BUG-4931: Simple routing policy
[bgpcep.git] / bgp / rib-impl / src / main / java / org / opendaylight / protocol / bgp / rib / impl / ApplicationPeer.java
index a04410dfe143572ea75cbb3c9adb27cd5c8cb9a2..7b3d04fc070a51b901795497182b1c74a81ee5da 100644 (file)
 package org.opendaylight.protocol.bgp.rib.impl;
 
 import com.google.common.base.Preconditions;
+import com.google.common.base.Verify;
 import com.google.common.net.InetAddresses;
 import java.util.Arrays;
-import java.util.Map.Entry;
-import org.opendaylight.controller.md.sal.binding.api.DataChangeListener;
-import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeEvent;
-import org.opendaylight.protocol.bgp.rib.spi.AbstractAdjRIBs;
-import org.opendaylight.protocol.bgp.rib.spi.AdjRIBsIn;
-import org.opendaylight.protocol.bgp.rib.spi.Peer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Optional;
+import org.opendaylight.controller.md.sal.common.api.data.AsyncTransaction;
+import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
+import org.opendaylight.controller.md.sal.common.api.data.TransactionChain;
+import org.opendaylight.controller.md.sal.common.api.data.TransactionChainListener;
+import org.opendaylight.controller.md.sal.dom.api.DOMDataTreeChangeListener;
+import org.opendaylight.controller.md.sal.dom.api.DOMDataWriteTransaction;
+import org.opendaylight.controller.md.sal.dom.api.DOMTransactionChain;
+import org.opendaylight.protocol.bgp.openconfig.spi.BGPConfigModuleTracker;
+import org.opendaylight.protocol.bgp.rib.spi.IdentifierUtils;
+import org.opendaylight.protocol.bgp.rib.spi.RouterIds;
 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.inet.types.rev100924.Ipv4Address;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.message.rev130919.UpdateBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.message.rev130919.path.attributes.Attributes;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.message.rev130919.path.attributes.AttributesBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.multiprotocol.rev130919.Attributes1;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.multiprotocol.rev130919.Attributes1Builder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.multiprotocol.rev130919.Attributes2;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.multiprotocol.rev130919.Attributes2Builder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.multiprotocol.rev130919.BgpTableType;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.multiprotocol.rev130919.update.attributes.MpReachNlriBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.multiprotocol.rev130919.update.attributes.MpUnreachNlriBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev130925.ApplicationRibId;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev130925.Route;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev130925.PeerRole;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev130925.SimpleRoutingPolicy;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev130925.bgp.rib.rib.Peer;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev130925.bgp.rib.rib.peer.AdjRibIn;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev130925.rib.Tables;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.bgp.rib.rev130925.rib.TablesKey;
-import org.opendaylight.yangtools.yang.binding.DataObject;
-import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+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.NormalizedNode;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateNode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class ApplicationPeer implements AutoCloseable, Peer, DataChangeListener {
+/**
+ * Application Peer is a special case of BGP peer. It serves as an interface
+ * for user to advertise user routes to ODL and through ODL to other BGP peers.
+ *
+ * This peer has it's own RIB, where it stores all user routes. This RIB is
+ * located in configurational datastore. Routes are added through RESTCONF.
+ *
+ * They are then processed as routes from any other peer, through AdjRib,
+ * EffectiveRib,LocRib and if they are advertised further, through AdjRibOut.
+ *
+ * For purposed of import policies such as Best Path Selection, application
+ * peer needs to have a BGP-ID that is configurable.
+ */
+public class ApplicationPeer implements AutoCloseable, org.opendaylight.protocol.bgp.rib.spi.Peer, DOMDataTreeChangeListener, TransactionChainListener {
 
     private static final Logger LOG = LoggerFactory.getLogger(ApplicationPeer.class);
 
     private final byte[] rawIdentifier;
-    private final RIBImpl targetRib;
     private final String name;
+    private final YangInstanceIdentifier adjRibsInId;
+    private final DOMTransactionChain chain;
+    private final DOMTransactionChain writerChain;
+    private final BGPConfigModuleTracker moduleTracker;
+    private final EffectiveRibInWriter effectiveRibInWriter;
+    private AdjRibInWriter writer;
 
-    public ApplicationPeer(final ApplicationRibId applicationRibId, final Ipv4Address ipAddress, final RIBImpl targetRib) {
+    public ApplicationPeer(final ApplicationRibId applicationRibId, final Ipv4Address ipAddress, final RIBImpl rib, final BGPConfigModuleTracker
+        moduleTracker) {
         this.name = applicationRibId.getValue().toString();
-        this.targetRib = Preconditions.checkNotNull(targetRib);
+        final RIBImpl targetRib = Preconditions.checkNotNull(rib);
         this.rawIdentifier = InetAddresses.forString(ipAddress.getValue()).getAddress();
-    }
-
-    @Override
-    public void onDataChanged(final AsyncDataChangeEvent<InstanceIdentifier<?>, DataObject> change) {
-        final UpdateBuilder ub = new UpdateBuilder();
-        for (final Entry<InstanceIdentifier<?>, DataObject> data : change.getCreatedData().entrySet()) {
-            fillMpReach(ub, data);
-        }
-        for (final Entry<InstanceIdentifier<?>, DataObject> data : change.getUpdatedData().entrySet()) {
-            fillMpReach(ub, data);
-        }
-        for (final InstanceIdentifier<?> data : change.getRemovedPaths()) {
-            final MpUnreachNlriBuilder unreachBuilder = new MpUnreachNlriBuilder();
-            final TablesKey key = data.firstKeyOf(Tables.class, TablesKey.class);
-            unreachBuilder.setAfi(key.getAfi());
-            unreachBuilder.setSafi(key.getSafi());
-            final AbstractAdjRIBs<?,?,?> ribsIn = (AbstractAdjRIBs<?,?,?>)this.targetRib.getTable(key);
-            ribsIn.addWith(unreachBuilder, data);
-            ub.setAttributes(new AttributesBuilder().addAugmentation(Attributes2.class, new Attributes2Builder().setMpUnreachNlri(unreachBuilder.build()).build()).build());
-            LOG.debug("Updating RIB with {}", ub.build());
-            this.targetRib.updateTables(this, ub.build());
+        final NodeIdentifierWithPredicates peerId = IdentifierUtils.domPeerId(RouterIds.createPeerId(ipAddress));
+        final YangInstanceIdentifier peerIId = targetRib.getYangRibId().node(Peer.QNAME).node(peerId);
+        this.adjRibsInId = peerIId.node(AdjRibIn.QNAME).node(Tables.QNAME);
+        this.chain = targetRib.createPeerChain(this);
+        this.effectiveRibInWriter = EffectiveRibInWriter.create(targetRib.getService(), targetRib.createPeerChain(this), peerIId,
+            targetRib.getImportPolicyPeerTracker(), targetRib.getRibSupportContext(), PeerRole.Internal);
+        this.writerChain = targetRib.createPeerChain(this);
+        this.writer = AdjRibInWriter.create(targetRib.getYangRibId(), PeerRole.Internal, Optional.of(SimpleRoutingPolicy.AnnounceNone), this.writerChain);
+        this.writer = this.writer.transform(RouterIds.createPeerId(ipAddress), targetRib.getRibSupportContext(), targetRib.getLocalTablesKeys(),
+            Collections.emptyList());
+        this.moduleTracker = moduleTracker;
+        if (moduleTracker != null) {
+            moduleTracker.onInstanceCreate();
         }
+    }
 
+    public ApplicationPeer(final ApplicationRibId applicationRibId, final Ipv4Address bgpPeerId, final RIBImpl targetRibDependency) {
+        this(applicationRibId, bgpPeerId, targetRibDependency, null);
     }
 
-    private void fillMpReach(final UpdateBuilder ub, final Entry<InstanceIdentifier<?>, DataObject> data) {
-        if (data.getValue() instanceof Route) {
-            final Route r = (Route) data.getValue();
-            final MpReachNlriBuilder reachBuilder = new MpReachNlriBuilder();
-            final TablesKey key = data.getKey().firstKeyOf(Tables.class, TablesKey.class);
-            reachBuilder.setAfi(key.getAfi());
-            reachBuilder.setSafi(key.getSafi());
-            final AdjRIBsIn<?,Route> ribsIn = this.targetRib.getTable(key);
-            ribsIn.addAdvertisement(reachBuilder, (Route)data.getValue());
-            final AttributesBuilder pa = new AttributesBuilder();
-            pa.addAugmentation(Attributes1.class, new Attributes1Builder().setMpReachNlri(reachBuilder.build()).build());
-            this.addAttributes(pa, r.getAttributes());
-            pa.setCNextHop(reachBuilder.getCNextHop());
-            ub.setAttributes(pa.build());
-            LOG.debug("Updating RIB with {}", ub.build());
-            this.targetRib.updateTables(this, ub.build());
+    /**
+     * Routes come from application RIB that is identified by (configurable) name.
+     * Each route is pushed into AdjRibsInWriter with it's whole context. In this
+     * method, it doesn't matter if the routes are removed or added, this will
+     * be determined in LocRib.
+     */
+    @Override
+    public void onDataTreeChanged(final Collection<DataTreeCandidate> changes) {
+        final DOMDataWriteTransaction tx = this.chain.newWriteOnlyTransaction();
+        LOG.debug("Received data change to ApplicationRib {}", changes);
+        for (final DataTreeCandidate tc : changes) {
+            LOG.debug("Modification Type {}", tc.getRootNode().getModificationType());
+            final YangInstanceIdentifier path = tc.getRootPath();
+            final PathArgument lastArg = path.getLastPathArgument();
+            Verify.verify(lastArg instanceof NodeIdentifierWithPredicates, "Unexpected type %s in path %s", lastArg.getClass(), path);
+            final NodeIdentifierWithPredicates tableKey = (NodeIdentifierWithPredicates) lastArg;
+            for (final DataTreeCandidateNode child : tc.getRootNode().getChildNodes()) {
+                final PathArgument childIdentifier = child.getIdentifier();
+                final YangInstanceIdentifier tableId = this.adjRibsInId.node(tableKey).node(childIdentifier);
+                switch (child.getModificationType()) {
+                case DELETE:
+                    LOG.trace("App peer -> AdjRibsIn path delete: {}", childIdentifier);
+                    tx.delete(LogicalDatastoreType.OPERATIONAL, tableId);
+                    break;
+                case UNMODIFIED:
+                    // No-op
+                    break;
+                case SUBTREE_MODIFIED:
+                    if (EffectiveRibInWriter.TABLE_ROUTES.equals(childIdentifier)) {
+                        processRoutesTable(child, tableId, tx, tableId);
+                        break;
+                    }
+                case WRITE:
+                    if (child.getDataAfter().isPresent()) {
+                        final NormalizedNode<?,?> dataAfter = child.getDataAfter().get();
+                        LOG.trace("App peer -> AdjRibsIn path : {}", tableId);
+                        LOG.trace("App peer -> AdjRibsIn data : {}", dataAfter);
+                        tx.put(LogicalDatastoreType.OPERATIONAL, tableId, dataAfter);
+                    }
+                    break;
+                default:
+                    break;
+                }
+            }
         }
+        tx.submit();
     }
 
-    private void addAttributes(final AttributesBuilder pa, final Attributes a) {
-        if (a != null) {
-            pa.setAggregator(a.getAggregator());
-            pa.setAsPath(a.getAsPath());
-            pa.setAtomicAggregate(a.getAtomicAggregate());
-            pa.setClusterId(a.getClusterId());
-            pa.setCommunities(a.getCommunities());
-            pa.setExtendedCommunities(a.getExtendedCommunities());
-            pa.setLocalPref(a.getLocalPref());
-            pa.setMultiExitDisc(a.getMultiExitDisc());
-            pa.setOrigin(a.getOrigin());
-            pa.setOriginatorId(a.getOriginatorId());
+    /**
+     * Applies modification under table routes based on modification type instead of only put. BUG 4438
+     * @param node
+     * @param identifier
+     * @param tx
+     * @param routeTableIdentifier
+     */
+    private void processRoutesTable(final DataTreeCandidateNode node, final YangInstanceIdentifier identifier,
+        final DOMDataWriteTransaction tx, final YangInstanceIdentifier routeTableIdentifier) {
+        for (final DataTreeCandidateNode child : node.getChildNodes()) {
+            final YangInstanceIdentifier childIdentifier = identifier.node(child.getIdentifier());
+            switch (child.getModificationType()) {
+            case DELETE:
+                LOG.trace("App peer -> AdjRibsIn path delete: {}", childIdentifier);
+                tx.delete(LogicalDatastoreType.OPERATIONAL, childIdentifier);
+                break;
+            case UNMODIFIED:
+                // No-op
+                break;
+            case SUBTREE_MODIFIED:
+                //For be ables to use DELETE when we remove specific routes as we do when we remove the whole routes,
+                // we need to go deeper three levels
+                if (!routeTableIdentifier.equals(childIdentifier.getParent().getParent().getParent())) {
+                    processRoutesTable(child, childIdentifier, tx, routeTableIdentifier);
+                    break;
+                }
+            case WRITE:
+                if (child.getDataAfter().isPresent()) {
+                    final NormalizedNode<?,?> dataAfter = child.getDataAfter().get();
+                    LOG.trace("App peer -> AdjRibsIn path : {}", childIdentifier);
+                    LOG.trace("App peer -> AdjRibsIn data : {}", dataAfter);
+                    tx.put(LogicalDatastoreType.OPERATIONAL, childIdentifier, dataAfter);
+                }
+                break;
+            default:
+                break;
+            }
         }
     }
 
@@ -114,8 +186,12 @@ public class ApplicationPeer implements AutoCloseable, Peer, DataChangeListener
 
     @Override
     public void close() {
-        for (final BgpTableType t : this.targetRib.getLocalTables()) {
-            this.targetRib.clearTable(this, new TablesKey(t.getAfi(), t.getSafi()));
+        this.effectiveRibInWriter.close();
+        this.writer.removePeer();
+        this.chain.close();
+        this.writerChain.close();
+        if (this.moduleTracker != null) {
+            this.moduleTracker.onInstanceClose();
         }
     }
 
@@ -123,4 +199,15 @@ public class ApplicationPeer implements AutoCloseable, Peer, DataChangeListener
     public byte[] getRawIdentifier() {
         return Arrays.copyOf(this.rawIdentifier, this.rawIdentifier.length);
     }
+
+    @Override
+    public void onTransactionChainFailed(final TransactionChain<?, ?> chain, final AsyncTransaction<?, ?> transaction,
+        final Throwable cause) {
+        LOG.error("Transaction chain failed.", cause);
+    }
+
+    @Override
+    public void onTransactionChainSuccessful(final TransactionChain<?, ?> chain) {
+        LOG.debug("Transaction chain {} successfull.", chain);
+    }
 }