BGPCEP-704: Add restart session rpc
[bgpcep.git] / pcep / topology / topology-provider / src / main / java / org / opendaylight / bgpcep / pcep / topology / provider / AbstractTopologySessionListener.java
1 /*
2  * Copyright (c) 2013 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.bgpcep.pcep.topology.provider;
9
10 import static java.util.Objects.requireNonNull;
11
12 import com.google.common.base.Optional;
13 import com.google.common.base.Preconditions;
14 import com.google.common.util.concurrent.FutureCallback;
15 import com.google.common.util.concurrent.Futures;
16 import com.google.common.util.concurrent.ListenableFuture;
17 import com.google.common.util.concurrent.MoreExecutors;
18 import io.netty.util.concurrent.FutureListener;
19 import java.net.InetAddress;
20 import java.util.ArrayList;
21 import java.util.Collection;
22 import java.util.HashMap;
23 import java.util.List;
24 import java.util.Map;
25 import java.util.Map.Entry;
26 import java.util.Objects;
27 import java.util.Timer;
28 import java.util.TimerTask;
29 import java.util.concurrent.TimeUnit;
30 import javax.annotation.concurrent.GuardedBy;
31 import org.opendaylight.controller.config.yang.pcep.topology.provider.ListenerStateRuntimeMXBean;
32 import org.opendaylight.controller.config.yang.pcep.topology.provider.PeerCapabilities;
33 import org.opendaylight.controller.config.yang.pcep.topology.provider.ReplyTime;
34 import org.opendaylight.controller.config.yang.pcep.topology.provider.SessionState;
35 import org.opendaylight.controller.config.yang.pcep.topology.provider.StatefulMessages;
36 import org.opendaylight.controller.md.sal.binding.api.ReadWriteTransaction;
37 import org.opendaylight.controller.md.sal.binding.api.WriteTransaction;
38 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
39 import org.opendaylight.protocol.pcep.PCEPCloseTermination;
40 import org.opendaylight.protocol.pcep.PCEPSession;
41 import org.opendaylight.protocol.pcep.PCEPTerminationReason;
42 import org.opendaylight.protocol.pcep.TerminationReason;
43 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.inet.types.rev130715.IpAddressBuilder;
44 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev171025.LspObject;
45 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev171025.Path1;
46 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.ietf.stateful.rev171025.lsp.object.Lsp;
47 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.types.rev131005.Message;
48 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.types.rev131005.MessageHeader;
49 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.types.rev131005.Object;
50 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.types.rev131005.ProtocolVersion;
51 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev171025.LspId;
52 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev171025.Node1;
53 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev171025.Node1Builder;
54 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev171025.OperationResult;
55 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev171025.PccSyncState;
56 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev171025.TearDownSessionInput;
57 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev171025.lsp.metadata.Metadata;
58 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev171025.pcep.client.attributes.PathComputationClient;
59 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev171025.pcep.client.attributes.PathComputationClientBuilder;
60 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev171025.pcep.client.attributes.path.computation.client.ReportedLsp;
61 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev171025.pcep.client.attributes.path.computation.client.ReportedLspBuilder;
62 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev171025.pcep.client.attributes.path.computation.client.ReportedLspKey;
63 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev171025.pcep.client.attributes.path.computation.client.reported.lsp.Path;
64 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.Node;
65 import org.opendaylight.yangtools.yang.binding.DataContainer;
66 import org.opendaylight.yangtools.yang.binding.DataObject;
67 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
68 import org.slf4j.Logger;
69 import org.slf4j.LoggerFactory;
70
71 /**
72  * Base class for PCEP topology providers. It handles the common tasks involved in managing a PCEP server (PCE)
73  * endpoint, and exposing a network topology based on it. It needs to be subclassed to form a fully functional block,
74  * where the subclass provides handling of incoming messages.
75  *
76  * @param <S> identifier type of requests
77  * @param <L> identifier type for LSPs
78  */
79 public abstract class AbstractTopologySessionListener<S, L> implements TopologySessionListener, ListenerStateRuntimeMXBean {
80     protected static final MessageHeader MESSAGE_HEADER = new MessageHeader() {
81         private final ProtocolVersion version = new ProtocolVersion((short) 1);
82
83         @Override
84         public Class<? extends DataContainer> getImplementedInterface() {
85             return MessageHeader.class;
86         }
87
88         @Override
89         public ProtocolVersion getVersion() {
90             return this.version;
91         }
92     };
93     protected static final String MISSING_XML_TAG = "Mandatory XML tags are missing.";
94     private static final Logger LOG = LoggerFactory.getLogger(AbstractTopologySessionListener.class);
95     @GuardedBy("this")
96     protected final Map<L, String> lsps = new HashMap<>();
97     @GuardedBy("this")
98     private final Map<S, PCEPRequest> requests = new HashMap<>();
99
100     @GuardedBy("this")
101     private final Map<String, ReportedLsp> lspData = new HashMap<>();
102     private final ServerSessionManager serverSessionManager;
103     @GuardedBy("this")
104     private final SessionListenerState listenerState;
105     private InstanceIdentifier<PathComputationClient> pccIdentifier;
106     private TopologyNodeState nodeState;
107     private boolean synced = false;
108     private PCEPSession session;
109     private SyncOptimization syncOptimization;
110     private boolean triggeredResyncInProcess;
111
112     protected AbstractTopologySessionListener(final ServerSessionManager serverSessionManager) {
113         this.serverSessionManager = requireNonNull(serverSessionManager);
114         this.listenerState = new SessionListenerState();
115     }
116
117     @Override
118     public final synchronized void onSessionUp(final PCEPSession session) {
119         /*
120          * The session went up. Look up the router in Inventory model,
121          * create it if it is not there (marking that fact for later
122          * deletion), and mark it as synchronizing. Also create it in
123          * the topology model, with empty LSP list.
124          */
125         final InetAddress peerAddress = session.getRemoteAddress();
126
127         this.syncOptimization = new SyncOptimization(session);
128
129         final TopologyNodeState state = this.serverSessionManager.takeNodeState(peerAddress, this, isLspDbRetreived());
130         // takeNodeState(..) may fail when the server session manager is being restarted due to configuration change
131         if (state == null) {
132             LOG.error("Unable to fetch topology node state for PCEP session. Closing session {}", session);
133             session.close(TerminationReason.UNKNOWN);
134             this.onSessionTerminated(session, new PCEPCloseTermination(TerminationReason.UNKNOWN));
135             return;
136         }
137
138         if (this.session != null || this.nodeState != null) {
139             LOG.error("PCEP session is already up with {}. Closing session {}", session.getRemoteAddress(), session);
140             session.close(TerminationReason.UNKNOWN);
141             this.onSessionTerminated(session, new PCEPCloseTermination(TerminationReason.UNKNOWN));
142             return;
143         }
144
145         this.session = session;
146         this.nodeState = state;
147
148         LOG.trace("Peer {} resolved to topology node {}", peerAddress, state.getNodeId());
149
150         // Our augmentation in the topology node
151         final PathComputationClientBuilder pccBuilder = new PathComputationClientBuilder();
152
153         onSessionUp(session, pccBuilder);
154         this.synced = isSynchronized();
155
156         pccBuilder.setIpAddress(IpAddressBuilder.getDefaultInstance(peerAddress.getHostAddress()));
157         final InstanceIdentifier<Node1> topologyAugment = state.getNodeId().augmentation(Node1.class);
158         this.pccIdentifier = topologyAugment.child(PathComputationClient.class);
159         final Node initialNodeState = state.getInitialNodeState();
160         final boolean isNodePresent = isLspDbRetreived() && initialNodeState != null;
161         if (isNodePresent) {
162             loadLspData(initialNodeState, this.lspData, this.lsps, isIncrementalSynchro());
163             pccBuilder.setReportedLsp(initialNodeState.getAugmentation(Node1.class).getPathComputationClient().getReportedLsp());
164         }
165         writeNode(pccBuilder, state, topologyAugment);
166         this.listenerState.init(session);
167         LOG.info("Session with {} attached to topology node {}", session.getRemoteAddress(), state.getNodeId());
168     }
169
170     private void writeNode(final PathComputationClientBuilder pccBuilder, final TopologyNodeState state,
171             final InstanceIdentifier<Node1> topologyAugment) {
172         final Node1 ta = new Node1Builder().setPathComputationClient(pccBuilder.build()).build();
173
174         final ReadWriteTransaction trans = state.rwTransaction();
175         trans.put(LogicalDatastoreType.OPERATIONAL, topologyAugment, ta);
176         LOG.trace("Peer data {} set to {}", topologyAugment, ta);
177
178         // All set, commit the modifications
179         Futures.addCallback(trans.submit(), new FutureCallback<Void>() {
180             @Override
181             public void onSuccess(final Void result) {
182                 LOG.trace("Internal state for session {} updated successfully", AbstractTopologySessionListener.this.session);
183             }
184
185             @Override
186             public void onFailure(final Throwable t) {
187                 LOG.error("Failed to update internal state for session {}, terminating it", AbstractTopologySessionListener.this.session, t);
188                 AbstractTopologySessionListener.this.session.close(TerminationReason.UNKNOWN);
189             }
190         }, MoreExecutors.directExecutor());
191     }
192
193     protected void updatePccState(final PccSyncState pccSyncState) {
194         if (this.nodeState == null) {
195             LOG.info("Server Session Manager is closed.");
196             AbstractTopologySessionListener.this.session.close(TerminationReason.UNKNOWN);
197             return;
198         }
199         final MessageContext ctx = new MessageContext(this.nodeState.beginTransaction());
200         updatePccNode(ctx, new PathComputationClientBuilder().setStateSync(pccSyncState).build());
201         if (pccSyncState != PccSyncState.Synchronized) {
202             this.synced = false;
203             this.triggeredResyncInProcess = true;
204         }
205         // All set, commit the modifications
206         Futures.addCallback(ctx.trans.submit(), new FutureCallback<Void>() {
207             @Override
208             public void onSuccess(final Void result) {
209                 LOG.trace("Internal state for session {} updated successfully", AbstractTopologySessionListener.this.session);
210             }
211
212             @Override
213             public void onFailure(final Throwable t) {
214                 LOG.error("Failed to update internal state for session {}", AbstractTopologySessionListener.this.session, t);
215                 AbstractTopologySessionListener.this.session.close(TerminationReason.UNKNOWN);
216             }
217         }, MoreExecutors.directExecutor());
218     }
219
220     protected boolean isTriggeredSyncInProcess() {
221         return this.triggeredResyncInProcess;
222     }
223
224     /**
225      * Tear down the given PCEP session. It's OK to call this method even after the session
226      * is already down. It always clear up the current session status.
227      *
228      * @param session
229      */
230     @GuardedBy("this")
231     private synchronized void tearDown(final PCEPSession session) {
232         requireNonNull(session);
233         this.serverSessionManager.releaseNodeState(this.nodeState, session, isLspDbPersisted());
234         this.nodeState = null;
235         try {
236             if (this.session != null) {
237                 this.session.close();
238             }
239             session.close();
240         } catch (Exception e) {
241             LOG.error("Session {} cannot be closed.", session, e);
242         }
243         this.session = null;
244         this.syncOptimization = null;
245
246         // Clear all requests we know about
247         for (final Entry<S, PCEPRequest> e : this.requests.entrySet()) {
248             final PCEPRequest r = e.getValue();
249             switch (r.getState()) {
250                 case DONE:
251                     // Done is done, nothing to do
252                     LOG.trace("Request {} was done when session went down.", e.getKey());
253                     break;
254                 case UNACKED:
255                     // Peer has not acked: results in failure
256                     LOG.info("Request {} was incomplete when session went down, failing the instruction", e.getKey());
257                     r.done(OperationResults.NOACK);
258                     break;
259                 case UNSENT:
260                     // Peer has not been sent to the peer: results in cancellation
261                     LOG.debug("Request {} was not sent when session went down, cancelling the instruction", e.getKey());
262                     r.done(OperationResults.UNSENT);
263                     break;
264                 default:
265                     break;
266             }
267         }
268         this.requests.clear();
269     }
270
271     @Override
272     public final synchronized void onSessionDown(final PCEPSession session, final Exception e) {
273         LOG.warn("Session {} went down unexpectedly", session, e);
274         tearDown(session);
275     }
276
277     @Override
278     public final synchronized void onSessionTerminated(final PCEPSession session, final PCEPTerminationReason reason) {
279         LOG.info("Session {} terminated by peer with reason {}", session, reason);
280         tearDown(session);
281     }
282
283     @Override
284     public final synchronized void onMessage(final PCEPSession session, final Message message) {
285         if (this.nodeState == null) {
286             LOG.warn("Topology node state is null. Unhandled message {} on session {}", message, session);
287             session.close(TerminationReason.UNKNOWN);
288             return;
289         }
290         final MessageContext ctx = new MessageContext(this.nodeState.beginTransaction());
291
292         if (onMessage(ctx, message)) {
293             LOG.warn("Unhandled message {} on session {}", message, session);
294             //cancel not supported, submit empty transaction
295             ctx.trans.submit();
296             return;
297         }
298
299         Futures.addCallback(ctx.trans.submit(), new FutureCallback<Void>() {
300             @Override
301             public void onSuccess(final Void result) {
302                 LOG.trace("Internal state for session {} updated successfully", session);
303                 ctx.notifyRequests();
304             }
305
306             @Override
307             public void onFailure(final Throwable t) {
308                 LOG.error("Failed to update internal state for session {}, closing it", session, t);
309                 ctx.notifyRequests();
310                 session.close(TerminationReason.UNKNOWN);
311             }
312         }, MoreExecutors.directExecutor());
313     }
314
315     @Override
316     public void close() {
317         if (this.session != null) {
318             this.session.close(TerminationReason.UNKNOWN);
319         }
320     }
321
322     protected final synchronized PCEPRequest removeRequest(final S id) {
323         final PCEPRequest ret = this.requests.remove(id);
324         if (ret != null) {
325             this.listenerState.processRequestStats(ret.getElapsedMillis());
326         }
327         LOG.trace("Removed request {} object {}", id, ret);
328         return ret;
329     }
330
331     protected final synchronized ListenableFuture<OperationResult> sendMessage(final Message message, final S requestId,
332             final Metadata metadata) {
333         final io.netty.util.concurrent.Future<Void> f = this.session.sendMessage(message);
334         this.listenerState.updateStatefulSentMsg(message);
335         final PCEPRequest req = new PCEPRequest(metadata);
336         this.requests.put(requestId, req);
337         final short rpcTimeout = this.serverSessionManager.getRpcTimeout();
338         LOG.trace("RPC response timeout value is {} seconds", rpcTimeout);
339         if (rpcTimeout > 0) {
340             setupTimeoutHandler(requestId, req, rpcTimeout);
341         }
342
343         f.addListener((FutureListener<Void>) future -> {
344             if (!future.isSuccess()) {
345                 synchronized (AbstractTopologySessionListener.this) {
346                     AbstractTopologySessionListener.this.requests.remove(requestId);
347                 }
348                 req.done(OperationResults.UNSENT);
349                 LOG.info("Failed to send request {}, instruction cancelled", requestId, future.cause());
350             } else {
351                 req.sent();
352                 LOG.trace("Request {} sent to peer (object {})", requestId, req);
353             }
354         });
355
356         return req.getFuture();
357     }
358
359     private void setupTimeoutHandler(final S requestId, final PCEPRequest req, final short timeout) {
360         final Timer timer = req.getTimer();
361         timer.schedule(new TimerTask() {
362             @Override
363             public void run() {
364                 synchronized (AbstractTopologySessionListener.this) {
365                     AbstractTopologySessionListener.this.requests.remove(requestId);
366                 }
367                 req.done();
368                 LOG.info("Request {} timed-out waiting for response", requestId);
369             }
370         }, TimeUnit.SECONDS.toMillis(timeout));
371         LOG.trace("Set up response timeout handler for request {}", requestId);
372     }
373
374     /**
375      * Update an LSP in the data store
376      *
377      * @param ctx       Message context
378      * @param id        Revision-specific LSP identifier
379      * @param lspName   LSP name
380      * @param rlb       Reported LSP builder
381      * @param solicited True if the update was solicited
382      * @param remove    True if this is an LSP path removal
383      */
384     protected final synchronized void updateLsp(final MessageContext ctx, final L id, final String lspName,
385             final ReportedLspBuilder rlb, final boolean solicited, final boolean remove) {
386
387         final String name;
388         if (lspName == null) {
389             name = this.lsps.get(id);
390             if (name == null) {
391                 LOG.error("PLSPID {} seen for the first time, not reporting the LSP", id);
392                 return;
393             }
394         } else {
395             name = lspName;
396         }
397
398         LOG.debug("Saved LSP {} with name {}", id, name);
399         this.lsps.put(id, name);
400
401
402         final ReportedLsp previous = this.lspData.get(name);
403         // if no previous report about the lsp exist, just proceed
404         if (previous != null) {
405             final List<Path> updatedPaths = makeBeforeBreak(rlb, previous, name, remove);
406             // if all paths or the last path were deleted, delete whole tunnel
407             if (updatedPaths == null || updatedPaths.isEmpty()) {
408                 LOG.debug("All paths were removed, removing LSP with {}.", id);
409                 removeLsp(ctx, id);
410                 return;
411             }
412             rlb.setPath(updatedPaths);
413         }
414         rlb.setKey(new ReportedLspKey(name));
415         rlb.setName(name);
416
417         // If this is an unsolicited update. We need to make sure we retain the metadata already present
418         if (solicited) {
419             this.nodeState.setLspMetadata(name, rlb.getMetadata());
420         } else {
421             rlb.setMetadata(this.nodeState.getLspMetadata(name));
422         }
423
424         final ReportedLsp rl = rlb.build();
425         ctx.trans.put(LogicalDatastoreType.OPERATIONAL, this.pccIdentifier.child(ReportedLsp.class, rlb.getKey()), rl);
426         LOG.debug("LSP {} updated to MD-SAL", name);
427
428         this.lspData.put(name, rl);
429     }
430
431     private List<Path> makeBeforeBreak(final ReportedLspBuilder rlb, final ReportedLsp previous, final String name, final boolean remove) {
432         // just one path should be reported
433         Preconditions.checkState(rlb.getPath().size() == 1);
434         final org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.rsvp.rev150820.LspId reportedLspId = rlb.getPath().get(0).getLspId();
435         final List<Path> updatedPaths;
436         //lspId = 0 and remove = false -> tunnel is down, still exists but no path is signaled
437         //remove existing tunnel's paths now, as explicit path remove will not come
438         if (!remove && reportedLspId.getValue() == 0) {
439             updatedPaths = new ArrayList<>();
440             LOG.debug("Remove previous paths {} to this lsp name {}", previous.getPath(), name);
441         } else {
442             // check previous report for existing paths
443             updatedPaths = new ArrayList<>(previous.getPath());
444             LOG.debug("Found previous paths {} to this lsp name {}", updatedPaths, name);
445             for (final Path path : previous.getPath()) {
446                 //we found reported path in previous reports
447                 if (path.getLspId().getValue() == 0 || path.getLspId().equals(reportedLspId)) {
448                     LOG.debug("Match on lsp-id {}", path.getLspId().getValue());
449                     // path that was reported previously and does have the same lsp-id, path will be updated
450                     final boolean r = updatedPaths.remove(path);
451                     LOG.trace("Request removed? {}", r);
452                 }
453             }
454         }
455         // if the path does not exist in previous report, add it to path list, it's a new ERO
456         // only one path will be added
457         //lspId is 0 means confirmation message that shouldn't be added (because we have no means of deleting it later)
458         LOG.trace("Adding new path {} to {}", rlb.getPath(), updatedPaths);
459         updatedPaths.addAll(rlb.getPath());
460         if (remove) {
461             if (reportedLspId.getValue() == 0) {
462                 // if lsp-id also 0, remove all paths
463                 LOG.debug("Removing all paths.");
464                 updatedPaths.clear();
465             } else {
466                 // path is marked to be removed
467                 LOG.debug("Removing path {} from {}", rlb.getPath(), updatedPaths);
468                 final boolean r = updatedPaths.removeAll(rlb.getPath());
469                 LOG.trace("Request removed? {}", r);
470             }
471         }
472         LOG.debug("Setting new paths {} to lsp {}", updatedPaths, name);
473         return updatedPaths;
474     }
475
476     /**
477      * Indicate that the peer has completed state synchronization.
478      *
479      * @param ctx Message context
480      */
481     protected final synchronized void stateSynchronizationAchieved(final MessageContext ctx) {
482         if (this.synced) {
483             LOG.debug("State synchronization achieved while synchronized, not updating state");
484             return;
485         }
486
487         // Update synchronization flag
488         this.synced = true;
489         if (this.triggeredResyncInProcess) {
490             this.triggeredResyncInProcess = false;
491         }
492         updatePccNode(ctx, new PathComputationClientBuilder().setStateSync(PccSyncState.Synchronized).build());
493
494         // The node has completed synchronization, cleanup metadata no longer reported back
495         this.nodeState.cleanupExcept(this.lsps.values());
496         LOG.debug("Session {} achieved synchronized state", this.session);
497     }
498
499     protected final synchronized void updatePccNode(final MessageContext ctx, final PathComputationClient pcc) {
500         ctx.trans.merge(LogicalDatastoreType.OPERATIONAL, this.pccIdentifier, pcc);
501     }
502
503     protected final InstanceIdentifier<ReportedLsp> lspIdentifier(final String name) {
504         return this.pccIdentifier.child(ReportedLsp.class, new ReportedLspKey(name));
505     }
506
507     /**
508      * Remove LSP from the database.
509      *
510      * @param ctx Message Context
511      * @param id  Revision-specific LSP identifier
512      */
513     protected final synchronized void removeLsp(final MessageContext ctx, final L id) {
514         final String name = this.lsps.remove(id);
515         LOG.debug("LSP {} removed", name);
516         ctx.trans.delete(LogicalDatastoreType.OPERATIONAL, lspIdentifier(name));
517         this.lspData.remove(name);
518     }
519
520     protected abstract void onSessionUp(PCEPSession session, PathComputationClientBuilder pccBuilder);
521
522     /**
523      * Perform revision-specific message processing when a message arrives.
524      *
525      * @param ctx     Message processing context
526      * @param message Protocol message
527      * @return True if the message type is not handle.
528      */
529     protected abstract boolean onMessage(MessageContext ctx, Message message);
530
531     protected final String lookupLspName(final L id) {
532         requireNonNull(id, "ID parameter null.");
533         return this.lsps.get(id);
534     }
535
536     /**
537      * Reads operational data on this node. Doesn't attempt to read the data,
538      * if the node does not exist. In this case returns null.
539      *
540      * @param id InstanceIdentifier of the node
541      * @return null if the node does not exists, or operational data
542      */
543     protected final synchronized <T extends DataObject> ListenableFuture<Optional<T>> readOperationalData(final InstanceIdentifier<T> id) {
544         if (this.nodeState == null) {
545             return null;
546         }
547         return this.nodeState.readOperationalData(id);
548     }
549
550     protected abstract Object validateReportedLsp(final Optional<ReportedLsp> rep, final LspId input);
551
552     protected abstract void loadLspData(final Node node, final Map<String, ReportedLsp> lspData, final Map<L, String> lsps, final boolean incrementalSynchro);
553
554     protected final boolean isLspDbPersisted() {
555         if (this.syncOptimization != null) {
556             return this.syncOptimization.isSyncAvoidanceEnabled();
557         }
558         return false;
559     }
560
561     protected final boolean isLspDbRetreived() {
562         if (this.syncOptimization != null) {
563             return this.syncOptimization.isDbVersionPresent();
564         }
565         return false;
566     }
567
568     /**
569      * Is Incremental synchronization if LSP-DB-VERSION are included,
570      * LSP-DB-VERSION TLV values doesnt match, and  LSP-SYNC-CAPABILITY is enabled
571      *
572      * @return
573      */
574     protected final boolean isIncrementalSynchro() {
575         if (this.syncOptimization != null) {
576             return this.syncOptimization.isSyncAvoidanceEnabled() && this.syncOptimization.isDeltaSyncEnabled();
577         }
578         return false;
579     }
580
581     protected final boolean isTriggeredInitialSynchro() {
582         if (this.syncOptimization != null) {
583             return this.syncOptimization.isTriggeredInitSyncEnabled();
584         }
585         return false;
586     }
587
588     protected final boolean isTriggeredReSyncEnabled() {
589         if (this.syncOptimization != null) {
590             return this.syncOptimization.isTriggeredReSyncEnabled();
591         }
592         return false;
593     }
594
595     protected final boolean isSynchronized() {
596         if (this.syncOptimization != null) {
597             return this.syncOptimization.doesLspDbMatch();
598         }
599         return false;
600     }
601
602     protected synchronized SessionListenerState getSessionListenerState() {
603         return this.listenerState;
604     }
605
606     @Override
607     public synchronized Integer getDelegatedLspsCount() {
608         return Math.toIntExact(this.lspData.values().stream()
609                 .map(ReportedLsp::getPath).filter(Objects::nonNull).filter(pathList -> !pathList.isEmpty())
610                 // pick the first path, as delegate status should be same in each path
611                 .map(pathList -> pathList.get(0))
612                 .map(path -> path.getAugmentation(Path1.class)).filter(Objects::nonNull)
613                 .map(LspObject::getLsp).filter(Objects::nonNull)
614                 .filter(Lsp::isDelegate)
615                 .count());
616     }
617
618     @Override
619     public Boolean getSynchronized() {
620         return this.synced;
621     }
622
623     @Override
624     public StatefulMessages getStatefulMessages() {
625         return this.listenerState.getStatefulMessages();
626     }
627
628     @Override
629     public synchronized ReplyTime getReplyTime() {
630         return this.listenerState.getReplyTime();
631     }
632
633     @Override
634     public synchronized PeerCapabilities getPeerCapabilities() {
635         return this.listenerState.getPeerCapabilities();
636     }
637
638     @Override
639     public synchronized ListenableFuture<Void> tearDownSession(final TearDownSessionInput input) {
640         close();
641         return Futures.immediateFuture(null);
642     }
643
644     @Override
645     public synchronized SessionState getSessionState() {
646         return this.listenerState.getSessionState(this.session);
647     }
648
649     @Override
650     public synchronized String getPeerId() {
651         return this.session.getPeerPref().getIpAddress();
652     }
653
654     protected static final class MessageContext {
655         private final Collection<PCEPRequest> requests = new ArrayList<>();
656         private final WriteTransaction trans;
657
658         private MessageContext(final WriteTransaction trans) {
659             this.trans = requireNonNull(trans);
660         }
661
662         void resolveRequest(final PCEPRequest req) {
663             this.requests.add(req);
664         }
665
666         private void notifyRequests() {
667             for (final PCEPRequest r : this.requests) {
668                 r.done(OperationResults.SUCCESS);
669             }
670         }
671     }
672 }