blob: b323c17351cdae1da5ae597779051fc370f356cb [file] [log] [blame]
Charles Chand55e84d2016-03-30 17:54:24 -07001/*
Pier Luigi96fe0772018-02-28 12:10:50 +01002 * Copyright 2018-present Open Networking Foundation
Charles Chand55e84d2016-03-30 17:54:24 -07003 *
4 * Licensed under the Apache License, Version 2.0 (the "License");
5 * you may not use this file except in compliance with the License.
6 * You may obtain a copy of the License at
7 *
8 * http://www.apache.org/licenses/LICENSE-2.0
9 *
10 * Unless required by applicable law or agreed to in writing, software
11 * distributed under the License is distributed on an "AS IS" BASIS,
12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 * See the License for the specific language governing permissions and
14 * limitations under the License.
15 */
16
Pier Luigi96fe0772018-02-28 12:10:50 +010017package org.onosproject.segmentrouting.mcast;
Charles Chand55e84d2016-03-30 17:54:24 -070018
Pier Luigi05514fd2018-02-28 17:24:03 +010019import com.google.common.cache.Cache;
20import com.google.common.cache.CacheBuilder;
21import com.google.common.cache.RemovalCause;
22import com.google.common.cache.RemovalNotification;
Charles Chand55e84d2016-03-30 17:54:24 -070023import com.google.common.collect.ImmutableSet;
24import com.google.common.collect.Lists;
Pier Luigibad6d6c2018-01-23 16:06:38 +010025import com.google.common.collect.Maps;
Charles Chand55e84d2016-03-30 17:54:24 -070026import com.google.common.collect.Sets;
27import org.onlab.packet.Ethernet;
28import org.onlab.packet.IpAddress;
29import org.onlab.packet.IpPrefix;
30import org.onlab.packet.MacAddress;
31import org.onlab.packet.VlanId;
32import org.onlab.util.KryoNamespace;
Pier Luigieba73a02018-01-16 10:47:50 +010033import org.onosproject.cluster.NodeId;
Charles Chand55e84d2016-03-30 17:54:24 -070034import org.onosproject.core.ApplicationId;
35import org.onosproject.core.CoreService;
Ray Milkey6c1f0f02017-08-15 11:02:29 -070036import org.onosproject.net.config.basics.McastConfig;
Charles Chand55e84d2016-03-30 17:54:24 -070037import org.onosproject.net.ConnectPoint;
38import org.onosproject.net.DeviceId;
39import org.onosproject.net.Link;
40import org.onosproject.net.Path;
41import org.onosproject.net.PortNumber;
42import org.onosproject.net.flow.DefaultTrafficSelector;
43import org.onosproject.net.flow.DefaultTrafficTreatment;
44import org.onosproject.net.flow.TrafficSelector;
45import org.onosproject.net.flow.TrafficTreatment;
46import org.onosproject.net.flow.criteria.Criteria;
Pier Luigi05514fd2018-02-28 17:24:03 +010047import org.onosproject.net.flow.criteria.VlanIdCriterion;
Charles Chand55e84d2016-03-30 17:54:24 -070048import org.onosproject.net.flow.instructions.Instructions.OutputInstruction;
49import org.onosproject.net.flowobjective.DefaultFilteringObjective;
50import org.onosproject.net.flowobjective.DefaultForwardingObjective;
51import org.onosproject.net.flowobjective.DefaultNextObjective;
Charles Chan2199c302016-04-23 17:36:10 -070052import org.onosproject.net.flowobjective.DefaultObjectiveContext;
Charles Chand55e84d2016-03-30 17:54:24 -070053import org.onosproject.net.flowobjective.FilteringObjective;
54import org.onosproject.net.flowobjective.ForwardingObjective;
55import org.onosproject.net.flowobjective.NextObjective;
Charles Chan2199c302016-04-23 17:36:10 -070056import org.onosproject.net.flowobjective.ObjectiveContext;
Charles Chand55e84d2016-03-30 17:54:24 -070057import org.onosproject.net.mcast.McastEvent;
Pier Luigib72201b2018-01-25 16:16:02 +010058import org.onosproject.net.mcast.McastRoute;
Charles Chand55e84d2016-03-30 17:54:24 -070059import org.onosproject.net.mcast.McastRouteInfo;
Pier Luigi83f919b2018-02-15 16:33:08 +010060import org.onosproject.net.topology.Topology;
Charles Chand55e84d2016-03-30 17:54:24 -070061import org.onosproject.net.topology.TopologyService;
Pier Luigi96fe0772018-02-28 12:10:50 +010062import org.onosproject.segmentrouting.SegmentRoutingManager;
63import org.onosproject.segmentrouting.SegmentRoutingService;
Pier Luigif7049c52018-02-23 19:57:40 +010064import org.onosproject.segmentrouting.config.DeviceConfigNotFoundException;
Charles Chan6ea94fc2016-05-10 17:29:47 -070065import org.onosproject.segmentrouting.config.SegmentRoutingAppConfig;
Charles Chan2199c302016-04-23 17:36:10 -070066import org.onosproject.segmentrouting.storekey.McastStoreKey;
Charles Chand55e84d2016-03-30 17:54:24 -070067import org.onosproject.store.serializers.KryoNamespaces;
68import org.onosproject.store.service.ConsistentMap;
69import org.onosproject.store.service.Serializer;
70import org.onosproject.store.service.StorageService;
Pier Luigieba73a02018-01-16 10:47:50 +010071import org.onosproject.store.service.Versioned;
Charles Chand55e84d2016-03-30 17:54:24 -070072import org.slf4j.Logger;
73import org.slf4j.LoggerFactory;
74
Pier Luigib72201b2018-01-25 16:16:02 +010075import java.time.Instant;
Charles Chand55e84d2016-03-30 17:54:24 -070076import java.util.Collection;
77import java.util.Collections;
Pier Luigibad6d6c2018-01-23 16:06:38 +010078import java.util.Comparator;
Charles Chand55e84d2016-03-30 17:54:24 -070079import java.util.List;
Charles Chan2199c302016-04-23 17:36:10 -070080import java.util.Map;
Charles Chand55e84d2016-03-30 17:54:24 -070081import java.util.Optional;
82import java.util.Set;
Pier Luigib72201b2018-01-25 16:16:02 +010083import java.util.concurrent.ScheduledExecutorService;
84import java.util.concurrent.TimeUnit;
85import java.util.concurrent.locks.Lock;
86import java.util.concurrent.locks.ReentrantLock;
Charles Chan2199c302016-04-23 17:36:10 -070087import java.util.stream.Collectors;
88
89import static com.google.common.base.Preconditions.checkState;
Pier Luigib72201b2018-01-25 16:16:02 +010090import static java.util.concurrent.Executors.newScheduledThreadPool;
91import static org.onlab.util.Tools.groupedThreads;
Pier Luigi05514fd2018-02-28 17:24:03 +010092import static org.onosproject.net.flow.criteria.Criterion.Type.VLAN_VID;
93import static org.onosproject.net.mcast.McastEvent.Type.ROUTE_REMOVED;
94import static org.onosproject.net.mcast.McastEvent.Type.SOURCE_ADDED;
95import static org.onosproject.net.mcast.McastEvent.Type.SOURCE_UPDATED;
Charles Chan59cc16d2017-02-02 16:20:42 -080096import static org.onosproject.segmentrouting.SegmentRoutingManager.INTERNAL_VLAN;
Charles Chand55e84d2016-03-30 17:54:24 -070097
98/**
Pier Luigi96fe0772018-02-28 12:10:50 +010099 * Handles Multicast related events.
Charles Chand55e84d2016-03-30 17:54:24 -0700100 */
Charles Chand2990362016-04-18 13:44:03 -0700101public class McastHandler {
102 private static final Logger log = LoggerFactory.getLogger(McastHandler.class);
Charles Chand55e84d2016-03-30 17:54:24 -0700103 private final SegmentRoutingManager srManager;
104 private final ApplicationId coreAppId;
Charles Chanfc5c7802016-05-17 13:13:55 -0700105 private final StorageService storageService;
106 private final TopologyService topologyService;
Charles Chan2199c302016-04-23 17:36:10 -0700107 private final ConsistentMap<McastStoreKey, NextObjective> mcastNextObjStore;
108 private final KryoNamespace.Builder mcastKryo;
109 private final ConsistentMap<McastStoreKey, McastRole> mcastRoleStore;
110
Pier Luigi05514fd2018-02-28 17:24:03 +0100111 // Wait time for the cache
112 private static final int WAIT_TIME_MS = 1000;
113 /**
114 * The mcastEventCache is implemented to avoid race condition by giving more time to the
115 * underlying subsystems to process previous calls.
116 */
117 private Cache<McastCacheKey, McastEvent> mcastEventCache = CacheBuilder.newBuilder()
118 .expireAfterWrite(WAIT_TIME_MS, TimeUnit.MILLISECONDS)
119 .removalListener((RemovalNotification<McastCacheKey, McastEvent> notification) -> {
120 // Get group ip, sink and related event
121 IpAddress mcastIp = notification.getKey().mcastIp();
122 ConnectPoint sink = notification.getKey().sink();
123 McastEvent mcastEvent = notification.getValue();
124 RemovalCause cause = notification.getCause();
125 log.debug("mcastEventCache removal event. group={}, sink={}, mcastEvent={}, cause={}",
126 mcastIp, sink, mcastEvent, cause);
127 // If it expires or it has been replaced, we deque the event
128 switch (notification.getCause()) {
129 case REPLACED:
130 case EXPIRED:
131 dequeueMcastEvent(mcastEvent);
132 break;
133 default:
134 break;
135 }
136 }).build();
137
138 private void enqueueMcastEvent(McastEvent mcastEvent) {
139 log.debug("Enqueue mcastEvent {}", mcastEvent);
140 final McastRouteInfo mcastRouteInfo = mcastEvent.subject();
141 // Let's create the keys of the cache
142 ImmutableSet.Builder<ConnectPoint> sinksBuilder = ImmutableSet.builder();
143 // For this event we will have a set of sinks
144 if (mcastEvent.type() == SOURCE_ADDED ||
145 mcastEvent.type() == SOURCE_UPDATED ||
146 mcastEvent.type() == ROUTE_REMOVED) {
147 // Add all the sinks
148 sinksBuilder.addAll(mcastRouteInfo.sinks());
149 } else {
150 // We have just one sink in this case
151 ConnectPoint sink = mcastRouteInfo.sink().orElse(null);
152 // It is always true, unless something of bad happened
153 // in the mcast route store
154 if (sink != null) {
155 sinksBuilder.add(sink);
156 }
157 }
158 // Push the elements in the cache
159 sinksBuilder.build().forEach(sink -> {
160 McastCacheKey cacheKey = new McastCacheKey(mcastRouteInfo.route().group(),
161 sink);
162 mcastEventCache.put(cacheKey, mcastEvent);
163 });
164 }
165
166 private void dequeueMcastEvent(McastEvent mcastEvent) {
167 log.debug("Dequeue mcastEvent {}", mcastEvent);
168 final McastRouteInfo mcastRouteInfo = mcastEvent.subject();
169 // Get source, mcast group
170 ConnectPoint source = mcastRouteInfo.source().orElse(null);
171 IpAddress mcastIp = mcastRouteInfo.route().group();
172 // According to the event type let's call the proper method
173 switch (mcastEvent.type()) {
174 case SOURCE_ADDED:
175 // Get all the sinks and process
176 Set<ConnectPoint> sinks = mcastRouteInfo.sinks();
177 sinks.forEach(sink -> processSinkAddedInternal(source, sink, mcastIp));
178 break;
179 case SOURCE_UPDATED:
180 // Get old source
181 ConnectPoint oldSource = mcastEvent.prevSubject().source().orElse(null);
182 // Just the first cached element will be processed
183 processSourceUpdatedInternal(mcastIp, source, oldSource);
184 break;
185 case ROUTE_REMOVED:
186 // Process the route removed, just the first cached element will be processed
187 processRouteRemovedInternal(source, mcastIp);
188 break;
189 case SINK_ADDED:
190 // Get the only sink and process
191 ConnectPoint sink = mcastRouteInfo.sink().orElse(null);
192 processSinkAddedInternal(source, sink, mcastIp);
193 break;
194 case SINK_REMOVED:
195 sink = mcastRouteInfo.sink().orElse(null);
196 processSinkRemovedInternal(source, sink, mcastIp);
197 break;
198 default:
199 break;
200 }
201 }
202
Pier Luigib72201b2018-01-25 16:16:02 +0100203 // Mcast lock to serialize local operations
204 private final Lock mcastLock = new ReentrantLock();
205
206 /**
207 * Acquires the lock used when making mcast changes.
208 */
209 private void mcastLock() {
210 mcastLock.lock();
211 }
212
213 /**
214 * Releases the lock used when making mcast changes.
215 */
216 private void mcastUnlock() {
217 mcastLock.unlock();
218 }
219
220 // Stability threshold for Mcast. Seconds
221 private static final long MCAST_STABLITY_THRESHOLD = 5;
222 // Last change done
223 private Instant lastMcastChange = Instant.now();
224
225 /**
226 * Determines if mcast in the network has been stable in the last
227 * MCAST_STABLITY_THRESHOLD seconds, by comparing the current time
228 * to the last mcast change timestamp.
229 *
230 * @return true if stable
231 */
232 private boolean isMcastStable() {
233 long last = (long) (lastMcastChange.toEpochMilli() / 1000.0);
234 long now = (long) (Instant.now().toEpochMilli() / 1000.0);
Saurav Dasa4020382018-02-14 14:14:54 -0800235 log.trace("Mcast stable since {}s", now - last);
Pier Luigib72201b2018-01-25 16:16:02 +0100236 return (now - last) > MCAST_STABLITY_THRESHOLD;
237 }
238
239 // Verify interval for Mcast
240 private static final long MCAST_VERIFY_INTERVAL = 30;
241
242 // Executor for mcast bucket corrector
243 private ScheduledExecutorService executorService
Pier Luigi05514fd2018-02-28 17:24:03 +0100244 = newScheduledThreadPool(1, groupedThreads("mcastWorker", "mcastWorker-%d", log));
Pier Luigib72201b2018-01-25 16:16:02 +0100245
Charles Chan2199c302016-04-23 17:36:10 -0700246 /**
Charles Chand55e84d2016-03-30 17:54:24 -0700247 * Constructs the McastEventHandler.
248 *
249 * @param srManager Segment Routing manager
250 */
Charles Chand2990362016-04-18 13:44:03 -0700251 public McastHandler(SegmentRoutingManager srManager) {
Charles Chand55e84d2016-03-30 17:54:24 -0700252 coreAppId = srManager.coreService.getAppId(CoreService.CORE_APP_NAME);
Charles Chand55e84d2016-03-30 17:54:24 -0700253 this.srManager = srManager;
254 this.storageService = srManager.storageService;
255 this.topologyService = srManager.topologyService;
Charles Chan2199c302016-04-23 17:36:10 -0700256 mcastKryo = new KryoNamespace.Builder()
Charles Chand55e84d2016-03-30 17:54:24 -0700257 .register(KryoNamespaces.API)
Charles Chan2199c302016-04-23 17:36:10 -0700258 .register(McastStoreKey.class)
259 .register(McastRole.class);
Charles Chand55e84d2016-03-30 17:54:24 -0700260 mcastNextObjStore = storageService
Charles Chan2199c302016-04-23 17:36:10 -0700261 .<McastStoreKey, NextObjective>consistentMapBuilder()
Charles Chand55e84d2016-03-30 17:54:24 -0700262 .withName("onos-mcast-nextobj-store")
Charles Chaneefdedf2016-05-23 16:45:45 -0700263 .withSerializer(Serializer.using(mcastKryo.build("McastHandler-NextObj")))
Charles Chand55e84d2016-03-30 17:54:24 -0700264 .build();
Charles Chan2199c302016-04-23 17:36:10 -0700265 mcastRoleStore = storageService
266 .<McastStoreKey, McastRole>consistentMapBuilder()
267 .withName("onos-mcast-role-store")
Charles Chaneefdedf2016-05-23 16:45:45 -0700268 .withSerializer(Serializer.using(mcastKryo.build("McastHandler-Role")))
Charles Chan2199c302016-04-23 17:36:10 -0700269 .build();
Pier Luigib72201b2018-01-25 16:16:02 +0100270 // Init the executor service and the buckets corrector
271 executorService.scheduleWithFixedDelay(new McastBucketCorrector(), 10,
272 MCAST_VERIFY_INTERVAL,
273 TimeUnit.SECONDS);
Pier Luigi05514fd2018-02-28 17:24:03 +0100274 // Schedule the clean up, this will allow the processing of the expired events
275 executorService.scheduleAtFixedRate(mcastEventCache::cleanUp, 0,
276 WAIT_TIME_MS, TimeUnit.MILLISECONDS);
Charles Chan2199c302016-04-23 17:36:10 -0700277 }
278
279 /**
280 * Read initial multicast from mcast store.
281 */
Pier Luigi96fe0772018-02-28 12:10:50 +0100282 public void init() {
Charles Chan2199c302016-04-23 17:36:10 -0700283 srManager.multicastRouteService.getRoutes().forEach(mcastRoute -> {
284 ConnectPoint source = srManager.multicastRouteService.fetchSource(mcastRoute);
285 Set<ConnectPoint> sinks = srManager.multicastRouteService.fetchSinks(mcastRoute);
286 sinks.forEach(sink -> {
287 processSinkAddedInternal(source, sink, mcastRoute.group());
288 });
289 });
Charles Chand55e84d2016-03-30 17:54:24 -0700290 }
291
292 /**
Pier Luigib72201b2018-01-25 16:16:02 +0100293 * Clean up when deactivating the application.
294 */
Pier Luigi96fe0772018-02-28 12:10:50 +0100295 public void terminate() {
Pier Luigib72201b2018-01-25 16:16:02 +0100296 executorService.shutdown();
297 }
298
299 /**
Pier Luigi05514fd2018-02-28 17:24:03 +0100300 * Processes the SOURCE_ADDED, SOURCE_UPDATED, SINK_ADDED,
301 * SINK_REMOVED and ROUTE_REMOVED events.
Charles Chand55e84d2016-03-30 17:54:24 -0700302 *
303 * @param event McastEvent with SOURCE_ADDED type
304 */
Pier Luigi05514fd2018-02-28 17:24:03 +0100305 public void processMcastEvent(McastEvent event) {
306 log.info("process {}", event);
307 // Verify if it is a complete event
Charles Chand55e84d2016-03-30 17:54:24 -0700308 McastRouteInfo mcastRouteInfo = event.subject();
309 if (!mcastRouteInfo.isComplete()) {
Pier Luigi05514fd2018-02-28 17:24:03 +0100310 log.info("Incompleted McastRouteInfo. Abort {}", event.type());
Charles Chand55e84d2016-03-30 17:54:24 -0700311 return;
312 }
Pier Luigi05514fd2018-02-28 17:24:03 +0100313 // Just enqueue for now
314 enqueueMcastEvent(event);
Pier Luigi9930da52018-02-02 16:19:11 +0100315 }
316
317 /**
Pier Luigi57d41792018-02-26 12:31:38 +0100318 * Process the SOURCE_UPDATED event.
319 *
320 * @param newSource the updated srouce info
321 * @param oldSource the outdated source info
322 */
323 private void processSourceUpdatedInternal(IpAddress mcastIp,
324 ConnectPoint newSource,
325 ConnectPoint oldSource) {
326 lastMcastChange = Instant.now();
327 mcastLock();
328 try {
329 log.debug("Processing source updated for group {}", mcastIp);
330
331 // Build key for the store and retrieve old data
332 McastStoreKey mcastStoreKey = new McastStoreKey(mcastIp, oldSource.deviceId());
333
334 // Verify leadership on the operation
335 if (!isLeader(oldSource)) {
336 log.debug("Skip {} due to lack of leadership", mcastIp);
337 return;
338 }
339
340 // This device is not serving this multicast group
341 if (!mcastRoleStore.containsKey(mcastStoreKey) ||
342 !mcastNextObjStore.containsKey(mcastStoreKey)) {
343 log.warn("{} is not serving {}. Abort.", oldSource.deviceId(), mcastIp);
344 return;
345 }
346 NextObjective nextObjective = mcastNextObjStore.get(mcastStoreKey).value();
347 Set<PortNumber> outputPorts = getPorts(nextObjective.next());
348
Pier Luigi05514fd2018-02-28 17:24:03 +0100349 // This an optimization to avoid unnecessary removal and add
350 if (!assignedVlanFromNext(nextObjective).equals(assignedVlan(newSource))) {
351 // Let's remove old flows and groups
352 removeGroupFromDevice(oldSource.deviceId(), mcastIp, assignedVlan(oldSource));
353 // Push new flows and group
354 outputPorts.forEach(portNumber -> addPortToDevice(newSource.deviceId(), portNumber,
355 mcastIp, assignedVlan(newSource)));
356 }
Pier Luigi57d41792018-02-26 12:31:38 +0100357 addFilterToDevice(newSource.deviceId(), newSource.port(),
358 assignedVlan(newSource), mcastIp);
359 // Setup mcast roles
360 mcastRoleStore.put(new McastStoreKey(mcastIp, newSource.deviceId()),
361 McastRole.INGRESS);
362 } finally {
363 mcastUnlock();
364 }
365 }
366
367 /**
Pier Luigi9930da52018-02-02 16:19:11 +0100368 * Removes the entire mcast tree related to this group.
369 *
370 * @param mcastIp multicast group IP address
371 */
372 private void processRouteRemovedInternal(ConnectPoint source, IpAddress mcastIp) {
373 lastMcastChange = Instant.now();
374 mcastLock();
375 try {
Pier Luigi57d41792018-02-26 12:31:38 +0100376 log.debug("Processing route removed for group {}", mcastIp);
Pier Luigi9930da52018-02-02 16:19:11 +0100377
378 // Find out the ingress, transit and egress device of the affected group
379 DeviceId ingressDevice = getDevice(mcastIp, McastRole.INGRESS)
380 .stream().findAny().orElse(null);
381 DeviceId transitDevice = getDevice(mcastIp, McastRole.TRANSIT)
382 .stream().findAny().orElse(null);
383 Set<DeviceId> egressDevices = getDevice(mcastIp, McastRole.EGRESS);
384
385 // Verify leadership on the operation
386 if (!isLeader(source)) {
387 log.debug("Skip {} due to lack of leadership", mcastIp);
388 return;
389 }
390
391 // If there are egress devices, sinks could be only on the ingress
392 if (!egressDevices.isEmpty()) {
393 egressDevices.forEach(
394 deviceId -> removeGroupFromDevice(deviceId, mcastIp, assignedVlan(null))
395 );
396 }
397 // Transit could be null
398 if (transitDevice != null) {
399 removeGroupFromDevice(transitDevice, mcastIp, assignedVlan(null));
400 }
401 // Ingress device should be not null
402 if (ingressDevice != null) {
403 removeGroupFromDevice(ingressDevice, mcastIp, assignedVlan(source));
404 }
Pier Luigi9930da52018-02-02 16:19:11 +0100405 } finally {
406 mcastUnlock();
407 }
408 }
409
410 /**
Pier Luigib72201b2018-01-25 16:16:02 +0100411 * Removes a path from source to sink for given multicast group.
412 *
413 * @param source connect point of the multicast source
414 * @param sink connection point of the multicast sink
415 * @param mcastIp multicast group IP address
416 */
417 private void processSinkRemovedInternal(ConnectPoint source, ConnectPoint sink,
418 IpAddress mcastIp) {
419 lastMcastChange = Instant.now();
420 mcastLock();
421 try {
Pier Luigi9930da52018-02-02 16:19:11 +0100422 // Verify leadership on the operation
423 if (!isLeader(source)) {
424 log.debug("Skip {} due to lack of leadership", mcastIp);
Charles Chand55e84d2016-03-30 17:54:24 -0700425 return;
426 }
Charles Chand55e84d2016-03-30 17:54:24 -0700427
Pier Luigib87b8ab2018-03-02 12:53:37 +0100428 boolean isLast = false;
Pier Luigib72201b2018-01-25 16:16:02 +0100429 // When source and sink are on the same device
430 if (source.deviceId().equals(sink.deviceId())) {
431 // Source and sink are on even the same port. There must be something wrong.
432 if (source.port().equals(sink.port())) {
433 log.warn("Skip {} since sink {} is on the same port of source {}. Abort",
434 mcastIp, sink, source);
435 return;
436 }
Pier Luigib87b8ab2018-03-02 12:53:37 +0100437 isLast = removePortFromDevice(sink.deviceId(), sink.port(), mcastIp, assignedVlan(source));
438 if (isLast) {
439 mcastRoleStore.remove(new McastStoreKey(mcastIp, sink.deviceId()));
440 }
Pier Luigib72201b2018-01-25 16:16:02 +0100441 return;
442 }
Charles Chand55e84d2016-03-30 17:54:24 -0700443
Pier Luigib72201b2018-01-25 16:16:02 +0100444 // Process the egress device
Pier Luigib87b8ab2018-03-02 12:53:37 +0100445 isLast = removePortFromDevice(sink.deviceId(), sink.port(), mcastIp, assignedVlan(null));
Pier Luigib72201b2018-01-25 16:16:02 +0100446 if (isLast) {
447 mcastRoleStore.remove(new McastStoreKey(mcastIp, sink.deviceId()));
448 }
449
450 // If this is the last sink on the device, also update upstream
451 Optional<Path> mcastPath = getPath(source.deviceId(), sink.deviceId(), mcastIp);
452 if (mcastPath.isPresent()) {
453 List<Link> links = Lists.newArrayList(mcastPath.get().links());
454 Collections.reverse(links);
455 for (Link link : links) {
456 if (isLast) {
457 isLast = removePortFromDevice(
458 link.src().deviceId(),
459 link.src().port(),
460 mcastIp,
461 assignedVlan(link.src().deviceId().equals(source.deviceId()) ? source : null)
462 );
Pier Luigib87b8ab2018-03-02 12:53:37 +0100463 if (isLast) {
464 mcastRoleStore.remove(new McastStoreKey(mcastIp, link.src().deviceId()));
465 }
Pier Luigib72201b2018-01-25 16:16:02 +0100466 }
Charles Chand55e84d2016-03-30 17:54:24 -0700467 }
468 }
Pier Luigib72201b2018-01-25 16:16:02 +0100469 } finally {
470 mcastUnlock();
Charles Chand55e84d2016-03-30 17:54:24 -0700471 }
472 }
473
474 /**
475 * Establishes a path from source to sink for given multicast group.
476 *
477 * @param source connect point of the multicast source
478 * @param sink connection point of the multicast sink
479 * @param mcastIp multicast group IP address
480 */
481 private void processSinkAddedInternal(ConnectPoint source, ConnectPoint sink,
482 IpAddress mcastIp) {
Pier Luigib72201b2018-01-25 16:16:02 +0100483 lastMcastChange = Instant.now();
484 mcastLock();
485 try {
486 // Continue only when this instance is the master of source device
487 if (!srManager.mastershipService.isLocalMaster(source.deviceId())) {
488 log.debug("Skip {} due to lack of mastership of the source device {}",
489 mcastIp, source.deviceId());
Charles Chand55e84d2016-03-30 17:54:24 -0700490 return;
491 }
Charles Chand55e84d2016-03-30 17:54:24 -0700492
Pier Luigib72201b2018-01-25 16:16:02 +0100493 // Process the ingress device
494 addFilterToDevice(source.deviceId(), source.port(), assignedVlan(source), mcastIp);
Charles Chan2199c302016-04-23 17:36:10 -0700495
Pier Luigib72201b2018-01-25 16:16:02 +0100496 // When source and sink are on the same device
497 if (source.deviceId().equals(sink.deviceId())) {
498 // Source and sink are on even the same port. There must be something wrong.
499 if (source.port().equals(sink.port())) {
500 log.warn("Skip {} since sink {} is on the same port of source {}. Abort",
501 mcastIp, sink, source);
502 return;
503 }
504 addPortToDevice(sink.deviceId(), sink.port(), mcastIp, assignedVlan(source));
505 mcastRoleStore.put(new McastStoreKey(mcastIp, sink.deviceId()), McastRole.INGRESS);
506 return;
507 }
Charles Chan2199c302016-04-23 17:36:10 -0700508
Pier Luigib72201b2018-01-25 16:16:02 +0100509 // Find a path. If present, create/update groups and flows for each hop
510 Optional<Path> mcastPath = getPath(source.deviceId(), sink.deviceId(), mcastIp);
511 if (mcastPath.isPresent()) {
512 List<Link> links = mcastPath.get().links();
513 checkState(links.size() == 2,
514 "Path in leaf-spine topology should always be two hops: ", links);
Charles Chan2199c302016-04-23 17:36:10 -0700515
Pier Luigib72201b2018-01-25 16:16:02 +0100516 links.forEach(link -> {
517 addPortToDevice(link.src().deviceId(), link.src().port(), mcastIp,
518 assignedVlan(link.src().deviceId().equals(source.deviceId()) ? source : null));
519 addFilterToDevice(link.dst().deviceId(), link.dst().port(), assignedVlan(null), mcastIp);
520 });
521
522 // Process the egress device
523 addPortToDevice(sink.deviceId(), sink.port(), mcastIp, assignedVlan(null));
524
525 // Setup mcast roles
526 mcastRoleStore.put(new McastStoreKey(mcastIp, source.deviceId()),
527 McastRole.INGRESS);
528 mcastRoleStore.put(new McastStoreKey(mcastIp, links.get(0).dst().deviceId()),
529 McastRole.TRANSIT);
530 mcastRoleStore.put(new McastStoreKey(mcastIp, sink.deviceId()),
531 McastRole.EGRESS);
532 } else {
533 log.warn("Unable to find a path from {} to {}. Abort sinkAdded",
534 source.deviceId(), sink.deviceId());
535 }
536 } finally {
537 mcastUnlock();
Charles Chand55e84d2016-03-30 17:54:24 -0700538 }
539 }
540
541 /**
Charles Chan2199c302016-04-23 17:36:10 -0700542 * Processes the LINK_DOWN event.
543 *
544 * @param affectedLink Link that is going down
545 */
Pier Luigi96fe0772018-02-28 12:10:50 +0100546 public void processLinkDown(Link affectedLink) {
Pier Luigib72201b2018-01-25 16:16:02 +0100547 lastMcastChange = Instant.now();
548 mcastLock();
549 try {
550 // Get groups affected by the link down event
551 getAffectedGroups(affectedLink).forEach(mcastIp -> {
552 // TODO Optimize when the group editing is in place
553 log.debug("Processing link down {} for group {}",
554 affectedLink, mcastIp);
Pier Luigieba73a02018-01-16 10:47:50 +0100555
Pier Luigib72201b2018-01-25 16:16:02 +0100556 // Find out the ingress, transit and egress device of affected group
557 DeviceId ingressDevice = getDevice(mcastIp, McastRole.INGRESS)
558 .stream().findAny().orElse(null);
559 DeviceId transitDevice = getDevice(mcastIp, McastRole.TRANSIT)
560 .stream().findAny().orElse(null);
561 Set<DeviceId> egressDevices = getDevice(mcastIp, McastRole.EGRESS);
562 ConnectPoint source = getSource(mcastIp);
Charles Chan8d449862016-05-16 18:44:13 -0700563
Pier Luigib72201b2018-01-25 16:16:02 +0100564 // Do not proceed if any of these info is missing
565 if (ingressDevice == null || transitDevice == null
566 || egressDevices == null || source == null) {
567 log.warn("Missing ingress {}, transit {}, egress {} devices or source {}",
568 ingressDevice, transitDevice, egressDevices, source);
569 return;
Charles Chan2199c302016-04-23 17:36:10 -0700570 }
Pier Luigib72201b2018-01-25 16:16:02 +0100571
572 // Continue only when this instance is the master of source device
573 if (!srManager.mastershipService.isLocalMaster(source.deviceId())) {
574 log.debug("Skip {} due to lack of mastership of the source device {}",
575 source.deviceId());
576 return;
577 }
578
579 // Remove entire transit
580 removeGroupFromDevice(transitDevice, mcastIp, assignedVlan(null));
581
582 // Remove transit-facing port on ingress device
583 PortNumber ingressTransitPort = ingressTransitPort(mcastIp);
584 if (ingressTransitPort != null) {
Pier Luigib87b8ab2018-03-02 12:53:37 +0100585 boolean isLast = removePortFromDevice(ingressDevice, ingressTransitPort,
586 mcastIp, assignedVlan(source));
587 if (isLast) {
588 mcastRoleStore.remove(new McastStoreKey(mcastIp, ingressDevice));
589 }
Pier Luigib72201b2018-01-25 16:16:02 +0100590 }
591
592 // Construct a new path for each egress device
593 egressDevices.forEach(egressDevice -> {
594 Optional<Path> mcastPath = getPath(ingressDevice, egressDevice, mcastIp);
595 if (mcastPath.isPresent()) {
596 installPath(mcastIp, source, mcastPath.get());
597 } else {
598 log.warn("Fail to recover egress device {} from link failure {}",
599 egressDevice, affectedLink);
600 removeGroupFromDevice(egressDevice, mcastIp, assignedVlan(null));
601 }
602 });
Charles Chan2199c302016-04-23 17:36:10 -0700603 });
Pier Luigib72201b2018-01-25 16:16:02 +0100604 } finally {
605 mcastUnlock();
606 }
Charles Chan2199c302016-04-23 17:36:10 -0700607 }
608
609 /**
Pier Luigieba73a02018-01-16 10:47:50 +0100610 * Process the DEVICE_DOWN event.
611 *
612 * @param deviceDown device going down
613 */
Pier Luigi96fe0772018-02-28 12:10:50 +0100614 public void processDeviceDown(DeviceId deviceDown) {
Pier Luigib72201b2018-01-25 16:16:02 +0100615 lastMcastChange = Instant.now();
616 mcastLock();
617 try {
618 // Get the mcast groups affected by the device going down
619 getAffectedGroups(deviceDown).forEach(mcastIp -> {
620 // TODO Optimize when the group editing is in place
621 log.debug("Processing device down {} for group {}",
622 deviceDown, mcastIp);
Pier Luigieba73a02018-01-16 10:47:50 +0100623
Pier Luigib72201b2018-01-25 16:16:02 +0100624 // Find out the ingress, transit and egress device of affected group
625 DeviceId ingressDevice = getDevice(mcastIp, McastRole.INGRESS)
626 .stream().findAny().orElse(null);
627 DeviceId transitDevice = getDevice(mcastIp, McastRole.TRANSIT)
628 .stream().findAny().orElse(null);
629 Set<DeviceId> egressDevices = getDevice(mcastIp, McastRole.EGRESS);
630 ConnectPoint source = getSource(mcastIp);
Pier Luigieba73a02018-01-16 10:47:50 +0100631
Pier Luigib72201b2018-01-25 16:16:02 +0100632 // Do not proceed if ingress device or source of this group are missing
633 // If sinks are in other leafs, we have ingress, transit, egress, and source
634 // If sinks are in the same leaf, we have just ingress and source
635 if (ingressDevice == null || source == null) {
636 log.warn("Missing ingress {} or source {} for group {}",
637 ingressDevice, source, mcastIp);
Pier Luigieba73a02018-01-16 10:47:50 +0100638 return;
639 }
Pier Luigieba73a02018-01-16 10:47:50 +0100640
Pier Luigi9930da52018-02-02 16:19:11 +0100641 // Verify leadership on the operation
642 if (!isLeader(source)) {
643 log.debug("Skip {} due to lack of leadership", mcastIp);
644 return;
Pier Luigieba73a02018-01-16 10:47:50 +0100645 }
Pier Luigib72201b2018-01-25 16:16:02 +0100646
647 // If it exists, we have to remove it in any case
648 if (transitDevice != null) {
649 // Remove entire transit
650 removeGroupFromDevice(transitDevice, mcastIp, assignedVlan(null));
651 }
652 // If the ingress is down
653 if (ingressDevice.equals(deviceDown)) {
654 // Remove entire ingress
655 removeGroupFromDevice(ingressDevice, mcastIp, assignedVlan(source));
656 // If other sinks different from the ingress exist
657 if (!egressDevices.isEmpty()) {
658 // Remove all the remaining egress
659 egressDevices.forEach(
660 egressDevice -> removeGroupFromDevice(egressDevice, mcastIp, assignedVlan(null))
661 );
Pier Luigieba73a02018-01-16 10:47:50 +0100662 }
Pier Luigib72201b2018-01-25 16:16:02 +0100663 } else {
664 // Egress or transit could be down at this point
665 // Get the ingress-transit port if it exists
666 PortNumber ingressTransitPort = ingressTransitPort(mcastIp);
667 if (ingressTransitPort != null) {
668 // Remove transit-facing port on ingress device
Pier Luigib87b8ab2018-03-02 12:53:37 +0100669 boolean isLast = removePortFromDevice(ingressDevice, ingressTransitPort,
670 mcastIp, assignedVlan(source));
671 // There are no further ports
672 if (isLast) {
673 // Remove entire ingress
674 mcastRoleStore.remove(new McastStoreKey(mcastIp, ingressDevice));
675 }
Pier Luigib72201b2018-01-25 16:16:02 +0100676 }
677 // One of the egress device is down
678 if (egressDevices.contains(deviceDown)) {
679 // Remove entire device down
680 removeGroupFromDevice(deviceDown, mcastIp, assignedVlan(null));
681 // Remove the device down from egress
682 egressDevices.remove(deviceDown);
683 // If there are no more egress and ingress does not have sinks
684 if (egressDevices.isEmpty() && !hasSinks(ingressDevice, mcastIp)) {
Pier Luigib72201b2018-01-25 16:16:02 +0100685 // We have done
686 return;
687 }
688 }
689 // Construct a new path for each egress device
690 egressDevices.forEach(egressDevice -> {
691 Optional<Path> mcastPath = getPath(ingressDevice, egressDevice, mcastIp);
692 // If there is a new path
693 if (mcastPath.isPresent()) {
694 // Let's install the new mcast path for this egress
695 installPath(mcastIp, source, mcastPath.get());
696 } else {
697 // We were not able to find an alternative path for this egress
698 log.warn("Fail to recover egress device {} from device down {}",
699 egressDevice, deviceDown);
700 removeGroupFromDevice(egressDevice, mcastIp, assignedVlan(null));
701 }
702 });
703 }
704 });
705 } finally {
706 mcastUnlock();
707 }
Pier Luigieba73a02018-01-16 10:47:50 +0100708 }
709
710 /**
Charles Chand55e84d2016-03-30 17:54:24 -0700711 * Adds filtering objective for given device and port.
712 *
713 * @param deviceId device ID
714 * @param port ingress port number
715 * @param assignedVlan assigned VLAN ID
716 */
Julia Ferguson65428c32017-08-10 18:15:24 +0000717 private void addFilterToDevice(DeviceId deviceId, PortNumber port, VlanId assignedVlan, IpAddress mcastIp) {
Charles Chand55e84d2016-03-30 17:54:24 -0700718 // Do nothing if the port is configured as suppressed
Charles Chan6ea94fc2016-05-10 17:29:47 -0700719 ConnectPoint connectPoint = new ConnectPoint(deviceId, port);
720 SegmentRoutingAppConfig appConfig = srManager.cfgService
Pier Luigi96fe0772018-02-28 12:10:50 +0100721 .getConfig(srManager.appId(), SegmentRoutingAppConfig.class);
Charles Chan6ea94fc2016-05-10 17:29:47 -0700722 if (appConfig != null && appConfig.suppressSubnet().contains(connectPoint)) {
723 log.info("Ignore suppressed port {}", connectPoint);
Charles Chand55e84d2016-03-30 17:54:24 -0700724 return;
725 }
726
Charles Chan50d900c2018-03-02 13:26:22 -0800727 MacAddress routerMac;
728 try {
729 routerMac = srManager.deviceConfiguration().getDeviceMac(deviceId);
730 } catch (DeviceConfigNotFoundException dcnfe) {
731 log.warn("Fail to push filtering objective since device is not configured. Abort");
732 return;
733 }
734
Charles Chand55e84d2016-03-30 17:54:24 -0700735 FilteringObjective.Builder filtObjBuilder =
Charles Chan45b69ab2018-03-02 15:41:41 -0800736 filterObjBuilder(port, assignedVlan, mcastIp, routerMac);
Charles Chan2199c302016-04-23 17:36:10 -0700737 ObjectiveContext context = new DefaultObjectiveContext(
738 (objective) -> log.debug("Successfully add filter on {}/{}, vlan {}",
Charles Chan59cc16d2017-02-02 16:20:42 -0800739 deviceId, port.toLong(), assignedVlan),
Charles Chan2199c302016-04-23 17:36:10 -0700740 (objective, error) ->
741 log.warn("Failed to add filter on {}/{}, vlan {}: {}",
Charles Chan59cc16d2017-02-02 16:20:42 -0800742 deviceId, port.toLong(), assignedVlan, error));
Charles Chan2199c302016-04-23 17:36:10 -0700743 srManager.flowObjectiveService.filter(deviceId, filtObjBuilder.add(context));
Charles Chand55e84d2016-03-30 17:54:24 -0700744 }
745
746 /**
747 * Adds a port to given multicast group on given device. This involves the
748 * update of L3 multicast group and multicast routing table entry.
749 *
750 * @param deviceId device ID
751 * @param port port to be added
752 * @param mcastIp multicast group
753 * @param assignedVlan assigned VLAN ID
754 */
755 private void addPortToDevice(DeviceId deviceId, PortNumber port,
756 IpAddress mcastIp, VlanId assignedVlan) {
Charles Chan2199c302016-04-23 17:36:10 -0700757 McastStoreKey mcastStoreKey = new McastStoreKey(mcastIp, deviceId);
Charles Chand55e84d2016-03-30 17:54:24 -0700758 ImmutableSet.Builder<PortNumber> portBuilder = ImmutableSet.builder();
Pier Luigi21fffd22018-01-19 10:24:53 +0100759 NextObjective newNextObj;
Charles Chan2199c302016-04-23 17:36:10 -0700760 if (!mcastNextObjStore.containsKey(mcastStoreKey)) {
Charles Chand55e84d2016-03-30 17:54:24 -0700761 // First time someone request this mcast group via this device
762 portBuilder.add(port);
Pier Luigi21fffd22018-01-19 10:24:53 +0100763 // New nextObj
764 newNextObj = nextObjBuilder(mcastIp, assignedVlan,
765 portBuilder.build(), null).add();
766 // Store the new port
767 mcastNextObjStore.put(mcastStoreKey, newNextObj);
Charles Chand55e84d2016-03-30 17:54:24 -0700768 } else {
769 // This device already serves some subscribers of this mcast group
Charles Chan2199c302016-04-23 17:36:10 -0700770 NextObjective nextObj = mcastNextObjStore.get(mcastStoreKey).value();
Charles Chand55e84d2016-03-30 17:54:24 -0700771 // Stop if the port is already in the nextobj
772 Set<PortNumber> existingPorts = getPorts(nextObj.next());
773 if (existingPorts.contains(port)) {
774 log.info("NextObj for {}/{} already exists. Abort", deviceId, port);
775 return;
776 }
Pier Luigi21fffd22018-01-19 10:24:53 +0100777 // Let's add the port and reuse the previous one
Yuta HIGUCHI0eb68e12018-02-09 18:05:23 -0800778 portBuilder.addAll(existingPorts).add(port);
Pier Luigi21fffd22018-01-19 10:24:53 +0100779 // Reuse previous nextObj
780 newNextObj = nextObjBuilder(mcastIp, assignedVlan,
781 portBuilder.build(), nextObj.id()).addToExisting();
782 // Store the final next objective and send only the difference to the driver
783 mcastNextObjStore.put(mcastStoreKey, newNextObj);
784 // Add just the new port
785 portBuilder = ImmutableSet.builder();
786 portBuilder.add(port);
787 newNextObj = nextObjBuilder(mcastIp, assignedVlan,
788 portBuilder.build(), nextObj.id()).addToExisting();
Charles Chand55e84d2016-03-30 17:54:24 -0700789 }
790 // Create, store and apply the new nextObj and fwdObj
Charles Chan2199c302016-04-23 17:36:10 -0700791 ObjectiveContext context = new DefaultObjectiveContext(
792 (objective) -> log.debug("Successfully add {} on {}/{}, vlan {}",
793 mcastIp, deviceId, port.toLong(), assignedVlan),
794 (objective, error) ->
795 log.warn("Failed to add {} on {}/{}, vlan {}: {}",
796 mcastIp, deviceId, port.toLong(), assignedVlan, error));
Charles Chand55e84d2016-03-30 17:54:24 -0700797 ForwardingObjective fwdObj =
Charles Chan2199c302016-04-23 17:36:10 -0700798 fwdObjBuilder(mcastIp, assignedVlan, newNextObj.id()).add(context);
Charles Chand55e84d2016-03-30 17:54:24 -0700799 srManager.flowObjectiveService.next(deviceId, newNextObj);
800 srManager.flowObjectiveService.forward(deviceId, fwdObj);
Charles Chand55e84d2016-03-30 17:54:24 -0700801 }
802
803 /**
804 * Removes a port from given multicast group on given device.
805 * This involves the update of L3 multicast group and multicast routing
806 * table entry.
807 *
808 * @param deviceId device ID
809 * @param port port to be added
810 * @param mcastIp multicast group
811 * @param assignedVlan assigned VLAN ID
812 * @return true if this is the last sink on this device
813 */
814 private boolean removePortFromDevice(DeviceId deviceId, PortNumber port,
815 IpAddress mcastIp, VlanId assignedVlan) {
Charles Chan2199c302016-04-23 17:36:10 -0700816 McastStoreKey mcastStoreKey =
817 new McastStoreKey(mcastIp, deviceId);
Charles Chand55e84d2016-03-30 17:54:24 -0700818 // This device is not serving this multicast group
Charles Chan2199c302016-04-23 17:36:10 -0700819 if (!mcastNextObjStore.containsKey(mcastStoreKey)) {
Charles Chand55e84d2016-03-30 17:54:24 -0700820 log.warn("{} is not serving {} on port {}. Abort.", deviceId, mcastIp, port);
821 return false;
822 }
Charles Chan2199c302016-04-23 17:36:10 -0700823 NextObjective nextObj = mcastNextObjStore.get(mcastStoreKey).value();
Charles Chand55e84d2016-03-30 17:54:24 -0700824
825 Set<PortNumber> existingPorts = getPorts(nextObj.next());
Charles Chan2199c302016-04-23 17:36:10 -0700826 // This port does not serve this multicast group
Charles Chand55e84d2016-03-30 17:54:24 -0700827 if (!existingPorts.contains(port)) {
828 log.warn("{} is not serving {} on port {}. Abort.", deviceId, mcastIp, port);
829 return false;
830 }
831 // Copy and modify the ImmutableSet
832 existingPorts = Sets.newHashSet(existingPorts);
833 existingPorts.remove(port);
834
835 NextObjective newNextObj;
Pier Luigid1be7b12018-01-19 10:24:53 +0100836 ObjectiveContext context;
Charles Chand55e84d2016-03-30 17:54:24 -0700837 ForwardingObjective fwdObj;
838 if (existingPorts.isEmpty()) {
Pier Luigid1be7b12018-01-19 10:24:53 +0100839 // If this is the last sink, remove flows and last bucket
Charles Chand55e84d2016-03-30 17:54:24 -0700840 // NOTE: Rely on GroupStore garbage collection rather than explicitly
841 // remove L3MG since there might be other flows/groups refer to
842 // the same L2IG
Pier Luigid1be7b12018-01-19 10:24:53 +0100843 context = new DefaultObjectiveContext(
Charles Chan2199c302016-04-23 17:36:10 -0700844 (objective) -> log.debug("Successfully remove {} on {}/{}, vlan {}",
845 mcastIp, deviceId, port.toLong(), assignedVlan),
846 (objective, error) ->
847 log.warn("Failed to remove {} on {}/{}, vlan {}: {}",
848 mcastIp, deviceId, port.toLong(), assignedVlan, error));
849 fwdObj = fwdObjBuilder(mcastIp, assignedVlan, nextObj.id()).remove(context);
850 mcastNextObjStore.remove(mcastStoreKey);
Charles Chand55e84d2016-03-30 17:54:24 -0700851 } else {
852 // If this is not the last sink, update flows and groups
Pier Luigid1be7b12018-01-19 10:24:53 +0100853 context = new DefaultObjectiveContext(
Charles Chan2199c302016-04-23 17:36:10 -0700854 (objective) -> log.debug("Successfully update {} on {}/{}, vlan {}",
855 mcastIp, deviceId, port.toLong(), assignedVlan),
856 (objective, error) ->
857 log.warn("Failed to update {} on {}/{}, vlan {}: {}",
858 mcastIp, deviceId, port.toLong(), assignedVlan, error));
Pier Luigid1be7b12018-01-19 10:24:53 +0100859 // Here we store the next objective with the remaining port
860 newNextObj = nextObjBuilder(mcastIp, assignedVlan,
861 existingPorts, nextObj.id()).removeFromExisting();
Charles Chanfc5c7802016-05-17 13:13:55 -0700862 fwdObj = fwdObjBuilder(mcastIp, assignedVlan, newNextObj.id()).add(context);
Charles Chan2199c302016-04-23 17:36:10 -0700863 mcastNextObjStore.put(mcastStoreKey, newNextObj);
Charles Chand55e84d2016-03-30 17:54:24 -0700864 }
Pier Luigid1be7b12018-01-19 10:24:53 +0100865 // Let's modify the next objective removing the bucket
866 newNextObj = nextObjBuilder(mcastIp, assignedVlan,
867 ImmutableSet.of(port), nextObj.id()).removeFromExisting();
868 srManager.flowObjectiveService.next(deviceId, newNextObj);
869 srManager.flowObjectiveService.forward(deviceId, fwdObj);
Charles Chand55e84d2016-03-30 17:54:24 -0700870 return existingPorts.isEmpty();
871 }
872
Charles Chan2199c302016-04-23 17:36:10 -0700873 /**
874 * Removes entire group on given device.
875 *
876 * @param deviceId device ID
877 * @param mcastIp multicast group to be removed
878 * @param assignedVlan assigned VLAN ID
879 */
880 private void removeGroupFromDevice(DeviceId deviceId, IpAddress mcastIp,
881 VlanId assignedVlan) {
882 McastStoreKey mcastStoreKey = new McastStoreKey(mcastIp, deviceId);
883 // This device is not serving this multicast group
884 if (!mcastNextObjStore.containsKey(mcastStoreKey)) {
885 log.warn("{} is not serving {}. Abort.", deviceId, mcastIp);
886 return;
887 }
888 NextObjective nextObj = mcastNextObjStore.get(mcastStoreKey).value();
889 // NOTE: Rely on GroupStore garbage collection rather than explicitly
890 // remove L3MG since there might be other flows/groups refer to
891 // the same L2IG
892 ObjectiveContext context = new DefaultObjectiveContext(
893 (objective) -> log.debug("Successfully remove {} on {}, vlan {}",
894 mcastIp, deviceId, assignedVlan),
895 (objective, error) ->
896 log.warn("Failed to remove {} on {}, vlan {}: {}",
897 mcastIp, deviceId, assignedVlan, error));
898 ForwardingObjective fwdObj = fwdObjBuilder(mcastIp, assignedVlan, nextObj.id()).remove(context);
899 srManager.flowObjectiveService.forward(deviceId, fwdObj);
900 mcastNextObjStore.remove(mcastStoreKey);
901 mcastRoleStore.remove(mcastStoreKey);
902 }
903
Pier Luigieba73a02018-01-16 10:47:50 +0100904 private void installPath(IpAddress mcastIp, ConnectPoint source, Path mcastPath) {
905 // Get Links
906 List<Link> links = mcastPath.links();
907 // For each link, modify the next on the source device adding the src port
908 // and a new filter objective on the destination port
909 links.forEach(link -> {
910 addPortToDevice(link.src().deviceId(), link.src().port(), mcastIp,
911 assignedVlan(link.src().deviceId().equals(source.deviceId()) ? source : null));
912 addFilterToDevice(link.dst().deviceId(), link.dst().port(), assignedVlan(null),
913 mcastIp);
914 });
915 // Setup new transit mcast role
916 mcastRoleStore.put(new McastStoreKey(mcastIp, links.get(0).dst().deviceId()),
917 McastRole.TRANSIT);
Pier Luigib87b8ab2018-03-02 12:53:37 +0100918 // Setup new ingress mcast role
919 mcastRoleStore.put(new McastStoreKey(mcastIp, links.get(0).src().deviceId()),
920 McastRole.INGRESS);
Charles Chan2199c302016-04-23 17:36:10 -0700921 }
922
Charles Chand55e84d2016-03-30 17:54:24 -0700923 /**
924 * Creates a next objective builder for multicast.
925 *
926 * @param mcastIp multicast group
927 * @param assignedVlan assigned VLAN ID
928 * @param outPorts set of output port numbers
929 * @return next objective builder
930 */
931 private NextObjective.Builder nextObjBuilder(IpAddress mcastIp,
Pier Luigi21fffd22018-01-19 10:24:53 +0100932 VlanId assignedVlan, Set<PortNumber> outPorts, Integer nextId) {
933 // If nextId is null allocate a new one
934 if (nextId == null) {
935 nextId = srManager.flowObjectiveService.allocateNextId();
936 }
Charles Chand55e84d2016-03-30 17:54:24 -0700937
938 TrafficSelector metadata =
939 DefaultTrafficSelector.builder()
940 .matchVlanId(assignedVlan)
941 .matchIPDst(mcastIp.toIpPrefix())
942 .build();
943
944 NextObjective.Builder nextObjBuilder = DefaultNextObjective
945 .builder().withId(nextId)
Pier Luigi96fe0772018-02-28 12:10:50 +0100946 .withType(NextObjective.Type.BROADCAST).fromApp(srManager.appId())
Charles Chand55e84d2016-03-30 17:54:24 -0700947 .withMeta(metadata);
948
949 outPorts.forEach(port -> {
950 TrafficTreatment.Builder tBuilder = DefaultTrafficTreatment.builder();
951 if (egressVlan().equals(VlanId.NONE)) {
952 tBuilder.popVlan();
953 }
954 tBuilder.setOutput(port);
955 nextObjBuilder.addTreatment(tBuilder.build());
956 });
957
958 return nextObjBuilder;
959 }
960
961 /**
962 * Creates a forwarding objective builder for multicast.
963 *
964 * @param mcastIp multicast group
965 * @param assignedVlan assigned VLAN ID
966 * @param nextId next ID of the L3 multicast group
967 * @return forwarding objective builder
968 */
969 private ForwardingObjective.Builder fwdObjBuilder(IpAddress mcastIp,
970 VlanId assignedVlan, int nextId) {
971 TrafficSelector.Builder sbuilder = DefaultTrafficSelector.builder();
Julia Ferguson65428c32017-08-10 18:15:24 +0000972 IpPrefix mcastPrefix = mcastIp.toIpPrefix();
973
974 if (mcastIp.isIp4()) {
975 sbuilder.matchEthType(Ethernet.TYPE_IPV4);
976 sbuilder.matchIPDst(mcastPrefix);
977 } else {
978 sbuilder.matchEthType(Ethernet.TYPE_IPV6);
979 sbuilder.matchIPv6Dst(mcastPrefix);
980 }
981
982
Charles Chand55e84d2016-03-30 17:54:24 -0700983 TrafficSelector.Builder metabuilder = DefaultTrafficSelector.builder();
984 metabuilder.matchVlanId(assignedVlan);
985
986 ForwardingObjective.Builder fwdBuilder = DefaultForwardingObjective.builder();
987 fwdBuilder.withSelector(sbuilder.build())
988 .withMeta(metabuilder.build())
989 .nextStep(nextId)
990 .withFlag(ForwardingObjective.Flag.SPECIFIC)
Pier Luigi96fe0772018-02-28 12:10:50 +0100991 .fromApp(srManager.appId())
Charles Chand55e84d2016-03-30 17:54:24 -0700992 .withPriority(SegmentRoutingService.DEFAULT_PRIORITY);
993 return fwdBuilder;
994 }
995
996 /**
997 * Creates a filtering objective builder for multicast.
998 *
Charles Chand55e84d2016-03-30 17:54:24 -0700999 * @param ingressPort ingress port of the multicast stream
1000 * @param assignedVlan assigned VLAN ID
Charles Chan50d900c2018-03-02 13:26:22 -08001001 * @param routerMac router MAC. This is carried in metadata and used from some switches that
1002 * need to put unicast entry before multicast entry in TMAC table.
Charles Chand55e84d2016-03-30 17:54:24 -07001003 * @return filtering objective builder
1004 */
Charles Chan45b69ab2018-03-02 15:41:41 -08001005 private FilteringObjective.Builder filterObjBuilder(PortNumber ingressPort,
Charles Chan50d900c2018-03-02 13:26:22 -08001006 VlanId assignedVlan, IpAddress mcastIp, MacAddress routerMac) {
Charles Chand55e84d2016-03-30 17:54:24 -07001007 FilteringObjective.Builder filtBuilder = DefaultFilteringObjective.builder();
Charles Chan1588e7b2016-06-28 16:50:13 -07001008
Julia Ferguson65428c32017-08-10 18:15:24 +00001009 if (mcastIp.isIp4()) {
1010 filtBuilder.withKey(Criteria.matchInPort(ingressPort))
1011 .addCondition(Criteria.matchEthDstMasked(MacAddress.IPV4_MULTICAST,
1012 MacAddress.IPV4_MULTICAST_MASK))
1013 .addCondition(Criteria.matchVlanId(egressVlan()))
1014 .withPriority(SegmentRoutingService.DEFAULT_PRIORITY);
1015 } else {
1016 filtBuilder.withKey(Criteria.matchInPort(ingressPort))
1017 .addCondition(Criteria.matchEthDstMasked(MacAddress.IPV6_MULTICAST,
1018 MacAddress.IPV6_MULTICAST_MASK))
1019 .addCondition(Criteria.matchVlanId(egressVlan()))
1020 .withPriority(SegmentRoutingService.DEFAULT_PRIORITY);
1021 }
Charles Chan1588e7b2016-06-28 16:50:13 -07001022 TrafficTreatment tt = DefaultTrafficTreatment.builder()
Charles Chan50d900c2018-03-02 13:26:22 -08001023 .pushVlan().setVlanId(assignedVlan)
1024 .setEthDst(routerMac)
1025 .build();
Charles Chan1588e7b2016-06-28 16:50:13 -07001026 filtBuilder.withMeta(tt);
1027
Pier Luigi96fe0772018-02-28 12:10:50 +01001028 return filtBuilder.permit().fromApp(srManager.appId());
Charles Chand55e84d2016-03-30 17:54:24 -07001029 }
1030
1031 /**
1032 * Gets output ports information from treatments.
1033 *
1034 * @param treatments collection of traffic treatments
1035 * @return set of output port numbers
1036 */
1037 private Set<PortNumber> getPorts(Collection<TrafficTreatment> treatments) {
1038 ImmutableSet.Builder<PortNumber> builder = ImmutableSet.builder();
1039 treatments.forEach(treatment -> {
1040 treatment.allInstructions().stream()
1041 .filter(instr -> instr instanceof OutputInstruction)
1042 .forEach(instr -> {
1043 builder.add(((OutputInstruction) instr).port());
1044 });
1045 });
1046 return builder.build();
1047 }
1048
Pier Luigif7049c52018-02-23 19:57:40 +01001049 // Utility method to verify is a link is a pair-link
1050 private boolean isPairLink(Link link) {
1051 // Take src id, src port, dst id and dst port
1052 final DeviceId srcId = link.src().deviceId();
1053 final PortNumber srcPort = link.src().port();
1054 final DeviceId dstId = link.dst().deviceId();
1055 final PortNumber dstPort = link.dst().port();
1056 // init as true
1057 boolean isPairLink = true;
1058 try {
1059 // If one of this condition is not true; it is not a pair link
Pier Luigi96fe0772018-02-28 12:10:50 +01001060 if (!(srManager.deviceConfiguration().isEdgeDevice(srcId) &&
1061 srManager.deviceConfiguration().isEdgeDevice(dstId) &&
1062 srManager.deviceConfiguration().getPairDeviceId(srcId).equals(dstId) &&
1063 srManager.deviceConfiguration().getPairLocalPort(srcId).equals(srcPort) &&
1064 srManager.deviceConfiguration().getPairLocalPort(dstId).equals(dstPort))) {
Pier Luigif7049c52018-02-23 19:57:40 +01001065 isPairLink = false;
1066 }
1067 } catch (DeviceConfigNotFoundException e) {
1068 // Configuration not provided
1069 log.warn("Could not check if the link {} is pairlink "
1070 + "config not yet provided", link);
1071 isPairLink = false;
1072 }
1073 return isPairLink;
1074 }
1075
Charles Chand55e84d2016-03-30 17:54:24 -07001076 /**
1077 * Gets a path from src to dst.
1078 * If a path was allocated before, returns the allocated path.
1079 * Otherwise, randomly pick one from available paths.
1080 *
1081 * @param src source device ID
1082 * @param dst destination device ID
1083 * @param mcastIp multicast group
1084 * @return an optional path from src to dst
1085 */
1086 private Optional<Path> getPath(DeviceId src, DeviceId dst, IpAddress mcastIp) {
Pier Luigi83f919b2018-02-15 16:33:08 +01001087 // Takes a snapshot of the topology
1088 final Topology currentTopology = topologyService.currentTopology();
Charles Chand55e84d2016-03-30 17:54:24 -07001089 List<Path> allPaths = Lists.newArrayList(
Pier Luigi83f919b2018-02-15 16:33:08 +01001090 topologyService.getPaths(currentTopology, src, dst)
1091 );
Pier Luigif7049c52018-02-23 19:57:40 +01001092 // Create list of valid paths
1093 allPaths.removeIf(path -> path.links().stream().anyMatch(this::isPairLink));
1094 // If there are no valid paths, just exit
Charles Chan2199c302016-04-23 17:36:10 -07001095 log.debug("{} path(s) found from {} to {}", allPaths.size(), src, dst);
Charles Chand55e84d2016-03-30 17:54:24 -07001096 if (allPaths.isEmpty()) {
Charles Chand55e84d2016-03-30 17:54:24 -07001097 return Optional.empty();
1098 }
1099
Pier Luigibad6d6c2018-01-23 16:06:38 +01001100 // Create a map index of suitablity-to-list of paths. For example
1101 // a path in the list associated to the index 1 shares only the
1102 // first hop and it is less suitable of a path belonging to the index
1103 // 2 that shares leaf-spine.
1104 Map<Integer, List<Path>> eligiblePaths = Maps.newHashMap();
1105 // Some init steps
1106 int nhop;
1107 McastStoreKey mcastStoreKey;
1108 Link hop;
1109 PortNumber srcPort;
1110 Set<PortNumber> existingPorts;
1111 NextObjective nextObj;
1112 // Iterate over paths looking for eligible paths
1113 for (Path path : allPaths) {
1114 // Unlikely, it will happen...
1115 if (!src.equals(path.links().get(0).src().deviceId())) {
1116 continue;
1117 }
1118 nhop = 0;
1119 // Iterate over the links
1120 while (nhop < path.links().size()) {
1121 // Get the link and verify if a next related
1122 // to the src device exist in the store
1123 hop = path.links().get(nhop);
1124 mcastStoreKey = new McastStoreKey(mcastIp, hop.src().deviceId());
1125 // It does not exist in the store, exit
1126 if (!mcastNextObjStore.containsKey(mcastStoreKey)) {
1127 break;
Charles Chand55e84d2016-03-30 17:54:24 -07001128 }
Pier Luigibad6d6c2018-01-23 16:06:38 +01001129 // Get the output ports on the next
1130 nextObj = mcastNextObjStore.get(mcastStoreKey).value();
1131 existingPorts = getPorts(nextObj.next());
1132 // And the src port on the link
1133 srcPort = hop.src().port();
1134 // the src port is not used as output, exit
1135 if (!existingPorts.contains(srcPort)) {
1136 break;
1137 }
1138 nhop++;
1139 }
1140 // n_hop defines the index
1141 if (nhop > 0) {
1142 eligiblePaths.compute(nhop, (index, paths) -> {
1143 paths = paths == null ? Lists.newArrayList() : paths;
1144 paths.add(path);
1145 return paths;
1146 });
Charles Chand55e84d2016-03-30 17:54:24 -07001147 }
1148 }
Pier Luigibad6d6c2018-01-23 16:06:38 +01001149
1150 // No suitable paths
1151 if (eligiblePaths.isEmpty()) {
1152 log.debug("No eligiblePath(s) found from {} to {}", src, dst);
1153 // Otherwise, randomly pick a path
1154 Collections.shuffle(allPaths);
1155 return allPaths.stream().findFirst();
1156 }
1157
1158 // Let's take the best ones
1159 Integer bestIndex = eligiblePaths.keySet()
1160 .stream()
1161 .sorted(Comparator.reverseOrder())
1162 .findFirst().orElse(null);
1163 List<Path> bestPaths = eligiblePaths.get(bestIndex);
1164 log.debug("{} eligiblePath(s) found from {} to {}",
1165 bestPaths.size(), src, dst);
1166 // randomly pick a path on the highest index
1167 Collections.shuffle(bestPaths);
1168 return bestPaths.stream().findFirst();
Charles Chand55e84d2016-03-30 17:54:24 -07001169 }
1170
1171 /**
Charles Chan2199c302016-04-23 17:36:10 -07001172 * Gets device(s) of given role in given multicast group.
1173 *
1174 * @param mcastIp multicast IP
1175 * @param role multicast role
1176 * @return set of device ID or empty set if not found
1177 */
1178 private Set<DeviceId> getDevice(IpAddress mcastIp, McastRole role) {
1179 return mcastRoleStore.entrySet().stream()
1180 .filter(entry -> entry.getKey().mcastIp().equals(mcastIp) &&
1181 entry.getValue().value() == role)
1182 .map(Map.Entry::getKey).map(McastStoreKey::deviceId)
1183 .collect(Collectors.toSet());
1184 }
1185
1186 /**
Charles Chan8d449862016-05-16 18:44:13 -07001187 * Gets source connect point of given multicast group.
1188 *
1189 * @param mcastIp multicast IP
1190 * @return source connect point or null if not found
1191 */
1192 private ConnectPoint getSource(IpAddress mcastIp) {
1193 return srManager.multicastRouteService.getRoutes().stream()
1194 .filter(mcastRoute -> mcastRoute.group().equals(mcastIp))
1195 .map(mcastRoute -> srManager.multicastRouteService.fetchSource(mcastRoute))
1196 .findAny().orElse(null);
1197 }
Pier Luigib87b8ab2018-03-02 12:53:37 +01001198 /**
1199 * Gets sinks of given multicast group.
1200 *
1201 * @param mcastIp multicast IP
1202 * @return set of sinks or empty set if not found
1203 */
1204 private Set<ConnectPoint> getSinks(IpAddress mcastIp) {
1205 return srManager.multicastRouteService.getRoutes().stream()
1206 .filter(mcastRoute -> mcastRoute.group().equals(mcastIp))
1207 .map(mcastRoute -> srManager.multicastRouteService.fetchSinks(mcastRoute))
1208 .findAny().orElse(Collections.emptySet());
1209 }
Charles Chan8d449862016-05-16 18:44:13 -07001210
1211 /**
Charles Chan2199c302016-04-23 17:36:10 -07001212 * Gets groups which is affected by the link down event.
1213 *
1214 * @param link link going down
1215 * @return a set of multicast IpAddress
1216 */
1217 private Set<IpAddress> getAffectedGroups(Link link) {
1218 DeviceId deviceId = link.src().deviceId();
1219 PortNumber port = link.src().port();
1220 return mcastNextObjStore.entrySet().stream()
1221 .filter(entry -> entry.getKey().deviceId().equals(deviceId) &&
1222 getPorts(entry.getValue().value().next()).contains(port))
1223 .map(Map.Entry::getKey).map(McastStoreKey::mcastIp)
1224 .collect(Collectors.toSet());
1225 }
1226
1227 /**
Pier Luigieba73a02018-01-16 10:47:50 +01001228 * Gets groups which are affected by the device down event.
1229 *
1230 * @param deviceId device going down
1231 * @return a set of multicast IpAddress
1232 */
1233 private Set<IpAddress> getAffectedGroups(DeviceId deviceId) {
1234 return mcastNextObjStore.entrySet().stream()
1235 .filter(entry -> entry.getKey().deviceId().equals(deviceId))
1236 .map(Map.Entry::getKey).map(McastStoreKey::mcastIp)
1237 .collect(Collectors.toSet());
1238 }
1239
1240 /**
Charles Chand55e84d2016-03-30 17:54:24 -07001241 * Gets egress VLAN from McastConfig.
1242 *
1243 * @return egress VLAN or VlanId.NONE if not configured
1244 */
1245 private VlanId egressVlan() {
1246 McastConfig mcastConfig =
1247 srManager.cfgService.getConfig(coreAppId, McastConfig.class);
1248 return (mcastConfig != null) ? mcastConfig.egressVlan() : VlanId.NONE;
1249 }
1250
1251 /**
1252 * Gets assigned VLAN according to the value of egress VLAN.
Charles Chan8d449862016-05-16 18:44:13 -07001253 * If connect point is specified, try to reuse the assigned VLAN on the connect point.
Charles Chand55e84d2016-03-30 17:54:24 -07001254 *
Charles Chan8d449862016-05-16 18:44:13 -07001255 * @param cp connect point; Can be null if not specified
1256 * @return assigned VLAN ID
Charles Chand55e84d2016-03-30 17:54:24 -07001257 */
Charles Chan8d449862016-05-16 18:44:13 -07001258 private VlanId assignedVlan(ConnectPoint cp) {
1259 // Use the egressVlan if it is tagged
1260 if (!egressVlan().equals(VlanId.NONE)) {
1261 return egressVlan();
1262 }
1263 // Reuse unicast VLAN if the port has subnet configured
1264 if (cp != null) {
Charles Chanb4879a52017-10-20 19:09:16 -07001265 VlanId untaggedVlan = srManager.getInternalVlanId(cp);
Charles Chan59cc16d2017-02-02 16:20:42 -08001266 return (untaggedVlan != null) ? untaggedVlan : INTERNAL_VLAN;
Charles Chan8d449862016-05-16 18:44:13 -07001267 }
Charles Chan59cc16d2017-02-02 16:20:42 -08001268 // Use DEFAULT_VLAN if none of the above matches
1269 return SegmentRoutingManager.INTERNAL_VLAN;
Charles Chand55e84d2016-03-30 17:54:24 -07001270 }
Charles Chan2199c302016-04-23 17:36:10 -07001271
1272 /**
Pier Luigi05514fd2018-02-28 17:24:03 +01001273 * Gets assigned VLAN according to the value in the meta.
1274 *
1275 * @param nextObjective nextObjective to analyze
1276 * @return assigned VLAN ID
1277 */
1278 private VlanId assignedVlanFromNext(NextObjective nextObjective) {
1279 return ((VlanIdCriterion) nextObjective.meta().getCriterion(VLAN_VID)).vlanId();
1280 }
1281
1282 /**
Charles Chan2199c302016-04-23 17:36:10 -07001283 * Gets the spine-facing port on ingress device of given multicast group.
1284 *
1285 * @param mcastIp multicast IP
1286 * @return spine-facing port on ingress device
1287 */
1288 private PortNumber ingressTransitPort(IpAddress mcastIp) {
1289 DeviceId ingressDevice = getDevice(mcastIp, McastRole.INGRESS)
1290 .stream().findAny().orElse(null);
1291 if (ingressDevice != null) {
1292 NextObjective nextObj = mcastNextObjStore
1293 .get(new McastStoreKey(mcastIp, ingressDevice)).value();
1294 Set<PortNumber> ports = getPorts(nextObj.next());
1295
1296 for (PortNumber port : ports) {
1297 // Spine-facing port should have no subnet and no xconnect
Pier Luigi96fe0772018-02-28 12:10:50 +01001298 if (srManager.deviceConfiguration() != null &&
1299 srManager.deviceConfiguration().getPortSubnets(ingressDevice, port).isEmpty() &&
Charles Chanfc5c7802016-05-17 13:13:55 -07001300 !srManager.xConnectHandler.hasXConnect(new ConnectPoint(ingressDevice, port))) {
Charles Chan2199c302016-04-23 17:36:10 -07001301 return port;
1302 }
1303 }
1304 }
1305 return null;
1306 }
Jonghwan Hyun42fe1052017-08-25 17:48:36 -07001307
1308 /**
Pier Luigieba73a02018-01-16 10:47:50 +01001309 * Verify if the given device has sinks
1310 * for the multicast group.
1311 *
1312 * @param deviceId device Id
1313 * @param mcastIp multicast IP
1314 * @return true if the device has sink for the group.
1315 * False otherwise.
1316 */
1317 private boolean hasSinks(DeviceId deviceId, IpAddress mcastIp) {
1318 if (deviceId != null) {
1319 // Get the nextobjective
1320 Versioned<NextObjective> versionedNextObj = mcastNextObjStore.get(
1321 new McastStoreKey(mcastIp, deviceId)
1322 );
1323 // If it exists
1324 if (versionedNextObj != null) {
1325 NextObjective nextObj = versionedNextObj.value();
1326 // Retrieves all the output ports
1327 Set<PortNumber> ports = getPorts(nextObj.next());
1328 // Tries to find at least one port that is not spine-facing
1329 for (PortNumber port : ports) {
1330 // Spine-facing port should have no subnet and no xconnect
Pier Luigi96fe0772018-02-28 12:10:50 +01001331 if (srManager.deviceConfiguration() != null &&
1332 (!srManager.deviceConfiguration().getPortSubnets(deviceId, port).isEmpty() ||
Pier Luigieba73a02018-01-16 10:47:50 +01001333 srManager.xConnectHandler.hasXConnect(new ConnectPoint(deviceId, port)))) {
1334 return true;
1335 }
1336 }
1337 }
1338 }
1339 return false;
1340 }
1341
1342 /**
Jonghwan Hyun42fe1052017-08-25 17:48:36 -07001343 * Removes filtering objective for given device and port.
1344 *
1345 * @param deviceId device ID
1346 * @param port ingress port number
1347 * @param assignedVlan assigned VLAN ID
1348 * @param mcastIp multicast IP address
1349 */
1350 private void removeFilterToDevice(DeviceId deviceId, PortNumber port, VlanId assignedVlan, IpAddress mcastIp) {
1351 // Do nothing if the port is configured as suppressed
1352 ConnectPoint connectPoint = new ConnectPoint(deviceId, port);
1353 SegmentRoutingAppConfig appConfig = srManager.cfgService
Pier Luigi96fe0772018-02-28 12:10:50 +01001354 .getConfig(srManager.appId(), SegmentRoutingAppConfig.class);
Jonghwan Hyun42fe1052017-08-25 17:48:36 -07001355 if (appConfig != null && appConfig.suppressSubnet().contains(connectPoint)) {
1356 log.info("Ignore suppressed port {}", connectPoint);
1357 return;
1358 }
1359
Charles Chan50d900c2018-03-02 13:26:22 -08001360 MacAddress routerMac;
1361 try {
1362 routerMac = srManager.deviceConfiguration().getDeviceMac(deviceId);
1363 } catch (DeviceConfigNotFoundException dcnfe) {
1364 log.warn("Fail to push filtering objective since device is not configured. Abort");
1365 return;
1366 }
1367
Jonghwan Hyun42fe1052017-08-25 17:48:36 -07001368 FilteringObjective.Builder filtObjBuilder =
Charles Chan45b69ab2018-03-02 15:41:41 -08001369 filterObjBuilder(port, assignedVlan, mcastIp, routerMac);
Jonghwan Hyun42fe1052017-08-25 17:48:36 -07001370 ObjectiveContext context = new DefaultObjectiveContext(
1371 (objective) -> log.debug("Successfully removed filter on {}/{}, vlan {}",
1372 deviceId, port.toLong(), assignedVlan),
1373 (objective, error) ->
1374 log.warn("Failed to remove filter on {}/{}, vlan {}: {}",
1375 deviceId, port.toLong(), assignedVlan, error));
1376 srManager.flowObjectiveService.filter(deviceId, filtObjBuilder.remove(context));
1377 }
1378
1379 /**
Pier Luigib72201b2018-01-25 16:16:02 +01001380 * Updates filtering objective for given device and port.
1381 * It is called in general when the mcast config has been
1382 * changed.
Jonghwan Hyun42fe1052017-08-25 17:48:36 -07001383 *
1384 * @param deviceId device ID
1385 * @param portNum ingress port number
1386 * @param vlanId assigned VLAN ID
1387 * @param install true to add, false to remove
1388 */
Pier Luigi96fe0772018-02-28 12:10:50 +01001389 public void updateFilterToDevice(DeviceId deviceId, PortNumber portNum,
Jonghwan Hyun42fe1052017-08-25 17:48:36 -07001390 VlanId vlanId, boolean install) {
Pier Luigib72201b2018-01-25 16:16:02 +01001391 lastMcastChange = Instant.now();
1392 mcastLock();
1393 try {
1394 // Iterates over the route and updates properly the filtering objective
1395 // on the source device.
1396 srManager.multicastRouteService.getRoutes().forEach(mcastRoute -> {
1397 ConnectPoint source = srManager.multicastRouteService.fetchSource(mcastRoute);
1398 if (source.deviceId().equals(deviceId) && source.port().equals(portNum)) {
1399 if (install) {
1400 addFilterToDevice(deviceId, portNum, vlanId, mcastRoute.group());
1401 } else {
1402 removeFilterToDevice(deviceId, portNum, vlanId, mcastRoute.group());
1403 }
Jonghwan Hyun42fe1052017-08-25 17:48:36 -07001404 }
Pier Luigib72201b2018-01-25 16:16:02 +01001405 });
1406 } finally {
1407 mcastUnlock();
1408 }
1409 }
1410
Pier Luigi9930da52018-02-02 16:19:11 +01001411 private boolean isLeader(ConnectPoint source) {
1412 // Continue only when we have the mastership on the operation
1413 if (!srManager.mastershipService.isLocalMaster(source.deviceId())) {
1414 // When the source is available we just check the mastership
1415 if (srManager.deviceService.isAvailable(source.deviceId())) {
1416 return false;
1417 }
1418 // Fallback with Leadership service
1419 // source id is used a topic
1420 NodeId leader = srManager.leadershipService.runForLeadership(
1421 source.deviceId().toString()).leaderNodeId();
1422 // Verify if this node is the leader
1423 if (!srManager.clusterService.getLocalNode().id().equals(leader)) {
1424 return false;
1425 }
1426 }
1427 // Done
1428 return true;
1429 }
1430
Pier Luigib72201b2018-01-25 16:16:02 +01001431 /**
1432 * Performs bucket verification operation for all mcast groups in the devices.
1433 * Firstly, it verifies that mcast is stable before trying verification operation.
1434 * Verification consists in creating new nexts with VERIFY operation. Actually,
1435 * the operation is totally delegated to the driver.
1436 */
1437 private final class McastBucketCorrector implements Runnable {
1438
1439 @Override
1440 public void run() {
1441 // Verify if the Mcast has been stable for MCAST_STABLITY_THRESHOLD
1442 if (!isMcastStable()) {
1443 return;
Jonghwan Hyun42fe1052017-08-25 17:48:36 -07001444 }
Pier Luigib72201b2018-01-25 16:16:02 +01001445 // Acquires lock
1446 mcastLock();
1447 try {
1448 // Iterates over the routes and verify the related next objectives
1449 srManager.multicastRouteService.getRoutes()
1450 .stream()
1451 .map(McastRoute::group)
1452 .forEach(mcastIp -> {
1453 log.trace("Running mcast buckets corrector for mcast group: {}",
1454 mcastIp);
1455
1456 // For each group we get current information in the store
1457 // and issue a check of the next objectives in place
1458 DeviceId ingressDevice = getDevice(mcastIp, McastRole.INGRESS)
1459 .stream().findAny().orElse(null);
1460 DeviceId transitDevice = getDevice(mcastIp, McastRole.TRANSIT)
1461 .stream().findAny().orElse(null);
1462 Set<DeviceId> egressDevices = getDevice(mcastIp, McastRole.EGRESS);
Pier Luigib87b8ab2018-03-02 12:53:37 +01001463 // Get source and sinks from Mcast Route Service and warn about errors
Pier Luigib72201b2018-01-25 16:16:02 +01001464 ConnectPoint source = getSource(mcastIp);
Pier Luigib87b8ab2018-03-02 12:53:37 +01001465 Set<ConnectPoint> sinks = getSinks(mcastIp);
Pier Luigib72201b2018-01-25 16:16:02 +01001466
1467 // Do not proceed if ingress device or source of this group are missing
1468 if (ingressDevice == null || source == null) {
Pier Luigib87b8ab2018-03-02 12:53:37 +01001469 if (!sinks.isEmpty()) {
1470 log.warn("Unable to run buckets corrector. " +
1471 "Missing ingress {} or source {} for group {}",
1472 ingressDevice, source, mcastIp);
1473 }
Pier Luigib72201b2018-01-25 16:16:02 +01001474 return;
1475 }
1476
1477 // Continue only when this instance is the master of source device
1478 if (!srManager.mastershipService.isLocalMaster(source.deviceId())) {
1479 log.trace("Unable to run buckets corrector. " +
1480 "Skip {} due to lack of mastership " +
1481 "of the source device {}",
1482 mcastIp, source.deviceId());
1483 return;
1484 }
1485
1486 // Create the set of the devices to be processed
1487 ImmutableSet.Builder<DeviceId> devicesBuilder = ImmutableSet.builder();
1488 devicesBuilder.add(ingressDevice);
1489 if (transitDevice != null) {
1490 devicesBuilder.add(transitDevice);
1491 }
1492 if (!egressDevices.isEmpty()) {
1493 devicesBuilder.addAll(egressDevices);
1494 }
1495 Set<DeviceId> devicesToProcess = devicesBuilder.build();
1496
1497 // Iterate over the devices
1498 devicesToProcess.forEach(deviceId -> {
1499 McastStoreKey currentKey = new McastStoreKey(mcastIp, deviceId);
1500 // If next exists in our store verify related next objective
1501 if (mcastNextObjStore.containsKey(currentKey)) {
1502 NextObjective currentNext = mcastNextObjStore.get(currentKey).value();
1503 // Get current ports
1504 Set<PortNumber> currentPorts = getPorts(currentNext.next());
1505 // Rebuild the next objective
1506 currentNext = nextObjBuilder(
1507 mcastIp,
1508 assignedVlan(deviceId.equals(source.deviceId()) ? source : null),
1509 currentPorts,
1510 currentNext.id()
1511 ).verify();
1512 // Send to the flowobjective service
1513 srManager.flowObjectiveService.next(deviceId, currentNext);
1514 } else {
Pier Luigi83f919b2018-02-15 16:33:08 +01001515 log.warn("Unable to run buckets corrector. " +
Pier Luigib72201b2018-01-25 16:16:02 +01001516 "Missing next for {} and group {}",
1517 deviceId, mcastIp);
1518 }
1519 });
1520
1521 });
1522 } finally {
1523 // Finally, it releases the lock
1524 mcastUnlock();
1525 }
1526
1527 }
Jonghwan Hyun42fe1052017-08-25 17:48:36 -07001528 }
Pier Luigib29144d2018-01-15 18:06:43 +01001529
1530 public Map<McastStoreKey, Integer> getMcastNextIds(IpAddress mcastIp) {
1531 // If mcast ip is present
1532 if (mcastIp != null) {
1533 return mcastNextObjStore.entrySet().stream()
1534 .filter(mcastEntry -> mcastIp.equals(mcastEntry.getKey().mcastIp()))
1535 .collect(Collectors.toMap(Map.Entry::getKey,
1536 entry -> entry.getValue().value().id()));
1537 }
1538 // Otherwise take all the groups
1539 return mcastNextObjStore.entrySet().stream()
1540 .collect(Collectors.toMap(Map.Entry::getKey,
1541 entry -> entry.getValue().value().id()));
1542 }
1543
Pier Luigi96fe0772018-02-28 12:10:50 +01001544 public Map<McastStoreKey, McastRole> getMcastRoles(IpAddress mcastIp) {
Pier Luigib29144d2018-01-15 18:06:43 +01001545 // If mcast ip is present
1546 if (mcastIp != null) {
1547 return mcastRoleStore.entrySet().stream()
1548 .filter(mcastEntry -> mcastIp.equals(mcastEntry.getKey().mcastIp()))
1549 .collect(Collectors.toMap(Map.Entry::getKey,
1550 entry -> entry.getValue().value()));
1551 }
1552 // Otherwise take all the groups
1553 return mcastRoleStore.entrySet().stream()
1554 .collect(Collectors.toMap(Map.Entry::getKey,
1555 entry -> entry.getValue().value()));
1556 }
1557
1558 public Map<ConnectPoint, List<ConnectPoint>> getMcastPaths(IpAddress mcastIp) {
1559 Map<ConnectPoint, List<ConnectPoint>> mcastPaths = Maps.newHashMap();
1560 // Get the source
1561 ConnectPoint source = getSource(mcastIp);
1562 // Source cannot be null, we don't know the starting point
1563 if (source != null) {
1564 // Init steps
1565 Set<DeviceId> visited = Sets.newHashSet();
1566 List<ConnectPoint> currentPath = Lists.newArrayList(
1567 source
1568 );
1569 // Build recursively the mcast paths
1570 buildMcastPaths(source.deviceId(), visited, mcastPaths, currentPath, mcastIp);
1571 }
1572 return mcastPaths;
1573 }
1574
1575 private void buildMcastPaths(DeviceId toVisit, Set<DeviceId> visited,
1576 Map<ConnectPoint, List<ConnectPoint>> mcastPaths,
1577 List<ConnectPoint> currentPath, IpAddress mcastIp) {
1578 // If we have visited the node to visit
1579 // there is a loop
1580 if (visited.contains(toVisit)) {
1581 return;
1582 }
1583 // Visit next-hop
1584 visited.add(toVisit);
1585 McastStoreKey mcastStoreKey = new McastStoreKey(mcastIp, toVisit);
1586 // Looking for next-hops
1587 if (mcastNextObjStore.containsKey(mcastStoreKey)) {
1588 // Build egress connectpoints
1589 NextObjective nextObjective = mcastNextObjStore.get(mcastStoreKey).value();
1590 // Get Ports
1591 Set<PortNumber> outputPorts = getPorts(nextObjective.next());
1592 // Build relative cps
1593 ImmutableSet.Builder<ConnectPoint> cpBuilder = ImmutableSet.builder();
1594 outputPorts.forEach(portNumber -> cpBuilder.add(new ConnectPoint(toVisit, portNumber)));
1595 Set<ConnectPoint> egressPoints = cpBuilder.build();
1596 // Define other variables for the next steps
1597 Set<Link> egressLinks;
1598 List<ConnectPoint> newCurrentPath;
1599 Set<DeviceId> newVisited;
1600 DeviceId newToVisit;
1601 for (ConnectPoint egressPoint : egressPoints) {
1602 egressLinks = srManager.linkService.getEgressLinks(egressPoint);
1603 // If it does not have egress links, stop
1604 if (egressLinks.isEmpty()) {
1605 // Add the connect points to the path
1606 newCurrentPath = Lists.newArrayList(currentPath);
1607 newCurrentPath.add(0, egressPoint);
1608 // Save in the map
1609 mcastPaths.put(egressPoint, newCurrentPath);
1610 } else {
1611 newVisited = Sets.newHashSet(visited);
1612 // Iterate over the egress links for the next hops
1613 for (Link egressLink : egressLinks) {
1614 // Update to visit
1615 newToVisit = egressLink.dst().deviceId();
1616 // Add the connect points to the path
1617 newCurrentPath = Lists.newArrayList(currentPath);
1618 newCurrentPath.add(0, egressPoint);
1619 newCurrentPath.add(0, egressLink.dst());
1620 // Go to the next hop
1621 buildMcastPaths(newToVisit, newVisited, mcastPaths, newCurrentPath, mcastIp);
1622 }
1623 }
1624 }
1625 }
1626 }
1627
Charles Chand55e84d2016-03-30 17:54:24 -07001628}