Merge "Bug 1965: Fixed DataChangedReply sent to deadletters"
authorMoiz Raja <moraja@cisco.com>
Thu, 16 Oct 2014 20:51:59 +0000 (20:51 +0000)
committerGerrit Code Review <gerrit@opendaylight.org>
Thu, 16 Oct 2014 20:51:59 +0000 (20:51 +0000)
28 files changed:
opendaylight/config/config-api/src/main/yang/config.yang
opendaylight/config/config-manager/src/main/java/org/opendaylight/controller/config/manager/impl/jmx/InternalJMXRegistrator.java
opendaylight/config/netty-event-executor-config/src/test/java/org/opendaylight/controller/config/yang/netty/eventexecutor/ImmediateEventExecutorModuleTest.java
opendaylight/md-sal/compatibility/sal-compatibility/src/main/java/org/opendaylight/controller/sal/compatibility/ComponentActivator.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardTransaction.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ThreePhaseCommitCohortProxy.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionChainProxy.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/messages/ReadyTransactionReply.java
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/BasicIntegrationTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ThreePhaseCommitCohortProxyTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/TransactionProxyTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/utils/ActorContextTest.java
opendaylight/md-sal/sal-rest-connector/src/main/java/org/opendaylight/controller/sal/rest/impl/NormalizedNodeXmlBodyWriter.java
opendaylight/md-sal/sal-rest-connector/src/main/java/org/opendaylight/controller/sal/restconf/impl/BrokerFacade.java
opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/NetconfMonitoringServiceImplTest.java [new file with mode: 0644]
opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/mapping/operations/DefaultCloseSessionTest.java [new file with mode: 0644]
opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/mapping/operations/DefaultCommitTest.java [new file with mode: 0644]
opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/mapping/operations/DefaultGetSchemaTest.java [new file with mode: 0644]
opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/mapping/operations/DefaultStopExiTest.java [new file with mode: 0644]
opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/osgi/NetconfImplActivatorTest.java [new file with mode: 0644]
opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/osgi/NetconfOperationServiceFactoryTrackerTest.java [new file with mode: 0644]
opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/util/DeserializerExceptionHandlerTest.java [new file with mode: 0644]
opendaylight/netconf/netconf-monitoring/src/test/java/org/opendaylight/controller/netconf/monitoring/osgi/NetconfMonitoringActivatorTest.java [new file with mode: 0644]
opendaylight/netconf/netconf-monitoring/src/test/java/org/opendaylight/controller/netconf/monitoring/osgi/NetconfMonitoringOperationServiceTest.java [new file with mode: 0644]
opendaylight/netconf/netconf-monitoring/src/test/java/org/opendaylight/controller/netconf/monitoring/osgi/NetconfMonitoringServiceTrackerTest.java [new file with mode: 0644]
opendaylight/switchmanager/implementation/src/main/java/org/opendaylight/controller/switchmanager/internal/Activator.java

index a0a4292adf406a597fa120cdbf869c5b4cc0e0c6..5d6c11fbeead51a0652ce151265ca3e50313f7d0 100644 (file)
@@ -122,7 +122,9 @@ module config {
                  the actual service-type which is actually required.";
 
             mandatory true;
-            type service-type-ref;
+            type leafref {
+                path "/config:services/config:service/config:type";
+            }
         }
 
         leaf name {
index 98f0908dc710cf40fbb93b88004d411d03f594fd..c6f4be64bb91609dfd4630f768ab114f4d69ab2e 100644 (file)
@@ -53,6 +53,7 @@ public class InternalJMXRegistrator implements Closeable {
 
     @GuardedBy("this")
     private final Set<ObjectName> registeredObjectNames = new HashSet<>();
+    @GuardedBy("this")
     private final List<InternalJMXRegistrator> children = new ArrayList<>();
 
     public synchronized InternalJMXRegistration registerMBean(Object object,
@@ -79,9 +80,8 @@ public class InternalJMXRegistrator implements Closeable {
         }
     }
 
-    public InternalJMXRegistrator createChild() {
-        InternalJMXRegistrator child = new InternalJMXRegistrator(
-                configMBeanServer);
+    public synchronized InternalJMXRegistrator createChild() {
+        InternalJMXRegistrator child = new InternalJMXRegistrator(configMBeanServer);
         children.add(child);
         return child;
     }
@@ -137,7 +137,7 @@ public class InternalJMXRegistrator implements Closeable {
         return getSameNames(result);
     }
 
-    private Set<ObjectName> getSameNames(Set<ObjectName> superSet) {
+    private synchronized Set<ObjectName> getSameNames(Set<ObjectName> superSet) {
         Set<ObjectName> result = new HashSet<>(superSet);
         result.retainAll(registeredObjectNames);
         for (InternalJMXRegistrator child : children) {
index 4cc9cc3dde0e91b7c35f98f28206ae69845c04c2..54c87604a433298824fc33ae12a6dc11732dfa92 100644 (file)
@@ -21,12 +21,12 @@ import org.opendaylight.controller.config.util.ConfigTransactionJMXClient;
 
 public class ImmediateEventExecutorModuleTest extends AbstractConfigTest {
 
-    private GlobalEventExecutorModuleFactory factory;
+    private ImmediateEventExecutorModuleFactory factory;
     private final String instanceName = ImmediateEventExecutorModuleFactory.SINGLETON_NAME;
 
     @Before
     public void setUp() {
-        factory = new GlobalEventExecutorModuleFactory();
+        factory = new ImmediateEventExecutorModuleFactory();
         super.initConfigTransactionManagerImpl(new HardcodedModuleFactoriesResolver(mockedContext,factory));
     }
 
index 4fc0cf75b33702574f12ecb605d87d5392566086..6a8e3c0e5bc62ee437ce2661f755d288e4caab55 100644 (file)
@@ -7,9 +7,7 @@
  */
 package org.opendaylight.controller.sal.compatibility;
 
-import java.util.Dictionary;
-import java.util.Hashtable;
-
+import com.google.common.base.Preconditions;
 import org.apache.felix.dm.Component;
 import org.opendaylight.controller.clustering.services.IClusterGlobalServices;
 import org.opendaylight.controller.sal.binding.api.BindingAwareBroker;
@@ -36,7 +34,8 @@ import org.opendaylight.controller.sal.utils.INodeConnectorFactory;
 import org.opendaylight.controller.sal.utils.INodeFactory;
 import org.osgi.framework.BundleContext;
 
-import com.google.common.base.Preconditions;
+import java.util.Dictionary;
+import java.util.Hashtable;
 
 public class ComponentActivator extends ComponentActivatorAbstractBase {
     private final INodeConnectorFactory nodeConnectorFactory = new MDSalNodeConnectorFactory();
@@ -90,8 +89,8 @@ public class ComponentActivator extends ComponentActivatorAbstractBase {
 
     @Override
     public void start(final BundleContext context) {
-        super.start(context);
         this.context = Preconditions.checkNotNull(context);
+        super.start(context);
     }
 
     public ProviderContext setBroker(final BindingAwareBroker broker) {
@@ -101,14 +100,14 @@ public class ComponentActivator extends ComponentActivatorAbstractBase {
     @Override
     protected Object[] getGlobalImplementations() {
         return new Object[] {
+                this, // Used for setBroker callback
                 flow,
                 inventory,
                 dataPacket,
                 nodeFactory,
                 nodeConnectorFactory,
                 topology,
-                tpProvider,
-                this // Used for setBroker callback
+                tpProvider
         };
     }
 
@@ -216,6 +215,9 @@ public class ComponentActivator extends ComponentActivatorAbstractBase {
                 .setService(IDiscoveryService.class)
                 .setCallbacks("setDiscoveryPublisher", "setDiscoveryPublisher")
                 .setRequired(false));
+        it.add(createServiceDependency()
+                .setService(BindingAwareBroker.class)
+                .setRequired(true));
     }
 
     private void _instanceConfigure(final InventoryAndReadAdapter imp, final Component it, String containerName) {
@@ -232,6 +234,9 @@ public class ComponentActivator extends ComponentActivatorAbstractBase {
                 .setService(IPluginOutInventoryService.class)
                 .setCallbacks("setInventoryPublisher", "unsetInventoryPublisher")
                 .setRequired(false));
+        it.add(createServiceDependency()
+                .setService(BindingAwareBroker.class)
+                .setRequired(true));
     }
 
     private void _configure(final TopologyAdapter imp, final Component it) {
index f5ca6e3c5aa2334eb26c52408dd7279f08e33d3a..b16ec0ac9f919b52855aaa7c608801b6a7ce2d01 100644 (file)
@@ -13,12 +13,10 @@ import akka.actor.PoisonPill;
 import akka.actor.Props;
 import akka.actor.ReceiveTimeout;
 import akka.japi.Creator;
-
+import akka.serialization.Serialization;
 import com.google.common.base.Optional;
 import com.google.common.util.concurrent.CheckedFuture;
 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedActor;
-
-
 import org.opendaylight.controller.cluster.datastore.exceptions.UnknownMessageException;
 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
 import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
@@ -207,8 +205,8 @@ public abstract class ShardTransaction extends AbstractUntypedActor {
         DOMStoreThreePhaseCommitCohort cohort =  transaction.ready();
         ActorRef cohortActor = getContext().actorOf(
             ThreePhaseCommitCohort.props(cohort, shardActor, modification, shardStats), "cohort");
-        getSender()
-        .tell(new ReadyTransactionReply(cohortActor.path()).toSerializable(), getSelf());
+        getSender().tell(new ReadyTransactionReply(
+            Serialization.serializedActorPath(cohortActor)).toSerializable(), getSelf());
 
     }
 
index 515be372e8c76b7008b7356a666bbf94e19a1972..6e7669695eb2e6317d00b6c69cc33861bb24fa48 100644 (file)
@@ -8,16 +8,13 @@
 
 package org.opendaylight.controller.cluster.datastore;
 
-import akka.actor.ActorPath;
 import akka.actor.ActorSelection;
 import akka.dispatch.Futures;
 import akka.dispatch.OnComplete;
-
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
-
 import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransaction;
@@ -30,7 +27,6 @@ import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import scala.concurrent.Future;
 import scala.runtime.AbstractFunction1;
 
@@ -45,29 +41,29 @@ public class ThreePhaseCommitCohortProxy implements DOMStoreThreePhaseCommitCoho
     private static final Logger LOG = LoggerFactory.getLogger(ThreePhaseCommitCohortProxy.class);
 
     private final ActorContext actorContext;
-    private final List<Future<ActorPath>> cohortPathFutures;
-    private volatile List<ActorPath> cohortPaths;
+    private final List<Future<ActorSelection>> cohortFutures;
+    private volatile List<ActorSelection> cohorts;
     private final String transactionId;
 
     public ThreePhaseCommitCohortProxy(ActorContext actorContext,
-            List<Future<ActorPath>> cohortPathFutures, String transactionId) {
+            List<Future<ActorSelection>> cohortFutures, String transactionId) {
         this.actorContext = actorContext;
-        this.cohortPathFutures = cohortPathFutures;
+        this.cohortFutures = cohortFutures;
         this.transactionId = transactionId;
     }
 
-    private Future<Void> buildCohortPathsList() {
+    private Future<Void> buildCohortList() {
 
-        Future<Iterable<ActorPath>> combinedFutures = Futures.sequence(cohortPathFutures,
+        Future<Iterable<ActorSelection>> combinedFutures = Futures.sequence(cohortFutures,
                 actorContext.getActorSystem().dispatcher());
 
-        return combinedFutures.transform(new AbstractFunction1<Iterable<ActorPath>, Void>() {
+        return combinedFutures.transform(new AbstractFunction1<Iterable<ActorSelection>, Void>() {
             @Override
-            public Void apply(Iterable<ActorPath> paths) {
-                cohortPaths = Lists.newArrayList(paths);
+            public Void apply(Iterable<ActorSelection> actorSelections) {
+                cohorts = Lists.newArrayList(actorSelections);
                 if(LOG.isDebugEnabled()) {
                     LOG.debug("Tx {} successfully built cohort path list: {}",
-                        transactionId, cohortPaths);
+                        transactionId, cohorts);
                 }
                 return null;
             }
@@ -87,12 +83,12 @@ public class ThreePhaseCommitCohortProxy implements DOMStoreThreePhaseCommitCoho
         // extracted from ReadyTransactionReply messages by the Futures that were obtained earlier
         // and passed to us from upstream processing. If any one fails then  we'll fail canCommit.
 
-        buildCohortPathsList().onComplete(new OnComplete<Void>() {
+        buildCohortList().onComplete(new OnComplete<Void>() {
             @Override
             public void onComplete(Throwable failure, Void notUsed) throws Throwable {
                 if(failure != null) {
                     if(LOG.isDebugEnabled()) {
-                        LOG.debug("Tx {}: a cohort path Future failed: {}", transactionId, failure);
+                        LOG.debug("Tx {}: a cohort Future failed: {}", transactionId, failure);
                     }
                     returnFuture.setException(failure);
                 } else {
@@ -150,12 +146,11 @@ public class ThreePhaseCommitCohortProxy implements DOMStoreThreePhaseCommitCoho
     }
 
     private Future<Iterable<Object>> invokeCohorts(Object message) {
-        List<Future<Object>> futureList = Lists.newArrayListWithCapacity(cohortPaths.size());
-        for(ActorPath actorPath : cohortPaths) {
+        List<Future<Object>> futureList = Lists.newArrayListWithCapacity(cohorts.size());
+        for(ActorSelection cohort : cohorts) {
             if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {}: Sending {} to cohort {}", transactionId, message, actorPath);
+                LOG.debug("Tx {}: Sending {} to cohort {}", transactionId, message, cohort);
             }
-            ActorSelection cohort = actorContext.actorSelection(actorPath);
 
             futureList.add(actorContext.executeOperationAsync(cohort, message));
         }
@@ -198,11 +193,11 @@ public class ThreePhaseCommitCohortProxy implements DOMStoreThreePhaseCommitCoho
         // The cohort actor list should already be built at this point by the canCommit phase but,
         // if not for some reason, we'll try to build it here.
 
-        if(cohortPaths != null) {
+        if(cohorts != null) {
             finishVoidOperation(operationName, message, expectedResponseClass, propagateException,
                     returnFuture);
         } else {
-            buildCohortPathsList().onComplete(new OnComplete<Void>() {
+            buildCohortList().onComplete(new OnComplete<Void>() {
                 @Override
                 public void onComplete(Throwable failure, Void notUsed) throws Throwable {
                     if(failure != null) {
@@ -280,7 +275,7 @@ public class ThreePhaseCommitCohortProxy implements DOMStoreThreePhaseCommitCoho
     }
 
     @VisibleForTesting
-    List<Future<ActorPath>> getCohortPathFutures() {
-        return Collections.unmodifiableList(cohortPathFutures);
+    List<Future<ActorSelection>> getCohortFutures() {
+        return Collections.unmodifiableList(cohortFutures);
     }
 }
index b74c89d727c2a1761a7c993272abcab1e4ad9999..b467ee4ddbf56c456c2e9f0f62381eefa173e31d 100644 (file)
@@ -8,7 +8,7 @@
 
 package org.opendaylight.controller.cluster.datastore;
 
-import akka.actor.ActorPath;
+import akka.actor.ActorSelection;
 import akka.dispatch.Futures;
 import org.opendaylight.controller.cluster.datastore.messages.CloseTransactionChain;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
@@ -28,7 +28,7 @@ import java.util.List;
 public class TransactionChainProxy implements DOMStoreTransactionChain{
     private final ActorContext actorContext;
     private final String transactionChainId;
-    private volatile List<Future<ActorPath>> cohortPathFutures = Collections.emptyList();
+    private volatile List<Future<ActorSelection>> cohortFutures = Collections.emptyList();
 
     public TransactionChainProxy(ActorContext actorContext) {
         this.actorContext = actorContext;
@@ -63,14 +63,14 @@ public class TransactionChainProxy implements DOMStoreTransactionChain{
         return transactionChainId;
     }
 
-    public void onTransactionReady(List<Future<ActorPath>> cohortPathFutures){
-        this.cohortPathFutures = cohortPathFutures;
+    public void onTransactionReady(List<Future<ActorSelection>> cohortFutures){
+        this.cohortFutures = cohortFutures;
     }
 
     public void waitTillCurrentTransactionReady(){
         try {
             Await.result(Futures
-                .sequence(this.cohortPathFutures, actorContext.getActorSystem().dispatcher()),
+                .sequence(this.cohortFutures, actorContext.getActorSystem().dispatcher()),
                 actorContext.getOperationDuration());
         } catch (Exception e) {
             throw new IllegalStateException("Failed when waiting for transaction on a chain to become ready", e);
index 19d9a66a528eb417d5bff41948641b68e9c8e481..ec198510d3586f88dee40d04f9294a3fb370a9ae 100644 (file)
@@ -8,7 +8,6 @@
 
 package org.opendaylight.controller.cluster.datastore;
 
-import akka.actor.ActorPath;
 import akka.actor.ActorSelection;
 import akka.dispatch.OnComplete;
 
@@ -315,7 +314,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             LOG.debug("Tx {} Trying to get {} transactions ready for commit", identifier,
                 remoteTransactionPaths.size());
         }
-        List<Future<ActorPath>> cohortPathFutures = Lists.newArrayList();
+        List<Future<ActorSelection>> cohortFutures = Lists.newArrayList();
 
         for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
 
@@ -323,14 +322,14 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                 LOG.debug("Tx {} Readying transaction for shard {}", identifier,
                     transactionContext.getShardName());
             }
-            cohortPathFutures.add(transactionContext.readyTransaction());
+            cohortFutures.add(transactionContext.readyTransaction());
         }
 
         if(transactionChainProxy != null){
-            transactionChainProxy.onTransactionReady(cohortPathFutures);
+            transactionChainProxy.onTransactionReady(cohortFutures);
         }
 
-        return new ThreePhaseCommitCohortProxy(actorContext, cohortPathFutures,
+        return new ThreePhaseCommitCohortProxy(actorContext, cohortFutures,
                 identifier.toString());
     }
 
@@ -439,7 +438,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         void closeTransaction();
 
-        Future<ActorPath> readyTransaction();
+        Future<ActorSelection> readyTransaction();
 
         void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
 
@@ -499,10 +498,6 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             return actor;
         }
 
-        private String getResolvedCohortPath(String cohortPath) {
-            return actorContext.resolvePath(actorPath, cohortPath);
-        }
-
         @Override
         public void closeTransaction() {
             if(LOG.isDebugEnabled()) {
@@ -512,7 +507,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         }
 
         @Override
-        public Future<ActorPath> readyTransaction() {
+        public Future<ActorSelection> readyTransaction() {
             if(LOG.isDebugEnabled()) {
                 LOG.debug("Tx {} readyTransaction called with {} previous recorded operations pending",
                     identifier, recordedOperationFutures.size());
@@ -538,9 +533,9 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             // Transform the combined Future into a Future that returns the cohort actor path from
             // the ReadyTransactionReply. That's the end result of the ready operation.
 
-            return combinedFutures.transform(new AbstractFunction1<Iterable<Object>, ActorPath>() {
+            return combinedFutures.transform(new AbstractFunction1<Iterable<Object>, ActorSelection>() {
                 @Override
-                public ActorPath apply(Iterable<Object> notUsed) {
+                public ActorSelection apply(Iterable<Object> notUsed) {
                     if(LOG.isDebugEnabled()) {
                         LOG.debug("Tx {} readyTransaction: pending recorded operations succeeded",
                             identifier);
@@ -557,16 +552,9 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                     if(serializedReadyReply.getClass().equals(
                                                      ReadyTransactionReply.SERIALIZABLE_CLASS)) {
                         ReadyTransactionReply reply = ReadyTransactionReply.fromSerializable(
-                                actorContext.getActorSystem(), serializedReadyReply);
-
-                        String resolvedCohortPath = getResolvedCohortPath(
-                                reply.getCohortPath().toString());
+                               serializedReadyReply);
 
-                        if(LOG.isDebugEnabled()) {
-                            LOG.debug("Tx {} readyTransaction: resolved cohort path {}",
-                                identifier, resolvedCohortPath);
-                        }
-                        return actorContext.actorFor(resolvedCohortPath);
+                        return actorContext.actorSelection(reply.getCohortPath());
                     } else {
                         // Throwing an exception here will fail the Future.
 
@@ -805,7 +793,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         }
 
         @Override
-        public Future<ActorPath> readyTransaction() {
+        public Future<ActorSelection> readyTransaction() {
             if(LOG.isDebugEnabled()) {
                 LOG.debug("Tx {} readyTransaction called", identifier);
             }
index 5273dc247925608326f9756e5dcbb061b106e5e2..59dd6db06e81c619f81edafe73a664d0e14189ed 100644 (file)
@@ -8,32 +8,32 @@
 
 package org.opendaylight.controller.cluster.datastore.messages;
 
-import akka.actor.ActorPath;
-import akka.actor.ActorSystem;
 import org.opendaylight.controller.protobuff.messages.transaction.ShardTransactionMessages;
 
 public class ReadyTransactionReply implements SerializableMessage {
   public static final Class SERIALIZABLE_CLASS = ShardTransactionMessages.ReadyTransactionReply.class;
-  private final ActorPath cohortPath;
+  private final String cohortPath;
 
-  public ReadyTransactionReply(ActorPath cohortPath) {
+  public ReadyTransactionReply(String cohortPath) {
 
     this.cohortPath = cohortPath;
   }
 
-  public ActorPath getCohortPath() {
+  public String getCohortPath() {
     return cohortPath;
   }
 
   @Override
   public ShardTransactionMessages.ReadyTransactionReply toSerializable() {
     return ShardTransactionMessages.ReadyTransactionReply.newBuilder()
-        .setActorPath(cohortPath.toString()).build();
+        .setActorPath(cohortPath).build();
   }
 
-  public static ReadyTransactionReply fromSerializable(ActorSystem actorSystem,Object serializable){
-    ShardTransactionMessages.ReadyTransactionReply o = (ShardTransactionMessages.ReadyTransactionReply) serializable;
-    return new ReadyTransactionReply(
-        actorSystem.actorFor(o.getActorPath()).path());
+  public static ReadyTransactionReply fromSerializable(Object serializable) {
+      ShardTransactionMessages.ReadyTransactionReply o =
+          (ShardTransactionMessages.ReadyTransactionReply) serializable;
+
+      return new ReadyTransactionReply(o.getActorPath());
+
   }
 }
index 44f4ef77d7ff057dbea32ebfd9d3404f5b30f862..d8af09c86b6ec01479f038104fdad350170f4991 100644 (file)
@@ -237,34 +237,6 @@ public class ActorContext {
         actorSystem.shutdown();
     }
 
-    /**
-     * @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();
-        String[] primaryPathElements = primaryPath.split("/");
-        builder.append(primaryPathElements[0]).append("//")
-            .append(primaryPathElements[1]).append(primaryPathElements[2]);
-        String[] remotePathElements = localPathOfRemoteActor.split("/");
-        for (int i = 3; i < remotePathElements.length; i++) {
-            builder.append("/").append(remotePathElements[i]);
-        }
-
-        return builder.toString();
-
-    }
-
-    public ActorPath actorFor(String path){
-        return actorSystem.actorFor(path).path();
-    }
-
     public String getCurrentMemberName(){
         return clusterWrapper.getCurrentMemberName();
     }
index 7b826302f588ad76be85f54c9e17bcc1c8dff56a..a718ca7e4c31ef16d6be1fb26e839f3f551450e2 100644 (file)
@@ -8,7 +8,6 @@
 
 package org.opendaylight.controller.cluster.datastore;
 
-import akka.actor.ActorPath;
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
 import akka.actor.Props;
@@ -134,11 +133,10 @@ public class BasicIntegrationTest extends AbstractActorTest {
                             @Override
                             protected ActorSelection match(Object in) {
                                 if (in.getClass().equals(ReadyTransactionReply.SERIALIZABLE_CLASS)) {
-                                    ActorPath cohortPath =
-                                        ReadyTransactionReply.fromSerializable(getSystem(),in)
+                                    String cohortPath =
+                                        ReadyTransactionReply.fromSerializable(in)
                                             .getCohortPath();
-                                    return getSystem()
-                                        .actorSelection(cohortPath);
+                                    return getSystem().actorSelection(cohortPath);
                                 } else {
                                     throw noMatch();
                                 }
index 3c9d857fe81db981d341233e2c7249f06273bee6..b7ac371812df897c5c86d7970cb309049aa2810a 100644 (file)
@@ -4,19 +4,8 @@ import akka.actor.ActorPath;
 import akka.actor.ActorSelection;
 import akka.actor.Props;
 import akka.dispatch.Futures;
-
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.ListenableFuture;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.isA;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.times;
-
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mock;
@@ -33,13 +22,20 @@ import org.opendaylight.controller.cluster.datastore.messages.PreCommitTransacti
 import org.opendaylight.controller.cluster.datastore.messages.SerializableMessage;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
 import org.opendaylight.controller.cluster.datastore.utils.DoNothingActor;
-
 import scala.concurrent.Future;
 
 import java.util.List;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.isA;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
 public class ThreePhaseCommitCohortProxyTest extends AbstractActorTest {
 
     @SuppressWarnings("serial")
@@ -56,28 +52,28 @@ public class ThreePhaseCommitCohortProxyTest extends AbstractActorTest {
         doReturn(getSystem()).when(actorContext).getActorSystem();
     }
 
-    private Future<ActorPath> newCohortPath() {
+    private Future<ActorSelection> newCohort() {
         ActorPath path = getSystem().actorOf(Props.create(DoNothingActor.class)).path();
-        doReturn(mock(ActorSelection.class)).when(actorContext).actorSelection(path);
-        return Futures.successful(path);
+        ActorSelection actorSelection = getSystem().actorSelection(path);
+        return Futures.successful(actorSelection);
     }
 
     private final ThreePhaseCommitCohortProxy setupProxy(int nCohorts) throws Exception {
-        List<Future<ActorPath>> cohortPathFutures = Lists.newArrayList();
+        List<Future<ActorSelection>> cohortFutures = Lists.newArrayList();
         for(int i = 1; i <= nCohorts; i++) {
-            cohortPathFutures.add(newCohortPath());
+            cohortFutures.add(newCohort());
         }
 
-        return new ThreePhaseCommitCohortProxy(actorContext, cohortPathFutures, "txn-1");
+        return new ThreePhaseCommitCohortProxy(actorContext, cohortFutures, "txn-1");
     }
 
     private ThreePhaseCommitCohortProxy setupProxyWithFailedCohortPath()
             throws Exception {
-        List<Future<ActorPath>> cohortPathFutures = Lists.newArrayList();
-        cohortPathFutures.add(newCohortPath());
-        cohortPathFutures.add(Futures.<ActorPath>failed(new TestException()));
+        List<Future<ActorSelection>> cohortFutures = Lists.newArrayList();
+        cohortFutures.add(newCohort());
+        cohortFutures.add(Futures.<ActorSelection>failed(new TestException()));
 
-        return new ThreePhaseCommitCohortProxy(actorContext, cohortPathFutures, "txn-1");
+        return new ThreePhaseCommitCohortProxy(actorContext, cohortFutures, "txn-1");
     }
 
     private void setupMockActorContext(Class<?> requestType, Object... responses) {
index bdcca42d15c45553d6feb8ff6cc9bd7a792795dc..592337f93f2db9cce227fb9db1f76c1dad06d508 100644 (file)
@@ -1,12 +1,12 @@
 package org.opendaylight.controller.cluster.datastore;
 
-import akka.actor.ActorPath;
+import com.google.common.util.concurrent.CheckedFuture;
+
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
 import akka.actor.Props;
 import akka.dispatch.Futures;
 import com.google.common.base.Optional;
-import com.google.common.util.concurrent.CheckedFuture;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentMatcher;
@@ -182,7 +182,7 @@ public class TransactionProxyTest extends AbstractActorTest {
         return argThat(matcher);
     }
 
-    private Future<Object> readyTxReply(ActorPath path) {
+    private Future<Object> readyTxReply(String path) {
         return Futures.successful((Object)new ReadyTransactionReply(path).toSerializable());
     }
 
@@ -227,12 +227,6 @@ public class TransactionProxyTest extends AbstractActorTest {
         doReturn(createTransactionReply(actorRef)).when(mockActorContext).
                 executeOperation(eq(getSystem().actorSelection(actorRef.path())),
                         eqCreateTransaction(memberName, type));
-
-        doReturn(actorRef.path().toString()).when(mockActorContext).resolvePath(
-                anyString(), eq(actorRef.path().toString()));
-
-        doReturn(actorRef.path()).when(mockActorContext).actorFor(actorRef.path().toString());
-
         return actorRef;
     }
 
@@ -631,19 +625,19 @@ public class TransactionProxyTest extends AbstractActorTest {
                 DeleteDataReply.SERIALIZABLE_CLASS);
     }
 
-    private void verifyCohortPathFutures(ThreePhaseCommitCohortProxy proxy,
-            Object... expReplies) throws Exception {
+    private void verifyCohortFutures(ThreePhaseCommitCohortProxy proxy,
+        Object... expReplies) throws Exception {
         assertEquals("getReadyOperationFutures size", expReplies.length,
-                proxy.getCohortPathFutures().size());
+                proxy.getCohortFutures().size());
 
         int i = 0;
-        for( Future<ActorPath> future: proxy.getCohortPathFutures()) {
+        for( Future<ActorSelection> future: proxy.getCohortFutures()) {
             assertNotNull("Ready operation Future is null", future);
 
             Object expReply = expReplies[i++];
-            if(expReply instanceof ActorPath) {
-                ActorPath actual = Await.result(future, Duration.create(5, TimeUnit.SECONDS));
-                assertEquals("Cohort actor path", expReply, actual);
+            if(expReply instanceof ActorSelection) {
+                ActorSelection actual = Await.result(future, Duration.create(5, TimeUnit.SECONDS));
+                assertEquals("Cohort actor path", (ActorSelection) expReply, actual);
             } else {
                 // Expecting exception.
                 try {
@@ -669,7 +663,7 @@ public class TransactionProxyTest extends AbstractActorTest {
         doReturn(writeDataReply()).when(mockActorContext).executeOperationAsync(
                 eq(actorSelection(actorRef)), eqWriteData(nodeToWrite));
 
-        doReturn(readyTxReply(actorRef.path())).when(mockActorContext).executeOperationAsync(
+        doReturn(readyTxReply(actorRef.path().toString())).when(mockActorContext).executeOperationAsync(
                 eq(actorSelection(actorRef)), isA(ReadyTransaction.SERIALIZABLE_CLASS));
 
         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext,
@@ -688,7 +682,7 @@ public class TransactionProxyTest extends AbstractActorTest {
         verifyRecordingOperationFutures(transactionProxy.getRecordedOperationFutures(),
                 WriteDataReply.SERIALIZABLE_CLASS);
 
-        verifyCohortPathFutures(proxy, actorRef.path());
+        verifyCohortFutures(proxy, getSystem().actorSelection(actorRef.path()));
     }
 
     @SuppressWarnings("unchecked")
@@ -704,7 +698,7 @@ public class TransactionProxyTest extends AbstractActorTest {
         doReturn(Futures.failed(new TestException())).when(mockActorContext).
                 executeOperationAsync(eq(actorSelection(actorRef)), eqWriteData(nodeToWrite));
 
-        doReturn(readyTxReply(actorRef.path())).when(mockActorContext).executeOperationAsync(
+        doReturn(readyTxReply(actorRef.path().toString())).when(mockActorContext).executeOperationAsync(
                 eq(actorSelection(actorRef)), isA(ReadyTransaction.SERIALIZABLE_CLASS));
 
         TransactionProxy transactionProxy = new TransactionProxy(mockActorContext,
@@ -723,7 +717,7 @@ public class TransactionProxyTest extends AbstractActorTest {
         verifyRecordingOperationFutures(transactionProxy.getRecordedOperationFutures(),
                 MergeDataReply.SERIALIZABLE_CLASS, TestException.class);
 
-        verifyCohortPathFutures(proxy, TestException.class);
+        verifyCohortFutures(proxy, TestException.class);
     }
 
     @SuppressWarnings("unchecked")
@@ -754,7 +748,7 @@ public class TransactionProxyTest extends AbstractActorTest {
         verifyRecordingOperationFutures(transactionProxy.getRecordedOperationFutures(),
                 MergeDataReply.SERIALIZABLE_CLASS);
 
-        verifyCohortPathFutures(proxy, TestException.class);
+        verifyCohortFutures(proxy, TestException.class);
     }
 
     @Test
@@ -781,7 +775,7 @@ public class TransactionProxyTest extends AbstractActorTest {
 
         ThreePhaseCommitCohortProxy proxy = (ThreePhaseCommitCohortProxy) ready;
 
-        verifyCohortPathFutures(proxy, PrimaryNotFoundException.class);
+        verifyCohortFutures(proxy, PrimaryNotFoundException.class);
     }
 
     @SuppressWarnings("unchecked")
@@ -809,7 +803,7 @@ public class TransactionProxyTest extends AbstractActorTest {
 
         ThreePhaseCommitCohortProxy proxy = (ThreePhaseCommitCohortProxy) ready;
 
-        verifyCohortPathFutures(proxy, IllegalArgumentException.class);
+        verifyCohortFutures(proxy, IllegalArgumentException.class);
     }
 
     @Test
index fa6d0b060f222f28ec723f1b4bc52b2f1ade3aa4..8426b03a37de93242b506df7fd9d4368d24a169f 100644 (file)
@@ -2,12 +2,12 @@ package org.opendaylight.controller.cluster.datastore.utils;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
-import akka.actor.ActorSystem;
 import akka.actor.Props;
 import akka.actor.UntypedActor;
 import akka.japi.Creator;
 import akka.testkit.JavaTestKit;
 import com.google.common.base.Optional;
+
 import org.junit.Test;
 import org.opendaylight.controller.cluster.datastore.AbstractActorTest;
 import org.opendaylight.controller.cluster.datastore.ClusterWrapper;
@@ -26,40 +26,6 @@ import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 
 public class ActorContextTest extends AbstractActorTest{
-    @Test
-    public void testResolvePathForRemoteActor(){
-        ActorContext actorContext =
-            new ActorContext(mock(ActorSystem.class), mock(ActorRef.class),mock(
-                ClusterWrapper.class),
-                mock(Configuration.class));
-
-        String actual = actorContext.resolvePath(
-            "akka.tcp://system@127.0.0.1:2550/user/shardmanager/shard",
-            "akka://system/user/shardmanager/shard/transaction");
-
-        String expected = "akka.tcp://system@127.0.0.1:2550/user/shardmanager/shard/transaction";
-
-        assertEquals(expected, actual);
-    }
-
-    @Test
-    public void testResolvePathForLocalActor(){
-        ActorContext actorContext =
-            new ActorContext(getSystem(), mock(ActorRef.class), mock(ClusterWrapper.class),
-                mock(Configuration.class));
-
-        String actual = actorContext.resolvePath(
-            "akka://system/user/shardmanager/shard",
-            "akka://system/user/shardmanager/shard/transaction");
-
-        String expected = "akka://system/user/shardmanager/shard/transaction";
-
-        assertEquals(expected, actual);
-
-        System.out.println(actorContext
-            .actorFor("akka://system/user/shardmanager/shard/transaction"));
-    }
-
 
     private static class MockShardManager extends UntypedActor {
 
index 3a6de300a0463da6cec1b0a2466375211b5adabd..bab26dfc012b1a620f2e0a619ef3d93d10b0bb2e 100644 (file)
@@ -16,7 +16,6 @@ import javax.ws.rs.Produces;
 import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.MultivaluedMap;
-import javax.ws.rs.core.Response;
 import javax.ws.rs.ext.MessageBodyWriter;
 import javax.ws.rs.ext.Provider;
 import javax.xml.stream.FactoryConfigurationError;
@@ -28,6 +27,8 @@ import org.opendaylight.controller.sal.rest.api.RestconfService;
 import org.opendaylight.controller.sal.restconf.impl.InstanceIdentifierContext;
 import org.opendaylight.controller.sal.restconf.impl.NormalizedNodeContext;
 import org.opendaylight.controller.sal.restconf.impl.RestconfDocumentedException;
+import org.opendaylight.controller.sal.restconf.impl.RestconfError.ErrorTag;
+import org.opendaylight.controller.sal.restconf.impl.RestconfError.ErrorType;
 import org.opendaylight.yangtools.yang.common.QName;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument;
 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
@@ -72,7 +73,9 @@ public class NormalizedNodeXmlBodyWriter implements MessageBodyWriter<Normalized
             WebApplicationException {
         InstanceIdentifierContext pathContext = t.getInstanceIdentifierContext();
         if (t.getData() == null) {
-            throw new RestconfDocumentedException(Response.Status.NOT_FOUND);
+            throw new RestconfDocumentedException(
+                    "Request could not be completed because the relevant data model content does not exist.",
+                    ErrorType.APPLICATION, ErrorTag.DATA_MISSING);
         }
 
         XMLStreamWriter xmlWriter;
index 8dbc5b50ee5dbd8ac374ef7d1452384442923085..3988a495cb15de246c29647bbf042d7a40be8702 100644 (file)
@@ -231,7 +231,7 @@ public class BrokerFacade {
     private CheckedFuture<Void, TransactionCommitFailedException> deleteDataViaTransaction(
             final DOMDataWriteTransaction writeTransaction, final LogicalDatastoreType datastore,
             YangInstanceIdentifier path) {
-        LOG.info("Delete " + datastore.name() + " via Restconf: {}", path);
+        LOG.trace("Delete " + datastore.name() + " via Restconf: {}", path);
         writeTransaction.delete(datastore, path);
         return writeTransaction.submit();
     }
diff --git a/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/NetconfMonitoringServiceImplTest.java b/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/NetconfMonitoringServiceImplTest.java
new file mode 100644 (file)
index 0000000..1b078be
--- /dev/null
@@ -0,0 +1,111 @@
+/*
+ * 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.netconf.impl;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import io.netty.channel.Channel;
+import java.util.List;
+import java.util.Set;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+import org.opendaylight.controller.netconf.api.monitoring.NetconfManagementSession;
+import org.opendaylight.controller.netconf.impl.NetconfServerSession;
+import org.opendaylight.controller.netconf.impl.NetconfServerSessionListener;
+import org.opendaylight.controller.netconf.impl.osgi.NetconfMonitoringServiceImpl;
+import org.opendaylight.controller.netconf.mapping.api.Capability;
+import org.opendaylight.controller.netconf.mapping.api.NetconfOperationProvider;
+import org.opendaylight.controller.netconf.mapping.api.NetconfOperationService;
+import org.opendaylight.controller.netconf.mapping.api.NetconfOperationServiceSnapshot;
+import org.opendaylight.controller.netconf.util.messages.NetconfHelloMessageAdditionalHeader;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.*;
+
+public class NetconfMonitoringServiceImplTest {
+
+    private NetconfMonitoringServiceImpl service;
+
+    @Mock
+    private NetconfOperationProvider operationProvider;
+    @Mock
+    private NetconfManagementSession managementSession;
+    @Mock
+    private NetconfOperationServiceSnapshot snapshot;
+    @Mock
+    private NetconfOperationService operationService;
+
+    @Before
+    public void setUp() throws Exception {
+        MockitoAnnotations.initMocks(this);
+        service = new NetconfMonitoringServiceImpl(operationProvider);
+    }
+
+    @Test
+    public void testSessions() throws Exception {
+        doReturn("sessToStr").when(managementSession).toString();
+        service.onSessionUp(managementSession);
+        List list = Lists.newArrayList(managementSession);
+    }
+
+    @Test(expected = RuntimeException.class)
+    public void testGetSchemas() throws Exception {
+        doThrow(RuntimeException.class).when(operationProvider).openSnapshot(anyString());
+        service.getSchemas();
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void testGetSchemas2() throws Exception {
+        doThrow(Exception.class).when(operationProvider).openSnapshot(anyString());
+        service.getSchemas();
+    }
+
+    @Test
+    public void testGetSchemas3() throws Exception {
+        doReturn("").when(managementSession).toString();
+        Capability cap = mock(Capability.class);
+        Set caps = Sets.newHashSet(cap);
+        Set services = Sets.newHashSet(operationService);
+        doReturn(snapshot).when(operationProvider).openSnapshot(anyString());
+        doReturn(services).when(snapshot).getServices();
+        doReturn(caps).when(operationService).getCapabilities();
+        Optional opt = mock(Optional.class);
+        doReturn(opt).when(cap).getCapabilitySchema();
+        doReturn(true).when(opt).isPresent();
+        doReturn(opt).when(cap).getModuleNamespace();
+        doReturn("namespace").when(opt).get();
+        Optional optRev = Optional.of("rev");
+        doReturn(optRev).when(cap).getRevision();
+        doReturn(Optional.of("modName")).when(cap).getModuleName();
+        doReturn(Optional.of(Lists.newArrayList("loc"))).when(cap).getLocation();
+        doNothing().when(snapshot).close();
+
+        assertNotNull(service.getSchemas());
+        verify(snapshot, times(1)).close();
+
+        NetconfServerSessionListener sessionListener = mock(NetconfServerSessionListener.class);
+        Channel channel = mock(Channel.class);
+        NetconfHelloMessageAdditionalHeader header = new NetconfHelloMessageAdditionalHeader("name", "addr", "2", "tcp", "id");
+        NetconfServerSession sm = new NetconfServerSession(sessionListener, channel, 10, header);
+        doNothing().when(sessionListener).onSessionUp(any(NetconfServerSession.class));
+        sm.sessionUp();
+        service.onSessionUp(sm);
+        assertEquals(1, service.getSessions().getSession().size());
+
+        assertEquals(Long.valueOf(10), service.getSessions().getSession().get(0).getSessionId());
+
+        service.onSessionDown(sm);
+        assertEquals(0, service.getSessions().getSession().size());
+    }
+}
diff --git a/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/mapping/operations/DefaultCloseSessionTest.java b/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/mapping/operations/DefaultCloseSessionTest.java
new file mode 100644 (file)
index 0000000..ae3d656
--- /dev/null
@@ -0,0 +1,42 @@
+/*
+ * 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.netconf.impl.mapping.operations;
+
+import org.junit.Test;
+import org.opendaylight.controller.netconf.api.NetconfDocumentedException;
+import org.opendaylight.controller.netconf.util.xml.XmlElement;
+import org.opendaylight.controller.netconf.util.xml.XmlUtil;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+
+public class DefaultCloseSessionTest {
+    @Test
+    public void testDefaultCloseSession() throws Exception {
+        AutoCloseable res = mock(AutoCloseable.class);
+        doNothing().when(res).close();
+        DefaultCloseSession session = new DefaultCloseSession("", res);
+        Document doc = XmlUtil.newDocument();
+        XmlElement elem = XmlElement.fromDomElement(XmlUtil.readXmlToElement("<elem/>"));
+        session.handleWithNoSubsequentOperations(doc, elem);
+    }
+
+    @Test(expected = NetconfDocumentedException.class)
+    public void testDefaultCloseSession2() throws Exception {
+        AutoCloseable res = mock(AutoCloseable.class);
+        doThrow(NetconfDocumentedException.class).when(res).close();
+        DefaultCloseSession session = new DefaultCloseSession("", res);
+        Document doc = XmlUtil.newDocument();
+        XmlElement elem = XmlElement.fromDomElement(XmlUtil.readXmlToElement("<elem/>"));
+        session.handleWithNoSubsequentOperations(doc, elem);
+    }
+}
diff --git a/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/mapping/operations/DefaultCommitTest.java b/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/mapping/operations/DefaultCommitTest.java
new file mode 100644 (file)
index 0000000..98050de
--- /dev/null
@@ -0,0 +1,76 @@
+/*
+ * 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.netconf.impl.mapping.operations;
+
+import com.google.common.collect.Sets;
+import org.junit.Before;
+import org.junit.Test;
+import org.opendaylight.controller.netconf.api.NetconfDocumentedException;
+import org.opendaylight.controller.netconf.impl.DefaultCommitNotificationProducer;
+import org.opendaylight.controller.netconf.impl.NetconfServerSession;
+import org.opendaylight.controller.netconf.impl.mapping.CapabilityProvider;
+import org.opendaylight.controller.netconf.impl.osgi.NetconfOperationRouter;
+import org.opendaylight.controller.netconf.mapping.api.NetconfOperationChainedExecution;
+import org.opendaylight.controller.netconf.util.test.XmlFileLoader;
+import org.opendaylight.controller.netconf.util.xml.XmlUtil;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+
+import static org.mockito.Mockito.*;
+
+public class DefaultCommitTest {
+
+    private NetconfOperationChainedExecution operation;
+    private Document requestMessage;
+    private NetconfOperationRouter router;
+    private DefaultCommitNotificationProducer notifier;
+    private CapabilityProvider cap;
+    private DefaultCommit commit;
+
+    @Before
+    public void setUp() throws Exception {
+        operation = mock(NetconfOperationChainedExecution.class);
+        doReturn(XmlUtil.newDocument()).when(operation).execute(any(Document.class));
+        router = mock(NetconfOperationRouter.class);
+        doReturn(false).when(operation).isExecutionTermination();
+        notifier = mock(DefaultCommitNotificationProducer.class);
+        doNothing().when(notifier).sendCommitNotification(anyString(), any(Element.class), anySetOf(String.class));
+        cap = mock(CapabilityProvider.class);
+        doReturn(Sets.newHashSet()).when(cap).getCapabilities();
+        Document rpcData = XmlFileLoader.xmlFileToDocument("netconfMessages/editConfig_expectedResult.xml");
+        doReturn(rpcData).when(router).onNetconfMessage(any(Document.class), any(NetconfServerSession.class));
+        commit = new DefaultCommit(notifier, cap, "", router);
+    }
+
+    @Test
+    public void testHandleWithNotification() throws Exception {
+        requestMessage = XmlFileLoader.xmlFileToDocument("netconfMessages/commit.xml");
+        commit.handle(requestMessage, operation);
+        verify(operation, times(1)).execute(requestMessage);
+        verify(notifier, times(1)).sendCommitNotification(anyString(), any(Element.class), anySetOf(String.class));
+    }
+
+    @Test
+    public void testHandleWithoutNotification() throws Exception {
+        requestMessage = XmlFileLoader.xmlFileToDocument("netconfMessages/commit.xml");
+        Element elem = requestMessage.getDocumentElement();
+        elem.setAttribute("notify", "false");
+        commit.handle(requestMessage, operation);
+        verify(operation, times(1)).execute(requestMessage);
+        verify(notifier, never()).sendCommitNotification(anyString(), any(Element.class), anySetOf(String.class));
+    }
+
+    @Test(expected = NetconfDocumentedException.class)
+    public void testHandle() throws Exception {
+        Document rpcData = XmlFileLoader.xmlFileToDocument("netconfMessages/get.xml");
+        doReturn(rpcData).when(router).onNetconfMessage(any(Document.class), any(NetconfServerSession.class));
+        requestMessage = XmlFileLoader.xmlFileToDocument("netconfMessages/commit.xml");
+        commit.handle(requestMessage, operation);
+    }
+}
diff --git a/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/mapping/operations/DefaultGetSchemaTest.java b/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/mapping/operations/DefaultGetSchemaTest.java
new file mode 100644 (file)
index 0000000..b655e90
--- /dev/null
@@ -0,0 +1,60 @@
+/*
+ * 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.netconf.impl.mapping.operations;
+
+import com.google.common.base.Optional;
+import junit.framework.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.opendaylight.controller.netconf.api.NetconfDocumentedException;
+import org.opendaylight.controller.netconf.impl.mapping.CapabilityProvider;
+import org.opendaylight.controller.netconf.util.xml.XmlElement;
+import org.opendaylight.controller.netconf.util.xml.XmlUtil;
+import org.w3c.dom.Document;
+
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+
+public class DefaultGetSchemaTest {
+
+    private CapabilityProvider cap;
+    private Document doc;
+    private String getSchema;
+
+    @Before
+    public void setUp() throws Exception {
+        cap = mock(CapabilityProvider.class);
+        doc = XmlUtil.newDocument();
+        getSchema = "<get-schema xmlns=\"urn:ietf:params:xml:ns:yang:ietf-netconf-monitoring\">\n" +
+                "        <identifier>threadpool-api</identifier>\n" +
+                "        <version>2010-09-24</version>\n" +
+                "        <format\n" +
+                "                xmlns:ncm=\"urn:ietf:params:xml:ns:yang:ietf-netconf-monitoring\">ncm:yang\n" +
+                "        </format>\n" +
+                "    </get-schema>";
+    }
+
+    @Test(expected = NetconfDocumentedException.class)
+    public void testDefaultGetSchema() throws Exception {
+        DefaultGetSchema schema = new DefaultGetSchema(cap, "");
+        doThrow(IllegalStateException.class).when(cap).getSchemaForCapability(anyString(), any(Optional.class));
+        schema.handleWithNoSubsequentOperations(doc, XmlElement.fromDomElement(XmlUtil.readXmlToElement(getSchema)));
+    }
+
+    @Test
+    public void handleWithNoSubsequentOperations() throws Exception {
+        DefaultGetSchema schema = new DefaultGetSchema(cap, "");
+        doReturn("").when(cap).getSchemaForCapability(anyString(), any(Optional.class));
+        assertNotNull(schema.handleWithNoSubsequentOperations(doc, XmlElement.fromDomElement(XmlUtil.readXmlToElement(getSchema))));
+    }
+}
diff --git a/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/mapping/operations/DefaultStopExiTest.java b/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/mapping/operations/DefaultStopExiTest.java
new file mode 100644 (file)
index 0000000..b335165
--- /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.netconf.impl.mapping.operations;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelPipeline;
+import org.junit.Test;
+import org.opendaylight.controller.netconf.impl.NetconfServerSession;
+import org.opendaylight.controller.netconf.util.xml.XmlElement;
+import org.opendaylight.controller.netconf.util.xml.XmlUtil;
+import org.w3c.dom.Document;
+
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Mockito.*;
+
+public class DefaultStopExiTest {
+    @Test
+    public void testHandleWithNoSubsequentOperations() throws Exception {
+        DefaultStopExi exi = new DefaultStopExi("");
+        Document doc = XmlUtil.newDocument();
+        Channel channel = mock(Channel.class);
+        ChannelPipeline pipeline = mock(ChannelPipeline.class);
+        doReturn(pipeline).when(channel).pipeline();
+        ChannelHandler channelHandler = mock(ChannelHandler.class);
+        doReturn(channelHandler).when(pipeline).replace(anyString(), anyString(), any(ChannelHandler.class));
+
+        NetconfServerSession serverSession = new NetconfServerSession(null, channel, 2L, null);
+        exi.setNetconfSession(serverSession);
+
+        assertNotNull(exi.handleWithNoSubsequentOperations(doc, XmlElement.fromDomElement(XmlUtil.readXmlToElement("<elem/>"))));
+        verify(pipeline, times(1)).replace(anyString(), anyString(), any(ChannelHandler.class));
+    }
+}
diff --git a/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/osgi/NetconfImplActivatorTest.java b/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/osgi/NetconfImplActivatorTest.java
new file mode 100644 (file)
index 0000000..b59ea88
--- /dev/null
@@ -0,0 +1,58 @@
+/*
+ * 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.netconf.impl.osgi;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Dictionary;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+import org.opendaylight.controller.netconf.api.monitoring.NetconfMonitoringService;
+import org.opendaylight.controller.netconf.mapping.api.NetconfOperationProvider;
+import org.osgi.framework.*;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.*;
+
+public class NetconfImplActivatorTest {
+
+    @Mock
+    private BundleContext bundle;
+    @Mock
+    private Filter filter;
+    @Mock
+    private ServiceReference reference;
+    @Mock
+    private ServiceRegistration registration;
+
+    @Before
+    public void setUp() throws Exception {
+        MockitoAnnotations.initMocks(this);
+        doReturn(filter).when(bundle).createFilter(anyString());
+        doNothing().when(bundle).addServiceListener(any(ServiceListener.class), anyString());
+
+        ServiceReference[] refs = new ServiceReference[0];
+        doReturn(refs).when(bundle).getServiceReferences(anyString(), anyString());
+        doReturn(Arrays.asList(refs)).when(bundle).getServiceReferences(any(Class.class), anyString());
+        doReturn("").when(bundle).getProperty(anyString());
+        doReturn(registration).when(bundle).registerService(any(Class.class), any(NetconfOperationServiceFactoryListenerImpl.class), any(Dictionary.class));
+        doNothing().when(registration).unregister();
+        doNothing().when(bundle).removeServiceListener(any(ServiceListener.class));
+    }
+
+    @Test
+    public void testStart() throws Exception {
+        NetconfImplActivator activator = new NetconfImplActivator();
+        activator.start(bundle);
+        verify(bundle, times(2)).registerService(any(Class.class), any(NetconfOperationServiceFactoryListenerImpl.class), any(Dictionary.class));
+        activator.stop(bundle);
+    }
+}
diff --git a/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/osgi/NetconfOperationServiceFactoryTrackerTest.java b/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/osgi/NetconfOperationServiceFactoryTrackerTest.java
new file mode 100644 (file)
index 0000000..374e8ae
--- /dev/null
@@ -0,0 +1,62 @@
+/*
+ * 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.netconf.impl.osgi;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+import org.opendaylight.controller.netconf.mapping.api.NetconfOperationServiceFactory;
+import org.osgi.framework.BundleContext;
+import org.osgi.framework.Filter;
+import org.osgi.framework.ServiceReference;
+
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.*;
+
+public class NetconfOperationServiceFactoryTrackerTest {
+
+    @Mock
+    private Filter filter;
+    @Mock
+    private BundleContext context;
+    @Mock
+    private NetconfOperationServiceFactoryListener listener;
+    @Mock
+    private NetconfOperationServiceFactory factory;
+    @Mock
+    private ServiceReference reference;
+
+    private NetconfOperationServiceFactoryTracker tracker;
+
+    @Before
+    public void setUp() throws Exception {
+        MockitoAnnotations.initMocks(this);
+        doNothing().when(listener).onRemoveNetconfOperationServiceFactory(any(NetconfOperationServiceFactory.class));
+        doReturn(filter).when(context).createFilter(anyString());
+        doReturn("").when(reference).toString();
+        doReturn(factory).when(context).getService(any(ServiceReference.class));
+        doReturn("").when(factory).toString();
+        doNothing().when(listener).onAddNetconfOperationServiceFactory(any(NetconfOperationServiceFactory.class));
+        tracker = new NetconfOperationServiceFactoryTracker(context, listener);
+    }
+
+    @Test
+    public void testNetconfOperationServiceFactoryTracker() throws Exception {
+        tracker.removedService(null, factory);
+        verify(listener, times(1)).onRemoveNetconfOperationServiceFactory(any(NetconfOperationServiceFactory.class));
+    }
+
+    @Test
+    public void testAddingService() throws Exception {
+        assertNotNull(tracker.addingService(reference));
+        verify(listener, times(1)).onAddNetconfOperationServiceFactory(any(NetconfOperationServiceFactory.class));
+    }
+}
diff --git a/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/util/DeserializerExceptionHandlerTest.java b/opendaylight/netconf/netconf-impl/src/test/java/org/opendaylight/controller/netconf/impl/util/DeserializerExceptionHandlerTest.java
new file mode 100644 (file)
index 0000000..6512b4b
--- /dev/null
@@ -0,0 +1,43 @@
+/*
+ * 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.netconf.impl.util;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.util.concurrent.GenericFutureListener;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.mockito.Mockito.*;
+
+public class DeserializerExceptionHandlerTest {
+
+    private DeserializerExceptionHandler handler;
+    private ChannelFuture channelFuture;
+    private ChannelHandlerContext context;
+    private Channel channel;
+
+    @Before
+    public void setUp() throws Exception {
+        handler = new DeserializerExceptionHandler();
+        context = mock(ChannelHandlerContext.class);
+        channel = mock(Channel.class);
+        doReturn(channel).when(context).channel();
+        channelFuture = mock(ChannelFuture.class);
+        doReturn(channelFuture).when(channelFuture).addListener(any(GenericFutureListener.class));
+        doReturn(channelFuture).when(channel).writeAndFlush(anyObject());
+    }
+
+    @Test
+    public void testExceptionCaught() throws Exception {
+        handler.exceptionCaught(context, new Exception());
+        verify(context, times(1)).channel();
+    }
+}
diff --git a/opendaylight/netconf/netconf-monitoring/src/test/java/org/opendaylight/controller/netconf/monitoring/osgi/NetconfMonitoringActivatorTest.java b/opendaylight/netconf/netconf-monitoring/src/test/java/org/opendaylight/controller/netconf/monitoring/osgi/NetconfMonitoringActivatorTest.java
new file mode 100644 (file)
index 0000000..4049356
--- /dev/null
@@ -0,0 +1,51 @@
+/*
+ * 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.netconf.monitoring.osgi;
+
+import java.util.Arrays;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+import org.osgi.framework.BundleContext;
+import org.osgi.framework.Filter;
+import org.osgi.framework.ServiceListener;
+import org.osgi.framework.ServiceReference;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.*;
+
+public class NetconfMonitoringActivatorTest {
+
+    @Mock
+    BundleContext context;
+    @Mock
+    Filter filter;
+
+    @Before
+    public void setUp() throws Exception {
+        MockitoAnnotations.initMocks(this);
+        doReturn(filter).when(context).createFilter(anyString());
+        doNothing().when(context).addServiceListener(any(ServiceListener.class), anyString());
+        ServiceReference[] refs = new ServiceReference[2];
+        doReturn(Arrays.asList(refs)).when(context).getServiceReferences(any(Class.class), anyString());
+        doReturn(refs).when(context).getServiceReferences(anyString(), anyString());
+    }
+
+    @Test
+    public void testNetconfMonitoringActivator() throws Exception {
+        NetconfMonitoringActivator activator = new NetconfMonitoringActivator();
+        activator.start(context);
+        verify(context, times(1)).addServiceListener(any(ServiceListener.class), anyString());
+
+        activator.stop(context);
+        verify(context, times(1)).removeServiceListener(any(ServiceListener.class));
+    }
+}
diff --git a/opendaylight/netconf/netconf-monitoring/src/test/java/org/opendaylight/controller/netconf/monitoring/osgi/NetconfMonitoringOperationServiceTest.java b/opendaylight/netconf/netconf-monitoring/src/test/java/org/opendaylight/controller/netconf/monitoring/osgi/NetconfMonitoringOperationServiceTest.java
new file mode 100644 (file)
index 0000000..b8e35e9
--- /dev/null
@@ -0,0 +1,34 @@
+/*
+ * 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.netconf.monitoring.osgi;
+
+import com.google.common.base.Optional;
+import org.junit.Test;
+import org.opendaylight.controller.netconf.api.monitoring.NetconfMonitoringService;
+import org.opendaylight.controller.netconf.monitoring.MonitoringConstants;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+
+public class NetconfMonitoringOperationServiceTest {
+    @Test
+    public void testGetters() throws Exception {
+        NetconfMonitoringService monitor = mock(NetconfMonitoringService.class);
+        NetconfMonitoringOperationService service = new NetconfMonitoringOperationService(monitor);
+
+        assertEquals(1, service.getNetconfOperations().size());
+
+        assertEquals(Optional.absent(), service.getCapabilities().iterator().next().getCapabilitySchema());
+        assertEquals(Optional.absent(), service.getCapabilities().iterator().next().getLocation());
+        assertEquals(Optional.of(MonitoringConstants.MODULE_REVISION), service.getCapabilities().iterator().next().getRevision());
+        assertEquals(Optional.of(MonitoringConstants.MODULE_NAME), service.getCapabilities().iterator().next().getModuleName());
+        assertEquals(Optional.of(MonitoringConstants.NAMESPACE), service.getCapabilities().iterator().next().getModuleNamespace());
+        assertEquals(MonitoringConstants.URI, service.getCapabilities().iterator().next().getCapabilityUri());
+    }
+}
diff --git a/opendaylight/netconf/netconf-monitoring/src/test/java/org/opendaylight/controller/netconf/monitoring/osgi/NetconfMonitoringServiceTrackerTest.java b/opendaylight/netconf/netconf-monitoring/src/test/java/org/opendaylight/controller/netconf/monitoring/osgi/NetconfMonitoringServiceTrackerTest.java
new file mode 100644 (file)
index 0000000..2a53a6c
--- /dev/null
@@ -0,0 +1,58 @@
+/*
+ * 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.netconf.monitoring.osgi;
+
+import java.util.Hashtable;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+import org.opendaylight.controller.netconf.api.monitoring.NetconfMonitoringService;
+import org.opendaylight.controller.netconf.mapping.api.NetconfOperationServiceFactory;
+import org.osgi.framework.BundleContext;
+import org.osgi.framework.Filter;
+import org.osgi.framework.ServiceReference;
+import org.osgi.framework.ServiceRegistration;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyCollection;
+import static org.mockito.Mockito.*;
+
+public class NetconfMonitoringServiceTrackerTest {
+
+    @Mock
+    private ServiceReference reference;
+    @Mock
+    private BundleContext context;
+    @Mock
+    private ServiceRegistration serviceRegistration;
+    @Mock
+    private Filter filter;
+    @Mock
+    private NetconfMonitoringService monitoringService;
+
+    @Before
+    public void setUp() throws Exception {
+        MockitoAnnotations.initMocks(this);
+        doReturn(serviceRegistration).when(context).registerService(any(Class.class), any(NetconfOperationServiceFactory.class), any(Hashtable.class));
+        doNothing().when(serviceRegistration).unregister();
+        doReturn(filter).when(context).createFilter(anyString());
+        doReturn("").when(reference).toString();
+        doReturn(monitoringService).when(context).getService(any(ServiceReference.class));
+    }
+
+    @Test
+    public void testAddingService() throws Exception {
+        NetconfMonitoringServiceTracker tracker = new NetconfMonitoringServiceTracker(context);
+        tracker.addingService(reference);
+        verify(context, times(1)).registerService(any(Class.class), any(NetconfOperationServiceFactory.class), any(Hashtable.class));
+        tracker.removedService(reference, null);
+        verify(serviceRegistration, times(1)).unregister();
+    }
+}
index 0f11284e53a4bce1db970d66ccc1a86cf024dc67..deecf26e0d99227069114722f4b92addf7c136e1 100644 (file)
@@ -77,7 +77,7 @@ public class Activator extends ComponentActivatorAbstractBase {
             c.add(createContainerServiceDependency(containerName).setService(
                     IInventoryService.class).setCallbacks(
                     "setInventoryService", "unsetInventoryService")
-                    .setRequired(false));
+                    .setRequired(true));
             c.add(createContainerServiceDependency(containerName).setService(
                     IStatisticsManager.class).setCallbacks(
                     "setStatisticsManager", "unsetStatisticsManager")