Merge "Bug 6146 - Some Flows not found in DeviceFlowRegistry"
authorAbhijit Kumbhare <abhijit.kumbhare@ericsson.com>
Thu, 28 Jul 2016 20:34:37 +0000 (20:34 +0000)
committerGerrit Code Review <gerrit@opendaylight.org>
Thu, 28 Jul 2016 20:34:37 +0000 (20:34 +0000)
77 files changed:
applications/forwardingrules-sync/pom.xml
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/ForwardingRulesAddCommitter.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/ForwardingRulesRemoveCommitter.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/ForwardingRulesUpdateCommitter.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/NodeListener.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/SyncReactor.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/dao/FlowCapableNodeOdlDao.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/AbstractFrmSyncListener.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/ForwardingRulesSyncProvider.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/SimplifiedConfigListener.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/SimplifiedOperationalListener.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/SimplifiedOperationalRetryListener.java [deleted file]
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/SyncReactorClusterDecorator.java [new file with mode: 0644]
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/SyncReactorFutureDecorator.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/SyncReactorFutureZipDecorator.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/SyncReactorGuardDecorator.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/SyncReactorImpl.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/SyncReactorRetryDecorator.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/clustering/DeviceMastership.java [new file with mode: 0644]
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/clustering/DeviceMastershipManager.java [new file with mode: 0644]
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/strategy/FlowForwarder.java [moved from applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/FlowForwarder.java with 97% similarity]
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/strategy/GroupForwarder.java [moved from applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/GroupForwarder.java with 98% similarity]
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/strategy/MeterForwarder.java [moved from applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/MeterForwarder.java with 98% similarity]
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/strategy/SyncPlanPushStrategyFlatBatchImpl.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/strategy/SyncPlanPushStrategyIncrementalImpl.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/strategy/TableForwarder.java [moved from applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/TableForwarder.java with 97% similarity]
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/FlowCapableNodeLookups.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/FxChainUtil.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/ItemSyncBox.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/ModificationUtil.java [new file with mode: 0644]
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/ReconcileUtil.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/ReconciliationRegistry.java [moved from applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/RetryRegistry.java with 78% similarity]
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/SemaphoreKeeperGuavaImpl.java
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/SwitchFlowId.java [moved from applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/markandsweep/SwitchFlowId.java with 96% similarity]
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/SyncupEntry.java [new file with mode: 0644]
applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/ZipQueueEntry.java [deleted file]
applications/forwardingrules-sync/src/main/resources/org/opendaylight/blueprint/forwardingrules-sync.xml
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/ForwardingRulesSyncProviderTest.java
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/SimplifiedConfigListenerTest.java
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/SimplifiedOperationalListenerTest.java
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/SimplifiedOperationalRetryListenerTest.java [deleted file]
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/SyncReactorClusterDecoratorTest.java [new file with mode: 0644]
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/SyncReactorFutureZipDecoratorTest.java
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/SyncReactorGuardDecoratorTest.java
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/SyncReactorImplTest.java
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/SyncReactorRetryDecoratorTest.java
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/clustering/DeviceMastershipManagerTest.java [new file with mode: 0644]
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/clustering/DeviceMastershipTest.java [new file with mode: 0644]
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/strategy/FlowForwarderTest.java [moved from applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/FlowForwarderTest.java with 99% similarity]
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/strategy/GroupForwarderTest.java [moved from applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/GroupForwarderTest.java with 99% similarity]
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/strategy/MeterForwarderTest.java [moved from applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/MeterForwarderTest.java with 99% similarity]
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/strategy/SyncPlanPushStrategyFlatBatchImplTest.java
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/strategy/SyncPlanPushStrategyIncrementalImplTest.java
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/strategy/TableForwarderTest.java [moved from applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/TableForwarderTest.java with 99% similarity]
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/util/ReconcileUtilTest.java
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/util/ReconciliationRegistryTest.java [moved from applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/util/RetryRegistryTest.java with 60% similarity]
applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/util/SemaphoreKeeperTest.java
applications/topology-manager/src/main/java/org/opendaylight/openflowplugin/applications/topology/manager/FlowCapableNodeMapping.java
applications/topology-manager/src/main/java/org/opendaylight/openflowplugin/applications/topology/manager/FlowCapableTopologyExporter.java
applications/topology-manager/src/main/java/org/opendaylight/openflowplugin/applications/topology/manager/NodeChangeListenerImpl.java
applications/topology-manager/src/main/java/org/opendaylight/openflowplugin/applications/topology/manager/OperationProcessor.java
applications/topology-manager/src/main/java/org/opendaylight/openflowplugin/applications/topology/manager/TopologyManagerUtil.java
applications/topology-manager/src/test/java/org/opendaylight/openflowplugin/applications/topology/manager/DataChangeListenerBase.java
applications/topology-manager/src/test/java/org/opendaylight/openflowplugin/applications/topology/manager/FlowCapableTopologyExporterTest.java
applications/topology-manager/src/test/java/org/opendaylight/openflowplugin/applications/topology/manager/NodeChangeListenerImplTest.java
applications/topology-manager/src/test/java/org/opendaylight/openflowplugin/applications/topology/manager/OperationProcessorTest.java
applications/topology-manager/src/test/java/org/opendaylight/openflowplugin/applications/topology/manager/TerminationPointChangeListenerImplTest.java
openflowplugin-api/src/main/java/org/opendaylight/openflowplugin/api/OFConstants.java
openflowplugin-impl/src/main/java/org/opendaylight/openflowplugin/impl/registry/flow/DeviceFlowRegistryImpl.java
openflowplugin-impl/src/main/java/org/opendaylight/openflowplugin/impl/registry/flow/FlowDescriptorFactory.java
openflowplugin-impl/src/main/java/org/opendaylight/openflowplugin/impl/registry/flow/FlowRegistryKeyFactory.java
openflowplugin-impl/src/main/java/org/opendaylight/openflowplugin/impl/services/SalFlowServiceImpl.java
openflowplugin-impl/src/main/java/org/opendaylight/openflowplugin/impl/statistics/SinglePurposeMultipartReplyTranslator.java
openflowplugin-impl/src/main/java/org/opendaylight/openflowplugin/impl/util/FlowUtil.java
openflowplugin-impl/src/test/java/org/opendaylight/openflowplugin/impl/registry/flow/DeviceFlowRegistryImplTest.java
openflowplugin-impl/src/test/java/org/opendaylight/openflowplugin/impl/services/SalFlowServiceImplTest.java
openflowplugin-impl/src/test/java/org/opendaylight/openflowplugin/impl/util/FlowUtilTest.java

index 067a462f2ee9dccdbbc66a5a3de49400a003ea3d..02b725fc27beccbc72ad3160ac6c22914aad3ebe 100644 (file)
             <artifactId>model-flow-service</artifactId>
         </dependency>
 
-        <dependency>
-            <groupId>org.opendaylight.openflowplugin</groupId>
-            <artifactId>openflowplugin-common</artifactId>
-        </dependency>
-
         <dependency>
             <groupId>org.opendaylight.yangtools</groupId>
             <artifactId>yang-common</artifactId>
             <artifactId>sal-common-util</artifactId>
         </dependency>
 
+        <dependency>
+            <groupId>org.opendaylight.mdsal</groupId>
+            <artifactId>mdsal-singleton-common-api</artifactId>
+        </dependency>
+
         <dependency>
             <groupId>com.google.code.findbugs</groupId>
             <artifactId>jsr305</artifactId>
index 7b227fd8153423a0e3e65bf8dc0b4c1a52304379..11e399c62e22bb409384f01eeae77f4e32abcabd 100644 (file)
@@ -26,6 +26,7 @@ public interface ForwardingRulesAddCommitter<D extends DataObject, A extends Dat
      * @param identifier - the whole path to new DataObject
      * @param add        - new DataObject
      * @param nodeIdent  - Node InstanceIdentifier
+     * @return RpcResult of action
      */
     Future<RpcResult<A>> add(InstanceIdentifier<D> identifier, D add, InstanceIdentifier<FlowCapableNode> nodeIdent);
 
index f21bf98a39ae2255926a7cbbbcd0dbb682079613..04f6554ad1fc7a8b4c3565c79ad833be5e6278ee 100644 (file)
@@ -26,6 +26,7 @@ public interface ForwardingRulesRemoveCommitter<D extends DataObject, R extends
      * @param identifier - the whole path to DataObject
      * @param del        - DataObject for removing
      * @param nodeIdent  - Node InstanceIdentifier
+     * @return RpcResult of action
      */
     Future<RpcResult<R>> remove(InstanceIdentifier<D> identifier, D del, InstanceIdentifier<FlowCapableNode> nodeIdent);
 
index d1b3eead9ac6f074065f4d932d3781f8b66e41be..3302a28ad62b7a8333d4db18a9bcf95763d97452 100644 (file)
@@ -27,6 +27,7 @@ public interface ForwardingRulesUpdateCommitter<D extends DataObject, U extends
      * @param original   - original DataObject (for update)
      * @param update     - changed DataObject (contain updates)
      * @param nodeIdent  - Node InstanceIdentifier
+     * @return RpcResult of action
      */
     Future<RpcResult<U>> update(InstanceIdentifier<D> identifier, D original, D update,
                                 InstanceIdentifier<FlowCapableNode> nodeIdent);
index f3bc24928f31b93f4188d706568c66ae0c9883fd..05ca0c20f6ee73838a4b4ea454dc26055ad308a6 100644 (file)
@@ -9,11 +9,11 @@
 package org.opendaylight.openflowplugin.applications.frsync;
 
 import java.util.EventListener;
-import org.opendaylight.controller.md.sal.binding.api.DataTreeChangeListener;
+import org.opendaylight.controller.md.sal.binding.api.ClusteredDataTreeChangeListener;
 import org.opendaylight.yangtools.yang.binding.DataObject;
 
 /**
  * Unifying listener for data and event changes on node.
  */
-public interface NodeListener<T extends DataObject> extends EventListener, DataTreeChangeListener<T> {
+public interface NodeListener<T extends DataObject> extends EventListener, ClusteredDataTreeChangeListener<T> {
 }
index cf92bbae60840ff5b1111f0c066ef9a23a6084be..d38d7844323b0e370ab6e3186192b35a00e2b769 100644 (file)
@@ -9,7 +9,7 @@
 package org.opendaylight.openflowplugin.applications.frsync;
 
 import com.google.common.util.concurrent.ListenableFuture;
-import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 
@@ -19,12 +19,10 @@ import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 public interface SyncReactor {
     /**
      * @param flowcapableNodePath path to openflow augmentation of node
-     * @param configTree configured node
-     * @param operationalTree device reflection
-     * @param dsType type of DS change
+     * @param syncupEntry configured node + device reflection
      * @return synchronization outcome
+     * @throws InterruptedException
      */
     ListenableFuture<Boolean> syncup(final InstanceIdentifier<FlowCapableNode> flowcapableNodePath,
-                                     final FlowCapableNode configTree, final FlowCapableNode operationalTree,
-                                     final LogicalDatastoreType dsType) throws InterruptedException;
+                                     final SyncupEntry syncupEntry) throws InterruptedException;
 }
index 73467086f47cfd969836ba3210037ebd44c07a7c..062f1d4299aafe506212b199ef9ea1dfab780a3d 100644 (file)
@@ -47,7 +47,7 @@ public class FlowCapableNodeOdlDao implements FlowCapableNodeDao {
                     NODES_IID.child(Node.class, new NodeKey(nodeId)).augmentation(FlowCapableNode.class);
             return roTx.read(logicalDatastoreType, path).checkedGet(5000, TimeUnit.MILLISECONDS);
         } catch (ReadFailedException | TimeoutException e) {
-            LOG.error("error reading {}", nodeId, e);
+            LOG.error("error reading {} -> {}", nodeId.getValue(), e);
         }
 
         return Optional.absent();
index aaf4063bd0f21a6e609d686404383a90a8a7223f..744b531e1f61317cc3f6cd3aed2de2e053cec919 100644 (file)
@@ -38,7 +38,7 @@ public abstract class AbstractFrmSyncListener<T extends DataObject> implements N
                 if (optFuture.isPresent()) {
                     final ListenableFuture<Boolean> future = optFuture.get();
                     final Boolean ret = future.get(15000, TimeUnit.MILLISECONDS);
-                    LOG.debug("syncup ret {} {} {} thread:{}", dsType(), ret, nodeId.getValue(), threadName());
+                    LOG.debug("syncup return in {} listener for: {} [{}] thread:{}", dsType(), nodeId.getValue(), ret, threadName());
                 }
             } catch (InterruptedException e) {
                 LOG.warn("permit for forwarding rules sync not acquired: {}", nodeId.getValue());
index 0e5ae5e568439bec77f662be6b8920c537ab698e..996f3ef4c06d3a40e81d82fc32d831d8e32474a2 100644 (file)
@@ -14,13 +14,14 @@ import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
 import org.opendaylight.controller.md.sal.binding.api.DataBroker;
 import org.opendaylight.controller.md.sal.binding.api.DataTreeIdentifier;
 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.controller.sal.binding.api.BindingAwareBroker;
+import org.opendaylight.controller.sal.binding.api.BindingAwareBroker.ProviderContext;
 import org.opendaylight.controller.sal.binding.api.BindingAwareProvider;
 import org.opendaylight.controller.sal.binding.api.RpcConsumerRegistry;
+import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceProvider;
 import org.opendaylight.openflowplugin.applications.frsync.NodeListener;
 import org.opendaylight.openflowplugin.applications.frsync.SyncPlanPushStrategy;
 import org.opendaylight.openflowplugin.applications.frsync.SyncReactor;
@@ -28,8 +29,10 @@ import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeCa
 import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeDao;
 import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeOdlDao;
 import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeSnapshotDao;
+import org.opendaylight.openflowplugin.applications.frsync.impl.clustering.DeviceMastershipManager;
 import org.opendaylight.openflowplugin.applications.frsync.impl.strategy.SyncPlanPushStrategyFlatBatchImpl;
-import org.opendaylight.openflowplugin.applications.frsync.util.RetryRegistry;
+import org.opendaylight.openflowplugin.applications.frsync.impl.strategy.TableForwarder;
+import org.opendaylight.openflowplugin.applications.frsync.util.ReconciliationRegistry;
 import org.opendaylight.openflowplugin.applications.frsync.util.SemaphoreKeeperGuavaImpl;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flat.batch.service.rev160321.SalFlatBatchService;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
@@ -49,6 +52,7 @@ public class ForwardingRulesSyncProvider implements AutoCloseable, BindingAwareP
     private static final Logger LOG = LoggerFactory.getLogger(ForwardingRulesSyncProvider.class);
 
     private final DataBroker dataService;
+    private final ClusterSingletonServiceProvider clusterSingletonService;
     private final SalTableService salTableService;
     private final SalFlatBatchService flatBatchService;
 
@@ -69,9 +73,12 @@ public class ForwardingRulesSyncProvider implements AutoCloseable, BindingAwareP
 
     public ForwardingRulesSyncProvider(final BindingAwareBroker broker,
                                        final DataBroker dataBroker,
-                                       final RpcConsumerRegistry rpcRegistry) {
-        Preconditions.checkArgument(rpcRegistry != null, "RpcConsumerRegistry can not be null !");
+                                       final RpcConsumerRegistry rpcRegistry,
+                                       final ClusterSingletonServiceProvider clusterSingletonService) {
+        Preconditions.checkArgument(rpcRegistry != null, "RpcConsumerRegistry can not be null!");
         this.dataService = Preconditions.checkNotNull(dataBroker, "DataBroker can not be null!");
+        this.clusterSingletonService = Preconditions.checkNotNull(clusterSingletonService,
+                "ClusterSingletonServiceProvider can not be null!");
         this.salTableService = Preconditions.checkNotNull(rpcRegistry.getRpcService(SalTableService.class),
                 "RPC SalTableService not found.");
         this.flatBatchService = Preconditions.checkNotNull(rpcRegistry.getRpcService(SalFlatBatchService.class),
@@ -86,26 +93,28 @@ public class ForwardingRulesSyncProvider implements AutoCloseable, BindingAwareP
                 .setUncaughtExceptionHandler((thread, e) -> LOG.error("Uncaught exception {}", thread, e))
                 .build());
         syncThreadPool = MoreExecutors.listeningDecorator(executorService);
-
         broker.registerProvider(this);
     }
 
     @Override
-    public void onSessionInitiated(final BindingAwareBroker.ProviderContext providerContext) {
+    public void onSessionInitiated(final ProviderContext providerContext) {
         final TableForwarder tableForwarder = new TableForwarder(salTableService);
 
         final SyncPlanPushStrategy syncPlanPushStrategy = new SyncPlanPushStrategyFlatBatchImpl()
                 .setFlatBatchService(flatBatchService)
                 .setTableForwarder(tableForwarder);
 
-        final RetryRegistry retryRegistry = new RetryRegistry();
+        final ReconciliationRegistry reconciliationRegistry = new ReconciliationRegistry();
+        final DeviceMastershipManager deviceMastershipManager =
+                new DeviceMastershipManager(clusterSingletonService, reconciliationRegistry);
 
         final SyncReactor syncReactorImpl = new SyncReactorImpl(syncPlanPushStrategy);
-        final SyncReactor syncReactorRetry = new SyncReactorRetryDecorator(syncReactorImpl, retryRegistry);
+        final SyncReactor syncReactorRetry = new SyncReactorRetryDecorator(syncReactorImpl, reconciliationRegistry);
         final SyncReactor syncReactorGuard = new SyncReactorGuardDecorator(syncReactorRetry,
-                new SemaphoreKeeperGuavaImpl<InstanceIdentifier<FlowCapableNode>>(1, true));
+                new SemaphoreKeeperGuavaImpl<>(1, true));
+        final SyncReactor syncReactorFutureZip = new SyncReactorFutureZipDecorator(syncReactorGuard, syncThreadPool);
 
-        final SyncReactor reactor = new SyncReactorFutureZipDecorator(syncReactorGuard, syncThreadPool);
+        final SyncReactor reactor = new SyncReactorClusterDecorator(syncReactorFutureZip, deviceMastershipManager);
 
         final FlowCapableNodeSnapshotDao configSnapshot = new FlowCapableNodeSnapshotDao();
         final FlowCapableNodeSnapshotDao operationalSnapshot = new FlowCapableNodeSnapshotDao();
@@ -117,7 +126,7 @@ public class ForwardingRulesSyncProvider implements AutoCloseable, BindingAwareP
         final NodeListener<FlowCapableNode> nodeListenerConfig =
                 new SimplifiedConfigListener(reactor, configSnapshot, operationalDao);
         final NodeListener<Node> nodeListenerOperational =
-                new SimplifiedOperationalRetryListener(reactor, operationalSnapshot, configDao, retryRegistry);
+                new SimplifiedOperationalListener(reactor, operationalSnapshot, configDao, reconciliationRegistry, deviceMastershipManager);
 
         dataTreeConfigChangeListener =
                 dataService.registerDataTreeChangeListener(nodeConfigDataTreePath, nodeListenerConfig);
@@ -140,4 +149,5 @@ public class ForwardingRulesSyncProvider implements AutoCloseable, BindingAwareP
 
         syncThreadPool.shutdown();
     }
+
 }
index 8cad7e50ed5b1ac36f8b3c32d5b06a1bb043ef3f..f4188e2926aa8767ef6111b6849d1a5cf491011a 100644 (file)
@@ -18,6 +18,7 @@ import org.opendaylight.openflowplugin.applications.frsync.SyncReactor;
 import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeDao;
 import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeSnapshotDao;
 import org.opendaylight.openflowplugin.applications.frsync.util.PathUtil;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
@@ -30,19 +31,20 @@ import org.slf4j.LoggerFactory;
 public class SimplifiedConfigListener extends AbstractFrmSyncListener<FlowCapableNode> {
     private static final Logger LOG = LoggerFactory.getLogger(SimplifiedConfigListener.class);
     private final SyncReactor reactor;
-    private final FlowCapableNodeSnapshotDao configSnaphot;
+    private final FlowCapableNodeSnapshotDao configSnapshot;
     private final FlowCapableNodeDao operationalDao;
 
-    public SimplifiedConfigListener(final SyncReactor reactor, FlowCapableNodeSnapshotDao configSnaphot,
-            FlowCapableNodeDao operationalDao) {
+    public SimplifiedConfigListener(final SyncReactor reactor,
+                                    final FlowCapableNodeSnapshotDao configSnapshot,
+                                    final FlowCapableNodeDao operationalDao) {
         this.reactor = reactor;
-        this.configSnaphot = configSnaphot;
+        this.configSnapshot = configSnapshot;
         this.operationalDao = operationalDao;
     }
 
     @Override
     public void onDataTreeChanged(Collection<DataTreeModification<FlowCapableNode>> modifications) {
-        LOG.trace("Inventory Config changes {}", modifications.size());
+        LOG.trace("Config changes: {}", modifications.size());
         super.onDataTreeChanged(modifications);
     }
 
@@ -56,8 +58,7 @@ public class SimplifiedConfigListener extends AbstractFrmSyncListener<FlowCapabl
         final InstanceIdentifier<FlowCapableNode> nodePath = modification.getRootPath().getRootIdentifier();
         final NodeId nodeId = PathUtil.digNodeId(nodePath);
 
-        configSnaphot.updateCache(nodeId, Optional.fromNullable(modification.getRootNode().getDataAfter()));
-
+        configSnapshot.updateCache(nodeId, Optional.fromNullable(modification.getRootNode().getDataAfter()));
 
         final Optional<FlowCapableNode> operationalNode = operationalDao.loadByNodeId(nodeId);
         if (!operationalNode.isPresent()) {
@@ -90,12 +91,13 @@ public class SimplifiedConfigListener extends AbstractFrmSyncListener<FlowCapabl
      * optimal case (but the switch could be reprogrammed by another person/system.</li>
      * </ul>
      */
-    private ListenableFuture<Boolean> onNodeAdded(InstanceIdentifier<FlowCapableNode> nodePath,
-                           FlowCapableNode dataAfter, FlowCapableNode operationalNode) throws InterruptedException {
+    private ListenableFuture<Boolean> onNodeAdded(final InstanceIdentifier<FlowCapableNode> nodePath,
+                                                  final FlowCapableNode dataAfter,
+                                                  final FlowCapableNode operationalNode) throws InterruptedException {
         NodeId nodeId = PathUtil.digNodeId(nodePath);
-        LOG.trace("onNodeAdded {}", nodeId);
-        final ListenableFuture<Boolean> endResult = reactor.syncup(nodePath, dataAfter, operationalNode, dsType());
-        return endResult;
+        LOG.trace("onNodeAdded {}", nodeId.getValue());
+        final SyncupEntry syncupEntry = new SyncupEntry(dataAfter, dsType(), operationalNode, LogicalDatastoreType.OPERATIONAL);
+        return reactor.syncup(nodePath, syncupEntry);
     }
 
     /**
@@ -105,12 +107,13 @@ public class SimplifiedConfigListener extends AbstractFrmSyncListener<FlowCapabl
      * system which is updating operational store (that components is also trying to solve
      * scale/performance issues on several layers).
      */
-    private ListenableFuture<Boolean> onNodeUpdated(InstanceIdentifier<FlowCapableNode> nodePath,
-                          FlowCapableNode dataBefore, FlowCapableNode dataAfter) throws InterruptedException {
+    private ListenableFuture<Boolean> onNodeUpdated(final InstanceIdentifier<FlowCapableNode> nodePath,
+                                                    final FlowCapableNode dataBefore,
+                                                    final FlowCapableNode dataAfter) throws InterruptedException {
         NodeId nodeId = PathUtil.digNodeId(nodePath);
-        LOG.trace("onNodeUpdated {}", nodeId);
-        final ListenableFuture<Boolean> endResult = reactor.syncup(nodePath, dataAfter, dataBefore, dsType());
-        return endResult;
+        LOG.trace("onNodeUpdated {}", nodeId.getValue());
+        final SyncupEntry syncupEntry = new SyncupEntry(dataAfter, dsType(), dataBefore, dsType());
+        return reactor.syncup(nodePath, syncupEntry);
     }
 
     /**
@@ -118,16 +121,17 @@ public class SimplifiedConfigListener extends AbstractFrmSyncListener<FlowCapabl
      * probably optimized using dedicated wipe-out RPC, but it has impact on switch if it is
      * programmed by two person/system
      */
-    private ListenableFuture<Boolean> onNodeDeleted(InstanceIdentifier<FlowCapableNode> nodePath,
-                                                    FlowCapableNode dataBefore) throws InterruptedException {
+    private ListenableFuture<Boolean> onNodeDeleted(final InstanceIdentifier<FlowCapableNode> nodePath,
+                                                    final FlowCapableNode dataBefore) throws InterruptedException {
         NodeId nodeId = PathUtil.digNodeId(nodePath);
-        LOG.trace("onNodeDeleted {}", nodeId);
-        final ListenableFuture<Boolean> endResult = reactor.syncup(nodePath, null, dataBefore, dsType());
-        return endResult;
+        LOG.trace("onNodeDeleted {}", nodeId.getValue());
+        final SyncupEntry syncupEntry = new SyncupEntry(null, dsType(), dataBefore, dsType());
+        return reactor.syncup(nodePath, syncupEntry);
     }
 
     @Override
     public LogicalDatastoreType dsType() {
         return LogicalDatastoreType.CONFIGURATION;
     }
+
 }
index 738a652e5e0e08741a0f47d37d850223e235f573..eb70f5d4931a61acb897a1799ebabefd6b8063c1 100644 (file)
@@ -10,7 +10,10 @@ package org.opendaylight.openflowplugin.applications.frsync.impl;
 
 import com.google.common.base.Optional;
 import com.google.common.util.concurrent.ListenableFuture;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
 import java.util.Collection;
+import java.util.Date;
 import java.util.List;
 import org.opendaylight.controller.md.sal.binding.api.DataObjectModification;
 import org.opendaylight.controller.md.sal.binding.api.DataObjectModification.ModificationType;
@@ -19,7 +22,14 @@ import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.openflowplugin.applications.frsync.SyncReactor;
 import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeDao;
 import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeSnapshotDao;
+import org.opendaylight.openflowplugin.applications.frsync.impl.clustering.DeviceMastershipManager;
+import org.opendaylight.openflowplugin.applications.frsync.util.ModificationUtil;
+import org.opendaylight.openflowplugin.applications.frsync.util.PathUtil;
+import org.opendaylight.openflowplugin.applications.frsync.util.ReconciliationRegistry;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableStatisticsGatheringStatus;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.snapshot.gathering.status.grouping.SnapshotGatheringStatusEnd;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.Nodes;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.node.NodeConnector;
@@ -34,21 +44,29 @@ import org.slf4j.LoggerFactory;
  */
 public class SimplifiedOperationalListener extends AbstractFrmSyncListener<Node> {
     private static final Logger LOG = LoggerFactory.getLogger(SimplifiedOperationalListener.class);
+    public static final String DATE_AND_TIME_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSSXXX";
 
     private final SyncReactor reactor;
     private final FlowCapableNodeSnapshotDao operationalSnapshot;
     private final FlowCapableNodeDao configDao;
-
-    public SimplifiedOperationalListener(SyncReactor reactor, FlowCapableNodeSnapshotDao operationalSnapshot,
-                                         FlowCapableNodeDao configDao) {
+    private final ReconciliationRegistry reconciliationRegistry;
+    private final DeviceMastershipManager deviceMastershipManager;
+
+    public SimplifiedOperationalListener(final SyncReactor reactor,
+                                         final FlowCapableNodeSnapshotDao operationalSnapshot,
+                                         final FlowCapableNodeDao configDao,
+                                         final ReconciliationRegistry reconciliationRegistry,
+                                         final DeviceMastershipManager deviceMastershipManager) {
         this.reactor = reactor;
         this.operationalSnapshot = operationalSnapshot;
         this.configDao = configDao;
+        this.reconciliationRegistry = reconciliationRegistry;
+        this.deviceMastershipManager = deviceMastershipManager;
     }
 
     @Override
     public void onDataTreeChanged(Collection<DataTreeModification<Node>> modifications) {
-        LOG.trace("Inventory Operational changes {}", modifications.size());
+        LOG.trace("Operational changes: {}", modifications.size());
         super.onDataTreeChanged(modifications);
     }
 
@@ -64,31 +82,38 @@ public class SimplifiedOperationalListener extends AbstractFrmSyncListener<Node>
      */
     protected Optional<ListenableFuture<Boolean>> processNodeModification(
             DataTreeModification<Node> modification) throws InterruptedException {
-
+        final NodeId nodeId = ModificationUtil.nodeId(modification);
         updateCache(modification);
-        if (isReconciliationNeeded(modification)) {
+
+        if (isAdd(modification) || isAddLogical(modification)) {
+            deviceMastershipManager.onDeviceConnected(nodeId);
+        }
+
+        if (isRegisteredAndConsistentForReconcile(modification)) {
             return reconciliation(modification);
+        } else {
+            return skipModification(modification);
         }
-        return skipModification(modification);
     }
 
     /**
      * Remove if delete. Update only if FlowCapableNode Augmentation modified.
      *
      * @param modification Datastore modification
-     * @return true for cache update, false for cache remove
      */
-    protected boolean updateCache(DataTreeModification<Node> modification) {
+    private void updateCache(DataTreeModification<Node> modification) {
+        NodeId nodeId = ModificationUtil.nodeId(modification);
         if (isDelete(modification) || isDeleteLogical(modification)) {
-            operationalSnapshot.updateCache(nodeId(modification), Optional.<FlowCapableNode>absent());
-            return false;
+            operationalSnapshot.updateCache(nodeId, Optional.absent());
+            deviceMastershipManager.onDeviceDisconnected(nodeId);
+            return;
         }
-        operationalSnapshot.updateCache(nodeId(modification), Optional.fromNullable(flowCapableNodeAfter(modification)));
-        return true;
+        operationalSnapshot.updateCache(nodeId, Optional.fromNullable(ModificationUtil.flowCapableNodeAfter(modification)));
     }
 
     private Optional<ListenableFuture<Boolean>> skipModification(DataTreeModification<Node> modification) {
-        LOG.trace("Skipping Inventory Operational modification {}, before {}, after {}", nodeIdValue(modification),
+        LOG.trace("Skipping operational modification: {}, before {}, after {}",
+                ModificationUtil.nodeIdValue(modification),
                 modification.getRootNode().getDataBefore() == null ? "null" : "nonnull",
                 modification.getRootNode().getDataAfter() == null ? "null" : "nonnull");
         return Optional.absent();
@@ -99,7 +124,7 @@ public class SimplifiedOperationalListener extends AbstractFrmSyncListener<Node>
      */
     private boolean isDelete(DataTreeModification<Node> modification) {
         if (ModificationType.DELETE == modification.getRootNode().getModificationType()) {
-            LOG.trace("Delete {} (physical)", nodeIdValue(modification));
+            LOG.trace("Delete {} (physical)", ModificationUtil.nodeIdValue(modification));
             return true;
         }
 
@@ -112,7 +137,7 @@ public class SimplifiedOperationalListener extends AbstractFrmSyncListener<Node>
     private boolean isDeleteLogical(DataTreeModification<Node> modification) {
         final DataObjectModification<Node> rootNode = modification.getRootNode();
         if (!safeConnectorsEmpty(rootNode.getDataBefore()) && safeConnectorsEmpty(rootNode.getDataAfter())) {
-            LOG.trace("Delete {} (logical)", nodeIdValue(modification));
+            LOG.trace("Delete {} (logical)", ModificationUtil.nodeIdValue(modification));
             return true;
         }
 
@@ -126,7 +151,7 @@ public class SimplifiedOperationalListener extends AbstractFrmSyncListener<Node>
 
         final boolean nodeAppearedInOperational = dataBefore == null && dataAfter != null;
         if (nodeAppearedInOperational) {
-            LOG.trace("Add {} (physical)", nodeIdValue(modification));
+            LOG.trace("Add {} (physical)", ModificationUtil.nodeIdValue(modification));
         }
         return nodeAppearedInOperational;
     }
@@ -137,77 +162,87 @@ public class SimplifiedOperationalListener extends AbstractFrmSyncListener<Node>
     private boolean isAddLogical(DataTreeModification<Node> modification) {
         final DataObjectModification<Node> rootNode = modification.getRootNode();
         if (safeConnectorsEmpty(rootNode.getDataBefore()) && !safeConnectorsEmpty(rootNode.getDataAfter())) {
-            LOG.trace("Add {} (logical)", nodeIdValue(modification));
+            LOG.trace("Add {} (logical)", ModificationUtil.nodeIdValue(modification));
             return true;
         }
 
         return false;
     }
 
-    protected boolean isReconciliationNeeded(DataTreeModification<Node> modification) {
-        return isAdd(modification) || isAddLogical(modification);
-    }
-
     private Optional<ListenableFuture<Boolean>> reconciliation(DataTreeModification<Node> modification) throws InterruptedException {
-        final NodeId nodeId = nodeId(modification);
+        final NodeId nodeId = ModificationUtil.nodeId(modification);
         final Optional<FlowCapableNode> nodeConfiguration = configDao.loadByNodeId(nodeId);
 
         if (nodeConfiguration.isPresent()) {
             LOG.debug("Reconciliation: {}", nodeId.getValue());
             final InstanceIdentifier<FlowCapableNode> nodePath = InstanceIdentifier.create(Nodes.class)
-                    .child(Node.class, new NodeKey(nodeId(modification))).augmentation(FlowCapableNode.class);
-            return Optional.of(reactor.syncup(nodePath, nodeConfiguration.get(), flowCapableNodeAfter(modification), dsType()));
+                    .child(Node.class, new NodeKey(ModificationUtil.nodeId(modification)))
+                    .augmentation(FlowCapableNode.class);
+            final FlowCapableNode fcOperationalNode = ModificationUtil.flowCapableNodeAfter(modification);
+            final SyncupEntry syncupEntry = new SyncupEntry(nodeConfiguration.get(), LogicalDatastoreType.CONFIGURATION,
+                                                            fcOperationalNode, dsType());
+            return Optional.of(reactor.syncup(nodePath, syncupEntry));
         } else {
+            LOG.debug("Config not present for reconciliation: {}", nodeId.getValue());
             return skipModification(modification);
         }
     }
 
-    private static FlowCapableNode flowCapableNodeAfter(DataTreeModification<Node> modification) {
-        final Node dataAfter = modification.getRootNode().getDataAfter();
-        if (dataAfter == null) {
-            return null;
-        }
-        return dataAfter.getAugmentation(FlowCapableNode.class);
-    }
+    private boolean isRegisteredAndConsistentForReconcile(DataTreeModification<Node> modification) {
+        final NodeId nodeId = PathUtil.digNodeId(modification.getRootPath().getRootIdentifier());
 
-    private static boolean safeConnectorsEmpty(Node node) {
-        if (node == null) {
-            return true;
+        if (!reconciliationRegistry.isRegistered(nodeId)) {
+            return false;
         }
 
-        final List<NodeConnector> nodeConnectors = node.getNodeConnector();
+        final FlowCapableStatisticsGatheringStatus gatheringStatus = modification.getRootNode().getDataAfter()
+                .getAugmentation(FlowCapableStatisticsGatheringStatus.class);
 
-        return nodeConnectors == null || nodeConnectors.isEmpty();
-    }
+        if (gatheringStatus == null) {
+            LOG.trace("Statistics gathering never started: {}", nodeId.getValue());
+            return false;
+        }
 
-    private static String nodeIdValue(DataTreeModification<Node> modification) {
-        final NodeId nodeId = nodeId(modification);
+        final SnapshotGatheringStatusEnd gatheringStatusEnd = gatheringStatus.getSnapshotGatheringStatusEnd();
 
-        if (nodeId == null) {
-            return null;
+        if (gatheringStatusEnd == null) {
+            LOG.trace("Statistics gathering is not over yet: {}", nodeId.getValue());
+            return false;
         }
 
-        return nodeId.getValue();
-    }
-
-    static NodeId nodeId(DataTreeModification<Node> modification) {
-        final DataObjectModification<Node> rootNode = modification.getRootNode();
-        final Node dataAfter = rootNode.getDataAfter();
+        if (!gatheringStatusEnd.isSucceeded()) {
+            LOG.trace("Statistics gathering was not successful: {}", nodeId.getValue());
+            return false;
+        }
 
-        if (dataAfter != null) {
-            return dataAfter.getId();
+        try {
+            Date timestampOfRegistration = reconciliationRegistry.getRegistrationTimestamp(nodeId);
+            final SimpleDateFormat simpleDateFormat = new SimpleDateFormat(DATE_AND_TIME_FORMAT);
+            Date timestampOfStatistics = simpleDateFormat.parse(gatheringStatusEnd.getEnd().getValue());
+            if (timestampOfStatistics.after(timestampOfRegistration)) {
+                LOG.debug("Fresh operational present: {}", nodeId.getValue());
+                return true;
+            }
+        } catch (ParseException e) {
+            LOG.error("Timestamp parsing error {}", e);
         }
+        LOG.debug("Fresh operational not present: {}", nodeId.getValue());
+        return false;
+    }
 
-        final Node dataBefore = rootNode.getDataBefore();
-        if (dataBefore != null) {
-            return dataBefore.getId();
+    private static boolean safeConnectorsEmpty(Node node) {
+        if (node == null) {
+            return true;
         }
 
-        return null;
+        final List<NodeConnector> nodeConnectors = node.getNodeConnector();
+
+        return nodeConnectors == null || nodeConnectors.isEmpty();
     }
 
     @Override
     public LogicalDatastoreType dsType() {
         return LogicalDatastoreType.OPERATIONAL;
     }
+    
 }
diff --git a/applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/SimplifiedOperationalRetryListener.java b/applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/SimplifiedOperationalRetryListener.java
deleted file mode 100644 (file)
index 924e26f..0000000
+++ /dev/null
@@ -1,112 +0,0 @@
-/**
- * Copyright (c) 2016 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.openflowplugin.applications.frsync.impl;
-
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import org.opendaylight.controller.md.sal.binding.api.DataTreeModification;
-import org.opendaylight.openflowplugin.applications.frsync.SyncReactor;
-import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeDao;
-import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeSnapshotDao;
-import org.opendaylight.openflowplugin.applications.frsync.util.PathUtil;
-import org.opendaylight.openflowplugin.applications.frsync.util.RetryRegistry;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableStatisticsGatheringStatus;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.snapshot.gathering.status.grouping.SnapshotGatheringStatusEnd;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.nodes.Node;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Modified {@link SimplifiedOperationalListener} for usage of retry mechanism.
- */
-public class SimplifiedOperationalRetryListener extends SimplifiedOperationalListener {
-
-    private static final Logger LOG = LoggerFactory.getLogger(SimplifiedOperationalRetryListener.class);
-    private final RetryRegistry retryRegistry;
-
-    public SimplifiedOperationalRetryListener(SyncReactor reactor, FlowCapableNodeSnapshotDao operationalSnapshot,
-                                              FlowCapableNodeDao configDao, RetryRegistry retryRegistry) {
-        super(reactor, operationalSnapshot, configDao);
-        this.retryRegistry = retryRegistry;
-    }
-
-    /**
-     * Adding condition check for retry.
-     *
-     * @param modification operational datastore modification
-     * @return true if reconciliation is needed, false otherwise
-     */
-    protected boolean isReconciliationNeeded(DataTreeModification<Node> modification) {
-        return super.isReconciliationNeeded(modification) || isRegisteredAndConsistentForRetry(modification);
-    }
-
-    /**
-     * If node is removed unregister for retry in addition.
-     *
-     * @param modification operational datastore modification
-     * @return true for cache update, false for cache remove and retry unregister
-     */
-    protected boolean updateCache(DataTreeModification<Node> modification) {
-        boolean nodeUpdated = super.updateCache(modification);
-        if (!nodeUpdated) { // node removed if not updated
-            retryRegistry.unregisterIfRegistered(nodeId(modification));
-        }
-        return nodeUpdated;
-    }
-
-    /**
-     * Check if retry should be proceeded.
-     *
-     * @param modification operational modification
-     * @return true if device is registered for retry and actual modification is consistent, false otherwise
-     */
-    private boolean isRegisteredAndConsistentForRetry(DataTreeModification<Node> modification) {
-        final NodeId nodeId = PathUtil.digNodeId(modification.getRootPath().getRootIdentifier());
-
-        if (!retryRegistry.isRegistered(nodeId)) {
-            return false;
-        }
-
-        final FlowCapableStatisticsGatheringStatus gatheringStatus = modification.getRootNode().getDataAfter()
-                .getAugmentation(FlowCapableStatisticsGatheringStatus.class);
-
-        if (gatheringStatus == null) {
-            LOG.trace("Statistics gathering never started for: {}", nodeId.getValue());
-            return false;
-        }
-
-        final SnapshotGatheringStatusEnd gatheringStatusEnd = gatheringStatus.getSnapshotGatheringStatusEnd();
-
-        if (gatheringStatusEnd == null) {
-            LOG.trace("Statistics gathering is not over yet for: {}", nodeId.getValue());
-            return false;
-        }
-
-        if (!gatheringStatusEnd.isSucceeded()) {
-            LOG.debug("Statistics gathering was not successful for: {}", nodeId.getValue());
-            return false;
-        }
-
-        try {
-            Date timestampOfRegistration = retryRegistry.getRegistration(nodeId);
-            final SimpleDateFormat simpleDateFormat = new SimpleDateFormat(RetryRegistry.DATE_AND_TIME_FORMAT);
-            Date timestampOfStatistics = simpleDateFormat.parse(gatheringStatusEnd.getEnd().getValue());
-            if (timestampOfStatistics.after(timestampOfRegistration)) {
-                LOG.debug("Fresh operational present for: {} -> going retry!", nodeId.getValue());
-                return true;
-            }
-        } catch (ParseException e) {
-            LOG.error("Timestamp parsing error {}", e);
-        }
-        LOG.debug("Fresh operational not present for: {}", nodeId.getValue());
-        return false;
-    }
-}
diff --git a/applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/SyncReactorClusterDecorator.java b/applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/SyncReactorClusterDecorator.java
new file mode 100644 (file)
index 0000000..e508e27
--- /dev/null
@@ -0,0 +1,50 @@
+/**
+ * Copyright (c) 2016 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.openflowplugin.applications.frsync.impl;
+
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.opendaylight.openflowplugin.applications.frsync.SyncReactor;
+import org.opendaylight.openflowplugin.applications.frsync.impl.clustering.DeviceMastershipManager;
+import org.opendaylight.openflowplugin.applications.frsync.util.PathUtil;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
+import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Decorator for cluster related issues.
+ */
+public class SyncReactorClusterDecorator implements SyncReactor {
+    private static final Logger LOG = LoggerFactory.getLogger(SyncReactorClusterDecorator.class);
+    private final SyncReactor delegate;
+    private final DeviceMastershipManager deviceMastershipManager;
+
+    public SyncReactorClusterDecorator(final SyncReactor delegate,
+                                       final DeviceMastershipManager deviceMastershipManager) {
+        this.delegate = delegate;
+        this.deviceMastershipManager = deviceMastershipManager;
+    }
+
+    @Override
+    public ListenableFuture<Boolean> syncup(final InstanceIdentifier<FlowCapableNode> flowcapableNodePath,
+                                            final SyncupEntry syncupEntry) throws InterruptedException {
+        final NodeId nodeId = PathUtil.digNodeId(flowcapableNodePath);
+        LOG.trace("Syncup cluster decorator: {}", nodeId.getValue());
+
+        if (!deviceMastershipManager.isDeviceMastered(nodeId)) {
+            LOG.debug("Skip syncup since not master for: {}", nodeId.getValue());
+            return Futures.immediateFuture(Boolean.TRUE);
+        } else {
+            return delegate.syncup(flowcapableNodePath, syncupEntry);
+        }
+    }
+}
index f05db11009bb3e718b00e15c9008a893caf3e824..ba3a1d8c59e9d858a8c85b78f70b047ecb9c3226 100644 (file)
@@ -10,12 +10,11 @@ package org.opendaylight.openflowplugin.applications.frsync.impl;
 
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
-import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.openflowplugin.applications.frsync.SyncReactor;
 import org.opendaylight.openflowplugin.applications.frsync.util.PathUtil;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
@@ -38,39 +37,33 @@ public class SyncReactorFutureDecorator implements SyncReactor {
     }
 
     public ListenableFuture<Boolean> syncup(final InstanceIdentifier<FlowCapableNode> flowcapableNodePath,
-                                            final FlowCapableNode configTree, final FlowCapableNode operationalTree,
-                                            final LogicalDatastoreType dsType) throws InterruptedException {
+                                            final SyncupEntry syncupEntry) throws InterruptedException {
         final NodeId nodeId = PathUtil.digNodeId(flowcapableNodePath);
-        LOG.trace("syncup future {}", nodeId.getValue());
+        LOG.trace("syncup future decorator: {}", nodeId.getValue());
 
-        final ListenableFuture<Boolean> syncup = executorService.submit(new Callable<Boolean>() {
-            public Boolean call() throws Exception {
-                final String oldThreadName = updateThreadName(nodeId);
+        return executorService.submit(() -> {
+            final String oldThreadName = updateThreadName(nodeId);
 
-                try {
-                    final Boolean ret = doSyncupInFuture(flowcapableNodePath, configTree, operationalTree, dsType)
-                            .get(10000, TimeUnit.MILLISECONDS);
-                    LOG.trace("ret {} {}", nodeId.getValue(), ret);
-                    return true;
-                } catch (TimeoutException e) {
-                    LOG.error("doSyncupInFuture timeout occured {}", nodeId.getValue(), e);
-                    return false;
-                } finally {
-                    updateThreadName(oldThreadName);
-                }
+            try {
+                final Boolean ret = doSyncupInFuture(flowcapableNodePath, syncupEntry)
+                        .get(10000, TimeUnit.MILLISECONDS);
+                LOG.trace("syncup return in future decorator: {} [{}]", nodeId.getValue(), ret);
+                return true;
+            } catch (TimeoutException e) {
+                LOG.error("doSyncupInFuture timeout occured {}", nodeId.getValue(), e);
+                return false;
+            } finally {
+                updateThreadName(oldThreadName);
             }
         });
-
-        return syncup;
     }
 
     protected ListenableFuture<Boolean> doSyncupInFuture(final InstanceIdentifier<FlowCapableNode> flowcapableNodePath,
-                                                         final FlowCapableNode configTree, final FlowCapableNode operationalTree,
-                                                         final LogicalDatastoreType dsType) throws InterruptedException {
+                                                         final SyncupEntry syncupEntry) throws InterruptedException {
         final NodeId nodeId = PathUtil.digNodeId(flowcapableNodePath);
-        LOG.trace("doSyncupInFuture future {}", nodeId.getValue());
+        LOG.trace("doSyncupInFuture future decorator: {}", nodeId.getValue());
 
-        return delegate.syncup(flowcapableNodePath, configTree, operationalTree, dsType);
+        return delegate.syncup(flowcapableNodePath, syncupEntry);
     }
 
     private String updateThreadName(NodeId nodeId) {
@@ -88,7 +81,7 @@ public class SyncReactorFutureDecorator implements SyncReactor {
         return oldName;
     }
 
-    private String updateThreadName(String name) {
+    private void updateThreadName(String name) {
         final Thread currentThread = Thread.currentThread();
         final String oldName = currentThread.getName();
         try {
@@ -100,6 +93,5 @@ public class SyncReactorFutureDecorator implements SyncReactor {
         } catch (Exception e) {
             LOG.error("failed updating threadName {}", name, e);
         }
-        return oldName;
     }
 }
index a3edde05a884882b7f86eaa5029012346f1af9af..060a04964c02aff5ba2f9a5ca79604e9bc67c87c 100644 (file)
@@ -15,10 +15,9 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.Semaphore;
 import javax.annotation.concurrent.GuardedBy;
-import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.openflowplugin.applications.frsync.SyncReactor;
 import org.opendaylight.openflowplugin.applications.frsync.util.PathUtil;
-import org.opendaylight.openflowplugin.applications.frsync.util.ZipQueueEntry;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
@@ -33,7 +32,7 @@ public class SyncReactorFutureZipDecorator extends SyncReactorFutureDecorator {
     private static final Logger LOG = LoggerFactory.getLogger(SyncReactorFutureZipDecorator.class);
 
     @GuardedBy("compressionGuard")
-    private final Map<InstanceIdentifier<FlowCapableNode>, ZipQueueEntry> compressionQueue = new HashMap<>();
+    private final Map<InstanceIdentifier<FlowCapableNode>, SyncupEntry> compressionQueue = new HashMap<>();
     private final Semaphore compressionGuard = new Semaphore(1, false);
 
     public SyncReactorFutureZipDecorator(SyncReactor delegate, ListeningExecutorService executorService) {
@@ -41,17 +40,16 @@ public class SyncReactorFutureZipDecorator extends SyncReactorFutureDecorator {
     }
 
     public ListenableFuture<Boolean> syncup(final InstanceIdentifier<FlowCapableNode> flowcapableNodePath,
-                                            final FlowCapableNode configTree, final FlowCapableNode operationalTree,
-                                            final LogicalDatastoreType dsType) throws InterruptedException {
+                                            final SyncupEntry syncupEntry) throws InterruptedException {
         final NodeId nodeId = PathUtil.digNodeId(flowcapableNodePath);
-        LOG.trace("syncup zip {}", nodeId.getValue());
+        LOG.trace("syncup zip decorator: {}", nodeId.getValue());
 
         try {
             compressionGuard.acquire();
 
-            final boolean newFutureNecessary = updateCompressionState(flowcapableNodePath, configTree, operationalTree, dsType);
+            final boolean newFutureNecessary = updateCompressionState(flowcapableNodePath, syncupEntry);
             if (newFutureNecessary) {
-                super.syncup(flowcapableNodePath, configTree, operationalTree, dsType);
+                super.syncup(flowcapableNodePath, syncupEntry);
             }
             return Futures.immediateFuture(true);
         } finally {
@@ -60,17 +58,15 @@ public class SyncReactorFutureZipDecorator extends SyncReactorFutureDecorator {
     }
 
     protected ListenableFuture<Boolean> doSyncupInFuture(final InstanceIdentifier<FlowCapableNode> flowcapableNodePath,
-                                                         final FlowCapableNode configTree, final FlowCapableNode operationalTree,
-                                                         final LogicalDatastoreType dsType) throws InterruptedException {
+                                                         final SyncupEntry syncupEntry) throws InterruptedException {
         final NodeId nodeId = PathUtil.digNodeId(flowcapableNodePath);
-        LOG.trace("doSyncupInFuture zip {}", nodeId.getValue());
+        LOG.trace("doSyncupInFuture zip decorator: {}", nodeId.getValue());
 
-        final ZipQueueEntry lastCompressionState = removeLastCompressionState(flowcapableNodePath);
+        final SyncupEntry lastCompressionState = removeLastCompressionState(flowcapableNodePath);
         if (lastCompressionState == null) {
             return Futures.immediateFuture(true);
         } else {
-            return super.doSyncupInFuture(flowcapableNodePath,
-                    lastCompressionState.getLeft(), lastCompressionState.getRight(), dsType);
+            return super.doSyncupInFuture(flowcapableNodePath, lastCompressionState);
         }
     }
 
@@ -79,30 +75,24 @@ public class SyncReactorFutureZipDecorator extends SyncReactorFutureDecorator {
      * update its zip queue entry. Create/replace zip queue entry for the device with operational delta otherwise.
      */
     private boolean updateCompressionState(final InstanceIdentifier<FlowCapableNode> flowcapableNodePath,
-                                           final FlowCapableNode configTree, final FlowCapableNode operationalTree,
-                                           final LogicalDatastoreType dsType) {
-        final ZipQueueEntry previousEntry = compressionQueue.get(flowcapableNodePath);
+                                           final SyncupEntry syncupEntry) {
+        final SyncupEntry previousEntry = compressionQueue.get(flowcapableNodePath);
 
-        if (previousEntry != null && dsType == LogicalDatastoreType.CONFIGURATION
-                && previousEntry.getDsType() == LogicalDatastoreType.CONFIGURATION) {
-            putOptimizedConfigDelta(flowcapableNodePath, configTree, previousEntry);
+        if (previousEntry != null && syncupEntry.isOptimizedConfigDelta() && previousEntry.isOptimizedConfigDelta()) {
+            updateOptimizedConfigDelta(flowcapableNodePath, syncupEntry, previousEntry);
         } else {
-            putLatestOperationalDelta(flowcapableNodePath, configTree, operationalTree, dsType);
+            compressionQueue.put(flowcapableNodePath, syncupEntry);
         }
         return previousEntry == null;
     }
 
-    private void putOptimizedConfigDelta(InstanceIdentifier<FlowCapableNode> flowcapableNodePath, FlowCapableNode configTree,
-                                         ZipQueueEntry previous) {
-        compressionQueue.put(flowcapableNodePath, new ZipQueueEntry(configTree, previous.getRight(), previous.getDsType()));
+    private void updateOptimizedConfigDelta(InstanceIdentifier<FlowCapableNode> flowcapableNodePath, SyncupEntry actual,
+                                            SyncupEntry previous) {
+        compressionQueue.put(flowcapableNodePath, new SyncupEntry(actual.getAfter(), actual.getDsTypeAfter(),
+                                                                  previous.getBefore(), previous.getDsTypeBefore()));
     }
 
-    private void putLatestOperationalDelta(InstanceIdentifier<FlowCapableNode> flowcapableNodePath, FlowCapableNode configTree,
-                                           FlowCapableNode operationalTree, LogicalDatastoreType dsType) {
-        compressionQueue.put(flowcapableNodePath, new ZipQueueEntry(configTree, operationalTree, dsType));
-    }
-
-    private ZipQueueEntry removeLastCompressionState(
+    private SyncupEntry removeLastCompressionState(
             final InstanceIdentifier<FlowCapableNode> flowcapableNodePath) {
         try {
             try {
index e40b3b2200a1eda0fd070ea7553a6f8ba75e5f49..c51f0bb6e0a6df346efe2024a49cb776cfeba5f2 100644 (file)
@@ -15,10 +15,10 @@ import com.google.common.util.concurrent.ListenableFuture;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 import javax.annotation.Nullable;
-import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.openflowplugin.applications.frsync.SemaphoreKeeper;
 import org.opendaylight.openflowplugin.applications.frsync.SyncReactor;
 import org.opendaylight.openflowplugin.applications.frsync.util.PathUtil;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
@@ -42,10 +42,9 @@ public class SyncReactorGuardDecorator implements SyncReactor {
     }
 
     public ListenableFuture<Boolean> syncup(final InstanceIdentifier<FlowCapableNode> flowcapableNodePath,
-                                            final FlowCapableNode configTree, final FlowCapableNode operationalTree,
-                                            final LogicalDatastoreType dsType) throws InterruptedException {
+                                            final SyncupEntry syncupEntry) throws InterruptedException {
         final NodeId nodeId = PathUtil.digNodeId(flowcapableNodePath);
-        LOG.trace("syncup guard {}", nodeId.getValue());
+        LOG.trace("syncup guard decorator: {}", nodeId.getValue());
 
         final long stampBeforeGuard = System.nanoTime();
         final Semaphore guard = summonGuardAndAcquire(flowcapableNodePath);
@@ -62,7 +61,7 @@ public class SyncReactorGuardDecorator implements SyncReactor {
             }
 
             final ListenableFuture<Boolean> endResult =
-                    delegate.syncup(flowcapableNodePath, configTree, operationalTree, dsType);
+                    delegate.syncup(flowcapableNodePath, syncupEntry);
 
             Futures.addCallback(endResult, new FutureCallback<Boolean>() {
                 @Override
index 504d583d032f8a02dc029067e20e4cc593265a9f..5eb8860f884ebb35dcb609f414f5e77898bb4d85 100644 (file)
@@ -18,7 +18,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
-import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.openflowplugin.applications.frsync.SyncPlanPushStrategy;
 import org.opendaylight.openflowplugin.applications.frsync.SyncReactor;
 import org.opendaylight.openflowplugin.applications.frsync.impl.strategy.SynchronizationDiffInput;
@@ -29,6 +28,7 @@ import org.opendaylight.openflowplugin.applications.frsync.util.ItemSyncBox;
 import org.opendaylight.openflowplugin.applications.frsync.util.PathUtil;
 import org.opendaylight.openflowplugin.applications.frsync.util.ReconcileUtil;
 import org.opendaylight.openflowplugin.applications.frsync.util.SyncCrudCounters;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.meters.Meter;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.Table;
@@ -57,11 +57,15 @@ public class SyncReactorImpl implements SyncReactor {
 
     @Override
     public ListenableFuture<Boolean> syncup(final InstanceIdentifier<FlowCapableNode> nodeIdent,
-                                            final FlowCapableNode configTree, final FlowCapableNode operationalTree,
-                                            final LogicalDatastoreType dsType) {
-
+                                            final SyncupEntry syncupEntry) {
         final NodeId nodeId = PathUtil.digNodeId(nodeIdent);
-        LOG.trace("syncup impl {} cfg:{} oper:{}", nodeId.getValue(), configTree == null ? "is null" : "non null", operationalTree == null ? "is null" : "non null");
+        FlowCapableNode configTree = syncupEntry.getAfter();
+        FlowCapableNode operationalTree = syncupEntry.getBefore();
+
+        LOG.trace("syncup reactor {} cfg:{} oper:{}",
+                nodeId.getValue(),
+                configTree == null ? "is null" : "non null",
+                operationalTree == null ? "is null" : "non null");
         final SyncCrudCounters counters = new SyncCrudCounters();
 
         /**
@@ -108,7 +112,7 @@ public class SyncReactorImpl implements SyncReactor {
                     final CrudCounts flowCrudCounts = counters.getFlowCrudCounts();
                     final CrudCounts meterCrudCounts = counters.getMeterCrudCounts();
                     final CrudCounts groupCrudCounts = counters.getGroupCrudCounts();
-                    LOG.debug("sync-outcome[{}] (added/updated/removed): flow={}/{}/{}, meter={}/{}/{}, group={}/{}/{}, took={} ms",
+                    LOG.debug("syncup outcome[{}] (added/updated/removed): flow={}/{}/{}, meter={}/{}/{}, group={}/{}/{}, took={} ms",
                             nodeId.getValue(),
                             flowCrudCounts.getAdded(),
                             flowCrudCounts.getUpdated(),
index d504855c9283077bb032c7457b3115e48e7aa84b..9ab76dfe19847b074e9949f694039d78b808005f 100644 (file)
@@ -11,10 +11,10 @@ package org.opendaylight.openflowplugin.applications.frsync.impl;
 import com.google.common.base.Function;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
-import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.openflowplugin.applications.frsync.SyncReactor;
 import org.opendaylight.openflowplugin.applications.frsync.util.PathUtil;
-import org.opendaylight.openflowplugin.applications.frsync.util.RetryRegistry;
+import org.opendaylight.openflowplugin.applications.frsync.util.ReconciliationRegistry;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
@@ -29,38 +29,35 @@ public class SyncReactorRetryDecorator implements SyncReactor {
     private static final Logger LOG = LoggerFactory.getLogger(SyncReactorRetryDecorator.class);
 
     private final SyncReactor delegate;
-    private final RetryRegistry retryRegistry;
+    private final ReconciliationRegistry reconciliationRegistry;
 
-    public SyncReactorRetryDecorator(final SyncReactor delegate, RetryRegistry retryRegistry) {
+    public SyncReactorRetryDecorator(final SyncReactor delegate, final ReconciliationRegistry reconciliationRegistry) {
         this.delegate = delegate;
-        this.retryRegistry = retryRegistry;
+        this.reconciliationRegistry = reconciliationRegistry;
     }
 
     public ListenableFuture<Boolean> syncup(final InstanceIdentifier<FlowCapableNode> flowcapableNodePath,
-                                            final FlowCapableNode configTree, final FlowCapableNode operationalTree,
-                                            final LogicalDatastoreType dsType) throws InterruptedException {
+                                            final SyncupEntry syncupEntry) throws InterruptedException {
 
         final NodeId nodeId = PathUtil.digNodeId(flowcapableNodePath);
-        LOG.trace("syncup retry {}", nodeId.getValue());
+        LOG.trace("syncup retry decorator: {}", nodeId.getValue());
 
-        if (dsType == LogicalDatastoreType.CONFIGURATION && retryRegistry.isRegistered(nodeId)) {
-            LOG.trace("Config change ignored because device is in retry [{}]", nodeId);
+        if (syncupEntry.isOptimizedConfigDelta() && reconciliationRegistry.isRegistered(nodeId)) {
+            LOG.debug("Config change ignored because {} is in reconcile.", nodeId.getValue());
             return Futures.immediateFuture(Boolean.FALSE);
         }
 
-        ListenableFuture<Boolean> syncupResult = delegate.syncup(flowcapableNodePath, configTree, operationalTree, dsType);
+        ListenableFuture<Boolean> syncupResult = delegate.syncup(flowcapableNodePath,syncupEntry);
 
         return Futures.transform(syncupResult, new Function<Boolean, Boolean>() {
             @Override
             public Boolean apply(Boolean result) {
-                LOG.trace("syncup ret in retry {}", result);
+                LOG.trace("syncup return in retry decorator: {} [{}]", nodeId.getValue(), result);
                 if (result) {
-                    retryRegistry.unregisterIfRegistered(nodeId);
+                    reconciliationRegistry.unregisterIfRegistered(nodeId);
                     return true;
                 } else {
-                    retryRegistry.register(nodeId);
-                    // TODO  elicit statistics gathering if not running actually
-                    // triggerStatisticsGathering(nodeId);
+                    reconciliationRegistry.register(nodeId);
                     return false;
                 }
             }
diff --git a/applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/clustering/DeviceMastership.java b/applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/clustering/DeviceMastership.java
new file mode 100644 (file)
index 0000000..211b4f4
--- /dev/null
@@ -0,0 +1,71 @@
+/**
+ * Copyright (c) 2016 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.openflowplugin.applications.frsync.impl.clustering;
+
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonService;
+import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceRegistration;
+import org.opendaylight.mdsal.singleton.common.api.ServiceGroupIdentifier;
+import org.opendaylight.openflowplugin.applications.frsync.util.ReconciliationRegistry;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link ClusterSingletonService} clusterSingletonServiceRegistration per connected device.
+ */
+public class DeviceMastership implements ClusterSingletonService {
+    private static final Logger LOG = LoggerFactory.getLogger(DeviceMastership.class);
+    private final NodeId nodeId;
+    private final ServiceGroupIdentifier identifier;
+    private final ReconciliationRegistry reconciliationRegistry;
+    private ClusterSingletonServiceRegistration clusterSingletonServiceRegistration;
+    private boolean deviceMastered;
+
+    public DeviceMastership(final NodeId nodeId, final ReconciliationRegistry reconciliationRegistry) {
+        this.nodeId = nodeId;
+        this.identifier = ServiceGroupIdentifier.create(nodeId.getValue());
+        this.reconciliationRegistry = reconciliationRegistry;
+        this.deviceMastered = false;
+    }
+
+    @Override
+    public void instantiateServiceInstance() {
+        LOG.debug("FRS started for: {}", nodeId.getValue());
+        deviceMastered = true;
+        reconciliationRegistry.register(nodeId);
+    }
+
+    @Override
+    public ListenableFuture<Void> closeServiceInstance() {
+        LOG.debug("FRS stopped for: {}", nodeId.getValue());
+        deviceMastered = false;
+        reconciliationRegistry.unregisterIfRegistered(nodeId);
+        return Futures.immediateFuture(null);
+    }
+
+    @Override
+    public ServiceGroupIdentifier getIdentifier() {
+        return identifier;
+    }
+
+    public boolean isDeviceMastered() {
+        return deviceMastered;
+    }
+
+    public void setClusterSingletonServiceRegistration(final ClusterSingletonServiceRegistration registration) {
+        this.clusterSingletonServiceRegistration = registration;
+    }
+
+    public ClusterSingletonServiceRegistration getClusterSingletonServiceRegistration() {
+        return clusterSingletonServiceRegistration;
+    }
+
+}
diff --git a/applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/clustering/DeviceMastershipManager.java b/applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/impl/clustering/DeviceMastershipManager.java
new file mode 100644 (file)
index 0000000..41f736c
--- /dev/null
@@ -0,0 +1,65 @@
+/*
+ * Copyright (c) 2016 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.openflowplugin.applications.frsync.impl.clustering;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.concurrent.ConcurrentHashMap;
+import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceProvider;
+import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceRegistration;
+import org.opendaylight.openflowplugin.applications.frsync.util.ReconciliationRegistry;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Manager for clustering service registrations of {@link DeviceMastership}.
+ */
+public class DeviceMastershipManager {
+    private static final Logger LOG = LoggerFactory.getLogger(DeviceMastershipManager.class);
+    private final ClusterSingletonServiceProvider clusterSingletonService;
+    private final ConcurrentHashMap<NodeId, DeviceMastership> deviceMasterships = new ConcurrentHashMap();
+    private final ReconciliationRegistry reconciliationRegistry;
+
+    public DeviceMastershipManager(final ClusterSingletonServiceProvider clusterSingletonService,
+                                   final ReconciliationRegistry reconciliationRegistry) {
+        this.clusterSingletonService = clusterSingletonService;
+        this.reconciliationRegistry = reconciliationRegistry;
+    }
+
+    public void onDeviceConnected(final NodeId nodeId) {
+        final DeviceMastership mastership = new DeviceMastership(nodeId, reconciliationRegistry);
+        final ClusterSingletonServiceRegistration registration = clusterSingletonService.registerClusterSingletonService(mastership);
+        mastership.setClusterSingletonServiceRegistration(registration);
+        deviceMasterships.put(nodeId, mastership);
+        LOG.debug("FRS service registered for: {}", nodeId.getValue());
+    }
+
+
+    public void onDeviceDisconnected(final NodeId nodeId) {
+        final DeviceMastership mastership = deviceMasterships.remove(nodeId);
+        final ClusterSingletonServiceRegistration registration = mastership.getClusterSingletonServiceRegistration();
+        if (registration != null) {
+            try {
+                registration.close();
+            } catch (Exception e) {
+                LOG.error("FRS cluster service close fail: {}", nodeId.getValue());
+            }
+        }
+        LOG.debug("FRS service unregistered for: {}", nodeId.getValue());
+    }
+
+    public boolean isDeviceMastered(final NodeId nodeId) {
+        return deviceMasterships.get(nodeId) != null && deviceMasterships.get(nodeId).isDeviceMastered();
+    }
+
+    @VisibleForTesting
+    ConcurrentHashMap<NodeId, DeviceMastership> getDeviceMasterships() {
+        return deviceMasterships;
+    }
+}
@@ -1,12 +1,12 @@
-/**
- * Copyright (c) 2016 Cisco Systems, Inc. and others.  All rights reserved.
+/*
+ * Copyright (c) 2016 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.openflowplugin.applications.frsync.impl;
+package org.opendaylight.openflowplugin.applications.frsync.impl.strategy;
 
 import com.google.common.base.Preconditions;
 import java.util.concurrent.Future;
@@ -126,7 +126,7 @@ public class FlowForwarder implements ForwardingRulesCommitter<Flow, AddFlowOutp
         Preconditions.checkNotNull(tableKey, "TableKey can not be null or empty!");
         Preconditions.checkNotNull(flow, "Flow can not be null or empty!");
         if (!tableKey.getId().equals(flow.getTableId())) {
-            LOG.warn("TableID in URI tableId={} and in palyload tableId={} is not same.",
+            LOG.warn("TableID in URI tableId={} and in payload tableId={} is not same.",
                     flow.getTableId(), tableKey.getId());
             return false;
         }
@@ -1,12 +1,12 @@
-/**
- * Copyright (c) 2016 Cisco Systems, Inc. and others.  All rights reserved.
+/*
+ * Copyright (c) 2016 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.openflowplugin.applications.frsync.impl;
+package org.opendaylight.openflowplugin.applications.frsync.impl.strategy;
 
 import java.util.concurrent.Future;
 import org.opendaylight.openflowplugin.applications.frsync.ForwardingRulesCommitter;
@@ -1,12 +1,12 @@
-/**
- * Copyright (c) 2016 Cisco Systems, Inc. and others.  All rights reserved.
+/*
+ * Copyright (c) 2016 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.openflowplugin.applications.frsync.impl;
+package org.opendaylight.openflowplugin.applications.frsync.impl.strategy;
 
 import java.util.concurrent.Future;
 import org.opendaylight.openflowplugin.applications.frsync.ForwardingRulesCommitter;
index ed90d2854624002de508eda5556f6ce1b3fca027..70ce219708a1480efdca8f214dd9e51898e689f9 100644 (file)
@@ -24,7 +24,6 @@ import java.util.Map;
 import java.util.concurrent.Future;
 import javax.annotation.Nullable;
 import org.opendaylight.openflowplugin.applications.frsync.SyncPlanPushStrategy;
-import org.opendaylight.openflowplugin.applications.frsync.impl.TableForwarder;
 import org.opendaylight.openflowplugin.applications.frsync.util.FxChainUtil;
 import org.opendaylight.openflowplugin.applications.frsync.util.ItemSyncBox;
 import org.opendaylight.openflowplugin.applications.frsync.util.PathUtil;
@@ -129,13 +128,6 @@ public class SyncPlanPushStrategyFlatBatchImpl implements SyncPlanPushStrategy {
         resultVehicle = Futures.transform(resultVehicle, new AsyncFunction<RpcResult<Void>, RpcResult<Void>>() {
             @Override
             public ListenableFuture<RpcResult<Void>> apply(final RpcResult<Void> input) throws Exception {
-                if (!input.isSuccessful()) {
-                    //TODO chain errors but not skip processing on first error return Futures.immediateFuture(input);
-                    //final ListenableFuture<RpcResult<Void>> singleVoidUpdateResult = Futures.transform(
-                    //        Futures.asList Arrays.asList(input, output),
-                    //        ReconcileUtil.<UpdateFlowOutput>createRpcResultCondenser("TODO"));
-                }
-
                 final List<Batch> batchBag = new ArrayList<>();
                 int batchOrder = 0;
 
@@ -151,7 +143,8 @@ public class SyncPlanPushStrategyFlatBatchImpl implements SyncPlanPushStrategy {
 
                 final ProcessFlatBatchInput flatBatchInput = new ProcessFlatBatchInputBuilder()
                         .setNode(new NodeRef(PathUtil.digNodePath(diffInput.getNodeIdent())))
-                        .setExitOnFirstError(false) // TODO: propagate from input
+                        // TODO: propagate from input
+                        .setExitOnFirstError(false)
                         .setBatch(batchBag)
                         .build();
 
@@ -237,12 +230,6 @@ public class SyncPlanPushStrategyFlatBatchImpl implements SyncPlanPushStrategy {
         return batchMap;
     }
 
-    private int getNextBatchLimit(final PeekingIterator<Batch> inputBatchIterator, final int failureIndexLimit) {
-        return inputBatchIterator.hasNext()
-                ? inputBatchIterator.peek().getBatchOrder()
-                : failureIndexLimit;
-    }
-
     @VisibleForTesting
     static int assembleRemoveFlows(final List<Batch> batchBag, int batchOrder, final Map<TableKey, ItemSyncBox<Flow>> flowItemSyncTableMap) {
         // process flow remove
@@ -395,23 +382,21 @@ public class SyncPlanPushStrategyFlatBatchImpl implements SyncPlanPushStrategy {
     @VisibleForTesting
     static int assembleRemoveMeters(final List<Batch> batchBag, int batchOrder, final ItemSyncBox<Meter> meterItemSyncBox) {
         // process meter remove
-        if (meterItemSyncBox != null) {
-            if (!meterItemSyncBox.getItemsToPush().isEmpty()) {
-                final List<FlatBatchRemoveMeter> flatBatchRemoveMeterBag =
-                        new ArrayList<>(meterItemSyncBox.getItemsToUpdate().size());
-                int itemOrder = 0;
-                for (Meter meter : meterItemSyncBox.getItemsToPush()) {
-                    flatBatchRemoveMeterBag.add(new FlatBatchRemoveMeterBuilder(meter).setBatchOrder(itemOrder++).build());
-                }
-                final Batch batch = new BatchBuilder()
-                        .setBatchChoice(new FlatBatchRemoveMeterCaseBuilder()
-                                .setFlatBatchRemoveMeter(flatBatchRemoveMeterBag)
-                                .build())
-                        .setBatchOrder(batchOrder)
-                        .build();
-                batchOrder += itemOrder;
-                batchBag.add(batch);
+        if (meterItemSyncBox != null && !meterItemSyncBox.getItemsToPush().isEmpty()) {
+            final List<FlatBatchRemoveMeter> flatBatchRemoveMeterBag =
+                    new ArrayList<>(meterItemSyncBox.getItemsToUpdate().size());
+            int itemOrder = 0;
+            for (Meter meter : meterItemSyncBox.getItemsToPush()) {
+                flatBatchRemoveMeterBag.add(new FlatBatchRemoveMeterBuilder(meter).setBatchOrder(itemOrder++).build());
             }
+            final Batch batch = new BatchBuilder()
+                    .setBatchChoice(new FlatBatchRemoveMeterCaseBuilder()
+                            .setFlatBatchRemoveMeter(flatBatchRemoveMeterBag)
+                            .build())
+                    .setBatchOrder(batchOrder)
+                    .build();
+            batchOrder += itemOrder;
+            batchBag.add(batch);
         }
         return batchOrder;
     }
index 3f4b53efdb850e00fbe6792a3000fc50d54eb415..6d8ca69066e8ae0e1bc80e2ee0ed498d06231340 100644 (file)
@@ -18,10 +18,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import org.opendaylight.openflowplugin.applications.frsync.SyncPlanPushStrategy;
-import org.opendaylight.openflowplugin.applications.frsync.impl.FlowForwarder;
-import org.opendaylight.openflowplugin.applications.frsync.impl.GroupForwarder;
-import org.opendaylight.openflowplugin.applications.frsync.impl.MeterForwarder;
-import org.opendaylight.openflowplugin.applications.frsync.impl.TableForwarder;
 import org.opendaylight.openflowplugin.applications.frsync.util.CrudCounts;
 import org.opendaylight.openflowplugin.applications.frsync.util.FxChainUtil;
 import org.opendaylight.openflowplugin.applications.frsync.util.ItemSyncBox;
@@ -67,9 +63,9 @@ public class SyncPlanPushStrategyIncrementalImpl implements SyncPlanPushStrategy
     private static final Logger LOG = LoggerFactory.getLogger(SyncPlanPushStrategyIncrementalImpl.class);
 
     private FlowForwarder flowForwarder;
-    private TableForwarder tableForwarder;
     private MeterForwarder meterForwarder;
     private GroupForwarder groupForwarder;
+    private TableForwarder tableForwarder;
     private FlowCapableTransactionService transactionService;
 
     @Override
@@ -153,9 +149,9 @@ public class SyncPlanPushStrategyIncrementalImpl implements SyncPlanPushStrategy
 
 
     ListenableFuture<RpcResult<Void>> addMissingFlows(final NodeId nodeId,
-                                                                final InstanceIdentifier<FlowCapableNode> nodeIdent,
-                                                                final Map<TableKey, ItemSyncBox<Flow>> flowsInTablesSyncBox,
-                                                                final SyncCrudCounters counters) {
+                                                      final InstanceIdentifier<FlowCapableNode> nodeIdent,
+                                                      final Map<TableKey, ItemSyncBox<Flow>> flowsInTablesSyncBox,
+                                                      final SyncCrudCounters counters) {
         if (flowsInTablesSyncBox.isEmpty()) {
             LOG.trace("no tables in config for node: {} -> SKIPPING", nodeId.getValue());
             return RpcResultBuilder.<Void>success().buildFuture();
@@ -204,22 +200,14 @@ public class SyncPlanPushStrategyIncrementalImpl implements SyncPlanPushStrategy
                 Futures.allAsList(allUpdateResults),
                 ReconcileUtil.<UpdateFlowOutput>createRpcResultCondenser("flow updating"));
 
-        final ListenableFuture<RpcResult<Void>> summaryResult = Futures.transform(
-                Futures.allAsList(singleVoidAddResult, singleVoidUpdateResult),
+        return Futures.transform(Futures.allAsList(singleVoidAddResult, singleVoidUpdateResult),
                 ReconcileUtil.<Void>createRpcResultCondenser("flow add/update"));
-
-        return summaryResult;
-
-        /*
-        return Futures.transform(summaryResult,
-                ReconcileUtil.chainBarrierFlush(PathUtil.digNodePath(nodeIdent), transactionService));
-                */
     }
 
     ListenableFuture<RpcResult<Void>> removeRedundantFlows(final NodeId nodeId,
-                                                                     final InstanceIdentifier<FlowCapableNode> nodeIdent,
-                                                                     final Map<TableKey, ItemSyncBox<Flow>> removalPlan,
-                                                                     final SyncCrudCounters counters) {
+                                                           final InstanceIdentifier<FlowCapableNode> nodeIdent,
+                                                           final Map<TableKey, ItemSyncBox<Flow>> removalPlan,
+                                                           final SyncCrudCounters counters) {
         if (removalPlan.isEmpty()) {
             LOG.trace("no tables in operational for node: {} -> SKIPPING", nodeId.getValue());
             return RpcResultBuilder.<Void>success().buildFuture();
@@ -250,9 +238,9 @@ public class SyncPlanPushStrategyIncrementalImpl implements SyncPlanPushStrategy
     }
 
     ListenableFuture<RpcResult<Void>> removeRedundantMeters(final NodeId nodeId,
-                                                                      final InstanceIdentifier<FlowCapableNode> nodeIdent,
-                                                                      final ItemSyncBox<Meter> meterRemovalPlan,
-                                                                      final SyncCrudCounters counters) {
+                                                            final InstanceIdentifier<FlowCapableNode> nodeIdent,
+                                                            final ItemSyncBox<Meter> meterRemovalPlan,
+                                                            final SyncCrudCounters counters) {
         if (meterRemovalPlan.isEmpty()) {
             LOG.trace("no meters on device for node: {} -> SKIPPING", nodeId.getValue());
             return RpcResultBuilder.<Void>success().buildFuture();
@@ -271,14 +259,8 @@ public class SyncPlanPushStrategyIncrementalImpl implements SyncPlanPushStrategy
             meterCrudCounts.incRemoved();
         }
 
-        final ListenableFuture<RpcResult<Void>> singleVoidResult = Futures.transform(
-                Futures.allAsList(allResults),
+        return Futures.transform(Futures.allAsList(allResults),
                 ReconcileUtil.<RemoveMeterOutput>createRpcResultCondenser("meter remove"));
-        return singleVoidResult;
-        /*
-        return Futures.transform(singleVoidResult,
-                ReconcileUtil.chainBarrierFlush(PathUtil.digNodePath(nodeIdent), transactionService));
-                */
     }
 
     ListenableFuture<RpcResult<Void>> removeRedundantGroups(final NodeId nodeId,
@@ -407,15 +389,14 @@ public class SyncPlanPushStrategyIncrementalImpl implements SyncPlanPushStrategy
                 ReconcileUtil.<Void>createRpcResultCondenser("group add/update"));
 
 
-        return Futures.transform(summaryResult,
-                ReconcileUtil.chainBarrierFlush(
-                        PathUtil.digNodePath(nodeIdent), transactionService));
+        return Futures.transform(summaryResult, ReconcileUtil.chainBarrierFlush(
+                PathUtil.digNodePath(nodeIdent), transactionService));
     }
 
     ListenableFuture<RpcResult<Void>> addMissingMeters(final NodeId nodeId,
-                                                                 final InstanceIdentifier<FlowCapableNode> nodeIdent,
-                                                                 final ItemSyncBox<Meter> syncBox,
-                                                                 final SyncCrudCounters counters) {
+                                                       final InstanceIdentifier<FlowCapableNode> nodeIdent,
+                                                       final ItemSyncBox<Meter> syncBox,
+                                                       final SyncCrudCounters counters) {
         if (syncBox.isEmpty()) {
             LOG.trace("no meters configured for node: {} -> SKIPPING", nodeId.getValue());
             return RpcResultBuilder.<Void>success().buildFuture();
@@ -451,22 +432,14 @@ public class SyncPlanPushStrategyIncrementalImpl implements SyncPlanPushStrategy
                 Futures.allAsList(allUpdateResults),
                 ReconcileUtil.<UpdateMeterOutput>createRpcResultCondenser("meter update"));
 
-        final ListenableFuture<RpcResult<Void>> summaryResults = Futures.transform(
-                Futures.allAsList(singleVoidUpdateResult, singleVoidAddResult),
+        return Futures.transform(Futures.allAsList(singleVoidUpdateResult, singleVoidAddResult),
                 ReconcileUtil.<Void>createRpcResultCondenser("meter add/update"));
-
-        return summaryResults;
-
-        /*
-        return Futures.transform(summaryResults,
-                ReconcileUtil.chainBarrierFlush(PathUtil.digNodePath(nodeIdent), transactionService));
-                */
     }
 
     ListenableFuture<RpcResult<Void>> addMissingGroups(final NodeId nodeId,
-                                                                 final InstanceIdentifier<FlowCapableNode> nodeIdent,
-                                                                 final List<ItemSyncBox<Group>> groupsAddPlan,
-                                                                 final SyncCrudCounters counters) {
+                                                       final InstanceIdentifier<FlowCapableNode> nodeIdent,
+                                                       final List<ItemSyncBox<Group>> groupsAddPlan,
+                                                       final SyncCrudCounters counters) {
         if (groupsAddPlan.isEmpty()) {
             LOG.trace("no groups configured for node: {} -> SKIPPING", nodeId.getValue());
             return RpcResultBuilder.<Void>success().buildFuture();
@@ -1,12 +1,12 @@
-/**
- * Copyright (c) 2016 Cisco Systems, Inc. and others.  All rights reserved.
+/*
+ * Copyright (c) 2016 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.openflowplugin.applications.frsync.impl;
+package org.opendaylight.openflowplugin.applications.frsync.impl.strategy;
 
 import java.util.Collections;
 import java.util.concurrent.Future;
index 52d0623d00d7a35ce044ce464525bdd7ee768b37..cd92bb083e3f6771e32fe6fc84edb0ac68c6f62a 100644 (file)
@@ -14,7 +14,6 @@ import java.util.List;
 import java.util.Map;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
-import org.opendaylight.openflowplugin.applications.frsync.markandsweep.SwitchFlowId;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.meters.Meter;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.Table;
index 7eddebb01eb6cc566debf741015b5170aaa66ca8..903dd207f018b2de94fcf6e503d7927d7b464648 100644 (file)
@@ -40,13 +40,13 @@ public class FxChainUtil {
                         LOG.debug(prefix + " failed: {} -> {}", nodeId.getValue(), Arrays.toString(errors.toArray()));
                     }
                 } else {
-                    LOG.debug(prefix + "reconciliation failed: {} -> null result", nodeId.getValue());
+                    LOG.debug(prefix + " reconciliation failed: {} -> null result", nodeId.getValue());
                 }
             }
 
             @Override
             public void onFailure(final Throwable t) {
-                LOG.debug(prefix + "reconciliation failed seriously: {}", nodeId.getValue(), t);
+                LOG.debug(prefix + " reconciliation failed seriously: {}", nodeId.getValue(), t);
             }
         };
     }
index 1e9b49ea9ae1d1b70ea53cab3e63f0d7b37d6c2f..5067a13523e0f81c10f52f3ae5155f5c8658cc88 100644 (file)
@@ -17,8 +17,8 @@ import java.util.Set;
  */
 public class ItemSyncBox<I> {
 
-    private Set<I> itemsToPush = new LinkedHashSet<>();
-    private Set<ItemUpdateTuple<I>> itemsToUpdate = new LinkedHashSet<>();
+    private final Set<I> itemsToPush = new LinkedHashSet<>();
+    private final Set<ItemUpdateTuple<I>> itemsToUpdate = new LinkedHashSet<>();
 
     public Set<I> getItemsToPush() {
         return itemsToPush;
diff --git a/applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/ModificationUtil.java b/applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/ModificationUtil.java
new file mode 100644 (file)
index 0000000..1df1f7c
--- /dev/null
@@ -0,0 +1,54 @@
+/**
+ * Copyright (c) 2016 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.openflowplugin.applications.frsync.util;
+
+import org.opendaylight.controller.md.sal.binding.api.DataObjectModification;
+import org.opendaylight.controller.md.sal.binding.api.DataTreeModification;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.nodes.Node;
+
+/**
+ * Basic {@link DataTreeModification} related tools.
+ */
+public class ModificationUtil {
+    public static String nodeIdValue(DataTreeModification<Node> modification) {
+        final NodeId nodeId = nodeId(modification);
+
+        if (nodeId == null) {
+            return null;
+        }
+
+        return nodeId.getValue();
+    }
+
+    public static NodeId nodeId(DataTreeModification<Node> modification) {
+        final DataObjectModification<Node> rootNode = modification.getRootNode();
+        final Node dataAfter = rootNode.getDataAfter();
+
+        if (dataAfter != null) {
+            return dataAfter.getId();
+        }
+
+        final Node dataBefore = rootNode.getDataBefore();
+        if (dataBefore != null) {
+            return dataBefore.getId();
+        }
+
+        return null;
+    }
+
+    public static FlowCapableNode flowCapableNodeAfter(DataTreeModification<Node> modification) {
+        final Node dataAfter = modification.getRootNode().getDataAfter();
+        if (dataAfter == null) {
+            return null;
+        }
+        return dataAfter.getAugmentation(FlowCapableNode.class);
+    }
+}
index e339ccdd4a7f181077a8e25bdf8dde17e7d9c753..79fd17a69774be0ca6c2587ab4009fcfd2fa83b7 100644 (file)
@@ -8,14 +8,12 @@
 
 package org.opendaylight.openflowplugin.applications.frsync.util;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.AsyncFunction;
 import com.google.common.util.concurrent.JdkFutureAdapters;
-import com.google.common.util.concurrent.ListenableFuture;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -26,7 +24,6 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import javax.annotation.Nullable;
-import org.opendaylight.openflowplugin.applications.frsync.markandsweep.SwitchFlowId;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.action.types.rev131112.action.action.GroupActionCase;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.action.types.rev131112.action.list.Action;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
@@ -129,14 +126,11 @@ public class ReconcileUtil {
     public static AsyncFunction<RpcResult<Void>, RpcResult<Void>> chainBarrierFlush(
             final InstanceIdentifier<Node> nodeIdent,
             final FlowCapableTransactionService flowCapableTransactionService) {
-        return new AsyncFunction<RpcResult<Void>, RpcResult<Void>>() {
-            @Override
-            public ListenableFuture<RpcResult<Void>> apply(final RpcResult<Void> input) throws Exception {
-                final SendBarrierInput barrierInput = new SendBarrierInputBuilder()
-                        .setNode(new NodeRef(nodeIdent))
-                        .build();
-                return JdkFutureAdapters.listenInPoolThread(flowCapableTransactionService.sendBarrier(barrierInput));
-            }
+        return input -> {
+            final SendBarrierInput barrierInput = new SendBarrierInputBuilder()
+                    .setNode(new NodeRef(nodeIdent))
+                    .build();
+            return JdkFutureAdapters.listenInPoolThread(flowCapableTransactionService.sendBarrier(barrierInput));
         };
     }
 
@@ -262,7 +256,7 @@ public class ReconcileUtil {
                                                        final Map<MeterId, Meter> meterOperationalMap,
                                                        final List<Meter> metersConfigured,
                                                        final boolean gatherUpdates) {
-        LOG.trace("resolving meters for {}", nodeId);
+        LOG.trace("resolving meters for {}", nodeId.getValue());
         final ItemSyncBox<Meter> syncBox = new ItemSyncBox<>();
         for (Meter meter : metersConfigured) {
             final Meter existingMeter = meterOperationalMap.get(meter.getMeterId());
@@ -284,8 +278,7 @@ public class ReconcileUtil {
      * @param gatherUpdates      check content of pending item if present on device (and create update task eventually)
      * @return list of safe synchronization steps
      */
-    @VisibleForTesting
-    static ItemSyncBox<Flow> resolveFlowDiffsInTable(final List<Flow> flowsConfigured,
+    public static ItemSyncBox<Flow> resolveFlowDiffsInTable(final List<Flow> flowsConfigured,
                                                      final Map<SwitchFlowId, Flow> flowOperationalMap,
                                                      final boolean gatherUpdates) {
         final ItemSyncBox<Flow> flowsSyncBox = new ItemSyncBox<>();
@@ -316,7 +309,7 @@ public class ReconcileUtil {
                                                                                final Map<Short, Table> tableOperationalMap,
                                                                                final List<Table> tablesConfigured,
                                                                                final boolean gatherUpdates) {
-        LOG.trace("resolving flows in tables for {}", nodeId);
+        LOG.trace("resolving flows in tables for {}", nodeId.getValue());
         final Map<TableKey, ItemSyncBox<Flow>> tableFlowSyncBoxes = new HashMap<>();
         for (final Table tableConfigured : tablesConfigured) {
             final List<Flow> flowsConfigured = tableConfigured.getFlow();
@@ -16,20 +16,17 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Holder of registration request for fresh operational.
+ * Holder of registration request for reconciliation (fresh operational).
  */
-public class RetryRegistry {
-
-    private static final Logger LOG = LoggerFactory.getLogger(RetryRegistry.class);
+public class ReconciliationRegistry {
+    private static final Logger LOG = LoggerFactory.getLogger(ReconciliationRegistry.class);
     private final Map<NodeId, Date> registration = new ConcurrentHashMap<>();
-    public static final String DATE_AND_TIME_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSSXXX";
 
     public Date register(NodeId nodeId) {
         Date timestamp = new Date();
         registration.put(nodeId, timestamp);
-        if (timestamp != null) {
-            LOG.debug("Registered for next consistent operational: {}", nodeId.getValue());
-        }
+        LOG.debug("Registered for next consistent operational: {}", nodeId.getValue());
+        // TODO  elicit statistics gathering if not running actually
         return timestamp;
     }
 
@@ -45,7 +42,7 @@ public class RetryRegistry {
         return registration.get(nodeId) != null;
     }
 
-    public Date getRegistration(NodeId nodeId) {
+    public Date getRegistrationTimestamp(NodeId nodeId) {
         return registration.get(nodeId);
     }
 
index 419525a1326b97c4d2e008bb47d3fa8faf1df534..240905bdf28929a5b9083bfd6a5d827487660f67 100644 (file)
@@ -20,7 +20,7 @@ import org.opendaylight.openflowplugin.applications.frsync.SemaphoreKeeper;
  */
 public class SemaphoreKeeperGuavaImpl<K> implements SemaphoreKeeper<K> {
 
-    private LoadingCache<K, Semaphore> semaphoreCache;
+    private final LoadingCache<K, Semaphore> semaphoreCache;
 
     public SemaphoreKeeperGuavaImpl(final int permits, final boolean fair) {
         semaphoreCache = CacheBuilder.newBuilder()
@@ -6,7 +6,7 @@
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
 
-package org.opendaylight.openflowplugin.applications.frsync.markandsweep;
+package org.opendaylight.openflowplugin.applications.frsync.util;
 
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.table.Flow;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.types.rev131026.flow.Match;
diff --git a/applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/SyncupEntry.java b/applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/SyncupEntry.java
new file mode 100644 (file)
index 0000000..151724c
--- /dev/null
@@ -0,0 +1,74 @@
+/**
+ * Copyright (c) 2016 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.openflowplugin.applications.frsync.util;
+
+import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
+
+/**
+ * Data entry of before and after data for syncup in {@link org.opendaylight.openflowplugin.applications.frsync.SyncReactor}.
+ */
+public class SyncupEntry {
+    private final FlowCapableNode after;
+    private final LogicalDatastoreType dsTypeAfter;
+    private final FlowCapableNode before;
+    private final LogicalDatastoreType dsTypeBefore;
+
+    public SyncupEntry(final FlowCapableNode after, final LogicalDatastoreType dsTypeAfter,
+                       final FlowCapableNode before, final LogicalDatastoreType dsTypeBefore) {
+        this.after = after;
+        this.dsTypeAfter = dsTypeAfter;
+        this.before = before;
+        this.dsTypeBefore = dsTypeBefore;
+    }
+
+    public FlowCapableNode getAfter() {
+        return after;
+    }
+
+    public FlowCapableNode getBefore() {
+        return before;
+    }
+
+    public LogicalDatastoreType getDsTypeAfter() {
+        return dsTypeAfter;
+    }
+
+    public LogicalDatastoreType getDsTypeBefore() {
+        return dsTypeBefore;
+    }
+
+    public boolean isOptimizedConfigDelta() {
+        return dsTypeAfter == LogicalDatastoreType.CONFIGURATION && dsTypeBefore == LogicalDatastoreType.CONFIGURATION;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        SyncupEntry that = (SyncupEntry) o;
+
+        if (after != null ? !after.equals(that.after) : that.after != null) {
+            return false;
+        }
+        if (dsTypeAfter != that.dsTypeAfter) {
+            return false;
+        }
+        if (before != null ? !before.equals(that.before) : that.before != null) {
+            return false;
+        }
+        return dsTypeBefore == that.dsTypeBefore;
+    }
+
+}
diff --git a/applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/ZipQueueEntry.java b/applications/forwardingrules-sync/src/main/java/org/opendaylight/openflowplugin/applications/frsync/util/ZipQueueEntry.java
deleted file mode 100644 (file)
index ec4d8a8..0000000
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Copyright (c) 2016 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.openflowplugin.applications.frsync.util;
-
-import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
-import org.opendaylight.openflowplugin.applications.frsync.impl.SyncReactorFutureZipDecorator;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
-
-/**
- * Simple compression queue entry for {@link SyncReactorFutureZipDecorator}.
- */
-public class ZipQueueEntry {
-    private final FlowCapableNode after;
-    private final FlowCapableNode before;
-    private final LogicalDatastoreType dsTypeBefore;
-
-    public ZipQueueEntry(final FlowCapableNode after, final FlowCapableNode before,
-                         final LogicalDatastoreType dsTypeBefore) {
-        this.after = after;
-        this.before = before;
-        this.dsTypeBefore = dsTypeBefore;
-    }
-
-    public FlowCapableNode getLeft() {
-        return after;
-    }
-
-    public FlowCapableNode getRight() {
-        return before;
-    }
-
-    public LogicalDatastoreType getDsType() {
-        return dsTypeBefore;
-    }
-
-}
index c60e9600e736638782cea112a49c5651e5506602..e1f24449239d298a1f222537faa40fbfe5317d58 100644 (file)
            odl:use-default-for-reference-types="true">
 
     <reference id="broker" interface="org.opendaylight.controller.sal.binding.api.BindingAwareBroker"/>
-    <reference id="dataBroker" interface="org.opendaylight.controller.md.sal.binding.api.DataBroker"/>
+    <reference id="dataBroker" interface="org.opendaylight.controller.md.sal.binding.api.DataBroker" odl:type="pingpong"/>
     <reference id="rpcRegistry" interface="org.opendaylight.controller.sal.binding.api.RpcProviderRegistry"/>
+    <reference id="clusterSingletonService" interface="org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceProvider"/>
 
     <bean id="frSync" class="org.opendaylight.openflowplugin.applications.frsync.impl.ForwardingRulesSyncProvider"
           destroy-method="close">
         <argument ref="broker"/>
         <argument ref="dataBroker"/>
         <argument ref="rpcRegistry"/>
+        <argument ref="clusterSingletonService"/>
     </bean>
 
 </blueprint>
\ No newline at end of file
index 76fb1fad803b57f91e90879625058f6bb523502f..b795b2f728b73f8059ec48a185f2a90e14c6397a 100644 (file)
@@ -20,7 +20,9 @@ import org.opendaylight.controller.md.sal.binding.api.DataBroker;
 import org.opendaylight.controller.md.sal.binding.api.DataTreeChangeListener;
 import org.opendaylight.controller.md.sal.binding.api.DataTreeIdentifier;
 import org.opendaylight.controller.sal.binding.api.BindingAwareBroker;
+import org.opendaylight.controller.sal.binding.api.BindingAwareBroker.ProviderContext;
 import org.opendaylight.controller.sal.binding.api.RpcConsumerRegistry;
+import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceProvider;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flat.batch.service.rev160321.SalFlatBatchService;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.table.service.rev131026.SalTableService;
@@ -40,7 +42,9 @@ public class ForwardingRulesSyncProviderTest {
     @Mock
     private BindingAwareBroker broker;
     @Mock
-    private BindingAwareBroker.ProviderContext providerContext;
+    private ProviderContext providerContext;
+    @Mock
+    private ClusterSingletonServiceProvider clusterSingletonService;
 
     @Before
     public void setUp() throws Exception {
@@ -53,7 +57,7 @@ public class ForwardingRulesSyncProviderTest {
                     }
                 });
 
-        provider = new ForwardingRulesSyncProvider(broker, dataBroker, rpcRegistry);
+        provider = new ForwardingRulesSyncProvider(broker, dataBroker, rpcRegistry, clusterSingletonService);
         Mockito.verify(rpcRegistry).getRpcService(SalTableService.class);
         Mockito.verify(rpcRegistry).getRpcService(SalFlatBatchService.class);
         Mockito.verify(broker).registerProvider(provider);
index 4c20088ded8fa1897bc91bf5c00357c90e348e02..888f31776109fed27b87a1b78235cf6107b371f9 100644 (file)
@@ -30,6 +30,7 @@ import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeCa
 import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeDao;
 import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeOdlDao;
 import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeSnapshotDao;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.Nodes;
@@ -46,7 +47,8 @@ public class SimplifiedConfigListenerTest {
     private static final NodeId NODE_ID = new NodeId("testNode");
     private InstanceIdentifier<FlowCapableNode> fcNodePath;
     private SimplifiedConfigListener nodeListenerConfig;
-    private LogicalDatastoreType dsType = LogicalDatastoreType.CONFIGURATION;
+    private final LogicalDatastoreType confgDS = LogicalDatastoreType.CONFIGURATION;
+    private final LogicalDatastoreType operationalDS = LogicalDatastoreType.OPERATIONAL;
 
     @Mock
     private SyncReactor reactor;
@@ -79,8 +81,6 @@ public class SimplifiedConfigListenerTest {
         Mockito.when(db.newReadOnlyTransaction()).thenReturn(roTx);
         Mockito.when(dataTreeModification.getRootPath()).thenReturn(dataTreeIdentifier);
         Mockito.when(dataTreeModification.getRootNode()).thenReturn(configModification);
-        Mockito.when(reactor.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Matchers.<FlowCapableNode>any(),
-                Matchers.<FlowCapableNode>any(), Matchers.<LogicalDatastoreType>any())).thenReturn(Futures.immediateFuture(Boolean.TRUE));
     }
 
     @Test
@@ -89,41 +89,40 @@ public class SimplifiedConfigListenerTest {
     }
 
     @Test
-    public void testOnDataTreeChangedSyncupAdd() throws InterruptedException {
-        Mockito.when(roTx.read(LogicalDatastoreType.OPERATIONAL, fcNodePath))
-                .thenReturn(Futures.immediateCheckedFuture(Optional.of(dataBefore)));
+    public void testOnDataTreeChangedAdd() throws InterruptedException {
+        Mockito.when(configModification.getDataBefore()).thenReturn(null);
         Mockito.when(configModification.getDataAfter()).thenReturn(dataAfter);
+        final SyncupEntry syncupEntry = loadOperationalDSAndPrepareSyncupEntry(dataAfter, confgDS, dataBefore, operationalDS);
 
         nodeListenerConfig.onDataTreeChanged(Collections.singleton(dataTreeModification));
 
-        Mockito.verify(reactor).syncup(fcNodePath, dataAfter, dataBefore, dsType);
+        Mockito.verify(reactor).syncup(fcNodePath, syncupEntry);
         Mockito.verifyNoMoreInteractions(reactor);
         Mockito.verify(roTx).close();
     }
 
     @Test
-    public void testOnDataTreeChangedSyncupUpdate() throws InterruptedException {
-        Mockito.when(roTx.read(LogicalDatastoreType.OPERATIONAL, fcNodePath))
-                .thenReturn(Futures.immediateCheckedFuture(Optional.of(dataBefore)));
+    public void testOnDataTreeChangedUpdate() throws InterruptedException {
         Mockito.when(configModification.getDataBefore()).thenReturn(dataBefore);
         Mockito.when(configModification.getDataAfter()).thenReturn(dataAfter);
+        final SyncupEntry syncupEntry = loadOperationalDSAndPrepareSyncupEntry(dataAfter, confgDS, dataBefore, confgDS);
 
         nodeListenerConfig.onDataTreeChanged(Collections.singleton(dataTreeModification));
 
-        Mockito.verify(reactor).syncup(fcNodePath, dataAfter, dataBefore, dsType);
+        Mockito.verify(reactor).syncup(fcNodePath, syncupEntry);
         Mockito.verifyNoMoreInteractions(reactor);
         Mockito.verify(roTx).close();
     }
 
     @Test
-    public void testOnDataTreeChangedSyncupDelete() throws InterruptedException {
-        Mockito.when(roTx.read(LogicalDatastoreType.OPERATIONAL, fcNodePath))
-                .thenReturn(Futures.immediateCheckedFuture(Optional.of(dataBefore)));
+    public void testOnDataTreeChangedDelete() throws InterruptedException {
         Mockito.when(configModification.getDataBefore()).thenReturn(dataBefore);
+        Mockito.when(configModification.getDataAfter()).thenReturn(null);
+        final SyncupEntry syncupEntry = loadOperationalDSAndPrepareSyncupEntry(null, confgDS, dataBefore, confgDS);
 
         nodeListenerConfig.onDataTreeChanged(Collections.singleton(dataTreeModification));
 
-        Mockito.verify(reactor).syncup(fcNodePath, null, dataBefore, dsType);
+        Mockito.verify(reactor).syncup(fcNodePath, syncupEntry);
         Mockito.verifyNoMoreInteractions(reactor);
         Mockito.verify(roTx).close();
     }
@@ -138,4 +137,15 @@ public class SimplifiedConfigListenerTest {
         Mockito.verifyZeroInteractions(reactor);
         Mockito.verify(roTx).close();
     }
+
+    private SyncupEntry loadOperationalDSAndPrepareSyncupEntry(final FlowCapableNode after, final LogicalDatastoreType dsTypeAfter,
+                                                               final FlowCapableNode before, final LogicalDatastoreType dsTypeBefore) throws InterruptedException {
+        Mockito.when(roTx.read(LogicalDatastoreType.OPERATIONAL, fcNodePath))
+                .thenReturn(Futures.immediateCheckedFuture(Optional.of(dataBefore)));
+        final SyncupEntry syncupEntry = new SyncupEntry(after, dsTypeAfter, before, dsTypeBefore);
+        Mockito.when(reactor.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Mockito.eq(syncupEntry)))
+                .thenReturn(Futures.immediateFuture(Boolean.TRUE));
+        return syncupEntry;
+    }
+
 }
\ No newline at end of file
index bd106a81cc94ecb6aaf2c02ba9ffabbc662ac9c9..4eb2e5b9685e7c6fc6d33c789f717af64a2fdebe 100644 (file)
@@ -1,5 +1,5 @@
-/*
- * Copyright (c) 2016 Cisco Systems, Inc. and others.  All rights reserved.
+/**
+ * Copyright (c) 2016 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,
@@ -10,6 +10,8 @@ package org.opendaylight.openflowplugin.applications.frsync.impl;
 
 import com.google.common.base.Optional;
 import com.google.common.util.concurrent.Futures;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
 import java.util.Collections;
 import java.util.List;
 import org.junit.Assert;
@@ -32,7 +34,13 @@ import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeCa
 import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeDao;
 import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeOdlDao;
 import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeSnapshotDao;
+import org.opendaylight.openflowplugin.applications.frsync.impl.clustering.DeviceMastershipManager;
+import org.opendaylight.openflowplugin.applications.frsync.util.ReconciliationRegistry;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
+import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.yang.types.rev130715.DateAndTime;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableStatisticsGatheringStatus;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.snapshot.gathering.status.grouping.SnapshotGatheringStatusEnd;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.Nodes;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.node.NodeConnector;
@@ -49,7 +57,9 @@ public class SimplifiedOperationalListenerTest {
     private static final NodeId NODE_ID = new NodeId("testNode");
     private InstanceIdentifier<FlowCapableNode> fcNodePath;
     private SimplifiedOperationalListener nodeListenerOperational;
-    private final LogicalDatastoreType dsType = LogicalDatastoreType.OPERATIONAL;
+    private final LogicalDatastoreType configDS = LogicalDatastoreType.CONFIGURATION;
+    private final LogicalDatastoreType operationalDS = LogicalDatastoreType.OPERATIONAL;
+    private final SimpleDateFormat simpleDateFormat = new SimpleDateFormat(SimplifiedOperationalListener.DATE_AND_TIME_FORMAT);
 
     @Mock
     private SyncReactor reactor;
@@ -65,6 +75,14 @@ public class SimplifiedOperationalListenerTest {
     private Node operationalNode;
     @Mock
     private FlowCapableNode fcOperationalNode;
+    @Mock
+    private FlowCapableStatisticsGatheringStatus statisticsGatheringStatus;
+    @Mock
+    private SnapshotGatheringStatusEnd snapshotGatheringStatusEnd;
+    @Mock
+    private ReconciliationRegistry reconciliationRegistry;
+    @Mock
+    private DeviceMastershipManager deviceMastershipManager;
 
     @Before
     public void setUp() throws Exception {
@@ -74,7 +92,7 @@ public class SimplifiedOperationalListenerTest {
         final FlowCapableNodeDao configDao = new FlowCapableNodeCachedDao(configSnapshot,
                 new FlowCapableNodeOdlDao(db, LogicalDatastoreType.CONFIGURATION));
 
-        nodeListenerOperational = new SimplifiedOperationalListener(reactor, operationalSnapshot, configDao);
+        nodeListenerOperational = new SimplifiedOperationalListener(reactor, operationalSnapshot, configDao, reconciliationRegistry, deviceMastershipManager);
         InstanceIdentifier<Node> nodePath = InstanceIdentifier.create(Nodes.class).child(Node.class, new NodeKey(NODE_ID));
         fcNodePath = nodePath.augmentation(FlowCapableNode.class);
 
@@ -94,51 +112,165 @@ public class SimplifiedOperationalListenerTest {
     }
 
     @Test
-    public void testOnDataTreeChangedSyncupAdd() throws InterruptedException {
-        Mockito.when(roTx.read(LogicalDatastoreType.CONFIGURATION, fcNodePath))
-                .thenReturn(Futures.immediateCheckedFuture(Optional.of(configNode)));
-        Mockito.when(reactor.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Matchers.<FlowCapableNode>any(),
-                Matchers.<FlowCapableNode>any(), Matchers.<LogicalDatastoreType>any())).thenReturn(Futures.immediateFuture(Boolean.TRUE));
-        Mockito.when(operationalModification.getDataAfter()).thenReturn(operationalNode);
+    public void testOnDataTreeChangeAddSyncup() throws Exception {
+        Mockito.when(reconciliationRegistry.isRegistered(NODE_ID)).thenReturn(true);
+        operationalAdd();
+        prepareFreshOperational(true);
+        final SyncupEntry syncupEntry = loadConfigDSAndPrepareSyncupEntry(configNode, configDS, fcOperationalNode, operationalDS);
 
         nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
 
-        Mockito.verify(reactor).syncup(fcNodePath, configNode, fcOperationalNode, dsType);
-        Mockito.verifyNoMoreInteractions(reactor);
+        Mockito.verify(deviceMastershipManager).onDeviceConnected(NODE_ID);
+        Mockito.verify(reactor).syncup(fcNodePath, syncupEntry);
         Mockito.verify(roTx).close();
     }
 
     @Test
-    public void testOnDataTreeChangedAddSkip() {
+    public void testOnDataTreeChangedAddSkip() throws Exception {
         // Related to bug 5920 -> https://bugs.opendaylight.org/show_bug.cgi?id=5920
+        Mockito.when(reconciliationRegistry.isRegistered(NODE_ID)).thenReturn(true);
+        operationalAdd();
+        prepareFreshOperational(true);
+
         Mockito.when(roTx.read(LogicalDatastoreType.CONFIGURATION, fcNodePath))
                 .thenReturn(Futures.immediateCheckedFuture(Optional.absent()));
-        Mockito.when(operationalModification.getDataAfter()).thenReturn(operationalNode);
 
         nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
 
+        Mockito.verify(deviceMastershipManager).onDeviceConnected(NODE_ID);
         Mockito.verifyZeroInteractions(reactor);
         Mockito.verify(roTx).close();
     }
 
     @Test
-    public void testOnDataTreeChangedSyncupDeletePhysical() {
+    public void testOnDataTreeChangedDeletePhysical() throws Exception {
         Mockito.when(operationalModification.getDataBefore()).thenReturn(operationalNode);
+        Mockito.when(operationalModification.getDataAfter()).thenReturn(null);
         Mockito.when(dataTreeModification.getRootNode().getModificationType()).thenReturn(ModificationType.DELETE);
+        Mockito.when(reconciliationRegistry.isRegistered(NODE_ID)).thenReturn(false);
 
         nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
 
+        Mockito.verify(deviceMastershipManager).onDeviceDisconnected(NODE_ID);
         Mockito.verifyZeroInteractions(reactor);
     }
 
     @Test
-    public void testOnDataTreeChangedSyncupDeleteLogical() {
+    public void testOnDataTreeChangedDeleteLogical() {
         Mockito.when(operationalModification.getDataBefore()).thenReturn(operationalNode);
         List<NodeConnector> nodeConnectorList = Mockito.mock(List.class);
         Mockito.when(operationalNode.getNodeConnector()).thenReturn(nodeConnectorList);
+        Mockito.when(reconciliationRegistry.isRegistered(NODE_ID)).thenReturn(false);
+
+        nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
+
+        Mockito.verify(deviceMastershipManager).onDeviceDisconnected(NODE_ID);
+        Mockito.verifyZeroInteractions(reactor);
+    }
+
+    @Test
+    public void testOnDataTreeChangedReconcileNotRegistered() {
+        operationalUpdate();
+        Mockito.when(reconciliationRegistry.isRegistered(NODE_ID)).thenReturn(false);
 
         nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
 
         Mockito.verifyZeroInteractions(reactor);
     }
+
+    @Test
+    public void testOnDataTreeChangedReconcileButStaticsGatheringNotStarted() {
+        Mockito.when(reconciliationRegistry.isRegistered(NODE_ID)).thenReturn(true);
+        operationalUpdate();
+        Mockito.when(operationalNode.getAugmentation(FlowCapableStatisticsGatheringStatus.class)).thenReturn(null);
+
+        nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
+
+        Mockito.verifyZeroInteractions(reactor);
+    }
+
+    @Test
+    public void testOnDataTreeChangedReconcileButStaticsGatheringNotFinished() {
+        Mockito.when(reconciliationRegistry.isRegistered(NODE_ID)).thenReturn(true);
+        operationalUpdate();
+        Mockito.when(operationalNode.getAugmentation(FlowCapableStatisticsGatheringStatus.class)).thenReturn(statisticsGatheringStatus);
+        Mockito.when(statisticsGatheringStatus.getSnapshotGatheringStatusEnd()).thenReturn(null);
+
+        nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
+
+        Mockito.verifyZeroInteractions(reactor);
+    }
+
+    @Test
+    public void testOnDataTreeChangedReconcileButStaticsGatheringNotSuccessful() {
+        Mockito.when(reconciliationRegistry.isRegistered(NODE_ID)).thenReturn(true);
+        operationalUpdate();
+        Mockito.when(operationalNode.getAugmentation(FlowCapableStatisticsGatheringStatus.class)).thenReturn(statisticsGatheringStatus);
+        Mockito.when(statisticsGatheringStatus.getSnapshotGatheringStatusEnd()).thenReturn(snapshotGatheringStatusEnd);
+        Mockito.when(snapshotGatheringStatusEnd.isSucceeded()).thenReturn(false);
+
+        nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
+
+        Mockito.verifyZeroInteractions(reactor);
+    }
+
+    @Test
+    public void testOnDataTreeChangedReconcileAndFreshOperationalNotPresent() throws ParseException {
+        Mockito.when(reconciliationRegistry.isRegistered(NODE_ID)).thenReturn(true);
+        operationalUpdate();
+        prepareFreshOperational(false);
+
+        nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
+
+        Mockito.verifyZeroInteractions(reactor);
+    }
+
+    @Test
+    public void testOnDataTreeChangedReconcileAndFreshOperationalPresent() throws Exception {
+        Mockito.when(reconciliationRegistry.isRegistered(NODE_ID)).thenReturn(true);
+        operationalUpdate();
+        prepareFreshOperational(true);
+        final SyncupEntry syncupEntry = loadConfigDSAndPrepareSyncupEntry(configNode, configDS, fcOperationalNode, operationalDS);
+
+        nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
+
+        Mockito.verify(reactor).syncup(fcNodePath, syncupEntry);
+        Mockito.verify(roTx).close();
+    }
+
+    private void prepareFreshOperational(final boolean afterRegistration) throws ParseException {
+        Mockito.when(operationalNode.getAugmentation(FlowCapableStatisticsGatheringStatus.class)).thenReturn(statisticsGatheringStatus);
+        Mockito.when(statisticsGatheringStatus.getSnapshotGatheringStatusEnd()).thenReturn(snapshotGatheringStatusEnd);
+        Mockito.when(snapshotGatheringStatusEnd.isSucceeded()).thenReturn(true);
+        Mockito.when(snapshotGatheringStatusEnd.getEnd()).thenReturn(Mockito.mock(DateAndTime.class));
+        final String timestampBefore = "0000-12-12T01:01:01.000-07:00";
+        final String timestampAfter = "9999-12-12T01:01:01.000-07:00";
+        if (afterRegistration) {
+            Mockito.when(snapshotGatheringStatusEnd.getEnd().getValue()).thenReturn(timestampAfter);
+            Mockito.when(reconciliationRegistry.getRegistrationTimestamp(NODE_ID)).thenReturn(simpleDateFormat.parse(timestampBefore));
+        } else {
+            Mockito.when(snapshotGatheringStatusEnd.getEnd().getValue()).thenReturn(timestampBefore);
+            Mockito.when(reconciliationRegistry.getRegistrationTimestamp(NODE_ID)).thenReturn(simpleDateFormat.parse(timestampAfter));
+        }
+    }
+
+    private void operationalAdd() {
+        Mockito.when(operationalModification.getDataBefore()).thenReturn(null);
+        Mockito.when(operationalModification.getDataAfter()).thenReturn(operationalNode);
+    }
+
+    private void operationalUpdate() {
+        Mockito.when(operationalModification.getDataBefore()).thenReturn(operationalNode);
+        Mockito.when(operationalModification.getDataAfter()).thenReturn(operationalNode);
+    }
+
+    private SyncupEntry loadConfigDSAndPrepareSyncupEntry(final FlowCapableNode after, final LogicalDatastoreType dsTypeAfter,
+                                                          final FlowCapableNode before, final LogicalDatastoreType dsTypeBefore) throws InterruptedException {
+        Mockito.when(roTx.read(LogicalDatastoreType.CONFIGURATION, fcNodePath))
+                .thenReturn(Futures.immediateCheckedFuture(Optional.of(configNode)));
+        final SyncupEntry syncupEntry = new SyncupEntry(after, dsTypeAfter, before, dsTypeBefore);
+        Mockito.when(reactor.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Mockito.eq(syncupEntry)))
+                .thenReturn(Futures.immediateFuture(Boolean.TRUE));
+        return syncupEntry;
+    }
 }
diff --git a/applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/SimplifiedOperationalRetryListenerTest.java b/applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/SimplifiedOperationalRetryListenerTest.java
deleted file mode 100644 (file)
index a2e6cc5..0000000
+++ /dev/null
@@ -1,205 +0,0 @@
-/**
- * Copyright (c) 2016 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.openflowplugin.applications.frsync.impl;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.Futures;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Collections;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.Matchers;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.runners.MockitoJUnitRunner;
-import org.opendaylight.controller.md.sal.binding.api.DataBroker;
-import org.opendaylight.controller.md.sal.binding.api.DataObjectModification;
-import org.opendaylight.controller.md.sal.binding.api.DataTreeIdentifier;
-import org.opendaylight.controller.md.sal.binding.api.DataTreeModification;
-import org.opendaylight.controller.md.sal.binding.api.ReadOnlyTransaction;
-import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
-import org.opendaylight.openflowplugin.applications.frsync.SyncReactor;
-import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeCachedDao;
-import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeDao;
-import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeOdlDao;
-import org.opendaylight.openflowplugin.applications.frsync.dao.FlowCapableNodeSnapshotDao;
-import org.opendaylight.openflowplugin.applications.frsync.util.RetryRegistry;
-import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.yang.types.rev130715.DateAndTime;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableStatisticsGatheringStatus;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.snapshot.gathering.status.grouping.SnapshotGatheringStatusEnd;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.Nodes;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.nodes.Node;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.nodes.NodeKey;
-import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
-
-/**
- * Test for {@link SimplifiedOperationalRetryListener}.
- */
-@RunWith(MockitoJUnitRunner.class)
-public class SimplifiedOperationalRetryListenerTest {
-
-    private final SimpleDateFormat simpleDateFormat = new SimpleDateFormat(RetryRegistry.DATE_AND_TIME_FORMAT);
-    private static final NodeId NODE_ID = new NodeId("testNode");
-    private InstanceIdentifier<FlowCapableNode> fcNodePath;
-    private SimplifiedOperationalRetryListener nodeListenerOperational;
-    private final String timestampBefore = "0000-12-12T01:01:01.000-07:00";
-    private final String timestampAfter = "9999-12-12T01:01:01.000-07:00";
-    private final LogicalDatastoreType dsType = LogicalDatastoreType.OPERATIONAL;
-
-    @Mock
-    private SyncReactor reactor;
-    @Mock
-    private ReadOnlyTransaction roTx;
-    @Mock
-    private DataTreeModification<Node> dataTreeModification;
-    @Mock
-    private DataObjectModification<Node> operationalModification;
-    @Mock
-    private FlowCapableNode configNode;
-    @Mock
-    private Node operationalNode;
-    @Mock
-    private FlowCapableNode fcOperationalNode;
-    @Mock
-    private RetryRegistry retryRegistry;
-    @Mock
-    private FlowCapableStatisticsGatheringStatus statisticsGatheringStatus;
-    @Mock
-    private SnapshotGatheringStatusEnd snapshotGatheringStatusEnd;
-
-    @Before
-    public void setUp() throws Exception {
-        final DataBroker db = Mockito.mock(DataBroker.class);
-        final FlowCapableNodeSnapshotDao configSnapshot = new FlowCapableNodeSnapshotDao();
-        final FlowCapableNodeSnapshotDao operationalSnapshot = new FlowCapableNodeSnapshotDao();
-        final FlowCapableNodeDao configDao = new FlowCapableNodeCachedDao(configSnapshot,
-                new FlowCapableNodeOdlDao(db, LogicalDatastoreType.CONFIGURATION));
-
-        nodeListenerOperational = new SimplifiedOperationalRetryListener(reactor, operationalSnapshot, configDao, retryRegistry);
-        InstanceIdentifier<Node> nodePath = InstanceIdentifier.create(Nodes.class).child(Node.class, new NodeKey(NODE_ID));
-        fcNodePath = nodePath.augmentation(FlowCapableNode.class);
-
-        final DataTreeIdentifier<Node> dataTreeIdentifier =
-                new DataTreeIdentifier<>(LogicalDatastoreType.OPERATIONAL, nodePath);
-
-        Mockito.when(db.newReadOnlyTransaction()).thenReturn(roTx);
-        Mockito.when(operationalNode.getId()).thenReturn(NODE_ID);
-        Mockito.when(dataTreeModification.getRootPath()).thenReturn(dataTreeIdentifier);
-        Mockito.when(dataTreeModification.getRootNode()).thenReturn(operationalModification);
-        Mockito.when(operationalNode.getAugmentation(FlowCapableNode.class)).thenReturn(fcOperationalNode);
-    }
-
-    @Test
-    public void testOnDataTreeChangedRetryNotRegistered() {
-        Mockito.when(operationalModification.getDataBefore()).thenReturn(operationalNode);
-        Mockito.when(operationalModification.getDataAfter()).thenReturn(operationalNode);
-        Mockito.when(retryRegistry.isRegistered(NODE_ID)).thenReturn(false);
-
-        nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
-
-        Mockito.verifyZeroInteractions(reactor);
-    }
-
-    @Test
-    public void testOnDataTreeChangedRetryButStaticsGatheringNotStarted() {
-        Mockito.when(operationalModification.getDataBefore()).thenReturn(operationalNode);
-        Mockito.when(operationalModification.getDataAfter()).thenReturn(operationalNode);
-        Mockito.when(retryRegistry.isRegistered(NODE_ID)).thenReturn(true);
-        Mockito.when(operationalNode.getAugmentation(FlowCapableStatisticsGatheringStatus.class)).thenReturn(null);
-
-        nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
-
-        Mockito.verifyZeroInteractions(reactor);
-    }
-
-    @Test
-    public void testOnDataTreeChangedRetryButStaticsGatheringNotFinished() {
-        Mockito.when(operationalModification.getDataBefore()).thenReturn(operationalNode);
-        Mockito.when(operationalModification.getDataAfter()).thenReturn(operationalNode);
-        Mockito.when(retryRegistry.isRegistered(NODE_ID)).thenReturn(true);
-        Mockito.when(operationalNode.getAugmentation(FlowCapableStatisticsGatheringStatus.class)).thenReturn(statisticsGatheringStatus);
-        Mockito.when(statisticsGatheringStatus.getSnapshotGatheringStatusEnd()).thenReturn(null);
-
-        nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
-
-        Mockito.verifyZeroInteractions(reactor);
-    }
-
-    @Test
-    public void testOnDataTreeChangedRetryButStaticsGatheringNotSuccessful() {
-        Mockito.when(operationalModification.getDataBefore()).thenReturn(operationalNode);
-        Mockito.when(operationalModification.getDataAfter()).thenReturn(operationalNode);
-        Mockito.when(retryRegistry.isRegistered(NODE_ID)).thenReturn(true);
-        Mockito.when(operationalNode.getAugmentation(FlowCapableStatisticsGatheringStatus.class)).thenReturn(statisticsGatheringStatus);
-        Mockito.when(statisticsGatheringStatus.getSnapshotGatheringStatusEnd()).thenReturn(snapshotGatheringStatusEnd);
-        Mockito.when(snapshotGatheringStatusEnd.isSucceeded()).thenReturn(false);
-
-        nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
-
-        Mockito.verifyZeroInteractions(reactor);
-    }
-
-    @Test
-    public void testOnDataTreeChangedRetryAndFreshOperationalNotPresent() throws ParseException {
-        final DateAndTime timestamp = Mockito.mock(DateAndTime.class);
-        Mockito.when(operationalModification.getDataBefore()).thenReturn(operationalNode);
-        Mockito.when(operationalModification.getDataAfter()).thenReturn(operationalNode);
-        Mockito.when(retryRegistry.isRegistered(NODE_ID)).thenReturn(true);
-        Mockito.when(operationalNode.getAugmentation(FlowCapableStatisticsGatheringStatus.class)).thenReturn(statisticsGatheringStatus);
-        Mockito.when(statisticsGatheringStatus.getSnapshotGatheringStatusEnd()).thenReturn(snapshotGatheringStatusEnd);
-        Mockito.when(snapshotGatheringStatusEnd.isSucceeded()).thenReturn(true);
-        Mockito.when(snapshotGatheringStatusEnd.getEnd()).thenReturn(timestamp);
-        Mockito.when(snapshotGatheringStatusEnd.getEnd().getValue()).thenReturn(timestampBefore);
-        Mockito.when(retryRegistry.getRegistration(NODE_ID)).thenReturn(simpleDateFormat.parse(timestampAfter));
-
-        nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
-
-        Mockito.verifyZeroInteractions(reactor);
-    }
-
-    @Test
-    public void testOnDataTreeChangedRetryAndFreshOperationalPresent() throws Exception {
-        final DateAndTime timestamp = Mockito.mock(DateAndTime.class);
-        Mockito.when(roTx.read(LogicalDatastoreType.CONFIGURATION, fcNodePath))
-                .thenReturn(Futures.immediateCheckedFuture(Optional.of(configNode)));
-        Mockito.when(reactor.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Matchers.<FlowCapableNode>any(),
-                Matchers.<FlowCapableNode>any(), Matchers.<LogicalDatastoreType>any())).thenReturn(Futures.immediateFuture(Boolean.TRUE));
-        Mockito.when(operationalModification.getDataBefore()).thenReturn(operationalNode);
-        Mockito.when(operationalModification.getDataAfter()).thenReturn(operationalNode);
-        Mockito.when(retryRegistry.isRegistered(NODE_ID)).thenReturn(true);
-        Mockito.when(operationalNode.getAugmentation(FlowCapableStatisticsGatheringStatus.class)).thenReturn(statisticsGatheringStatus);
-        Mockito.when(statisticsGatheringStatus.getSnapshotGatheringStatusEnd()).thenReturn(snapshotGatheringStatusEnd);
-        Mockito.when(snapshotGatheringStatusEnd.isSucceeded()).thenReturn(true);
-        Mockito.when(snapshotGatheringStatusEnd.getEnd()).thenReturn(timestamp);
-        Mockito.when(snapshotGatheringStatusEnd.getEnd().getValue()).thenReturn(timestampAfter);
-        Mockito.when(retryRegistry.getRegistration(NODE_ID)).thenReturn(simpleDateFormat.parse(timestampBefore));
-
-        nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
-
-        Mockito.verify(reactor).syncup(fcNodePath, configNode, fcOperationalNode, dsType);
-        Mockito.verifyNoMoreInteractions(reactor);
-        Mockito.verify(roTx).close();
-    }
-
-    @Test
-    public void testOnDataTreeChangedRetryAndNodeDeleted() {
-        Mockito.when(operationalModification.getDataBefore()).thenReturn(operationalNode);
-        Mockito.when(dataTreeModification.getRootNode().getModificationType()).thenReturn(DataObjectModification.ModificationType.DELETE);
-
-        nodeListenerOperational.onDataTreeChanged(Collections.singleton(dataTreeModification));
-
-        Mockito.verify(retryRegistry).unregisterIfRegistered(NODE_ID);
-        Mockito.verifyZeroInteractions(reactor);
-    }
-
-}
diff --git a/applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/SyncReactorClusterDecoratorTest.java b/applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/SyncReactorClusterDecoratorTest.java
new file mode 100644 (file)
index 0000000..3330dd4
--- /dev/null
@@ -0,0 +1,71 @@
+/**
+ * Copyright (c) 2016 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.openflowplugin.applications.frsync.impl;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.runners.MockitoJUnitRunner;
+import org.opendaylight.openflowplugin.applications.frsync.SyncReactor;
+import org.opendaylight.openflowplugin.applications.frsync.impl.clustering.DeviceMastershipManager;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.Nodes;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.nodes.Node;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.nodes.NodeKey;
+import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
+
+/**
+ * Test for {@link SyncReactorClusterDecorator}.
+ */
+@RunWith(MockitoJUnitRunner.class)
+public class SyncReactorClusterDecoratorTest {
+
+    private static final NodeId NODE_ID = new NodeId("test-node");
+    private SyncReactorClusterDecorator reactor;
+    private InstanceIdentifier<FlowCapableNode> fcNodePath;
+
+    @Mock
+    private SyncReactor delegate;
+    @Mock
+    private DeviceMastershipManager deviceMastershipManager;
+    @Mock
+    private SyncupEntry syncupEntry;
+
+    @Before
+    public void setUp() {
+        reactor = new SyncReactorClusterDecorator(delegate, deviceMastershipManager);
+
+        InstanceIdentifier<Node> nodePath = InstanceIdentifier.create(Nodes.class).child(Node.class, new NodeKey(NODE_ID));
+        fcNodePath = nodePath.augmentation(FlowCapableNode.class);
+    }
+
+    @Test
+    public void testSyncupMaster() throws InterruptedException {
+        Mockito.when(deviceMastershipManager.isDeviceMastered(NODE_ID)).thenReturn(true);
+
+        reactor.syncup(fcNodePath, syncupEntry);
+
+        Mockito.verify(delegate).syncup(fcNodePath, syncupEntry);
+        Mockito.verifyNoMoreInteractions(delegate);
+    }
+
+    @Test
+    public void testSyncupSlave() throws InterruptedException {
+        Mockito.when(deviceMastershipManager.isDeviceMastered(NODE_ID)).thenReturn(false);
+
+        reactor.syncup(fcNodePath, syncupEntry);
+
+        Mockito.verifyZeroInteractions(delegate);
+    }
+
+}
\ No newline at end of file
index 6a059bc1da7caaee7b6e693f82ef6b29510bf9e3..9180e62b87e06813a6a25c5d923526ae3537f463 100644 (file)
@@ -31,6 +31,8 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.runners.MockitoJUnitRunner;
 import org.mockito.stubbing.Answer;
 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
+import org.opendaylight.openflowplugin.applications.frsync.SyncReactor;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.Nodes;
@@ -51,9 +53,13 @@ public class SyncReactorFutureZipDecoratorTest {
     private SyncReactorFutureZipDecorator reactor;
     private InstanceIdentifier<FlowCapableNode> fcNodePath;
     private ListeningExecutorService syncThreadPool;
+    private final LogicalDatastoreType configDS = LogicalDatastoreType.CONFIGURATION;
+    private final LogicalDatastoreType operationalDS = LogicalDatastoreType.OPERATIONAL;
 
     @Mock
-    private SyncReactorGuardDecorator delegate;
+    private SyncReactor delegate;
+    @Mock
+    private SyncupEntry syncupEntry;
 
     @Before
     public void setUp() {
@@ -75,10 +81,16 @@ public class SyncReactorFutureZipDecoratorTest {
         final FlowCapableNode dataAfter2 = Mockito.mock(FlowCapableNode.class);
         final CountDownLatch latchForFirst = new CountDownLatch(1);
         final CountDownLatch latchForNext = new CountDownLatch(1);
-        final LogicalDatastoreType dsType = LogicalDatastoreType.CONFIGURATION;
 
-        Mockito.when(delegate.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Matchers.<FlowCapableNode>any(),
-                Matchers.<FlowCapableNode>any(), Matchers.<LogicalDatastoreType>any())).thenAnswer(new Answer<ListenableFuture<Boolean>>() {
+        final SyncupEntry first = new SyncupEntry(dataBefore, configDS, null, configDS);
+        final SyncupEntry second = new SyncupEntry(dataAfter, configDS, dataBefore, configDS);
+        final SyncupEntry third = new SyncupEntry(null, configDS, dataAfter, configDS);
+        final SyncupEntry fourth = new SyncupEntry(dataAfter2, configDS, null, configDS);
+        final SyncupEntry zipped = new SyncupEntry(dataAfter2, configDS, dataBefore, configDS);
+        final List<ListenableFuture<Boolean>> allResults = new ArrayList<>();
+
+        Mockito.when(delegate.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Mockito.eq(first)))
+                .thenAnswer(new Answer<ListenableFuture<Boolean>>() {
                     @Override
                     public ListenableFuture<Boolean> answer(final InvocationOnMock invocationOnMock) throws Throwable {
                         LOG.info("unlocking next configs");
@@ -87,15 +99,17 @@ public class SyncReactorFutureZipDecoratorTest {
                         LOG.info("unlocking first delegate");
                         return Futures.immediateFuture(Boolean.TRUE);
                     }
-                }).thenReturn(Futures.immediateFuture(Boolean.TRUE));
+                });
 
-        final List<ListenableFuture<Boolean>> allResults = new ArrayList<>();
-        allResults.add(reactor.syncup(fcNodePath, dataBefore, null, dsType));
+        allResults.add(reactor.syncup(fcNodePath, first));
         latchForNext.await();
 
-        allResults.add(reactor.syncup(fcNodePath, dataAfter, dataBefore, dsType));
-        allResults.add(reactor.syncup(fcNodePath, null, dataAfter, dsType));
-        allResults.add(reactor.syncup(fcNodePath, dataAfter2, null, dsType));
+        mockSyncupWithEntry(second);
+        allResults.add(reactor.syncup(fcNodePath, second));
+        mockSyncupWithEntry(third);
+        allResults.add(reactor.syncup(fcNodePath, third));
+        mockSyncupWithEntry(fourth);
+        allResults.add(reactor.syncup(fcNodePath, fourth));
         latchForFirst.countDown();
 
         Futures.allAsList(allResults).get(1, TimeUnit.SECONDS);
@@ -108,8 +122,8 @@ public class SyncReactorFutureZipDecoratorTest {
             syncThreadPool.shutdownNow();
         }
         final InOrder inOrder = Mockito.inOrder(delegate);
-        inOrder.verify(delegate).syncup(fcNodePath, dataBefore, null, dsType);
-        inOrder.verify(delegate).syncup(fcNodePath, dataAfter2, dataBefore, dsType);
+        inOrder.verify(delegate).syncup(fcNodePath, first);
+        inOrder.verify(delegate).syncup(fcNodePath, zipped);
         inOrder.verifyNoMoreInteractions();
     }
 
@@ -118,21 +132,24 @@ public class SyncReactorFutureZipDecoratorTest {
         final FlowCapableNode dataBefore = Mockito.mock(FlowCapableNode.class);
         final FlowCapableNode dataAfter = Mockito.mock(FlowCapableNode.class);
         final CountDownLatch latchForNext = new CountDownLatch(1);
-        final LogicalDatastoreType dsType = LogicalDatastoreType.CONFIGURATION;
 
-        Mockito.when(delegate.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Matchers.<FlowCapableNode>any(),
-                Matchers.<FlowCapableNode>any(), Matchers.<LogicalDatastoreType>any())).thenAnswer(new Answer<ListenableFuture<Boolean>>() {
+        final SyncupEntry first = new SyncupEntry(dataBefore, configDS, null, configDS);
+        final SyncupEntry second = new SyncupEntry(dataAfter, configDS, dataBefore, configDS);
+
+        Mockito.when(delegate.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Mockito.eq(first)))
+                .thenAnswer(new Answer<ListenableFuture<Boolean>>() {
             @Override
             public ListenableFuture<Boolean> answer(final InvocationOnMock invocationOnMock) throws Throwable {
                 LOG.info("unlocking next config");
                 latchForNext.countDown();
                 return Futures.immediateFuture(Boolean.TRUE);
             }
-            }).thenReturn(Futures.immediateFuture(Boolean.TRUE));
+            });
 
-        reactor.syncup(fcNodePath, dataBefore, null, dsType);
+        reactor.syncup(fcNodePath, first);
         latchForNext.await();
-        reactor.syncup(fcNodePath, dataAfter, dataBefore, dsType);
+        mockSyncupWithEntry(second);
+        reactor.syncup(fcNodePath, second);
 
         boolean terminated = syncThreadPool.awaitTermination(1, TimeUnit.SECONDS);
         if (!terminated) {
@@ -140,10 +157,9 @@ public class SyncReactorFutureZipDecoratorTest {
             syncThreadPool.shutdownNow();
         }
         final InOrder inOrder = Mockito.inOrder(delegate);
-        inOrder.verify(delegate).syncup(fcNodePath, dataBefore, null, dsType);
-        inOrder.verify(delegate).syncup(fcNodePath, dataAfter, dataBefore, dsType);
+        inOrder.verify(delegate).syncup(fcNodePath, first);
+        inOrder.verify(delegate).syncup(fcNodePath, second);
         inOrder.verifyNoMoreInteractions();
-
     }
 
     @Test
@@ -155,8 +171,11 @@ public class SyncReactorFutureZipDecoratorTest {
         final CountDownLatch latchForFirst = new CountDownLatch(1);
         final CountDownLatch latchForNext = new CountDownLatch(1);
 
-        Mockito.when(delegate.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Matchers.<FlowCapableNode>any(),
-                Matchers.<FlowCapableNode>any(), Matchers.<LogicalDatastoreType>any())).thenAnswer(new Answer<ListenableFuture<Boolean>>() {
+        final SyncupEntry first = new SyncupEntry(configAfter, configDS, configBefore, configDS);
+        final SyncupEntry second = new SyncupEntry(configActual, configDS, freshOperational, operationalDS);
+
+        Mockito.when(delegate.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Mockito.eq(first)))
+                .thenAnswer(new Answer<ListenableFuture<Boolean>>() {
             @Override
             public ListenableFuture<Boolean> answer(final InvocationOnMock invocationOnMock) throws Throwable {
                 LOG.info("unlocking for fresh operational");
@@ -165,12 +184,13 @@ public class SyncReactorFutureZipDecoratorTest {
                 LOG.info("unlocking first delegate");
                 return Futures.immediateFuture(Boolean.TRUE);
             }
-        }).thenReturn(Futures.immediateFuture(Boolean.TRUE));
+        });
 
-        reactor.syncup(fcNodePath, configAfter, configBefore, LogicalDatastoreType.CONFIGURATION);
+        reactor.syncup(fcNodePath, first);
         latchForNext.await();
 
-        reactor.syncup(fcNodePath, configActual, freshOperational, LogicalDatastoreType.OPERATIONAL);
+        mockSyncupWithEntry(second);
+        reactor.syncup(fcNodePath, second);
         latchForFirst.countDown();
 
         syncThreadPool.shutdown();
@@ -179,7 +199,12 @@ public class SyncReactorFutureZipDecoratorTest {
             LOG.info("thread pool not terminated.");
             syncThreadPool.shutdownNow();
         }
-        Mockito.verify(delegate, Mockito.times(1)).syncup(fcNodePath, configActual, freshOperational, LogicalDatastoreType.OPERATIONAL);
+        Mockito.verify(delegate, Mockito.times(1)).syncup(fcNodePath, second);
+    }
+
+    private void mockSyncupWithEntry(final SyncupEntry entry) throws InterruptedException {
+        Mockito.when(delegate.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Mockito.eq(entry)))
+                .thenReturn(Futures.immediateFuture(Boolean.TRUE));
     }
 
     @After
index d10a228bb7c94c0fa831a19b624807d2c19a9b52..5b43181d65d6bdedc117abcc9713ecb2d763728e 100644 (file)
@@ -16,8 +16,9 @@ import org.mockito.Matchers;
 import org.mockito.Mock;
 import org.mockito.Mockito;
 import org.mockito.runners.MockitoJUnitRunner;
-import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
+import org.opendaylight.openflowplugin.applications.frsync.SyncReactor;
 import org.opendaylight.openflowplugin.applications.frsync.util.SemaphoreKeeperGuavaImpl;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.Nodes;
@@ -34,14 +35,15 @@ public class SyncReactorGuardDecoratorTest {
     private static final NodeId NODE_ID = new NodeId("test-node");
     private SyncReactorGuardDecorator reactor;
     private InstanceIdentifier<FlowCapableNode> fcNodePath;
-    private final LogicalDatastoreType dsType = LogicalDatastoreType.CONFIGURATION;
 
     @Mock
-    private SyncReactorRetryDecorator delegate;
+    private SyncReactor delegate;
     @Mock
     private FlowCapableNode fcConfigNode;
     @Mock
     private FlowCapableNode fcOperationalNode;
+    @Mock
+    private SyncupEntry syncupEntry;
 
     @Before
     public void setUp() throws Exception {
@@ -57,23 +59,23 @@ public class SyncReactorGuardDecoratorTest {
 
     @Test
     public void testSyncupSuccess() throws Exception {
-        Mockito.when(delegate.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Matchers.<FlowCapableNode>any(),
-                Matchers.<FlowCapableNode>any(), Matchers.<LogicalDatastoreType>any())).thenReturn(Futures.immediateFuture(Boolean.TRUE));
+        Mockito.when(delegate.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Matchers.<SyncupEntry>any()))
+                .thenReturn(Futures.immediateFuture(Boolean.TRUE));
 
-        reactor.syncup(fcNodePath, fcConfigNode, fcOperationalNode, dsType);
+        reactor.syncup(fcNodePath, syncupEntry);
 
-        Mockito.verify(delegate).syncup(fcNodePath, fcConfigNode, fcOperationalNode, dsType);
+        Mockito.verify(delegate).syncup(fcNodePath, syncupEntry);
         Mockito.verifyNoMoreInteractions(delegate);
     }
 
     @Test
     public void testSyncupFail() throws Exception {
-        Mockito.when(delegate.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Matchers.<FlowCapableNode>any(),
-                Matchers.<FlowCapableNode>any(), Matchers.<LogicalDatastoreType>any())).thenReturn(Futures.immediateFailedFuture(new Exception()));
+        Mockito.when(delegate.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Matchers.<SyncupEntry>any()))
+                .thenReturn(Futures.immediateFailedFuture(new Exception()));
 
-        reactor.syncup(fcNodePath, fcConfigNode, fcOperationalNode, dsType);
+        reactor.syncup(fcNodePath, syncupEntry);
 
-        Mockito.verify(delegate).syncup(fcNodePath, fcConfigNode, fcOperationalNode, dsType);
+        Mockito.verify(delegate).syncup(fcNodePath, syncupEntry);
         Mockito.verifyNoMoreInteractions(delegate);
 
     }
index 588a4019369bafdfb9983a4df32ecb11fb5d820d..5e6112ee12473aa686eba3d81fde8e7b6f2aeeab 100644 (file)
@@ -27,6 +27,7 @@ import org.opendaylight.openflowplugin.applications.frsync.SyncPlanPushStrategy;
 import org.opendaylight.openflowplugin.applications.frsync.impl.strategy.SynchronizationDiffInput;
 import org.opendaylight.openflowplugin.applications.frsync.util.ReconcileUtil;
 import org.opendaylight.openflowplugin.applications.frsync.util.SyncCrudCounters;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNodeBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.meters.Meter;
@@ -101,13 +102,16 @@ public class SyncReactorImplTest {
                 .setMeter(Collections.singletonList(DSInputFactory.createMeter(2L)))
                 .build();
 
+        final SyncupEntry syncupEntry = new SyncupEntry(configFcn, LogicalDatastoreType.CONFIGURATION,
+                                                        operationalFcn, LogicalDatastoreType.OPERATIONAL);
+
         Mockito.when(syncPlanPushStrategy.executeSyncStrategy(
                 Matchers.<ListenableFuture<RpcResult<Void>>>any(),
                 Matchers.<SynchronizationDiffInput>any(),
                 Matchers.<SyncCrudCounters>any()))
                 .thenReturn(RpcResultBuilder.<Void>success().buildFuture());
 
-        final ListenableFuture<Boolean> syncupResult = reactor.syncup(NODE_IDENT, configFcn, operationalFcn, LogicalDatastoreType.CONFIGURATION);
+        final ListenableFuture<Boolean> syncupResult = reactor.syncup(NODE_IDENT, syncupEntry);
         try {
             Assert.assertTrue(syncupResult.isDone());
             final Boolean voidRpcResult = syncupResult.get(2, TimeUnit.SECONDS);
index 1bf41a49cbb9ec9c20d63761c999f8638a6ae6c5..8151e620df889e0ebe4cb74e5e329f64d25ba88f 100644 (file)
@@ -16,8 +16,9 @@ import org.mockito.Matchers;
 import org.mockito.Mock;
 import org.mockito.Mockito;
 import org.mockito.runners.MockitoJUnitRunner;
-import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
-import org.opendaylight.openflowplugin.applications.frsync.util.RetryRegistry;
+import org.opendaylight.openflowplugin.applications.frsync.SyncReactor;
+import org.opendaylight.openflowplugin.applications.frsync.util.ReconciliationRegistry;
+import org.opendaylight.openflowplugin.applications.frsync.util.SyncupEntry;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.Nodes;
@@ -34,57 +35,51 @@ public class SyncReactorRetryDecoratorTest {
     private static final NodeId NODE_ID = new NodeId("test-node");
     private SyncReactorRetryDecorator reactor;
     private InstanceIdentifier<FlowCapableNode> fcNodePath;
-    private final LogicalDatastoreType dsType = LogicalDatastoreType.CONFIGURATION;
 
     @Mock
-    private SyncReactorImpl delegate;
+    private SyncReactor delegate;
     @Mock
-    private RetryRegistry retryRegistry;
+    private ReconciliationRegistry reconciliationRegistry;
     @Mock
-    private FlowCapableNode fcConfigNode;
-    @Mock
-    private FlowCapableNode fcOperationalNode;
+    private SyncupEntry syncupEntry;
 
     @Before
     public void setUp() {
-        reactor = new SyncReactorRetryDecorator(delegate, retryRegistry);
+        reactor = new SyncReactorRetryDecorator(delegate, reconciliationRegistry);
         InstanceIdentifier<Node> nodePath = InstanceIdentifier.create(Nodes.class).child(Node.class, new NodeKey(NODE_ID));
         fcNodePath = nodePath.augmentation(FlowCapableNode.class);
-
-        final Node operationalNode = Mockito.mock(Node.class);
-        Mockito.when(operationalNode.getId()).thenReturn(NODE_ID);
-        Mockito.when(operationalNode.getAugmentation(FlowCapableNode.class)).thenReturn(fcOperationalNode);
     }
 
     @Test
     public void testSyncupSuccess() throws InterruptedException {
-        Mockito.when(delegate.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Matchers.<FlowCapableNode>any(),
-                Matchers.<FlowCapableNode>any(), Matchers.<LogicalDatastoreType>any())).thenReturn(Futures.immediateFuture(Boolean.TRUE));
+        Mockito.when(delegate.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Matchers.<SyncupEntry>any()))
+                .thenReturn(Futures.immediateFuture(Boolean.TRUE));
 
-        reactor.syncup(fcNodePath, fcConfigNode, fcOperationalNode, dsType);
+        reactor.syncup(fcNodePath, syncupEntry);
 
-        Mockito.verify(delegate).syncup(fcNodePath, fcConfigNode, fcOperationalNode, dsType);
+        Mockito.verify(delegate).syncup(fcNodePath, syncupEntry);
         Mockito.verifyNoMoreInteractions(delegate);
-        Mockito.verify(retryRegistry).unregisterIfRegistered(NODE_ID);
+        Mockito.verify(reconciliationRegistry).unregisterIfRegistered(NODE_ID);
     }
 
     @Test
     public void testSyncupFail() throws InterruptedException {
-        Mockito.when(delegate.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Matchers.<FlowCapableNode>any(),
-                Matchers.<FlowCapableNode>any(), Matchers.<LogicalDatastoreType>any())).thenReturn(Futures.immediateFuture(Boolean.FALSE));
+        Mockito.when(delegate.syncup(Matchers.<InstanceIdentifier<FlowCapableNode>>any(), Matchers.<SyncupEntry>any()))
+                .thenReturn(Futures.immediateFuture(Boolean.FALSE));
 
-        reactor.syncup(fcNodePath, fcConfigNode, fcOperationalNode, dsType);
+        reactor.syncup(fcNodePath, syncupEntry);
 
-        Mockito.verify(delegate).syncup(fcNodePath, fcConfigNode, fcOperationalNode, dsType);
+        Mockito.verify(delegate).syncup(fcNodePath, syncupEntry);
         Mockito.verifyNoMoreInteractions(delegate);
-        Mockito.verify(retryRegistry).register(NODE_ID);
+        Mockito.verify(reconciliationRegistry).register(NODE_ID);
     }
 
     @Test
     public void testSyncupConfigIgnoreInRetry() throws InterruptedException {
-        Mockito.when(retryRegistry.isRegistered(NODE_ID)).thenReturn(true);
+        Mockito.when(reconciliationRegistry.isRegistered(NODE_ID)).thenReturn(true);
+        Mockito.when(syncupEntry.isOptimizedConfigDelta()).thenReturn(true);
 
-        reactor.syncup(fcNodePath, fcConfigNode, fcOperationalNode, dsType);
+        reactor.syncup(fcNodePath, syncupEntry);
 
         Mockito.verifyZeroInteractions(delegate);
     }
diff --git a/applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/clustering/DeviceMastershipManagerTest.java b/applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/clustering/DeviceMastershipManagerTest.java
new file mode 100644 (file)
index 0000000..d73df5b
--- /dev/null
@@ -0,0 +1,71 @@
+/*
+ * Copyright (c) 2016 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.openflowplugin.applications.frsync.impl.clustering;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Matchers;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.runners.MockitoJUnitRunner;
+import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonService;
+import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceProvider;
+import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceRegistration;
+import org.opendaylight.openflowplugin.applications.frsync.util.ReconciliationRegistry;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
+
+/**
+ * Test for {@link DeviceMastershipManager}.
+ */
+@RunWith(MockitoJUnitRunner.class)
+public class DeviceMastershipManagerTest {
+    private static final NodeId NODE_ID = new NodeId("testNode");
+    private DeviceMastershipManager deviceMastershipManager;
+    @Mock
+    private ClusterSingletonServiceRegistration registration;
+    @Mock
+    private ClusterSingletonServiceProvider clusterSingletonService;
+
+    @Before
+    public void setUp() throws Exception {
+        deviceMastershipManager = new DeviceMastershipManager(clusterSingletonService, new ReconciliationRegistry());
+        Mockito.when(clusterSingletonService.registerClusterSingletonService(Matchers.<ClusterSingletonService>any()))
+                .thenReturn(registration);
+    }
+
+    @Test
+    public void testOnDeviceConnectedAndDisconnected() {
+        // no context
+        Assert.assertNull(deviceMastershipManager.getDeviceMasterships().get(NODE_ID));
+        // create context - register
+        deviceMastershipManager.onDeviceConnected(NODE_ID);
+        DeviceMastership registration = deviceMastershipManager.getDeviceMasterships().get(NODE_ID);
+        Assert.assertNotNull(registration);
+        Mockito.verify(clusterSingletonService).registerClusterSingletonService(registration);
+        // destroy context - unregister
+        deviceMastershipManager.onDeviceDisconnected(NODE_ID);
+        Assert.assertNull(deviceMastershipManager.getDeviceMasterships().get(NODE_ID));
+    }
+
+    @Test
+    public void testIsDeviceMasteredOrSlaved() {
+        // no context
+        Assert.assertFalse(deviceMastershipManager.isDeviceMastered(NODE_ID));
+        deviceMastershipManager.onDeviceConnected(NODE_ID);
+        // is master
+        deviceMastershipManager.getDeviceMasterships().get(NODE_ID).instantiateServiceInstance();
+        Assert.assertTrue(deviceMastershipManager.isDeviceMastered(NODE_ID));
+        // is not master
+        deviceMastershipManager.getDeviceMasterships().get(NODE_ID).closeServiceInstance();
+        Assert.assertFalse(deviceMastershipManager.isDeviceMastered(NODE_ID));
+    }
+
+}
diff --git a/applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/clustering/DeviceMastershipTest.java b/applications/forwardingrules-sync/src/test/java/org/opendaylight/openflowplugin/applications/frsync/impl/clustering/DeviceMastershipTest.java
new file mode 100644 (file)
index 0000000..be45a9d
--- /dev/null
@@ -0,0 +1,52 @@
+/*
+ * Copyright (c) 2016 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.openflowplugin.applications.frsync.impl.clustering;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.runners.MockitoJUnitRunner;
+import org.opendaylight.openflowplugin.applications.frsync.util.ReconciliationRegistry;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
+
+/**
+ * Test for {@link DeviceMastership}.
+ */
+@RunWith(MockitoJUnitRunner.class)
+public class DeviceMastershipTest {
+    private static final NodeId NODE_ID = new NodeId("testNode");
+    private DeviceMastership deviceMastership;
+
+    @Mock
+    private DeviceMastershipManager deviceMastershipManager;
+    @Mock
+    private ReconciliationRegistry reconciliationRegistry;
+
+    @Before
+    public void setUp() throws Exception {
+        deviceMastership = new DeviceMastership(NODE_ID, reconciliationRegistry);
+    }
+
+    @Test
+    public void instantiateServiceInstance() {
+        deviceMastership.instantiateServiceInstance();
+        Mockito.verify(reconciliationRegistry).register(NODE_ID);
+        Assert.assertTrue(deviceMastership.isDeviceMastered());
+    }
+
+    @Test
+    public void closeServiceInstance() {
+        deviceMastership.closeServiceInstance();
+        Mockito.verify(reconciliationRegistry).unregisterIfRegistered(NODE_ID);
+        Assert.assertFalse(deviceMastership.isDeviceMastered());
+    }
+}
\ No newline at end of file
@@ -1,4 +1,4 @@
-/**
+/*
  * Copyright (c) 2016 Cisco Systems, Inc. and others. All rights reserved.
  *
  * This program and the accompanying materials are made available under the
@@ -6,7 +6,7 @@
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
 
-package org.opendaylight.openflowplugin.applications.frsync.impl;
+package org.opendaylight.openflowplugin.applications.frsync.impl.strategy;
 
 import java.math.BigInteger;
 import java.util.Collections;
@@ -1,4 +1,4 @@
-/**
+/*
  * Copyright (c) 2016 Cisco Systems, Inc. and others. All rights reserved.
  *
  * This program and the accompanying materials are made available under the
@@ -6,7 +6,7 @@
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
 
-package org.opendaylight.openflowplugin.applications.frsync.impl;
+package org.opendaylight.openflowplugin.applications.frsync.impl.strategy;
 
 import java.math.BigInteger;
 import java.util.concurrent.Future;
@@ -1,4 +1,4 @@
-/**
+/*
  * Copyright (c) 2016 Cisco Systems, Inc. and others. All rights reserved.
  *
  * This program and the accompanying materials are made available under the
@@ -6,7 +6,7 @@
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
 
-package org.opendaylight.openflowplugin.applications.frsync.impl;
+package org.opendaylight.openflowplugin.applications.frsync.impl.strategy;
 
 import java.math.BigInteger;
 import java.util.concurrent.Future;
index 90b7cf8b876e0c0a44a960641d7b1c526be5562e..4bea89fd48c2f248841135bbdf2ca90ea001b79a 100644 (file)
@@ -25,7 +25,6 @@ import org.mockito.Matchers;
 import org.mockito.Mock;
 import org.mockito.Mockito;
 import org.mockito.runners.MockitoJUnitRunner;
-import org.opendaylight.openflowplugin.applications.frsync.impl.TableForwarder;
 import org.opendaylight.openflowplugin.applications.frsync.util.ItemSyncBox;
 import org.opendaylight.openflowplugin.applications.frsync.util.SyncCrudCounters;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flat.batch.service.rev160321.ProcessFlatBatchInput;
@@ -81,12 +80,12 @@ public class SyncPlanPushStrategyFlatBatchImplTest {
     @Captor
     private ArgumentCaptor<ProcessFlatBatchInput> processFlatBatchInputCpt;
 
-    private List<ItemSyncBox<Group>> groupsToAddOrUpdate;
-    private List<ItemSyncBox<Group>> groupsToRemove;
-    private ItemSyncBox<Meter> metersToAddOrUpdate;
-    private ItemSyncBox<Meter> metersToRemove;
-    private Map<TableKey, ItemSyncBox<Flow>> flowsToAddOrUpdate;
-    private Map<TableKey, ItemSyncBox<Flow>> flowsToRemove;
+    private final List<ItemSyncBox<Group>> groupsToAddOrUpdate;
+    private final List<ItemSyncBox<Group>> groupsToRemove;
+    private final ItemSyncBox<Meter> metersToAddOrUpdate;
+    private final ItemSyncBox<Meter> metersToRemove;
+    private final Map<TableKey, ItemSyncBox<Flow>> flowsToAddOrUpdate;
+    private final Map<TableKey, ItemSyncBox<Flow>> flowsToRemove;
     private List<Batch> batchBag;
 
     private SyncPlanPushStrategyFlatBatchImpl syncPlanPushStrategy;
@@ -280,7 +279,7 @@ public class SyncPlanPushStrategyFlatBatchImplTest {
     }
 
     @Test
-    public void testMapBachesToRanges() throws Exception {
+    public void testMapBatchesToRanges() throws Exception {
         final List<Batch> inputBatchBag = Lists.newArrayList(
                 new BatchBuilder().setBatchOrder(0).build(),
                 new BatchBuilder().setBatchOrder(5).build(),
index 99358de3e2b0e71b0b56889bcef398813342af01..a0d84c429f229fa4ab4663141c57859ef68d9509 100644 (file)
@@ -26,15 +26,10 @@ import org.mockito.InOrder;
 import org.mockito.Matchers;
 import org.mockito.Mock;
 import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
 import org.mockito.runners.MockitoJUnitRunner;
 import org.mockito.stubbing.Answer;
 import org.opendaylight.controller.md.sal.binding.api.DataBroker;
 import org.opendaylight.openflowplugin.applications.frsync.impl.DSInputFactory;
-import org.opendaylight.openflowplugin.applications.frsync.impl.FlowForwarder;
-import org.opendaylight.openflowplugin.applications.frsync.impl.GroupForwarder;
-import org.opendaylight.openflowplugin.applications.frsync.impl.MeterForwarder;
-import org.opendaylight.openflowplugin.applications.frsync.impl.TableForwarder;
 import org.opendaylight.openflowplugin.applications.frsync.util.ItemSyncBox;
 import org.opendaylight.openflowplugin.applications.frsync.util.SyncCrudCounters;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
@@ -108,12 +103,12 @@ public class SyncPlanPushStrategyIncrementalImplTest {
 
     private SyncCrudCounters counters;
 
-    private List<ItemSyncBox<Group>> groupsToAddOrUpdate;
-    private List<ItemSyncBox<Group>> groupsToRemove;
-    private ItemSyncBox<Meter> metersToAddOrUpdate;
-    private ItemSyncBox<Meter> metersToRemove;
-    private Map<TableKey, ItemSyncBox<Flow>> flowsToAddOrUpdate;
-    private Map<TableKey, ItemSyncBox<Flow>> flowsToRemove;
+    private final List<ItemSyncBox<Group>> groupsToAddOrUpdate;
+    private final List<ItemSyncBox<Group>> groupsToRemove;
+    private final ItemSyncBox<Meter> metersToAddOrUpdate;
+    private final ItemSyncBox<Meter> metersToRemove;
+    private final Map<TableKey, ItemSyncBox<Flow>> flowsToAddOrUpdate;
+    private final Map<TableKey, ItemSyncBox<Flow>> flowsToRemove;
 
     public SyncPlanPushStrategyIncrementalImplTest() {
         groupsToAddOrUpdate = Lists.newArrayList(DiffInputFactory.createGroupSyncBox(1, 2, 3),
@@ -226,12 +221,7 @@ public class SyncPlanPushStrategyIncrementalImplTest {
     }
 
     private <O> Answer<Future<RpcResult<O>>> createSalServiceFutureAnswer() {
-        return new Answer<Future<RpcResult<O>>>() {
-            @Override
-            public Future<RpcResult<O>> answer(final InvocationOnMock invocation) throws Throwable {
-                return RpcResultBuilder.<O>success().buildFuture();
-            }
-        };
+        return invocation -> RpcResultBuilder.<O>success().buildFuture();
     }
 
     @Test
@@ -1,4 +1,4 @@
-/**
+/*
  * Copyright (c) 2016 Cisco Systems, Inc. and others. All rights reserved.
  *
  * This program and the accompanying materials are made available under the
@@ -6,7 +6,7 @@
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
 
-package org.opendaylight.openflowplugin.applications.frsync.impl;
+package org.opendaylight.openflowplugin.applications.frsync.impl.strategy;
 
 import java.math.BigInteger;
 import java.util.concurrent.Future;
index 61fd6384b828e844fafbb5af1c2bd972a9313799..727510286481bba5781b416fa926765b4546cd09 100644 (file)
@@ -89,11 +89,6 @@ public class ReconcileUtilTest {
         Assert.assertTrue(vehicle.get().isSuccessful());
     }
 
-    @Test
-    public void testCreateRpcResultCondenser() throws Exception {
-
-    }
-
     /**
      * add one missing group
      *
@@ -335,19 +330,4 @@ public class ReconcileUtilTest {
         }
         return syncBox1;
     }
-
-    @Test
-    public void testResolveMeterDiffs() throws Exception {
-
-    }
-
-    @Test
-    public void testResolveFlowDiffsInTable() throws Exception {
-
-    }
-
-    @Test
-    public void testResolveFlowDiffsInAllTables() throws Exception {
-
-    }
 }
\ No newline at end of file
@@ -17,38 +17,38 @@ import org.mockito.runners.MockitoJUnitRunner;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
 
 /**
- * Test for {@link RetryRegistry}.
+ * Test for {@link ReconciliationRegistry}.
  */
 @RunWith(MockitoJUnitRunner.class)
-public class RetryRegistryTest {
+public class ReconciliationRegistryTest {
 
     private static final NodeId NODE_ID = new NodeId("testNode");
-    private RetryRegistry retryRegistry;
+    private ReconciliationRegistry reconciliationRegistry;
 
     @Before
     public void setUp() throws Exception {
-        retryRegistry = new RetryRegistry();
+        reconciliationRegistry = new ReconciliationRegistry();
     }
 
     @Test
     public void testRegister() {
-        Date timestamp = retryRegistry.register(NODE_ID);
-        Assert.assertEquals(true, retryRegistry.isRegistered(NODE_ID));
+        Date timestamp = reconciliationRegistry.register(NODE_ID);
+        Assert.assertEquals(true, reconciliationRegistry.isRegistered(NODE_ID));
         Assert.assertNotNull(timestamp);
     }
 
     @Test
     public void testUnregisterIfRegistered() {
-        retryRegistry.register(NODE_ID);
-        Date timestamp = retryRegistry.unregisterIfRegistered(NODE_ID);
-        Assert.assertEquals(false, retryRegistry.isRegistered(NODE_ID));
+        reconciliationRegistry.register(NODE_ID);
+        Date timestamp = reconciliationRegistry.unregisterIfRegistered(NODE_ID);
+        Assert.assertEquals(false, reconciliationRegistry.isRegistered(NODE_ID));
         Assert.assertNotNull(timestamp);
     }
 
     @Test
     public void testUnregisterIfNotRegistered() {
-        Date timestamp = retryRegistry.unregisterIfRegistered(NODE_ID);
-        Assert.assertEquals(false, retryRegistry.isRegistered(NODE_ID));
+        Date timestamp = reconciliationRegistry.unregisterIfRegistered(NODE_ID);
+        Assert.assertEquals(false, reconciliationRegistry.isRegistered(NODE_ID));
         Assert.assertNull(timestamp);
     }
 
index 8affd2a5b2444bed518cfd6019d796c810d42ef8..510dce07d168d05dc5933b45fa3d093bcedb84eb 100644 (file)
@@ -111,7 +111,7 @@ public class SemaphoreKeeperTest {
     private static class Worker implements Runnable {
         private final SemaphoreKeeper<String> keeper;
         private final String key;
-        private ConcurrentMap<Integer, Integer> counter = new ConcurrentHashMap<>();
+        private final ConcurrentMap<Integer, Integer> counter = new ConcurrentHashMap<>();
         private volatile int index = 0;
 
         public Worker(SemaphoreKeeper<String> keeper, final String key) {
index f76f650a0a22abb96bd6c5ffe2672fa684af4774..d2e1550505de9c130888dff0bc3f824138032c36 100644 (file)
@@ -66,35 +66,35 @@ public final class FlowCapableNodeMapping {
 
     public static org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.Node toTopologyNode(
             final NodeId nodeId, final NodeRef invNodeRef) {
-        return new NodeBuilder() //
-                .setNodeId(nodeId) //
-                .addAugmentation(InventoryNode.class, new InventoryNodeBuilder() //
-                        .setInventoryNodeRef(invNodeRef) //
-                        .build()) //
+        return new NodeBuilder()
+                .setNodeId(nodeId)
+                .addAugmentation(InventoryNode.class, new InventoryNodeBuilder()
+                        .setInventoryNodeRef(invNodeRef)
+                        .build())
                 .build();
     }
 
     public static TerminationPoint toTerminationPoint(final TpId id, final NodeConnectorRef invNodeConnectorRef) {
-        return new TerminationPointBuilder() //
-                .setTpId(id) //
-                .addAugmentation(InventoryNodeConnector.class, new InventoryNodeConnectorBuilder() //
-                        .setInventoryNodeConnectorRef(invNodeConnectorRef) //
-                        .build()) //
+        return new TerminationPointBuilder()
+                .setTpId(id)
+                .addAugmentation(InventoryNodeConnector.class, new InventoryNodeConnectorBuilder()
+                    .setInventoryNodeConnectorRef(invNodeConnectorRef)
+                    .build())
                 .build();
     }
 
     public static Link toTopologyLink(
             final org.opendaylight.yang.gen.v1.urn.opendaylight.flow.topology.discovery.rev130819.Link link) {
-        return new LinkBuilder() //
-                .setSource(new SourceBuilder() //
-                        .setSourceNode(toTopologyNodeId(link.getSource())) //
-                        .setSourceTp(toTerminationPointId(link.getSource())) //
-                        .build()) //
-                .setDestination(new DestinationBuilder() //
-                        .setDestNode(toTopologyNodeId(link.getDestination())) //
-                        .setDestTp(toTerminationPointId(link.getDestination())) //
-                        .build()) //
-                .setLinkId(new LinkId(getNodeConnectorKey(link.getSource()).getId())) //
+        return new LinkBuilder()
+                .setSource(new SourceBuilder()
+                        .setSourceNode(toTopologyNodeId(link.getSource()))
+                        .setSourceTp(toTerminationPointId(link.getSource()))
+                        .build())
+                .setDestination(new DestinationBuilder()
+                        .setDestNode(toTopologyNodeId(link.getDestination()))
+                        .setDestTp(toTerminationPointId(link.getDestination()))
+                        .build())
+                .setLinkId(new LinkId(getNodeConnectorKey(link.getSource()).getId()))
                 .build();
     }
 }
index cb39795b86297250336915a7e0f90445fd027446..cb7d973c18793978f57edd1c9f23b3bc7a45396a 100644 (file)
@@ -115,8 +115,6 @@ class FlowCapableTopologyExporter implements FlowTopologyDiscoveryListener {
         return tpPath(toTopologyNodeId(invNodeKey.getId()), toTerminationPointId(invNodeConnectorKey.getId()));
     }
 
-
-
     private InstanceIdentifier<Node> getNodePath(final NodeId nodeId) {
         return iiToTopology.child(Node.class, new NodeKey(nodeId));
     }
index 653e2bed290fa8bf5c3d6b8e2ce27e7375ea577f..b0ed434bb4d5449c1c02f69396c4edd390845fbb 100644 (file)
@@ -45,7 +45,7 @@ public class NodeChangeListenerImpl extends DataChangeListenerImpl {
     }
 
     /**
-     * @param removedPaths
+     * @param removedNodes
      */
     private void processRemovedNode(final Set<InstanceIdentifier<?>> removedNodes) {
         for (InstanceIdentifier<?> removedNode : removedNodes) {
@@ -66,17 +66,17 @@ public class NodeChangeListenerImpl extends DataChangeListenerImpl {
         }
     }
 
-    /**
-     * @param updatedData
-     */
-    // private void processUpdatedNode(Map<InstanceIdentifier<?>, DataObject> updatedData) {
-    // //TODO: only node id is used from incomming data object.
-    // //if it is changed what should happen? Listener is on FlocCapableNode so change
-    // //of node id (only data which are used) isn't caught.
-    // }
+//    /**
+//     * @param updatedData
+//     */
+//     private void processUpdatedNode(Map<InstanceIdentifier<?>, DataObject> updatedData) {
+//         //TODO: only node id is used from incoming data object.
+//         //if it is changed what should happen? Listener is on FlowCapableNode so change
+//         //of node id (only data which are used) isn't caught.
+//     }
 
     /**
-     * @param createdData
+     * @param addedDatas
      */
     private void processAddedNode(final Map<InstanceIdentifier<?>, DataObject> addedDatas) {
         for (Entry<InstanceIdentifier<?>, DataObject> addedData : addedDatas.entrySet()) {
@@ -97,7 +97,7 @@ public class NodeChangeListenerImpl extends DataChangeListenerImpl {
     /**
      * @param nodeIdInTopology
      * @param iiToNodeInInventory
-     * @return
+     * @return Node
      */
     private static org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.Node prepareTopologyNode(final NodeId nodeIdInTopology, final InstanceIdentifier<?> iiToNodeInInventory) {
         final InventoryNode inventoryNode = new InventoryNodeBuilder()
index 96365a338868467d728dcd204d384633be24c106..bbfae52b84910b179fe182d9b3fa345a0ef90934 100644 (file)
@@ -77,16 +77,7 @@ public final class OperationProcessor implements AutoCloseable, Runnable, Transa
                     } while (op != null);
 
                     LOG.debug("Processed {} operations, submitting transaction", ops);
-
-                    try {
-                        tx.submit().checkedGet();
-                    } catch (final TransactionCommitFailedException e) {
-                        LOG.warn("Stat DataStoreOperation unexpected State!", e);
-                        transactionChain.close();
-                        transactionChain = dataBroker.createTransactionChain(this);
-                        cleanDataStoreOperQueue();
-                    }
-
+                    submitTransaction(tx);
                 } catch (final IllegalStateException e) {
                     LOG.warn("Stat DataStoreOperation unexpected State!", e);
                     transactionChain.close();
@@ -94,7 +85,7 @@ public final class OperationProcessor implements AutoCloseable, Runnable, Transa
                     cleanDataStoreOperQueue();
                 } catch (final InterruptedException e) {
                     // This should mean we're shutting down.
-                    LOG.debug("Stat Manager DS Operation thread interupted!", e);
+                    LOG.debug("Stat Manager DS Operation thread interrupted!", e);
                     finishing = true;
                 } catch (final Exception e) {
                     LOG.warn("Stat DataStore Operation executor fail!", e);
@@ -104,6 +95,17 @@ public final class OperationProcessor implements AutoCloseable, Runnable, Transa
         cleanDataStoreOperQueue();
     }
 
+    private void submitTransaction(ReadWriteTransaction tx) {
+        try {
+            tx.submit().checkedGet();
+        } catch (final TransactionCommitFailedException e) {
+            LOG.warn("Stat DataStoreOperation unexpected State!", e);
+            transactionChain.close();
+            transactionChain = dataBroker.createTransactionChain(this);
+            cleanDataStoreOperQueue();
+        }
+    }
+
     private void cleanDataStoreOperQueue() {
         while (!queue.isEmpty()) {
             queue.poll();
index e380c6af6a279e8154848033cc1166de43c8e812..96dd36e582958688b83e6d2acb4373beb5caab13 100644 (file)
@@ -25,6 +25,8 @@ public class TopologyManagerUtil {
 
     private static final Logger LOG = LoggerFactory.getLogger(TopologyManagerUtil.class);
 
+    private TopologyManagerUtil() {}
+
     static void removeAffectedLinks(final NodeId id, final ReadWriteTransaction transaction, InstanceIdentifier<Topology> topology) {
         Optional<Topology> topologyOptional = Optional.absent();
         try {
index e88e27a7c0f6b656dc415404fa3e6e40df36a9c0..0f8a35e89ffe0032742c9bcd4a46a28efe046f0b 100644 (file)
@@ -9,27 +9,28 @@ package org.opendaylight.openflowplugin.applications.topology.manager;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doReturn;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import org.junit.After;
+import org.junit.Before;
 import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
 import org.opendaylight.controller.md.sal.binding.api.BindingTransactionChain;
 import org.opendaylight.controller.md.sal.binding.api.DataBroker;
 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeEvent;
-import org.junit.After;
-import org.junit.Before;
-import org.mockito.MockitoAnnotations;
 import org.opendaylight.controller.md.sal.common.api.data.TransactionChainListener;
-import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.NetworkTopology;
-import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.TopologyId;
-import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.Topology;
-import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.TopologyKey;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Set;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNodeConnector;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNodeConnectorBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.types.port.rev130925.PortConfig;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.types.port.rev130925.flow.capable.port.StateBuilder;
+import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.NetworkTopology;
+import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.TopologyId;
+import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.Topology;
+import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.TopologyKey;
 import org.opendaylight.yangtools.yang.binding.DataObject;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 
index e3e669adc9ed1b8924c9e4c9645732db2c2aa6f1..883d0722f0803bce71b7c06bc8618501a7fb76ba 100644 (file)
@@ -8,11 +8,27 @@
 
 package org.opendaylight.openflowplugin.applications.topology.manager;
 
-import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.*;
-import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeEvent;
-import org.opendaylight.yangtools.yang.binding.DataObject;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newDestTp;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newInvNodeConnKey;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newInvNodeKey;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newLink;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newNodeConnID;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newSourceTp;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.setupStubbedSubmit;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.waitForSubmit;
+
 import com.google.common.base.Optional;
 import com.google.common.util.concurrent.Futures;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -22,10 +38,9 @@ import org.mockito.MockitoAnnotations;
 import org.opendaylight.controller.md.sal.binding.api.BindingTransactionChain;
 import org.opendaylight.controller.md.sal.binding.api.DataBroker;
 import org.opendaylight.controller.md.sal.binding.api.ReadWriteTransaction;
+import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeEvent;
 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.controller.md.sal.common.api.data.TransactionChainListener;
-import org.opendaylight.openflowplugin.applications.topology.manager.FlowCapableTopologyExporter;
-import org.opendaylight.openflowplugin.applications.topology.manager.OperationProcessor;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.topology.discovery.rev130819.LinkDiscoveredBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.topology.discovery.rev130819.LinkRemovedBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeConnectorRef;
@@ -36,17 +51,8 @@ import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.TopologyKey;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.Link;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.LinkKey;
+import org.opendaylight.yangtools.yang.binding.DataObject;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
 
 public class FlowCapableTopologyExporterTest {
 
index 2c8a2601300444412870e32b302ef7b1bb8958d6..bb24ce44353f80a5fc1897fad72b22675a3797e0 100644 (file)
@@ -13,11 +13,17 @@ import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
-import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.*;
-
-import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeRef;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.model.topology.inventory.rev131030.InventoryNode;
-import org.opendaylight.yangtools.yang.binding.DataObject;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.assertDeletedIDs;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newDestNode;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newInvNodeKey;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newLink;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newSourceNode;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.setReadFutureAsync;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.setupStubbedDeletes;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.setupStubbedSubmit;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.verifyMockTx;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.waitForDeletes;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.waitForSubmit;
 
 import com.google.common.base.Optional;
 import com.google.common.util.concurrent.Futures;
@@ -31,7 +37,9 @@ import org.mockito.ArgumentCaptor;
 import org.opendaylight.controller.md.sal.binding.api.ReadWriteTransaction;
 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeRef;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.Nodes;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.model.topology.inventory.rev131030.InventoryNode;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.NodeId;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.Topology;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.TopologyBuilder;
@@ -39,6 +47,7 @@ import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.Node;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.NodeBuilder;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.NodeKey;
+import org.opendaylight.yangtools.yang.binding.DataObject;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 
 /**
index 8f46f13128f7b1bcad3176b2935981ecfdb923c9..f495b840dd2606e59f430e4e7f2a6eb120e84034 100644 (file)
@@ -14,11 +14,19 @@ import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
-import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.*;
-
-import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeConnectorRef;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.model.topology.inventory.rev131030.InventoryNodeConnector;
-import org.opendaylight.yangtools.yang.binding.DataObject;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.assertDeletedIDs;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newDestTp;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newInvNodeConnKey;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newInvNodeKey;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newLink;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newNodeConnID;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.newSourceTp;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.setReadFutureAsync;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.setupStubbedDeletes;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.setupStubbedSubmit;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.verifyMockTx;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.waitForDeletes;
+import static org.opendaylight.openflowplugin.applications.topology.manager.TestUtils.waitForSubmit;
 
 import com.google.common.base.Optional;
 import com.google.common.util.concurrent.Futures;
@@ -32,6 +40,8 @@ import org.mockito.ArgumentCaptor;
 import org.opendaylight.controller.md.sal.binding.api.ReadWriteTransaction;
 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeConnectorRef;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.model.topology.inventory.rev131030.InventoryNodeConnector;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.NodeId;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.TpId;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.Topology;
@@ -42,6 +52,7 @@ import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.NodeKey;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.node.TerminationPoint;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.node.TerminationPointKey;
+import org.opendaylight.yangtools.yang.binding.DataObject;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 
 /**
index dce622964b81d15152f15dca1e181da17659d17f..fce41bd0c98011ed9f8ac7e43f14bbef3f929c03 100644 (file)
@@ -9,6 +9,8 @@ package org.opendaylight.openflowplugin.api;
 
 import java.math.BigInteger;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.types.rev131026.FlowCookie;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.types.rev131026.flow.Match;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.types.rev131026.flow.MatchBuilder;
 
 /**
  * OFP related constants
@@ -26,7 +28,6 @@ public final class OFConstants {
     /** enum ofp_port_no, reserved port: local openflow port  */
     public static final short OFPP_LOCAL = ((short)0xfffe);
 
-
     /** openflow protocol 1.0 - version identifier */
     public static final short OFP_VERSION_1_0 = 0x01;
     /** openflow protocol 1.3 - version identifier */
@@ -54,6 +55,8 @@ public final class OFConstants {
     public static final BigInteger DEFAULT_COOKIE_MASK = BigInteger.ZERO;
     public static final FlowCookie DEFAULT_FLOW_COOKIE = new FlowCookie(DEFAULT_COOKIE);
     public static final Integer DEFAULT_FLOW_PRIORITY = 0x8000;
+    /** Empty flow match */
+    public static final Match EMPTY_MATCH = new MatchBuilder().build();
 
     /** indicates that no buffering should be applied and the whole packet is to be
      *  sent to the controller. */
@@ -62,7 +65,6 @@ public final class OFConstants {
      *  sent to the controller. */
     public static final Integer OFPCML_NO_BUFFER = 0xffff;
 
-
     public static final int MAC_ADDRESS_LENGTH = 6;
     public static final int SIZE_OF_LONG_IN_BYTES = 8;
     public static final int SIGNUM_UNSIGNED = 1;
index ede4d6905e4eeb982caf5d8bc90af7896cbe9b65..0f051f7230588392a5c1aa3068b3a80654bbab0e 100644 (file)
@@ -7,6 +7,7 @@
  */
 package org.opendaylight.openflowplugin.impl.registry.flow;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.util.concurrent.CheckedFuture;
 import com.google.common.util.concurrent.FutureCallback;
@@ -22,6 +23,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Consumer;
 import javax.annotation.concurrent.GuardedBy;
 import org.opendaylight.controller.md.sal.binding.api.DataBroker;
@@ -31,7 +33,6 @@ import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
 import org.opendaylight.openflowplugin.api.openflow.registry.flow.DeviceFlowRegistry;
 import org.opendaylight.openflowplugin.api.openflow.registry.flow.FlowDescriptor;
 import org.opendaylight.openflowplugin.api.openflow.registry.flow.FlowRegistryKey;
-import org.opendaylight.openflowplugin.impl.util.FlowUtil;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowId;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.table.Flow;
@@ -47,6 +48,8 @@ import org.slf4j.LoggerFactory;
  */
 public class DeviceFlowRegistryImpl implements DeviceFlowRegistry {
     private static final Logger LOG = LoggerFactory.getLogger(DeviceFlowRegistryImpl.class);
+    private static final String ALIEN_SYSTEM_FLOW_ID = "#UF$TABLE*";
+    private static final AtomicInteger UNACCOUNTED_FLOWS_COUNTER = new AtomicInteger(0);
 
     private final ConcurrentMap<FlowRegistryKey, FlowDescriptor> flowRegistry = new TrieMap<>();
     @GuardedBy("marks")
@@ -173,7 +176,7 @@ public class DeviceFlowRegistryImpl implements DeviceFlowRegistry {
         // We was not able to retrieve FlowDescriptor, so we will at least try to generate it
         if (flowDescriptor == null) {
             final short tableId = flowRegistryKey.getTableId();
-            final FlowId alienFlowId = FlowUtil.createAlienFlowId(tableId);
+            final FlowId alienFlowId = createAlienFlowId(tableId);
             flowDescriptor = FlowDescriptorFactory.create(tableId, alienFlowId);
 
             // Finally we got flowDescriptor, so now we will store it to registry,
@@ -224,4 +227,10 @@ public class DeviceFlowRegistryImpl implements DeviceFlowRegistry {
         flowRegistry.clear();
         marks.clear();
     }
+
+    @VisibleForTesting
+    static FlowId createAlienFlowId(final short tableId) {
+        final String alienId = ALIEN_SYSTEM_FLOW_ID + tableId + '-' + UNACCOUNTED_FLOWS_COUNTER.incrementAndGet();
+        return new FlowId(alienId);
+    }
 }
index 1399980bf1c2abb5b24a247727e4b853bb96bc88..a751be2e5fb5df1675b66e29998af05aa367a375 100644 (file)
@@ -18,6 +18,9 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.ta
  * Created by Martin Bobak &lt;mbobak@cisco.com&gt; on 9.4.2015.
  */
 public class FlowDescriptorFactory {
+    private FlowDescriptorFactory() {
+        // Hide implicit constructor
+    }
 
     public static FlowDescriptor create(final short tableId, final FlowId fLowId) {
         final TableKey tableKey = new TableKey(tableId);
@@ -29,7 +32,7 @@ public class FlowDescriptorFactory {
         private final FlowId flowId;
         private final TableKey tableKey;
 
-        public FlowDescriptorDto(final TableKey tableKey, final FlowId flowId) {
+        private FlowDescriptorDto(final TableKey tableKey, final FlowId flowId) {
             Preconditions.checkNotNull(tableKey);
             Preconditions.checkNotNull(flowId);
             this.flowId = flowId;
index 140d7878f3c06670b75c2538db4f715ca81715dd..026aceedf2541b9b87526b5256e95d74649cb159 100644 (file)
@@ -22,9 +22,8 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.types.rev131026.flow.M
  * Created by Martin Bobak &lt;mbobak@cisco.com&gt; on 8.4.2015.
  */
 public class FlowRegistryKeyFactory {
-
-
-    public FlowRegistryKeyFactory() {
+    private FlowRegistryKeyFactory() {
+        // Hide implicit constructor
     }
 
     public static FlowRegistryKey create(final Flow flow) {
@@ -32,18 +31,16 @@ public class FlowRegistryKeyFactory {
     }
 
     private static final class FlowRegistryKeyDto implements FlowRegistryKey {
-
         private final short tableId;
         private final int priority;
         private final BigInteger cookie;
         private final Match match;
-        private static final Match EMPTY_MATCH = new MatchBuilder().build();
 
-        public FlowRegistryKeyDto(final Flow flow) {
+        private FlowRegistryKeyDto(final Flow flow) {
             //TODO: mandatory flow input values (or default values) should be specified via yang model
             tableId = Preconditions.checkNotNull(flow.getTableId(), "flow tableId must not be null");
             priority = MoreObjects.firstNonNull(flow.getPriority(), OFConstants.DEFAULT_FLOW_PRIORITY);
-            match = flow.getMatch()==null ? EMPTY_MATCH : flow.getMatch();
+            match = MoreObjects.firstNonNull(flow.getMatch(), OFConstants.EMPTY_MATCH);
             cookie = MoreObjects.firstNonNull(flow.getCookie(), OFConstants.DEFAULT_FLOW_COOKIE).getValue();
         }
 
index 71656eca9cd04203e9b0f36caa74cbfa079bbf2c..c0ab557425994f2d06426f6820db02fdbb4099ab 100644 (file)
@@ -14,6 +14,7 @@ import com.google.common.util.concurrent.ListenableFuture;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import java.util.Objects;
 import java.util.concurrent.Future;
 import javax.annotation.Nullable;
 import org.opendaylight.openflowplugin.api.openflow.device.DeviceContext;
@@ -25,7 +26,6 @@ import org.opendaylight.openflowplugin.api.openflow.rpc.ItemLifeCycleSource;
 import org.opendaylight.openflowplugin.api.openflow.rpc.listener.ItemLifecycleListener;
 import org.opendaylight.openflowplugin.impl.registry.flow.FlowDescriptorFactory;
 import org.opendaylight.openflowplugin.impl.registry.flow.FlowRegistryKeyFactory;
-import org.opendaylight.openflowplugin.impl.util.FlowUtil;
 import org.opendaylight.openflowplugin.openflow.md.util.FlowCreatorUtil;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowId;
@@ -76,16 +76,20 @@ public class SalFlowServiceImpl implements SalFlowService, ItemLifeCycleSource {
 
     @Override
     public Future<RpcResult<AddFlowOutput>> addFlow(final AddFlowInput input) {
+        final FlowRegistryKey flowRegistryKey = FlowRegistryKeyFactory.create(input);
         final FlowId flowId;
-        if (null != input.getFlowRef()) {
+        final FlowDescriptor flowDescriptor;
+
+        if (Objects.nonNull(input.getFlowRef())) {
             flowId = input.getFlowRef().getValue().firstKeyOf(Flow.class, FlowKey.class).getId();
+            flowDescriptor = FlowDescriptorFactory.create(input.getTableId(), flowId);
+            deviceContext.getDeviceFlowRegistry().store(flowRegistryKey, flowDescriptor);
         } else {
-            flowId = FlowUtil.createAlienFlowId(input.getTableId());
+            flowId = deviceContext.getDeviceFlowRegistry().storeIfNecessary(flowRegistryKey);
+            flowDescriptor = FlowDescriptorFactory.create(input.getTableId(), flowId);
         }
+
         LOG.trace("Calling add flow for flow with ID ={}.", flowId);
-        final FlowRegistryKey flowRegistryKey = FlowRegistryKeyFactory.create(input);
-        final FlowDescriptor flowDescriptor = FlowDescriptorFactory.create(input.getTableId(), flowId);
-        deviceContext.getDeviceFlowRegistry().store(flowRegistryKey, flowDescriptor);
         final ListenableFuture<RpcResult<AddFlowOutput>> future =
                 flowAdd.processFlowModInputBuilders(flowAdd.toFlowModInputs(input));
         Futures.addCallback(future, new FutureCallback<RpcResult<AddFlowOutput>>() {
index 154582e61746fc2f8574e7a70cff9de4911157e6..732bc4f465ba2cce2f0a1213424fbacc0bd56810 100644 (file)
@@ -52,6 +52,7 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.meter.types.rev130918.Meter
 import org.opendaylight.yang.gen.v1.urn.opendaylight.model.statistics.types.rev130925.duration.DurationBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.model.statistics.types.rev130925.node.connector.statistics.BytesBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.model.statistics.types.rev130925.node.connector.statistics.PacketsBuilder;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.common.types.rev130731.MultipartType;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.MultipartReplyMessage;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.OfHeader;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.multipart.reply.multipart.reply.body.MultipartReplyAggregateCase;
@@ -118,312 +119,384 @@ public class SinglePurposeMultipartReplyTranslator {
         if (msg instanceof MultipartReplyMessage) {
             MultipartReplyMessage mpReply = (MultipartReplyMessage) msg;
             NodeId node = SinglePurposeMultipartReplyTranslator.nodeIdFromDatapathId(datapathId);
-            switch (mpReply.getType()) {
-                case OFPMPFLOW: {
-                    FlowsStatisticsUpdateBuilder message = new FlowsStatisticsUpdateBuilder();
-                    message.setId(node);
-                    message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
-                    message.setTransactionId(generateTransactionId(mpReply.getXid()));
-                    MultipartReplyFlowCase caseBody = (MultipartReplyFlowCase) mpReply.getMultipartReplyBody();
-                    MultipartReplyFlow replyBody = caseBody.getMultipartReplyFlow();
-                    message.setFlowAndStatisticsMapList(flowStatsConvertor.toSALFlowStatsList(replyBody.getFlowStats(), datapathId, ofVersion));
-
-                    listDataObject.add(message.build());
-                    return listDataObject;
-                }
-                case OFPMPAGGREGATE: {
-                    AggregateFlowStatisticsUpdateBuilder message = new AggregateFlowStatisticsUpdateBuilder();
-                    message.setId(node);
-                    message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
-                    message.setTransactionId(generateTransactionId(mpReply.getXid()));
-
-                    MultipartReplyAggregateCase caseBody = (MultipartReplyAggregateCase) mpReply.getMultipartReplyBody();
-                    MultipartReplyAggregate replyBody = caseBody.getMultipartReplyAggregate();
-                    message.setByteCount(new Counter64(replyBody.getByteCount()));
-                    message.setPacketCount(new Counter64(replyBody.getPacketCount()));
-                    message.setFlowCount(new Counter32(replyBody.getFlowCount()));
-
-                    listDataObject.add(message.build());
-                    return listDataObject;
-                }
-                case OFPMPPORTSTATS: {
-
-
-                    NodeConnectorStatisticsUpdateBuilder message = new NodeConnectorStatisticsUpdateBuilder();
-                    message.setId(node);
-                    message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
-                    message.setTransactionId(generateTransactionId(mpReply.getXid()));
-
-                    MultipartReplyPortStatsCase caseBody = (MultipartReplyPortStatsCase) mpReply.getMultipartReplyBody();
-                    MultipartReplyPortStats replyBody = caseBody.getMultipartReplyPortStats();
-
-                    List<NodeConnectorStatisticsAndPortNumberMap> statsMap =
-                            new ArrayList<NodeConnectorStatisticsAndPortNumberMap>();
-                    for (PortStats portStats : replyBody.getPortStats()) {
-
-                        NodeConnectorStatisticsAndPortNumberMapBuilder statsBuilder =
-                                new NodeConnectorStatisticsAndPortNumberMapBuilder();
-                        statsBuilder.setNodeConnectorId(
-                                InventoryDataServiceUtil.nodeConnectorIdfromDatapathPortNo(datapathId,
-                                        portStats.getPortNo(), ofVersion));
-
-                        BytesBuilder bytesBuilder = new BytesBuilder();
-                        bytesBuilder.setReceived(portStats.getRxBytes());
-                        bytesBuilder.setTransmitted(portStats.getTxBytes());
-                        statsBuilder.setBytes(bytesBuilder.build());
-
-                        PacketsBuilder packetsBuilder = new PacketsBuilder();
-                        packetsBuilder.setReceived(portStats.getRxPackets());
-                        packetsBuilder.setTransmitted(portStats.getTxPackets());
-                        statsBuilder.setPackets(packetsBuilder.build());
-
-                        DurationBuilder durationBuilder = new DurationBuilder();
-                        if (portStats.getDurationSec() != null) {
-                            durationBuilder.setSecond(new Counter32(portStats.getDurationSec()));
-                        }
-                        if (portStats.getDurationNsec() != null) {
-                            durationBuilder.setNanosecond(new Counter32(portStats.getDurationNsec()));
-                        }
-                        statsBuilder.setDuration(durationBuilder.build());
-                        statsBuilder.setCollisionCount(portStats.getCollisions());
-                        statsBuilder.setKey(new NodeConnectorStatisticsAndPortNumberMapKey(statsBuilder.getNodeConnectorId()));
-                        statsBuilder.setReceiveCrcError(portStats.getRxCrcErr());
-                        statsBuilder.setReceiveDrops(portStats.getRxDropped());
-                        statsBuilder.setReceiveErrors(portStats.getRxErrors());
-                        statsBuilder.setReceiveFrameError(portStats.getRxFrameErr());
-                        statsBuilder.setReceiveOverRunError(portStats.getRxOverErr());
-                        statsBuilder.setTransmitDrops(portStats.getTxDropped());
-                        statsBuilder.setTransmitErrors(portStats.getTxErrors());
-
-                        statsMap.add(statsBuilder.build());
-                    }
-                    message.setNodeConnectorStatisticsAndPortNumberMap(statsMap);
-
-
-                    listDataObject.add(message.build());
-                    return listDataObject;
-                }
-                case OFPMPGROUP: {
-                    GroupStatisticsUpdatedBuilder message = new GroupStatisticsUpdatedBuilder();
-                    message.setId(node);
-                    message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
-                    message.setTransactionId(generateTransactionId(mpReply.getXid()));
-                    MultipartReplyGroupCase caseBody = (MultipartReplyGroupCase) mpReply.getMultipartReplyBody();
-                    MultipartReplyGroup replyBody = caseBody.getMultipartReplyGroup();
-                    message.setGroupStats(groupStatsConvertor.toSALGroupStatsList(replyBody.getGroupStats()));
-
-                    listDataObject.add(message.build());
-                    return listDataObject;
-                }
-                case OFPMPGROUPDESC: {
-
-                    GroupDescStatsUpdatedBuilder message = new GroupDescStatsUpdatedBuilder();
-                    message.setId(node);
-                    message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
-                    message.setTransactionId(generateTransactionId(mpReply.getXid()));
-                    MultipartReplyGroupDescCase caseBody = (MultipartReplyGroupDescCase) mpReply.getMultipartReplyBody();
-                    MultipartReplyGroupDesc replyBody = caseBody.getMultipartReplyGroupDesc();
-
-                    message.setGroupDescStats(groupStatsConvertor.toSALGroupDescStatsList(replyBody.getGroupDesc(), ofVersion));
-
-                    listDataObject.add(message.build());
-                    return listDataObject;
-                }
-                case OFPMPGROUPFEATURES: {
-                    GroupFeaturesUpdatedBuilder message = new GroupFeaturesUpdatedBuilder();
-                    message.setId(node);
-                    message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
-                    message.setTransactionId(generateTransactionId(mpReply.getXid()));
-                    MultipartReplyGroupFeaturesCase caseBody = (MultipartReplyGroupFeaturesCase) mpReply.getMultipartReplyBody();
-                    MultipartReplyGroupFeatures replyBody = caseBody.getMultipartReplyGroupFeatures();
-                    List<Class<? extends GroupType>> supportedGroups =
-                            new ArrayList<Class<? extends GroupType>>();
-
-                    if (replyBody.getTypes().isOFPGTALL()) {
-                        supportedGroups.add(GroupAll.class);
-                    }
-                    if (replyBody.getTypes().isOFPGTSELECT()) {
-                        supportedGroups.add(GroupSelect.class);
-                    }
-                    if (replyBody.getTypes().isOFPGTINDIRECT()) {
-                        supportedGroups.add(GroupIndirect.class);
-                    }
-                    if (replyBody.getTypes().isOFPGTFF()) {
-                        supportedGroups.add(GroupFf.class);
-                    }
-                    message.setGroupTypesSupported(supportedGroups);
-                    message.setMaxGroups(replyBody.getMaxGroups());
-
-                    List<Class<? extends GroupCapability>> supportedCapabilities =
-                            new ArrayList<Class<? extends GroupCapability>>();
-
-                    if (replyBody.getCapabilities().isOFPGFCCHAINING()) {
-                        supportedCapabilities.add(Chaining.class);
-                    }
-                    if (replyBody.getCapabilities().isOFPGFCCHAININGCHECKS()) {
-                        supportedCapabilities.add(ChainingChecks.class);
-                    }
-                    if (replyBody.getCapabilities().isOFPGFCSELECTLIVENESS()) {
-                        supportedCapabilities.add(SelectLiveness.class);
-                    }
-                    if (replyBody.getCapabilities().isOFPGFCSELECTWEIGHT()) {
-                        supportedCapabilities.add(SelectWeight.class);
-                    }
-
-                    message.setGroupCapabilitiesSupported(supportedCapabilities);
-
-                    message.setActions(getGroupActionsSupportBitmap(replyBody.getActionsBitmap()));
-                    listDataObject.add(message.build());
-
-                    return listDataObject;
-                }
-                case OFPMPMETER: {
-                    MeterStatisticsUpdatedBuilder message = new MeterStatisticsUpdatedBuilder();
-                    message.setId(node);
-                    message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
-                    message.setTransactionId(generateTransactionId(mpReply.getXid()));
-
-                    MultipartReplyMeterCase caseBody = (MultipartReplyMeterCase) mpReply.getMultipartReplyBody();
-                    MultipartReplyMeter replyBody = caseBody.getMultipartReplyMeter();
-                    message.setMeterStats(meterStatsConvertor.toSALMeterStatsList(replyBody.getMeterStats()));
-
-                    listDataObject.add(message.build());
-                    return listDataObject;
-                }
-                case OFPMPMETERCONFIG: {
-
-                    MeterConfigStatsUpdatedBuilder message = new MeterConfigStatsUpdatedBuilder();
-                    message.setId(node);
-                    message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
-                    message.setTransactionId(generateTransactionId(mpReply.getXid()));
-
-                    MultipartReplyMeterConfigCase caseBody = (MultipartReplyMeterConfigCase) mpReply.getMultipartReplyBody();
-                    MultipartReplyMeterConfig replyBody = caseBody.getMultipartReplyMeterConfig();
-                    message.setMeterConfigStats(meterStatsConvertor.toSALMeterConfigList(replyBody.getMeterConfig()));
-
-                    listDataObject.add(message.build());
-                    return listDataObject;
-                }
-                case OFPMPMETERFEATURES: {
-                    //Convert OF message and send it to SAL listener
-                    MeterFeaturesUpdatedBuilder message = new MeterFeaturesUpdatedBuilder();
-                    message.setId(node);
-                    message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
-                    message.setTransactionId(generateTransactionId(mpReply.getXid()));
-
-                    MultipartReplyMeterFeaturesCase caseBody = (MultipartReplyMeterFeaturesCase) mpReply.getMultipartReplyBody();
-                    MultipartReplyMeterFeatures replyBody = caseBody.getMultipartReplyMeterFeatures();
-                    message.setMaxBands(replyBody.getMaxBands());
-                    message.setMaxColor(replyBody.getMaxColor());
-                    message.setMaxMeter(new Counter32(replyBody.getMaxMeter()));
-
-                    List<Class<? extends MeterCapability>> supportedCapabilities =
-                            new ArrayList<Class<? extends MeterCapability>>();
-                    if (replyBody.getCapabilities().isOFPMFBURST()) {
-                        supportedCapabilities.add(MeterBurst.class);
-                    }
-                    if (replyBody.getCapabilities().isOFPMFKBPS()) {
-                        supportedCapabilities.add(MeterKbps.class);
-
-                    }
-                    if (replyBody.getCapabilities().isOFPMFPKTPS()) {
-                        supportedCapabilities.add(MeterPktps.class);
-
-                    }
-                    if (replyBody.getCapabilities().isOFPMFSTATS()) {
-                        supportedCapabilities.add(MeterStats.class);
-
-                    }
-                    message.setMeterCapabilitiesSupported(supportedCapabilities);
-
-                    List<Class<? extends MeterBand>> supportedMeterBand =
-                            new ArrayList<Class<? extends MeterBand>>();
-                    if (replyBody.getBandTypes().isOFPMBTDROP()) {
-                        supportedMeterBand.add(MeterBandDrop.class);
-                    }
-                    if (replyBody.getBandTypes().isOFPMBTDSCPREMARK()) {
-                        supportedMeterBand.add(MeterBandDscpRemark.class);
-                    }
-                    message.setMeterBandSupported(supportedMeterBand);
-                    listDataObject.add(message.build());
-
-                    return listDataObject;
-                }
-                case OFPMPTABLE: {
-
-                    FlowTableStatisticsUpdateBuilder message = new FlowTableStatisticsUpdateBuilder();
-                    message.setId(node);
-                    message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
-                    message.setTransactionId(generateTransactionId(mpReply.getXid()));
-
-                    MultipartReplyTableCase caseBody = (MultipartReplyTableCase) mpReply.getMultipartReplyBody();
-                    MultipartReplyTable replyBody = caseBody.getMultipartReplyTable();
-                    List<TableStats> swTablesStats = replyBody.getTableStats();
-
-                    List<FlowTableAndStatisticsMap> salFlowStats = new ArrayList<FlowTableAndStatisticsMap>();
-                    for (TableStats swTableStats : swTablesStats) {
-                        FlowTableAndStatisticsMapBuilder statisticsBuilder = new FlowTableAndStatisticsMapBuilder();
-
-                        statisticsBuilder.setActiveFlows(new Counter32(swTableStats.getActiveCount()));
-                        statisticsBuilder.setPacketsLookedUp(new Counter64(swTableStats.getLookupCount()));
-                        statisticsBuilder.setPacketsMatched(new Counter64(swTableStats.getMatchedCount()));
-                        statisticsBuilder.setTableId(new TableId(swTableStats.getTableId()));
-                        salFlowStats.add(statisticsBuilder.build());
-                    }
-
-                    message.setFlowTableAndStatisticsMap(salFlowStats);
-                    listDataObject.add(message.build());
-                    return listDataObject;
-                }
-                case OFPMPQUEUE: {
-
-                    QueueStatisticsUpdateBuilder message = new QueueStatisticsUpdateBuilder();
-                    message.setId(node);
-                    message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
-                    message.setTransactionId(generateTransactionId(mpReply.getXid()));
-
-                    MultipartReplyQueueCase caseBody = (MultipartReplyQueueCase) mpReply.getMultipartReplyBody();
-                    MultipartReplyQueue replyBody = caseBody.getMultipartReplyQueue();
-
-                    List<QueueIdAndStatisticsMap> statsMap =
-                            new ArrayList<QueueIdAndStatisticsMap>();
-
-                    for (QueueStats queueStats : replyBody.getQueueStats()) {
-
-                        QueueIdAndStatisticsMapBuilder statsBuilder =
-                                new QueueIdAndStatisticsMapBuilder();
-                        statsBuilder.setNodeConnectorId(
-                                InventoryDataServiceUtil.nodeConnectorIdfromDatapathPortNo(datapathId,
-                                        queueStats.getPortNo(), ofVersion));
-                        statsBuilder.setTransmissionErrors(new Counter64(queueStats.getTxErrors()));
-                        statsBuilder.setTransmittedBytes(new Counter64(queueStats.getTxBytes()));
-                        statsBuilder.setTransmittedPackets(new Counter64(queueStats.getTxPackets()));
-
-                        DurationBuilder durationBuilder = new DurationBuilder();
-                        durationBuilder.setSecond(new Counter32(queueStats.getDurationSec()));
-                        durationBuilder.setNanosecond(new Counter32(queueStats.getDurationNsec()));
-                        statsBuilder.setDuration(durationBuilder.build());
-
-                        statsBuilder.setQueueId(new QueueId(queueStats.getQueueId()));
-                        statsBuilder.setNodeConnectorId(InventoryDataServiceUtil.nodeConnectorIdfromDatapathPortNo(datapathId,
-                                queueStats.getPortNo(), ofVersion));
-
-                        statsMap.add(statsBuilder.build());
-                    }
-                    message.setQueueIdAndStatisticsMap(statsMap);
-
-                    listDataObject.add(message.build());
-                    return listDataObject;
-                }
-
-                default:
-                    return listDataObject;
-            }
+
+            translateFlow(listDataObject, mpReply, node, ofVersion, datapathId);
+            translateAggregate(listDataObject, mpReply, node);
+            translatePortStats(listDataObject, mpReply, node, ofVersion, datapathId);
+            translateGroup(listDataObject, mpReply, node);
+            translateGroupDesc(listDataObject, mpReply, node, ofVersion);
+            translateGroupFeatures(listDataObject, mpReply, node);
+            translateMeter(listDataObject, mpReply, node);
+            translateMeterConfig(listDataObject, mpReply, node);
+            translateMeterFeatures(listDataObject, mpReply, node);
+            translateTable(listDataObject, mpReply, node);
+            translateQueue(listDataObject, mpReply, node, ofVersion, datapathId);
         }
 
         return listDataObject;
     }
 
+    private static void translateFlow(final List<DataObject> listDataObject,
+                                      final MultipartReplyMessage mpReply,
+                                      final NodeId node,
+                                      final OpenflowVersion ofVersion,
+                                      final BigInteger datapathId) {
+        if (!MultipartType.OFPMPFLOW.equals(mpReply.getType())) {
+            return;
+        }
+
+        FlowsStatisticsUpdateBuilder message = new FlowsStatisticsUpdateBuilder();
+        message.setId(node);
+        message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
+        message.setTransactionId(generateTransactionId(mpReply.getXid()));
+        MultipartReplyFlowCase caseBody = (MultipartReplyFlowCase) mpReply.getMultipartReplyBody();
+        MultipartReplyFlow replyBody = caseBody.getMultipartReplyFlow();
+        message.setFlowAndStatisticsMapList(flowStatsConvertor.toSALFlowStatsList(replyBody.getFlowStats(), datapathId, ofVersion));
+
+        listDataObject.add(message.build());
+    }
+
+    private static void translateAggregate(final List<DataObject> listDataObject,
+                                           final MultipartReplyMessage mpReply,
+                                           final NodeId node) {
+        if (!MultipartType.OFPMPAGGREGATE.equals(mpReply.getType())) {
+            return;
+        }
+
+        AggregateFlowStatisticsUpdateBuilder message = new AggregateFlowStatisticsUpdateBuilder();
+        message.setId(node);
+        message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
+        message.setTransactionId(generateTransactionId(mpReply.getXid()));
+
+        MultipartReplyAggregateCase caseBody = (MultipartReplyAggregateCase) mpReply.getMultipartReplyBody();
+        MultipartReplyAggregate replyBody = caseBody.getMultipartReplyAggregate();
+        message.setByteCount(new Counter64(replyBody.getByteCount()));
+        message.setPacketCount(new Counter64(replyBody.getPacketCount()));
+        message.setFlowCount(new Counter32(replyBody.getFlowCount()));
+
+        listDataObject.add(message.build());
+    }
+
+    private static void translatePortStats(final List<DataObject> listDataObject,
+                                           final MultipartReplyMessage mpReply,
+                                           final NodeId node,
+                                           final OpenflowVersion ofVersion,
+                                           final BigInteger datapathId) {
+        if (!MultipartType.OFPMPPORTSTATS.equals(mpReply.getType())) {
+            return;
+        }
+
+        NodeConnectorStatisticsUpdateBuilder message = new NodeConnectorStatisticsUpdateBuilder();
+        message.setId(node);
+        message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
+        message.setTransactionId(generateTransactionId(mpReply.getXid()));
+
+        MultipartReplyPortStatsCase caseBody = (MultipartReplyPortStatsCase) mpReply.getMultipartReplyBody();
+        MultipartReplyPortStats replyBody = caseBody.getMultipartReplyPortStats();
+
+        List<NodeConnectorStatisticsAndPortNumberMap> statsMap =
+                new ArrayList<>();
+        for (PortStats portStats : replyBody.getPortStats()) {
+
+            NodeConnectorStatisticsAndPortNumberMapBuilder statsBuilder =
+                    new NodeConnectorStatisticsAndPortNumberMapBuilder();
+            statsBuilder.setNodeConnectorId(
+                    InventoryDataServiceUtil.nodeConnectorIdfromDatapathPortNo(datapathId,
+                            portStats.getPortNo(), ofVersion));
+
+            BytesBuilder bytesBuilder = new BytesBuilder();
+            bytesBuilder.setReceived(portStats.getRxBytes());
+            bytesBuilder.setTransmitted(portStats.getTxBytes());
+            statsBuilder.setBytes(bytesBuilder.build());
+
+            PacketsBuilder packetsBuilder = new PacketsBuilder();
+            packetsBuilder.setReceived(portStats.getRxPackets());
+            packetsBuilder.setTransmitted(portStats.getTxPackets());
+            statsBuilder.setPackets(packetsBuilder.build());
+
+            DurationBuilder durationBuilder = new DurationBuilder();
+            if (portStats.getDurationSec() != null) {
+                durationBuilder.setSecond(new Counter32(portStats.getDurationSec()));
+            }
+            if (portStats.getDurationNsec() != null) {
+                durationBuilder.setNanosecond(new Counter32(portStats.getDurationNsec()));
+            }
+            statsBuilder.setDuration(durationBuilder.build());
+            statsBuilder.setCollisionCount(portStats.getCollisions());
+            statsBuilder.setKey(new NodeConnectorStatisticsAndPortNumberMapKey(statsBuilder.getNodeConnectorId()));
+            statsBuilder.setReceiveCrcError(portStats.getRxCrcErr());
+            statsBuilder.setReceiveDrops(portStats.getRxDropped());
+            statsBuilder.setReceiveErrors(portStats.getRxErrors());
+            statsBuilder.setReceiveFrameError(portStats.getRxFrameErr());
+            statsBuilder.setReceiveOverRunError(portStats.getRxOverErr());
+            statsBuilder.setTransmitDrops(portStats.getTxDropped());
+            statsBuilder.setTransmitErrors(portStats.getTxErrors());
+
+            statsMap.add(statsBuilder.build());
+        }
+        message.setNodeConnectorStatisticsAndPortNumberMap(statsMap);
+
+
+        listDataObject.add(message.build());
+    }
+
+    private static void translateGroup(final List<DataObject> listDataObject,
+                                       final MultipartReplyMessage mpReply,
+                                       final NodeId node) {
+        if (!MultipartType.OFPMPGROUP.equals(mpReply.getType())) {
+            return;
+        }
+
+        GroupStatisticsUpdatedBuilder message = new GroupStatisticsUpdatedBuilder();
+        message.setId(node);
+        message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
+        message.setTransactionId(generateTransactionId(mpReply.getXid()));
+        MultipartReplyGroupCase caseBody = (MultipartReplyGroupCase) mpReply.getMultipartReplyBody();
+        MultipartReplyGroup replyBody = caseBody.getMultipartReplyGroup();
+        message.setGroupStats(groupStatsConvertor.toSALGroupStatsList(replyBody.getGroupStats()));
+
+        listDataObject.add(message.build());
+    }
+
+    private static void translateGroupDesc(final List<DataObject> listDataObject,
+                                           final MultipartReplyMessage mpReply,
+                                           final NodeId node,
+                                           final OpenflowVersion ofVersion) {
+        if (!MultipartType.OFPMPGROUPDESC.equals(mpReply.getType())) {
+            return;
+        }
+
+        GroupDescStatsUpdatedBuilder message = new GroupDescStatsUpdatedBuilder();
+        message.setId(node);
+        message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
+        message.setTransactionId(generateTransactionId(mpReply.getXid()));
+        MultipartReplyGroupDescCase caseBody = (MultipartReplyGroupDescCase) mpReply.getMultipartReplyBody();
+        MultipartReplyGroupDesc replyBody = caseBody.getMultipartReplyGroupDesc();
+
+        message.setGroupDescStats(groupStatsConvertor.toSALGroupDescStatsList(replyBody.getGroupDesc(), ofVersion));
+
+        listDataObject.add(message.build());
+    }
+
+    private static void translateGroupFeatures(final List<DataObject> listDataObject,
+                                               final MultipartReplyMessage mpReply,
+                                               final NodeId node) {
+        if (!MultipartType.OFPMPGROUPFEATURES.equals(mpReply.getType())) {
+            return;
+        }
+
+        GroupFeaturesUpdatedBuilder message = new GroupFeaturesUpdatedBuilder();
+        message.setId(node);
+        message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
+        message.setTransactionId(generateTransactionId(mpReply.getXid()));
+        MultipartReplyGroupFeaturesCase caseBody = (MultipartReplyGroupFeaturesCase) mpReply.getMultipartReplyBody();
+        MultipartReplyGroupFeatures replyBody = caseBody.getMultipartReplyGroupFeatures();
+        List<Class<? extends GroupType>> supportedGroups =
+                new ArrayList<>();
+
+        if (replyBody.getTypes().isOFPGTALL()) {
+            supportedGroups.add(GroupAll.class);
+        }
+        if (replyBody.getTypes().isOFPGTSELECT()) {
+            supportedGroups.add(GroupSelect.class);
+        }
+        if (replyBody.getTypes().isOFPGTINDIRECT()) {
+            supportedGroups.add(GroupIndirect.class);
+        }
+        if (replyBody.getTypes().isOFPGTFF()) {
+            supportedGroups.add(GroupFf.class);
+        }
+        message.setGroupTypesSupported(supportedGroups);
+        message.setMaxGroups(replyBody.getMaxGroups());
+
+        List<Class<? extends GroupCapability>> supportedCapabilities =
+                new ArrayList<>();
+
+        if (replyBody.getCapabilities().isOFPGFCCHAINING()) {
+            supportedCapabilities.add(Chaining.class);
+        }
+        if (replyBody.getCapabilities().isOFPGFCCHAININGCHECKS()) {
+            supportedCapabilities.add(ChainingChecks.class);
+        }
+        if (replyBody.getCapabilities().isOFPGFCSELECTLIVENESS()) {
+            supportedCapabilities.add(SelectLiveness.class);
+        }
+        if (replyBody.getCapabilities().isOFPGFCSELECTWEIGHT()) {
+            supportedCapabilities.add(SelectWeight.class);
+        }
+
+        message.setGroupCapabilitiesSupported(supportedCapabilities);
+
+        message.setActions(getGroupActionsSupportBitmap(replyBody.getActionsBitmap()));
+        listDataObject.add(message.build());
+    }
+
+    private static void translateMeter(final List<DataObject> listDataObject,
+                                       final MultipartReplyMessage mpReply,
+                                       final NodeId node) {
+        if (!MultipartType.OFPMPMETER.equals(mpReply.getType())) {
+            return;
+        }
+
+        MeterStatisticsUpdatedBuilder message = new MeterStatisticsUpdatedBuilder();
+        message.setId(node);
+        message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
+        message.setTransactionId(generateTransactionId(mpReply.getXid()));
+
+        MultipartReplyMeterCase caseBody = (MultipartReplyMeterCase) mpReply.getMultipartReplyBody();
+        MultipartReplyMeter replyBody = caseBody.getMultipartReplyMeter();
+        message.setMeterStats(meterStatsConvertor.toSALMeterStatsList(replyBody.getMeterStats()));
+
+        listDataObject.add(message.build());
+    }
+
+    private static void translateMeterConfig(final List<DataObject> listDataObject,
+                                             final MultipartReplyMessage mpReply,
+                                             final NodeId node) {
+        if (!MultipartType.OFPMPMETERCONFIG.equals(mpReply.getType())) {
+            return;
+        }
+
+        MeterConfigStatsUpdatedBuilder message = new MeterConfigStatsUpdatedBuilder();
+        message.setId(node);
+        message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
+        message.setTransactionId(generateTransactionId(mpReply.getXid()));
+
+        MultipartReplyMeterConfigCase caseBody = (MultipartReplyMeterConfigCase) mpReply.getMultipartReplyBody();
+        MultipartReplyMeterConfig replyBody = caseBody.getMultipartReplyMeterConfig();
+        message.setMeterConfigStats(meterStatsConvertor.toSALMeterConfigList(replyBody.getMeterConfig()));
+
+        listDataObject.add(message.build());
+    }
+
+    private static void translateMeterFeatures(final List<DataObject> listDataObject,
+                                               final MultipartReplyMessage mpReply,
+                                               final NodeId node) {
+        if (!MultipartType.OFPMPMETERFEATURES.equals(mpReply.getType())) {
+            return;
+        }
+
+        //Convert OF message and send it to SAL listener
+        MeterFeaturesUpdatedBuilder message = new MeterFeaturesUpdatedBuilder();
+        message.setId(node);
+        message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
+        message.setTransactionId(generateTransactionId(mpReply.getXid()));
+
+        MultipartReplyMeterFeaturesCase caseBody = (MultipartReplyMeterFeaturesCase) mpReply.getMultipartReplyBody();
+        MultipartReplyMeterFeatures replyBody = caseBody.getMultipartReplyMeterFeatures();
+        message.setMaxBands(replyBody.getMaxBands());
+        message.setMaxColor(replyBody.getMaxColor());
+        message.setMaxMeter(new Counter32(replyBody.getMaxMeter()));
+
+        List<Class<? extends MeterCapability>> supportedCapabilities =
+                new ArrayList<>();
+        if (replyBody.getCapabilities().isOFPMFBURST()) {
+            supportedCapabilities.add(MeterBurst.class);
+        }
+        if (replyBody.getCapabilities().isOFPMFKBPS()) {
+            supportedCapabilities.add(MeterKbps.class);
+
+        }
+        if (replyBody.getCapabilities().isOFPMFPKTPS()) {
+            supportedCapabilities.add(MeterPktps.class);
+
+        }
+        if (replyBody.getCapabilities().isOFPMFSTATS()) {
+            supportedCapabilities.add(MeterStats.class);
+
+        }
+        message.setMeterCapabilitiesSupported(supportedCapabilities);
+
+        List<Class<? extends MeterBand>> supportedMeterBand =
+                new ArrayList<>();
+        if (replyBody.getBandTypes().isOFPMBTDROP()) {
+            supportedMeterBand.add(MeterBandDrop.class);
+        }
+        if (replyBody.getBandTypes().isOFPMBTDSCPREMARK()) {
+            supportedMeterBand.add(MeterBandDscpRemark.class);
+        }
+        message.setMeterBandSupported(supportedMeterBand);
+        listDataObject.add(message.build());
+    }
+
+    private static void translateTable(final List<DataObject> listDataObject,
+                                       final MultipartReplyMessage mpReply,
+                                       final NodeId node) {
+        if (!MultipartType.OFPMPTABLE.equals(mpReply.getType())) {
+            return;
+        }
+
+        FlowTableStatisticsUpdateBuilder message = new FlowTableStatisticsUpdateBuilder();
+        message.setId(node);
+        message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
+        message.setTransactionId(generateTransactionId(mpReply.getXid()));
+
+        MultipartReplyTableCase caseBody = (MultipartReplyTableCase) mpReply.getMultipartReplyBody();
+        MultipartReplyTable replyBody = caseBody.getMultipartReplyTable();
+        List<TableStats> swTablesStats = replyBody.getTableStats();
+
+        List<FlowTableAndStatisticsMap> salFlowStats = new ArrayList<FlowTableAndStatisticsMap>();
+        for (TableStats swTableStats : swTablesStats) {
+            FlowTableAndStatisticsMapBuilder statisticsBuilder = new FlowTableAndStatisticsMapBuilder();
+
+            statisticsBuilder.setActiveFlows(new Counter32(swTableStats.getActiveCount()));
+            statisticsBuilder.setPacketsLookedUp(new Counter64(swTableStats.getLookupCount()));
+            statisticsBuilder.setPacketsMatched(new Counter64(swTableStats.getMatchedCount()));
+            statisticsBuilder.setTableId(new TableId(swTableStats.getTableId()));
+            salFlowStats.add(statisticsBuilder.build());
+        }
+
+        message.setFlowTableAndStatisticsMap(salFlowStats);
+        listDataObject.add(message.build());
+    }
+
+    private static void translateQueue(final List<DataObject> listDataObject,
+                                       final MultipartReplyMessage mpReply,
+                                       final NodeId node,
+                                       final OpenflowVersion ofVersion,
+                                       final BigInteger datapathId) {
+        if (!MultipartType.OFPMPQUEUE.equals(mpReply.getType())) {
+            return;
+        }
+
+        QueueStatisticsUpdateBuilder message = new QueueStatisticsUpdateBuilder();
+        message.setId(node);
+        message.setMoreReplies(mpReply.getFlags().isOFPMPFREQMORE());
+        message.setTransactionId(generateTransactionId(mpReply.getXid()));
+
+        MultipartReplyQueueCase caseBody = (MultipartReplyQueueCase) mpReply.getMultipartReplyBody();
+        MultipartReplyQueue replyBody = caseBody.getMultipartReplyQueue();
+
+        List<QueueIdAndStatisticsMap> statsMap =
+                new ArrayList<QueueIdAndStatisticsMap>();
+
+        for (QueueStats queueStats : replyBody.getQueueStats()) {
+
+            QueueIdAndStatisticsMapBuilder statsBuilder =
+                    new QueueIdAndStatisticsMapBuilder();
+            statsBuilder.setNodeConnectorId(
+                    InventoryDataServiceUtil.nodeConnectorIdfromDatapathPortNo(datapathId,
+                            queueStats.getPortNo(), ofVersion));
+            statsBuilder.setTransmissionErrors(new Counter64(queueStats.getTxErrors()));
+            statsBuilder.setTransmittedBytes(new Counter64(queueStats.getTxBytes()));
+            statsBuilder.setTransmittedPackets(new Counter64(queueStats.getTxPackets()));
+
+            DurationBuilder durationBuilder = new DurationBuilder();
+            durationBuilder.setSecond(new Counter32(queueStats.getDurationSec()));
+            durationBuilder.setNanosecond(new Counter32(queueStats.getDurationNsec()));
+            statsBuilder.setDuration(durationBuilder.build());
+
+            statsBuilder.setQueueId(new QueueId(queueStats.getQueueId()));
+            statsBuilder.setNodeConnectorId(InventoryDataServiceUtil.nodeConnectorIdfromDatapathPortNo(datapathId,
+                    queueStats.getPortNo(), ofVersion));
+
+            statsMap.add(statsBuilder.build());
+        }
+        message.setQueueIdAndStatisticsMap(statsMap);
+
+        listDataObject.add(message.build());
+    }
+
     private static NodeId nodeIdFromDatapathId(final BigInteger datapathId) {
         String current = datapathId.toString();
         return new NodeId("openflow:" + current);
index 931532e41e5e7c404619ff8aa5a12e095194524f..2aad0ecc568cb8d21d3a723d4da58d424eb32b11 100644 (file)
@@ -16,7 +16,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
 import javax.annotation.Nullable;
 import org.apache.commons.lang3.tuple.Pair;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
@@ -45,13 +44,7 @@ import org.opendaylight.yangtools.yang.common.RpcResultBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * Created by Martin Bobak &lt;mbobak@cisco.com&gt; on 8.4.2015.
- */
 public final class FlowUtil {
-
-    private static final String ALIEN_SYSTEM_FLOW_ID = "#UF$TABLE*";
-    private static final AtomicInteger unaccountedFlowsCounter = new AtomicInteger(0);
     private static final Logger LOG = LoggerFactory.getLogger(FlowUtil.class);
     private static final RpcResultBuilder<List<BatchFailedFlowsOutput>> SUCCESSFUL_FLOW_OUTPUT_RPC_RESULT =
             RpcResultBuilder.success(Collections.<BatchFailedFlowsOutput>emptyList());
@@ -151,20 +144,6 @@ public final class FlowUtil {
         return resultBld;
     }
 
-    /**
-     * Create alien flow id
-     *
-     * @param tableId the table id
-     * @return the flow id
-     */
-    public static FlowId createAlienFlowId(final short tableId) {
-        final StringBuilder sBuilder = new StringBuilder(ALIEN_SYSTEM_FLOW_ID)
-                .append(tableId).append('-').append(unaccountedFlowsCounter.incrementAndGet());
-        String alienId = sBuilder.toString();
-        return new FlowId(alienId);
-
-    }
-
     /**
      * Factory method: create {@link Function} which attaches barrier response to given {@link RpcResult}&lt;T&gt;
      * and changes success flag if needed.
index 6ee4a3c4ce43d9f82b658fafddaee60ac9f77f29..43544aff61ddbab1d37510be01b095eb378e0e41 100644 (file)
@@ -8,6 +8,8 @@
 
 package org.opendaylight.openflowplugin.impl.registry.flow;
 
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
@@ -15,6 +17,8 @@ import static org.mockito.Mockito.when;
 
 import com.google.common.base.Optional;
 import com.google.common.util.concurrent.Futures;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -42,6 +46,9 @@ import org.opendaylight.yangtools.yang.binding.KeyedInstanceIdentifier;
 @RunWith(MockitoJUnitRunner.class)
 public class DeviceFlowRegistryImplTest {
     private static final String NODE_ID = "openflow:1";
+    private static final Pattern INDEX_PATTERN = Pattern.compile("^#UF\\$TABLE\\*1-([0-9]+)$");
+    private static final Short DUMMY_TABLE_ID = 1;
+
     private DeviceFlowRegistryImpl deviceFlowRegistry;
     private FlowRegistryKey key;
     private FlowDescriptor descriptor;
@@ -146,4 +153,26 @@ public class DeviceFlowRegistryImplTest {
         deviceFlowRegistry.removeMarked();
         Assert.assertEquals(1, deviceFlowRegistry.getAllFlowDescriptors().size());
     }
+
+    @Test
+    public void createAlienFlowIdTest() throws Exception {
+        final String alienFlowId1 = DeviceFlowRegistryImpl.createAlienFlowId(DUMMY_TABLE_ID).getValue();
+        final Integer index1 = parseIndex(alienFlowId1);
+        final String alienFlowId2 = DeviceFlowRegistryImpl.createAlienFlowId(DUMMY_TABLE_ID).getValue();
+        final Integer index2 = parseIndex(alienFlowId2);
+
+        assertNotNull("index1 parsing failed: " + alienFlowId1, index1);
+        assertNotNull("index2 parsing failed: " + alienFlowId2, index2);
+        assertTrue(index1 < index2);
+    }
+
+    private static Integer parseIndex(String alienFlowIdValue) {
+        final Matcher mach = INDEX_PATTERN.matcher(alienFlowIdValue);
+
+        if (mach.find()) {
+            return Integer.valueOf(mach.group(1));
+        }
+
+        return null;
+    }
 }
\ No newline at end of file
index 17eacd497b6dfae4b1f0001faaa6422f7717abf1..6600c9e8a5ec0ab102e997dad54ed54643cd6335 100644 (file)
@@ -149,6 +149,7 @@ public class SalFlowServiceImplTest extends TestCase {
         Mockito.doReturn(Futures.<RequestContext<Object>>immediateFailedFuture(new Exception("ut-failed-response")))
                 .when(requestContext).getFuture();
 
+        mockingFlowRegistryLookup();
         final Future<RpcResult<AddFlowOutput>> rpcResultFuture = salFlowService.addFlow(mockedAddFlowInput);
 
         assertNotNull(rpcResultFuture);
@@ -180,6 +181,7 @@ public class SalFlowServiceImplTest extends TestCase {
         AddFlowInput mockedAddFlowInput = createFlowMock(AddFlowInput.class);
         salFlowService.setItemLifecycleListener(itemLifecycleListener);
 
+        mockingFlowRegistryLookup();
         verifyOutput(salFlowService.addFlow(mockedAddFlowInput));
         if (itemLifecycleListener != null) {
             Mockito.verify(itemLifecycleListener).onAdded(Matchers.<KeyedInstanceIdentifier<Flow, FlowKey>>any(), Matchers.<Flow>any());
@@ -250,9 +252,11 @@ public class SalFlowServiceImplTest extends TestCase {
 
     private void mockingFlowRegistryLookup() {
         FlowDescriptor mockedFlowDescriptor = mock(FlowDescriptor.class);
-        when(mockedFlowDescriptor.getFlowId()).thenReturn(new FlowId(DUMMY_FLOW_ID));
+        FlowId flowId = new FlowId(DUMMY_FLOW_ID);
+        when(mockedFlowDescriptor.getFlowId()).thenReturn(flowId);
         when(mockedFlowDescriptor.getTableKey()).thenReturn(new TableKey(DUMMY_TABLE_ID));
 
+        when(deviceFlowRegistry.storeIfNecessary(Matchers.any(FlowRegistryKey.class))).thenReturn(flowId);
         when(deviceFlowRegistry.retrieveIdForFlow(Matchers.any(FlowRegistryKey.class))).thenReturn(mockedFlowDescriptor);
     }
 
index 144472a6a637103ef682ab8dea05fa72b4b016a6..b5fe5d8d23ad05ca4ca4c5d5c4d2e5ba92114d19 100644 (file)
@@ -8,15 +8,10 @@
 
 package org.opendaylight.openflowplugin.impl.util;
 
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
 import com.google.common.base.Function;
 import com.google.common.collect.Lists;
 import java.util.Collections;
 import java.util.List;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 import org.apache.commons.lang3.tuple.Pair;
 import org.junit.Assert;
 import org.junit.Test;
@@ -41,33 +36,11 @@ import org.opendaylight.yangtools.yang.common.RpcResult;
 import org.opendaylight.yangtools.yang.common.RpcResultBuilder;
 
 public class FlowUtilTest {
-
-    public static final Pattern INDEX_PATTERN = Pattern.compile("^#UF\\$TABLE\\*1-([0-9]+)$");
     public static final NodeId DUMMY_NODE_ID = new NodeId("dummyNodeId");
     public static final FlowId DUMMY_FLOW_ID = new FlowId("dummyFlowId");
     public static final FlowId DUMMY_FLOW_ID_2 = new FlowId("dummyFlowId_2");
     public static final Short DUMMY_TABLE_ID = 1;
 
-    @Test
-    public void createAlienFlowIdTest() {
-        final String alienFlowId1 = FlowUtil.createAlienFlowId(DUMMY_TABLE_ID).getValue();
-        final Integer index1 = parseIndex(alienFlowId1);
-        final String alienFlowId2 = FlowUtil.createAlienFlowId(DUMMY_TABLE_ID).getValue();
-        final Integer index2 = parseIndex(alienFlowId2);
-
-        assertNotNull("index1 parsing failed: " + alienFlowId1, index1);
-        assertNotNull("index2 parsing failed: " + alienFlowId2, index2);
-        assertTrue(index1 < index2);
-    }
-
-    private static Integer parseIndex(String alienFlowIdValue) {
-        final Matcher mach = INDEX_PATTERN.matcher(alienFlowIdValue);
-        if (mach.find()) {
-            return Integer.valueOf(mach.group(1));
-        }
-        return null;
-    }
-
     @Test
     public void testBuildFlowPath() throws Exception {
         final InstanceIdentifier<Node> nodePath = InstanceIdentifier