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