BUG-731: hide serverSessionManager
[bgpcep.git] / pcep / topology-provider / src / main / java / org / opendaylight / bgpcep / pcep / topology / provider / AbstractTopologySessionListener.java
index 264c59a1ca34130d4d00dc86f66f03105adc5025..c3cdc667a2fe9abc5dd2697eac8d8b67cf7ac277 100644 (file)
@@ -23,12 +23,9 @@ 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;
@@ -36,12 +33,16 @@ 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.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.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.DataObject;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier.InstanceIdentifierBuilder;
 import org.opendaylight.yangtools.yang.common.RpcResult;
@@ -49,12 +50,13 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 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 {
+public abstract class AbstractTopologySessionListener<SRPID, PLSPID> implements PCEPSessionListener, TopologySessionListener {
        protected static final MessageHeader MESSAGE_HEADER = new MessageHeader() {
                private final ProtocolVersion version = new ProtocolVersion((short) 1);
 
@@ -68,24 +70,20 @@ public abstract class AbstractTopologySessionListener<SRPID, PLSPID, PATHNAME> i
                        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 static final Logger LOG = LoggerFactory.getLogger(AbstractTopologySessionListener.class);
 
        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 final Map<String, ReportedLsp> lspData = new HashMap<>();
+       private final Map<PLSPID, String> lsps = new HashMap<>();
+       private final ServerSessionManager serverSessionManager;
        private InstanceIdentifier<Node> topologyNode;
        private InstanceIdentifier<Node1> topologyAugment;
        private PathComputationClientBuilder pccBuilder;
        private Node1Builder topologyAugmentBuilder;
-       private TopologyNodeState<PATHNAME> nodeState;
+       private TopologyNodeState nodeState;
        private boolean ownsTopology = false;
-       private boolean synced = false;
+       private boolean synced = false, dirty;
        private PCEPSession session;
 
        protected AbstractTopologySessionListener(final ServerSessionManager serverSessionManager) {
@@ -103,8 +101,7 @@ public abstract class AbstractTopologySessionListener<SRPID, PLSPID, PATHNAME> i
                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();
+                               this.topologyNode = this.serverSessionManager.getTopology().child(Node.class, n.getKey());
                                LOG.debug("Reusing topology node {} for id {} at {}", n, pccId, this.topologyNode);
                                return n;
                        }
@@ -116,7 +113,7 @@ public abstract class AbstractTopologySessionListener<SRPID, PLSPID, PATHNAME> i
                 */
                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 InstanceIdentifier<Node> nti = this.serverSessionManager.getTopology().child(Node.class, nk);
 
                final Node ret = new NodeBuilder().setKey(nk).setNodeId(id).build();
 
@@ -149,8 +146,8 @@ public abstract class AbstractTopologySessionListener<SRPID, PLSPID, PATHNAME> i
                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();
+               this.topologyAugment = this.topologyNode.augmentation(Node1.class);
+               final Node1 ta = this.topologyAugmentBuilder.build();
 
                trans.putOperationalData(this.topologyAugment, ta);
                LOG.debug("Peer data {} set to {}", this.topologyAugment, ta);
@@ -228,8 +225,54 @@ public abstract class AbstractTopologySessionListener<SRPID, PLSPID, PATHNAME> i
                tearDown(session);
        }
 
+       @Override
+       public final synchronized void onMessage(final PCEPSession session, final Message message) {
+               final DataModificationTransaction trans = this.serverSessionManager.beginTransaction();
+
+               dirty = false;
+
+               if (onMessage(trans, message)) {
+                       LOG.info("Unhandled message {} on session {}", message, session);
+                       return;
+               }
+
+               if (dirty) {
+                       LOG.debug("Internal state changed, forcing sync");
+                       this.pccBuilder.setReportedLsp(Lists.newArrayList(lspData.values()));
+                       this.topologyAugmentBuilder.setPathComputationClient(this.pccBuilder.build());
+                       final Node1 ta = this.topologyAugmentBuilder.build();
+
+                       trans.removeOperationalData(this.topologyAugment);
+                       trans.putOperationalData(this.topologyAugment, ta);
+                       LOG.debug("Peer data {} set to {}", this.topologyAugment, ta);
+                       dirty = false;
+               } else {
+                       LOG.debug("State has not changed, skipping sync");
+               }
+
+               Futures.addCallback(JdkFutureAdapters.listenInPoolThread(trans.commit()), 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 {}, closing it", session, t);
+                               session.close(TerminationReason.Unknown);
+                       }
+               });
+       }
+
+       @Override
+       public void close() {
+               if (this.session != null) {
+                       this.session.close(TerminationReason.Unknown);
+               }
+       }
+
        protected InstanceIdentifierBuilder<PathComputationClient> pccIdentifier() {
-               return InstanceIdentifier.builder(this.topologyAugment).child(PathComputationClient.class);
+               return this.topologyAugment.builder().child(PathComputationClient.class);
        }
 
        protected final synchronized PCEPRequest removeRequest(final SRPID id) {
@@ -264,6 +307,39 @@ public abstract class AbstractTopologySessionListener<SRPID, PLSPID, PATHNAME> i
                return req.getFuture();
        }
 
+       protected final synchronized void updateLsp(final DataModificationTransaction trans, final PLSPID id, final String lspName,
+                       final ReportedLspBuilder rlb, final boolean solicited) {
+
+               final String name;
+               if (lspName == null) {
+                       name = this.lsps.get(id);
+                       if (name == null) {
+                               LOG.error("PLSPID {} seen for the first time, not reporting the LSP", id);
+                               return;
+                       }
+               } else {
+                       name = lspName;
+               }
+
+               LOG.debug("Saved LSP {} with name {}", id, name);
+               this.lsps.put(id, name);
+
+               Preconditions.checkState(name != null);
+               rlb.setKey(new ReportedLspKey(name));
+               rlb.setName(name);
+
+               // If this is an unsolicited update. We need to make sure we retain the metadata already present
+               if (solicited) {
+                       this.nodeState.setLspMetadata(name, rlb.getMetadata());
+               } else {
+                       rlb.setMetadata(this.nodeState.getLspMetadata(name));
+               }
+
+               LOG.debug("LSP {} forcing update to MD-SAL", name);
+               dirty = true;
+               lspData .put(name, rlb.build());
+       }
+
        protected final synchronized void stateSynchronizationAchieved(final DataModificationTransaction trans) {
                if (this.synced) {
                        LOG.debug("State synchronization achieved while synchronized, not updating state");
@@ -272,36 +348,35 @@ public abstract class AbstractTopologySessionListener<SRPID, PLSPID, PATHNAME> i
 
                // 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);
+               this.pccBuilder.setStateSync(PccSyncState.Synchronized).build();
+               this.dirty = true;
 
                // The node has completed synchronization, cleanup metadata no longer reported back
                this.nodeState.cleanupExcept(this.lsps.values());
-               LOG.debug("Session {} achieved synchronized state", session);
+               LOG.debug("Session {} achieved synchronized state", this.session);
        }
 
-       protected final synchronized void addLsp(final PLSPID id, final PATHNAME name) {
-               Preconditions.checkState(lsps.containsKey(id) == false);
-               lsps.put(id, name);
+       protected final InstanceIdentifierBuilder<ReportedLsp> lspIdentifier(final String name) {
+               return pccIdentifier().child(ReportedLsp.class, new ReportedLspKey(name));
        }
 
-       protected final synchronized PATHNAME getLsp(final PLSPID id) {
-               return lsps.get(id);
+       protected final synchronized void removeLsp(final DataModificationTransaction trans, final PLSPID id) {
+               final String name = this.lsps.remove(id);
+               dirty = true;
+               LOG.debug("LSP {} removed", name);
+               lspData.remove(name);
        }
 
-       protected final synchronized PATHNAME removeLsp(final PLSPID id) {
-               return lsps.remove(id);
-       }
+       protected abstract void onSessionUp(PCEPSession session, PathComputationClientBuilder pccBuilder);
 
-       protected final synchronized Metadata getLspMetadata(final PATHNAME name) {
-               return this.nodeState.getLspMetadata(name);
-       }
+       protected abstract boolean onMessage(DataModificationTransaction trans, Message message);
 
-       protected final synchronized void updateLspMetadata(final PATHNAME name, final Metadata metadata) {
-               this.nodeState.setLspMetadata(name, metadata);
+       protected String lookupLspName(final PLSPID id) {
+               Preconditions.checkNotNull(id, "ID parameter null.");
+               return this.lsps.get(id);
        }
 
-       abstract protected void onSessionUp(PCEPSession session, PathComputationClientBuilder pccBuilder);
+       protected final <T extends DataObject> T readOperationalData(final InstanceIdentifier<T> id) {
+               return serverSessionManager.readOperationalData(id);
+       }
 }