Create model-agnostic AbstractTopologySessionListener 22/4622/2
authorRobert Varga <rovarga@cisco.com>
Thu, 23 Jan 2014 04:05:24 +0000 (05:05 +0100)
committerRobert Varga <rovarga@cisco.com>
Thu, 23 Jan 2014 14:14:17 +0000 (15:14 +0100)
This finishes the refactor for support of multiple stateful models: we
now have an Abstract class which each of the model-specific listeners
need to implement.

Change-Id: I074e8c2a593a5e34a652364daa3e4aa21390e516
Signed-off-by: Robert Varga <rovarga@cisco.com>
pcep/topology-provider/src/main/java/org/opendaylight/bgpcep/pcep/topology/provider/AbstractTopologySessionListener.java [new file with mode: 0644]
pcep/topology-provider/src/main/java/org/opendaylight/bgpcep/pcep/topology/provider/ServerSessionManager.java
pcep/topology-provider/src/main/java/org/opendaylight/bgpcep/pcep/topology/provider/Stateful07TopologySessionListener.java
pcep/topology-provider/src/main/java/org/opendaylight/bgpcep/pcep/topology/provider/TopologyNodeState.java

diff --git a/pcep/topology-provider/src/main/java/org/opendaylight/bgpcep/pcep/topology/provider/AbstractTopologySessionListener.java b/pcep/topology-provider/src/main/java/org/opendaylight/bgpcep/pcep/topology/provider/AbstractTopologySessionListener.java
new file mode 100644 (file)
index 0000000..264c59a
--- /dev/null
@@ -0,0 +1,307 @@
+/*
+ * Copyright (c) 2013 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.bgpcep.pcep.topology.provider;
+
+import io.netty.util.concurrent.FutureListener;
+
+import java.net.InetAddress;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import org.opendaylight.controller.md.sal.common.api.TransactionStatus;
+import org.opendaylight.controller.sal.binding.api.data.DataModificationTransaction;
+import org.opendaylight.protocol.pcep.PCEPSession;
+import org.opendaylight.protocol.pcep.PCEPSessionListener;
+import org.opendaylight.protocol.pcep.PCEPTerminationReason;
+import org.opendaylight.protocol.pcep.TerminationReason;
+import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.inet.types.rev100924.IpAddressBuilder;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.message.rev131007.Pcerr;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.message.rev131007.PcerrBuilder;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.types.rev131005.Message;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.types.rev131005.MessageHeader;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.types.rev131005.ProtocolVersion;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.types.rev131005.pcerr.message.PcerrMessageBuilder;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.Node1;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.Node1Builder;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.OperationResult;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.PccSyncState;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.lsp.metadata.Metadata;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.pcep.client.attributes.PathComputationClient;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.pcep.client.attributes.PathComputationClientBuilder;
+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.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.DataContainer;
+import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
+import org.opendaylight.yangtools.yang.binding.InstanceIdentifier.InstanceIdentifierBuilder;
+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;
+
+public abstract class AbstractTopologySessionListener<SRPID, PLSPID, PATHNAME> implements PCEPSessionListener, TopologySessionListener {
+       protected static final MessageHeader MESSAGE_HEADER = new MessageHeader() {
+               private final ProtocolVersion version = new ProtocolVersion((short) 1);
+
+               @Override
+               public Class<? extends DataContainer> getImplementedInterface() {
+                       return MessageHeader.class;
+               }
+
+               @Override
+               public ProtocolVersion getVersion() {
+                       return this.version;
+               }
+       };
+       protected static final Pcerr UNHANDLED_MESSAGE_ERROR = new PcerrBuilder().setPcerrMessage(
+                       new PcerrMessageBuilder().setErrorType(null).build()).build();
+
+       private static final Logger LOG = LoggerFactory.getLogger(Stateful07TopologySessionListener.class);
+
+       // FIXME: make this private
+       protected final ServerSessionManager serverSessionManager;
+
+       private final Map<SRPID, PCEPRequest> waitingRequests = new HashMap<>();
+       private final Map<SRPID, PCEPRequest> sendingRequests = new HashMap<>();
+       private final Map<PLSPID, PATHNAME> lsps = new HashMap<>();
+       private InstanceIdentifier<Node> topologyNode;
+       private InstanceIdentifier<Node1> topologyAugment;
+       private PathComputationClientBuilder pccBuilder;
+       private Node1Builder topologyAugmentBuilder;
+       private TopologyNodeState<PATHNAME> nodeState;
+       private boolean ownsTopology = false;
+       private boolean synced = false;
+       private PCEPSession session;
+
+       protected AbstractTopologySessionListener(final ServerSessionManager serverSessionManager) {
+               this.serverSessionManager = Preconditions.checkNotNull(serverSessionManager);
+       }
+
+       private static String createNodeId(final InetAddress addr) {
+               return "pcc://" + addr.getHostAddress();
+       }
+
+       private Node topologyNode(final DataModificationTransaction trans, final InetAddress address) {
+               final String pccId = createNodeId(address);
+               final Topology topo = (Topology) trans.readOperationalData(this.serverSessionManager.getTopology());
+
+               for (final Node n : topo.getNode()) {
+                       LOG.debug("Matching topology node {} to id {}", n, pccId);
+                       if (n.getNodeId().getValue().equals(pccId)) {
+                               this.topologyNode =
+                                               InstanceIdentifier.builder(this.serverSessionManager.getTopology()).child(Node.class, n.getKey()).toInstance();
+                               LOG.debug("Reusing topology node {} for id {} at {}", n, pccId, this.topologyNode);
+                               return n;
+                       }
+               }
+
+               /*
+                * We failed to find a matching node. Let's create a dynamic one
+                * and note that we are the owner (so we clean it up afterwards).
+                */
+               final NodeId id = new NodeId(pccId);
+               final NodeKey nk = new NodeKey(id);
+               final InstanceIdentifier<Node> nti = InstanceIdentifier.builder(this.serverSessionManager.getTopology()).child(Node.class, nk).toInstance();
+
+               final Node ret = new NodeBuilder().setKey(nk).setNodeId(id).build();
+
+               trans.putOperationalData(nti, ret);
+               LOG.debug("Created topology node {} for id {} at {}", ret, pccId, nti);
+               this.ownsTopology = true;
+               this.topologyNode = nti;
+               return ret;
+       }
+
+       @Override
+       public final synchronized void onSessionUp(final PCEPSession session) {
+               /*
+                * The session went up. Look up the router in Inventory model,
+                * create it if it is not there (marking that fact for later
+                * deletion), and mark it as synchronizing. Also create it in
+                * the topology model, with empty LSP list.
+                */
+               final InetAddress peerAddress = session.getRemoteAddress();
+               final DataModificationTransaction trans = this.serverSessionManager.beginTransaction();
+
+               final Node topoNode = topologyNode(trans, peerAddress);
+               LOG.debug("Peer {} resolved to topology node {}", peerAddress, topoNode);
+
+               // Our augmentation in the topology node
+               this.synced = false;
+               this.pccBuilder = new PathComputationClientBuilder();
+               this.pccBuilder.setIpAddress(IpAddressBuilder.getDefaultInstance(peerAddress.getHostAddress()));
+
+               onSessionUp(session, this.pccBuilder);
+
+               this.topologyAugmentBuilder = new Node1Builder().setPathComputationClient(this.pccBuilder.build());
+               this.topologyAugment = InstanceIdentifier.builder(this.topologyNode).augmentation(Node1.class).toInstance();
+               final Node1 ta = topologyAugmentBuilder.build();
+
+               trans.putOperationalData(this.topologyAugment, ta);
+               LOG.debug("Peer data {} set to {}", this.topologyAugment, ta);
+
+               // All set, commit the modifications
+               final ListenableFuture<RpcResult<TransactionStatus>> f = JdkFutureAdapters.listenInPoolThread(trans.commit());
+               Futures.addCallback(f, new FutureCallback<RpcResult<TransactionStatus>>() {
+                       @Override
+                       public void onSuccess(final RpcResult<TransactionStatus> result) {
+                               LOG.trace("Internal state for session {} updated successfully", session);
+                       }
+
+                       @Override
+                       public void onFailure(final Throwable t) {
+                               LOG.error("Failed to update internal state for session {}, terminating it", session, t);
+                               session.close(TerminationReason.Unknown);
+                       }
+               });
+
+               this.nodeState = this.serverSessionManager.takeNodeState(topoNode.getNodeId(), this);
+               this.session = session;
+               LOG.info("Session with {} attached to topology node {}", session.getRemoteAddress(), topoNode.getNodeId());
+       }
+
+       @GuardedBy("this")
+       private void tearDown(final PCEPSession session) {
+               this.serverSessionManager.releaseNodeState(this.nodeState);
+               this.nodeState = null;
+               this.session = null;
+
+               final DataModificationTransaction trans = this.serverSessionManager.beginTransaction();
+
+               // The session went down. Undo all the Topology changes we have done.
+               trans.removeOperationalData(this.topologyAugment);
+               if (this.ownsTopology) {
+                       trans.removeOperationalData(this.topologyNode);
+               }
+
+               Futures.addCallback(JdkFutureAdapters.listenInPoolThread(trans.commit()), new FutureCallback<RpcResult<TransactionStatus>>() {
+                       @Override
+                       public void onSuccess(final RpcResult<TransactionStatus> result) {
+                               LOG.trace("Internal state for session {} cleaned up successfully", session);
+                       }
+
+                       @Override
+                       public void onFailure(final Throwable t) {
+                               LOG.error("Failed to cleanup internal state for session {}", session, t);
+                       }
+               });
+
+               // Clear all requests which have not been sent to the peer: they result in cancellation
+               for (final Entry<SRPID, PCEPRequest> e : this.sendingRequests.entrySet()) {
+                       LOG.debug("Request {} was not sent when session went down, cancelling the instruction", e.getKey());
+                       e.getValue().setResult(OperationResults.UNSENT);
+               }
+               this.sendingRequests.clear();
+
+               // CLear all requests which have not been acked by the peer: they result in failure
+               for (final Entry<SRPID, PCEPRequest> e : this.waitingRequests.entrySet()) {
+                       LOG.info("Request {} was incomplete when session went down, failing the instruction", e.getKey());
+                       e.getValue().setResult(OperationResults.NOACK);
+               }
+               this.waitingRequests.clear();
+       }
+
+       @Override
+       public final synchronized void onSessionDown(final PCEPSession session, final Exception e) {
+               LOG.warn("Session {} went down unexpectedly", e);
+               tearDown(session);
+       }
+
+       @Override
+       public final synchronized void onSessionTerminated(final PCEPSession session, final PCEPTerminationReason reason) {
+               LOG.info("Session {} terminated by peer with reason {}", session, reason);
+               tearDown(session);
+       }
+
+       protected InstanceIdentifierBuilder<PathComputationClient> pccIdentifier() {
+               return InstanceIdentifier.builder(this.topologyAugment).child(PathComputationClient.class);
+       }
+
+       protected final synchronized PCEPRequest removeRequest(final SRPID id) {
+               return this.waitingRequests.remove(id);
+       }
+
+       private synchronized void messageSendingComplete(final SRPID requestId, final io.netty.util.concurrent.Future<Void> future) {
+               final PCEPRequest req = this.sendingRequests.remove(requestId);
+
+               if (future.isSuccess()) {
+                       this.waitingRequests.put(requestId, req);
+               } else {
+                       LOG.info("Failed to send request {}, instruction cancelled", requestId, future.cause());
+                       req.setResult(OperationResults.UNSENT);
+               }
+       }
+
+       protected final synchronized ListenableFuture<OperationResult> sendMessage(final Message message, final SRPID requestId,
+                       final Metadata metadata) {
+               final io.netty.util.concurrent.Future<Void> f = this.session.sendMessage(message);
+               final PCEPRequest req = new PCEPRequest(metadata);
+
+               this.sendingRequests.put(requestId, req);
+
+               f.addListener(new FutureListener<Void>() {
+                       @Override
+                       public void operationComplete(final io.netty.util.concurrent.Future<Void> future) {
+                               messageSendingComplete(requestId, future);
+                       }
+               });
+
+               return req.getFuture();
+       }
+
+       protected final synchronized void stateSynchronizationAchieved(final DataModificationTransaction trans) {
+               if (this.synced) {
+                       LOG.debug("State synchronization achieved while synchronized, not updating state");
+                       return;
+               }
+
+               // Update synchronization flag
+               this.synced = true;
+               this.topologyAugmentBuilder.setPathComputationClient(this.pccBuilder.setStateSync(PccSyncState.Synchronized).build());
+               final Node1 ta = this.topologyAugmentBuilder.build();
+               trans.putOperationalData(this.topologyAugment, ta);
+               LOG.debug("Peer data {} set to {}", this.topologyAugment, ta);
+
+               // The node has completed synchronization, cleanup metadata no longer reported back
+               this.nodeState.cleanupExcept(this.lsps.values());
+               LOG.debug("Session {} achieved synchronized state", session);
+       }
+
+       protected final synchronized void addLsp(final PLSPID id, final PATHNAME name) {
+               Preconditions.checkState(lsps.containsKey(id) == false);
+               lsps.put(id, name);
+       }
+
+       protected final synchronized PATHNAME getLsp(final PLSPID id) {
+               return lsps.get(id);
+       }
+
+       protected final synchronized PATHNAME removeLsp(final PLSPID id) {
+               return lsps.remove(id);
+       }
+
+       protected final synchronized Metadata getLspMetadata(final PATHNAME name) {
+               return this.nodeState.getLspMetadata(name);
+       }
+
+       protected final synchronized void updateLspMetadata(final PATHNAME name, final Metadata metadata) {
+               this.nodeState.setLspMetadata(name, metadata);
+       }
+
+       abstract protected void onSessionUp(PCEPSession session, PathComputationClientBuilder pccBuilder);
+}
index 7413f436807e4b93840465dbca5b60464b0751fd..2f33bf9a9427fdbea1522027d28a99067dfc3758 100644 (file)
@@ -52,7 +52,7 @@ final class ServerSessionManager implements SessionListenerFactory<PCEPSessionLi
        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 Map<NodeId, TopologyNodeState<?>> state = new HashMap<>();
        private final InstanceIdentifier<Topology> topology;
        private final DataProviderService dataProvider;
 
@@ -87,18 +87,19 @@ final class ServerSessionManager implements SessionListenerFactory<PCEPSessionLi
                });
        }
 
-       public void releaseNodeState(final TopologyNodeState nodeState) {
+       public void releaseNodeState(final TopologyNodeState<?> nodeState) {
                LOG.debug("Node {} unbound", nodeState.getNodeId());
                this.nodes.remove(nodeState.getNodeId());
                nodeState.released();
        }
 
-       synchronized TopologyNodeState takeNodeState(final NodeId id, final TopologySessionListener sessionListener) {
+       synchronized <T> TopologyNodeState<T> takeNodeState(final NodeId id, final TopologySessionListener sessionListener) {
                LOG.debug("Node {} bound to listener {}", id, sessionListener);
 
-               TopologyNodeState ret = this.state.get(id);
+               @SuppressWarnings("unchecked")
+               TopologyNodeState<T> ret = (TopologyNodeState<T>) this.state.get(id);
                if (ret == null) {
-                       ret = new TopologyNodeState(id, DEFAULT_HOLD_STATE_NANOS);
+                       ret = new TopologyNodeState<T>(id, DEFAULT_HOLD_STATE_NANOS);
                        this.state.put(id, ret);
                }
 
index 5dc8466eb66d39452daf02345f5ca8386571f380..685b2380d0cd4dd6bb7985a4ae716b9434b080f8 100644 (file)
@@ -7,13 +7,8 @@
  */
 package org.opendaylight.bgpcep.pcep.topology.provider;
 
-import io.netty.util.concurrent.FutureListener;
-
 import java.net.InetAddress;
 import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
 
 import javax.annotation.concurrent.GuardedBy;
 
@@ -21,9 +16,7 @@ import org.opendaylight.controller.md.sal.common.api.TransactionStatus;
 import org.opendaylight.controller.sal.binding.api.data.DataModificationTransaction;
 import org.opendaylight.protocol.pcep.PCEPSession;
 import org.opendaylight.protocol.pcep.PCEPSessionListener;
-import org.opendaylight.protocol.pcep.PCEPTerminationReason;
 import org.opendaylight.protocol.pcep.TerminationReason;
-import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.inet.types.rev100924.IpAddressBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.crabbe.initiated.rev131126.PcinitiateBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.crabbe.initiated.rev131126.pcinitiate.message.PcinitiateMessageBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.crabbe.initiated.rev131126.pcinitiate.message.pcinitiate.message.RequestsBuilder;
@@ -31,6 +24,7 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.iet
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.PcupdBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.PlspId;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.SrpIdNumber;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.SymbolicPathName;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.Tlvs2;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.lsp.object.Lsp;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.lsp.object.LspBuilder;
@@ -42,242 +36,67 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.iet
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.srp.object.Srp;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.srp.object.SrpBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.stateful.capability.tlv.Stateful;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.symbolic.path.name.tlv.SymbolicPathName;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.symbolic.path.name.tlv.SymbolicPathNameBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.message.rev131007.Pcerr;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.message.rev131007.PcerrBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.types.rev131005.Message;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.types.rev131005.MessageHeader;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.types.rev131005.ProtocolVersion;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.types.rev131005.open.object.open.Tlvs;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.types.rev131005.pcerr.message.PcerrMessageBuilder;
 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;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.Node1;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.Node1Builder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.OperationResult;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.PccSyncState;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.RemoveLspArgs;
 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.lsp.metadata.Metadata;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.pcep.client.attributes.PathComputationClient;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.pcep.client.attributes.PathComputationClientBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.pcep.client.attributes.path.computation.client.ReportedLsp;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.pcep.client.attributes.path.computation.client.ReportedLspBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.pcep.client.attributes.path.computation.client.ReportedLspKey;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.pcep.client.attributes.path.computation.client.StatefulTlvBuilder;
-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.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.DataContainer;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier.InstanceIdentifierBuilder;
 import org.opendaylight.yangtools.yang.common.RpcResult;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Collections2;
 import com.google.common.collect.ImmutableList;
 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 Stateful07TopologySessionListener implements PCEPSessionListener, TopologySessionListener {
+final class Stateful07TopologySessionListener extends AbstractTopologySessionListener<SrpIdNumber, PlspId, SymbolicPathName> implements PCEPSessionListener {
        private static final Logger LOG = LoggerFactory.getLogger(Stateful07TopologySessionListener.class);
-       private static final MessageHeader MESSAGE_HEADER = new MessageHeader() {
-               private final ProtocolVersion version = new ProtocolVersion((short) 1);
-
-               @Override
-               public Class<? extends DataContainer> getImplementedInterface() {
-                       return MessageHeader.class;
-               }
-
-               @Override
-               public ProtocolVersion getVersion() {
-                       return this.version;
-               }
-       };
-       private static final Pcerr UNHANDLED_MESSAGE_ERROR = new PcerrBuilder().setPcerrMessage(
-                       new PcerrMessageBuilder().setErrorType(null).build()).build();
-       private final ServerSessionManager serverSessionManager;
 
        /**
         * @param serverSessionManager
         */
        Stateful07TopologySessionListener(final ServerSessionManager serverSessionManager) {
-               this.serverSessionManager = Preconditions.checkNotNull(serverSessionManager);
+               super(serverSessionManager);
        }
 
-       private final Map<SrpIdNumber, PCEPRequest> waitingRequests = new HashMap<>();
-       private final Map<SrpIdNumber, PCEPRequest> sendingRequests = new HashMap<>();
-       private final Map<PlspId, SymbolicPathName> lsps = new HashMap<>();
-       private PathComputationClientBuilder pccBuilder;
-       private InstanceIdentifier<Node1> topologyAugment;
-       private InstanceIdentifier<Node> topologyNode;
-       private Node1Builder topologyAugmentBuilder;
-       private boolean ownsTopology = false;
-       private boolean synced = false;
-       private PCEPSession session;
-
        @GuardedBy("this")
        private long requestId = 1;
-       private TopologyNodeState nodeState;
-
-       private static String createNodeId(final InetAddress addr) {
-               return "pcc://" + addr.getHostAddress();
-       }
-
-       private Node topologyNode(final DataModificationTransaction trans, final InetAddress address) {
-               final String pccId = createNodeId(address);
-               final Topology topo = (Topology) trans.readOperationalData(this.serverSessionManager.getTopology());
-
-               for (final Node n : topo.getNode()) {
-                       LOG.debug("Matching topology node {} to id {}", n, pccId);
-                       if (n.getNodeId().getValue().equals(pccId)) {
-                               this.topologyNode =
-                                               InstanceIdentifier.builder(this.serverSessionManager.getTopology()).child(Node.class, n.getKey()).toInstance();
-                               LOG.debug("Reusing topology node {} for id {} at {}", n, pccId, this.topologyNode);
-                               return n;
-                       }
-               }
-
-               /*
-                * We failed to find a matching node. Let's create a dynamic one
-                * and note that we are the owner (so we clean it up afterwards).
-                */
-               final NodeId id = new NodeId(pccId);
-               final NodeKey nk = new NodeKey(id);
-               final InstanceIdentifier<Node> nti = InstanceIdentifier.builder(this.serverSessionManager.getTopology()).child(Node.class, nk).toInstance();
-
-               final Node ret = new NodeBuilder().setKey(nk).setNodeId(id).build();
-
-               trans.putOperationalData(nti, ret);
-               LOG.debug("Created topology node {} for id {} at {}", ret, pccId, nti);
-               this.ownsTopology = true;
-               this.topologyNode = nti;
-               return ret;
-       }
 
        @Override
-       public synchronized void onSessionUp(final PCEPSession session) {
-               /*
-                * The session went up. Look up the router in Inventory model,
-                * create it if it is not there (marking that fact for later
-                * deletion), and mark it as synchronizing. Also create it in
-                * the topology model, with empty LSP list.
-                */
+       protected void onSessionUp(final PCEPSession session, final PathComputationClientBuilder pccBuilder) {
                final InetAddress peerAddress = session.getRemoteAddress();
-               final DataModificationTransaction trans = this.serverSessionManager.beginTransaction();
-
-               final Node topoNode = topologyNode(trans, peerAddress);
-               LOG.debug("Peer {} resolved to topology node {}", peerAddress, topoNode);
-
-               // Our augmentation in the topology node
-               this.synced = false;
-               this.pccBuilder = new PathComputationClientBuilder();
-               this.pccBuilder.setIpAddress(IpAddressBuilder.getDefaultInstance(peerAddress.getHostAddress()));
 
                final Tlvs tlvs = session.getRemoteTlvs();
                if (tlvs.getAugmentation(Tlvs2.class) != null) {
                        final Stateful stateful = tlvs.getAugmentation(Tlvs2.class).getStateful();
                        if (stateful != null) {
-                               this.pccBuilder.setReportedLsp(Collections.<ReportedLsp> emptyList());
-                               this.pccBuilder.setStatefulTlv(new StatefulTlvBuilder().setStateful(stateful).build());
-                               this.pccBuilder.setStateSync(PccSyncState.InitialResync);
+                               pccBuilder.setReportedLsp(Collections.<ReportedLsp> emptyList());
+                               pccBuilder.setStatefulTlv(new StatefulTlvBuilder().setStateful(stateful).build());
+                               pccBuilder.setStateSync(PccSyncState.InitialResync);
                        } else {
                                LOG.debug("Peer {} does not advertise stateful TLV", peerAddress);
                        }
                } else {
                        LOG.debug("Peer {} does not advertise stateful TLV", peerAddress);
                }
-
-               this.topologyAugmentBuilder = new Node1Builder().setPathComputationClient(this.pccBuilder.build());
-               this.topologyAugment = InstanceIdentifier.builder(this.topologyNode).augmentation(Node1.class).toInstance();
-               final Node1 ta = topologyAugmentBuilder.build();
-
-               trans.putOperationalData(this.topologyAugment, ta);
-               LOG.debug("Peer data {} set to {}", this.topologyAugment, ta);
-
-               // All set, commit the modifications
-               final ListenableFuture<RpcResult<TransactionStatus>> f = JdkFutureAdapters.listenInPoolThread(trans.commit());
-               Futures.addCallback(f, new FutureCallback<RpcResult<TransactionStatus>>() {
-                       @Override
-                       public void onSuccess(final RpcResult<TransactionStatus> result) {
-                               LOG.trace("Internal state for session {} updated successfully", session);
-                       }
-
-                       @Override
-                       public void onFailure(final Throwable t) {
-                               LOG.error("Failed to update internal state for session {}, terminating it", session, t);
-                               session.close(TerminationReason.Unknown);
-                       }
-               });
-
-               this.nodeState = this.serverSessionManager.takeNodeState(topoNode.getNodeId(), this);
-               this.session = session;
-               LOG.info("Session with {} attached to topology node {}", session.getRemoteAddress(), topoNode.getNodeId());
-       }
-
-       @GuardedBy("this")
-       private void tearDown(final PCEPSession session) {
-               this.serverSessionManager.releaseNodeState(this.nodeState);
-               this.nodeState = null;
-               this.session = null;
-
-               final DataModificationTransaction trans = this.serverSessionManager.beginTransaction();
-
-               // The session went down. Undo all the Topology changes we have done.
-               trans.removeOperationalData(this.topologyAugment);
-               if (this.ownsTopology) {
-                       trans.removeOperationalData(this.topologyNode);
-               }
-
-               Futures.addCallback(JdkFutureAdapters.listenInPoolThread(trans.commit()), new FutureCallback<RpcResult<TransactionStatus>>() {
-                       @Override
-                       public void onSuccess(final RpcResult<TransactionStatus> result) {
-                               LOG.trace("Internal state for session {} cleaned up successfully", session);
-                       }
-
-                       @Override
-                       public void onFailure(final Throwable t) {
-                               LOG.error("Failed to cleanup internal state for session {}", session, t);
-                       }
-               });
-
-               // Clear all requests which have not been sent to the peer: they result in cancellation
-               for (final Entry<SrpIdNumber, PCEPRequest> e : this.sendingRequests.entrySet()) {
-                       LOG.debug("Request {} was not sent when session went down, cancelling the instruction", e.getKey());
-                       e.getValue().setResult(OperationResults.UNSENT);
-               }
-               this.sendingRequests.clear();
-
-               // CLear all requests which have not been acked by the peer: they result in failure
-               for (final Entry<SrpIdNumber, PCEPRequest> e : this.waitingRequests.entrySet()) {
-                       LOG.info("Request {} was incomplete when session went down, failing the instruction", e.getKey());
-                       e.getValue().setResult(OperationResults.NOACK);
-               }
-               this.waitingRequests.clear();
-       }
-
-       @Override
-       public synchronized void onSessionDown(final PCEPSession session, final Exception e) {
-               LOG.warn("Session {} went down unexpectedly", e);
-               tearDown(session);
-       }
-
-       @Override
-       public synchronized void onSessionTerminated(final PCEPSession session, final PCEPTerminationReason reason) {
-               LOG.info("Session {} terminated by peer with reason {}", session, reason);
-               tearDown(session);
        }
 
        private InstanceIdentifierBuilder<ReportedLsp> lspIdentifier(final ReportedLspKey key) {
-               return InstanceIdentifier.builder(this.topologyAugment).child(PathComputationClient.class).child(ReportedLsp.class, key);
+               return pccIdentifier().child(ReportedLsp.class, key);
        }
 
        @Override
@@ -294,25 +113,8 @@ final class Stateful07TopologySessionListener implements PCEPSessionListener, To
                for (final Reports r : rpt.getReports()) {
                        final Lsp lsp = r.getLsp();
 
-                       if (!lsp.isSync() && !this.synced) {
-                               // Update synchronization flag
-                               this.synced = true;
-                               this.topologyAugmentBuilder.setPathComputationClient(this.pccBuilder.setStateSync(PccSyncState.Synchronized).build());
-                               final Node1 ta = this.topologyAugmentBuilder.build();
-                               trans.putOperationalData(this.topologyAugment, ta);
-                               LOG.debug("Peer data {} set to {}", this.topologyAugment, ta);
-
-                               // The node has completed synchronization, cleanup metadata no longer reported back
-                               this.nodeState.cleanupExcept(Collections2.transform(
-                                               this.lsps.values(),
-                                               new Function<SymbolicPathName, org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.SymbolicPathName>() {
-                                                       @Override
-                                                       public org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.SymbolicPathName apply(
-                                                                       final SymbolicPathName input) {
-                                                               return input.getPathName();
-                                                       }
-                                               }));
-                               LOG.debug("Session {} achieved synchronized state", session);
+                       if (!lsp.isSync()) {
+                               stateSynchronizationAchieved(trans);
                                continue;
                        }
 
@@ -329,7 +131,7 @@ final class Stateful07TopologySessionListener implements PCEPSessionListener, To
                                        case Active:
                                        case Down:
                                        case Up:
-                                               final PCEPRequest req = this.waitingRequests.remove(id);
+                                               final PCEPRequest req = removeRequest(id);
                                                if (req != null) {
                                                        LOG.debug("Request {} resulted in LSP operational state {}", id, lsp.getOperational());
                                                        rlb.setMetadata(req.getMetadata());
@@ -349,33 +151,35 @@ final class Stateful07TopologySessionListener implements PCEPSessionListener, To
 
                        final PlspId id = lsp.getPlspId();
                        if (lsp.isRemove()) {
-                               final SymbolicPathName name = this.lsps.remove(id);
+                               final SymbolicPathName name = removeLsp(id);
                                if (name != null) {
-                                       this.nodeState.removeLspMetadata(name.getPathName());
-                                       trans.removeOperationalData(lspIdentifier(new ReportedLspKey(name.getPathName())).build());
+                                       trans.removeOperationalData(lspIdentifier(new ReportedLspKey(name)).build());
                                }
 
                                LOG.debug("LSP {} removed", lsp);
                        } else {
-                               if (!this.lsps.containsKey(id)) {
+                               SymbolicPathName name = getLsp(id);
+                               if (name == null) {
                                        LOG.debug("PLSPID {} not known yet, looking for a symbolic name", id);
 
                                        final org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.lsp.object.lsp.Tlvs tlvs = r.getLsp().getTlvs();
                                        if (tlvs != null && tlvs.getSymbolicPathName() != null) {
-                                               this.lsps.put(id, tlvs.getSymbolicPathName());
+                                               name = tlvs.getSymbolicPathName().getPathName();
+                                               addLsp(id, name);
                                        } else {
                                                LOG.error("PLSPID {} seen for the first time, not reporting the LSP", id);
                                                continue;
                                        }
                                }
 
-                               rlb.setKey(new ReportedLspKey(this.lsps.get(id).getPathName()));
+                               Preconditions.checkState(name != null);
+                               rlb.setKey(new ReportedLspKey(name));
 
                                // If this is an unsolicited update. We need to make sure we retain the metadata already present
                                if (solicited) {
-                                       this.nodeState.setLspMetadata(rlb.getName(), rlb.getMetadata());
+                                       updateLspMetadata(name, rlb.getMetadata());
                                } else {
-                                       rlb.setMetadata(this.nodeState.getLspMetadata(rlb.getName()));
+                                       rlb.setMetadata(getLspMetadata(name));
                                }
 
                                trans.putOperationalData(lspIdentifier(rlb.getKey()).build(), rlb.build());
@@ -403,39 +207,10 @@ final class Stateful07TopologySessionListener implements PCEPSessionListener, To
                return new SrpIdNumber(this.requestId++);
        }
 
-       private synchronized void messageSendingComplete(final SrpIdNumber requestId, final io.netty.util.concurrent.Future<Void> future) {
-               final PCEPRequest req = this.sendingRequests.remove(requestId);
-
-               if (future.isSuccess()) {
-                       this.waitingRequests.put(requestId, req);
-               } else {
-                       LOG.info("Failed to send request {}, instruction cancelled", requestId, future.cause());
-                       req.setResult(OperationResults.UNSENT);
-               }
-       }
-
-       synchronized ListenableFuture<OperationResult> sendMessage(final Message message, final SrpIdNumber requestId,
-                       final Metadata metadata) {
-               final io.netty.util.concurrent.Future<Void> f = this.session.sendMessage(message);
-               final PCEPRequest req = new PCEPRequest(metadata);
-
-               this.sendingRequests.put(requestId, req);
-
-               f.addListener(new FutureListener<Void>() {
-                       @Override
-                       public void operationComplete(final io.netty.util.concurrent.Future<Void> future) {
-                               messageSendingComplete(requestId, future);
-                       }
-               });
-
-               return req.getFuture();
-       }
-
        @Override
        public synchronized ListenableFuture<OperationResult> addLsp(final AddLspArgs input) {
                // Make sure there is no such LSP
-               final InstanceIdentifier<ReportedLsp> lsp = InstanceIdentifier.builder(topologyAugment).child(PathComputationClient.class).child(
-                               ReportedLsp.class, new ReportedLspKey(input.getName())).toInstance();
+               final InstanceIdentifier<ReportedLsp> lsp = lspIdentifier(new ReportedLspKey(input.getName())).build();
                if (serverSessionManager.readOperationalData(lsp) != null) {
                        LOG.debug("Node {} already contains lsp {} at {}", input.getNode(), input.getName(), lsp);
                        return OperationResults.UNSENT.future();
@@ -459,8 +234,7 @@ final class Stateful07TopologySessionListener implements PCEPSessionListener, To
        @Override
        public synchronized ListenableFuture<OperationResult> removeLsp(final RemoveLspArgs input) {
                // Make sure the LSP exists, we need it for PLSP-ID
-               final InstanceIdentifier<ReportedLsp> lsp = InstanceIdentifier.builder(topologyAugment).child(PathComputationClient.class).child(
-                               ReportedLsp.class, new ReportedLspKey(input.getName())).toInstance();
+               final InstanceIdentifier<ReportedLsp> lsp = lspIdentifier(new ReportedLspKey(input.getName())).build();
                final ReportedLsp rep = serverSessionManager.readOperationalData(lsp);
                if (rep == null) {
                        LOG.debug("Node {} does not contain LSP {}", input.getNode(), input.getName());
@@ -480,8 +254,7 @@ final class Stateful07TopologySessionListener implements PCEPSessionListener, To
        @Override
        public synchronized ListenableFuture<OperationResult> updateLsp(final UpdateLspArgs input) {
                // Make sure the LSP exists
-               final InstanceIdentifier<ReportedLsp> lsp = InstanceIdentifier.builder(topologyAugment).child(PathComputationClient.class).child(
-                               ReportedLsp.class, new ReportedLspKey(input.getName())).toInstance();
+               final InstanceIdentifier<ReportedLsp> lsp = lspIdentifier(new ReportedLspKey(input.getName())).build();
                final ReportedLsp rep = serverSessionManager.readOperationalData(lsp);
                if (rep == null) {
                        LOG.debug("Node {} does not contain LSP {}", input.getNode(), input.getName());
@@ -504,8 +277,7 @@ final class Stateful07TopologySessionListener implements PCEPSessionListener, To
        @Override
        public synchronized ListenableFuture<OperationResult> ensureLspOperational(final EnsureLspOperationalInput input) {
                // Make sure the LSP exists
-               final InstanceIdentifier<ReportedLsp> lsp = InstanceIdentifier.builder(topologyAugment).child(PathComputationClient.class).child(
-                               ReportedLsp.class, new ReportedLspKey(input.getName())).toInstance();
+               final InstanceIdentifier<ReportedLsp> lsp = lspIdentifier(new ReportedLspKey(input.getName())).build();
                LOG.debug("Checking if LSP {} has operational state {}", lsp, input.getArguments().getOperational());
                final ReportedLsp rep = serverSessionManager.readOperationalData(lsp);
                if (rep == null) {
index 6f01b317c62ad74413f0d5b74145a9641579d698..24d40b634310e68080f111c7e278d585def2935b 100644 (file)
@@ -14,15 +14,14 @@ import java.util.Map;
 
 import javax.annotation.concurrent.ThreadSafe;
 
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev131222.SymbolicPathName;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev131024.lsp.metadata.Metadata;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.NodeId;
 
 import com.google.common.base.Preconditions;
 
 @ThreadSafe
-final class TopologyNodeState {
-       private final Map<SymbolicPathName, Metadata> metadata = new HashMap<>();
+final class TopologyNodeState<PATHNAME> {
+       private final Map<PATHNAME, Metadata> metadata = new HashMap<>();
        private final long holdStateNanos;
        private final NodeId nodeId;
        private long lastReleased = 0;
@@ -37,11 +36,11 @@ final class TopologyNodeState {
                return nodeId;
        }
 
-       public synchronized Metadata getLspMetadata(final SymbolicPathName name) {
+       public synchronized Metadata getLspMetadata(final PATHNAME name) {
                return metadata.get(name);
        }
 
-       public synchronized void setLspMetadata(final SymbolicPathName name, final Metadata value) {
+       public synchronized void setLspMetadata(final PATHNAME name, final Metadata value) {
                if (value == null) {
                        metadata.remove(name);
                } else {
@@ -49,12 +48,12 @@ final class TopologyNodeState {
                }
        }
 
-       public synchronized void removeLspMetadata(final SymbolicPathName name) {
+       public synchronized void removeLspMetadata(final PATHNAME name) {
                metadata.remove(name);
        }
 
-       public synchronized void cleanupExcept(final Collection<SymbolicPathName> values) {
-               final Iterator<SymbolicPathName> it = metadata.keySet().iterator();
+       public synchronized void cleanupExcept(final Collection<PATHNAME> values) {
+               final Iterator<PATHNAME> it = metadata.keySet().iterator();
                while (it.hasNext()) {
                        if (!values.contains(it.next())) {
                                it.remove();