blob: 01b4f1ac3ad40764a40efc7dcd7a8383d12ec572 [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 Chigrinbf14b372019-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 Chigrinbf14b372019-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 Chigrinbf14b372019-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 Chigrinbf14b372019-01-14 17:29:56 +020034import java.util.Map;
Yuta HIGUCHI2341e602017-03-08 20:10:08 -080035import java.util.Optional;
Anton Chigrinbf14b372019-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 Chigrinbf14b372019-01-14 17:29:56 +020039import java.util.concurrent.ConcurrentHashMap;
tom7ef8ff92014-09-17 13:08:06 -070040import java.util.concurrent.CopyOnWriteArrayList;
Anton Chigrinbf14b372019-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 Chigrinbf14b372019-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 Chigrinbf14b372019-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 Chigrinbf14b372019-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 Chigrinbf14b372019-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";
pierc684ee12019-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 Chigrinbf14b372019-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 Chigrinbf14b372019-01-14 17:29:56 +0200177 @Reference(cardinality = ReferenceCardinality.MANDATORY)
178 private OpenFlowService openFlowManager;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700179
Anton Chigrinbf14b372019-01-14 17:29:56 +0200180 private static final int BACKLOG_READ_BUFFER_DEFAULT = 1000;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700181
182 /**
Anton Chigrinbf14b372019-01-14 17:29:56 +0200183 * Map with all LinkedBlockingMessagesQueue queues which contains OFMessages.
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700184 */
Anton Chigrinbf14b372019-01-14 17:29:56 +0200185 private Map<Integer, LinkedBlockingMessagesQueue<OFMessage>> dispatchQueuesMapProducer = new ConcurrentHashMap<>();
186
187 /**
188 * OFMessage classifiers map.
189 */
190 private List<Set<OpenFlowClassifier>> messageClassifiersMapProducer =
191 new CopyOnWriteArrayList<Set<OpenFlowClassifier>>();
192
193
194 /**
195 * Lock held by take, poll, etc.
196 */
197 private final ReentrantLock takeLock = new ReentrantLock();
198
199 /**
200 * Wait queue for waiting takes.
201 */
202 private final Condition notEmpty = takeLock.newCondition();
203
204 /**
205 * Current number of elements in enabled sub-queues.
206 */
207 private final AtomicInteger totalCount = new AtomicInteger();
208
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700209
210 /**
211 * Single thread executor for OFMessage dispatching.
212 *
213 * Gets initialized on channelActive, shutdown on channelInactive.
214 */
215 private ExecutorService dispatcher;
216
217 /**
218 * Handle for dispatcher thread.
219 * <p>
220 * Should only be touched from the Channel I/O thread
221 */
222 private Future<?> dispatcherHandle = CompletableFuture.completedFuture(null);
223
224 /**
225 * Dispatch backlog.
226 * <p>
227 * Should only be touched from the Channel I/O thread
228 */
Anton Chigrinbf14b372019-01-14 17:29:56 +0200229 private final Deque<OFMessage> dispatchBacklog;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700230
tom7ef8ff92014-09-17 13:08:06 -0700231 /**
232 * Create a new unconnected OFChannelHandler.
Thomas Vachuskab14c77a2014-11-04 18:08:01 -0800233 * @param controller parent controller
tom7ef8ff92014-09-17 13:08:06 -0700234 */
235 OFChannelHandler(Controller controller) {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700236
tom7ef8ff92014-09-17 13:08:06 -0700237 this.controller = controller;
238 this.state = ChannelState.INIT;
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800239 this.pendingPortStatusMsg = new CopyOnWriteArrayList<>();
240 this.portDescReplies = new ArrayList<>();
tom7ef8ff92014-09-17 13:08:06 -0700241 duplicateDpidFound = Boolean.FALSE;
Anton Chigrinbf14b372019-01-14 17:29:56 +0200242 //Initialize queues and classifiers
243 dispatchBacklog = new LinkedBlockingDeque<>(BACKLOG_READ_BUFFER_DEFAULT);
244 for (int i = 0; i < NUM_OF_QUEUES; i++) {
245 if (controller.getQueueSize(i) > 0) {
246 dispatchQueuesMapProducer.put(i,
247 new LinkedBlockingMessagesQueue<>(i, controller.getQueueSize(i), controller.getBulkSize(i)));
248 }
249 if (i != NUM_OF_QUEUES) {
250 messageClassifiersMapProducer.add(i, new CopyOnWriteArraySet<>());
251 }
252 }
tom7ef8ff92014-09-17 13:08:06 -0700253 }
254
255
256
257 // XXX S consider if necessary
258 public void disconnectSwitch() {
259 sw.disconnectSwitch();
260 }
261
262
263
264 //*************************
265 // Channel State Machine
266 //*************************
267
268 /**
269 * The state machine for handling the switch/channel state. All state
270 * transitions should happen from within the state machine (and not from other
271 * parts of the code)
272 */
273 enum ChannelState {
274 /**
275 * Initial state before channel is connected.
276 */
277 INIT(false) {
278 @Override
279 void processOFMessage(OFChannelHandler h, OFMessage m)
280 throws IOException, SwitchStateException {
281 illegalMessageReceived(h, m);
282 }
283
284 @Override
285 void processOFError(OFChannelHandler h, OFErrorMsg m)
286 throws IOException {
287 // need to implement since its abstract but it will never
288 // be called
289 }
290
291 @Override
292 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
293 throws IOException {
294 unhandledMessageReceived(h, m);
295 }
296 },
297
298 /**
299 * We send a OF 1.3 HELLO to the switch and wait for a Hello from the switch.
300 * Once we receive the reply, we decide on OF 1.3 or 1.0 switch - no other
301 * protocol version is accepted.
302 * We send an OFFeaturesRequest depending on the protocol version selected
303 * Next state is WAIT_FEATURES_REPLY
304 */
305 WAIT_HELLO(false) {
306 @Override
307 void processOFHello(OFChannelHandler h, OFHello m)
308 throws IOException {
309 // TODO We could check for the optional bitmap, but for now
310 // we are just checking the version number.
Chip Boling68bc6562015-07-06 10:00:01 -0500311 if (m.getVersion().getWireVersion() >= OFVersion.OF_13.getWireVersion()) {
312 log.debug("Received {} Hello from {} - switching to OF "
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800313 + "version 1.3+", m.getVersion(),
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700314 h.channel.remoteAddress());
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800315 h.ofVersion = m.getVersion();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700316 h.factory = OFFactories.getFactory(h.ofVersion);
alshabib70fc7fb2015-01-06 11:04:29 -0800317 h.sendHandshakeHelloMessage();
Chip Boling68bc6562015-07-06 10:00:01 -0500318 } else if (m.getVersion().getWireVersion() >= OFVersion.OF_10.getWireVersion()) {
alshabib09d48be2014-10-03 15:43:33 -0700319 log.debug("Received {} Hello from {} - switching to OF "
tom7ef8ff92014-09-17 13:08:06 -0700320 + "version 1.0", m.getVersion(),
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700321 h.channel.remoteAddress());
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800322 h.ofVersion = m.getVersion();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700323 h.factory = OFFactories.getFactory(h.ofVersion);
alshabib70fc7fb2015-01-06 11:04:29 -0800324 OFHello hi =
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700325 h.factory.buildHello()
alshabib70fc7fb2015-01-06 11:04:29 -0800326 .setXid(h.handshakeTransactionIds--)
327 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700328 h.channel.writeAndFlush(Collections.singletonList(hi));
tom7ef8ff92014-09-17 13:08:06 -0700329 } else {
330 log.error("Received Hello of version {} from switch at {}. "
331 + "This controller works with OF1.0 and OF1.3 "
332 + "switches. Disconnecting switch ...",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700333 m.getVersion(), h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -0700334 h.channel.disconnect();
335 return;
336 }
337 h.sendHandshakeFeaturesRequestMessage();
338 h.setState(WAIT_FEATURES_REPLY);
339 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700340
tom7ef8ff92014-09-17 13:08:06 -0700341 @Override
342 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
343 throws IOException, SwitchStateException {
344 illegalMessageReceived(h, m);
345 }
346 @Override
347 void processOFStatisticsReply(OFChannelHandler h,
348 OFStatsReply m)
349 throws IOException, SwitchStateException {
350 illegalMessageReceived(h, m);
351 }
352 @Override
353 void processOFError(OFChannelHandler h, OFErrorMsg m) {
354 logErrorDisconnect(h, m);
355 }
356
357 @Override
358 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
359 throws IOException {
360 unhandledMessageReceived(h, m);
361 }
362 },
363
364
365 /**
366 * We are waiting for a features reply message. Once we receive it, the
367 * behavior depends on whether this is a 1.0 or 1.3 switch. For 1.0,
368 * we send a SetConfig request, barrier, and GetConfig request and the
369 * next state is WAIT_CONFIG_REPLY. For 1.3, we send a Port description
370 * request and the next state is WAIT_PORT_DESC_REPLY.
371 */
372 WAIT_FEATURES_REPLY(false) {
373 @Override
Brian O'Connorf69e3e32018-05-10 02:25:09 -0700374 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
tom7ef8ff92014-09-17 13:08:06 -0700375 throws IOException {
Brian O'Connorf69e3e32018-05-10 02:25:09 -0700376 Long dpid = m.getDatapathId().getLong();
377 if (!h.setDpid(dpid, h.channel)) {
378 log.error("Switch presented invalid certificate for dpid {}. Disconnecting",
379 dpid);
380 h.channel.disconnect();
381 return;
382 }
Anton Chigrinbf14b372019-01-14 17:29:56 +0200383 h.deviceId = DeviceId.deviceId(uri(h.thisdpid));
alshabib09d48be2014-10-03 15:43:33 -0700384 log.debug("Received features reply for switch at {} with dpid {}",
tom7ef8ff92014-09-17 13:08:06 -0700385 h.getSwitchInfoString(), h.thisdpid);
386
387 h.featuresReply = m; //temp store
388 if (h.ofVersion == OFVersion.OF_10) {
389 h.sendHandshakeSetConfig();
390 h.setState(WAIT_CONFIG_REPLY);
391 } else {
392 //version is 1.3, must get switchport information
393 h.sendHandshakeOFPortDescRequest();
394 h.setState(WAIT_PORT_DESC_REPLY);
395 }
396 }
397 @Override
398 void processOFStatisticsReply(OFChannelHandler h,
399 OFStatsReply m)
400 throws IOException, SwitchStateException {
401 illegalMessageReceived(h, m);
402 }
403 @Override
404 void processOFError(OFChannelHandler h, OFErrorMsg m) {
405 logErrorDisconnect(h, m);
406 }
407
408 @Override
409 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
410 throws IOException {
Thomas Vachuska39274462014-12-02 13:23:50 -0800411 h.pendingPortStatusMsg.add(m);
tom7ef8ff92014-09-17 13:08:06 -0700412 }
413 },
414
415 /**
416 * We are waiting for a description of the 1.3 switch ports.
417 * Once received, we send a SetConfig request
418 * Next State is WAIT_CONFIG_REPLY
419 */
420 WAIT_PORT_DESC_REPLY(false) {
421
422 @Override
423 void processOFStatisticsReply(OFChannelHandler h, OFStatsReply m)
424 throws SwitchStateException {
425 // Read port description
426 if (m.getStatsType() != OFStatsType.PORT_DESC) {
427 log.warn("Expecting port description stats but received stats "
428 + "type {} from {}. Ignoring ...", m.getStatsType(),
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700429 h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -0700430 return;
431 }
432 if (m.getFlags().contains(OFStatsReplyFlags.REPLY_MORE)) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700433 log.debug("Stats reply indicates more stats from sw {} for "
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700434 + "port description",
tom7ef8ff92014-09-17 13:08:06 -0700435 h.getSwitchInfoString());
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800436 h.portDescReplies.add((OFPortDescStatsReply) m);
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700437 return;
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800438 } else {
439 h.portDescReplies.add((OFPortDescStatsReply) m);
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700440 }
441 //h.portDescReply = (OFPortDescStatsReply) m; // temp store
Saurav Das45f48152018-01-18 12:07:33 -0800442 log.debug("Received port desc reply for switch at {}: {}",
443 h.getSwitchInfoString(),
444 ((OFPortDescStatsReply) m).getEntries());
tom7ef8ff92014-09-17 13:08:06 -0700445 try {
446 h.sendHandshakeSetConfig();
447 } catch (IOException e) {
448 log.error("Unable to send setConfig after PortDescReply. "
449 + "Error: {}", e.getMessage());
450 }
451 h.setState(WAIT_CONFIG_REPLY);
452 }
453
454 @Override
455 void processOFError(OFChannelHandler h, OFErrorMsg m)
456 throws IOException, SwitchStateException {
457 logErrorDisconnect(h, m);
458
459 }
460
461 @Override
462 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
463 throws IOException, SwitchStateException {
Thomas Vachuska39274462014-12-02 13:23:50 -0800464 h.pendingPortStatusMsg.add(m);
tom7ef8ff92014-09-17 13:08:06 -0700465
466 }
467 },
468
469 /**
470 * We are waiting for a config reply message. Once we receive it
471 * we send a DescriptionStatsRequest to the switch.
472 * Next state: WAIT_DESCRIPTION_STAT_REPLY
473 */
474 WAIT_CONFIG_REPLY(false) {
475 @Override
476 void processOFGetConfigReply(OFChannelHandler h, OFGetConfigReply m)
477 throws IOException {
478 if (m.getMissSendLen() == 0xffff) {
479 log.trace("Config Reply from switch {} confirms "
480 + "miss length set to 0xffff",
481 h.getSwitchInfoString());
482 } else {
483 // FIXME: we can't really deal with switches that don't send
484 // full packets. Shouldn't we drop the connection here?
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800485 log.warn("Config Reply from switch {} has "
tom7ef8ff92014-09-17 13:08:06 -0700486 + "miss length set to {}",
487 h.getSwitchInfoString(),
488 m.getMissSendLen());
489 }
Jordi Ortiz91477b82016-11-29 15:22:50 +0100490
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800491 nextState(h);
492 }
493
494 /**
495 * Transition to next state based on OF version.
496 *
497 * @param h current channel handler
498 * @throws IOException
499 */
500 private void nextState(OFChannelHandler h) throws IOException {
501 if (h.ofVersion.getWireVersion() >= OFVersion.OF_13.getWireVersion()) {
Jordi Ortiz91477b82016-11-29 15:22:50 +0100502 // Meters were introduced in OpenFlow 1.3
503 h.sendMeterFeaturesRequest();
504 h.setState(WAIT_METER_FEATURES_REPLY);
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800505 } else {
Jordi Ortiz91477b82016-11-29 15:22:50 +0100506 h.sendHandshakeDescriptionStatsRequest();
507 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
508 }
tom7ef8ff92014-09-17 13:08:06 -0700509 }
510
511 @Override
512 void processOFBarrierReply(OFChannelHandler h, OFBarrierReply m) {
513 // do nothing;
514 }
515
516 @Override
517 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
518 throws IOException, SwitchStateException {
519 illegalMessageReceived(h, m);
520 }
Anton Chigrinbf14b372019-01-14 17:29:56 +0200521
tom7ef8ff92014-09-17 13:08:06 -0700522 @Override
523 void processOFStatisticsReply(OFChannelHandler h,
524 OFStatsReply m)
525 throws IOException, SwitchStateException {
526 log.error("Received multipart(stats) message sub-type {}",
527 m.getStatsType());
528 illegalMessageReceived(h, m);
529 }
530
531 @Override
532 void processOFError(OFChannelHandler h, OFErrorMsg m) {
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800533 if (m.getErrType() == OFErrorType.BAD_REQUEST) {
534 OFBadRequestErrorMsg badRequest = (OFBadRequestErrorMsg) m;
535 if (badRequest.getCode() == OFBadRequestCode.BAD_TYPE) {
536 log.debug("{} does not support GetConfig, moving on", h.getSwitchInfoString());
537 try {
538 nextState(h);
539 return;
540 } catch (IOException e) {
541 log.error("Exception thrown transitioning to next", e);
542 logErrorDisconnect(h, m);
543 }
544 }
545 }
tom7ef8ff92014-09-17 13:08:06 -0700546 logErrorDisconnect(h, m);
547 }
548
549 @Override
550 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
551 throws IOException {
552 h.pendingPortStatusMsg.add(m);
553 }
554 },
555
556
557 /**
558 * We are waiting for a OFDescriptionStat message from the switch.
559 * Once we receive any stat message we try to parse it. If it's not
560 * a description stats message we disconnect. If its the expected
561 * description stats message, we:
562 * - use the switch driver to bind the switch and get an IOFSwitch instance
563 * - setup the IOFSwitch instance
564 * - add switch controller and send the initial role
565 * request to the switch.
566 * Next state: WAIT_INITIAL_ROLE
567 * In the typical case, where switches support role request messages
568 * the next state is where we expect the role reply message.
569 * In the special case that where the switch does not support any kind
570 * of role request messages, we don't send a role message, but we do
571 * request mastership from the registry service. This controller
572 * should become master once we hear back from the registry service.
573 * All following states will have a h.sw instance!
574 */
575 WAIT_DESCRIPTION_STAT_REPLY(false) {
576 @Override
577 void processOFStatisticsReply(OFChannelHandler h, OFStatsReply m)
578 throws SwitchStateException {
579 // Read description, if it has been updated
580 if (m.getStatsType() != OFStatsType.DESC) {
581 log.warn("Expecting Description stats but received stats "
582 + "type {} from {}. Ignoring ...", m.getStatsType(),
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700583 h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -0700584 return;
585 }
tom7ef8ff92014-09-17 13:08:06 -0700586 OFDescStatsReply drep = (OFDescStatsReply) m;
Saurav Dasf9ba4222015-05-07 17:13:59 -0700587 log.info("Received switch description reply {} from switch at {}",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700588 drep, h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -0700589 // Here is where we differentiate between different kinds of switches
590 h.sw = h.controller.getOFSwitchInstance(h.thisdpid, drep, h.ofVersion);
591
Ray Milkey31b00482019-02-07 08:06:28 -0800592 if (h.sw == null) {
593 log.info("Switch not found for {}", h.thisdpid);
594 return;
595 }
596
tom7ef8ff92014-09-17 13:08:06 -0700597 h.sw.setOFVersion(h.ofVersion);
598 h.sw.setFeaturesReply(h.featuresReply);
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700599 h.sw.setPortDescReplies(h.portDescReplies);
Jordi Ortiz91477b82016-11-29 15:22:50 +0100600 h.sw.setMeterFeaturesReply(h.meterFeaturesReply);
tom7ef8ff92014-09-17 13:08:06 -0700601 h.sw.setConnected(true);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700602 h.sw.setChannel(h);
Praseed Balakrishnana22eadf2014-10-20 14:21:45 -0700603// boolean success = h.sw.connectSwitch();
604//
605// if (!success) {
606// disconnectDuplicate(h);
607// return;
608// }
tom7ef8ff92014-09-17 13:08:06 -0700609 // set switch information
610
611
612
alshabib09d48be2014-10-03 15:43:33 -0700613 log.debug("Switch {} bound to class {}, description {}",
Ray Milkey6bc43c22015-11-06 13:22:38 -0800614 h.sw, h.sw.getClass(), drep);
tom7ef8ff92014-09-17 13:08:06 -0700615 //Put switch in EQUAL mode until we hear back from the global registry
616 //log.debug("Setting new switch {} to EQUAL and sending Role request",
617 // h.sw.getStringId());
618 //h.sw.activateEqualSwitch();
619 //h.setSwitchRole(RoleState.EQUAL);
620
621 h.sw.startDriverHandshake();
alshabib9eab22f2014-10-20 17:17:31 -0700622 if (h.sw.isDriverHandshakeComplete()) {
pierc684ee12019-07-16 15:52:50 +0200623 // We are not able to complete the connection for a dpid collision.
624 // Same device reconnecting or different device configured with
625 // the same dpid.
alshabib9eab22f2014-10-20 17:17:31 -0700626 if (!h.sw.connectSwitch()) {
pierc684ee12019-07-16 15:52:50 +0200627 // Disconnect from the device and return
alshabib9eab22f2014-10-20 17:17:31 -0700628 disconnectDuplicate(h);
pierc684ee12019-07-16 15:52:50 +0200629 return;
Anton Chigrinbf14b372019-01-14 17:29:56 +0200630 } else {
631 h.initClassifiers();
alshabib9eab22f2014-10-20 17:17:31 -0700632 }
Thomas Vachuska39274462014-12-02 13:23:50 -0800633 handlePendingPortStatusMessages(h);
alshabib9eab22f2014-10-20 17:17:31 -0700634 h.setState(ACTIVE);
635 } else {
636 h.setState(WAIT_SWITCH_DRIVER_SUB_HANDSHAKE);
637 }
tom7ef8ff92014-09-17 13:08:06 -0700638
639 }
640
641 @Override
642 void processOFError(OFChannelHandler h, OFErrorMsg m) {
643 logErrorDisconnect(h, m);
644 }
645
646 @Override
647 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
648 throws IOException, SwitchStateException {
649 illegalMessageReceived(h, m);
650 }
651
652 @Override
653 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
654 throws IOException {
655 h.pendingPortStatusMsg.add(m);
656 }
657 },
658
659
660 /**
661 * We are waiting for the respective switch driver to complete its
662 * configuration. Notice that we do not consider this to be part of the main
663 * switch-controller handshake. But we do consider it as a step that comes
664 * before we declare the switch as available to the controller.
665 * Next State: depends on the role of this controller for this switch - either
666 * MASTER or EQUAL.
667 */
668 WAIT_SWITCH_DRIVER_SUB_HANDSHAKE(true) {
669
670 @Override
671 void processOFError(OFChannelHandler h, OFErrorMsg m)
672 throws IOException {
673 // will never be called. We override processOFMessage
674 }
675
alshabibd7963912014-10-20 14:52:04 -0700676
677
tom7ef8ff92014-09-17 13:08:06 -0700678 @Override
679 void processOFMessage(OFChannelHandler h, OFMessage m)
680 throws IOException, SwitchStateException {
alshabibd7963912014-10-20 14:52:04 -0700681
682 if (h.sw.isDriverHandshakeComplete()) {
683 moveToActive(h);
alshabib9eab22f2014-10-20 17:17:31 -0700684 h.state.processOFMessage(h, m);
685 return;
alshabibd7963912014-10-20 14:52:04 -0700686
687 }
688
tom7ef8ff92014-09-17 13:08:06 -0700689 if (m.getType() == OFType.ECHO_REQUEST) {
690 processOFEchoRequest(h, (OFEchoRequest) m);
Praseed Balakrishnana22eadf2014-10-20 14:21:45 -0700691 } else if (m.getType() == OFType.ECHO_REPLY) {
692 processOFEchoReply(h, (OFEchoReply) m);
tom7ef8ff92014-09-17 13:08:06 -0700693 } else if (m.getType() == OFType.ROLE_REPLY) {
694 h.sw.handleRole(m);
695 } else if (m.getType() == OFType.ERROR) {
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800696 if (!h.sw.handleRoleError((OFErrorMsg) m)) {
tom7ef8ff92014-09-17 13:08:06 -0700697 h.sw.processDriverHandshakeMessage(m);
698 if (h.sw.isDriverHandshakeComplete()) {
alshabibd7963912014-10-20 14:52:04 -0700699 moveToActive(h);
tom7ef8ff92014-09-17 13:08:06 -0700700 }
701 }
702 } else {
703 if (m.getType() == OFType.EXPERIMENTER &&
704 ((OFExperimenter) m).getExperimenter() ==
705 RoleManager.NICIRA_EXPERIMENTER) {
706 h.sw.handleNiciraRole(m);
707 } else {
708 h.sw.processDriverHandshakeMessage(m);
709 if (h.sw.isDriverHandshakeComplete()) {
alshabibd7963912014-10-20 14:52:04 -0700710 moveToActive(h);
tom7ef8ff92014-09-17 13:08:06 -0700711 }
712 }
713 }
714 }
715
716 @Override
717 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
718 throws IOException, SwitchStateException {
719 h.pendingPortStatusMsg.add(m);
720 }
alshabibd7963912014-10-20 14:52:04 -0700721
722 private void moveToActive(OFChannelHandler h) {
723 boolean success = h.sw.connectSwitch();
pierc684ee12019-07-16 15:52:50 +0200724 // Disconnect from the device and return
alshabibd7963912014-10-20 14:52:04 -0700725 if (!success) {
726 disconnectDuplicate(h);
pierc684ee12019-07-16 15:52:50 +0200727 return;
alshabibd7963912014-10-20 14:52:04 -0700728 }
pierc684ee12019-07-16 15:52:50 +0200729 handlePendingPortStatusMessages(h);
730 h.setState(ACTIVE);
alshabibd7963912014-10-20 14:52:04 -0700731 }
732
tom7ef8ff92014-09-17 13:08:06 -0700733 },
734
Jordi Ortiz91477b82016-11-29 15:22:50 +0100735 /**
736 * We are expecting a OF Multi Part Meter Features Stats Reply.
737 * Notice that this information is only available for switches running
738 * OpenFlow 1.3
739 */
740 WAIT_METER_FEATURES_REPLY(true) {
Yuta HIGUCHI10f45132017-03-01 17:09:32 -0800741
742 @Override
743 void processOFEchoRequest(OFChannelHandler h, OFEchoRequest m)
744 throws IOException {
745 super.processOFEchoRequest(h, m);
746 if (System.currentTimeMillis() - h.lastStateChange > METER_TIMEOUT) {
747 log.info("{} did not respond to MeterFeaturesRequest on time, " +
748 "moving on without it.",
749 h.getSwitchInfoString());
750 h.sendHandshakeDescriptionStatsRequest();
751 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
752 }
753 }
754
Jordi Ortiz91477b82016-11-29 15:22:50 +0100755 @Override
756 void processOFError(OFChannelHandler h, OFErrorMsg m)
757 throws IOException {
Charles Chan34155e52016-11-30 18:28:11 -0800758 // Hardware switches may reply OFErrorMsg if meter is not supported
759 log.warn("Received OFError {}. It seems {} does not support Meter.",
760 m.getErrType().name(), Dpid.uri(h.thisdpid));
761 log.debug("{}", m);
762 h.sendHandshakeDescriptionStatsRequest();
763 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
Jordi Ortiz91477b82016-11-29 15:22:50 +0100764 }
765
766 @Override
767 void processOFStatisticsReply(OFChannelHandler h,
768 OFStatsReply m)
769 throws IOException, SwitchStateException {
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800770 switch (m.getStatsType()) {
Jordi Ortiz91477b82016-11-29 15:22:50 +0100771 case METER_FEATURES:
772
773 log.debug("Received Meter Features");
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800774 OFMeterFeaturesStatsReply ofmfsr = (OFMeterFeaturesStatsReply) m;
Jordi Ortiz91477b82016-11-29 15:22:50 +0100775 log.info("Received meter features from {} with max meters: {}",
776 h.getSwitchInfoString(),
777 ofmfsr.getFeatures().getMaxMeter());
778 h.meterFeaturesReply = ofmfsr;
779 h.sendHandshakeDescriptionStatsRequest();
780 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
781 break;
782 default:
783 log.error("Unexpected OF Multi Part stats reply");
784 illegalMessageReceived(h, m);
785 break;
786 }
787 }
788
789 @Override
790 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
791 throws IOException, SwitchStateException {
792 illegalMessageReceived(h, m);
793 }
794
795 @Override
796 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
797 throws IOException {
798 h.pendingPortStatusMsg.add(m);
799 }
Yuta HIGUCHI1745e5a2017-01-15 21:43:02 -0800800
801 @Override
802 void processIdle(OFChannelHandler h) throws IOException {
803 log.info("{} did not respond to MeterFeaturesRequest, " +
804 "moving on without it.",
805 h.getSwitchInfoString());
806 h.sendHandshakeDescriptionStatsRequest();
807 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
808 }
Jordi Ortiz91477b82016-11-29 15:22:50 +0100809 },
810
tom7ef8ff92014-09-17 13:08:06 -0700811
812 /**
813 * This controller is in MASTER role for this switch. We enter this state
814 * after requesting and winning control from the global registry.
815 * The main handshake as well as the switch-driver sub-handshake
816 * is complete at this point.
817 * // XXX S reconsider below
818 * In the (near) future we may deterministically assign controllers to
819 * switches at startup.
820 * We only leave this state if the switch disconnects or
821 * if we send a role request for SLAVE /and/ receive the role reply for
822 * SLAVE.
823 */
824 ACTIVE(true) {
825 @Override
826 void processOFError(OFChannelHandler h, OFErrorMsg m)
827 throws IOException, SwitchStateException {
828 // if we get here, then the error message is for something else
829 if (m.getErrType() == OFErrorType.BAD_REQUEST &&
Ray Milkey30d19652016-09-06 12:09:46 -0700830 (((OFBadRequestErrorMsg) m).getCode() ==
Charles Chan9d7465e2016-09-09 19:02:34 -0700831 OFBadRequestCode.EPERM ||
tom7ef8ff92014-09-17 13:08:06 -0700832 ((OFBadRequestErrorMsg) m).getCode() ==
Charles Chan9d7465e2016-09-09 19:02:34 -0700833 OFBadRequestCode.IS_SLAVE)) {
tom7ef8ff92014-09-17 13:08:06 -0700834 // We are the master controller and the switch returned
835 // a permission error. This is a likely indicator that
836 // the switch thinks we are slave. Reassert our
837 // role
838 // FIXME: this could be really bad during role transitions
839 // if two controllers are master (even if its only for
840 // a brief period). We might need to see if these errors
841 // persist before we reassert
alshabib339a3d92014-09-26 17:54:32 -0700842
tom7ef8ff92014-09-17 13:08:06 -0700843 h.sw.reassertRole();
844 } else if (m.getErrType() == OFErrorType.FLOW_MOD_FAILED &&
845 ((OFFlowModFailedErrorMsg) m).getCode() ==
846 OFFlowModFailedCode.ALL_TABLES_FULL) {
847 h.sw.setTableFull(true);
848 } else {
849 logError(h, m);
850 }
851 h.dispatchMessage(m);
852 }
853
854 @Override
855 void processOFStatisticsReply(OFChannelHandler h,
856 OFStatsReply m) {
Ayaka Koshibe38594c22014-10-22 13:36:12 -0700857 if (m.getStatsType().equals(OFStatsType.PORT_DESC)) {
Saurav Das45f48152018-01-18 12:07:33 -0800858 log.debug("Received port desc message from {}: {}",
859 h.sw.getDpid(),
860 ((OFPortDescStatsReply) m).getEntries());
Ayaka Koshibe38594c22014-10-22 13:36:12 -0700861 h.sw.setPortDescReply((OFPortDescStatsReply) m);
862 }
tom7ef8ff92014-09-17 13:08:06 -0700863 h.dispatchMessage(m);
864 }
865
866 @Override
867 void processOFExperimenter(OFChannelHandler h, OFExperimenter m)
868 throws SwitchStateException {
869 h.sw.handleNiciraRole(m);
870 }
871
872 @Override
873 void processOFRoleReply(OFChannelHandler h, OFRoleReply m)
874 throws SwitchStateException {
875 h.sw.handleRole(m);
876 }
877
878 @Override
879 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
880 throws SwitchStateException {
881 handlePortStatusMessage(h, m, true);
Thomas Vachuska39274462014-12-02 13:23:50 -0800882 //h.dispatchMessage(m);
tom7ef8ff92014-09-17 13:08:06 -0700883 }
884
885 @Override
886 void processOFPacketIn(OFChannelHandler h, OFPacketIn m) {
alshabib9eab22f2014-10-20 17:17:31 -0700887// OFPacketOut out =
888// h.sw.factory().buildPacketOut()
889// .setXid(m.getXid())
890// .setBufferId(m.getBufferId()).build();
891// h.sw.sendMsg(out);
tom7ef8ff92014-09-17 13:08:06 -0700892 h.dispatchMessage(m);
893 }
894
895 @Override
896 void processOFFlowRemoved(OFChannelHandler h,
897 OFFlowRemoved m) {
898 h.dispatchMessage(m);
899 }
900
901 @Override
902 void processOFBarrierReply(OFChannelHandler h, OFBarrierReply m) {
903 h.dispatchMessage(m);
904 }
905
Ayaka Koshibee8708e32014-10-22 13:40:18 -0700906 @Override
907 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m) {
Ayaka Koshibe38594c22014-10-22 13:36:12 -0700908 h.sw.setFeaturesReply(m);
Ayaka Koshibee8708e32014-10-22 13:40:18 -0700909 h.dispatchMessage(m);
910 }
911
Yuta HIGUCHI1745e5a2017-01-15 21:43:02 -0800912 @Override
913 void processIdle(OFChannelHandler h) throws IOException {
914 log.info("{} idle", h.getSwitchInfoString());
915 }
916
tom7ef8ff92014-09-17 13:08:06 -0700917 };
918
919 private final boolean handshakeComplete;
920 ChannelState(boolean handshakeComplete) {
921 this.handshakeComplete = handshakeComplete;
922 }
923
924 /**
925 * Is this a state in which the handshake has completed?
926 * @return true if the handshake is complete
927 */
928 public boolean isHandshakeComplete() {
929 return handshakeComplete;
930 }
931
932 /**
933 * Get a string specifying the switch connection, state, and
934 * message received. To be used as message for SwitchStateException
935 * or log messages
936 * @param h The channel handler (to get switch information_
937 * @param m The OFMessage that has just been received
938 * @param details A string giving more details about the exact nature
939 * of the problem.
940 * @return display string
941 */
942 // needs to be protected because enum members are actually subclasses
943 protected String getSwitchStateMessage(OFChannelHandler h,
944 OFMessage m,
945 String details) {
946 return String.format("Switch: [%s], State: [%s], received: [%s]"
947 + ", details: %s",
948 h.getSwitchInfoString(),
949 this.toString(),
950 m.getType().toString(),
951 details);
952 }
953
954 /**
955 * We have an OFMessage we didn't expect given the current state and
956 * we want to treat this as an error.
957 * We currently throw an exception that will terminate the connection
958 * However, we could be more forgiving
959 * @param h the channel handler that received the message
960 * @param m the message
Jonathan Hart147b2ac2014-10-23 10:03:52 -0700961 * @throws SwitchStateException we always throw the exception
tom7ef8ff92014-09-17 13:08:06 -0700962 */
Jonathan Hart147b2ac2014-10-23 10:03:52 -0700963 // needs to be protected because enum members are actually subclasses
tom7ef8ff92014-09-17 13:08:06 -0700964 protected void illegalMessageReceived(OFChannelHandler h, OFMessage m)
965 throws SwitchStateException {
966 String msg = getSwitchStateMessage(h, m,
967 "Switch should never send this message in the current state");
968 throw new SwitchStateException(msg);
969
970 }
971
972 /**
973 * We have an OFMessage we didn't expect given the current state and
974 * we want to ignore the message.
975 * @param h the channel handler the received the message
976 * @param m the message
977 */
978 protected void unhandledMessageReceived(OFChannelHandler h,
979 OFMessage m) {
980 if (log.isDebugEnabled()) {
981 String msg = getSwitchStateMessage(h, m,
982 "Ignoring unexpected message");
983 log.debug(msg);
984 }
985 }
986
987 /**
988 * Log an OpenFlow error message from a switch.
989 * @param h The switch that sent the error
990 * @param error The error message
991 */
992 protected void logError(OFChannelHandler h, OFErrorMsg error) {
alshabib09d48be2014-10-03 15:43:33 -0700993 log.error("{} from switch {} in state {}",
tom7ef8ff92014-09-17 13:08:06 -0700994 error,
995 h.getSwitchInfoString(),
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800996 this);
tom7ef8ff92014-09-17 13:08:06 -0700997 }
998
999 /**
1000 * Log an OpenFlow error message from a switch and disconnect the
1001 * channel.
1002 *
1003 * @param h the IO channel for this switch.
1004 * @param error The error message
1005 */
1006 protected void logErrorDisconnect(OFChannelHandler h, OFErrorMsg error) {
1007 logError(h, error);
HIGUCHI Yutadc5cf8a2016-04-29 15:17:06 -07001008 log.error("Disconnecting switch {}", h.getSwitchInfoString());
tom7ef8ff92014-09-17 13:08:06 -07001009 h.channel.disconnect();
1010 }
1011
1012 /**
1013 * log an error message for a duplicate dpid and disconnect this channel.
1014 * @param h the IO channel for this switch.
1015 */
1016 protected void disconnectDuplicate(OFChannelHandler h) {
1017 log.error("Duplicated dpid or incompleted cleanup - "
1018 + "disconnecting channel {}", h.getSwitchInfoString());
1019 h.duplicateDpidFound = Boolean.TRUE;
1020 h.channel.disconnect();
1021 }
1022
1023
1024
1025 /**
1026 * Handles all pending port status messages before a switch is declared
1027 * activated in MASTER or EQUAL role. Note that since this handling
1028 * precedes the activation (and therefore notification to IOFSwitchListerners)
1029 * the changes to ports will already be visible once the switch is
1030 * activated. As a result, no notifications are sent out for these
1031 * pending portStatus messages.
Thomas Vachuska4b420772014-10-30 16:46:17 -07001032 *
1033 * @param h the channel handler that received the message
tom7ef8ff92014-09-17 13:08:06 -07001034 */
1035 protected void handlePendingPortStatusMessages(OFChannelHandler h) {
1036 try {
1037 handlePendingPortStatusMessages(h, 0);
1038 } catch (SwitchStateException e) {
1039 log.error(e.getMessage());
1040 }
1041 }
1042
1043 private void handlePendingPortStatusMessages(OFChannelHandler h, int index)
1044 throws SwitchStateException {
1045 if (h.sw == null) {
1046 String msg = "State machine error: switch is null. Should never " +
1047 "happen";
1048 throw new SwitchStateException(msg);
1049 }
Thomas Vachuska39274462014-12-02 13:23:50 -08001050 log.info("Processing {} pending port status messages for {}",
1051 h.pendingPortStatusMsg.size(), h.sw.getStringId());
1052
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001053 ArrayList<OFPortStatus> temp = new ArrayList<>();
tom7ef8ff92014-09-17 13:08:06 -07001054 for (OFPortStatus ps: h.pendingPortStatusMsg) {
1055 temp.add(ps);
1056 handlePortStatusMessage(h, ps, false);
1057 }
tom7ef8ff92014-09-17 13:08:06 -07001058 // expensive but ok - we don't expect too many port-status messages
1059 // note that we cannot use clear(), because of the reasons below
1060 h.pendingPortStatusMsg.removeAll(temp);
Thomas Vachuska39274462014-12-02 13:23:50 -08001061 temp.clear();
tom7ef8ff92014-09-17 13:08:06 -07001062 // the iterator above takes a snapshot of the list - so while we were
1063 // dealing with the pending port-status messages, we could have received
1064 // newer ones. Handle them recursively, but break the recursion after
1065 // five steps to avoid an attack.
1066 if (!h.pendingPortStatusMsg.isEmpty() && ++index < 5) {
1067 handlePendingPortStatusMessages(h, index);
1068 }
1069 }
1070
1071 /**
1072 * Handle a port status message.
1073 *
1074 * Handle a port status message by updating the port maps in the
1075 * IOFSwitch instance and notifying Controller about the change so
1076 * it can dispatch a switch update.
1077 *
1078 * @param h The OFChannelHhandler that received the message
1079 * @param m The PortStatus message we received
1080 * @param doNotify if true switch port changed events will be
1081 * dispatched
Thomas Vachuskab14c77a2014-11-04 18:08:01 -08001082 * @throws SwitchStateException if the switch is not bound to the channel
tom7ef8ff92014-09-17 13:08:06 -07001083 *
1084 */
1085 protected void handlePortStatusMessage(OFChannelHandler h, OFPortStatus m,
1086 boolean doNotify) throws SwitchStateException {
1087 if (h.sw == null) {
1088 String msg = getSwitchStateMessage(h, m,
1089 "State machine error: switch is null. Should never " +
1090 "happen");
1091 throw new SwitchStateException(msg);
1092 }
Saurav Dasbd071d82018-01-09 17:38:44 -08001093 log.info("Received port status message from {}/{}: {}",
1094 h.sw.getDpid(), m.getDesc().getPortNo(), m);
tom7ef8ff92014-09-17 13:08:06 -07001095
1096 h.sw.handleMessage(m);
1097 }
1098
1099
1100 /**
1101 * Process an OF message received on the channel and
1102 * update state accordingly.
1103 *
1104 * The main "event" of the state machine. Process the received message,
1105 * send follow up message if required and update state if required.
1106 *
1107 * Switches on the message type and calls more specific event handlers
1108 * for each individual OF message type. If we receive a message that
1109 * is supposed to be sent from a controller to a switch we throw
1110 * a SwitchStateExeption.
1111 *
1112 * The more specific handlers can also throw SwitchStateExceptions
1113 *
1114 * @param h The OFChannelHandler that received the message
1115 * @param m The message we received.
Thomas Vachuskab14c77a2014-11-04 18:08:01 -08001116 * @throws SwitchStateException if the switch is not bound to the channel
1117 * @throws IOException if unable to send message back to the switch
tom7ef8ff92014-09-17 13:08:06 -07001118 */
1119 void processOFMessage(OFChannelHandler h, OFMessage m)
1120 throws IOException, SwitchStateException {
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001121 switch (m.getType()) {
tom7ef8ff92014-09-17 13:08:06 -07001122 case HELLO:
1123 processOFHello(h, (OFHello) m);
1124 break;
1125 case BARRIER_REPLY:
1126 processOFBarrierReply(h, (OFBarrierReply) m);
1127 break;
1128 case ECHO_REPLY:
1129 processOFEchoReply(h, (OFEchoReply) m);
1130 break;
1131 case ECHO_REQUEST:
1132 processOFEchoRequest(h, (OFEchoRequest) m);
1133 break;
1134 case ERROR:
1135 processOFError(h, (OFErrorMsg) m);
1136 break;
1137 case FEATURES_REPLY:
1138 processOFFeaturesReply(h, (OFFeaturesReply) m);
1139 break;
1140 case FLOW_REMOVED:
1141 processOFFlowRemoved(h, (OFFlowRemoved) m);
1142 break;
1143 case GET_CONFIG_REPLY:
1144 processOFGetConfigReply(h, (OFGetConfigReply) m);
1145 break;
1146 case PACKET_IN:
1147 processOFPacketIn(h, (OFPacketIn) m);
1148 break;
1149 case PORT_STATUS:
1150 processOFPortStatus(h, (OFPortStatus) m);
1151 break;
1152 case QUEUE_GET_CONFIG_REPLY:
1153 processOFQueueGetConfigReply(h, (OFQueueGetConfigReply) m);
1154 break;
1155 case STATS_REPLY: // multipart_reply in 1.3
1156 processOFStatisticsReply(h, (OFStatsReply) m);
1157 break;
1158 case EXPERIMENTER:
1159 processOFExperimenter(h, (OFExperimenter) m);
1160 break;
1161 case ROLE_REPLY:
1162 processOFRoleReply(h, (OFRoleReply) m);
1163 break;
1164 case GET_ASYNC_REPLY:
1165 processOFGetAsyncReply(h, (OFAsyncGetReply) m);
1166 break;
1167
1168 // The following messages are sent to switches. The controller
1169 // should never receive them
1170 case SET_CONFIG:
1171 case GET_CONFIG_REQUEST:
1172 case PACKET_OUT:
1173 case PORT_MOD:
1174 case QUEUE_GET_CONFIG_REQUEST:
1175 case BARRIER_REQUEST:
1176 case STATS_REQUEST: // multipart request in 1.3
1177 case FEATURES_REQUEST:
1178 case FLOW_MOD:
1179 case GROUP_MOD:
1180 case TABLE_MOD:
1181 case GET_ASYNC_REQUEST:
1182 case SET_ASYNC:
1183 case METER_MOD:
1184 default:
1185 illegalMessageReceived(h, m);
1186 break;
1187 }
1188 }
1189
1190 /*-----------------------------------------------------------------
1191 * Default implementation for message handlers in any state.
1192 *
1193 * Individual states must override these if they want a behavior
1194 * that differs from the default.
1195 *
1196 * In general, these handlers simply ignore the message and do
1197 * nothing.
1198 *
1199 * There are some exceptions though, since some messages really
1200 * are handled the same way in every state (e.g., ECHO_REQUST) or
1201 * that are only valid in a single state (e.g., HELLO, GET_CONFIG_REPLY
1202 -----------------------------------------------------------------*/
1203
1204 void processOFHello(OFChannelHandler h, OFHello m)
1205 throws IOException, SwitchStateException {
1206 // we only expect hello in the WAIT_HELLO state
alshabib45fd88a2015-09-24 17:34:35 -07001207 log.warn("Received Hello outside WAIT_HELLO state; switch {} is not complaint.",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001208 h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001209 }
1210
1211 void processOFBarrierReply(OFChannelHandler h, OFBarrierReply m)
1212 throws IOException {
1213 // Silently ignore.
1214 }
1215
1216 void processOFEchoRequest(OFChannelHandler h, OFEchoRequest m)
1217 throws IOException {
1218 if (h.ofVersion == null) {
1219 log.error("No OF version set for {}. Not sending Echo REPLY",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001220 h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001221 return;
1222 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001223 OFEchoReply reply = h.factory
1224 .buildEchoReply()
1225 .setXid(m.getXid())
1226 .setData(m.getData())
1227 .build();
1228 h.channel.writeAndFlush(Collections.singletonList(reply));
tom7ef8ff92014-09-17 13:08:06 -07001229 }
1230
1231 void processOFEchoReply(OFChannelHandler h, OFEchoReply m)
1232 throws IOException {
1233 // Do nothing with EchoReplies !!
1234 }
1235
1236 // no default implementation for OFError
1237 // every state must override it
1238 abstract void processOFError(OFChannelHandler h, OFErrorMsg m)
1239 throws IOException, SwitchStateException;
1240
1241
1242 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
1243 throws IOException, SwitchStateException {
1244 unhandledMessageReceived(h, m);
1245 }
1246
1247 void processOFFlowRemoved(OFChannelHandler h, OFFlowRemoved m)
1248 throws IOException {
1249 unhandledMessageReceived(h, m);
1250 }
1251
1252 void processOFGetConfigReply(OFChannelHandler h, OFGetConfigReply m)
1253 throws IOException, SwitchStateException {
1254 // we only expect config replies in the WAIT_CONFIG_REPLY state
1255 illegalMessageReceived(h, m);
1256 }
1257
1258 void processOFPacketIn(OFChannelHandler h, OFPacketIn m)
1259 throws IOException {
1260 unhandledMessageReceived(h, m);
1261 }
1262
1263 // no default implementation. Every state needs to handle it.
1264 abstract void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
1265 throws IOException, SwitchStateException;
1266
1267 void processOFQueueGetConfigReply(OFChannelHandler h,
1268 OFQueueGetConfigReply m)
1269 throws IOException {
1270 unhandledMessageReceived(h, m);
1271 }
1272
1273 void processOFStatisticsReply(OFChannelHandler h, OFStatsReply m)
1274 throws IOException, SwitchStateException {
1275 unhandledMessageReceived(h, m);
1276 }
1277
1278 void processOFExperimenter(OFChannelHandler h, OFExperimenter m)
1279 throws IOException, SwitchStateException {
1280 // TODO: it might make sense to parse the vendor message here
1281 // into the known vendor messages we support and then call more
1282 // specific event handlers
1283 unhandledMessageReceived(h, m);
1284 }
1285
1286 void processOFRoleReply(OFChannelHandler h, OFRoleReply m)
1287 throws SwitchStateException, IOException {
1288 unhandledMessageReceived(h, m);
1289 }
1290
1291 void processOFGetAsyncReply(OFChannelHandler h,
1292 OFAsyncGetReply m) {
1293 unhandledMessageReceived(h, m);
1294 }
1295
Yuta HIGUCHI1745e5a2017-01-15 21:43:02 -08001296 void processIdle(OFChannelHandler h) throws IOException {
1297 // disconnect channel which did no complete handshake
1298 log.error("{} idle in state {}, disconnecting", h.getSwitchInfoString(), this);
1299 h.channel.disconnect();
1300 }
tom7ef8ff92014-09-17 13:08:06 -07001301 }
1302
1303
1304
1305 //*************************
1306 // Channel handler methods
1307 //*************************
1308
1309 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001310 public void channelActive(ChannelHandlerContext ctx)
1311 throws Exception {
1312
1313 channel = ctx.channel();
tom7ef8ff92014-09-17 13:08:06 -07001314 log.info("New switch connection from {}",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001315 channel.remoteAddress());
1316
1317 SocketAddress address = channel.remoteAddress();
1318 if (address instanceof InetSocketAddress) {
1319 final InetSocketAddress inetAddress = (InetSocketAddress) address;
1320 final IpAddress ipAddress = IpAddress.valueOf(inetAddress.getAddress());
1321 if (ipAddress.isIp4()) {
1322 channelId = ipAddress.toString() + ':' + inetAddress.getPort();
1323 } else {
1324 channelId = '[' + ipAddress.toString() + "]:" + inetAddress.getPort();
1325 }
1326 } else {
1327 channelId = channel.toString();
1328 }
1329
1330 dispatcher = Executors.newSingleThreadExecutor(groupedThreads("onos/of/dispatcher", channelId, log));
1331
alshabib70fc7fb2015-01-06 11:04:29 -08001332 /*
1333 hack to wait for the switch to tell us what it's
1334 max version is. This is not spec compliant and should
1335 be removed as soon as switches behave better.
1336 */
1337 //sendHandshakeHelloMessage();
tom7ef8ff92014-09-17 13:08:06 -07001338 setState(ChannelState.WAIT_HELLO);
1339 }
1340
1341 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001342 public void channelInactive(ChannelHandlerContext ctx)
1343 throws Exception {
1344
tom7ef8ff92014-09-17 13:08:06 -07001345 log.info("Switch disconnected callback for sw:{}. Cleaning up ...",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001346 getSwitchInfoString());
1347
1348 if (dispatcher != null) {
Harold Huang828cd592017-11-04 10:46:04 +08001349 dispatcher.shutdownNow();
Thomas Vachuskad75684a2018-01-03 09:04:47 -08001350 dispatcher = null;
tom7ef8ff92014-09-17 13:08:06 -07001351 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001352
1353 if (thisdpid != 0) {
1354 if (!duplicateDpidFound) {
1355 // if the disconnected switch (on this ChannelHandler)
1356 // was not one with a duplicate-dpid, it is safe to remove all
1357 // state for it at the controller. Notice that if the disconnected
1358 // switch was a duplicate-dpid, calling the method below would clear
1359 // all state for the original switch (with the same dpid),
1360 // which we obviously don't want.
1361 log.info("{}:removal called", getSwitchInfoString());
1362 if (sw != null) {
1363 sw.removeConnectedSwitch();
1364 }
1365 } else {
1366 // A duplicate was disconnected on this ChannelHandler,
1367 // this is the same switch reconnecting, but the original state was
1368 // not cleaned up - XXX check liveness of original ChannelHandler
1369 log.info("{}:duplicate found", getSwitchInfoString());
1370 duplicateDpidFound = Boolean.FALSE;
1371 }
1372 } else {
1373 log.warn("no dpid in channelHandler registered for "
1374 + "disconnected switch {}", getSwitchInfoString());
1375 }
tom7ef8ff92014-09-17 13:08:06 -07001376 }
1377
1378 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001379 public void exceptionCaught(ChannelHandlerContext ctx,
1380 Throwable cause)
tom7ef8ff92014-09-17 13:08:06 -07001381 throws Exception {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001382
1383 if (cause instanceof ReadTimeoutException) {
tom7ef8ff92014-09-17 13:08:06 -07001384 // switch timeout
1385 log.error("Disconnecting switch {} due to read timeout",
1386 getSwitchInfoString());
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001387 ctx.channel().close();
1388 } else if (cause instanceof HandshakeTimeoutException) {
tom7ef8ff92014-09-17 13:08:06 -07001389 log.error("Disconnecting switch {}: failed to complete handshake",
1390 getSwitchInfoString());
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001391 ctx.channel().close();
1392 } else if (cause instanceof ClosedChannelException) {
tom7ef8ff92014-09-17 13:08:06 -07001393 log.debug("Channel for sw {} already closed", getSwitchInfoString());
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001394 } else if (cause instanceof IOException) {
1395 if (!cause.getMessage().equals(RESET_BY_PEER) &&
1396 !cause.getMessage().equals(BROKEN_PIPE)) {
Thomas Vachuskae9af1f42015-07-06 08:42:18 -07001397 log.error("Disconnecting switch {} due to IO Error: {}",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001398 getSwitchInfoString(), cause.getMessage());
Thomas Vachuskae9af1f42015-07-06 08:42:18 -07001399 if (log.isDebugEnabled()) {
1400 // still print stack trace if debug is enabled
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001401 log.debug("StackTrace for previous Exception: ", cause);
Thomas Vachuskae9af1f42015-07-06 08:42:18 -07001402 }
tom7ef8ff92014-09-17 13:08:06 -07001403 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001404 ctx.channel().close();
1405 } else if (cause instanceof SwitchStateException) {
tom7ef8ff92014-09-17 13:08:06 -07001406 log.error("Disconnecting switch {} due to switch state error: {}",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001407 getSwitchInfoString(), cause.getMessage());
tom7ef8ff92014-09-17 13:08:06 -07001408 if (log.isDebugEnabled()) {
1409 // still print stack trace if debug is enabled
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001410 log.debug("StackTrace for previous Exception: ", cause);
tom7ef8ff92014-09-17 13:08:06 -07001411 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001412 ctx.channel().close();
1413 } else if (cause instanceof OFParseError) {
tom7ef8ff92014-09-17 13:08:06 -07001414 log.error("Disconnecting switch "
1415 + getSwitchInfoString() +
1416 " due to message parse failure",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001417 cause);
1418 ctx.channel().close();
1419 } else if (cause instanceof RejectedExecutionException) {
tom7ef8ff92014-09-17 13:08:06 -07001420 log.warn("Could not process message: queue full");
1421 } else {
1422 log.error("Error while processing message from switch "
1423 + getSwitchInfoString()
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001424 + "state " + this.state, cause);
1425 ctx.channel().close();
tom7ef8ff92014-09-17 13:08:06 -07001426 }
1427 }
1428
1429 @Override
1430 public String toString() {
1431 return getSwitchInfoString();
1432 }
1433
Ray Milkey986a47a2018-01-25 11:38:51 -08001434 private void channelIdle(ChannelHandlerContext ctx,
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001435 IdleStateEvent e)
Ray Milkey986a47a2018-01-25 11:38:51 -08001436 throws IOException {
Charles Chan982d3902018-03-21 14:58:53 -07001437 // Factory can be null if the channel goes idle during initial handshake. Since the switch
1438 // is not even initialized properly, we just skip this and disconnect the channel.
1439 if (factory != null) {
1440 OFMessage m = factory.buildEchoRequest().build();
1441 log.debug("Sending Echo Request on idle channel: {}", ctx.channel());
1442 ctx.write(Collections.singletonList(m), ctx.voidPromise());
1443 // XXX S some problems here -- echo request has no transaction id, and
1444 // echo reply is not correlated to the echo request.
1445 }
Yuta HIGUCHI1745e5a2017-01-15 21:43:02 -08001446 state.processIdle(this);
tom7ef8ff92014-09-17 13:08:06 -07001447 }
1448
1449 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001450 public void userEventTriggered(ChannelHandlerContext ctx,
1451 Object evt)
tom7ef8ff92014-09-17 13:08:06 -07001452 throws Exception {
tom7ef8ff92014-09-17 13:08:06 -07001453
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001454 if (evt instanceof IdleStateEvent) {
1455 channelIdle(ctx, (IdleStateEvent) evt);
1456 }
tom7ef8ff92014-09-17 13:08:06 -07001457
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001458 super.userEventTriggered(ctx, evt);
1459 }
1460
1461 // SimpleChannelInboundHandler without dependency to TypeParameterMatcher
1462 @Override
1463 public void channelRead(ChannelHandlerContext ctx,
1464 Object msg) throws Exception {
1465
1466 boolean release = true;
1467 try {
1468 if (msg instanceof OFMessage) {
1469 // channelRead0 inlined
1470 state.processOFMessage(this, (OFMessage) msg);
1471 } else {
1472 release = false;
1473 ctx.fireChannelRead(msg);
tom7ef8ff92014-09-17 13:08:06 -07001474 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001475 } finally {
1476 if (release) {
1477 ReferenceCountUtil.release(msg);
1478 }
tom7ef8ff92014-09-17 13:08:06 -07001479 }
1480 }
1481
1482
1483
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001484
tom7ef8ff92014-09-17 13:08:06 -07001485 //*************************
1486 // Channel utility methods
1487 //*************************
1488
1489 /**
1490 * Is this a state in which the handshake has completed?
Anton Chigrinbf14b372019-01-14 17:29:56 +02001491 *
tom7ef8ff92014-09-17 13:08:06 -07001492 * @return true if the handshake is complete
1493 */
1494 public boolean isHandshakeComplete() {
1495 return this.state.isHandshakeComplete();
1496 }
1497
Anton Chigrinbf14b372019-01-14 17:29:56 +02001498 /**
1499 * Increment totalCount variable and send signal to executor.
1500 */
1501 private void incrementAndSignal() {
1502 try {
1503 totalCount.incrementAndGet();
1504 takeLock.lockInterruptibly();
1505 try {
1506 notEmpty.signal();
1507 } finally {
1508 takeLock.unlock();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001509 }
Anton Chigrinbf14b372019-01-14 17:29:56 +02001510 } catch (InterruptedException e) {
1511 e.printStackTrace();
1512 }
1513 }
1514
1515 /**
1516 * Try to push OpenFlow message to queue.
1517 *
1518 * @param message OpenFlow message
1519 * @param idQueue id of Queue
1520 * @return true if message was successful added to queue
1521 */
1522 private boolean pushMessageToQueue(OFMessage message, int idQueue) {
1523 if (!dispatchQueuesMapProducer.get(idQueue).offer(message)) {
1524 return false;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001525 } else {
Anton Chigrinbf14b372019-01-14 17:29:56 +02001526 incrementAndSignal();
1527 return true;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001528 }
Anton Chigrinbf14b372019-01-14 17:29:56 +02001529 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001530
Anton Chigrinbf14b372019-01-14 17:29:56 +02001531 /**
1532 * Process backlog - move messages from backlog to default queue.
1533 *
1534 * @return true if whole backlog was processed, otherwise false
1535 */
1536 private boolean processDispatchBacklogQueue() {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001537 while (!dispatchBacklog.isEmpty()) {
Anton Chigrinbf14b372019-01-14 17:29:56 +02001538 OFMessage msgFromBacklog = dispatchBacklog.removeFirst();
1539 if (!pushMessageToQueue(msgFromBacklog, NUM_OF_QUEUES - 1)) {
1540 dispatchBacklog.addFirst(msgFromBacklog);
1541 return false;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001542 }
1543 }
Anton Chigrinbf14b372019-01-14 17:29:56 +02001544 return true;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001545
Anton Chigrinbf14b372019-01-14 17:29:56 +02001546 }
1547
1548 /**
1549 * Parse OpenFlow message context for get Ethernet packet.
1550 *
1551 * @param message OpenFlow message
1552 * @return parsed Ethernet packet
1553 */
1554 private Ethernet parsePacketInMessage(OFMessage message) {
1555 OpenFlowPacketContext pktCtx = DefaultOpenFlowPacketContext
1556 .packetContextFromPacketIn(sw, (OFPacketIn) message);
1557 DeviceId id = DeviceId.deviceId(Dpid.uri(pktCtx.dpid().value()));
1558 DefaultInboundPacket inPkt = new DefaultInboundPacket(
1559 new ConnectPoint(id, PortNumber.portNumber(pktCtx.inPort())),
1560 pktCtx.parsed(), ByteBuffer.wrap(pktCtx.unparsed()),
1561 pktCtx.cookie());
1562 return inPkt.parsed();
1563 }
1564
1565 /**
1566 * Classify the Ethernet packet for membership on one of the queues.
1567 *
1568 * @param packet ethernet packet
1569 * @return Id of destination Queue
1570 */
1571 private int classifyEthernetPacket(Ethernet packet) {
1572 for (Set<OpenFlowClassifier> classifiers : this.messageClassifiersMapProducer) {
1573 for (OpenFlowClassifier classifier : classifiers) {
1574 if (classifier.ethernetType() == packet.getEtherType()) {
1575 return classifier.idQueue();
1576 }
1577 }
1578 }
1579 return NUM_OF_QUEUES - 1;
1580 }
1581
1582 /**
1583 * Process messages from dispatch queues.
1584 *
1585 * @param queuesSize count of messages in all queues
1586 */
1587 private void processMessages(int queuesSize) {
1588 List<OFMessage> msgs = new ArrayList<>();
1589 int processed;
1590 do {
1591 processed = 0;
1592 while (processed < queuesSize) {
1593 for (LinkedBlockingMessagesQueue<OFMessage> queue :
1594 dispatchQueuesMapProducer.values()) {
1595 processed += queue.drainTo(msgs);
1596 }
1597 }
1598
1599 msgs.forEach(sw::handleMessage);
1600 msgs.clear();
1601 /* Decrement conditional variable */
1602 queuesSize = totalCount.addAndGet(-1 * processed);
1603 } while (queuesSize > 0);
1604 }
1605
1606 private void dispatchMessage(OFMessage m) {
1607 log.debug("Begin dispatch OpenFlow Message");
1608 boolean backlogEmpty = processDispatchBacklogQueue();
1609 if (m.getType() == OFType.PACKET_IN) {
1610 Ethernet pkt = parsePacketInMessage(m);
1611 pushMessageToQueue(m, classifyEthernetPacket(pkt));
1612 } else {
1613 if (!backlogEmpty || !pushMessageToQueue(m, NUM_OF_QUEUES - 1)) {
1614 dispatchBacklog.offer(m);
1615 }
1616 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001617
1618 if (dispatcherHandle.isDone()) {
1619 // dispatcher terminated for some reason, restart
Yuta HIGUCHIfbd9ae92018-01-24 23:39:06 -08001620 dispatcherHandle = dispatcher.submit((Runnable) () -> {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001621 try {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001622 for (;;) {
Anton Chigrinbf14b372019-01-14 17:29:56 +02001623 int tc = 0;
1624 takeLock.lockInterruptibly();
1625 try {
1626 while ((tc = totalCount.get()) == 0) {
1627 notEmpty.await();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001628 }
Anton Chigrinbf14b372019-01-14 17:29:56 +02001629 } finally {
1630 takeLock.unlock();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001631 }
1632
Anton Chigrinbf14b372019-01-14 17:29:56 +02001633 processMessages(tc);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001634 }
1635 } catch (InterruptedException e) {
Anton Chigrinbf14b372019-01-14 17:29:56 +02001636 log.error("executor thread InterruptedException: {}", e);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001637 Thread.currentThread().interrupt();
1638 // interrupted. gracefully shutting down
1639 return;
1640 }
1641
1642 });
1643 }
tom7ef8ff92014-09-17 13:08:06 -07001644 }
1645
1646 /**
1647 * Return a string describing this switch based on the already available
1648 * information (DPID and/or remote socket).
1649 * @return display string
1650 */
1651 private String getSwitchInfoString() {
1652 if (sw != null) {
1653 return sw.toString();
1654 }
1655 String channelString;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001656 if (channel == null || channel.remoteAddress() == null) {
tom7ef8ff92014-09-17 13:08:06 -07001657 channelString = "?";
1658 } else {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001659 channelString = channel.remoteAddress().toString();
tom7ef8ff92014-09-17 13:08:06 -07001660 }
1661 String dpidString;
1662 if (featuresReply == null) {
1663 dpidString = "?";
1664 } else {
1665 dpidString = featuresReply.getDatapathId().toString();
1666 }
1667 return String.format("[%s DPID[%s]]", channelString, dpidString);
1668 }
1669
1670 /**
1671 * Update the channels state. Only called from the state machine.
1672 * TODO: enforce restricted state transitions
pierc684ee12019-07-16 15:52:50 +02001673 * @param state new state
tom7ef8ff92014-09-17 13:08:06 -07001674 */
pierc684ee12019-07-16 15:52:50 +02001675 void setState(ChannelState state) {
tom7ef8ff92014-09-17 13:08:06 -07001676 this.state = state;
Yuta HIGUCHI10f45132017-03-01 17:09:32 -08001677 this.lastStateChange = System.currentTimeMillis();
tom7ef8ff92014-09-17 13:08:06 -07001678 }
1679
Brian O'Connorf69e3e32018-05-10 02:25:09 -07001680 private boolean setDpid(Long dpid, Channel channel) {
1681 ChannelHandlerContext sslContext = channel.pipeline().context(SslHandler.class);
1682 if (sslContext != null) {
1683 try {
1684 SslHandler sslHandler = (SslHandler) sslContext.handler();
1685 Certificate[] certs = sslHandler.engine().getSession().getPeerCertificates();
1686 Certificate cert = certs.length > 0 ? certs[0] : null;
1687 if (!controller.isValidCertificate(dpid, cert)) {
1688 return false;
1689 }
1690 } catch (SSLPeerUnverifiedException e) {
1691 log.info("Switch with dpid {} is an unverified SSL peer.", dpid, e);
1692 return false;
1693 }
1694 }
1695 this.thisdpid = dpid;
1696 return true;
1697 }
1698
tom7ef8ff92014-09-17 13:08:06 -07001699 /**
1700 * Send hello message to the switch using the handshake transactions ids.
1701 * @throws IOException
1702 */
1703 private void sendHandshakeHelloMessage() throws IOException {
1704 // The OF protocol requires us to start things off by sending the highest
1705 // version of the protocol supported.
1706
Yuta HIGUCHI6512f3e2017-05-18 17:21:24 -07001707 // bitmap represents OF1.0, OF1.3, OF1.4, and OF1.5
tom7ef8ff92014-09-17 13:08:06 -07001708 // see Sec. 7.5.1 of the OF1.3.4 spec
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001709 U32 bitmap = U32.ofRaw((0b1 << OFVersion.OF_10.getWireVersion()) |
1710 (0b1 << OFVersion.OF_13.getWireVersion()) |
Yuta HIGUCHI6512f3e2017-05-18 17:21:24 -07001711 (0b1 << OFVersion.OF_14.getWireVersion()) |
1712 (0b1 << OFVersion.OF_15.getWireVersion()));
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001713 OFVersion version = Optional.ofNullable(ofVersion).orElse(OFVersion.OF_13);
1714 OFHelloElem hem = OFFactories.getFactory(version)
1715 .buildHelloElemVersionbitmap()
tom7ef8ff92014-09-17 13:08:06 -07001716 .setBitmaps(Collections.singletonList(bitmap))
1717 .build();
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001718 OFMessage.Builder mb = OFFactories.getFactory(version)
1719 .buildHello()
tom7ef8ff92014-09-17 13:08:06 -07001720 .setXid(this.handshakeTransactionIds--)
1721 .setElements(Collections.singletonList(hem));
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001722 log.info("Sending {} Hello to {}", version, channel.remoteAddress());
1723 channel.writeAndFlush(Collections.singletonList(mb.build()));
tom7ef8ff92014-09-17 13:08:06 -07001724 }
1725
1726 /**
1727 * Send featuresRequest msg to the switch using the handshake transactions ids.
1728 * @throws IOException
1729 */
1730 private void sendHandshakeFeaturesRequestMessage() throws IOException {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001731 log.debug("Sending FEATURES_REQUEST to {}", channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001732 OFMessage m = factory.buildFeaturesRequest()
1733 .setXid(this.handshakeTransactionIds--)
1734 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001735 channel.writeAndFlush(Collections.singletonList(m));
tom7ef8ff92014-09-17 13:08:06 -07001736 }
1737
1738 /**
1739 * Send the configuration requests to tell the switch we want full
1740 * packets.
1741 * @throws IOException
1742 */
1743 private void sendHandshakeSetConfig() throws IOException {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001744 log.debug("Sending CONFIG_REQUEST to {}", channel.remoteAddress());
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001745 List<OFMessage> msglist = new ArrayList<>(3);
tom7ef8ff92014-09-17 13:08:06 -07001746
1747 // Ensure we receive the full packet via PacketIn
1748 // FIXME: We don't set the reassembly flags.
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001749 // Only send config to switches to send full packets, if they have a buffer.
Michael Jarschel7f521a32015-08-12 16:31:07 +02001750 // Saves a packet & OFSetConfig can't be handled by certain switches.
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001751 if (this.featuresReply.getNBuffers() > 0) {
Michael Jarschel7f521a32015-08-12 16:31:07 +02001752 OFSetConfig sc = factory
1753 .buildSetConfig()
1754 .setMissSendLen((short) 0xffff)
1755 .setXid(this.handshakeTransactionIds--)
1756 .build();
1757 msglist.add(sc);
1758 }
tom7ef8ff92014-09-17 13:08:06 -07001759
1760 // Barrier
1761 OFBarrierRequest br = factory
1762 .buildBarrierRequest()
1763 .setXid(this.handshakeTransactionIds--)
1764 .build();
1765 msglist.add(br);
1766
1767 // Verify (need barrier?)
1768 OFGetConfigRequest gcr = factory
1769 .buildGetConfigRequest()
1770 .setXid(this.handshakeTransactionIds--)
1771 .build();
1772 msglist.add(gcr);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001773 channel.writeAndFlush(msglist);
tom7ef8ff92014-09-17 13:08:06 -07001774 }
1775
1776 /**
1777 * send a description state request.
1778 * @throws IOException
1779 */
1780 private void sendHandshakeDescriptionStatsRequest() throws IOException {
1781 // Get Description to set switch-specific flags
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001782 log.debug("Sending DESC_STATS_REQUEST to {}", channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001783 OFDescStatsRequest dreq = factory
1784 .buildDescStatsRequest()
1785 .setXid(handshakeTransactionIds--)
1786 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001787 channel.writeAndFlush(Collections.singletonList(dreq));
tom7ef8ff92014-09-17 13:08:06 -07001788 }
1789
Jordi Ortiz91477b82016-11-29 15:22:50 +01001790 /**
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001791 * send a meter features request.
1792 *
Jordi Ortiz91477b82016-11-29 15:22:50 +01001793 * @throws IOException
1794 */
1795 private void sendMeterFeaturesRequest() throws IOException {
1796 // Get meter features including the MaxMeters value available for the device
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001797 OFFactory factory = OFFactories.getFactory(ofVersion);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001798 log.debug("Sending METER_FEATURES_REQUEST to {}", channel.remoteAddress());
Jordi Ortiz91477b82016-11-29 15:22:50 +01001799 OFMeterFeaturesStatsRequest mfreq = factory
1800 .buildMeterFeaturesStatsRequest()
1801 .setXid(handshakeTransactionIds--)
1802 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001803 channel.writeAndFlush(Collections.singletonList(mfreq));
Jordi Ortiz91477b82016-11-29 15:22:50 +01001804 }
1805
tom7ef8ff92014-09-17 13:08:06 -07001806 private void sendHandshakeOFPortDescRequest() throws IOException {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001807 log.debug("Sending OF_PORT_DESC_REQUEST to {}", channel.remoteAddress());
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001808 // Get port description for 1.3+ switch
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001809 OFPortDescStatsRequest preq = factory
tom7ef8ff92014-09-17 13:08:06 -07001810 .buildPortDescStatsRequest()
1811 .setXid(handshakeTransactionIds--)
1812 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001813 channel.writeAndFlush(Collections.singletonList(preq));
tom7ef8ff92014-09-17 13:08:06 -07001814 }
1815
1816 ChannelState getStateForTesting() {
1817 return state;
1818 }
1819
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001820
1821 @Override
1822 public boolean isActive() {
1823 if (channel != null) {
1824 return channel.isActive();
1825 }
1826 return false;
1827 }
1828
1829 @Override
1830 public void closeSession() {
1831 if (channel != null) {
1832 channel.close();
1833 }
1834 }
1835
1836 @Override
1837 public boolean sendMsg(Iterable<OFMessage> msgs) {
1838 if (channel.isActive()) {
Laszlo Pappb68fe7e2017-11-24 17:06:59 +00001839 if (log.isTraceEnabled()) {
1840 log.trace("Sending messages for switch {} via openflow channel: {}", getSwitchInfoString(), msgs);
1841 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001842 channel.writeAndFlush(msgs, channel.voidPromise());
1843 return true;
1844 } else {
1845 log.warn("Dropping messages for switch {} because channel is not connected: {}",
1846 getSwitchInfoString(), msgs);
1847 return false;
1848 }
1849 }
1850
1851 @Override
1852 public CharSequence sessionInfo() {
1853 return channelId;
1854 }
1855
Anton Chigrinbf14b372019-01-14 17:29:56 +02001856 @Override
1857 public void addClassifier(OpenFlowClassifier classifier) {
1858 if (this.deviceId.equals(classifier.deviceId())) {
1859 log.debug("Add OpenFlow Classifier for switch {} to queue {} with type {}",
1860 classifier.deviceId().toString(), classifier.idQueue(), classifier.ethernetType());
1861 this.messageClassifiersMapProducer.get(classifier.idQueue()).add(classifier);
1862 }
1863 }
1864
1865 @Override
1866 public void removeClassifier(OpenFlowClassifier classifier) {
1867 if (this.deviceId.equals(classifier.deviceId())) {
1868 log.debug("Remove OpenFlow Classifier for switch {} from queue {} with type {}",
1869 classifier.deviceId().toString(), classifier.idQueue(), classifier.ethernetType());
1870 this.messageClassifiersMapProducer.get(classifier.idQueue()).remove(classifier);
1871 }
1872 }
1873
1874 /**
1875 * Init classifier configuration for the switch. Use stored configuration if exist.
1876 * Otherwise add LLDP and BDDP classifiers for Queue N0.
1877 */
1878 private void initClassifiers() {
1879 try {
1880 openFlowManager = DefaultServiceDirectory.getService(OpenFlowService.class);
1881 DeviceId did = DeviceId.deviceId(uri(thisdpid));
1882 Set<OpenFlowClassifier> classifiers = openFlowManager.getClassifiersByDeviceId(did);
1883 if (classifiers == null) {
1884 OpenFlowClassifier classifier =
1885 new OpenFlowClassifier.Builder(did, 0).ethernetType(TYPE_LLDP).build();
1886 openFlowManager.add(classifier);
1887 classifier = new OpenFlowClassifier.Builder(did, 0).ethernetType(TYPE_BSN).build();
1888 openFlowManager.add(classifier);
1889 } else {
1890 this.messageClassifiersMapProducer.forEach((v) -> {
1891 v.clear();
1892 });
1893 classifiers.forEach((c) -> {
1894 messageClassifiersMapProducer.get(c.idQueue()).add(c);
1895 });
1896 }
1897 } catch (Exception e) {
1898 log.error("Initialize default classifier failed: {}", e.toString());
1899 e.printStackTrace();
1900 }
1901 }
tom7ef8ff92014-09-17 13:08:06 -07001902}