Table 220 flows not removed
[genius.git] / interfacemanager / interfacemanager-impl / src / main / java / org / opendaylight / genius / interfacemanager / listeners / InterfaceInventoryStateListener.java
1 /*
2  * Copyright (c) 2016, 2018 Ericsson India Global Services Pvt Ltd. 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.genius.interfacemanager.listeners;
9
10 import static org.opendaylight.mdsal.binding.util.Datastore.CONFIGURATION;
11 import static org.opendaylight.mdsal.binding.util.Datastore.OPERATIONAL;
12
13 import com.google.common.util.concurrent.FutureCallback;
14 import com.google.common.util.concurrent.Futures;
15 import com.google.common.util.concurrent.ListenableFuture;
16 import com.google.common.util.concurrent.MoreExecutors;
17 import java.util.ArrayList;
18 import java.util.Collections;
19 import java.util.HashMap;
20 import java.util.List;
21 import java.util.Map;
22 import java.util.Objects;
23 import java.util.Optional;
24 import java.util.concurrent.Callable;
25 import java.util.concurrent.ExecutionException;
26 import javax.inject.Inject;
27 import javax.inject.Singleton;
28 import org.apache.aries.blueprint.annotation.service.Reference;
29 import org.checkerframework.checker.nullness.qual.Nullable;
30 import org.opendaylight.genius.interfacemanager.IfmConstants;
31 import org.opendaylight.genius.interfacemanager.IfmUtil;
32 import org.opendaylight.genius.interfacemanager.InterfacemgrProvider;
33 import org.opendaylight.genius.interfacemanager.commons.AlivenessMonitorUtils;
34 import org.opendaylight.genius.interfacemanager.commons.InterfaceManagerCommonUtils;
35 import org.opendaylight.genius.interfacemanager.commons.InterfaceMetaUtils;
36 import org.opendaylight.genius.interfacemanager.recovery.impl.InterfaceServiceRecoveryHandler;
37 import org.opendaylight.genius.interfacemanager.renderer.ovs.statehelpers.OvsInterfaceStateAddHelper;
38 import org.opendaylight.genius.interfacemanager.renderer.ovs.statehelpers.OvsInterfaceStateUpdateHelper;
39 import org.opendaylight.genius.interfacemanager.servicebindings.flowbased.utilities.FlowBasedServicesUtils;
40 import org.opendaylight.genius.utils.clustering.EntityOwnershipUtils;
41 import org.opendaylight.infrautils.jobcoordinator.JobCoordinator;
42 import org.opendaylight.infrautils.utils.concurrent.Executors;
43 import org.opendaylight.mdsal.binding.api.DataBroker;
44 import org.opendaylight.mdsal.binding.util.Datastore.Configuration;
45 import org.opendaylight.mdsal.binding.util.InterruptibleCheckedConsumer;
46 import org.opendaylight.mdsal.binding.util.ManagedNewTransactionRunner;
47 import org.opendaylight.mdsal.binding.util.ManagedNewTransactionRunnerImpl;
48 import org.opendaylight.mdsal.binding.util.TypedReadWriteTransaction;
49 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
50 import org.opendaylight.serviceutils.srm.RecoverableListener;
51 import org.opendaylight.serviceutils.srm.ServiceRecoveryRegistry;
52 import org.opendaylight.serviceutils.tools.listener.AbstractClusteredAsyncDataTreeChangeListener;
53 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.interfaces.rev140508.interfaces.state.Interface;
54 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNodeConnector;
55 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.types.port.rev130925.PortReason;
56 import org.opendaylight.yang.gen.v1.urn.opendaylight.genius.idmanager.rev160406.IdManagerService;
57 import org.opendaylight.yang.gen.v1.urn.opendaylight.genius.interfacemanager.meta.rev160406._interface.child.info.InterfaceParentEntry;
58 import org.opendaylight.yang.gen.v1.urn.opendaylight.genius.interfacemanager.meta.rev160406._interface.child.info._interface.parent.entry.InterfaceChildEntry;
59 import org.opendaylight.yang.gen.v1.urn.opendaylight.genius.interfacemanager.meta.rev160406._interface.child.info._interface.parent.entry.InterfaceChildEntryKey;
60 import org.opendaylight.yang.gen.v1.urn.opendaylight.genius.interfacemanager.rev160406.IfTunnel;
61 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeConnectorId;
62 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.Nodes;
63 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.node.NodeConnector;
64 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.nodes.Node;
65 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
66 import org.opendaylight.yangtools.yang.common.Uint64;
67 import org.slf4j.Logger;
68 import org.slf4j.LoggerFactory;
69
70
71 /**
72  * This Class is a Data Change Listener for FlowCapableNodeConnector updates.
73  * This creates an entry in the interface-state OperDS for every node-connector
74  * used.
75  *
76  * <p>
77  * NOTE: This class just creates an ifstate entry whose interface-name will be
78  * the same as the node-connector portname. If PortName is not unique across
79  * DPNs, this implementation can have problems.
80  */
81 @Singleton
82 public class InterfaceInventoryStateListener
83         extends AbstractClusteredAsyncDataTreeChangeListener<FlowCapableNodeConnector>
84         implements RecoverableListener {
85
86     private static final Logger LOG = LoggerFactory.getLogger(InterfaceInventoryStateListener.class);
87     private static final Logger EVENT_LOGGER = LoggerFactory.getLogger("GeniusEventLogger");
88
89     private final DataBroker dataBroker;
90     private final ManagedNewTransactionRunner txRunner;
91     private final IdManagerService idManager;
92     private final EntityOwnershipUtils entityOwnershipUtils;
93     private final JobCoordinator coordinator;
94     private final InterfaceManagerCommonUtils interfaceManagerCommonUtils;
95     private final AlivenessMonitorUtils alivenessMonitorUtils;
96     private final OvsInterfaceStateUpdateHelper ovsInterfaceStateUpdateHelper;
97     private final OvsInterfaceStateAddHelper ovsInterfaceStateAddHelper;
98     private final InterfaceMetaUtils interfaceMetaUtils;
99     private final PortNameCache portNameCache;
100     private final InterfacemgrProvider interfacemgrProvider;
101     private final MigrationInProgressCache migrationInProgressCache;
102
103     @Inject
104     public InterfaceInventoryStateListener(@Reference final DataBroker dataBroker,
105                                            final IdManagerService idManagerService,
106                                            final EntityOwnershipUtils entityOwnershipUtils,
107                                            @Reference final JobCoordinator coordinator,
108                                            final InterfaceManagerCommonUtils interfaceManagerCommonUtils,
109                                            final OvsInterfaceStateAddHelper ovsInterfaceStateAddHelper,
110                                            final OvsInterfaceStateUpdateHelper ovsInterfaceStateUpdateHelper,
111                                            final AlivenessMonitorUtils alivenessMonitorUtils,
112                                            final InterfaceMetaUtils interfaceMetaUtils,
113                                            final PortNameCache portNameCache,
114                                            final InterfaceServiceRecoveryHandler interfaceServiceRecoveryHandler,
115                                            @Reference final ServiceRecoveryRegistry serviceRecoveryRegistry,
116                                            final InterfacemgrProvider interfacemgrProvider) {
117         super(dataBroker, LogicalDatastoreType.OPERATIONAL, InstanceIdentifier.create(Nodes.class)
118                         .child(Node.class).child(NodeConnector.class)
119                         .augmentation(FlowCapableNodeConnector.class),
120                 Executors.newFixedThreadPool(1, "InterfaceInventoryStateListener", LOG));
121         this.dataBroker = dataBroker;
122         this.txRunner = new ManagedNewTransactionRunnerImpl(dataBroker);
123         this.idManager = idManagerService;
124         this.entityOwnershipUtils = entityOwnershipUtils;
125         this.coordinator = coordinator;
126         this.interfaceManagerCommonUtils = interfaceManagerCommonUtils;
127         this.alivenessMonitorUtils = alivenessMonitorUtils;
128         this.ovsInterfaceStateUpdateHelper = ovsInterfaceStateUpdateHelper;
129         this.ovsInterfaceStateAddHelper = ovsInterfaceStateAddHelper;
130         this.interfaceMetaUtils = interfaceMetaUtils;
131         this.portNameCache = portNameCache;
132         this.interfacemgrProvider = interfacemgrProvider;
133         this.migrationInProgressCache = new MigrationInProgressCache();
134         serviceRecoveryRegistry.addRecoverableListener(interfaceServiceRecoveryHandler.buildServiceRegistryKey(),
135                 this);
136     }
137
138     @Override
139     public void registerListener() {
140         super.register();
141     }
142
143     @Override
144     public void deregisterListener() {
145         close();
146     }
147
148     @Override
149     public void remove(InstanceIdentifier<FlowCapableNodeConnector> key,
150                           FlowCapableNodeConnector flowCapableNodeConnectorOld) {
151         String interfaceName = flowCapableNodeConnectorOld.getName();
152         EVENT_LOGGER.debug("IFM-InterfaceInventoryState,REMOVE {}", interfaceName);
153         if (interfacemgrProvider.isItmDirectTunnelsEnabled()
154             && InterfaceManagerCommonUtils.isTunnelPort(interfaceName)
155             && interfaceManagerCommonUtils.getInterfaceFromConfigDS(interfaceName) == null) {
156             LOG.debug("ITM Direct Tunnels is enabled, node connector removed event for"
157                     + " internal tunnel {}", interfaceName);
158             return;
159         }
160
161         NodeConnectorId nodeConnectorId = InstanceIdentifier.keyOf(key.firstIdentifierOf(NodeConnector.class))
162                 .getId();
163         LOG.trace("Removing entry for port id {} from map",nodeConnectorId.getValue());
164         portNameCache.remove(nodeConnectorId.getValue());
165
166
167         if (!entityOwnershipUtils.isEntityOwner(IfmConstants.INTERFACE_CONFIG_ENTITY,
168                 IfmConstants.INTERFACE_CONFIG_ENTITY)) {
169             return;
170         }
171
172         LOG.debug("Received NodeConnector Remove Event: {}, {}", key, flowCapableNodeConnectorOld);
173         String portName = InterfaceManagerCommonUtils.getPortNameForInterface(nodeConnectorId,
174             flowCapableNodeConnectorOld.getName());
175         EVENT_LOGGER.debug("IFM-InterfaceInventoryState Entity Owner,REMOVE {},{}", portName,
176                 nodeConnectorId.getValue());
177
178         remove(nodeConnectorId, null, flowCapableNodeConnectorOld, portName, true);
179     }
180
181     private void remove(NodeConnectorId nodeConnectorIdNew, NodeConnectorId nodeConnectorIdOld,
182                         FlowCapableNodeConnector fcNodeConnectorNew, String portName, boolean isNetworkEvent) {
183         InterfaceStateRemoveWorker portStateRemoveWorker = new InterfaceStateRemoveWorker(idManager,
184                 nodeConnectorIdNew, nodeConnectorIdOld, fcNodeConnectorNew, portName,
185                 isNetworkEvent, true);
186         coordinator.enqueueJob(portName, portStateRemoveWorker, IfmConstants.JOB_MAX_RETRIES);
187     }
188
189     @Override
190     public void update(InstanceIdentifier<FlowCapableNodeConnector> key, FlowCapableNodeConnector fcNodeConnectorOld,
191         FlowCapableNodeConnector fcNodeConnectorNew) {
192         String interfaceName = fcNodeConnectorNew.getName();
193         EVENT_LOGGER.debug("IFM-InterfaceInventoryState,UPDATE {},{}", fcNodeConnectorNew.getName(),
194                 fcNodeConnectorNew.getReason());
195         if (interfacemgrProvider.isItmDirectTunnelsEnabled()
196             && InterfaceManagerCommonUtils.isTunnelPort(interfaceName)
197             && interfaceManagerCommonUtils.getInterfaceFromConfigDS(interfaceName) == null) {
198             LOG.debug("ITM Direct Tunnels is enabled, hence ignoring node connector Update event for"
199                     + " internal tunnel {}", interfaceName);
200             return;
201         }
202
203
204         if (fcNodeConnectorNew.getReason() == PortReason.Delete
205                 || !entityOwnershipUtils.isEntityOwner(IfmConstants.INTERFACE_CONFIG_ENTITY,
206                 IfmConstants.INTERFACE_CONFIG_ENTITY)) {
207             return;
208         }
209
210         LOG.debug("Received NodeConnector Update Event: {}, {}, {}", key, fcNodeConnectorOld, fcNodeConnectorNew);
211         NodeConnectorId nodeConnectorId = InstanceIdentifier.keyOf(key.firstIdentifierOf(NodeConnector.class)).getId();
212         String portName = InterfaceManagerCommonUtils.getPortNameForInterface(nodeConnectorId,
213                 fcNodeConnectorNew.getName());
214         EVENT_LOGGER.debug("IFM-InterfaceInventoryState Entity Owner,UPDATE {},{}", portName,
215                 nodeConnectorId.getValue());
216
217         InterfaceStateUpdateWorker portStateUpdateWorker = new InterfaceStateUpdateWorker(key, fcNodeConnectorOld,
218             fcNodeConnectorNew, portName);
219         coordinator.enqueueJob(portName, portStateUpdateWorker, IfmConstants.JOB_MAX_RETRIES);
220     }
221
222     @Override
223     public void add(InstanceIdentifier<FlowCapableNodeConnector> key, FlowCapableNodeConnector fcNodeConnectorNew) {
224         String interfaceName = fcNodeConnectorNew.getName();
225         EVENT_LOGGER.debug("IFM-InterfaceInventoryState,ADD {}", interfaceName);
226         if (interfacemgrProvider.isItmDirectTunnelsEnabled()
227             && InterfaceManagerCommonUtils.isTunnelPort(interfaceName)
228             && interfaceManagerCommonUtils.getInterfaceFromConfigDS(interfaceName) == null) {
229             LOG.debug("ITM Direct Tunnels is enabled, ignoring node connector add for"
230                     + " internal tunnel {}", interfaceName);
231             return;
232         }
233
234         NodeConnectorId nodeConnectorId = InstanceIdentifier.keyOf(key.firstIdentifierOf(NodeConnector.class))
235                 .getId();
236         LOG.trace("Adding entry for portid {} portname {} in map", nodeConnectorId.getValue(),
237                 fcNodeConnectorNew.getName());
238         portNameCache.put(nodeConnectorId.getValue(),fcNodeConnectorNew.getName());
239         if (!entityOwnershipUtils.isEntityOwner(IfmConstants.INTERFACE_CONFIG_ENTITY,
240                 IfmConstants.INTERFACE_CONFIG_ENTITY)) {
241             return;
242         }
243
244         LOG.debug("Received NodeConnector Add Event: {}, {}", key, fcNodeConnectorNew);
245         String portName = InterfaceManagerCommonUtils.getPortNameForInterface(nodeConnectorId,
246             fcNodeConnectorNew.getName());
247         EVENT_LOGGER.debug("IFM-InterfaceInventoryState Entity Owner,ADD {},{}", portName, nodeConnectorId.getValue());
248
249         if (InterfaceManagerCommonUtils.isNovaPort(portName) || InterfaceManagerCommonUtils.isK8SPort(portName)) {
250             Optional<NodeConnectorId> nodeConnectorIdFromCache = migrationInProgressCache.get(interfaceName);
251             if (nodeConnectorIdFromCache.isPresent() && nodeConnectorIdFromCache.get().equals(nodeConnectorId)) {
252                 LOG.error("NodeConnectorId is changed. Dropping Port update event for {} from NodeConnectorId {}",
253                         fcNodeConnectorNew.getName(), nodeConnectorId);
254                 return;
255             }
256             NodeConnectorId nodeConnectorIdOld = null;
257             org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang
258                     .ietf.interfaces.rev140508.interfaces.state.Interface interfaceState = interfaceManagerCommonUtils
259                     .getInterfaceState(interfaceName);
260             if (interfaceState != null) {
261                 List<String> ofportIds = interfaceState.getLowerLayerIf();
262                 nodeConnectorIdOld = new NodeConnectorId(ofportIds.get(0));
263             }
264             if (nodeConnectorIdOld != null && !nodeConnectorId.equals(nodeConnectorIdOld)) {
265                 Uint64 dpnIdOld = IfmUtil.getDpnFromNodeConnectorId(nodeConnectorIdOld);
266                 Uint64 dpnIdNew = IfmUtil.getDpnFromNodeConnectorId(nodeConnectorId);
267                 if (!Objects.equals(dpnIdOld, dpnIdNew)) {
268                     if (fcNodeConnectorNew.getReason() != PortReason.Add
269                             && interfaceState.getOperStatus()
270                             != Interface.OperStatus.Unknown) {
271                         LOG.error("Dropping Port update event for {}, as DPN id is changed from {} to {}",
272                             fcNodeConnectorNew.getName(), dpnIdOld, dpnIdNew);
273                         return;
274                     }
275                 } else {
276                     LOG.warn("Port number update detected for {}", fcNodeConnectorNew.getName());
277                 }
278                 //VM Migration or Port Number Update: Delete existing interface entry for older DPN
279                 migrationInProgressCache.put(interfaceName, nodeConnectorIdOld);
280                 LOG.trace("Removing entry for port id {} from map",nodeConnectorIdOld.getValue());
281                 portNameCache.remove(nodeConnectorIdOld.getValue());
282                 EVENT_LOGGER.debug("IFM-VMMigration,{}", portName);
283                 LOG.debug("Triggering NodeConnector Remove Event for the interface: {}, {}, {}", portName,
284                     nodeConnectorId, nodeConnectorIdOld);
285                 remove(nodeConnectorId, nodeConnectorIdOld, fcNodeConnectorNew, portName, false);
286                 // Adding a delay of 10sec for VM migration, so applications will have sufficient time
287                 // for processing remove before add
288                 try {
289                     Thread.sleep(IfmConstants.DELAY_TIME_IN_MILLISECOND);
290                 } catch (final InterruptedException e) {
291                     LOG.error("Error while waiting for the vm migration remove events to get processed");
292                 }
293             }
294         }
295
296         InterfaceStateAddWorker ifStateAddWorker = new InterfaceStateAddWorker(idManager, nodeConnectorId,
297             fcNodeConnectorNew, portName, interfaceName, IfmConstants.JOB_MAX_RETRIES);
298         coordinator.enqueueJob(portName, ifStateAddWorker, IfmConstants.JOB_MAX_RETRIES);
299     }
300
301
302     private class InterfaceStateAddWorker implements Callable {
303         private final NodeConnectorId nodeConnectorId;
304         private final FlowCapableNodeConnector fcNodeConnectorNew;
305         private final String interfaceName;
306         private final IdManagerService idManager;
307         private final String migrationCacheKey;
308         private int maxRetries;
309
310         InterfaceStateAddWorker(IdManagerService idManager, NodeConnectorId nodeConnectorId,
311                                 FlowCapableNodeConnector fcNodeConnectorNew, String portName, String migrationCacheKey,
312                                 int maxRetries) {
313             this.nodeConnectorId = nodeConnectorId;
314             this.fcNodeConnectorNew = fcNodeConnectorNew;
315             this.interfaceName = portName;
316             this.idManager = idManager;
317             this.migrationCacheKey = migrationCacheKey;
318             this.maxRetries = maxRetries;
319         }
320
321         @Override
322         public Object call() {
323             List<? extends ListenableFuture<?>> futures = ovsInterfaceStateAddHelper.addState(nodeConnectorId,
324                     interfaceName, fcNodeConnectorNew);
325             Map<InterfaceChildEntryKey, InterfaceChildEntry> interfaceChildEntries =
326                     getInterfaceChildEntries(interfaceName);
327             for (InterfaceChildEntry interfaceChildEntry : interfaceChildEntries.values()) {
328                 InterfaceStateAddWorker interfaceStateAddWorker = new InterfaceStateAddWorker(idManager,
329                         nodeConnectorId, fcNodeConnectorNew, interfaceChildEntry.getChildInterface(), null, 0);
330                 coordinator.enqueueJob(interfaceName, interfaceStateAddWorker);
331             }
332
333             if (migrationCacheKey != null && futures != null && !futures.isEmpty()) {
334                 ListenableFuture<List<Object>> completedFuture = Futures.allAsList(futures);
335                 Futures.addCallback(completedFuture, new FutureCallback<List<Object>>() {
336                     @Override
337                     public void onFailure(Throwable error) {
338                         maxRetries--;
339                         if (maxRetries == 0) {
340                             migrationInProgressCache.remove(migrationCacheKey);
341                             LOG.error("OvsInterfaceStateAddHelper addState failed for interface {}",
342                                     migrationCacheKey);
343                         }
344                     }
345
346                     @Override
347                     public void onSuccess(List<Object> result) {
348                         migrationInProgressCache.remove(migrationCacheKey);
349                     }
350                 }, MoreExecutors.directExecutor());
351             }
352             return futures;
353         }
354
355         @Override
356         public String toString() {
357             return "InterfaceStateAddWorker{" + "nodeConnectorId=" + nodeConnectorId + ", fcNodeConnectorNew="
358                     + fcNodeConnectorNew + ", interfaceName='" + interfaceName + '\'' + '}';
359         }
360     }
361
362     private class InterfaceStateUpdateWorker implements Callable<List<? extends ListenableFuture<?>>> {
363         private final InstanceIdentifier<FlowCapableNodeConnector> key;
364         private final FlowCapableNodeConnector fcNodeConnectorOld;
365         private final FlowCapableNodeConnector fcNodeConnectorNew;
366         private final String interfaceName;
367
368         InterfaceStateUpdateWorker(InstanceIdentifier<FlowCapableNodeConnector> key,
369                                    FlowCapableNodeConnector fcNodeConnectorOld,
370                                    FlowCapableNodeConnector fcNodeConnectorNew,
371                                    String portName) {
372             this.key = key;
373             this.fcNodeConnectorOld = fcNodeConnectorOld;
374             this.fcNodeConnectorNew = fcNodeConnectorNew;
375             this.interfaceName = portName;
376         }
377
378         @Override
379         public List<? extends ListenableFuture<?>> call() {
380             List<? extends ListenableFuture<?>> futures = ovsInterfaceStateUpdateHelper.updateState(
381                     interfaceName, fcNodeConnectorNew, fcNodeConnectorOld);
382             Map<InterfaceChildEntryKey, InterfaceChildEntry> interfaceChildEntries =
383                     getInterfaceChildEntries(interfaceName);
384             for (InterfaceChildEntry interfaceChildEntry : interfaceChildEntries.values()) {
385                 InterfaceStateUpdateWorker interfaceStateUpdateWorker = new InterfaceStateUpdateWorker(key,
386                         fcNodeConnectorOld, fcNodeConnectorNew, interfaceChildEntry.getChildInterface());
387                 coordinator.enqueueJob(interfaceName, interfaceStateUpdateWorker);
388             }
389             return futures;
390         }
391
392         @Override
393         public String toString() {
394             return "InterfaceStateUpdateWorker{" + "key=" + key + ", fcNodeConnectorOld=" + fcNodeConnectorOld
395                     + ", fcNodeConnectorNew=" + fcNodeConnectorNew + ", interfaceName='" + interfaceName + '\'' + '}';
396         }
397     }
398
399     private class InterfaceStateRemoveWorker implements Callable<List<? extends ListenableFuture<?>>> {
400         private final NodeConnectorId nodeConnectorIdNew;
401         private NodeConnectorId nodeConnectorIdOld;
402         private final FlowCapableNodeConnector fcNodeConnectorOld;
403         private final String interfaceName;
404         private final IdManagerService idManager;
405         private final boolean isNetworkEvent;
406         private final boolean isParentInterface;
407
408         InterfaceStateRemoveWorker(IdManagerService idManager, NodeConnectorId nodeConnectorIdNew,
409                                    NodeConnectorId nodeConnectorIdOld,
410                                    FlowCapableNodeConnector fcNodeConnectorOld, String interfaceName,
411                                    boolean isNetworkEvent,
412                                    boolean isParentInterface) {
413             this.nodeConnectorIdNew = nodeConnectorIdNew;
414             this.nodeConnectorIdOld = nodeConnectorIdOld;
415             this.fcNodeConnectorOld = fcNodeConnectorOld;
416             this.interfaceName = interfaceName;
417             this.idManager = idManager;
418             this.isNetworkEvent = isNetworkEvent;
419             this.isParentInterface = isParentInterface;
420         }
421
422         @Override
423         public List<? extends ListenableFuture<?>> call() {
424             // VM Migration: Skip OFPPR_DELETE event received after OFPPR_ADD
425             // for same interface from Older DPN
426             if (isParentInterface && isNetworkEvent) {
427                 nodeConnectorIdOld = FlowBasedServicesUtils.getNodeConnectorIdFromInterface(interfaceName,
428                         interfaceManagerCommonUtils);
429                 if (nodeConnectorIdOld != null && !nodeConnectorIdNew.equals(nodeConnectorIdOld)) {
430                     LOG.debug("Dropping the NodeConnector Remove Event for the interface: {}, {}, {}", interfaceName,
431                             nodeConnectorIdNew, nodeConnectorIdOld);
432                     return Collections.emptyList();
433                 }
434             }
435
436             List<? extends ListenableFuture<?>> futures = removeInterfaceStateConfiguration();
437
438             Map<InterfaceChildEntryKey, InterfaceChildEntry> interfaceChildEntries =
439                     getInterfaceChildEntries(interfaceName);
440             for (InterfaceChildEntry interfaceChildEntry : interfaceChildEntries.values()) {
441                 // Fetch all interfaces on this port and trigger remove worker
442                 // for each of them
443                 InterfaceStateRemoveWorker interfaceStateRemoveWorker = new InterfaceStateRemoveWorker(idManager,
444                         nodeConnectorIdNew, nodeConnectorIdOld, fcNodeConnectorOld,
445                         interfaceChildEntry.getChildInterface(), isNetworkEvent, false);
446                 coordinator.enqueueJob(interfaceName, interfaceStateRemoveWorker);
447             }
448             return futures;
449         }
450
451         private List<? extends ListenableFuture<?>> removeInterfaceStateConfiguration() {
452             List<ListenableFuture<?>> futures = new ArrayList<>();
453
454             //VM Migration: Use old nodeConnectorId to delete the interface entry
455             NodeConnectorId nodeConnectorId = nodeConnectorIdOld != null
456                     && !nodeConnectorIdNew.equals(nodeConnectorIdOld) ? nodeConnectorIdOld : nodeConnectorIdNew;
457             // delete the port entry from interface operational DS
458             Uint64 dpId = IfmUtil.getDpnFromNodeConnectorId(nodeConnectorId);
459
460             futures.add(txRunner.applyWithNewTransactionChainAndClose(txChain ->
461                 txChain.applyWithNewReadWriteTransactionAndSubmit(OPERATIONAL, operTx -> {
462                     // In a genuine port delete scenario, the reason will be there in the incoming event, for all
463                     // remaining
464                     // cases treat the event as DPN disconnect, if old and new ports are same. Else, this is a VM
465                     // migration
466                     // scenario, and should be treated as port removal.
467                     LOG.debug("Removing interface state information for interface: {}", interfaceName);
468                     if (fcNodeConnectorOld.getReason() != PortReason.Delete
469                         && nodeConnectorIdNew.equals(nodeConnectorIdOld)) {
470                         //Remove event is because of connection lost between controller and switch, or switch shutdown.
471                         // Hence, don't remove the interface but set the status as "unknown"
472                         ovsInterfaceStateUpdateHelper.updateInterfaceStateOnNodeRemove(interfaceName,
473                             fcNodeConnectorOld,
474                             operTx);
475                     } else {
476                         EVENT_LOGGER.debug("IFM-OvsInterfaceState,REMOVE {}", interfaceName);
477                         InterfaceManagerCommonUtils.deleteStateEntry(operTx, interfaceName);
478                         org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.interfaces.rev140508.interfaces
479                             .Interface iface = interfaceManagerCommonUtils.getInterfaceFromConfigDS(interfaceName);
480
481                         if (InterfaceManagerCommonUtils.isTunnelInterface(iface)) {
482                             // If this interface is a tunnel interface, remove the tunnel ingress flow and stop LLDP
483                             // monitoring
484                             interfaceMetaUtils.removeLportTagInterfaceMap(operTx, interfaceName);
485                             return Optional.of((InterruptibleCheckedConsumer<TypedReadWriteTransaction<Configuration>,
486                                 ExecutionException>) confTx -> handleTunnelMonitoringRemoval(confTx, dpId,
487                                 iface.getName(),
488                                 iface.augmentation(IfTunnel.class)));
489                         }
490                         // remove ingress flow only for northbound configured interfaces
491                         // skip this check for non-unique ports(Ex: br-int,br-ex)
492
493                         if (iface != null) {
494                             FlowBasedServicesUtils.removeIngressFlow(interfaceName, dpId, txRunner, futures);
495                             IfmUtil.unbindService(txRunner, coordinator, iface.getName(),
496                                     FlowBasedServicesUtils.buildDefaultServiceId(iface.getName()));
497                             EVENT_LOGGER.debug("IFM-InterfaceState, REMOVE, IngressFlow {}", interfaceName);
498                         }
499                         // Delete the Vpn Interface from DpnToInterface Op DS.
500                         InterfaceManagerCommonUtils.deleteDpnToInterface(dpId, interfaceName, operTx);
501                     }
502                     return Optional.empty();
503                 }).transform((@Nullable Optional<?> optionalJob) -> {
504                     if (optionalJob != null && optionalJob.isPresent()) {
505                         txChain.callWithNewReadWriteTransactionAndSubmit(CONFIGURATION,
506                             (InterruptibleCheckedConsumer<TypedReadWriteTransaction<Configuration>, ?
507                                 extends Exception>) optionalJob.get());
508                     }
509                     return null;
510                 }, MoreExecutors.directExecutor())));
511             return futures;
512         }
513
514         private void handleTunnelMonitoringRemoval(TypedReadWriteTransaction<Configuration> tx, Uint64 dpId,
515             String removedInterfaceName, IfTunnel ifTunnel) throws ExecutionException, InterruptedException {
516             interfaceManagerCommonUtils.removeTunnelIngressFlow(tx, ifTunnel, dpId, removedInterfaceName);
517
518             IfmUtil.unbindService(txRunner, coordinator, removedInterfaceName,
519                     FlowBasedServicesUtils.buildDefaultServiceId(removedInterfaceName));
520
521             alivenessMonitorUtils.stopLLDPMonitoring(ifTunnel, removedInterfaceName);
522         }
523
524         @Override
525         public String toString() {
526             return "InterfaceStateRemoveWorker{" + "nodeConnectorIdNew=" + nodeConnectorIdNew + ", nodeConnectorIdOld="
527                     + nodeConnectorIdOld + ", fcNodeConnectorOld=" + fcNodeConnectorOld + ", interfaceName='"
528                     + interfaceName + '\'' + '}';
529         }
530     }
531
532     public Map<InterfaceChildEntryKey, InterfaceChildEntry> getInterfaceChildEntries(String interfaceName) {
533         InterfaceParentEntry interfaceParentEntry =
534                 interfaceMetaUtils.getInterfaceParentEntryFromConfigDS(interfaceName);
535         if (interfaceParentEntry != null && interfaceParentEntry.getInterfaceChildEntry() != null) {
536             return interfaceParentEntry.getInterfaceChildEntry();
537         }
538         return new HashMap<>();
539     }
540 }