Make subnetroute label and flow management more robust
[netvirt.git] / fibmanager / impl / src / main / java / org / opendaylight / netvirt / fibmanager / VrfEntryListener.java
1 /*
2  * Copyright © 2015, 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.fibmanager;
9
10 import static org.opendaylight.genius.mdsalutil.NWUtil.isIpv4Address;
11 import static org.opendaylight.mdsal.binding.util.Datastore.CONFIGURATION;
12 import static org.opendaylight.mdsal.binding.util.Datastore.OPERATIONAL;
13
14 import com.google.common.base.Preconditions;
15 import com.google.common.collect.Lists;
16 import com.google.common.util.concurrent.FutureCallback;
17 import com.google.common.util.concurrent.Futures;
18 import com.google.common.util.concurrent.ListenableFuture;
19 import com.google.common.util.concurrent.MoreExecutors;
20 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
21 import java.math.BigInteger;
22 import java.net.InetAddress;
23 import java.net.UnknownHostException;
24 import java.util.ArrayList;
25 import java.util.Arrays;
26 import java.util.Collections;
27 import java.util.List;
28 import java.util.Map;
29 import java.util.Objects;
30 import java.util.Optional;
31 import java.util.concurrent.Callable;
32 import java.util.concurrent.CopyOnWriteArrayList;
33 import java.util.concurrent.ExecutionException;
34 import java.util.concurrent.locks.ReentrantLock;
35 import javax.inject.Inject;
36 import javax.inject.Singleton;
37 import org.eclipse.jdt.annotation.NonNull;
38 import org.eclipse.jdt.annotation.Nullable;
39 import org.opendaylight.genius.datastoreutils.SingleTransactionDataBroker;
40 import org.opendaylight.genius.datastoreutils.listeners.DataTreeEventCallbackRegistrar;
41 import org.opendaylight.genius.mdsalutil.ActionInfo;
42 import org.opendaylight.genius.mdsalutil.FlowEntity;
43 import org.opendaylight.genius.mdsalutil.InstructionInfo;
44 import org.opendaylight.genius.mdsalutil.MDSALUtil;
45 import org.opendaylight.genius.mdsalutil.MatchInfo;
46 import org.opendaylight.genius.mdsalutil.MetaDataUtil;
47 import org.opendaylight.genius.mdsalutil.NWUtil;
48 import org.opendaylight.genius.mdsalutil.NwConstants;
49 import org.opendaylight.genius.mdsalutil.actions.ActionDrop;
50 import org.opendaylight.genius.mdsalutil.actions.ActionGroup;
51 import org.opendaylight.genius.mdsalutil.actions.ActionPopMpls;
52 import org.opendaylight.genius.mdsalutil.instructions.InstructionApplyActions;
53 import org.opendaylight.genius.mdsalutil.instructions.InstructionGotoTable;
54 import org.opendaylight.genius.mdsalutil.instructions.InstructionWriteMetadata;
55 import org.opendaylight.genius.mdsalutil.interfaces.IMdsalApiManager;
56 import org.opendaylight.genius.mdsalutil.matches.MatchEthernetType;
57 import org.opendaylight.genius.mdsalutil.matches.MatchIpv4Destination;
58 import org.opendaylight.genius.mdsalutil.matches.MatchMetadata;
59 import org.opendaylight.genius.mdsalutil.matches.MatchMplsLabel;
60 import org.opendaylight.genius.mdsalutil.matches.MatchTunnelId;
61 import org.opendaylight.genius.utils.JvmGlobalLocks;
62 import org.opendaylight.genius.utils.batching.SubTransaction;
63 import org.opendaylight.infrautils.jobcoordinator.JobCoordinator;
64 import org.opendaylight.infrautils.utils.concurrent.Executors;
65 import org.opendaylight.infrautils.utils.concurrent.LoggingFutures;
66 import org.opendaylight.mdsal.binding.api.DataBroker;
67 import org.opendaylight.mdsal.binding.util.Datastore;
68 import org.opendaylight.mdsal.binding.util.Datastore.Configuration;
69 import org.opendaylight.mdsal.binding.util.Datastore.Operational;
70 import org.opendaylight.mdsal.binding.util.ManagedNewTransactionRunner;
71 import org.opendaylight.mdsal.binding.util.ManagedNewTransactionRunnerImpl;
72 import org.opendaylight.mdsal.binding.util.RetryingManagedNewTransactionRunner;
73 import org.opendaylight.mdsal.binding.util.TransactionAdapter;
74 import org.opendaylight.mdsal.binding.util.TypedReadWriteTransaction;
75 import org.opendaylight.mdsal.binding.util.TypedWriteTransaction;
76 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
77 import org.opendaylight.netvirt.elanmanager.api.IElanService;
78 import org.opendaylight.netvirt.fibmanager.NexthopManager.AdjacencyResult;
79 import org.opendaylight.netvirt.fibmanager.api.FibHelper;
80 import org.opendaylight.netvirt.fibmanager.api.RouteOrigin;
81 import org.opendaylight.netvirt.vpnmanager.api.VpnExtraRouteHelper;
82 import org.opendaylight.netvirt.vpnmanager.api.VpnHelper;
83 import org.opendaylight.serviceutils.tools.listener.AbstractAsyncDataTreeChangeListener;
84 import org.opendaylight.serviceutils.upgrade.UpgradeState;
85 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.yang.types.rev130715.MacAddress;
86 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
87 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowId;
88 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.Table;
89 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.TableKey;
90 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.table.Flow;
91 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.table.FlowBuilder;
92 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.table.FlowKey;
93 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.Nodes;
94 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.nodes.Node;
95 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.fibmanager.rev150330.FibEntries;
96 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.fibmanager.rev150330.LabelRouteMap;
97 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.fibmanager.rev150330.RouterInterface;
98 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.fibmanager.rev150330.SubnetRoute;
99 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.fibmanager.rev150330.fibentries.VrfTables;
100 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.fibmanager.rev150330.fibentries.VrfTablesKey;
101 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.fibmanager.rev150330.label.route.map.LabelRouteInfo;
102 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.fibmanager.rev150330.label.route.map.LabelRouteInfoBuilder;
103 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.fibmanager.rev150330.label.route.map.LabelRouteInfoKey;
104 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.fibmanager.rev150330.vrfentries.VrfEntry;
105 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.fibmanager.rev150330.vrfentries.VrfEntryKey;
106 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.fibmanager.rev150330.vrfentrybase.RoutePaths;
107 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.fibmanager.rev150330.vrfentrybase.RoutePathsKey;
108 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.l3vpn.rev130911.AdjacenciesOp;
109 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.l3vpn.rev130911.prefix.to._interface.vpn.ids.Prefixes;
110 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.l3vpn.rev130911.prefix.to._interface.vpn.ids.PrefixesBuilder;
111 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.l3vpn.rev130911.vpn._interface.op.data.VpnInterfaceOpDataEntry;
112 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.l3vpn.rev130911.vpn.instance.op.data.VpnInstanceOpDataEntry;
113 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.l3vpn.rev130911.vpn.instance.op.data.vpn.instance.op.data.entry.VpnToDpnList;
114 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.l3vpn.rev130911.vpn.instance.op.data.vpn.instance.op.data.entry.VpnToDpnListKey;
115 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.l3vpn.rev130911.vpn.to.extraroutes.vpn.extra.routes.Routes;
116 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.neutronvpn.l3vpn.rev200204.adjacency.list.Adjacency;
117 import org.opendaylight.yang.gen.v1.urn.opendaylight.netvirt.neutronvpn.l3vpn.rev200204.adjacency.list.AdjacencyBuilder;
118 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
119 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier.InstanceIdentifierBuilder;
120 import org.opendaylight.yangtools.yang.common.Uint32;
121 import org.opendaylight.yangtools.yang.common.Uint64;
122 import org.slf4j.Logger;
123 import org.slf4j.LoggerFactory;
124
125 @Singleton
126 public class VrfEntryListener extends AbstractAsyncDataTreeChangeListener<VrfEntry> {
127
128     private static final Logger LOG = LoggerFactory.getLogger(VrfEntryListener.class);
129     private static final String FLOWID_PREFIX = "L3.";
130     private static final Uint64 COOKIE_VM_FIB_TABLE =  Uint64.valueOf("8000003", 16).intern();
131     private static final int DEFAULT_FIB_FLOW_PRIORITY = 10;
132     private static final int IPV4_ADDR_PREFIX_LENGTH = 32;
133     private static final int LFIB_INTERVPN_PRIORITY = 15;
134     public static final Uint64 COOKIE_TUNNEL = Uint64.valueOf("9000000", 16).intern();
135     private static final int MAX_RETRIES = 3;
136     private static final Uint64 COOKIE_TABLE_MISS = Uint64.valueOf("8000004", 16).intern();
137
138     private final DataBroker dataBroker;
139     private final ManagedNewTransactionRunner txRunner;
140     private final RetryingManagedNewTransactionRunner retryingTxRunner;
141     private final IMdsalApiManager mdsalManager;
142     private final NexthopManager nextHopManager;
143     private final BgpRouteVrfEntryHandler bgpRouteVrfEntryHandler;
144     private final BaseVrfEntryHandler baseVrfEntryHandler;
145     private final RouterInterfaceVrfEntryHandler routerInterfaceVrfEntryHandler;
146     private final JobCoordinator jobCoordinator;
147     private final IElanService elanManager;
148     private final FibUtil fibUtil;
149     private final List<AutoCloseable> closeables = new CopyOnWriteArrayList<>();
150     private final UpgradeState upgradeState;
151     private final DataTreeEventCallbackRegistrar eventCallbacks;
152
153     @Inject
154     public VrfEntryListener(final DataBroker dataBroker, final IMdsalApiManager mdsalApiManager,
155                             final NexthopManager nexthopManager,
156                             final IElanService elanManager,
157                             final BaseVrfEntryHandler vrfEntryHandler,
158                             final BgpRouteVrfEntryHandler bgpRouteVrfEntryHandler,
159                             final RouterInterfaceVrfEntryHandler routerInterfaceVrfEntryHandler,
160                             final JobCoordinator jobCoordinator,
161                             final FibUtil fibUtil,
162                             final UpgradeState upgradeState,
163                             final DataTreeEventCallbackRegistrar eventCallbacks) {
164         super(dataBroker, LogicalDatastoreType.CONFIGURATION, InstanceIdentifier.create(FibEntries.class)
165                 .child(VrfTables.class).child(VrfEntry.class),
166                 Executors.newListeningSingleThreadExecutor("VrfEntryListener", LOG));
167         this.dataBroker = dataBroker;
168         this.txRunner = new ManagedNewTransactionRunnerImpl(dataBroker);
169         this.retryingTxRunner = new RetryingManagedNewTransactionRunner(dataBroker, MAX_RETRIES);
170         this.mdsalManager = mdsalApiManager;
171         this.nextHopManager = nexthopManager;
172         this.elanManager = elanManager;
173         this.baseVrfEntryHandler = vrfEntryHandler;
174         this.bgpRouteVrfEntryHandler = bgpRouteVrfEntryHandler;
175         this.routerInterfaceVrfEntryHandler = routerInterfaceVrfEntryHandler;
176         this.jobCoordinator = jobCoordinator;
177         this.fibUtil = fibUtil;
178         this.upgradeState = upgradeState;
179         this.eventCallbacks = eventCallbacks;
180     }
181
182     public void init() {
183         LOG.info("{} init", getClass().getSimpleName());
184     }
185
186     @Override
187     @SuppressWarnings("checkstyle:IllegalCatch")
188     public void close() {
189         closeables.forEach(c -> {
190             try {
191                 c.close();
192             } catch (Exception e) {
193                 LOG.warn("Error closing {}", c, e);
194             }
195         });
196         Executors.shutdownAndAwaitTermination(getExecutorService());
197     }
198
199     @Override
200     public void add(final InstanceIdentifier<VrfEntry> identifier, final VrfEntry vrfEntry) {
201         Preconditions.checkNotNull(vrfEntry, "VrfEntry should not be null or empty.");
202         String rd = identifier.firstKeyOf(VrfTables.class).getRouteDistinguisher();
203         LOG.debug("ADD: Adding Fib Entry rd {} prefix {} route-paths {}",
204                 rd, vrfEntry.getDestPrefix(), vrfEntry.getRoutePaths());
205         addFibEntries(identifier, vrfEntry, rd);
206         LOG.info("ADD: Added Fib Entry rd {} prefix {} route-paths {}",
207                  rd, vrfEntry.getDestPrefix(), vrfEntry.getRoutePaths());
208     }
209
210     //This method is temporary. Eventually Factory design pattern will be used to get
211     // right VrfEntryhandle and invoke its methods.
212     private void addFibEntries(InstanceIdentifier<VrfEntry> identifier, VrfEntry vrfEntry, String rd) {
213         if (RouteOrigin.value(vrfEntry.getOrigin()) == RouteOrigin.BGP) {
214             bgpRouteVrfEntryHandler.createFlows(identifier, vrfEntry, rd);
215             return;
216         }
217         if (VrfEntry.EncapType.Vxlan.equals(vrfEntry.getEncapType())) {
218             LOG.info("EVPN flows need to be programmed.");
219             EvpnVrfEntryHandler evpnVrfEntryHandler = new EvpnVrfEntryHandler(dataBroker, this, bgpRouteVrfEntryHandler,
220                     nextHopManager, jobCoordinator, fibUtil, upgradeState, eventCallbacks);
221             evpnVrfEntryHandler.createFlows(identifier, vrfEntry, rd);
222             closeables.add(evpnVrfEntryHandler);
223             return;
224         }
225         RouterInterface routerInt = vrfEntry.augmentation(RouterInterface.class);
226         if (routerInt != null) {
227             // ping responder for router interfaces
228             routerInterfaceVrfEntryHandler.createFlows(vrfEntry, rd);
229             return;
230         }
231         if (RouteOrigin.value(vrfEntry.getOrigin()) != RouteOrigin.BGP) {
232             createFibEntries(identifier, vrfEntry);
233             return;
234         }
235     }
236
237     @Override
238     public void remove(InstanceIdentifier<VrfEntry> identifier, VrfEntry vrfEntry) {
239         Preconditions.checkNotNull(vrfEntry, "VrfEntry should not be null or empty.");
240         String rd = identifier.firstKeyOf(VrfTables.class).getRouteDistinguisher();
241         LOG.debug("REMOVE: Removing Fib Entry rd {} prefix {} route-paths {}",
242                 rd, vrfEntry.getDestPrefix(), vrfEntry.getRoutePaths());
243         removeFibEntries(identifier, vrfEntry, rd);
244         LOG.info("REMOVE: Removed Fib Entry rd {} prefix {} route-paths {}",
245             rd, vrfEntry.getDestPrefix(), vrfEntry.getRoutePaths());
246     }
247
248     //This method is temporary. Eventually Factory design pattern will be used to get
249     // right VrfEntryhandle and invoke its methods.
250     private void removeFibEntries(InstanceIdentifier<VrfEntry> identifier, VrfEntry vrfEntry, String rd) {
251         if (VrfEntry.EncapType.Vxlan.equals(vrfEntry.getEncapType())) {
252             LOG.info("EVPN flows to be deleted");
253             EvpnVrfEntryHandler evpnVrfEntryHandler = new EvpnVrfEntryHandler(dataBroker, this, bgpRouteVrfEntryHandler,
254                     nextHopManager, jobCoordinator, fibUtil, upgradeState, eventCallbacks);
255             evpnVrfEntryHandler.removeFlows(identifier, vrfEntry, rd);
256             closeables.add(evpnVrfEntryHandler);
257             return;
258         }
259         RouterInterface routerInt = vrfEntry.augmentation(RouterInterface.class);
260         if (routerInt != null) {
261             // ping responder for router interfaces
262             routerInterfaceVrfEntryHandler.removeFlows(vrfEntry, rd);
263             return;
264         }
265         if (RouteOrigin.value(vrfEntry.getOrigin()) != RouteOrigin.BGP) {
266             deleteFibEntries(identifier, vrfEntry);
267             return;
268         }
269         if (RouteOrigin.value(vrfEntry.getOrigin()) == RouteOrigin.BGP) {
270             bgpRouteVrfEntryHandler.removeFlows(identifier, vrfEntry, rd);
271             return;
272         }
273     }
274
275     @Override
276     // "Redundant nullcheck of originalRoutePath, which is known to be non-null" - the null checking for
277     // originalRoutePath is a little dicey - safest to keep the checking even if not needed.
278     @SuppressFBWarnings("RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
279     public void update(InstanceIdentifier<VrfEntry> identifier, VrfEntry original, VrfEntry update) {
280         Preconditions.checkNotNull(update, "VrfEntry should not be null or empty.");
281         final String rd = identifier.firstKeyOf(VrfTables.class).getRouteDistinguisher();
282         LOG.debug("UPDATE: Updating Fib Entries to rd {} prefix {} route-paths {} origin {} old-origin {}", rd,
283                 update.getDestPrefix(), update.getRoutePaths(), update.getOrigin(), original.getOrigin());
284
285         //Ignore any Fib Entry changes for subneroute
286         if (RouteOrigin.value(update.getOrigin()) == RouteOrigin.CONNECTED) {
287             LOG.info("UPDATE: Updated Connected Fib Entries with rd {} prefix {} old-nexthop {} "
288                             + "new-nexthop {} origin {}",
289                     rd, update.getDestPrefix(), original.getRoutePaths(), update.getRoutePaths(),
290                     update.getOrigin());
291             return;
292         }
293
294         // Handle BGP Routes first
295         if (RouteOrigin.value(update.getOrigin()) == RouteOrigin.BGP) {
296             bgpRouteVrfEntryHandler.updateFlows(identifier, original, update, rd);
297             LOG.info("UPDATE: Updated BGP advertised Fib Entry with rd {} prefix {} route-paths {}",
298                     rd, update.getDestPrefix(), update.getRoutePaths());
299             return;
300         }
301
302         if (RouteOrigin.value(update.getOrigin()) == RouteOrigin.STATIC) {
303             List<RoutePaths> originalRoutePath = new ArrayList<RoutePaths>(original.nonnullRoutePaths().values());
304             List<RoutePaths> updateRoutePath = new ArrayList<RoutePaths>(update.nonnullRoutePaths().values());
305             LOG.info("UPDATE: Original route-path {} update route-path {} ", originalRoutePath, updateRoutePath);
306
307             //Updates need to be handled for extraroute even if original vrf entry route path is null or
308             //updated vrf entry route path is null. This can happen during tunnel events.
309             Optional<VpnInstanceOpDataEntry> optVpnInstance = fibUtil.getVpnInstanceOpData(rd);
310             List<String> usedRds = new ArrayList<>();
311             if (optVpnInstance.isPresent()) {
312                 usedRds = VpnExtraRouteHelper.getUsedRds(dataBroker,optVpnInstance.get().getVpnId(),
313                         update.getDestPrefix());
314             }
315             // If original VRF Entry had nexthop null , but update VRF Entry
316             // has nexthop , route needs to be created on remote Dpns
317             if (originalRoutePath == null || originalRoutePath.isEmpty()
318                     && updateRoutePath != null && !updateRoutePath.isEmpty() && usedRds.isEmpty()) {
319                 // TODO(vivek): Though ugly, Not handling this code now, as each
320                 // tep add event will invoke flow addition
321                 LOG.trace("Original VRF entry NH is null for destprefix {}. And the prefix is not an extra route."
322                         + " This event is IGNORED here.", update.getDestPrefix());
323                 return;
324             }
325
326             // If original VRF Entry had valid nexthop , but update VRF Entry
327             // has nexthop empty'ed out, route needs to be removed from remote Dpns
328             if (updateRoutePath == null || updateRoutePath.isEmpty()
329                     && originalRoutePath != null && !originalRoutePath.isEmpty() && usedRds.isEmpty()) {
330                 LOG.trace("Original VRF entry had valid NH for destprefix {}. And the prefix is not an extra route."
331                         + "This event is IGNORED here.", update.getDestPrefix());
332                 return;
333             }
334             //Update the used rds and vpntoextraroute containers only for the deleted nextHops.
335             List<String> nextHopsRemoved = FibHelper.getNextHopListFromRoutePaths(original);
336             nextHopsRemoved.removeAll(FibHelper.getNextHopListFromRoutePaths(update));
337             List<ListenableFuture<?>> futures = new ArrayList<>();
338             ListenableFuture<?> configFuture =
339                 txRunner.callWithNewReadWriteTransactionAndSubmit(CONFIGURATION, configTx ->
340                     futures.add(txRunner.callWithNewReadWriteTransactionAndSubmit(OPERATIONAL, operTx ->
341                         nextHopsRemoved.parallelStream()
342                             .forEach(nextHopRemoved -> {
343                                 try {
344                                     fibUtil.updateUsedRdAndVpnToExtraRoute(
345                                         configTx, operTx, nextHopRemoved, rd, update.getDestPrefix());
346                                 } catch (ExecutionException | InterruptedException e) {
347                                     throw new RuntimeException(e);
348                                 }
349                             }))));
350             futures.add(configFuture);
351             Futures.addCallback(configFuture, new FutureCallback<Object>() {
352                 @Override
353                 public void onSuccess(Object result) {
354                     createFibEntries(identifier, update);
355                     LOG.info("UPDATE: Updated static Fib Entry with rd {} prefix {} route-paths {}",
356                             rd, update.getDestPrefix(), update.getRoutePaths());
357                 }
358
359                 @Override
360                 public void onFailure(Throwable throwable) {
361                     LOG.error("Exception encountered while submitting operational future for update vrfentry {}",
362                             update, throwable);
363                 }
364             }, MoreExecutors.directExecutor());
365             return;
366         }
367
368         //Handle all other routes only on a cluster reboot
369         if (original.equals(update)) {
370             //Reboot use-case
371             createFibEntries(identifier, update);
372             LOG.info("UPDATE: Updated Non-static Fib Entry with rd {} prefix {} route-paths {}",
373                     rd, update.getDestPrefix(), update.getRoutePaths());
374             return;
375         }
376
377         LOG.info("UPDATE: Ignoring update for FIB entry with rd {} prefix {} route-origin {} route-paths {}",
378                 rd, update.getDestPrefix(), update.getOrigin(), update.getRoutePaths());
379     }
380
381     private void createFibEntries(final InstanceIdentifier<VrfEntry> vrfEntryIid, final VrfEntry vrfEntry) {
382         final VrfTablesKey vrfTableKey = vrfEntryIid.firstKeyOf(VrfTables.class);
383         List<SubTransaction> txnObjects =  new ArrayList<>();
384         final VpnInstanceOpDataEntry vpnInstance =
385                 fibUtil.getVpnInstance(vrfTableKey.getRouteDistinguisher());
386         Preconditions.checkNotNull(vpnInstance, "Vpn Instance not available " + vrfTableKey.getRouteDistinguisher());
387         Preconditions.checkNotNull(vpnInstance.getVpnId(), "Vpn Instance with rd " + vpnInstance.getVrfId()
388                 + " has null vpnId!");
389         final Map<VpnToDpnListKey, VpnToDpnList> keyVpnToDpnListMap;
390         if (vrfEntry.getParentVpnRd() != null
391                 && FibHelper.isControllerManagedNonSelfImportedRoute(RouteOrigin.value(vrfEntry.getOrigin()))) {
392             // This block MUST BE HIT only for PNF (Physical Network Function) FIB Entries.
393             VpnInstanceOpDataEntry parentVpnInstance = fibUtil.getVpnInstance(vrfEntry.getParentVpnRd());
394             keyVpnToDpnListMap = parentVpnInstance != null ? parentVpnInstance.nonnullVpnToDpnList() :
395                 vpnInstance.getVpnToDpnList();
396             LOG.info("createFibEntries: Processing creation of PNF FIB entry with rd {} prefix {}",
397                     vrfEntry.getParentVpnRd(), vrfEntry.getDestPrefix());
398         } else {
399             keyVpnToDpnListMap = vpnInstance.nonnullVpnToDpnList();
400         }
401         final Uint32 vpnId = vpnInstance.getVpnId();
402         final String rd = vrfTableKey.getRouteDistinguisher();
403         SubnetRoute subnetRoute = vrfEntry.augmentation(SubnetRoute.class);
404         if (subnetRoute != null) {
405             final long elanTag = subnetRoute.getElantag().toJava();
406             LOG.trace("SUBNETROUTE: createFibEntries: SubnetRoute augmented vrfentry found for rd {} prefix {}"
407                     + " with elantag {}", rd, vrfEntry.getDestPrefix(), elanTag);
408             if (keyVpnToDpnListMap != null) {
409                 jobCoordinator.enqueueJob(FibUtil.getJobKeyForRdPrefix(rd, vrfEntry.getDestPrefix()),
410                     () -> Collections.singletonList(
411                         txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION, tx -> {
412                             for (final VpnToDpnList curDpn : keyVpnToDpnListMap.values()) {
413                                 if (curDpn.getDpnState() == VpnToDpnList.DpnState.Active) {
414                                     installSubnetRouteInFib(curDpn.getDpnId(),
415                                                                 elanTag, rd, vpnId, vrfEntry, tx);
416                                     installSubnetBroadcastAddrDropRule(curDpn.getDpnId(), rd,
417                                         vpnId, vrfEntry, NwConstants.ADD_FLOW, tx);
418                                 }
419                             }
420                         })));
421             }
422             return;
423         }
424         // Get etherType value based on the IpPrefix address family type
425         int etherType;
426         try {
427             etherType = NWUtil.getEtherTypeFromIpPrefix(vrfEntry.getDestPrefix());
428         } catch (IllegalArgumentException ex) {
429             LOG.error("Unable to get etherType for IP Prefix {}", vrfEntry.getDestPrefix());
430             return;
431         }
432
433         final List<Uint64> localDpnIdList = createLocalFibEntry(vpnInstance.getVpnId(),
434                                                                         rd, vrfEntry, etherType);
435         if (!localDpnIdList.isEmpty() && keyVpnToDpnListMap != null) {
436             jobCoordinator.enqueueJob(FibUtil.getJobKeyForRdPrefix(rd, vrfEntry.getDestPrefix()),
437                 () -> Collections.singletonList(txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION, tx -> {
438                     final ReentrantLock lock = lockFor(vpnInstance);
439                     lock.lock();
440                     try {
441                         for (VpnToDpnList vpnDpn : keyVpnToDpnListMap.values()) {
442                             if (!localDpnIdList.contains(vpnDpn.getDpnId())) {
443                                 if (vpnDpn.getDpnState() == VpnToDpnList.DpnState.Active) {
444                                     try {
445                                         if (RouteOrigin.BGP.getValue().equals(vrfEntry.getOrigin())) {
446                                             bgpRouteVrfEntryHandler.createRemoteFibEntry(vpnDpn.getDpnId(),
447                                                     vpnId, vrfTableKey.getRouteDistinguisher(), vrfEntry,
448                                                     TransactionAdapter.toWriteTransaction(tx), txnObjects);
449                                         } else {
450                                             createRemoteFibEntry(vpnDpn.getDpnId(),
451                                                     vpnInstance.getVpnId(),
452                                                     vrfTableKey.getRouteDistinguisher(), vrfEntry, tx);
453                                         }
454                                     } catch (NullPointerException e) {
455                                         LOG.error("Failed to get create remote fib flows for prefix {} ",
456                                                 vrfEntry.getDestPrefix(), e);
457                                     }
458                                 }
459                             }
460                         }
461                     } finally {
462                         lock.unlock();
463                     }
464                 })), MAX_RETRIES);
465         }
466     }
467
468     void refreshFibTables(String rd, String prefix) {
469         InstanceIdentifier<VrfEntry> vrfEntryId =
470                 InstanceIdentifier.builder(FibEntries.class).child(VrfTables.class, new VrfTablesKey(rd))
471                         .child(VrfEntry.class, new VrfEntryKey(prefix)).build();
472         Optional<VrfEntry> vrfEntry;
473         try {
474             vrfEntry = SingleTransactionDataBroker.syncReadOptional(dataBroker, LogicalDatastoreType.CONFIGURATION,
475                     vrfEntryId);
476         } catch (ExecutionException | InterruptedException e) {
477             LOG.error("refreshFibTables: Exception while reading VrfEntry Ds for the prefix {} rd {}", prefix, rd, e);
478             return;
479         }
480         if (vrfEntry.isPresent()) {
481             createFibEntries(vrfEntryId, vrfEntry.get());
482         }
483     }
484
485     private Prefixes updateVpnReferencesInLri(LabelRouteInfo lri, String vpnInstanceName, boolean isPresentInList) {
486         LOG.debug("updating LRI : for label {} vpninstancename {}", lri.getLabel(), vpnInstanceName);
487         PrefixesBuilder prefixBuilder = new PrefixesBuilder();
488         prefixBuilder.setDpnId(lri.getDpnId());
489         prefixBuilder.setVpnInterfaceName(lri.getVpnInterfaceName());
490         prefixBuilder.setIpAddress(lri.getPrefix());
491         // Increment the refCount here
492         InstanceIdentifier<LabelRouteInfo> lriId = InstanceIdentifier.builder(LabelRouteMap.class)
493             .child(LabelRouteInfo.class, new LabelRouteInfoKey(lri.getLabel())).build();
494         LabelRouteInfoBuilder builder = new LabelRouteInfoBuilder(lri);
495         if (!isPresentInList) {
496             LOG.debug("vpnName {} is not present in LRI with label {}..", vpnInstanceName, lri.getLabel());
497             List<String> vpnInstanceNames =
498                 lri.getVpnInstanceList() != null ? new ArrayList<>(lri.getVpnInstanceList()) : new ArrayList<>();
499             vpnInstanceNames.add(vpnInstanceName);
500             builder.setVpnInstanceList(vpnInstanceNames);
501             MDSALUtil.syncWrite(dataBroker, LogicalDatastoreType.OPERATIONAL, lriId, builder.build());
502         } else {
503             LOG.debug("vpnName {} is present in LRI with label {}..", vpnInstanceName, lri.getLabel());
504         }
505         return prefixBuilder.build();
506     }
507
508     void installSubnetRouteInFib(final Uint64 dpnId, final long elanTag, final String rd,
509             final Uint32 vpnId, final VrfEntry vrfEntry, TypedWriteTransaction<Configuration> tx) {
510         if (tx == null) {
511             LoggingFutures.addErrorLogging(txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION,
512                 newTx -> installSubnetRouteInFib(dpnId, elanTag, rd, vpnId, vrfEntry, newTx)), LOG,
513                 "Error installing subnet route in FIB");
514             return;
515         }
516         int etherType;
517         try {
518             etherType = NWUtil.getEtherTypeFromIpPrefix(vrfEntry.getDestPrefix());
519         } catch (IllegalArgumentException ex) {
520             LOG.error("Unable to get etherType for IP Prefix {}", vrfEntry.getDestPrefix());
521             return;
522         }
523         final List<InstructionInfo> instructions = new ArrayList<>();
524         Uint64 subnetRouteMeta = Uint64.valueOf(BigInteger.valueOf(elanTag).shiftLeft(24)
525             .or(BigInteger.valueOf(vpnId.longValue()).shiftLeft(1)));
526         instructions.add(new InstructionWriteMetadata(subnetRouteMeta, MetaDataUtil.METADATA_MASK_SUBNET_ROUTE));
527         instructions.add(new InstructionGotoTable(NwConstants.L3_SUBNET_ROUTE_TABLE));
528         baseVrfEntryHandler.makeConnectedRoute(dpnId, vpnId, vrfEntry, rd, instructions,
529                 NwConstants.ADD_FLOW, TransactionAdapter.toWriteTransaction(tx), null);
530         if (vrfEntry.getRoutePaths() != null) {
531             for (RoutePaths routePath : vrfEntry.getRoutePaths().values()) {
532                 if (RouteOrigin.value(vrfEntry.getOrigin()) != RouteOrigin.SELF_IMPORTED) {
533                     List<ActionInfo> actionsInfos = new ArrayList<>();
534                     // reinitialize instructions list for LFIB Table
535                     final List<InstructionInfo> LFIBinstructions = new ArrayList<>();
536                     actionsInfos.add(new ActionPopMpls(etherType));
537                     LFIBinstructions.add(new InstructionApplyActions(actionsInfos));
538                     LFIBinstructions.add(new InstructionWriteMetadata(subnetRouteMeta,
539                             MetaDataUtil.METADATA_MASK_SUBNET_ROUTE));
540                     LFIBinstructions.add(new InstructionGotoTable(NwConstants.L3_SUBNET_ROUTE_TABLE));
541
542                     makeLFibTableEntry(dpnId, routePath.getLabel(), LFIBinstructions,
543                             DEFAULT_FIB_FLOW_PRIORITY, NwConstants.ADD_FLOW, tx);
544                 }
545             }
546         }
547     }
548
549     private void installSubnetBroadcastAddrDropRule(final Uint64 dpnId, final String rd, final Uint32 vpnId,
550             final VrfEntry vrfEntry, int addOrRemove, TypedWriteTransaction<Configuration> tx) {
551         List<MatchInfo> matches = new ArrayList<>();
552
553         LOG.debug("SUBNETROUTE: installSubnetBroadcastAddrDropRule: destPrefix {} rd {} vpnId {} dpnId {}",
554                 vrfEntry.getDestPrefix(), rd, vpnId, dpnId);
555         String[] ipAddress = vrfEntry.getDestPrefix().split("/");
556         String subnetBroadcastAddr = FibUtil.getBroadcastAddressFromCidr(vrfEntry.getDestPrefix());
557         final int prefixLength = ipAddress.length == 1 ? 0 : Integer.parseInt(ipAddress[1]);
558
559         InetAddress destPrefix;
560         try {
561             destPrefix = InetAddress.getByName(subnetBroadcastAddr);
562         } catch (UnknownHostException e) {
563             LOG.error("Failed to get destPrefix for prefix {} rd {} VpnId {} DPN {}",
564                     vrfEntry.getDestPrefix(), rd, vpnId, dpnId, e);
565             return;
566         }
567
568         // Match on VpnId and SubnetBroadCast IP address
569         matches.add(new MatchMetadata(MetaDataUtil.getVpnIdMetadata(vpnId.longValue()),
570             MetaDataUtil.METADATA_MASK_VRFID));
571         matches.add(MatchEthernetType.IPV4);
572
573         if (prefixLength != 0) {
574             matches.add(new MatchIpv4Destination(subnetBroadcastAddr, Integer.toString(IPV4_ADDR_PREFIX_LENGTH)));
575         }
576
577         //Action is to drop the packet
578         List<InstructionInfo> dropInstructions = new ArrayList<>();
579         List<ActionInfo> actionsInfos = new ArrayList<>();
580         actionsInfos.add(new ActionDrop());
581         dropInstructions.add(new InstructionApplyActions(actionsInfos));
582
583         int priority = DEFAULT_FIB_FLOW_PRIORITY + IPV4_ADDR_PREFIX_LENGTH;
584         String flowRef = FibUtil.getFlowRef(dpnId, NwConstants.L3_SUBNET_ROUTE_TABLE, rd, priority, destPrefix);
585         FlowEntity flowEntity = MDSALUtil.buildFlowEntity(dpnId, NwConstants.L3_SUBNET_ROUTE_TABLE, flowRef, priority,
586                 flowRef, 0, 0, COOKIE_TABLE_MISS, matches, dropInstructions);
587
588         Flow flow = flowEntity.getFlowBuilder().build();
589         String flowId = flowEntity.getFlowId();
590         FlowKey flowKey = new FlowKey(new FlowId(flowId));
591         Node nodeDpn = FibUtil.buildDpnNode(dpnId);
592
593         InstanceIdentifier<Flow> flowInstanceId = InstanceIdentifier.builder(Nodes.class)
594                 .child(Node.class, nodeDpn.key()).augmentation(FlowCapableNode.class)
595                 .child(Table.class, new TableKey(flow.getTableId())).child(Flow.class, flowKey).build();
596
597         if (addOrRemove == NwConstants.ADD_FLOW) {
598             tx.mergeParentStructurePut(flowInstanceId,flow);
599         } else {
600             tx.delete(flowInstanceId);
601         }
602     }
603
604     private List<Uint64> createLocalFibEntry(Uint32 vpnId, String rd, VrfEntry vrfEntry, int etherType) {
605         List<Uint64> returnLocalDpnId = new ArrayList<>();
606         String localNextHopIP = vrfEntry.getDestPrefix();
607         Prefixes localNextHopInfo = fibUtil.getPrefixToInterface(vpnId, localNextHopIP);
608         String vpnName = fibUtil.getVpnNameFromId(vpnId);
609         if (localNextHopInfo == null) {
610             boolean localNextHopSeen = false;
611             List<Routes> vpnExtraRoutes = null;
612             //Synchronized to prevent missing bucket action due to race condition between refreshFib and
613             // add/updateFib threads on missing nexthop in VpnToExtraroutes
614             // FIXME: use an Identifier structure?
615             final ReentrantLock lock = JvmGlobalLocks.getLockForString(localNextHopIP + FibConstants.SEPARATOR + rd);
616             lock.lock();
617             try {
618                 List<String> usedRds = VpnExtraRouteHelper.getUsedRds(dataBroker, vpnId, localNextHopIP);
619                 vpnExtraRoutes = VpnExtraRouteHelper.getAllVpnExtraRoutes(dataBroker,
620                         vpnName, usedRds, localNextHopIP);
621                 if (LOG.isDebugEnabled()) {
622                     LOG.debug("Creating Local fib entry with vpnName {} usedRds {} localNextHopIP {} vpnExtraRoutes {}",
623                             vpnName, usedRds, localNextHopIP, vpnExtraRoutes);
624                 }
625
626                 //Is this fib route an extra route? If yes, get the nexthop which would be an adjacency in the vpn
627                 for (Routes vpnExtraRoute : vpnExtraRoutes) {
628                     String ipPrefix;
629                     if (isIpv4Address(vpnExtraRoute.getNexthopIpList().get(0))) {
630                         ipPrefix = vpnExtraRoute.getNexthopIpList().get(0) + NwConstants.IPV4PREFIX;
631                     } else {
632                         ipPrefix = vpnExtraRoute.getNexthopIpList().get(0) + NwConstants.IPV6PREFIX;
633                     }
634                     Prefixes localNextHopInfoLocal = fibUtil.getPrefixToInterface(vpnId,
635                             ipPrefix);
636                     if (localNextHopInfoLocal != null) {
637                         localNextHopSeen = true;
638                         Uint64 dpnId =
639                                 checkCreateLocalFibEntry(localNextHopInfoLocal, localNextHopInfoLocal.getIpAddress(),
640                                         vpnId, rd, vrfEntry, vpnExtraRoute, vpnExtraRoutes, etherType,
641                                         /*parentVpnId*/ null);
642                         returnLocalDpnId.add(dpnId);
643                     }
644                 }
645             } finally {
646                 lock.unlock();
647             }
648             if (!localNextHopSeen && RouteOrigin.value(vrfEntry.getOrigin()) == RouteOrigin.SELF_IMPORTED) {
649                 java.util.Optional<Uint32> optionalLabel = FibUtil.getLabelFromRoutePaths(vrfEntry);
650                 if (optionalLabel.isPresent()) {
651                     Uint32 label = optionalLabel.get();
652                     List<String> nextHopAddressList = FibHelper.getNextHopListFromRoutePaths(vrfEntry);
653                     final LabelRouteInfoKey lriKey = new LabelRouteInfoKey(label);
654                     final ReentrantLock labelLock = lockFor(lriKey);
655                     labelLock.lock();
656                     try {
657                         LabelRouteInfo lri = getLabelRouteInfo(lriKey);
658                         Uint32 parentVpnId = lri.getParentVpnid();
659                         if (isPrefixAndNextHopPresentInLri(localNextHopIP, nextHopAddressList, lri)) {
660                             Optional<VpnInstanceOpDataEntry> vpnInstanceOpDataEntryOptional =
661                                     fibUtil.getVpnInstanceOpData(rd);
662                             if (vpnInstanceOpDataEntryOptional.isPresent()) {
663                                 String vpnInstanceName = vpnInstanceOpDataEntryOptional.get().getVpnInstanceName();
664                                 if (lri.getVpnInstanceList() != null && lri.getVpnInstanceList().contains(
665                                        vpnInstanceName)) {
666                                     localNextHopInfo = updateVpnReferencesInLri(lri, vpnInstanceName, true);
667                                     localNextHopIP = lri.getPrefix();
668                                 } else {
669                                     localNextHopInfo = updateVpnReferencesInLri(lri, vpnInstanceName, false);
670                                     localNextHopIP = lri.getPrefix();
671                                 }
672                             }
673                             if (localNextHopInfo != null) {
674                                 LOG.debug("Fetched labelRouteInfo for label {} interface {} and got dpn {}",
675                                         label, localNextHopInfo.getVpnInterfaceName(), lri.getDpnId());
676                                 if (vpnExtraRoutes.isEmpty()) {
677                                     Uint64 dpnId = checkCreateLocalFibEntry(localNextHopInfo, localNextHopIP,
678                                             vpnId, rd, vrfEntry, null, vpnExtraRoutes, etherType, parentVpnId);
679                                     returnLocalDpnId.add(dpnId);
680                                 } else {
681                                     for (Routes extraRoutes : vpnExtraRoutes) {
682                                         Uint64 dpnId = checkCreateLocalFibEntry(localNextHopInfo, localNextHopIP,
683                                                 vpnId, rd, vrfEntry, extraRoutes, vpnExtraRoutes, etherType,
684                                                 parentVpnId);
685                                         returnLocalDpnId.add(dpnId);
686                                     }
687                                 }
688                             }
689                         }
690                     } finally {
691                         labelLock.unlock();
692                     }
693                 }
694             }
695             if (returnLocalDpnId.isEmpty()) {
696                 LOG.error("Local DPNID is empty for rd {}, vpnId {}, vrfEntry {}", rd, vpnId, vrfEntry);
697             }
698         } else {
699             Uint64 dpnId = checkCreateLocalFibEntry(localNextHopInfo, localNextHopIP, vpnId,
700                     rd, vrfEntry, /*routes*/ null, /*vpnExtraRoutes*/ null, etherType,
701                     /*parentVpnId*/ null);
702             if (dpnId != null) {
703                 returnLocalDpnId.add(dpnId);
704             }
705         }
706         return returnLocalDpnId;
707     }
708
709     private Uint64 checkCreateLocalFibEntry(Prefixes localNextHopInfo, String localNextHopIP,
710                                                 final Uint32 vpnId, final String rd,
711                                                 final VrfEntry vrfEntry,
712                                                 @Nullable Routes routes, @Nullable List<Routes> vpnExtraRoutes,
713                                                 int etherType, Uint32 parentVpnId) {
714         String vpnName = fibUtil.getVpnNameFromId(vpnId);
715         if (localNextHopInfo != null) {
716             long groupId;
717             long localGroupId;
718             final Uint64 dpnId = localNextHopInfo.getDpnId();
719             if (Prefixes.PrefixCue.Nat.equals(localNextHopInfo.getPrefixCue())) {
720                 LOG.debug("checkCreateLocalFibEntry: NAT Prefix {} with vpnId {} rd {}. Skip local dpn {}"
721                         + " FIB processing", vrfEntry.getDestPrefix(), vpnId, rd, dpnId);
722                 return dpnId;
723             }
724             if (Prefixes.PrefixCue.PhysNetFunc.equals(localNextHopInfo.getPrefixCue())) {
725                 LOG.debug("checkCreateLocalFibEntry: PNF Prefix {} with vpnId {} rd {}. Skip local dpn {}"
726                         + " FIB processing", vrfEntry.getDestPrefix(), vpnId, rd, dpnId);
727                 return dpnId;
728             }
729             if (!isVpnPresentInDpn(rd, dpnId)) {
730                 LOG.error("checkCreateLocalFibEntry: The VPN with id {} rd {} is not available on dpn {}",
731                         vpnId, rd, dpnId.toString());
732                 return Uint64.ZERO;
733             }
734             String interfaceName = localNextHopInfo.getVpnInterfaceName();
735             String prefix = vrfEntry.getDestPrefix();
736             String gwMacAddress = vrfEntry.getGatewayMacAddress();
737             //The loadbalancing group is created only if the extra route has multiple nexthops
738             //to avoid loadbalancing the discovered routes
739             if (RouteOrigin.STATIC.getValue().equals(vrfEntry.getOrigin()) && vpnExtraRoutes != null
740                     && routes != null) {
741                 if (vpnExtraRoutes.size() > 1) {
742                     groupId = nextHopManager.createNextHopGroups(vpnId, rd, dpnId, vrfEntry, routes, vpnExtraRoutes);
743                     localGroupId = nextHopManager.getLocalSelectGroup(vpnId, vrfEntry.getDestPrefix());
744                 } else {
745                     groupId = nextHopManager.createNextHopGroups(vpnId, rd, dpnId, vrfEntry, routes, vpnExtraRoutes);
746                     localGroupId = groupId;
747                 }
748             } else {
749                 groupId = nextHopManager.createLocalNextHop(vpnId, dpnId, interfaceName, localNextHopIP, prefix,
750                         gwMacAddress, parentVpnId);
751                 localGroupId = groupId;
752             }
753             if (groupId == FibConstants.INVALID_GROUP_ID) {
754                 LOG.error("Unable to create Group for local prefix {} on rd {} for vpninterface {} on Node {}",
755                         prefix, rd, interfaceName, dpnId.toString());
756                 return Uint64.ZERO;
757             }
758             final List<InstructionInfo> instructions = Collections.singletonList(
759                     new InstructionApplyActions(
760                             Collections.singletonList(new ActionGroup(groupId))));
761             final List<InstructionInfo> lfibinstructions = Collections.singletonList(
762                     new InstructionApplyActions(
763                             Arrays.asList(new ActionPopMpls(etherType), new ActionGroup(localGroupId))));
764             java.util.Optional<Uint32> optLabel = FibUtil.getLabelFromRoutePaths(vrfEntry);
765             List<String> nextHopAddressList = FibHelper.getNextHopListFromRoutePaths(vrfEntry);
766             String jobKey = FibUtil.getCreateLocalNextHopJobKey(vpnId, dpnId, vrfEntry.getDestPrefix());
767             jobCoordinator.enqueueJob(jobKey,
768                 () -> Collections.singletonList(txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION, tx -> {
769                     baseVrfEntryHandler.makeConnectedRoute(dpnId, vpnId, vrfEntry, rd, instructions,
770                             NwConstants.ADD_FLOW, TransactionAdapter.toWriteTransaction(tx), null);
771                     if (FibUtil.isBgpVpn(vpnName, rd)) {
772                         optLabel.ifPresent(label -> {
773                             if (RouteOrigin.value(vrfEntry.getOrigin()) != RouteOrigin.SELF_IMPORTED) {
774                                 LOG.debug(
775                                         "Installing LFIB and tunnel table entry on dpn {} for interface {} with label "
776                                                 + "{}, rd {}, prefix {}, nexthop {}", dpnId,
777                                         localNextHopInfo.getVpnInterfaceName(), optLabel, rd, vrfEntry.getDestPrefix(),
778                                         nextHopAddressList);
779                                 makeLFibTableEntry(dpnId, label, lfibinstructions, DEFAULT_FIB_FLOW_PRIORITY,
780                                         NwConstants.ADD_FLOW, tx);
781                                 makeTunnelTableEntry(dpnId, label, localGroupId, tx);
782                             } else {
783                                 LOG.debug("Route with rd {} prefix {} label {} nexthop {} for vpn {} is an imported "
784                                                 + "route. LFib and Terminating table entries will not be created.",
785                                         rd, vrfEntry.getDestPrefix(), optLabel, nextHopAddressList, vpnId);
786                             }
787                         });
788                     }
789                 })));
790             return dpnId;
791         }
792         LOG.error("localNextHopInfo received is null for prefix {} on rd {} on vpn {}", vrfEntry.getDestPrefix(), rd,
793                 vpnName);
794         return Uint64.ZERO;
795     }
796
797     private boolean isVpnPresentInDpn(String rd, Uint64 dpnId) {
798         InstanceIdentifier<VpnToDpnList> id = VpnHelper.getVpnToDpnListIdentifier(rd, dpnId);
799         Optional<VpnToDpnList> dpnInVpn;
800         try {
801             dpnInVpn = SingleTransactionDataBroker.syncReadOptional(dataBroker, LogicalDatastoreType.OPERATIONAL, id);
802         } catch (ExecutionException | InterruptedException e) {
803             LOG.error("isVpnPresentInDpn: Exception while reading VpnToDpnList Ds for the rd {} dpnId {}", rd,
804                     dpnId, e);
805             return false;
806         }
807         return dpnInVpn.isPresent();
808     }
809
810     @Nullable
811     private LabelRouteInfo getLabelRouteInfo(Uint32 label) {
812         return getLabelRouteInfo(new LabelRouteInfoKey(label));
813     }
814
815     @Nullable
816     private LabelRouteInfo getLabelRouteInfo(LabelRouteInfoKey label) {
817         InstanceIdentifier<LabelRouteInfo> lriIid = InstanceIdentifier.builder(LabelRouteMap.class)
818             .child(LabelRouteInfo.class, label).build();
819         Optional<LabelRouteInfo> opResult = null;
820         try {
821             opResult = SingleTransactionDataBroker.syncReadOptional(dataBroker, LogicalDatastoreType.OPERATIONAL,
822                     lriIid);
823         } catch (ExecutionException | InterruptedException e) {
824             LOG.error("refreshFibTables: Exception while reading LabelRouteInfo Ds for the label {}", label, e);
825             return null;
826         }
827         if (opResult.isPresent()) {
828             return opResult.get();
829         }
830         return null;
831     }
832
833     @SuppressFBWarnings
834     private boolean deleteLabelRouteInfo(LabelRouteInfo lri, String vpnInstanceName,
835             @Nullable TypedWriteTransaction<Operational> tx) {
836         if (lri == null) {
837             return true;
838         }
839         LOG.debug("deleting LRI : for label {} vpninstancename {}", lri.getLabel(), vpnInstanceName);
840         InstanceIdentifier<LabelRouteInfo> lriId = InstanceIdentifier.builder(LabelRouteMap.class)
841             .child(LabelRouteInfo.class, new LabelRouteInfoKey(lri.getLabel())).build();
842
843         List<String> vpnInstancesList = lri.getVpnInstanceList() != null
844             ? new ArrayList<>(lri.getVpnInstanceList()) : new ArrayList<>();
845         if (vpnInstancesList.contains(vpnInstanceName)) {
846             LOG.debug("vpninstance {} name is present", vpnInstanceName);
847             vpnInstancesList.remove(vpnInstanceName);
848         }
849         if (vpnInstancesList.isEmpty()) {
850             LOG.debug("deleting LRI instance object for label {}", lri.getLabel());
851             if (tx != null) {
852                 tx.delete(lriId);
853             } else {
854                 MDSALUtil.syncDelete(dataBroker, LogicalDatastoreType.OPERATIONAL, lriId);
855             }
856             return true;
857         } else {
858             LOG.debug("updating LRI instance object for label {}", lri.getLabel());
859             LabelRouteInfoBuilder builder = new LabelRouteInfoBuilder(lri).setVpnInstanceList(vpnInstancesList);
860             MDSALUtil.syncWrite(dataBroker, LogicalDatastoreType.OPERATIONAL, lriId, builder.build());
861         }
862         return false;
863     }
864
865     void makeTunnelTableEntry(Uint64 dpId, Uint32 label, long groupId/*String egressInterfaceName*/,
866                                       TypedWriteTransaction<Configuration> tx) {
867         List<ActionInfo> actionsInfos = Collections.singletonList(new ActionGroup(groupId));
868
869         createTerminatingServiceActions(dpId, label, actionsInfos, tx);
870
871         LOG.debug("Terminating service Entry for dpID {} : label : {} egress : {} installed successfully",
872             dpId, label, groupId);
873     }
874
875     public void createTerminatingServiceActions(Uint64 destDpId, Uint32 label, List<ActionInfo> actionsInfos,
876                                                 TypedWriteTransaction<Configuration> tx) {
877         List<MatchInfo> mkMatches = new ArrayList<>();
878
879         LOG.debug("create terminatingServiceAction on DpnId = {} and serviceId = {} and actions = {}",
880             destDpId, label, actionsInfos);
881
882         // Matching metadata
883         // FIXME vxlan vni bit set is not working properly with OVS.need to revisit
884         mkMatches.add(new MatchTunnelId(Uint64.valueOf(label.longValue())));
885
886         List<InstructionInfo> mkInstructions = new ArrayList<>();
887         mkInstructions.add(new InstructionApplyActions(actionsInfos));
888
889         FlowEntity terminatingServiceTableFlowEntity =
890             MDSALUtil.buildFlowEntity(destDpId, NwConstants.INTERNAL_TUNNEL_TABLE,
891             getTableMissFlowRef(destDpId, NwConstants.INTERNAL_TUNNEL_TABLE, label),
892                     FibConstants.DEFAULT_VPN_INTERNAL_TUNNEL_TABLE_PRIORITY,
893                     String.format("%s:%s", "TST Flow Entry ", label), 0, 0,
894                     Uint64.valueOf(COOKIE_TUNNEL.longValue() + label.longValue()),
895                     mkMatches, mkInstructions);
896
897         FlowKey flowKey = new FlowKey(new FlowId(terminatingServiceTableFlowEntity.getFlowId()));
898
899         FlowBuilder flowbld = terminatingServiceTableFlowEntity.getFlowBuilder();
900
901         Node nodeDpn = FibUtil.buildDpnNode(terminatingServiceTableFlowEntity.getDpnId());
902         InstanceIdentifier<Flow> flowInstanceId = InstanceIdentifier.builder(Nodes.class)
903             .child(Node.class, nodeDpn.key()).augmentation(FlowCapableNode.class)
904             .child(Table.class, new TableKey(terminatingServiceTableFlowEntity.getTableId()))
905             .child(Flow.class, flowKey).build();
906         tx.mergeParentStructurePut(flowInstanceId, flowbld.build());
907     }
908
909     private void removeTunnelTableEntry(Uint64 dpId, Uint32 label, TypedWriteTransaction<Configuration> tx) {
910         FlowEntity flowEntity;
911         LOG.debug("remove terminatingServiceActions called with DpnId = {} and label = {}", dpId, label);
912         List<MatchInfo> mkMatches = new ArrayList<>();
913         // Matching metadata
914         mkMatches.add(new MatchTunnelId(Uint64.valueOf(label.longValue())));
915         flowEntity = MDSALUtil.buildFlowEntity(dpId,
916             NwConstants.INTERNAL_TUNNEL_TABLE,
917             getTableMissFlowRef(dpId, NwConstants.INTERNAL_TUNNEL_TABLE, label),
918                 FibConstants.DEFAULT_VPN_INTERNAL_TUNNEL_TABLE_PRIORITY,
919                 String.format("%s:%s", "TST Flow Entry ", label), 0, 0,
920                 Uint64.valueOf(COOKIE_TUNNEL.longValue() + label.longValue()), mkMatches, null);
921         Node nodeDpn = FibUtil.buildDpnNode(flowEntity.getDpnId());
922         FlowKey flowKey = new FlowKey(new FlowId(flowEntity.getFlowId()));
923         InstanceIdentifier<Flow> flowInstanceId = InstanceIdentifier.builder(Nodes.class)
924             .child(Node.class, nodeDpn.key()).augmentation(FlowCapableNode.class)
925             .child(Table.class, new TableKey(flowEntity.getTableId())).child(Flow.class, flowKey).build();
926
927         tx.delete(flowInstanceId);
928         LOG.debug("Terminating service Entry for dpID {} : label : {} removed successfully", dpId, label);
929     }
930
931     public List<Uint64> deleteLocalFibEntry(Uint32 vpnId, String rd, VrfEntry vrfEntry) {
932         List<Uint64> returnLocalDpnId = new ArrayList<>();
933         Prefixes localNextHopInfo = fibUtil.getPrefixToInterface(vpnId, vrfEntry.getDestPrefix());
934         String vpnName = fibUtil.getVpnNameFromId(vpnId);
935         boolean shouldUpdateNonEcmpLocalNextHop = true;
936         if (localNextHopInfo == null) {
937             List<String> usedRds = VpnExtraRouteHelper.getUsedRds(dataBroker, vpnId, vrfEntry.getDestPrefix());
938             if (usedRds.size() > 1) {
939                 LOG.error("The extra route prefix {} is still present in some DPNs in vpn {} on rd {}",
940                         vrfEntry.getDestPrefix(), vpnName, rd);
941                 return returnLocalDpnId;
942             }
943             String vpnRd = !usedRds.isEmpty() ? usedRds.get(0) : rd;
944             //Is this fib route an extra route? If yes, get the nexthop which would be an adjacency
945             //in the vpn
946             Optional<Routes> extraRouteOptional = VpnExtraRouteHelper.getVpnExtraroutes(dataBroker,
947                     vpnName, vpnRd, vrfEntry.getDestPrefix());
948             if (extraRouteOptional.isPresent()) {
949                 Routes extraRoute = extraRouteOptional.get();
950                 String ipPrefix;
951                 if (isIpv4Address(extraRoute.getNexthopIpList().get(0))) {
952                     ipPrefix = extraRoute.getNexthopIpList().get(0) + NwConstants.IPV4PREFIX;
953                 } else {
954                     ipPrefix = extraRoute.getNexthopIpList().get(0) + NwConstants.IPV6PREFIX;
955                 }
956                 if (extraRoute.getNexthopIpList().size() > 1) {
957                     shouldUpdateNonEcmpLocalNextHop = false;
958                 }
959                 localNextHopInfo = fibUtil.getPrefixToInterface(vpnId, ipPrefix);
960                 if (localNextHopInfo != null) {
961                     String localNextHopIP = localNextHopInfo.getIpAddress();
962                     Uint64 dpnId = checkDeleteLocalFibEntry(localNextHopInfo, localNextHopIP, vpnName, vpnId, rd,
963                             vrfEntry, shouldUpdateNonEcmpLocalNextHop);
964                     if (!dpnId.equals(Uint64.ZERO)) {
965                         LOG.trace("Deleting ECMP group for prefix {}, dpn {}", vrfEntry.getDestPrefix(), dpnId);
966                         nextHopManager.deleteLoadBalancingNextHop(vpnId, dpnId, vrfEntry.getDestPrefix());
967                         returnLocalDpnId.add(dpnId);
968                     }
969                 } else {
970                     LOG.error("localNextHopInfo unavailable while deleting prefix {} with rds {}, primary rd {} in "
971                             + "vpn {}", vrfEntry.getDestPrefix(), usedRds, rd, vpnName);
972                 }
973             }
974
975             if (localNextHopInfo == null) {
976                 /* Imported VRF entry */
977                 java.util.Optional<Uint32> optionalLabel = FibUtil.getLabelFromRoutePaths(vrfEntry);
978                 if (optionalLabel.isPresent()) {
979                     Uint32 label = optionalLabel.get();
980                     List<String> nextHopAddressList = FibHelper.getNextHopListFromRoutePaths(vrfEntry);
981                     LabelRouteInfo lri = getLabelRouteInfo(label);
982                     if (isPrefixAndNextHopPresentInLri(vrfEntry.getDestPrefix(), nextHopAddressList, lri)) {
983                         PrefixesBuilder prefixBuilder = new PrefixesBuilder();
984                         prefixBuilder.setDpnId(lri.getDpnId());
985                         Uint64 dpnId = checkDeleteLocalFibEntry(prefixBuilder.build(), nextHopAddressList.get(0),
986                                 vpnName, vpnId, rd, vrfEntry, shouldUpdateNonEcmpLocalNextHop);
987                         if (!dpnId.equals(Uint64.ZERO)) {
988                             returnLocalDpnId.add(dpnId);
989                         }
990                     }
991                 }
992             }
993
994         } else {
995             LOG.debug("Obtained prefix to interface for rd {} prefix {}", rd, vrfEntry.getDestPrefix());
996             String localNextHopIP = localNextHopInfo.getIpAddress();
997             Uint64 dpnId = checkDeleteLocalFibEntry(localNextHopInfo, localNextHopIP, vpnName, vpnId, rd, vrfEntry,
998                     shouldUpdateNonEcmpLocalNextHop);
999             if (!dpnId.equals(Uint64.ZERO)) {
1000                 returnLocalDpnId.add(dpnId);
1001             }
1002         }
1003
1004         return returnLocalDpnId;
1005     }
1006
1007     private Uint64 checkDeleteLocalFibEntry(Prefixes localNextHopInfo, final String localNextHopIP,
1008             final String vpnName, final Uint32 vpnId, final String rd, final VrfEntry vrfEntry,
1009             boolean shouldUpdateNonEcmpLocalNextHop) {
1010         if (localNextHopInfo != null) {
1011             final Uint64 dpnId = localNextHopInfo.getDpnId();
1012             if (Prefixes.PrefixCue.Nat.equals(localNextHopInfo.getPrefixCue())) {
1013                 LOG.debug("checkDeleteLocalFibEntry: NAT Prefix {} with vpnId {} rd {}. Skip local dpn {}"
1014                         + " FIB processing", vrfEntry.getDestPrefix(), vpnId, rd, dpnId);
1015                 return dpnId;
1016             }
1017             if (Prefixes.PrefixCue.PhysNetFunc.equals(localNextHopInfo.getPrefixCue())) {
1018                 LOG.debug("checkDeleteLocalFibEntry: PNF Prefix {} with vpnId {} rd {}. Skip local dpn {}"
1019                         + " FIB processing", vrfEntry.getDestPrefix(), vpnId, rd, dpnId);
1020                 return dpnId;
1021             }
1022
1023             jobCoordinator.enqueueJob(FibUtil.getCreateLocalNextHopJobKey(vpnId, dpnId, vrfEntry.getDestPrefix()),
1024                 () -> Collections.singletonList(txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION, tx -> {
1025                     baseVrfEntryHandler.makeConnectedRoute(dpnId, vpnId, vrfEntry, rd, null,
1026                             NwConstants.DEL_FLOW, TransactionAdapter.toWriteTransaction(tx), null);
1027                     if (FibUtil.isBgpVpn(vpnName, rd)) {
1028                         if (RouteOrigin.value(vrfEntry.getOrigin()) != RouteOrigin.SELF_IMPORTED) {
1029                             FibUtil.getLabelFromRoutePaths(vrfEntry).ifPresent(label -> {
1030                                 makeLFibTableEntry(dpnId, label, null /* instructions */, DEFAULT_FIB_FLOW_PRIORITY,
1031                                         NwConstants.DEL_FLOW, tx);
1032                                 removeTunnelTableEntry(dpnId, label, tx);
1033                             });
1034                         }
1035                     }
1036                 })));
1037             //TODO: verify below adjacency call need to be optimized (?)
1038             //In case of the removal of the extra route, the loadbalancing group is updated
1039             if (shouldUpdateNonEcmpLocalNextHop) {
1040                 baseVrfEntryHandler.deleteLocalAdjacency(dpnId, vpnId, localNextHopIP, vrfEntry.getDestPrefix());
1041             }
1042             return dpnId;
1043         }
1044         return Uint64.ZERO;
1045     }
1046
1047     private void createRemoteFibEntry(final Uint64 remoteDpnId, final Uint32 vpnId, String rd,
1048             final VrfEntry vrfEntry, TypedWriteTransaction<Configuration> tx) {
1049         if (tx == null) {
1050             LoggingFutures.addErrorLogging(txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION,
1051                 newTx -> createRemoteFibEntry(remoteDpnId, vpnId, rd, vrfEntry, newTx)), LOG,
1052                 "Error creating remote FIB entry");
1053             return;
1054         }
1055
1056         String vpnName = fibUtil.getVpnNameFromId(vpnId);
1057         LOG.debug("createremotefibentry: adding route {} for rd {} on remoteDpnId {}", vrfEntry.getDestPrefix(), rd,
1058                 remoteDpnId);
1059
1060         if (RouteOrigin.value(vrfEntry.getOrigin()) != RouteOrigin.STATIC) {
1061             programRemoteFibEntry(remoteDpnId, vpnId, rd, vrfEntry, tx);
1062             return;
1063         }
1064         // Handling static VRF entries
1065         List<String> usedRds = VpnExtraRouteHelper.getUsedRds(dataBroker, vpnId, vrfEntry.getDestPrefix());
1066         List<Routes> vpnExtraRoutes =
1067                 VpnExtraRouteHelper.getAllVpnExtraRoutes(dataBroker, vpnName, usedRds, vrfEntry.getDestPrefix());
1068         if (!vpnExtraRoutes.isEmpty()) {
1069             programRemoteFibWithLoadBalancingGroups(remoteDpnId, vpnId, rd, vrfEntry, vpnExtraRoutes);
1070         } else {
1071             // Program in case of other static VRF entries like floating IPs
1072             programRemoteFibEntry(remoteDpnId, vpnId, rd, vrfEntry, tx);
1073         }
1074     }
1075
1076     // Allow deprecated TransactionRunner calls for now
1077     @SuppressWarnings("ForbidCertainMethod")
1078     private void programRemoteFibWithLoadBalancingGroups(final Uint64 remoteDpnId, final Uint32 vpnId, String rd,
1079             final VrfEntry vrfEntry, List<Routes> vpnExtraRoutes) {
1080         // create loadbalancing groups for extra routes only when the extra route is
1081         // present behind multiple VMs
1082         // Obtain the local routes for this particular dpn.
1083         java.util.Optional<Routes> routes = vpnExtraRoutes.stream().filter(route -> {
1084             Prefixes prefixToInterface =
1085                     fibUtil.getPrefixToInterface(vpnId, FibUtil.getIpPrefix(route.getNexthopIpList().get(0)));
1086             if (prefixToInterface == null) {
1087                 return false;
1088             }
1089             return remoteDpnId.equals(prefixToInterface.getDpnId());
1090         }).findFirst();
1091         long groupId = nextHopManager.createNextHopGroups(vpnId, rd, remoteDpnId, vrfEntry,
1092                 routes.isPresent() ? routes.get() : null, vpnExtraRoutes);
1093         if (groupId == FibConstants.INVALID_GROUP_ID) {
1094             LOG.error("Unable to create Group for local prefix {} on rd {} on Node {}", vrfEntry.getDestPrefix(), rd,
1095                     remoteDpnId);
1096             return;
1097         }
1098         List<ActionInfo> actionInfos = Collections.singletonList(new ActionGroup(groupId));
1099         List<InstructionInfo> instructions = Lists.newArrayList(new InstructionApplyActions(actionInfos));
1100         String jobKey = FibUtil.getCreateRemoteNextHopJobKey(vpnId, remoteDpnId, vrfEntry.getDestPrefix());
1101         jobCoordinator.enqueueJob(jobKey,
1102             () -> Collections.singletonList(txRunner.callWithNewWriteOnlyTransactionAndSubmit(Datastore.CONFIGURATION,
1103                 txn -> {
1104                     baseVrfEntryHandler.makeConnectedRoute(remoteDpnId, vpnId, vrfEntry, rd, instructions,
1105                             NwConstants.ADD_FLOW, TransactionAdapter.toWriteTransaction(txn), null);
1106                 })));
1107
1108         LOG.debug("Successfully added FIB entry for prefix {} in vpnId {}", vrfEntry.getDestPrefix(), vpnId);
1109     }
1110
1111     private void programRemoteFibEntry(final Uint64 remoteDpnId, final Uint32 vpnId, String rd,
1112             final VrfEntry vrfEntry, TypedWriteTransaction<Configuration> tx) {
1113         List<AdjacencyResult> adjacencyResults = baseVrfEntryHandler.resolveAdjacency(remoteDpnId, vpnId, vrfEntry, rd);
1114         if (adjacencyResults.isEmpty()) {
1115             LOG.error("Could not get interface for route-paths: {} in vpn {} on DPN {}", vrfEntry.getRoutePaths(), rd,
1116                     remoteDpnId);
1117             LOG.error("Failed to add Route: {} in vpn: {}", vrfEntry.getDestPrefix(), rd);
1118             return;
1119         }
1120         baseVrfEntryHandler.programRemoteFib(remoteDpnId, vpnId, vrfEntry, TransactionAdapter.toWriteTransaction(tx),
1121                 rd, adjacencyResults, null);
1122         LOG.debug("Successfully programmed FIB entry for prefix {} in vpnId {}", vrfEntry.getDestPrefix(), vpnId);
1123     }
1124
1125     protected void cleanUpOpDataForFib(Uint32 vpnId, String primaryRd, final VrfEntry vrfEntry) {
1126     /* Get interface info from prefix to interface mapping;
1127         Use the interface info to get the corresponding vpn interface op DS entry,
1128         remove the adjacency corresponding to this fib entry.
1129         If adjacency removed is the last adjacency, clean up the following:
1130          - vpn interface from dpntovpn list, dpn if last vpn interface on dpn
1131          - prefix to interface entry
1132          - vpn interface op DS
1133      */
1134         LOG.debug("Cleanup of prefix {} in VPN {}", vrfEntry.getDestPrefix(), vpnId);
1135         Prefixes prefixInfo = fibUtil.getPrefixToInterface(vpnId, vrfEntry.getDestPrefix());
1136         if (prefixInfo == null) {
1137             List<String> usedRds = VpnExtraRouteHelper.getUsedRds(dataBroker, vpnId, vrfEntry.getDestPrefix());
1138             String usedRd = usedRds.isEmpty() ? primaryRd : usedRds.get(0);
1139             Routes extraRoute = baseVrfEntryHandler.getVpnToExtraroute(vpnId, usedRd, vrfEntry.getDestPrefix());
1140             if (extraRoute != null && extraRoute.getNexthopIpList() != null) {
1141                 for (String nextHopIp : extraRoute.getNexthopIpList()) {
1142                     LOG.debug("NextHop IP for destination {} is {}", vrfEntry.getDestPrefix(), nextHopIp);
1143                     if (nextHopIp != null) {
1144                         String ipPrefix;
1145                         if (isIpv4Address(nextHopIp)) {
1146                             ipPrefix = nextHopIp + NwConstants.IPV4PREFIX;
1147                         } else {
1148                             ipPrefix = nextHopIp + NwConstants.IPV6PREFIX;
1149                         }
1150                         prefixInfo = fibUtil.getPrefixToInterface(vpnId, ipPrefix);
1151                         checkCleanUpOpDataForFib(prefixInfo, vpnId, primaryRd, vrfEntry, extraRoute);
1152                     }
1153                 }
1154             }
1155             if (prefixInfo == null) {
1156                 java.util.Optional<Uint32> optionalLabel = FibUtil.getLabelFromRoutePaths(vrfEntry);
1157                 if (optionalLabel.isPresent()) {
1158                     Uint32 label = optionalLabel.get();
1159                     List<String> nextHopAddressList = FibHelper.getNextHopListFromRoutePaths(vrfEntry);
1160                     LabelRouteInfo lri = getLabelRouteInfo(label);
1161                     if (isPrefixAndNextHopPresentInLri(vrfEntry.getDestPrefix(), nextHopAddressList, lri)) {
1162                         PrefixesBuilder prefixBuilder = new PrefixesBuilder();
1163                         prefixBuilder.setDpnId(lri.getDpnId());
1164                         prefixBuilder.setVpnInterfaceName(lri.getVpnInterfaceName());
1165                         prefixBuilder.setIpAddress(lri.getPrefix());
1166                         prefixInfo = prefixBuilder.build();
1167                         LOG.debug("Fetched labelRouteInfo for label {} interface {} and got dpn {}",
1168                                 label, prefixInfo.getVpnInterfaceName(), lri.getDpnId());
1169                         checkCleanUpOpDataForFib(prefixInfo, vpnId, primaryRd, vrfEntry, extraRoute);
1170                     }
1171                 }
1172             }
1173         } else {
1174             checkCleanUpOpDataForFib(prefixInfo, vpnId, primaryRd, vrfEntry, null /*Routes*/);
1175         }
1176     }
1177
1178     private void checkCleanUpOpDataForFib(final Prefixes prefixInfo, final Uint32 vpnId, final String rd,
1179                                           final VrfEntry vrfEntry, @Nullable final Routes extraRoute) {
1180
1181         if (prefixInfo == null) {
1182             LOG.error("Cleanup VPN Data Failed as unable to find prefix Info for prefix {} VpnId {} rd {}",
1183                     vrfEntry.getDestPrefix(), vpnId, rd);
1184             return; //Don't have any info for this prefix (shouldn't happen); need to return
1185         }
1186
1187         if (Prefixes.PrefixCue.Nat.equals(prefixInfo.getPrefixCue())) {
1188             LOG.debug("NAT Prefix {} with vpnId {} rd {}. Skip FIB processing",
1189                     vrfEntry.getDestPrefix(), vpnId, rd);
1190             return;
1191         }
1192
1193         String ifName = prefixInfo.getVpnInterfaceName();
1194         jobCoordinator.enqueueJob("VPNINTERFACE-" + ifName,
1195             new CleanupVpnInterfaceWorker(prefixInfo, vpnId, rd, vrfEntry, extraRoute));
1196     }
1197
1198     private class CleanupVpnInterfaceWorker implements Callable<List<? extends ListenableFuture<?>>> {
1199         Prefixes prefixInfo;
1200         Uint32 vpnId;
1201         String rd;
1202         VrfEntry vrfEntry;
1203         Routes extraRoute;
1204
1205         CleanupVpnInterfaceWorker(final Prefixes prefixInfo, final Uint32 vpnId, final String rd,
1206                                          final VrfEntry vrfEntry, final Routes extraRoute) {
1207             this.prefixInfo = prefixInfo;
1208             this.vpnId = vpnId;
1209             this.rd = rd;
1210             this.vrfEntry = vrfEntry;
1211             this.extraRoute = extraRoute;
1212         }
1213
1214         @Override
1215         public List<? extends ListenableFuture<?>> call() {
1216             // If another renderer(for eg : CSS) needs to be supported, check can be performed here
1217             // to call the respective helpers.
1218             return Collections.singletonList(txRunner.callWithNewReadWriteTransactionAndSubmit(OPERATIONAL, tx -> {
1219                 //First Cleanup LabelRouteInfo
1220                 //TODO(KIRAN) : Move the below block when addressing iRT/eRT for L3VPN Over VxLan
1221                 LOG.debug("cleanupVpnInterfaceWorker: rd {} prefix {}", rd, prefixInfo.getIpAddress());
1222                 if (VrfEntry.EncapType.Mplsgre.equals(vrfEntry.getEncapType())) {
1223                     FibUtil.getLabelFromRoutePaths(vrfEntry).ifPresent(label -> {
1224                         List<String> nextHopAddressList = FibHelper.getNextHopListFromRoutePaths(vrfEntry);
1225                         final LabelRouteInfoKey lriKey = new LabelRouteInfoKey(label);
1226                         final ReentrantLock lock = lockFor(lriKey);
1227                         lock.lock();
1228                         try {
1229                             LabelRouteInfo lri = getLabelRouteInfo(lriKey);
1230                             if (lri != null && Objects.equals(lri.getPrefix(), vrfEntry.getDestPrefix())
1231                                     && nextHopAddressList.contains(lri.getNextHopIpList().get(0))) {
1232                                 Optional<VpnInstanceOpDataEntry> vpnInstanceOpDataEntryOptional =
1233                                         fibUtil.getVpnInstanceOpData(rd);
1234                                 String vpnInstanceName = "";
1235                                 if (vpnInstanceOpDataEntryOptional.isPresent()) {
1236                                     vpnInstanceName = vpnInstanceOpDataEntryOptional.get().getVpnInstanceName();
1237                                 }
1238                                 boolean lriRemoved = deleteLabelRouteInfo(lri, vpnInstanceName, tx);
1239                                 if (lriRemoved) {
1240                                     String parentRd = lri.getParentVpnRd();
1241                                     fibUtil.releaseId(FibConstants.VPN_IDPOOL_NAME, FibUtil.getNextHopLabelKey(
1242                                             parentRd, vrfEntry.getDestPrefix()));
1243                                 }
1244                             } else {
1245                                 fibUtil.releaseId(FibConstants.VPN_IDPOOL_NAME, FibUtil.getNextHopLabelKey(
1246                                         rd, vrfEntry.getDestPrefix()));
1247                             }
1248                         } finally {
1249                             lock.unlock();
1250                         }
1251                     });
1252                 }
1253                 String ifName = prefixInfo.getVpnInterfaceName();
1254                 Optional<String> optVpnName = fibUtil.getVpnNameFromRd(rd);
1255                 String vpnName = null;
1256
1257                 if (Prefixes.PrefixCue.PhysNetFunc.equals(prefixInfo.getPrefixCue())) {
1258                     // Get vpnId for rd = networkId since op vpnInterface will be pointing to rd = networkId
1259                     Optional<String> vpnNameOpt = fibUtil.getVpnNameFromRd(vrfEntry.getParentVpnRd());
1260                     if (vpnNameOpt.isPresent()) {
1261                         vpnId = fibUtil.getVpnId(vpnNameOpt.get());
1262                     }
1263                 }
1264                 if (optVpnName.isPresent()) {
1265                     vpnName = optVpnName.get();
1266                     Optional<VpnInterfaceOpDataEntry> opVpnInterface = tx
1267                             .read(FibUtil.getVpnInterfaceOpDataEntryIdentifier(ifName, vpnName)).get();
1268                     if (opVpnInterface.isPresent()) {
1269                         Uint32 associatedVpnId = fibUtil.getVpnId(vpnName);
1270                         if (!Objects.equals(vpnId, associatedVpnId)) {
1271                             LOG.warn("Prefixes {} are associated with different vpn instance with id {} rather than {}",
1272                                     vrfEntry.getDestPrefix(), associatedVpnId, vpnId);
1273                             LOG.warn("Not proceeding with Cleanup op data for prefix {}", vrfEntry.getDestPrefix());
1274                             return;
1275                         } else {
1276                             LOG.debug("Processing cleanup of prefix {} associated with vpn {}",
1277                                     vrfEntry.getDestPrefix(), associatedVpnId);
1278                         }
1279                     }
1280                 }
1281                 if (extraRoute != null) {
1282                     List<String> usedRds = VpnExtraRouteHelper.getUsedRds(dataBroker, vpnId, vrfEntry.getDestPrefix());
1283                     //Only one used Rd present in case of removal event
1284                     String usedRd = usedRds.get(0);
1285                     if (optVpnName.isPresent()) {
1286                         tx.delete(BaseVrfEntryHandler.getVpnToExtrarouteIdentifier(vpnName, usedRd,
1287                                         vrfEntry.getDestPrefix()));
1288                         txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION, configTx ->
1289                             configTx.delete(VpnExtraRouteHelper.getUsedRdsIdentifier(vpnId, vrfEntry.getDestPrefix())));
1290                     }
1291                 }
1292                 handleAdjacencyAndVpnOpInterfaceDeletion(vrfEntry, ifName, vpnName, tx);
1293             }));
1294         }
1295     }
1296
1297     /**
1298      * Check all the adjacency in VpnInterfaceOpData and decide whether to delete the entire interface or only adj.
1299      * Remove Adjacency from VPNInterfaceOpData.
1300      * if Adjacency != primary.
1301      * if Adjacency == primary , then mark it for deletion.
1302      * Remove entire VPNinterfaceOpData Entry.
1303      * if sie of Adjacency <= 2 and all are marked for deletion , delete the entire VPNinterface Op entry.
1304      * @param vrfEntry - VrfEntry removed
1305      * @param ifName - Interface name from VRFentry
1306      * @param vpnName - VPN name of corresponding VRF
1307      * @param tx - ReadWrite Tx
1308      */
1309     @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD",
1310             justification = "https://github.com/spotbugs/spotbugs/issues/811")
1311     private void handleAdjacencyAndVpnOpInterfaceDeletion(VrfEntry vrfEntry, String ifName, String vpnName,
1312                                                           TypedReadWriteTransaction<Operational> tx)
1313             throws ExecutionException, InterruptedException {
1314         InstanceIdentifier<Adjacency> adjacencyIid =
1315                 FibUtil.getAdjacencyIdentifierOp(ifName, vpnName, vrfEntry.getDestPrefix());
1316         Optional<Adjacency> adjacencyOptional = tx.read(adjacencyIid).get();
1317         if (adjacencyOptional.isPresent()) {
1318             if (adjacencyOptional.get().getAdjacencyType() != Adjacency.AdjacencyType.PrimaryAdjacency) {
1319                 tx.delete(FibUtil.getAdjacencyIdentifierOp(ifName, vpnName, vrfEntry.getDestPrefix()));
1320             } else {
1321                 tx.merge(adjacencyIid,
1322                         new AdjacencyBuilder(adjacencyOptional.get()).setMarkedForDeletion(true).build());
1323             }
1324         }
1325
1326         Optional<AdjacenciesOp> optAdjacencies = tx.read(FibUtil.getAdjListPathOp(ifName, vpnName)).get();
1327
1328         if (!optAdjacencies.isPresent() || optAdjacencies.get().getAdjacency() == null) {
1329             return;
1330         }
1331
1332         @NonNull List<Adjacency> adjacencies
1333                 = new ArrayList<Adjacency>(optAdjacencies.get().nonnullAdjacency().values());
1334         if (adjacencies.size() <= 2
1335                 && adjacencies.stream().allMatch(adjacency ->
1336                 adjacency.getAdjacencyType() == Adjacency.AdjacencyType.PrimaryAdjacency
1337                         && adjacency.isMarkedForDeletion() != null
1338                         && adjacency.isMarkedForDeletion()
1339         )) {
1340             LOG.info("Clean up vpn interface {} to vpn {} list.", ifName, vpnName);
1341             tx.delete(FibUtil.getVpnInterfaceOpDataEntryIdentifier(ifName, vpnName));
1342         }
1343     }
1344
1345     private void deleteFibEntries(final InstanceIdentifier<VrfEntry> identifier, final VrfEntry vrfEntry) {
1346         final VrfTablesKey vrfTableKey = identifier.firstKeyOf(VrfTables.class);
1347         final String rd = vrfTableKey.getRouteDistinguisher();
1348         final VpnInstanceOpDataEntry vpnInstance = fibUtil.getVpnInstance(vrfTableKey.getRouteDistinguisher());
1349         if (vpnInstance == null) {
1350             LOG.error("VPN Instance for rd {} is not available from VPN Op Instance Datastore", rd);
1351             return;
1352         }
1353         final Map<VpnToDpnListKey, VpnToDpnList> keyVpnToDpnListMap;
1354         if (vrfEntry.getParentVpnRd() != null
1355                 && FibHelper.isControllerManagedNonSelfImportedRoute(RouteOrigin.value(vrfEntry.getOrigin()))) {
1356             // This block MUST BE HIT only for PNF (Physical Network Function) FIB Entries.
1357             VpnInstanceOpDataEntry parentVpnInstance = fibUtil.getVpnInstance(vrfEntry.getParentVpnRd());
1358             keyVpnToDpnListMap = parentVpnInstance != null ? parentVpnInstance.getVpnToDpnList() :
1359                     vpnInstance.getVpnToDpnList();
1360             LOG.info("deleteFibEntries: Processing deletion of PNF FIB entry with rd {} prefix {}",
1361                     vrfEntry.getParentVpnRd(), vrfEntry.getDestPrefix());
1362         } else {
1363             keyVpnToDpnListMap = vpnInstance.getVpnToDpnList();
1364         }
1365
1366         SubnetRoute subnetRoute = vrfEntry.augmentation(SubnetRoute.class);
1367         final java.util.Optional<Uint32> optionalLabel = FibUtil.getLabelFromRoutePaths(vrfEntry);
1368         String vpnName = fibUtil.getVpnNameFromId(vpnInstance.getVpnId());
1369         if (subnetRoute != null) {
1370             long elanTag = subnetRoute.getElantag().toJava();
1371             LOG.trace("SUBNETROUTE: deleteFibEntries: SubnetRoute augmented vrfentry found for rd {} prefix {}"
1372                     + " with elantag {}", rd, vrfEntry.getDestPrefix(), elanTag);
1373             if (keyVpnToDpnListMap != null) {
1374                 jobCoordinator.enqueueJob(FibUtil.getJobKeyForRdPrefix(rd, vrfEntry.getDestPrefix()),
1375                     () -> Collections.singletonList(
1376                         txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION, tx -> {
1377                             for (final VpnToDpnList curDpn : keyVpnToDpnListMap.values()) {
1378
1379                                 baseVrfEntryHandler.makeConnectedRoute(curDpn.getDpnId(),
1380                                     vpnInstance.getVpnId(),
1381                                     vrfEntry, vrfTableKey.getRouteDistinguisher(), null,
1382                                         NwConstants.DEL_FLOW, TransactionAdapter.toWriteTransaction(tx), null);
1383                                 if (RouteOrigin.value(vrfEntry.getOrigin()) != RouteOrigin.SELF_IMPORTED) {
1384                                     optionalLabel.ifPresent(label -> makeLFibTableEntry(curDpn.getDpnId(),
1385                                         label, null, DEFAULT_FIB_FLOW_PRIORITY, NwConstants.DEL_FLOW, tx));
1386                                 }
1387
1388                                 installSubnetBroadcastAddrDropRule(curDpn.getDpnId(), rd,
1389                                     vpnInstance.getVpnId(),
1390                                     vrfEntry, NwConstants.DEL_FLOW, tx);
1391                             }
1392                         })));
1393             }
1394             return;
1395         }
1396
1397         final List<Uint64> localDpnIdList = deleteLocalFibEntry(vpnInstance.getVpnId(),
1398             vrfTableKey.getRouteDistinguisher(), vrfEntry);
1399         if (keyVpnToDpnListMap != null) {
1400             List<String> usedRds = VpnExtraRouteHelper.getUsedRds(dataBroker,
1401                     vpnInstance.getVpnId(), vrfEntry.getDestPrefix());
1402             String jobKey;
1403             Optional<Routes> extraRouteOptional;
1404             //Is this fib route an extra route? If yes, get the nexthop which would be an adjacency in the vpn
1405             if (usedRds != null && !usedRds.isEmpty()) {
1406                 if (usedRds.size() > 1) {
1407                     LOG.error("The extra route prefix is still present in some DPNs");
1408                     return ;
1409                 } else {
1410                     // The first rd is retrieved from usedrds as Only 1 rd would be present as extra route prefix
1411                     //is not present in any other DPN
1412                     extraRouteOptional = VpnExtraRouteHelper
1413                             .getVpnExtraroutes(dataBroker, vpnName, usedRds.get(0), vrfEntry.getDestPrefix());
1414                 }
1415             } else {
1416                 extraRouteOptional = Optional.empty();
1417             }
1418
1419             jobCoordinator.enqueueJob(FibUtil.getJobKeyForRdPrefix(rd, vrfEntry.getDestPrefix()),
1420                 () -> Collections.singletonList(txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION, tx -> {
1421                     if (localDpnIdList.size() <= 0) {
1422                         for (VpnToDpnList curDpn : keyVpnToDpnListMap.values()) {
1423                             baseVrfEntryHandler.deleteRemoteRoute(Uint64.ZERO, curDpn.getDpnId(),
1424                                 vpnInstance.getVpnId(), vrfTableKey, vrfEntry, extraRouteOptional,
1425                                     TransactionAdapter.toWriteTransaction(tx));
1426                         }
1427                     } else {
1428                         for (Uint64 localDpnId : localDpnIdList) {
1429                             for (VpnToDpnList curDpn : keyVpnToDpnListMap.values()) {
1430                                 if (!Objects.equals(curDpn.getDpnId(), localDpnId)) {
1431                                     baseVrfEntryHandler.deleteRemoteRoute(localDpnId, curDpn.getDpnId(),
1432                                         vpnInstance.getVpnId(), vrfTableKey, vrfEntry, extraRouteOptional,
1433                                             TransactionAdapter.toWriteTransaction(tx));
1434                                 }
1435                             }
1436                         }
1437                     }
1438                     if (extraRouteOptional.isPresent()) {
1439                         //Remove select groups only for extra-routes
1440                         nextHopManager.removeNextHopPointer(nextHopManager
1441                                 .getRemoteSelectGroupKey(vpnInstance.getVpnId(), vrfEntry.getDestPrefix()));
1442                         nextHopManager.removeNextHopPointer(nextHopManager
1443                                 .getLocalSelectGroupKey(vpnInstance.getVpnId(), vrfEntry.getDestPrefix()));
1444                     }
1445                 })), MAX_RETRIES);
1446         }
1447
1448         //The flow/group entry has been deleted from config DS; need to clean up associated operational
1449         //DS entries in VPN Op DS, VpnInstanceOpData and PrefixToInterface to complete deletion
1450         cleanUpOpDataForFib(vpnInstance.getVpnId(), vrfTableKey.getRouteDistinguisher(), vrfEntry);
1451
1452     }
1453
1454     private void makeLFibTableEntry(Uint64 dpId, Uint32 label, @Nullable List<InstructionInfo> instructions,
1455                                     int priority, int addOrRemove, TypedWriteTransaction<Configuration> tx) {
1456         if (tx == null) {
1457             LoggingFutures.addErrorLogging(txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION,
1458                 newTx -> makeLFibTableEntry(dpId, label, instructions, priority, addOrRemove, newTx)), LOG,
1459                 "Error making LFIB table entry");
1460             return;
1461         }
1462
1463         List<MatchInfo> matches = new ArrayList<>();
1464         matches.add(MatchEthernetType.MPLS_UNICAST);
1465         matches.add(new MatchMplsLabel(label.longValue()));
1466
1467         // Install the flow entry in L3_LFIB_TABLE
1468         String flowRef = FibUtil.getFlowRef(dpId, NwConstants.L3_LFIB_TABLE, label, priority);
1469
1470         FlowEntity flowEntity;
1471         flowEntity = MDSALUtil.buildFlowEntity(dpId, NwConstants.L3_LFIB_TABLE, flowRef, priority, flowRef, 0, 0,
1472             NwConstants.COOKIE_VM_LFIB_TABLE, matches, instructions);
1473         Flow flow = flowEntity.getFlowBuilder().build();
1474         String flowId = flowEntity.getFlowId();
1475         FlowKey flowKey = new FlowKey(new FlowId(flowId));
1476         Node nodeDpn = FibUtil.buildDpnNode(dpId);
1477         InstanceIdentifier<Flow> flowInstanceId = InstanceIdentifier.builder(Nodes.class)
1478             .child(Node.class, nodeDpn.key()).augmentation(FlowCapableNode.class)
1479             .child(Table.class, new TableKey(flow.getTableId())).child(Flow.class, flowKey).build();
1480
1481         if (addOrRemove == NwConstants.ADD_FLOW) {
1482             tx.mergeParentStructurePut(flowInstanceId, flow);
1483         } else {
1484             tx.delete(flowInstanceId);
1485         }
1486
1487         LOG.debug("LFIB Entry for dpID {} : label : {} instructions {} : key {} {} successfully",
1488             dpId, label, instructions, flowKey, NwConstants.ADD_FLOW == addOrRemove ? "ADDED" : "REMOVED");
1489     }
1490
1491     public void populateFibOnNewDpn(final Uint64 dpnId, final Uint32 vpnId, final String rd,
1492                                     final FutureCallback<List<?>> callback) {
1493         LOG.trace("New dpn {} for vpn {} : populateFibOnNewDpn", dpnId, rd);
1494         jobCoordinator.enqueueJob(FibUtil.getJobKeyForVpnIdDpnId(vpnId, dpnId),
1495             () -> {
1496                 InstanceIdentifier<VrfTables> id = buildVrfId(rd);
1497                 final VpnInstanceOpDataEntry vpnInstance = fibUtil.getVpnInstance(rd);
1498                 final Optional<VrfTables> vrfTable = MDSALUtil.read(dataBroker,
1499                         LogicalDatastoreType.CONFIGURATION, id);
1500                 List<ListenableFuture<?>> futures = new ArrayList<>();
1501                 if (!vrfTable.isPresent()) {
1502                     LOG.info("populateFibOnNewDpn: dpn: {}: VRF Table not yet available for RD {}", dpnId, rd);
1503                     if (callback != null) {
1504                         ListenableFuture<List<Object>> listenableFuture = Futures.allAsList(futures);
1505                         Futures.addCallback(listenableFuture, callback, MoreExecutors.directExecutor());
1506                     }
1507                     return futures;
1508                 }
1509
1510                 final ReentrantLock lock = lockFor(vpnInstance);
1511                 lock.lock();
1512                 try {
1513                     futures.add(retryingTxRunner.callWithNewReadWriteTransactionAndSubmit(CONFIGURATION, tx -> {
1514                         for (final VrfEntry vrfEntry : vrfTable.get().nonnullVrfEntry().values()) {
1515                             SubnetRoute subnetRoute = vrfEntry.augmentation(SubnetRoute.class);
1516                             if (subnetRoute != null) {
1517                                 long elanTag = subnetRoute.getElantag().toJava();
1518                                 installSubnetRouteInFib(dpnId, elanTag, rd, vpnId, vrfEntry, tx);
1519                                 installSubnetBroadcastAddrDropRule(dpnId, rd, vpnId, vrfEntry, NwConstants.ADD_FLOW,
1520                                         tx);
1521                                 continue;
1522                             }
1523                             RouterInterface routerInt = vrfEntry.augmentation(RouterInterface.class);
1524                             if (routerInt != null) {
1525                                 LOG.trace("Router augmented vrfentry found rd:{}, uuid:{}, ip:{}, mac:{}",
1526                                         rd, routerInt.getUuid(), routerInt.getIpAddress(), routerInt.getMacAddress());
1527                                 routerInterfaceVrfEntryHandler.installRouterFibEntry(vrfEntry, dpnId, vpnId,
1528                                         routerInt.getIpAddress(), new MacAddress(routerInt.getMacAddress()),
1529                                         NwConstants.ADD_FLOW);
1530                                 continue;
1531                             }
1532                             //Handle local flow creation for imports
1533                             if (RouteOrigin.value(vrfEntry.getOrigin()) == RouteOrigin.SELF_IMPORTED) {
1534                                 java.util.Optional<Uint32> optionalLabel = FibUtil.getLabelFromRoutePaths(vrfEntry);
1535                                 if (optionalLabel.isPresent()) {
1536                                     List<String> nextHopList = FibHelper.getNextHopListFromRoutePaths(vrfEntry);
1537                                     LabelRouteInfo lri = getLabelRouteInfo(optionalLabel.get());
1538                                     if (isPrefixAndNextHopPresentInLri(vrfEntry.getDestPrefix(), nextHopList, lri)) {
1539                                         if (Objects.equals(lri.getDpnId(), dpnId)) {
1540                                             try {
1541                                                 int etherType = NWUtil.getEtherTypeFromIpPrefix(
1542                                                         vrfEntry.getDestPrefix());
1543                                                 createLocalFibEntry(vpnId, rd, vrfEntry, etherType);
1544                                             } catch (IllegalArgumentException ex) {
1545                                                 LOG.warn("Unable to get etherType for IP Prefix {}",
1546                                                         vrfEntry.getDestPrefix());
1547                                             }
1548                                             continue;
1549                                         }
1550                                     }
1551                                 }
1552                             }
1553                             boolean shouldCreateRemoteFibEntry = shouldCreateFibEntryForVrfAndVpnIdOnDpn(vpnId,
1554                                     vrfEntry, dpnId);
1555                             if (shouldCreateRemoteFibEntry) {
1556                                 LOG.trace("Will create remote FIB entry for vrfEntry {} on DPN {}", vrfEntry, dpnId);
1557                                 if (RouteOrigin.BGP.getValue().equals(vrfEntry.getOrigin())) {
1558                                     List<SubTransaction> txnObjects =  new ArrayList<>();
1559                                     bgpRouteVrfEntryHandler.createRemoteFibEntry(dpnId, vpnId,
1560                                             vrfTable.get().getRouteDistinguisher(), vrfEntry,
1561                                             TransactionAdapter.toWriteTransaction(tx), txnObjects);
1562                                 } else {
1563                                     createRemoteFibEntry(dpnId, vpnId, vrfTable.get().getRouteDistinguisher(),
1564                                             vrfEntry, tx);
1565                                 }
1566                             }
1567                         }
1568                     }));
1569                     if (callback != null) {
1570                         ListenableFuture<List<Object>> listenableFuture = Futures.allAsList(futures);
1571                         Futures.addCallback(listenableFuture, callback, MoreExecutors.directExecutor());
1572                     }
1573                 } finally {
1574                     lock.unlock();
1575                 }
1576                 return futures;
1577             });
1578     }
1579
1580     public void populateExternalRoutesOnDpn(final Uint64 dpnId, final Uint32 vpnId, final String rd,
1581                                             final String localNextHopIp, final String remoteNextHopIp) {
1582         LOG.trace("populateExternalRoutesOnDpn : dpn {}, vpn {}, rd {}, localNexthopIp {} , remoteNextHopIp {} ",
1583             dpnId, vpnId, rd, localNextHopIp, remoteNextHopIp);
1584         InstanceIdentifier<VrfTables> id = buildVrfId(rd);
1585         final VpnInstanceOpDataEntry vpnInstance = fibUtil.getVpnInstance(rd);
1586         List<SubTransaction> txnObjects =  new ArrayList<>();
1587         final Optional<VrfTables> vrfTable;
1588         try {
1589             vrfTable = SingleTransactionDataBroker.syncReadOptional(dataBroker, LogicalDatastoreType.CONFIGURATION, id);
1590         } catch (ExecutionException | InterruptedException e) {
1591             LOG.error("populateExternalRoutesOnDpn: Exception while reading the VrfTable for the rd {}", rd, e);
1592             return;
1593         }
1594         if (vrfTable.isPresent()) {
1595             jobCoordinator.enqueueJob(FibUtil.getJobKeyForVpnIdDpnId(vpnId, dpnId),
1596                 () -> Collections.singletonList(txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION, tx -> {
1597                     final ReentrantLock lock = lockFor(vpnInstance);
1598                     lock.lock();
1599                     try {
1600                         vrfTable.get().nonnullVrfEntry().values().stream()
1601                             .filter(vrfEntry -> RouteOrigin.BGP == RouteOrigin.value(vrfEntry.getOrigin()))
1602                             .forEach(bgpRouteVrfEntryHandler.getConsumerForCreatingRemoteFib(dpnId, vpnId,
1603                                 rd, remoteNextHopIp, vrfTable, TransactionAdapter.toWriteTransaction(tx), txnObjects));
1604                     } finally {
1605                         lock.unlock();
1606                     }
1607                 })));
1608         }
1609     }
1610
1611     public void manageRemoteRouteOnDPN(final boolean action,
1612                                        final Uint64 localDpnId,
1613                                        final Uint32 vpnId,
1614                                        final String rd,
1615                                        final String destPrefix,
1616                                        final String destTepIp,
1617                                        final Uint32 label) {
1618         final VpnInstanceOpDataEntry vpnInstance = fibUtil.getVpnInstance(rd);
1619
1620         if (vpnInstance == null) {
1621             LOG.error("VpnInstance for rd {} not present for prefix {}", rd, destPrefix);
1622             return;
1623         }
1624
1625         jobCoordinator.enqueueJob(FibUtil.getJobKeyForVpnIdDpnId(vpnId, localDpnId),
1626             () -> Collections.singletonList(txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION, tx -> {
1627                 final ReentrantLock lock = lockFor(vpnInstance);
1628                 lock.lock();
1629                 try {
1630                     VrfTablesKey vrfTablesKey = new VrfTablesKey(rd);
1631                     VrfEntry vrfEntry = getVrfEntry(dataBroker, rd, destPrefix);
1632                     if (vrfEntry == null) {
1633                         return;
1634                     }
1635                     LOG.trace("manageRemoteRouteOnDPN :: action {}, DpnId {}, vpnId {}, rd {}, destPfx {}",
1636                             action, localDpnId, vpnId, rd, destPrefix);
1637                     Map<RoutePathsKey, RoutePaths> keyRoutePathsMap = vrfEntry.getRoutePaths();
1638                     VrfEntry modVrfEntry;
1639                     if (keyRoutePathsMap == null || keyRoutePathsMap.isEmpty()) {
1640                         modVrfEntry = FibHelper.getVrfEntryBuilder(vrfEntry, label,
1641                                 Collections.singletonList(destTepIp),
1642                                 RouteOrigin.value(vrfEntry.getOrigin()), null /* parentVpnRd */).build();
1643                     } else {
1644                         modVrfEntry = vrfEntry;
1645                     }
1646
1647                     if (action) {
1648                         LOG.trace("manageRemoteRouteOnDPN updated(add)  vrfEntry :: {}", modVrfEntry);
1649                         createRemoteFibEntry(localDpnId, vpnId, vrfTablesKey.getRouteDistinguisher(),
1650                                 modVrfEntry, tx);
1651                     } else {
1652                         LOG.trace("manageRemoteRouteOnDPN updated(remove)  vrfEntry :: {}", modVrfEntry);
1653                         List<String> usedRds = VpnExtraRouteHelper.getUsedRds(dataBroker,
1654                                 vpnInstance.getVpnId(), vrfEntry.getDestPrefix());
1655                         if (usedRds.size() > 1) {
1656                             LOG.debug("The extra route prefix is still present in some DPNs");
1657                             return;
1658                         }
1659                         //Is this fib route an extra route? If yes, get the nexthop which would be
1660                         //an adjacency in the vpn
1661                         Optional<Routes> extraRouteOptional = Optional.empty();
1662                         if (RouteOrigin.value(vrfEntry.getOrigin()) == RouteOrigin.STATIC && usedRds.size() != 0) {
1663                             extraRouteOptional = VpnExtraRouteHelper.getVpnExtraroutes(dataBroker,
1664                                     fibUtil.getVpnNameFromId(vpnInstance.getVpnId()),
1665                                     usedRds.get(0), vrfEntry.getDestPrefix());
1666                         }
1667                         baseVrfEntryHandler.deleteRemoteRoute(null, localDpnId, vpnId, vrfTablesKey, modVrfEntry,
1668                                 extraRouteOptional, TransactionAdapter.toWriteTransaction(tx));
1669                     }
1670                 } finally {
1671                     lock.unlock();
1672                 }
1673             })));
1674     }
1675
1676     public void cleanUpDpnForVpn(final Uint64 dpnId, final Uint32 vpnId, final String rd,
1677                                  final FutureCallback<List<?>> callback) {
1678         LOG.trace("cleanUpDpnForVpn: Remove dpn {} for vpn {} : cleanUpDpnForVpn", dpnId, rd);
1679         jobCoordinator.enqueueJob(FibUtil.getJobKeyForVpnIdDpnId(vpnId, dpnId),
1680             () -> {
1681                 InstanceIdentifier<VrfTables> id = buildVrfId(rd);
1682                 final VpnInstanceOpDataEntry vpnInstance = fibUtil.getVpnInstance(rd);
1683                 List<SubTransaction> txnObjects = new ArrayList<>();
1684                 final Optional<VrfTables> vrfTable = MDSALUtil.read(dataBroker,
1685                         LogicalDatastoreType.CONFIGURATION, id);
1686                 List<ListenableFuture<?>> futures = new ArrayList<>();
1687                 if (!vrfTable.isPresent()) {
1688                     LOG.error("cleanUpDpnForVpn: VRF Table not available for RD {}", rd);
1689                     if (callback != null) {
1690                         ListenableFuture<List<Object>> listenableFuture = Futures.allAsList(futures);
1691                         Futures.addCallback(listenableFuture, callback, MoreExecutors.directExecutor());
1692                     }
1693                     return futures;
1694                 }
1695                 final ReentrantLock lock = lockFor(vpnInstance);
1696                 lock.lock();
1697                 try {
1698                     futures.add(retryingTxRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION, tx -> {
1699                         String vpnName = fibUtil.getVpnNameFromId(vpnInstance.getVpnId());
1700                         for (final VrfEntry vrfEntry : vrfTable.get().nonnullVrfEntry().values()) {
1701                             /* parentRd is only filled for external PNF cases where the interface on the external
1702                              * network VPN are used to cleanup the flows. For all other cases, use "rd" for
1703                              * #fibUtil.isInterfacePresentInDpn().
1704                              * */
1705                             String parentRd = vrfEntry.getParentVpnRd() != null ? vrfEntry.getParentVpnRd()
1706                                     : rd;
1707                             /* Handle subnet routes here */
1708                             SubnetRoute subnetRoute = vrfEntry.augmentation(SubnetRoute.class);
1709                             if (subnetRoute != null && !fibUtil
1710                                     .isInterfacePresentInDpn(parentRd, dpnId)) {
1711                                 LOG.trace("SUBNETROUTE: cleanUpDpnForVpn: Cleaning subnetroute {} on dpn {}"
1712                                         + " for vpn {}", vrfEntry.getDestPrefix(), dpnId, rd);
1713                                 baseVrfEntryHandler.makeConnectedRoute(dpnId, vpnId, vrfEntry, rd, null,
1714                                         NwConstants.DEL_FLOW, TransactionAdapter.toWriteTransaction(tx), null);
1715                                 Map<RoutePathsKey, RoutePaths> keyRoutePathsMap = vrfEntry.getRoutePaths();
1716                                 if (keyRoutePathsMap != null) {
1717                                     for (RoutePaths routePath : keyRoutePathsMap.values()) {
1718                                         makeLFibTableEntry(dpnId, routePath.getLabel(), null,
1719                                                 DEFAULT_FIB_FLOW_PRIORITY,
1720                                                 NwConstants.DEL_FLOW, tx);
1721                                         LOG.trace("SUBNETROUTE: cleanUpDpnForVpn: Released subnetroute label {}"
1722                                                         + " for rd {} prefix {}", routePath.getLabel(), rd,
1723                                                 vrfEntry.getDestPrefix());
1724                                     }
1725                                 }
1726                                 installSubnetBroadcastAddrDropRule(dpnId, rd, vpnId, vrfEntry,
1727                                         NwConstants.DEL_FLOW, tx);
1728                                 continue;
1729                             }
1730                             // ping responder for router interfaces
1731                             RouterInterface routerInt = vrfEntry.augmentation(RouterInterface.class);
1732                             if (routerInt != null) {
1733                                 LOG.trace("Router augmented vrfentry found for rd:{}, uuid:{}, ip:{}, mac:{}",
1734                                         rd, routerInt.getUuid(), routerInt.getIpAddress(),
1735                                         routerInt.getMacAddress());
1736                                 routerInterfaceVrfEntryHandler.installRouterFibEntry(vrfEntry, dpnId, vpnId,
1737                                         routerInt.getIpAddress(), new MacAddress(routerInt.getMacAddress()),
1738                                         NwConstants.DEL_FLOW);
1739                                 continue;
1740                             }
1741                             //Handle local flow deletion for imports
1742                             if (RouteOrigin.value(vrfEntry.getOrigin()) == RouteOrigin.SELF_IMPORTED) {
1743                                 java.util.Optional<Uint32> optionalLabel = FibUtil.getLabelFromRoutePaths(vrfEntry);
1744                                 if (optionalLabel.isPresent()) {
1745                                     List<String> nextHopList = FibHelper.getNextHopListFromRoutePaths(vrfEntry);
1746                                     LabelRouteInfo lri = getLabelRouteInfo(optionalLabel.get());
1747                                     if (isPrefixAndNextHopPresentInLri(vrfEntry.getDestPrefix(), nextHopList,
1748                                             lri) && Objects.equals(lri.getDpnId(), dpnId)) {
1749                                         deleteLocalFibEntry(vpnId, rd, vrfEntry);
1750                                     }
1751                                 }
1752                             }
1753                             // Passing null as we don't know the dpn
1754                             // to which prefix is attached at this point
1755                             List<String> usedRds = VpnExtraRouteHelper.getUsedRds(dataBroker,
1756                                     vpnInstance.getVpnId(), vrfEntry.getDestPrefix());
1757                             Optional<Routes> extraRouteOptional;
1758                             //Is this fib route an extra route? If yes, get the nexthop which would be
1759                             //an adjacency in the vpn
1760                             if (usedRds != null && !usedRds.isEmpty()) {
1761                                 if (usedRds.size() > 1) {
1762                                     LOG.error("The extra route prefix is still present in some DPNs");
1763                                     return;
1764                                 } else {
1765                                     extraRouteOptional = VpnExtraRouteHelper.getVpnExtraroutes(dataBroker, vpnName,
1766                                             usedRds.get(0), vrfEntry.getDestPrefix());
1767
1768                                 }
1769                             } else {
1770                                 extraRouteOptional = Optional.empty();
1771                             }
1772                             if (RouteOrigin.BGP.getValue().equals(vrfEntry.getOrigin())) {
1773                                 bgpRouteVrfEntryHandler.deleteRemoteRoute(null, dpnId, vpnId,
1774                                         vrfTable.get().key(), vrfEntry, extraRouteOptional,
1775                                         TransactionAdapter.toWriteTransaction(tx), txnObjects);
1776                             } else {
1777                                 if (subnetRoute == null || !fibUtil
1778                                         .isInterfacePresentInDpn(parentRd, dpnId)) {
1779                                     baseVrfEntryHandler.deleteRemoteRoute(null, dpnId, vpnId,
1780                                             vrfTable.get().key(), vrfEntry, extraRouteOptional,
1781                                             TransactionAdapter.toWriteTransaction(tx));
1782                                 }
1783                             }
1784                         }
1785                     }));
1786                 } finally {
1787                     lock.unlock();
1788                 }
1789                 if (callback != null) {
1790                     ListenableFuture<List<Object>> listenableFuture = Futures.allAsList(futures);
1791                     Futures.addCallback(listenableFuture, callback, MoreExecutors.directExecutor());
1792                 }
1793                 return futures;
1794             });
1795     }
1796
1797     public void cleanUpExternalRoutesOnDpn(final Uint64 dpnId, final Uint32 vpnId, final String rd,
1798                                            final String localNextHopIp, final String remoteNextHopIp) {
1799         LOG.trace("cleanUpExternalRoutesOnDpn : cleanup remote routes on dpn {} for vpn {}, rd {}, "
1800                 + " localNexthopIp {} , remoteNexhtHopIp {}",
1801             dpnId, vpnId, rd, localNextHopIp, remoteNextHopIp);
1802         InstanceIdentifier<VrfTables> id = buildVrfId(rd);
1803         final VpnInstanceOpDataEntry vpnInstance = fibUtil.getVpnInstance(rd);
1804         List<SubTransaction> txnObjects =  new ArrayList<>();
1805         final Optional<VrfTables> vrfTable;
1806         try {
1807             vrfTable = SingleTransactionDataBroker.syncReadOptional(dataBroker, LogicalDatastoreType.CONFIGURATION, id);
1808         } catch (ExecutionException | InterruptedException e) {
1809             LOG.error("getVrfEntry: Exception while reading VrfTable for the rd {} vpnId {}", rd, vpnId, e);
1810             return;
1811         }
1812         if (vrfTable.isPresent()) {
1813             jobCoordinator.enqueueJob(FibUtil.getJobKeyForVpnIdDpnId(vpnId, dpnId),
1814                 () -> {
1815                     final ReentrantLock lock = lockFor(vpnInstance);
1816                     lock.lock();
1817                     try {
1818                         return Collections.singletonList(
1819                             txRunner.callWithNewWriteOnlyTransactionAndSubmit(CONFIGURATION,
1820                                 tx -> vrfTable.get().nonnullVrfEntry().values().stream()
1821                                     .filter(vrfEntry -> RouteOrigin.value(vrfEntry.getOrigin()) == RouteOrigin.BGP)
1822                                     .forEach(bgpRouteVrfEntryHandler.getConsumerForDeletingRemoteFib(dpnId, vpnId,
1823                                         remoteNextHopIp, vrfTable, TransactionAdapter.toWriteTransaction(tx),
1824                                             txnObjects))));
1825                     } finally {
1826                         lock.unlock();
1827                     }
1828                 });
1829         }
1830     }
1831
1832     public static InstanceIdentifier<VrfTables> buildVrfId(String rd) {
1833         InstanceIdentifierBuilder<VrfTables> idBuilder =
1834             InstanceIdentifier.builder(FibEntries.class).child(VrfTables.class, new VrfTablesKey(rd));
1835         return idBuilder.build();
1836     }
1837
1838     private String getTableMissFlowRef(Uint64 dpnId, short tableId, Uint32 tableMiss) {
1839         return FLOWID_PREFIX + dpnId + NwConstants.FLOWID_SEPARATOR + tableId + NwConstants.FLOWID_SEPARATOR
1840                 + tableMiss + FLOWID_PREFIX;
1841     }
1842
1843     @Nullable
1844     private VrfEntry getVrfEntry(DataBroker broker, String rd, String ipPrefix) {
1845         InstanceIdentifier<VrfEntry> vrfEntryId = InstanceIdentifier.builder(FibEntries.class)
1846             .child(VrfTables.class, new VrfTablesKey(rd))
1847             .child(VrfEntry.class, new VrfEntryKey(ipPrefix)).build();
1848         Optional<VrfEntry> vrfEntry;
1849         try {
1850             vrfEntry = SingleTransactionDataBroker.syncReadOptional(broker, LogicalDatastoreType.CONFIGURATION,
1851                     vrfEntryId);
1852         } catch (ExecutionException | InterruptedException e) {
1853             LOG.error("getVrfEntry: Exception while reading VrfEntry for the prefix {} rd {}", ipPrefix, rd, e);
1854             return null;
1855         }
1856         if (vrfEntry.isPresent()) {
1857             return vrfEntry.get();
1858         }
1859         return null;
1860     }
1861
1862     private static boolean isPrefixAndNextHopPresentInLri(String prefix,
1863             List<String> nextHopAddressList, LabelRouteInfo lri) {
1864         return lri != null && Objects.equals(lri.getPrefix(), prefix)
1865                 && nextHopAddressList.contains(lri.getNextHopIpList().get(0));
1866     }
1867
1868     private boolean shouldCreateFibEntryForVrfAndVpnIdOnDpn(Uint32 vpnId, VrfEntry vrfEntry, Uint64 dpnId) {
1869         if (RouteOrigin.value(vrfEntry.getOrigin()) == RouteOrigin.BGP) {
1870             return true;
1871         }
1872
1873         Prefixes prefix = fibUtil.getPrefixToInterface(vpnId, vrfEntry.getDestPrefix());
1874         if (prefix != null) {
1875             Uint64 prefixDpnId = prefix.getDpnId();
1876             if (dpnId.equals(prefixDpnId)) {
1877                 LOG.trace("Should not create remote FIB entry for vrfEntry {} on DPN {}",
1878                         vrfEntry, dpnId);
1879                 return false;
1880             }
1881         }
1882         return true;
1883     }
1884
1885     private static ReentrantLock lockFor(final VpnInstanceOpDataEntry vpnInstance) {
1886         // FIXME: use vpnInstance.key() instead?
1887         return JvmGlobalLocks.getLockForString(vpnInstance.getVpnInstanceName());
1888     }
1889
1890     private static ReentrantLock lockFor(LabelRouteInfoKey label) {
1891         return JvmGlobalLocks.getLockFor(label);
1892     }
1893 }