BUG-2299: allow PCEP extensions to be reconfigured
[bgpcep.git] / pcep / topology-provider / src / main / java / org / opendaylight / bgpcep / pcep / topology / provider / ServerSessionManager.java
index 2f33bf9a9427fdbea1522027d28a99067dfc3758..535ba28d066166c0357d737623e32beb732754c7 100644 (file)
@@ -7,16 +7,25 @@
  */
 package org.opendaylight.bgpcep.pcep.topology.provider;
 
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ListenableFuture;
+import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
-
-import org.opendaylight.controller.md.sal.common.api.TransactionStatus;
-import org.opendaylight.controller.sal.binding.api.data.DataModificationTransaction;
-import org.opendaylight.controller.sal.binding.api.data.DataProviderService;
+import org.opendaylight.controller.config.yang.pcep.topology.provider.PCEPTopologyProviderRuntimeMXBean;
+import org.opendaylight.controller.config.yang.pcep.topology.provider.PCEPTopologyProviderRuntimeRegistration;
+import org.opendaylight.controller.config.yang.pcep.topology.provider.PCEPTopologyProviderRuntimeRegistrator;
+import org.opendaylight.controller.md.sal.binding.api.DataBroker;
+import org.opendaylight.controller.md.sal.binding.api.ReadWriteTransaction;
+import org.opendaylight.controller.md.sal.binding.api.WriteTransaction;
+import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
+import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
+import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
 import org.opendaylight.protocol.framework.SessionListenerFactory;
+import org.opendaylight.protocol.pcep.PCEPSession;
 import org.opendaylight.protocol.pcep.PCEPSessionListener;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.AddLspArgs;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.EnsureLspOperationalInput;
@@ -26,158 +35,145 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.TopologyTypes1Builder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.UpdateLspArgs;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.topology.pcep.type.TopologyPcepBuilder;
+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.NetworkTopologyBuilder;
 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;
 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.Node;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.TopologyTypesBuilder;
-import org.opendaylight.yangtools.yang.binding.DataObject;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
-import org.opendaylight.yangtools.yang.common.RpcResult;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Preconditions;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.JdkFutureAdapters;
-import com.google.common.util.concurrent.ListenableFuture;
-
 /**
  *
  */
-final class ServerSessionManager implements SessionListenerFactory<PCEPSessionListener>, AutoCloseable, TopologySessionRPCs {
-       private static final Logger LOG = LoggerFactory.getLogger(ServerSessionManager.class);
-       private static final long DEFAULT_HOLD_STATE_NANOS = TimeUnit.MINUTES.toNanos(5);
-
-       private final Map<NodeId, TopologySessionListener> nodes = new HashMap<>();
-       private final Map<NodeId, TopologyNodeState<?>> state = new HashMap<>();
-       private final InstanceIdentifier<Topology> topology;
-       private final DataProviderService dataProvider;
-
-       public ServerSessionManager(final DataProviderService dataProvider, final InstanceIdentifier<Topology> topology) {
-               this.dataProvider = Preconditions.checkNotNull(dataProvider);
-               this.topology = Preconditions.checkNotNull(topology);
-
-               // Make sure the topology does not exist
-               final Object c = dataProvider.readOperationalData(topology);
-               Preconditions.checkArgument(c == null, "Topology %s already exists", topology);
-
-               // Now create the base topology
-               final TopologyKey k = InstanceIdentifier.keyOf(topology);
-               final DataModificationTransaction t = dataProvider.beginTransaction();
-               t.putOperationalData(
-                               topology,
-                               new TopologyBuilder().setKey(k).setTopologyId(k.getTopologyId()).setTopologyTypes(
-                                               new TopologyTypesBuilder().addAugmentation(TopologyTypes1.class,
-                                                               new TopologyTypes1Builder().setTopologyPcep(new TopologyPcepBuilder().build()).build()).build()).setNode(
-                                                                               new ArrayList<Node>()).build());
-
-               Futures.addCallback(JdkFutureAdapters.listenInPoolThread(t.commit()), new FutureCallback<RpcResult<TransactionStatus>>() {
-                       @Override
-                       public void onSuccess(final RpcResult<TransactionStatus> result) {
-                               LOG.trace("Topology {} created successfully", topology);
-                       }
-
-                       @Override
-                       public void onFailure(final Throwable t) {
-                               LOG.error("Failed to create topology {}", topology);
-                       }
-               });
-       }
-
-       public void releaseNodeState(final TopologyNodeState<?> nodeState) {
-               LOG.debug("Node {} unbound", nodeState.getNodeId());
-               this.nodes.remove(nodeState.getNodeId());
-               nodeState.released();
-       }
-
-       synchronized <T> TopologyNodeState<T> takeNodeState(final NodeId id, final TopologySessionListener sessionListener) {
-               LOG.debug("Node {} bound to listener {}", id, sessionListener);
-
-               @SuppressWarnings("unchecked")
-               TopologyNodeState<T> ret = (TopologyNodeState<T>) this.state.get(id);
-               if (ret == null) {
-                       ret = new TopologyNodeState<T>(id, DEFAULT_HOLD_STATE_NANOS);
-                       this.state.put(id, ret);
-               }
-
-               this.nodes.put(id, sessionListener);
-               ret.taken();
-               return ret;
-       }
-
-       @Override
-       public PCEPSessionListener getSessionListener() {
-               return new Stateful07TopologySessionListener(this);
-       }
-
-       @Override
-       public synchronized ListenableFuture<OperationResult> addLsp(final AddLspArgs input) {
-               // Get the listener corresponding to the node
-               final TopologySessionListener l = this.nodes.get(input.getNode());
-               if (l == null) {
-                       LOG.debug("Session for node {} not found", input.getNode());
-                       return OperationResults.UNSENT.future();
-               }
-
-               return l.addLsp(input);
-       }
-
-       @Override
-       public synchronized ListenableFuture<OperationResult> removeLsp(final RemoveLspArgs input) {
-               // Get the listener corresponding to the node
-               final TopologySessionListener l = this.nodes.get(input.getNode());
-               if (l == null) {
-                       LOG.debug("Session for node {} not found", input.getNode());
-                       return OperationResults.UNSENT.future();
-               }
-
-               return l.removeLsp(input);
-       }
-
-       @Override
-       public synchronized ListenableFuture<OperationResult> updateLsp(final UpdateLspArgs input) {
-               // Get the listener corresponding to the node
-               final TopologySessionListener l = this.nodes.get(input.getNode());
-               if (l == null) {
-                       LOG.debug("Session for node {} not found", input.getNode());
-                       return OperationResults.UNSENT.future();
-               }
-
-               return l.updateLsp(input);
-       }
-
-       @Override
-       public synchronized ListenableFuture<OperationResult> ensureLspOperational(final EnsureLspOperationalInput input) {
-               // Get the listener corresponding to the node
-               final TopologySessionListener l = this.nodes.get(input.getNode());
-               if (l == null) {
-                       LOG.debug("Session for node {} not found", input.getNode());
-                       return OperationResults.UNSENT.future();
-               }
-
-               return l.ensureLspOperational(input);
-       }
-
-       InstanceIdentifier<Topology> getTopology() {
-               return topology;
-       }
-
-       DataModificationTransaction beginTransaction() {
-               return dataProvider.beginTransaction();
-       }
-
-       @SuppressWarnings("unchecked")
-       <T extends DataObject> T readOperationalData(final InstanceIdentifier<T> id) {
-               return (T)dataProvider.readOperationalData(id);
-       }
-
-       @Override
-       public void close() throws InterruptedException, ExecutionException {
-               final DataModificationTransaction t = this.dataProvider.beginTransaction();
-               t.removeOperationalData(this.topology);
-               t.commit().get();
-       }
+final class ServerSessionManager implements SessionListenerFactory<PCEPSessionListener>, AutoCloseable, TopologySessionRPCs, PCEPTopologyProviderRuntimeMXBean {
+    private static final Logger LOG = LoggerFactory.getLogger(ServerSessionManager.class);
+    private static final long DEFAULT_HOLD_STATE_NANOS = TimeUnit.MINUTES.toNanos(5);
+
+    private final Map<NodeId, TopologySessionListener> nodes = new HashMap<>();
+    private final Map<NodeId, TopologyNodeState> state = new HashMap<>();
+    private final TopologySessionListenerFactory listenerFactory;
+    private final InstanceIdentifier<Topology> topology;
+    private final DataBroker broker;
+    private Optional<PCEPTopologyProviderRuntimeRegistration> runtimeRootRegistration = Optional.absent();
+
+    public ServerSessionManager(final DataBroker broker, final InstanceIdentifier<Topology> topology,
+            final TopologySessionListenerFactory listenerFactory) throws ReadFailedException, TransactionCommitFailedException {
+        this.broker = Preconditions.checkNotNull(broker);
+        this.topology = Preconditions.checkNotNull(topology);
+        this.listenerFactory = Preconditions.checkNotNull(listenerFactory);
+
+
+        // Make sure the topology does not exist
+        final ReadWriteTransaction tx = broker.newReadWriteTransaction();
+        final Optional<?> c = tx.read(LogicalDatastoreType.OPERATIONAL, topology).checkedGet();
+        Preconditions.checkArgument(!c.isPresent(), "Topology %s already exists", topology);
+
+        // create empty network-topology if not exists
+        tx.merge(LogicalDatastoreType.OPERATIONAL, InstanceIdentifier.builder(NetworkTopology.class).build(), new NetworkTopologyBuilder().build());
+        // Now create the base topology
+        final TopologyKey k = InstanceIdentifier.keyOf(topology);
+        tx.put(LogicalDatastoreType.OPERATIONAL, topology, new TopologyBuilder().setKey(k).setTopologyId(k.getTopologyId()).setTopologyTypes(
+                new TopologyTypesBuilder().addAugmentation(TopologyTypes1.class,
+                        new TopologyTypes1Builder().setTopologyPcep(new TopologyPcepBuilder().build()).build()).build()).setNode(
+                                new ArrayList<Node>()).build());
+
+        tx.submit().checkedGet();
+    }
+
+    private static NodeId createNodeId(final InetAddress addr) {
+        return new NodeId("pcc://" + addr.getHostAddress());
+    }
+
+    synchronized void releaseNodeState(final TopologyNodeState nodeState, final PCEPSession session) {
+        LOG.debug("Node {} unbound", nodeState.getNodeId());
+        this.nodes.remove(createNodeId(session.getRemoteAddress()));
+        nodeState.released();
+    }
+
+    synchronized TopologyNodeState takeNodeState(final InetAddress address, final TopologySessionListener sessionListener) {
+        final NodeId id = createNodeId(address);
+
+        LOG.debug("Node {} requested by listener {}", id, sessionListener);
+        TopologyNodeState ret = this.state.get(id);
+
+        if (ret == null) {
+            ret = new TopologyNodeState(this.broker, this.topology, id, DEFAULT_HOLD_STATE_NANOS);
+            LOG.debug("Created topology node {} for id {} at {}", ret, id, ret.getNodeId());
+            this.state.put(id, ret);
+        }
+        // FIXME: else check for conflicting session
+
+        ret.taken();
+        this.nodes.put(id, sessionListener);
+        LOG.debug("Node {} bound to listener {}", id, sessionListener);
+        return ret;
+    }
+
+    @Override
+    public PCEPSessionListener getSessionListener() {
+        return this.listenerFactory.createTopologySessionListener(this);
+    }
+
+    private synchronized TopologySessionListener checkSessionPresence(final NodeId nodeId) {
+        // Get the listener corresponding to the node
+        final TopologySessionListener l = this.nodes.get(nodeId);
+        if (l == null) {
+            LOG.debug("Session for node {} not found", nodeId);
+            return null;
+        }
+        return l;
+    }
+
+    @Override
+    public synchronized ListenableFuture<OperationResult> addLsp(final AddLspArgs input) {
+        final TopologySessionListener l = checkSessionPresence(input.getNode());
+        return (l != null) ? l.addLsp(input) : OperationResults.UNSENT.future();
+    }
+
+    @Override
+    public synchronized ListenableFuture<OperationResult> removeLsp(final RemoveLspArgs input) {
+        final TopologySessionListener l = checkSessionPresence(input.getNode());
+        return (l != null) ? l.removeLsp(input) : OperationResults.UNSENT.future();
+    }
+
+    @Override
+    public synchronized ListenableFuture<OperationResult> updateLsp(final UpdateLspArgs input) {
+        final TopologySessionListener l = checkSessionPresence(input.getNode());
+        return (l != null) ? l.updateLsp(input) : OperationResults.UNSENT.future();
+    }
+
+    @Override
+    public synchronized ListenableFuture<OperationResult> ensureLspOperational(final EnsureLspOperationalInput input) {
+        final TopologySessionListener l = checkSessionPresence(input.getNode());
+        return (l != null) ? l.ensureLspOperational(input) : OperationResults.UNSENT.future();
+    }
+
+    @Override
+    public void close() throws TransactionCommitFailedException {
+        if (this.runtimeRootRegistration.isPresent()) {
+            this.runtimeRootRegistration.get().close();
+        }
+        for (final TopologySessionListener sessionListener : this.nodes.values()) {
+            sessionListener.close();
+        }
+        for (final TopologyNodeState nodeState : this.state.values()) {
+            nodeState.close();
+        }
+        final WriteTransaction t = this.broker.newWriteOnlyTransaction();
+        t.delete(LogicalDatastoreType.OPERATIONAL, this.topology);
+        t.submit().checkedGet();
+    }
+
+    public void registerRuntimeRootRegistartion(final PCEPTopologyProviderRuntimeRegistrator runtimeRootRegistrator) {
+        this.runtimeRootRegistration = Optional.of(runtimeRootRegistrator.register(this));
+    }
+
+    public Optional<PCEPTopologyProviderRuntimeRegistration> getRuntimeRootRegistration() {
+        return this.runtimeRootRegistration;
+    }
 }