CORD-1578 CORD-1708 Handle dual single homed router and single dual homed router
In addition
- Carry all alternatives in ROUTE_UPDATED and ROUTE_REMOVED event
- More unit tests
- More debug messages
- Fix routing table out-of-sync issue in MockRoutingService
- Fix populateRoute/revokeRoute mastership issue
Change-Id: I22d537625b570b09ecd4e22e6e14bb1ee27f8bcb
diff --git a/src/main/java/org/onosproject/segmentrouting/DefaultRoutingHandler.java b/src/main/java/org/onosproject/segmentrouting/DefaultRoutingHandler.java
index 50b0cd2..0d263e7 100644
--- a/src/main/java/org/onosproject/segmentrouting/DefaultRoutingHandler.java
+++ b/src/main/java/org/onosproject/segmentrouting/DefaultRoutingHandler.java
@@ -27,11 +27,14 @@
import org.onlab.packet.Ip4Address;
import org.onlab.packet.Ip6Address;
import org.onlab.packet.IpPrefix;
+import org.onlab.packet.MacAddress;
+import org.onlab.packet.VlanId;
import org.onosproject.cluster.NodeId;
import org.onosproject.net.ConnectPoint;
import org.onosproject.net.Device;
import org.onosproject.net.DeviceId;
import org.onosproject.net.Link;
+import org.onosproject.net.PortNumber;
import org.onosproject.segmentrouting.config.DeviceConfigNotFoundException;
import org.onosproject.segmentrouting.config.DeviceConfiguration;
import org.onosproject.segmentrouting.grouphandler.DefaultGroupHandler;
@@ -238,7 +241,8 @@
*
* @param cpts connect point(s) of the subnets being added
* @param subnets subnets being added
- */ //XXX refactor
+ */
+ // XXX refactor
protected void populateSubnet(Set<ConnectPoint> cpts, Set<IpPrefix> subnets) {
lastRoutingChange = DateTime.now();
statusLock.lock();
@@ -250,7 +254,8 @@
}
populationStatus = Status.STARTED;
rulePopulator.resetCounter();
- log.info("Starting to populate routing rules for added routes");
+ log.info("Starting to populate routing rules for added routes, subnets={}, cpts={}",
+ subnets, cpts);
// Take snapshots of the topology
updatedEcmpSpgMap = new HashMap<>();
Set<EdgePair> edgePairs = new HashSet<>();
@@ -972,6 +977,40 @@
}
/**
+ * Populates IP rules for a route that has direct connection to the switch
+ * if the current instance is the master of the switch.
+ *
+ * @param deviceId device ID of the device that next hop attaches to
+ * @param prefix IP prefix of the route
+ * @param hostMac MAC address of the next hop
+ * @param hostVlanId Vlan ID of the nexthop
+ * @param outPort port where the next hop attaches to
+ */
+ void populateRoute(DeviceId deviceId, IpPrefix prefix,
+ MacAddress hostMac, VlanId hostVlanId, PortNumber outPort) {
+ if (srManager.mastershipService.isLocalMaster(deviceId)) {
+ srManager.routingRulePopulator.populateRoute(deviceId, prefix, hostMac, hostVlanId, outPort);
+ }
+ }
+
+ /**
+ * Removes IP rules for a route when the next hop is gone.
+ * if the current instance is the master of the switch.
+ *
+ * @param deviceId device ID of the device that next hop attaches to
+ * @param prefix IP prefix of the route
+ * @param hostMac MAC address of the next hop
+ * @param hostVlanId Vlan ID of the nexthop
+ * @param outPort port that next hop attaches to
+ */
+ void revokeRoute(DeviceId deviceId, IpPrefix prefix,
+ MacAddress hostMac, VlanId hostVlanId, PortNumber outPort) {
+ if (srManager.mastershipService.isLocalMaster(deviceId)) {
+ srManager.routingRulePopulator.revokeRoute(deviceId, prefix, hostMac, hostVlanId, outPort);
+ }
+ }
+
+ /**
* Remove ECMP graph entry for the given device. Typically called when
* device is no longer available.
*
diff --git a/src/main/java/org/onosproject/segmentrouting/HostHandler.java b/src/main/java/org/onosproject/segmentrouting/HostHandler.java
index 5029b30..7a205ee 100644
--- a/src/main/java/org/onosproject/segmentrouting/HostHandler.java
+++ b/src/main/java/org/onosproject/segmentrouting/HostHandler.java
@@ -50,7 +50,6 @@
* Handles host-related events.
*/
public class HostHandler {
- private static final String NOT_MASTER = "Current instance is not the master of {}. Ignore.";
private static final Logger log = LoggerFactory.getLogger(HostHandler.class);
protected final SegmentRoutingManager srManager;
@@ -87,8 +86,7 @@
void processHostAddedAtLocation(Host host, HostLocation location) {
checkArgument(host.locations().contains(location), "{} is not a location of {}", location, host);
- if (!isMasterOf(location)) {
- log.debug(NOT_MASTER, location);
+ if (!srManager.isMasterOf(location)) {
return;
}
@@ -115,16 +113,19 @@
Set<IpAddress> ips = host.ipAddresses();
log.info("Host {}/{} is removed from {}", hostMac, hostVlanId, locations);
- locations.stream().filter(this::isMasterOf).forEach(location -> {
- processBridgingRule(location.deviceId(), location.port(), hostMac, hostVlanId, true);
- ips.forEach(ip ->
- processRoutingRule(location.deviceId(), location.port(), hostMac, hostVlanId, ip, true)
- );
+ locations.forEach(location -> {
+ if (srManager.isMasterOf(location)) {
+ processBridgingRule(location.deviceId(), location.port(), hostMac, hostVlanId, true);
+ ips.forEach(ip ->
+ processRoutingRule(location.deviceId(), location.port(), hostMac, hostVlanId, ip, true)
+ );
+ }
// Also remove redirection flows on the pair device if exists.
Optional<DeviceId> pairDeviceId = srManager.getPairDeviceId(location.deviceId());
Optional<PortNumber> pairLocalPort = srManager.getPairLocalPorts(location.deviceId());
- if (pairDeviceId.isPresent() && pairLocalPort.isPresent()) {
+ if (pairDeviceId.isPresent() && pairLocalPort.isPresent() &&
+ srManager.mastershipService.isLocalMaster(pairDeviceId.get())) {
// NOTE: Since the pairLocalPort is trunk port, use assigned vlan of original port
// when the host is untagged
VlanId vlanId = Optional.ofNullable(srManager.getInternalVlanId(location)).orElse(hostVlanId);
@@ -150,7 +151,7 @@
.collect(Collectors.toSet());
// For each old location
- Sets.difference(prevLocations, newLocations).stream().filter(this::isMasterOf)
+ Sets.difference(prevLocations, newLocations).stream().filter(srManager::isMasterOf)
.forEach(prevLocation -> {
// Remove routing rules for old IPs
Sets.difference(prevIps, newIps).forEach(ip ->
@@ -211,7 +212,7 @@
});
// For each new location, add all new IPs.
- Sets.difference(newLocations, prevLocations).stream().filter(this::isMasterOf)
+ Sets.difference(newLocations, prevLocations).stream().filter(srManager::isMasterOf)
.forEach(newLocation -> {
processBridgingRule(newLocation.deviceId(), newLocation.port(), hostMac, hostVlanId, false);
newIps.forEach(ip ->
@@ -221,7 +222,7 @@
});
// For each unchanged location, add new IPs and remove old IPs.
- Sets.intersection(newLocations, prevLocations).stream().filter(this::isMasterOf)
+ Sets.intersection(newLocations, prevLocations).stream().filter(srManager::isMasterOf)
.forEach(unchangedLocation -> {
Sets.difference(prevIps, newIps).forEach(ip ->
processRoutingRule(unchangedLocation.deviceId(), unchangedLocation.port(), hostMac,
@@ -243,7 +244,7 @@
Set<IpAddress> newIps = event.subject().ipAddresses();
log.info("Host {}/{} is updated", mac, vlanId);
- locations.stream().filter(this::isMasterOf).forEach(location -> {
+ locations.stream().filter(srManager::isMasterOf).forEach(location -> {
Sets.difference(prevIps, newIps).forEach(ip ->
processRoutingRule(location.deviceId(), location.port(), mac, vlanId, ip, true)
);
@@ -378,20 +379,9 @@
log.info("{} routing rule for {} at {}", revoke ? "Revoking" : "Populating", ip, location);
if (revoke) {
- srManager.routingRulePopulator.revokeRoute(deviceId, ip.toIpPrefix(), mac, vlanId, port);
+ srManager.defaultRoutingHandler.revokeRoute(deviceId, ip.toIpPrefix(), mac, vlanId, port);
} else {
- srManager.routingRulePopulator.populateRoute(deviceId, ip.toIpPrefix(), mac, vlanId, port);
+ srManager.defaultRoutingHandler.populateRoute(deviceId, ip.toIpPrefix(), mac, vlanId, port);
}
}
-
- /**
- * Determine if current instance is the master of given connect point.
- *
- * @param cp connect point
- * @return true if current instance is the master of given connect point
- */
- private boolean isMasterOf(ConnectPoint cp) {
- log.debug(NOT_MASTER, cp);
- return srManager.mastershipService.isLocalMaster(cp.deviceId());
- }
}
diff --git a/src/main/java/org/onosproject/segmentrouting/RouteHandler.java b/src/main/java/org/onosproject/segmentrouting/RouteHandler.java
index 214ba6b..c80bc09 100644
--- a/src/main/java/org/onosproject/segmentrouting/RouteHandler.java
+++ b/src/main/java/org/onosproject/segmentrouting/RouteHandler.java
@@ -16,20 +16,33 @@
package org.onosproject.segmentrouting;
-import com.google.common.collect.ImmutableSet;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.RemovalCause;
+import com.google.common.cache.RemovalNotification;
import com.google.common.collect.Sets;
import org.onlab.packet.IpPrefix;
import org.onlab.packet.MacAddress;
import org.onlab.packet.VlanId;
import org.onosproject.net.ConnectPoint;
+import org.onosproject.net.HostLocation;
+import org.onosproject.net.PortNumber;
+import org.onosproject.net.host.HostEvent;
import org.onosproject.routeservice.ResolvedRoute;
import org.onosproject.routeservice.RouteEvent;
import org.onosproject.net.DeviceId;
+import org.onosproject.routeservice.RouteInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.util.Collection;
import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
/**
* Handles RouteEvent and manages routing entries.
@@ -38,8 +51,35 @@
private static final Logger log = LoggerFactory.getLogger(RouteHandler.class);
private final SegmentRoutingManager srManager;
+ private static final int WAIT_TIME_MS = 1000;
+ /**
+ * The routeEventCache is implemented to avoid race condition by giving more time to the
+ * underlying flow subsystem to process previous populateSubnet call.
+ */
+ private Cache<IpPrefix, RouteEvent> routeEventCache = CacheBuilder.newBuilder()
+ .expireAfterWrite(WAIT_TIME_MS, TimeUnit.MILLISECONDS)
+ .removalListener((RemovalNotification<IpPrefix, RouteEvent> notification) -> {
+ IpPrefix prefix = notification.getKey();
+ RouteEvent routeEvent = notification.getValue();
+ RemovalCause cause = notification.getCause();
+ log.debug("routeEventCache removal event. prefix={}, routeEvent={}, cause={}",
+ prefix, routeEvent, cause);
+
+ switch (notification.getCause()) {
+ case REPLACED:
+ case EXPIRED:
+ dequeueRouteEvent(routeEvent);
+ break;
+ default:
+ break;
+ }
+ }).build();
+
RouteHandler(SegmentRoutingManager srManager) {
this.srManager = srManager;
+
+ Executors.newSingleThreadScheduledExecutor()
+ .scheduleAtFixedRate(routeEventCache::cleanUp, 0, WAIT_TIME_MS, TimeUnit.MILLISECONDS);
}
protected void init(DeviceId deviceId) {
@@ -56,70 +96,239 @@
}
void processRouteAdded(RouteEvent event) {
- log.info("processRouteAdded {}", event);
- processRouteAddedInternal(event.subject());
+ enqueueRouteEvent(event);
}
private void processRouteAddedInternal(ResolvedRoute route) {
+ processRouteAddedInternal(Sets.newHashSet(route));
+ }
+
+ private void processRouteAddedInternal(Collection<ResolvedRoute> routes) {
if (!isReady()) {
- log.info("System is not ready. Skip adding route for {}", route.prefix());
+ log.info("System is not ready. Skip adding route for {}", routes);
return;
}
- IpPrefix prefix = route.prefix();
- MacAddress nextHopMac = route.nextHopMac();
- VlanId nextHopVlan = route.nextHopVlan();
- ConnectPoint location = srManager.nextHopLocations(route).stream().findFirst().orElse(null);
+ log.info("processRouteAddedInternal. routes={}", routes);
- if (location == null) {
- log.info("{} ignored. Cannot find nexthop location", prefix);
- return;
- }
+ Set<ConnectPoint> allLocations = Sets.newHashSet();
+ Set<IpPrefix> allPrefixes = Sets.newHashSet();
+ routes.forEach(route -> {
+ allLocations.addAll(srManager.nextHopLocations(route));
+ allPrefixes.add(route.prefix());
+ });
+ log.debug("RouteAdded. populateSubnet {}, {}", allLocations, allPrefixes);
+ srManager.defaultRoutingHandler.populateSubnet(allLocations, allPrefixes);
- srManager.deviceConfiguration.addSubnet(location, prefix);
- // XXX need to handle the case where there are two connectpoints
- srManager.defaultRoutingHandler.populateSubnet(Sets.newHashSet(location),
- Sets.newHashSet(prefix));
- srManager.routingRulePopulator.populateRoute(location.deviceId(), prefix,
- nextHopMac, nextHopVlan, location.port());
+ routes.forEach(route -> {
+ IpPrefix prefix = route.prefix();
+ MacAddress nextHopMac = route.nextHopMac();
+ VlanId nextHopVlan = route.nextHopVlan();
+ Set<ConnectPoint> locations = srManager.nextHopLocations(route);
+
+ locations.forEach(location -> {
+ log.debug("RouteAdded. addSubnet {}, {}", location, prefix);
+ srManager.deviceConfiguration.addSubnet(location, prefix);
+ log.debug("RouteAdded populateRoute {}, {}, {}, {}", location, prefix, nextHopMac, nextHopVlan);
+ srManager.defaultRoutingHandler.populateRoute(location.deviceId(), prefix,
+ nextHopMac, nextHopVlan, location.port());
+ });
+ });
}
void processRouteUpdated(RouteEvent event) {
- log.info("processRouteUpdated {}", event);
- processRouteRemovedInternal(event.prevSubject());
- processRouteAddedInternal(event.subject());
+ enqueueRouteEvent(event);
+ }
+
+ void processAlternativeRoutesChanged(RouteEvent event) {
+ enqueueRouteEvent(event);
+ }
+
+ private void processRouteUpdatedInternal(Set<ResolvedRoute> routes, Set<ResolvedRoute> oldRoutes) {
+ if (!isReady()) {
+ log.info("System is not ready. Skip updating route for {} -> {}", oldRoutes, routes);
+ return;
+ }
+
+ log.info("processRouteUpdatedInternal. routes={}, oldRoutes={}", routes, oldRoutes);
+
+ Set<ConnectPoint> allLocations = Sets.newHashSet();
+ Set<IpPrefix> allPrefixes = Sets.newHashSet();
+ routes.forEach(route -> {
+ allLocations.addAll(srManager.nextHopLocations(route));
+ allPrefixes.add(route.prefix());
+ });
+ log.debug("RouteUpdated. populateSubnet {}, {}", allLocations, allPrefixes);
+ srManager.defaultRoutingHandler.populateSubnet(allLocations, allPrefixes);
+
+
+ Set<ResolvedRoute> toBeRemoved = Sets.difference(oldRoutes, routes).immutableCopy();
+ Set<ResolvedRoute> toBeAdded = Sets.difference(routes, oldRoutes).immutableCopy();
+
+ toBeRemoved.forEach(route -> {
+ srManager.nextHopLocations(route).forEach(oldLocation -> {
+ if (toBeAdded.stream().map(srManager::nextHopLocations)
+ .flatMap(Set::stream).map(ConnectPoint::deviceId)
+ .noneMatch(deviceId -> deviceId.equals(oldLocation.deviceId()))) {
+ IpPrefix prefix = route.prefix();
+ MacAddress nextHopMac = route.nextHopMac();
+ VlanId nextHopVlan = route.nextHopVlan();
+
+ log.debug("RouteUpdated. removeSubnet {}, {}", oldLocation, prefix);
+ srManager.deviceConfiguration.removeSubnet(oldLocation, prefix);
+ log.debug("RouteUpdated. revokeRoute {}, {}, {}, {}", oldLocation, prefix, nextHopMac, nextHopVlan);
+ srManager.defaultRoutingHandler.revokeRoute(oldLocation.deviceId(), prefix,
+ nextHopMac, nextHopVlan, oldLocation.port());
+ }
+ });
+ });
+
+ toBeAdded.forEach(route -> {
+ IpPrefix prefix = route.prefix();
+ MacAddress nextHopMac = route.nextHopMac();
+ VlanId nextHopVlan = route.nextHopVlan();
+ Set<ConnectPoint> locations = srManager.nextHopLocations(route);
+
+ locations.forEach(location -> {
+ log.debug("RouteUpdated. addSubnet {}, {}", location, prefix);
+ srManager.deviceConfiguration.addSubnet(location, prefix);
+ log.debug("RouteUpdated. populateRoute {}, {}, {}, {}", location, prefix, nextHopMac, nextHopVlan);
+ srManager.defaultRoutingHandler.populateRoute(location.deviceId(), prefix,
+ nextHopMac, nextHopVlan, location.port());
+ });
+ });
+
}
void processRouteRemoved(RouteEvent event) {
- log.info("processRouteRemoved {}", event);
- processRouteRemovedInternal(event.subject());
+ enqueueRouteEvent(event);
}
- private void processRouteRemovedInternal(ResolvedRoute route) {
+ private void processRouteRemovedInternal(Collection<ResolvedRoute> routes) {
if (!isReady()) {
- log.info("System is not ready. Skip removing route for {}", route.prefix());
+ log.info("System is not ready. Skip removing route for {}", routes);
return;
}
- IpPrefix prefix = route.prefix();
- MacAddress nextHopMac = route.nextHopMac();
- VlanId nextHopVlan = route.nextHopVlan();
- ConnectPoint location = srManager.nextHopLocations(route).stream().findFirst().orElse(null);
+ log.info("processRouteRemovedInternal. routes={}", routes);
- if (location == null) {
- log.info("{} ignored. Cannot find nexthop location", prefix);
- return;
- }
+ Set<IpPrefix> allPrefixes = Sets.newHashSet();
+ routes.forEach(route -> {
+ allPrefixes.add(route.prefix());
+ });
+ log.debug("RouteRemoved. revokeSubnet {}", allPrefixes);
+ srManager.defaultRoutingHandler.revokeSubnet(allPrefixes);
- srManager.deviceConfiguration.removeSubnet(location, prefix);
- srManager.defaultRoutingHandler.revokeSubnet(ImmutableSet.of(prefix));
- srManager.routingRulePopulator.revokeRoute(
- location.deviceId(), prefix, nextHopMac, nextHopVlan, location.port());
+ routes.forEach(route -> {
+ IpPrefix prefix = route.prefix();
+ MacAddress nextHopMac = route.nextHopMac();
+ VlanId nextHopVlan = route.nextHopVlan();
+ Set<ConnectPoint> locations = srManager.nextHopLocations(route);
+
+ locations.forEach(location -> {
+ log.debug("RouteRemoved. removeSubnet {}, {}", location, prefix);
+ srManager.deviceConfiguration.removeSubnet(location, prefix);
+ log.debug("RouteRemoved. revokeRoute {}, {}, {}, {}", location, prefix, nextHopMac, nextHopVlan);
+ srManager.defaultRoutingHandler.revokeRoute(location.deviceId(), prefix,
+ nextHopMac, nextHopVlan, location.port());
+
+ // Also remove redirection flows on the pair device if exists.
+ Optional<DeviceId> pairDeviceId = srManager.getPairDeviceId(location.deviceId());
+ Optional<PortNumber> pairLocalPort = srManager.getPairLocalPorts(location.deviceId());
+ if (pairDeviceId.isPresent() && pairLocalPort.isPresent()) {
+ // NOTE: Since the pairLocalPort is trunk port, use assigned vlan of original port
+ // when the host is untagged
+ VlanId vlanId = Optional.ofNullable(srManager.getInternalVlanId(location)).orElse(nextHopVlan);
+
+ log.debug("RouteRemoved. revokeRoute {}, {}, {}, {}", location, prefix, nextHopMac, nextHopVlan);
+ srManager.defaultRoutingHandler.revokeRoute(pairDeviceId.get(), prefix,
+ nextHopMac, vlanId, pairLocalPort.get());
+ }
+ });
+ });
+ }
+
+ void processHostMovedEvent(HostEvent event) {
+ log.info("processHostMovedEvent {}", event);
+ MacAddress hostMac = event.subject().mac();
+ VlanId hostVlanId = event.subject().vlan();
+
+ affectedRoutes(hostMac, hostVlanId).forEach(affectedRoute -> {
+ IpPrefix prefix = affectedRoute.prefix();
+ Set<HostLocation> prevLocations = event.prevSubject().locations();
+ Set<HostLocation> newLocations = event.subject().locations();
+
+ // For each old location
+ Sets.difference(prevLocations, newLocations).stream().filter(srManager::isMasterOf)
+ .forEach(prevLocation -> {
+ // Redirect the flows to pair link if configured
+ // Note: Do not continue removing any rule
+ Optional<DeviceId> pairDeviceId = srManager.getPairDeviceId(prevLocation.deviceId());
+ Optional<PortNumber> pairLocalPort = srManager.getPairLocalPorts(prevLocation.deviceId());
+ if (pairDeviceId.isPresent() && pairLocalPort.isPresent() && newLocations.stream()
+ .anyMatch(location -> location.deviceId().equals(pairDeviceId.get()))) {
+ // NOTE: Since the pairLocalPort is trunk port, use assigned vlan of original port
+ // when the host is untagged
+ VlanId vlanId = Optional.ofNullable(srManager.getInternalVlanId(prevLocation)).orElse(hostVlanId);
+ log.debug("HostMoved. populateRoute {}, {}, {}, {}", prevLocation, prefix, hostMac, vlanId);
+ srManager.defaultRoutingHandler.populateRoute(prevLocation.deviceId(), prefix,
+ hostMac, vlanId, pairLocalPort.get());
+ return;
+ }
+
+ // No pair information supplied. Remove route
+ log.debug("HostMoved. revokeRoute {}, {}, {}, {}", prevLocation, prefix, hostMac, hostVlanId);
+ srManager.defaultRoutingHandler.revokeRoute(prevLocation.deviceId(), prefix,
+ hostMac, hostVlanId, prevLocation.port());
+ });
+
+ // For each new location, add all new IPs.
+ Sets.difference(newLocations, prevLocations).stream().filter(srManager::isMasterOf)
+ .forEach(newLocation -> {
+ log.debug("HostMoved. populateRoute {}, {}, {}, {}", newLocation, prefix, hostMac, hostVlanId);
+ srManager.defaultRoutingHandler.populateRoute(newLocation.deviceId(), prefix,
+ hostMac, hostVlanId, newLocation.port());
+ });
+
+ });
+ }
+
+ private Set<ResolvedRoute> affectedRoutes(MacAddress mac, VlanId vlanId) {
+ return srManager.routeService.getRouteTables().stream()
+ .map(routeTableId -> srManager.routeService.getRoutes(routeTableId))
+ .flatMap(Collection::stream)
+ .map(RouteInfo::allRoutes)
+ .flatMap(Collection::stream)
+ .filter(resolvedRoute -> mac.equals(resolvedRoute.nextHopMac()) &&
+ vlanId.equals(resolvedRoute.nextHopVlan())).collect(Collectors.toSet());
}
private boolean isReady() {
return Objects.nonNull(srManager.deviceConfiguration) &&
- Objects.nonNull(srManager.defaultRoutingHandler) &&
- Objects.nonNull(srManager.routingRulePopulator);
+ Objects.nonNull(srManager.defaultRoutingHandler);
+ }
+
+ void enqueueRouteEvent(RouteEvent routeEvent) {
+ log.debug("Enqueue routeEvent {}", routeEvent);
+ routeEventCache.put(routeEvent.subject().prefix(), routeEvent);
+ }
+
+ void dequeueRouteEvent(RouteEvent routeEvent) {
+ log.debug("Dequeue routeEvent {}", routeEvent);
+ switch (routeEvent.type()) {
+ case ROUTE_ADDED:
+ processRouteAddedInternal(routeEvent.alternatives());
+ break;
+ case ROUTE_REMOVED:
+ processRouteRemovedInternal(routeEvent.alternatives());
+ break;
+ case ROUTE_UPDATED:
+ case ALTERNATIVE_ROUTES_CHANGED:
+ processRouteUpdatedInternal(Sets.newHashSet(routeEvent.alternatives()),
+ Sets.newHashSet(routeEvent.prevAlternatives()));
+ break;
+ default:
+ break;
+ }
}
}
diff --git a/src/main/java/org/onosproject/segmentrouting/RoutingRulePopulator.java b/src/main/java/org/onosproject/segmentrouting/RoutingRulePopulator.java
index dd11423..3fb38ba 100644
--- a/src/main/java/org/onosproject/segmentrouting/RoutingRulePopulator.java
+++ b/src/main/java/org/onosproject/segmentrouting/RoutingRulePopulator.java
@@ -119,7 +119,7 @@
* @param hostVlanId Vlan ID of the nexthop
* @param outPort port where the next hop attaches to
*/
- public void populateRoute(DeviceId deviceId, IpPrefix prefix,
+ void populateRoute(DeviceId deviceId, IpPrefix prefix,
MacAddress hostMac, VlanId hostVlanId, PortNumber outPort) {
log.debug("Populate direct routing entry for route {} at {}:{}",
prefix, deviceId, outPort);
@@ -136,9 +136,11 @@
+ "to error for dev:{} route:{}", deviceId, prefix);
return;
}
+
+ int nextId = fwdBuilder.add().nextId();
ObjectiveContext context = new DefaultObjectiveContext(
- (objective) -> log.debug("Direct routing rule for route {} populated",
- prefix),
+ (objective) -> log.debug("Direct routing rule for route {} populated. nextId={}",
+ prefix, nextId),
(objective, error) ->
log.warn("Failed to populate direct routing rule for route {}: {}",
prefix, error));
@@ -155,7 +157,7 @@
* @param hostVlanId Vlan ID of the nexthop
* @param outPort port that next hop attaches to
*/
- public void revokeRoute(DeviceId deviceId, IpPrefix prefix,
+ void revokeRoute(DeviceId deviceId, IpPrefix prefix,
MacAddress hostMac, VlanId hostVlanId, PortNumber outPort) {
log.debug("Revoke IP table entry for route {} at {}:{}",
prefix, deviceId, outPort);
diff --git a/src/main/java/org/onosproject/segmentrouting/SegmentRoutingManager.java b/src/main/java/org/onosproject/segmentrouting/SegmentRoutingManager.java
index 6be675a..79bd8d3 100644
--- a/src/main/java/org/onosproject/segmentrouting/SegmentRoutingManager.java
+++ b/src/main/java/org/onosproject/segmentrouting/SegmentRoutingManager.java
@@ -130,6 +130,7 @@
public class SegmentRoutingManager implements SegmentRoutingService {
private static Logger log = LoggerFactory.getLogger(SegmentRoutingManager.class);
+ private static final String NOT_MASTER = "Current instance is not the master of {}. Ignore.";
@Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
private ComponentConfigService compCfgService;
@@ -700,6 +701,20 @@
}
/**
+ * Determine if current instance is the master of given connect point.
+ *
+ * @param cp connect point
+ * @return true if current instance is the master of given connect point
+ */
+ boolean isMasterOf(ConnectPoint cp) {
+ boolean isMaster = mastershipService.isLocalMaster(cp.deviceId());
+ if (!isMaster) {
+ log.debug(NOT_MASTER, cp);
+ }
+ return isMaster;
+ }
+
+ /**
* Returns locations of given resolved route.
*
* @param resolvedRoute resolved route
@@ -1539,6 +1554,7 @@
break;
case HOST_MOVED:
hostHandler.processHostMovedEvent(event);
+ routeHandler.processHostMovedEvent(event);
break;
case HOST_REMOVED:
hostHandler.processHostRemovedEvent(event);
@@ -1587,6 +1603,9 @@
case ROUTE_REMOVED:
routeHandler.processRouteRemoved(event);
break;
+ case ALTERNATIVE_ROUTES_CHANGED:
+ routeHandler.processAlternativeRoutesChanged(event);
+ break;
default:
break;
}
diff --git a/src/test/java/org/onosproject/segmentrouting/HostHandlerTest.java b/src/test/java/org/onosproject/segmentrouting/HostHandlerTest.java
index 3ad2d52..3e576b9 100644
--- a/src/test/java/org/onosproject/segmentrouting/HostHandlerTest.java
+++ b/src/test/java/org/onosproject/segmentrouting/HostHandlerTest.java
@@ -60,6 +60,7 @@
Maps.newConcurrentMap();
private static final Map<MockRoutingTableKey, MockRoutingTableValue> ROUTING_TABLE =
Maps.newConcurrentMap();
+ private static final Map<ConnectPoint, Set<IpPrefix>> SUBNET_TABLE = Maps.newConcurrentMap();
// Mocked Next Id
private static final Map<Integer, TrafficTreatment> NEXT_TABLE = Maps.newConcurrentMap();
@@ -191,6 +192,7 @@
srManager.deviceConfiguration = new DeviceConfiguration(srManager);
srManager.flowObjectiveService = new MockFlowObjectiveService(BRIDGING_TABLE, NEXT_TABLE);
srManager.routingRulePopulator = new MockRoutingRulePopulator(srManager, ROUTING_TABLE);
+ srManager.defaultRoutingHandler = new MockDefaultRoutingHandler(srManager, SUBNET_TABLE);
srManager.interfaceService = new MockInterfaceService(INTERFACES);
srManager.mastershipService = new MockMastershipService(LOCAL_DEVICES);
srManager.hostService = new MockHostService(HOSTS);
diff --git a/src/test/java/org/onosproject/segmentrouting/MockDefaultRoutingHandler.java b/src/test/java/org/onosproject/segmentrouting/MockDefaultRoutingHandler.java
index a69d5e7..ec07238 100644
--- a/src/test/java/org/onosproject/segmentrouting/MockDefaultRoutingHandler.java
+++ b/src/test/java/org/onosproject/segmentrouting/MockDefaultRoutingHandler.java
@@ -36,6 +36,15 @@
@Override
protected void populateSubnet(Set<ConnectPoint> cpts, Set<IpPrefix> subnets) {
+ subnetTable.forEach((k, v) -> {
+ if (!cpts.contains(k)) {
+ subnetTable.get(k).removeAll(subnets);
+ if (subnetTable.get(k).isEmpty()) {
+ subnetTable.remove(k);
+ }
+ }
+ });
+
cpts.forEach(cpt -> subnetTable.put(cpt, subnets));
}
diff --git a/src/test/java/org/onosproject/segmentrouting/MockRouteService.java b/src/test/java/org/onosproject/segmentrouting/MockRouteService.java
index 14e2166..7927636 100644
--- a/src/test/java/org/onosproject/segmentrouting/MockRouteService.java
+++ b/src/test/java/org/onosproject/segmentrouting/MockRouteService.java
@@ -16,29 +16,44 @@
package org.onosproject.segmentrouting;
-import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
+import org.onlab.packet.IpAddress;
+import org.onlab.packet.IpPrefix;
+import org.onlab.packet.MacAddress;
+import org.onlab.packet.VlanId;
+import org.onosproject.routeservice.ResolvedRoute;
+import org.onosproject.routeservice.Route;
import org.onosproject.routeservice.RouteInfo;
import org.onosproject.routeservice.RouteServiceAdapter;
import org.onosproject.routeservice.RouteTableId;
import java.util.Collection;
-import java.util.Set;
+import java.util.Map;
+import java.util.stream.Collectors;
/**
* Mock Route Service.
* We assume there is only one routing table named "default".
*/
public class MockRouteService extends RouteServiceAdapter {
- private Set<RouteInfo> routes;
+ private Map<MockRoutingTableKey, MockRoutingTableValue> routingTable;
- MockRouteService(Set<RouteInfo> routes) {
- this.routes = ImmutableSet.copyOf(routes);
+ MockRouteService(Map<MockRoutingTableKey, MockRoutingTableValue> routingTable) {
+ this.routingTable = routingTable;
}
@Override
public Collection<RouteInfo> getRoutes(RouteTableId id) {
- return routes;
+ return routingTable.entrySet().stream().map(e -> {
+ IpPrefix prefix = e.getKey().ipPrefix;
+ IpAddress nextHop = IpAddress.valueOf(0); // dummy
+ MacAddress mac = e.getValue().macAddress;
+ VlanId vlan = e.getValue().vlanId;
+ Route route = new Route(Route.Source.STATIC, prefix, nextHop);
+ ResolvedRoute rr = new ResolvedRoute(route, mac, vlan);
+
+ return new RouteInfo(prefix, rr, Sets.newHashSet(rr));
+ }).collect(Collectors.toSet());
}
@Override
diff --git a/src/test/java/org/onosproject/segmentrouting/RouteHandlerTest.java b/src/test/java/org/onosproject/segmentrouting/RouteHandlerTest.java
index 383f800..775e5ee 100644
--- a/src/test/java/org/onosproject/segmentrouting/RouteHandlerTest.java
+++ b/src/test/java/org/onosproject/segmentrouting/RouteHandlerTest.java
@@ -16,6 +16,8 @@
package org.onosproject.segmentrouting;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import org.junit.Before;
@@ -24,21 +26,25 @@
import org.onlab.packet.IpPrefix;
import org.onlab.packet.MacAddress;
import org.onlab.packet.VlanId;
+import org.onosproject.net.config.ConfigApplyDelegate;
import org.onosproject.net.ConnectPoint;
import org.onosproject.net.DefaultHost;
import org.onosproject.net.DeviceId;
import org.onosproject.net.Host;
import org.onosproject.net.HostId;
import org.onosproject.net.HostLocation;
+import org.onosproject.net.PortNumber;
import org.onosproject.net.config.NetworkConfigRegistryAdapter;
import org.onosproject.net.flow.TrafficTreatment;
+import org.onosproject.net.host.HostEvent;
+import org.onosproject.net.host.HostService;
import org.onosproject.net.intf.Interface;
import org.onosproject.net.provider.ProviderId;
import org.onosproject.routeservice.ResolvedRoute;
import org.onosproject.routeservice.Route;
import org.onosproject.routeservice.RouteEvent;
-import org.onosproject.routeservice.RouteInfo;
import org.onosproject.segmentrouting.config.DeviceConfiguration;
+import org.onosproject.segmentrouting.config.SegmentRoutingDeviceConfig;
import java.util.Map;
import java.util.Set;
@@ -50,6 +56,7 @@
*/
public class RouteHandlerTest {
private RouteHandler routeHandler;
+ private HostService hostService;
// Mocked routing and bridging tables
private static final Map<MockBridgingTableKey, MockBridgingTableValue> BRIDGING_TABLE =
@@ -61,6 +68,8 @@
private static final Map<Integer, TrafficTreatment> NEXT_TABLE = Maps.newConcurrentMap();
private static final IpPrefix P1 = IpPrefix.valueOf("10.0.0.0/24");
+
+ // Single homed router 1
private static final IpAddress N1 = IpAddress.valueOf("10.0.1.254");
private static final MacAddress M1 = MacAddress.valueOf("00:00:00:00:00:01");
private static final VlanId V1 = VlanId.vlanId((short) 1);
@@ -68,48 +77,61 @@
private static final Route R1 = new Route(Route.Source.STATIC, P1, N1);
private static final ResolvedRoute RR1 = new ResolvedRoute(R1, M1, V1);
+ // Single homed router 2
private static final IpAddress N2 = IpAddress.valueOf("10.0.2.254");
private static final MacAddress M2 = MacAddress.valueOf("00:00:00:00:00:02");
private static final VlanId V2 = VlanId.vlanId((short) 2);
- private static final ConnectPoint CP2 = ConnectPoint.deviceConnectPoint("of:0000000000000001/2");
+ private static final ConnectPoint CP2 = ConnectPoint.deviceConnectPoint("of:0000000000000002/2");
private static final Route R2 = new Route(Route.Source.STATIC, P1, N2);
private static final ResolvedRoute RR2 = new ResolvedRoute(R2, M2, V2);
- private static final RouteInfo RI1 = new RouteInfo(P1, RR1, Sets.newHashSet(RR1));
+ // Dual homed router 1
+ private static final IpAddress N3 = IpAddress.valueOf("10.0.3.254");
+ private static final MacAddress M3 = MacAddress.valueOf("00:00:00:00:00:03");
+ private static final VlanId V3 = VlanId.vlanId((short) 3);
+ private static final Route R3 = new Route(Route.Source.STATIC, P1, N3);
+ private static final ResolvedRoute RR3 = new ResolvedRoute(R3, M3, V3);
+ // Hosts
private static final Host H1 = new DefaultHost(ProviderId.NONE, HostId.hostId(M1, V1), M1, V1,
Sets.newHashSet(new HostLocation(CP1, 0)), Sets.newHashSet(N1), false);
private static final Host H2 = new DefaultHost(ProviderId.NONE, HostId.hostId(M2, V2), M2, V2,
Sets.newHashSet(new HostLocation(CP2, 0)), Sets.newHashSet(N2), false);
+ private static final Host H3D = new DefaultHost(ProviderId.NONE, HostId.hostId(M3, V3), M3, V3,
+ Sets.newHashSet(new HostLocation(CP1, 0), new HostLocation(CP2, 0)), Sets.newHashSet(N3), false);
+ private static final Host H3S = new DefaultHost(ProviderId.NONE, HostId.hostId(M3, V3), M3, V3,
+ Sets.newHashSet(new HostLocation(CP1, 0)), Sets.newHashSet(N3), false);
+
+ // Pair Local Port
+ private static final PortNumber P9 = PortNumber.portNumber(9);
// A set of hosts
- private static final Set<Host> HOSTS = Sets.newHashSet(H1, H2);
+ private static final Set<Host> HOSTS = Sets.newHashSet(H1, H2, H3D);
+ private static final Set<Host> HOSTS_ONE_FAIL = Sets.newHashSet(H1, H2, H3S);
+ private static final Set<Host> HOSTS_BOTH_FAIL = Sets.newHashSet(H1, H2);
// A set of devices of which we have mastership
- private static final Set<DeviceId> LOCAL_DEVICES = Sets.newHashSet();
+ private static final Set<DeviceId> LOCAL_DEVICES = Sets.newHashSet(CP1.deviceId(), CP2.deviceId());
// A set of interfaces
private static final Set<Interface> INTERFACES = Sets.newHashSet();
- // A set of routes
- private static final Set<RouteInfo> ROUTE_INFOS = Sets.newHashSet(RI1);
@Before
public void setUp() throws Exception {
-// TODO Initialize pairDevice and pairLocalPort config
-// ObjectMapper mapper = new ObjectMapper();
-// ConfigApplyDelegate delegate = config -> {};
-//
-// SegmentRoutingDeviceConfig dev3Config = new SegmentRoutingDeviceConfig();
-// JsonNode dev3Tree = mapper.createObjectNode();
-// dev3Config.init(DEV3, "host-handler-test", dev3Tree, mapper, delegate);
-// dev3Config.setPairDeviceId(DEV4).setPairLocalPort(P9);
-//
-// SegmentRoutingDeviceConfig dev4Config = new SegmentRoutingDeviceConfig();
-// JsonNode dev4Tree = mapper.createObjectNode();
-// dev4Config.init(DEV4, "host-handler-test", dev4Tree, mapper, delegate);
-// dev4Config.setPairDeviceId(DEV3).setPairLocalPort(P9);
+ ObjectMapper mapper = new ObjectMapper();
+ ConfigApplyDelegate delegate = config -> { };
+
+ SegmentRoutingDeviceConfig dev1Config = new SegmentRoutingDeviceConfig();
+ JsonNode dev1Tree = mapper.createObjectNode();
+ dev1Config.init(CP1.deviceId(), "host-handler-test", dev1Tree, mapper, delegate);
+ dev1Config.setPairDeviceId(CP2.deviceId()).setPairLocalPort(P9);
+
+ SegmentRoutingDeviceConfig dev2Config = new SegmentRoutingDeviceConfig();
+ JsonNode dev2Tree = mapper.createObjectNode();
+ dev2Config.init(CP2.deviceId(), "host-handler-test", dev2Tree, mapper, delegate);
+ dev2Config.setPairDeviceId(CP1.deviceId()).setPairLocalPort(P9);
MockNetworkConfigRegistry mockNetworkConfigRegistry = new MockNetworkConfigRegistry();
-// mockNetworkConfigRegistry.applyConfig(dev3Config);
-// mockNetworkConfigRegistry.applyConfig(dev4Config);
+ mockNetworkConfigRegistry.applyConfig(dev1Config);
+ mockNetworkConfigRegistry.applyConfig(dev2Config);
// Initialize Segment Routing Manager
SegmentRoutingManager srManager = new MockSegmentRoutingManager(NEXT_TABLE);
@@ -120,11 +142,18 @@
srManager.defaultRoutingHandler = new MockDefaultRoutingHandler(srManager, SUBNET_TABLE);
srManager.interfaceService = new MockInterfaceService(INTERFACES);
srManager.mastershipService = new MockMastershipService(LOCAL_DEVICES);
- srManager.hostService = new MockHostService(HOSTS);
+ hostService = new MockHostService(HOSTS);
+ srManager.hostService = hostService;
srManager.cfgService = mockNetworkConfigRegistry;
- srManager.routeService = new MockRouteService(ROUTE_INFOS);
+ srManager.routeService = new MockRouteService(ROUTING_TABLE);
- routeHandler = new RouteHandler(srManager);
+ routeHandler = new RouteHandler(srManager) {
+ // routeEventCache is not necessary for unit tests
+ @Override
+ void enqueueRouteEvent(RouteEvent routeEvent) {
+ dequeueRouteEvent(routeEvent);
+ }
+ };
ROUTING_TABLE.clear();
BRIDGING_TABLE.clear();
@@ -133,6 +162,10 @@
@Test
public void init() throws Exception {
+ MockRoutingTableKey rtk = new MockRoutingTableKey(CP1.deviceId(), P1);
+ MockRoutingTableValue rtv = new MockRoutingTableValue(CP1.port(), M1, V1);
+ ROUTING_TABLE.put(rtk, rtv);
+
routeHandler.init(CP1.deviceId());
assertEquals(1, ROUTING_TABLE.size());
@@ -147,7 +180,7 @@
@Test
public void processRouteAdded() throws Exception {
- RouteEvent re = new RouteEvent(RouteEvent.Type.ROUTE_ADDED, RR1);
+ RouteEvent re = new RouteEvent(RouteEvent.Type.ROUTE_ADDED, RR1, Sets.newHashSet(RR1));
routeHandler.processRouteAdded(re);
assertEquals(1, ROUTING_TABLE.size());
@@ -164,11 +197,12 @@
public void processRouteUpdated() throws Exception {
processRouteAdded();
- RouteEvent re = new RouteEvent(RouteEvent.Type.ROUTE_UPDATED, RR2, RR1);
+ RouteEvent re = new RouteEvent(RouteEvent.Type.ROUTE_UPDATED, RR2, RR1, Sets.newHashSet(RR2),
+ Sets.newHashSet(RR1));
routeHandler.processRouteUpdated(re);
assertEquals(1, ROUTING_TABLE.size());
- MockRoutingTableValue rtv2 = ROUTING_TABLE.get(new MockRoutingTableKey(CP1.deviceId(), P1));
+ MockRoutingTableValue rtv2 = ROUTING_TABLE.get(new MockRoutingTableKey(CP2.deviceId(), P1));
assertEquals(M2, rtv2.macAddress);
assertEquals(V2, rtv2.vlanId);
assertEquals(CP2.port(), rtv2.portNumber);
@@ -181,7 +215,148 @@
public void processRouteRemoved() throws Exception {
processRouteAdded();
- RouteEvent re = new RouteEvent(RouteEvent.Type.ROUTE_REMOVED, RR1);
+ RouteEvent re = new RouteEvent(RouteEvent.Type.ROUTE_REMOVED, RR1, Sets.newHashSet(RR1));
+ routeHandler.processRouteRemoved(re);
+
+ assertEquals(0, ROUTING_TABLE.size());
+ assertEquals(0, SUBNET_TABLE.size());
+ }
+
+ @Test
+ public void testTwoSingleHomedAdded() throws Exception {
+ RouteEvent re = new RouteEvent(RouteEvent.Type.ROUTE_ADDED, RR1, Sets.newHashSet(RR1, RR2));
+ routeHandler.processRouteAdded(re);
+
+ assertEquals(2, ROUTING_TABLE.size());
+ MockRoutingTableValue rtv1 = ROUTING_TABLE.get(new MockRoutingTableKey(CP1.deviceId(), P1));
+ MockRoutingTableValue rtv2 = ROUTING_TABLE.get(new MockRoutingTableKey(CP2.deviceId(), P1));
+ assertEquals(M1, rtv1.macAddress);
+ assertEquals(M2, rtv2.macAddress);
+ assertEquals(V1, rtv1.vlanId);
+ assertEquals(V2, rtv2.vlanId);
+ assertEquals(CP1.port(), rtv1.portNumber);
+ assertEquals(CP2.port(), rtv2.portNumber);
+
+ assertEquals(2, SUBNET_TABLE.size());
+ assertTrue(SUBNET_TABLE.get(CP1).contains(P1));
+ assertTrue(SUBNET_TABLE.get(CP2).contains(P1));
+ }
+
+ @Test
+ public void testOneDualHomedAdded() throws Exception {
+ RouteEvent re = new RouteEvent(RouteEvent.Type.ROUTE_ADDED, RR3, Sets.newHashSet(RR3));
+ routeHandler.processRouteAdded(re);
+
+ assertEquals(2, ROUTING_TABLE.size());
+ MockRoutingTableValue rtv1 = ROUTING_TABLE.get(new MockRoutingTableKey(CP1.deviceId(), P1));
+ MockRoutingTableValue rtv2 = ROUTING_TABLE.get(new MockRoutingTableKey(CP2.deviceId(), P1));
+ assertEquals(M3, rtv1.macAddress);
+ assertEquals(M3, rtv2.macAddress);
+ assertEquals(V3, rtv1.vlanId);
+ assertEquals(V3, rtv2.vlanId);
+ assertEquals(CP1.port(), rtv1.portNumber);
+ assertEquals(CP2.port(), rtv2.portNumber);
+
+ assertEquals(2, SUBNET_TABLE.size());
+ assertTrue(SUBNET_TABLE.get(CP1).contains(P1));
+ assertTrue(SUBNET_TABLE.get(CP2).contains(P1));
+ }
+
+ @Test
+ public void testOneSingleHomedToTwoSingleHomed() throws Exception {
+ processRouteAdded();
+
+ RouteEvent re = new RouteEvent(RouteEvent.Type.ALTERNATIVE_ROUTES_CHANGED, RR1, null,
+ Sets.newHashSet(RR1, RR2), Sets.newHashSet(RR1));
+ routeHandler.processAlternativeRoutesChanged(re);
+
+ assertEquals(2, ROUTING_TABLE.size());
+ MockRoutingTableValue rtv1 = ROUTING_TABLE.get(new MockRoutingTableKey(CP1.deviceId(), P1));
+ MockRoutingTableValue rtv2 = ROUTING_TABLE.get(new MockRoutingTableKey(CP2.deviceId(), P1));
+ assertEquals(M1, rtv1.macAddress);
+ assertEquals(M2, rtv2.macAddress);
+ assertEquals(V1, rtv1.vlanId);
+ assertEquals(V2, rtv2.vlanId);
+ assertEquals(CP1.port(), rtv1.portNumber);
+ assertEquals(CP2.port(), rtv2.portNumber);
+
+ assertEquals(2, SUBNET_TABLE.size());
+ assertTrue(SUBNET_TABLE.get(CP1).contains(P1));
+ assertTrue(SUBNET_TABLE.get(CP2).contains(P1));
+ }
+
+ @Test
+ public void testTwoSingleHomedToOneSingleHomed() throws Exception {
+ testTwoSingleHomedAdded();
+
+ RouteEvent re = new RouteEvent(RouteEvent.Type.ALTERNATIVE_ROUTES_CHANGED, RR1, null,
+ Sets.newHashSet(RR1), Sets.newHashSet(RR1, RR2));
+ routeHandler.processAlternativeRoutesChanged(re);
+
+ assertEquals(1, ROUTING_TABLE.size());
+ MockRoutingTableValue rtv1 = ROUTING_TABLE.get(new MockRoutingTableKey(CP1.deviceId(), P1));
+ assertEquals(M1, rtv1.macAddress);
+ assertEquals(V1, rtv1.vlanId);
+ assertEquals(CP1.port(), rtv1.portNumber);
+
+ assertEquals(1, SUBNET_TABLE.size());
+ assertTrue(SUBNET_TABLE.get(CP1).contains(P1));
+ }
+
+ @Test
+ public void testDualHomedSingleLocationFail() throws Exception {
+ testOneDualHomedAdded();
+
+ HostEvent he = new HostEvent(HostEvent.Type.HOST_MOVED, H3S, H3D);
+ routeHandler.processHostMovedEvent(he);
+
+ assertEquals(2, ROUTING_TABLE.size());
+ MockRoutingTableValue rtv1 = ROUTING_TABLE.get(new MockRoutingTableKey(CP1.deviceId(), P1));
+ MockRoutingTableValue rtv2 = ROUTING_TABLE.get(new MockRoutingTableKey(CP2.deviceId(), P1));
+ assertEquals(M3, rtv1.macAddress);
+ assertEquals(M3, rtv2.macAddress);
+ assertEquals(V3, rtv1.vlanId);
+ assertEquals(V3, rtv2.vlanId);
+ assertEquals(CP1.port(), rtv1.portNumber);
+ assertEquals(P9, rtv2.portNumber);
+
+ // ECMP route table hasn't changed
+ assertEquals(2, SUBNET_TABLE.size());
+ assertTrue(SUBNET_TABLE.get(CP1).contains(P1));
+ assertTrue(SUBNET_TABLE.get(CP2).contains(P1));
+ }
+
+ @Test
+ public void testDualHomedBothLocationFail() throws Exception {
+ testDualHomedSingleLocationFail();
+
+ hostService = new MockHostService(HOSTS_ONE_FAIL);
+
+ RouteEvent re = new RouteEvent(RouteEvent.Type.ROUTE_REMOVED, RR3, Sets.newHashSet(RR3));
+ routeHandler.processRouteRemoved(re);
+
+ assertEquals(0, ROUTING_TABLE.size());
+ assertEquals(0, SUBNET_TABLE.size());
+ }
+
+ @Test
+ public void testTwoSingleHomedRemoved() throws Exception {
+ testTwoSingleHomedAdded();
+
+ hostService = new MockHostService(HOSTS_BOTH_FAIL);
+
+ RouteEvent re = new RouteEvent(RouteEvent.Type.ROUTE_REMOVED, RR1, Sets.newHashSet(RR1, RR2));
+ routeHandler.processRouteRemoved(re);
+
+ assertEquals(0, ROUTING_TABLE.size());
+ assertEquals(0, SUBNET_TABLE.size());
+ }
+
+ @Test
+ public void testOneDualHomeRemoved() throws Exception {
+ testOneDualHomedAdded();
+
+ RouteEvent re = new RouteEvent(RouteEvent.Type.ROUTE_REMOVED, RR3, Sets.newHashSet(RR3));
routeHandler.processRouteRemoved(re);
assertEquals(0, ROUTING_TABLE.size());