Merge "BUG-868: Migrate to SchemaContextListener"
authorTony Tkacik <ttkacik@cisco.com>
Fri, 1 Aug 2014 08:00:43 +0000 (08:00 +0000)
committerGerrit Code Review <gerrit@opendaylight.org>
Fri, 1 Aug 2014 08:00:43 +0000 (08:00 +0000)
113 files changed:
opendaylight/commons/protocol-framework/pom.xml
opendaylight/commons/protocol-framework/src/main/java/org/opendaylight/protocol/framework/AbstractDispatcher.java
opendaylight/commons/protocol-framework/src/main/java/org/opendaylight/protocol/framework/ProtocolSessionPromise.java
opendaylight/commons/protocol-framework/src/main/java/org/opendaylight/protocol/framework/ReconnectPromise.java
opendaylight/commons/protocol-framework/src/main/java/org/opendaylight/protocol/framework/SessionListener.java
opendaylight/commons/protocol-framework/src/test/java/org/opendaylight/protocol/framework/ServerTest.java
opendaylight/commons/protocol-framework/src/test/java/org/opendaylight/protocol/framework/SimpleDispatcher.java
opendaylight/distribution/opendaylight/src/main/resources/configuration/initial/01-md-sal.xml
opendaylight/md-sal/pom.xml
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorContext.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorContextImpl.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/ApplySnapshot.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/ApplySnapshot.java with 89% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/ApplyState.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/ApplyState.java with 94% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/CommitEntry.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/CommitEntry.java with 86% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/ElectionTimeout.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/ElectionTimeout.java with 85% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/PersistEntry.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/PersistEntry.java with 87% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/Replicate.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/Replicate.java with 94% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SaveSnapshot.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/SaveSnapshot.java with 88% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SendHeartBeat.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/SendHeartBeat.java with 88% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SendInstallSnapshot.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/SendInstallSnapshot.java with 85% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehavior.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Leader.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntries.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntriesReply.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshot.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotReply.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RaftRPC.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVote.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVoteReply.java
opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/MockRaftActorContext.java
opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/CandidateTest.java
opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/FollowerTest.java
opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java
opendaylight/md-sal/sal-binding-broker/src/main/java/org/opendaylight/controller/md/sal/binding/impl/AbstractForwardedDataBroker.java
opendaylight/md-sal/sal-distributed-datastore/pom.xml
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/CompositeModificationPayload.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Configuration.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/NoOpCohort.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardManager.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/exceptions/PrimaryNotFoundException.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/exceptions/TimeoutException.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/messages/PeerAddressResolved.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/utils/ActorContext.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/AbstractActorTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/BasicIntegrationTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/DistributedDataStoreIntegrationTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/md/cluster/datastore/model/CarsModel.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/md/cluster/datastore/model/PeopleModel.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/md/cluster/datastore/model/SampleModelsTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/resources/module-shards.conf
opendaylight/md-sal/sal-distributed-datastore/src/test/resources/modules.conf
opendaylight/md-sal/sal-protocolbuffer-encoding/pom.xml
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/mdsal/CompositeModificationPayload.java [new file with mode: 0644]
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/cohort3pc/ThreePhaseCommitCohortMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/common/NormalizedNodeMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/common/SimpleNormalizedNodeMessage.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/datachange/notification/DataChangeListenerMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/persistent/PersistentMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/registration/ListenerRegistrationMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/shard/ShardManagerMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/transaction/ShardTransactionChainMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/transaction/ShardTransactionMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/resources/CompositeModificationPayload.proto [new file with mode: 0644]
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/resources/Persistent.proto
opendaylight/md-sal/sal-remoterpc-connector/pom.xml
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/config/yang/config/remote_rpc_connector/RemoteRPCBrokerModule.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/config/yang/config/remote_rpc_connector/RemoteRPCBrokerModuleFactory.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/RemoteRpcImplementation.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/RemoteRpcProvider.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/RemoteRpcProviderFactory.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/RouteIdentifierImpl.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/RoutedRpcListener.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/RpcBroker.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/RpcListener.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/RpcManager.java [new file with mode: 0644]
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/XmlUtils.java [deleted file]
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/AddRoutedRpc.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/AddRpc.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/ErrorResponse.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/ExecuteRpc.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/GetRoutedRpc.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/GetRoutedRpcReply.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/GetRpc.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/GetRpcReply.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/InvokeRoutedRpc.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/InvokeRpc.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/RemoveRoutedRpc.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/RemoveRpc.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/RoutingTableData.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/RpcResponse.java
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/UpdateSchemaContext.java [new file with mode: 0644]
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/ActorUtil.java [moved from opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/ActorUtil.java with 97% similarity]
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/InstanceIdentifierForXmlCodec.java [new file with mode: 0644]
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/RandomPrefix.java [new file with mode: 0644]
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/XmlDocumentUtils.java [new file with mode: 0644]
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/XmlStreamUtils.java [new file with mode: 0644]
opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/XmlUtils.java [new file with mode: 0644]
opendaylight/md-sal/sal-remoterpc-connector/src/main/resources/application.conf
opendaylight/md-sal/sal-remoterpc-connector/src/test/java/org/opendaylight/controller/remote/rpc/RpcBrokerTest.java
opendaylight/md-sal/sal-remoterpc-connector/src/test/java/org/opendaylight/controller/remote/rpc/registry/RoutingTableTest.java
opendaylight/md-sal/sal-remoterpc-connector/src/test/java/org/opendaylight/controller/remote/rpc/utils/XmlUtilsTest.java [new file with mode: 0644]
opendaylight/md-sal/sal-remoterpc-connector/src/test/resources/org/opendaylight/controller/remote/rpc/utils/rpcTest.yang [new file with mode: 0644]
opendaylight/netconf/netconf-cli/src/main/java/org/opendaylight/controller/netconf/cli/writer/impl/CliOutputFromNormalizedNodeSerializerFactory.java
opendaylight/networkconfiguration/neutron/src/main/java/org/opendaylight/controller/networkconfig/neutron/NeutronSubnet.java
opendaylight/northbound/commons/pom.xml
opendaylight/northbound/networkconfiguration/neutron/src/main/java/org/opendaylight/controller/networkconfig/neutron/northbound/NeutronRoutersNorthbound.java

index f70698731aa0aea9e1cc6f1cff86d4ded144b454..774bc7c23f178b30f26b94d65212a595d5941b4a 100644 (file)
       <artifactId>netty-event-executor-config</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>ch.qos.logback</groupId>
+      <artifactId>logback-classic</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>
index a62bd7da06501b355c41f36230dbf30a982d61fa..a05d02cd09ad4e1b447a61da687768bc58e3dd94 100644 (file)
@@ -7,12 +7,19 @@
  */
 package org.opendaylight.protocol.framework;
 
+import java.io.Closeable;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import com.google.common.base.Preconditions;
 
 import io.netty.bootstrap.Bootstrap;
 import io.netty.bootstrap.ServerBootstrap;
-import io.netty.channel.Channel;
 import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;
@@ -28,13 +35,6 @@ import io.netty.util.concurrent.Future;
 import io.netty.util.concurrent.GlobalEventExecutor;
 import io.netty.util.concurrent.Promise;
 
-import java.io.Closeable;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 /**
  * Dispatcher class for creating servers and clients. The idea is to first create servers and clients and the run the
  * start method that will handle sockets in different thread.
@@ -155,7 +155,7 @@ public abstract class AbstractDispatcher<S extends ProtocolSession<?>, L extends
      */
     protected Future<S> createClient(final InetSocketAddress address, final ReconnectStrategy strategy, final PipelineInitializer<S> initializer) {
         final Bootstrap b = new Bootstrap();
-        final ProtocolSessionPromise<S> p = new ProtocolSessionPromise<S>(executor, address, strategy, b);
+        final ProtocolSessionPromise<S> p = new ProtocolSessionPromise<>(executor, address, strategy, b);
         b.option(ChannelOption.SO_KEEPALIVE, true).handler(
                 new ChannelInitializer<SocketChannel>() {
                     @Override
@@ -165,18 +165,36 @@ public abstract class AbstractDispatcher<S extends ProtocolSession<?>, L extends
                 });
 
         customizeBootstrap(b);
+        setWorkerGroup(b);
+        setChannelFactory(b);
+
+        p.connect();
+        LOG.debug("Client created.");
+        return p;
+    }
 
+    private void setWorkerGroup(final Bootstrap b) {
         if (b.group() == null) {
             b.group(workerGroup);
         }
+    }
 
-        // There is no way to detect if this was already set by
-        // customizeBootstrap()
-        try {
-            b.channel(NioSocketChannel.class);
-        } catch (IllegalStateException e) {
-            LOG.trace("Not overriding channelFactory on bootstrap {}", b, e);
-        }
+    /**
+     * Create a client but use a pre-configured bootstrap.
+     * This method however replaces the ChannelInitializer in the bootstrap. All other configuration is preserved.
+     *
+     * @param address remote address
+     */
+    protected Future<S> createClient(final InetSocketAddress address, final ReconnectStrategy strategy, final Bootstrap bootstrap, final PipelineInitializer<S> initializer) {
+        final ProtocolSessionPromise<S> p = new ProtocolSessionPromise<>(executor, address, strategy, bootstrap);
+
+        bootstrap.handler(
+                new ChannelInitializer<SocketChannel>() {
+                    @Override
+                    protected void initChannel(final SocketChannel ch) {
+                        initializer.initializeChannel(ch, p);
+                    }
+                });
 
         p.connect();
         LOG.debug("Client created.");
@@ -195,6 +213,9 @@ public abstract class AbstractDispatcher<S extends ProtocolSession<?>, L extends
     }
 
     /**
+     *
+     * @deprecated use {@link org.opendaylight.protocol.framework.AbstractDispatcher#createReconnectingClient(java.net.InetSocketAddress, ReconnectStrategyFactory, org.opendaylight.protocol.framework.AbstractDispatcher.PipelineInitializer)} with only one reconnectStrategyFactory instead.
+     *
      * Creates a client.
      *
      * @param address remote address
@@ -204,15 +225,47 @@ public abstract class AbstractDispatcher<S extends ProtocolSession<?>, L extends
      * @return Future representing the reconnection task. It will report completion based on reestablishStrategy, e.g.
      *         success if it indicates no further attempts should be made and failure if it reports an error
      */
+    @Deprecated
     protected Future<Void> createReconnectingClient(final InetSocketAddress address, final ReconnectStrategyFactory connectStrategyFactory,
             final ReconnectStrategy reestablishStrategy, final PipelineInitializer<S> initializer) {
+        return createReconnectingClient(address, connectStrategyFactory, initializer);
+    }
 
-        final ReconnectPromise<S, L> p = new ReconnectPromise<S, L>(GlobalEventExecutor.INSTANCE, this, address, connectStrategyFactory, reestablishStrategy, initializer);
-        p.connect();
+    /**
+     * Creates a reconnecting client.
+     *
+     * @param address remote address
+     * @param connectStrategyFactory Factory for creating reconnection strategy for every reconnect attempt
+     *
+     * @return Future representing the reconnection task. It will report completion based on reestablishStrategy, e.g.
+     *         success if it indicates no further attempts should be made and failure if it reports an error
+     */
+    protected Future<Void> createReconnectingClient(final InetSocketAddress address, final ReconnectStrategyFactory connectStrategyFactory,
+            final PipelineInitializer<S> initializer) {
+        final Bootstrap b = new Bootstrap();
+
+        final ReconnectPromise<S, L> p = new ReconnectPromise<>(GlobalEventExecutor.INSTANCE, this, address, connectStrategyFactory, b, initializer);
+
+        b.option(ChannelOption.SO_KEEPALIVE, true);
 
+        customizeBootstrap(b);
+        setWorkerGroup(b);
+        setChannelFactory(b);
+
+        p.connect();
         return p;
     }
 
+    private void setChannelFactory(final Bootstrap b) {
+        // There is no way to detect if this was already set by
+        // customizeBootstrap()
+        try {
+            b.channel(NioSocketChannel.class);
+        } catch (final IllegalStateException e) {
+            LOG.trace("Not overriding channelFactory on bootstrap {}", b, e);
+        }
+    }
+
     /**
      * @deprecated Should only be used with {@link AbstractDispatcher#AbstractDispatcher()}
      */
@@ -225,5 +278,4 @@ public abstract class AbstractDispatcher<S extends ProtocolSession<?>, L extends
             this.bossGroup.shutdownGracefully();
         }
     }
-
 }
index a78274cca0ca4f49e3c14ce69e6e6815b7a7e395..a38db61eaddcdb4ebd2daa79f03790fc6bde2007 100644 (file)
@@ -7,6 +7,7 @@
  */
 package org.opendaylight.protocol.framework;
 
+import com.google.common.base.Preconditions;
 import io.netty.bootstrap.Bootstrap;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelFutureListener;
@@ -16,17 +17,12 @@ import io.netty.util.concurrent.EventExecutor;
 import io.netty.util.concurrent.Future;
 import io.netty.util.concurrent.FutureListener;
 import io.netty.util.concurrent.Promise;
-
 import java.net.InetSocketAddress;
-
 import javax.annotation.concurrent.GuardedBy;
 import javax.annotation.concurrent.ThreadSafe;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Preconditions;
-
 @ThreadSafe
 final class ProtocolSessionPromise<S extends ProtocolSession<?>> extends DefaultPromise<S> {
     private static final Logger LOG = LoggerFactory.getLogger(ProtocolSessionPromise.class);
@@ -54,72 +50,12 @@ final class ProtocolSessionPromise<S extends ProtocolSession<?>> extends Default
             LOG.debug("Promise {} attempting connect for {}ms", lock, timeout);
 
             this.b.option(ChannelOption.CONNECT_TIMEOUT_MILLIS, timeout);
-            this.pending = this.b.connect(this.address).addListener(new ChannelFutureListener() {
-                @Override
-                public void operationComplete(final ChannelFuture cf) throws Exception {
-                    synchronized (lock) {
-
-                        LOG.debug("Promise {} connection resolved", lock);
-
-                        // Triggered when a connection attempt is resolved.
-                        Preconditions.checkState(ProtocolSessionPromise.this.pending.equals(cf));
-
-                        /*
-                         * The promise we gave out could have been cancelled,
-                         * which cascades to the connect getting cancelled,
-                         * but there is a slight race window, where the connect
-                         * is already resolved, but the listener has not yet
-                         * been notified -- cancellation at that point won't
-                         * stop the notification arriving, so we have to close
-                         * the race here.
-                         */
-                        if (isCancelled()) {
-                            if (cf.isSuccess()) {
-                                LOG.debug("Closing channel for cancelled promise {}", lock);
-                                cf.channel().close();
-                            }
-                            return;
-                        }
-
-                        if (!cf.isSuccess()) {
-                            LOG.debug("Attempt to connect to {} failed", ProtocolSessionPromise.this.address, cf.cause());
-
-                            final Future<Void> rf = ProtocolSessionPromise.this.strategy.scheduleReconnect(cf.cause());
-                            rf.addListener(new FutureListener<Void>() {
-                                @Override
-                                public void operationComplete(final Future<Void> sf) {
-                                    synchronized (lock) {
-                                        // Triggered when a connection attempt is to be made.
-                                        Preconditions.checkState(ProtocolSessionPromise.this.pending.equals(sf));
-
-                                        /*
-                                         * The promise we gave out could have been cancelled,
-                                         * which cascades to the reconnect attempt getting
-                                         * cancelled, but there is a slight race window, where
-                                         * the reconnect attempt is already enqueued, but the
-                                         * listener has not yet been notified -- if cancellation
-                                         * happens at that point, we need to catch it here.
-                                         */
-                                        if (!isCancelled()) {
-                                            if (sf.isSuccess()) {
-                                                connect();
-                                            } else {
-                                                setFailure(sf.cause());
-                                            }
-                                        }
-                                    }
-                                }
-                            });
-
-                            ProtocolSessionPromise.this.pending = rf;
-                        } else {
-                            LOG.debug("Promise {} connection successful", lock);
-                        }
-                    }
-                }
-            });
+            final ChannelFuture connectFuture = this.b.connect(this.address);
+            // Add listener that attempts reconnect by invoking this method again.
+            connectFuture.addListener(new BootstrapConnectListener(lock));
+            this.pending = connectFuture;
         } catch (final Exception e) {
-            LOG.info("Failed to connect to {}", e);
+            LOG.info("Failed to connect to {}", address, e);
             setFailure(e);
         }
     }
@@ -140,4 +76,79 @@ final class ProtocolSessionPromise<S extends ProtocolSession<?>> extends Default
         this.strategy.reconnectSuccessful();
         return super.setSuccess(result);
     }
+
+    private class BootstrapConnectListener implements ChannelFutureListener {
+        private final Object lock;
+
+        public BootstrapConnectListener(final Object lock) {
+            this.lock = lock;
+        }
+
+        @Override
+        public void operationComplete(final ChannelFuture cf) throws Exception {
+            synchronized (lock) {
+
+                LOG.debug("Promise {} connection resolved", lock);
+
+                // Triggered when a connection attempt is resolved.
+                Preconditions.checkState(ProtocolSessionPromise.this.pending.equals(cf));
+
+                /*
+                 * The promise we gave out could have been cancelled,
+                 * which cascades to the connect getting cancelled,
+                 * but there is a slight race window, where the connect
+                 * is already resolved, but the listener has not yet
+                 * been notified -- cancellation at that point won't
+                 * stop the notification arriving, so we have to close
+                 * the race here.
+                 */
+                if (isCancelled()) {
+                    if (cf.isSuccess()) {
+                        LOG.debug("Closing channel for cancelled promise {}", lock);
+                        cf.channel().close();
+                    }
+                    return;
+                }
+
+                if(cf.isSuccess()) {
+                    LOG.debug("Promise {} connection successful", lock);
+                    return;
+                }
+
+                LOG.debug("Attempt to connect to {} failed", ProtocolSessionPromise.this.address, cf.cause());
+
+                final Future<Void> rf = ProtocolSessionPromise.this.strategy.scheduleReconnect(cf.cause());
+                rf.addListener(new ReconnectingStrategyListener());
+                ProtocolSessionPromise.this.pending = rf;
+            }
+        }
+
+        private class ReconnectingStrategyListener implements FutureListener<Void> {
+            @Override
+            public void operationComplete(final Future<Void> sf) {
+                synchronized (lock) {
+                    // Triggered when a connection attempt is to be made.
+                    Preconditions.checkState(ProtocolSessionPromise.this.pending.equals(sf));
+
+                    /*
+                     * The promise we gave out could have been cancelled,
+                     * which cascades to the reconnect attempt getting
+                     * cancelled, but there is a slight race window, where
+                     * the reconnect attempt is already enqueued, but the
+                     * listener has not yet been notified -- if cancellation
+                     * happens at that point, we need to catch it here.
+                     */
+                    if (!isCancelled()) {
+                        if (sf.isSuccess()) {
+                            connect();
+                        } else {
+                            setFailure(sf.cause());
+                        }
+                    }
+                }
+            }
+        }
+
+    }
+
 }
index 1fa6a8175352617aa8f87a56e90f6d391bdef029..fe1012f443fc7824b1e2b8b3d3c4ed15ecbfb30d 100644 (file)
  */
 package org.opendaylight.protocol.framework;
 
-import io.netty.channel.ChannelFuture;
+import com.google.common.base.Preconditions;
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
 import io.netty.channel.socket.SocketChannel;
 import io.netty.util.concurrent.DefaultPromise;
 import io.netty.util.concurrent.EventExecutor;
 import io.netty.util.concurrent.Future;
-import io.netty.util.concurrent.FutureListener;
 import io.netty.util.concurrent.Promise;
-
-import java.io.Closeable;
 import java.net.InetSocketAddress;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.opendaylight.protocol.framework.AbstractDispatcher.PipelineInitializer;
-
-import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 final class ReconnectPromise<S extends ProtocolSession<?>, L extends SessionListener<?, ?, ?>> extends DefaultPromise<Void> {
+    private static final Logger LOG = LoggerFactory.getLogger(ReconnectPromise.class);
+
     private final AbstractDispatcher<S, L> dispatcher;
     private final InetSocketAddress address;
     private final ReconnectStrategyFactory strategyFactory;
-    private final ReconnectStrategy strategy;
-    private final PipelineInitializer<S> initializer;
+    private final Bootstrap b;
+    private final AbstractDispatcher.PipelineInitializer<S> initializer;
     private Future<?> pending;
 
-    private final AtomicBoolean negotiationFinished = new AtomicBoolean(false);
-
     public ReconnectPromise(final EventExecutor executor, final AbstractDispatcher<S, L> dispatcher, final InetSocketAddress address,
-            final ReconnectStrategyFactory connectStrategyFactory, final ReconnectStrategy reestablishStrategy,
-            final PipelineInitializer<S> initializer) {
+                            final ReconnectStrategyFactory connectStrategyFactory, final Bootstrap b, final AbstractDispatcher.PipelineInitializer<S> initializer) {
         super(executor);
+        this.b = b;
+        this.initializer = Preconditions.checkNotNull(initializer);
         this.dispatcher = Preconditions.checkNotNull(dispatcher);
         this.address = Preconditions.checkNotNull(address);
         this.strategyFactory = Preconditions.checkNotNull(connectStrategyFactory);
-        this.strategy = Preconditions.checkNotNull(reestablishStrategy);
-        this.initializer = Preconditions.checkNotNull(initializer);
     }
 
-    // FIXME: BUG-190: refactor
-
     synchronized void connect() {
-        negotiationFinished.set(false);
-
         final ReconnectStrategy cs = this.strategyFactory.createReconnectStrategy();
-        final ReconnectStrategy rs = new ReconnectStrategy() {
-            @Override
-            public Future<Void> scheduleReconnect(final Throwable cause) {
-                return cs.scheduleReconnect(cause);
-            }
 
-            @Override
-            public void reconnectSuccessful() {
-                cs.reconnectSuccessful();
-            }
-
-            @Override
-            public int getConnectTimeout() throws Exception {
-                final int cst = cs.getConnectTimeout();
-                final int rst = ReconnectPromise.this.strategy.getConnectTimeout();
-
-                if (cst == 0) {
-                    return rst;
-                }
-                if (rst == 0) {
-                    return cst;
-                }
-                return Math.min(cst, rst);
-            }
-        };
-
-        final Future<S> cf = this.dispatcher.createClient(this.address, rs, new PipelineInitializer<S>() {
+        // Set up a client with pre-configured bootstrap, but add a closed channel handler into the pipeline to support reconnect attempts
+        pending = this.dispatcher.createClient(this.address, cs, b, new AbstractDispatcher.PipelineInitializer<S>() {
             @Override
             public void initializeChannel(final SocketChannel channel, final Promise<S> promise) {
-                addChannelClosedListener(channel.closeFuture());
                 initializer.initializeChannel(channel, promise);
+
+                // add closed channel handler
+                channel.pipeline().addFirst(new ClosedChannelHandler(ReconnectPromise.this));
             }
         });
+    }
 
-        final Object lock = this;
-        this.pending = cf;
+    /**
+     *
+     * @return true if initial connection was established successfully, false if initial connection failed due to e.g. Connection refused, Negotiation failed
+     */
+    private boolean isInitialConnectFinished() {
+        Preconditions.checkNotNull(pending);
+        return pending.isDone() && pending.isSuccess();
+    }
 
-        cf.addListener(new FutureListener<S>() {
+    @Override
+    public synchronized boolean cancel(final boolean mayInterruptIfRunning) {
+        if (super.cancel(mayInterruptIfRunning)) {
+            Preconditions.checkNotNull(pending);
+            this.pending.cancel(mayInterruptIfRunning);
+            return true;
+        }
 
-            @Override
-            public void operationComplete(final Future<S> future) {
-                synchronized (lock) {
-                    if (!future.isSuccess()) {
-                        final Future<Void> rf = ReconnectPromise.this.strategy.scheduleReconnect(cf.cause());
-
-                        if(rf == null) {
-                            // This should reflect: no more reconnecting strategies, enough
-                            // Currently all reconnect strategies fail with exception, should return null
-                            return;
-                        }
-
-                        ReconnectPromise.this.pending = rf;
-
-                        rf.addListener(new FutureListener<Void>() {
-                            @Override
-                            public void operationComplete(final Future<Void> sf) {
-                                synchronized (lock) {
-                                    /*
-                                     * The promise we gave out could have been cancelled,
-                                     * which cascades to the reconnect attempt getting
-                                     * cancelled, but there is a slight race window, where
-                                     * the reconnect attempt is already enqueued, but the
-                                     * listener has not yet been notified -- if cancellation
-                                     * happens at that point, we need to catch it here.
-                                     */
-                                    if (!isCancelled()) {
-                                        if (sf.isSuccess()) {
-                                            connect();
-                                        } else {
-                                            setFailure(sf.cause());
-                                        }
-                                    }
-                                }
-                            }
-                        });
-                    } else {
-                        /*
-                         *  FIXME: BUG-190: we have a slight race window with cancellation
-                         *         here. Analyze and define its semantics.
-                         */
-                        ReconnectPromise.this.strategy.reconnectSuccessful();
-                        negotiationFinished.set(true);
-                    }
-                }
-            }
-        });
+        return false;
     }
 
-    private final ClosedChannelListener closedChannelListener = new ClosedChannelListener();
-
-    class ClosedChannelListener implements Closeable, FutureListener<Void> {
+    /**
+     * Channel handler that responds to channelInactive event and reconnects the session.
+     * Only if the initial connection was successfully established and promise was not canceled.
+     */
+    private static final class ClosedChannelHandler extends ChannelInboundHandlerAdapter {
+        private final ReconnectPromise<?, ?> promise;
 
-        private final AtomicBoolean stop = new AtomicBoolean(false);
+        public ClosedChannelHandler(final ReconnectPromise<?, ?> promise) {
+            this.promise = promise;
+        }
 
         @Override
-        public void operationComplete(final Future<Void> future) throws Exception {
-            if (stop.get()) {
+        public void channelInactive(final ChannelHandlerContext ctx) throws Exception {
+            if (promise.isCancelled()) {
                 return;
             }
 
-            // Start reconnecting crashed session after negotiation was successful
-            if (!negotiationFinished.get()) {
+            // Check if initial connection was fully finished. If the session was dropped during negotiation, reconnect will not happen.
+            // Session can be dropped during negotiation on purpose by the client side and would make no sense to initiate reconnect
+            if (promise.isInitialConnectFinished() == false) {
                 return;
             }
 
-            connect();
-        }
-
-        @Override
-        public void close() {
-            this.stop.set(true);
+            LOG.debug("Reconnecting after connection to {} was dropped", promise.address);
+            promise.connect();
         }
     }
 
-    private void addChannelClosedListener(final ChannelFuture channelFuture) {
-        channelFuture.addListener(closedChannelListener);
-    }
-
-    @Override
-    public synchronized boolean cancel(final boolean mayInterruptIfRunning) {
-        closedChannelListener.close();
-
-        if (super.cancel(mayInterruptIfRunning)) {
-            this.pending.cancel(mayInterruptIfRunning);
-            return true;
-        }
-
-        return false;
-    }
 }
index 3c429fc7749be6b4ac37cf5e5800f5978861fa2c..a756a0da7e780e84510d8e329b475b9342e0c9e7 100644 (file)
@@ -10,7 +10,7 @@ package org.opendaylight.protocol.framework;
 import java.util.EventListener;
 
 /**
- * Listener that receives session state informations. This interface should be
+ * Listener that receives session state information. This interface should be
  * implemented by a protocol specific abstract class, that is extended by
  * a final class that implements the methods.
  */
index bead1ee49e64bd1b708ecc6d23c51321f4cc72bd..63026e384c498ce12dca810c94886f9a562215d8 100644 (file)
@@ -9,6 +9,14 @@ package org.opendaylight.protocol.framework;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.timeout;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.nio.NioEventLoopGroup;
@@ -16,50 +24,139 @@ import io.netty.util.concurrent.DefaultPromise;
 import io.netty.util.concurrent.Future;
 import io.netty.util.concurrent.GlobalEventExecutor;
 import io.netty.util.concurrent.Promise;
-
+import io.netty.util.concurrent.SucceededFuture;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 public class ServerTest {
     SimpleDispatcher clientDispatcher, dispatcher;
 
-    final SimpleSessionListener pce = new SimpleSessionListener();
-
     SimpleSession session = null;
 
     ChannelFuture server = null;
 
     InetSocketAddress serverAddress;
     private NioEventLoopGroup eventLoopGroup;
-
+    // Dedicated loop group for server, needed for testing reconnection client
+    // With dedicated server group we can simulate session drop by shutting only the server group down
+    private NioEventLoopGroup serverLoopGroup;
 
     @Before
     public void setUp() {
         final int port = 10000 + (int)(10000 * Math.random());
         serverAddress = new InetSocketAddress("127.0.0.1", port);
         eventLoopGroup = new NioEventLoopGroup();
+        serverLoopGroup = new NioEventLoopGroup();
+    }
+
+    @After
+    public void tearDown() throws IOException, InterruptedException, ExecutionException {
+        if(server != null) {
+            this.server.channel().close();
+        }
+        this.eventLoopGroup.shutdownGracefully().get();
+        this.serverLoopGroup.shutdownGracefully().get();
+        try {
+            Thread.sleep(500);
+        } catch (final InterruptedException e) {
+            throw new RuntimeException(e);
+        }
     }
 
     @Test
-    public void testConnectionEstablished() throws Exception {
+    public void testConnectionRefused() throws Exception {
+        this.clientDispatcher = getClientDispatcher();
+
+        final ReconnectStrategy mockReconnectStrategy = getMockedReconnectStrategy();
+
+        this.clientDispatcher.createClient(this.serverAddress,
+                mockReconnectStrategy, new SessionListenerFactory<SimpleSessionListener>() {
+                    @Override
+                    public SimpleSessionListener getSessionListener() {
+                        return new SimpleSessionListener();
+                    }
+                });
+
+        Mockito.verify(mockReconnectStrategy, timeout(5000).atLeast(2)).scheduleReconnect(any(Throwable.class));
+    }
+
+    @Test
+    public void testConnectionReestablishInitial() throws Exception {
+        this.clientDispatcher = getClientDispatcher();
+
+        final ReconnectStrategy mockReconnectStrategy = getMockedReconnectStrategy();
+
+        this.clientDispatcher.createClient(this.serverAddress,
+                mockReconnectStrategy, new SessionListenerFactory<SimpleSessionListener>() {
+                    @Override
+                    public SimpleSessionListener getSessionListener() {
+                        return new SimpleSessionListener();
+                    }
+                });
+
+        Mockito.verify(mockReconnectStrategy, timeout(5000).atLeast(2)).scheduleReconnect(any(Throwable.class));
+
+        final Promise<Boolean> p = new DefaultPromise<>(GlobalEventExecutor.INSTANCE);
+        this.dispatcher = getServerDispatcher(p);
+
+        this.server = this.dispatcher.createServer(this.serverAddress, new SessionListenerFactory<SimpleSessionListener>() {
+            @Override
+            public SimpleSessionListener getSessionListener() {
+                return new SimpleSessionListener();
+            }
+        });
+
+        this.server.get();
+
+        assertEquals(true, p.get(3, TimeUnit.SECONDS));
+    }
+
+    @Test
+    public void testConnectionDrop() throws Exception {
         final Promise<Boolean> p = new DefaultPromise<>(GlobalEventExecutor.INSTANCE);
 
-        this.dispatcher = new SimpleDispatcher(new SessionNegotiatorFactory<SimpleMessage, SimpleSession, SimpleSessionListener>() {
+        this.dispatcher = getServerDispatcher(p);
 
+        this.server = this.dispatcher.createServer(this.serverAddress, new SessionListenerFactory<SimpleSessionListener>() {
             @Override
-            public SessionNegotiator<SimpleSession> getSessionNegotiator(final SessionListenerFactory<SimpleSessionListener> factory,
-                    final Channel channel, final Promise<SimpleSession> promise) {
-                p.setSuccess(true);
-                return new SimpleSessionNegotiator(promise, channel);
+            public SimpleSessionListener getSessionListener() {
+                return new SimpleSessionListener();
             }
-        }, new DefaultPromise<SimpleSession>(GlobalEventExecutor.INSTANCE), eventLoopGroup);
+        });
+
+        this.server.get();
+
+        this.clientDispatcher = getClientDispatcher();
+
+        final ReconnectStrategy reconnectStrategy = getMockedReconnectStrategy();
+        this.session = this.clientDispatcher.createClient(this.serverAddress,
+                reconnectStrategy, new SessionListenerFactory<SimpleSessionListener>() {
+                    @Override
+                    public SimpleSessionListener getSessionListener() {
+                        return new SimpleSessionListener();
+                    }
+                }).get(6, TimeUnit.SECONDS);
+
+        assertEquals(true, p.get(3, TimeUnit.SECONDS));
+
+        shutdownServer();
+
+        // No reconnect should be scheduled after server drops connection with not-reconnecting client
+        verify(reconnectStrategy, times(0)).scheduleReconnect(any(Throwable.class));
+    }
+
+    @Test
+    public void testConnectionReestablishAfterDrop() throws Exception {
+        final Promise<Boolean> p = new DefaultPromise<>(GlobalEventExecutor.INSTANCE);
+
+        this.dispatcher = getServerDispatcher(p);
 
         this.server = this.dispatcher.createServer(this.serverAddress, new SessionListenerFactory<SimpleSessionListener>() {
             @Override
@@ -70,13 +167,42 @@ public class ServerTest {
 
         this.server.get();
 
-        this.clientDispatcher = new SimpleDispatcher(new SessionNegotiatorFactory<SimpleMessage, SimpleSession, SimpleSessionListener>() {
+        this.clientDispatcher = getClientDispatcher();
+
+        final ReconnectStrategyFactory reconnectStrategyFactory = mock(ReconnectStrategyFactory.class);
+        final ReconnectStrategy reconnectStrategy = getMockedReconnectStrategy();
+        doReturn(reconnectStrategy).when(reconnectStrategyFactory).createReconnectStrategy();
+
+        this.clientDispatcher.createReconnectingClient(this.serverAddress,
+                reconnectStrategyFactory, new SessionListenerFactory<SimpleSessionListener>() {
+                    @Override
+                    public SimpleSessionListener getSessionListener() {
+                        return new SimpleSessionListener();
+                    }
+                });
+
+        assertEquals(true, p.get(3, TimeUnit.SECONDS));
+        shutdownServer();
+
+        verify(reconnectStrategyFactory, timeout(20000).atLeast(2)).createReconnectStrategy();
+    }
+
+    @Test
+    public void testConnectionEstablished() throws Exception {
+        final Promise<Boolean> p = new DefaultPromise<>(GlobalEventExecutor.INSTANCE);
+
+        this.dispatcher = getServerDispatcher(p);
+
+        this.server = this.dispatcher.createServer(this.serverAddress, new SessionListenerFactory<SimpleSessionListener>() {
             @Override
-            public SessionNegotiator<SimpleSession> getSessionNegotiator(final SessionListenerFactory<SimpleSessionListener> factory,
-                    final Channel channel, final Promise<SimpleSession> promise) {
-                return new SimpleSessionNegotiator(promise, channel);
+            public SimpleSessionListener getSessionListener() {
+                return new SimpleSessionListener();
             }
-        }, new DefaultPromise<SimpleSession>(GlobalEventExecutor.INSTANCE), eventLoopGroup);
+        });
+
+        this.server.get();
+
+        this.clientDispatcher = getClientDispatcher();
 
         this.session = this.clientDispatcher.createClient(this.serverAddress,
                 new NeverReconnectStrategy(GlobalEventExecutor.INSTANCE, 5000), new SessionListenerFactory<SimpleSessionListener>() {
@@ -93,15 +219,7 @@ public class ServerTest {
     public void testConnectionFailed() throws IOException, InterruptedException, ExecutionException, TimeoutException {
         final Promise<Boolean> p = new DefaultPromise<>(GlobalEventExecutor.INSTANCE);
 
-        this.dispatcher = new SimpleDispatcher(new SessionNegotiatorFactory<SimpleMessage, SimpleSession, SimpleSessionListener>() {
-
-            @Override
-            public SessionNegotiator<SimpleSession> getSessionNegotiator(final SessionListenerFactory<SimpleSessionListener> factory,
-                    final Channel channel, final Promise<SimpleSession> promise) {
-                p.setSuccess(true);
-                return new SimpleSessionNegotiator(promise, channel);
-            }
-        }, new DefaultPromise<SimpleSession>(GlobalEventExecutor.INSTANCE), eventLoopGroup);
+        this.dispatcher = getServerDispatcher(p);
 
         this.server = this.dispatcher.createServer(this.serverAddress, new SessionListenerFactory<SimpleSessionListener>() {
             @Override
@@ -112,13 +230,7 @@ public class ServerTest {
 
         this.server.get();
 
-        this.clientDispatcher = new SimpleDispatcher(new SessionNegotiatorFactory<SimpleMessage, SimpleSession, SimpleSessionListener>() {
-            @Override
-            public SessionNegotiator<SimpleSession> getSessionNegotiator(final SessionListenerFactory<SimpleSessionListener> factory,
-                    final Channel channel, final Promise<SimpleSession> promise) {
-                return new SimpleSessionNegotiator(promise, channel);
-            }
-        }, new DefaultPromise<SimpleSession>(GlobalEventExecutor.INSTANCE), eventLoopGroup);
+        this.clientDispatcher = getClientDispatcher();
 
         this.session = this.clientDispatcher.createClient(this.serverAddress,
                 new NeverReconnectStrategy(GlobalEventExecutor.INSTANCE, 5000), new SessionListenerFactory<SimpleSessionListener>() {
@@ -138,14 +250,89 @@ public class ServerTest {
         assertFalse(session.isSuccess());
     }
 
-    @After
-    public void tearDown() throws IOException, InterruptedException {
-        this.server.channel().close();
-        this.eventLoopGroup.shutdownGracefully();
-        try {
-            Thread.sleep(500);
-        } catch (final InterruptedException e) {
-            throw new RuntimeException(e);
-        }
+    @Test
+    public void testNegotiationFailedNoReconnect() throws Exception {
+        final Promise<Boolean> p = new DefaultPromise<>(GlobalEventExecutor.INSTANCE);
+
+        this.dispatcher = getServerDispatcher(p);
+
+        this.server = this.dispatcher.createServer(this.serverAddress, new SessionListenerFactory<SimpleSessionListener>() {
+            @Override
+            public SimpleSessionListener getSessionListener() {
+                return new SimpleSessionListener();
+            }
+        });
+
+        this.server.get();
+
+        this.clientDispatcher = new SimpleDispatcher(new SessionNegotiatorFactory<SimpleMessage, SimpleSession, SimpleSessionListener>() {
+            @Override
+            public SessionNegotiator<SimpleSession> getSessionNegotiator(final SessionListenerFactory<SimpleSessionListener> factory,
+                                                                         final Channel channel, final Promise<SimpleSession> promise) {
+
+                return new SimpleSessionNegotiator(promise, channel) {
+                    @Override
+                    protected void startNegotiation() throws Exception {
+                        negotiationFailed(new IllegalStateException("Negotiation failed"));
+                    }
+                };
+            }
+        }, new DefaultPromise<SimpleSession>(GlobalEventExecutor.INSTANCE), eventLoopGroup);
+
+        final ReconnectStrategyFactory reconnectStrategyFactory = mock(ReconnectStrategyFactory.class);
+        final ReconnectStrategy reconnectStrategy = getMockedReconnectStrategy();
+        doReturn(reconnectStrategy).when(reconnectStrategyFactory).createReconnectStrategy();
+
+        this.clientDispatcher.createReconnectingClient(this.serverAddress,
+                reconnectStrategyFactory, new SessionListenerFactory<SimpleSessionListener>() {
+                    @Override
+                    public SimpleSessionListener getSessionListener() {
+                        return new SimpleSessionListener();
+                    }
+                });
+
+
+        // Only one strategy should be created for initial connect, no more = no reconnects
+        verify(reconnectStrategyFactory, times(1)).createReconnectStrategy();
     }
+
+    private SimpleDispatcher getClientDispatcher() {
+        return new SimpleDispatcher(new SessionNegotiatorFactory<SimpleMessage, SimpleSession, SimpleSessionListener>() {
+            @Override
+            public SessionNegotiator<SimpleSession> getSessionNegotiator(final SessionListenerFactory<SimpleSessionListener> factory,
+                                                                         final Channel channel, final Promise<SimpleSession> promise) {
+                return new SimpleSessionNegotiator(promise, channel);
+            }
+        }, new DefaultPromise<SimpleSession>(GlobalEventExecutor.INSTANCE), eventLoopGroup);
+    }
+
+    private ReconnectStrategy getMockedReconnectStrategy() throws Exception {
+        final ReconnectStrategy mockReconnectStrategy = mock(ReconnectStrategy.class);
+        final Future<Void> future = new SucceededFuture<>(GlobalEventExecutor.INSTANCE, null);
+        doReturn(future).when(mockReconnectStrategy).scheduleReconnect(any(Throwable.class));
+        doReturn(5000).when(mockReconnectStrategy).getConnectTimeout();
+        doNothing().when(mockReconnectStrategy).reconnectSuccessful();
+        return mockReconnectStrategy;
+    }
+
+
+    private void shutdownServer() throws InterruptedException, ExecutionException {
+        // Shutdown server
+        server.channel().close().get();
+        // Closing server channel does not close established connections, eventLoop has to be closed as well to simulate dropped session
+        serverLoopGroup.shutdownGracefully().get();
+    }
+
+    private SimpleDispatcher getServerDispatcher(final Promise<Boolean> p) {
+        return new SimpleDispatcher(new SessionNegotiatorFactory<SimpleMessage, SimpleSession, SimpleSessionListener>() {
+
+            @Override
+            public SessionNegotiator<SimpleSession> getSessionNegotiator(final SessionListenerFactory<SimpleSessionListener> factory,
+                                                                         final Channel channel, final Promise<SimpleSession> promise) {
+                p.setSuccess(true);
+                return new SimpleSessionNegotiator(promise, channel);
+            }
+        }, null, serverLoopGroup);
+    }
+
 }
index 12aac9ecc5257432ee282159f24dbb1fd2c319a5..d83738520cccc1e7bc24b4f97cb31c30ff6e51ad 100644 (file)
@@ -54,6 +54,10 @@ public class SimpleDispatcher extends AbstractDispatcher<SimpleSession, SimpleSe
         return super.createClient(address, strategy, new SimplePipelineInitializer(listenerFactory));
     }
 
+    public Future<Void> createReconnectingClient(final InetSocketAddress address, final ReconnectStrategyFactory strategy, final SessionListenerFactory<SimpleSessionListener> listenerFactory) {
+        return super.createReconnectingClient(address, strategy, new SimplePipelineInitializer(listenerFactory));
+    }
+
     public ChannelFuture createServer(final InetSocketAddress address, final SessionListenerFactory<SimpleSessionListener> listenerFactory) {
         return super.createServer(address, new SimplePipelineInitializer(listenerFactory));
     }
index 03da7f0ccd6cd37a4044404e3426a8f5307f580b..f25b7d91bdb38b2f357ec5faa91283d2929e3128 100644 (file)
                     </schema-service>
                 </module>
                 -->
-                <!-- Cluster RPC -->
-                <!-- Enable the following module if you want to use remote rpc connector
-                <module>
-                    <type xmlns:prefix="urn:opendaylight:params:xml:ns:yang:controller:config:remote-rpc-connector">prefix:remote-rpc-connector</type>
-                    <name>remote-rpc-connector</name>
-                    <dom-broker xmlns="urn:opendaylight:params:xml:ns:yang:controller:config:remote-rpc-connector">
-                        <type xmlns:dom="urn:opendaylight:params:xml:ns:yang:controller:md:sal:dom">dom:dom-broker-osgi-registry</type>
-                        <name>dom-broker</name>
-                    </dom-broker>
-                </module>
-                -->
+
                 <module>
                     <type xmlns:prefix="urn:opendaylight:params:xml:ns:yang:controller:inmemory-datastore-provider">prefix:inmemory-operational-datastore-provider</type>
                     <name>operational-store-service</name>
                         </binding-mapping-service>
                     </binding-forwarded-data-broker>
                 </module>
+                <!-- Cluster RPC -->
+                <!-- Enable the following module if you want to use remote rpc connector
+                <module>
+                    <type xmlns:prefix="urn:opendaylight:params:xml:ns:yang:controller:config:remote-rpc-connector">prefix:remote-rpc-connector</type>
+                    <name>remote-rpc-connector</name>
+                    <dom-broker xmlns="urn:opendaylight:params:xml:ns:yang:controller:config:remote-rpc-connector">
+                        <type xmlns:dom="urn:opendaylight:params:xml:ns:yang:controller:md:sal:dom">dom:dom-broker-osgi-registry</type>
+                        <name>dom-broker</name>
+                    </dom-broker>
+                </module>
+                -->
             </modules>
             <services xmlns="urn:opendaylight:params:xml:ns:yang:controller:config">
                     <service>
index cadd8ca87d784095c9a9a6bc5f08b396f399691d..37da4d2b880b73689d0fae3df5c9d21e08156aec 100644 (file)
     <!--sal-protocolbuffer-encoding-->
     <module>sal-protocolbuffer-encoding</module>
 
-    <!--sal-distributed-datastore-->
+    <!-- sal-distributed-datastore -->
     <module>sal-distributed-datastore</module>
 
+    <!-- XSQL -->
+    <module>sal-dom-xsql</module>
+
     <!-- Yang Test Models for MD-SAL -->
     <module>sal-test-model</module>
+
+    <!-- Clustering -->
+    <module>sal-remoterpc-connector</module>
   </modules>
 
   <build>
index 15239795a8abc2aab0019439e2a64bbab2701afc..b8e9653bc5db92ffa4cb15afd391c6a9ec05cd16 100644 (file)
@@ -19,6 +19,9 @@ import akka.persistence.SaveSnapshotSuccess;
 import akka.persistence.SnapshotOffer;
 import akka.persistence.SnapshotSelectionCriteria;
 import akka.persistence.UntypedPersistentActor;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
+import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
 import org.opendaylight.controller.cluster.raft.behaviors.Candidate;
 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
 import org.opendaylight.controller.cluster.raft.behaviors.Leader;
@@ -26,10 +29,7 @@ import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
 import org.opendaylight.controller.cluster.raft.client.messages.AddRaftPeer;
 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
-import org.opendaylight.controller.cluster.raft.internal.messages.ApplySnapshot;
 import org.opendaylight.controller.cluster.raft.client.messages.RemoveRaftPeer;
-import org.opendaylight.controller.cluster.raft.internal.messages.ApplyState;
-import org.opendaylight.controller.cluster.raft.internal.messages.Replicate;
 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
 
 import java.io.Serializable;
@@ -250,6 +250,24 @@ public abstract class RaftActor extends UntypedPersistentActor {
         return currentBehavior.state();
     }
 
+    /**
+     * setPeerAddress sets the address of a known peer at a later time.
+     * <p>
+     * This is to account for situations where a we know that a peer
+     * exists but we do not know an address up-front. This may also be used in
+     * situations where a known peer starts off in a different location and we
+     * need to change it's address
+     * <p>
+     * Note that if the peerId does not match the list of peers passed to
+     * this actor during construction an IllegalStateException will be thrown.
+     *
+     * @param peerId
+     * @param peerAddress
+     */
+    protected void setPeerAddress(String peerId, String peerAddress){
+        context.setPeerAddress(peerId, peerAddress);
+    }
+
 
 
     /**
@@ -451,7 +469,7 @@ public abstract class RaftActor extends UntypedPersistentActor {
             final String identifier,
             final ReplicatedLogEntry replicatedLogEntry) {
             context.getLogger().debug(
-                "Append log entry and persist " + replicatedLogEntry);
+                "Append log entry and persist {} ", replicatedLogEntry);
             // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
             journal.add(replicatedLogEntry);
 
index 7150ec0e6e8fd889f383a4aaace4b250a44061bc..ae9431a43acf576f2769672b49d3ce234014e3c0 100644 (file)
@@ -96,21 +96,27 @@ public interface RaftActorContext {
     ActorSystem getActorSystem();
 
     /**
+     * Get the logger to be used for logging messages
      *
      * @return
      */
     LoggingAdapter getLogger();
 
     /**
-     * Get a mapping of peer id's their addresses
+     * Get a mapping of peerId's to their addresses
+     *
      * @return
+     *
      */
     Map<String, String> getPeerAddresses();
 
     /**
+     * Get the address of the peer as a String. This is the same format in
+     * which a consumer would provide the address
      *
      * @param peerId
-     * @return
+     * @return The address of the peer or null if the address has not yet been
+     *         resolved
      */
     String getPeerAddress(String peerId);
 
@@ -126,4 +132,27 @@ public interface RaftActorContext {
      * @param name
      */
     public void removePeer(String name);
+
+    /**
+     * Given a peerId return the corresponding actor
+     * <p>
+     *
+     *
+     * @param peerId
+     * @return The actorSelection corresponding to the peer or null if the
+     *         address has not yet been resolved
+     */
+    ActorSelection getPeerActorSelection(String peerId);
+
+    /**
+     * Set Peer Address can be called at a later time to change the address of
+     * a known peer.
+     *
+     * <p>
+     * Throws an IllegalStateException if the peer is unknown
+     *
+     * @param peerId
+     * @param peerAddress
+     */
+    void setPeerAddress(String peerId, String peerAddress);
 }
index a0f13280c28f68cb3225499924597578302489c2..833c8a9e8ad1ca7b8fe170aadffc25b3c12b6b24 100644 (file)
@@ -17,6 +17,8 @@ import akka.event.LoggingAdapter;
 
 import java.util.Map;
 
+import static com.google.common.base.Preconditions.checkState;
+
 public class RaftActorContextImpl implements RaftActorContext{
 
     private final ActorRef actor;
@@ -37,7 +39,6 @@ public class RaftActorContextImpl implements RaftActorContext{
 
     private final LoggingAdapter LOG;
 
-
     public RaftActorContextImpl(ActorRef actor, UntypedActorContext context,
         String id,
         ElectionTerm termInformation, long commitIndex,
@@ -110,12 +111,25 @@ public class RaftActorContextImpl implements RaftActorContext{
     }
 
     @Override public void addToPeers(String name, String address) {
-        LOG.debug("Kamal--> addToPeer for:"+name);
         peerAddresses.put(name, address);
     }
 
     @Override public void removePeer(String name) {
-        LOG.debug("Kamal--> removePeer for:"+name);
         peerAddresses.remove(name);
     }
+
+    @Override public ActorSelection getPeerActorSelection(String peerId) {
+        String peerAddress = getPeerAddress(peerId);
+        if(peerAddress != null){
+            return actorSelection(peerAddress);
+        }
+        return null;
+    }
+
+    @Override public void setPeerAddress(String peerId, String peerAddress) {
+        LOG.info("Peer address for peer {} set to {}", peerId, peerAddress);
+        checkState(peerAddresses.containsKey(peerId), peerId + " is unknown");
+
+        peerAddresses.put(peerId, peerAddress);
+    }
 }
@@ -6,7 +6,7 @@
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
 
-package org.opendaylight.controller.cluster.raft.internal.messages;
+package org.opendaylight.controller.cluster.raft.base.messages;
 
 import akka.actor.ActorRef;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
@@ -6,7 +6,7 @@
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
 
-package org.opendaylight.controller.cluster.raft.internal.messages;
+package org.opendaylight.controller.cluster.raft.base.messages;
 
 import akka.actor.ActorRef;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
index bc87370b159cbd52ce829f1c65df564073e5558d..f7281bb8e3967c3ca2be583adb35210cbd07e87b 100644 (file)
@@ -15,8 +15,8 @@ import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftState;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
 import org.opendaylight.controller.cluster.raft.SerializationUtils;
-import org.opendaylight.controller.cluster.raft.internal.messages.ApplyState;
-import org.opendaylight.controller.cluster.raft.internal.messages.ElectionTimeout;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
+import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
index ecd49012461a7b1ee76f4678c7451a4fa1edbfb1..c125bd32b60a5c5d714ea13e7008417f904d88dc 100644 (file)
@@ -12,16 +12,14 @@ import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftState;
-import org.opendaylight.controller.cluster.raft.internal.messages.ElectionTimeout;
+import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
 
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
+import java.util.Set;
 
 /**
  * The behavior of a RaftActor when it is in the CandidateState
@@ -43,24 +41,20 @@ import java.util.Map;
  */
 public class Candidate extends AbstractRaftActorBehavior {
 
-    private final Map<String, ActorSelection> peerToActor = new HashMap<>();
-
     private int voteCount;
 
     private final int votesRequired;
 
+    private final Set<String> peers;
+
     public Candidate(RaftActorContext context) {
         super(context);
 
-        Collection<String> peerPaths = context.getPeerAddresses().values();
+        peers = context.getPeerAddresses().keySet();
 
-        for (String peerPath : peerPaths) {
-            peerToActor.put(peerPath,
-                context.actorSelection(peerPath));
-        }
+        context.getLogger().debug("Election:Candidate has following peers:"+ peers);
 
-        context.getLogger().debug("Election:Candidate has following peers:"+peerToActor.keySet());
-        if(peerPaths.size() > 0) {
+        if(peers.size() > 0) {
             // Votes are required from a majority of the peers including self.
             // The votesRequired field therefore stores a calculated value
             // of the number of votes required for this candidate to win an
@@ -73,7 +67,7 @@ public class Candidate extends AbstractRaftActorBehavior {
             // 0 peers = 1 votesRequired (0 + 1) / 2 + 1 = 1
             // 2 peers = 2 votesRequired (2 + 1) / 2 + 1 = 2
             // 4 peers = 3 votesRequired (4 + 1) / 2 + 1 = 3
-            int noOfPeers = peerPaths.size();
+            int noOfPeers = peers.size();
             int self = 1;
             votesRequired = (noOfPeers + self) / 2 + 1;
         } else {
@@ -87,6 +81,8 @@ public class Candidate extends AbstractRaftActorBehavior {
     @Override protected RaftState handleAppendEntries(ActorRef sender,
         AppendEntries appendEntries) {
 
+        context.getLogger().info("Candidate: Received {}", appendEntries.toString());
+
         return state();
     }
 
@@ -115,10 +111,16 @@ public class Candidate extends AbstractRaftActorBehavior {
     }
 
     @Override
-    public RaftState handleMessage(ActorRef sender, Object message) {
+    public RaftState handleMessage(ActorRef sender, Object originalMessage) {
+
+        Object message = fromSerializableMessage(originalMessage);
 
         if (message instanceof RaftRPC) {
+
             RaftRPC rpc = (RaftRPC) message;
+
+            context.getLogger().debug("RaftRPC message received {} my term is {}", rpc.toString(), context.getTermInformation().getCurrentTerm());
+
             // If RPC request or response contains term T > currentTerm:
             // set currentTerm = T, convert to follower (§5.1)
             // This applies to all RPC messages and responses
@@ -141,6 +143,7 @@ public class Candidate extends AbstractRaftActorBehavior {
             scheduleElection(electionDuration());
             return state();
         }
+
         return super.handleMessage(sender, message);
     }
 
@@ -153,21 +156,25 @@ public class Candidate extends AbstractRaftActorBehavior {
 
         // Increment the election term and vote for self
         long currentTerm = context.getTermInformation().getCurrentTerm();
-        context.getTermInformation().updateAndPersist(currentTerm + 1, context.getId());
+        context.getTermInformation().updateAndPersist(currentTerm + 1,
+            context.getId());
 
-        context.getLogger().debug("Starting new term " + (currentTerm+1));
+        context.getLogger().debug("Starting new term " + (currentTerm + 1));
 
         // Request for a vote
         // TODO: Retry request for vote if replies do not arrive in a reasonable
         // amount of time TBD
-        for (ActorSelection peerActor : peerToActor.values()) {
-            peerActor.tell(new RequestVote(
-                    context.getTermInformation().getCurrentTerm(),
-                    context.getId(),
-                    context.getReplicatedLog().lastIndex(),
-                    context.getReplicatedLog().lastTerm()),
-                context.getActor()
-            );
+        for (String peerId : peers) {
+            ActorSelection peerActor = context.getPeerActorSelection(peerId);
+            if(peerActor != null) {
+                peerActor.tell(new RequestVote(
+                        context.getTermInformation().getCurrentTerm(),
+                        context.getId(),
+                        context.getReplicatedLog().lastIndex(),
+                        context.getReplicatedLog().lastTerm()),
+                    context.getActor()
+                );
+            }
         }
 
 
index 68b444b2bac7d20ae0ab7a307ede546191392938..db62dfc2ac1cd4f10c59849c0666b9291ca946a3 100644 (file)
@@ -12,8 +12,8 @@ import akka.actor.ActorRef;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftState;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
-import org.opendaylight.controller.cluster.raft.internal.messages.ApplySnapshot;
-import org.opendaylight.controller.cluster.raft.internal.messages.ElectionTimeout;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
+import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
@@ -40,6 +40,11 @@ public class Follower extends AbstractRaftActorBehavior {
     @Override protected RaftState handleAppendEntries(ActorRef sender,
         AppendEntries appendEntries) {
 
+        if(appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) {
+            context.getLogger()
+                .info("Follower: Received {}", appendEntries.toString());
+        }
+
         // TODO : Refactor this method into a bunch of smaller methods
         // to make it easier to read. Before refactoring ensure tests
         // cover the code properly
@@ -162,8 +167,9 @@ public class Follower extends AbstractRaftActorBehavior {
             for (int i = addEntriesFrom;
                  i < appendEntries.getEntries().size(); i++) {
 
-                context.getLogger().debug(
-                    "Append entry to log " + appendEntries.getEntries().get(i).getData()
+                context.getLogger().info(
+                    "Append entry to log " + appendEntries.getEntries().get(
+                        i).getData()
                         .toString()
                 );
                 context.getReplicatedLog()
index 26beed2f7a69b5191407f230c839edd1e4178ade..53e47c2f842f17ac0b2766811745a7f1ec6eabde 100644 (file)
@@ -19,10 +19,10 @@ import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftState;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
-import org.opendaylight.controller.cluster.raft.internal.messages.ApplyState;
-import org.opendaylight.controller.cluster.raft.internal.messages.Replicate;
-import org.opendaylight.controller.cluster.raft.internal.messages.SendHeartBeat;
-import org.opendaylight.controller.cluster.raft.internal.messages.SendInstallSnapshot;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
+import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
+import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
+import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
@@ -36,6 +36,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -67,7 +68,7 @@ public class Leader extends AbstractRaftActorBehavior {
     private final Map<String, FollowerLogInformation> followerToLog =
         new HashMap();
 
-    private final Map<String, ActorSelection> followerToActor = new HashMap<>();
+    private final Set<String> followers;
 
     private Cancellable heartbeatSchedule = null;
     private Cancellable appendEntriesSchedule = null;
@@ -84,22 +85,21 @@ public class Leader extends AbstractRaftActorBehavior {
             context.setCommitIndex(lastIndex());
         }
 
-        for (String followerId : context.getPeerAddresses().keySet()) {
+        followers = context.getPeerAddresses().keySet();
+
+        for (String followerId : followers) {
             FollowerLogInformation followerLogInformation =
                 new FollowerLogInformationImpl(followerId,
                     new AtomicLong(lastIndex()),
                     new AtomicLong(-1));
 
-            followerToActor.put(followerId,
-                context.actorSelection(context.getPeerAddress(followerId)));
-
             followerToLog.put(followerId, followerLogInformation);
         }
 
-        context.getLogger().debug("Election:Leader has following peers:"+followerToActor.keySet());
+        context.getLogger().debug("Election:Leader has following peers:"+ followers);
 
-        if (followerToActor.size() > 0) {
-            minReplicationCount = (followerToActor.size() + 1) / 2 + 1;
+        if (followers.size() > 0) {
+            minReplicationCount = (followers.size() + 1) / 2 + 1;
         } else {
             minReplicationCount = 0;
         }
@@ -121,16 +121,29 @@ public class Leader extends AbstractRaftActorBehavior {
     @Override protected RaftState handleAppendEntries(ActorRef sender,
         AppendEntries appendEntries) {
 
+        context.getLogger().info("Leader: Received {}", appendEntries.toString());
+
         return state();
     }
 
     @Override protected RaftState handleAppendEntriesReply(ActorRef sender,
         AppendEntriesReply appendEntriesReply) {
 
+        if(! appendEntriesReply.isSuccess()) {
+            context.getLogger()
+                .info("Leader: Received {}", appendEntriesReply.toString());
+        }
+
         // Update the FollowerLogInformation
         String followerId = appendEntriesReply.getFollowerId();
         FollowerLogInformation followerLogInformation =
             followerToLog.get(followerId);
+
+        if(followerLogInformation == null){
+            context.getLogger().error("Unknown follower {}", followerId);
+            return state();
+        }
+
         if (appendEntriesReply.isSuccess()) {
             followerLogInformation
                 .setMatchIndex(appendEntriesReply.getLogLastIndex());
@@ -251,7 +264,7 @@ public class Leader extends AbstractRaftActorBehavior {
 
         context.getLogger().debug("Replicate message " + logIndex);
 
-        if (followerToActor.size() == 0) {
+        if (followers.size() == 0) {
             context.setCommitIndex(
                 replicate.getReplicatedLogEntry().getIndex());
 
@@ -277,32 +290,37 @@ public class Leader extends AbstractRaftActorBehavior {
 
     private void sendAppendEntries() {
         // Send an AppendEntries to all followers
-        for (String followerId : followerToActor.keySet()) {
+        for (String followerId : followers) {
             ActorSelection followerActor =
-                followerToActor.get(followerId);
+                context.getPeerActorSelection(followerId);
 
-            FollowerLogInformation followerLogInformation =
-                followerToLog.get(followerId);
+            if (followerActor != null) {
+                FollowerLogInformation followerLogInformation =
+                    followerToLog.get(followerId);
 
-            long nextIndex = followerLogInformation.getNextIndex().get();
+                long nextIndex = followerLogInformation.getNextIndex().get();
 
-            List<ReplicatedLogEntry> entries = Collections.emptyList();
+                List<ReplicatedLogEntry> entries = Collections.emptyList();
 
-            if(context.getReplicatedLog().isPresent(nextIndex)){
-                // TODO: Instead of sending all entries from nextIndex
-                // only send a fixed number of entries to each follower
-                // This is to avoid the situation where there are a lot of
-                // entries to install for a fresh follower or to a follower
-                // that has fallen too far behind with the log but yet is not
-                // eligible to receive a snapshot
-                entries =
-                    context.getReplicatedLog().getFrom(nextIndex);
-            }
+                if (context.getReplicatedLog().isPresent(nextIndex)) {
+                    // TODO: Instead of sending all entries from nextIndex
+                    // only send a fixed number of entries to each follower
+                    // This is to avoid the situation where there are a lot of
+                    // entries to install for a fresh follower or to a follower
+                    // that has fallen too far behind with the log but yet is not
+                    // eligible to receive a snapshot
+                    entries =
+                        context.getReplicatedLog().getFrom(nextIndex);
+                }
 
-            followerActor.tell(
-                new AppendEntries(currentTerm(), context.getId(), prevLogIndex(nextIndex),
-                    prevLogTerm(nextIndex), entries, context.getCommitIndex()).toSerializable(),
-                actor());
+                followerActor.tell(
+                    new AppendEntries(currentTerm(), context.getId(),
+                        prevLogIndex(nextIndex),
+                        prevLogTerm(nextIndex), entries,
+                        context.getCommitIndex()).toSerializable(),
+                    actor()
+                );
+            }
         }
     }
 
@@ -312,30 +330,33 @@ public class Leader extends AbstractRaftActorBehavior {
      * snapshots at every heartbeat.
      */
     private void installSnapshotIfNeeded(){
-        for (String followerId : followerToActor.keySet()) {
+        for (String followerId : followers) {
             ActorSelection followerActor =
-                followerToActor.get(followerId);
-
-            FollowerLogInformation followerLogInformation =
-                followerToLog.get(followerId);
-
-            long nextIndex = followerLogInformation.getNextIndex().get();
-
-            if(!context.getReplicatedLog().isPresent(nextIndex) && context.getReplicatedLog().isInSnapshot(nextIndex)){
-                followerActor.tell(
-                    new InstallSnapshot(currentTerm(), context.getId(),
-                        context.getReplicatedLog().getSnapshotIndex(),
-                        context.getReplicatedLog().getSnapshotTerm(),
-                        context.getReplicatedLog().getSnapshot()
-                    ),
-                    actor()
-                );
+                context.getPeerActorSelection(followerId);
+
+            if(followerActor != null) {
+                FollowerLogInformation followerLogInformation =
+                    followerToLog.get(followerId);
+
+                long nextIndex = followerLogInformation.getNextIndex().get();
+
+                if (!context.getReplicatedLog().isPresent(nextIndex) && context
+                    .getReplicatedLog().isInSnapshot(nextIndex)) {
+                    followerActor.tell(
+                        new InstallSnapshot(currentTerm(), context.getId(),
+                            context.getReplicatedLog().getSnapshotIndex(),
+                            context.getReplicatedLog().getSnapshotTerm(),
+                            context.getReplicatedLog().getSnapshot()
+                        ),
+                        actor()
+                    );
+                }
             }
         }
     }
 
     private RaftState sendHeartBeat() {
-        if (followerToActor.size() > 0) {
+        if (followers.size() > 0) {
             sendAppendEntries();
         }
         return state();
@@ -354,7 +375,7 @@ public class Leader extends AbstractRaftActorBehavior {
     }
 
     private void scheduleHeartBeat(FiniteDuration interval) {
-        if(followerToActor.keySet().size() == 0){
+        if(followers.size() == 0){
             // Optimization - do not bother scheduling a heartbeat as there are
             // no followers
             return;
@@ -376,7 +397,7 @@ public class Leader extends AbstractRaftActorBehavior {
 
 
     private void scheduleInstallSnapshotCheck(FiniteDuration interval) {
-        if(followerToActor.keySet().size() == 0){
+        if(followers.size() == 0){
             // Optimization - do not bother scheduling a heartbeat as there are
             // no followers
             return;
index 45ac9a9ebfbecbb2fb13881dd3f365f8ac9f7e8e..94366efd5e897657744df37b0a475a847273e1b4 100644 (file)
@@ -14,7 +14,6 @@ import org.opendaylight.controller.cluster.raft.ReplicatedLogImplEntry;
 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
 import org.opendaylight.controller.cluster.raft.protobuff.messages.AppendEntriesMessages;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
@@ -24,7 +23,7 @@ import java.util.Map;
  * Invoked by leader to replicate log entries (§5.3); also used as
  * heartbeat (§5.2).
  */
-public class AppendEntries extends AbstractRaftRPC implements Serializable {
+public class AppendEntries extends AbstractRaftRPC {
 
     public static final Class SERIALIZABLE_CLASS = AppendEntriesMessages.AppendEntries.class;
 
@@ -77,13 +76,16 @@ public class AppendEntries extends AbstractRaftRPC implements Serializable {
     }
 
     @Override public String toString() {
-        return "AppendEntries{" +
-            "leaderId='" + leaderId + '\'' +
-            ", prevLogIndex=" + prevLogIndex +
-            ", prevLogTerm=" + prevLogTerm +
-            ", entries=" + entries +
-            ", leaderCommit=" + leaderCommit +
-            '}';
+        final StringBuilder sb =
+            new StringBuilder("AppendEntries{");
+        sb.append("term=").append(getTerm());
+        sb.append("leaderId='").append(leaderId).append('\'');
+        sb.append(", prevLogIndex=").append(prevLogIndex);
+        sb.append(", prevLogTerm=").append(prevLogTerm);
+        sb.append(", entries=").append(entries);
+        sb.append(", leaderCommit=").append(leaderCommit);
+        sb.append('}');
+        return sb.toString();
     }
 
     public <T extends Object> Object toSerializable(){
index 7e6628abe1464800ef2449501a165a89c19cbcb3..b923baa7168df3d34ae7064c722222b66bbf9c51 100644 (file)
@@ -8,12 +8,10 @@
 
 package org.opendaylight.controller.cluster.raft.messages;
 
-import java.io.Serializable;
-
 /**
  * Reply for the AppendEntriesRpc message
  */
-public class AppendEntriesReply extends AbstractRaftRPC implements Serializable {
+public class AppendEntriesReply extends AbstractRaftRPC {
 
     // true if follower contained entry matching
     // prevLogIndex and prevLogTerm
@@ -58,4 +56,16 @@ public class AppendEntriesReply extends AbstractRaftRPC implements Serializable
     public String getFollowerId() {
         return followerId;
     }
+
+    @Override public String toString() {
+        final StringBuilder sb =
+            new StringBuilder("AppendEntriesReply{");
+        sb.append("term=").append(term);
+        sb.append(", success=").append(success);
+        sb.append(", logLastIndex=").append(logLastIndex);
+        sb.append(", logLastTerm=").append(logLastTerm);
+        sb.append(", followerId='").append(followerId).append('\'');
+        sb.append('}');
+        return sb.toString();
+    }
 }
index 5053560b411c014e9dd756c48fd84e9a17ab4cab..888854fa71eaf4f745e11456c7b09349a7d8a443 100644 (file)
@@ -8,9 +8,7 @@
 
 package org.opendaylight.controller.cluster.raft.messages;
 
-import java.io.Serializable;
-
-public class InstallSnapshot extends AbstractRaftRPC implements Serializable {
+public class InstallSnapshot extends AbstractRaftRPC {
 
     private final String leaderId;
     private final long lastIncludedIndex;
index 02a3252776167580b3bfa444ee765223ad6ded3b..85b89b70ae2b84c8ab26fcb7e16ba041fbd2040a 100644 (file)
@@ -8,9 +8,7 @@
 
 package org.opendaylight.controller.cluster.raft.messages;
 
-import java.io.Serializable;
-
-public class InstallSnapshotReply extends AbstractRaftRPC implements Serializable {
+public class InstallSnapshotReply extends AbstractRaftRPC {
 
     // The followerId - this will be used to figure out which follower is
     // responding
index a770e54f580a971b679b923ef5d44e37d9fc98dd..10d99988d8bf6ac58a16a784a4fcc00f6c1d8ece 100644 (file)
@@ -8,6 +8,8 @@
 
 package org.opendaylight.controller.cluster.raft.messages;
 
-public interface RaftRPC {
+import java.io.Serializable;
+
+public interface RaftRPC extends Serializable {
     public long getTerm();
 }
index 310968de950bf2f5e4ff31bcbcc3d7718ad618a8..6ef2a06285fedb85997f79b3763d367d46ceb663 100644 (file)
@@ -8,12 +8,10 @@
 
 package org.opendaylight.controller.cluster.raft.messages;
 
-import java.io.Serializable;
-
 /**
  * Invoked by candidates to gather votes (§5.2).
  */
-public class RequestVote extends AbstractRaftRPC implements Serializable{
+public class RequestVote extends AbstractRaftRPC {
 
     // candidate requesting vote
     private String candidateId;
@@ -63,4 +61,15 @@ public class RequestVote extends AbstractRaftRPC implements Serializable{
     public void setLastLogTerm(long lastLogTerm) {
         this.lastLogTerm = lastLogTerm;
     }
+
+    @Override public String toString() {
+        final StringBuilder sb =
+            new StringBuilder("RequestVote{");
+        sb.append("term='").append(getTerm()).append('\'');
+        sb.append("candidateId='").append(candidateId).append('\'');
+        sb.append(", lastLogIndex=").append(lastLogIndex);
+        sb.append(", lastLogTerm=").append(lastLogTerm);
+        sb.append('}');
+        return sb.toString();
+    }
 }
index a658ab7d810685c53b90ed6955ce5df5273b1b10..df80b4e5e24e34f0d9d4a3017d7dbfdea1b181ea 100644 (file)
@@ -8,9 +8,7 @@
 
 package org.opendaylight.controller.cluster.raft.messages;
 
-import java.io.Serializable;
-
-public class RequestVoteReply extends AbstractRaftRPC implements Serializable {
+public class RequestVoteReply extends AbstractRaftRPC {
 
     // true means candidate received vot
     private final boolean voteGranted;
index 406c164cdbd66acbc7ce452ad6ee4640ec4521b3..2e200cba275d9d4265f56da69ee36e61acd0db76 100644 (file)
@@ -18,6 +18,7 @@ import com.google.protobuf.GeneratedMessage;
 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
 import org.opendaylight.controller.cluster.raft.protobuff.messages.AppendEntriesMessages;
 import org.opendaylight.controller.cluster.raft.protobuff.messages.MockPayloadMessages;
+import com.google.common.base.Preconditions;
 
 import java.io.Serializable;
 import java.util.ArrayList;
@@ -159,6 +160,19 @@ public class MockRaftActorContext implements RaftActorContext {
         peerAddresses.remove(name);
     }
 
+    @Override public ActorSelection getPeerActorSelection(String peerId) {
+        String peerAddress = getPeerAddress(peerId);
+        if(peerAddress != null){
+            return actorSelection(peerAddress);
+        }
+        return null;
+    }
+
+    @Override public void setPeerAddress(String peerId, String peerAddress) {
+        Preconditions.checkState(peerAddresses.containsKey(peerId));
+        peerAddresses.put(peerId, peerAddress);
+    }
+
     public void setPeerAddresses(Map<String, String> peerAddresses) {
         this.peerAddresses = peerAddresses;
     }
index 8bcee58afe11b40c03bf8fa36eeb0b338a1ae3ff..c76368370506af15ab4b7b567672514514e41690 100644 (file)
@@ -9,7 +9,7 @@ import org.junit.Test;
 import org.opendaylight.controller.cluster.raft.MockRaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftState;
-import org.opendaylight.controller.cluster.raft.internal.messages.ElectionTimeout;
+import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
index 126e7d10c25a1981ab3e4ba30021e68cf07df214..c015d950c42055799bdd5826f0cca1aaf398f357 100644 (file)
@@ -9,7 +9,7 @@ import org.opendaylight.controller.cluster.raft.MockRaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftState;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
-import org.opendaylight.controller.cluster.raft.internal.messages.ElectionTimeout;
+import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
index bc9d4a2767b130e1afde72c40641ae90af9fd31a..d33b33925b5e7aba358152ff6067e4842fec3b71 100644 (file)
@@ -8,9 +8,9 @@ import org.junit.Test;
 import org.opendaylight.controller.cluster.raft.MockRaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftState;
-import org.opendaylight.controller.cluster.raft.internal.messages.ApplyState;
-import org.opendaylight.controller.cluster.raft.internal.messages.Replicate;
-import org.opendaylight.controller.cluster.raft.internal.messages.SendHeartBeat;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
+import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
+import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.utils.DoNothingActor;
 
index 3c364b35e1af9c47028a2874f2372ef8f018cb82..2d81b6022d35af1c70fba4adb269ffc2d8c93dce 100644 (file)
@@ -7,11 +7,15 @@
  */
 package org.opendaylight.controller.md.sal.binding.impl;
 
+import com.google.common.base.Objects;
+import com.google.common.base.Optional;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -31,6 +35,7 @@ import org.opendaylight.yangtools.concepts.Delegator;
 import org.opendaylight.yangtools.concepts.ListenerRegistration;
 import org.opendaylight.yangtools.yang.binding.DataObject;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.opendaylight.yangtools.yang.data.impl.codec.BindingIndependentMappingService;
 import org.opendaylight.yangtools.yang.data.impl.codec.DeserializationException;
@@ -39,12 +44,7 @@ import org.opendaylight.yangtools.yang.model.api.SchemaContextListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Objects;
-import com.google.common.base.Optional;
-import com.google.common.collect.Iterables;
-
-public abstract class AbstractForwardedDataBroker implements Delegator<DOMDataBroker>, DomForwardedBroker,
-        SchemaContextListener, AutoCloseable {
+public abstract class AbstractForwardedDataBroker implements Delegator<DOMDataBroker>, DomForwardedBroker, SchemaContextListener, AutoCloseable {
 
     private static final Logger LOG = LoggerFactory.getLogger(AbstractForwardedDataBroker.class);
     // The Broker to whom we do all forwarding
@@ -90,17 +90,17 @@ public abstract class AbstractForwardedDataBroker implements Delegator<DOMDataBr
             final DataChangeScope triggeringScope) {
         DOMDataChangeListener domDataChangeListener = new TranslatingDataChangeInvoker(store, path, listener,
                 triggeringScope);
-        org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier domPath = codec.toNormalized(path);
+        YangInstanceIdentifier domPath = codec.toNormalized(path);
         ListenerRegistration<DOMDataChangeListener> domRegistration = domDataBroker.registerDataChangeListener(store,
                 domPath, domDataChangeListener, triggeringScope);
         return new ListenerRegistrationImpl(listener, domRegistration);
     }
 
     protected Map<InstanceIdentifier<?>, DataObject> toBinding(
-            final Map<org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier, ? extends NormalizedNode<?, ?>> normalized) {
+            final Map<YangInstanceIdentifier, ? extends NormalizedNode<?, ?>> normalized) {
         Map<InstanceIdentifier<?>, DataObject> newMap = new HashMap<>();
 
-        for (Map.Entry<org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier, ? extends NormalizedNode<?, ?>> entry : sortedEntries(normalized)) {
+        for (Map.Entry<YangInstanceIdentifier, ? extends NormalizedNode<?, ?>> entry : sortedEntries(normalized)) {
             try {
                 Optional<Entry<InstanceIdentifier<? extends DataObject>, DataObject>> potential = getCodec().toBinding(
                         entry);
@@ -115,25 +115,43 @@ public abstract class AbstractForwardedDataBroker implements Delegator<DOMDataBr
         return newMap;
     }
 
-    private static <T> Iterable<Entry<org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier,T>> sortedEntries(final Map<org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier, T> map) {
-        ArrayList<Entry<org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier, T>> entries = new ArrayList<>(map.entrySet());
-        Collections.sort(entries, new Comparator<Entry<org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier, T>>() {
+    private static final Comparator<Entry<YangInstanceIdentifier, ?>> MAP_ENTRY_COMPARATOR = new Comparator<Entry<YangInstanceIdentifier, ?>>() {
+        @Override
+        public int compare(final Entry<YangInstanceIdentifier, ?> left,
+                final Entry<YangInstanceIdentifier, ?> right) {
+            final Iterator<?> li = left.getKey().getPathArguments().iterator();
+            final Iterator<?> ri = right.getKey().getPathArguments().iterator();
+
+            // Iterate until left is exhausted...
+            while (li.hasNext()) {
+                if (!ri.hasNext()) {
+                    // Left is deeper
+                    return 1;
+                }
 
-            @Override
-            public int compare(final Entry<org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier, T> left,
-                    final Entry<org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier, T> right) {
-                int leftSize = Iterables.size(left.getKey().getPathArguments());
-                int rightSize = Iterables.size(right.getKey().getPathArguments());
-                return Integer.compare(leftSize, rightSize);
+                li.next();
+                ri.next();
             }
-        });
-        return entries;
+
+            // Check if right is exhausted
+            return ri.hasNext() ? -1 : 0;
+        }
+    };
+
+    private static <T> Iterable<Entry<YangInstanceIdentifier,T>> sortedEntries(final Map<YangInstanceIdentifier, T> map) {
+        if (!map.isEmpty()) {
+            ArrayList<Entry<YangInstanceIdentifier, T>> entries = new ArrayList<>(map.entrySet());
+            Collections.sort(entries, MAP_ENTRY_COMPARATOR);
+            return entries;
+        } else {
+            return Collections.emptySet();
+        }
     }
 
     protected Set<InstanceIdentifier<?>> toBinding(
-            final Set<org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier> normalized) {
+            final Set<YangInstanceIdentifier> normalized) {
         Set<InstanceIdentifier<?>> hashSet = new HashSet<>();
-        for (org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier normalizedPath : normalized) {
+        for (YangInstanceIdentifier normalizedPath : normalized) {
             try {
                 Optional<InstanceIdentifier<? extends DataObject>> potential = getCodec().toBinding(normalizedPath);
                 if (potential.isPresent()) {
@@ -175,13 +193,13 @@ public abstract class AbstractForwardedDataBroker implements Delegator<DOMDataBr
 
         @Override
         public void onDataChanged(
-                final AsyncDataChangeEvent<org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier, NormalizedNode<?, ?>> change) {
+                final AsyncDataChangeEvent<YangInstanceIdentifier, NormalizedNode<?, ?>> change) {
             bindingDataChangeListener.onDataChanged(new TranslatedDataChangeEvent(change, path));
         }
     }
 
     private class TranslatedDataChangeEvent implements AsyncDataChangeEvent<InstanceIdentifier<?>, DataObject> {
-        private final AsyncDataChangeEvent<org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier, NormalizedNode<?, ?>> domEvent;
+        private final AsyncDataChangeEvent<YangInstanceIdentifier, NormalizedNode<?, ?>> domEvent;
         private final InstanceIdentifier<?> path;
 
         private Map<InstanceIdentifier<?>, DataObject> createdCache;
@@ -192,7 +210,7 @@ public abstract class AbstractForwardedDataBroker implements Delegator<DOMDataBr
         private Optional<DataObject> updatedDataCache;
 
         public TranslatedDataChangeEvent(
-                final AsyncDataChangeEvent<org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier, NormalizedNode<?, ?>> change,
+                final AsyncDataChangeEvent<YangInstanceIdentifier, NormalizedNode<?, ?>> change,
                 final InstanceIdentifier<?> path) {
             this.domEvent = change;
             this.path = path;
index 16b97b7855eb4755b63ebe6bac07784e460d20c5..94c895dab385a92e3f547099ac83010cb7e517ec 100644 (file)
         <version>1.1-SNAPSHOT</version>
     </dependency>
 
+    <dependency>
+        <groupId>org.opendaylight.controller</groupId>
+        <artifactId>sal-akka-raft</artifactId>
+        <version>1.1-SNAPSHOT</version>
+    </dependency>
+
     <!-- Test Dependencies -->
     <dependency>
       <groupId>junit</groupId>
             <Import-Package>!*snappy;!org.jboss.*;*</Import-Package>
             <Embed-Dependency>
                 sal-protocolbuffer-encoding;
+                sal-akka-raft;
                 !sal*;
                 !*config-api*;
                 !*testkit*;
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/CompositeModificationPayload.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/CompositeModificationPayload.java
new file mode 100644 (file)
index 0000000..955e4bb
--- /dev/null
@@ -0,0 +1,53 @@
+/*
+ * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+
+package org.opendaylight.controller.cluster.datastore;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.GeneratedMessage;
+import org.opendaylight.controller.cluster.example.protobuff.messages.KeyValueMessages;
+import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
+import org.opendaylight.controller.cluster.raft.protobuff.messages.AppendEntriesMessages;
+import org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+public class CompositeModificationPayload extends Payload implements
+    Serializable {
+
+    private final PersistentMessages.CompositeModification modification;
+
+    public CompositeModificationPayload(){
+        modification = null;
+    }
+    public CompositeModificationPayload(Object modification){
+        this.modification = (PersistentMessages.CompositeModification) modification;
+    }
+
+    @Override public Map<GeneratedMessage.GeneratedExtension, PersistentMessages.CompositeModification> encode() {
+        Preconditions.checkState(modification!=null);
+        Map<GeneratedMessage.GeneratedExtension, PersistentMessages.CompositeModification> map = new HashMap<>();
+        map.put(org.opendaylight.controller.mdsal.CompositeModificationPayload.modification, this.modification);
+        return map;
+    }
+
+    @Override public Payload decode(
+        AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload payload) {
+        PersistentMessages.CompositeModification modification = payload
+            .getExtension(
+                org.opendaylight.controller.mdsal.CompositeModificationPayload.modification);
+        payload.getExtension(KeyValueMessages.value);
+        return new CompositeModificationPayload(modification);
+    }
+
+    public Object getModification(){
+        return this.modification;
+    }
+}
index a67f58c760bbdaeda30f2f6b9c32512537ee52c0..34239070a3dfafe15c8ba7e71c630eaf56136e3b 100644 (file)
@@ -15,9 +15,41 @@ import java.util.List;
 import java.util.Map;
 
 public interface Configuration {
+
+    /**
+     * Given a memberName find all the shards that belong on that member and
+     * return the names of those shards
+     *
+     * @param memberName
+     * @return
+     */
     List<String> getMemberShardNames(String memberName);
+
+    /**
+     * Given a module namespace return the name of a module
+     * @param nameSpace
+     * @return
+     */
     Optional<String> getModuleNameFromNameSpace(String nameSpace);
+
+    /**
+     * Get a mapping of the module names to it's corresponding ShardStrategy
+     * @return
+     */
     Map<String, ShardStrategy> getModuleNameToShardStrategyMap();
+
+    /**
+     * Given a module name find all the shardNames corresponding to it
+     * @param moduleName
+     * @return
+     */
     List<String> getShardNamesFromModuleName(String moduleName);
+
+    /**
+     * Given a shardName find all the members on which it belongs
+     *
+     * @param shardName
+     * @return
+     */
     List<String> getMembersFromShardName(String shardName);
 }
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/NoOpCohort.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/NoOpCohort.java
new file mode 100644 (file)
index 0000000..eb28159
--- /dev/null
@@ -0,0 +1,40 @@
+/*
+ * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+
+package org.opendaylight.controller.cluster.datastore;
+
+import akka.actor.UntypedActor;
+import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction;
+import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply;
+import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransaction;
+import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransactionReply;
+import org.opendaylight.controller.cluster.datastore.messages.CommitTransaction;
+import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
+import org.opendaylight.controller.cluster.datastore.messages.PreCommitTransaction;
+import org.opendaylight.controller.cluster.datastore.messages.PreCommitTransactionReply;
+
+public class NoOpCohort extends UntypedActor {
+
+    @Override public void onReceive(Object message) throws Exception {
+        if (message.getClass().equals(CanCommitTransaction.SERIALIZABLE_CLASS)) {
+            getSender().tell(new CanCommitTransactionReply(false).toSerializable(), getSelf());
+        } else if (message.getClass().equals(PreCommitTransaction.SERIALIZABLE_CLASS)) {
+            getSender().tell(
+                new PreCommitTransactionReply().toSerializable(),
+                getSelf());
+        } else if (message.getClass().equals(CommitTransaction.SERIALIZABLE_CLASS)) {
+            getSender().tell(new CommitTransactionReply().toSerializable(), getSelf());
+        } else if (message.getClass().equals(AbortTransaction.SERIALIZABLE_CLASS)) {
+            getSender().tell(new AbortTransactionReply().toSerializable(), getSelf());
+        } else {
+            throw new Exception ("Not recognized message received,message="+message);
+        }
+
+    }
+}
+
index d6ad553cf3a237002645811dad1b299c3f2fd309..999d0f8bafca9639baa70ea7c893656f87704ae7 100644 (file)
@@ -14,9 +14,7 @@ import akka.actor.Props;
 import akka.event.Logging;
 import akka.event.LoggingAdapter;
 import akka.japi.Creator;
-import akka.persistence.Persistent;
-import akka.persistence.RecoveryCompleted;
-import akka.persistence.UntypedProcessor;
+import akka.serialization.Serialization;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
@@ -28,12 +26,13 @@ import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionC
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionChainReply;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.ForwardedCommitTransaction;
-import org.opendaylight.controller.cluster.datastore.messages.NonPersistent;
+import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener;
 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply;
 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
 import org.opendaylight.controller.cluster.datastore.modification.Modification;
 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
+import org.opendaylight.controller.cluster.raft.RaftActor;
 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
@@ -54,7 +53,7 @@ import java.util.concurrent.Executors;
  * Our Shard uses InMemoryDataStore as it's internal representation and delegates all requests it
  * </p>
  */
-public class Shard extends UntypedProcessor {
+public class Shard extends RaftActor {
 
     public static final String DEFAULT_NAME = "default";
 
@@ -73,11 +72,16 @@ public class Shard extends UntypedProcessor {
     // property persistent
     private final boolean persistent;
 
+    private final String name;
+
     private SchemaContext schemaContext;
 
     private final ShardStats shardMBean;
 
-    private Shard(String name) {
+    private Shard(String name, Map<String, String> peerAddresses) {
+        super(name, peerAddresses);
+
+        this.name = name;
 
         String setting = System.getProperty("shard.persistent");
 
@@ -91,78 +95,88 @@ public class Shard extends UntypedProcessor {
 
     }
 
-    public static Props props(final String name) {
+    public static Props props(final String name, final Map<String, String> peerAddresses) {
         return Props.create(new Creator<Shard>() {
 
             @Override
             public Shard create() throws Exception {
-                return new Shard(name);
+                return new Shard(name, peerAddresses);
             }
 
         });
     }
 
 
-    @Override
-    public void onReceive(Object message) throws Exception {
-        LOG.debug("Received message " + message.getClass().toString());
-
-        if(!recoveryFinished()){
-            // FIXME : Properly handle recovery
-            return;
-        }
+    @Override public void onReceiveCommand(Object message){
+        LOG.debug("Received message {} from {}", message.getClass().toString(), getSender());
 
         if (message.getClass().equals(CreateTransactionChain.SERIALIZABLE_CLASS)) {
-            createTransactionChain();
+            if(isLeader()) {
+                createTransactionChain();
+            } else if(getLeader() != null){
+                getLeader().forward(message, getContext());
+            }
         } else if (message.getClass().equals(RegisterChangeListener.SERIALIZABLE_CLASS)) {
             registerChangeListener(RegisterChangeListener.fromSerializable(getContext().system(), message));
         } else if (message instanceof UpdateSchemaContext) {
             updateSchemaContext((UpdateSchemaContext) message);
         } else if (message instanceof ForwardedCommitTransaction) {
             handleForwardedCommit((ForwardedCommitTransaction) message);
-        } else if (message instanceof Persistent) {
-            commit(((Persistent)message).payload());
         } else if (message.getClass().equals(CreateTransaction.SERIALIZABLE_CLASS)) {
-            createTransaction(CreateTransaction.fromSerializable(message));
-        } else if(message instanceof NonPersistent){
-            commit(((NonPersistent)message).payload());
-        }else if (message instanceof RecoveryCompleted) {
-            //FIXME: PROPERLY HANDLE RECOVERY COMPLETED
-
-        }else {
-          throw new Exception("Not recognized message found message=" + message);
+            if(isLeader()) {
+                createTransaction(CreateTransaction.fromSerializable(message));
+            } else if(getLeader() != null){
+                getLeader().forward(message, getContext());
+            }
+        } else if (message instanceof PeerAddressResolved){
+            PeerAddressResolved resolved = (PeerAddressResolved) message;
+            setPeerAddress(resolved.getPeerId(), resolved.getPeerAddress());
+        } else {
+          super.onReceiveCommand(message);
         }
     }
 
     private void createTransaction(CreateTransaction createTransaction) {
         DOMStoreReadWriteTransaction transaction =
             store.newReadWriteTransaction();
+        String transactionId = "shard-" + createTransaction.getTransactionId();
+        LOG.info("Creating transaction : {} " , transactionId);
         ActorRef transactionActor = getContext().actorOf(
-            ShardTransaction.props(transaction, getSelf(), schemaContext), "shard-" + createTransaction.getTransactionId());
+            ShardTransaction.props(transaction, getSelf(), schemaContext), transactionId);
+
         getSender()
-            .tell(new CreateTransactionReply(transactionActor.path().toString(), createTransaction.getTransactionId()).toSerializable(),
+            .tell(new CreateTransactionReply(Serialization.serializedActorPath(transactionActor), createTransaction.getTransactionId()).toSerializable(),
                 getSelf());
     }
 
-    private void commit(Object serialized) {
+    private void commit(final ActorRef sender, Object serialized) {
         Modification modification = MutableCompositeModification.fromSerializable(serialized, schemaContext);
         DOMStoreThreePhaseCommitCohort cohort =
             modificationToCohort.remove(serialized);
         if (cohort == null) {
             LOG.error(
                 "Could not find cohort for modification : " + modification);
+            LOG.info("Writing modification using a new transaction");
+            modification.apply(store.newReadWriteTransaction());
             return;
         }
+
         final ListenableFuture<Void> future = cohort.commit();
         shardMBean.incrementCommittedTransactionCount();
-        final ActorRef sender = getSender();
         final ActorRef self = getSelf();
         future.addListener(new Runnable() {
             @Override
             public void run() {
                 try {
                     future.get();
-                    sender.tell(new CommitTransactionReply().toSerializable(), self);
+
+                    if(sender != null) {
+                        sender
+                            .tell(new CommitTransactionReply().toSerializable(),
+                                self);
+                    } else {
+                        LOG.error("sender is null ???");
+                    }
                 } catch (InterruptedException | ExecutionException e) {
                     // FIXME : Handle this properly
                     LOG.error(e, "An exception happened when committing");
@@ -176,12 +190,11 @@ public class Shard extends UntypedProcessor {
 
         modificationToCohort
             .put(serializedModification , message.getCohort());
+
         if(persistent) {
-            getSelf().forward(Persistent.create(serializedModification),
-                getContext());
+            this.persistData(getSender(), "identifier", new CompositeModificationPayload(serializedModification));
         } else {
-            getSelf().forward(NonPersistent.create(serializedModification),
-                getContext());
+            this.commit(getSender(), serializedModification);
         }
     }
 
@@ -197,7 +210,8 @@ public class Shard extends UntypedProcessor {
 
 
         ActorSelection dataChangeListenerPath = getContext()
-            .system().actorSelection(registerChangeListener.getDataChangeListenerPath());
+            .system().actorSelection(
+                registerChangeListener.getDataChangeListenerPath());
 
         AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>
             listener = new DataChangeListenerProxy(schemaContext,dataChangeListenerPath);
@@ -220,9 +234,36 @@ public class Shard extends UntypedProcessor {
     private void createTransactionChain() {
         DOMStoreTransactionChain chain = store.createTransactionChain();
         ActorRef transactionChain =
-            getContext().actorOf(ShardTransactionChain.props(chain, schemaContext));
+            getContext().actorOf(
+                ShardTransactionChain.props(chain, schemaContext));
         getSender()
-            .tell(new CreateTransactionChainReply(transactionChain.path()).toSerializable(),
+            .tell(new CreateTransactionChainReply(transactionChain.path())
+                .toSerializable(),
                 getSelf());
     }
+
+    @Override protected void applyState(ActorRef clientActor, String identifier,
+        Object data) {
+
+        if(data instanceof CompositeModificationPayload){
+            Object modification =
+                ((CompositeModificationPayload) data).getModification();
+            commit(clientActor, modification);
+        } else {
+            LOG.error("Unknown state received {}", data);
+        }
+
+    }
+
+    @Override protected Object createSnapshot() {
+        throw new UnsupportedOperationException("createSnapshot");
+    }
+
+    @Override protected void applySnapshot(Object snapshot) {
+        throw new UnsupportedOperationException("applySnapshot");
+    }
+
+    @Override public String persistenceId() {
+        return this.name;
+    }
 }
index 0363b3ceb370772ca617bb50abcddef95ed7da5e..5fbce4cd98900be65053939ca7a51d4f7a929a98 100644 (file)
@@ -19,6 +19,7 @@ import akka.japi.Creator;
 import akka.japi.Function;
 import com.google.common.base.Preconditions;
 import org.opendaylight.controller.cluster.datastore.messages.FindPrimary;
+import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
 import org.opendaylight.controller.cluster.datastore.messages.PrimaryFound;
 import org.opendaylight.controller.cluster.datastore.messages.PrimaryNotFound;
 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
@@ -56,7 +57,7 @@ public class ShardManager extends AbstractUntypedActor {
     // Stores a mapping between a member name and the address of the member
     private final Map<String, Address> memberNameToAddress = new HashMap<>();
 
-    private final Map<String, ActorPath> localShards = new HashMap<>();
+    private final Map<String, ShardInformation> localShards = new HashMap<>();
 
 
     private final String type;
@@ -125,14 +126,20 @@ public class ShardManager extends AbstractUntypedActor {
     }
 
     private void memberUp(ClusterEvent.MemberUp message) {
-        memberNameToAddress.put(message.member().roles().head(), message.member().address());
+        String memberName = message.member().roles().head();
+
+        memberNameToAddress.put(memberName , message.member().address());
+
+        for(ShardInformation info : localShards.values()){
+            String shardName = info.getShardName();
+            info.updatePeerAddress(getShardActorName(memberName, shardName),
+                getShardActorPath(shardName, memberName));
+        }
     }
 
     private void updateSchemaContext(Object message) {
-        for(ActorPath path : localShards.values()){
-            getContext().system().actorSelection(path)
-                .forward(message,
-                    getContext());
+        for(ShardInformation info : localShards.values()){
+            info.getActor().tell(message,getSelf());
         }
     }
 
@@ -142,35 +149,50 @@ public class ShardManager extends AbstractUntypedActor {
         List<String> members =
             configuration.getMembersFromShardName(shardName);
 
-        for(String memberName : members) {
-            if (memberName.equals(cluster.getCurrentMemberName())) {
-                // This is a local shard
-                ActorPath shardPath = localShards.get(shardName);
-                if (shardPath == null) {
-                    getSender()
-                        .tell(new PrimaryNotFound(shardName).toSerializable(), getSelf());
-                    return;
-                }
-                getSender().tell(new PrimaryFound(shardPath.toString()).toSerializable(),
-                    getSelf());
+        // First see if the there is a local replica for the shard
+        ShardInformation info = localShards.get(shardName);
+        if(info != null) {
+            ActorPath shardPath = info.getActorPath();
+            if (shardPath != null) {
+                getSender()
+                    .tell(
+                        new PrimaryFound(shardPath.toString()).toSerializable(),
+                        getSelf());
                 return;
-            } else {
-                Address address = memberNameToAddress.get(memberName);
-                if(address != null){
-                    String path =
-                        address.toString() + "/user/shardmanager-" + this.type + "/" + getShardActorName(
-                            memberName, shardName);
-                    getSender().tell(new PrimaryFound(path).toSerializable(), getSelf());
-                    return;
-                }
+            }
+        }
 
+        if(cluster.getCurrentMemberName() != null) {
+            members.remove(cluster.getCurrentMemberName());
+        }
 
+        // There is no way for us to figure out the primary (for now) so assume
+        // that one of the remote nodes is a primary
+        for(String memberName : members) {
+            Address address = memberNameToAddress.get(memberName);
+            if(address != null){
+                String path =
+                    getShardActorPath(shardName, memberName);
+                getSender().tell(new PrimaryFound(path).toSerializable(), getSelf());
+                return;
             }
         }
-
         getSender().tell(new PrimaryNotFound(shardName).toSerializable(), getSelf());
     }
 
+    private String
+
+
+    getShardActorPath(String shardName, String memberName) {
+        Address address = memberNameToAddress.get(memberName);
+        if(address != null) {
+            return address.toString() + "/user/shardmanager-" + this.type + "/"
+                + getShardActorName(
+                memberName, shardName);
+        }
+        return null;
+    }
+
     private String getShardActorName(String memberName, String shardName){
         return memberName + "-shard-" + shardName + "-" + this.type;
     }
@@ -183,14 +205,35 @@ public class ShardManager extends AbstractUntypedActor {
 
         for(String shardName : memberShardNames){
             String shardActorName = getShardActorName(memberName, shardName);
+            Map<String, String> peerAddresses = getPeerAddresses(shardName);
             ActorRef actor = getContext()
-                .actorOf(Shard.props(shardActorName), shardActorName);
-            ActorPath path = actor.path();
-            localShards.put(shardName, path);
+                .actorOf(Shard.props(shardActorName, peerAddresses),
+                    shardActorName);
+            localShards.put(shardName, new ShardInformation(shardName, actor, peerAddresses));
         }
 
     }
 
+    private Map<String, String> getPeerAddresses(String shardName){
+
+        Map<String, String> peerAddresses = new HashMap<>();
+
+        List<String> members =
+            this.configuration.getMembersFromShardName(shardName);
+
+        String currentMemberName = this.cluster.getCurrentMemberName();
+
+        for(String memberName : members){
+            if(!currentMemberName.equals(memberName)){
+                String shardActorName = getShardActorName(memberName, shardName);
+                String path =
+                    getShardActorPath(shardName, currentMemberName);
+                peerAddresses.put(shardActorName, path);
+            }
+        }
+        return peerAddresses;
+    }
+
 
     @Override
     public SupervisorStrategy supervisorStrategy() {
@@ -204,4 +247,49 @@ public class ShardManager extends AbstractUntypedActor {
         );
 
     }
+
+    private class ShardInformation {
+        private final String shardName;
+        private final ActorRef actor;
+        private final ActorPath actorPath;
+        private final Map<String, String> peerAddresses;
+
+        private ShardInformation(String shardName, ActorRef actor,
+            Map<String, String> peerAddresses) {
+            this.shardName = shardName;
+            this.actor = actor;
+            this.actorPath = actor.path();
+            this.peerAddresses = peerAddresses;
+        }
+
+        public String getShardName() {
+            return shardName;
+        }
+
+        public ActorRef getActor(){
+            return actor;
+        }
+
+        public ActorPath getActorPath() {
+            return actorPath;
+        }
+
+        public Map<String, String> getPeerAddresses() {
+            return peerAddresses;
+        }
+
+        public void updatePeerAddress(String peerId, String peerAddress){
+            LOG.info("updatePeerAddress for peer {} with address {}", peerId, peerAddress);
+            if(peerAddresses.containsKey(peerId)){
+                peerAddresses.put(peerId, peerAddress);
+
+                LOG.info("Sending PeerAddressResolved for peer {} with address {} to {}", peerId, peerAddress, actor.path());
+
+                actor
+                    .tell(new PeerAddressResolved(peerId, peerAddress),
+                        getSelf());
+
+            }
+        }
+    }
 }
index 5a0049aa6d05acd5f0e9114ff93e5dcc4ca53dad..c85d32012fed9ee036a96b2e3663c061eabd0385 100644 (file)
@@ -9,10 +9,14 @@
 package org.opendaylight.controller.cluster.datastore;
 
 import akka.actor.ActorPath;
+import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
+import akka.actor.Props;
 import com.google.common.base.Optional;
+import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListenableFutureTask;
+import org.opendaylight.controller.cluster.datastore.exceptions.TimeoutException;
 import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
@@ -30,6 +34,8 @@ import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCoh
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -60,6 +66,10 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
     private static final AtomicLong counter = new AtomicLong();
 
+    private static final Logger
+        LOG = LoggerFactory.getLogger(TransactionProxy.class);
+
+
     private final TransactionType transactionType;
     private final ActorContext actorContext;
     private final Map<String, TransactionContext> remoteTransactionPaths = new HashMap<>();
@@ -72,7 +82,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         TransactionType transactionType,
         ExecutorService executor,
         SchemaContext schemaContext
-        ) {
+    ) {
 
         this.identifier = actorContext.getCurrentMemberName() + "-txn-" + counter.getAndIncrement();
         this.transactionType = transactionType;
@@ -88,33 +98,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         createTransactionIfMissing(actorContext, path);
 
-        final ActorSelection remoteTransaction = remoteTransactionFromIdentifier(path);
-
-        Callable<Optional<NormalizedNode<?,?>>> call = new Callable() {
-
-            @Override public Optional<NormalizedNode<?,?>> call() throws Exception {
-                Object response = actorContext
-                    .executeRemoteOperation(remoteTransaction, new ReadData(path).toSerializable(),
-                        ActorContext.ASK_DURATION);
-                if(response.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)){
-                    ReadDataReply reply = ReadDataReply.fromSerializable(schemaContext,path, response);
-                    if(reply.getNormalizedNode() == null){
-                        return Optional.absent();
-                    }
-                    //FIXME : A cast should not be required here ???
-                    return (Optional<NormalizedNode<?, ?>>) Optional.of(reply.getNormalizedNode());
-                }
-
-                return Optional.absent();
-            }
-        };
-
-        ListenableFutureTask<Optional<NormalizedNode<?, ?>>>
-            future = ListenableFutureTask.create(call);
-
-        executor.submit(future);
-
-        return future;
+        return transactionContext(path).readData(path);
     }
 
     @Override
@@ -122,8 +106,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         createTransactionIfMissing(actorContext, path);
 
-        final ActorSelection remoteTransaction = remoteTransactionFromIdentifier(path);
-        remoteTransaction.tell(new WriteData(path, data, schemaContext).toSerializable(), null);
+        transactionContext(path).writeData(path, data);
     }
 
     @Override
@@ -131,8 +114,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         createTransactionIfMissing(actorContext, path);
 
-        final ActorSelection remoteTransaction = remoteTransactionFromIdentifier(path);
-        remoteTransaction.tell(new MergeData(path, data, schemaContext).toSerializable(), null);
+        transactionContext(path).mergeData(path, data);
     }
 
     @Override
@@ -140,8 +122,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         createTransactionIfMissing(actorContext, path);
 
-        final ActorSelection remoteTransaction = remoteTransactionFromIdentifier(path);
-        remoteTransaction.tell(new DeleteData(path).toSerializable(), null);
+        transactionContext(path).deleteData(path);
     }
 
     @Override
@@ -149,10 +130,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         List<ActorPath> cohortPaths = new ArrayList<>();
 
         for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
-            Object result = actorContext.executeRemoteOperation(transactionContext.getActor(),
-                new ReadyTransaction().toSerializable(),
-                ActorContext.ASK_DURATION
-            );
+            Object result = transactionContext.readyTransaction();
 
             if(result.getClass().equals(ReadyTransactionReply.SERIALIZABLE_CLASS)){
                 ReadyTransactionReply reply = ReadyTransactionReply.fromSerializable(actorContext.getActorSystem(),result);
@@ -173,14 +151,13 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     @Override
     public void close() {
         for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
-            transactionContext.getActor().tell(
-                new CloseTransaction().toSerializable(), null);
+            transactionContext.closeTransaction();
         }
     }
 
-    private ActorSelection remoteTransactionFromIdentifier(YangInstanceIdentifier path){
+    private TransactionContext transactionContext(YangInstanceIdentifier path){
         String shardName = shardNameFromIdentifier(path);
-        return remoteTransactionPaths.get(shardName).getActor();
+        return remoteTransactionPaths.get(shardName);
     }
 
     private String shardNameFromIdentifier(YangInstanceIdentifier path){
@@ -198,48 +175,186 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             return;
         }
 
-        Object response = actorContext.executeShardOperation(shardName, new CreateTransaction(identifier).toSerializable(), ActorContext.ASK_DURATION);
-        if(response.getClass().equals(CreateTransactionReply.SERIALIZABLE_CLASS)){
-            CreateTransactionReply reply = CreateTransactionReply.fromSerializable(response);
-            String transactionPath = actorContext.getRemoteActorPath(shardName, reply.getTransactionPath());
+        try {
+            Object response = actorContext.executeShardOperation(shardName,
+                new CreateTransaction(identifier).toSerializable(),
+                ActorContext.ASK_DURATION);
+            if (response.getClass()
+                .equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
+                CreateTransactionReply reply =
+                    CreateTransactionReply.fromSerializable(response);
+
+                String transactionPath = reply.getTransactionPath();
 
-            ActorSelection transactionActor = actorContext.actorSelection(transactionPath);
-            transactionContext = new TransactionContext(shardName, transactionPath, transactionActor);
+                LOG.info("Received transaction path = {}"  , transactionPath );
 
-            remoteTransactionPaths.put(shardName, transactionContext);
+                ActorSelection transactionActor =
+                    actorContext.actorSelection(transactionPath);
+                transactionContext =
+                    new TransactionContextImpl(shardName, transactionPath,
+                        transactionActor);
+
+                remoteTransactionPaths.put(shardName, transactionContext);
+            }
+        } catch(TimeoutException e){
+            remoteTransactionPaths.put(shardName, new NoOpTransactionContext(shardName));
         }
     }
 
+    private interface TransactionContext {
+        String getShardName();
+
+        String getResolvedCohortPath(String cohortPath);
+
+        public void closeTransaction();
+
+        public Object readyTransaction();
 
-    private class TransactionContext {
+        void deleteData(YangInstanceIdentifier path);
+
+        void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
+
+        ListenableFuture<Optional<NormalizedNode<?, ?>>> readData(final YangInstanceIdentifier path);
+
+        void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
+    }
+
+
+    private class TransactionContextImpl implements TransactionContext{
         private final String shardName;
         private final String actorPath;
         private final ActorSelection  actor;
 
 
-        private TransactionContext(String shardName, String actorPath,
+        private TransactionContextImpl(String shardName, String actorPath,
             ActorSelection actor) {
             this.shardName = shardName;
             this.actorPath = actorPath;
             this.actor = actor;
         }
 
-
-        public String getShardName() {
+        @Override public String getShardName() {
             return shardName;
         }
 
-        public String getActorPath() {
-            return actorPath;
-        }
-
-        public ActorSelection getActor() {
+        private ActorSelection getActor() {
             return actor;
         }
 
-        public String getResolvedCohortPath(String cohortPath){
+        @Override public String getResolvedCohortPath(String cohortPath){
             return actorContext.resolvePath(actorPath, cohortPath);
         }
+
+        @Override public void closeTransaction() {
+            getActor().tell(
+                new CloseTransaction().toSerializable(), null);
+        }
+
+        @Override public Object readyTransaction() {
+            return actorContext.executeRemoteOperation(getActor(),
+                new ReadyTransaction().toSerializable(),
+                ActorContext.ASK_DURATION
+            );
+
+        }
+
+        @Override public void deleteData(YangInstanceIdentifier path) {
+            getActor().tell(new DeleteData(path).toSerializable(), null);
+        }
+
+        @Override public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data){
+            getActor().tell(new MergeData(path, data, schemaContext).toSerializable(), null);
+        }
+
+        @Override public ListenableFuture<Optional<NormalizedNode<?, ?>>> readData(final YangInstanceIdentifier path) {
+
+            Callable<Optional<NormalizedNode<?,?>>> call = new Callable() {
+
+                @Override public Optional<NormalizedNode<?,?>> call() throws Exception {
+                    Object response = actorContext
+                        .executeRemoteOperation(getActor(), new ReadData(path).toSerializable(),
+                            ActorContext.ASK_DURATION);
+                    if(response.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)){
+                        ReadDataReply reply = ReadDataReply.fromSerializable(schemaContext,path, response);
+                        if(reply.getNormalizedNode() == null){
+                            return Optional.absent();
+                        }
+                        //FIXME : A cast should not be required here ???
+                        return (Optional<NormalizedNode<?, ?>>) Optional.of(reply.getNormalizedNode());
+                    }
+
+                    return Optional.absent();
+                }
+            };
+
+            ListenableFutureTask<Optional<NormalizedNode<?, ?>>>
+                future = ListenableFutureTask.create(call);
+
+            executor.submit(future);
+
+            return future;
+        }
+
+        @Override public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
+            getActor().tell(new WriteData(path, data, schemaContext).toSerializable(), null);
+        }
+
+    }
+
+    private class NoOpTransactionContext implements TransactionContext {
+
+        private final Logger
+            LOG = LoggerFactory.getLogger(NoOpTransactionContext.class);
+
+        private final String shardName;
+
+        private ActorRef cohort;
+
+        public NoOpTransactionContext(String shardName){
+            this.shardName = shardName;
+        }
+        @Override public String getShardName() {
+            return  shardName;
+
+        }
+
+        @Override public String getResolvedCohortPath(String cohortPath) {
+            return cohort.path().toString();
+        }
+
+        @Override public void closeTransaction() {
+            LOG.error("closeTransaction called");
+        }
+
+        @Override public Object readyTransaction() {
+            LOG.error("readyTransaction called");
+            cohort = actorContext.getActorSystem().actorOf(Props.create(NoOpCohort.class));
+            return new ReadyTransactionReply(cohort.path()).toSerializable();
+        }
+
+        @Override public void deleteData(YangInstanceIdentifier path) {
+            LOG.error("deleteData called path = {}", path);
+        }
+
+        @Override public void mergeData(YangInstanceIdentifier path,
+            NormalizedNode<?, ?> data) {
+            LOG.error("mergeData called path = {}", path);
+        }
+
+        @Override
+        public ListenableFuture<Optional<NormalizedNode<?, ?>>> readData(
+            YangInstanceIdentifier path) {
+            LOG.error("readData called path = {}", path);
+            return Futures.immediateFuture(
+                Optional.<NormalizedNode<?, ?>>absent());
+        }
+
+        @Override public void writeData(YangInstanceIdentifier path,
+            NormalizedNode<?, ?> data) {
+            LOG.error("writeData called path = {}", path);
+        }
     }
 
+
+
 }
index 4780aaccfb960b0254dde011a543666a373282dc..472cd387345a5ffd61747af83629dcb86a8100f3 100644 (file)
@@ -9,7 +9,7 @@
 package org.opendaylight.controller.cluster.datastore.exceptions;
 
 public class TimeoutException extends RuntimeException {
-    public TimeoutException(Exception e){
-        super(e);
+    public TimeoutException(String message, Exception e){
+        super(message, e);
     }
 }
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/messages/PeerAddressResolved.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/messages/PeerAddressResolved.java
new file mode 100644 (file)
index 0000000..8c2543e
--- /dev/null
@@ -0,0 +1,27 @@
+/*
+ * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+
+package org.opendaylight.controller.cluster.datastore.messages;
+
+public class PeerAddressResolved {
+    private final String peerId;
+    private final String peerAddress;
+
+    public PeerAddressResolved(String peerId, String peerAddress) {
+        this.peerId = peerId;
+        this.peerAddress = peerAddress;
+    }
+
+    public String getPeerId() {
+        return peerId;
+    }
+
+    public String getPeerAddress() {
+        return peerAddress;
+    }
+}
index c7ee7d8c2cca9ced8f17cf9f4fbc2eecd5cd3f45..ac0893da5ad157c230fad2aab7401b27d0054751 100644 (file)
@@ -14,9 +14,6 @@ import akka.actor.ActorSelection;
 import akka.actor.ActorSystem;
 import akka.actor.PoisonPill;
 import akka.util.Timeout;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
 import org.opendaylight.controller.cluster.datastore.ClusterWrapper;
 import org.opendaylight.controller.cluster.datastore.Configuration;
 import org.opendaylight.controller.cluster.datastore.exceptions.PrimaryNotFoundException;
@@ -105,7 +102,7 @@ public class ActorContext {
 
             return found.getPrimaryPath();
         }
-        throw new PrimaryNotFoundException();
+        throw new PrimaryNotFoundException("Could not find primary for shardName " + shardName);
     }
 
 
@@ -125,7 +122,7 @@ public class ActorContext {
         try {
             return Await.result(future, AWAIT_DURATION);
         } catch (Exception e) {
-            throw new TimeoutException(e);
+            throw new TimeoutException("Sending message " + message.getClass().toString() + " to actor " + actor.toString() + " failed" , e);
         }
     }
 
@@ -148,7 +145,7 @@ public class ActorContext {
         try {
             return Await.result(future, AWAIT_DURATION);
         } catch (Exception e) {
-            throw new TimeoutException(e);
+            throw new TimeoutException("Sending message " + message.getClass().toString() + " to actor " + actor.toString() + " failed" , e);
         }
     }
 
@@ -178,22 +175,15 @@ public class ActorContext {
         actorSystem.shutdown();
     }
 
-    public String getRemoteActorPath(final String shardName,
-        final String localPathOfRemoteActor) {
-        final String path = findPrimaryPath(shardName);
-
-        LoadingCache<String, String> graphs = CacheBuilder.newBuilder()
-            .expireAfterAccess(2, TimeUnit.SECONDS)
-            .build(
-                new CacheLoader<String, String>() {
-                    public String load(String key) {
-                        return resolvePath(path, localPathOfRemoteActor);
-                    }
-                }
-            );
-        return graphs.getUnchecked(localPathOfRemoteActor);
-    }
-
+    /**
+     * @deprecated Need to stop using this method. There are ways to send a
+     * remote ActorRef as a string which should be used instead of this hack
+     *
+     * @param primaryPath
+     * @param localPathOfRemoteActor
+     * @return
+     */
+    @Deprecated
     public String resolvePath(final String primaryPath,
         final String localPathOfRemoteActor) {
         StringBuilder builder = new StringBuilder();
index 214b3e9d3d989d73d0cd618db937a0fa6bcfdcb6..e23a76b0b223ee2b8ee0d5398da878cd520caf65 100644 (file)
@@ -10,22 +10,38 @@ package org.opendaylight.controller.cluster.datastore;
 
 import akka.actor.ActorSystem;
 import akka.testkit.JavaTestKit;
+import org.apache.commons.io.FileUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
+import java.io.File;
+import java.io.IOException;
+
 public abstract class AbstractActorTest {
     private static ActorSystem system;
 
     @BeforeClass
-    public static void setUpClass() {
+    public static void setUpClass() throws IOException {
+        File journal = new File("journal");
+
+        if(journal.exists()) {
+            FileUtils.deleteDirectory(journal);
+        }
+
         System.setProperty("shard.persistent", "false");
         system = ActorSystem.create("test");
     }
 
     @AfterClass
-    public static void tearDownClass() {
+    public static void tearDownClass() throws IOException {
         JavaTestKit.shutdownActorSystem(system);
         system = null;
+
+        File journal = new File("journal");
+
+        if(journal.exists()) {
+            FileUtils.deleteDirectory(journal);
+        }
     }
 
     protected ActorSystem getSystem() {
index b62a4b36d5016a00b2b86ae54d1623a3ea76aba5..11ad559744a3923aebfbb07472771b98a4439ab0 100644 (file)
@@ -33,6 +33,8 @@ import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
+import java.util.Collections;
+
 public class BasicIntegrationTest extends AbstractActorTest {
 
     @Test
@@ -47,16 +49,25 @@ public class BasicIntegrationTest extends AbstractActorTest {
 
 
         new JavaTestKit(getSystem()) {{
-            final Props props = Shard.props("config");
+            final Props props = Shard.props("config", Collections.EMPTY_MAP);
             final ActorRef shard = getSystem().actorOf(props);
 
             new Within(duration("5 seconds")) {
                 protected void run() {
 
+
                     shard.tell(
                         new UpdateSchemaContext(TestModel.createTestContext()),
                         getRef());
 
+
+                    // Wait for Shard to become a Leader
+                    try {
+                        Thread.sleep(1000);
+                    } catch (InterruptedException e) {
+                        e.printStackTrace();
+                    }
+                    // 1. Create a TransactionChain
                     shard.tell(new CreateTransactionChain().toSerializable(), getRef());
 
                     final ActorSelection transactionChain =
@@ -76,6 +87,9 @@ public class BasicIntegrationTest extends AbstractActorTest {
 
                     Assert.assertNotNull(transactionChain);
 
+                    System.out.println("Successfully created transaction chain");
+
+                    // 2. Create a Transaction on the TransactionChain
                     transactionChain.tell(new CreateTransaction("txn-1").toSerializable(), getRef());
 
                     final ActorSelection transaction =
@@ -94,9 +108,9 @@ public class BasicIntegrationTest extends AbstractActorTest {
 
                     Assert.assertNotNull(transaction);
 
-                    // Add a watch on the transaction actor so that we are notified when it dies
-                    final ActorRef transactionActorRef = watchActor(transaction);
+                    System.out.println("Successfully created transaction");
 
+                    // 3. Write some data
                     transaction.tell(new WriteData(TestModel.TEST_PATH,
                         ImmutableNodes.containerNode(TestModel.TEST_QNAME), TestModel.createTestContext()).toSerializable(),
                         getRef());
@@ -113,6 +127,10 @@ public class BasicIntegrationTest extends AbstractActorTest {
 
                     Assert.assertTrue(writeDone);
 
+                    System.out.println("Successfully wrote data");
+
+                    // 4. Ready the transaction for commit
+
                     transaction.tell(new ReadyTransaction().toSerializable(), getRef());
 
                     final ActorSelection cohort =
@@ -132,8 +150,9 @@ public class BasicIntegrationTest extends AbstractActorTest {
 
                     Assert.assertNotNull(cohort);
 
-                    // Add a watch on the transaction actor so that we are notified when it dies
-                    final ActorRef cohorActorRef = watchActor(cohort);
+                    System.out.println("Successfully readied the transaction");
+
+                    // 5. PreCommit the transaction
 
                     cohort.tell(new PreCommitTransaction().toSerializable(), getRef());
 
@@ -150,10 +169,14 @@ public class BasicIntegrationTest extends AbstractActorTest {
 
                     Assert.assertTrue(preCommitDone);
 
+                    System.out.println("Successfully pre-committed the transaction");
+
+                    // 6. Commit the transaction
                     cohort.tell(new CommitTransaction().toSerializable(), getRef());
 
                     // FIXME : Add assertions that the commit worked and that the cohort and transaction actors were terminated
 
+                    System.out.println("TODO : Check Successfully committed the transaction");
                 }
 
 
index 116e5e75b50d261e3d9673378ee643cfe2f2e31f..b5e3d24ef6d536d6afe2c9a6abc111bbb4b8688b 100644 (file)
@@ -9,7 +9,6 @@ import org.junit.Before;
 import org.junit.Test;
 import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategyFactory;
 import org.opendaylight.controller.cluster.datastore.utils.MockClusterWrapper;
-import org.opendaylight.controller.cluster.datastore.utils.MockConfiguration;
 import org.opendaylight.controller.md.cluster.datastore.model.CarsModel;
 import org.opendaylight.controller.md.cluster.datastore.model.PeopleModel;
 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
@@ -46,12 +45,15 @@ public class DistributedDataStoreIntegrationTest{
 
     @Test
     public void integrationTest() throws Exception {
-        ShardStrategyFactory.setConfiguration(new MockConfiguration());
+        Configuration configuration = new ConfigurationImpl("module-shards.conf", "modules.conf");
+        ShardStrategyFactory.setConfiguration(configuration);
         DistributedDataStore distributedDataStore =
-            new DistributedDataStore(getSystem(), "config", new MockClusterWrapper(), new MockConfiguration());
+            new DistributedDataStore(getSystem(), "config", new MockClusterWrapper(), configuration);
 
         distributedDataStore.onGlobalContextUpdated(TestModel.createTestContext());
 
+        Thread.sleep(1000);
+
         DOMStoreReadWriteTransaction transaction =
             distributedDataStore.newReadWriteTransaction();
 
@@ -95,6 +97,8 @@ public class DistributedDataStoreIntegrationTest{
 
         distributedDataStore.onGlobalContextUpdated(SchemaContextHelper.full());
 
+        Thread.sleep(1000);
+
         DOMStoreReadWriteTransaction transaction =
             distributedDataStore.newReadWriteTransaction();
 
index f20cd8c3d7ef2a18bbc375ce3fa770975c26bc9c..7d57ea8284e90dc3f941b7b82385e5db28ed75f4 100644 (file)
@@ -7,6 +7,7 @@ import org.junit.Test;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionChain;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionChainReply;
+import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener;
 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply;
 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
@@ -19,6 +20,10 @@ import org.opendaylight.controller.protobuff.messages.transaction.ShardTransacti
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -26,10 +31,18 @@ public class ShardTest extends AbstractActorTest {
     @Test
     public void testOnReceiveCreateTransactionChain() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final Props props = Shard.props("config");
+            final Props props = Shard.props("config", Collections.EMPTY_MAP);
             final ActorRef subject =
                 getSystem().actorOf(props, "testCreateTransactionChain");
 
+
+            // Wait for Shard to become a Leader
+            try {
+                Thread.sleep(1000);
+            } catch (InterruptedException e) {
+                e.printStackTrace();
+            }
+
             new Within(duration("1 seconds")) {
                 protected void run() {
 
@@ -64,7 +77,7 @@ public class ShardTest extends AbstractActorTest {
     @Test
     public void testOnReceiveRegisterListener() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final Props props = Shard.props("config");
+            final Props props = Shard.props("config", Collections.EMPTY_MAP);
             final ActorRef subject =
                 getSystem().actorOf(props, "testRegisterChangeListener");
 
@@ -107,10 +120,19 @@ public class ShardTest extends AbstractActorTest {
     @Test
     public void testCreateTransaction(){
         new JavaTestKit(getSystem()) {{
-            final Props props = Shard.props("config");
+            final Props props = Shard.props("config", Collections.EMPTY_MAP);
             final ActorRef subject =
                 getSystem().actorOf(props, "testCreateTransaction");
 
+
+            // Wait for Shard to become a Leader
+            try {
+                Thread.sleep(1000);
+            } catch (InterruptedException e) {
+                e.printStackTrace();
+            }
+
+
             new Within(duration("1 seconds")) {
                 protected void run() {
 
@@ -135,9 +157,8 @@ public class ShardTest extends AbstractActorTest {
                         }
                     }.get(); // this extracts the received message
 
-                    assertEquals("Unexpected transaction path " + out,
-                        "akka://test/user/testCreateTransaction/shard-txn-1",
-                        out);
+                    assertTrue("Unexpected transaction path " + out,
+                        out.contains("akka://test/user/testCreateTransaction/shard-txn-1"));
                     expectNoMsg();
                 }
 
@@ -146,7 +167,29 @@ public class ShardTest extends AbstractActorTest {
         }};
     }
 
+    @Test
+    public void testPeerAddressResolved(){
+        new JavaTestKit(getSystem()) {{
+            Map<String, String> peerAddresses = new HashMap<>();
+            peerAddresses.put("member-2", null);
+            final Props props = Shard.props("config", peerAddresses);
+            final ActorRef subject =
+                getSystem().actorOf(props, "testPeerAddressResolved");
+
+            new Within(duration("1 seconds")) {
+                protected void run() {
 
+                    subject.tell(
+                        new PeerAddressResolved("member-2", "akka://foobar"),
+                        getRef());
+
+                    expectNoMsg();
+                }
+
+
+            };
+        }};
+    }
 
     private AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>> noOpDataChangeListener() {
         return new AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>() {
index 4d7c61a197e08420e7cec90753aa1ef427f4ce4d..7884eeccdae7874f8d19c9090bf08d755cc991be 100644 (file)
@@ -30,6 +30,8 @@ import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 
+import java.util.Collections;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -49,7 +51,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveReadData() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props("config"));
+            final ActorRef shard = getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
             final Props props =
                 ShardTransaction.props(store.newReadWriteTransaction(), shard, testSchemaContext);
             final ActorRef subject = getSystem().actorOf(props, "testReadData");
@@ -89,7 +91,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveReadDataWhenDataNotFound() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props("config"));
+            final ActorRef shard = getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
             final Props props =
                 ShardTransaction.props(store.newReadWriteTransaction(), shard, testSchemaContext);
             final ActorRef subject = getSystem().actorOf(props, "testReadDataWhenDataNotFound");
@@ -163,7 +165,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveWriteData() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props("config"));
+            final ActorRef shard = getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
             final Props props =
                 ShardTransaction.props(store.newReadWriteTransaction(), shard, TestModel.createTestContext());
             final ActorRef subject =
@@ -201,7 +203,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveMergeData() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props("config"));
+            final ActorRef shard = getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
             final Props props =
                 ShardTransaction.props(store.newReadWriteTransaction(), shard, testSchemaContext);
             final ActorRef subject =
@@ -240,7 +242,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveDeleteData() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props("config"));
+            final ActorRef shard = getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
             final Props props =
                 ShardTransaction.props(store.newReadWriteTransaction(), shard, TestModel.createTestContext());
             final ActorRef subject =
@@ -277,7 +279,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveReadyTransaction() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props("config"));
+            final ActorRef shard = getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
             final Props props =
                 ShardTransaction.props(store.newReadWriteTransaction(), shard, TestModel.createTestContext());
             final ActorRef subject =
@@ -313,7 +315,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveCloseTransaction() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props("config"));
+            final ActorRef shard = getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
             final Props props =
                 ShardTransaction.props(store.newReadWriteTransaction(), shard, TestModel.createTestContext());
             final ActorRef subject =
index 57df20172d59b6008c691617363788d4b64e7d23..6860872b75fddaf0c952af3943a8713f977531f5 100644 (file)
@@ -8,6 +8,7 @@
 
 package org.opendaylight.controller.md.cluster.datastore.model;
 
+import java.math.BigInteger;
 import org.opendaylight.yangtools.yang.common.QName;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
@@ -42,14 +43,14 @@ public class CarsModel {
         MapEntryNode altima =
             ImmutableNodes.mapEntryBuilder(CAR_QNAME, CAR_NAME_QNAME, "altima")
                 .withChild(ImmutableNodes.leafNode(CAR_NAME_QNAME, "altima"))
-                .withChild(ImmutableNodes.leafNode(CAR_PRICE_QNAME, 1000))
+                .withChild(ImmutableNodes.leafNode(CAR_PRICE_QNAME, new BigInteger("1000")))
                 .build();
 
         // Create an entry for the car accord
         MapEntryNode honda =
             ImmutableNodes.mapEntryBuilder(CAR_QNAME, CAR_NAME_QNAME, "accord")
                 .withChild(ImmutableNodes.leafNode(CAR_NAME_QNAME, "accord"))
-                .withChild(ImmutableNodes.leafNode(CAR_PRICE_QNAME, 2000))
+                .withChild(ImmutableNodes.leafNode(CAR_PRICE_QNAME, new BigInteger("2000")))
                 .build();
 
         cars.withChild(altima);
index 1b4020af438c356a3b07b01ded9c179d02a7bdc1..e637920e7855c9859ee9cab249f31b20766e9ef2 100644 (file)
@@ -42,14 +42,14 @@ public class PeopleModel {
         MapEntryNode jack =
             ImmutableNodes.mapEntryBuilder(PERSON_QNAME, PERSON_NAME_QNAME, "jack")
                 .withChild(ImmutableNodes.leafNode(PERSON_NAME_QNAME, "jack"))
-                .withChild(ImmutableNodes.leafNode(PERSON_AGE_QNAME, 100))
+                .withChild(ImmutableNodes.leafNode(PERSON_AGE_QNAME, 100L))
                 .build();
 
         // Create an entry for the person jill
         MapEntryNode jill =
             ImmutableNodes.mapEntryBuilder(PERSON_QNAME, PERSON_NAME_QNAME, "jill")
                 .withChild(ImmutableNodes.leafNode(PERSON_NAME_QNAME, "jill"))
-                .withChild(ImmutableNodes.leafNode(PERSON_AGE_QNAME, 200))
+                .withChild(ImmutableNodes.leafNode(PERSON_AGE_QNAME, 200L))
                 .build();
 
         cars.withChild(jack);
index d8fefcd986cc8033d6530622039c5e0ce005114f..be8713c7020c5848e7eec03278a95a9794a095ad 100644 (file)
@@ -18,45 +18,45 @@ import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 public class SampleModelsTest {
     @Test
     public void testPeopleModel(){
-        NormalizedNode<?, ?> expected = PeopleModel.create();
+        final NormalizedNode<?, ?> expected = PeopleModel.create();
 
 
-        NormalizedNodeMessages.Container node =
+        final NormalizedNodeMessages.Container node =
             new NormalizedNodeToNodeCodec(SchemaContextHelper.full())
                 .encode(YangInstanceIdentifier.of(PeopleModel.BASE_QNAME),
                     expected);
 
-        NormalizedNodeMessages.Node normalizedNode =
+        final NormalizedNodeMessages.Node normalizedNode =
             node.getNormalizedNode();
 
-        NormalizedNode<?,?> actual = new NormalizedNodeToNodeCodec(SchemaContextHelper.full()).decode(YangInstanceIdentifier.of(PeopleModel.BASE_QNAME),
+        final NormalizedNode<?,?> actual = new NormalizedNodeToNodeCodec(SchemaContextHelper.full()).decode(YangInstanceIdentifier.of(PeopleModel.BASE_QNAME),
             normalizedNode);
 
 
-        Assert.assertEquals(expected.toString(), actual.toString());
+        Assert.assertEquals(expected, actual);
 
     }
 
 
     @Test
     public void testCarsModel(){
-        NormalizedNode<?, ?> expected = CarsModel.create();
+        final NormalizedNode<?, ?> expected = CarsModel.create();
 
 
-        NormalizedNodeMessages.Container node =
+        final NormalizedNodeMessages.Container node =
             new NormalizedNodeToNodeCodec(SchemaContextHelper.full())
                 .encode(YangInstanceIdentifier.of(CarsModel.BASE_QNAME),
                     expected);
 
-        NormalizedNodeMessages.Node normalizedNode =
+        final NormalizedNodeMessages.Node normalizedNode =
             node.getNormalizedNode();
 
-        NormalizedNode<?,?> actual = new NormalizedNodeToNodeCodec(SchemaContextHelper.full()).decode(
+        final NormalizedNode<?,?> actual = new NormalizedNodeToNodeCodec(SchemaContextHelper.full()).decode(
             YangInstanceIdentifier.of(CarsModel.BASE_QNAME),
             normalizedNode);
 
 
-        Assert.assertEquals(expected.toString(), actual.toString());
+        Assert.assertEquals(expected, actual);
 
     }
 }
index 9f35f2b9f24f4547c5989e7b7bd6c8458ff23d23..b3de998479cf37d84c211e033884230386096b72 100644 (file)
@@ -18,9 +18,7 @@ module-shards = [
             {
                 name="people-1"
                 replicas = [
-                    "member-1",
-                    "member-2",
-                    "member-3"
+                    "member-1"
                 ]
             }
         ]
@@ -31,9 +29,18 @@ module-shards = [
             {
                 name="cars-1"
                 replicas = [
-                    "member-4",
-                    "member-1",
-                    "member-5"
+                    "member-1"
+                ]
+            }
+        ]
+    },
+    {
+        name = "test"
+        shards = [
+            {
+                name="test-1"
+                replicas = [
+                    "member-1"
                 ]
             }
         ]
index 52f249a7ff57fa1a3ef3a0b8ca12faac09c06d6e..22854cb11a968bd66f1b969d95a5cc07da7941de 100644 (file)
@@ -8,6 +8,11 @@ modules = [
         name = "cars"
         namespace = "urn:opendaylight:params:xml:ns:yang:controller:md:sal:dom:store:test:cars"
         shard-strategy = "module"
+    },
+    {
+     name = "test"
+     namespace = "urn:opendaylight:params:xml:ns:yang:controller:md:sal:dom:store:test"
+     shard-strategy = "module"
     }
 
 ]
index e3fac63a8335b9e2fed6ef6844dc4ea3125a6cd9..28e629a92c92fa7b165a390e12e2fd89a15e2ba5 100644 (file)
           <groupId>org.opendaylight.yangtools</groupId>
           <artifactId>yang-binding</artifactId>
       </dependency>
+
+      <dependency>
+          <groupId>org.opendaylight.controller</groupId>
+          <artifactId>sal-akka-raft</artifactId>
+          <version>1.1-SNAPSHOT</version>
+      </dependency>
+
       <dependency>
           <groupId>com.google.guava</groupId>
           <artifactId>guava</artifactId>
diff --git a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/mdsal/CompositeModificationPayload.java b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/mdsal/CompositeModificationPayload.java
new file mode 100644 (file)
index 0000000..87b246b
--- /dev/null
@@ -0,0 +1,60 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: CompositeModificationPayload.proto
+
+package org.opendaylight.controller.mdsal;
+
+public final class CompositeModificationPayload {
+  private CompositeModificationPayload() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+    registry.add(org.opendaylight.controller.mdsal.CompositeModificationPayload.modification);
+  }
+  public static final int MODIFICATION_FIELD_NUMBER = 2;
+  /**
+   * <code>extend .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload { ... }</code>
+   */
+  public static final
+    com.google.protobuf.GeneratedMessage.GeneratedExtension<
+      org.opendaylight.controller.cluster.raft.protobuff.messages.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload,
+      org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.CompositeModification> modification = com.google.protobuf.GeneratedMessage
+          .newFileScopedGeneratedExtension(
+        org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.CompositeModification.class,
+        org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.CompositeModification.getDefaultInstance());
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\"CompositeModificationPayload.proto\022!or" +
+      "g.opendaylight.controller.mdsal\032\033AppendE" +
+      "ntriesMessages.proto\032\014Common.proto\032\020Pers" +
+      "istent.proto:\242\001\n\014modification\022R.org.open" +
+      "daylight.controller.cluster.raft.AppendE" +
+      "ntries.ReplicatedLogEntry.Payload\030\002 \001(\0132" +
+      "8.org.opendaylight.controller.mdsal.Comp" +
+      "ositeModification"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          modification.internalInit(descriptor.getExtensions().get(0));
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+          org.opendaylight.controller.cluster.raft.protobuff.messages.AppendEntriesMessages.getDescriptor(),
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.getDescriptor(),
+          org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.getDescriptor(),
+        }, assigner);
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}
index 35c2940be3e05aa9e03d8c922ecffde2cd0e823a..81e5b462cc56a6d27b95f1dd050f330b4dfd1520 100644 (file)
@@ -179,7 +179,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -195,7 +195,7 @@ public final class NormalizedNodeMessages {
         getNameBytes() {
       java.lang.Object ref = name_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         name_ = b;
@@ -222,7 +222,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -238,7 +238,7 @@ public final class NormalizedNodeMessages {
         getValueBytes() {
       java.lang.Object ref = value_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         value_ = b;
@@ -265,7 +265,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -281,7 +281,7 @@ public final class NormalizedNodeMessages {
         getTypeBytes() {
       java.lang.Object ref = type_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         type_ = b;
@@ -541,7 +541,7 @@ public final class NormalizedNodeMessages {
 
       public final boolean isInitialized() {
         if (!hasName()) {
-
+          
           return false;
         }
         return true;
@@ -595,7 +595,7 @@ public final class NormalizedNodeMessages {
           getNameBytes() {
         java.lang.Object ref = name_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           name_ = b;
@@ -669,7 +669,7 @@ public final class NormalizedNodeMessages {
           getValueBytes() {
         java.lang.Object ref = value_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           value_ = b;
@@ -743,7 +743,7 @@ public final class NormalizedNodeMessages {
           getTypeBytes() {
         java.lang.Object ref = type_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           type_ = b;
@@ -930,7 +930,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -946,7 +946,7 @@ public final class NormalizedNodeMessages {
         getValueBytes() {
       java.lang.Object ref = value_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         value_ = b;
@@ -1168,7 +1168,7 @@ public final class NormalizedNodeMessages {
 
       public final boolean isInitialized() {
         if (!hasValue()) {
-
+          
           return false;
         }
         return true;
@@ -1222,7 +1222,7 @@ public final class NormalizedNodeMessages {
           getValueBytes() {
         java.lang.Object ref = value_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           value_ = b;
@@ -1341,7 +1341,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4;</code>
      */
-    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute>
+    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute> 
         getAttributesList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4;</code>
@@ -1354,7 +1354,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4;</code>
      */
-    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
         getAttributesOrBuilderList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4;</code>
@@ -1504,7 +1504,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -1520,7 +1520,7 @@ public final class NormalizedNodeMessages {
         getValueBytes() {
       java.lang.Object ref = value_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         value_ = b;
@@ -1555,7 +1555,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -1575,7 +1575,7 @@ public final class NormalizedNodeMessages {
         getTypeBytes() {
       java.lang.Object ref = type_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         type_ = b;
@@ -1619,7 +1619,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4;</code>
      */
-    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
         getAttributesOrBuilderList() {
       return attributes_;
     }
@@ -1948,7 +1948,7 @@ public final class NormalizedNodeMessages {
               attributesBuilder_ = null;
               attributes_ = other.attributes_;
               bitField0_ = (bitField0_ & ~0x00000008);
-              attributesBuilder_ =
+              attributesBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getAttributesFieldBuilder() : null;
             } else {
@@ -1962,18 +1962,18 @@ public final class NormalizedNodeMessages {
 
       public final boolean isInitialized() {
         if (!hasValue()) {
-
+          
           return false;
         }
         if (hasNodeType()) {
           if (!getNodeType().isInitialized()) {
-
+            
             return false;
           }
         }
         for (int i = 0; i < getAttributesCount(); i++) {
           if (!getAttributes(i).isInitialized()) {
-
+            
             return false;
           }
         }
@@ -2028,7 +2028,7 @@ public final class NormalizedNodeMessages {
           getValueBytes() {
         java.lang.Object ref = value_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           value_ = b;
@@ -2114,7 +2114,7 @@ public final class NormalizedNodeMessages {
           getTypeBytes() {
         java.lang.Object ref = type_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           type_ = b;
@@ -2275,7 +2275,7 @@ public final class NormalizedNodeMessages {
        * <code>optional .org.opendaylight.controller.mdsal.QName nodeType = 3;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.QName, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.QName.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.QNameOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.QName, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.QName.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.QNameOrBuilder> 
           getNodeTypeFieldBuilder() {
         if (nodeTypeBuilder_ == null) {
           nodeTypeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -2483,7 +2483,7 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4;</code>
        */
-      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
            getAttributesOrBuilderList() {
         if (attributesBuilder_ != null) {
           return attributesBuilder_.getMessageOrBuilderList();
@@ -2509,12 +2509,12 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4;</code>
        */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder>
+      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder> 
            getAttributesBuilderList() {
         return getAttributesFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
           getAttributesFieldBuilder() {
         if (attributesBuilder_ == null) {
           attributesBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
@@ -2546,7 +2546,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1;</code>
      */
-    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument>
+    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument> 
         getArgumentsList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1;</code>
@@ -2559,7 +2559,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1;</code>
      */
-    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder>
+    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder> 
         getArgumentsOrBuilderList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1;</code>
@@ -2680,7 +2680,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1;</code>
      */
-    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder>
+    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder> 
         getArgumentsOrBuilderList() {
       return arguments_;
     }
@@ -2933,7 +2933,7 @@ public final class NormalizedNodeMessages {
               argumentsBuilder_ = null;
               arguments_ = other.arguments_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              argumentsBuilder_ =
+              argumentsBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getArgumentsFieldBuilder() : null;
             } else {
@@ -2948,7 +2948,7 @@ public final class NormalizedNodeMessages {
       public final boolean isInitialized() {
         for (int i = 0; i < getArgumentsCount(); i++) {
           if (!getArguments(i).isInitialized()) {
-
+            
             return false;
           }
         }
@@ -3169,7 +3169,7 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1;</code>
        */
-      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder>
+      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder> 
            getArgumentsOrBuilderList() {
         if (argumentsBuilder_ != null) {
           return argumentsBuilder_.getMessageOrBuilderList();
@@ -3195,12 +3195,12 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1;</code>
        */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument.Builder>
+      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument.Builder> 
            getArgumentsBuilderList() {
         return getArgumentsFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder> 
           getArgumentsFieldBuilder() {
         if (argumentsBuilder_ == null) {
           argumentsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
@@ -3262,7 +3262,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3;</code>
      */
-    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute>
+    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute> 
         getAttributesList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3;</code>
@@ -3275,7 +3275,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3;</code>
      */
-    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
         getAttributesOrBuilderList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3;</code>
@@ -3287,7 +3287,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Node child = 4;</code>
      */
-    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node>
+    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node> 
         getChildList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Node child = 4;</code>
@@ -3300,7 +3300,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Node child = 4;</code>
      */
-    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
         getChildOrBuilderList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Node child = 4;</code>
@@ -3546,7 +3546,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -3562,7 +3562,7 @@ public final class NormalizedNodeMessages {
         getPathBytes() {
       java.lang.Object ref = path_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         path_ = b;
@@ -3589,7 +3589,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -3605,7 +3605,7 @@ public final class NormalizedNodeMessages {
         getTypeBytes() {
       java.lang.Object ref = type_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         type_ = b;
@@ -3627,7 +3627,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3;</code>
      */
-    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
         getAttributesOrBuilderList() {
       return attributes_;
     }
@@ -3663,7 +3663,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Node child = 4;</code>
      */
-    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
         getChildOrBuilderList() {
       return child_;
     }
@@ -3704,7 +3704,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -3720,7 +3720,7 @@ public final class NormalizedNodeMessages {
         getValueBytes() {
       java.lang.Object ref = value_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         value_ = b;
@@ -3747,7 +3747,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -3763,7 +3763,7 @@ public final class NormalizedNodeMessages {
         getValueTypeBytes() {
       java.lang.Object ref = valueType_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         valueType_ = b;
@@ -4202,7 +4202,7 @@ public final class NormalizedNodeMessages {
               attributesBuilder_ = null;
               attributes_ = other.attributes_;
               bitField0_ = (bitField0_ & ~0x00000004);
-              attributesBuilder_ =
+              attributesBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getAttributesFieldBuilder() : null;
             } else {
@@ -4228,7 +4228,7 @@ public final class NormalizedNodeMessages {
               childBuilder_ = null;
               child_ = other.child_;
               bitField0_ = (bitField0_ & ~0x00000008);
-              childBuilder_ =
+              childBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getChildFieldBuilder() : null;
             } else {
@@ -4266,19 +4266,19 @@ public final class NormalizedNodeMessages {
       public final boolean isInitialized() {
         for (int i = 0; i < getAttributesCount(); i++) {
           if (!getAttributes(i).isInitialized()) {
-
+            
             return false;
           }
         }
         for (int i = 0; i < getChildCount(); i++) {
           if (!getChild(i).isInitialized()) {
-
+            
             return false;
           }
         }
         if (hasInstanceIdentifierValue()) {
           if (!getInstanceIdentifierValue().isInitialized()) {
-
+            
             return false;
           }
         }
@@ -4333,7 +4333,7 @@ public final class NormalizedNodeMessages {
           getPathBytes() {
         java.lang.Object ref = path_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           path_ = b;
@@ -4407,7 +4407,7 @@ public final class NormalizedNodeMessages {
           getTypeBytes() {
         java.lang.Object ref = type_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           type_ = b;
@@ -4647,7 +4647,7 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3;</code>
        */
-      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
            getAttributesOrBuilderList() {
         if (attributesBuilder_ != null) {
           return attributesBuilder_.getMessageOrBuilderList();
@@ -4673,12 +4673,12 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3;</code>
        */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder>
+      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder> 
            getAttributesBuilderList() {
         return getAttributesFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
           getAttributesFieldBuilder() {
         if (attributesBuilder_ == null) {
           attributesBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
@@ -4887,7 +4887,7 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Node child = 4;</code>
        */
-      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
            getChildOrBuilderList() {
         if (childBuilder_ != null) {
           return childBuilder_.getMessageOrBuilderList();
@@ -4913,12 +4913,12 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Node child = 4;</code>
        */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder>
+      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder> 
            getChildBuilderList() {
         return getChildFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getChildFieldBuilder() {
         if (childBuilder_ == null) {
           childBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
@@ -4961,7 +4961,7 @@ public final class NormalizedNodeMessages {
           getValueBytes() {
         java.lang.Object ref = value_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           value_ = b;
@@ -5035,7 +5035,7 @@ public final class NormalizedNodeMessages {
           getValueTypeBytes() {
         java.lang.Object ref = valueType_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           valueType_ = b;
@@ -5277,7 +5277,7 @@ public final class NormalizedNodeMessages {
        * <code>optional .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierValue = 8;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getInstanceIdentifierValueFieldBuilder() {
         if (instanceIdentifierValueBuilder_ == null) {
           instanceIdentifierValueBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -5459,7 +5459,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -5475,7 +5475,7 @@ public final class NormalizedNodeMessages {
         getParentPathBytes() {
       java.lang.Object ref = parentPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         parentPath_ = b;
@@ -5751,12 +5751,12 @@ public final class NormalizedNodeMessages {
 
       public final boolean isInitialized() {
         if (!hasParentPath()) {
-
+          
           return false;
         }
         if (hasNormalizedNode()) {
           if (!getNormalizedNode().isInitialized()) {
-
+            
             return false;
           }
         }
@@ -5811,7 +5811,7 @@ public final class NormalizedNodeMessages {
           getParentPathBytes() {
         java.lang.Object ref = parentPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           parentPath_ = b;
@@ -5960,7 +5960,7 @@ public final class NormalizedNodeMessages {
        * <code>optional .org.opendaylight.controller.mdsal.Node normalizedNode = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getNormalizedNodeFieldBuilder() {
         if (normalizedNodeBuilder_ == null) {
           normalizedNodeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -6431,16 +6431,16 @@ public final class NormalizedNodeMessages {
 
       public final boolean isInitialized() {
         if (!hasInstanceIdentifierPath()) {
-
+          
           return false;
         }
         if (!getInstanceIdentifierPath().isInitialized()) {
-
+          
           return false;
         }
         if (hasNormalizedNode()) {
           if (!getNormalizedNode().isInitialized()) {
-
+            
             return false;
           }
         }
@@ -6570,7 +6570,7 @@ public final class NormalizedNodeMessages {
        * <code>required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPath = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getInstanceIdentifierPathFieldBuilder() {
         if (instanceIdentifierPathBuilder_ == null) {
           instanceIdentifierPathBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -6687,7 +6687,7 @@ public final class NormalizedNodeMessages {
        * <code>optional .org.opendaylight.controller.mdsal.Node normalizedNode = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getNormalizedNodeFieldBuilder() {
         if (normalizedNodeBuilder_ == null) {
           normalizedNodeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -6718,7 +6718,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1;</code>
      */
-    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry>
+    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry> 
         getMapEntriesList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1;</code>
@@ -6731,7 +6731,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1;</code>
      */
-    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder>
+    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder> 
         getMapEntriesOrBuilderList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1;</code>
@@ -6852,7 +6852,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1;</code>
      */
-    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder>
+    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder> 
         getMapEntriesOrBuilderList() {
       return mapEntries_;
     }
@@ -7105,7 +7105,7 @@ public final class NormalizedNodeMessages {
               mapEntriesBuilder_ = null;
               mapEntries_ = other.mapEntries_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              mapEntriesBuilder_ =
+              mapEntriesBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getMapEntriesFieldBuilder() : null;
             } else {
@@ -7120,7 +7120,7 @@ public final class NormalizedNodeMessages {
       public final boolean isInitialized() {
         for (int i = 0; i < getMapEntriesCount(); i++) {
           if (!getMapEntries(i).isInitialized()) {
-
+            
             return false;
           }
         }
@@ -7341,7 +7341,7 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1;</code>
        */
-      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder>
+      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder> 
            getMapEntriesOrBuilderList() {
         if (mapEntriesBuilder_ != null) {
           return mapEntriesBuilder_.getMessageOrBuilderList();
@@ -7367,12 +7367,12 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1;</code>
        */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry.Builder>
+      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry.Builder> 
            getMapEntriesBuilderList() {
         return getMapEntriesFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder> 
           getMapEntriesFieldBuilder() {
         if (mapEntriesBuilder_ == null) {
           mapEntriesBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
index 67ab472d0ce27973757a8c22b1e104bd72550fe2..29e54571d30b6ddbb9392c6f3edbed83f09c5db8 100644 (file)
@@ -159,7 +159,7 @@ public final class SimpleNormalizedNodeMessage {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -175,7 +175,7 @@ public final class SimpleNormalizedNodeMessage {
         getNodeIdentifierBytes() {
       java.lang.Object ref = nodeIdentifier_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         nodeIdentifier_ = b;
@@ -202,7 +202,7 @@ public final class SimpleNormalizedNodeMessage {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -218,7 +218,7 @@ public final class SimpleNormalizedNodeMessage {
         getXmlStringBytes() {
       java.lang.Object ref = xmlString_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         xmlString_ = b;
@@ -463,11 +463,11 @@ public final class SimpleNormalizedNodeMessage {
 
       public final boolean isInitialized() {
         if (!hasNodeIdentifier()) {
-
+          
           return false;
         }
         if (!hasXmlString()) {
-
+          
           return false;
         }
         return true;
@@ -521,7 +521,7 @@ public final class SimpleNormalizedNodeMessage {
           getNodeIdentifierBytes() {
         java.lang.Object ref = nodeIdentifier_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           nodeIdentifier_ = b;
@@ -595,7 +595,7 @@ public final class SimpleNormalizedNodeMessage {
           getXmlStringBytes() {
         java.lang.Object ref = xmlString_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           xmlString_ = b;
index 384b389f9251b9a233cfbe817262eb10a4829443..201883476838f3c592f63e47c08fcde5a661b29a 100644 (file)
@@ -85,7 +85,7 @@ public final class DataChangeListenerMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6;</code>
      */
-    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier>
+    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier> 
         getRemovedPathsList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6;</code>
@@ -98,7 +98,7 @@ public final class DataChangeListenerMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6;</code>
      */
-    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
         getRemovedPathsOrBuilderList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6;</code>
@@ -395,7 +395,7 @@ public final class DataChangeListenerMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6;</code>
      */
-    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
         getRemovedPathsOrBuilderList() {
       return removedPaths_;
     }
@@ -810,7 +810,7 @@ public final class DataChangeListenerMessages {
               removedPathsBuilder_ = null;
               removedPaths_ = other.removedPaths_;
               bitField0_ = (bitField0_ & ~0x00000020);
-              removedPathsBuilder_ =
+              removedPathsBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getRemovedPathsFieldBuilder() : null;
             } else {
@@ -825,37 +825,37 @@ public final class DataChangeListenerMessages {
       public final boolean isInitialized() {
         if (hasOriginalSubTree()) {
           if (!getOriginalSubTree().isInitialized()) {
-
+            
             return false;
           }
         }
         if (hasUpdatedSubTree()) {
           if (!getUpdatedSubTree().isInitialized()) {
-
+            
             return false;
           }
         }
         if (hasOriginalData()) {
           if (!getOriginalData().isInitialized()) {
-
+            
             return false;
           }
         }
         if (hasUpdatedData()) {
           if (!getUpdatedData().isInitialized()) {
-
+            
             return false;
           }
         }
         if (hasCreatedData()) {
           if (!getCreatedData().isInitialized()) {
-
+            
             return false;
           }
         }
         for (int i = 0; i < getRemovedPathsCount(); i++) {
           if (!getRemovedPaths(i).isInitialized()) {
-
+            
             return false;
           }
         }
@@ -985,7 +985,7 @@ public final class DataChangeListenerMessages {
        * <code>optional .org.opendaylight.controller.mdsal.Node originalSubTree = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getOriginalSubTreeFieldBuilder() {
         if (originalSubTreeBuilder_ == null) {
           originalSubTreeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -1102,7 +1102,7 @@ public final class DataChangeListenerMessages {
        * <code>optional .org.opendaylight.controller.mdsal.Node updatedSubTree = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getUpdatedSubTreeFieldBuilder() {
         if (updatedSubTreeBuilder_ == null) {
           updatedSubTreeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -1219,7 +1219,7 @@ public final class DataChangeListenerMessages {
        * <code>optional .org.opendaylight.controller.mdsal.NodeMap originalData = 3;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapOrBuilder> 
           getOriginalDataFieldBuilder() {
         if (originalDataBuilder_ == null) {
           originalDataBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -1336,7 +1336,7 @@ public final class DataChangeListenerMessages {
        * <code>optional .org.opendaylight.controller.mdsal.NodeMap updatedData = 4;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapOrBuilder> 
           getUpdatedDataFieldBuilder() {
         if (updatedDataBuilder_ == null) {
           updatedDataBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -1453,7 +1453,7 @@ public final class DataChangeListenerMessages {
        * <code>optional .org.opendaylight.controller.mdsal.NodeMap createdData = 5;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapOrBuilder> 
           getCreatedDataFieldBuilder() {
         if (createdDataBuilder_ == null) {
           createdDataBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -1661,7 +1661,7 @@ public final class DataChangeListenerMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6;</code>
        */
-      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
            getRemovedPathsOrBuilderList() {
         if (removedPathsBuilder_ != null) {
           return removedPathsBuilder_.getMessageOrBuilderList();
@@ -1687,12 +1687,12 @@ public final class DataChangeListenerMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6;</code>
        */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder>
+      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder> 
            getRemovedPathsBuilderList() {
         return getRemovedPathsFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getRemovedPathsFieldBuilder() {
         if (removedPathsBuilder_ == null) {
           removedPathsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
index 6c1e2722f65cfb69ba10a73683456d67ea012a78..eaa90012dbdc58425adc8fbfa2ea4e467f0c4379 100644 (file)
@@ -193,7 +193,7 @@ public final class PersistentMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -209,7 +209,7 @@ public final class PersistentMessages {
         getTypeBytes() {
       java.lang.Object ref = type_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         type_ = b;
@@ -541,20 +541,20 @@ public final class PersistentMessages {
 
       public final boolean isInitialized() {
         if (!hasType()) {
-
+          
           return false;
         }
         if (!hasPath()) {
-
+          
           return false;
         }
         if (!getPath().isInitialized()) {
-
+          
           return false;
         }
         if (hasData()) {
           if (!getData().isInitialized()) {
-
+            
             return false;
           }
         }
@@ -609,7 +609,7 @@ public final class PersistentMessages {
           getTypeBytes() {
         java.lang.Object ref = type_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           type_ = b;
@@ -758,7 +758,7 @@ public final class PersistentMessages {
        * <code>required .org.opendaylight.controller.mdsal.InstanceIdentifier path = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getPathFieldBuilder() {
         if (pathBuilder_ == null) {
           pathBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -875,7 +875,7 @@ public final class PersistentMessages {
        * <code>optional .org.opendaylight.controller.mdsal.Node data = 3;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getDataFieldBuilder() {
         if (dataBuilder_ == null) {
           dataBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -906,7 +906,7 @@ public final class PersistentMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Modification modification = 1;</code>
      */
-    java.util.List<org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification>
+    java.util.List<org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification> 
         getModificationList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Modification modification = 1;</code>
@@ -919,7 +919,7 @@ public final class PersistentMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Modification modification = 1;</code>
      */
-    java.util.List<? extends org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder>
+    java.util.List<? extends org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder> 
         getModificationOrBuilderList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Modification modification = 1;</code>
@@ -1040,7 +1040,7 @@ public final class PersistentMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Modification modification = 1;</code>
      */
-    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder>
+    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder> 
         getModificationOrBuilderList() {
       return modification_;
     }
@@ -1293,7 +1293,7 @@ public final class PersistentMessages {
               modificationBuilder_ = null;
               modification_ = other.modification_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              modificationBuilder_ =
+              modificationBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getModificationFieldBuilder() : null;
             } else {
@@ -1308,7 +1308,7 @@ public final class PersistentMessages {
       public final boolean isInitialized() {
         for (int i = 0; i < getModificationCount(); i++) {
           if (!getModification(i).isInitialized()) {
-
+            
             return false;
           }
         }
@@ -1529,7 +1529,7 @@ public final class PersistentMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Modification modification = 1;</code>
        */
-      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder>
+      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder> 
            getModificationOrBuilderList() {
         if (modificationBuilder_ != null) {
           return modificationBuilder_.getMessageOrBuilderList();
@@ -1555,12 +1555,12 @@ public final class PersistentMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Modification modification = 1;</code>
        */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification.Builder>
+      public java.util.List<org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification.Builder> 
            getModificationBuilderList() {
         return getModificationFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification, org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification.Builder, org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder>
+          org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification, org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification.Builder, org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder> 
           getModificationFieldBuilder() {
         if (modificationBuilder_ == null) {
           modificationBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
@@ -1605,15 +1605,16 @@ public final class PersistentMessages {
   static {
     java.lang.String[] descriptorData = {
       "\n\020Persistent.proto\022!org.opendaylight.con" +
-      "troller.mdsal\032\014Common.proto\"\230\001\n\014Modifica" +
-      "tion\022\014\n\004type\030\001 \002(\t\022C\n\004path\030\002 \002(\01325.org.o" +
-      "pendaylight.controller.mdsal.InstanceIde" +
-      "ntifier\0225\n\004data\030\003 \001(\0132\'.org.opendaylight" +
-      ".controller.mdsal.Node\"^\n\025CompositeModif" +
-      "ication\022E\n\014modification\030\001 \003(\0132/.org.open" +
-      "daylight.controller.mdsal.ModificationBO" +
-      "\n9org.opendaylight.controller.protobuff." +
-      "messages.persistentB\022PersistentMessages"
+      "troller.mdsal\032\014Common.proto\032\033AppendEntri" +
+      "esMessages.proto\"\230\001\n\014Modification\022\014\n\004typ" +
+      "e\030\001 \002(\t\022C\n\004path\030\002 \002(\01325.org.opendaylight" +
+      ".controller.mdsal.InstanceIdentifier\0225\n\004" +
+      "data\030\003 \001(\0132\'.org.opendaylight.controller" +
+      ".mdsal.Node\"^\n\025CompositeModification\022E\n\014" +
+      "modification\030\001 \003(\0132/.org.opendaylight.co" +
+      "ntroller.mdsal.ModificationBO\n9org.opend" +
+      "aylight.controller.protobuff.messages.pe",
+      "rsistentB\022PersistentMessages"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -1639,6 +1640,7 @@ public final class PersistentMessages {
       .internalBuildGeneratedFileFrom(descriptorData,
         new com.google.protobuf.Descriptors.FileDescriptor[] {
           org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.getDescriptor(),
+          org.opendaylight.controller.cluster.raft.protobuff.messages.AppendEntriesMessages.getDescriptor(),
         }, assigner);
   }
 
index 77cbd4da46c811e43a3a2692a40f007ab51e5097..e06dd0d42982d779b8aa40931338b88c528d6069 100644 (file)
@@ -837,7 +837,7 @@ public final class ListenerRegistrationMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -853,7 +853,7 @@ public final class ListenerRegistrationMessages {
         getDataChangeListenerActorPathBytes() {
       java.lang.Object ref = dataChangeListenerActorPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         dataChangeListenerActorPath_ = b;
@@ -1146,19 +1146,19 @@ public final class ListenerRegistrationMessages {
 
       public final boolean isInitialized() {
         if (!hasInstanceIdentifierPath()) {
-
+          
           return false;
         }
         if (!hasDataChangeListenerActorPath()) {
-
+          
           return false;
         }
         if (!hasDataChangeScope()) {
-
+          
           return false;
         }
         if (!getInstanceIdentifierPath().isInitialized()) {
-
+          
           return false;
         }
         return true;
@@ -1287,7 +1287,7 @@ public final class ListenerRegistrationMessages {
        * <code>required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPath = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getInstanceIdentifierPathFieldBuilder() {
         if (instanceIdentifierPathBuilder_ == null) {
           instanceIdentifierPathBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -1329,7 +1329,7 @@ public final class ListenerRegistrationMessages {
           getDataChangeListenerActorPathBytes() {
         java.lang.Object ref = dataChangeListenerActorPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           dataChangeListenerActorPath_ = b;
@@ -1556,7 +1556,7 @@ public final class ListenerRegistrationMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -1572,7 +1572,7 @@ public final class ListenerRegistrationMessages {
         getListenerRegistrationPathBytes() {
       java.lang.Object ref = listenerRegistrationPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         listenerRegistrationPath_ = b;
@@ -1801,7 +1801,7 @@ public final class ListenerRegistrationMessages {
 
       public final boolean isInitialized() {
         if (!hasListenerRegistrationPath()) {
-
+          
           return false;
         }
         return true;
@@ -1855,7 +1855,7 @@ public final class ListenerRegistrationMessages {
           getListenerRegistrationPathBytes() {
         java.lang.Object ref = listenerRegistrationPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           listenerRegistrationPath_ = b;
index 2324dfc2a238b1e197a04c360ee1c1b0db66ebf4..7c2a47e1b0daa740f96acf355b4281f9331ec7f2 100644 (file)
@@ -139,7 +139,7 @@ public final class ShardManagerMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -155,7 +155,7 @@ public final class ShardManagerMessages {
         getShardNameBytes() {
       java.lang.Object ref = shardName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         shardName_ = b;
@@ -377,7 +377,7 @@ public final class ShardManagerMessages {
 
       public final boolean isInitialized() {
         if (!hasShardName()) {
-
+          
           return false;
         }
         return true;
@@ -431,7 +431,7 @@ public final class ShardManagerMessages {
           getShardNameBytes() {
         java.lang.Object ref = shardName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           shardName_ = b;
@@ -618,7 +618,7 @@ public final class ShardManagerMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -634,7 +634,7 @@ public final class ShardManagerMessages {
         getPrimaryPathBytes() {
       java.lang.Object ref = primaryPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         primaryPath_ = b;
@@ -856,7 +856,7 @@ public final class ShardManagerMessages {
 
       public final boolean isInitialized() {
         if (!hasPrimaryPath()) {
-
+          
           return false;
         }
         return true;
@@ -910,7 +910,7 @@ public final class ShardManagerMessages {
           getPrimaryPathBytes() {
         java.lang.Object ref = primaryPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           primaryPath_ = b;
@@ -1097,7 +1097,7 @@ public final class ShardManagerMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -1113,7 +1113,7 @@ public final class ShardManagerMessages {
         getShardNameBytes() {
       java.lang.Object ref = shardName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         shardName_ = b;
@@ -1335,7 +1335,7 @@ public final class ShardManagerMessages {
 
       public final boolean isInitialized() {
         if (!hasShardName()) {
-
+          
           return false;
         }
         return true;
@@ -1389,7 +1389,7 @@ public final class ShardManagerMessages {
           getShardNameBytes() {
         java.lang.Object ref = shardName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           shardName_ = b;
index 63dd5e7081603aed89168980a5f6ad6bb27de28e..3f354ba40eeea6751b1410c47a5bcf7b39db86a6 100644 (file)
@@ -1066,7 +1066,7 @@ public final class ShardTransactionChainMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -1082,7 +1082,7 @@ public final class ShardTransactionChainMessages {
         getTransactionChainPathBytes() {
       java.lang.Object ref = transactionChainPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         transactionChainPath_ = b;
@@ -1304,7 +1304,7 @@ public final class ShardTransactionChainMessages {
 
       public final boolean isInitialized() {
         if (!hasTransactionChainPath()) {
-
+          
           return false;
         }
         return true;
@@ -1358,7 +1358,7 @@ public final class ShardTransactionChainMessages {
           getTransactionChainPathBytes() {
         java.lang.Object ref = transactionChainPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           transactionChainPath_ = b;
index 7ce3b586b43a6e5e470297b13d65bb5a8e199c80..ee2c70423e14fcaa8775d285f22cd2a8bb02f324 100644 (file)
@@ -757,7 +757,7 @@ public final class ShardTransactionMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -773,7 +773,7 @@ public final class ShardTransactionMessages {
         getTransactionIdBytes() {
       java.lang.Object ref = transactionId_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         transactionId_ = b;
@@ -995,7 +995,7 @@ public final class ShardTransactionMessages {
 
       public final boolean isInitialized() {
         if (!hasTransactionId()) {
-
+          
           return false;
         }
         return true;
@@ -1049,7 +1049,7 @@ public final class ShardTransactionMessages {
           getTransactionIdBytes() {
         java.lang.Object ref = transactionId_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           transactionId_ = b;
@@ -1256,7 +1256,7 @@ public final class ShardTransactionMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -1272,7 +1272,7 @@ public final class ShardTransactionMessages {
         getTransactionActorPathBytes() {
       java.lang.Object ref = transactionActorPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         transactionActorPath_ = b;
@@ -1299,7 +1299,7 @@ public final class ShardTransactionMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -1315,7 +1315,7 @@ public final class ShardTransactionMessages {
         getTransactionIdBytes() {
       java.lang.Object ref = transactionId_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         transactionId_ = b;
@@ -1560,11 +1560,11 @@ public final class ShardTransactionMessages {
 
       public final boolean isInitialized() {
         if (!hasTransactionActorPath()) {
-
+          
           return false;
         }
         if (!hasTransactionId()) {
-
+          
           return false;
         }
         return true;
@@ -1618,7 +1618,7 @@ public final class ShardTransactionMessages {
           getTransactionActorPathBytes() {
         java.lang.Object ref = transactionActorPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           transactionActorPath_ = b;
@@ -1692,7 +1692,7 @@ public final class ShardTransactionMessages {
           getTransactionIdBytes() {
         java.lang.Object ref = transactionId_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           transactionId_ = b;
@@ -2188,7 +2188,7 @@ public final class ShardTransactionMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -2204,7 +2204,7 @@ public final class ShardTransactionMessages {
         getActorPathBytes() {
       java.lang.Object ref = actorPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         actorPath_ = b;
@@ -2426,7 +2426,7 @@ public final class ShardTransactionMessages {
 
       public final boolean isInitialized() {
         if (!hasActorPath()) {
-
+          
           return false;
         }
         return true;
@@ -2480,7 +2480,7 @@ public final class ShardTransactionMessages {
           getActorPathBytes() {
         java.lang.Object ref = actorPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           actorPath_ = b;
@@ -2902,11 +2902,11 @@ public final class ShardTransactionMessages {
 
       public final boolean isInitialized() {
         if (!hasInstanceIdentifierPathArguments()) {
-
+          
           return false;
         }
         if (!getInstanceIdentifierPathArguments().isInitialized()) {
-
+          
           return false;
         }
         return true;
@@ -3035,7 +3035,7 @@ public final class ShardTransactionMessages {
        * <code>required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPathArguments = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getInstanceIdentifierPathArgumentsFieldBuilder() {
         if (instanceIdentifierPathArgumentsBuilder_ == null) {
           instanceIdentifierPathArgumentsBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -3734,11 +3734,11 @@ public final class ShardTransactionMessages {
 
       public final boolean isInitialized() {
         if (!hasInstanceIdentifierPathArguments()) {
-
+          
           return false;
         }
         if (!getInstanceIdentifierPathArguments().isInitialized()) {
-
+          
           return false;
         }
         return true;
@@ -3867,7 +3867,7 @@ public final class ShardTransactionMessages {
        * <code>required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPathArguments = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getInstanceIdentifierPathArgumentsFieldBuilder() {
         if (instanceIdentifierPathArgumentsBuilder_ == null) {
           instanceIdentifierPathArgumentsBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -4256,7 +4256,7 @@ public final class ShardTransactionMessages {
       public final boolean isInitialized() {
         if (hasNormalizedNode()) {
           if (!getNormalizedNode().isInitialized()) {
-
+            
             return false;
           }
         }
@@ -4386,7 +4386,7 @@ public final class ShardTransactionMessages {
        * <code>optional .org.opendaylight.controller.mdsal.Node normalizedNode = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getNormalizedNodeFieldBuilder() {
         if (normalizedNodeBuilder_ == null) {
           normalizedNodeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -4859,19 +4859,19 @@ public final class ShardTransactionMessages {
 
       public final boolean isInitialized() {
         if (!hasInstanceIdentifierPathArguments()) {
-
+          
           return false;
         }
         if (!hasNormalizedNode()) {
-
+          
           return false;
         }
         if (!getInstanceIdentifierPathArguments().isInitialized()) {
-
+          
           return false;
         }
         if (!getNormalizedNode().isInitialized()) {
-
+          
           return false;
         }
         return true;
@@ -5000,7 +5000,7 @@ public final class ShardTransactionMessages {
        * <code>required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPathArguments = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getInstanceIdentifierPathArgumentsFieldBuilder() {
         if (instanceIdentifierPathArgumentsBuilder_ == null) {
           instanceIdentifierPathArgumentsBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -5117,7 +5117,7 @@ public final class ShardTransactionMessages {
        * <code>required .org.opendaylight.controller.mdsal.Node normalizedNode = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getNormalizedNodeFieldBuilder() {
         if (normalizedNodeBuilder_ == null) {
           normalizedNodeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -5899,19 +5899,19 @@ public final class ShardTransactionMessages {
 
       public final boolean isInitialized() {
         if (!hasInstanceIdentifierPathArguments()) {
-
+          
           return false;
         }
         if (!hasNormalizedNode()) {
-
+          
           return false;
         }
         if (!getInstanceIdentifierPathArguments().isInitialized()) {
-
+          
           return false;
         }
         if (!getNormalizedNode().isInitialized()) {
-
+          
           return false;
         }
         return true;
@@ -6040,7 +6040,7 @@ public final class ShardTransactionMessages {
        * <code>required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPathArguments = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getInstanceIdentifierPathArgumentsFieldBuilder() {
         if (instanceIdentifierPathArgumentsBuilder_ == null) {
           instanceIdentifierPathArgumentsBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -6157,7 +6157,7 @@ public final class ShardTransactionMessages {
        * <code>required .org.opendaylight.controller.mdsal.Node normalizedNode = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getNormalizedNodeFieldBuilder() {
         if (normalizedNodeBuilder_ == null) {
           normalizedNodeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
diff --git a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/resources/CompositeModificationPayload.proto b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/resources/CompositeModificationPayload.proto
new file mode 100644 (file)
index 0000000..b571cd2
--- /dev/null
@@ -0,0 +1,11 @@
+package org.opendaylight.controller.mdsal;
+
+import "AppendEntriesMessages.proto";
+import "Common.proto";
+import "Persistent.proto";
+
+extend org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload {
+    optional CompositeModification modification=2;
+}
+
+
index b3f28197ed3e24459d721071c476dae1b8d52bb3..8e834494cbe590aabdb2c68a49389f5d05c040aa 100644 (file)
@@ -1,6 +1,7 @@
 package org.opendaylight.controller.mdsal;
 
 import "Common.proto";
+import "AppendEntriesMessages.proto";
 
 option java_package = "org.opendaylight.controller.protobuff.messages.persistent";
 option java_outer_classname = "PersistentMessages";
@@ -16,3 +17,4 @@ message Modification {
 message CompositeModification {
     repeated Modification modification=1;
 }
+
index 6ee301d827ec3caa91f1b6f81dd13c6c21784afb..ce3bfe9a4c81a65decaf51625c7505c07cafc05e 100644 (file)
       <groupId>org.opendaylight.controller</groupId>
       <artifactId>netconf-util</artifactId>
     </dependency>
-
+      <dependency>
+          <groupId>org.opendaylight.controller</groupId>
+          <artifactId>sal-core-spi</artifactId>
+      </dependency>
+      <dependency>
+          <groupId>org.opendaylight.controller</groupId>
+          <artifactId>sal-common-impl</artifactId>
+      </dependency>
     <!-- Yang tools-->
 
     <dependency>
       <groupId>org.opendaylight.yangtools</groupId>
       <artifactId>yang-data-api</artifactId>
+
     </dependency>
+      <dependency>
+          <groupId>org.opendaylight.yangtools</groupId>
+          <artifactId>yang-model-api</artifactId>
+
+      </dependency>
 
     <dependency>
       <groupId>org.opendaylight.yangtools</groupId>
       <artifactId>yang-data-impl</artifactId>
+
     </dependency>
 
     <dependency>
       <groupId>org.opendaylight.yangtools</groupId>
       <artifactId>yang-common</artifactId>
+
     </dependency>
 
 
       <version>${slf4j.version}</version>
       <scope>test</scope>
     </dependency>
+      <dependency>
+          <groupId>org.opendaylight.yangtools</groupId>
+          <artifactId>yang-parser-impl</artifactId>
+          <scope>test</scope>
+      </dependency>
+      <dependency>
+          <groupId>com.google.collections</groupId>
+          <artifactId>google-collections</artifactId>
+          <version>1.0</version>
+          <scope>test</scope>
+      </dependency>
 
   </dependencies>
 
index 8315bbeeb33b8beef8101c4c820ebd63a4ae1f21..9824889b8023d4e31e20d835731c59ef7047c314 100644 (file)
@@ -2,10 +2,8 @@ package org.opendaylight.controller.config.yang.config.remote_rpc_connector;
 
 import org.opendaylight.controller.remote.rpc.RemoteRpcProviderFactory;
 import org.opendaylight.controller.sal.core.api.Broker;
-import org.osgi.framework.BundleContext;
 
 public class RemoteRPCBrokerModule extends org.opendaylight.controller.config.yang.config.remote_rpc_connector.AbstractRemoteRPCBrokerModule {
-  private BundleContext bundleContext;
   public RemoteRPCBrokerModule(org.opendaylight.controller.config.api.ModuleIdentifier identifier, org.opendaylight.controller.config.api.DependencyResolver dependencyResolver) {
     super(identifier, dependencyResolver);
   }
@@ -22,10 +20,6 @@ public class RemoteRPCBrokerModule extends org.opendaylight.controller.config.ya
   @Override
   public java.lang.AutoCloseable createInstance() {
     Broker broker = getDomBrokerDependency();
-    return RemoteRpcProviderFactory.createInstance(broker, bundleContext);
-  }
-
-  public void setBundleContext(final BundleContext bundleContext) {
-    this.bundleContext = bundleContext;
+    return RemoteRpcProviderFactory.createInstance(broker);
   }
 }
index e1ba46ae15fb94c993c1069b4dd2e627bb803ada..330845b14fc28bedbc9268467a2fa5075addc5a4 100644 (file)
@@ -9,27 +9,7 @@
 */
 package org.opendaylight.controller.config.yang.config.remote_rpc_connector;
 
-import org.opendaylight.controller.config.api.DependencyResolver;
-import org.opendaylight.controller.config.api.DynamicMBeanWithInstance;
-import org.opendaylight.controller.config.spi.Module;
-import org.osgi.framework.BundleContext;
 
 public class RemoteRPCBrokerModuleFactory extends org.opendaylight.controller.config.yang.config.remote_rpc_connector.AbstractRemoteRPCBrokerModuleFactory {
 
-  @Override
-  public Module createModule(String instanceName, DependencyResolver dependencyResolver, BundleContext bundleContext) {
-    RemoteRPCBrokerModule module = (RemoteRPCBrokerModule)super.createModule(instanceName,dependencyResolver,bundleContext);
-    module.setBundleContext(bundleContext);
-    return module;
-  }
-
-  @Override
-  public Module createModule(String instanceName, DependencyResolver dependencyResolver,
-                             DynamicMBeanWithInstance old, BundleContext bundleContext) throws Exception {
-    RemoteRPCBrokerModule module = (RemoteRPCBrokerModule)super.createModule(instanceName, dependencyResolver,
-        old, bundleContext);
-    module.setBundleContext(bundleContext);
-    return module;
-  }
-
 }
index 43aa5b7e85a4a3da136de723210c5e3ae3f34cca..d384144f4fde6389861597777bd795b30478e773 100644 (file)
@@ -7,21 +7,19 @@ import org.opendaylight.controller.remote.rpc.messages.ErrorResponse;
 import org.opendaylight.controller.remote.rpc.messages.InvokeRoutedRpc;
 import org.opendaylight.controller.remote.rpc.messages.InvokeRpc;
 import org.opendaylight.controller.remote.rpc.messages.RpcResponse;
-import org.opendaylight.controller.sal.common.util.RpcErrors;
-import org.opendaylight.controller.sal.common.util.Rpcs;
+import org.opendaylight.controller.remote.rpc.utils.ActorUtil;
+import org.opendaylight.controller.remote.rpc.utils.XmlUtils;
 import org.opendaylight.controller.sal.core.api.RoutedRpcDefaultImplementation;
 import org.opendaylight.controller.sal.core.api.RpcImplementation;
 import org.opendaylight.yangtools.yang.common.QName;
-import org.opendaylight.yangtools.yang.common.RpcError;
 import org.opendaylight.yangtools.yang.common.RpcResult;
+import org.opendaylight.yangtools.yang.common.RpcResultBuilder;
 import org.opendaylight.yangtools.yang.data.api.CompositeNode;
-import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Set;
 
@@ -37,7 +35,7 @@ public class RemoteRpcImplementation implements RpcImplementation,
   }
 
   @Override
-  public ListenableFuture<RpcResult<CompositeNode>> invokeRpc(QName rpc, InstanceIdentifier identifier, CompositeNode input) {
+  public ListenableFuture<RpcResult<CompositeNode>> invokeRpc(QName rpc, YangInstanceIdentifier identifier, CompositeNode input) {
     InvokeRoutedRpc rpcMsg = new InvokeRoutedRpc(rpc, identifier, input);
 
     return executeMsg(rpcMsg);
@@ -56,22 +54,33 @@ public class RemoteRpcImplementation implements RpcImplementation,
   }
 
   private ListenableFuture<RpcResult<CompositeNode>> executeMsg(Object rpcMsg) {
-    CompositeNode result = null;
-    Collection<RpcError> errors = errors = new ArrayList<>();
+    ListenableFuture<RpcResult<CompositeNode>> listenableFuture = null;
+
     try {
       Object response = ActorUtil.executeLocalOperation(rpcBroker, rpcMsg, ActorUtil.ASK_DURATION, ActorUtil.AWAIT_DURATION);
       if(response instanceof RpcResponse) {
+
         RpcResponse rpcResponse = (RpcResponse) response;
-        result = XmlUtils.xmlToCompositeNode(rpcResponse.getResultCompositeNode());
+        CompositeNode result = XmlUtils.xmlToCompositeNode(rpcResponse.getResultCompositeNode());
+        listenableFuture = Futures.immediateFuture(RpcResultBuilder.success(result).build());
+
       } else if(response instanceof ErrorResponse) {
+
         ErrorResponse errorResponse = (ErrorResponse) response;
         Exception e = errorResponse.getException();
-        errors.add(RpcErrors.getRpcError(null, null, null, null, e.getMessage(), null, e.getCause()));
+        final RpcResultBuilder<CompositeNode> failed = RpcResultBuilder.failed();
+        failed.withError(null, null, e.getMessage(), null, null, e.getCause());
+        listenableFuture = Futures.immediateFuture(failed.build());
+
       }
     } catch (Exception e) {
       LOG.error("Error occurred while invoking RPC actor {}", e.toString());
-      errors.add(RpcErrors.getRpcError(null, null, null, null, e.getMessage(), null, e.getCause()));
+
+      final RpcResultBuilder<CompositeNode> failed = RpcResultBuilder.failed();
+      failed.withError(null, null, e.getMessage(), null, null, e.getCause());
+      listenableFuture = Futures.immediateFuture(failed.build());
     }
-    return Futures.immediateFuture(Rpcs.getRpcResult(true, result, errors));
+
+    return listenableFuture;
   }
 }
index 1bb7ea451441683750dcd8e4a18051e61f829e01..3df572d7c27226511eaa08e096d4477807099f6f 100644 (file)
@@ -11,37 +11,35 @@ package org.opendaylight.controller.remote.rpc;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
+import org.opendaylight.controller.remote.rpc.messages.UpdateSchemaContext;
 import org.opendaylight.controller.remote.rpc.registry.ClusterWrapper;
 import org.opendaylight.controller.remote.rpc.registry.ClusterWrapperImpl;
-import org.opendaylight.controller.remote.rpc.registry.RpcRegistry;
 import org.opendaylight.controller.sal.core.api.Broker;
 import org.opendaylight.controller.sal.core.api.Provider;
 import org.opendaylight.controller.sal.core.api.RpcProvisionRegistry;
 import org.opendaylight.controller.sal.core.api.model.SchemaService;
-import org.opendaylight.yangtools.yang.common.QName;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.opendaylight.yangtools.yang.model.api.SchemaContextListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.Collection;
-import java.util.Set;
 
 /**
  * This is the base class which initialize all the actors, listeners and
  * default RPc implementation so remote invocation of rpcs.
  */
-public class RemoteRpcProvider implements AutoCloseable, Provider{
+public class RemoteRpcProvider implements AutoCloseable, Provider, SchemaContextListener {
 
   private static final Logger LOG = LoggerFactory.getLogger(RemoteRpcProvider.class);
 
   private final ActorSystem actorSystem;
-  private ActorRef rpcBroker;
-  private ActorRef rpcRegistry;
   private final RpcProvisionRegistry rpcProvisionRegistry;
   private Broker.ProviderSession brokerSession;
-  private RpcListener rpcListener;
-  private RoutedRpcListener routeChangeListener;
-  private RemoteRpcImplementation rpcImplementation;
+  private SchemaContext schemaContext;
+  private ActorRef rpcManager;
+
+
   public RemoteRpcProvider(ActorSystem actorSystem, RpcProvisionRegistry rpcProvisionRegistry) {
     this.actorSystem = actorSystem;
     this.rpcProvisionRegistry = rpcProvisionRegistry;
@@ -50,8 +48,6 @@ public class RemoteRpcProvider implements AutoCloseable, Provider{
   @Override
   public void close() throws Exception {
     this.actorSystem.shutdown();
-    unregisterSupportedRpcs();
-    unregisterSupportedRoutedRpcs();
   }
 
   @Override
@@ -66,64 +62,22 @@ public class RemoteRpcProvider implements AutoCloseable, Provider{
   }
 
   private void start() {
-    LOG.debug("Starting all rpc listeners.");
+    LOG.info("Starting all rpc listeners and actors.");
     // Create actor to handle and sync routing table in cluster
     ClusterWrapper clusterWrapper = new ClusterWrapperImpl(actorSystem);
-    rpcRegistry = actorSystem.actorOf(RpcRegistry.props(clusterWrapper), "rpc-registry");
-
-    // Create actor to invoke and execute rpc
     SchemaService schemaService = brokerSession.getService(SchemaService.class);
-    SchemaContext schemaContext = schemaService.getGlobalContext();
-    rpcBroker = actorSystem.actorOf(RpcBroker.props(brokerSession, rpcRegistry, schemaContext), "rpc-broker");
-    String rpcBrokerPath = clusterWrapper.getAddress().toString() + "/user/rpc-broker";
-    rpcListener = new RpcListener(rpcRegistry, rpcBrokerPath);
-    routeChangeListener = new RoutedRpcListener(rpcRegistry, rpcBrokerPath);
-    rpcImplementation = new RemoteRpcImplementation(rpcBroker, schemaContext);
-    brokerSession.addRpcRegistrationListener(rpcListener);
-    rpcProvisionRegistry.registerRouteChangeListener(routeChangeListener);
-    rpcProvisionRegistry.setRoutedRpcDefaultDelegate(rpcImplementation);
-    announceSupportedRpcs();
-    announceSupportedRoutedRpcs();
+    schemaContext = schemaService.getGlobalContext();
 
-  }
+    rpcManager = actorSystem.actorOf(RpcManager.props(clusterWrapper, schemaContext, brokerSession, rpcProvisionRegistry), "rpc");
 
-  /**
-   * Add all the locally registered RPCs in the clustered routing table
-   */
-  private void announceSupportedRpcs(){
-    LOG.debug("Adding all supported rpcs to routing table");
-    Set<QName> currentlySupported = brokerSession.getSupportedRpcs();
-    for (QName rpc : currentlySupported) {
-      rpcListener.onRpcImplementationAdded(rpc);
-    }
+    LOG.debug("Rpc actors are created.");
   }
 
-  /**
-   * Add all the locally registered Routed RPCs in the clustered routing table
-   */
-  private void announceSupportedRoutedRpcs(){
-
-    //TODO: announce all routed RPCs as well
 
-  }
-
-  /**
-   * Un-Register all the supported RPCs from clustered routing table
-   */
-  private void unregisterSupportedRpcs(){
-    LOG.debug("removing all supported rpcs to routing table");
-    Set<QName> currentlySupported = brokerSession.getSupportedRpcs();
-    for (QName rpc : currentlySupported) {
-      rpcListener.onRpcImplementationRemoved(rpc);
-    }
-  }
-
-  /**
-   * Un-Register all the locally supported Routed RPCs from clustered routing table
-   */
-  private void unregisterSupportedRoutedRpcs(){
-
-    //TODO: remove all routed RPCs as well
+  @Override
+  public void onGlobalContextUpdated(SchemaContext schemaContext) {
+    this.schemaContext = schemaContext;
+    rpcManager.tell(new UpdateSchemaContext(schemaContext), null);
 
   }
 }
index 61dc8183df662b8fc7d4545519661fe9412a253a..4c40ca177756877f121f0c5187b747241342655c 100644 (file)
@@ -11,13 +11,12 @@ package org.opendaylight.controller.remote.rpc;
 
 import org.opendaylight.controller.sal.core.api.Broker;
 import org.opendaylight.controller.sal.core.api.RpcProvisionRegistry;
-import org.osgi.framework.BundleContext;
 
 public class RemoteRpcProviderFactory {
-    public static RemoteRpcProvider createInstance(final Broker broker, final BundleContext bundleContext){
+    public static RemoteRpcProvider createInstance(final Broker broker){
       RemoteRpcProvider rpcProvider =
           new RemoteRpcProvider(ActorSystemFactory.getInstance(), (RpcProvisionRegistry) broker);
-      broker.registerProvider(rpcProvider, bundleContext);
+      broker.registerProvider(rpcProvider);
       return rpcProvider;
     }
 }
index ea72238fbcb3aa6cc29621c76821d08a0eb4e158..85d21381b61bf127fb95e0e11a67275d6d25ec86 100644 (file)
@@ -7,20 +7,23 @@
  */
 package org.opendaylight.controller.remote.rpc;
 
+import com.google.common.base.Preconditions;
 import org.opendaylight.controller.sal.connector.api.RpcRouter;
 import org.opendaylight.yangtools.yang.common.QName;
-import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+
 
 import java.io.Serializable;
 
-public class RouteIdentifierImpl implements RpcRouter.RouteIdentifier<QName, QName, InstanceIdentifier>,Serializable {
+public class RouteIdentifierImpl implements RpcRouter.RouteIdentifier<QName, QName, YangInstanceIdentifier>,Serializable {
   private static final long serialVersionUID = 1L;
 
-  private QName context;
-  private QName type;
-  private InstanceIdentifier route;
+  private final QName context;
+  private final QName type;
+  private final YangInstanceIdentifier route;
 
-  public RouteIdentifierImpl(QName context, QName type, InstanceIdentifier route) {
+  public RouteIdentifierImpl(final QName context, final QName type, final YangInstanceIdentifier route) {
+    Preconditions.checkNotNull(type, "Rpc type should not be null");
     this.context = context;
     this.type = type;
     this.route = route;
@@ -37,7 +40,7 @@ public class RouteIdentifierImpl implements RpcRouter.RouteIdentifier<QName, QNa
   }
 
   @Override
-  public InstanceIdentifier getRoute() {
+  public YangInstanceIdentifier getRoute() {
     return this.route;
   }
 
index a0df3629fa521367f68ca48263d913343c189a68..a6eeac02708a20e8a746581d80c5054a776ed671 100644 (file)
@@ -10,13 +10,15 @@ package org.opendaylight.controller.remote.rpc;
 
 
 import akka.actor.ActorRef;
+import com.google.common.base.Preconditions;
 import org.opendaylight.controller.md.sal.common.api.routing.RouteChange;
 import org.opendaylight.controller.md.sal.common.api.routing.RouteChangeListener;
 import org.opendaylight.controller.remote.rpc.messages.AddRoutedRpc;
 import org.opendaylight.controller.remote.rpc.messages.RemoveRoutedRpc;
+import org.opendaylight.controller.remote.rpc.utils.ActorUtil;
 import org.opendaylight.controller.sal.connector.api.RpcRouter;
 import org.opendaylight.controller.sal.core.api.RpcRoutingContext;
-import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -24,22 +26,25 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-public class RoutedRpcListener implements RouteChangeListener<RpcRoutingContext, InstanceIdentifier>{
+public class RoutedRpcListener implements RouteChangeListener<RpcRoutingContext, YangInstanceIdentifier>{
   private static final Logger LOG = LoggerFactory.getLogger(RoutedRpcListener.class);
   private final ActorRef rpcRegistry;
   private final String actorPath;
 
   public RoutedRpcListener(ActorRef rpcRegistry, String actorPath) {
+    Preconditions.checkNotNull(rpcRegistry, "rpc registry actor should not be null");
+    Preconditions.checkNotNull(actorPath, "actor path of rpc broker on current node should not be null");
+
     this.rpcRegistry = rpcRegistry;
     this.actorPath = actorPath;
   }
 
   @Override
-  public void onRouteChange(RouteChange<RpcRoutingContext, InstanceIdentifier> routeChange) {
-    Map<RpcRoutingContext, Set<InstanceIdentifier>> announcements = routeChange.getAnnouncements();
+  public void onRouteChange(RouteChange<RpcRoutingContext, YangInstanceIdentifier> routeChange) {
+    Map<RpcRoutingContext, Set<YangInstanceIdentifier>> announcements = routeChange.getAnnouncements();
     announce(getRouteIdentifiers(announcements));
 
-    Map<RpcRoutingContext, Set<InstanceIdentifier>> removals = routeChange.getRemovals();
+    Map<RpcRoutingContext, Set<YangInstanceIdentifier>> removals = routeChange.getRemovals();
     remove(getRouteIdentifiers(removals));
   }
 
@@ -78,12 +83,12 @@ public class RoutedRpcListener implements RouteChangeListener<RpcRoutingContext,
    * @param changes
    * @return
    */
-  private Set<RpcRouter.RouteIdentifier<?, ?, ?>> getRouteIdentifiers(Map<RpcRoutingContext, Set<InstanceIdentifier>> changes) {
+  private Set<RpcRouter.RouteIdentifier<?, ?, ?>> getRouteIdentifiers(Map<RpcRoutingContext, Set<YangInstanceIdentifier>> changes) {
     RouteIdentifierImpl routeId = null;
     Set<RpcRouter.RouteIdentifier<?, ?, ?>> routeIdSet = new HashSet<>();
 
     for (RpcRoutingContext context : changes.keySet()){
-      for (InstanceIdentifier instanceId : changes.get(context)){
+      for (YangInstanceIdentifier instanceId : changes.get(context)){
         routeId = new RouteIdentifierImpl(null, context.getRpc(), instanceId);
         routeIdSet.add(routeId);
       }
index 3354fc3da25e990279cacec849fa131e57e5b963..26e8e960e3ce46d5f9ea9b3047160134a1f2df36 100644 (file)
@@ -20,6 +20,8 @@ import org.opendaylight.controller.remote.rpc.messages.GetRpcReply;
 import org.opendaylight.controller.remote.rpc.messages.InvokeRoutedRpc;
 import org.opendaylight.controller.remote.rpc.messages.InvokeRpc;
 import org.opendaylight.controller.remote.rpc.messages.RpcResponse;
+import org.opendaylight.controller.remote.rpc.utils.ActorUtil;
+import org.opendaylight.controller.remote.rpc.utils.XmlUtils;
 import org.opendaylight.controller.sal.core.api.Broker;
 import org.opendaylight.yangtools.yang.common.RpcResult;
 import org.opendaylight.yangtools.yang.data.api.CompositeNode;
@@ -38,7 +40,7 @@ public class RpcBroker extends AbstractUntypedActor {
   private static final Logger LOG = LoggerFactory.getLogger(RpcBroker.class);
   private final Broker.ProviderSession brokerSession;
   private final ActorRef rpcRegistry;
-  private final SchemaContext schemaContext;
+  private SchemaContext schemaContext;
 
   private RpcBroker(Broker.ProviderSession brokerSession, ActorRef rpcRegistry, SchemaContext schemaContext){
     this.brokerSession = brokerSession;
@@ -72,7 +74,7 @@ public class RpcBroker extends AbstractUntypedActor {
     try {
       RouteIdentifierImpl routeId = new RouteIdentifierImpl(null, msg.getRpc(), msg.getIdentifier());
       GetRoutedRpc routedRpcMsg = new GetRoutedRpc(routeId);
-      GetRoutedRpcReply rpcReply = (GetRoutedRpcReply)ActorUtil.executeLocalOperation(rpcRegistry, routedRpcMsg, ActorUtil.LOCAL_ASK_DURATION, ActorUtil.LOCAL_AWAIT_DURATION);
+      GetRoutedRpcReply rpcReply = (GetRoutedRpcReply) ActorUtil.executeLocalOperation(rpcRegistry, routedRpcMsg, ActorUtil.LOCAL_ASK_DURATION, ActorUtil.LOCAL_AWAIT_DURATION);
 
       String remoteActorPath = rpcReply.getRoutePath();
       if(remoteActorPath == null) {
@@ -105,11 +107,12 @@ public class RpcBroker extends AbstractUntypedActor {
       String remoteActorPath = rpcReply.getRoutePath();
 
       if(remoteActorPath == null) {
-        LOG.debug("No remote actor found for rpc execution.");
+        LOG.debug("No remote actor found for rpc {{}}.", msg.getRpc());
 
         getSender().tell(new ErrorResponse(
-          new IllegalStateException("No remote actor found for rpc execution.")), self());
+          new IllegalStateException("No remote actor found for rpc execution of : " + msg.getRpc())), self());
       } else {
+
         ExecuteRpc executeMsg = new ExecuteRpc(XmlUtils.inputCompositeNodeToXml(msg.getInput(), schemaContext), msg.getRpc());
         Object operationRes = ActorUtil.executeRemoteOperation(this.context().actorSelection(remoteActorPath),
             executeMsg, ActorUtil.REMOTE_ASK_DURATION, ActorUtil.REMOTE_AWAIT_DURATION);
@@ -127,7 +130,6 @@ public class RpcBroker extends AbstractUntypedActor {
     try {
       Future<RpcResult<CompositeNode>> rpc = brokerSession.rpc(msg.getRpc(), XmlUtils.inputXmlToCompositeNode(msg.getRpc(), msg.getInputCompositeNode(), schemaContext));
       RpcResult<CompositeNode> rpcResult = rpc != null ? rpc.get():null;
-
       CompositeNode result = rpcResult != null ? rpcResult.getResult() : null;
       getSender().tell(new RpcResponse(XmlUtils.outputCompositeNodeToXml(result, schemaContext)), self());
     } catch (Exception e) {
index ae760fadc41649033ffac4d803d684c8a4963ac0..f6149906692ad0e5133938068508e80364d0769e 100644 (file)
@@ -12,6 +12,7 @@ package org.opendaylight.controller.remote.rpc;
 import akka.actor.ActorRef;
 import org.opendaylight.controller.remote.rpc.messages.AddRpc;
 import org.opendaylight.controller.remote.rpc.messages.RemoveRpc;
+import org.opendaylight.controller.remote.rpc.utils.ActorUtil;
 import org.opendaylight.controller.sal.core.api.RpcRegistrationListener;
 import org.opendaylight.yangtools.yang.common.QName;
 import org.slf4j.Logger;
diff --git a/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/RpcManager.java b/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/RpcManager.java
new file mode 100644 (file)
index 0000000..4925a17
--- /dev/null
@@ -0,0 +1,128 @@
+/*
+ * 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.remote.rpc;
+
+
+import akka.actor.ActorRef;
+import akka.actor.OneForOneStrategy;
+import akka.actor.Props;
+import akka.actor.SupervisorStrategy;
+import akka.japi.Creator;
+import akka.japi.Function;
+import org.opendaylight.controller.remote.rpc.messages.UpdateSchemaContext;
+import org.opendaylight.controller.remote.rpc.registry.ClusterWrapper;
+import org.opendaylight.controller.remote.rpc.registry.RpcRegistry;
+import org.opendaylight.controller.sal.core.api.Broker;
+import org.opendaylight.controller.sal.core.api.RpcProvisionRegistry;
+import org.opendaylight.yangtools.yang.common.QName;
+import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.duration.Duration;
+import java.util.Set;
+
+/**
+ * This class acts as a supervisor, creates all the actors, resumes them, if an exception is thrown.
+ *
+ * It also starts the rpc listeners
+ */
+
+public class RpcManager extends AbstractUntypedActor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RpcManager.class);
+
+  private SchemaContext schemaContext;
+  private final ClusterWrapper clusterWrapper;
+  private ActorRef rpcBroker;
+  private ActorRef rpcRegistry;
+  private final Broker.ProviderSession brokerSession;
+  private RpcListener rpcListener;
+  private RoutedRpcListener routeChangeListener;
+  private RemoteRpcImplementation rpcImplementation;
+  private final RpcProvisionRegistry rpcProvisionRegistry;
+
+  private RpcManager(ClusterWrapper clusterWrapper, SchemaContext schemaContext,
+                     Broker.ProviderSession brokerSession, RpcProvisionRegistry rpcProvisionRegistry) {
+    this.clusterWrapper = clusterWrapper;
+    this.schemaContext = schemaContext;
+    this.brokerSession = brokerSession;
+    this.rpcProvisionRegistry = rpcProvisionRegistry;
+
+    createRpcActors();
+    startListeners();
+  }
+
+
+  public static Props props(final ClusterWrapper clusterWrapper, final SchemaContext schemaContext,
+                            final Broker.ProviderSession brokerSession, final RpcProvisionRegistry rpcProvisionRegistry) {
+    return Props.create(new Creator<RpcManager>() {
+      @Override
+      public RpcManager create() throws Exception {
+        return new RpcManager(clusterWrapper, schemaContext, brokerSession, rpcProvisionRegistry);
+      }
+    });
+  }
+
+  private void createRpcActors() {
+    LOG.debug("Create rpc registry and broker actors");
+
+    rpcRegistry = getContext().actorOf(RpcRegistry.props(clusterWrapper), "rpc-registry");
+    rpcBroker = getContext().actorOf(RpcBroker.props(brokerSession, rpcRegistry, schemaContext), "rpc-broker");
+  }
+
+  private void startListeners() {
+    LOG.debug("Registers rpc listeners");
+
+    String rpcBrokerPath = clusterWrapper.getAddress().toString() + "/user/rpc/rpc-broker";
+    rpcListener = new RpcListener(rpcRegistry, rpcBrokerPath);
+    routeChangeListener = new RoutedRpcListener(rpcRegistry, rpcBrokerPath);
+    rpcImplementation = new RemoteRpcImplementation(rpcBroker, schemaContext);
+
+    brokerSession.addRpcRegistrationListener(rpcListener);
+    rpcProvisionRegistry.registerRouteChangeListener(routeChangeListener);
+    rpcProvisionRegistry.setRoutedRpcDefaultDelegate(rpcImplementation);
+    announceSupportedRpcs();
+  }
+
+  /**
+   * Add all the locally registered RPCs in the clustered routing table
+   */
+  private void announceSupportedRpcs(){
+    LOG.debug("Adding all supported rpcs to routing table");
+    Set<QName> currentlySupported = brokerSession.getSupportedRpcs();
+    for (QName rpc : currentlySupported) {
+      rpcListener.onRpcImplementationAdded(rpc);
+    }
+  }
+
+
+  @Override
+  protected void handleReceive(Object message) throws Exception {
+    if(message instanceof UpdateSchemaContext) {
+      updateSchemaContext((UpdateSchemaContext) message);
+    }
+
+  }
+
+  private void updateSchemaContext(UpdateSchemaContext message) {
+    this.schemaContext = message.getSchemaContext();
+  }
+
+  @Override
+  public SupervisorStrategy supervisorStrategy() {
+    return new OneForOneStrategy(10, Duration.create("1 minute"),
+        new Function<Throwable, SupervisorStrategy.Directive>() {
+          @Override
+          public SupervisorStrategy.Directive apply(Throwable t) {
+            return SupervisorStrategy.resume();
+          }
+        }
+    );
+  }
+}
diff --git a/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/XmlUtils.java b/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/XmlUtils.java
deleted file mode 100644 (file)
index 5fb2bb8..0000000
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
- *
- * This program and the accompanying materials are made available under the
- * terms of the Eclipse Public License v1.0 which accompanies this distribution,
- * and is available at http://www.eclipse.org/legal/epl-v10.html
- */
-
-package org.opendaylight.controller.remote.rpc;
-
-import com.google.common.base.Optional;
-import org.opendaylight.yangtools.yang.common.QName;
-import org.opendaylight.yangtools.yang.data.api.CompositeNode;
-import org.opendaylight.yangtools.yang.data.api.Node;
-import org.opendaylight.yangtools.yang.data.api.SimpleNode;
-import org.opendaylight.yangtools.yang.data.impl.XmlTreeBuilder;
-import org.opendaylight.yangtools.yang.data.impl.codec.xml.XmlDocumentUtils;
-import org.opendaylight.yangtools.yang.model.api.DataSchemaNode;
-import org.opendaylight.yangtools.yang.model.api.RpcDefinition;
-import org.opendaylight.yangtools.yang.model.api.SchemaContext;
-import org.opendaylight.controller.netconf.util.xml.XmlUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.w3c.dom.Document;
-import org.xml.sax.SAXException;
-
-import javax.activation.UnsupportedDataTypeException;
-import javax.xml.stream.XMLStreamException;
-import javax.xml.transform.OutputKeys;
-import javax.xml.transform.Transformer;
-import javax.xml.transform.TransformerException;
-import javax.xml.transform.TransformerFactory;
-import javax.xml.transform.dom.DOMSource;
-import javax.xml.transform.stream.StreamResult;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.Set;
-
-public class XmlUtils {
-
-  private static final Logger LOG = LoggerFactory.getLogger(XmlUtils.class);
-
-  public static String inputCompositeNodeToXml(CompositeNode cNode, SchemaContext schemaContext){
-    if (cNode == null) return new String();
-
-    //Document domTree = NodeUtils.buildShadowDomTree(cNode);
-    Document domTree = null;
-    try {
-      Set<RpcDefinition> rpcs =  schemaContext.getOperations();
-      for(RpcDefinition rpc : rpcs) {
-        if(rpc.getQName().equals(cNode.getNodeType())){
-          domTree = XmlDocumentUtils.toDocument(cNode, rpc.getInput(), XmlDocumentUtils.defaultValueCodecProvider());
-          break;
-        }
-      }
-
-    } catch (UnsupportedDataTypeException e) {
-      LOG.error("Error during translation of CompositeNode to Document", e);
-    }
-    return domTransformer(domTree);
-  }
-
-  public static String outputCompositeNodeToXml(CompositeNode cNode, SchemaContext schemaContext){
-    if (cNode == null) return new String();
-
-    //Document domTree = NodeUtils.buildShadowDomTree(cNode);
-    Document domTree = null;
-    try {
-      Set<RpcDefinition> rpcs =  schemaContext.getOperations();
-      for(RpcDefinition rpc : rpcs) {
-        if(rpc.getQName().equals(cNode.getNodeType())){
-          domTree = XmlDocumentUtils.toDocument(cNode, rpc.getInput(), XmlDocumentUtils.defaultValueCodecProvider());
-          break;
-        }
-      }
-
-    } catch (UnsupportedDataTypeException e) {
-      LOG.error("Error during translation of CompositeNode to Document", e);
-    }
-    return domTransformer(domTree);
-  }
-
-  private static String domTransformer(Document domTree) {
-    StringWriter writer = new StringWriter();
-    try {
-      TransformerFactory tf = TransformerFactory.newInstance();
-      Transformer transformer = tf.newTransformer();
-      transformer.setOutputProperty(OutputKeys.OMIT_XML_DECLARATION, "yes");
-      transformer.transform(new DOMSource(domTree), new StreamResult(writer));
-    } catch (TransformerException e) {
-
-      LOG.error("Error during translation of Document to OutputStream", e);
-    }
-    LOG.debug("compositeNodeToXml " + writer.toString());
-
-    return writer.toString();
-  }
-
-  public static CompositeNode xmlToCompositeNode(String xml){
-    if (xml==null || xml.length()==0) return null;
-
-    Node<?> dataTree;
-    try {
-      dataTree = XmlTreeBuilder.buildDataTree(new ByteArrayInputStream(xml.getBytes()));
-    } catch (XMLStreamException e) {
-      LOG.error("Error during building data tree from XML", e);
-      return null;
-    }
-    if (dataTree == null) {
-      LOG.error("data tree is null");
-      return null;
-    }
-    if (dataTree instanceof SimpleNode) {
-      LOG.error("RPC XML was resolved as SimpleNode");
-      return null;
-    }
-    return (CompositeNode) dataTree;
-  }
-
-  public static CompositeNode inputXmlToCompositeNode(QName rpc, String xml,  SchemaContext schemaContext){
-    if (xml==null || xml.length()==0) return null;
-
-    Node<?> dataTree = null;
-    try {
-
-      Document doc = XmlUtil.readXmlToDocument(xml);
-      LOG.debug("xmlToCompositeNode Document is " + xml );
-      Set<RpcDefinition> rpcs =  schemaContext.getOperations();
-      for(RpcDefinition rpcDef : rpcs) {
-        if(rpcDef.getQName().equals(rpc)){
-          dataTree = XmlDocumentUtils.toDomNode(doc.getDocumentElement(), Optional.<DataSchemaNode>of(rpcDef.getInput()), Optional.of(XmlDocumentUtils.defaultValueCodecProvider()));
-          break;
-        }
-      }
-    } catch (SAXException e) {
-      LOG.error("Error during building data tree from XML", e);
-    } catch (IOException e) {
-      LOG.error("Error during building data tree from XML", e);
-    }
-
-    LOG.debug("xmlToCompositeNode " + dataTree.toString());
-    return (CompositeNode) dataTree;
-  }
-}
index 25773bb8a69531b4b07f28755062671eaf4a8975..fd1af2b33ce6a9ddfc6a756644da9c51cb1e9c3d 100644 (file)
@@ -8,6 +8,7 @@
 
 package org.opendaylight.controller.remote.rpc.messages;
 
+import com.google.common.base.Preconditions;
 import org.opendaylight.controller.sal.connector.api.RpcRouter;
 
 import java.io.Serializable;
@@ -15,10 +16,13 @@ import java.util.Set;
 
 public class AddRoutedRpc implements Serializable {
 
-  Set<RpcRouter.RouteIdentifier<?, ?, ?>> announcements;
-  String actorPath;
+  private final Set<RpcRouter.RouteIdentifier<?, ?, ?>> announcements;
+  private final String actorPath;
+
+  public AddRoutedRpc(final Set<RpcRouter.RouteIdentifier<?, ?, ?>> announcements, final String actorPath) {
+    Preconditions.checkNotNull(announcements, "Route identifier should not be null");
+    Preconditions.checkNotNull(actorPath, "Actor path should not be null");
 
-  public AddRoutedRpc(Set<RpcRouter.RouteIdentifier<?, ?, ?>> announcements, String actorPath) {
     this.announcements = announcements;
     this.actorPath = actorPath;
   }
index eac973137e19592f577d3ce586d8345d18eff1ce..7eaa8f0618615ca1e0b1afc7963b737ad53ef9d8 100644 (file)
@@ -8,16 +8,20 @@
 
 package org.opendaylight.controller.remote.rpc.messages;
 
+import com.google.common.base.Preconditions;
 import org.opendaylight.controller.remote.rpc.RouteIdentifierImpl;
 
 import java.io.Serializable;
 
 public class AddRpc implements Serializable {
 
-  RouteIdentifierImpl routeId;
-  String actorPath;
+  private final RouteIdentifierImpl routeId;
+  private final String actorPath;
+
+  public AddRpc(final RouteIdentifierImpl routeId, final String actorPath) {
+    Preconditions.checkNotNull(routeId, "Route identifier should not be null");
+    Preconditions.checkNotNull(actorPath, "Actor path should not be null");
 
-  public AddRpc(RouteIdentifierImpl routeId, String actorPath) {
     this.routeId = routeId;
     this.actorPath = actorPath;
   }
index ef3f5281122116113e1064de06e98ab6842f9782..2c26243fe990659708961c38b1bc8e1a9c828409 100644 (file)
@@ -7,13 +7,16 @@
  */
 package org.opendaylight.controller.remote.rpc.messages;
 
+import com.google.common.base.Preconditions;
+
 import java.io.Serializable;
 
 public class ErrorResponse implements Serializable {
 
-  Exception exception;
+  private final Exception exception;
 
-  public ErrorResponse(Exception e) {
+  public ErrorResponse(final Exception e) {
+    Preconditions.checkNotNull(e, "Exception should be present for error message");
     this.exception = e;
   }
 
index 030d81ac7edd1293709af83ebcf4a8f07d8c6cc2..522dd44f5bf934d95b91d40f1b5656e80f703b8e 100644 (file)
@@ -8,16 +8,20 @@
 package org.opendaylight.controller.remote.rpc.messages;
 
 
+import com.google.common.base.Preconditions;
 import org.opendaylight.yangtools.yang.common.QName;
 
 import java.io.Serializable;
 
 public class ExecuteRpc implements Serializable {
 
-  private String inputCompositeNode;
-  private QName rpc;
+  private final String inputCompositeNode;
+  private final QName rpc;
+
+  public ExecuteRpc(final String inputCompositeNode, final QName rpc) {
+    Preconditions.checkNotNull(inputCompositeNode, "Composite Node input string should be present");
+    Preconditions.checkNotNull(rpc, "rpc Qname should not be null");
 
-  public ExecuteRpc(String inputCompositeNode, QName rpc) {
     this.inputCompositeNode = inputCompositeNode;
     this.rpc = rpc;
   }
index b1fa410e8d984e4b3449c849e9c4d3eb207c05a6..e8d226218226c68c15550dd642b27c2ddc1568de 100644 (file)
@@ -9,15 +9,17 @@
 package org.opendaylight.controller.remote.rpc.messages;
 
 
+import com.google.common.base.Preconditions;
 import org.opendaylight.controller.remote.rpc.RouteIdentifierImpl;
 
 import java.io.Serializable;
 
 public class GetRoutedRpc implements Serializable {
 
-  RouteIdentifierImpl routeId;
+  private final RouteIdentifierImpl routeId;
 
-  public GetRoutedRpc(RouteIdentifierImpl routeId) {
+  public GetRoutedRpc(final RouteIdentifierImpl routeId) {
+    Preconditions.checkNotNull(routeId, "route id should not be null");
     this.routeId = routeId;
   }
 
index 0e1563340bf44ad8f781badcc438d79ac26a05f5..d42666219289688af8075d8922b0f4bf1b4e23e9 100644 (file)
@@ -12,9 +12,9 @@ import java.io.Serializable;
 
 public class GetRoutedRpcReply implements Serializable {
 
-  private String routePath;
+  private final String routePath;
 
-  public GetRoutedRpcReply(String routePath) {
+  public GetRoutedRpcReply(final String routePath) {
     this.routePath = routePath;
   }
 
index c55627961a77683fa669ab3a09a8948886a5a276..c1d4240dca8544758adfd93c4eccb034abb02cb2 100644 (file)
@@ -7,15 +7,17 @@
  */
 package org.opendaylight.controller.remote.rpc.messages;
 
+import com.google.common.base.Preconditions;
 import org.opendaylight.controller.remote.rpc.RouteIdentifierImpl;
 
 import java.io.Serializable;
 
 public class GetRpc implements Serializable {
 
-  RouteIdentifierImpl routeId;
+  private final RouteIdentifierImpl routeId;
 
-  public GetRpc(RouteIdentifierImpl routeId) {
+  public GetRpc(final RouteIdentifierImpl routeId) {
+    Preconditions.checkNotNull(routeId, "Route Id should not be null");
     this.routeId = routeId;
   }
 
index 3309b989c54b614b0d288aa013e388ea6a3db107..aaf089d16f2e1e85e3ca060e5214e3fc927e3993 100644 (file)
@@ -12,9 +12,9 @@ import java.io.Serializable;
 
 public class GetRpcReply implements Serializable {
 
-  private String routePath;
+  private final String routePath;
 
-  public GetRpcReply(String routePath) {
+  public GetRpcReply(final String routePath) {
     this.routePath = routePath;
   }
 
index 00ef980bb18829618dcb857ce0d1b6150ff85d63..fd73144e1d9f68378320b1c3bec347ea4f7f90fe 100644 (file)
@@ -7,26 +7,26 @@
  */
 package org.opendaylight.controller.remote.rpc.messages;
 
+import com.google.common.base.Preconditions;
 import org.opendaylight.yangtools.yang.common.QName;
 import org.opendaylight.yangtools.yang.data.api.CompositeNode;
-import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 
 import java.io.Serializable;
 
 public class InvokeRoutedRpc implements Serializable {
 
-  private QName rpc;
-  private InstanceIdentifier identifier;
-  private CompositeNode input;
+  private final QName rpc;
+  private final YangInstanceIdentifier identifier;
+  private final CompositeNode input;
 
-  public InvokeRoutedRpc(QName rpc, InstanceIdentifier identifier, CompositeNode input) {
-    this.rpc = rpc;
-    this.identifier = identifier;
-    this.input = input;
-  }
+  public InvokeRoutedRpc(final QName rpc, final YangInstanceIdentifier identifier, final CompositeNode input) {
+    Preconditions.checkNotNull(rpc, "rpc qname should not be null");
+    Preconditions.checkNotNull(identifier, "instance identifier of routed rpc should not be null");
+    Preconditions.checkNotNull(input, "rpc input should not be null");
 
-  public InvokeRoutedRpc(QName rpc, CompositeNode input) {
     this.rpc = rpc;
+    this.identifier = identifier;
     this.input = input;
   }
 
@@ -34,7 +34,7 @@ public class InvokeRoutedRpc implements Serializable {
     return rpc;
   }
 
-  public InstanceIdentifier getIdentifier() {
+  public YangInstanceIdentifier getIdentifier() {
     return identifier;
   }
 
index 1f4eab0971b79452f7997da0770425639b4232bf..94b7fe40244838f6115469502af862cde4daa7cc 100644 (file)
@@ -7,6 +7,7 @@
  */
 package org.opendaylight.controller.remote.rpc.messages;
 
+import com.google.common.base.Preconditions;
 import org.opendaylight.yangtools.yang.common.QName;
 import org.opendaylight.yangtools.yang.data.api.CompositeNode;
 
@@ -14,10 +15,13 @@ import java.io.Serializable;
 
 public class InvokeRpc implements Serializable {
 
-  private QName rpc;
-  private CompositeNode input;
+  private final QName rpc;
+  private final CompositeNode input;
+
+  public InvokeRpc(final QName rpc, final CompositeNode input) {
+    Preconditions.checkNotNull(rpc, "rpc qname should not be null");
+    Preconditions.checkNotNull(input, "rpc input should not be null");
 
-  public InvokeRpc(QName rpc, CompositeNode input) {
     this.rpc = rpc;
     this.input = input;
   }
index a3aa9d12fa97e206c480c2fc0651fc9216aba60f..b560b8c8c064c9bb9c17d06121bd0e4e67b5fd70 100644 (file)
@@ -8,6 +8,7 @@
 
 package org.opendaylight.controller.remote.rpc.messages;
 
+import com.google.common.base.Preconditions;
 import org.opendaylight.controller.sal.connector.api.RpcRouter;
 
 import java.io.Serializable;
@@ -15,10 +16,13 @@ import java.util.Set;
 
 public class RemoveRoutedRpc implements Serializable {
 
-  Set<RpcRouter.RouteIdentifier<?, ?, ?>> announcements;
-  String actorPath;
+  private final Set<RpcRouter.RouteIdentifier<?, ?, ?>> announcements;
+  private final String actorPath;
+
+  public RemoveRoutedRpc(final Set<RpcRouter.RouteIdentifier<?, ?, ?>> announcements, final String actorPath) {
+    Preconditions.checkNotNull(announcements, "Route identifier should not be null");
+    Preconditions.checkNotNull(actorPath, "Actor path should not be null");
 
-  public RemoveRoutedRpc(Set<RpcRouter.RouteIdentifier<?, ?, ?>> announcements, String actorPath) {
     this.announcements = announcements;
     this.actorPath = actorPath;
   }
index 0bfd78aaae8e724eadaa8c07407a5abbcf1d7609..289334fccc9af3031fc64d9622336d6344fa411a 100644 (file)
@@ -8,15 +8,18 @@
 
 package org.opendaylight.controller.remote.rpc.messages;
 
+import com.google.common.base.Preconditions;
 import org.opendaylight.controller.remote.rpc.RouteIdentifierImpl;
 
 import java.io.Serializable;
 
 public class RemoveRpc implements Serializable {
 
-  RouteIdentifierImpl routeId;
+  private final RouteIdentifierImpl routeId;
+
+  public RemoveRpc(final RouteIdentifierImpl routeId) {
+    Preconditions.checkNotNull(routeId, "Route Id should not be null");
 
-  public RemoveRpc(RouteIdentifierImpl routeId) {
     this.routeId = routeId;
   }
 
index 132fdba0545c95b0dcbf4ea485c1a8662bb1a781..c57a258426ec54992b158ffa37ccb77288022547 100644 (file)
@@ -15,11 +15,11 @@ import java.util.LinkedHashSet;
 import java.util.Map;
 
 public class RoutingTableData implements Serializable {
-  private Map<RpcRouter.RouteIdentifier<?, ?, ?>, String> rpcMap;
-  private Map<RpcRouter.RouteIdentifier<?, ?, ?>, LinkedHashSet<String>> routedRpcMap;
+  private final Map<RpcRouter.RouteIdentifier<?, ?, ?>, String> rpcMap;
+  private final Map<RpcRouter.RouteIdentifier<?, ?, ?>, LinkedHashSet<String>> routedRpcMap;
 
-  public RoutingTableData(Map<RpcRouter.RouteIdentifier<?, ?, ?>, String> rpcMap,
-                          Map<RpcRouter.RouteIdentifier<?, ?, ?>, LinkedHashSet<String>> routedRpcMap) {
+  public RoutingTableData(final Map<RpcRouter.RouteIdentifier<?, ?, ?>, String> rpcMap,
+                          final Map<RpcRouter.RouteIdentifier<?, ?, ?>, LinkedHashSet<String>> routedRpcMap) {
     this.rpcMap = rpcMap;
     this.routedRpcMap = routedRpcMap;
   }
index cbfecb1918abc4849a3606efa84f780399b16411..17766f15b9af7c22112852c16686c02684155924 100644 (file)
@@ -12,9 +12,9 @@ package org.opendaylight.controller.remote.rpc.messages;
 import java.io.Serializable;
 
 public class RpcResponse implements Serializable {
-  private String resultCompositeNode;
+  private final String resultCompositeNode;
 
-  public RpcResponse(String resultCompositeNode) {
+  public RpcResponse(final String resultCompositeNode) {
     this.resultCompositeNode = resultCompositeNode;
   }
 
diff --git a/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/UpdateSchemaContext.java b/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/messages/UpdateSchemaContext.java
new file mode 100644 (file)
index 0000000..83fc772
--- /dev/null
@@ -0,0 +1,24 @@
+/*
+ * 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.remote.rpc.messages;
+
+import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+
+public class UpdateSchemaContext {
+
+  private final SchemaContext schemaContext;
+
+  public UpdateSchemaContext(final SchemaContext schemaContext) {
+    this.schemaContext = schemaContext;
+  }
+
+  public SchemaContext getSchemaContext() {
+    return schemaContext;
+  }
+}
diff --git a/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/InstanceIdentifierForXmlCodec.java b/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/InstanceIdentifierForXmlCodec.java
new file mode 100644 (file)
index 0000000..92a7fba
--- /dev/null
@@ -0,0 +1,219 @@
+/*
+ * 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.remote.rpc.utils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
+import org.opendaylight.yangtools.yang.common.QName;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument;
+import org.opendaylight.yangtools.yang.model.api.Module;
+import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Element;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public final class InstanceIdentifierForXmlCodec {
+  private static final Pattern PREDICATE_PATTERN = Pattern.compile("\\[(.*?)\\]");
+  private static final Splitter SLASH_SPLITTER = Splitter.on('/');
+  private static final Splitter COLON_SPLITTER = Splitter.on(':');
+  private static final Splitter AT_SPLITTER = Splitter.on('@');
+  private static final Logger logger = LoggerFactory.getLogger(InstanceIdentifierForXmlCodec.class);
+
+  private InstanceIdentifierForXmlCodec() {
+    throw new UnsupportedOperationException("Utility class");
+  }
+
+  public static YangInstanceIdentifier deserialize(final Element element, final SchemaContext schemaContext) {
+    Preconditions.checkNotNull(element, "Value of element for deserialization can't be null");
+    Preconditions.checkNotNull(schemaContext,
+        "Schema context for deserialization of instance identifier type can't be null");
+
+    final String valueTrimmed = element.getTextContent().trim();
+    logger.debug("Instance identifier derserialize: splitting the text {} with Slash to find path arguments", valueTrimmed);
+    final Iterator<String> xPathParts = SLASH_SPLITTER.split(valueTrimmed).iterator();
+
+    // must be at least "/pr:node"
+    if (!xPathParts.hasNext() || !xPathParts.next().isEmpty() || !xPathParts.hasNext()) {
+      logger.debug("Instance identifier derserialize: No path argument found for element.");
+      return null;
+    }
+
+    List<PathArgument> result = new ArrayList<>();
+    while (xPathParts.hasNext()) {
+      String xPathPartTrimmed = xPathParts.next().trim();
+
+      PathArgument pathArgument = toPathArgument(xPathPartTrimmed, element, schemaContext);
+      if (pathArgument != null) {
+        result.add(pathArgument);
+      }
+    }
+    return YangInstanceIdentifier.create(result);
+  }
+
+  public static Element serialize(final YangInstanceIdentifier id, final Element element) {
+    Preconditions.checkNotNull(id, "Variable should contain instance of instance identifier and can't be null");
+    Preconditions.checkNotNull(element, "DOM element can't be null");
+
+    final RandomPrefix prefixes = new RandomPrefix();
+    final String str = XmlUtils.encodeIdentifier(prefixes, id);
+
+    for (Entry<URI, String> e: prefixes.getPrefixes()) {
+      element.setAttribute("xmlns:" + e.getValue(), e.getKey().toString());
+    }
+    element.setTextContent(str);
+    return element;
+  }
+
+  private static String getIdAndPrefixAsStr(final String pathPart) {
+    int predicateStartIndex = pathPart.indexOf('[');
+    return predicateStartIndex == -1 ? pathPart : pathPart.substring(0, predicateStartIndex);
+  }
+
+  private static PathArgument toPathArgument(final String xPathArgument, final Element element, final SchemaContext schemaContext) {
+    final QName mainQName = toIdentity(xPathArgument, element, schemaContext);
+
+    // predicates
+    final Matcher matcher = PREDICATE_PATTERN.matcher(xPathArgument);
+    final Map<QName, Object> predicates = new HashMap<>();
+    QName currentQName = mainQName;
+
+    while (matcher.find()) {
+      final String predicateStr = matcher.group(1).trim();
+      final int indexOfEqualityMark = predicateStr.indexOf('=');
+      if (indexOfEqualityMark != -1) {
+        final Object predicateValue = toPredicateValue(predicateStr.substring(indexOfEqualityMark + 1));
+        if (predicateValue == null) {
+          return null;
+        }
+
+        if (predicateStr.charAt(0) != '.') {
+          // target is not a leaf-list
+          currentQName = toIdentity(predicateStr.substring(0, indexOfEqualityMark), element, schemaContext);
+          if (currentQName == null) {
+            return null;
+          }
+        }
+        logger.debug("Instance identifier derserialize: finding predicates of node {}", predicateValue);
+        predicates.put(currentQName, predicateValue);
+      }
+    }
+
+    if (predicates.isEmpty()) {
+      return new YangInstanceIdentifier.NodeIdentifier(mainQName);
+    } else {
+      return new YangInstanceIdentifier.NodeIdentifierWithPredicates(mainQName, predicates);
+    }
+
+  }
+
+  public static QName toIdentity(final String xPathArgument, final Element element, final SchemaContext schemaContext) {
+    final String xPathPartTrimmed = getIdAndPrefixAsStr(xPathArgument).trim();
+    final Iterator<String> it = COLON_SPLITTER.split(xPathPartTrimmed).iterator();
+
+    // Empty string
+    if (!it.hasNext()) {
+      return null;
+    }
+
+    final String prefix = it.next().trim();
+    if (prefix.isEmpty()) {
+      return null;
+    }
+
+    // it is not "prefix:value"
+    if (!it.hasNext()) {
+      return null;
+    }
+
+    final String identifier = it.next().trim();
+    if (identifier.isEmpty()) {
+      return null;
+    }
+
+    URI namespace = null;
+    String namespaceStr = null;
+    try {
+      namespaceStr = element.getAttribute("xmlns:"+prefix);
+      namespace = new URI(namespaceStr);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException("It wasn't possible to convert " + namespaceStr + " to URI object.");
+    } catch (NullPointerException e) {
+      throw new IllegalArgumentException("I wasn't possible to get namespace for prefix " + prefix);
+    }
+
+    Module module = schemaContext.findModuleByNamespaceAndRevision(namespace, null);
+    return QName.create(module.getQNameModule(), identifier);
+  }
+
+  private static String trimIfEndIs(final String str, final char end) {
+    final int l = str.length() - 1;
+    if (str.charAt(l) != end) {
+      return null;
+    }
+
+    return str.substring(1, l);
+  }
+
+  private static Object toPredicateValue(final String predicatedValue) {
+    logger.debug("Instance identifier derserialize: converting the predicate vstring to object {}", predicatedValue);
+    final String predicatedValueTrimmed = predicatedValue.trim();
+    if (predicatedValue.isEmpty()) {
+      return null;
+    }
+    String updatedValue = null;
+    switch (predicatedValueTrimmed.charAt(0)) {
+      case '"':
+        updatedValue =  trimIfEndIs(predicatedValueTrimmed, '"');
+        break;
+      case '\'':
+        updatedValue =  trimIfEndIs(predicatedValueTrimmed, '\'');
+        break;
+      default:
+        updatedValue =  predicatedValueTrimmed;
+    }
+    Iterator<String> it = AT_SPLITTER.split(updatedValue).iterator();
+    // Empty string
+    if (!it.hasNext()) {
+      return null;
+    }
+
+    final String value = it.next().trim();
+    if (value.isEmpty()) {
+      return null;
+    }
+
+    if (!it.hasNext()) {
+      return value;
+    }
+
+    final String type = it.next().trim();
+    if (type.isEmpty()) {
+      return value;
+    }
+    Object predicateObject = null;
+    try {
+      logger.debug("Instance identifier derserialize: converting the predicate value {{}}to correct object type {{}}", value, type);
+      predicateObject = Class.forName(type).getConstructor(String.class).newInstance(value);
+    } catch (Exception e) {
+      logger.error("Could not convert to valid type of value", e);
+    }
+    return predicateObject;
+  }
+}
diff --git a/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/RandomPrefix.java b/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/RandomPrefix.java
new file mode 100644 (file)
index 0000000..55cc819
--- /dev/null
@@ -0,0 +1,50 @@
+/*
+ * 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.remote.rpc.utils;
+
+import org.opendaylight.yangtools.yang.common.QName;
+
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * Picked from Yang tools because it was not public class
+ */
+
+final class RandomPrefix {
+    final Map<URI, String> prefixes = new HashMap<>();
+
+    Iterable<Entry<URI, String>> getPrefixes() {
+        return prefixes.entrySet();
+    }
+
+    String encodeQName(final QName qname) {
+        String prefix = prefixes.get(qname.getNamespace());
+        if (prefix == null) {
+            prefix = qname.getPrefix();
+            if (prefix == null || prefix.isEmpty() || prefixes.containsValue(prefix)) {
+                final ThreadLocalRandom random = ThreadLocalRandom.current();
+                do {
+                    final StringBuilder sb = new StringBuilder();
+                    for (int i = 0; i < 4; i++) {
+                        sb.append((char)('a' + random.nextInt(25)));
+                    }
+
+                    prefix = sb.toString();
+                } while (prefixes.containsValue(prefix));
+            }
+
+            prefixes.put(qname.getNamespace(), prefix);
+        }
+
+        return prefix + ':' + qname.getLocalName();
+    }
+}
diff --git a/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/XmlDocumentUtils.java b/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/XmlDocumentUtils.java
new file mode 100644 (file)
index 0000000..127aa07
--- /dev/null
@@ -0,0 +1,311 @@
+/*
+ * Copyright (c) 2013 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.remote.rpc.utils;
+
+import com.google.common.base.Function;
+import com.google.common.base.Objects;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import org.opendaylight.yangtools.yang.common.QName;
+import org.opendaylight.yangtools.yang.data.api.CompositeNode;
+import org.opendaylight.yangtools.yang.data.api.ModifyAction;
+import org.opendaylight.yangtools.yang.data.api.Node;
+import org.opendaylight.yangtools.yang.data.impl.ImmutableCompositeNode;
+import org.opendaylight.yangtools.yang.data.impl.SimpleNodeTOImpl;
+import org.opendaylight.yangtools.yang.data.impl.codec.TypeDefinitionAwareCodec;
+import org.opendaylight.yangtools.yang.data.impl.codec.xml.XmlCodecProvider;
+import org.opendaylight.yangtools.yang.model.api.ChoiceCaseNode;
+import org.opendaylight.yangtools.yang.model.api.ChoiceNode;
+import org.opendaylight.yangtools.yang.model.api.ContainerSchemaNode;
+import org.opendaylight.yangtools.yang.model.api.DataNodeContainer;
+import org.opendaylight.yangtools.yang.model.api.DataSchemaNode;
+import org.opendaylight.yangtools.yang.model.api.LeafListSchemaNode;
+import org.opendaylight.yangtools.yang.model.api.LeafSchemaNode;
+import org.opendaylight.yangtools.yang.model.api.ListSchemaNode;
+import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.opendaylight.yangtools.yang.model.api.SchemaNode;
+import org.opendaylight.yangtools.yang.model.api.TypeDefinition;
+import org.opendaylight.yangtools.yang.model.api.type.IdentityrefTypeDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Attr;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.NodeList;
+
+import javax.activation.UnsupportedDataTypeException;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamWriter;
+import javax.xml.transform.dom.DOMResult;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static com.google.common.base.Preconditions.checkState;
+
+public class XmlDocumentUtils {
+  private static class ElementWithSchemaContext {
+    Element element;
+    SchemaContext schemaContext;
+
+    ElementWithSchemaContext(final Element element,final SchemaContext schemaContext) {
+      this.schemaContext = schemaContext;
+      this.element = element;
+    }
+
+    Element getElement() {
+      return element;
+    }
+
+    SchemaContext getSchemaContext() {
+      return schemaContext;
+    }
+  }
+
+  public static final QName OPERATION_ATTRIBUTE_QNAME = QName.create(URI.create("urn:ietf:params:xml:ns:netconf:base:1.0"), null, "operation");
+  private static final Logger logger = LoggerFactory.getLogger(XmlDocumentUtils.class);
+  private static final XMLOutputFactory FACTORY = XMLOutputFactory.newFactory();
+
+  /**
+   * Converts Data DOM structure to XML Document for specified XML Codec Provider and corresponding
+   * Data Node Container schema. The CompositeNode data parameter enters as root of Data DOM tree and will
+   * be transformed to root in XML Document. Each element of Data DOM tree is compared against specified Data
+   * Node Container Schema and transformed accordingly.
+   *
+   * @param data Data DOM root element
+   * @param schema Data Node Container Schema
+   * @param codecProvider XML Codec Provider
+   * @return new instance of XML Document
+   * @throws javax.activation.UnsupportedDataTypeException
+   */
+  public static Document toDocument(final CompositeNode data, final DataNodeContainer schema, final XmlCodecProvider codecProvider)
+      throws UnsupportedDataTypeException {
+    Preconditions.checkNotNull(data);
+    Preconditions.checkNotNull(schema);
+
+    if (!(schema instanceof ContainerSchemaNode || schema instanceof ListSchemaNode)) {
+      throw new UnsupportedDataTypeException("Schema can be ContainerSchemaNode or ListSchemaNode. Other types are not supported yet.");
+    }
+
+    final DOMResult result = new DOMResult(getDocument());
+    try {
+      final XMLStreamWriter writer = FACTORY.createXMLStreamWriter(result);
+      XmlStreamUtils.create(codecProvider).writeDocument(writer, data, (SchemaNode)schema);
+      writer.close();
+      return (Document)result.getNode();
+    } catch (XMLStreamException e) {
+      logger.error("Failed to serialize data {}", data, e);
+      return null;
+    }
+  }
+
+  public static Document getDocument() {
+    DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+    Document doc = null;
+    try {
+      DocumentBuilder bob = dbf.newDocumentBuilder();
+      doc = bob.newDocument();
+    } catch (ParserConfigurationException e) {
+      throw new RuntimeException(e);
+    }
+    return doc;
+  }
+
+
+  public static QName qNameFromElement(final Element xmlElement) {
+    String namespace = xmlElement.getNamespaceURI();
+    String localName = xmlElement.getLocalName();
+    return QName.create(namespace != null ? URI.create(namespace) : null, null, localName);
+  }
+
+  private static Node<?> toNodeWithSchema(final Element xmlElement, final DataSchemaNode schema, final XmlCodecProvider codecProvider,final SchemaContext schemaCtx) {
+    checkQName(xmlElement, schema.getQName());
+    if (schema instanceof DataNodeContainer) {
+      return toCompositeNodeWithSchema(xmlElement, schema.getQName(), (DataNodeContainer) schema, schemaCtx);
+    } else if (schema instanceof LeafSchemaNode) {
+      return toSimpleNodeWithType(xmlElement, (LeafSchemaNode) schema, codecProvider,schemaCtx);
+    } else if (schema instanceof LeafListSchemaNode) {
+      return toSimpleNodeWithType(xmlElement, (LeafListSchemaNode) schema, codecProvider,schemaCtx);
+    }
+    return null;
+  }
+
+
+
+  private static Node<?> toSimpleNodeWithType(final Element xmlElement, final LeafSchemaNode schema,
+                                              final XmlCodecProvider codecProvider,final SchemaContext schemaCtx) {
+    TypeDefinitionAwareCodec<? extends Object, ? extends TypeDefinition<?>> codec = codecProvider.codecFor(schema.getType());
+    String text = xmlElement.getTextContent();
+    Object value = null;
+    if (codec != null) {
+      logger.debug("toSimpleNodeWithType: found codec, deserializing text {}", text);
+      value = codec.deserialize(text);
+    }
+
+    final TypeDefinition<?> baseType = XmlUtils.resolveBaseTypeFrom(schema.getType());
+    if (baseType instanceof org.opendaylight.yangtools.yang.model.util.InstanceIdentifier) {
+      logger.debug("toSimpleNodeWithType: base type of node is instance identifier, deserializing element", xmlElement);
+      value = InstanceIdentifierForXmlCodec.deserialize(xmlElement,schemaCtx);
+
+    } else if(baseType instanceof IdentityrefTypeDefinition){
+      logger.debug("toSimpleNodeWithType: base type of node is IdentityrefTypeDefinition, deserializing element", xmlElement);
+      value = InstanceIdentifierForXmlCodec.toIdentity(xmlElement.getTextContent(), xmlElement, schemaCtx);
+
+    }
+
+    if (value == null) {
+      logger.debug("toSimpleNodeWithType: no type found for element, returning just the text string value of element {}", xmlElement);
+      value = xmlElement.getTextContent();
+    }
+
+    Optional<ModifyAction> modifyAction = getModifyOperationFromAttributes(xmlElement);
+    return new SimpleNodeTOImpl<>(schema.getQName(), null, value, modifyAction.orNull());
+  }
+
+  private static Node<?> toSimpleNodeWithType(final Element xmlElement, final LeafListSchemaNode schema,
+                                              final XmlCodecProvider codecProvider,final SchemaContext schemaCtx) {
+    TypeDefinitionAwareCodec<? extends Object, ? extends TypeDefinition<?>> codec = codecProvider.codecFor(schema.getType());
+    String text = xmlElement.getTextContent();
+    Object value = null;
+    if (codec != null) {
+      logger.debug("toSimpleNodeWithType: found codec, deserializing text {}", text);
+      value = codec.deserialize(text);
+    }
+
+    if (schema.getType() instanceof org.opendaylight.yangtools.yang.model.util.InstanceIdentifier) {
+      logger.debug("toSimpleNodeWithType: base type of node is instance identifier, deserializing element", xmlElement);
+      value = InstanceIdentifierForXmlCodec.deserialize(xmlElement,schemaCtx);
+    }
+
+    if (value == null) {
+      logger.debug("toSimpleNodeWithType: no type found for element, returning just the text string value of element {}", xmlElement);
+      value = xmlElement.getTextContent();
+    }
+
+    Optional<ModifyAction> modifyAction = getModifyOperationFromAttributes(xmlElement);
+    return new SimpleNodeTOImpl<>(schema.getQName(), null, value, modifyAction.orNull());
+  }
+
+  private static Node<?> toCompositeNodeWithSchema(final Element xmlElement, final QName qName, final DataNodeContainer schema,
+                                                   final SchemaContext schemaCtx) {
+    List<Node<?>> values = toDomNodes(xmlElement, Optional.fromNullable(schema.getChildNodes()),schemaCtx);
+    Optional<ModifyAction> modifyAction = getModifyOperationFromAttributes(xmlElement);
+    return ImmutableCompositeNode.create(qName, values, modifyAction.orNull());
+  }
+
+  private static Optional<ModifyAction> getModifyOperationFromAttributes(final Element xmlElement) {
+    Attr attributeNodeNS = xmlElement.getAttributeNodeNS(OPERATION_ATTRIBUTE_QNAME.getNamespace().toString(), OPERATION_ATTRIBUTE_QNAME.getLocalName());
+    if(attributeNodeNS == null) {
+      return Optional.absent();
+    }
+
+    ModifyAction action = ModifyAction.fromXmlValue(attributeNodeNS.getValue());
+    Preconditions.checkArgument(action.isOnElementPermitted(), "Unexpected operation %s on %s", action, xmlElement);
+
+    return Optional.of(action);
+  }
+
+  private static void checkQName(final Element xmlElement, final QName qName) {
+    checkState(Objects.equal(xmlElement.getNamespaceURI(), qName.getNamespace().toString()));
+    checkState(qName.getLocalName().equals(xmlElement.getLocalName()));
+  }
+
+  public static final Optional<DataSchemaNode> findFirstSchema(final QName qname, final Collection<DataSchemaNode> dataSchemaNode) {
+    if (dataSchemaNode != null && !dataSchemaNode.isEmpty() && qname != null) {
+      for (DataSchemaNode dsn : dataSchemaNode) {
+        if (qname.isEqualWithoutRevision(dsn.getQName())) {
+          return Optional.<DataSchemaNode> of(dsn);
+        } else if (dsn instanceof ChoiceNode) {
+          for (ChoiceCaseNode choiceCase : ((ChoiceNode) dsn).getCases()) {
+            Optional<DataSchemaNode> foundDsn = findFirstSchema(qname, choiceCase.getChildNodes());
+            if (foundDsn != null && foundDsn.isPresent()) {
+              return foundDsn;
+            }
+          }
+        }
+      }
+    }
+    return Optional.absent();
+  }
+
+  private static Node<?> toDomNode(Element element) {
+    QName qname = qNameFromElement(element);
+
+    ImmutableList.Builder<Node<?>> values = ImmutableList.<Node<?>> builder();
+    NodeList nodes = element.getChildNodes();
+    boolean isSimpleObject = true;
+    String value = null;
+    for (int i = 0; i < nodes.getLength(); i++) {
+      org.w3c.dom.Node child = nodes.item(i);
+      if (child instanceof Element) {
+        isSimpleObject = false;
+        values.add(toDomNode((Element) child));
+      }
+      if (isSimpleObject && child instanceof org.w3c.dom.Text) {
+        value = element.getTextContent();
+        if (!Strings.isNullOrEmpty(value)) {
+          isSimpleObject = true;
+        }
+      }
+    }
+    if (isSimpleObject) {
+      return new SimpleNodeTOImpl<>(qname, null, value);
+    }
+    return ImmutableCompositeNode.create(qname, values.build());
+  }
+
+  public static List<Node<?>> toDomNodes(final Element element, final Optional<Collection<DataSchemaNode>> context,SchemaContext schemaCtx) {
+    return forEachChild(element.getChildNodes(),schemaCtx, new Function<ElementWithSchemaContext, Optional<Node<?>>>() {
+
+      @Override
+      public Optional<Node<?>> apply(ElementWithSchemaContext input) {
+        if (context.isPresent()) {
+          QName partialQName = qNameFromElement(input.getElement());
+          Optional<DataSchemaNode> schemaNode = XmlDocumentUtils.findFirstSchema(partialQName, context.get());
+          if (schemaNode.isPresent()) {
+            return Optional.<Node<?>> fromNullable(//
+                toNodeWithSchema(input.getElement(), schemaNode.get(), XmlDocumentUtils.defaultValueCodecProvider(),input.getSchemaContext()));
+          }
+        }
+        return Optional.<Node<?>> fromNullable(toDomNode(input.getElement()));
+      }
+
+    });
+
+  }
+
+  private static final <T> List<T> forEachChild(final NodeList nodes, final SchemaContext schemaContext, final Function<ElementWithSchemaContext, Optional<T>> forBody) {
+    final int l = nodes.getLength();
+    if (l == 0) {
+      return ImmutableList.of();
+    }
+
+    final List<T> list = new ArrayList<>(l);
+    for (int i = 0; i < l; i++) {
+      org.w3c.dom.Node child = nodes.item(i);
+      if (child instanceof Element) {
+        Optional<T> result = forBody.apply(new ElementWithSchemaContext((Element) child,schemaContext));
+        if (result.isPresent()) {
+          list.add(result.get());
+        }
+      }
+    }
+    return ImmutableList.copyOf(list);
+  }
+
+  public static final XmlCodecProvider defaultValueCodecProvider() {
+    return XmlUtils.DEFAULT_XML_CODEC_PROVIDER;
+  }
+}
diff --git a/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/XmlStreamUtils.java b/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/XmlStreamUtils.java
new file mode 100644 (file)
index 0000000..e4576c4
--- /dev/null
@@ -0,0 +1,249 @@
+/*
+ * 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.remote.rpc.utils;
+
+import com.google.common.annotations.Beta;
+import com.google.common.base.Preconditions;
+import org.opendaylight.yangtools.yang.common.QName;
+import org.opendaylight.yangtools.yang.data.api.AttributesContainer;
+import org.opendaylight.yangtools.yang.data.api.CompositeNode;
+import org.opendaylight.yangtools.yang.data.api.Node;
+import org.opendaylight.yangtools.yang.data.api.SimpleNode;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.impl.codec.TypeDefinitionAwareCodec;
+import org.opendaylight.yangtools.yang.data.impl.codec.xml.XmlCodecProvider;
+import org.opendaylight.yangtools.yang.data.impl.schema.SchemaUtils;
+import org.opendaylight.yangtools.yang.model.api.DataNodeContainer;
+import org.opendaylight.yangtools.yang.model.api.DataSchemaNode;
+import org.opendaylight.yangtools.yang.model.api.LeafListSchemaNode;
+import org.opendaylight.yangtools.yang.model.api.LeafSchemaNode;
+import org.opendaylight.yangtools.yang.model.api.SchemaNode;
+import org.opendaylight.yangtools.yang.model.api.TypeDefinition;
+import org.opendaylight.yangtools.yang.model.api.type.IdentityrefTypeDefinition;
+import org.opendaylight.yangtools.yang.model.api.type.InstanceIdentifierTypeDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamWriter;
+import java.net.URI;
+import java.util.Map.Entry;
+
+/**
+ * Utility class for bridging JAXP Stream and YANG Data APIs. Note that the definition of this class
+ * by no means final and subject to change as more functionality is centralized here.
+ */
+@Beta
+public class XmlStreamUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(XmlStreamUtils.class);
+  private final XmlCodecProvider codecProvider;
+
+  protected XmlStreamUtils(final XmlCodecProvider codecProvider) {
+    this.codecProvider = Preconditions.checkNotNull(codecProvider);
+  }
+
+  /**
+   * Create a new instance encapsulating a particular codec provider.
+   *
+   * @param codecProvider XML codec provider
+   * @return A new instance
+   */
+  public static XmlStreamUtils create(final XmlCodecProvider codecProvider) {
+    return new XmlStreamUtils(codecProvider);
+  }
+
+  /**
+   * Check if a particular data element can be emitted as an empty element, bypassing value encoding. This
+   * functionality is optional, as valid XML stream is produced even if start/end element is produced unconditionally.
+   *
+   * @param data Data node
+   * @return True if the data node will result in empty element body.
+   */
+  public static boolean isEmptyElement(final Node<?> data) {
+    if (data == null) {
+      return true;
+    }
+
+    if (data instanceof CompositeNode) {
+      return ((CompositeNode) data).getValue().isEmpty();
+    }
+    if (data instanceof SimpleNode) {
+      return data.getValue() == null;
+    }
+
+    // Safe default
+    return false;
+  }
+
+  /**
+   * Write an InstanceIdentifier into the output stream. Calling corresponding {@link javax.xml.stream.XMLStreamWriter#writeStartElement(String)}
+   * and {@link javax.xml.stream.XMLStreamWriter#writeEndElement()} is the responsibility of the caller.
+   *
+   * @param writer XML Stream writer
+   * @param id InstanceIdentifier
+   * @throws javax.xml.stream.XMLStreamException
+   */
+  public static void write(final @Nonnull XMLStreamWriter writer, final @Nonnull YangInstanceIdentifier id) throws XMLStreamException {
+    Preconditions.checkNotNull(writer, "Writer may not be null");
+    Preconditions.checkNotNull(id, "Variable should contain instance of instance identifier and can't be null");
+    LOG.debug("Writing Instance identifier with Random prefix");
+    final RandomPrefix prefixes = new RandomPrefix();
+    final String str = XmlUtils.encodeIdentifier(prefixes, id);
+
+    for (Entry<URI, String> e: prefixes.getPrefixes()) {
+      writer.writeNamespace(e.getValue(), e.getKey().toString());
+    }
+    LOG.debug("Instance identifier with Random prefix is now {}", str);
+    writer.writeCharacters(str);
+  }
+
+  /**
+   * Write a full XML document corresponding to a CompositeNode into an XML stream writer.
+   *
+   * @param writer XML Stream writer
+   * @param data data node
+   * @param schema corresponding schema node, may be null
+   * @throws javax.xml.stream.XMLStreamException if an encoding problem occurs
+   */
+  public void writeDocument(final @Nonnull XMLStreamWriter writer, final @Nonnull CompositeNode data, final @Nullable SchemaNode schema) throws XMLStreamException {
+    // final Boolean repairing = (Boolean) writer.getProperty(XMLOutputFactory.IS_REPAIRING_NAMESPACES);
+    // Preconditions.checkArgument(repairing == true, "XML Stream Writer has to be repairing namespaces");
+
+    writer.writeStartDocument();
+    writeElement(writer, data, schema);
+    writer.writeEndDocument();
+    writer.flush();
+  }
+
+
+  /**
+   * Write an element into a XML stream writer. This includes the element start/end tags and
+   * the value of the element.
+   *
+   * @param writer XML Stream writer
+   * @param data data node
+   * @param schema Schema node
+   * @throws javax.xml.stream.XMLStreamException if an encoding problem occurs
+   */
+  public void writeElement(final XMLStreamWriter writer, final @Nonnull Node<?> data, final SchemaNode schema) throws XMLStreamException {
+    final QName qname = data.getNodeType();
+    final String pfx = qname.getPrefix() != null ? qname.getPrefix() : "";
+    final String ns = qname.getNamespace() != null ? qname.getNamespace().toString() : "";
+
+    if (isEmptyElement(data)) {
+      writer.writeEmptyElement(pfx, qname.getLocalName(), ns);
+      return;
+    }
+
+    writer.writeStartElement(pfx, qname.getLocalName(), ns);
+    if (data instanceof AttributesContainer && ((AttributesContainer) data).getAttributes() != null) {
+      for (Entry<QName, String> attribute : ((AttributesContainer) data).getAttributes().entrySet()) {
+        writer.writeAttribute(attribute.getKey().getNamespace().toString(), attribute.getKey().getLocalName(), attribute.getValue());
+      }
+    }
+
+    if (data instanceof SimpleNode<?>) {
+      LOG.debug("writeElement : node is of type SimpleNode");
+      // Simple node
+      if (schema instanceof LeafListSchemaNode) {
+        writeValue(writer, ((LeafListSchemaNode) schema).getType(), data.getValue());
+      } else if (schema instanceof LeafSchemaNode) {
+        writeValue(writer, ((LeafSchemaNode) schema).getType(), data.getValue());
+      } else {
+        Object value = data.getValue();
+        if (value != null) {
+          writer.writeCharacters(String.valueOf(value));
+        }
+      }
+    } else {
+      LOG.debug("writeElement : node is of type CompositeNode");
+      // CompositeNode
+      for (Node<?> child : ((CompositeNode) data).getValue()) {
+        DataSchemaNode childSchema = null;
+        if (schema instanceof DataNodeContainer) {
+          childSchema = SchemaUtils.findFirstSchema(child.getNodeType(), ((DataNodeContainer) schema).getChildNodes()).orNull();
+          if (childSchema == null) {
+            LOG.debug("Probably the data node \"{}\" does not conform to schema", child == null ? "" : child.getNodeType().getLocalName());
+          }
+        }
+
+        writeElement(writer, child, childSchema);
+      }
+    }
+
+    writer.writeEndElement();
+  }
+
+  /**
+   * Write a value into a XML stream writer. This method assumes the start and end of element is
+   * emitted by the caller.
+   *
+   * @param writer XML Stream writer
+   * @param type type definitions
+   * @param value object value
+   * @throws javax.xml.stream.XMLStreamException if an encoding problem occurs
+   */
+  public void writeValue(final @Nonnull XMLStreamWriter writer, final @Nonnull TypeDefinition<?> type, final Object value) throws XMLStreamException {
+    if (value == null) {
+      LOG.debug("Value of {}:{} is null, not encoding it", type.getQName().getNamespace(), type.getQName().getLocalName());
+      return;
+    }
+
+    final TypeDefinition<?> baseType = XmlUtils.resolveBaseTypeFrom(type);
+    if (baseType instanceof IdentityrefTypeDefinition) {
+      write(writer, (IdentityrefTypeDefinition) baseType, value);
+    } else if (baseType instanceof InstanceIdentifierTypeDefinition) {
+      write(writer, (InstanceIdentifierTypeDefinition) baseType, value);
+    } else {
+      final TypeDefinitionAwareCodec<Object, ?> codec = codecProvider.codecFor(baseType);
+      String text;
+      if (codec != null) {
+        try {
+          text = codec.serialize(value);
+        } catch (ClassCastException e) {
+          LOG.error("Provided node value {} did not have type {} required by mapping. Using stream instead.", value, baseType, e);
+          text = String.valueOf(value);
+        }
+      } else {
+        LOG.error("Failed to find codec for {}, falling back to using stream", baseType);
+        text = String.valueOf(value);
+      }
+      writer.writeCharacters(text);
+    }
+  }
+
+  private static void write(final @Nonnull XMLStreamWriter writer, final @Nonnull IdentityrefTypeDefinition type, final @Nonnull Object value) throws XMLStreamException {
+    if (value instanceof QName) {
+      final QName qname = (QName) value;
+      final String prefix;
+      if (qname.getPrefix() != null && !qname.getPrefix().isEmpty()) {
+        prefix = qname.getPrefix();
+      } else {
+        prefix = "x";
+      }
+
+      writer.writeNamespace(prefix, qname.getNamespace().toString());
+      writer.writeCharacters(prefix + ':' + qname.getLocalName());
+    } else {
+      LOG.debug("Value of {}:{} is not a QName but {}", type.getQName().getNamespace(), type.getQName().getLocalName(), value.getClass());
+      writer.writeCharacters(String.valueOf(value));
+    }
+  }
+
+  private static void write(final @Nonnull XMLStreamWriter writer, final @Nonnull InstanceIdentifierTypeDefinition type, final @Nonnull Object value) throws XMLStreamException {
+    if (value instanceof YangInstanceIdentifier) {
+      LOG.debug("Writing InstanceIdentifier object {}", value);
+      write(writer, (YangInstanceIdentifier)value);
+    } else {
+      LOG.debug("Value of {}:{} is not an InstanceIdentifier but {}", type.getQName().getNamespace(), type.getQName().getLocalName(), value.getClass());
+      writer.writeCharacters(String.valueOf(value));
+    }
+  }
+}
diff --git a/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/XmlUtils.java b/opendaylight/md-sal/sal-remoterpc-connector/src/main/java/org/opendaylight/controller/remote/rpc/utils/XmlUtils.java
new file mode 100644 (file)
index 0000000..e07401a
--- /dev/null
@@ -0,0 +1,281 @@
+/*
+ * 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.remote.rpc.utils;
+
+import com.google.common.base.Optional;
+import org.opendaylight.controller.netconf.util.xml.XmlUtil;
+import org.opendaylight.yangtools.yang.common.QName;
+import org.opendaylight.yangtools.yang.data.api.CompositeNode;
+import org.opendaylight.yangtools.yang.data.api.Node;
+import org.opendaylight.yangtools.yang.data.api.SimpleNode;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifierWithPredicates;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeWithValue;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument;
+import org.opendaylight.yangtools.yang.data.impl.ImmutableCompositeNode;
+import org.opendaylight.yangtools.yang.data.impl.XmlTreeBuilder;
+import org.opendaylight.yangtools.yang.data.impl.codec.TypeDefinitionAwareCodec;
+import org.opendaylight.yangtools.yang.data.impl.codec.xml.XmlCodecProvider;
+import org.opendaylight.yangtools.yang.data.impl.util.CompositeNodeBuilder;
+import org.opendaylight.yangtools.yang.model.api.RpcDefinition;
+import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.opendaylight.yangtools.yang.model.api.TypeDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.NodeList;
+import org.xml.sax.SAXException;
+
+import javax.activation.UnsupportedDataTypeException;
+import javax.annotation.Nonnull;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.transform.OutputKeys;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Common XML-related utility methods, which are not specific to a particular
+ * JAXP API.
+ */
+public class XmlUtils {
+
+  public static final XmlCodecProvider DEFAULT_XML_CODEC_PROVIDER = new XmlCodecProvider() {
+    @Override
+    public TypeDefinitionAwareCodec<Object, ? extends TypeDefinition<?>> codecFor(final TypeDefinition<?> baseType) {
+      return TypeDefinitionAwareCodec.from(baseType);
+    }
+  };
+
+  private XmlUtils() {
+  }
+
+  private static final String BLANK = "";
+  private static final Logger LOG = LoggerFactory.getLogger(XmlUtils.class);
+
+  /**
+   * Converts the composite node to xml using rpc input schema node
+   * @param cNode
+   * @param schemaContext
+   * @return xml String
+   */
+  public static String inputCompositeNodeToXml(CompositeNode cNode, SchemaContext schemaContext){
+    LOG.debug("Converting input composite node to xml {}", cNode);
+    if (cNode == null) return BLANK;
+
+    if(schemaContext == null) return BLANK;
+
+    Document domTree = null;
+    try {
+      Set<RpcDefinition> rpcs =  schemaContext.getOperations();
+      for(RpcDefinition rpc : rpcs) {
+        if(rpc.getQName().equals(cNode.getNodeType())){
+          LOG.debug("Found the rpc definition from schema context matching with input composite node  {}", rpc.getQName());
+
+          CompositeNode inputContainer = cNode.getFirstCompositeByName(QName.create(cNode.getNodeType(), "input"));
+          domTree = XmlDocumentUtils.toDocument(inputContainer, rpc.getInput(), XmlDocumentUtils.defaultValueCodecProvider());
+
+          LOG.debug("input composite node to document conversion complete, document is   {}", domTree);
+          break;
+        }
+      }
+
+    } catch (UnsupportedDataTypeException e) {
+      LOG.error("Error during translation of CompositeNode to Document", e);
+    }
+    return domTransformer(domTree);
+  }
+
+  /**
+   * Converts the composite node to xml String using rpc output schema node
+   * @param cNode
+   * @param schemaContext
+   * @return xml string
+   */
+  public static String outputCompositeNodeToXml(CompositeNode cNode, SchemaContext schemaContext){
+    LOG.debug("Converting output composite node to xml {}", cNode);
+    if (cNode == null) return BLANK;
+
+    if(schemaContext == null) return BLANK;
+
+    Document domTree = null;
+    try {
+      Set<RpcDefinition> rpcs =  schemaContext.getOperations();
+      for(RpcDefinition rpc : rpcs) {
+        if(rpc.getQName().equals(cNode.getNodeType())){
+          LOG.debug("Found the rpc definition from schema context matching with output composite node  {}", rpc.getQName());
+
+          CompositeNode outputContainer = cNode.getFirstCompositeByName(QName.create(cNode.getNodeType(), "output"));
+          domTree = XmlDocumentUtils.toDocument(outputContainer, rpc.getOutput(), XmlDocumentUtils.defaultValueCodecProvider());
+
+          LOG.debug("output composite node to document conversion complete, document is   {}", domTree);
+          break;
+        }
+      }
+
+    } catch (UnsupportedDataTypeException e) {
+      LOG.error("Error during translation of CompositeNode to Document", e);
+    }
+    return domTransformer(domTree);
+  }
+
+  private static String domTransformer(Document domTree) {
+    StringWriter writer = new StringWriter();
+    try {
+      TransformerFactory tf = TransformerFactory.newInstance();
+      Transformer transformer = tf.newTransformer();
+      transformer.setOutputProperty(OutputKeys.OMIT_XML_DECLARATION, "yes");
+      transformer.transform(new DOMSource(domTree), new StreamResult(writer));
+    } catch (TransformerException e) {
+
+      LOG.error("Error during translation of Document to OutputStream", e);
+    }
+    LOG.debug("Document to string conversion complete, xml string is  {} ",  writer.toString());
+
+    return writer.toString();
+  }
+
+  public static CompositeNode xmlToCompositeNode(String xml){
+    if (xml==null || xml.length()==0) return null;
+
+    Node<?> dataTree;
+    try {
+      dataTree = XmlTreeBuilder.buildDataTree(new ByteArrayInputStream(xml.getBytes()));
+    } catch (XMLStreamException e) {
+      LOG.error("Error during building data tree from XML", e);
+      return null;
+    }
+    if (dataTree == null) {
+      LOG.error("data tree is null");
+      return null;
+    }
+    if (dataTree instanceof SimpleNode) {
+      LOG.error("RPC XML was resolved as SimpleNode");
+      return null;
+    }
+    return (CompositeNode) dataTree;
+  }
+
+  /**
+   * Converts the xml to composite node using rpc input schema node
+   * @param rpc
+   * @param xml
+   * @param schemaContext
+   * @return CompositeNode object based on the input, if any of the input parameter is null, a null object is returned
+   */
+  public static CompositeNode inputXmlToCompositeNode(QName rpc, String xml,  SchemaContext schemaContext){
+    LOG.debug("Converting input xml to composite node {}", xml);
+    if (xml==null || xml.length()==0) return null;
+
+    if(rpc == null) return null;
+
+    if(schemaContext == null) return null;
+
+    CompositeNode compositeNode = null;
+    try {
+
+      Document doc = XmlUtil.readXmlToDocument(xml);
+      Set<RpcDefinition> rpcs =  schemaContext.getOperations();
+      for(RpcDefinition rpcDef : rpcs) {
+        if(rpcDef.getQName().equals(rpc)){
+          LOG.debug("found the rpc definition from schema context matching rpc  {}", rpc);
+
+          if(rpcDef.getInput() == null) {
+            LOG.warn("found rpc definition's input is null");
+            return null;
+          }
+
+          QName input = rpcDef.getInput().getQName();
+          NodeList nodeList = doc.getElementsByTagNameNS(input.getNamespace().toString(), "input");
+          if(nodeList == null || nodeList.getLength() < 1) {
+            LOG.warn("xml does not have input entry. {}", xml);
+            return null;
+          }
+          Element xmlData = (Element)nodeList.item(0);
+
+          List<Node<?>> dataNodes = XmlDocumentUtils.toDomNodes(xmlData,
+              Optional.of(rpcDef.getInput().getChildNodes()), schemaContext);
+
+          LOG.debug("Converted xml input to list of nodes  {}", dataNodes);
+
+          final CompositeNodeBuilder<ImmutableCompositeNode> it = ImmutableCompositeNode.builder();
+          it.setQName(input);
+          it.add(ImmutableCompositeNode.create(input, dataNodes));
+          compositeNode = it.toInstance();
+          break;
+        }
+      }
+    } catch (SAXException e) {
+      LOG.error("Error during building data tree from XML", e);
+    } catch (IOException e) {
+      LOG.error("Error during building data tree from XML", e);
+    }
+
+    LOG.debug("Xml to composite node conversion complete {} ", compositeNode);
+    return compositeNode;
+  }
+
+  public static TypeDefinition<?> resolveBaseTypeFrom(final @Nonnull TypeDefinition<?> type) {
+    TypeDefinition<?> superType = type;
+    while (superType.getBaseType() != null) {
+      superType = superType.getBaseType();
+    }
+    return superType;
+  }
+
+  /**
+   * This code is picked from yangtools and modified to add type of instance identifier
+   * output of instance identifier something like below for a flow ref composite node of type instance identifier,
+   * which has path arguments with predicates, whose value is of type java.lang.short
+   * <flow-ref xmlns:bgkj="urn:opendaylight:flow:inventory" xmlns:jdlk="urn:opendaylight:inventory">
+   *   /jdlk:nodes/jdlk:node[jdlk:id='openflow:205558455098190@java.lang.String']
+   *   /bgkj:table[bgkj:id='3@java.lang.Short']
+   *   /bgkj:flow[bgkj:id='156@java.lang.String']
+   * </flow-ref>
+   *
+   */
+
+  public static String encodeIdentifier(final RandomPrefix prefixes, final YangInstanceIdentifier id) {
+    StringBuilder textContent = new StringBuilder();
+    for (PathArgument pathArgument : id.getPathArguments()) {
+      textContent.append('/');
+      textContent.append(prefixes.encodeQName(pathArgument.getNodeType()));
+      if (pathArgument instanceof NodeIdentifierWithPredicates) {
+        Map<QName, Object> predicates = ((NodeIdentifierWithPredicates) pathArgument).getKeyValues();
+
+        for (QName keyValue : predicates.keySet()) {
+          Object value = predicates.get(keyValue);
+          String type = value.getClass().getName();
+          String predicateValue = String.valueOf(value);
+          textContent.append('[');
+          textContent.append(prefixes.encodeQName(keyValue));
+          textContent.append("='");
+          textContent.append(predicateValue);
+          textContent.append("@");
+          textContent.append(type);
+          textContent.append("']");
+        }
+      } else if (pathArgument instanceof NodeWithValue) {
+        textContent.append("[.='");
+        textContent.append(((NodeWithValue) pathArgument).getValue());
+        textContent.append("']");
+      }
+    }
+
+    return textContent.toString();
+  }
+}
index 9585e9ffb6b10a45340011f9aa9fbcc34f46b9e1..6088dd0e0ea517c27bd74e79272a92f4b5b7ef1d 100644 (file)
@@ -7,7 +7,7 @@ odl-cluster{
     remote {
       log-remote-lifecycle-events = off
       netty.tcp {
-        hostname = "192.168.141.142"
+        hostname = "192.168.141.141"
         port = 2551
       }
     }
index 595d8331eb48ffba9bbef69da7bb1e293e98fbd8..392c1e637d848e1ccc47d43b3a951011a2e25a72 100644 (file)
@@ -35,6 +35,7 @@ import org.opendaylight.yangtools.yang.common.RpcResult;
 import org.opendaylight.yangtools.yang.data.api.CompositeNode;
 import org.opendaylight.yangtools.yang.data.api.ModifyAction;
 import org.opendaylight.yangtools.yang.data.api.Node;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.impl.ImmutableCompositeNode;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 
@@ -72,15 +73,16 @@ public class RpcBrokerTest {
       Broker.ProviderSession brokerSession = Mockito.mock(Broker.ProviderSession.class);
       SchemaContext schemaContext = mock(SchemaContext.class);
       ActorRef rpcBroker = system.actorOf(RpcBroker.props(brokerSession, rpcRegistry, schemaContext));
-      QName rpc = new QName(new URI("actor1"), "actor1");
-      InvokeRpc invokeMsg = new InvokeRpc(rpc, null);
+      QName rpc = new QName(new URI("noactor1"), "noactor1");
+      CompositeNode input = new ImmutableCompositeNode(QName.create("ns", "2013-12-09", "no child"), new ArrayList<Node<?>>(), ModifyAction.REPLACE);
+      InvokeRpc invokeMsg = new InvokeRpc(rpc, input);
       rpcBroker.tell(invokeMsg, getRef());
 
       Boolean getMsg = new ExpectMsg<Boolean>("ErrorResponse") {
         protected Boolean match(Object in) {
           if (in instanceof ErrorResponse) {
             ErrorResponse reply = (ErrorResponse)in;
-            return "No remote actor found for rpc execution.".equals(reply.getException().getMessage());
+            return reply.getException().getMessage().contains("No remote actor found for rpc execution of :");
           } else {
             throw noMatch();
           }
@@ -144,7 +146,8 @@ public class RpcBrokerTest {
       SchemaContext schemaContext = mock(SchemaContext.class);
       ActorRef rpcBroker = system.actorOf(RpcBroker.props(brokerSession, rpcRegistry, schemaContext));
       QName rpc = new QName(new URI("actor1"), "actor1");
-      InvokeRoutedRpc invokeMsg = new InvokeRoutedRpc(rpc, null);
+      CompositeNode input = new ImmutableCompositeNode(QName.create("ns", "2013-12-09", "child1"), new ArrayList<Node<?>>(), ModifyAction.REPLACE);
+      InvokeRoutedRpc invokeMsg = new InvokeRoutedRpc(rpc, YangInstanceIdentifier.create(new YangInstanceIdentifier.NodeIdentifier(rpc)), input);
       rpcBroker.tell(invokeMsg, getRef());
 
       Boolean getMsg = new ExpectMsg<Boolean>("ErrorResponse") {
@@ -176,7 +179,8 @@ public class RpcBrokerTest {
       ActorRef rpcBrokerRemote = system.actorOf(RpcBroker.props(brokerSession, rpcRegistry, schemaContext), "actor2");
       // Add Routed RPC in table
       QName rpc = new QName(new URI("actor2"), "actor2");
-      RouteIdentifierImpl routeId = new RouteIdentifierImpl(null, rpc, null);
+      YangInstanceIdentifier identifier = YangInstanceIdentifier.create(new YangInstanceIdentifier.NodeIdentifier(rpc));
+      RouteIdentifierImpl routeId = new RouteIdentifierImpl(null, rpc, identifier);
       final String route = rpcBrokerRemote.path().toString();
       Set<RpcRouter.RouteIdentifier<?, ?, ?>> routeIds = new HashSet<>();
       routeIds.add(routeId);
@@ -192,7 +196,7 @@ public class RpcBrokerTest {
       RpcResult<CompositeNode> result = Rpcs.getRpcResult(true, invokeRpcResult, errors);
       Future<RpcResult<CompositeNode>> rpcResult = Futures.immediateFuture(result);
       when(brokerSession.rpc(rpc, input)).thenReturn(rpcResult);
-      InvokeRoutedRpc invokeMsg = new InvokeRoutedRpc(rpc, input);
+      InvokeRoutedRpc invokeMsg = new InvokeRoutedRpc(rpc, identifier, input);
       rpcBroker.tell(invokeMsg, getRef());
 
       //verify response msg
index a57402a793ae06ca5dc4f4259c8e2d2b74b3d3aa..129a5a56e838298416716ff4e056faec7ae1370c 100644 (file)
@@ -39,7 +39,8 @@ public class RoutingTableTest {
   @Test
   public void addGlobalRouteNullRouteTest() {
     try {
-      RouteIdentifierImpl routeId = new RouteIdentifierImpl(null, null, null);
+      QName type = new QName(new URI("actor1"), "actor1");
+      RouteIdentifierImpl routeId = new RouteIdentifierImpl(null, type, null);
       routingTable.addGlobalRoute(routeId, null);
 
       Assert.fail("Null pointer exception was not thrown.");
@@ -109,7 +110,8 @@ public class RoutingTableTest {
   @Test
   public void addRoutedRpcNullRouteTest() {
     try {
-      RouteIdentifierImpl routeId = new RouteIdentifierImpl(null, null, null);
+      QName type = new QName(new URI("actor1"), "actor1");
+      RouteIdentifierImpl routeId = new RouteIdentifierImpl(null, type, null);
 
       routingTable.addRoutedRpc(routeId, null);
 
diff --git a/opendaylight/md-sal/sal-remoterpc-connector/src/test/java/org/opendaylight/controller/remote/rpc/utils/XmlUtilsTest.java b/opendaylight/md-sal/sal-remoterpc-connector/src/test/java/org/opendaylight/controller/remote/rpc/utils/XmlUtilsTest.java
new file mode 100644 (file)
index 0000000..a408e1d
--- /dev/null
@@ -0,0 +1,133 @@
+/*
+ * 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.remote.rpc.utils;
+
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.io.ByteSource;
+import junit.framework.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.opendaylight.yangtools.yang.common.QName;
+import org.opendaylight.yangtools.yang.data.api.CompositeNode;
+import org.opendaylight.yangtools.yang.data.api.ModifyAction;
+import org.opendaylight.yangtools.yang.data.api.Node;
+import org.opendaylight.yangtools.yang.data.api.SimpleNode;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.impl.ImmutableCompositeNode;
+import org.opendaylight.yangtools.yang.model.api.Module;
+import org.opendaylight.yangtools.yang.model.api.RpcDefinition;
+import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.opendaylight.yangtools.yang.parser.impl.YangParserImpl;
+
+import javax.xml.parsers.DocumentBuilderFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+
+public class XmlUtilsTest {
+
+  private static final DocumentBuilderFactory BUILDERFACTORY;
+
+  static {
+    final DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
+    factory.setNamespaceAware(true);
+    factory.setCoalescing(true);
+    factory.setIgnoringElementContentWhitespace(true);
+    factory.setIgnoringComments(true);
+    BUILDERFACTORY = factory;
+  }
+
+  private SchemaContext schema;
+  private RpcDefinition testRpc;
+
+  public static final String XML_CONTENT = "<add-flow xmlns=\"urn:opendaylight:controller:rpc:test\"><input xmlns=\"urn:opendaylight:controller:rpc:test\">" +
+      "<id>flowid</id>" +
+      "<flow xmlns:ltha=\"urn:opendaylight:controller:rpc:test\">/ltha:node/ltha:node1[ltha:id='3@java.lang.Short']</flow>" +
+      "</input></add-flow>";
+
+  @Before
+  public void setUp() throws Exception {
+    final ByteSource byteSource = new ByteSource() {
+      @Override
+      public InputStream openStream() throws IOException {
+        return XmlUtilsTest.this.getClass().getResourceAsStream("rpcTest.yang");
+      }
+    };
+    schema = new YangParserImpl().parseSources(Lists.newArrayList(byteSource));
+    final Module rpcTestModule = schema.getModules().iterator().next();
+    testRpc = rpcTestModule.getRpcs().iterator().next();
+  }
+
+  @Test
+  public void testNullInputXmlToComposite() {
+    CompositeNode node = XmlUtils.inputXmlToCompositeNode(testRpc.getQName(), null, schema);
+    Assert.assertNull(node);
+  }
+
+  @Test
+  public void testNullRpcXmlToComposite() {
+    CompositeNode node = XmlUtils.inputXmlToCompositeNode(null, XML_CONTENT, schema);
+    Assert.assertNull(node);
+  }
+
+  @Test
+  public void testInputXmlToCompositeNode() {
+    CompositeNode node = XmlUtils.inputXmlToCompositeNode(testRpc.getQName(), XML_CONTENT, schema);
+    ImmutableList<SimpleNode> input = (ImmutableList)node.getValue().get(0).getValue();
+    SimpleNode firstNode = input.get(0);
+
+    Assert.assertEquals("id", firstNode.getNodeType().getLocalName());
+    Assert.assertEquals("flowid", firstNode.getValue());
+
+    SimpleNode secondNode = input.get(1);
+    Assert.assertEquals("flow", secondNode.getNodeType().getLocalName());
+
+    YangInstanceIdentifier instance = (YangInstanceIdentifier) secondNode.getValue();
+    Iterable<YangInstanceIdentifier.PathArgument> iterable = instance.getPathArguments();
+    Iterator it = iterable.iterator();
+    YangInstanceIdentifier.NodeIdentifier firstPath = (YangInstanceIdentifier.NodeIdentifier) it.next();
+    Assert.assertEquals("node", firstPath.getNodeType().getLocalName());
+    YangInstanceIdentifier.NodeIdentifierWithPredicates secondPath = (YangInstanceIdentifier.NodeIdentifierWithPredicates)it.next();
+    Short value = (Short)secondPath.getKeyValues().values().iterator().next();
+    Short expected = 3;
+    Assert.assertEquals(expected, value);
+  }
+
+  @Test
+  public void testInputCompositeNodeToXML() {
+    CompositeNode input = XmlUtils.inputXmlToCompositeNode(testRpc.getQName(), XML_CONTENT, schema);
+    List<Node<?>> childNodes = new ArrayList();
+    childNodes.add(input);
+    QName rpcQName = schema.getOperations().iterator().next().getQName();
+    CompositeNode node = new ImmutableCompositeNode(rpcQName, input.getValue(), ModifyAction.REPLACE);
+    String xml = XmlUtils.inputCompositeNodeToXml(node, schema);
+    Assert.assertNotNull(xml);
+    Assert.assertTrue(xml.contains("3@java.lang.Short"));
+  }
+
+  @Test
+  public void testNullCompositeNodeToXml(){
+    String xml = XmlUtils.inputCompositeNodeToXml(null, schema);
+    Assert.assertTrue(xml.isEmpty());
+  }
+
+  @Test
+  public void testNullSchemaCompositeNodeToXml(){
+    String xml = XmlUtils.inputCompositeNodeToXml(new ImmutableCompositeNode(QName.create("ns", "2013-12-09", "child1"), new ArrayList<Node<?>>(), ModifyAction.REPLACE), null);
+    Assert.assertTrue(xml.isEmpty());
+  }
+
+
+}
diff --git a/opendaylight/md-sal/sal-remoterpc-connector/src/test/resources/org/opendaylight/controller/remote/rpc/utils/rpcTest.yang b/opendaylight/md-sal/sal-remoterpc-connector/src/test/resources/org/opendaylight/controller/remote/rpc/utils/rpcTest.yang
new file mode 100644 (file)
index 0000000..5fc564f
--- /dev/null
@@ -0,0 +1,32 @@
+/*
+ * 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
+ */
+module rpc-test {
+    yang-version 1;
+    namespace "urn:opendaylight:controller:rpc:test";
+    prefix "rpct";
+
+    revision 2014-07-29 {
+       description "rpc test";
+    }
+
+    typedef flow-ref {
+        type instance-identifier;
+    }
+
+    rpc add-flow {
+        input {
+            leaf id {
+                type string;
+            }
+
+            leaf flow {
+                type flow-ref;
+            }
+        }
+    }
+}
\ No newline at end of file
index cab07afe6c36778e02938d3d63b604fba4ab0f1c..a62af7651eb74e7b009d302af1c577ac9fc121cf 100644 (file)
@@ -8,6 +8,7 @@
 package org.opendaylight.controller.netconf.cli.writer.impl;
 
 import org.opendaylight.controller.netconf.cli.writer.OutFormatter;
+import org.opendaylight.yangtools.yang.data.api.schema.AnyXmlNode;
 import org.opendaylight.yangtools.yang.data.api.schema.AugmentationNode;
 import org.opendaylight.yangtools.yang.data.api.schema.ChoiceNode;
 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
@@ -20,6 +21,7 @@ import org.opendaylight.yangtools.yang.data.impl.codec.xml.XmlCodecProvider;
 import org.opendaylight.yangtools.yang.data.impl.schema.transform.FromNormalizedNodeSerializer;
 import org.opendaylight.yangtools.yang.data.impl.schema.transform.FromNormalizedNodeSerializerFactory;
 import org.opendaylight.yangtools.yang.data.impl.schema.transform.base.serializer.NodeSerializerDispatcher;
+import org.opendaylight.yangtools.yang.model.api.AnyXmlSchemaNode;
 import org.opendaylight.yangtools.yang.model.api.AugmentationSchema;
 import org.opendaylight.yangtools.yang.model.api.ContainerSchemaNode;
 import org.opendaylight.yangtools.yang.model.api.LeafListSchemaNode;
@@ -100,4 +102,9 @@ public final class CliOutputFromNormalizedNodeSerializerFactory implements FromN
         return mapNodeSerializer;
     }
 
+    @Override
+    public FromNormalizedNodeSerializer<String, AnyXmlNode, AnyXmlSchemaNode> getAnyXmlNodeSerializer() {
+        throw new UnsupportedOperationException();
+    }
+
 }
\ No newline at end of file
index 840029006be5b16a4d394bae0d6daa0b5efd65a5..1f10b395137c498b3e542e974545132332e84be4 100644 (file)
@@ -75,7 +75,7 @@ public class NeutronSubnet extends ConfigurationObject implements Serializable,
      */
     List<NeutronPort> myPorts;
 
-    boolean gatewayIPAssigned;
+    Boolean gatewayIPAssigned;
 
     public NeutronSubnet() {
         myPorts = new ArrayList<NeutronPort>();
@@ -299,7 +299,7 @@ public class NeutronSubnet extends ConfigurationObject implements Serializable,
             try {
                 SubnetUtils util = new SubnetUtils(cidr);
                 SubnetInfo info = util.getInfo();
-                if (gatewayIP == null) {
+                if (gatewayIP == null || ("").equals(gatewayIP)) {
                     gatewayIP = info.getLowAddress();
                 }
                 if (allocationPools.size() < 1) {
@@ -460,6 +460,10 @@ public class NeutronSubnet extends ConfigurationObject implements Serializable,
         gatewayIPAssigned = false;
     }
 
+    public Boolean getGatewayIPAllocated() {
+        return gatewayIPAssigned;
+    }
+
     @Override
     public String toString() {
         return "NeutronSubnet [subnetUUID=" + subnetUUID + ", networkUUID=" + networkUUID + ", name=" + name
index a2d2dac112e932d7578698bd1eccf54f54942f50..cbc1f0c328e58be31f53304c657ed050f746ffe1 100644 (file)
@@ -91,7 +91,6 @@
               javax.ws.rs.core,
               javax.xml.bind,
               javax.xml.bind.annotation,
-              org.objectweb.asm,
               org.opendaylight.controller.sal.utils,
               org.opendaylight.controller.sal.core,
               org.opendaylight.controller.sal.authorization,
index 17b2fcfcf9ef43437903f499dc4da2aceb61983b..806e853b3604ec7848e08176aaf7f8d8e8f3996f 100644 (file)
@@ -422,7 +422,9 @@ public class NeutronRoutersNorthbound {
         if (instances != null) {
             for (Object instance : instances) {
                 INeutronRouterAware service = (INeutronRouterAware) instance;
-                service.canAttachInterface(target, input);
+                int status = service.canAttachInterface(target, input);
+                if (status < 200 || status > 299)
+                    return Response.status(status).build();
             }
         }
 
@@ -498,7 +500,9 @@ public class NeutronRoutersNorthbound {
             if (instances != null) {
                 for (Object instance : instances) {
                     INeutronRouterAware service = (INeutronRouterAware) instance;
-                    service.canDetachInterface(target, input);
+                    int status = service.canDetachInterface(target, input);
+                    if (status < 200 || status > 299)
+                        return Response.status(status).build();
                 }
             }