YANG revision dates mass-update
[bgpcep.git] / bgp / rib-impl / src / main / java / org / opendaylight / protocol / bgp / rib / impl / EffectiveRibInWriter.java
index e39e6f9e015cd935f12bd52d7181cb237c212542..993b34f6938e3bbabb38eb5893491d7e3bd3dbf8 100644 (file)
  */
 package org.opendaylight.protocol.bgp.rib.impl;
 
+import static com.google.common.base.Verify.verify;
 import static java.util.Objects.requireNonNull;
-
-import com.google.common.base.Verify;
+import static org.opendaylight.protocol.bgp.rib.spi.RIBNodeIdentifiers.ADJRIBIN_ATTRIBUTES_AID;
+import static org.opendaylight.protocol.bgp.rib.spi.RIBNodeIdentifiers.ADJRIBIN_NID;
+import static org.opendaylight.protocol.bgp.rib.spi.RIBNodeIdentifiers.ATTRIBUTES_NID;
+import static org.opendaylight.protocol.bgp.rib.spi.RIBNodeIdentifiers.EFFRIBIN_NID;
+import static org.opendaylight.protocol.bgp.rib.spi.RIBNodeIdentifiers.LLGR_STALE_NID;
+import static org.opendaylight.protocol.bgp.rib.spi.RIBNodeIdentifiers.ROUTES_NID;
+import static org.opendaylight.protocol.bgp.rib.spi.RIBNodeIdentifiers.TABLES_NID;
+import static org.opendaylight.protocol.bgp.rib.spi.RIBNodeIdentifiers.UPTODATE_NID;
+
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.FluentFuture;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.MoreExecutors;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.LongAdder;
-import javax.annotation.Nonnull;
-import javax.annotation.concurrent.NotThreadSafe;
-import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
-import org.opendaylight.controller.md.sal.dom.api.ClusteredDOMDataTreeChangeListener;
-import org.opendaylight.controller.md.sal.dom.api.DOMDataTreeIdentifier;
-import org.opendaylight.controller.md.sal.dom.api.DOMDataWriteTransaction;
-import org.opendaylight.controller.md.sal.dom.api.DOMTransactionChain;
-import org.opendaylight.protocol.bgp.rib.impl.spi.AbstractImportPolicy;
-import org.opendaylight.protocol.bgp.rib.impl.spi.ImportPolicyPeerTracker;
+import org.checkerframework.checker.lock.qual.GuardedBy;
+import org.checkerframework.checker.lock.qual.Holding;
+import org.opendaylight.mdsal.common.api.CommitInfo;
+import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
+import org.opendaylight.mdsal.dom.api.ClusteredDOMDataTreeChangeListener;
+import org.opendaylight.mdsal.dom.api.DOMDataTreeChangeService;
+import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
+import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteTransaction;
+import org.opendaylight.mdsal.dom.api.DOMTransactionChain;
+import org.opendaylight.protocol.bgp.openconfig.spi.BGPTableTypeRegistryConsumer;
+import org.opendaylight.protocol.bgp.parser.impl.message.update.CommunityUtil;
 import org.opendaylight.protocol.bgp.rib.impl.spi.RIB;
 import org.opendaylight.protocol.bgp.rib.impl.spi.RIBSupportContext;
 import org.opendaylight.protocol.bgp.rib.impl.spi.RIBSupportContextRegistry;
+import org.opendaylight.protocol.bgp.rib.impl.spi.RibOutRefresh;
 import org.opendaylight.protocol.bgp.rib.impl.state.peer.PrefixesInstalledCounters;
 import org.opendaylight.protocol.bgp.rib.impl.state.peer.PrefixesReceivedCounters;
-import org.opendaylight.protocol.bgp.rib.spi.BGPPeerTracker;
-import org.opendaylight.protocol.bgp.rib.spi.IdentifierUtils;
+import org.opendaylight.protocol.bgp.rib.spi.RIBNormalizedNodes;
 import org.opendaylight.protocol.bgp.rib.spi.RIBSupport;
-import org.opendaylight.protocol.bgp.rib.spi.entry.AttributeBindingCodecSerializer;
 import org.opendaylight.protocol.bgp.rib.spi.policy.BGPRibRoutingPolicy;
 import org.opendaylight.protocol.bgp.rib.spi.policy.BGPRouteEntryImportParameters;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.message.rev171207.path.attributes.Attributes;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev171207.PeerId;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev171207.PeerRole;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev171207.bgp.rib.rib.peer.AdjRibIn;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev171207.bgp.rib.rib.peer.EffectiveRibIn;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev171207.rib.Tables;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev171207.rib.TablesKey;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev171207.rib.tables.Routes;
+import org.opendaylight.protocol.bgp.route.targetcontrain.spi.ClientRouteTargetContrainCache;
+import org.opendaylight.protocol.bgp.route.targetcontrain.spi.RouteTargetMembeshipUtil;
+import org.opendaylight.yang.gen.v1.http.openconfig.net.yang.bgp.types.rev151009.AfiSafiType;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.message.rev200120.path.attributes.Attributes;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.message.rev200120.path.attributes.attributes.Communities;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev180329.PeerRole;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev180329.rib.TablesKey;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.route.target.constrain.rev180618.RouteTargetConstrainSubsequentAddressFamily;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.route.target.constrain.rev180618.route.target.constrain.routes.route.target.constrain.routes.RouteTargetConstrainRoute;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.types.rev200120.Ipv4AddressFamily;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.types.rev200120.Ipv6AddressFamily;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.types.rev200120.MplsLabeledVpnSubsequentAddressFamily;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.types.rev200120.RouteTarget;
 import org.opendaylight.yangtools.concepts.ListenerRegistration;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
-import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifier;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifierWithPredicates;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument;
+import org.opendaylight.yangtools.yang.data.api.schema.ChoiceNode;
 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
+import org.opendaylight.yangtools.yang.data.api.schema.DataContainerChild;
+import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
+import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
+import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNodes;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateNode;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.ModificationType;
+import org.opendaylight.yangtools.yang.data.impl.schema.Builders;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
  * Implementation of the BGP import policy. Listens on peer's Adj-RIB-In, inspects all inbound
  * routes in the context of the advertising peer's role and applies the inbound policy.
+ *
  * <p>
  * Inbound policy is applied as follows:
+ *
  * <p>
  * 1) if the peer is an eBGP peer, perform attribute replacement and filtering
  * 2) check if a route is admissible based on attributes attached to it, as well as the
  * advertising peer's role
  * 3) output admitting routes with edited attributes into /bgp-rib/rib/peer/effective-rib-in/tables/routes
+ *
+ * <p>
+ * This class is NOT thread-safe.
  */
-@NotThreadSafe
-final class EffectiveRibInWriter implements PrefixesReceivedCounters, PrefixesInstalledCounters, AutoCloseable {
+final class EffectiveRibInWriter implements PrefixesReceivedCounters, PrefixesInstalledCounters,
+        AutoCloseable, ClusteredDOMDataTreeChangeListener {
+
     private static final Logger LOG = LoggerFactory.getLogger(EffectiveRibInWriter.class);
-    static final NodeIdentifier TABLE_ROUTES = new NodeIdentifier(Routes.QNAME);
-
-    private final class AdjInTracker implements PrefixesReceivedCounters, PrefixesInstalledCounters, AutoCloseable,
-            ClusteredDOMDataTreeChangeListener {
-        private final RIBSupportContextRegistry registry;
-        private final YangInstanceIdentifier peerIId;
-        private final YangInstanceIdentifier effRibTables;
-        private final ListenerRegistration<?> reg;
-        private final DOMTransactionChain chain;
-        private final Map<TablesKey, LongAdder> prefixesReceived;
-        private final Map<TablesKey, LongAdder> prefixesInstalled;
-        private final BGPRibRoutingPolicy ribPolicies;
-        private final BGPPeerTracker peerTracker;
-        private final AttributeBindingCodecSerializer attBindingCodecSerializer;
-        private final PeerId peerId;
-
-        AdjInTracker(final RIB rib,
-                final DOMTransactionChain chain,
-                final YangInstanceIdentifier peerIId,
-                @Nonnull final Set<TablesKey> tables) {
-            this.registry = requireNonNull(rib.getRibSupportContext());
-            this.chain = requireNonNull(chain);
-            this.peerIId = requireNonNull(peerIId);
-            this.effRibTables = this.peerIId.node(EffectiveRibIn.QNAME).node(Tables.QNAME);
-            this.prefixesInstalled = buildPrefixesTables(tables);
-            this.prefixesReceived = buildPrefixesTables(tables);
-            this.ribPolicies = requireNonNull(rib.getRibPolicies());
-            this.peerTracker = requireNonNull(rib.getPeerTracker());
-            this.attBindingCodecSerializer = rib;
-            this.peerId = IdentifierUtils.peerId((NodeIdentifierWithPredicates) peerIId.getLastPathArgument());
-            final DOMDataTreeIdentifier treeId = new DOMDataTreeIdentifier(LogicalDatastoreType.OPERATIONAL,
-                    this.peerIId.node(AdjRibIn.QNAME).node(Tables.QNAME));
-            LOG.debug("Registered Effective RIB on {}", this.peerIId);
-            this.reg = requireNonNull(rib.getService()).registerDataTreeChangeListener(treeId, this);
-        }
+    private static final TablesKey IVP4_VPN_TABLE_KEY = new TablesKey(Ipv4AddressFamily.class,
+            MplsLabeledVpnSubsequentAddressFamily.class);
+    private static final TablesKey IVP6_VPN_TABLE_KEY = new TablesKey(Ipv6AddressFamily.class,
+            MplsLabeledVpnSubsequentAddressFamily.class);
+    private static final ImmutableList<Communities> STALE_LLGR_COMMUNUTIES = ImmutableList.of(
+        StaleCommunities.STALE_LLGR);
+    private static final Attributes STALE_LLGR_ATTRIBUTES = new org.opendaylight.yang.gen.v1.urn.opendaylight.params
+            .xml.ns.yang.bgp.message.rev200120.path.attributes.AttributesBuilder()
+            .setCommunities(STALE_LLGR_COMMUNUTIES)
+            .build();
+    private static final ChoiceNode EMPTY_ROUTES = Builders.choiceBuilder().withNodeIdentifier(ROUTES_NID).build();
+
+    private final RIBSupportContextRegistry registry;
+    private final YangInstanceIdentifier peerIId;
+    private final YangInstanceIdentifier effRibTables;
+    private final DOMDataTreeChangeService service;
+    private final List<RouteTarget> rtMemberships;
+    private final RibOutRefresh vpnTableRefresher;
+    private final ClientRouteTargetContrainCache rtCache;
+    private ListenerRegistration<?> reg;
+    private DOMTransactionChain chain;
+    private final Map<TablesKey, LongAdder> prefixesReceived;
+    private final Map<TablesKey, LongAdder> prefixesInstalled;
+    private final BGPRibRoutingPolicy ribPolicies;
+    private final BGPRouteEntryImportParameters peerImportParameters;
+    private final BGPTableTypeRegistryConsumer tableTypeRegistry;
+    @GuardedBy("this")
+    private FluentFuture<? extends CommitInfo> submitted;
+    private boolean rtMembershipsUpdated;
+
+    EffectiveRibInWriter(
+            final BGPRouteEntryImportParameters peer,
+            final RIB rib,
+            final DOMTransactionChain chain,
+            final YangInstanceIdentifier peerIId,
+            final Set<TablesKey> tables,
+            final BGPTableTypeRegistryConsumer tableTypeRegistry,
+            final List<RouteTarget> rtMemberships,
+            final ClientRouteTargetContrainCache rtCache) {
+        this.registry = requireNonNull(rib.getRibSupportContext());
+        this.chain = requireNonNull(chain);
+        this.peerIId = requireNonNull(peerIId);
+        this.effRibTables = this.peerIId.node(EFFRIBIN_NID);
+        this.prefixesInstalled = buildPrefixesTables(tables);
+        this.prefixesReceived = buildPrefixesTables(tables);
+        this.ribPolicies = requireNonNull(rib.getRibPolicies());
+        this.service = requireNonNull(rib.getService());
+        this.tableTypeRegistry = requireNonNull(tableTypeRegistry);
+        this.peerImportParameters = peer;
+        this.rtMemberships = rtMemberships;
+        this.rtCache = rtCache;
+        this.vpnTableRefresher = rib;
+    }
 
-        private Map<TablesKey, LongAdder> buildPrefixesTables(final Set<TablesKey> tables) {
-            final ImmutableMap.Builder<TablesKey, LongAdder> b = ImmutableMap.builder();
-            tables.forEach(table -> b.put(table, new LongAdder()));
-            return b.build();
-        }
+    public void init() {
+        final DOMDataTreeIdentifier treeId = new DOMDataTreeIdentifier(LogicalDatastoreType.OPERATIONAL,
+            this.peerIId.node(ADJRIBIN_NID).node(TABLES_NID));
+        LOG.debug("Registered Effective RIB on {}", this.peerIId);
+        this.reg = requireNonNull(this.service).registerDataTreeChangeListener(treeId, this);
+    }
 
-        private void processRoute(final DOMDataWriteTransaction tx, final RIBSupport ribSupport, final AbstractImportPolicy policy,
-                final YangInstanceIdentifier routesPath, final DataTreeCandidateNode route) {
-            LOG.debug("Process route {}", route.getIdentifier());
-            final YangInstanceIdentifier routeId = ribSupport.routePath(routesPath, route.getIdentifier());
-            final TablesKey tablesKey = new TablesKey(ribSupport.getAfi(), ribSupport.getSafi());
-            switch (route.getModificationType()) {
-                case DELETE:
-                case DISAPPEARED:
-                    tx.delete(LogicalDatastoreType.OPERATIONAL, routeId);
-                    LOG.debug("Route deleted. routeId={}", routeId);
-                    CountersUtil.decrement(this.prefixesInstalled.get(tablesKey), tablesKey);
-                    break;
-                case UNMODIFIED:
-                    // No-op
-                    break;
-                case APPEARED:
-                case SUBTREE_MODIFIED:
-                case WRITE:
-                    final NormalizedNode<?, ?> advRoute = route.getDataAfter().get();
-                    tx.put(LogicalDatastoreType.OPERATIONAL, routeId, advRoute);
-                    CountersUtil.increment(this.prefixesReceived.get(tablesKey), tablesKey);
-                    // Lookup per-table attributes from RIBSupport
-
-                    final NodeIdentifierWithPredicates routeIdentifier = ribSupport
-                            .createRouteKeyPathArgument((NodeIdentifierWithPredicates) route.getIdentifier());
-                    Optional<Attributes> advertisedAttrs = this.attBindingCodecSerializer
-                            .getAttributes(ribSupport, routeIdentifier, advRoute);
-
-                    final Optional<Attributes> effectiveAttrs;
-                    if (advertisedAttrs.isPresent()) {
-                        final PeerRole peerRole = this.peerTracker.getRole(this.peerId);
-                        final BGPRouteEntryImportParameters ribPolicyParameters =
-                                new BGPRouteEntryImportParametersImpl(
-                                        (NodeIdentifierWithPredicates) route.getIdentifier(), this.peerId, peerRole);
-                        effectiveAttrs = this.ribPolicies
-                                .applyImportPolicies(ribPolicyParameters, advertisedAttrs.get());
-                        LOG.debug("Route {} effective attributes {} towards {}", route.getIdentifier(), effectiveAttrs,
-                                routeId);
-
-                    } else {
-                        effectiveAttrs = Optional.empty();
-                    }
+    private static Map<TablesKey, LongAdder> buildPrefixesTables(final Set<TablesKey> tables) {
+        final ImmutableMap.Builder<TablesKey, LongAdder> b = ImmutableMap.builder();
+        tables.forEach(table -> b.put(table, new LongAdder()));
+        return b.build();
+    }
 
-                    LOG.debug("Route {} effective attributes {} towards {}", route.getIdentifier(), effectiveAttrs, routeId);
-
-                    final Optional<ContainerNode> normEffAtt = this.attBindingCodecSerializer
-                            .toNormalizedNodeAttribute(ribSupport, routeIdentifier, effectiveAttrs);
-                    if (normEffAtt.isPresent()) {
-                        tx.put(LogicalDatastoreType.OPERATIONAL,
-                                routeId.node(ribSupport.routeAttributesIdentifier()), normEffAtt.get());
-                        if (route.getModificationType() == ModificationType.WRITE) {
-                            CountersUtil.increment(this.prefixesInstalled.get(tablesKey), tablesKey);
-                        }
-                    } else {
-                        LOG.warn("Route {} advertised empty attributes", routeId);
-                        tx.delete(LogicalDatastoreType.OPERATIONAL, routeId);
-                    }
-                    break;
-                default:
-                    LOG.warn("Ignoring unhandled route {}", route);
-                    break;
-            }
+    @Override
+    public synchronized void onDataTreeChanged(final Collection<DataTreeCandidate> changes) {
+        if (this.chain == null) {
+            LOG.trace("Chain closed. Ignoring Changes : {}", changes);
+            return;
         }
 
-        private void processTableChildren(final DOMDataWriteTransaction tx, final RIBSupport ribSupport, final YangInstanceIdentifier tablePath, final Collection<DataTreeCandidateNode> children) {
-            for (final DataTreeCandidateNode child : children) {
-                final PathArgument childIdentifier = child.getIdentifier();
-                final Optional<NormalizedNode<?, ?>> childDataAfter = child.getDataAfter();
-                final TablesKey tablesKey = new TablesKey(ribSupport.getAfi(), ribSupport.getSafi());
-                LOG.debug("Process table {} type {}, dataAfter {}, dataBefore {}", childIdentifier, child
-                        .getModificationType(), childDataAfter, child.getDataBefore());
-                final YangInstanceIdentifier childPath = tablePath.node(childIdentifier);
-                switch (child.getModificationType()) {
-                    case DELETE:
-                    case DISAPPEARED:
-                        tx.delete(LogicalDatastoreType.OPERATIONAL, childPath);
-                        LOG.debug("Route deleted. routeId={}", childPath);
-                        CountersUtil.decrement(this.prefixesInstalled.get(tablesKey), tablesKey);
-                        break;
-                    case UNMODIFIED:
-                        // No-op
-                        break;
-                    case SUBTREE_MODIFIED:
-                        processModifiedRouteTables(child, childIdentifier, tx, ribSupport, EffectiveRibInWriter.this.importPolicy, childPath, childDataAfter);
-                        break;
-                    case APPEARED:
-                    case WRITE:
-                        writeRouteTables(child, childIdentifier, tx, ribSupport, EffectiveRibInWriter.this.importPolicy, childPath, childDataAfter);
-
-                        break;
-                    default:
-                        LOG.warn("Ignoring unhandled child {}", child);
-                        break;
+        LOG.trace("Data changed called to effective RIB. Change : {}", changes);
+        DOMDataTreeWriteTransaction tx = null;
+        for (final DataTreeCandidate tc : changes) {
+            final YangInstanceIdentifier rootPath = tc.getRootPath();
+            final DataTreeCandidateNode root = tc.getRootNode();
+            for (final DataTreeCandidateNode table : root.getChildNodes()) {
+                if (tx == null) {
+                    tx = this.chain.newWriteOnlyTransaction();
                 }
+                changeDataTree(tx, rootPath, root, table);
             }
         }
 
-        private void processModifiedRouteTables(final DataTreeCandidateNode child, final PathArgument childIdentifier, final DOMDataWriteTransaction tx,
-                final RIBSupport ribSupport, final AbstractImportPolicy policy, final YangInstanceIdentifier childPath, final Optional<NormalizedNode<?, ?>> childDataAfter) {
-            if (TABLE_ROUTES.equals(childIdentifier)) {
-                for (final DataTreeCandidateNode route : ribSupport.changedRoutes(child)) {
-                    processRoute(tx, ribSupport, policy, childPath, route);
+        if (tx != null) {
+            final FluentFuture<? extends CommitInfo> future = tx.commit();
+            this.submitted = future;
+            future.addCallback(new FutureCallback<CommitInfo>() {
+                @Override
+                public void onSuccess(final CommitInfo result) {
+                    LOG.trace("Successful commit");
                 }
-            } else {
-                tx.put(LogicalDatastoreType.OPERATIONAL, childPath, childDataAfter.get());
-            }
-        }
 
-        private void writeRouteTables(final DataTreeCandidateNode child, final PathArgument childIdentifier, final DOMDataWriteTransaction tx, final RIBSupport ribSupport, final AbstractImportPolicy policy, final YangInstanceIdentifier childPath, final Optional<NormalizedNode<?, ?>> childDataAfter) {
-            if (TABLE_ROUTES.equals(childIdentifier)) {
-                final Collection<DataTreeCandidateNode> changedRoutes = ribSupport.changedRoutes(child);
-                if (!changedRoutes.isEmpty()) {
-                    tx.put(LogicalDatastoreType.OPERATIONAL, childPath, childDataAfter.get());
-                    // Routes are special, as they may end up being filtered. The previous put conveniently
-                    // ensured that we have them in at target, so a subsequent delete will not fail :)
-                    for (final DataTreeCandidateNode route : changedRoutes) {
-                        processRoute(tx, ribSupport, policy, childPath, route);
-                    }
+                @Override
+                public void onFailure(final Throwable trw) {
+                    LOG.error("Failed commit", trw);
                 }
-            }
+            }, MoreExecutors.directExecutor());
         }
 
-        private RIBSupportContext getRibSupport(final NodeIdentifierWithPredicates tableKey) {
-            return this.registry.getRIBSupportContext(tableKey);
+        //Refresh VPN Table if RT Memberships were updated
+        if (this.rtMembershipsUpdated) {
+            this.vpnTableRefresher.refreshTable(IVP4_VPN_TABLE_KEY, this.peerImportParameters.getFromPeerId());
+            this.vpnTableRefresher.refreshTable(IVP6_VPN_TABLE_KEY, this.peerImportParameters.getFromPeerId());
+            this.rtMembershipsUpdated = false;
         }
+    }
+
+    @Override
+    public synchronized void close() {
+        if (this.reg != null) {
+            this.reg.close();
+            this.reg = null;
+        }
+        if (this.submitted != null) {
+            try {
+                this.submitted.get();
+            } catch (final InterruptedException | ExecutionException throwable) {
+                LOG.error("Write routes failed", throwable);
+            }
+        }
+        if (this.chain != null) {
+            this.chain.close();
+            this.chain = null;
+        }
+        this.prefixesReceived.values().forEach(LongAdder::reset);
+        this.prefixesInstalled.values().forEach(LongAdder::reset);
+    }
 
-        private YangInstanceIdentifier effectiveTablePath(final NodeIdentifierWithPredicates tableKey) {
-            return this.effRibTables.node(tableKey);
+    @Override
+    public long getPrefixedReceivedCount(final TablesKey tablesKey) {
+        final LongAdder counter = this.prefixesReceived.get(tablesKey);
+        if (counter == null) {
+            return 0;
         }
+        return counter.longValue();
+    }
+
+    @Override
+    public Set<TablesKey> getTableKeys() {
+        return ImmutableSet.copyOf(this.prefixesReceived.keySet());
+    }
 
-        private void modifyTable(final DOMDataWriteTransaction tx, final NodeIdentifierWithPredicates tableKey, final DataTreeCandidateNode table) {
-            final RIBSupportContext ribSupport = getRibSupport(tableKey);
-            final YangInstanceIdentifier tablePath = effectiveTablePath(tableKey);
+    @Override
+    public boolean isSupported(final TablesKey tablesKey) {
+        return this.prefixesReceived.containsKey(tablesKey);
+    }
 
-            processTableChildren(tx, ribSupport.getRibSupport(), tablePath, table.getChildNodes());
+    @Override
+    public long getPrefixedInstalledCount(final TablesKey tablesKey) {
+        final LongAdder counter = this.prefixesInstalled.get(tablesKey);
+        if (counter == null) {
+            return 0;
         }
+        return counter.longValue();
+    }
 
-        private void writeTable(final DOMDataWriteTransaction tx, final NodeIdentifierWithPredicates tableKey, final DataTreeCandidateNode table) {
-            final RIBSupportContext ribSupport = getRibSupport(tableKey);
-            final YangInstanceIdentifier tablePath = effectiveTablePath(tableKey);
+    @Override
+    public long getTotalPrefixesInstalled() {
+        return this.prefixesInstalled.values().stream().mapToLong(LongAdder::longValue).sum();
+    }
 
-            // Create an empty table
-            LOG.trace("Create Empty table", tablePath);
-            ribSupport.createEmptyTableStructure(tx, tablePath);
+    @Holding("this")
+    private void changeDataTree(final DOMDataTreeWriteTransaction tx, final YangInstanceIdentifier rootPath,
+            final DataTreeCandidateNode root, final DataTreeCandidateNode table) {
+        final PathArgument lastArg = table.getIdentifier();
+        verify(lastArg instanceof NodeIdentifierWithPredicates, "Unexpected type %s in path %s", lastArg.getClass(),
+            rootPath);
+        final NodeIdentifierWithPredicates tableKey = (NodeIdentifierWithPredicates) lastArg;
+        final RIBSupportContext ribContext = this.registry.getRIBSupportContext(tableKey);
+        if (ribContext == null) {
+            LOG.warn("Table {} is not supported, ignoring event", tableKey);
+            return;
+        }
 
-            processTableChildren(tx, ribSupport.getRibSupport(), tablePath, table.getChildNodes());
+        final YangInstanceIdentifier effectiveTablePath = effectiveTablePath(tableKey);
+        final ModificationType modificationType = root.getModificationType();
+        LOG.debug("Effective table {} modification type {}", effectiveTablePath, modificationType);
+        switch (modificationType) {
+            case DISAPPEARED:
+            case DELETE:
+                deleteTable(tx, ribContext, effectiveTablePath, table);
+                break;
+            case APPEARED:
+            case WRITE:
+                writeTable(tx, ribContext, effectiveTablePath, table);
+                break;
+            case SUBTREE_MODIFIED:
+                modifyTable(tx, ribContext, effectiveTablePath, table);
+                break;
+            case UNMODIFIED:
+                LOG.info("Ignoring spurious notification on {} data {}", rootPath, table);
+                break;
+            default:
+                LOG.warn("Ignoring unhandled root {}", table);
+                break;
         }
+    }
 
-        @Override
-        public void onDataTreeChanged(@Nonnull final Collection<DataTreeCandidate> changes) {
-            LOG.trace("Data changed called to effective RIB. Change : {}", changes);
+    private void deleteTable(final DOMDataTreeWriteTransaction tx, final RIBSupportContext ribContext,
+            final YangInstanceIdentifier effectiveTablePath, final DataTreeCandidateNode table) {
+        LOG.debug("Delete Effective Table {}", effectiveTablePath);
+        onDeleteTable(ribContext.getRibSupport(), effectiveTablePath, table.getDataBefore());
+        tx.delete(LogicalDatastoreType.OPERATIONAL, effectiveTablePath);
+    }
 
-            // we have a lot of transactions created for 'nothing' because a lot of changes
-            // are skipped, so ensure we only create one transaction when we really need it
-            DOMDataWriteTransaction tx = null;
-            for (final DataTreeCandidate tc : changes) {
-                final YangInstanceIdentifier rootPath = tc.getRootPath();
+    private void modifyTable(final DOMDataTreeWriteTransaction tx, final RIBSupportContext ribContext,
+            final YangInstanceIdentifier effectiveTablePath, final DataTreeCandidateNode table) {
+        LOG.debug("Modify Effective Table {}", effectiveTablePath);
+
+        final boolean wasLongLivedStale = isLongLivedStaleTable(table.getDataBefore());
+        final boolean longLivedStale = isLongLivedStaleTable(table.getDataAfter());
+        if (wasLongLivedStale != longLivedStale) {
+            LOG.debug("LLGR_STALE flag flipped {}, overwriting table {}", longLivedStale ? "ON" : "OFF",
+                    effectiveTablePath);
+            writeTable(tx, ribContext, effectiveTablePath, table);
+            return;
+        }
 
-                final DataTreeCandidateNode root = tc.getRootNode();
-                for (final DataTreeCandidateNode table : root.getChildNodes()) {
-                    if (tx == null) {
-                        tx = this.chain.newWriteOnlyTransaction();
-                    }
-                    changeDataTree(tx, rootPath, root, table);
-                }
-            }
-            if (tx != null) {
-                tx.submit();
+        table.getModifiedChild(ATTRIBUTES_NID).ifPresent(modifiedAttrs -> {
+            final YangInstanceIdentifier effAttrsPath = effectiveTablePath.node(ATTRIBUTES_NID);
+            final Optional<NormalizedNode<?, ?>> optAttrsAfter = modifiedAttrs.getDataAfter();
+            if (optAttrsAfter.isPresent()) {
+                tx.put(LogicalDatastoreType.OPERATIONAL, effAttrsPath, effectiveAttributes(
+                    NormalizedNodes.findNode(optAttrsAfter.get(), UPTODATE_NID)));
+            } else {
+                tx.delete(LogicalDatastoreType.OPERATIONAL, effAttrsPath);
             }
-        }
+        });
 
-        private void changeDataTree(final DOMDataWriteTransaction tx, final YangInstanceIdentifier rootPath,
-                final DataTreeCandidateNode root, final DataTreeCandidateNode table) {
-            final PathArgument lastArg = table.getIdentifier();
-            Verify.verify(lastArg instanceof NodeIdentifierWithPredicates, "Unexpected type %s in path %s", lastArg.getClass(), rootPath);
-            final NodeIdentifierWithPredicates tableKey = (NodeIdentifierWithPredicates) lastArg;
-            final RIBSupport ribSupport = getRibSupport(tableKey).getRibSupport();
-            final ModificationType modificationType = root.getModificationType();
-            switch (modificationType) {
+        table.getModifiedChild(ROUTES_NID).ifPresent(modifiedRoutes -> {
+            final RIBSupport<?, ?, ?, ?> ribSupport = ribContext.getRibSupport();
+            switch (modifiedRoutes.getModificationType()) {
+                case APPEARED:
+                case WRITE:
+                    deleteRoutesBefore(tx, ribSupport, effectiveTablePath, modifiedRoutes);
+                    // XXX: YANG Tools seems to have an issue stacking DELETE with child WRITE
+                    tx.put(LogicalDatastoreType.OPERATIONAL, effectiveTablePath.node(ROUTES_NID), EMPTY_ROUTES);
+                    writeRoutesAfter(tx, ribSupport, effectiveTablePath, modifiedRoutes.getDataAfter(), longLivedStale);
+                    break;
                 case DELETE:
                 case DISAPPEARED:
-                    final YangInstanceIdentifier effectiveTablePath = effectiveTablePath(tableKey);
-                    LOG.debug("Delete Effective Table {} modification type {}, ", effectiveTablePath, modificationType);
-
-                    // delete the corresponding effective table
-                    tx.delete(LogicalDatastoreType.OPERATIONAL, effectiveTablePath);
-                    final TablesKey tk = new TablesKey(ribSupport.getAfi(), ribSupport.getSafi());
-                    CountersUtil.decrement(this.prefixesInstalled.get(tk), tk);
+                    deleteRoutesBefore(tx, ribSupport, effectiveTablePath, modifiedRoutes);
+                    tx.delete(LogicalDatastoreType.OPERATIONAL, effectiveTablePath.node(ROUTES_NID));
                     break;
                 case SUBTREE_MODIFIED:
-                    modifyTable(tx, tableKey, table);
+                    for (DataTreeCandidateNode modifiedRoute : ribSupport.changedRoutes(modifiedRoutes)) {
+                        processRoute(tx, ribSupport, effectiveTablePath, modifiedRoute, longLivedStale);
+                    }
                     break;
                 case UNMODIFIED:
-                    LOG.info("Ignoring spurious notification on {} data {}", rootPath, table);
-                    break;
-                case APPEARED:
-                case WRITE:
-                    writeTable(tx, tableKey, table);
-                    break;
+                    // No-op
+                    return;
                 default:
-                    LOG.warn("Ignoring unhandled root {}", root);
+                    LOG.warn("Ignoring modified routes {}", modifiedRoutes);
                     break;
             }
+        });
+    }
+
+    private void writeTable(final DOMDataTreeWriteTransaction tx, final RIBSupportContext ribContext,
+            final YangInstanceIdentifier effectiveTablePath, final DataTreeCandidateNode table) {
+        LOG.debug("Write Effective Table {}", effectiveTablePath);
+        onDeleteTable(ribContext.getRibSupport(), effectiveTablePath, table.getDataBefore());
+
+        final Optional<NormalizedNode<?, ?>> maybeTableAfter = table.getDataAfter();
+        if (maybeTableAfter.isPresent()) {
+            final MapEntryNode tableAfter = extractMapEntry(maybeTableAfter);
+            ribContext.createEmptyTableStructure(tx, effectiveTablePath);
+
+            final Optional<DataContainerChild<?, ?>> maybeAttrsAfter = tableAfter.getChild(ATTRIBUTES_NID);
+            final boolean longLivedStale;
+            if (maybeAttrsAfter.isPresent()) {
+                final ContainerNode attrsAfter = extractContainer(maybeAttrsAfter);
+                longLivedStale = isLongLivedStale(attrsAfter);
+                tx.put(LogicalDatastoreType.OPERATIONAL, effectiveTablePath.node(ATTRIBUTES_NID),
+                    effectiveAttributes(attrsAfter.getChild(UPTODATE_NID)));
+            } else {
+                longLivedStale = false;
+            }
+
+            writeRoutesAfter(tx, ribContext.getRibSupport(), effectiveTablePath,
+                NormalizedNodes.findNode(tableAfter, ROUTES_NID), longLivedStale);
         }
+    }
 
-        @Override
-        public synchronized void close() {
-            this.reg.close();
-            this.prefixesReceived.values().forEach(LongAdder::reset);
-            this.prefixesInstalled.values().forEach(LongAdder::reset);
+    // Performs house-keeping when the contents of a table is deleted
+    private void onDeleteTable(final RIBSupport<?, ?, ?, ?> ribSupport, final YangInstanceIdentifier effectiveTablePath,
+            final Optional<NormalizedNode<?, ?>> tableBefore) {
+        // Routes are special in that we need to process the to keep our counters accurate
+        final Optional<NormalizedNode<?, ?>> maybeRoutesBefore = findRoutesMap(ribSupport,
+                NormalizedNodes.findNode(tableBefore, ROUTES_NID));
+        if (maybeRoutesBefore.isPresent()) {
+            onRoutesDeleted(ribSupport, effectiveTablePath, extractMap(maybeRoutesBefore).getValue());
+        }
+    }
+
+    private void deleteRoutesBefore(final DOMDataTreeWriteTransaction tx, final RIBSupport<?, ?, ?, ?> ribSupport,
+            final YangInstanceIdentifier effectiveTablePath, final DataTreeCandidateNode modifiedRoutes) {
+        final Optional<NormalizedNode<?, ?>> maybeRoutesBefore = NormalizedNodes.findNode(
+            modifiedRoutes.getDataBefore(), ribSupport.relativeRoutesPath());
+        if (maybeRoutesBefore.isPresent()) {
+            onRoutesDeleted(ribSupport, effectiveTablePath, extractMap(maybeRoutesBefore).getValue());
         }
+    }
 
-        @Override
-        public long getPrefixedReceivedCount(final TablesKey tablesKey) {
-            final LongAdder counter = this.prefixesReceived.get(tablesKey);
-            if (counter == null) {
-                return 0;
+    private void writeRoutesAfter(final DOMDataTreeWriteTransaction tx, final RIBSupport<?, ?, ?, ?> ribSupport,
+            final YangInstanceIdentifier effectiveTablePath, final Optional<NormalizedNode<?, ?>> routesAfter,
+            final boolean longLivedStale) {
+        final Optional<NormalizedNode<?, ?>> maybeRoutesAfter = NormalizedNodes.findNode(routesAfter,
+            ribSupport.relativeRoutesPath());
+        if (maybeRoutesAfter.isPresent()) {
+            final YangInstanceIdentifier routesPath = routeMapPath(ribSupport, effectiveTablePath);
+            for (MapEntryNode routeAfter : extractMap(maybeRoutesAfter).getValue()) {
+                writeRoute(tx, ribSupport, routesPath.node(routeAfter.getIdentifier()), Optional.empty(), routeAfter,
+                    longLivedStale);
             }
-            return counter.longValue();
         }
+    }
 
-        @Override
-        public Set<TablesKey> getTableKeys() {
-            return ImmutableSet.copyOf(this.prefixesReceived.keySet());
+    private void onRoutesDeleted(final RIBSupport<?, ?, ?, ?> ribSupport,
+            final YangInstanceIdentifier effectiveTablePath, final Collection<MapEntryNode> deletedRoutes) {
+        if (ribSupport.getSafi() == RouteTargetConstrainSubsequentAddressFamily.class) {
+            final YangInstanceIdentifier routesPath = routeMapPath(ribSupport, effectiveTablePath);
+            for (final MapEntryNode routeBefore : deletedRoutes) {
+                deleteRouteTarget(ribSupport, routesPath.node(routeBefore.getIdentifier()), routeBefore);
+            }
+            this.rtMembershipsUpdated = true;
         }
 
-        @Override
-        public boolean isSupported(final TablesKey tablesKey) {
-            return this.prefixesReceived.containsKey(tablesKey);
+        final TablesKey tablesKey = ribSupport.getTablesKey();
+        CountersUtil.add(prefixesInstalled.get(tablesKey), tablesKey, -deletedRoutes.size());
+    }
+
+    private void processRoute(final DOMDataTreeWriteTransaction tx, final RIBSupport<?, ?, ?, ?> ribSupport,
+            final YangInstanceIdentifier routesPath, final DataTreeCandidateNode route, final boolean longLivedStale) {
+        LOG.debug("Process route {}", route.getIdentifier());
+        final YangInstanceIdentifier routePath = ribSupport.routePath(routesPath, route.getIdentifier());
+        switch (route.getModificationType()) {
+            case DELETE:
+            case DISAPPEARED:
+                deleteRoute(tx, ribSupport, routePath, route.getDataBefore().orElse(null));
+                break;
+            case UNMODIFIED:
+                // No-op
+                break;
+            case APPEARED:
+            case SUBTREE_MODIFIED:
+            case WRITE:
+                writeRoute(tx, ribSupport, routePath, route.getDataBefore(), route.getDataAfter().get(),
+                    longLivedStale);
+                break;
+            default:
+                LOG.warn("Ignoring unhandled route {}", route);
+                break;
         }
+    }
+
+    private void deleteRoute(final DOMDataTreeWriteTransaction tx, final RIBSupport<?, ?, ?, ?> ribSupport,
+            final YangInstanceIdentifier routeIdPath, final NormalizedNode<?, ?> route) {
+        handleRouteTarget(ModificationType.DELETE, ribSupport, routeIdPath, route);
+        tx.delete(LogicalDatastoreType.OPERATIONAL, routeIdPath);
+        LOG.debug("Route deleted. routeId={}", routeIdPath);
+        final TablesKey tablesKey = ribSupport.getTablesKey();
+        CountersUtil.decrement(this.prefixesInstalled.get(tablesKey), tablesKey);
+    }
 
-        @Override
-        public long getPrefixedInstalledCount(final TablesKey tablesKey) {
-            final LongAdder counter = this.prefixesInstalled.get(tablesKey);
-            if (counter == null) {
-                return 0;
+    private void writeRoute(final DOMDataTreeWriteTransaction tx, final RIBSupport<?, ?, ?, ?> ribSupport,
+            final YangInstanceIdentifier routePath, final Optional<NormalizedNode<?, ?>> routeBefore,
+            final NormalizedNode<?, ?> routeAfter, final boolean longLivedStale) {
+        final TablesKey tablesKey = ribSupport.getTablesKey();
+        CountersUtil.increment(this.prefixesReceived.get(tablesKey), tablesKey);
+        // Lookup per-table attributes from RIBSupport
+        final ContainerNode advertisedAttrs = (ContainerNode) NormalizedNodes.findNode(routeAfter,
+            ribSupport.routeAttributesIdentifier()).orElse(null);
+        final Attributes routeAttrs = ribSupport.attributeFromContainerNode(advertisedAttrs);
+        final Optional<Attributes> optEffAtt;
+        // In case we want to add LLGR_STALE we do not process route through policies since it may be
+        // considered as received with LLGR_STALE from peer which is not true.
+        if (longLivedStale) {
+            // LLGR procedures are in effect. If the route is tagged with NO_LLGR, it needs to be removed.
+            final List<Communities> effCommunities = routeAttrs.getCommunities();
+            if (effCommunities != null && effCommunities.contains(CommunityUtil.NO_LLGR)) {
+                deleteRoute(tx, ribSupport, routePath, routeBefore.orElse(null));
+                return;
             }
-            return counter.longValue();
+            optEffAtt = Optional.of(wrapLongLivedStale(routeAttrs));
+        } else {
+            final Class<? extends AfiSafiType> afiSafiType
+                = tableTypeRegistry.getAfiSafiType(ribSupport.getTablesKey()).get();
+            optEffAtt = this.ribPolicies
+                .applyImportPolicies(this.peerImportParameters, routeAttrs, afiSafiType);
+        }
+        if (!optEffAtt.isPresent()) {
+            deleteRoute(tx, ribSupport, routePath, routeBefore.orElse(null));
+            return;
         }
+        handleRouteTarget(ModificationType.WRITE, ribSupport, routePath, routeAfter);
+        tx.put(LogicalDatastoreType.OPERATIONAL, routePath, routeAfter);
+        CountersUtil.increment(this.prefixesInstalled.get(tablesKey), tablesKey);
+
+        final YangInstanceIdentifier attPath = routePath.node(ribSupport.routeAttributesIdentifier());
+        final Attributes attToStore = optEffAtt.get();
+        if (!attToStore.equals(routeAttrs)) {
+            final ContainerNode finalAttribute = ribSupport.attributeToContainerNode(attPath, attToStore);
+            tx.put(LogicalDatastoreType.OPERATIONAL, attPath, finalAttribute);
+        }
+    }
 
-        @Override
-        public long getTotalPrefixesInstalled() {
-            return this.prefixesInstalled.values().stream().mapToLong(LongAdder::longValue).sum();
+    private void addRouteTarget(final RouteTargetConstrainRoute rtc) {
+        final RouteTarget rtMembership = RouteTargetMembeshipUtil.getRT(rtc);
+        if (PeerRole.Ebgp != this.peerImportParameters.getFromPeerRole()) {
+            this.rtCache.cacheRoute(rtc);
         }
+        this.rtMemberships.add(rtMembership);
     }
 
-    private final AdjInTracker adjInTracker;
-    private final AbstractImportPolicy importPolicy;
+    private void deleteRouteTarget(final RIBSupport<?, ?, ?, ?> ribSupport, final YangInstanceIdentifier routeIdPath,
+            final NormalizedNode<?, ?> route) {
+        deleteRouteTarget((RouteTargetConstrainRoute) ribSupport.fromNormalizedNode(routeIdPath, route));
+    }
 
-    static EffectiveRibInWriter create(@Nonnull final RIB rib,
-            @Nonnull final DOMTransactionChain chain,
-            @Nonnull final YangInstanceIdentifier peerIId,
-            @Nonnull final ImportPolicyPeerTracker importPolicyPeerTracker,
-            final PeerRole peerRole,
-            @Nonnull final Set<TablesKey> tables) {
-        return new EffectiveRibInWriter(rib, chain, peerIId, importPolicyPeerTracker, peerRole, tables);
+    private void deleteRouteTarget(final RouteTargetConstrainRoute rtc) {
+        final RouteTarget rtMembership = RouteTargetMembeshipUtil.getRT(rtc);
+        if (PeerRole.Ebgp != this.peerImportParameters.getFromPeerRole()) {
+            this.rtCache.uncacheRoute(rtc);
+        }
+        this.rtMemberships.remove(rtMembership);
     }
 
-    private EffectiveRibInWriter(final RIB rib, final DOMTransactionChain chain, final YangInstanceIdentifier peerIId,
-            final ImportPolicyPeerTracker importPolicyPeerTracker, final PeerRole peerRole, @Nonnull final Set<TablesKey> tables) {
-        importPolicyPeerTracker.peerRoleChanged(peerIId, peerRole);
-        this.importPolicy = importPolicyPeerTracker.policyFor(IdentifierUtils.peerId((NodeIdentifierWithPredicates) peerIId.getLastPathArgument()));
-        this.adjInTracker = new AdjInTracker(rib, chain, peerIId, tables);
+    private void handleRouteTarget(final ModificationType modificationType, final RIBSupport<?, ?, ?, ?> ribSupport,
+            final YangInstanceIdentifier routeIdPath, final NormalizedNode<?, ?> route) {
+        if (ribSupport.getSafi() == RouteTargetConstrainSubsequentAddressFamily.class) {
+            final RouteTargetConstrainRoute rtc =
+                (RouteTargetConstrainRoute) ribSupport.fromNormalizedNode(routeIdPath, route);
+            if (ModificationType.DELETE == modificationType) {
+                deleteRouteTarget(rtc);
+            } else {
+                addRouteTarget(rtc);
+            }
+            this.rtMembershipsUpdated = true;
+        }
     }
 
-    @Override
-    public void close() {
-        this.adjInTracker.close();
+    @SuppressFBWarnings("UPM_UNCALLED_PRIVATE_METHOD")
+    private static Attributes wrapLongLivedStale(final Attributes attrs) {
+        if (attrs == null) {
+            return STALE_LLGR_ATTRIBUTES;
+        }
+
+        final List<Communities> oldCommunities = attrs.getCommunities();
+        final List<Communities> newCommunities;
+        if (oldCommunities != null) {
+            if (oldCommunities.contains(StaleCommunities.STALE_LLGR)) {
+                return attrs;
+            }
+            newCommunities = StaleCommunities.create(oldCommunities);
+        } else {
+            newCommunities = STALE_LLGR_COMMUNUTIES;
+        }
+
+        return new org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.message.rev200120
+                .path.attributes.AttributesBuilder(attrs).setCommunities(newCommunities).build();
     }
 
-    @Override
-    public long getPrefixedReceivedCount(final TablesKey tablesKey) {
-        return this.adjInTracker.getPrefixedReceivedCount(tablesKey);
+    // XXX: this should be moved to YangInstanceIdentifier at some point
+    private static YangInstanceIdentifier concat(final YangInstanceIdentifier parent, final List<PathArgument> args) {
+        YangInstanceIdentifier ret = parent;
+        for (PathArgument arg : args) {
+            ret = ret.node(arg);
+        }
+        return ret;
     }
 
-    @Override
-    public Set<TablesKey> getTableKeys() {
-        return this.adjInTracker.getTableKeys();
+    private YangInstanceIdentifier effectiveTablePath(final NodeIdentifierWithPredicates tableKey) {
+        return this.effRibTables.node(TABLES_NID).node(tableKey);
     }
 
-    @Override
-    public boolean isSupported(final TablesKey tablesKey) {
-        return this.adjInTracker.isSupported(tablesKey);
+    private static YangInstanceIdentifier routeMapPath(final RIBSupport<?, ?, ?, ?> ribSupport,
+            final YangInstanceIdentifier tablePath) {
+        return concat(tablePath.node(ROUTES_NID), ribSupport.relativeRoutesPath());
     }
 
-    @Override
-    public long getPrefixedInstalledCount(@Nonnull final TablesKey tablesKey) {
-        return this.adjInTracker.getPrefixedInstalledCount(tablesKey);
+    private static Optional<NormalizedNode<?, ?>> findRoutesMap(final RIBSupport<?, ?, ?, ?> ribSupport,
+            final Optional<NormalizedNode<?, ?>> optRoutes) {
+        return NormalizedNodes.findNode(optRoutes, ribSupport.relativeRoutesPath());
     }
 
-    @Override
-    public long getTotalPrefixesInstalled() {
-        return this.adjInTracker.getTotalPrefixesInstalled();
+    private static ContainerNode extractContainer(final Optional<? extends NormalizedNode<?, ?>> optNode) {
+        final NormalizedNode<?, ?> node = optNode.get();
+        verify(node instanceof ContainerNode, "Expected ContainerNode, got %s", node);
+        return (ContainerNode) node;
+    }
+
+    private static MapNode extractMap(final Optional<? extends NormalizedNode<?, ?>> optNode) {
+        final NormalizedNode<?, ?> node = optNode.get();
+        verify(node instanceof MapNode, "Expected MapNode, got %s", node);
+        return (MapNode) node;
+    }
+
+    private static MapEntryNode extractMapEntry(final Optional<? extends NormalizedNode<?, ?>> optNode) {
+        final NormalizedNode<?, ?> node = optNode.get();
+        verify(node instanceof MapEntryNode, "Expected MapEntryNode, got %s", node);
+        return (MapEntryNode) node;
+    }
+
+    private static boolean isLongLivedStale(final ContainerNode attributes) {
+        return NormalizedNodes.findNode(attributes, ADJRIBIN_ATTRIBUTES_AID, LLGR_STALE_NID).isPresent();
+    }
+
+    private static boolean isLongLivedStaleTable(final Optional<NormalizedNode<?, ?>> optTable) {
+        final Optional<NormalizedNode<?, ?>> optAttributes = NormalizedNodes.findNode(optTable, ATTRIBUTES_NID);
+        return optAttributes.isPresent() ? isLongLivedStale(extractContainer(optAttributes)) : false;
+    }
+
+    private static ContainerNode effectiveAttributes(final Optional<? extends NormalizedNode<?, ?>> optUptodate) {
+        return optUptodate.map(leaf -> {
+            final Object value = leaf.getValue();
+            verify(value instanceof Boolean, "Expected boolean uptodate, got %s", value);
+            return ((Boolean) value).booleanValue() ? RIBNormalizedNodes.UPTODATE_ATTRIBUTES
+                    : RIBNormalizedNodes.NOT_UPTODATE_ATTRIBUTES;
+        }).orElse(RIBNormalizedNodes.NOT_UPTODATE_ATTRIBUTES);
     }
 }