blob: 9f78a11382b1b8a5e997896a246e9cbb2f555399 [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 */
16
Brian O'Connorabafb502014-12-02 22:26:20 -080017package org.onosproject.openflow.controller.impl;
tom7ef8ff92014-09-17 13:08:06 -070018
Anton Chigrin4af4f872019-01-14 17:29:56 +020019import static org.onlab.packet.Ethernet.TYPE_BSN;
20import static org.onlab.packet.Ethernet.TYPE_LLDP;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070021import static org.onlab.util.Tools.groupedThreads;
Anton Chigrin4af4f872019-01-14 17:29:56 +020022import static org.onosproject.openflow.controller.Dpid.uri;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070023
tom7ef8ff92014-09-17 13:08:06 -070024import java.io.IOException;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070025import java.net.InetSocketAddress;
26import java.net.SocketAddress;
Anton Chigrin4af4f872019-01-14 17:29:56 +020027import java.nio.ByteBuffer;
tom7ef8ff92014-09-17 13:08:06 -070028import java.nio.channels.ClosedChannelException;
Brian O'Connorf69e3e32018-05-10 02:25:09 -070029import java.security.cert.Certificate;
tom7ef8ff92014-09-17 13:08:06 -070030import java.util.ArrayList;
31import java.util.Collections;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070032import java.util.Deque;
tom7ef8ff92014-09-17 13:08:06 -070033import java.util.List;
Anton Chigrin4af4f872019-01-14 17:29:56 +020034import java.util.Map;
Yuta HIGUCHI2341e602017-03-08 20:10:08 -080035import java.util.Optional;
Anton Chigrin4af4f872019-01-14 17:29:56 +020036import java.util.Set;
37import java.util.concurrent.LinkedBlockingDeque;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070038import java.util.concurrent.CompletableFuture;
Anton Chigrin4af4f872019-01-14 17:29:56 +020039import java.util.concurrent.ConcurrentHashMap;
tom7ef8ff92014-09-17 13:08:06 -070040import java.util.concurrent.CopyOnWriteArrayList;
Anton Chigrin4af4f872019-01-14 17:29:56 +020041import java.util.concurrent.CopyOnWriteArraySet;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070042import java.util.concurrent.ExecutorService;
43import java.util.concurrent.Executors;
44import java.util.concurrent.Future;
tom7ef8ff92014-09-17 13:08:06 -070045import java.util.concurrent.RejectedExecutionException;
Anton Chigrin4af4f872019-01-14 17:29:56 +020046import java.util.concurrent.atomic.AtomicInteger;
47import java.util.concurrent.locks.Condition;
48import java.util.concurrent.locks.ReentrantLock;
Brian O'Connorf69e3e32018-05-10 02:25:09 -070049
Anton Chigrin4af4f872019-01-14 17:29:56 +020050import org.osgi.service.component.annotations.Reference;
51import org.osgi.service.component.annotations.ReferenceCardinality;
52import org.onlab.osgi.DefaultServiceDirectory;
53import org.onlab.packet.Ethernet;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070054import org.onlab.packet.IpAddress;
Anton Chigrin4af4f872019-01-14 17:29:56 +020055import org.onosproject.net.ConnectPoint;
56import org.onosproject.net.DeviceId;
57import org.onosproject.net.PortNumber;
58import org.onosproject.net.packet.DefaultInboundPacket;
59import org.onosproject.openflow.controller.DefaultOpenFlowPacketContext;
Charles Chan34155e52016-11-30 18:28:11 -080060import org.onosproject.openflow.controller.Dpid;
Anton Chigrin4af4f872019-01-14 17:29:56 +020061import org.onosproject.openflow.controller.OpenFlowClassifier;
62import org.onosproject.openflow.controller.OpenFlowPacketContext;
63import org.onosproject.openflow.controller.OpenFlowService;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070064import org.onosproject.openflow.controller.OpenFlowSession;
Brian O'Connorabafb502014-12-02 22:26:20 -080065import org.onosproject.openflow.controller.driver.OpenFlowSwitchDriver;
66import org.onosproject.openflow.controller.driver.SwitchStateException;
tom7ef8ff92014-09-17 13:08:06 -070067import org.projectfloodlight.openflow.exceptions.OFParseError;
68import org.projectfloodlight.openflow.protocol.OFAsyncGetReply;
69import org.projectfloodlight.openflow.protocol.OFBadRequestCode;
70import org.projectfloodlight.openflow.protocol.OFBarrierReply;
71import org.projectfloodlight.openflow.protocol.OFBarrierRequest;
72import org.projectfloodlight.openflow.protocol.OFDescStatsReply;
73import org.projectfloodlight.openflow.protocol.OFDescStatsRequest;
74import org.projectfloodlight.openflow.protocol.OFEchoReply;
75import org.projectfloodlight.openflow.protocol.OFEchoRequest;
76import org.projectfloodlight.openflow.protocol.OFErrorMsg;
77import org.projectfloodlight.openflow.protocol.OFErrorType;
78import org.projectfloodlight.openflow.protocol.OFExperimenter;
Yuta HIGUCHI2341e602017-03-08 20:10:08 -080079import org.projectfloodlight.openflow.protocol.OFFactories;
tom7ef8ff92014-09-17 13:08:06 -070080import org.projectfloodlight.openflow.protocol.OFFactory;
81import org.projectfloodlight.openflow.protocol.OFFeaturesReply;
82import org.projectfloodlight.openflow.protocol.OFFlowModFailedCode;
83import org.projectfloodlight.openflow.protocol.OFFlowRemoved;
84import org.projectfloodlight.openflow.protocol.OFGetConfigReply;
85import org.projectfloodlight.openflow.protocol.OFGetConfigRequest;
86import org.projectfloodlight.openflow.protocol.OFHello;
87import org.projectfloodlight.openflow.protocol.OFHelloElem;
88import org.projectfloodlight.openflow.protocol.OFMessage;
Jordi Ortiz91477b82016-11-29 15:22:50 +010089import org.projectfloodlight.openflow.protocol.OFMeterFeaturesStatsReply;
90import org.projectfloodlight.openflow.protocol.OFMeterFeaturesStatsRequest;
tom7ef8ff92014-09-17 13:08:06 -070091import org.projectfloodlight.openflow.protocol.OFPacketIn;
92import org.projectfloodlight.openflow.protocol.OFPortDescStatsReply;
93import org.projectfloodlight.openflow.protocol.OFPortDescStatsRequest;
94import org.projectfloodlight.openflow.protocol.OFPortStatus;
95import org.projectfloodlight.openflow.protocol.OFQueueGetConfigReply;
96import org.projectfloodlight.openflow.protocol.OFRoleReply;
97import org.projectfloodlight.openflow.protocol.OFSetConfig;
98import org.projectfloodlight.openflow.protocol.OFStatsReply;
99import org.projectfloodlight.openflow.protocol.OFStatsReplyFlags;
100import org.projectfloodlight.openflow.protocol.OFStatsType;
101import org.projectfloodlight.openflow.protocol.OFType;
102import org.projectfloodlight.openflow.protocol.OFVersion;
103import org.projectfloodlight.openflow.protocol.errormsg.OFBadRequestErrorMsg;
104import org.projectfloodlight.openflow.protocol.errormsg.OFFlowModFailedErrorMsg;
105import org.projectfloodlight.openflow.types.U32;
106import org.slf4j.Logger;
107import org.slf4j.LoggerFactory;
108
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700109import io.netty.channel.Channel;
110import io.netty.channel.ChannelHandlerContext;
111import io.netty.channel.ChannelInboundHandlerAdapter;
Brian O'Connorf69e3e32018-05-10 02:25:09 -0700112import io.netty.handler.ssl.SslHandler;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700113import io.netty.handler.timeout.IdleStateEvent;
114import io.netty.handler.timeout.ReadTimeoutException;
115import io.netty.util.ReferenceCountUtil;
116
Brian O'Connorf69e3e32018-05-10 02:25:09 -0700117import javax.net.ssl.SSLPeerUnverifiedException;
118
tom7ef8ff92014-09-17 13:08:06 -0700119/**
120 * Channel handler deals with the switch connection and dispatches
121 * switch messages to the appropriate locations.
122 */
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700123class OFChannelHandler extends ChannelInboundHandlerAdapter
124 implements OpenFlowSession {
125
tom7ef8ff92014-09-17 13:08:06 -0700126 private static final Logger log = LoggerFactory.getLogger(OFChannelHandler.class);
Thomas Vachuskae9af1f42015-07-06 08:42:18 -0700127
128 private static final String RESET_BY_PEER = "Connection reset by peer";
129 private static final String BROKEN_PIPE = "Broken pipe";
pieref3d74c2019-07-16 15:52:50 +0200130 static final int NUM_OF_QUEUES = 8;
Thomas Vachuskae9af1f42015-07-06 08:42:18 -0700131
tom7ef8ff92014-09-17 13:08:06 -0700132 private final Controller controller;
133 private OpenFlowSwitchDriver sw;
134 private long thisdpid; // channelHandler cached value of connected switch id
Anton Chigrin4af4f872019-01-14 17:29:56 +0200135 private DeviceId deviceId;
tom7ef8ff92014-09-17 13:08:06 -0700136 private Channel channel;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700137 private String channelId;
138
139
tom7ef8ff92014-09-17 13:08:06 -0700140 // State needs to be volatile because the HandshakeTimeoutHandler
141 // needs to check if the handshake is complete
142 private volatile ChannelState state;
143
Yuta HIGUCHI10f45132017-03-01 17:09:32 -0800144 /**
145 * Timeout in ms to wait for meter feature reply.
146 */
147 private static final long METER_TIMEOUT = 60_000;
148
149 private volatile long lastStateChange = System.currentTimeMillis();
150
tom7ef8ff92014-09-17 13:08:06 -0700151 // When a switch with a duplicate dpid is found (i.e we already have a
152 // connected switch with the same dpid), the new switch is immediately
153 // disconnected. At that point netty callsback channelDisconnected() which
154 // proceeds to cleaup switch state - we need to ensure that it does not cleanup
155 // switch state for the older (still connected) switch
156 private volatile Boolean duplicateDpidFound;
157
158 // Temporary storage for switch-features and port-description
159 private OFFeaturesReply featuresReply;
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700160 private List<OFPortDescStatsReply> portDescReplies;
Jordi Ortiz91477b82016-11-29 15:22:50 +0100161 private OFMeterFeaturesStatsReply meterFeaturesReply;
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700162 //private OFPortDescStatsReply portDescReply;
tom7ef8ff92014-09-17 13:08:06 -0700163 // a concurrent ArrayList to temporarily store port status messages
164 // before we are ready to deal with them
165 private final CopyOnWriteArrayList<OFPortStatus> pendingPortStatusMsg;
166
167 //Indicates the openflow version used by this switch
168 protected OFVersion ofVersion;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700169 protected OFFactory factory;
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800170
tom7ef8ff92014-09-17 13:08:06 -0700171 /** transaction Ids to use during handshake. Since only one thread
172 * calls into an OFChannelHandler instance, we don't need atomic.
173 * We will count down
174 */
175 private int handshakeTransactionIds = -1;
176
Anton Chigrin4af4f872019-01-14 17:29:56 +0200177 @Reference(cardinality = ReferenceCardinality.MANDATORY)
178 private OpenFlowService openFlowManager;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700179
pier063e9032019-11-22 20:51:26 +0100180 // Each IDLE_INTERVAL (see OFChannelInitializer) we will perform a sanity check
181 // Idle timeout actions will be performed each MAX_IDLE_RETRY * IDLE_INTERVAL
182 private static final int MAX_IDLE_RETRY = 4;
183 private int maxIdleRetry = MAX_IDLE_RETRY;
184
185 // Dispatcher buffer/read size
Anton Chigrin4af4f872019-01-14 17:29:56 +0200186 private static final int BACKLOG_READ_BUFFER_DEFAULT = 1000;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700187
188 /**
Anton Chigrin4af4f872019-01-14 17:29:56 +0200189 * Map with all LinkedBlockingMessagesQueue queues which contains OFMessages.
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700190 */
Anton Chigrin4af4f872019-01-14 17:29:56 +0200191 private Map<Integer, LinkedBlockingMessagesQueue<OFMessage>> dispatchQueuesMapProducer = new ConcurrentHashMap<>();
192
193 /**
194 * OFMessage classifiers map.
195 */
196 private List<Set<OpenFlowClassifier>> messageClassifiersMapProducer =
197 new CopyOnWriteArrayList<Set<OpenFlowClassifier>>();
198
199
200 /**
201 * Lock held by take, poll, etc.
202 */
203 private final ReentrantLock takeLock = new ReentrantLock();
204
205 /**
206 * Wait queue for waiting takes.
207 */
208 private final Condition notEmpty = takeLock.newCondition();
209
210 /**
211 * Current number of elements in enabled sub-queues.
212 */
213 private final AtomicInteger totalCount = new AtomicInteger();
214
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700215
216 /**
217 * Single thread executor for OFMessage dispatching.
218 *
219 * Gets initialized on channelActive, shutdown on channelInactive.
220 */
221 private ExecutorService dispatcher;
222
223 /**
224 * Handle for dispatcher thread.
225 * <p>
226 * Should only be touched from the Channel I/O thread
227 */
228 private Future<?> dispatcherHandle = CompletableFuture.completedFuture(null);
229
230 /**
231 * Dispatch backlog.
232 * <p>
233 * Should only be touched from the Channel I/O thread
234 */
Anton Chigrin4af4f872019-01-14 17:29:56 +0200235 private final Deque<OFMessage> dispatchBacklog;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700236
tom7ef8ff92014-09-17 13:08:06 -0700237 /**
238 * Create a new unconnected OFChannelHandler.
Thomas Vachuskab14c77a2014-11-04 18:08:01 -0800239 * @param controller parent controller
tom7ef8ff92014-09-17 13:08:06 -0700240 */
241 OFChannelHandler(Controller controller) {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700242
tom7ef8ff92014-09-17 13:08:06 -0700243 this.controller = controller;
244 this.state = ChannelState.INIT;
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800245 this.pendingPortStatusMsg = new CopyOnWriteArrayList<>();
246 this.portDescReplies = new ArrayList<>();
tom7ef8ff92014-09-17 13:08:06 -0700247 duplicateDpidFound = Boolean.FALSE;
Anton Chigrin4af4f872019-01-14 17:29:56 +0200248 //Initialize queues and classifiers
249 dispatchBacklog = new LinkedBlockingDeque<>(BACKLOG_READ_BUFFER_DEFAULT);
250 for (int i = 0; i < NUM_OF_QUEUES; i++) {
251 if (controller.getQueueSize(i) > 0) {
252 dispatchQueuesMapProducer.put(i,
253 new LinkedBlockingMessagesQueue<>(i, controller.getQueueSize(i), controller.getBulkSize(i)));
254 }
255 if (i != NUM_OF_QUEUES) {
256 messageClassifiersMapProducer.add(i, new CopyOnWriteArraySet<>());
257 }
258 }
tom7ef8ff92014-09-17 13:08:06 -0700259 }
260
261
262
263 // XXX S consider if necessary
264 public void disconnectSwitch() {
265 sw.disconnectSwitch();
266 }
267
268
269
270 //*************************
271 // Channel State Machine
272 //*************************
273
274 /**
275 * The state machine for handling the switch/channel state. All state
276 * transitions should happen from within the state machine (and not from other
277 * parts of the code)
278 */
279 enum ChannelState {
280 /**
281 * Initial state before channel is connected.
282 */
283 INIT(false) {
284 @Override
285 void processOFMessage(OFChannelHandler h, OFMessage m)
286 throws IOException, SwitchStateException {
287 illegalMessageReceived(h, m);
288 }
289
290 @Override
291 void processOFError(OFChannelHandler h, OFErrorMsg m)
292 throws IOException {
293 // need to implement since its abstract but it will never
294 // be called
295 }
296
297 @Override
298 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
299 throws IOException {
300 unhandledMessageReceived(h, m);
301 }
302 },
303
304 /**
305 * We send a OF 1.3 HELLO to the switch and wait for a Hello from the switch.
306 * Once we receive the reply, we decide on OF 1.3 or 1.0 switch - no other
307 * protocol version is accepted.
308 * We send an OFFeaturesRequest depending on the protocol version selected
309 * Next state is WAIT_FEATURES_REPLY
310 */
311 WAIT_HELLO(false) {
312 @Override
313 void processOFHello(OFChannelHandler h, OFHello m)
314 throws IOException {
315 // TODO We could check for the optional bitmap, but for now
316 // we are just checking the version number.
Chip Boling68bc6562015-07-06 10:00:01 -0500317 if (m.getVersion().getWireVersion() >= OFVersion.OF_13.getWireVersion()) {
318 log.debug("Received {} Hello from {} - switching to OF "
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800319 + "version 1.3+", m.getVersion(),
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700320 h.channel.remoteAddress());
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800321 h.ofVersion = m.getVersion();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700322 h.factory = OFFactories.getFactory(h.ofVersion);
alshabib70fc7fb2015-01-06 11:04:29 -0800323 h.sendHandshakeHelloMessage();
Chip Boling68bc6562015-07-06 10:00:01 -0500324 } else if (m.getVersion().getWireVersion() >= OFVersion.OF_10.getWireVersion()) {
alshabib09d48be2014-10-03 15:43:33 -0700325 log.debug("Received {} Hello from {} - switching to OF "
tom7ef8ff92014-09-17 13:08:06 -0700326 + "version 1.0", m.getVersion(),
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700327 h.channel.remoteAddress());
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800328 h.ofVersion = m.getVersion();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700329 h.factory = OFFactories.getFactory(h.ofVersion);
alshabib70fc7fb2015-01-06 11:04:29 -0800330 OFHello hi =
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700331 h.factory.buildHello()
alshabib70fc7fb2015-01-06 11:04:29 -0800332 .setXid(h.handshakeTransactionIds--)
333 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700334 h.channel.writeAndFlush(Collections.singletonList(hi));
tom7ef8ff92014-09-17 13:08:06 -0700335 } else {
336 log.error("Received Hello of version {} from switch at {}. "
337 + "This controller works with OF1.0 and OF1.3 "
338 + "switches. Disconnecting switch ...",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700339 m.getVersion(), h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -0700340 h.channel.disconnect();
341 return;
342 }
343 h.sendHandshakeFeaturesRequestMessage();
344 h.setState(WAIT_FEATURES_REPLY);
345 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700346
tom7ef8ff92014-09-17 13:08:06 -0700347 @Override
348 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
349 throws IOException, SwitchStateException {
350 illegalMessageReceived(h, m);
351 }
352 @Override
353 void processOFStatisticsReply(OFChannelHandler h,
354 OFStatsReply m)
355 throws IOException, SwitchStateException {
356 illegalMessageReceived(h, m);
357 }
358 @Override
359 void processOFError(OFChannelHandler h, OFErrorMsg m) {
360 logErrorDisconnect(h, m);
361 }
362
363 @Override
364 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
365 throws IOException {
366 unhandledMessageReceived(h, m);
367 }
368 },
369
370
371 /**
372 * We are waiting for a features reply message. Once we receive it, the
373 * behavior depends on whether this is a 1.0 or 1.3 switch. For 1.0,
374 * we send a SetConfig request, barrier, and GetConfig request and the
375 * next state is WAIT_CONFIG_REPLY. For 1.3, we send a Port description
376 * request and the next state is WAIT_PORT_DESC_REPLY.
377 */
378 WAIT_FEATURES_REPLY(false) {
379 @Override
Brian O'Connorf69e3e32018-05-10 02:25:09 -0700380 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
tom7ef8ff92014-09-17 13:08:06 -0700381 throws IOException {
Brian O'Connorf69e3e32018-05-10 02:25:09 -0700382 Long dpid = m.getDatapathId().getLong();
383 if (!h.setDpid(dpid, h.channel)) {
384 log.error("Switch presented invalid certificate for dpid {}. Disconnecting",
385 dpid);
386 h.channel.disconnect();
387 return;
388 }
Anton Chigrin4af4f872019-01-14 17:29:56 +0200389 h.deviceId = DeviceId.deviceId(uri(h.thisdpid));
alshabib09d48be2014-10-03 15:43:33 -0700390 log.debug("Received features reply for switch at {} with dpid {}",
tom7ef8ff92014-09-17 13:08:06 -0700391 h.getSwitchInfoString(), h.thisdpid);
392
393 h.featuresReply = m; //temp store
394 if (h.ofVersion == OFVersion.OF_10) {
395 h.sendHandshakeSetConfig();
396 h.setState(WAIT_CONFIG_REPLY);
397 } else {
398 //version is 1.3, must get switchport information
399 h.sendHandshakeOFPortDescRequest();
400 h.setState(WAIT_PORT_DESC_REPLY);
401 }
402 }
403 @Override
404 void processOFStatisticsReply(OFChannelHandler h,
405 OFStatsReply m)
406 throws IOException, SwitchStateException {
407 illegalMessageReceived(h, m);
408 }
409 @Override
410 void processOFError(OFChannelHandler h, OFErrorMsg m) {
411 logErrorDisconnect(h, m);
412 }
413
414 @Override
415 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
416 throws IOException {
Thomas Vachuska39274462014-12-02 13:23:50 -0800417 h.pendingPortStatusMsg.add(m);
tom7ef8ff92014-09-17 13:08:06 -0700418 }
419 },
420
421 /**
422 * We are waiting for a description of the 1.3 switch ports.
423 * Once received, we send a SetConfig request
424 * Next State is WAIT_CONFIG_REPLY
425 */
426 WAIT_PORT_DESC_REPLY(false) {
427
428 @Override
429 void processOFStatisticsReply(OFChannelHandler h, OFStatsReply m)
430 throws SwitchStateException {
431 // Read port description
432 if (m.getStatsType() != OFStatsType.PORT_DESC) {
433 log.warn("Expecting port description stats but received stats "
434 + "type {} from {}. Ignoring ...", m.getStatsType(),
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700435 h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -0700436 return;
437 }
438 if (m.getFlags().contains(OFStatsReplyFlags.REPLY_MORE)) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700439 log.debug("Stats reply indicates more stats from sw {} for "
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700440 + "port description",
tom7ef8ff92014-09-17 13:08:06 -0700441 h.getSwitchInfoString());
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800442 h.portDescReplies.add((OFPortDescStatsReply) m);
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700443 return;
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800444 } else {
445 h.portDescReplies.add((OFPortDescStatsReply) m);
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700446 }
447 //h.portDescReply = (OFPortDescStatsReply) m; // temp store
Andrea Campanellaf9a5ecb2020-05-22 12:42:12 +0200448 if (log.isDebugEnabled()) {
449 log.debug("Received port desc reply for switch at {}: {}",
450 h.getSwitchInfoString(),
451 ((OFPortDescStatsReply) m).getEntries());
452 }
tom7ef8ff92014-09-17 13:08:06 -0700453 try {
454 h.sendHandshakeSetConfig();
455 } catch (IOException e) {
456 log.error("Unable to send setConfig after PortDescReply. "
457 + "Error: {}", e.getMessage());
458 }
459 h.setState(WAIT_CONFIG_REPLY);
460 }
461
462 @Override
463 void processOFError(OFChannelHandler h, OFErrorMsg m)
464 throws IOException, SwitchStateException {
465 logErrorDisconnect(h, m);
466
467 }
468
469 @Override
470 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
471 throws IOException, SwitchStateException {
Thomas Vachuska39274462014-12-02 13:23:50 -0800472 h.pendingPortStatusMsg.add(m);
tom7ef8ff92014-09-17 13:08:06 -0700473
474 }
475 },
476
477 /**
478 * We are waiting for a config reply message. Once we receive it
479 * we send a DescriptionStatsRequest to the switch.
480 * Next state: WAIT_DESCRIPTION_STAT_REPLY
481 */
482 WAIT_CONFIG_REPLY(false) {
483 @Override
484 void processOFGetConfigReply(OFChannelHandler h, OFGetConfigReply m)
485 throws IOException {
486 if (m.getMissSendLen() == 0xffff) {
487 log.trace("Config Reply from switch {} confirms "
488 + "miss length set to 0xffff",
489 h.getSwitchInfoString());
490 } else {
491 // FIXME: we can't really deal with switches that don't send
492 // full packets. Shouldn't we drop the connection here?
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800493 log.warn("Config Reply from switch {} has "
tom7ef8ff92014-09-17 13:08:06 -0700494 + "miss length set to {}",
495 h.getSwitchInfoString(),
496 m.getMissSendLen());
497 }
Jordi Ortiz91477b82016-11-29 15:22:50 +0100498
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800499 nextState(h);
500 }
501
502 /**
503 * Transition to next state based on OF version.
504 *
505 * @param h current channel handler
506 * @throws IOException
507 */
508 private void nextState(OFChannelHandler h) throws IOException {
509 if (h.ofVersion.getWireVersion() >= OFVersion.OF_13.getWireVersion()) {
Jordi Ortiz91477b82016-11-29 15:22:50 +0100510 // Meters were introduced in OpenFlow 1.3
511 h.sendMeterFeaturesRequest();
512 h.setState(WAIT_METER_FEATURES_REPLY);
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800513 } else {
Jordi Ortiz91477b82016-11-29 15:22:50 +0100514 h.sendHandshakeDescriptionStatsRequest();
515 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
516 }
tom7ef8ff92014-09-17 13:08:06 -0700517 }
518
519 @Override
520 void processOFBarrierReply(OFChannelHandler h, OFBarrierReply m) {
521 // do nothing;
522 }
523
524 @Override
525 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
526 throws IOException, SwitchStateException {
527 illegalMessageReceived(h, m);
528 }
Anton Chigrin4af4f872019-01-14 17:29:56 +0200529
tom7ef8ff92014-09-17 13:08:06 -0700530 @Override
531 void processOFStatisticsReply(OFChannelHandler h,
532 OFStatsReply m)
533 throws IOException, SwitchStateException {
534 log.error("Received multipart(stats) message sub-type {}",
535 m.getStatsType());
536 illegalMessageReceived(h, m);
537 }
538
539 @Override
540 void processOFError(OFChannelHandler h, OFErrorMsg m) {
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800541 if (m.getErrType() == OFErrorType.BAD_REQUEST) {
542 OFBadRequestErrorMsg badRequest = (OFBadRequestErrorMsg) m;
543 if (badRequest.getCode() == OFBadRequestCode.BAD_TYPE) {
544 log.debug("{} does not support GetConfig, moving on", h.getSwitchInfoString());
545 try {
546 nextState(h);
547 return;
548 } catch (IOException e) {
549 log.error("Exception thrown transitioning to next", e);
550 logErrorDisconnect(h, m);
551 }
552 }
553 }
tom7ef8ff92014-09-17 13:08:06 -0700554 logErrorDisconnect(h, m);
555 }
556
557 @Override
558 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
559 throws IOException {
560 h.pendingPortStatusMsg.add(m);
561 }
562 },
563
564
565 /**
566 * We are waiting for a OFDescriptionStat message from the switch.
567 * Once we receive any stat message we try to parse it. If it's not
568 * a description stats message we disconnect. If its the expected
569 * description stats message, we:
570 * - use the switch driver to bind the switch and get an IOFSwitch instance
571 * - setup the IOFSwitch instance
572 * - add switch controller and send the initial role
573 * request to the switch.
574 * Next state: WAIT_INITIAL_ROLE
575 * In the typical case, where switches support role request messages
576 * the next state is where we expect the role reply message.
577 * In the special case that where the switch does not support any kind
578 * of role request messages, we don't send a role message, but we do
579 * request mastership from the registry service. This controller
580 * should become master once we hear back from the registry service.
581 * All following states will have a h.sw instance!
582 */
583 WAIT_DESCRIPTION_STAT_REPLY(false) {
584 @Override
585 void processOFStatisticsReply(OFChannelHandler h, OFStatsReply m)
586 throws SwitchStateException {
587 // Read description, if it has been updated
588 if (m.getStatsType() != OFStatsType.DESC) {
589 log.warn("Expecting Description stats but received stats "
590 + "type {} from {}. Ignoring ...", m.getStatsType(),
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700591 h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -0700592 return;
593 }
tom7ef8ff92014-09-17 13:08:06 -0700594 OFDescStatsReply drep = (OFDescStatsReply) m;
Saurav Dasf9ba4222015-05-07 17:13:59 -0700595 log.info("Received switch description reply {} from switch at {}",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700596 drep, h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -0700597 // Here is where we differentiate between different kinds of switches
598 h.sw = h.controller.getOFSwitchInstance(h.thisdpid, drep, h.ofVersion);
599
Ray Milkey31b00482019-02-07 08:06:28 -0800600 if (h.sw == null) {
601 log.info("Switch not found for {}", h.thisdpid);
602 return;
603 }
604
Andrea Campanellaf9a5ecb2020-05-22 12:42:12 +0200605 // set switch information
tom7ef8ff92014-09-17 13:08:06 -0700606 h.sw.setOFVersion(h.ofVersion);
607 h.sw.setFeaturesReply(h.featuresReply);
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700608 h.sw.setPortDescReplies(h.portDescReplies);
Jordi Ortiz91477b82016-11-29 15:22:50 +0100609 h.sw.setMeterFeaturesReply(h.meterFeaturesReply);
tom7ef8ff92014-09-17 13:08:06 -0700610 h.sw.setConnected(true);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700611 h.sw.setChannel(h);
Andrea Campanellaf9a5ecb2020-05-22 12:42:12 +0200612
613 //Port Description List has served its purpose, clearing.
614 h.portDescReplies.clear();
615
Praseed Balakrishnana22eadf2014-10-20 14:21:45 -0700616// boolean success = h.sw.connectSwitch();
617//
618// if (!success) {
619// disconnectDuplicate(h);
620// return;
621// }
tom7ef8ff92014-09-17 13:08:06 -0700622
alshabib09d48be2014-10-03 15:43:33 -0700623 log.debug("Switch {} bound to class {}, description {}",
Ray Milkey6bc43c22015-11-06 13:22:38 -0800624 h.sw, h.sw.getClass(), drep);
tom7ef8ff92014-09-17 13:08:06 -0700625 //Put switch in EQUAL mode until we hear back from the global registry
626 //log.debug("Setting new switch {} to EQUAL and sending Role request",
627 // h.sw.getStringId());
628 //h.sw.activateEqualSwitch();
629 //h.setSwitchRole(RoleState.EQUAL);
630
631 h.sw.startDriverHandshake();
alshabib9eab22f2014-10-20 17:17:31 -0700632 if (h.sw.isDriverHandshakeComplete()) {
pieref3d74c2019-07-16 15:52:50 +0200633 // We are not able to complete the connection for a dpid collision.
634 // Same device reconnecting or different device configured with
635 // the same dpid.
alshabib9eab22f2014-10-20 17:17:31 -0700636 if (!h.sw.connectSwitch()) {
pieref3d74c2019-07-16 15:52:50 +0200637 // Disconnect from the device and return
alshabib9eab22f2014-10-20 17:17:31 -0700638 disconnectDuplicate(h);
pieref3d74c2019-07-16 15:52:50 +0200639 return;
Anton Chigrin4af4f872019-01-14 17:29:56 +0200640 } else {
641 h.initClassifiers();
alshabib9eab22f2014-10-20 17:17:31 -0700642 }
Thomas Vachuska39274462014-12-02 13:23:50 -0800643 handlePendingPortStatusMessages(h);
alshabib9eab22f2014-10-20 17:17:31 -0700644 h.setState(ACTIVE);
645 } else {
646 h.setState(WAIT_SWITCH_DRIVER_SUB_HANDSHAKE);
647 }
tom7ef8ff92014-09-17 13:08:06 -0700648
649 }
650
651 @Override
652 void processOFError(OFChannelHandler h, OFErrorMsg m) {
653 logErrorDisconnect(h, m);
654 }
655
656 @Override
657 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
658 throws IOException, SwitchStateException {
659 illegalMessageReceived(h, m);
660 }
661
662 @Override
663 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
664 throws IOException {
665 h.pendingPortStatusMsg.add(m);
666 }
667 },
668
669
670 /**
671 * We are waiting for the respective switch driver to complete its
672 * configuration. Notice that we do not consider this to be part of the main
673 * switch-controller handshake. But we do consider it as a step that comes
674 * before we declare the switch as available to the controller.
675 * Next State: depends on the role of this controller for this switch - either
676 * MASTER or EQUAL.
677 */
678 WAIT_SWITCH_DRIVER_SUB_HANDSHAKE(true) {
679
680 @Override
681 void processOFError(OFChannelHandler h, OFErrorMsg m)
682 throws IOException {
683 // will never be called. We override processOFMessage
684 }
685
alshabibd7963912014-10-20 14:52:04 -0700686
687
tom7ef8ff92014-09-17 13:08:06 -0700688 @Override
689 void processOFMessage(OFChannelHandler h, OFMessage m)
690 throws IOException, SwitchStateException {
alshabibd7963912014-10-20 14:52:04 -0700691
692 if (h.sw.isDriverHandshakeComplete()) {
693 moveToActive(h);
alshabib9eab22f2014-10-20 17:17:31 -0700694 h.state.processOFMessage(h, m);
695 return;
alshabibd7963912014-10-20 14:52:04 -0700696
697 }
698
tom7ef8ff92014-09-17 13:08:06 -0700699 if (m.getType() == OFType.ECHO_REQUEST) {
700 processOFEchoRequest(h, (OFEchoRequest) m);
Praseed Balakrishnana22eadf2014-10-20 14:21:45 -0700701 } else if (m.getType() == OFType.ECHO_REPLY) {
702 processOFEchoReply(h, (OFEchoReply) m);
tom7ef8ff92014-09-17 13:08:06 -0700703 } else if (m.getType() == OFType.ROLE_REPLY) {
704 h.sw.handleRole(m);
705 } else if (m.getType() == OFType.ERROR) {
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800706 if (!h.sw.handleRoleError((OFErrorMsg) m)) {
tom7ef8ff92014-09-17 13:08:06 -0700707 h.sw.processDriverHandshakeMessage(m);
708 if (h.sw.isDriverHandshakeComplete()) {
alshabibd7963912014-10-20 14:52:04 -0700709 moveToActive(h);
tom7ef8ff92014-09-17 13:08:06 -0700710 }
711 }
712 } else {
713 if (m.getType() == OFType.EXPERIMENTER &&
714 ((OFExperimenter) m).getExperimenter() ==
715 RoleManager.NICIRA_EXPERIMENTER) {
716 h.sw.handleNiciraRole(m);
717 } else {
718 h.sw.processDriverHandshakeMessage(m);
719 if (h.sw.isDriverHandshakeComplete()) {
alshabibd7963912014-10-20 14:52:04 -0700720 moveToActive(h);
tom7ef8ff92014-09-17 13:08:06 -0700721 }
722 }
723 }
724 }
725
726 @Override
727 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
728 throws IOException, SwitchStateException {
729 h.pendingPortStatusMsg.add(m);
730 }
alshabibd7963912014-10-20 14:52:04 -0700731
732 private void moveToActive(OFChannelHandler h) {
733 boolean success = h.sw.connectSwitch();
pieref3d74c2019-07-16 15:52:50 +0200734 // Disconnect from the device and return
alshabibd7963912014-10-20 14:52:04 -0700735 if (!success) {
736 disconnectDuplicate(h);
pieref3d74c2019-07-16 15:52:50 +0200737 return;
alshabibd7963912014-10-20 14:52:04 -0700738 }
pieref3d74c2019-07-16 15:52:50 +0200739 handlePendingPortStatusMessages(h);
740 h.setState(ACTIVE);
alshabibd7963912014-10-20 14:52:04 -0700741 }
742
tom7ef8ff92014-09-17 13:08:06 -0700743 },
744
Jordi Ortiz91477b82016-11-29 15:22:50 +0100745 /**
746 * We are expecting a OF Multi Part Meter Features Stats Reply.
747 * Notice that this information is only available for switches running
748 * OpenFlow 1.3
749 */
750 WAIT_METER_FEATURES_REPLY(true) {
Yuta HIGUCHI10f45132017-03-01 17:09:32 -0800751
752 @Override
753 void processOFEchoRequest(OFChannelHandler h, OFEchoRequest m)
754 throws IOException {
755 super.processOFEchoRequest(h, m);
756 if (System.currentTimeMillis() - h.lastStateChange > METER_TIMEOUT) {
757 log.info("{} did not respond to MeterFeaturesRequest on time, " +
758 "moving on without it.",
759 h.getSwitchInfoString());
760 h.sendHandshakeDescriptionStatsRequest();
761 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
762 }
763 }
764
Jordi Ortiz91477b82016-11-29 15:22:50 +0100765 @Override
766 void processOFError(OFChannelHandler h, OFErrorMsg m)
767 throws IOException {
Charles Chan34155e52016-11-30 18:28:11 -0800768 // Hardware switches may reply OFErrorMsg if meter is not supported
769 log.warn("Received OFError {}. It seems {} does not support Meter.",
770 m.getErrType().name(), Dpid.uri(h.thisdpid));
771 log.debug("{}", m);
772 h.sendHandshakeDescriptionStatsRequest();
773 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
Jordi Ortiz91477b82016-11-29 15:22:50 +0100774 }
775
776 @Override
777 void processOFStatisticsReply(OFChannelHandler h,
778 OFStatsReply m)
779 throws IOException, SwitchStateException {
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800780 switch (m.getStatsType()) {
Jordi Ortiz91477b82016-11-29 15:22:50 +0100781 case METER_FEATURES:
782
783 log.debug("Received Meter Features");
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800784 OFMeterFeaturesStatsReply ofmfsr = (OFMeterFeaturesStatsReply) m;
Jordi Ortiz91477b82016-11-29 15:22:50 +0100785 log.info("Received meter features from {} with max meters: {}",
786 h.getSwitchInfoString(),
787 ofmfsr.getFeatures().getMaxMeter());
788 h.meterFeaturesReply = ofmfsr;
789 h.sendHandshakeDescriptionStatsRequest();
790 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
791 break;
792 default:
793 log.error("Unexpected OF Multi Part stats reply");
794 illegalMessageReceived(h, m);
795 break;
796 }
797 }
798
799 @Override
800 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
801 throws IOException, SwitchStateException {
802 illegalMessageReceived(h, m);
803 }
804
805 @Override
806 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
807 throws IOException {
808 h.pendingPortStatusMsg.add(m);
809 }
Yuta HIGUCHI1745e5a2017-01-15 21:43:02 -0800810
811 @Override
812 void processIdle(OFChannelHandler h) throws IOException {
813 log.info("{} did not respond to MeterFeaturesRequest, " +
814 "moving on without it.",
815 h.getSwitchInfoString());
816 h.sendHandshakeDescriptionStatsRequest();
817 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
818 }
Jordi Ortiz91477b82016-11-29 15:22:50 +0100819 },
820
tom7ef8ff92014-09-17 13:08:06 -0700821
822 /**
823 * This controller is in MASTER role for this switch. We enter this state
824 * after requesting and winning control from the global registry.
825 * The main handshake as well as the switch-driver sub-handshake
826 * is complete at this point.
827 * // XXX S reconsider below
828 * In the (near) future we may deterministically assign controllers to
829 * switches at startup.
830 * We only leave this state if the switch disconnects or
831 * if we send a role request for SLAVE /and/ receive the role reply for
832 * SLAVE.
833 */
834 ACTIVE(true) {
835 @Override
836 void processOFError(OFChannelHandler h, OFErrorMsg m)
837 throws IOException, SwitchStateException {
838 // if we get here, then the error message is for something else
839 if (m.getErrType() == OFErrorType.BAD_REQUEST &&
Ray Milkey30d19652016-09-06 12:09:46 -0700840 (((OFBadRequestErrorMsg) m).getCode() ==
Charles Chan9d7465e2016-09-09 19:02:34 -0700841 OFBadRequestCode.EPERM ||
tom7ef8ff92014-09-17 13:08:06 -0700842 ((OFBadRequestErrorMsg) m).getCode() ==
Charles Chan9d7465e2016-09-09 19:02:34 -0700843 OFBadRequestCode.IS_SLAVE)) {
tom7ef8ff92014-09-17 13:08:06 -0700844 // We are the master controller and the switch returned
845 // a permission error. This is a likely indicator that
846 // the switch thinks we are slave. Reassert our
847 // role
848 // FIXME: this could be really bad during role transitions
849 // if two controllers are master (even if its only for
850 // a brief period). We might need to see if these errors
851 // persist before we reassert
alshabib339a3d92014-09-26 17:54:32 -0700852
tom7ef8ff92014-09-17 13:08:06 -0700853 h.sw.reassertRole();
854 } else if (m.getErrType() == OFErrorType.FLOW_MOD_FAILED &&
855 ((OFFlowModFailedErrorMsg) m).getCode() ==
856 OFFlowModFailedCode.ALL_TABLES_FULL) {
857 h.sw.setTableFull(true);
858 } else {
859 logError(h, m);
860 }
861 h.dispatchMessage(m);
862 }
863
864 @Override
865 void processOFStatisticsReply(OFChannelHandler h,
866 OFStatsReply m) {
Ayaka Koshibe38594c22014-10-22 13:36:12 -0700867 if (m.getStatsType().equals(OFStatsType.PORT_DESC)) {
Andrea Campanellaf9a5ecb2020-05-22 12:42:12 +0200868 if (log.isDebugEnabled()) {
869 log.debug("Received port desc message from {}: {}",
870 h.sw.getDpid(),
871 ((OFPortDescStatsReply) m).getEntries());
872 }
873 if (m.getFlags().contains(OFStatsReplyFlags.REPLY_MORE)) {
874 log.debug("Active Stats reply indicates more stats from sw {} for "
875 + "port description",
876 h.getSwitchInfoString());
877 h.portDescReplies.add((OFPortDescStatsReply) m);
878 h.dispatchMessage(m);
879 return;
880 }
881
882 h.portDescReplies.add((OFPortDescStatsReply) m);
883 if (log.isDebugEnabled()) {
884 log.debug("Adding all Port Desc Active Replies to {}: {}",
885 h.sw.getDpid(),
886 h.portDescReplies);
887 }
888 h.sw.setPortDescReplies(h.portDescReplies);
889 //clearing to wait for next full response
890 h.portDescReplies.clear();
Ayaka Koshibe38594c22014-10-22 13:36:12 -0700891 }
tom7ef8ff92014-09-17 13:08:06 -0700892 h.dispatchMessage(m);
893 }
894
895 @Override
896 void processOFExperimenter(OFChannelHandler h, OFExperimenter m)
897 throws SwitchStateException {
898 h.sw.handleNiciraRole(m);
899 }
900
901 @Override
902 void processOFRoleReply(OFChannelHandler h, OFRoleReply m)
903 throws SwitchStateException {
904 h.sw.handleRole(m);
905 }
906
907 @Override
908 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
909 throws SwitchStateException {
910 handlePortStatusMessage(h, m, true);
Thomas Vachuska39274462014-12-02 13:23:50 -0800911 //h.dispatchMessage(m);
tom7ef8ff92014-09-17 13:08:06 -0700912 }
913
914 @Override
915 void processOFPacketIn(OFChannelHandler h, OFPacketIn m) {
alshabib9eab22f2014-10-20 17:17:31 -0700916// OFPacketOut out =
917// h.sw.factory().buildPacketOut()
918// .setXid(m.getXid())
919// .setBufferId(m.getBufferId()).build();
920// h.sw.sendMsg(out);
tom7ef8ff92014-09-17 13:08:06 -0700921 h.dispatchMessage(m);
922 }
923
924 @Override
925 void processOFFlowRemoved(OFChannelHandler h,
926 OFFlowRemoved m) {
927 h.dispatchMessage(m);
928 }
929
930 @Override
931 void processOFBarrierReply(OFChannelHandler h, OFBarrierReply m) {
932 h.dispatchMessage(m);
933 }
934
Ayaka Koshibee8708e32014-10-22 13:40:18 -0700935 @Override
936 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m) {
Ayaka Koshibe38594c22014-10-22 13:36:12 -0700937 h.sw.setFeaturesReply(m);
Ayaka Koshibee8708e32014-10-22 13:40:18 -0700938 h.dispatchMessage(m);
939 }
940
Yuta HIGUCHI1745e5a2017-01-15 21:43:02 -0800941 @Override
942 void processIdle(OFChannelHandler h) throws IOException {
943 log.info("{} idle", h.getSwitchInfoString());
944 }
945
tom7ef8ff92014-09-17 13:08:06 -0700946 };
947
948 private final boolean handshakeComplete;
949 ChannelState(boolean handshakeComplete) {
950 this.handshakeComplete = handshakeComplete;
951 }
952
953 /**
954 * Is this a state in which the handshake has completed?
955 * @return true if the handshake is complete
956 */
957 public boolean isHandshakeComplete() {
958 return handshakeComplete;
959 }
960
961 /**
962 * Get a string specifying the switch connection, state, and
963 * message received. To be used as message for SwitchStateException
964 * or log messages
965 * @param h The channel handler (to get switch information_
966 * @param m The OFMessage that has just been received
967 * @param details A string giving more details about the exact nature
968 * of the problem.
969 * @return display string
970 */
971 // needs to be protected because enum members are actually subclasses
972 protected String getSwitchStateMessage(OFChannelHandler h,
973 OFMessage m,
974 String details) {
975 return String.format("Switch: [%s], State: [%s], received: [%s]"
976 + ", details: %s",
977 h.getSwitchInfoString(),
978 this.toString(),
979 m.getType().toString(),
980 details);
981 }
982
983 /**
984 * We have an OFMessage we didn't expect given the current state and
985 * we want to treat this as an error.
986 * We currently throw an exception that will terminate the connection
987 * However, we could be more forgiving
988 * @param h the channel handler that received the message
989 * @param m the message
Jonathan Hart147b2ac2014-10-23 10:03:52 -0700990 * @throws SwitchStateException we always throw the exception
tom7ef8ff92014-09-17 13:08:06 -0700991 */
Jonathan Hart147b2ac2014-10-23 10:03:52 -0700992 // needs to be protected because enum members are actually subclasses
tom7ef8ff92014-09-17 13:08:06 -0700993 protected void illegalMessageReceived(OFChannelHandler h, OFMessage m)
994 throws SwitchStateException {
995 String msg = getSwitchStateMessage(h, m,
996 "Switch should never send this message in the current state");
997 throw new SwitchStateException(msg);
998
999 }
1000
1001 /**
1002 * We have an OFMessage we didn't expect given the current state and
1003 * we want to ignore the message.
1004 * @param h the channel handler the received the message
1005 * @param m the message
1006 */
1007 protected void unhandledMessageReceived(OFChannelHandler h,
1008 OFMessage m) {
1009 if (log.isDebugEnabled()) {
1010 String msg = getSwitchStateMessage(h, m,
1011 "Ignoring unexpected message");
1012 log.debug(msg);
1013 }
1014 }
1015
1016 /**
1017 * Log an OpenFlow error message from a switch.
1018 * @param h The switch that sent the error
1019 * @param error The error message
1020 */
1021 protected void logError(OFChannelHandler h, OFErrorMsg error) {
alshabib09d48be2014-10-03 15:43:33 -07001022 log.error("{} from switch {} in state {}",
tom7ef8ff92014-09-17 13:08:06 -07001023 error,
1024 h.getSwitchInfoString(),
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001025 this);
tom7ef8ff92014-09-17 13:08:06 -07001026 }
1027
1028 /**
1029 * Log an OpenFlow error message from a switch and disconnect the
1030 * channel.
1031 *
1032 * @param h the IO channel for this switch.
1033 * @param error The error message
1034 */
1035 protected void logErrorDisconnect(OFChannelHandler h, OFErrorMsg error) {
1036 logError(h, error);
HIGUCHI Yutadc5cf8a2016-04-29 15:17:06 -07001037 log.error("Disconnecting switch {}", h.getSwitchInfoString());
tom7ef8ff92014-09-17 13:08:06 -07001038 h.channel.disconnect();
1039 }
1040
1041 /**
1042 * log an error message for a duplicate dpid and disconnect this channel.
1043 * @param h the IO channel for this switch.
1044 */
1045 protected void disconnectDuplicate(OFChannelHandler h) {
1046 log.error("Duplicated dpid or incompleted cleanup - "
1047 + "disconnecting channel {}", h.getSwitchInfoString());
1048 h.duplicateDpidFound = Boolean.TRUE;
1049 h.channel.disconnect();
1050 }
1051
1052
1053
1054 /**
1055 * Handles all pending port status messages before a switch is declared
1056 * activated in MASTER or EQUAL role. Note that since this handling
1057 * precedes the activation (and therefore notification to IOFSwitchListerners)
1058 * the changes to ports will already be visible once the switch is
1059 * activated. As a result, no notifications are sent out for these
1060 * pending portStatus messages.
Thomas Vachuska4b420772014-10-30 16:46:17 -07001061 *
1062 * @param h the channel handler that received the message
tom7ef8ff92014-09-17 13:08:06 -07001063 */
1064 protected void handlePendingPortStatusMessages(OFChannelHandler h) {
1065 try {
1066 handlePendingPortStatusMessages(h, 0);
1067 } catch (SwitchStateException e) {
1068 log.error(e.getMessage());
1069 }
1070 }
1071
1072 private void handlePendingPortStatusMessages(OFChannelHandler h, int index)
1073 throws SwitchStateException {
1074 if (h.sw == null) {
1075 String msg = "State machine error: switch is null. Should never " +
1076 "happen";
1077 throw new SwitchStateException(msg);
1078 }
Thomas Vachuska39274462014-12-02 13:23:50 -08001079 log.info("Processing {} pending port status messages for {}",
1080 h.pendingPortStatusMsg.size(), h.sw.getStringId());
1081
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001082 ArrayList<OFPortStatus> temp = new ArrayList<>();
tom7ef8ff92014-09-17 13:08:06 -07001083 for (OFPortStatus ps: h.pendingPortStatusMsg) {
1084 temp.add(ps);
1085 handlePortStatusMessage(h, ps, false);
1086 }
tom7ef8ff92014-09-17 13:08:06 -07001087 // expensive but ok - we don't expect too many port-status messages
1088 // note that we cannot use clear(), because of the reasons below
1089 h.pendingPortStatusMsg.removeAll(temp);
Thomas Vachuska39274462014-12-02 13:23:50 -08001090 temp.clear();
tom7ef8ff92014-09-17 13:08:06 -07001091 // the iterator above takes a snapshot of the list - so while we were
1092 // dealing with the pending port-status messages, we could have received
1093 // newer ones. Handle them recursively, but break the recursion after
1094 // five steps to avoid an attack.
1095 if (!h.pendingPortStatusMsg.isEmpty() && ++index < 5) {
1096 handlePendingPortStatusMessages(h, index);
1097 }
1098 }
1099
1100 /**
1101 * Handle a port status message.
1102 *
1103 * Handle a port status message by updating the port maps in the
1104 * IOFSwitch instance and notifying Controller about the change so
1105 * it can dispatch a switch update.
1106 *
1107 * @param h The OFChannelHhandler that received the message
1108 * @param m The PortStatus message we received
1109 * @param doNotify if true switch port changed events will be
1110 * dispatched
Thomas Vachuskab14c77a2014-11-04 18:08:01 -08001111 * @throws SwitchStateException if the switch is not bound to the channel
tom7ef8ff92014-09-17 13:08:06 -07001112 *
1113 */
1114 protected void handlePortStatusMessage(OFChannelHandler h, OFPortStatus m,
1115 boolean doNotify) throws SwitchStateException {
1116 if (h.sw == null) {
1117 String msg = getSwitchStateMessage(h, m,
1118 "State machine error: switch is null. Should never " +
1119 "happen");
1120 throw new SwitchStateException(msg);
1121 }
Saurav Dasbd071d82018-01-09 17:38:44 -08001122 log.info("Received port status message from {}/{}: {}",
1123 h.sw.getDpid(), m.getDesc().getPortNo(), m);
tom7ef8ff92014-09-17 13:08:06 -07001124
1125 h.sw.handleMessage(m);
1126 }
1127
1128
1129 /**
1130 * Process an OF message received on the channel and
1131 * update state accordingly.
1132 *
1133 * The main "event" of the state machine. Process the received message,
1134 * send follow up message if required and update state if required.
1135 *
1136 * Switches on the message type and calls more specific event handlers
1137 * for each individual OF message type. If we receive a message that
1138 * is supposed to be sent from a controller to a switch we throw
1139 * a SwitchStateExeption.
1140 *
1141 * The more specific handlers can also throw SwitchStateExceptions
1142 *
1143 * @param h The OFChannelHandler that received the message
1144 * @param m The message we received.
Thomas Vachuskab14c77a2014-11-04 18:08:01 -08001145 * @throws SwitchStateException if the switch is not bound to the channel
1146 * @throws IOException if unable to send message back to the switch
tom7ef8ff92014-09-17 13:08:06 -07001147 */
1148 void processOFMessage(OFChannelHandler h, OFMessage m)
1149 throws IOException, SwitchStateException {
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001150 switch (m.getType()) {
tom7ef8ff92014-09-17 13:08:06 -07001151 case HELLO:
1152 processOFHello(h, (OFHello) m);
1153 break;
1154 case BARRIER_REPLY:
1155 processOFBarrierReply(h, (OFBarrierReply) m);
1156 break;
1157 case ECHO_REPLY:
1158 processOFEchoReply(h, (OFEchoReply) m);
1159 break;
1160 case ECHO_REQUEST:
1161 processOFEchoRequest(h, (OFEchoRequest) m);
1162 break;
1163 case ERROR:
1164 processOFError(h, (OFErrorMsg) m);
1165 break;
1166 case FEATURES_REPLY:
1167 processOFFeaturesReply(h, (OFFeaturesReply) m);
1168 break;
1169 case FLOW_REMOVED:
1170 processOFFlowRemoved(h, (OFFlowRemoved) m);
1171 break;
1172 case GET_CONFIG_REPLY:
1173 processOFGetConfigReply(h, (OFGetConfigReply) m);
1174 break;
1175 case PACKET_IN:
1176 processOFPacketIn(h, (OFPacketIn) m);
1177 break;
1178 case PORT_STATUS:
1179 processOFPortStatus(h, (OFPortStatus) m);
1180 break;
1181 case QUEUE_GET_CONFIG_REPLY:
1182 processOFQueueGetConfigReply(h, (OFQueueGetConfigReply) m);
1183 break;
1184 case STATS_REPLY: // multipart_reply in 1.3
1185 processOFStatisticsReply(h, (OFStatsReply) m);
1186 break;
1187 case EXPERIMENTER:
1188 processOFExperimenter(h, (OFExperimenter) m);
1189 break;
1190 case ROLE_REPLY:
1191 processOFRoleReply(h, (OFRoleReply) m);
1192 break;
1193 case GET_ASYNC_REPLY:
1194 processOFGetAsyncReply(h, (OFAsyncGetReply) m);
1195 break;
1196
1197 // The following messages are sent to switches. The controller
1198 // should never receive them
1199 case SET_CONFIG:
1200 case GET_CONFIG_REQUEST:
1201 case PACKET_OUT:
1202 case PORT_MOD:
1203 case QUEUE_GET_CONFIG_REQUEST:
1204 case BARRIER_REQUEST:
1205 case STATS_REQUEST: // multipart request in 1.3
1206 case FEATURES_REQUEST:
1207 case FLOW_MOD:
1208 case GROUP_MOD:
1209 case TABLE_MOD:
1210 case GET_ASYNC_REQUEST:
1211 case SET_ASYNC:
1212 case METER_MOD:
1213 default:
1214 illegalMessageReceived(h, m);
1215 break;
1216 }
1217 }
1218
1219 /*-----------------------------------------------------------------
1220 * Default implementation for message handlers in any state.
1221 *
1222 * Individual states must override these if they want a behavior
1223 * that differs from the default.
1224 *
1225 * In general, these handlers simply ignore the message and do
1226 * nothing.
1227 *
1228 * There are some exceptions though, since some messages really
1229 * are handled the same way in every state (e.g., ECHO_REQUST) or
1230 * that are only valid in a single state (e.g., HELLO, GET_CONFIG_REPLY
1231 -----------------------------------------------------------------*/
1232
1233 void processOFHello(OFChannelHandler h, OFHello m)
1234 throws IOException, SwitchStateException {
1235 // we only expect hello in the WAIT_HELLO state
alshabib45fd88a2015-09-24 17:34:35 -07001236 log.warn("Received Hello outside WAIT_HELLO state; switch {} is not complaint.",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001237 h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001238 }
1239
1240 void processOFBarrierReply(OFChannelHandler h, OFBarrierReply m)
1241 throws IOException {
1242 // Silently ignore.
1243 }
1244
1245 void processOFEchoRequest(OFChannelHandler h, OFEchoRequest m)
1246 throws IOException {
1247 if (h.ofVersion == null) {
1248 log.error("No OF version set for {}. Not sending Echo REPLY",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001249 h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001250 return;
1251 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001252 OFEchoReply reply = h.factory
1253 .buildEchoReply()
1254 .setXid(m.getXid())
1255 .setData(m.getData())
1256 .build();
1257 h.channel.writeAndFlush(Collections.singletonList(reply));
tom7ef8ff92014-09-17 13:08:06 -07001258 }
1259
1260 void processOFEchoReply(OFChannelHandler h, OFEchoReply m)
1261 throws IOException {
1262 // Do nothing with EchoReplies !!
1263 }
1264
1265 // no default implementation for OFError
1266 // every state must override it
1267 abstract void processOFError(OFChannelHandler h, OFErrorMsg m)
1268 throws IOException, SwitchStateException;
1269
1270
1271 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
1272 throws IOException, SwitchStateException {
1273 unhandledMessageReceived(h, m);
1274 }
1275
1276 void processOFFlowRemoved(OFChannelHandler h, OFFlowRemoved m)
1277 throws IOException {
1278 unhandledMessageReceived(h, m);
1279 }
1280
1281 void processOFGetConfigReply(OFChannelHandler h, OFGetConfigReply m)
1282 throws IOException, SwitchStateException {
1283 // we only expect config replies in the WAIT_CONFIG_REPLY state
1284 illegalMessageReceived(h, m);
1285 }
1286
1287 void processOFPacketIn(OFChannelHandler h, OFPacketIn m)
1288 throws IOException {
1289 unhandledMessageReceived(h, m);
1290 }
1291
1292 // no default implementation. Every state needs to handle it.
1293 abstract void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
1294 throws IOException, SwitchStateException;
1295
1296 void processOFQueueGetConfigReply(OFChannelHandler h,
1297 OFQueueGetConfigReply m)
1298 throws IOException {
1299 unhandledMessageReceived(h, m);
1300 }
1301
1302 void processOFStatisticsReply(OFChannelHandler h, OFStatsReply m)
1303 throws IOException, SwitchStateException {
1304 unhandledMessageReceived(h, m);
1305 }
1306
1307 void processOFExperimenter(OFChannelHandler h, OFExperimenter m)
1308 throws IOException, SwitchStateException {
1309 // TODO: it might make sense to parse the vendor message here
1310 // into the known vendor messages we support and then call more
1311 // specific event handlers
1312 unhandledMessageReceived(h, m);
1313 }
1314
1315 void processOFRoleReply(OFChannelHandler h, OFRoleReply m)
1316 throws SwitchStateException, IOException {
1317 unhandledMessageReceived(h, m);
1318 }
1319
1320 void processOFGetAsyncReply(OFChannelHandler h,
1321 OFAsyncGetReply m) {
1322 unhandledMessageReceived(h, m);
1323 }
1324
Yuta HIGUCHI1745e5a2017-01-15 21:43:02 -08001325 void processIdle(OFChannelHandler h) throws IOException {
1326 // disconnect channel which did no complete handshake
1327 log.error("{} idle in state {}, disconnecting", h.getSwitchInfoString(), this);
1328 h.channel.disconnect();
1329 }
tom7ef8ff92014-09-17 13:08:06 -07001330 }
1331
1332
1333
1334 //*************************
1335 // Channel handler methods
1336 //*************************
1337
1338 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001339 public void channelActive(ChannelHandlerContext ctx)
1340 throws Exception {
1341
1342 channel = ctx.channel();
tom7ef8ff92014-09-17 13:08:06 -07001343 log.info("New switch connection from {}",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001344 channel.remoteAddress());
1345
1346 SocketAddress address = channel.remoteAddress();
1347 if (address instanceof InetSocketAddress) {
1348 final InetSocketAddress inetAddress = (InetSocketAddress) address;
1349 final IpAddress ipAddress = IpAddress.valueOf(inetAddress.getAddress());
1350 if (ipAddress.isIp4()) {
1351 channelId = ipAddress.toString() + ':' + inetAddress.getPort();
1352 } else {
1353 channelId = '[' + ipAddress.toString() + "]:" + inetAddress.getPort();
1354 }
1355 } else {
1356 channelId = channel.toString();
1357 }
1358
1359 dispatcher = Executors.newSingleThreadExecutor(groupedThreads("onos/of/dispatcher", channelId, log));
1360
alshabib70fc7fb2015-01-06 11:04:29 -08001361 /*
1362 hack to wait for the switch to tell us what it's
1363 max version is. This is not spec compliant and should
1364 be removed as soon as switches behave better.
1365 */
1366 //sendHandshakeHelloMessage();
tom7ef8ff92014-09-17 13:08:06 -07001367 setState(ChannelState.WAIT_HELLO);
1368 }
1369
1370 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001371 public void channelInactive(ChannelHandlerContext ctx)
1372 throws Exception {
1373
tom7ef8ff92014-09-17 13:08:06 -07001374 log.info("Switch disconnected callback for sw:{}. Cleaning up ...",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001375 getSwitchInfoString());
1376
1377 if (dispatcher != null) {
Harold Huang828cd592017-11-04 10:46:04 +08001378 dispatcher.shutdownNow();
Thomas Vachuskad75684a2018-01-03 09:04:47 -08001379 dispatcher = null;
tom7ef8ff92014-09-17 13:08:06 -07001380 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001381
1382 if (thisdpid != 0) {
1383 if (!duplicateDpidFound) {
1384 // if the disconnected switch (on this ChannelHandler)
1385 // was not one with a duplicate-dpid, it is safe to remove all
1386 // state for it at the controller. Notice that if the disconnected
1387 // switch was a duplicate-dpid, calling the method below would clear
1388 // all state for the original switch (with the same dpid),
1389 // which we obviously don't want.
1390 log.info("{}:removal called", getSwitchInfoString());
1391 if (sw != null) {
1392 sw.removeConnectedSwitch();
1393 }
1394 } else {
1395 // A duplicate was disconnected on this ChannelHandler,
1396 // this is the same switch reconnecting, but the original state was
1397 // not cleaned up - XXX check liveness of original ChannelHandler
1398 log.info("{}:duplicate found", getSwitchInfoString());
1399 duplicateDpidFound = Boolean.FALSE;
1400 }
1401 } else {
1402 log.warn("no dpid in channelHandler registered for "
1403 + "disconnected switch {}", getSwitchInfoString());
1404 }
tom7ef8ff92014-09-17 13:08:06 -07001405 }
1406
1407 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001408 public void exceptionCaught(ChannelHandlerContext ctx,
1409 Throwable cause)
tom7ef8ff92014-09-17 13:08:06 -07001410 throws Exception {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001411
1412 if (cause instanceof ReadTimeoutException) {
tom7ef8ff92014-09-17 13:08:06 -07001413 // switch timeout
1414 log.error("Disconnecting switch {} due to read timeout",
1415 getSwitchInfoString());
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001416 ctx.channel().close();
1417 } else if (cause instanceof HandshakeTimeoutException) {
tom7ef8ff92014-09-17 13:08:06 -07001418 log.error("Disconnecting switch {}: failed to complete handshake",
1419 getSwitchInfoString());
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001420 ctx.channel().close();
1421 } else if (cause instanceof ClosedChannelException) {
tom7ef8ff92014-09-17 13:08:06 -07001422 log.debug("Channel for sw {} already closed", getSwitchInfoString());
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001423 } else if (cause instanceof IOException) {
1424 if (!cause.getMessage().equals(RESET_BY_PEER) &&
1425 !cause.getMessage().equals(BROKEN_PIPE)) {
Thomas Vachuskae9af1f42015-07-06 08:42:18 -07001426 log.error("Disconnecting switch {} due to IO Error: {}",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001427 getSwitchInfoString(), cause.getMessage());
Thomas Vachuskae9af1f42015-07-06 08:42:18 -07001428 if (log.isDebugEnabled()) {
1429 // still print stack trace if debug is enabled
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001430 log.debug("StackTrace for previous Exception: ", cause);
Thomas Vachuskae9af1f42015-07-06 08:42:18 -07001431 }
tom7ef8ff92014-09-17 13:08:06 -07001432 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001433 ctx.channel().close();
1434 } else if (cause instanceof SwitchStateException) {
tom7ef8ff92014-09-17 13:08:06 -07001435 log.error("Disconnecting switch {} due to switch state error: {}",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001436 getSwitchInfoString(), cause.getMessage());
tom7ef8ff92014-09-17 13:08:06 -07001437 if (log.isDebugEnabled()) {
1438 // still print stack trace if debug is enabled
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001439 log.debug("StackTrace for previous Exception: ", cause);
tom7ef8ff92014-09-17 13:08:06 -07001440 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001441 ctx.channel().close();
1442 } else if (cause instanceof OFParseError) {
tom7ef8ff92014-09-17 13:08:06 -07001443 log.error("Disconnecting switch "
1444 + getSwitchInfoString() +
1445 " due to message parse failure",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001446 cause);
1447 ctx.channel().close();
1448 } else if (cause instanceof RejectedExecutionException) {
tom7ef8ff92014-09-17 13:08:06 -07001449 log.warn("Could not process message: queue full");
1450 } else {
1451 log.error("Error while processing message from switch "
1452 + getSwitchInfoString()
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001453 + "state " + this.state, cause);
1454 ctx.channel().close();
tom7ef8ff92014-09-17 13:08:06 -07001455 }
1456 }
1457
1458 @Override
1459 public String toString() {
1460 return getSwitchInfoString();
1461 }
1462
pier063e9032019-11-22 20:51:26 +01001463 // We have reduced the idle period, the idea is to use
1464 // the IdleHandler to perform also some sanity checks.
1465 // Previous code is still executed with the same frequency
1466 // which is IDLE_INTERVAL * MAX_IDLE_RETRY of inactivity
Ray Milkey986a47a2018-01-25 11:38:51 -08001467 private void channelIdle(ChannelHandlerContext ctx,
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001468 IdleStateEvent e)
Ray Milkey986a47a2018-01-25 11:38:51 -08001469 throws IOException {
pier063e9032019-11-22 20:51:26 +01001470 // dispatcher terminated for some reason, restart
1471 if (dispatcherHandle.isDone()) {
1472 dispatcherHandle = dispatcher.submit(new Dispatcher());
Charles Chan982d3902018-03-21 14:58:53 -07001473 }
pier063e9032019-11-22 20:51:26 +01001474 // drain the backlog
1475 processDispatchBacklogQueue();
1476 // Original timeout reached
1477 if (--maxIdleRetry == 0) {
1478 maxIdleRetry = MAX_IDLE_RETRY;
1479 // Factory can be null if the channel goes idle during initial handshake. Since the switch
1480 // is not even initialized properly, we just skip this and disconnect the channel.
1481 if (factory != null) {
1482 // send an echo request each time idle_timeout * TICK
1483 OFMessage m = factory.buildEchoRequest().build();
1484 log.info("Sending Echo Request on idle channel: {}", ctx.channel());
1485 // XXX S some problems here -- echo request has no transaction id, and
1486 // echo reply is not correlated to the echo request.
1487 ctx.writeAndFlush(Collections.singletonList(m), ctx.voidPromise());
1488 }
1489 state.processIdle(this);
1490 }
tom7ef8ff92014-09-17 13:08:06 -07001491 }
1492
1493 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001494 public void userEventTriggered(ChannelHandlerContext ctx,
1495 Object evt)
tom7ef8ff92014-09-17 13:08:06 -07001496 throws Exception {
pier63295402019-10-24 16:53:25 +02001497 // If the connection is READER/WRITER idle try to send an echo request
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001498 if (evt instanceof IdleStateEvent) {
pier063e9032019-11-22 20:51:26 +01001499 log.debug("Channel {} is {}", ctx.channel(), ((IdleStateEvent) evt).state());
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001500 channelIdle(ctx, (IdleStateEvent) evt);
pier63295402019-10-24 16:53:25 +02001501 } else {
1502 super.userEventTriggered(ctx, evt);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001503 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001504 }
1505
1506 // SimpleChannelInboundHandler without dependency to TypeParameterMatcher
1507 @Override
1508 public void channelRead(ChannelHandlerContext ctx,
1509 Object msg) throws Exception {
1510
1511 boolean release = true;
pier063e9032019-11-22 20:51:26 +01001512 maxIdleRetry = MAX_IDLE_RETRY;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001513 try {
1514 if (msg instanceof OFMessage) {
1515 // channelRead0 inlined
1516 state.processOFMessage(this, (OFMessage) msg);
1517 } else {
1518 release = false;
1519 ctx.fireChannelRead(msg);
tom7ef8ff92014-09-17 13:08:06 -07001520 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001521 } finally {
1522 if (release) {
1523 ReferenceCountUtil.release(msg);
1524 }
tom7ef8ff92014-09-17 13:08:06 -07001525 }
1526 }
1527
1528
1529
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001530
tom7ef8ff92014-09-17 13:08:06 -07001531 //*************************
1532 // Channel utility methods
1533 //*************************
1534
1535 /**
1536 * Is this a state in which the handshake has completed?
Anton Chigrin4af4f872019-01-14 17:29:56 +02001537 *
tom7ef8ff92014-09-17 13:08:06 -07001538 * @return true if the handshake is complete
1539 */
1540 public boolean isHandshakeComplete() {
1541 return this.state.isHandshakeComplete();
1542 }
1543
Anton Chigrin4af4f872019-01-14 17:29:56 +02001544 /**
1545 * Increment totalCount variable and send signal to executor.
1546 */
1547 private void incrementAndSignal() {
1548 try {
1549 totalCount.incrementAndGet();
1550 takeLock.lockInterruptibly();
1551 try {
1552 notEmpty.signal();
1553 } finally {
1554 takeLock.unlock();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001555 }
Anton Chigrin4af4f872019-01-14 17:29:56 +02001556 } catch (InterruptedException e) {
1557 e.printStackTrace();
1558 }
1559 }
1560
1561 /**
1562 * Try to push OpenFlow message to queue.
1563 *
1564 * @param message OpenFlow message
1565 * @param idQueue id of Queue
1566 * @return true if message was successful added to queue
1567 */
1568 private boolean pushMessageToQueue(OFMessage message, int idQueue) {
1569 if (!dispatchQueuesMapProducer.get(idQueue).offer(message)) {
1570 return false;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001571 } else {
Anton Chigrin4af4f872019-01-14 17:29:56 +02001572 incrementAndSignal();
1573 return true;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001574 }
Anton Chigrin4af4f872019-01-14 17:29:56 +02001575 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001576
Anton Chigrin4af4f872019-01-14 17:29:56 +02001577 /**
1578 * Process backlog - move messages from backlog to default queue.
1579 *
1580 * @return true if whole backlog was processed, otherwise false
1581 */
1582 private boolean processDispatchBacklogQueue() {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001583 while (!dispatchBacklog.isEmpty()) {
Anton Chigrin4af4f872019-01-14 17:29:56 +02001584 OFMessage msgFromBacklog = dispatchBacklog.removeFirst();
1585 if (!pushMessageToQueue(msgFromBacklog, NUM_OF_QUEUES - 1)) {
1586 dispatchBacklog.addFirst(msgFromBacklog);
1587 return false;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001588 }
1589 }
Anton Chigrin4af4f872019-01-14 17:29:56 +02001590 return true;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001591
Anton Chigrin4af4f872019-01-14 17:29:56 +02001592 }
1593
1594 /**
1595 * Parse OpenFlow message context for get Ethernet packet.
1596 *
1597 * @param message OpenFlow message
1598 * @return parsed Ethernet packet
1599 */
1600 private Ethernet parsePacketInMessage(OFMessage message) {
1601 OpenFlowPacketContext pktCtx = DefaultOpenFlowPacketContext
1602 .packetContextFromPacketIn(sw, (OFPacketIn) message);
1603 DeviceId id = DeviceId.deviceId(Dpid.uri(pktCtx.dpid().value()));
1604 DefaultInboundPacket inPkt = new DefaultInboundPacket(
1605 new ConnectPoint(id, PortNumber.portNumber(pktCtx.inPort())),
1606 pktCtx.parsed(), ByteBuffer.wrap(pktCtx.unparsed()),
1607 pktCtx.cookie());
1608 return inPkt.parsed();
1609 }
1610
1611 /**
1612 * Classify the Ethernet packet for membership on one of the queues.
1613 *
1614 * @param packet ethernet packet
1615 * @return Id of destination Queue
1616 */
1617 private int classifyEthernetPacket(Ethernet packet) {
1618 for (Set<OpenFlowClassifier> classifiers : this.messageClassifiersMapProducer) {
1619 for (OpenFlowClassifier classifier : classifiers) {
1620 if (classifier.ethernetType() == packet.getEtherType()) {
1621 return classifier.idQueue();
1622 }
1623 }
1624 }
1625 return NUM_OF_QUEUES - 1;
1626 }
1627
1628 /**
1629 * Process messages from dispatch queues.
1630 *
1631 * @param queuesSize count of messages in all queues
1632 */
1633 private void processMessages(int queuesSize) {
1634 List<OFMessage> msgs = new ArrayList<>();
1635 int processed;
1636 do {
1637 processed = 0;
1638 while (processed < queuesSize) {
1639 for (LinkedBlockingMessagesQueue<OFMessage> queue :
1640 dispatchQueuesMapProducer.values()) {
1641 processed += queue.drainTo(msgs);
1642 }
1643 }
1644
1645 msgs.forEach(sw::handleMessage);
1646 msgs.clear();
1647 /* Decrement conditional variable */
1648 queuesSize = totalCount.addAndGet(-1 * processed);
1649 } while (queuesSize > 0);
1650 }
1651
1652 private void dispatchMessage(OFMessage m) {
1653 log.debug("Begin dispatch OpenFlow Message");
1654 boolean backlogEmpty = processDispatchBacklogQueue();
1655 if (m.getType() == OFType.PACKET_IN) {
1656 Ethernet pkt = parsePacketInMessage(m);
1657 pushMessageToQueue(m, classifyEthernetPacket(pkt));
1658 } else {
1659 if (!backlogEmpty || !pushMessageToQueue(m, NUM_OF_QUEUES - 1)) {
1660 dispatchBacklog.offer(m);
1661 }
1662 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001663
1664 if (dispatcherHandle.isDone()) {
1665 // dispatcher terminated for some reason, restart
pier063e9032019-11-22 20:51:26 +01001666 dispatcherHandle = dispatcher.submit(new Dispatcher());
1667 }
1668 }
1669
1670 private final class Dispatcher implements Runnable {
1671 // dispatch loop
1672 @Override
1673 public void run() {
1674 try {
1675 for (;;) {
1676 int tc = 0;
1677 takeLock.lockInterruptibly();
1678 try {
1679 while ((tc = totalCount.get()) == 0) {
1680 notEmpty.await();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001681 }
pier063e9032019-11-22 20:51:26 +01001682 } finally {
1683 takeLock.unlock();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001684 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001685
pier063e9032019-11-22 20:51:26 +01001686 processMessages(tc);
1687 }
1688 } catch (InterruptedException e) {
1689 log.warn("Dispatcher interrupted");
1690 Thread.currentThread().interrupt();
1691 // interrupted. gracefully shutting down
1692 return;
1693 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001694 }
tom7ef8ff92014-09-17 13:08:06 -07001695 }
1696
1697 /**
1698 * Return a string describing this switch based on the already available
1699 * information (DPID and/or remote socket).
1700 * @return display string
1701 */
1702 private String getSwitchInfoString() {
1703 if (sw != null) {
1704 return sw.toString();
1705 }
1706 String channelString;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001707 if (channel == null || channel.remoteAddress() == null) {
tom7ef8ff92014-09-17 13:08:06 -07001708 channelString = "?";
1709 } else {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001710 channelString = channel.remoteAddress().toString();
tom7ef8ff92014-09-17 13:08:06 -07001711 }
1712 String dpidString;
1713 if (featuresReply == null) {
1714 dpidString = "?";
1715 } else {
1716 dpidString = featuresReply.getDatapathId().toString();
1717 }
1718 return String.format("[%s DPID[%s]]", channelString, dpidString);
1719 }
1720
1721 /**
1722 * Update the channels state. Only called from the state machine.
1723 * TODO: enforce restricted state transitions
pieref3d74c2019-07-16 15:52:50 +02001724 * @param state new state
tom7ef8ff92014-09-17 13:08:06 -07001725 */
pieref3d74c2019-07-16 15:52:50 +02001726 void setState(ChannelState state) {
tom7ef8ff92014-09-17 13:08:06 -07001727 this.state = state;
Yuta HIGUCHI10f45132017-03-01 17:09:32 -08001728 this.lastStateChange = System.currentTimeMillis();
tom7ef8ff92014-09-17 13:08:06 -07001729 }
1730
Brian O'Connorf69e3e32018-05-10 02:25:09 -07001731 private boolean setDpid(Long dpid, Channel channel) {
1732 ChannelHandlerContext sslContext = channel.pipeline().context(SslHandler.class);
1733 if (sslContext != null) {
1734 try {
1735 SslHandler sslHandler = (SslHandler) sslContext.handler();
1736 Certificate[] certs = sslHandler.engine().getSession().getPeerCertificates();
1737 Certificate cert = certs.length > 0 ? certs[0] : null;
1738 if (!controller.isValidCertificate(dpid, cert)) {
1739 return false;
1740 }
1741 } catch (SSLPeerUnverifiedException e) {
1742 log.info("Switch with dpid {} is an unverified SSL peer.", dpid, e);
1743 return false;
1744 }
1745 }
1746 this.thisdpid = dpid;
1747 return true;
1748 }
1749
tom7ef8ff92014-09-17 13:08:06 -07001750 /**
1751 * Send hello message to the switch using the handshake transactions ids.
1752 * @throws IOException
1753 */
1754 private void sendHandshakeHelloMessage() throws IOException {
1755 // The OF protocol requires us to start things off by sending the highest
1756 // version of the protocol supported.
1757
Yuta HIGUCHI6512f3e2017-05-18 17:21:24 -07001758 // bitmap represents OF1.0, OF1.3, OF1.4, and OF1.5
tom7ef8ff92014-09-17 13:08:06 -07001759 // see Sec. 7.5.1 of the OF1.3.4 spec
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001760 U32 bitmap = U32.ofRaw((0b1 << OFVersion.OF_10.getWireVersion()) |
1761 (0b1 << OFVersion.OF_13.getWireVersion()) |
Yuta HIGUCHI6512f3e2017-05-18 17:21:24 -07001762 (0b1 << OFVersion.OF_14.getWireVersion()) |
1763 (0b1 << OFVersion.OF_15.getWireVersion()));
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001764 OFVersion version = Optional.ofNullable(ofVersion).orElse(OFVersion.OF_13);
1765 OFHelloElem hem = OFFactories.getFactory(version)
1766 .buildHelloElemVersionbitmap()
tom7ef8ff92014-09-17 13:08:06 -07001767 .setBitmaps(Collections.singletonList(bitmap))
1768 .build();
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001769 OFMessage.Builder mb = OFFactories.getFactory(version)
1770 .buildHello()
tom7ef8ff92014-09-17 13:08:06 -07001771 .setXid(this.handshakeTransactionIds--)
1772 .setElements(Collections.singletonList(hem));
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001773 log.info("Sending {} Hello to {}", version, channel.remoteAddress());
1774 channel.writeAndFlush(Collections.singletonList(mb.build()));
tom7ef8ff92014-09-17 13:08:06 -07001775 }
1776
1777 /**
1778 * Send featuresRequest msg to the switch using the handshake transactions ids.
1779 * @throws IOException
1780 */
1781 private void sendHandshakeFeaturesRequestMessage() throws IOException {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001782 log.debug("Sending FEATURES_REQUEST to {}", channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001783 OFMessage m = factory.buildFeaturesRequest()
1784 .setXid(this.handshakeTransactionIds--)
1785 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001786 channel.writeAndFlush(Collections.singletonList(m));
tom7ef8ff92014-09-17 13:08:06 -07001787 }
1788
1789 /**
1790 * Send the configuration requests to tell the switch we want full
1791 * packets.
1792 * @throws IOException
1793 */
1794 private void sendHandshakeSetConfig() throws IOException {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001795 log.debug("Sending CONFIG_REQUEST to {}", channel.remoteAddress());
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001796 List<OFMessage> msglist = new ArrayList<>(3);
tom7ef8ff92014-09-17 13:08:06 -07001797
1798 // Ensure we receive the full packet via PacketIn
1799 // FIXME: We don't set the reassembly flags.
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001800 // Only send config to switches to send full packets, if they have a buffer.
Michael Jarschel7f521a32015-08-12 16:31:07 +02001801 // Saves a packet & OFSetConfig can't be handled by certain switches.
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001802 if (this.featuresReply.getNBuffers() > 0) {
Michael Jarschel7f521a32015-08-12 16:31:07 +02001803 OFSetConfig sc = factory
1804 .buildSetConfig()
1805 .setMissSendLen((short) 0xffff)
1806 .setXid(this.handshakeTransactionIds--)
1807 .build();
1808 msglist.add(sc);
1809 }
tom7ef8ff92014-09-17 13:08:06 -07001810
1811 // Barrier
1812 OFBarrierRequest br = factory
1813 .buildBarrierRequest()
1814 .setXid(this.handshakeTransactionIds--)
1815 .build();
1816 msglist.add(br);
1817
1818 // Verify (need barrier?)
1819 OFGetConfigRequest gcr = factory
1820 .buildGetConfigRequest()
1821 .setXid(this.handshakeTransactionIds--)
1822 .build();
1823 msglist.add(gcr);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001824 channel.writeAndFlush(msglist);
tom7ef8ff92014-09-17 13:08:06 -07001825 }
1826
1827 /**
1828 * send a description state request.
1829 * @throws IOException
1830 */
1831 private void sendHandshakeDescriptionStatsRequest() throws IOException {
1832 // Get Description to set switch-specific flags
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001833 log.debug("Sending DESC_STATS_REQUEST to {}", channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001834 OFDescStatsRequest dreq = factory
1835 .buildDescStatsRequest()
1836 .setXid(handshakeTransactionIds--)
1837 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001838 channel.writeAndFlush(Collections.singletonList(dreq));
tom7ef8ff92014-09-17 13:08:06 -07001839 }
1840
Jordi Ortiz91477b82016-11-29 15:22:50 +01001841 /**
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001842 * send a meter features request.
1843 *
Jordi Ortiz91477b82016-11-29 15:22:50 +01001844 * @throws IOException
1845 */
1846 private void sendMeterFeaturesRequest() throws IOException {
1847 // Get meter features including the MaxMeters value available for the device
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001848 OFFactory factory = OFFactories.getFactory(ofVersion);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001849 log.debug("Sending METER_FEATURES_REQUEST to {}", channel.remoteAddress());
Jordi Ortiz91477b82016-11-29 15:22:50 +01001850 OFMeterFeaturesStatsRequest mfreq = factory
1851 .buildMeterFeaturesStatsRequest()
1852 .setXid(handshakeTransactionIds--)
1853 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001854 channel.writeAndFlush(Collections.singletonList(mfreq));
Jordi Ortiz91477b82016-11-29 15:22:50 +01001855 }
1856
tom7ef8ff92014-09-17 13:08:06 -07001857 private void sendHandshakeOFPortDescRequest() throws IOException {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001858 log.debug("Sending OF_PORT_DESC_REQUEST to {}", channel.remoteAddress());
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001859 // Get port description for 1.3+ switch
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001860 OFPortDescStatsRequest preq = factory
tom7ef8ff92014-09-17 13:08:06 -07001861 .buildPortDescStatsRequest()
1862 .setXid(handshakeTransactionIds--)
1863 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001864 channel.writeAndFlush(Collections.singletonList(preq));
tom7ef8ff92014-09-17 13:08:06 -07001865 }
1866
1867 ChannelState getStateForTesting() {
1868 return state;
1869 }
1870
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001871
1872 @Override
1873 public boolean isActive() {
1874 if (channel != null) {
1875 return channel.isActive();
1876 }
1877 return false;
1878 }
1879
1880 @Override
1881 public void closeSession() {
1882 if (channel != null) {
1883 channel.close();
1884 }
1885 }
1886
1887 @Override
1888 public boolean sendMsg(Iterable<OFMessage> msgs) {
1889 if (channel.isActive()) {
Laszlo Pappb68fe7e2017-11-24 17:06:59 +00001890 if (log.isTraceEnabled()) {
1891 log.trace("Sending messages for switch {} via openflow channel: {}", getSwitchInfoString(), msgs);
1892 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001893 channel.writeAndFlush(msgs, channel.voidPromise());
1894 return true;
1895 } else {
1896 log.warn("Dropping messages for switch {} because channel is not connected: {}",
1897 getSwitchInfoString(), msgs);
1898 return false;
1899 }
1900 }
1901
1902 @Override
1903 public CharSequence sessionInfo() {
1904 return channelId;
1905 }
1906
Anton Chigrin4af4f872019-01-14 17:29:56 +02001907 @Override
1908 public void addClassifier(OpenFlowClassifier classifier) {
1909 if (this.deviceId.equals(classifier.deviceId())) {
1910 log.debug("Add OpenFlow Classifier for switch {} to queue {} with type {}",
1911 classifier.deviceId().toString(), classifier.idQueue(), classifier.ethernetType());
1912 this.messageClassifiersMapProducer.get(classifier.idQueue()).add(classifier);
1913 }
1914 }
1915
1916 @Override
1917 public void removeClassifier(OpenFlowClassifier classifier) {
1918 if (this.deviceId.equals(classifier.deviceId())) {
1919 log.debug("Remove OpenFlow Classifier for switch {} from queue {} with type {}",
1920 classifier.deviceId().toString(), classifier.idQueue(), classifier.ethernetType());
1921 this.messageClassifiersMapProducer.get(classifier.idQueue()).remove(classifier);
1922 }
1923 }
1924
1925 /**
1926 * Init classifier configuration for the switch. Use stored configuration if exist.
1927 * Otherwise add LLDP and BDDP classifiers for Queue N0.
1928 */
1929 private void initClassifiers() {
1930 try {
1931 openFlowManager = DefaultServiceDirectory.getService(OpenFlowService.class);
1932 DeviceId did = DeviceId.deviceId(uri(thisdpid));
1933 Set<OpenFlowClassifier> classifiers = openFlowManager.getClassifiersByDeviceId(did);
1934 if (classifiers == null) {
1935 OpenFlowClassifier classifier =
1936 new OpenFlowClassifier.Builder(did, 0).ethernetType(TYPE_LLDP).build();
1937 openFlowManager.add(classifier);
1938 classifier = new OpenFlowClassifier.Builder(did, 0).ethernetType(TYPE_BSN).build();
1939 openFlowManager.add(classifier);
1940 } else {
1941 this.messageClassifiersMapProducer.forEach((v) -> {
1942 v.clear();
1943 });
1944 classifiers.forEach((c) -> {
1945 messageClassifiersMapProducer.get(c.idQueue()).add(c);
1946 });
1947 }
1948 } catch (Exception e) {
1949 log.error("Initialize default classifier failed: {}", e.toString());
1950 e.printStackTrace();
1951 }
1952 }
tom7ef8ff92014-09-17 13:08:06 -07001953}