blob: 1c0f531f01eb83ad3e071cb318192813c05df8a7 [file] [log] [blame]
Jonathan Hart472062d2014-04-03 10:56:48 -07001package net.onrc.onos.core.topology;
Jonathan Hart062a2e82014-02-03 09:41:57 -08002
Pavlin Radoslavov018d5332014-02-19 23:08:35 -08003import java.nio.ByteBuffer;
Yuta HIGUCHI8d762e92014-02-12 14:10:25 -08004import java.util.ArrayList;
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07005import java.util.Arrays;
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -08006import java.util.Collection;
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07007import java.util.Comparator;
Pavlin Radoslavov018d5332014-02-19 23:08:35 -08008import java.util.HashMap;
Yuta HIGUCHI8d762e92014-02-12 14:10:25 -08009import java.util.HashSet;
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -080010import java.util.LinkedList;
Yuta HIGUCHI8d762e92014-02-12 14:10:25 -080011import java.util.List;
Pavlin Radoslavov018d5332014-02-19 23:08:35 -080012import java.util.Map;
Yuta HIGUCHI8d762e92014-02-12 14:10:25 -080013import java.util.Set;
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -070014import java.util.TreeSet;
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -080015import java.util.concurrent.BlockingQueue;
Yuta HIGUCHIa536e762014-02-17 21:47:28 -080016import java.util.concurrent.CopyOnWriteArrayList;
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -080017import java.util.concurrent.LinkedBlockingQueue;
Yuta HIGUCHI8d762e92014-02-12 14:10:25 -080018
Yuta HIGUCHIbc67a052014-06-30 10:37:09 -070019import javax.annotation.concurrent.GuardedBy;
20
TeruU28adcc32014-04-15 17:57:35 -070021import net.floodlightcontroller.util.MACAddress;
Jonathan Hart6df90172014-04-03 10:13:11 -070022import net.onrc.onos.core.datagrid.IDatagridService;
23import net.onrc.onos.core.datagrid.IEventChannel;
24import net.onrc.onos.core.datagrid.IEventChannelListener;
TeruU28adcc32014-04-15 17:57:35 -070025import net.onrc.onos.core.datastore.topology.KVDevice;
Jonathan Hart6df90172014-04-03 10:13:11 -070026import net.onrc.onos.core.datastore.topology.KVLink;
27import net.onrc.onos.core.datastore.topology.KVPort;
28import net.onrc.onos.core.datastore.topology.KVSwitch;
Pavlin Radoslavovd4f40372014-07-18 16:58:40 -070029import net.onrc.onos.core.metrics.OnosMetrics;
30import net.onrc.onos.core.metrics.OnosMetrics.MetricsComponent;
31import net.onrc.onos.core.metrics.OnosMetrics.MetricsFeature;
Jonathan Hartdeda0ba2014-04-03 11:14:12 -070032import net.onrc.onos.core.registry.IControllerRegistryService;
Yuta HIGUCHI8f3dfa32014-06-25 00:14:25 -070033import net.onrc.onos.core.util.Dpid;
Jonathan Hart23701d12014-04-03 10:45:48 -070034import net.onrc.onos.core.util.EventEntry;
Yuta HIGUCHIe2a4e172014-07-03 10:50:39 -070035import net.onrc.onos.core.util.PortNumber;
Yuta HIGUCHI5c8cbeb2014-06-27 11:13:48 -070036import net.onrc.onos.core.util.SwitchPort;
Pavlin Radoslavov4eaab992014-07-03 18:39:42 -070037import net.onrc.onos.core.util.serializers.KryoFactory;
Yuta HIGUCHI5c8cbeb2014-06-27 11:13:48 -070038
Jonathan Hart062a2e82014-02-03 09:41:57 -080039import org.slf4j.Logger;
40import org.slf4j.LoggerFactory;
41
Pavlin Radoslavovd4f40372014-07-18 16:58:40 -070042import com.codahale.metrics.Gauge;
43import com.codahale.metrics.Meter;
Pavlin Radoslavov4eaab992014-07-03 18:39:42 -070044import com.esotericsoftware.kryo.Kryo;
45
Yuta HIGUCHI181d34d2014-02-05 15:05:46 -080046/**
Jonathan Harte37e4e22014-05-13 19:12:02 -070047 * The TopologyManager receives topology updates from the southbound discovery
48 * modules and from other ONOS instances. These updates are processed and
49 * applied to the in-memory topology instance.
Ray Milkey269ffb92014-04-03 14:43:30 -070050 * <p/>
Yuta HIGUCHI4bfdd532014-02-07 13:47:36 -080051 * - Maintain Invariant/Relationships between Topology Objects.
Ray Milkey269ffb92014-04-03 14:43:30 -070052 * <p/>
Yuta HIGUCHI765cd0d2014-02-06 12:46:41 -080053 * TODO To be synchronized based on TopologyEvent Notification.
Ray Milkey269ffb92014-04-03 14:43:30 -070054 * <p/>
Yuta HIGUCHIcb951982014-02-11 13:31:44 -080055 * TODO TBD: Caller is expected to maintain parent/child calling order. Parent
Yuta HIGUCHI1c700102014-02-12 16:30:52 -080056 * Object must exist before adding sub component(Add Switch -> Port).
Ray Milkey269ffb92014-04-03 14:43:30 -070057 * <p/>
Yuta HIGUCHI4bfdd532014-02-07 13:47:36 -080058 * TODO TBD: This class may delay the requested change to handle event
59 * re-ordering. e.g.) Link Add came in, but Switch was not there.
Yuta HIGUCHI181d34d2014-02-05 15:05:46 -080060 */
Jonathan Harte37e4e22014-05-13 19:12:02 -070061public class TopologyManager implements TopologyDiscoveryInterface {
Jonathan Hart062a2e82014-02-03 09:41:57 -080062
Yuta HIGUCHI80829d12014-02-05 20:16:56 -080063 private static final Logger log = LoggerFactory
Ray Milkey269ffb92014-04-03 14:43:30 -070064 .getLogger(TopologyManager.class);
Yuta HIGUCHIcd922f42014-02-11 18:59:11 -080065
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -080066 private IEventChannel<byte[], TopologyEvent> eventChannel;
Jonathan Hart10a7e2b2014-02-21 18:30:08 -080067 public static final String EVENT_CHANNEL_NAME = "onos.topology";
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -080068 private EventHandler eventHandler = new EventHandler();
69
weibitf7c31a42014-06-23 16:51:01 -070070 private TopologyDatastore datastore;
Jonathan Harte37e4e22014-05-13 19:12:02 -070071 private final TopologyImpl topology = new TopologyImpl();
Yuta HIGUCHI170229f2014-02-17 15:47:54 -080072 private final IControllerRegistryService registryService;
Pavlin Radoslavov4eaab992014-07-03 18:39:42 -070073 private Kryo kryo = KryoFactory.newKryoObject();
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -070074 private TopologyEventPreprocessor eventPreprocessor;
Pavlin Radoslavov8a44b782014-08-07 12:53:27 -070075 private CopyOnWriteArrayList<ITopologyListener> topologyListeners =
76 new CopyOnWriteArrayList<>();
Yuta HIGUCHI181d34d2014-02-05 15:05:46 -080077
Pavlin Radoslavov706add22014-02-20 12:15:59 -080078 //
Pavlin Radoslavovd4f40372014-07-18 16:58:40 -070079 // Metrics
80 //
81 private static final MetricsComponent METRICS_COMPONENT =
82 OnosMetrics.registerComponent("Topology");
83 private static final MetricsFeature METRICS_FEATURE_EVENT_NOTIFICATION =
84 METRICS_COMPONENT.registerFeature("EventNotification");
85 //
Pavlin Radoslavovc49917c2014-07-23 12:16:29 -070086 // Timestamp of the last Topology event (ms from the Epoch)
87 private volatile long lastEventTimestampEpochMs = 0;
88 private final Gauge<Long> gaugeLastEventTimestampEpochMs =
Pavlin Radoslavovd4f40372014-07-18 16:58:40 -070089 OnosMetrics.registerMetric(METRICS_COMPONENT,
90 METRICS_FEATURE_EVENT_NOTIFICATION,
Pavlin Radoslavovc49917c2014-07-23 12:16:29 -070091 "LastEventTimestamp.EpochMs",
Pavlin Radoslavovd4f40372014-07-18 16:58:40 -070092 new Gauge<Long>() {
93 @Override
94 public Long getValue() {
Pavlin Radoslavovc49917c2014-07-23 12:16:29 -070095 return lastEventTimestampEpochMs;
Pavlin Radoslavovd4f40372014-07-18 16:58:40 -070096 }
97 });
98 // Rate of the Topology events published to the Topology listeners
99 private final Meter listenerEventRate =
100 OnosMetrics.createMeter(METRICS_COMPONENT,
101 METRICS_FEATURE_EVENT_NOTIFICATION,
102 "ListenerEventRate");
103
104 //
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800105 // Local state for keeping track of locally discovered events so we can
106 // cleanup properly when a Switch or Port is removed.
107 //
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700108 // We keep all Port, (incoming) Link and Host events per Switch DPID:
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800109 // - If a switch goes down, we remove all corresponding Port, Link and
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700110 // Host events.
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800111 // - If a port on a switch goes down, we remove all corresponding Link
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700112 // and Host events discovered by this instance.
Yuta HIGUCHIbc67a052014-06-30 10:37:09 -0700113 //
114 // How to handle side-effect of remote events.
115 // - Remote Port Down event -> Link Down
116 // Not handled. (XXX Shouldn't it be removed from discovered.. Map)
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700117 // - Remote Host Added -> lose ownership of Host)
Yuta HIGUCHIbc67a052014-06-30 10:37:09 -0700118 // Not handled. (XXX Shouldn't it be removed from discovered.. Map)
119 //
120 // XXX Domain knowledge based invariant maintenance should be moved to
121 // driver module, since the invariant may be different on optical, etc.
122 //
123 // What happens on leadership change?
Pavlin Radoslavov8a44b782014-08-07 12:53:27 -0700124 // - Probably should: remove from discovered.. Maps, but not send DELETE
125 // events
126 // XXX Switch/Port can be rediscovered by new leader, but Link, Host?
Yuta HIGUCHIbc67a052014-06-30 10:37:09 -0700127 // - Current: There is no way to recognize leadership change?
128 // ZookeeperRegistry.requestControl(long, ControlChangeCallback)
Pavlin Radoslavov8a44b782014-08-07 12:53:27 -0700129 // is the only way to register listener, and it allows only one
130 // listener, which is already used by Controller class.
Yuta HIGUCHIbc67a052014-06-30 10:37:09 -0700131 //
132 // FIXME Replace with concurrent variant.
133 // #removeSwitchDiscoveryEvent(SwitchEvent) runs in different thread.
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800134 //
Yuta HIGUCHI8f3dfa32014-06-25 00:14:25 -0700135 private Map<Dpid, Map<ByteBuffer, PortEvent>> discoveredAddedPortEvents =
Ray Milkey269ffb92014-04-03 14:43:30 -0700136 new HashMap<>();
Yuta HIGUCHI8f3dfa32014-06-25 00:14:25 -0700137 private Map<Dpid, Map<ByteBuffer, LinkEvent>> discoveredAddedLinkEvents =
Ray Milkey269ffb92014-04-03 14:43:30 -0700138 new HashMap<>();
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700139 private Map<Dpid, Map<ByteBuffer, HostEvent>> discoveredAddedHostEvents =
Ray Milkey269ffb92014-04-03 14:43:30 -0700140 new HashMap<>();
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800141
142 //
Pavlin Radoslavov8a44b782014-08-07 12:53:27 -0700143 // Local state for keeping the last ADD Mastership Event entries.
144 // TODO: In the future, we might have to keep this state somewhere else.
145 //
146 private Map<ByteBuffer, MastershipEvent> lastAddMastershipEvents =
147 new HashMap<>();
148
149 //
Pavlin Radoslavov74986ce2014-02-20 13:17:20 -0800150 // Local state for keeping track of the application event notifications
151 //
Yuta HIGUCHIbc67a052014-06-30 10:37:09 -0700152 // - Queue of events, which will be dispatched to local listeners
153 // on next notification.
Yuta HIGUCHI703696c2014-06-25 20:36:45 -0700154
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700155 private List<MastershipEvent> apiAddedMastershipEvents =
156 new LinkedList<>();
157 private List<MastershipEvent> apiRemovedMastershipEvents =
158 new LinkedList<>();
Yuta HIGUCHI703696c2014-06-25 20:36:45 -0700159 private List<SwitchEvent> apiAddedSwitchEvents = new LinkedList<>();
160 private List<SwitchEvent> apiRemovedSwitchEvents = new LinkedList<>();
161 private List<PortEvent> apiAddedPortEvents = new LinkedList<>();
162 private List<PortEvent> apiRemovedPortEvents = new LinkedList<>();
163 private List<LinkEvent> apiAddedLinkEvents = new LinkedList<>();
164 private List<LinkEvent> apiRemovedLinkEvents = new LinkedList<>();
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700165 private List<HostEvent> apiAddedHostEvents = new LinkedList<>();
166 private List<HostEvent> apiRemovedHostEvents = new LinkedList<>();
Pavlin Radoslavov74986ce2014-02-20 13:17:20 -0800167
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800168 /**
169 * Constructor.
170 *
Jonathan Harte37e4e22014-05-13 19:12:02 -0700171 * @param registryService the Registry Service to use.
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800172 */
Pavlin Radoslavov8a44b782014-08-07 12:53:27 -0700173 public TopologyManager(IControllerRegistryService registryService) {
Jonathan Harte37e4e22014-05-13 19:12:02 -0700174 datastore = new TopologyDatastore();
Ray Milkey269ffb92014-04-03 14:43:30 -0700175 this.registryService = registryService;
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700176 this.eventPreprocessor =
177 new TopologyEventPreprocessor(registryService);
Yuta HIGUCHI80829d12014-02-05 20:16:56 -0800178 }
Yuta HIGUCHI181d34d2014-02-05 15:05:46 -0800179
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800180 /**
Jonathan Harte37e4e22014-05-13 19:12:02 -0700181 * Get the Topology.
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800182 *
Jonathan Harte37e4e22014-05-13 19:12:02 -0700183 * @return the Topology.
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800184 */
Jonathan Harte37e4e22014-05-13 19:12:02 -0700185 Topology getTopology() {
186 return topology;
Pavlin Radoslavov6d224ee2014-02-18 16:43:15 -0800187 }
188
Yuta HIGUCHI4bfdd532014-02-07 13:47:36 -0800189 /**
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -0800190 * Event handler class.
191 */
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -0700192 class EventHandler extends Thread implements
Ray Milkey269ffb92014-04-03 14:43:30 -0700193 IEventChannelListener<byte[], TopologyEvent> {
194 private BlockingQueue<EventEntry<TopologyEvent>> topologyEvents =
195 new LinkedBlockingQueue<EventEntry<TopologyEvent>>();
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -0800196
Ray Milkey269ffb92014-04-03 14:43:30 -0700197 /**
198 * Startup processing.
199 */
200 private void startup() {
201 //
202 // TODO: Read all state from the database:
203 //
204 // Collection<EventEntry<TopologyEvent>> collection =
205 // readWholeTopologyFromDB();
206 //
207 // For now, as a shortcut we read it from the datagrid
208 //
Ray Milkey5df613b2014-04-15 10:50:56 -0700209 Collection<TopologyEvent> allTopologyEvents =
Ray Milkey269ffb92014-04-03 14:43:30 -0700210 eventChannel.getAllEntries();
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700211 List<EventEntry<TopologyEvent>> events =
212 new LinkedList<EventEntry<TopologyEvent>>();
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -0800213
Ray Milkey5df613b2014-04-15 10:50:56 -0700214 for (TopologyEvent topologyEvent : allTopologyEvents) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700215 EventEntry<TopologyEvent> eventEntry =
Pavlin Radoslavov8a44b782014-08-07 12:53:27 -0700216 new EventEntry<TopologyEvent>(EventEntry.Type.ENTRY_ADD,
217 topologyEvent);
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700218 events.add(eventEntry);
Ray Milkey269ffb92014-04-03 14:43:30 -0700219 }
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700220 processEvents(events);
Ray Milkey269ffb92014-04-03 14:43:30 -0700221 }
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -0800222
Ray Milkey269ffb92014-04-03 14:43:30 -0700223 /**
224 * Run the thread.
225 */
226 @Override
227 public void run() {
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700228 List<EventEntry<TopologyEvent>> events =
229 new LinkedList<EventEntry<TopologyEvent>>();
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -0800230
Ray Milkey269ffb92014-04-03 14:43:30 -0700231 this.setName("TopologyManager.EventHandler " + this.getId());
232 startup();
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -0800233
Ray Milkey269ffb92014-04-03 14:43:30 -0700234 //
235 // The main loop
236 //
Pavlin Radoslavov8e881a42014-06-24 16:58:07 -0700237 while (true) {
238 try {
239 EventEntry<TopologyEvent> eventEntry =
240 topologyEvents.take();
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700241 events.add(eventEntry);
242 topologyEvents.drainTo(events);
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -0800243
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700244 processEvents(events);
245 events.clear();
Pavlin Radoslavov8e881a42014-06-24 16:58:07 -0700246 } catch (Exception exception) {
247 log.debug("Exception processing Topology Events: ",
248 exception);
Ray Milkey269ffb92014-04-03 14:43:30 -0700249 }
Ray Milkey269ffb92014-04-03 14:43:30 -0700250 }
251 }
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -0800252
Ray Milkey269ffb92014-04-03 14:43:30 -0700253 /**
254 * Process all topology events.
255 *
256 * @param events the events to process.
257 */
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700258 private void processEvents(List<EventEntry<TopologyEvent>> events) {
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700259 //
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700260 // Pre-process the events
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700261 //
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700262 events = eventPreprocessor.processEvents(events);
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -0800263
Ray Milkey269ffb92014-04-03 14:43:30 -0700264 //
Jonathan Harte37e4e22014-05-13 19:12:02 -0700265 // Lock the topology while it is modified
Ray Milkey269ffb92014-04-03 14:43:30 -0700266 //
Jonathan Harte37e4e22014-05-13 19:12:02 -0700267 topology.acquireWriteLock();
Pavlin Radoslavov8ffb8bf2014-02-20 15:34:26 -0800268
Ray Milkey269ffb92014-04-03 14:43:30 -0700269 try {
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700270 // Apply the events
Ray Milkey269ffb92014-04-03 14:43:30 -0700271 //
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700272 // NOTE: The events are suppose to be in the proper order
273 // to naturally build and update the topology.
Ray Milkey269ffb92014-04-03 14:43:30 -0700274 //
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700275 for (EventEntry<TopologyEvent> event : events) {
276 TopologyEvent topologyEvent = event.eventData();
Yuta HIGUCHI3aca81a2014-02-23 12:41:19 -0800277
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700278 // Get the event itself
279 MastershipEvent mastershipEvent =
280 topologyEvent.getMastershipEvent();
281 SwitchEvent switchEvent = topologyEvent.getSwitchEvent();
282 PortEvent portEvent = topologyEvent.getPortEvent();
283 LinkEvent linkEvent = topologyEvent.getLinkEvent();
284 HostEvent hostEvent = topologyEvent.getHostEvent();
285 boolean wasAdded = false;
Yuta HIGUCHI3aca81a2014-02-23 12:41:19 -0800286
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700287 //
288 // Extract the events
289 //
290 switch (event.eventType()) {
291 case ENTRY_ADD:
292 if (mastershipEvent != null) {
293 wasAdded = addMastershipEvent(mastershipEvent);
294 }
295 if (switchEvent != null) {
296 wasAdded = addSwitch(switchEvent);
297 }
298 if (portEvent != null) {
299 wasAdded = addPort(portEvent);
300 }
301 if (linkEvent != null) {
302 wasAdded = addLink(linkEvent);
303 }
304 if (hostEvent != null) {
305 wasAdded = addHost(hostEvent);
306 }
307 // If the item wasn't added, probably it was reordered
308 if (!wasAdded) {
309 ByteBuffer id = topologyEvent.getIDasByteBuffer();
310 eventPreprocessor.reorderedEvents.put(id, topologyEvent);
311 }
312 break;
313 case ENTRY_REMOVE:
314 if (mastershipEvent != null) {
315 removeMastershipEvent(mastershipEvent);
316 }
317 if (switchEvent != null) {
318 removeSwitch(switchEvent);
319 }
320 if (portEvent != null) {
321 removePort(portEvent);
322 }
323 if (linkEvent != null) {
324 removeLink(linkEvent);
325 }
326 if (hostEvent != null) {
327 removeHost(hostEvent);
328 }
329 break;
330 default:
331 log.error("Unknown topology event {}",
332 event.eventType());
333 }
334 }
Ray Milkey269ffb92014-04-03 14:43:30 -0700335 } finally {
336 //
Jonathan Harte37e4e22014-05-13 19:12:02 -0700337 // Topology modifications completed: Release the lock
Ray Milkey269ffb92014-04-03 14:43:30 -0700338 //
Jonathan Harte37e4e22014-05-13 19:12:02 -0700339 topology.releaseWriteLock();
Ray Milkey269ffb92014-04-03 14:43:30 -0700340 }
Yuta HIGUCHI3aca81a2014-02-23 12:41:19 -0800341
Ray Milkey269ffb92014-04-03 14:43:30 -0700342 //
343 // Dispatch the Topology Notification Events to the applications
344 //
Jonathan Harte37e4e22014-05-13 19:12:02 -0700345 dispatchTopologyEvents();
Ray Milkey269ffb92014-04-03 14:43:30 -0700346 }
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -0800347
Ray Milkey269ffb92014-04-03 14:43:30 -0700348 /**
349 * Receive a notification that an entry is added.
350 *
351 * @param value the value for the entry.
352 */
353 @Override
354 public void entryAdded(TopologyEvent value) {
355 EventEntry<TopologyEvent> eventEntry =
356 new EventEntry<TopologyEvent>(EventEntry.Type.ENTRY_ADD,
357 value);
358 topologyEvents.add(eventEntry);
359 }
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -0800360
Ray Milkey269ffb92014-04-03 14:43:30 -0700361 /**
362 * Receive a notification that an entry is removed.
363 *
364 * @param value the value for the entry.
365 */
366 @Override
367 public void entryRemoved(TopologyEvent value) {
368 EventEntry<TopologyEvent> eventEntry =
369 new EventEntry<TopologyEvent>(EventEntry.Type.ENTRY_REMOVE,
370 value);
371 topologyEvents.add(eventEntry);
372 }
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -0800373
Ray Milkey269ffb92014-04-03 14:43:30 -0700374 /**
375 * Receive a notification that an entry is updated.
376 *
377 * @param value the value for the entry.
378 */
379 @Override
380 public void entryUpdated(TopologyEvent value) {
381 // NOTE: The ADD and UPDATE events are processed in same way
382 entryAdded(value);
383 }
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -0800384 }
385
386 /**
387 * Startup processing.
388 *
389 * @param datagridService the datagrid service to use.
390 */
391 void startup(IDatagridService datagridService) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700392 eventChannel = datagridService.addListener(EVENT_CHANNEL_NAME,
393 eventHandler,
394 byte[].class,
395 TopologyEvent.class);
396 eventHandler.start();
Pavlin Radoslavov721a2e02014-02-14 23:40:14 -0800397 }
398
Pavlin Radoslavov74986ce2014-02-20 13:17:20 -0800399 /**
Pavlin Radoslavov8a44b782014-08-07 12:53:27 -0700400 * Registers a listener for topology events.
401 *
402 * @param listener the listener to register
403 */
404 void registerTopologyListener(ITopologyListener listener) {
405 topologyListeners.addIfAbsent(listener);
406 }
407
408 /**
409 * Deregisters a listener for topology events. The listener will no longer
410 * receive topology events after this call.
411 *
412 * @param listener the listener to deregister
413 */
414 void deregisterTopologyListener(ITopologyListener listener) {
415 topologyListeners.remove(listener);
416 }
417
418 /**
Jonathan Harte37e4e22014-05-13 19:12:02 -0700419 * Dispatch Topology Events to the listeners.
Pavlin Radoslavov74986ce2014-02-20 13:17:20 -0800420 */
Jonathan Harte37e4e22014-05-13 19:12:02 -0700421 private void dispatchTopologyEvents() {
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700422 if (apiAddedMastershipEvents.isEmpty() &&
423 apiRemovedMastershipEvents.isEmpty() &&
424 apiAddedSwitchEvents.isEmpty() &&
Ray Milkey269ffb92014-04-03 14:43:30 -0700425 apiRemovedSwitchEvents.isEmpty() &&
426 apiAddedPortEvents.isEmpty() &&
427 apiRemovedPortEvents.isEmpty() &&
428 apiAddedLinkEvents.isEmpty() &&
429 apiRemovedLinkEvents.isEmpty() &&
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700430 apiAddedHostEvents.isEmpty() &&
431 apiRemovedHostEvents.isEmpty()) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700432 return; // No events to dispatch
433 }
Pavlin Radoslavov74986ce2014-02-20 13:17:20 -0800434
Ray Milkey269ffb92014-04-03 14:43:30 -0700435 if (log.isDebugEnabled()) {
436 //
437 // Debug statements
438 // TODO: Those statements should be removed in the future
439 //
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700440 for (MastershipEvent mastershipEvent : apiAddedMastershipEvents) {
441 log.debug("Dispatch Topology Event: ADDED {}",
442 mastershipEvent);
443 }
444 for (MastershipEvent mastershipEvent : apiRemovedMastershipEvents) {
445 log.debug("Dispatch Topology Event: REMOVED {}",
446 mastershipEvent);
447 }
Ray Milkeyb29e6262014-04-09 16:02:14 -0700448 for (SwitchEvent switchEvent : apiAddedSwitchEvents) {
Jonathan Harte37e4e22014-05-13 19:12:02 -0700449 log.debug("Dispatch Topology Event: ADDED {}", switchEvent);
Ray Milkeyb29e6262014-04-09 16:02:14 -0700450 }
451 for (SwitchEvent switchEvent : apiRemovedSwitchEvents) {
Jonathan Harte37e4e22014-05-13 19:12:02 -0700452 log.debug("Dispatch Topology Event: REMOVED {}", switchEvent);
Ray Milkeyb29e6262014-04-09 16:02:14 -0700453 }
454 for (PortEvent portEvent : apiAddedPortEvents) {
Jonathan Harte37e4e22014-05-13 19:12:02 -0700455 log.debug("Dispatch Topology Event: ADDED {}", portEvent);
Ray Milkeyb29e6262014-04-09 16:02:14 -0700456 }
457 for (PortEvent portEvent : apiRemovedPortEvents) {
Jonathan Harte37e4e22014-05-13 19:12:02 -0700458 log.debug("Dispatch Topology Event: REMOVED {}", portEvent);
Ray Milkeyb29e6262014-04-09 16:02:14 -0700459 }
460 for (LinkEvent linkEvent : apiAddedLinkEvents) {
Jonathan Harte37e4e22014-05-13 19:12:02 -0700461 log.debug("Dispatch Topology Event: ADDED {}", linkEvent);
Ray Milkeyb29e6262014-04-09 16:02:14 -0700462 }
463 for (LinkEvent linkEvent : apiRemovedLinkEvents) {
Jonathan Harte37e4e22014-05-13 19:12:02 -0700464 log.debug("Dispatch Topology Event: REMOVED {}", linkEvent);
Ray Milkeyb29e6262014-04-09 16:02:14 -0700465 }
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700466 for (HostEvent hostEvent : apiAddedHostEvents) {
467 log.debug("Dispatch Topology Event: ADDED {}", hostEvent);
Ray Milkeyb29e6262014-04-09 16:02:14 -0700468 }
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700469 for (HostEvent hostEvent : apiRemovedHostEvents) {
470 log.debug("Dispatch Topology Event: REMOVED {}", hostEvent);
Ray Milkeyb29e6262014-04-09 16:02:14 -0700471 }
Ray Milkey269ffb92014-04-03 14:43:30 -0700472 }
adminbc181552014-02-21 18:36:42 -0800473
Pavlin Radoslavovd4f40372014-07-18 16:58:40 -0700474 //
475 // Update the metrics
476 //
477 long totalEvents =
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700478 apiAddedMastershipEvents.size() + apiRemovedMastershipEvents.size() +
Pavlin Radoslavovd4f40372014-07-18 16:58:40 -0700479 apiAddedSwitchEvents.size() + apiRemovedSwitchEvents.size() +
480 apiAddedPortEvents.size() + apiRemovedPortEvents.size() +
481 apiAddedLinkEvents.size() + apiRemovedLinkEvents.size() +
482 apiAddedHostEvents.size() + apiRemovedHostEvents.size();
483 this.listenerEventRate.mark(totalEvents);
Pavlin Radoslavovc49917c2014-07-23 12:16:29 -0700484 this.lastEventTimestampEpochMs = System.currentTimeMillis();
Pavlin Radoslavovd4f40372014-07-18 16:58:40 -0700485
486 //
Ray Milkey269ffb92014-04-03 14:43:30 -0700487 // Deliver the events
Pavlin Radoslavovd4f40372014-07-18 16:58:40 -0700488 //
Jonathan Harte37e4e22014-05-13 19:12:02 -0700489 for (ITopologyListener listener : this.topologyListeners) {
Pavlin Radoslavov4eaab992014-07-03 18:39:42 -0700490 TopologyEvents events =
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700491 new TopologyEvents(kryo.copy(apiAddedMastershipEvents),
492 kryo.copy(apiRemovedMastershipEvents),
493 kryo.copy(apiAddedSwitchEvents),
Pavlin Radoslavov4eaab992014-07-03 18:39:42 -0700494 kryo.copy(apiRemovedSwitchEvents),
495 kryo.copy(apiAddedPortEvents),
496 kryo.copy(apiRemovedPortEvents),
497 kryo.copy(apiAddedLinkEvents),
498 kryo.copy(apiRemovedLinkEvents),
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700499 kryo.copy(apiAddedHostEvents),
500 kryo.copy(apiRemovedHostEvents));
Pavlin Radoslavov4eaab992014-07-03 18:39:42 -0700501 listener.topologyEvents(events);
Ray Milkey269ffb92014-04-03 14:43:30 -0700502 }
Pavlin Radoslavov74986ce2014-02-20 13:17:20 -0800503
Ray Milkey269ffb92014-04-03 14:43:30 -0700504 //
505 // Cleanup
506 //
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700507 apiAddedMastershipEvents.clear();
508 apiRemovedMastershipEvents.clear();
Ray Milkey269ffb92014-04-03 14:43:30 -0700509 apiAddedSwitchEvents.clear();
510 apiRemovedSwitchEvents.clear();
511 apiAddedPortEvents.clear();
512 apiRemovedPortEvents.clear();
513 apiAddedLinkEvents.clear();
514 apiRemovedLinkEvents.clear();
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700515 apiAddedHostEvents.clear();
516 apiRemovedHostEvents.clear();
Pavlin Radoslavov74986ce2014-02-20 13:17:20 -0800517 }
518
519 /**
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700520 * Mastership updated event.
521 *
522 * @param mastershipEvent the mastership event.
523 */
524 @Override
525 public void putSwitchMastershipEvent(MastershipEvent mastershipEvent) {
526 // Send out notification
527 TopologyEvent topologyEvent =
528 new TopologyEvent(mastershipEvent,
529 registryService.getOnosInstanceId());
530 eventChannel.addEntry(topologyEvent.getID(), topologyEvent);
531 }
532
533 /**
534 * Mastership removed event.
535 *
536 * @param mastershipEvent the mastership event.
537 */
538 @Override
539 public void removeSwitchMastershipEvent(MastershipEvent mastershipEvent) {
540 // Send out notification
Pavlin Radoslavovbb17de22014-08-06 15:34:37 -0700541 TopologyEvent topologyEvent =
542 new TopologyEvent(mastershipEvent,
543 registryService.getOnosInstanceId());
544 eventChannel.removeEntry(topologyEvent.getID());
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700545 }
546
547 /**
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800548 * Switch discovered event.
549 *
550 * @param switchEvent the switch event.
Ray Milkey269ffb92014-04-03 14:43:30 -0700551 * @param portEvents the corresponding port events for the switch.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800552 */
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800553 @Override
Pavlin Radoslavov018d5332014-02-19 23:08:35 -0800554 public void putSwitchDiscoveryEvent(SwitchEvent switchEvent,
Ray Milkey269ffb92014-04-03 14:43:30 -0700555 Collection<PortEvent> portEvents) {
556 if (datastore.addSwitch(switchEvent, portEvents)) {
Jonathan Hart92c819f2014-05-30 10:53:30 -0700557 log.debug("Sending add switch: {}", switchEvent);
Ray Milkey269ffb92014-04-03 14:43:30 -0700558 // Send out notification
Pavlin Radoslavova5637c02014-07-30 15:55:11 -0700559 TopologyEvent topologyEvent =
560 new TopologyEvent(switchEvent,
561 registryService.getOnosInstanceId());
Ray Milkey269ffb92014-04-03 14:43:30 -0700562 eventChannel.addEntry(topologyEvent.getID(), topologyEvent);
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800563
Ray Milkey269ffb92014-04-03 14:43:30 -0700564 // Send out notification for each port
565 for (PortEvent portEvent : portEvents) {
Jonathan Hart92c819f2014-05-30 10:53:30 -0700566 log.debug("Sending add port: {}", portEvent);
Pavlin Radoslavova5637c02014-07-30 15:55:11 -0700567 topologyEvent =
568 new TopologyEvent(portEvent,
569 registryService.getOnosInstanceId());
Ray Milkey269ffb92014-04-03 14:43:30 -0700570 eventChannel.addEntry(topologyEvent.getID(), topologyEvent);
571 }
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800572
Ray Milkey269ffb92014-04-03 14:43:30 -0700573 //
574 // Keep track of the added ports
575 //
576 // Get the old Port Events
577 Map<ByteBuffer, PortEvent> oldPortEvents =
578 discoveredAddedPortEvents.get(switchEvent.getDpid());
Ray Milkeyb29e6262014-04-09 16:02:14 -0700579 if (oldPortEvents == null) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700580 oldPortEvents = new HashMap<>();
Ray Milkeyb29e6262014-04-09 16:02:14 -0700581 }
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800582
Ray Milkey269ffb92014-04-03 14:43:30 -0700583 // Store the new Port Events in the local cache
584 Map<ByteBuffer, PortEvent> newPortEvents = new HashMap<>();
585 for (PortEvent portEvent : portEvents) {
586 ByteBuffer id = portEvent.getIDasByteBuffer();
587 newPortEvents.put(id, portEvent);
588 }
589 discoveredAddedPortEvents.put(switchEvent.getDpid(),
590 newPortEvents);
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800591
Ray Milkey269ffb92014-04-03 14:43:30 -0700592 //
593 // Extract the removed ports
594 //
595 List<PortEvent> removedPortEvents = new LinkedList<>();
596 for (Map.Entry<ByteBuffer, PortEvent> entry : oldPortEvents.entrySet()) {
597 ByteBuffer key = entry.getKey();
598 PortEvent portEvent = entry.getValue();
Ray Milkeyb29e6262014-04-09 16:02:14 -0700599 if (!newPortEvents.containsKey(key)) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700600 removedPortEvents.add(portEvent);
Ray Milkeyb29e6262014-04-09 16:02:14 -0700601 }
Ray Milkey269ffb92014-04-03 14:43:30 -0700602 }
603
604 // Cleanup old removed ports
Ray Milkeyb29e6262014-04-09 16:02:14 -0700605 for (PortEvent portEvent : removedPortEvents) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700606 removePortDiscoveryEvent(portEvent);
Ray Milkeyb29e6262014-04-09 16:02:14 -0700607 }
Ray Milkey269ffb92014-04-03 14:43:30 -0700608 }
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800609 }
610
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800611 /**
Yuta HIGUCHIbc67a052014-06-30 10:37:09 -0700612 * {@inheritDoc}
613 * <p/>
614 * Called by {@link TopologyPublisher.SwitchCleanup} thread.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800615 */
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800616 @Override
Pavlin Radoslavov6ea84a42014-02-19 15:50:01 -0800617 public void removeSwitchDiscoveryEvent(SwitchEvent switchEvent) {
Pavlin Radoslavovbb17de22014-08-06 15:34:37 -0700618 TopologyEvent topologyEvent;
619
Ray Milkey269ffb92014-04-03 14:43:30 -0700620 // Get the old Port Events
621 Map<ByteBuffer, PortEvent> oldPortEvents =
622 discoveredAddedPortEvents.get(switchEvent.getDpid());
Ray Milkeyb29e6262014-04-09 16:02:14 -0700623 if (oldPortEvents == null) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700624 oldPortEvents = new HashMap<>();
Ray Milkeyb29e6262014-04-09 16:02:14 -0700625 }
Pavlin Radoslavov018d5332014-02-19 23:08:35 -0800626
Ray Milkey269ffb92014-04-03 14:43:30 -0700627 if (datastore.deactivateSwitch(switchEvent, oldPortEvents.values())) {
Jonathan Hart92c819f2014-05-30 10:53:30 -0700628 log.debug("Sending remove switch: {}", switchEvent);
Ray Milkey269ffb92014-04-03 14:43:30 -0700629 // Send out notification
Pavlin Radoslavovbb17de22014-08-06 15:34:37 -0700630 topologyEvent =
631 new TopologyEvent(switchEvent,
632 registryService.getOnosInstanceId());
633 eventChannel.removeEntry(topologyEvent.getID());
Pavlin Radoslavov018d5332014-02-19 23:08:35 -0800634
Ray Milkey269ffb92014-04-03 14:43:30 -0700635 //
636 // Send out notification for each port.
637 //
638 // NOTE: We don't use removePortDiscoveryEvent() for the cleanup,
639 // because it will attempt to remove the port from the database,
640 // and the deactiveSwitch() call above already removed all ports.
641 //
Ray Milkeyb29e6262014-04-09 16:02:14 -0700642 for (PortEvent portEvent : oldPortEvents.values()) {
Jonathan Hart92c819f2014-05-30 10:53:30 -0700643 log.debug("Sending remove port:", portEvent);
Pavlin Radoslavovbb17de22014-08-06 15:34:37 -0700644 topologyEvent =
645 new TopologyEvent(portEvent,
646 registryService.getOnosInstanceId());
647 eventChannel.removeEntry(topologyEvent.getID());
Ray Milkeyb29e6262014-04-09 16:02:14 -0700648 }
Ray Milkey269ffb92014-04-03 14:43:30 -0700649 discoveredAddedPortEvents.remove(switchEvent.getDpid());
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800650
Ray Milkey269ffb92014-04-03 14:43:30 -0700651 // Cleanup for each link
652 Map<ByteBuffer, LinkEvent> oldLinkEvents =
653 discoveredAddedLinkEvents.get(switchEvent.getDpid());
654 if (oldLinkEvents != null) {
655 for (LinkEvent linkEvent : new ArrayList<>(oldLinkEvents.values())) {
656 removeLinkDiscoveryEvent(linkEvent);
657 }
658 discoveredAddedLinkEvents.remove(switchEvent.getDpid());
659 }
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800660
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700661 // Cleanup for each host
662 Map<ByteBuffer, HostEvent> oldHostEvents =
663 discoveredAddedHostEvents.get(switchEvent.getDpid());
664 if (oldHostEvents != null) {
665 for (HostEvent hostEvent : new ArrayList<>(oldHostEvents.values())) {
666 removeHostDiscoveryEvent(hostEvent);
Ray Milkey269ffb92014-04-03 14:43:30 -0700667 }
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700668 discoveredAddedHostEvents.remove(switchEvent.getDpid());
Ray Milkey269ffb92014-04-03 14:43:30 -0700669 }
670 }
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800671 }
672
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800673 /**
674 * Port discovered event.
675 *
676 * @param portEvent the port event.
677 */
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800678 @Override
Pavlin Radoslavov6ea84a42014-02-19 15:50:01 -0800679 public void putPortDiscoveryEvent(PortEvent portEvent) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700680 if (datastore.addPort(portEvent)) {
Jonathan Hart92c819f2014-05-30 10:53:30 -0700681 log.debug("Sending add port: {}", portEvent);
Ray Milkey269ffb92014-04-03 14:43:30 -0700682 // Send out notification
Pavlin Radoslavova5637c02014-07-30 15:55:11 -0700683 TopologyEvent topologyEvent =
684 new TopologyEvent(portEvent,
685 registryService.getOnosInstanceId());
Ray Milkey269ffb92014-04-03 14:43:30 -0700686 eventChannel.addEntry(topologyEvent.getID(), topologyEvent);
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800687
Ray Milkey269ffb92014-04-03 14:43:30 -0700688 // Store the new Port Event in the local cache
689 Map<ByteBuffer, PortEvent> oldPortEvents =
690 discoveredAddedPortEvents.get(portEvent.getDpid());
691 if (oldPortEvents == null) {
692 oldPortEvents = new HashMap<>();
693 discoveredAddedPortEvents.put(portEvent.getDpid(),
694 oldPortEvents);
695 }
696 ByteBuffer id = portEvent.getIDasByteBuffer();
697 oldPortEvents.put(id, portEvent);
698 }
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800699 }
700
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800701 /**
702 * Port removed event.
703 *
704 * @param portEvent the port event.
705 */
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800706 @Override
Pavlin Radoslavov6ea84a42014-02-19 15:50:01 -0800707 public void removePortDiscoveryEvent(PortEvent portEvent) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700708 if (datastore.deactivatePort(portEvent)) {
Jonathan Hart92c819f2014-05-30 10:53:30 -0700709 log.debug("Sending remove port: {}", portEvent);
Ray Milkey269ffb92014-04-03 14:43:30 -0700710 // Send out notification
Pavlin Radoslavovbb17de22014-08-06 15:34:37 -0700711 TopologyEvent topologyEvent =
712 new TopologyEvent(portEvent,
713 registryService.getOnosInstanceId());
714 eventChannel.removeEntry(topologyEvent.getID());
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800715
Ray Milkey269ffb92014-04-03 14:43:30 -0700716 // Cleanup the Port Event from the local cache
717 Map<ByteBuffer, PortEvent> oldPortEvents =
718 discoveredAddedPortEvents.get(portEvent.getDpid());
719 if (oldPortEvents != null) {
720 ByteBuffer id = portEvent.getIDasByteBuffer();
721 oldPortEvents.remove(id);
722 }
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800723
Ray Milkey269ffb92014-04-03 14:43:30 -0700724 // Cleanup for the incoming link
725 Map<ByteBuffer, LinkEvent> oldLinkEvents =
726 discoveredAddedLinkEvents.get(portEvent.getDpid());
727 if (oldLinkEvents != null) {
728 for (LinkEvent linkEvent : new ArrayList<>(oldLinkEvents.values())) {
Yuta HIGUCHIbf0a8712014-06-30 18:59:46 -0700729 if (linkEvent.getDst().equals(portEvent.getSwitchPort())) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700730 removeLinkDiscoveryEvent(linkEvent);
731 // XXX If we change our model to allow multiple Link on
732 // a Port, this loop must be fixed to allow continuing.
733 break;
734 }
735 }
736 }
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800737
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700738 // Cleanup for the connected hosts
Ray Milkey269ffb92014-04-03 14:43:30 -0700739 // TODO: The implementation below is probably wrong
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700740 List<HostEvent> removedHostEvents = new LinkedList<>();
741 Map<ByteBuffer, HostEvent> oldHostEvents =
742 discoveredAddedHostEvents.get(portEvent.getDpid());
743 if (oldHostEvents != null) {
744 for (HostEvent hostEvent : new ArrayList<>(oldHostEvents.values())) {
745 for (SwitchPort swp : hostEvent.getAttachmentPoints()) {
Yuta HIGUCHIbf0a8712014-06-30 18:59:46 -0700746 if (swp.equals(portEvent.getSwitchPort())) {
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700747 removedHostEvents.add(hostEvent);
Ray Milkey269ffb92014-04-03 14:43:30 -0700748 }
749 }
750 }
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700751 for (HostEvent hostEvent : removedHostEvents) {
752 removeHostDiscoveryEvent(hostEvent);
Ray Milkeyb29e6262014-04-09 16:02:14 -0700753 }
Ray Milkey269ffb92014-04-03 14:43:30 -0700754 }
755 }
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800756 }
757
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800758 /**
759 * Link discovered event.
760 *
761 * @param linkEvent the link event.
762 */
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800763 @Override
Pavlin Radoslavov6ea84a42014-02-19 15:50:01 -0800764 public void putLinkDiscoveryEvent(LinkEvent linkEvent) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700765 if (datastore.addLink(linkEvent)) {
Jonathan Hart92c819f2014-05-30 10:53:30 -0700766 log.debug("Sending add link: {}", linkEvent);
Ray Milkey269ffb92014-04-03 14:43:30 -0700767 // Send out notification
Pavlin Radoslavova5637c02014-07-30 15:55:11 -0700768 TopologyEvent topologyEvent =
769 new TopologyEvent(linkEvent,
770 registryService.getOnosInstanceId());
Ray Milkey269ffb92014-04-03 14:43:30 -0700771 eventChannel.addEntry(topologyEvent.getID(), topologyEvent);
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800772
Ray Milkey269ffb92014-04-03 14:43:30 -0700773 // Store the new Link Event in the local cache
774 Map<ByteBuffer, LinkEvent> oldLinkEvents =
775 discoveredAddedLinkEvents.get(linkEvent.getDst().getDpid());
776 if (oldLinkEvents == null) {
777 oldLinkEvents = new HashMap<>();
778 discoveredAddedLinkEvents.put(linkEvent.getDst().getDpid(),
779 oldLinkEvents);
780 }
781 ByteBuffer id = linkEvent.getIDasByteBuffer();
782 oldLinkEvents.put(id, linkEvent);
783 }
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800784 }
785
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800786 /**
787 * Link removed event.
788 *
789 * @param linkEvent the link event.
790 */
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800791 @Override
Pavlin Radoslavov6ea84a42014-02-19 15:50:01 -0800792 public void removeLinkDiscoveryEvent(LinkEvent linkEvent) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700793 if (datastore.removeLink(linkEvent)) {
Jonathan Hart92c819f2014-05-30 10:53:30 -0700794 log.debug("Sending remove link: {}", linkEvent);
Ray Milkey269ffb92014-04-03 14:43:30 -0700795 // Send out notification
Pavlin Radoslavovbb17de22014-08-06 15:34:37 -0700796 TopologyEvent topologyEvent =
797 new TopologyEvent(linkEvent,
798 registryService.getOnosInstanceId());
799 eventChannel.removeEntry(topologyEvent.getID());
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800800
Ray Milkey269ffb92014-04-03 14:43:30 -0700801 // Cleanup the Link Event from the local cache
802 Map<ByteBuffer, LinkEvent> oldLinkEvents =
803 discoveredAddedLinkEvents.get(linkEvent.getDst().getDpid());
804 if (oldLinkEvents != null) {
805 ByteBuffer id = linkEvent.getIDasByteBuffer();
806 oldLinkEvents.remove(id);
807 }
808 }
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800809 }
Jonathan Hart22eb9882014-02-11 15:52:59 -0800810
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800811 /**
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700812 * Host discovered event.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800813 *
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700814 * @param hostEvent the host event.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800815 */
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800816 @Override
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700817 public void putHostDiscoveryEvent(HostEvent hostEvent) {
818 if (datastore.addHost(hostEvent)) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700819 // Send out notification
Pavlin Radoslavova5637c02014-07-30 15:55:11 -0700820 TopologyEvent topologyEvent =
821 new TopologyEvent(hostEvent,
822 registryService.getOnosInstanceId());
Ray Milkey269ffb92014-04-03 14:43:30 -0700823 eventChannel.addEntry(topologyEvent.getID(), topologyEvent);
Pavlin Radoslavov8a44b782014-08-07 12:53:27 -0700824 log.debug("Put the host info into the cache of the topology. mac {}",
825 hostEvent.getMac());
Ray Milkey269ffb92014-04-03 14:43:30 -0700826
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700827 // Store the new Host Event in the local cache
Ray Milkey269ffb92014-04-03 14:43:30 -0700828 // TODO: The implementation below is probably wrong
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700829 for (SwitchPort swp : hostEvent.getAttachmentPoints()) {
830 Map<ByteBuffer, HostEvent> oldHostEvents =
831 discoveredAddedHostEvents.get(swp.getDpid());
832 if (oldHostEvents == null) {
833 oldHostEvents = new HashMap<>();
834 discoveredAddedHostEvents.put(swp.getDpid(),
835 oldHostEvents);
Ray Milkey269ffb92014-04-03 14:43:30 -0700836 }
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700837 ByteBuffer id = hostEvent.getIDasByteBuffer();
838 oldHostEvents.put(id, hostEvent);
Ray Milkey269ffb92014-04-03 14:43:30 -0700839 }
840 }
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800841 }
Jonathan Hart22eb9882014-02-11 15:52:59 -0800842
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800843 /**
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700844 * Host removed event.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800845 *
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700846 * @param hostEvent the host event.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800847 */
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800848 @Override
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700849 public void removeHostDiscoveryEvent(HostEvent hostEvent) {
850 if (datastore.removeHost(hostEvent)) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700851 // Send out notification
Pavlin Radoslavovbb17de22014-08-06 15:34:37 -0700852 TopologyEvent topologyEvent =
853 new TopologyEvent(hostEvent,
854 registryService.getOnosInstanceId());
855 eventChannel.removeEntry(topologyEvent.getID());
Pavlin Radoslavov8a44b782014-08-07 12:53:27 -0700856 log.debug("Remove the host info into the cache of the topology. mac {}",
857 hostEvent.getMac());
Pavlin Radoslavov26d83402014-02-20 15:24:30 -0800858
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700859 // Cleanup the Host Event from the local cache
Ray Milkey269ffb92014-04-03 14:43:30 -0700860 // TODO: The implementation below is probably wrong
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -0700861 ByteBuffer id = ByteBuffer.wrap(hostEvent.getID());
862 for (SwitchPort swp : hostEvent.getAttachmentPoints()) {
863 Map<ByteBuffer, HostEvent> oldHostEvents =
864 discoveredAddedHostEvents.get(swp.getDpid());
865 if (oldHostEvents != null) {
866 oldHostEvents.remove(id);
Ray Milkey269ffb92014-04-03 14:43:30 -0700867 }
868 }
869 }
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -0800870 }
Jonathan Hart22eb9882014-02-11 15:52:59 -0800871
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -0700872 //
873 // Methods to update topology replica
874 //
875
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800876 /**
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700877 * Adds Switch Mastership event.
Pavlin Radoslavov695f8952014-07-23 16:57:01 -0700878 *
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700879 * @param mastershipEvent the MastershipEvent to process.
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700880 * @return true if the item was successfully added, otherwise false.
Pavlin Radoslavov695f8952014-07-23 16:57:01 -0700881 */
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700882 @GuardedBy("topology.writeLock")
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700883 private boolean addMastershipEvent(MastershipEvent mastershipEvent) {
884 log.debug("Added Mastership event {}", mastershipEvent);
Pavlin Radoslavov8a44b782014-08-07 12:53:27 -0700885 lastAddMastershipEvents.put(mastershipEvent.getIDasByteBuffer(),
886 mastershipEvent);
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700887 apiAddedMastershipEvents.add(mastershipEvent);
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700888 return true;
Pavlin Radoslavov695f8952014-07-23 16:57:01 -0700889 }
890
891 /**
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700892 * Removes Switch Mastership event.
Pavlin Radoslavov695f8952014-07-23 16:57:01 -0700893 *
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700894 * @param mastershipEvent the MastershipEvent to process.
Pavlin Radoslavov695f8952014-07-23 16:57:01 -0700895 */
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700896 @GuardedBy("topology.writeLock")
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700897 private void removeMastershipEvent(MastershipEvent mastershipEvent) {
898 log.debug("Removed Mastership event {}", mastershipEvent);
Pavlin Radoslavov8a44b782014-08-07 12:53:27 -0700899 lastAddMastershipEvents.remove(mastershipEvent.getIDasByteBuffer());
Pavlin Radoslavovcac157d2014-07-31 13:54:08 -0700900 apiRemovedMastershipEvents.add(mastershipEvent);
Pavlin Radoslavov695f8952014-07-23 16:57:01 -0700901 }
902
903 /**
Yuta HIGUCHI7926ba32014-07-09 11:39:32 -0700904 * Adds a switch to the topology replica.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800905 *
Yuta HIGUCHIbf0a8712014-06-30 18:59:46 -0700906 * @param switchEvent the SwitchEvent with the switch to add.
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700907 * @return true if the item was successfully added, otherwise false.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800908 */
Yuta HIGUCHIbc67a052014-06-30 10:37:09 -0700909 @GuardedBy("topology.writeLock")
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700910 private boolean addSwitch(SwitchEvent switchEvent) {
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -0700911 if (log.isDebugEnabled()) {
912 SwitchEvent sw = topology.getSwitchEvent(switchEvent.getDpid());
913 if (sw != null) {
914 log.debug("Update {}", switchEvent);
915 } else {
916 log.debug("Added {}", switchEvent);
917 }
Ray Milkey269ffb92014-04-03 14:43:30 -0700918 }
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -0700919 topology.putSwitch(switchEvent.freeze());
Ray Milkey269ffb92014-04-03 14:43:30 -0700920 apiAddedSwitchEvents.add(switchEvent);
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700921 return true;
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -0800922 }
923
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800924 /**
Yuta HIGUCHI7926ba32014-07-09 11:39:32 -0700925 * Removes a switch from the topology replica.
Yuta HIGUCHIbf0a8712014-06-30 18:59:46 -0700926 * <p/>
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700927 * It will call {@link #removePort(PortEvent)} for each ports on this
928 * switch.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800929 *
Yuta HIGUCHIbf0a8712014-06-30 18:59:46 -0700930 * @param switchEvent the SwitchEvent with the switch to remove.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800931 */
Yuta HIGUCHIbc67a052014-06-30 10:37:09 -0700932 @GuardedBy("topology.writeLock")
Pavlin Radoslavov74986ce2014-02-20 13:17:20 -0800933 private void removeSwitch(SwitchEvent switchEvent) {
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -0700934 final Dpid dpid = switchEvent.getDpid();
935
936 SwitchEvent swInTopo = topology.getSwitchEvent(dpid);
937 if (swInTopo == null) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700938 log.warn("Switch {} already removed, ignoring", switchEvent);
939 return;
940 }
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -0800941
Ray Milkey269ffb92014-04-03 14:43:30 -0700942 //
943 // Remove all Ports on the Switch
944 //
945 ArrayList<PortEvent> portsToRemove = new ArrayList<>();
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -0700946 for (Port port : topology.getPorts(dpid)) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700947 log.warn("Port {} on Switch {} should be removed prior to removing Switch. Removing Port now.",
948 port, switchEvent);
Yuta HIGUCHIcd14dda2014-07-24 09:57:22 -0700949 PortEvent portEvent = new PortEvent(port.getSwitchPort());
Ray Milkey269ffb92014-04-03 14:43:30 -0700950 portsToRemove.add(portEvent);
951 }
Ray Milkeyb29e6262014-04-09 16:02:14 -0700952 for (PortEvent portEvent : portsToRemove) {
Ray Milkey269ffb92014-04-03 14:43:30 -0700953 removePort(portEvent);
Ray Milkeyb29e6262014-04-09 16:02:14 -0700954 }
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -0800955
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -0700956 log.debug("Removed {}", swInTopo);
957 topology.removeSwitch(dpid);
958 apiRemovedSwitchEvents.add(swInTopo);
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -0800959 }
960
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800961 /**
Yuta HIGUCHI7926ba32014-07-09 11:39:32 -0700962 * Adds a port to the topology replica.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800963 *
Yuta HIGUCHIbf0a8712014-06-30 18:59:46 -0700964 * @param portEvent the PortEvent with the port to add.
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700965 * @return true if the item was successfully added, otherwise false.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800966 */
Yuta HIGUCHIbc67a052014-06-30 10:37:09 -0700967 @GuardedBy("topology.writeLock")
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700968 private boolean addPort(PortEvent portEvent) {
Jonathan Harte37e4e22014-05-13 19:12:02 -0700969 Switch sw = topology.getSwitch(portEvent.getDpid());
Ray Milkey269ffb92014-04-03 14:43:30 -0700970 if (sw == null) {
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700971 // Reordered event
Jonathan Hartf1675202014-05-23 14:59:07 -0700972 log.debug("{} reordered because switch is null", portEvent);
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700973 return false;
Ray Milkey269ffb92014-04-03 14:43:30 -0700974 }
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -0800975
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -0700976 if (log.isDebugEnabled()) {
977 PortEvent port = topology.getPortEvent(portEvent.getSwitchPort());
978 if (port != null) {
979 log.debug("Update {}", portEvent);
980 } else {
981 log.debug("Added {}", portEvent);
982 }
Ray Milkey269ffb92014-04-03 14:43:30 -0700983 }
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -0700984 topology.putPort(portEvent.freeze());
Ray Milkey269ffb92014-04-03 14:43:30 -0700985 apiAddedPortEvents.add(portEvent);
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -0700986 return true;
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -0800987 }
988
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800989 /**
Yuta HIGUCHI7926ba32014-07-09 11:39:32 -0700990 * Removes a port from the topology replica.
Yuta HIGUCHIbf0a8712014-06-30 18:59:46 -0700991 * <p/>
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -0700992 * It will remove attachment points from each hosts on this port
Yuta HIGUCHIbf0a8712014-06-30 18:59:46 -0700993 * and call {@link #removeLink(LinkEvent)} for each links on this port.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800994 *
Yuta HIGUCHIbf0a8712014-06-30 18:59:46 -0700995 * @param portEvent the PortEvent with the port to remove.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -0800996 */
Yuta HIGUCHIbc67a052014-06-30 10:37:09 -0700997 @GuardedBy("topology.writeLock")
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -0800998 private void removePort(PortEvent portEvent) {
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -0700999 SwitchEvent sw = topology.getSwitchEvent(portEvent.getDpid());
Ray Milkey269ffb92014-04-03 14:43:30 -07001000 if (sw == null) {
1001 log.warn("Parent Switch for Port {} already removed, ignoring",
1002 portEvent);
1003 return;
1004 }
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001005
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001006 final SwitchPort switchPort = portEvent.getSwitchPort();
1007 PortEvent portInTopo = topology.getPortEvent(switchPort);
1008 if (portInTopo == null) {
Ray Milkey269ffb92014-04-03 14:43:30 -07001009 log.warn("Port {} already removed, ignoring", portEvent);
1010 return;
1011 }
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001012
Ray Milkey269ffb92014-04-03 14:43:30 -07001013 //
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001014 // Remove all Host attachment points bound to this Port
Ray Milkey269ffb92014-04-03 14:43:30 -07001015 //
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001016 List<HostEvent> hostsToUpdate = new ArrayList<>();
1017 for (Host host : topology.getHosts(switchPort)) {
1018 log.debug("Removing Host {} on Port {}", host, portInTopo);
1019 HostEvent hostEvent = topology.getHostEvent(host.getMacAddress());
1020 hostsToUpdate.add(hostEvent);
Ray Milkey269ffb92014-04-03 14:43:30 -07001021 }
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001022 for (HostEvent hostEvent : hostsToUpdate) {
1023 HostEvent newHostEvent = new HostEvent(hostEvent);
1024 newHostEvent.removeAttachmentPoint(switchPort);
1025 newHostEvent.freeze();
1026
1027 // TODO should this event be fired inside #addHost?
1028 if (newHostEvent.getAttachmentPoints().isEmpty()) {
1029 // No more attachment point left -> remove Host
1030 removeHost(hostEvent);
1031 } else {
1032 // Update Host
1033 addHost(newHostEvent);
1034 }
Ray Milkeyb29e6262014-04-09 16:02:14 -07001035 }
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001036
Ray Milkey269ffb92014-04-03 14:43:30 -07001037 //
1038 // Remove all Links connected to the Port
1039 //
1040 Set<Link> links = new HashSet<>();
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001041 links.addAll(topology.getOutgoingLinks(switchPort));
1042 links.addAll(topology.getIncomingLinks(switchPort));
Ray Milkey269ffb92014-04-03 14:43:30 -07001043 for (Link link : links) {
Ray Milkeyb29e6262014-04-09 16:02:14 -07001044 if (link == null) {
Ray Milkey269ffb92014-04-03 14:43:30 -07001045 continue;
Ray Milkeyb29e6262014-04-09 16:02:14 -07001046 }
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001047 LinkEvent linkEvent = topology.getLinkEvent(link.getLinkTuple());
1048 if (linkEvent != null) {
1049 log.debug("Removing Link {} on Port {}", link, portInTopo);
1050 removeLink(linkEvent);
1051 }
Ray Milkeyb29e6262014-04-09 16:02:14 -07001052 }
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001053
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001054 // Remove the Port from Topology
1055 log.debug("Removed {}", portInTopo);
1056 topology.removePort(switchPort);
Pavlin Radoslavov74986ce2014-02-20 13:17:20 -08001057
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001058 apiRemovedPortEvents.add(portInTopo);
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001059 }
1060
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -08001061 /**
Yuta HIGUCHI7926ba32014-07-09 11:39:32 -07001062 * Adds a link to the topology replica.
Yuta HIGUCHIbf0a8712014-06-30 18:59:46 -07001063 * <p/>
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001064 * It will remove attachment points from each hosts using the same ports.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -08001065 *
Yuta HIGUCHIbf0a8712014-06-30 18:59:46 -07001066 * @param linkEvent the LinkEvent with the link to add.
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001067 * @return true if the item was successfully added, otherwise false.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -08001068 */
Yuta HIGUCHIbc67a052014-06-30 10:37:09 -07001069 @GuardedBy("topology.writeLock")
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001070 private boolean addLink(LinkEvent linkEvent) {
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001071 PortEvent srcPort = topology.getPortEvent(linkEvent.getSrc());
1072 PortEvent dstPort = topology.getPortEvent(linkEvent.getDst());
Ray Milkey269ffb92014-04-03 14:43:30 -07001073 if ((srcPort == null) || (dstPort == null)) {
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001074 // Reordered event
Jonathan Hartf1675202014-05-23 14:59:07 -07001075 log.debug("{} reordered because {} port is null", linkEvent,
1076 (srcPort == null) ? "src" : "dst");
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001077 return false;
Ray Milkey269ffb92014-04-03 14:43:30 -07001078 }
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001079
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001080 //
1081 // XXX domain knowledge: Sanity check: Port cannot have both Link and
1082 // Host.
1083 //
1084 // FIXME: Potentially local replica may not be up-to-date yet due to
1085 // Hazelcast delay.
1086 // FIXME: May need to manage local truth and use them instead.
1087 //
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001088 if (topology.getLinkEvent(linkEvent.getLinkTuple()) == null) {
1089 // Only check for existing Host when adding new Link.
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001090 // Remove all Hosts attached to the ports on both ends
1091
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001092 Set<HostEvent> hostsToUpdate =
1093 new TreeSet<>(new Comparator<HostEvent>() {
1094 // Comparison only using ID(=MAC)
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001095 @Override
1096 public int compare(HostEvent o1, HostEvent o2) {
1097 return Long.compare(o1.getMac().toLong(), o2.getMac().toLong());
1098 }
1099 });
1100
1101 List<SwitchPort> portsToCheck = Arrays.asList(
1102 srcPort.getSwitchPort(),
1103 dstPort.getSwitchPort());
1104
1105 // Enumerate Host which needs to be updated by this Link add event
1106 for (SwitchPort port : portsToCheck) {
1107 for (Host host : topology.getHosts(port)) {
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -07001108 log.error("Host {} on Port {} should have been removed prior to adding Link {}",
1109 host, port, linkEvent);
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001110
Pavlin Radoslavov8a44b782014-08-07 12:53:27 -07001111 HostEvent hostEvent =
1112 topology.getHostEvent(host.getMacAddress());
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001113 hostsToUpdate.add(hostEvent);
Ray Milkey269ffb92014-04-03 14:43:30 -07001114 }
1115 }
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001116 // Remove attachment point from them
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001117 for (HostEvent hostEvent : hostsToUpdate) {
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001118 // Remove port from attachment point and update
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001119 HostEvent newHostEvent = new HostEvent(hostEvent);
1120 newHostEvent.removeAttachmentPoint(srcPort.getSwitchPort());
1121 newHostEvent.removeAttachmentPoint(dstPort.getSwitchPort());
1122 newHostEvent.freeze();
1123
1124 // TODO should this event be fired inside #addHost?
1125 if (newHostEvent.getAttachmentPoints().isEmpty()) {
1126 // No more attachment point left -> remove Host
1127 removeHost(hostEvent);
1128 } else {
1129 // Update Host
1130 addHost(newHostEvent);
1131 }
Ray Milkeyb29e6262014-04-09 16:02:14 -07001132 }
Ray Milkey269ffb92014-04-03 14:43:30 -07001133 }
Pavlin Radoslavov74986ce2014-02-20 13:17:20 -08001134
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001135 if (log.isDebugEnabled()) {
1136 LinkEvent link = topology.getLinkEvent(linkEvent.getLinkTuple());
1137 if (link != null) {
1138 log.debug("Update {}", linkEvent);
1139 } else {
1140 log.debug("Added {}", linkEvent);
1141 }
1142 }
1143 topology.putLink(linkEvent.freeze());
Ray Milkey269ffb92014-04-03 14:43:30 -07001144 apiAddedLinkEvents.add(linkEvent);
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001145 return true;
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001146 }
1147
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -08001148 /**
Yuta HIGUCHI7926ba32014-07-09 11:39:32 -07001149 * Removes a link from the topology replica.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -08001150 *
Yuta HIGUCHIbf0a8712014-06-30 18:59:46 -07001151 * @param linkEvent the LinkEvent with the link to remove.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -08001152 */
Yuta HIGUCHIbc67a052014-06-30 10:37:09 -07001153 @GuardedBy("topology.writeLock")
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001154 private void removeLink(LinkEvent linkEvent) {
Yuta HIGUCHI8f3dfa32014-06-25 00:14:25 -07001155 Port srcPort = topology.getPort(linkEvent.getSrc().getDpid(),
Yuta HIGUCHIb1e2ab72014-06-30 11:01:31 -07001156 linkEvent.getSrc().getPortNumber());
Ray Milkey269ffb92014-04-03 14:43:30 -07001157 if (srcPort == null) {
1158 log.warn("Src Port for Link {} already removed, ignoring",
1159 linkEvent);
1160 return;
1161 }
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001162
Yuta HIGUCHI8f3dfa32014-06-25 00:14:25 -07001163 Port dstPort = topology.getPort(linkEvent.getDst().getDpid(),
Yuta HIGUCHIb1e2ab72014-06-30 11:01:31 -07001164 linkEvent.getDst().getPortNumber());
Ray Milkey269ffb92014-04-03 14:43:30 -07001165 if (dstPort == null) {
1166 log.warn("Dst Port for Link {} already removed, ignoring",
1167 linkEvent);
1168 return;
1169 }
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001170
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001171 LinkEvent linkInTopo = topology.getLinkEvent(linkEvent.getLinkTuple(),
1172 linkEvent.getType());
1173 if (linkInTopo == null) {
1174 log.warn("Link {} already removed, ignoring", linkEvent);
1175 return;
Ray Milkey269ffb92014-04-03 14:43:30 -07001176 }
Jonathan Hart25bd53e2014-04-30 23:44:09 -07001177
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001178 if (log.isDebugEnabled()) {
1179 // only do sanity check on debug level
1180
1181 Link linkIn = dstPort.getIncomingLink(linkEvent.getType());
1182 if (linkIn == null) {
Pavlin Radoslavov8a44b782014-08-07 12:53:27 -07001183 log.warn("Link {} already removed on destination Port",
1184 linkEvent);
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001185 }
1186 Link linkOut = srcPort.getOutgoingLink(linkEvent.getType());
1187 if (linkOut == null) {
1188 log.warn("Link {} already removed on src Port", linkEvent);
1189 }
Jonathan Hart25bd53e2014-04-30 23:44:09 -07001190 }
Pavlin Radoslavov74986ce2014-02-20 13:17:20 -08001191
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001192 log.debug("Removed {}", linkInTopo);
1193 topology.removeLink(linkEvent.getLinkTuple(), linkEvent.getType());
1194 apiRemovedLinkEvents.add(linkInTopo);
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001195 }
1196
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -08001197 /**
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -07001198 * Adds a host to the topology replica.
Ray Milkey269ffb92014-04-03 14:43:30 -07001199 * <p/>
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -07001200 * TODO: Host-related work is incomplete.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -08001201 * TODO: Eventually, we might need to consider reordering
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001202 * or {@link #addLink(LinkEvent)} and {@link #addHost(HostEvent)} events
1203 * on the same port.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -08001204 *
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -07001205 * @param hostEvent the HostEvent with the host to add.
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001206 * @return true if the item was successfully added, otherwise false.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -08001207 */
Yuta HIGUCHIbc67a052014-06-30 10:37:09 -07001208 @GuardedBy("topology.writeLock")
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001209 private boolean addHost(HostEvent hostEvent) {
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001210
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001211 // TODO Decide how to handle update scenario.
1212 // If the new HostEvent has less attachment point compared to
1213 // existing HostEvent, what should the event be?
1214 // - AddHostEvent with some attachment point removed? (current behavior)
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001215
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001216 // create unfrozen copy
1217 // for removing attachment points which already has a link
1218 HostEvent modifiedHostEvent = new HostEvent(hostEvent);
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001219
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001220 // Verify each attachment point
Ray Milkey269ffb92014-04-03 14:43:30 -07001221 boolean attachmentFound = false;
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -07001222 for (SwitchPort swp : hostEvent.getAttachmentPoints()) {
1223 // XXX domain knowledge: Port must exist before Host
Yuta HIGUCHIbf0a8712014-06-30 18:59:46 -07001224 // but this knowledge cannot be pushed down to driver.
1225
Ray Milkey269ffb92014-04-03 14:43:30 -07001226 // Attached Ports must exist
Yuta HIGUCHIb1e2ab72014-06-30 11:01:31 -07001227 Port port = topology.getPort(swp.getDpid(), swp.getPortNumber());
Ray Milkey269ffb92014-04-03 14:43:30 -07001228 if (port == null) {
Pavlin Radoslavov8a44b782014-08-07 12:53:27 -07001229 log.debug("{} reordered because port {} was not there",
1230 hostEvent, swp);
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001231 // Reordered event
1232 return false; // should not continue if re-applying later
Ray Milkey269ffb92014-04-03 14:43:30 -07001233 }
1234 // Attached Ports must not have Link
1235 if (port.getOutgoingLink() != null ||
1236 port.getIncomingLink() != null) {
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001237 log.warn("Link (Out:{},In:{}) exist on the attachment point. "
1238 + "Ignoring this attachmentpoint ({}) from {}.",
1239 port.getOutgoingLink(), port.getIncomingLink(),
1240 swp, modifiedHostEvent);
1241 // FIXME Should either reject, reorder this HostEvent,
1242 // or remove attachment point from given HostEvent
1243 // Removing attachment point from given HostEvent for now.
1244 modifiedHostEvent.removeAttachmentPoint(swp);
Ray Milkey269ffb92014-04-03 14:43:30 -07001245 continue;
1246 }
1247
Ray Milkey269ffb92014-04-03 14:43:30 -07001248 attachmentFound = true;
1249 }
1250
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -07001251 // Update the host in the topology
Ray Milkey269ffb92014-04-03 14:43:30 -07001252 if (attachmentFound) {
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001253 if (modifiedHostEvent.getAttachmentPoints().isEmpty()) {
1254 log.warn("No valid attachment point left. Ignoring."
Pavlin Radoslavov8a44b782014-08-07 12:53:27 -07001255 + "original: {}, modified: {}",
1256 hostEvent, modifiedHostEvent);
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001257 // TODO Should we call #removeHost to trigger remove event?
1258 // only if this call is update.
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001259 return false;
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001260 }
1261
1262 if (log.isDebugEnabled()) {
1263 HostEvent host = topology.getHostEvent(hostEvent.getMac());
1264 if (host != null) {
1265 log.debug("Update {}", modifiedHostEvent);
1266 } else {
1267 log.debug("Added {}", modifiedHostEvent);
1268 }
1269 }
1270 topology.putHost(modifiedHostEvent.freeze());
1271 apiAddedHostEvents.add(modifiedHostEvent);
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001272 return true;
Ray Milkey269ffb92014-04-03 14:43:30 -07001273 }
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001274 return false;
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001275 }
1276
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -08001277 /**
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -07001278 * Removes a host from the topology replica.
Ray Milkey269ffb92014-04-03 14:43:30 -07001279 * <p/>
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -07001280 * TODO: Host-related work is incomplete.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -08001281 *
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -07001282 * @param hostEvent the Host Event with the host to remove.
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -08001283 */
Yuta HIGUCHIbc67a052014-06-30 10:37:09 -07001284 @GuardedBy("topology.writeLock")
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -07001285 private void removeHost(HostEvent hostEvent) {
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001286
1287 final MACAddress mac = hostEvent.getMac();
1288 HostEvent hostInTopo = topology.getHostEvent(mac);
1289 if (hostInTopo == null) {
Yuta HIGUCHIbfc77f02014-07-14 22:50:25 -07001290 log.warn("Host {} already removed, ignoring", hostEvent);
Ray Milkey269ffb92014-04-03 14:43:30 -07001291 return;
1292 }
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001293
Yuta HIGUCHI8b389a72014-07-18 13:50:00 -07001294 log.debug("Removed {}", hostInTopo);
1295 topology.removeHost(mac);
1296 apiRemovedHostEvents.add(hostInTopo);
Pavlin Radoslavov3c9cc552014-02-20 09:58:38 -08001297 }
Jonathan Hart22eb9882014-02-11 15:52:59 -08001298
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -08001299 /**
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -08001300 * Read the whole topology from the database.
1301 *
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001302 * @return a list of EventEntry-encapsulated Topology Events for
Pavlin Radoslavov734ff5a2014-02-26 10:20:43 -08001303 * the whole topology.
1304 */
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001305 private List<EventEntry<TopologyEvent>> readWholeTopologyFromDB() {
1306 List<EventEntry<TopologyEvent>> events =
Ray Milkey269ffb92014-04-03 14:43:30 -07001307 new LinkedList<EventEntry<TopologyEvent>>();
Pavlin Radoslavov018d5332014-02-19 23:08:35 -08001308
Ray Milkey269ffb92014-04-03 14:43:30 -07001309 // XXX May need to clear whole topology first, depending on
1310 // how we initially subscribe to replication events
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -08001311
Ray Milkey269ffb92014-04-03 14:43:30 -07001312 // Add all active switches
1313 for (KVSwitch sw : KVSwitch.getAllSwitches()) {
1314 if (sw.getStatus() != KVSwitch.STATUS.ACTIVE) {
1315 continue;
1316 }
Pavlin Radoslavov018d5332014-02-19 23:08:35 -08001317
Pavlin Radoslavova5637c02014-07-30 15:55:11 -07001318 //
1319 // TODO: Using the local ONOS Instance ID below is incorrect.
1320 // Currently, this code is not used, and it might go away in the
1321 // future.
1322 //
Yuta HIGUCHIe2a4e172014-07-03 10:50:39 -07001323 SwitchEvent switchEvent = new SwitchEvent(new Dpid(sw.getDpid()));
Pavlin Radoslavova5637c02014-07-30 15:55:11 -07001324 TopologyEvent topologyEvent =
1325 new TopologyEvent(switchEvent,
1326 registryService.getOnosInstanceId());
Ray Milkey269ffb92014-04-03 14:43:30 -07001327 EventEntry<TopologyEvent> eventEntry =
1328 new EventEntry<TopologyEvent>(EventEntry.Type.ENTRY_ADD,
1329 topologyEvent);
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001330 events.add(eventEntry);
Ray Milkey269ffb92014-04-03 14:43:30 -07001331 }
Yuta HIGUCHIa536e762014-02-17 21:47:28 -08001332
Ray Milkey269ffb92014-04-03 14:43:30 -07001333 // Add all active ports
1334 for (KVPort p : KVPort.getAllPorts()) {
1335 if (p.getStatus() != KVPort.STATUS.ACTIVE) {
1336 continue;
1337 }
Pavlin Radoslavov018d5332014-02-19 23:08:35 -08001338
Pavlin Radoslavova5637c02014-07-30 15:55:11 -07001339 //
1340 // TODO: Using the local ONOS Instance ID below is incorrect.
1341 // Currently, this code is not used, and it might go away in the
1342 // future.
1343 //
1344 PortEvent portEvent =
1345 new PortEvent(new Dpid(p.getDpid()),
1346 new PortNumber(p.getNumber().shortValue()));
1347 TopologyEvent topologyEvent =
1348 new TopologyEvent(portEvent,
1349 registryService.getOnosInstanceId());
Ray Milkey269ffb92014-04-03 14:43:30 -07001350 EventEntry<TopologyEvent> eventEntry =
1351 new EventEntry<TopologyEvent>(EventEntry.Type.ENTRY_ADD,
1352 topologyEvent);
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001353 events.add(eventEntry);
Ray Milkey269ffb92014-04-03 14:43:30 -07001354 }
Yuta HIGUCHIa536e762014-02-17 21:47:28 -08001355
Pavlin Radoslavova5637c02014-07-30 15:55:11 -07001356 for (KVDevice d : KVDevice.getAllDevices()) {
1357 //
1358 // TODO: Using the local ONOS Instance ID below is incorrect.
1359 // Currently, this code is not used, and it might go away in the
1360 // future.
1361 //
1362 HostEvent devEvent = new HostEvent(MACAddress.valueOf(d.getMac()));
1363 for (byte[] portId : d.getAllPortIds()) {
1364 devEvent.addAttachmentPoint(
1365 new SwitchPort(KVPort.getDpidFromKey(portId),
1366 KVPort.getNumberFromKey(portId)));
1367 }
1368 }
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -08001369
Ray Milkey269ffb92014-04-03 14:43:30 -07001370 for (KVLink l : KVLink.getAllLinks()) {
Pavlin Radoslavova5637c02014-07-30 15:55:11 -07001371 //
1372 // TODO: Using the local ONOS Instance ID below is incorrect.
1373 // Currently, this code is not used, and it might go away in the
1374 // future.
1375 //
Yuta HIGUCHIe2a4e172014-07-03 10:50:39 -07001376 LinkEvent linkEvent = new LinkEvent(
Pavlin Radoslavova5637c02014-07-30 15:55:11 -07001377 new SwitchPort(l.getSrc().dpid, l.getSrc().number),
1378 new SwitchPort(l.getDst().dpid, l.getDst().number));
1379 TopologyEvent topologyEvent =
1380 new TopologyEvent(linkEvent,
1381 registryService.getOnosInstanceId());
Ray Milkey269ffb92014-04-03 14:43:30 -07001382 EventEntry<TopologyEvent> eventEntry =
1383 new EventEntry<TopologyEvent>(EventEntry.Type.ENTRY_ADD,
1384 topologyEvent);
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001385 events.add(eventEntry);
Ray Milkey269ffb92014-04-03 14:43:30 -07001386 }
Pavlin Radoslavov018d5332014-02-19 23:08:35 -08001387
Pavlin Radoslavovd7b792e2014-08-01 02:47:47 -07001388 return events;
Pavlin Radoslavovc1cfde52014-02-19 11:35:29 -08001389 }
Jonathan Hart062a2e82014-02-03 09:41:57 -08001390}