NETVIRT-1630 migrate to md-sal APIs
[netvirt.git] / elanmanager / impl / src / main / java / org / opendaylight / netvirt / elan / internal / ElanInterfaceManager.java
1 /*
2  * Copyright (c) 2016, 2017 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.netvirt.elan.internal;
9
10 import static java.util.Collections.emptyList;
11 import static org.opendaylight.genius.infra.Datastore.CONFIGURATION;
12 import static org.opendaylight.genius.infra.Datastore.OPERATIONAL;
13 import static org.opendaylight.infrautils.utils.concurrent.LoggingFutures.addErrorLogging;
14 import static org.opendaylight.mdsal.binding.api.WriteTransaction.CREATE_MISSING_PARENTS;
15 import static org.opendaylight.netvirt.elan.utils.ElanUtils.isVxlanNetworkOrVxlanSegment;
16
17 import com.google.common.base.Preconditions;
18 import com.google.common.collect.Lists;
19 import com.google.common.util.concurrent.FluentFuture;
20 import com.google.common.util.concurrent.ListenableFuture;
21 import java.math.BigInteger;
22 import java.util.ArrayList;
23 import java.util.Collections;
24 import java.util.HashSet;
25 import java.util.List;
26 import java.util.Map;
27 import java.util.Objects;
28 import java.util.Optional;
29 import java.util.Queue;
30 import java.util.Set;
31 import java.util.concurrent.ConcurrentHashMap;
32 import java.util.concurrent.ConcurrentLinkedQueue;
33 import java.util.concurrent.ExecutionException;
34 import java.util.concurrent.locks.ReentrantLock;
35 import javax.annotation.PreDestroy;
36 import javax.inject.Inject;
37 import javax.inject.Singleton;
38 import org.apache.commons.lang3.StringUtils;
39 import org.opendaylight.genius.infra.Datastore.Configuration;
40 import org.opendaylight.genius.infra.Datastore.Operational;
41 import org.opendaylight.genius.infra.ManagedNewTransactionRunner;
42 import org.opendaylight.genius.infra.ManagedNewTransactionRunnerImpl;
43 import org.opendaylight.genius.infra.TypedReadWriteTransaction;
44 import org.opendaylight.genius.infra.TypedWriteTransaction;
45 import org.opendaylight.genius.interfacemanager.globals.InterfaceInfo;
46 import org.opendaylight.genius.interfacemanager.interfaces.IInterfaceManager;
47 import org.opendaylight.genius.itm.globals.ITMConstants;
48 import org.opendaylight.genius.mdsalutil.FlowEntity;
49 import org.opendaylight.genius.mdsalutil.FlowEntityBuilder;
50 import org.opendaylight.genius.mdsalutil.InstructionInfo;
51 import org.opendaylight.genius.mdsalutil.MDSALUtil;
52 import org.opendaylight.genius.mdsalutil.MatchInfo;
53 import org.opendaylight.genius.mdsalutil.MatchInfoBase;
54 import org.opendaylight.genius.mdsalutil.MetaDataUtil;
55 import org.opendaylight.genius.mdsalutil.NwConstants;
56 import org.opendaylight.genius.mdsalutil.actions.ActionDrop;
57 import org.opendaylight.genius.mdsalutil.actions.ActionGroup;
58 import org.opendaylight.genius.mdsalutil.actions.ActionNxResubmit;
59 import org.opendaylight.genius.mdsalutil.actions.ActionRegLoad;
60 import org.opendaylight.genius.mdsalutil.actions.ActionSetFieldTunnelId;
61 import org.opendaylight.genius.mdsalutil.instructions.InstructionGotoTable;
62 import org.opendaylight.genius.mdsalutil.instructions.InstructionWriteActions;
63 import org.opendaylight.genius.mdsalutil.instructions.InstructionWriteMetadata;
64 import org.opendaylight.genius.mdsalutil.interfaces.IMdsalApiManager;
65 import org.opendaylight.genius.mdsalutil.matches.MatchMetadata;
66 import org.opendaylight.genius.mdsalutil.matches.MatchTunnelId;
67 import org.opendaylight.genius.utils.JvmGlobalLocks;
68 import org.opendaylight.genius.utils.ServiceIndex;
69 import org.opendaylight.infrautils.jobcoordinator.JobCoordinator;
70 import org.opendaylight.infrautils.utils.concurrent.Executors;
71 import org.opendaylight.infrautils.utils.concurrent.LoggingFutures;
72 import org.opendaylight.infrautils.utils.concurrent.NamedSimpleReentrantLock.Acquired;
73 import org.opendaylight.mdsal.binding.api.DataBroker;
74 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
75 import org.opendaylight.netvirt.elan.cache.ElanInstanceCache;
76 import org.opendaylight.netvirt.elan.cache.ElanInterfaceCache;
77 import org.opendaylight.netvirt.elan.l2gw.utils.ElanL2GatewayMulticastUtils;
78 import org.opendaylight.netvirt.elan.l2gw.utils.ElanL2GatewayUtils;
79 import org.opendaylight.netvirt.elan.recovery.impl.ElanServiceRecoveryHandler;
80 import org.opendaylight.netvirt.elan.utils.ElanConstants;
81 import org.opendaylight.netvirt.elan.utils.ElanEtreeUtils;
82 import org.opendaylight.netvirt.elan.utils.ElanForwardingEntriesHandler;
83 import org.opendaylight.netvirt.elan.utils.ElanItmUtils;
84 import org.opendaylight.netvirt.elan.utils.ElanUtils;
85 import org.opendaylight.netvirt.elanmanager.api.ElanHelper;
86 import org.opendaylight.netvirt.elanmanager.utils.ElanL2GwCacheUtils;
87 import org.opendaylight.netvirt.neutronvpn.api.l2gw.L2GatewayDevice;
88 import org.opendaylight.netvirt.neutronvpn.api.utils.NeutronUtils;
89 import org.opendaylight.netvirt.neutronvpn.interfaces.INeutronVpnManager;
90 import org.opendaylight.serviceutils.srm.RecoverableListener;
91 import org.opendaylight.serviceutils.srm.ServiceRecoveryRegistry;
92 import org.opendaylight.serviceutils.tools.listener.AbstractAsyncDataTreeChangeListener;
93 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.inet.types.rev130715.IpAddress;
94 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.interfaces.rev140508.interfaces.state.Interface;
95 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.yang.types.rev130715.PhysAddress;
96 import org.opendaylight.yang.gen.v1.urn.opendaylight.action.types.rev131112.action.list.Action;
97 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
98 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowId;
99 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.table.Flow;
100 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.table.FlowBuilder;
101 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.types.rev131026.instruction.list.Instruction;
102 import org.opendaylight.yang.gen.v1.urn.opendaylight.genius.idmanager.rev160406.IdManagerService;
103 import org.opendaylight.yang.gen.v1.urn.opendaylight.genius.interfacemanager.servicebinding.rev160406.service.bindings.services.info.BoundServices;
104 import org.opendaylight.yang.gen.v1.urn.opendaylight.genius.itm.op.rev160406.external.tunnel.list.ExternalTunnel;
105 import org.opendaylight.yang.gen.v1.urn.opendaylight.group.types.rev131018.GroupId;
106 import org.opendaylight.yang.gen.v1.urn.opendaylight.group.types.rev131018.GroupTypes;
107 import org.opendaylight.yang.gen.v1.urn.opendaylight.group.types.rev131018.group.buckets.Bucket;
108 import org.opendaylight.yang.gen.v1.urn.opendaylight.group.types.rev131018.groups.Group;
109 import org.opendaylight.yang.gen.v1.urn.opendaylight.group.types.rev131018.groups.GroupKey;
110 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.Nodes;
111 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.nodes.Node;
112 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.nodes.NodeKey;
113 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.etree.rev160614.EtreeInstance;
114 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.etree.rev160614.EtreeInterface;
115 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.etree.rev160614.EtreeInterface.EtreeInterfaceType;
116 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.etree.rev160614.EtreeLeafTagName;
117 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.ElanDpnInterfaces;
118 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.ElanForwardingTables;
119 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.ElanInterfaces;
120 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.elan._interface.forwarding.entries.ElanInterfaceMac;
121 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.elan.dpn.interfaces.ElanDpnInterfacesList;
122 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.elan.dpn.interfaces.elan.dpn.interfaces.list.DpnInterfaces;
123 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.elan.dpn.interfaces.elan.dpn.interfaces.list.DpnInterfacesBuilder;
124 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.elan.dpn.interfaces.elan.dpn.interfaces.list.DpnInterfacesKey;
125 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.elan.forwarding.tables.MacTable;
126 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.elan.forwarding.tables.MacTableKey;
127 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.elan.instances.ElanInstance;
128 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.elan.instances.ElanInstanceBuilder;
129 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.elan.interfaces.ElanInterface;
130 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.elan.interfaces.elan._interface.StaticMacEntries;
131 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.elan.state.Elan;
132 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.elan.state.ElanBuilder;
133 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.elan.state.ElanKey;
134 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.forwarding.entries.MacEntry;
135 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.forwarding.entries.MacEntryBuilder;
136 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.forwarding.entries.MacEntryKey;
137 import org.opendaylight.yang.gen.v1.urn.opendaylight.neutron.ports.rev150712.ports.attributes.ports.Port;
138 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflowjava.nx.match.rev140421.NxmNxReg1;
139 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.hwvtep.global.attributes.RemoteMcastMacs;
140 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.NodeId;
141 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
142 import org.opendaylight.yangtools.yang.common.Uint32;
143 import org.opendaylight.yangtools.yang.common.Uint64;
144 import org.slf4j.Logger;
145 import org.slf4j.LoggerFactory;
146
147 /**
148  * Class in charge of handling creations, modifications and removals of
149  * ElanInterfaces.
150  *
151  * @see org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.elan.rev150602.elan.interfaces.ElanInterface
152  */
153 @Singleton
154 public class ElanInterfaceManager extends AbstractAsyncDataTreeChangeListener<ElanInterface>
155         implements RecoverableListener {
156     private static final Logger LOG = LoggerFactory.getLogger(ElanInterfaceManager.class);
157     private static final Logger EVENT_LOGGER = LoggerFactory.getLogger("NetvirtEventLogger");
158     public static final long WAIT_TIME_FOR_SYNC_INSTALL = Long.getLong("wait.time.sync.install", 300L);
159     private static final boolean SH_FLAG_SET = true;
160     private static final boolean SH_FLAG_UNSET = false;
161
162     private final DataBroker broker;
163     private final ManagedNewTransactionRunner txRunner;
164     private final IMdsalApiManager mdsalManager;
165     private final IInterfaceManager interfaceManager;
166     private final IdManagerService idManager;
167     private final ElanForwardingEntriesHandler elanForwardingEntriesHandler;
168     private final INeutronVpnManager neutronVpnManager;
169     private final ElanItmUtils elanItmUtils;
170     private final ElanEtreeUtils elanEtreeUtils;
171     private final ElanL2GatewayUtils elanL2GatewayUtils;
172     private final ElanL2GatewayMulticastUtils elanL2GatewayMulticastUtils;
173     private final ElanUtils elanUtils;
174     private final JobCoordinator jobCoordinator;
175     private final ElanInstanceCache elanInstanceCache;
176     private final ElanInterfaceCache elanInterfaceCache;
177     private final ElanGroupCache elanGroupCache;
178
179     private final Map<String, ConcurrentLinkedQueue<ElanInterface>>
180         unProcessedElanInterfaces = new ConcurrentHashMap<>();
181
182     @Inject
183     public ElanInterfaceManager(final DataBroker dataBroker, final IdManagerService managerService,
184                                 final IMdsalApiManager mdsalApiManager, IInterfaceManager interfaceManager,
185                                 final ElanForwardingEntriesHandler elanForwardingEntriesHandler,
186                                 final INeutronVpnManager neutronVpnManager, final ElanItmUtils elanItmUtils,
187                                 final ElanEtreeUtils elanEtreeUtils, final ElanL2GatewayUtils elanL2GatewayUtils,
188                                 final ElanUtils elanUtils, final JobCoordinator jobCoordinator,
189                                 final ElanL2GatewayMulticastUtils elanL2GatewayMulticastUtils,
190                                 final ElanInstanceCache elanInstanceCache,
191                                 final ElanInterfaceCache elanInterfaceCache,
192                                 final ElanServiceRecoveryHandler elanServiceRecoveryHandler,
193                                 ElanGroupCache elanGroupCache,
194                                 final ServiceRecoveryRegistry serviceRecoveryRegistry) {
195         super(dataBroker, LogicalDatastoreType.CONFIGURATION, InstanceIdentifier.create(ElanInterfaces.class)
196                 .child(ElanInterface.class),
197                 Executors.newListeningSingleThreadExecutor("ElanInterfaceManager", LOG));
198         this.broker = dataBroker;
199         this.txRunner = new ManagedNewTransactionRunnerImpl(dataBroker);
200         this.idManager = managerService;
201         this.mdsalManager = mdsalApiManager;
202         this.interfaceManager = interfaceManager;
203         this.elanForwardingEntriesHandler = elanForwardingEntriesHandler;
204         this.neutronVpnManager = neutronVpnManager;
205         this.elanItmUtils = elanItmUtils;
206         this.elanEtreeUtils = elanEtreeUtils;
207         this.elanL2GatewayUtils = elanL2GatewayUtils;
208         this.elanUtils = elanUtils;
209         this.jobCoordinator = jobCoordinator;
210         this.elanL2GatewayMulticastUtils = elanL2GatewayMulticastUtils;
211         this.elanInstanceCache = elanInstanceCache;
212         this.elanInterfaceCache = elanInterfaceCache;
213         this.elanGroupCache = elanGroupCache;
214         serviceRecoveryRegistry.addRecoverableListener(elanServiceRecoveryHandler.buildServiceRegistryKey(), this);
215     }
216
217     public void init() {
218         LOG.info("{} registered", getClass().getSimpleName());
219     }
220
221     @Override
222     public void registerListener() {
223         super.register();
224         LOG.info("Registering ElanInterfaceManager");
225     }
226
227     @Override
228     public void deregisterListener() {
229         super.close();
230         LOG.info("Deregistering ElanInterfaceManager");
231     }
232
233     @Override
234     @PreDestroy
235     public void close() {
236         super.close();
237         Executors.shutdownAndAwaitTermination(getExecutorService());
238     }
239
240     @Override
241     public void remove(InstanceIdentifier<ElanInterface> identifier, ElanInterface del) {
242         String interfaceName = del.getName();
243         String elanInstanceName = del.getElanInstanceName();
244         EVENT_LOGGER.debug("ELAN-Interface, REMOVE {} Instance {}", interfaceName, elanInstanceName);
245         Queue<ElanInterface> elanInterfaces = unProcessedElanInterfaces.get(elanInstanceName);
246         if (elanInterfaces != null && elanInterfaces.contains(del)) {
247             elanInterfaces.remove(del);
248             if (elanInterfaces.isEmpty()) {
249                 unProcessedElanInterfaces.remove(elanInstanceName);
250             }
251         }
252         ElanInstance elanInfo = elanInstanceCache.get(elanInstanceName).orElse(null);
253         /*
254          * Handling in case the elan instance is deleted.If the Elan instance is
255          * deleted, there is no need to explicitly delete the elan interfaces
256          */
257         if (elanInfo == null) {
258             return;
259         }
260         InterfaceInfo interfaceInfo = interfaceManager.getInterfaceInfo(interfaceName);
261         if (interfaceInfo == null && elanInfo.isExternal()) {
262             // In deleting external network, the underlying ietf Inteface might have been removed
263             // from the config DS prior to deleting the ELAN interface. We try to get the InterfaceInfo
264             // from Operational DS instead
265             interfaceInfo = interfaceManager.getInterfaceInfoFromOperationalDataStore(interfaceName);
266         }
267         InterfaceRemoveWorkerOnElan configWorker = new InterfaceRemoveWorkerOnElan(elanInstanceName, elanInfo,
268                 interfaceName, interfaceInfo, this);
269         jobCoordinator.enqueueJob(elanInstanceName, configWorker, ElanConstants.JOB_MAX_RETRIES);
270     }
271
272     private static class RemoveElanInterfaceHolder {
273         boolean isLastElanInterface = false;
274         Uint64 dpId = null;
275     }
276
277     @SuppressWarnings("checkstyle:ForbidCertainMethod")
278     public List<ListenableFuture<Void>> removeElanInterface(ElanInstance elanInfo, String interfaceName,
279             InterfaceInfo interfaceInfo) {
280         String elanName = elanInfo.getElanInstanceName();
281         EVENT_LOGGER.debug("ELAN-InterfaceState, REMOVE {} Instance {}", interfaceName, elanName);
282         Uint32 elanTag = elanInfo.getElanTag();
283         // We use two transaction so we don't suffer on multiple shards (interfaces and flows)
284         List<ListenableFuture<Void>> futures = new ArrayList<>();
285         RemoveElanInterfaceHolder holder = new RemoveElanInterfaceHolder();
286         futures.add(txRunner.callWithNewReadWriteTransactionAndSubmit(OPERATIONAL, interfaceTx -> {
287             Elan elanState = removeElanStateForInterface(elanInfo, interfaceName, interfaceTx);
288             if (elanState == null) {
289                 return;
290             }
291             futures.add(txRunner.callWithNewReadWriteTransactionAndSubmit(CONFIGURATION, flowTx -> {
292                 List<String> elanInterfaces = elanState.getElanInterfaces();
293                 if (elanInterfaces == null || elanInterfaces.isEmpty()) {
294                     holder.isLastElanInterface = true;
295                 }
296                 if (interfaceInfo != null) {
297                     holder.dpId = interfaceInfo.getDpId();
298                     DpnInterfaces dpnInterfaces = removeElanDpnInterfaceFromOperationalDataStore(elanName, holder.dpId,
299                         interfaceName, elanTag, interfaceTx);
300                     /*
301                      * If there are not elan ports, remove the unknown dmac, terminating
302                      * service table flows, remote/local bc group
303                      */
304                     if (dpnInterfaces == null || dpnInterfaces.getInterfaces() == null
305                         || dpnInterfaces.getInterfaces().isEmpty()) {
306                         // No more Elan Interfaces in this DPN
307                         EVENT_LOGGER.debug("ELAN-Flows, REMOVE {} Instance {}", interfaceName, elanName);
308                         LOG.debug("deleting the elan: {} present on dpId: {}", elanInfo.getElanInstanceName(),
309                             holder.dpId);
310                         if (!elanUtils.isOpenstackVniSemanticsEnforced()) {
311                             removeDefaultTermFlow(holder.dpId, elanInfo.getElanTag().toJava());
312                         }
313                         removeUnknownDmacFlow(holder.dpId, elanInfo, flowTx, elanInfo.getElanTag().toJava());
314                         removeEtreeUnknownDmacFlow(holder.dpId, elanInfo, flowTx);
315                         removeElanBroadcastGroup(elanInfo, interfaceInfo, flowTx);
316                         removeLocalBroadcastGroup(elanInfo, interfaceInfo, flowTx);
317                         removeEtreeBroadcastGrups(elanInfo, interfaceInfo, flowTx);
318                         if (isVxlanNetworkOrVxlanSegment(elanInfo)) {
319                             if (elanUtils.isOpenstackVniSemanticsEnforced()) {
320                                 elanUtils.removeTerminatingServiceAction(holder.dpId,
321                                     ElanUtils.getVxlanSegmentationId(elanInfo).intValue());
322                             }
323                             unsetExternalTunnelTable(holder.dpId, elanInfo, flowTx);
324                         }
325                     } else {
326                         setupLocalBroadcastGroups(elanInfo, dpnInterfaces, interfaceInfo, flowTx);
327                     }
328                 }
329             }));
330         }));
331         futures.forEach(ElanUtils::waitForTransactionToComplete);
332
333         InterfaceRemoveWorkerOnElanInterface removeInterfaceWorker = new InterfaceRemoveWorkerOnElanInterface(
334                 interfaceName, elanInfo, interfaceInfo, this, holder.isLastElanInterface);
335         jobCoordinator.enqueueJob(ElanUtils.getElanInterfaceJobKey(interfaceName), removeInterfaceWorker,
336                 ElanConstants.JOB_MAX_RETRIES);
337
338         return futures;
339     }
340
341     private void removeEtreeUnknownDmacFlow(Uint64 dpId, ElanInstance elanInfo,
342             TypedReadWriteTransaction<Configuration> deleteFlowGroupTx)
343             throws ExecutionException, InterruptedException {
344         EtreeLeafTagName etreeLeafTag = elanEtreeUtils.getEtreeLeafTagByElanTag(elanInfo.getElanTag().toJava());
345         if (etreeLeafTag != null) {
346             long leafTag = etreeLeafTag.getEtreeLeafTag().getValue().toJava();
347             removeUnknownDmacFlow(dpId, elanInfo, deleteFlowGroupTx, leafTag);
348         }
349     }
350
351     private void removeEtreeBroadcastGrups(ElanInstance elanInfo, InterfaceInfo interfaceInfo,
352             TypedReadWriteTransaction<Configuration> deleteFlowGroupTx)
353             throws ExecutionException, InterruptedException {
354         removeLeavesEtreeBroadcastGroup(elanInfo, interfaceInfo, deleteFlowGroupTx);
355         removeLeavesLocalBroadcastGroup(elanInfo, interfaceInfo, deleteFlowGroupTx);
356     }
357
358     private void removeLeavesLocalBroadcastGroup(ElanInstance elanInfo, InterfaceInfo interfaceInfo,
359             TypedReadWriteTransaction<Configuration> deleteFlowGroupTx)
360             throws ExecutionException, InterruptedException {
361         EtreeInstance etreeInstance = elanInfo.augmentation(EtreeInstance.class);
362         if (etreeInstance != null) {
363             Uint64 dpnId = interfaceInfo.getDpId();
364             long groupId = ElanUtils.getEtreeLeafLocalBCGId(etreeInstance.getEtreeLeafTagVal().getValue().toJava());
365             LOG.trace("deleted the localBroadCast Group:{}", groupId);
366             mdsalManager.removeGroup(deleteFlowGroupTx, dpnId, groupId);
367         }
368     }
369
370     private void removeLeavesEtreeBroadcastGroup(ElanInstance elanInfo, InterfaceInfo interfaceInfo,
371             TypedReadWriteTransaction<Configuration> deleteFlowGroupTx)
372             throws ExecutionException, InterruptedException {
373         EtreeInstance etreeInstance = elanInfo.augmentation(EtreeInstance.class);
374         if (etreeInstance != null) {
375             long etreeTag = etreeInstance.getEtreeLeafTagVal().getValue().toJava();
376             Uint64 dpnId = interfaceInfo.getDpId();
377             long groupId = ElanUtils.getEtreeLeafRemoteBCGId(etreeTag);
378             LOG.trace("deleting the remoteBroadCast group:{}", groupId);
379             mdsalManager.removeGroup(deleteFlowGroupTx, dpnId, groupId);
380         }
381     }
382
383     private static Elan removeElanStateForInterface(ElanInstance elanInfo, String interfaceName,
384             TypedReadWriteTransaction<Operational> tx) throws ExecutionException, InterruptedException {
385         String elanName = elanInfo.getElanInstanceName();
386         Elan elanState = ElanUtils.getElanByName(tx, elanName);
387         if (elanState == null) {
388             return elanState;
389         }
390         List<String> existingElanInterfaces = elanState.getElanInterfaces();
391         List<String> elanInterfaces = new ArrayList<>();
392         if (existingElanInterfaces != null) {
393             elanInterfaces.addAll(existingElanInterfaces);
394         }
395         boolean isRemoved = elanInterfaces.remove(interfaceName);
396         if (!isRemoved) {
397             return elanState;
398         }
399
400         if (elanInterfaces.isEmpty()) {
401             tx.delete(ElanUtils.getElanInstanceOperationalDataPath(elanName));
402             tx.delete(ElanUtils.getElanMacTableOperationalDataPath(elanName));
403             tx.delete(ElanUtils.getElanInfoEntriesOperationalDataPath(elanInfo.getElanTag()));
404             tx.delete(ElanUtils.getElanDpnOperationDataPath(elanName));
405         } else {
406             Elan updateElanState = new ElanBuilder().setElanInterfaces(elanInterfaces).setName(elanName)
407                     .withKey(new ElanKey(elanName)).build();
408             tx.put(ElanUtils.getElanInstanceOperationalDataPath(elanName), updateElanState);
409         }
410         return elanState;
411     }
412
413     private void deleteElanInterfaceFromConfigDS(String interfaceName, TypedReadWriteTransaction<Configuration> tx)
414             throws ExecutionException, InterruptedException {
415         // removing the ElanInterface from the config data_store if interface is
416         // not present in Interface config DS
417         InstanceIdentifier<ElanInterface> elanInterfaceId = ElanUtils
418                 .getElanInterfaceConfigurationDataPathId(interfaceName);
419         FluentFuture<Optional<ElanInterface>> interfaceOptional = tx.read(elanInterfaceId);
420         if (!interfaceOptional.get().isPresent() && elanInterfaceCache.get(interfaceName).isPresent()) {
421             tx.delete(ElanUtils.getElanInterfaceConfigurationDataPathId(interfaceName));
422         }
423     }
424
425     List<ListenableFuture<Void>> removeEntriesForElanInterface(ElanInstance elanInfo, InterfaceInfo
426             interfaceInfo, String interfaceName, boolean isLastElanInterface) {
427         String elanName = elanInfo.getElanInstanceName();
428         EVENT_LOGGER.debug("ELAN-InterfaceEntries, REMOVE {} Instance {}", interfaceName, elanName);
429         List<ListenableFuture<Void>> futures = new ArrayList<>();
430         futures.add(txRunner.callWithNewReadWriteTransactionAndSubmit(CONFIGURATION, flowTx -> {
431             futures.add(txRunner.callWithNewReadWriteTransactionAndSubmit(OPERATIONAL, interfaceTx -> {
432                 InstanceIdentifier<ElanInterfaceMac> elanInterfaceId = ElanUtils
433                         .getElanInterfaceMacEntriesOperationalDataPath(interfaceName);
434                 Optional<ElanInterfaceMac> existingElanInterfaceMac = interfaceTx.read(elanInterfaceId).get();
435                 LOG.debug("Removing the Interface:{} from elan:{}", interfaceName, elanName);
436                 if (interfaceInfo != null) {
437                     if (existingElanInterfaceMac.isPresent()) {
438                         List<MacEntry> existingMacEntries = existingElanInterfaceMac.get().getMacEntry();
439                         if (existingMacEntries != null) {
440                             List<PhysAddress> macAddresses = new ArrayList<>();
441                             for (MacEntry macEntry : existingMacEntries) {
442                                 PhysAddress macAddress = macEntry.getMacAddress();
443                                 LOG.debug("removing the  mac-entry:{} present on elanInterface:{}",
444                                         macAddress.getValue(), interfaceName);
445                                 Optional<MacEntry> macEntryOptional =
446                                         elanUtils.getMacEntryForElanInstance(interfaceTx, elanName, macAddress);
447                                 if (!isLastElanInterface && macEntryOptional.isPresent()) {
448                                     interfaceTx.delete(ElanUtils.getMacEntryOperationalDataPath(elanName, macAddress));
449                                 }
450                                 EVENT_LOGGER.debug("ELAN-MacFlows, REMOVE {} Instance {} Mac {}",
451                                         interfaceName, elanName, macAddress);
452                                 elanUtils.deleteMacFlows(elanInfo, interfaceInfo, macEntry, flowTx);
453                                 macAddresses.add(macAddress);
454                             }
455
456                             // Removing all those MACs from External Devices belonging
457                             // to this ELAN
458                             if (isVxlanNetworkOrVxlanSegment(elanInfo) && ! macAddresses.isEmpty()) {
459                                 elanL2GatewayUtils.removeMacsFromElanExternalDevices(elanInfo, macAddresses);
460                             }
461                         }
462                     }
463                     removeDefaultTermFlow(interfaceInfo.getDpId(), interfaceInfo.getInterfaceTag());
464                     removeFilterEqualsTable(elanInfo, interfaceInfo, flowTx);
465                 } else if (existingElanInterfaceMac.isPresent()) {
466                     // Interface does not exist in ConfigDS, so lets remove everything
467                     // about that interface related to Elan
468                     List<MacEntry> macEntries = existingElanInterfaceMac.get().getMacEntry();
469                     if (macEntries != null) {
470                         for (MacEntry macEntry : macEntries) {
471                             PhysAddress macAddress = macEntry.getMacAddress();
472                             if (elanUtils.getMacEntryForElanInstance(elanName, macAddress).isPresent()) {
473                                 interfaceTx.delete(ElanUtils.getMacEntryOperationalDataPath(elanName, macAddress));
474                             }
475                         }
476                     }
477                 }
478                 if (existingElanInterfaceMac.isPresent()) {
479                     interfaceTx.delete(elanInterfaceId);
480                 }
481                 unbindService(interfaceName, flowTx);
482                 deleteElanInterfaceFromConfigDS(interfaceName, flowTx);
483             }));
484         }));
485         return futures;
486     }
487
488     private DpnInterfaces removeElanDpnInterfaceFromOperationalDataStore(String elanName, Uint64 dpId,
489                                                                          String interfaceName, Uint32 elanTag,
490                                                                          TypedReadWriteTransaction<Operational> tx)
491             throws ExecutionException, InterruptedException {
492         // FIXME: pass in and use ElanInstanceKey instead?
493         final ReentrantLock lock = JvmGlobalLocks.getLockForString(elanName);
494         lock.lock();
495         try {
496             DpnInterfaces dpnInterfaces = elanUtils.getElanInterfaceInfoByElanDpn(elanName, dpId);
497             if (dpnInterfaces != null) {
498                 List<String> interfaceLists = null;
499                 if (dpnInterfaces.getInterfaces() != null) {
500                     interfaceLists = new ArrayList<>(dpnInterfaces.getInterfaces());
501                 }
502                 if (interfaceLists != null) {
503                     interfaceLists.remove(interfaceName);
504                 }
505
506                 if (interfaceLists == null || interfaceLists.isEmpty()) {
507                     deleteAllRemoteMacsInADpn(elanName, dpId, elanTag);
508                     deleteElanDpnInterface(elanName, dpId, tx);
509                 } else {
510                     dpnInterfaces = updateElanDpnInterfacesList(elanName, dpId, interfaceLists, tx);
511                 }
512             }
513             return dpnInterfaces;
514         } finally {
515             lock.unlock();
516         }
517     }
518
519     private void deleteAllRemoteMacsInADpn(String elanName, Uint64 dpId, Uint32 elanTag) {
520         List<DpnInterfaces> dpnInterfaces = elanUtils.getInvolvedDpnsInElan(elanName);
521         addErrorLogging(txRunner.callWithNewReadWriteTransactionAndSubmit(CONFIGURATION, confTx -> {
522             for (DpnInterfaces dpnInterface : dpnInterfaces) {
523                 Uint64 currentDpId = dpnInterface.getDpId();
524                 if (!currentDpId.equals(dpId) && dpnInterface.getInterfaces() != null) {
525                     for (String elanInterface : dpnInterface.getInterfaces()) {
526                         ElanInterfaceMac macs = elanUtils.getElanInterfaceMacByInterfaceName(elanInterface);
527                         if (macs == null || macs.getMacEntry() == null) {
528                             continue;
529                         }
530                         for (MacEntry mac : macs.getMacEntry()) {
531                             removeTheMacFlowInTheDPN(dpId, elanTag, mac, confTx);
532                             removeEtreeMacFlowInTheDPN(dpId, elanTag, mac, confTx);
533                         }
534                     }
535                 }
536             }
537         }), LOG, "Error deleting remote MACs in DPN {}", dpId);
538     }
539
540     private void removeEtreeMacFlowInTheDPN(Uint64 dpId, Uint32 elanTag, MacEntry mac,
541             TypedReadWriteTransaction<Configuration> confTx) throws ExecutionException, InterruptedException {
542         EtreeLeafTagName etreeLeafTag = elanEtreeUtils.getEtreeLeafTagByElanTag(elanTag.longValue());
543         if (etreeLeafTag != null) {
544             removeTheMacFlowInTheDPN(dpId, etreeLeafTag.getEtreeLeafTag().getValue(), mac, confTx);
545         }
546     }
547
548     private void removeTheMacFlowInTheDPN(Uint64 dpId, Uint32 elanTag, MacEntry mac,
549             TypedReadWriteTransaction<Configuration> confTx) throws ExecutionException, InterruptedException {
550         mdsalManager
551                 .removeFlow(confTx, dpId,
552                         MDSALUtil.buildFlow(NwConstants.ELAN_DMAC_TABLE,
553                                 ElanUtils.getKnownDynamicmacFlowRef(elanTag, mac.getMacAddress().getValue())));
554     }
555
556     /*
557     * Possible Scenarios for update
558     *   a. if orig={1,2,3,4}   and updated=null or updated={}
559         then all {1,2,3,4} should be removed
560
561         b. if orig=null or orig={}  and  updated ={1,2,3,4}
562         then all {1,2,3,4} should be added
563
564         c. if orig = {1,2,3,4} updated={2,3,4}
565         then 1 should be removed
566
567         d. basically if orig = { 1,2,3,4} and updated is {1,2,3,4,5}
568         then we should just add 5
569
570         e. if orig = {1,2,3,4} updated={2,3,4,5}
571         then 1 should be removed , 5 should be added
572     * */
573     @SuppressWarnings("checkstyle:ForbidCertainMethod")
574     @Override
575     public void update(InstanceIdentifier<ElanInterface> identifier, ElanInterface original, ElanInterface update) {
576         // updating the static-Mac Entries for the existing elanInterface
577         String elanName = update.getElanInstanceName();
578         String interfaceName = update.getName();
579         LOG.info("Update static mac entries for elan interface {} in elan instance {}", interfaceName, elanName);
580         EVENT_LOGGER.debug("ELAN-Interface, UPDATE {} Instance {}", original.getName(), elanName);
581
582         List<StaticMacEntries> originalStaticMacEntries = original.getStaticMacEntries();
583         List<StaticMacEntries> updatedStaticMacEntries = update.getStaticMacEntries();
584         List<StaticMacEntries> deletedEntries = ElanUtils.diffOf(originalStaticMacEntries, updatedStaticMacEntries);
585         List<StaticMacEntries> updatedEntries = ElanUtils.diffOf(updatedStaticMacEntries, originalStaticMacEntries);
586
587         deletedEntries.forEach((deletedEntry) -> removeInterfaceStaticMacEntries(elanName, interfaceName,
588                 deletedEntry.getMacAddress()));
589
590         /*if updatedStaticMacEntries is NOT NULL, which means as part of update call these entries were added.
591         * Hence add the macentries for the same.*/
592         for (StaticMacEntries staticMacEntry : updatedEntries) {
593             InstanceIdentifier<MacEntry> macEntryIdentifier = getMacEntryOperationalDataPath(elanName,
594                     staticMacEntry.getMacAddress());
595             addErrorLogging(ElanUtils.waitForTransactionToComplete(
596                 txRunner.callWithNewReadWriteTransactionAndSubmit(OPERATIONAL, tx -> {
597                     Optional<MacEntry> existingMacEntry = tx.read(macEntryIdentifier).get();
598                     if (existingMacEntry.isPresent()) {
599                         LOG.debug("updating elan interface forwarding table for mac entry {} elan instance {}",
600                                 existingMacEntry.get(), elanName);
601                         elanForwardingEntriesHandler.updateElanInterfaceForwardingTablesList(
602                             elanName, interfaceName, existingMacEntry.get().getInterface(), existingMacEntry.get(),
603                             tx);
604                     } else {
605                         LOG.info("Adding elan interface forwarding table for mac entry {} elan interface"
606                                 + " {} elan instance {}.", staticMacEntry.getMacAddress(), interfaceName, elanName);
607                         elanForwardingEntriesHandler.addElanInterfaceForwardingTableList(
608                             elanName, interfaceName, staticMacEntry, tx);
609                     }
610                 })), LOG, "Error in update: identifier={}, original={}, update={}", identifier, original, update);
611         }
612     }
613
614     @Override
615     public void add(InstanceIdentifier<ElanInterface> identifier, ElanInterface elanInterfaceAdded) {
616         LOG.info("Init for ELAN interface Add {}", elanInterfaceAdded);
617         addErrorLogging(txRunner.callWithNewWriteOnlyTransactionAndSubmit(OPERATIONAL, operTx -> {
618             String elanInstanceName = elanInterfaceAdded.getElanInstanceName();
619             String interfaceName = elanInterfaceAdded.getName();
620             EVENT_LOGGER.debug("ELAN-Interface, ADD {} Instance {}", interfaceName, elanInstanceName);
621             InterfaceInfo interfaceInfo = interfaceManager.getInterfaceInfo(interfaceName);
622             if (interfaceInfo == null) {
623                 LOG.info("Interface {} is removed from Interface Oper DS due to port down ", interfaceName);
624                 return;
625             }
626             ElanInstance elanInstance = elanInstanceCache.get(elanInstanceName).orElse(null);
627
628             if (elanInstance == null) {
629                 // Add the ElanInstance in the Configuration data-store
630                 List<String> elanInterfaces = new ArrayList<>();
631                 elanInterfaces.add(interfaceName);
632                 elanInstance = txRunner.applyWithNewReadWriteTransactionAndSubmit(CONFIGURATION,
633                     confTx -> ElanUtils.updateOperationalDataStore(idManager,
634                         new ElanInstanceBuilder().setElanInstanceName(elanInstanceName).setDescription(
635                             elanInterfaceAdded.getDescription()).build(), elanInterfaces, confTx, operTx)).get();
636             }
637
638             Uint32 elanTag = elanInstance.getElanTag();
639             // If elan tag is not updated, then put the elan interface into
640             // unprocessed entry map and entry. Let entries
641             // in this map get processed during ELAN update DCN.
642             if (elanTag == null || elanTag.longValue() == 0L) {
643                 ConcurrentLinkedQueue<ElanInterface> elanInterfaces = unProcessedElanInterfaces.get(elanInstanceName);
644                 if (elanInterfaces == null) {
645                     elanInterfaces = new ConcurrentLinkedQueue<>();
646                 }
647                 if (!elanInterfaces.contains(elanInterfaceAdded)) {
648                     elanInterfaces.add(elanInterfaceAdded);
649                 }
650                 LOG.error("ELAN tag for instance {} is not created. Adding it to unprocessed list."
651                         + " Recreate the network if this message is seen multiple times", elanInstanceName);
652                 unProcessedElanInterfaces.put(elanInstanceName, elanInterfaces);
653                 return;
654             }
655             InterfaceAddWorkerOnElan addWorker = new InterfaceAddWorkerOnElan(elanInstanceName, elanInterfaceAdded,
656                     interfaceInfo, elanInstance, this);
657             jobCoordinator.enqueueJob(elanInstanceName, addWorker, ElanConstants.JOB_MAX_RETRIES);
658         }), LOG, "Error processing added ELAN interface");
659     }
660
661     List<ListenableFuture<Void>> handleunprocessedElanInterfaces(ElanInstance elanInstance) {
662         LOG.trace("Handling unprocessed elan interfaces for elan instance {}", elanInstance.getElanInstanceName());
663         List<ListenableFuture<Void>> futures = new ArrayList<>();
664         Queue<ElanInterface> elanInterfaces = unProcessedElanInterfaces.get(elanInstance.getElanInstanceName());
665         if (elanInterfaces == null || elanInterfaces.isEmpty()) {
666             return futures;
667         }
668         for (ElanInterface elanInterface : elanInterfaces) {
669             String interfaceName = elanInterface.getName();
670             InterfaceInfo interfaceInfo = interfaceManager.getInterfaceInfo(interfaceName);
671             futures.addAll(addElanInterface(elanInterface, interfaceInfo, elanInstance));
672         }
673         unProcessedElanInterfaces.remove(elanInstance.getElanInstanceName());
674         LOG.info("Unprocessed elan interfaces for elan instance {} has been removed.",
675                 elanInstance.getElanInstanceName());
676         return futures;
677     }
678
679     void programRemoteDmacFlow(ElanInstance elanInstance, InterfaceInfo interfaceInfo,
680             TypedWriteTransaction<Configuration> writeFlowGroupTx) {
681         ElanDpnInterfacesList elanDpnInterfacesList = elanUtils
682                 .getElanDpnInterfacesList(elanInstance.getElanInstanceName());
683         List<DpnInterfaces> dpnInterfaceLists = null;
684         if (elanDpnInterfacesList != null) {
685             dpnInterfaceLists = elanDpnInterfacesList.getDpnInterfaces();
686         }
687         if (dpnInterfaceLists != null && !dpnInterfaceLists.isEmpty()) {
688             Uint64 dstDpId = interfaceInfo.getDpId();
689             processRemoteDmacFlowForInterface(dstDpId, elanInstance, dpnInterfaceLists, writeFlowGroupTx);
690         }
691     }
692
693     private void processRemoteDmacFlowForInterface(Uint64 dstDpId, ElanInstance elanInstance,
694              List<DpnInterfaces> dpnInterfaceLists, TypedWriteTransaction<Configuration> writeFlowGroupTx) {
695         for (DpnInterfaces dpnInterfaces : dpnInterfaceLists) {
696             if (Objects.equals(dpnInterfaces.getDpId(), dstDpId)) {
697                 continue;
698             }
699             List<String> remoteElanInterfaces = dpnInterfaces.getInterfaces();
700             if (remoteElanInterfaces == null || remoteElanInterfaces.isEmpty()) {
701                 continue;
702             }
703             for (String remoteIf : remoteElanInterfaces) {
704                 ElanInterfaceMac elanIfMac = elanUtils.getElanInterfaceMacByInterfaceName(remoteIf);
705                 InterfaceInfo remoteInterface = interfaceManager.getInterfaceInfo(remoteIf);
706                 if (elanIfMac == null || remoteInterface == null) {
707                     continue;
708                 }
709                 List<MacEntry> remoteMacEntries = elanIfMac.nonnullMacEntry();
710                 for (MacEntry macEntry : remoteMacEntries) {
711                     String macAddress = macEntry.getMacAddress().getValue();
712                     LOG.info("Programming remote dmac {} on the newly added DPN {} for elan {}", macAddress,
713                             dstDpId, elanInstance.getElanInstanceName());
714                     elanUtils.setupRemoteDmacFlow(dstDpId, remoteInterface.getDpId(),
715                             remoteInterface.getInterfaceTag(), elanInstance.getElanTag(), macAddress,
716                             elanInstance.getElanInstanceName(), writeFlowGroupTx, remoteIf, elanInstance);
717                 }
718             }
719         }
720     }
721
722     private static class AddElanInterfaceHolder {
723         private DpnInterfaces dpnInterfaces = null;
724         private boolean isFirstInterfaceInDpn = false;
725         private Uint64 dpId;
726     }
727
728     @SuppressWarnings("checkstyle:ForbidCertainMethod")
729     List<ListenableFuture<Void>> addElanInterface(ElanInterface elanInterface,
730             InterfaceInfo interfaceInfo, ElanInstance elanInstance) {
731         Preconditions.checkNotNull(elanInstance, "elanInstance cannot be null");
732         Preconditions.checkNotNull(interfaceInfo, "interfaceInfo cannot be null");
733         Preconditions.checkNotNull(elanInterface, "elanInterface cannot be null");
734
735         String interfaceName = elanInterface.getName();
736         String elanInstanceName = elanInterface.getElanInstanceName();
737         LOG.trace("Adding elan interface: interface name {} , instance name {}", interfaceName, elanInstanceName);
738         EVENT_LOGGER.debug("ELAN-InterfaceState, ADD {} Instance {}", interfaceName, elanInstanceName);
739
740         List<ListenableFuture<Void>> futures = new ArrayList<>();
741         AddElanInterfaceHolder holder = new AddElanInterfaceHolder();
742         futures.add(txRunner.callWithNewReadWriteTransactionAndSubmit(OPERATIONAL, operTx -> {
743             Elan elanInfo = ElanUtils.getElanByName(broker, elanInstanceName);
744             if (elanInfo == null) {
745                 LOG.trace("elanInfo is null for elan instance: {}", elanInstanceName);
746                 List<String> elanInterfaces = new ArrayList<>();
747                 elanInterfaces.add(interfaceName);
748                 futures.add(txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION,
749                     confTx -> ElanUtils.updateOperationalDataStore(idManager, elanInstance, elanInterfaces, confTx,
750                         operTx)));
751             } else {
752                 createElanStateList(elanInstanceName, interfaceName, operTx);
753             }
754             // Specific actions to the DPN where the ElanInterface has been added,
755             // for example, programming the
756             // External tunnel table if needed or adding the ElanInterface to the
757             // DpnInterfaces in the operational DS.
758             holder.dpId = interfaceInfo.getDpId();
759             if (holder.dpId != null && !holder.dpId.equals(ElanConstants.INVALID_DPN)) {
760                 // FIXME: use elanInstaince.key() instead?
761                 final ReentrantLock lock = JvmGlobalLocks.getLockForString(elanInstanceName);
762                 lock.lock();
763                 try {
764                     InstanceIdentifier<DpnInterfaces> elanDpnInterfaces = ElanUtils
765                         .getElanDpnInterfaceOperationalDataPath(elanInstanceName, holder.dpId);
766                     Optional<DpnInterfaces> existingElanDpnInterfaces = operTx.read(elanDpnInterfaces).get();
767                     if (ElanUtils.isVlan(elanInstance)) {
768                         holder.isFirstInterfaceInDpn =  checkIfFirstInterface(interfaceName,
769                             elanInstanceName, existingElanDpnInterfaces);
770                     } else {
771                         holder.isFirstInterfaceInDpn = !existingElanDpnInterfaces.isPresent();
772                     }
773                     if (holder.isFirstInterfaceInDpn) {
774                         // ELAN's 1st ElanInterface added to this DPN
775                         if (!existingElanDpnInterfaces.isPresent()) {
776                             holder.dpnInterfaces =
777                                 createElanInterfacesList(elanInstanceName, interfaceName, holder.dpId, operTx);
778                         } else {
779                             List<String> existingInterfaces = existingElanDpnInterfaces.get().getInterfaces();
780                             List<String> elanInterfaces =
781                                 existingInterfaces != null ? new ArrayList<>(existingInterfaces) : new ArrayList<>();
782                             elanInterfaces.add(interfaceName);
783                             holder.dpnInterfaces = updateElanDpnInterfacesList(elanInstanceName, holder.dpId,
784                                 elanInterfaces, operTx);
785                         }
786                         LOG.debug("1st interface {} for elan {} is added to dpn {}",
787                                 interfaceName, elanInstanceName, holder.dpId);
788                         // The 1st ElanInterface in a DPN must program the Ext Tunnel
789                         // table, but only if Elan has VNI
790                         if (isVxlanNetworkOrVxlanSegment(elanInstance)) {
791                             futures.add(txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION,
792                                 confTx -> setExternalTunnelTable(holder.dpId, elanInstance, confTx)));
793                         }
794                         elanL2GatewayUtils.installElanL2gwDevicesLocalMacsInDpn(holder.dpId, elanInstance,
795                             interfaceName);
796                     } else {
797                         List<String> existingInterfaces = existingElanDpnInterfaces.get().getInterfaces();
798                         List<String> elanInterfaces =
799                             existingInterfaces != null ? new ArrayList<>(existingInterfaces) : new ArrayList<>();
800                         elanInterfaces.add(interfaceName);
801                         if (elanInterfaces.size() == 1) { // 1st dpn interface
802                             elanL2GatewayUtils.installElanL2gwDevicesLocalMacsInDpn(holder.dpId, elanInstance,
803                                 interfaceName);
804                         }
805                         holder.dpnInterfaces =
806                             updateElanDpnInterfacesList(elanInstanceName, holder.dpId, elanInterfaces, operTx);
807                         LOG.debug("Interface {} for elan {} is added to dpn {}",
808                                 interfaceName, elanInstanceName, holder.dpId);
809                     }
810                 } finally {
811                     lock.unlock();
812                 }
813             }
814
815             // add code to install Local/Remote BC group, unknow DMAC entry,
816             // terminating service table flow entry
817             // call bindservice of interfacemanager to create ingress table flow
818             // enty.
819         }));
820         futures.forEach(ElanUtils::waitForTransactionToComplete);
821         futures.add(
822             ElanUtils.waitForTransactionToComplete(txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION,
823                 confTx -> installEntriesForFirstInterfaceonDpn(elanInstance, interfaceInfo, holder.dpnInterfaces,
824                     holder.isFirstInterfaceInDpn, confTx))));
825
826         // add the vlan provider interface to remote BC group for the elan
827         // for internal vlan networks
828         if (ElanUtils.isVlan(elanInstance) && !elanInstance.isExternal()) {
829             if (interfaceManager.isExternalInterface(interfaceName)) {
830                 LOG.debug("adding vlan prv intf {} to elan {} BC group", interfaceName, elanInstanceName);
831                 handleExternalInterfaceEvent(elanInstance, holder.dpnInterfaces, holder.dpId);
832             }
833         }
834         if (holder.isFirstInterfaceInDpn) {
835             // ELAN's 1st ElanInterface added to this DPN
836             LOG.debug("Adding dpn into operational dpn list {}", holder.dpId);
837             futures.add(txRunner.callWithNewReadWriteTransactionAndSubmit(OPERATIONAL, operTx -> {
838                 operTx.put(ElanUtils.getElanDpnInterfaceOperationalDataPath(elanInstanceName, holder.dpId),
839                         holder.dpnInterfaces, CREATE_MISSING_PARENTS);
840             }));
841         } else {
842             LOG.debug("Updated dpn into operational dpn list {}", holder.dpId);
843         }
844
845         scheduleElanInterfaceWorkerAfterRemoteBcGroup(elanInstance, interfaceInfo, holder.dpnInterfaces,
846                 holder.isFirstInterfaceInDpn, elanInterface);
847         return futures;
848     }
849
850     @SuppressWarnings("checkstyle:ForbidCertainMethod")
851     List<ListenableFuture<Void>> setupEntriesForElanInterface(ElanInstance elanInstance,
852             ElanInterface elanInterface, InterfaceInfo interfaceInfo, boolean isFirstInterfaceInDpn) {
853         String elanInstanceName = elanInstance.getElanInstanceName();
854         String interfaceName = elanInterface.getName();
855         List<ListenableFuture<Void>> futures = new ArrayList<>();
856         Uint64 dpId = interfaceInfo.getDpId();
857         boolean isInterfaceOperational = isOperational(interfaceInfo);
858         futures.add(txRunner.callWithNewReadWriteTransactionAndSubmit(CONFIGURATION, confTx -> {
859             futures.add(txRunner.callWithNewReadWriteTransactionAndSubmit(OPERATIONAL, operTx -> {
860                 installEntriesForElanInterface(elanInstance, elanInterface, interfaceInfo,
861                     isFirstInterfaceInDpn, confTx);
862
863                 List<StaticMacEntries> staticMacEntriesList = elanInterface.getStaticMacEntries();
864                 List<PhysAddress> staticMacAddresses = Lists.newArrayList();
865
866                 if (ElanUtils.isNotEmpty(staticMacEntriesList)) {
867                     for (StaticMacEntries staticMacEntry : staticMacEntriesList) {
868                         InstanceIdentifier<MacEntry> macId = getMacEntryOperationalDataPath(elanInstanceName,
869                             staticMacEntry.getMacAddress());
870                         Optional<MacEntry> existingMacEntry = ElanUtils.read(broker,
871                             LogicalDatastoreType.OPERATIONAL, macId);
872                         if (existingMacEntry.isPresent()) {
873                             elanForwardingEntriesHandler.updateElanInterfaceForwardingTablesList(
874                                 elanInstanceName, interfaceName, existingMacEntry.get().getInterface(),
875                                 existingMacEntry.get(), operTx);
876                         } else {
877                             elanForwardingEntriesHandler.addElanInterfaceForwardingTableList(elanInstanceName,
878                                 interfaceName, staticMacEntry, operTx);
879                         }
880
881                         if (isInterfaceOperational) {
882                             // Setting SMAC, DMAC, UDMAC in this DPN and also in other
883                             // DPNs
884                             String macAddress = staticMacEntry.getMacAddress().getValue();
885                             LOG.info(
886                                 "programming smac and dmacs for {} on source and other DPNs for elan {} and interface"
887                                     + " {}",
888                                 macAddress, elanInstanceName, interfaceName);
889                             EVENT_LOGGER.debug("ELAN-MacFlows, ADD {} Instance {} Mac {}",
890                                     interfaceName, elanInstanceName, macAddress);
891                             elanUtils.setupMacFlows(elanInstance, interfaceInfo, ElanConstants.STATIC_MAC_TIMEOUT,
892                                 staticMacEntry.getMacAddress().getValue(), true, confTx);
893                         }
894                     }
895
896                     if (isInterfaceOperational) {
897                         // Add MAC in TOR's remote MACs via OVSDB. Outside of the loop
898                         // on purpose.
899                         for (StaticMacEntries staticMacEntry : staticMacEntriesList) {
900                             staticMacAddresses.add(staticMacEntry.getMacAddress());
901                         }
902                         elanL2GatewayUtils.scheduleAddDpnMacInExtDevices(elanInstance.getElanInstanceName(), dpId,
903                             staticMacAddresses);
904                     }
905                 }
906             }));
907         }));
908         futures.forEach(ElanUtils::waitForTransactionToComplete);
909         if (isInterfaceOperational && !interfaceManager.isExternalInterface(interfaceName)) {
910             //At this point, the interface is operational and D/SMAC flows have been configured, mark the port active
911             try {
912                 Port neutronPort = neutronVpnManager.getNeutronPort(interfaceName);
913                 if (neutronPort != null) {
914                     NeutronUtils.updatePortStatus(interfaceName, NeutronUtils.PORT_STATUS_ACTIVE, broker);
915                 }
916             } catch (IllegalArgumentException ex) {
917                 LOG.trace("Interface: {} is not part of Neutron Network", interfaceName);
918             }
919         }
920         return futures;
921     }
922
923     protected void removeInterfaceStaticMacEntries(String elanInstanceName, String interfaceName,
924             PhysAddress physAddress) {
925         InterfaceInfo interfaceInfo = interfaceManager.getInterfaceInfo(interfaceName);
926         InstanceIdentifier<MacEntry> macId = getMacEntryOperationalDataPath(elanInstanceName, physAddress);
927         Optional<MacEntry> existingMacEntry = ElanUtils.read(broker,
928                 LogicalDatastoreType.OPERATIONAL, macId);
929
930         if (!existingMacEntry.isPresent()) {
931             return;
932         }
933
934         MacEntry macEntry = new MacEntryBuilder().setMacAddress(physAddress).setInterface(interfaceName)
935                 .withKey(new MacEntryKey(physAddress)).build();
936         elanForwardingEntriesHandler.deleteElanInterfaceForwardingEntries(
937                 elanInstanceCache.get(elanInstanceName).orElse(null), interfaceInfo, macEntry);
938     }
939
940     private boolean checkIfFirstInterface(String elanInterface, String elanInstanceName,
941             Optional<DpnInterfaces> existingElanDpnInterfaces) {
942         String routerPortUuid = ElanUtils.getRouterPordIdFromElanInstance(broker, elanInstanceName);
943         if (!existingElanDpnInterfaces.isPresent()) {
944             return true;
945         }
946         if (elanInterface.equals(elanInstanceName) || elanInterface.equals(routerPortUuid)) {
947             return false;
948         }
949         DpnInterfaces dpnInterfaces = existingElanDpnInterfaces.get();
950         int dummyInterfaceCount =  0;
951         List<String> interfaces = dpnInterfaces.getInterfaces();
952         if (interfaces == null) {
953             return true;
954         }
955         if (interfaces.contains(routerPortUuid)) {
956             dummyInterfaceCount++;
957         }
958         if (interfaces.contains(elanInstanceName)) {
959             dummyInterfaceCount++;
960         }
961         return interfaces.size() == dummyInterfaceCount;
962     }
963
964     private static InstanceIdentifier<MacEntry> getMacEntryOperationalDataPath(String elanName,
965             PhysAddress physAddress) {
966         return InstanceIdentifier.builder(ElanForwardingTables.class).child(MacTable.class, new MacTableKey(elanName))
967                 .child(MacEntry.class, new MacEntryKey(physAddress)).build();
968     }
969
970     private void installEntriesForElanInterface(ElanInstance elanInstance, ElanInterface elanInterface,
971             InterfaceInfo interfaceInfo, boolean isFirstInterfaceInDpn, TypedWriteTransaction<Configuration> confTx) {
972         if (!isOperational(interfaceInfo)) {
973             LOG.warn("Interface {} is not operational", elanInterface.getName());
974             EVENT_LOGGER.debug("ELAN-InterfaceEntries, ADD {} Instance {} Interface Status {}, returning",
975                     elanInterface.getName(), elanInstance.getElanInstanceName(), interfaceInfo.getOpState());
976             return;
977         }
978         Uint64 dpId = interfaceInfo.getDpId();
979         if (!elanUtils.isOpenstackVniSemanticsEnforced()) {
980             elanUtils.setupTermDmacFlows(interfaceInfo, mdsalManager, confTx);
981         }
982         setupFilterEqualsTable(elanInstance, interfaceInfo, confTx);
983         if (isFirstInterfaceInDpn) {
984             // Terminating Service , UnknownDMAC Table.
985             // The 1st ELAN Interface in a DPN must program the INTERNAL_TUNNEL_TABLE, but only if the network type
986             // for ELAN Instance is VxLAN
987             if (isVxlanNetworkOrVxlanSegment(elanInstance)) {
988                 setupTerminateServiceTable(elanInstance, dpId, confTx);
989             }
990             setupUnknownDMacTable(elanInstance, dpId, confTx);
991             /*
992              * Install remote DMAC flow. This is required since this DPN is
993              * added later to the elan instance and remote DMACs of other
994              * interfaces in this elan instance are not present in the current
995              * dpn.
996              */
997             if (!interfaceManager.isExternalInterface(interfaceInfo.getInterfaceName())) {
998                 LOG.info("Programming remote dmac flows on the newly connected dpn {} for elan {} ", dpId,
999                         elanInstance.getElanInstanceName());
1000                 programRemoteDmacFlow(elanInstance, interfaceInfo, confTx);
1001             }
1002         }
1003         // bind the Elan service to the Interface
1004         bindService(elanInstance, elanInterface, interfaceInfo.getInterfaceTag(), confTx);
1005     }
1006
1007     private void installEntriesForFirstInterfaceonDpn(ElanInstance elanInfo, InterfaceInfo interfaceInfo,
1008             DpnInterfaces dpnInterfaces, boolean isFirstInterfaceInDpn, TypedWriteTransaction<Configuration> confTx) {
1009         if (!isOperational(interfaceInfo)) {
1010             LOG.warn("Entries for interface on dpn {} not installed since interface {} is not operational",
1011                     dpnInterfaces.getDpId(), interfaceInfo.getInterfaceName());
1012             EVENT_LOGGER.debug("ELAN-1stInterfaceEntries, ADD {} Instance {} Interface Status {}, returning",
1013                     interfaceInfo.getInterfaceName(), elanInfo.getElanInstanceName(), interfaceInfo.getOpState());
1014             return;
1015         }
1016         EVENT_LOGGER.debug("ELAN-1stInterfaceEntries, ADD {} Instance {}",
1017                 interfaceInfo.getInterfaceName(), elanInfo.getElanInstanceName());
1018         // LocalBroadcast Group creation with elan-Interfaces
1019         LOG.info("Installing entries for interface {} on dpn {}", interfaceInfo.getInterfaceName(),
1020                 dpnInterfaces.getDpId());
1021         setupLocalBroadcastGroups(elanInfo, dpnInterfaces, interfaceInfo, confTx);
1022         if (isFirstInterfaceInDpn) {
1023             LOG.trace("waitTimeForSyncInstall is {}", WAIT_TIME_FOR_SYNC_INSTALL);
1024             try {
1025                 Thread.sleep(WAIT_TIME_FOR_SYNC_INSTALL);
1026             } catch (InterruptedException e1) {
1027                 LOG.warn("Error while waiting for local BC group for ELAN {} to install", elanInfo);
1028             }
1029         }
1030     }
1031
1032     public InstanceIdentifier<Group> getGroupIid(ElanInstance elanInfo, Uint64 dpnId) {
1033         long remoteBcGroupId = ElanUtils.getElanRemoteBCGId(elanInfo.getElanTag().toJava());
1034         return InstanceIdentifier.builder(Nodes.class)
1035                 .child(Node.class, new NodeKey(new org.opendaylight.yang.gen.v1.urn.opendaylight
1036                         .inventory.rev130819.NodeId("openflow:" + dpnId.toString())))
1037                 .augmentation(FlowCapableNode.class)
1038                 .child(Group.class, new GroupKey(new GroupId(remoteBcGroupId))).build();
1039     }
1040
1041     public void scheduleElanInterfaceWorkerAfterRemoteBcGroup(ElanInstance elanInfo,
1042                                                               InterfaceInfo interfaceInfo,
1043                                                               DpnInterfaces dpnInterfaces,
1044                                                               boolean isFirstInterfaceInDpn,
1045                                                               ElanInterface elanInterface) {
1046         if (!isOperational(interfaceInfo)) {
1047             LOG.debug("Interface {} is not operational", elanInterface.getName());
1048             return;
1049         }
1050         String elanInterfaceJobKey = ElanUtils.getElanInterfaceJobKey(interfaceInfo.getInterfaceName());
1051         InterfaceAddWorkerOnElanInterface addWorker = new InterfaceAddWorkerOnElanInterface(elanInterfaceJobKey,
1052                 elanInterface, interfaceInfo, elanInfo, isFirstInterfaceInDpn, this);
1053         InstanceIdentifier<Group> groupInstanceId = getGroupIid(elanInfo, dpnInterfaces.getDpId());
1054         elanGroupCache.addJobToWaitList(groupInstanceId, () -> {
1055             jobCoordinator.enqueueJob(elanInterfaceJobKey, addWorker, ElanConstants.JOB_MAX_RETRIES);
1056         });
1057     }
1058
1059     public void setupFilterEqualsTable(ElanInstance elanInfo, InterfaceInfo interfaceInfo,
1060             TypedWriteTransaction<Configuration> writeFlowGroupTx) {
1061         int ifTag = interfaceInfo.getInterfaceTag();
1062         Flow flow = MDSALUtil.buildFlowNew(NwConstants.ELAN_FILTER_EQUALS_TABLE,
1063                 getFlowRef(NwConstants.ELAN_FILTER_EQUALS_TABLE, ifTag, "group"), 11, elanInfo.getElanInstanceName(),
1064                 0, 0, Uint64.valueOf(ElanConstants.COOKIE_ELAN_FILTER_EQUALS.toJava().add(BigInteger.valueOf(ifTag))),
1065                 ElanUtils.getTunnelIdMatchForFilterEqualsLPortTag(ifTag),
1066                 elanUtils.getInstructionsInPortForOutGroup(interfaceInfo.getInterfaceName()));
1067
1068         mdsalManager.addFlow(writeFlowGroupTx, interfaceInfo.getDpId(), flow);
1069         LOG.trace("Filter equals table(55) flow entry created on dpn: {} for interface port: {}",
1070                 interfaceInfo.getDpId(), interfaceInfo.getPortName());
1071
1072         Flow flowEntry = MDSALUtil.buildFlowNew(NwConstants.ELAN_FILTER_EQUALS_TABLE,
1073                 getFlowRef(NwConstants.ELAN_FILTER_EQUALS_TABLE, ifTag, "drop"), 12, elanInfo.getElanInstanceName(), 0,
1074                 0, Uint64.valueOf(ElanConstants.COOKIE_ELAN_FILTER_EQUALS.toJava().add(BigInteger.valueOf(ifTag))),
1075                 getMatchesForFilterEqualsLPortTag(ifTag), MDSALUtil.buildInstructionsDrop());
1076
1077         mdsalManager.addFlow(writeFlowGroupTx, interfaceInfo.getDpId(), flowEntry);
1078         LOG.trace("Filter equals table(55) drop flow entry created on dpn: {} for interface port: {}",
1079                 interfaceInfo.getDpId(), interfaceInfo.getPortName());
1080     }
1081
1082     public void removeFilterEqualsTable(ElanInstance elanInfo, InterfaceInfo interfaceInfo,
1083             TypedReadWriteTransaction<Configuration> flowTx) throws ExecutionException, InterruptedException {
1084         int ifTag = interfaceInfo.getInterfaceTag();
1085         Flow flow = MDSALUtil.buildFlow(NwConstants.ELAN_FILTER_EQUALS_TABLE,
1086             getFlowRef(NwConstants.ELAN_FILTER_EQUALS_TABLE, ifTag, "group"));
1087
1088         mdsalManager.removeFlow(flowTx, interfaceInfo.getDpId(), flow);
1089
1090         Flow flowEntity = MDSALUtil.buildFlow(NwConstants.ELAN_FILTER_EQUALS_TABLE,
1091             getFlowRef(NwConstants.ELAN_FILTER_EQUALS_TABLE, ifTag, "drop"));
1092
1093         mdsalManager.removeFlow(flowTx, interfaceInfo.getDpId(), flowEntity);
1094     }
1095
1096     private static List<MatchInfo> buildMatchesForVni(Uint64 vni) {
1097         List<MatchInfo> mkMatches = new ArrayList<>();
1098         MatchInfo match = new MatchTunnelId(vni);
1099         mkMatches.add(match);
1100         return mkMatches;
1101     }
1102
1103     private static List<InstructionInfo> getInstructionsForOutGroup(long groupId) {
1104         List<InstructionInfo> mkInstructions = new ArrayList<>();
1105         mkInstructions.add(new InstructionWriteActions(Collections.singletonList(new ActionGroup(groupId))));
1106         return mkInstructions;
1107     }
1108
1109     private static List<MatchInfo> getMatchesForElanTag(long elanTag, boolean isSHFlagSet) {
1110         List<MatchInfo> mkMatches = new ArrayList<>();
1111         // Matching metadata
1112         mkMatches.add(new MatchMetadata(
1113                 ElanUtils.getElanMetadataLabel(elanTag, isSHFlagSet), MetaDataUtil.METADATA_MASK_SERVICE_SH_FLAG));
1114         return mkMatches;
1115     }
1116
1117     /**
1118      * Builds the list of instructions to be installed in the INTERNAL_TUNNEL_TABLE (36) / EXTERNAL_TUNNEL_TABLE (38)
1119      * which so far consists of writing the elanTag in metadata and send the packet to ELAN_DMAC_TABLE.
1120      *
1121      * @param elanTag
1122      *            elanTag to be written in metadata when flow is selected
1123      * @return the instructions ready to be installed in a flow
1124      */
1125     private static List<InstructionInfo> getInstructionsIntOrExtTunnelTable(Uint32 elanTag) {
1126         List<InstructionInfo> mkInstructions = new ArrayList<>();
1127         mkInstructions.add(new InstructionWriteMetadata(ElanHelper.getElanMetadataLabel(elanTag.longValue()),
1128                 ElanHelper.getElanMetadataMask()));
1129         /* applicable for EXTERNAL_TUNNEL_TABLE only
1130         * TODO: We should point to SMAC or DMAC depending on a configuration property to enable mac learning
1131         */
1132         mkInstructions.add(new InstructionGotoTable(NwConstants.ELAN_DMAC_TABLE));
1133         return mkInstructions;
1134     }
1135
1136     // Install DMAC entry on dst DPN
1137     public List<ListenableFuture<Void>> installDMacAddressTables(ElanInstance elanInfo, InterfaceInfo interfaceInfo,
1138             Uint64 dstDpId) {
1139         String interfaceName = interfaceInfo.getInterfaceName();
1140         ElanInterfaceMac elanInterfaceMac = elanUtils.getElanInterfaceMacByInterfaceName(interfaceName);
1141         if (elanInterfaceMac != null && elanInterfaceMac.getMacEntry() != null) {
1142             List<MacEntry> macEntries = elanInterfaceMac.getMacEntry();
1143             return Collections.singletonList(ElanUtils.waitForTransactionToComplete(
1144                 txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION, tx -> {
1145                     for (MacEntry macEntry : macEntries) {
1146                         String macAddress = macEntry.getMacAddress().getValue();
1147                         LOG.info("Installing remote dmac for mac address {} and interface {}", macAddress,
1148                             interfaceName);
1149                         try (Acquired lock = ElanUtils.lockElanMacDPN(elanInfo.getElanTag().toJava(), macAddress,
1150                             interfaceInfo.getDpId())) {
1151                             LOG.info("Acquired lock for mac : {}, proceeding with remote dmac install operation",
1152                                 macAddress);
1153                             elanUtils.setupDMacFlowOnRemoteDpn(elanInfo, interfaceInfo, dstDpId, macAddress, tx);
1154                         }
1155                     }
1156                 })));
1157         }
1158         return emptyList();
1159     }
1160
1161     private static void createDropBucket(List<Bucket> listBucket) {
1162         List<Action> actionsInfos = new ArrayList<>();
1163         actionsInfos.add(new ActionDrop().buildAction());
1164         Bucket dropBucket = MDSALUtil.buildBucket(actionsInfos, MDSALUtil.GROUP_WEIGHT, 0, MDSALUtil.WATCH_PORT,
1165                 MDSALUtil.WATCH_GROUP);
1166         listBucket.add(dropBucket);
1167     }
1168
1169     private void setupLocalBroadcastGroups(ElanInstance elanInfo, DpnInterfaces newDpnInterface,
1170             InterfaceInfo interfaceInfo, TypedWriteTransaction<Configuration> confTx) {
1171         if (!isOperational(interfaceInfo)) {
1172             EVENT_LOGGER.debug("ELAN-LBG, ADD {} Instance {} Interface Status {}, returning",
1173                     interfaceInfo.getInterfaceName(), elanInfo.getElanInstanceName(), interfaceInfo.getOpState());
1174             return;
1175         }
1176         setupStandardLocalBroadcastGroups(elanInfo, newDpnInterface, interfaceInfo, confTx);
1177         setupLeavesLocalBroadcastGroups(elanInfo, newDpnInterface, interfaceInfo, confTx);
1178     }
1179
1180     private void setupStandardLocalBroadcastGroups(ElanInstance elanInfo, DpnInterfaces newDpnInterface,
1181             InterfaceInfo interfaceInfo, TypedWriteTransaction<Configuration> confTx) {
1182         List<Bucket> listBucket = new ArrayList<>();
1183         int bucketId = 0;
1184         long groupId = ElanUtils.getElanLocalBCGId(elanInfo.getElanTag().toJava());
1185
1186         List<String> interfaces = new ArrayList<>();
1187         if (newDpnInterface != null && newDpnInterface.getInterfaces() != null) {
1188             interfaces = newDpnInterface.getInterfaces();
1189         }
1190         for (String ifName : interfaces) {
1191             // In case if there is a InterfacePort in the cache which is not in
1192             // operational state, skip processing it
1193             InterfaceInfo ifInfo = interfaceManager
1194                     .getInterfaceInfoFromOperationalDataStore(ifName, interfaceInfo.getInterfaceType());
1195             if (!isOperational(ifInfo)) {
1196                 continue;
1197             }
1198
1199             if (!interfaceManager.isExternalInterface(ifName)) {
1200                 listBucket.add(MDSALUtil.buildBucket(getInterfacePortActions(ifInfo), MDSALUtil.GROUP_WEIGHT, bucketId,
1201                         MDSALUtil.WATCH_PORT, MDSALUtil.WATCH_GROUP));
1202                 bucketId++;
1203             }
1204         }
1205
1206         Group group = MDSALUtil.buildGroup(groupId, elanInfo.getElanInstanceName(), GroupTypes.GroupAll,
1207                 MDSALUtil.buildBucketLists(listBucket));
1208         LOG.trace("installing the localBroadCast Group:{}", group);
1209         mdsalManager.addGroup(confTx, interfaceInfo.getDpId(), group);
1210     }
1211
1212     private void setupLeavesLocalBroadcastGroups(ElanInstance elanInfo, DpnInterfaces newDpnInterface,
1213             InterfaceInfo interfaceInfo, TypedWriteTransaction<Configuration> confTx) {
1214         EtreeInstance etreeInstance = elanInfo.augmentation(EtreeInstance.class);
1215         if (etreeInstance != null) {
1216             List<Bucket> listBucket = new ArrayList<>();
1217             int bucketId = 0;
1218
1219             List<String> interfaces = new ArrayList<>();
1220             if (newDpnInterface != null && newDpnInterface.getInterfaces() != null) {
1221                 interfaces = newDpnInterface.getInterfaces();
1222             }
1223             for (String ifName : interfaces) {
1224                 // In case if there is a InterfacePort in the cache which is not
1225                 // in
1226                 // operational state, skip processing it
1227                 InterfaceInfo ifInfo = interfaceManager
1228                         .getInterfaceInfoFromOperationalDataStore(ifName, interfaceInfo.getInterfaceType());
1229                 if (!isOperational(ifInfo)) {
1230                     continue;
1231                 }
1232
1233                 if (!interfaceManager.isExternalInterface(ifName)) {
1234                     // only add root interfaces
1235                     bucketId = addInterfaceIfRootInterface(bucketId, ifName, listBucket, ifInfo);
1236                 }
1237             }
1238
1239             if (listBucket.isEmpty()) { // No Buckets
1240                 createDropBucket(listBucket);
1241             }
1242
1243             long etreeLeafTag = etreeInstance.getEtreeLeafTagVal().getValue().toJava();
1244             long groupId = ElanUtils.getEtreeLeafLocalBCGId(etreeLeafTag);
1245             Group group = MDSALUtil.buildGroup(groupId, elanInfo.getElanInstanceName(), GroupTypes.GroupAll,
1246                     MDSALUtil.buildBucketLists(listBucket));
1247             LOG.trace("installing the localBroadCast Group:{}", group);
1248             mdsalManager.addGroup(confTx, interfaceInfo.getDpId(), group);
1249         }
1250     }
1251
1252     private int addInterfaceIfRootInterface(int bucketId, String ifName, List<Bucket> listBucket,
1253             InterfaceInfo ifInfo) {
1254         Optional<EtreeInterface> etreeInterface = elanInterfaceCache.getEtreeInterface(ifName);
1255         if (etreeInterface.isPresent() && etreeInterface.get().getEtreeInterfaceType() == EtreeInterfaceType.Root) {
1256             listBucket.add(MDSALUtil.buildBucket(getInterfacePortActions(ifInfo), MDSALUtil.GROUP_WEIGHT, bucketId,
1257                     MDSALUtil.WATCH_PORT, MDSALUtil.WATCH_GROUP));
1258             bucketId++;
1259         }
1260         return bucketId;
1261     }
1262
1263     public void removeLocalBroadcastGroup(ElanInstance elanInfo, InterfaceInfo interfaceInfo,
1264             TypedReadWriteTransaction<Configuration> deleteFlowGroupTx)
1265             throws ExecutionException, InterruptedException {
1266         Uint64 dpnId = interfaceInfo.getDpId();
1267         long groupId = ElanUtils.getElanLocalBCGId(elanInfo.getElanTag().toJava());
1268         LOG.trace("deleted the localBroadCast Group:{}", groupId);
1269         mdsalManager.removeGroup(deleteFlowGroupTx, dpnId, groupId);
1270     }
1271
1272     public void removeElanBroadcastGroup(ElanInstance elanInfo, InterfaceInfo interfaceInfo,
1273             TypedReadWriteTransaction<Configuration> deleteFlowGroupTx)
1274             throws ExecutionException, InterruptedException {
1275         Uint64 dpnId = interfaceInfo.getDpId();
1276         long groupId = ElanUtils.getElanRemoteBCGId(elanInfo.getElanTag().toJava());
1277         LOG.trace("deleting the remoteBroadCast group:{}", groupId);
1278         mdsalManager.removeGroup(deleteFlowGroupTx, dpnId, groupId);
1279     }
1280
1281     /**
1282      * Installs a flow in the External Tunnel table consisting in translating
1283      * the VNI retrieved from the packet that came over a tunnel with a TOR into
1284      * elanTag that will be used later in the ELANs pipeline.
1285      * @param dpnId the dpn id
1286      */
1287     private void setExternalTunnelTable(Uint64 dpnId, ElanInstance elanInfo,
1288             TypedWriteTransaction<Configuration> confTx) {
1289         Uint32 elanTag = elanInfo.getElanTag();
1290         FlowEntity flowEntity = MDSALUtil.buildFlowEntity(dpnId, NwConstants.EXTERNAL_TUNNEL_TABLE,
1291                 getFlowRef(NwConstants.EXTERNAL_TUNNEL_TABLE, elanTag.longValue()), 5, // prio
1292                 elanInfo.getElanInstanceName(), // flowName
1293                 0, // idleTimeout
1294                 0, // hardTimeout
1295                 Uint64.valueOf(ITMConstants.COOKIE_ITM_EXTERNAL.toJava().add(BigInteger.valueOf(elanTag.longValue()))),
1296                 buildMatchesForVni(Uint64.valueOf(ElanUtils.getVxlanSegmentationId(elanInfo).longValue())),
1297                 getInstructionsIntOrExtTunnelTable(elanTag));
1298
1299         mdsalManager.addFlow(confTx, flowEntity);
1300     }
1301
1302     /**
1303      * Removes, from External Tunnel table, the flow that translates from VNI to
1304      * elanTag. Important: ensure this method is only called whenever there is
1305      * no other ElanInterface in the specified DPN
1306      *
1307      * @param dpnId DPN whose Ext Tunnel table is going to be modified
1308      */
1309     private void unsetExternalTunnelTable(Uint64 dpnId, ElanInstance elanInfo,
1310             TypedReadWriteTransaction<Configuration> confTx) throws ExecutionException, InterruptedException {
1311         // TODO: Use DataStoreJobCoordinator in order to avoid that removing the
1312         // last ElanInstance plus
1313         // adding a new one does (almost at the same time) are executed in that
1314         // exact order
1315
1316         String flowId = getFlowRef(NwConstants.EXTERNAL_TUNNEL_TABLE, elanInfo.getElanTag().toJava());
1317         FlowEntity flowEntity = new FlowEntityBuilder()
1318             .setDpnId(dpnId)
1319             .setTableId(NwConstants.EXTERNAL_TUNNEL_TABLE)
1320             .setFlowId(flowId)
1321             .build();
1322         mdsalManager.removeFlow(confTx, flowEntity);
1323     }
1324
1325     public void setupTerminateServiceTable(ElanInstance elanInfo, Uint64 dpId,
1326             TypedWriteTransaction<Configuration> writeFlowGroupTx) {
1327         setupTerminateServiceTable(elanInfo, dpId, elanInfo.getElanTag(), writeFlowGroupTx);
1328         setupEtreeTerminateServiceTable(elanInfo, dpId, writeFlowGroupTx);
1329     }
1330
1331     public void setupTerminateServiceTable(ElanInstance elanInfo, Uint64 dpId, Uint32 elanTag,
1332             TypedWriteTransaction<Configuration> writeFlowGroupTx) {
1333         List<? extends MatchInfoBase> listMatchInfoBase;
1334         List<InstructionInfo> instructionInfos;
1335         Uint32 serviceId;
1336         if (!elanUtils.isOpenstackVniSemanticsEnforced()) {
1337             serviceId = elanTag;
1338             listMatchInfoBase = ElanUtils.getTunnelMatchesForServiceId(elanTag);
1339             instructionInfos = getInstructionsForOutGroup(ElanUtils.getElanLocalBCGId(elanTag.longValue()));
1340         } else {
1341             serviceId = ElanUtils.getVxlanSegmentationId(elanInfo);
1342             listMatchInfoBase = buildMatchesForVni(Uint64.valueOf(serviceId));
1343             instructionInfos = getInstructionsIntOrExtTunnelTable(elanTag);
1344         }
1345         FlowEntity flowEntity = MDSALUtil.buildFlowEntity(dpId, NwConstants.INTERNAL_TUNNEL_TABLE,
1346                 getFlowRef(NwConstants.INTERNAL_TUNNEL_TABLE, serviceId.longValue()), 5,
1347                 String.format("%s:%s", "ITM Flow Entry ", elanTag.toString()), 0, 0,
1348                 Uint64.valueOf(ITMConstants.COOKIE_ITM.toJava().add(BigInteger.valueOf(elanTag.longValue()))),
1349                 listMatchInfoBase, instructionInfos);
1350         mdsalManager.addFlow(writeFlowGroupTx, flowEntity);
1351         LOG.info("Installed internal tunnel table (36) flow entry on dpn: {} for elan: {}, tag: {}", dpId,
1352                 elanInfo.getElanInstanceName(), elanTag);
1353     }
1354
1355     private void setupEtreeTerminateServiceTable(ElanInstance elanInfo, Uint64 dpId,
1356             TypedWriteTransaction<Configuration> writeFlowGroupTx) {
1357         EtreeInstance etreeInstance = elanInfo.augmentation(EtreeInstance.class);
1358         if (etreeInstance != null) {
1359             setupTerminateServiceTable(elanInfo, dpId,
1360                 etreeInstance.getEtreeLeafTagVal().getValue(), writeFlowGroupTx);
1361         }
1362     }
1363
1364     public void setupUnknownDMacTable(ElanInstance elanInfo, Uint64 dpId,
1365             TypedWriteTransaction<Configuration> writeFlowGroupTx) {
1366         long elanTag = elanInfo.getElanTag().toJava();
1367         installLocalUnknownFlow(elanInfo, dpId, elanTag, writeFlowGroupTx);
1368         installRemoteUnknownFlow(elanInfo, dpId, elanTag, writeFlowGroupTx);
1369         setupEtreeUnknownDMacTable(elanInfo, dpId, elanTag, writeFlowGroupTx);
1370     }
1371
1372     private void setupEtreeUnknownDMacTable(ElanInstance elanInfo, Uint64 dpId, long elanTag,
1373             TypedWriteTransaction<Configuration> writeFlowGroupTx) {
1374         EtreeLeafTagName etreeLeafTag = elanEtreeUtils.getEtreeLeafTagByElanTag(elanTag);
1375         if (etreeLeafTag != null) {
1376             long leafTag = etreeLeafTag.getEtreeLeafTag().getValue().toJava();
1377             installRemoteUnknownFlow(elanInfo, dpId, leafTag, writeFlowGroupTx);
1378             installLocalUnknownFlow(elanInfo, dpId, leafTag, writeFlowGroupTx);
1379         }
1380     }
1381
1382     private void installLocalUnknownFlow(ElanInstance elanInfo, Uint64 dpId, long elanTag,
1383             TypedWriteTransaction<Configuration> writeFlowGroupTx) {
1384         FlowEntity flowEntity = MDSALUtil.buildFlowEntity(dpId, NwConstants.ELAN_UNKNOWN_DMAC_TABLE,
1385                 getUnknownDmacFlowRef(NwConstants.ELAN_UNKNOWN_DMAC_TABLE, elanTag,/* SH flag */false),
1386                 5, elanInfo.getElanInstanceName(), 0, 0,
1387                 Uint64.valueOf(ElanConstants.COOKIE_ELAN_UNKNOWN_DMAC.toJava().add(BigInteger.valueOf(elanTag))),
1388                 getMatchesForElanTag(elanTag, /* SH flag */false),
1389                 getInstructionsForOutGroup(ElanUtils.getElanRemoteBCGId(elanTag)));
1390
1391         mdsalManager.addFlow(writeFlowGroupTx, flowEntity);
1392         LOG.trace("Installed unknown dmac table (53) flow entry on dpn: {} for elan: {}, tag: {}",
1393                 dpId, elanInfo.getElanInstanceName(), elanTag);
1394     }
1395
1396     private void installRemoteUnknownFlow(ElanInstance elanInfo, Uint64 dpId, long elanTag,
1397             TypedWriteTransaction<Configuration> writeFlowGroupTx) {
1398         // only if ELAN can connect to external network, perform the following
1399
1400         if (isVxlanNetworkOrVxlanSegment(elanInfo) || ElanUtils.isVlan(elanInfo) || ElanUtils.isFlat(elanInfo)) {
1401             FlowEntity flowEntity = MDSALUtil.buildFlowEntity(dpId, NwConstants.ELAN_UNKNOWN_DMAC_TABLE,
1402                     getUnknownDmacFlowRef(NwConstants.ELAN_UNKNOWN_DMAC_TABLE, elanTag,/* SH flag */true),
1403                     5, elanInfo.getElanInstanceName(), 0, 0,
1404                     Uint64.valueOf(ElanConstants.COOKIE_ELAN_UNKNOWN_DMAC.toJava().add(BigInteger.valueOf(elanTag))),
1405                     getMatchesForElanTag(elanTag, /* SH flag */true),
1406                     getInstructionsForOutGroup(ElanUtils.getElanLocalBCGId(elanTag)));
1407             mdsalManager.addFlow(writeFlowGroupTx, flowEntity);
1408             LOG.trace("Installed unknown dmac table (53) flow entry on dpn: {} for elan connected to "
1409                     + "external network: {}, tag: {}", dpId, elanInfo.getElanInstanceName(), elanTag);
1410         }
1411     }
1412
1413
1414     private void removeUnknownDmacFlow(Uint64 dpId, ElanInstance elanInfo,
1415             TypedReadWriteTransaction<Configuration> deleteFlowGroupTx, long elanTag)
1416             throws ExecutionException, InterruptedException {
1417         Flow flow = new FlowBuilder().setId(new FlowId(getUnknownDmacFlowRef(NwConstants.ELAN_UNKNOWN_DMAC_TABLE,
1418                 elanTag, SH_FLAG_UNSET))).setTableId(NwConstants.ELAN_UNKNOWN_DMAC_TABLE).build();
1419         mdsalManager.removeFlow(deleteFlowGroupTx, dpId, flow);
1420
1421         if (isVxlanNetworkOrVxlanSegment(elanInfo)) {
1422             Flow flow2 = new FlowBuilder().setId(new FlowId(getUnknownDmacFlowRef(NwConstants.ELAN_UNKNOWN_DMAC_TABLE,
1423                     elanTag, SH_FLAG_SET))).setTableId(NwConstants.ELAN_UNKNOWN_DMAC_TABLE)
1424                     .build();
1425             mdsalManager.removeFlow(deleteFlowGroupTx, dpId, flow2);
1426         }
1427     }
1428
1429     private void removeDefaultTermFlow(Uint64 dpId, long elanTag) {
1430         elanUtils.removeTerminatingServiceAction(dpId, (int) elanTag);
1431     }
1432
1433     private void bindService(ElanInstance elanInfo, ElanInterface elanInterface, int lportTag,
1434             TypedWriteTransaction<Configuration> tx) {
1435         if (isStandardElanService(elanInterface)) {
1436             bindElanService(elanInfo.getElanTag().toJava(), elanInfo.getElanInstanceName(),
1437                     elanInterface.getName(), lportTag, tx);
1438         } else { // Etree service
1439             bindEtreeService(elanInfo, elanInterface, lportTag, tx);
1440         }
1441     }
1442
1443     private void bindElanService(long elanTag, String elanInstanceName, String interfaceName, int lportTag,
1444             TypedWriteTransaction<Configuration> tx) {
1445         int instructionKey = 0;
1446         List<Instruction> instructions = new ArrayList<>();
1447         instructions.add(MDSALUtil.buildAndGetWriteMetadaInstruction(ElanHelper.getElanMetadataLabel(elanTag),
1448                 MetaDataUtil.METADATA_MASK_SERVICE, ++instructionKey));
1449
1450         List<Action> actions = new ArrayList<>();
1451         actions.add(new ActionRegLoad(0, NxmNxReg1.class, 0, ElanConstants.INTERFACE_TAG_LENGTH - 1,
1452                 lportTag).buildAction());
1453         actions.add(new ActionRegLoad(1, ElanConstants.ELAN_REG_ID, 0, ElanConstants.ELAN_TAG_LENGTH - 1,
1454                 elanTag).buildAction());
1455         instructions.add(MDSALUtil.buildApplyActionsInstruction(actions, ++instructionKey));
1456
1457         instructions.add(MDSALUtil.buildAndGetGotoTableInstruction(NwConstants.ARP_CHECK_TABLE,
1458                 ++instructionKey));
1459
1460         short elanServiceIndex = ServiceIndex.getIndex(NwConstants.ELAN_SERVICE_NAME, NwConstants.ELAN_SERVICE_INDEX);
1461         BoundServices serviceInfo = ElanUtils.getBoundServices(
1462                 String.format("%s.%s.%s", "elan", elanInstanceName, interfaceName), elanServiceIndex,
1463                 NwConstants.ELAN_SERVICE_INDEX, NwConstants.COOKIE_ELAN_INGRESS_TABLE, instructions);
1464         InstanceIdentifier<BoundServices> bindServiceId = ElanUtils.buildServiceId(interfaceName, elanServiceIndex);
1465         Optional<BoundServices> existingElanService = ElanUtils.read(broker, LogicalDatastoreType.CONFIGURATION,
1466                 bindServiceId);
1467         if (!existingElanService.isPresent()) {
1468             tx.put(bindServiceId, serviceInfo, CREATE_MISSING_PARENTS);
1469             LOG.trace("Done binding elan service for elan: {} for interface: {}", elanInstanceName, interfaceName);
1470         }
1471     }
1472
1473     private void bindEtreeService(ElanInstance elanInfo, ElanInterface elanInterface, int lportTag,
1474             TypedWriteTransaction<Configuration> tx) {
1475         if (elanInterface.augmentation(EtreeInterface.class).getEtreeInterfaceType() == EtreeInterfaceType.Root) {
1476             bindElanService(elanInfo.getElanTag().toJava(), elanInfo.getElanInstanceName(), elanInterface.getName(),
1477                     lportTag, tx);
1478         } else {
1479             EtreeInstance etreeInstance = elanInfo.augmentation(EtreeInstance.class);
1480             if (etreeInstance == null) {
1481                 LOG.error("EtreeInterface {} is associated with a non EtreeInstance: {}",
1482                         elanInterface.getName(), elanInfo.getElanInstanceName());
1483             } else {
1484                 bindElanService(etreeInstance.getEtreeLeafTagVal().getValue().toJava(), elanInfo.getElanInstanceName(),
1485                         elanInterface.getName(), lportTag, tx);
1486             }
1487         }
1488     }
1489
1490     private static boolean isStandardElanService(ElanInterface elanInterface) {
1491         return elanInterface.augmentation(EtreeInterface.class) == null;
1492     }
1493
1494     protected void unbindService(String interfaceName, TypedReadWriteTransaction<Configuration> tx)
1495             throws ExecutionException, InterruptedException {
1496         short elanServiceIndex = ServiceIndex.getIndex(NwConstants.ELAN_SERVICE_NAME, NwConstants.ELAN_SERVICE_INDEX);
1497         InstanceIdentifier<BoundServices> bindServiceId = ElanUtils.buildServiceId(interfaceName, elanServiceIndex);
1498         if (tx.read(bindServiceId).get().isPresent()) {
1499             tx.delete(bindServiceId);
1500         }
1501     }
1502
1503     private static String getFlowRef(long tableId, long elanTag) {
1504         return String.valueOf(tableId) + elanTag;
1505     }
1506
1507     private static String getFlowRef(long tableId, long elanTag, String flowName) {
1508         return new StringBuilder().append(tableId).append(NwConstants.FLOWID_SEPARATOR).append(elanTag)
1509                 .append(NwConstants.FLOWID_SEPARATOR).append(flowName).toString();
1510     }
1511
1512     private static String getUnknownDmacFlowRef(long tableId, long elanTag, boolean shFlag) {
1513         return String.valueOf(tableId) + elanTag + shFlag;
1514     }
1515
1516     private static List<Action> getInterfacePortActions(InterfaceInfo interfaceInfo) {
1517         List<Action> listAction = new ArrayList<>();
1518         int actionKey = 0;
1519         listAction.add(
1520             new ActionSetFieldTunnelId(Uint64.valueOf(interfaceInfo.getInterfaceTag())).buildAction(actionKey));
1521         actionKey++;
1522         listAction.add(new ActionNxResubmit(NwConstants.ELAN_FILTER_EQUALS_TABLE).buildAction(actionKey));
1523         return listAction;
1524     }
1525
1526     private static DpnInterfaces updateElanDpnInterfacesList(String elanInstanceName, Uint64 dpId,
1527             List<String> interfaceNames, TypedWriteTransaction<Operational> tx) {
1528         DpnInterfaces dpnInterface = new DpnInterfacesBuilder().setDpId(dpId).setInterfaces(interfaceNames)
1529                 .withKey(new DpnInterfacesKey(dpId)).build();
1530         tx.put(ElanUtils.getElanDpnInterfaceOperationalDataPath(elanInstanceName, dpId), dpnInterface,
1531                 CREATE_MISSING_PARENTS);
1532         LOG.trace("Updated operational dpn interfaces for elan: {} with interfaces: {}", elanInstanceName,
1533                 interfaceNames);
1534         return dpnInterface;
1535     }
1536
1537     /**
1538      * Delete elan dpn interface from operational DS.
1539      *
1540      * @param elanInstanceName
1541      *            the elan instance name
1542      * @param dpId
1543      *            the dp id
1544      */
1545     private static void deleteElanDpnInterface(String elanInstanceName, Uint64 dpId,
1546             TypedReadWriteTransaction<Operational> tx) throws ExecutionException, InterruptedException {
1547         InstanceIdentifier<DpnInterfaces> dpnInterfacesId = ElanUtils
1548                 .getElanDpnInterfaceOperationalDataPath(elanInstanceName, dpId);
1549         Optional<DpnInterfaces> dpnInterfaces = tx.read(dpnInterfacesId).get();
1550         if (dpnInterfaces.isPresent()) {
1551             tx.delete(dpnInterfacesId);
1552         }
1553     }
1554
1555     private static DpnInterfaces createElanInterfacesList(String elanInstanceName, String interfaceName,
1556             Uint64 dpId, TypedWriteTransaction<Operational> tx) {
1557         List<String> interfaceNames = new ArrayList<>();
1558         interfaceNames.add(interfaceName);
1559         DpnInterfaces dpnInterface = new DpnInterfacesBuilder().setDpId(dpId).setInterfaces(interfaceNames)
1560                 .withKey(new DpnInterfacesKey(dpId)).build();
1561         return dpnInterface;
1562     }
1563
1564     private static void createElanStateList(String elanInstanceName, String interfaceName,
1565             TypedReadWriteTransaction<Operational> tx) throws ExecutionException, InterruptedException {
1566         InstanceIdentifier<Elan> elanInstance = ElanUtils.getElanInstanceOperationalDataPath(elanInstanceName);
1567         Optional<Elan> elanInterfaceLists = tx.read(elanInstance).get();
1568         // Adding new Elan Interface Port to the operational DataStore without
1569         // Static-Mac Entries..
1570         if (elanInterfaceLists.isPresent()) {
1571             List<String> interfaceLists = new ArrayList<>();
1572             interfaceLists.add(interfaceName);
1573             List<String> existingInterfaceLists = elanInterfaceLists.get().getElanInterfaces();
1574             if (existingInterfaceLists != null && !existingInterfaceLists.isEmpty()) {
1575                 existingInterfaceLists.forEach(iface -> interfaceLists.add(iface));
1576             }
1577
1578             Elan elanState = new ElanBuilder().setName(elanInstanceName).setElanInterfaces(interfaceLists)
1579                     .withKey(new ElanKey(elanInstanceName)).build();
1580             tx.put(ElanUtils.getElanInstanceOperationalDataPath(elanInstanceName), elanState, CREATE_MISSING_PARENTS);
1581             LOG.trace("Updated operational elan state for elan: {} with interfaces: {}", elanInstanceName,
1582                     interfaceLists);
1583         }
1584     }
1585
1586     private static boolean isOperational(InterfaceInfo interfaceInfo) {
1587         return interfaceInfo != null && interfaceInfo.getAdminState() == InterfaceInfo.InterfaceAdminState.ENABLED;
1588     }
1589
1590     @SuppressWarnings("checkstyle:IllegalCatch")
1591     public void handleInternalTunnelStateEvent(Uint64 srcDpId, Uint64 dstDpId) {
1592         ElanDpnInterfaces dpnInterfaceLists = elanUtils.getElanDpnInterfacesList();
1593         LOG.trace("processing tunnel state event for srcDpId {} dstDpId {}"
1594                 + " and dpnInterfaceList {}", srcDpId, dstDpId, dpnInterfaceLists);
1595         if (dpnInterfaceLists == null) {
1596             return;
1597         }
1598         List<ElanDpnInterfacesList> elanDpnIf = dpnInterfaceLists.nonnullElanDpnInterfacesList();
1599         for (ElanDpnInterfacesList elanDpns : elanDpnIf) {
1600             int cnt = 0;
1601             String elanName = elanDpns.getElanInstanceName();
1602             ElanInstance elanInfo = elanInstanceCache.get(elanName).orElse(null);
1603             if (elanInfo == null) {
1604                 LOG.warn("ELAN Info is null for elanName {} that does exist in elanDpnInterfaceList, "
1605                         + "skipping this ELAN for tunnel handling", elanName);
1606                 continue;
1607             }
1608             if (!isVxlanNetworkOrVxlanSegment(elanInfo)) {
1609                 LOG.debug("Ignoring internal tunnel state event for Flat/Vlan elan {}", elanName);
1610                 continue;
1611             }
1612             List<DpnInterfaces> dpnInterfaces = elanDpns.getDpnInterfaces();
1613             if (dpnInterfaces == null) {
1614                 continue;
1615             }
1616             DpnInterfaces dstDpnIf = null;
1617             for (DpnInterfaces dpnIf : dpnInterfaces) {
1618                 Uint64 dpnIfDpId = dpnIf.getDpId();
1619                 if (Objects.equals(dpnIfDpId, srcDpId)) {
1620                     cnt++;
1621                 } else if (Objects.equals(dpnIfDpId, dstDpId)) {
1622                     cnt++;
1623                     dstDpnIf = dpnIf;
1624                 }
1625             }
1626             if (cnt == 2) {
1627                 LOG.info("Elan instance:{} is present b/w srcDpn:{} and dstDpn:{}", elanName, srcDpId, dstDpId);
1628                 final DpnInterfaces finalDstDpnIf = dstDpnIf; // var needs to be final so it can be accessed in lambda
1629                 jobCoordinator.enqueueJob(elanName, () -> {
1630                     // update Remote BC Group
1631                     LOG.trace("procesing elan remote bc group for tunnel event {}", elanInfo);
1632                     try {
1633                         txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION,
1634                             confTx -> elanL2GatewayMulticastUtils.setupElanBroadcastGroups(elanInfo, srcDpId,
1635                                 confTx)).get();
1636                     } catch (RuntimeException e) {
1637                         LOG.error("Error while adding remote bc group for {} on dpId {} ", elanName, srcDpId);
1638                     }
1639                     Set<String> interfaceLists = new HashSet<>();
1640                     interfaceLists.addAll(finalDstDpnIf.getInterfaces());
1641                     for (String ifName : interfaceLists) {
1642                         jobCoordinator.enqueueJob(ElanUtils.getElanInterfaceJobKey(ifName), () -> {
1643                             LOG.info("Processing tunnel up event for elan {} and interface {}", elanName, ifName);
1644                             InterfaceInfo interfaceInfo = interfaceManager.getInterfaceInfo(ifName);
1645                             if (isOperational(interfaceInfo)) {
1646                                 return installDMacAddressTables(elanInfo, interfaceInfo, srcDpId);
1647                             }
1648                             return emptyList();
1649                         }, ElanConstants.JOB_MAX_RETRIES);
1650                     }
1651                     return emptyList();
1652                 }, ElanConstants.JOB_MAX_RETRIES);
1653             }
1654
1655         }
1656     }
1657
1658     /**
1659      * Handle external tunnel state event.
1660      *
1661      * @param externalTunnel
1662      *            the external tunnel
1663      * @param intrf
1664      *            the interface
1665      */
1666     void handleExternalTunnelStateEvent(ExternalTunnel externalTunnel, Interface intrf) {
1667         if (!validateExternalTunnelStateEvent(externalTunnel, intrf)) {
1668             return;
1669         }
1670         // dpId/externalNodeId will be available either in source or destination
1671         // based on the tunnel end point
1672         Uint64 dpId = null;
1673         NodeId externalNodeId = null;
1674         if (StringUtils.isNumeric(externalTunnel.getSourceDevice())) {
1675             dpId = Uint64.valueOf(externalTunnel.getSourceDevice());
1676             externalNodeId = new NodeId(externalTunnel.getDestinationDevice());
1677         } else if (StringUtils.isNumeric(externalTunnel.getDestinationDevice())) {
1678             dpId = Uint64.valueOf(externalTunnel.getDestinationDevice());
1679             externalNodeId = new NodeId(externalTunnel.getSourceDevice());
1680         }
1681         if (dpId == null || externalNodeId == null) {
1682             LOG.error("Dp ID / externalNodeId not found in external tunnel {}", externalTunnel);
1683             return;
1684         }
1685
1686         ElanDpnInterfaces dpnInterfaceLists = elanUtils.getElanDpnInterfacesList();
1687         if (dpnInterfaceLists == null) {
1688             return;
1689         }
1690         List<ElanDpnInterfacesList> elanDpnIf = dpnInterfaceLists.nonnullElanDpnInterfacesList();
1691         for (ElanDpnInterfacesList elanDpns : elanDpnIf) {
1692             String elanName = elanDpns.getElanInstanceName();
1693             ElanInstance elanInfo = elanInstanceCache.get(elanName).orElse(null);
1694
1695             DpnInterfaces dpnInterfaces = elanUtils.getElanInterfaceInfoByElanDpn(elanName, dpId);
1696             if (elanInfo == null || dpnInterfaces == null || dpnInterfaces.getInterfaces() == null
1697                     || dpnInterfaces.getInterfaces().isEmpty()) {
1698                 continue;
1699             }
1700             LOG.debug("Elan instance:{} is present in Dpn:{} ", elanName, dpId);
1701
1702             final Uint64 finalDpId = dpId;
1703             LoggingFutures.addErrorLogging(txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION,
1704                 confTx -> elanL2GatewayMulticastUtils.setupElanBroadcastGroups(elanInfo, finalDpId, confTx)), LOG,
1705                 "Error setting up ELAN BGs");
1706             // install L2gwDevices local macs in dpn.
1707             elanL2GatewayUtils.installL2gwDeviceMacsInDpn(dpId, externalNodeId, elanInfo, intrf.getName());
1708             // Install dpn macs on external device
1709             installDpnMacsInL2gwDevice(elanName, new HashSet<>(dpnInterfaces.getInterfaces()), dpId,
1710                     externalNodeId);
1711         }
1712         LOG.info("Handled ExternalTunnelStateEvent for {}", externalTunnel);
1713     }
1714
1715     /**
1716      * Installs dpn macs in external device. first it checks if the physical
1717      * locator towards this dpn tep is present or not if the physical locator is
1718      * present go ahead and add the ucast macs otherwise update the mcast mac
1719      * entry to include this dpn tep ip and schedule the job to put ucast macs
1720      * once the physical locator is programmed in device
1721      *
1722      * @param elanName
1723      *            the elan name
1724      * @param lstElanInterfaceNames
1725      *            the lst Elan interface names
1726      * @param dpnId
1727      *            the dpn id
1728      * @param externalNodeId
1729      *            the external node id
1730      */
1731     private void installDpnMacsInL2gwDevice(String elanName, Set<String> lstElanInterfaceNames, Uint64 dpnId,
1732             NodeId externalNodeId) {
1733         L2GatewayDevice elanL2GwDevice = ElanL2GwCacheUtils.getL2GatewayDeviceFromCache(elanName,
1734                 externalNodeId.getValue());
1735         if (elanL2GwDevice == null) {
1736             LOG.debug("L2 gw device not found in elan cache for device name {}", externalNodeId);
1737             return;
1738         }
1739         IpAddress dpnTepIp = elanItmUtils.getSourceDpnTepIp(dpnId, externalNodeId);
1740         if (dpnTepIp == null) {
1741             LOG.warn("Could not install dpn macs in l2gw device , dpnTepIp not found dpn : {} , nodeid : {}", dpnId,
1742                     externalNodeId);
1743             return;
1744         }
1745
1746         String logicalSwitchName = ElanL2GatewayUtils.getLogicalSwitchFromElan(elanName);
1747         RemoteMcastMacs remoteMcastMac = elanL2GatewayUtils.readRemoteMcastMac(externalNodeId, logicalSwitchName,
1748                 LogicalDatastoreType.OPERATIONAL);
1749         boolean phyLocAlreadyExists =
1750                 ElanL2GatewayUtils.checkIfPhyLocatorAlreadyExistsInRemoteMcastEntry(externalNodeId, remoteMcastMac,
1751                 dpnTepIp);
1752         LOG.debug("phyLocAlreadyExists = {} for locator [{}] in remote mcast entry for elan [{}], nodeId [{}]",
1753                 phyLocAlreadyExists, dpnTepIp.stringValue(), elanName, externalNodeId.getValue());
1754         List<PhysAddress> staticMacs = elanL2GatewayUtils.getElanDpnMacsFromInterfaces(lstElanInterfaceNames);
1755
1756         if (phyLocAlreadyExists) {
1757             elanL2GatewayUtils.scheduleAddDpnMacsInExtDevice(elanName, dpnId, staticMacs, elanL2GwDevice);
1758             return;
1759         }
1760         elanL2GatewayMulticastUtils.scheduleMcastMacUpdateJob(elanName, elanL2GwDevice);
1761         elanL2GatewayUtils.scheduleAddDpnMacsInExtDevice(elanName, dpnId, staticMacs, elanL2GwDevice);
1762     }
1763
1764     /**
1765      * Validate external tunnel state event.
1766      *
1767      * @param externalTunnel
1768      *            the external tunnel
1769      * @param intrf
1770      *            the intrf
1771      * @return true, if successful
1772      */
1773     private boolean validateExternalTunnelStateEvent(ExternalTunnel externalTunnel, Interface intrf) {
1774         if (intrf.getOperStatus() == Interface.OperStatus.Up) {
1775             String srcDevice = externalTunnel.getDestinationDevice();
1776             String destDevice = externalTunnel.getSourceDevice();
1777             ExternalTunnel otherEndPointExtTunnel = elanUtils.getExternalTunnel(srcDevice, destDevice,
1778                     LogicalDatastoreType.CONFIGURATION);
1779             LOG.trace("Validating external tunnel state: src tunnel {}, dest tunnel {}", externalTunnel,
1780                     otherEndPointExtTunnel);
1781             if (otherEndPointExtTunnel != null) {
1782                 boolean otherEndPointInterfaceOperational = ElanUtils.isInterfaceOperational(
1783                         otherEndPointExtTunnel.getTunnelInterfaceName(), broker);
1784                 if (otherEndPointInterfaceOperational) {
1785                     return true;
1786                 } else {
1787                     LOG.debug("Other end [{}] of the external tunnel is not yet UP for {}",
1788                             otherEndPointExtTunnel.getTunnelInterfaceName(), externalTunnel);
1789                 }
1790             }
1791         }
1792         return false;
1793     }
1794
1795     private static List<MatchInfo> getMatchesForFilterEqualsLPortTag(int lportTag) {
1796         List<MatchInfo> mkMatches = new ArrayList<>();
1797         // Matching metadata
1798         mkMatches.add(
1799                 new MatchMetadata(MetaDataUtil.getLportTagMetaData(lportTag), MetaDataUtil.METADATA_MASK_LPORT_TAG));
1800         mkMatches.add(new MatchTunnelId(Uint64.valueOf(lportTag)));
1801         return mkMatches;
1802     }
1803
1804     public void handleExternalInterfaceEvent(ElanInstance elanInstance, DpnInterfaces dpnInterfaces,
1805                                              Uint64 dpId) {
1806         LOG.debug("setting up remote BC group for elan {}", elanInstance.getPhysicalNetworkName());
1807         addErrorLogging(txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION,
1808             confTx -> elanL2GatewayMulticastUtils.setupStandardElanBroadcastGroups(elanInstance, dpnInterfaces, dpId,
1809                 confTx)), LOG, "Error setting up remote BC group for ELAN {}", elanInstance.getPhysicalNetworkName());
1810         try {
1811             Thread.sleep(WAIT_TIME_FOR_SYNC_INSTALL);
1812         } catch (InterruptedException e) {
1813             LOG.warn("Error while waiting for local BC group for ELAN {} to install", elanInstance);
1814         }
1815     }
1816 }