Integrate TopologyStatsProviderImpl
[bgpcep.git] / pcep / topology / topology-provider / src / main / java / org / opendaylight / bgpcep / pcep / topology / provider / AbstractTopologySessionListener.java
index ebebc3c3515328adde3fc8d1fe5916efc98c7fa0..cf90355f2a47269b3952f4cd589470db339a6a1d 100644 (file)
@@ -7,6 +7,7 @@
  */
 package org.opendaylight.bgpcep.pcep.topology.provider;
 
+import static com.google.common.base.Verify.verifyNotNull;
 import static java.util.Objects.requireNonNull;
 
 import com.google.common.collect.Iterables;
@@ -69,6 +70,7 @@ 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.rev220730.pcep.client.attributes.path.computation.client.reported.lsp.Path;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.topology.pcep.rev220730.pcep.client.attributes.path.computation.client.reported.lsp.PathKey;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.Node;
+import org.opendaylight.yangtools.concepts.ObjectRegistration;
 import org.opendaylight.yangtools.yang.binding.DataObject;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 import org.opendaylight.yangtools.yang.common.RpcResult;
@@ -91,15 +93,15 @@ public abstract class AbstractTopologySessionListener implements TopologySession
     @GuardedBy("this")
     final Map<PlspId, String> lsps = new HashMap<>();
     @GuardedBy("this")
-    SessionStateImpl listenerState;
+    private ObjectRegistration<SessionStateImpl> listenerState;
 
     // FIXME: clarify lifecycle rules of this map, most notably the interaction of multiple SrpIdNumbers
     @GuardedBy("this")
     private final Map<SrpIdNumber, PCEPRequest> requests = new HashMap<>();
     @GuardedBy("this")
     private final Map<String, ReportedLsp> lspData = new ConcurrentHashMap<>();
-    private final TopologySessionStatsRegistry statsProvider;
     private final ServerSessionManager serverSessionManager;
+    private final SessionStateRegistry stateRegistry;
 
     private InstanceIdentifier<PathComputationClient> pccIdentifier;
     @GuardedBy("this")
@@ -112,9 +114,9 @@ public abstract class AbstractTopologySessionListener implements TopologySession
     @GuardedBy("this")
     private boolean triggeredResyncInProcess;
 
-    AbstractTopologySessionListener(final TopologySessionStatsRegistry statsProvider,
+    AbstractTopologySessionListener(final SessionStateRegistry stateRegistry,
             final ServerSessionManager serverSessionManager) {
-        this.statsProvider = requireNonNull(statsProvider);
+        this.stateRegistry = requireNonNull(stateRegistry);
         this.serverSessionManager = requireNonNull(serverSessionManager);
     }
 
@@ -129,8 +131,8 @@ public abstract class AbstractTopologySessionListener implements TopologySession
                  */
                 final InetAddress peerAddress = psession.getRemoteAddress();
 
-                syncOptimization = new SyncOptimization(psession);
-                final boolean haveLspDbVersion = syncOptimization.isDbVersionPresent();
+                syncOptimization = SyncOptimization.of(psession.getLocalTlvs(), psession.getRemoteTlvs());
+                final boolean haveLspDbVersion = syncOptimization.dbVersionPresent();
 
                 final TopologyNodeState state =
                         serverSessionManager.takeNodeState(peerAddress, this, haveLspDbVersion);
@@ -153,7 +155,8 @@ public abstract class AbstractTopologySessionListener implements TopologySession
                 session = psession;
                 nodeState = state;
 
-                LOG.trace("Peer {} resolved to topology node {}", peerAddress, state.getNodeId());
+                final var nodeId = state.getNodeId();
+                LOG.trace("Peer {} resolved to topology node {}", peerAddress, nodeId);
 
                 // Our augmentation in the topology node
                 final PathComputationClientBuilder pccBuilder = new PathComputationClientBuilder()
@@ -164,7 +167,7 @@ public abstract class AbstractTopologySessionListener implements TopologySession
 
                 synced.set(isSynchronized());
 
-                final InstanceIdentifier<Node1> topologyAugment = state.getNodeId().augmentation(Node1.class);
+                final InstanceIdentifier<Node1> topologyAugment = nodeId.augmentation(Node1.class);
                 pccIdentifier = topologyAugment.child(PathComputationClient.class);
 
                 if (haveLspDbVersion) {
@@ -178,11 +181,8 @@ public abstract class AbstractTopologySessionListener implements TopologySession
                 state.storeNode(topologyAugment,
                         new Node1Builder().setPathComputationClient(pccBuilder.build()).build(), psession);
 
-                // TODO: collapse assignment? needs to be verified through bytecode
-                final var sessionState = new SessionStateImpl(this, psession);
-                listenerState = sessionState;
-                statsProvider.bind(state.getNodeId(), sessionState);
-                LOG.info("Session with {} attached to topology node {}", peerAddress, state.getNodeId());
+                listenerState = stateRegistry.bind(nodeId, new SessionStateImpl(this, psession));
+                LOG.info("Session with {} attached to topology node {}", peerAddress, nodeId);
             }
         }
     }
@@ -280,7 +280,6 @@ public abstract class AbstractTopologySessionListener implements TopologySession
                     LOG.error("Session {} cannot be closed.", psession, e);
                 }
                 session = null;
-                listenerState = null;
                 syncOptimization = null;
                 clearRequests();
             }
@@ -362,7 +361,6 @@ public abstract class AbstractTopologySessionListener implements TopologySession
                     session.close(TerminationReason.UNKNOWN);
                     session = null;
                 }
-                listenerState = null;
                 syncOptimization = null;
                 clearRequests();
             }
@@ -372,8 +370,11 @@ public abstract class AbstractTopologySessionListener implements TopologySession
     @Holding({"this.serverSessionManager", "this"})
     private void clearNodeState() {
         if (nodeState != null) {
-            statsProvider.unbind(nodeState.getNodeId());
             LOG.debug("Clear Node state: {}", nodeState.getNodeId());
+            if (listenerState != null) {
+                listenerState.close();
+                listenerState = null;
+            }
             nodeState = null;
         }
     }
@@ -408,7 +409,8 @@ public abstract class AbstractTopologySessionListener implements TopologySession
     final synchronized PCEPRequest removeRequest(final SrpIdNumber id) {
         final PCEPRequest ret = requests.remove(id);
         if (ret != null && listenerState != null) {
-            listenerState.processRequestStats(ret.getElapsedMillis());
+            // FIXME: just update fields
+            listenerState.getInstance().processRequestStats(ret.getElapsedMillis());
         }
         LOG.trace("Removed request {} object {}", id, ret);
         return ret;
@@ -417,7 +419,8 @@ public abstract class AbstractTopologySessionListener implements TopologySession
     final synchronized ListenableFuture<OperationResult> sendMessage(final Message message, final SrpIdNumber requestId,
             final Metadata metadata) {
         final var sendFuture = session.sendMessage(message);
-        listenerState.updateStatefulSentMsg(message);
+        // FIXME: just update fields
+        listenerState().updateStatefulSentMsg(message);
 
         // Note: the timeout is held back by us holding the 'this' monitor, which timeoutExpired re-acquires
         final var timeout = serverSessionManager.newRpcTimeout(this::timeoutExpired, requestId);
@@ -624,7 +627,7 @@ public abstract class AbstractTopologySessionListener implements TopologySession
             boolean incrementalSynchro);
 
     final boolean isLspDbPersisted() {
-        return syncOptimization != null && syncOptimization.isSyncAvoidanceEnabled();
+        return syncOptimization != null && syncOptimization.syncAvoidanceEnabled();
     }
 
     /**
@@ -632,20 +635,20 @@ public abstract class AbstractTopologySessionListener implements TopologySession
      * LSP-DB-VERSION TLV values doesnt match, and  LSP-SYNC-CAPABILITY is enabled.
      */
     final synchronized boolean isIncrementalSynchro() {
-        return syncOptimization != null && syncOptimization.isSyncAvoidanceEnabled()
-                && syncOptimization.isDeltaSyncEnabled();
+        return syncOptimization != null && syncOptimization.syncAvoidanceEnabled()
+                && syncOptimization.deltaSyncEnabled();
     }
 
     final synchronized boolean isTriggeredInitialSynchro() {
-        return syncOptimization != null && syncOptimization.isTriggeredInitSyncEnabled();
+        return syncOptimization != null && syncOptimization.triggeredInitialSyncEnabled();
     }
 
     final synchronized boolean isTriggeredReSyncEnabled() {
-        return syncOptimization != null && syncOptimization.isTriggeredReSyncEnabled();
+        return syncOptimization != null && syncOptimization.triggeredReSyncEnabled();
     }
 
     protected final synchronized boolean isSynchronized() {
-        return syncOptimization != null && syncOptimization.doesLspDbMatch();
+        return syncOptimization != null && syncOptimization.dbVersionMatch();
     }
 
     @Override
@@ -680,13 +683,16 @@ public abstract class AbstractTopologySessionListener implements TopologySession
         return lspUpdateCapability.get();
     }
 
-
     @Override
     public synchronized ListenableFuture<RpcResult<Void>> tearDownSession(final TearDownSessionInput input) {
         close();
         return RpcResultBuilder.<Void>success().buildFuture();
     }
 
+    final synchronized @NonNull SessionStateImpl listenerState() {
+        return verifyNotNull(listenerState).getInstance();
+    }
+
     static final class MessageContext {
         private final Collection<PCEPRequest> requests = new ArrayList<>();
         private final WriteTransaction trans;