blob: ab7009f301638c4ef3bfae493e1b0dbb96f054ab [file] [log] [blame]
Thomas Vachuska781d18b2014-10-27 10:31:25 -07001/*
Brian O'Connora09fe5b2017-08-03 21:12:30 -07002 * Copyright 2015-present Open Networking Foundation
Thomas Vachuska781d18b2014-10-27 10:31:25 -07003 *
Thomas Vachuska4f1a60c2014-10-28 13:39:07 -07004 * 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
Thomas Vachuska781d18b2014-10-27 10:31:25 -07007 *
Thomas Vachuska4f1a60c2014-10-28 13:39:07 -07008 * 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.
Thomas Vachuska781d18b2014-10-27 10:31:25 -070015 */
Brian O'Connorabafb502014-12-02 22:26:20 -080016package org.onosproject.openflow.controller.impl;
tom7ef8ff92014-09-17 13:08:06 -070017
Thomas Vachuska6f94ded2015-02-21 14:02:38 -080018import com.google.common.collect.ArrayListMultimap;
Yuta HIGUCHI7b41dc92017-06-22 19:37:06 -070019import com.google.common.collect.ImmutableList;
Thomas Vachuska6f94ded2015-02-21 14:02:38 -080020import com.google.common.collect.Multimap;
tom7ef8ff92014-09-17 13:08:06 -070021import org.apache.felix.scr.annotations.Activate;
22import org.apache.felix.scr.annotations.Component;
23import org.apache.felix.scr.annotations.Deactivate;
Jonathan Hartbbd91d42015-02-27 11:18:04 -080024import org.apache.felix.scr.annotations.Modified;
Charles Chan3b00e1b2015-08-26 23:12:52 +080025import org.apache.felix.scr.annotations.Property;
alshabibb452fd72015-04-22 20:46:20 -070026import org.apache.felix.scr.annotations.Reference;
27import org.apache.felix.scr.annotations.ReferenceCardinality;
tom7ef8ff92014-09-17 13:08:06 -070028import org.apache.felix.scr.annotations.Service;
Charles Chan3b00e1b2015-08-26 23:12:52 +080029import org.onosproject.cfg.ComponentConfigService;
Charles Chanecfdfb72015-11-24 19:05:50 -080030import org.onosproject.core.CoreService;
alshabibb452fd72015-04-22 20:46:20 -070031import org.onosproject.net.driver.DriverService;
Brian O'Connorabafb502014-12-02 22:26:20 -080032import org.onosproject.openflow.controller.DefaultOpenFlowPacketContext;
33import org.onosproject.openflow.controller.Dpid;
34import org.onosproject.openflow.controller.OpenFlowController;
35import org.onosproject.openflow.controller.OpenFlowEventListener;
Jian Lia78cdb22016-04-21 13:03:58 -070036import org.onosproject.openflow.controller.OpenFlowMessageListener;
Brian O'Connorabafb502014-12-02 22:26:20 -080037import org.onosproject.openflow.controller.OpenFlowPacketContext;
38import org.onosproject.openflow.controller.OpenFlowSwitch;
39import org.onosproject.openflow.controller.OpenFlowSwitchListener;
40import org.onosproject.openflow.controller.PacketListener;
41import org.onosproject.openflow.controller.RoleState;
42import org.onosproject.openflow.controller.driver.OpenFlowAgent;
Jonathan Hartbbd91d42015-02-27 11:18:04 -080043import org.osgi.service.component.ComponentContext;
Marc De Leenheerb9311372015-07-09 11:36:49 -070044import org.projectfloodlight.openflow.protocol.OFCalientFlowStatsEntry;
45import org.projectfloodlight.openflow.protocol.OFCalientFlowStatsReply;
Marc De Leenheer631ffce2014-10-28 16:29:07 -070046import org.projectfloodlight.openflow.protocol.OFCircuitPortStatus;
47import org.projectfloodlight.openflow.protocol.OFExperimenter;
alshabib64def642014-12-02 23:27:37 -080048import org.projectfloodlight.openflow.protocol.OFFactories;
Cem Türker3baff672017-10-12 15:09:01 +030049import org.projectfloodlight.openflow.protocol.OFFlowLightweightStatsEntry;
50import org.projectfloodlight.openflow.protocol.OFFlowLightweightStatsReply;
alshabib64def642014-12-02 23:27:37 -080051import org.projectfloodlight.openflow.protocol.OFFlowStatsEntry;
52import org.projectfloodlight.openflow.protocol.OFFlowStatsReply;
sangho6a0bb172015-02-05 12:24:48 -080053import org.projectfloodlight.openflow.protocol.OFGroupDescStatsEntry;
54import org.projectfloodlight.openflow.protocol.OFGroupDescStatsReply;
55import org.projectfloodlight.openflow.protocol.OFGroupStatsEntry;
56import org.projectfloodlight.openflow.protocol.OFGroupStatsReply;
tom7ef8ff92014-09-17 13:08:06 -070057import org.projectfloodlight.openflow.protocol.OFMessage;
58import org.projectfloodlight.openflow.protocol.OFPacketIn;
Marc De Leenheer631ffce2014-10-28 16:29:07 -070059import org.projectfloodlight.openflow.protocol.OFPortDesc;
sangho538108b2015-04-08 14:29:20 -070060import org.projectfloodlight.openflow.protocol.OFPortStatsEntry;
tom7ef8ff92014-09-17 13:08:06 -070061import org.projectfloodlight.openflow.protocol.OFPortStatus;
Cem Türker3baff672017-10-12 15:09:01 +030062import org.projectfloodlight.openflow.protocol.OFQueueStatsEntry;
63import org.projectfloodlight.openflow.protocol.OFQueueStatsReply;
Ayaka Koshibe38594c22014-10-22 13:36:12 -070064import org.projectfloodlight.openflow.protocol.OFStatsReply;
alshabib64def642014-12-02 23:27:37 -080065import org.projectfloodlight.openflow.protocol.OFStatsReplyFlags;
Jian Li2266bff2016-04-21 11:01:25 -070066import org.projectfloodlight.openflow.protocol.OFTableStatsEntry;
67import org.projectfloodlight.openflow.protocol.OFTableStatsReply;
Marc De Leenheerb9311372015-07-09 11:36:49 -070068import org.projectfloodlight.openflow.protocol.action.OFActionOutput;
69import org.projectfloodlight.openflow.protocol.instruction.OFInstruction;
tom7ef8ff92014-09-17 13:08:06 -070070import org.slf4j.Logger;
71import org.slf4j.LoggerFactory;
72
Yuta HIGUCHI7b41dc92017-06-22 19:37:06 -070073import java.util.ArrayList;
Thomas Vachuska6f94ded2015-02-21 14:02:38 -080074import java.util.Collection;
Marc De Leenheerb9311372015-07-09 11:36:49 -070075import java.util.Collections;
Marc De Leenheerb9311372015-07-09 11:36:49 -070076import java.util.List;
Thomas Vachuska6f94ded2015-02-21 14:02:38 -080077import java.util.Set;
Marc De Leenheer8aba62f2017-04-25 14:33:37 -070078import java.util.concurrent.CompletableFuture;
Thomas Vachuska6f94ded2015-02-21 14:02:38 -080079import java.util.concurrent.ConcurrentHashMap;
HIGUCHI Yuta1979f552015-12-28 21:24:26 -080080import java.util.concurrent.ConcurrentMap;
Jonathan Hart6d44d192015-05-11 18:01:19 -070081import java.util.concurrent.CopyOnWriteArraySet;
Thomas Vachuska6f94ded2015-02-21 14:02:38 -080082import java.util.concurrent.ExecutorService;
83import java.util.concurrent.Executors;
84import java.util.concurrent.locks.Lock;
85import java.util.concurrent.locks.ReentrantLock;
Jian Li2266bff2016-04-21 11:01:25 -070086
Thomas Vachuska6f94ded2015-02-21 14:02:38 -080087import static org.onlab.util.Tools.groupedThreads;
Ozge AYAZ60aded22017-06-20 08:35:30 +000088
tom7ef8ff92014-09-17 13:08:06 -070089
90@Component(immediate = true)
91@Service
92public class OpenFlowControllerImpl implements OpenFlowController {
Charles Chanecfdfb72015-11-24 19:05:50 -080093 private static final String APP_ID = "org.onosproject.openflow-base";
Brian O'Connorff278502015-09-22 14:49:52 -070094 private static final String DEFAULT_OFPORT = "6633,6653";
Yuta HIGUCHI8552b172016-07-25 12:10:08 -070095 private static final int DEFAULT_WORKER_THREADS = 0;
Andrea Campanella86e0c562017-11-23 16:38:24 +010096 protected static final String SCHEME = "of";
tom7ef8ff92014-09-17 13:08:06 -070097
98 private static final Logger log =
99 LoggerFactory.getLogger(OpenFlowControllerImpl.class);
100
alshabibb452fd72015-04-22 20:46:20 -0700101 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
Charles Chanecfdfb72015-11-24 19:05:50 -0800102 protected CoreService coreService;
103
104 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
alshabibb452fd72015-04-22 20:46:20 -0700105 protected DriverService driverService;
106
Charles Chan3b00e1b2015-08-26 23:12:52 +0800107 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
108 protected ComponentConfigService cfgService;
109
Brian O'Connore2a399e2015-09-22 15:32:50 -0700110 @Property(name = "openflowPorts", value = DEFAULT_OFPORT,
111 label = "Port numbers (comma separated) used by OpenFlow protocol; default is 6633,6653")
112 private String openflowPorts = DEFAULT_OFPORT;
Charles Chan3b00e1b2015-08-26 23:12:52 +0800113
114 @Property(name = "workerThreads", intValue = DEFAULT_WORKER_THREADS,
Yuta HIGUCHI8552b172016-07-25 12:10:08 -0700115 label = "Number of controller worker threads")
Charles Chan3b00e1b2015-08-26 23:12:52 +0800116 private int workerThreads = DEFAULT_WORKER_THREADS;
117
Ray Milkey7ec0d1b2015-11-13 08:51:35 -0800118 protected ExecutorService executorMsgs =
Andrea Campanelladda93562016-03-02 11:08:12 -0800119 Executors.newFixedThreadPool(32, groupedThreads("onos/of", "event-stats-%d", log));
Pavlin Radoslavov369c6432014-12-03 16:25:14 -0800120
121 private final ExecutorService executorBarrier =
Andrea Campanelladda93562016-03-02 11:08:12 -0800122 Executors.newFixedThreadPool(4, groupedThreads("onos/of", "event-barrier-%d", log));
alshabib8f1cf4a2014-09-17 14:44:48 -0700123
Prince Pereirae7798032016-07-08 16:31:58 +0530124 //Separate executor thread for handling error messages and barrier replies for same failed
125 // transactions to avoid context switching of thread
126 protected ExecutorService executorErrorMsgs =
127 Executors.newSingleThreadExecutor(groupedThreads("onos/of", "event-error-msg-%d", log));
128
129 //concurrent hashmap to track failed transactions
130 protected ConcurrentMap<Long, Boolean> errorMsgs =
131 new ConcurrentHashMap<>();
HIGUCHI Yuta1979f552015-12-28 21:24:26 -0800132 protected ConcurrentMap<Dpid, OpenFlowSwitch> connectedSwitches =
Jonathan Hart6d44d192015-05-11 18:01:19 -0700133 new ConcurrentHashMap<>();
HIGUCHI Yuta1979f552015-12-28 21:24:26 -0800134 protected ConcurrentMap<Dpid, OpenFlowSwitch> activeMasterSwitches =
Jonathan Hart6d44d192015-05-11 18:01:19 -0700135 new ConcurrentHashMap<>();
HIGUCHI Yuta1979f552015-12-28 21:24:26 -0800136 protected ConcurrentMap<Dpid, OpenFlowSwitch> activeEqualSwitches =
Jonathan Hart6d44d192015-05-11 18:01:19 -0700137 new ConcurrentHashMap<>();
tom7ef8ff92014-09-17 13:08:06 -0700138
Marc De Leenheer8aba62f2017-04-25 14:33:37 -0700139 // Key: dpid, value: map with key: long (XID), value: completable future
140 protected ConcurrentMap<Dpid, ConcurrentMap<Long, CompletableFuture<OFMessage>>> responses =
141 new ConcurrentHashMap<>();
142
tom7ef8ff92014-09-17 13:08:06 -0700143 protected OpenFlowSwitchAgent agent = new OpenFlowSwitchAgent();
Jonathan Hart6d44d192015-05-11 18:01:19 -0700144 protected Set<OpenFlowSwitchListener> ofSwitchListener = new CopyOnWriteArraySet<>();
tom7ef8ff92014-09-17 13:08:06 -0700145
146 protected Multimap<Integer, PacketListener> ofPacketListener =
147 ArrayListMultimap.create();
148
Jonathan Hart6d44d192015-05-11 18:01:19 -0700149 protected Set<OpenFlowEventListener> ofEventListener = new CopyOnWriteArraySet<>();
tom7ef8ff92014-09-17 13:08:06 -0700150
Jian Lia78cdb22016-04-21 13:03:58 -0700151 protected Set<OpenFlowMessageListener> ofMessageListener = new CopyOnWriteArraySet<>();
Jian Li28247b52016-01-07 17:24:15 -0800152
sangho6a0bb172015-02-05 12:24:48 -0800153 protected Multimap<Dpid, OFFlowStatsEntry> fullFlowStats =
154 ArrayListMultimap.create();
155
Cem Türker3baff672017-10-12 15:09:01 +0300156 protected Multimap<Dpid, OFFlowLightweightStatsEntry> fullFlowLightweightStats =
157 ArrayListMultimap.create();
158
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700159 protected Multimap<Dpid, OFTableStatsEntry> fullTableStats =
160 ArrayListMultimap.create();
161
sangho6a0bb172015-02-05 12:24:48 -0800162 protected Multimap<Dpid, OFGroupStatsEntry> fullGroupStats =
163 ArrayListMultimap.create();
164
165 protected Multimap<Dpid, OFGroupDescStatsEntry> fullGroupDescStats =
alshabib64def642014-12-02 23:27:37 -0800166 ArrayListMultimap.create();
167
Yuta HIGUCHI7b41dc92017-06-22 19:37:06 -0700168 // deprecated in 1.11.0, no longer referenced from anywhere
169 @Deprecated
sangho538108b2015-04-08 14:29:20 -0700170 protected Multimap<Dpid, OFPortStatsEntry> fullPortStats =
171 ArrayListMultimap.create();
172
Ozge AYAZ60aded22017-06-20 08:35:30 +0000173 protected Multimap<Dpid, OFQueueStatsEntry> fullQueueStats =
174 ArrayListMultimap.create();
175
tom7ef8ff92014-09-17 13:08:06 -0700176 private final Controller ctrl = new Controller();
177
178 @Activate
Jonathan Hartbbd91d42015-02-27 11:18:04 -0800179 public void activate(ComponentContext context) {
Andrea Campanella3556f362016-04-28 15:18:10 -0700180 coreService.registerApplication(APP_ID, this::cleanup);
Charles Chan3b00e1b2015-08-26 23:12:52 +0800181 cfgService.registerProperties(getClass());
Brian O'Connorff278502015-09-22 14:49:52 -0700182 ctrl.setConfigParams(context.getProperties());
alshabibb452fd72015-04-22 20:46:20 -0700183 ctrl.start(agent, driverService);
tom7ef8ff92014-09-17 13:08:06 -0700184 }
185
Andrea Campanella3556f362016-04-28 15:18:10 -0700186 private void cleanup() {
187 // Close listening channel and all OF channels. Clean information about switches
188 // before deactivating
Charles Chanecfdfb72015-11-24 19:05:50 -0800189 ctrl.stop();
190 connectedSwitches.values().forEach(OpenFlowSwitch::disconnectSwitch);
Andrea Campanella3556f362016-04-28 15:18:10 -0700191 connectedSwitches.clear();
192 activeMasterSwitches.clear();
193 activeEqualSwitches.clear();
Charles Chanecfdfb72015-11-24 19:05:50 -0800194 }
195
tom7ef8ff92014-09-17 13:08:06 -0700196 @Deactivate
197 public void deactivate() {
Thiago Santos61725402016-08-05 17:58:56 -0300198 cleanup();
Charles Chan3b00e1b2015-08-26 23:12:52 +0800199 cfgService.unregisterProperties(getClass(), false);
tom7ef8ff92014-09-17 13:08:06 -0700200 }
201
Jonathan Hartbbd91d42015-02-27 11:18:04 -0800202 @Modified
203 public void modified(ComponentContext context) {
Charles Chan3b00e1b2015-08-26 23:12:52 +0800204 ctrl.stop();
Brian O'Connorff278502015-09-22 14:49:52 -0700205 ctrl.setConfigParams(context.getProperties());
Charles Chan3b00e1b2015-08-26 23:12:52 +0800206 ctrl.start(agent, driverService);
Jonathan Hartbbd91d42015-02-27 11:18:04 -0800207 }
208
tom7ef8ff92014-09-17 13:08:06 -0700209 @Override
210 public Iterable<OpenFlowSwitch> getSwitches() {
211 return connectedSwitches.values();
212 }
213
214 @Override
215 public Iterable<OpenFlowSwitch> getMasterSwitches() {
216 return activeMasterSwitches.values();
217 }
218
219 @Override
220 public Iterable<OpenFlowSwitch> getEqualSwitches() {
221 return activeEqualSwitches.values();
222 }
223
224 @Override
225 public OpenFlowSwitch getSwitch(Dpid dpid) {
226 return connectedSwitches.get(dpid);
227 }
228
229 @Override
230 public OpenFlowSwitch getMasterSwitch(Dpid dpid) {
231 return activeMasterSwitches.get(dpid);
232 }
233
234 @Override
235 public OpenFlowSwitch getEqualSwitch(Dpid dpid) {
236 return activeEqualSwitches.get(dpid);
237 }
238
239 @Override
240 public void addListener(OpenFlowSwitchListener listener) {
alshabib8f1cf4a2014-09-17 14:44:48 -0700241 if (!ofSwitchListener.contains(listener)) {
242 this.ofSwitchListener.add(listener);
tom7ef8ff92014-09-17 13:08:06 -0700243 }
244 }
245
246 @Override
247 public void removeListener(OpenFlowSwitchListener listener) {
alshabib8f1cf4a2014-09-17 14:44:48 -0700248 this.ofSwitchListener.remove(listener);
tom7ef8ff92014-09-17 13:08:06 -0700249 }
250
251 @Override
Jian Lia78cdb22016-04-21 13:03:58 -0700252 public void addMessageListener(OpenFlowMessageListener listener) {
253 ofMessageListener.add(listener);
254 }
255
256 @Override
257 public void removeMessageListener(OpenFlowMessageListener listener) {
258 ofMessageListener.remove(listener);
259 }
260
261 @Override
tom7ef8ff92014-09-17 13:08:06 -0700262 public void addPacketListener(int priority, PacketListener listener) {
263 ofPacketListener.put(priority, listener);
264 }
265
266 @Override
267 public void removePacketListener(PacketListener listener) {
268 ofPacketListener.values().remove(listener);
269 }
270
271 @Override
alshabibeec3a062014-09-17 18:01:26 -0700272 public void addEventListener(OpenFlowEventListener listener) {
273 ofEventListener.add(listener);
274 }
275
276 @Override
277 public void removeEventListener(OpenFlowEventListener listener) {
278 ofEventListener.remove(listener);
279 }
280
281 @Override
tom7ef8ff92014-09-17 13:08:06 -0700282 public void write(Dpid dpid, OFMessage msg) {
283 this.getSwitch(dpid).sendMsg(msg);
284 }
285
286 @Override
Marc De Leenheer8aba62f2017-04-25 14:33:37 -0700287 public CompletableFuture<OFMessage> writeResponse(Dpid dpid, OFMessage msg) {
288 write(dpid, msg);
289
290 ConcurrentMap<Long, CompletableFuture<OFMessage>> xids =
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700291 responses.computeIfAbsent(dpid, k -> new ConcurrentHashMap<>());
Marc De Leenheer8aba62f2017-04-25 14:33:37 -0700292
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700293 CompletableFuture<OFMessage> future = new CompletableFuture<>();
Marc De Leenheer8aba62f2017-04-25 14:33:37 -0700294 xids.put(msg.getXid(), future);
295
296 return future;
297 }
298
299 @Override
tom7ef8ff92014-09-17 13:08:06 -0700300 public void processPacket(Dpid dpid, OFMessage msg) {
sangyun-han69ed4462016-07-27 12:10:12 +0900301 OpenFlowSwitch sw = this.getSwitch(dpid);
Laszlo Pappb68fe7e2017-11-24 17:06:59 +0000302 if (log.isTraceEnabled()) {
303 log.trace("Processing message from switch {} via openflow: {}", dpid, msg);
304 }
sangyun-han69ed4462016-07-27 12:10:12 +0900305
Marc De Leenheer8aba62f2017-04-25 14:33:37 -0700306 // Check if someone is waiting for this message
307 ConcurrentMap<Long, CompletableFuture<OFMessage>> xids = responses.get(dpid);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700308 if (xids != null) {
309 CompletableFuture<OFMessage> future = xids.remove(msg.getXid());
310 if (future != null) {
311 future.complete(msg);
312 }
Marc De Leenheer8aba62f2017-04-25 14:33:37 -0700313 }
314
tom7ef8ff92014-09-17 13:08:06 -0700315 switch (msg.getType()) {
316 case PORT_STATUS:
alshabib8f1cf4a2014-09-17 14:44:48 -0700317 for (OpenFlowSwitchListener l : ofSwitchListener) {
tom7ef8ff92014-09-17 13:08:06 -0700318 l.portChanged(dpid, (OFPortStatus) msg);
319 }
320 break;
Ayaka Koshibe38594c22014-10-22 13:36:12 -0700321 case FEATURES_REPLY:
322 for (OpenFlowSwitchListener l : ofSwitchListener) {
323 l.switchChanged(dpid);
324 }
325 break;
tom7ef8ff92014-09-17 13:08:06 -0700326 case PACKET_IN:
sangyun-han69ed4462016-07-27 12:10:12 +0900327 if (sw == null) {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700328 log.error("Ignoring PACKET_IN, switch {} is not found", dpid);
sangyun-han69ed4462016-07-27 12:10:12 +0900329 break;
330 }
tom7ef8ff92014-09-17 13:08:06 -0700331 OpenFlowPacketContext pktCtx = DefaultOpenFlowPacketContext
Ozge AYAZ60aded22017-06-20 08:35:30 +0000332 .packetContextFromPacketIn(sw, (OFPacketIn) msg);
tom7ef8ff92014-09-17 13:08:06 -0700333 for (PacketListener p : ofPacketListener.values()) {
334 p.handlePacket(pktCtx);
335 }
336 break;
Pavlin Radoslavov369c6432014-12-03 16:25:14 -0800337 // TODO: Consider using separate threadpool for sensitive messages.
338 // ie. Back to back error could cause us to starve.
339 case FLOW_REMOVED:
Andrea Campanelladda93562016-03-02 11:08:12 -0800340 executorMsgs.execute(new OFMessageHandler(dpid, msg));
Pavlin Radoslavov369c6432014-12-03 16:25:14 -0800341 break;
Prince Pereirae7798032016-07-08 16:31:58 +0530342 case ERROR:
343 log.debug("Received error message from {}: {}", dpid, msg);
344 errorMsgs.putIfAbsent(msg.getXid(), true);
345 executorErrorMsgs.execute(new OFMessageHandler(dpid, msg));
346 break;
Ayaka Koshibe38594c22014-10-22 13:36:12 -0700347 case STATS_REPLY:
Yuta HIGUCHI7b41dc92017-06-22 19:37:06 -0700348 processStatsReply(dpid, (OFStatsReply) msg);
alshabib64def642014-12-02 23:27:37 -0800349 break;
alshabib8f1cf4a2014-09-17 14:44:48 -0700350 case BARRIER_REPLY:
Prince Pereirae7798032016-07-08 16:31:58 +0530351 if (errorMsgs.containsKey(msg.getXid())) {
352 //To make oferror msg handling and corresponding barrier reply serialized,
353 // executorErrorMsgs is used for both transaction
354 errorMsgs.remove(msg.getXid());
355 executorErrorMsgs.execute(new OFMessageHandler(dpid, msg));
356 } else {
357 executorBarrier.execute(new OFMessageHandler(dpid, msg));
358 }
alshabib8f1cf4a2014-09-17 14:44:48 -0700359 break;
Marc De Leenheer631ffce2014-10-28 16:29:07 -0700360 case EXPERIMENTER:
sangyun-han69ed4462016-07-27 12:10:12 +0900361 if (sw == null) {
362 log.error("Switch {} is not found", dpid);
363 break;
364 }
Marc De Leenheerb9311372015-07-09 11:36:49 -0700365 long experimenter = ((OFExperimenter) msg).getExperimenter();
366 if (experimenter == 0x748771) {
367 // LINC-OE port stats
Marc De Leenheer631ffce2014-10-28 16:29:07 -0700368 OFCircuitPortStatus circuitPortStatus = (OFCircuitPortStatus) msg;
sangyun-han69ed4462016-07-27 12:10:12 +0900369 OFPortStatus.Builder portStatus = sw.factory().buildPortStatus();
370 OFPortDesc.Builder portDesc = sw.factory().buildPortDesc();
Marc De Leenheer631ffce2014-10-28 16:29:07 -0700371 portDesc.setPortNo(circuitPortStatus.getPortNo())
372 .setHwAddr(circuitPortStatus.getHwAddr())
373 .setName(circuitPortStatus.getName())
374 .setConfig(circuitPortStatus.getConfig())
375 .setState(circuitPortStatus.getState());
376 portStatus.setReason(circuitPortStatus.getReason()).setDesc(portDesc.build());
377 for (OpenFlowSwitchListener l : ofSwitchListener) {
378 l.portChanged(dpid, portStatus.build());
379 }
380 } else {
381 log.warn("Handling experimenter type {} not yet implemented",
382 ((OFExperimenter) msg).getExperimenter(), msg);
383 }
384 break;
tom7ef8ff92014-09-17 13:08:06 -0700385 default:
386 log.warn("Handling message type {} not yet implemented {}",
387 msg.getType(), msg);
388 }
389 }
390
Yuta HIGUCHI7b41dc92017-06-22 19:37:06 -0700391 private void processStatsReply(Dpid dpid, OFStatsReply reply) {
392 switch (reply.getStatsType()) {
393 case QUEUE:
394 Collection<OFQueueStatsEntry> queueStatsEntries = publishQueueStats(dpid, (OFQueueStatsReply) reply);
395 if (queueStatsEntries != null) {
396 OFQueueStatsReply.Builder rep =
397 OFFactories.getFactory(reply.getVersion()).buildQueueStatsReply();
398 rep.setEntries(ImmutableList.copyOf(queueStatsEntries));
399 rep.setXid(reply.getXid());
400 executorMsgs.execute(new OFMessageHandler(dpid, rep.build()));
401 }
402 break;
403
404 case PORT_DESC:
405 for (OpenFlowSwitchListener l : ofSwitchListener) {
406 l.switchChanged(dpid);
407 }
408 break;
409
410 case FLOW:
411 Collection<OFFlowStatsEntry> flowStats = publishFlowStats(dpid, (OFFlowStatsReply) reply);
412 if (flowStats != null) {
413 OFFlowStatsReply.Builder rep =
414 OFFactories.getFactory(reply.getVersion()).buildFlowStatsReply();
415 rep.setEntries(ImmutableList.copyOf(flowStats));
416 rep.setXid(reply.getXid());
417 executorMsgs.execute(new OFMessageHandler(dpid, rep.build()));
418 }
419 break;
Cem Türker3baff672017-10-12 15:09:01 +0300420 case FLOW_LIGHTWEIGHT:
421 Collection<OFFlowLightweightStatsEntry> flowLightweightStats =
422 publishFlowStatsLightweight(dpid, (OFFlowLightweightStatsReply) reply);
423 if (flowLightweightStats != null) {
424 OFFlowLightweightStatsReply.Builder rep =
425 OFFactories.getFactory(reply.getVersion()).buildFlowLightweightStatsReply();
426 rep.setEntries(ImmutableList.copyOf(flowLightweightStats));
427 rep.setXid(reply.getXid());
428 executorMsgs.execute(new OFMessageHandler(dpid, rep.build()));
429 }
430 break;
Yuta HIGUCHI7b41dc92017-06-22 19:37:06 -0700431 case TABLE:
432 Collection<OFTableStatsEntry> tableStats = publishTableStats(dpid, (OFTableStatsReply) reply);
433 if (tableStats != null) {
434 OFTableStatsReply.Builder rep =
435 OFFactories.getFactory(reply.getVersion()).buildTableStatsReply();
436 rep.setEntries(ImmutableList.copyOf(tableStats));
437 executorMsgs.execute(new OFMessageHandler(dpid, rep.build()));
438 }
439 break;
440
441 case GROUP:
442 Collection<OFGroupStatsEntry> groupStats = publishGroupStats(dpid, (OFGroupStatsReply) reply);
443 if (groupStats != null) {
444 OFGroupStatsReply.Builder rep =
445 OFFactories.getFactory(reply.getVersion()).buildGroupStatsReply();
446 rep.setEntries(ImmutableList.copyOf(groupStats));
447 rep.setXid(reply.getXid());
448 executorMsgs.execute(new OFMessageHandler(dpid, rep.build()));
449 }
450 break;
451
452 case GROUP_DESC:
453 Collection<OFGroupDescStatsEntry> groupDescStats = publishGroupDescStats(dpid,
454 (OFGroupDescStatsReply) reply);
455 if (groupDescStats != null) {
456 OFGroupDescStatsReply.Builder rep =
457 OFFactories.getFactory(reply.getVersion()).buildGroupDescStatsReply();
458 rep.setEntries(ImmutableList.copyOf(groupDescStats));
459 rep.setXid(reply.getXid());
460 executorMsgs.execute(new OFMessageHandler(dpid, rep.build()));
461 }
462 break;
463
464 case PORT:
465 executorMsgs.execute(new OFMessageHandler(dpid, reply));
466 break;
467
468 case METER:
469 executorMsgs.execute(new OFMessageHandler(dpid, reply));
470 break;
471
472 case EXPERIMENTER:
473 if (reply instanceof OFCalientFlowStatsReply) {
474 OpenFlowSwitch sw = this.getSwitch(dpid);
475 // Convert Calient flow statistics to regular flow stats
476 // TODO: parse remaining fields such as power levels etc. when we have proper monitoring API
477 if (sw == null) {
478 log.error("Switch {} is not found", dpid);
479 break;
480 }
481 OFFlowStatsReply.Builder fsr = sw.factory().buildFlowStatsReply();
482 List<OFFlowStatsEntry> entries = new ArrayList<>();
483 for (OFCalientFlowStatsEntry entry : ((OFCalientFlowStatsReply) reply).getEntries()) {
484
485 // Single instruction, i.e., output to port
486 OFActionOutput action = sw.factory()
487 .actions()
488 .buildOutput()
489 .setPort(entry.getOutPort())
490 .build();
491 OFInstruction instruction = sw.factory()
492 .instructions()
493 .applyActions(Collections.singletonList(action));
494 OFFlowStatsEntry fs = sw.factory().buildFlowStatsEntry()
495 .setMatch(entry.getMatch())
496 .setTableId(entry.getTableId())
497 .setDurationSec(entry.getDurationSec())
498 .setDurationNsec(entry.getDurationNsec())
499 .setPriority(entry.getPriority())
500 .setIdleTimeout(entry.getIdleTimeout())
501 .setHardTimeout(entry.getHardTimeout())
502 .setFlags(entry.getFlags())
503 .setCookie(entry.getCookie())
504 .setInstructions(Collections.singletonList(instruction))
505 .build();
506 entries.add(fs);
507 }
508 fsr.setEntries(entries);
509
510 flowStats = publishFlowStats(dpid, fsr.build());
511 if (flowStats != null) {
512 OFFlowStatsReply.Builder rep =
513 sw.factory().buildFlowStatsReply();
514 rep.setEntries(ImmutableList.copyOf(flowStats));
515 executorMsgs.execute(new OFMessageHandler(dpid, rep.build()));
516 }
517 } else {
518 executorMsgs.execute(new OFMessageHandler(dpid, reply));
519 }
520 break;
521 default:
522 log.warn("Discarding unknown stats reply type {}", reply.getStatsType());
523 break;
524 }
525 }
526
sangho6a0bb172015-02-05 12:24:48 -0800527 private synchronized Collection<OFFlowStatsEntry> publishFlowStats(Dpid dpid,
528 OFFlowStatsReply reply) {
alshabib64def642014-12-02 23:27:37 -0800529 //TODO: Get rid of synchronized
sangho6a0bb172015-02-05 12:24:48 -0800530 fullFlowStats.putAll(dpid, reply.getEntries());
alshabib64def642014-12-02 23:27:37 -0800531 if (!reply.getFlags().contains(OFStatsReplyFlags.REPLY_MORE)) {
sangho6a0bb172015-02-05 12:24:48 -0800532 return fullFlowStats.removeAll(dpid);
533 }
534 return null;
535 }
536
Cem Türker3baff672017-10-12 15:09:01 +0300537 private synchronized Collection<OFFlowLightweightStatsEntry> publishFlowStatsLightweight(
538 Dpid dpid,
539 OFFlowLightweightStatsReply reply) {
540 //TODO: Get rid of synchronized
541 fullFlowLightweightStats.putAll(dpid, reply.getEntries());
542 if (!reply.getFlags().contains(OFStatsReplyFlags.REPLY_MORE)) {
543 return fullFlowLightweightStats.removeAll(dpid);
544 }
545 return null;
546 }
547
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700548 private synchronized Collection<OFTableStatsEntry> publishTableStats(Dpid dpid,
549 OFTableStatsReply reply) {
550 //TODO: Get rid of synchronized
551 fullTableStats.putAll(dpid, reply.getEntries());
552 if (!reply.getFlags().contains(OFStatsReplyFlags.REPLY_MORE)) {
553 return fullTableStats.removeAll(dpid);
554 }
555 return null;
556 }
557
sangho6a0bb172015-02-05 12:24:48 -0800558 private synchronized Collection<OFGroupStatsEntry> publishGroupStats(Dpid dpid,
559 OFGroupStatsReply reply) {
560 //TODO: Get rid of synchronized
561 fullGroupStats.putAll(dpid, reply.getEntries());
562 if (!reply.getFlags().contains(OFStatsReplyFlags.REPLY_MORE)) {
563 return fullGroupStats.removeAll(dpid);
564 }
565 return null;
566 }
567
568 private synchronized Collection<OFGroupDescStatsEntry> publishGroupDescStats(Dpid dpid,
569 OFGroupDescStatsReply reply) {
570 //TODO: Get rid of synchronized
571 fullGroupDescStats.putAll(dpid, reply.getEntries());
572 if (!reply.getFlags().contains(OFStatsReplyFlags.REPLY_MORE)) {
573 return fullGroupDescStats.removeAll(dpid);
alshabib64def642014-12-02 23:27:37 -0800574 }
575 return null;
576 }
577
Ozge AYAZ60aded22017-06-20 08:35:30 +0000578 private synchronized Collection<OFQueueStatsEntry> publishQueueStats(Dpid dpid, OFQueueStatsReply reply) {
579 fullQueueStats.putAll(dpid, reply.getEntries());
580 if (!reply.getFlags().contains(OFStatsReplyFlags.REPLY_MORE)) {
581 return fullQueueStats.removeAll(dpid);
582 }
583 return null;
584 }
585
tom7ef8ff92014-09-17 13:08:06 -0700586 @Override
587 public void setRole(Dpid dpid, RoleState role) {
Yuta HIGUCHI79cbd1c2014-10-02 16:57:57 -0700588 final OpenFlowSwitch sw = getSwitch(dpid);
589 if (sw == null) {
590 log.debug("Switch not connected. Ignoring setRole({}, {})", dpid, role);
591 return;
592 }
593 sw.setRole(role);
tom7ef8ff92014-09-17 13:08:06 -0700594 }
595
596 /**
597 * Implementation of an OpenFlow Agent which is responsible for
598 * keeping track of connected switches and the state in which
599 * they are.
600 */
601 public class OpenFlowSwitchAgent implements OpenFlowAgent {
602
603 private final Logger log = LoggerFactory.getLogger(OpenFlowSwitchAgent.class);
604 private final Lock switchLock = new ReentrantLock();
605
606 @Override
607 public boolean addConnectedSwitch(Dpid dpid, OpenFlowSwitch sw) {
alshabib9eab22f2014-10-20 17:17:31 -0700608
tom7ef8ff92014-09-17 13:08:06 -0700609 if (connectedSwitches.get(dpid) != null) {
610 log.error("Trying to add connectedSwitch but found a previous "
611 + "value for dpid: {}", dpid);
612 return false;
613 } else {
Yuta HIGUCHIeb3f30b2014-10-22 11:34:49 -0700614 log.info("Added switch {}", dpid);
tom7ef8ff92014-09-17 13:08:06 -0700615 connectedSwitches.put(dpid, sw);
alshabib8f1cf4a2014-09-17 14:44:48 -0700616 for (OpenFlowSwitchListener l : ofSwitchListener) {
tom7ef8ff92014-09-17 13:08:06 -0700617 l.switchAdded(dpid);
618 }
619 return true;
620 }
621 }
622
623 @Override
624 public boolean validActivation(Dpid dpid) {
625 if (connectedSwitches.get(dpid) == null) {
626 log.error("Trying to activate switch but is not in "
627 + "connected switches: dpid {}. Aborting ..",
628 dpid);
629 return false;
630 }
631 if (activeMasterSwitches.get(dpid) != null ||
632 activeEqualSwitches.get(dpid) != null) {
633 log.error("Trying to activate switch but it is already "
634 + "activated: dpid {}. Found in activeMaster: {} "
Ray Milkey6bc43c22015-11-06 13:22:38 -0800635 + "Found in activeEqual: {}. Aborting ..",
636 dpid,
637 (activeMasterSwitches.get(dpid) == null) ? 'N' : 'Y',
638 (activeEqualSwitches.get(dpid) == null) ? 'N' : 'Y');
tom7ef8ff92014-09-17 13:08:06 -0700639 return false;
640 }
641 return true;
642 }
643
644
645 @Override
646 public boolean addActivatedMasterSwitch(Dpid dpid, OpenFlowSwitch sw) {
647 switchLock.lock();
648 try {
649 if (!validActivation(dpid)) {
650 return false;
651 }
652 activeMasterSwitches.put(dpid, sw);
653 return true;
654 } finally {
655 switchLock.unlock();
656 }
657 }
658
659 @Override
660 public boolean addActivatedEqualSwitch(Dpid dpid, OpenFlowSwitch sw) {
661 switchLock.lock();
662 try {
663 if (!validActivation(dpid)) {
664 return false;
665 }
666 activeEqualSwitches.put(dpid, sw);
667 log.info("Added Activated EQUAL Switch {}", dpid);
668 return true;
669 } finally {
670 switchLock.unlock();
671 }
672 }
673
674 @Override
675 public void transitionToMasterSwitch(Dpid dpid) {
676 switchLock.lock();
677 try {
678 if (activeMasterSwitches.containsKey(dpid)) {
679 return;
680 }
681 OpenFlowSwitch sw = activeEqualSwitches.remove(dpid);
682 if (sw == null) {
683 sw = getSwitch(dpid);
684 if (sw == null) {
685 log.error("Transition to master called on sw {}, but switch "
686 + "was not found in controller-cache", dpid);
687 return;
688 }
689 }
690 log.info("Transitioned switch {} to MASTER", dpid);
691 activeMasterSwitches.put(dpid, sw);
692 } finally {
693 switchLock.unlock();
694 }
695 }
696
697
698 @Override
699 public void transitionToEqualSwitch(Dpid dpid) {
700 switchLock.lock();
701 try {
702 if (activeEqualSwitches.containsKey(dpid)) {
703 return;
704 }
705 OpenFlowSwitch sw = activeMasterSwitches.remove(dpid);
706 if (sw == null) {
707 sw = getSwitch(dpid);
708 if (sw == null) {
709 log.error("Transition to equal called on sw {}, but switch "
710 + "was not found in controller-cache", dpid);
711 return;
712 }
713 }
714 log.info("Transitioned switch {} to EQUAL", dpid);
715 activeEqualSwitches.put(dpid, sw);
716 } finally {
717 switchLock.unlock();
718 }
719
720 }
721
722 @Override
723 public void removeConnectedSwitch(Dpid dpid) {
724 connectedSwitches.remove(dpid);
725 OpenFlowSwitch sw = activeMasterSwitches.remove(dpid);
726 if (sw == null) {
Thomas Vachuska3358af22015-05-19 18:40:34 -0700727 log.debug("sw was null for {}", dpid);
tom7ef8ff92014-09-17 13:08:06 -0700728 sw = activeEqualSwitches.remove(dpid);
729 }
alshabib8f1cf4a2014-09-17 14:44:48 -0700730 for (OpenFlowSwitchListener l : ofSwitchListener) {
tom7ef8ff92014-09-17 13:08:06 -0700731 l.switchRemoved(dpid);
732 }
733 }
734
735 @Override
Jian Lia78cdb22016-04-21 13:03:58 -0700736 public void processDownstreamMessage(Dpid dpid, List<OFMessage> m) {
737 for (OpenFlowMessageListener listener : ofMessageListener) {
738 listener.handleOutgoingMessage(dpid, m);
739 }
740 }
741
742
743 @Override
tom7ef8ff92014-09-17 13:08:06 -0700744 public void processMessage(Dpid dpid, OFMessage m) {
745 processPacket(dpid, m);
Jian Lia78cdb22016-04-21 13:03:58 -0700746
747 for (OpenFlowMessageListener listener : ofMessageListener) {
748 listener.handleIncomingMessage(dpid, m);
749 }
tom7ef8ff92014-09-17 13:08:06 -0700750 }
Ayaka Koshibeab91cc42014-09-25 10:20:52 -0700751
752 @Override
Ayaka Koshibe3ef2b0d2014-10-31 13:58:27 -0700753 public void returnRoleReply(Dpid dpid, RoleState requested, RoleState response) {
Ayaka Koshibeab91cc42014-09-25 10:20:52 -0700754 for (OpenFlowSwitchListener l : ofSwitchListener) {
Ayaka Koshibe3ef2b0d2014-10-31 13:58:27 -0700755 l.receivedRoleReply(dpid, requested, response);
Ayaka Koshibeab91cc42014-09-25 10:20:52 -0700756 }
757 }
tom7ef8ff92014-09-17 13:08:06 -0700758 }
759
Jian Li152b8852015-12-07 14:47:25 -0800760 /**
Jian Li2266bff2016-04-21 11:01:25 -0700761 * OpenFlow message handler.
Jian Li152b8852015-12-07 14:47:25 -0800762 */
Ray Milkey7ec0d1b2015-11-13 08:51:35 -0800763 protected final class OFMessageHandler implements Runnable {
alshabib8f1cf4a2014-09-17 14:44:48 -0700764
Ray Milkey9c9cde42018-01-12 14:22:06 -0800765 final OFMessage msg;
766 final Dpid dpid;
alshabib8f1cf4a2014-09-17 14:44:48 -0700767
768 public OFMessageHandler(Dpid dpid, OFMessage msg) {
769 this.msg = msg;
770 this.dpid = dpid;
771 }
772
773 @Override
774 public void run() {
alshabibeec3a062014-09-17 18:01:26 -0700775 for (OpenFlowEventListener listener : ofEventListener) {
alshabib8f1cf4a2014-09-17 14:44:48 -0700776 listener.handleMessage(dpid, msg);
777 }
778 }
alshabib8f1cf4a2014-09-17 14:44:48 -0700779 }
tom7ef8ff92014-09-17 13:08:06 -0700780}