blob: 5a95188b40e94824a5e7c51b3acc9d5ce763c68d [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";
Anton Chigrin4af4f872019-01-14 17:29:56 +0200130 private 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
Anton Chigrin4af4f872019-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 Chigrin4af4f872019-01-14 17:29:56 +0200183 * Map with all LinkedBlockingMessagesQueue queues which contains OFMessages.
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700184 */
Anton Chigrin4af4f872019-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 Chigrin4af4f872019-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 Chigrin4af4f872019-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 Chigrin4af4f872019-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 Chigrin4af4f872019-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()) {
623 if (!h.sw.connectSwitch()) {
624 disconnectDuplicate(h);
Anton Chigrin4af4f872019-01-14 17:29:56 +0200625 } else {
626 h.initClassifiers();
alshabib9eab22f2014-10-20 17:17:31 -0700627 }
Thomas Vachuska39274462014-12-02 13:23:50 -0800628 handlePendingPortStatusMessages(h);
alshabib9eab22f2014-10-20 17:17:31 -0700629 h.setState(ACTIVE);
630 } else {
631 h.setState(WAIT_SWITCH_DRIVER_SUB_HANDSHAKE);
632 }
tom7ef8ff92014-09-17 13:08:06 -0700633
634 }
635
636 @Override
637 void processOFError(OFChannelHandler h, OFErrorMsg m) {
638 logErrorDisconnect(h, m);
639 }
640
641 @Override
642 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
643 throws IOException, SwitchStateException {
644 illegalMessageReceived(h, m);
645 }
646
647 @Override
648 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
649 throws IOException {
650 h.pendingPortStatusMsg.add(m);
651 }
652 },
653
654
655 /**
656 * We are waiting for the respective switch driver to complete its
657 * configuration. Notice that we do not consider this to be part of the main
658 * switch-controller handshake. But we do consider it as a step that comes
659 * before we declare the switch as available to the controller.
660 * Next State: depends on the role of this controller for this switch - either
661 * MASTER or EQUAL.
662 */
663 WAIT_SWITCH_DRIVER_SUB_HANDSHAKE(true) {
664
665 @Override
666 void processOFError(OFChannelHandler h, OFErrorMsg m)
667 throws IOException {
668 // will never be called. We override processOFMessage
669 }
670
alshabibd7963912014-10-20 14:52:04 -0700671
672
tom7ef8ff92014-09-17 13:08:06 -0700673 @Override
674 void processOFMessage(OFChannelHandler h, OFMessage m)
675 throws IOException, SwitchStateException {
alshabibd7963912014-10-20 14:52:04 -0700676
677 if (h.sw.isDriverHandshakeComplete()) {
678 moveToActive(h);
alshabib9eab22f2014-10-20 17:17:31 -0700679 h.state.processOFMessage(h, m);
680 return;
alshabibd7963912014-10-20 14:52:04 -0700681
682 }
683
tom7ef8ff92014-09-17 13:08:06 -0700684 if (m.getType() == OFType.ECHO_REQUEST) {
685 processOFEchoRequest(h, (OFEchoRequest) m);
Praseed Balakrishnana22eadf2014-10-20 14:21:45 -0700686 } else if (m.getType() == OFType.ECHO_REPLY) {
687 processOFEchoReply(h, (OFEchoReply) m);
tom7ef8ff92014-09-17 13:08:06 -0700688 } else if (m.getType() == OFType.ROLE_REPLY) {
689 h.sw.handleRole(m);
690 } else if (m.getType() == OFType.ERROR) {
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800691 if (!h.sw.handleRoleError((OFErrorMsg) m)) {
tom7ef8ff92014-09-17 13:08:06 -0700692 h.sw.processDriverHandshakeMessage(m);
693 if (h.sw.isDriverHandshakeComplete()) {
alshabibd7963912014-10-20 14:52:04 -0700694 moveToActive(h);
tom7ef8ff92014-09-17 13:08:06 -0700695 }
696 }
697 } else {
698 if (m.getType() == OFType.EXPERIMENTER &&
699 ((OFExperimenter) m).getExperimenter() ==
700 RoleManager.NICIRA_EXPERIMENTER) {
701 h.sw.handleNiciraRole(m);
702 } else {
703 h.sw.processDriverHandshakeMessage(m);
704 if (h.sw.isDriverHandshakeComplete()) {
alshabibd7963912014-10-20 14:52:04 -0700705 moveToActive(h);
tom7ef8ff92014-09-17 13:08:06 -0700706 }
707 }
708 }
709 }
710
711 @Override
712 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
713 throws IOException, SwitchStateException {
714 h.pendingPortStatusMsg.add(m);
715 }
alshabibd7963912014-10-20 14:52:04 -0700716
717 private void moveToActive(OFChannelHandler h) {
718 boolean success = h.sw.connectSwitch();
Thomas Vachuska39274462014-12-02 13:23:50 -0800719 handlePendingPortStatusMessages(h);
alshabibd7963912014-10-20 14:52:04 -0700720 h.setState(ACTIVE);
721 if (!success) {
722 disconnectDuplicate(h);
alshabibd7963912014-10-20 14:52:04 -0700723 }
724 }
725
tom7ef8ff92014-09-17 13:08:06 -0700726 },
727
Jordi Ortiz91477b82016-11-29 15:22:50 +0100728 /**
729 * We are expecting a OF Multi Part Meter Features Stats Reply.
730 * Notice that this information is only available for switches running
731 * OpenFlow 1.3
732 */
733 WAIT_METER_FEATURES_REPLY(true) {
Yuta HIGUCHI10f45132017-03-01 17:09:32 -0800734
735 @Override
736 void processOFEchoRequest(OFChannelHandler h, OFEchoRequest m)
737 throws IOException {
738 super.processOFEchoRequest(h, m);
739 if (System.currentTimeMillis() - h.lastStateChange > METER_TIMEOUT) {
740 log.info("{} did not respond to MeterFeaturesRequest on time, " +
741 "moving on without it.",
742 h.getSwitchInfoString());
743 h.sendHandshakeDescriptionStatsRequest();
744 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
745 }
746 }
747
Jordi Ortiz91477b82016-11-29 15:22:50 +0100748 @Override
749 void processOFError(OFChannelHandler h, OFErrorMsg m)
750 throws IOException {
Charles Chan34155e52016-11-30 18:28:11 -0800751 // Hardware switches may reply OFErrorMsg if meter is not supported
752 log.warn("Received OFError {}. It seems {} does not support Meter.",
753 m.getErrType().name(), Dpid.uri(h.thisdpid));
754 log.debug("{}", m);
755 h.sendHandshakeDescriptionStatsRequest();
756 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
Jordi Ortiz91477b82016-11-29 15:22:50 +0100757 }
758
759 @Override
760 void processOFStatisticsReply(OFChannelHandler h,
761 OFStatsReply m)
762 throws IOException, SwitchStateException {
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800763 switch (m.getStatsType()) {
Jordi Ortiz91477b82016-11-29 15:22:50 +0100764 case METER_FEATURES:
765
766 log.debug("Received Meter Features");
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800767 OFMeterFeaturesStatsReply ofmfsr = (OFMeterFeaturesStatsReply) m;
Jordi Ortiz91477b82016-11-29 15:22:50 +0100768 log.info("Received meter features from {} with max meters: {}",
769 h.getSwitchInfoString(),
770 ofmfsr.getFeatures().getMaxMeter());
771 h.meterFeaturesReply = ofmfsr;
772 h.sendHandshakeDescriptionStatsRequest();
773 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
774 break;
775 default:
776 log.error("Unexpected OF Multi Part stats reply");
777 illegalMessageReceived(h, m);
778 break;
779 }
780 }
781
782 @Override
783 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
784 throws IOException, SwitchStateException {
785 illegalMessageReceived(h, m);
786 }
787
788 @Override
789 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
790 throws IOException {
791 h.pendingPortStatusMsg.add(m);
792 }
Yuta HIGUCHI1745e5a2017-01-15 21:43:02 -0800793
794 @Override
795 void processIdle(OFChannelHandler h) throws IOException {
796 log.info("{} did not respond to MeterFeaturesRequest, " +
797 "moving on without it.",
798 h.getSwitchInfoString());
799 h.sendHandshakeDescriptionStatsRequest();
800 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
801 }
Jordi Ortiz91477b82016-11-29 15:22:50 +0100802 },
803
tom7ef8ff92014-09-17 13:08:06 -0700804
805 /**
806 * This controller is in MASTER role for this switch. We enter this state
807 * after requesting and winning control from the global registry.
808 * The main handshake as well as the switch-driver sub-handshake
809 * is complete at this point.
810 * // XXX S reconsider below
811 * In the (near) future we may deterministically assign controllers to
812 * switches at startup.
813 * We only leave this state if the switch disconnects or
814 * if we send a role request for SLAVE /and/ receive the role reply for
815 * SLAVE.
816 */
817 ACTIVE(true) {
818 @Override
819 void processOFError(OFChannelHandler h, OFErrorMsg m)
820 throws IOException, SwitchStateException {
821 // if we get here, then the error message is for something else
822 if (m.getErrType() == OFErrorType.BAD_REQUEST &&
Ray Milkey30d19652016-09-06 12:09:46 -0700823 (((OFBadRequestErrorMsg) m).getCode() ==
Charles Chan9d7465e2016-09-09 19:02:34 -0700824 OFBadRequestCode.EPERM ||
tom7ef8ff92014-09-17 13:08:06 -0700825 ((OFBadRequestErrorMsg) m).getCode() ==
Charles Chan9d7465e2016-09-09 19:02:34 -0700826 OFBadRequestCode.IS_SLAVE)) {
tom7ef8ff92014-09-17 13:08:06 -0700827 // We are the master controller and the switch returned
828 // a permission error. This is a likely indicator that
829 // the switch thinks we are slave. Reassert our
830 // role
831 // FIXME: this could be really bad during role transitions
832 // if two controllers are master (even if its only for
833 // a brief period). We might need to see if these errors
834 // persist before we reassert
alshabib339a3d92014-09-26 17:54:32 -0700835
tom7ef8ff92014-09-17 13:08:06 -0700836 h.sw.reassertRole();
837 } else if (m.getErrType() == OFErrorType.FLOW_MOD_FAILED &&
838 ((OFFlowModFailedErrorMsg) m).getCode() ==
839 OFFlowModFailedCode.ALL_TABLES_FULL) {
840 h.sw.setTableFull(true);
841 } else {
842 logError(h, m);
843 }
844 h.dispatchMessage(m);
845 }
846
847 @Override
848 void processOFStatisticsReply(OFChannelHandler h,
849 OFStatsReply m) {
Ayaka Koshibe38594c22014-10-22 13:36:12 -0700850 if (m.getStatsType().equals(OFStatsType.PORT_DESC)) {
Saurav Das45f48152018-01-18 12:07:33 -0800851 log.debug("Received port desc message from {}: {}",
852 h.sw.getDpid(),
853 ((OFPortDescStatsReply) m).getEntries());
Ayaka Koshibe38594c22014-10-22 13:36:12 -0700854 h.sw.setPortDescReply((OFPortDescStatsReply) m);
855 }
tom7ef8ff92014-09-17 13:08:06 -0700856 h.dispatchMessage(m);
857 }
858
859 @Override
860 void processOFExperimenter(OFChannelHandler h, OFExperimenter m)
861 throws SwitchStateException {
862 h.sw.handleNiciraRole(m);
863 }
864
865 @Override
866 void processOFRoleReply(OFChannelHandler h, OFRoleReply m)
867 throws SwitchStateException {
868 h.sw.handleRole(m);
869 }
870
871 @Override
872 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
873 throws SwitchStateException {
874 handlePortStatusMessage(h, m, true);
Thomas Vachuska39274462014-12-02 13:23:50 -0800875 //h.dispatchMessage(m);
tom7ef8ff92014-09-17 13:08:06 -0700876 }
877
878 @Override
879 void processOFPacketIn(OFChannelHandler h, OFPacketIn m) {
alshabib9eab22f2014-10-20 17:17:31 -0700880// OFPacketOut out =
881// h.sw.factory().buildPacketOut()
882// .setXid(m.getXid())
883// .setBufferId(m.getBufferId()).build();
884// h.sw.sendMsg(out);
tom7ef8ff92014-09-17 13:08:06 -0700885 h.dispatchMessage(m);
886 }
887
888 @Override
889 void processOFFlowRemoved(OFChannelHandler h,
890 OFFlowRemoved m) {
891 h.dispatchMessage(m);
892 }
893
894 @Override
895 void processOFBarrierReply(OFChannelHandler h, OFBarrierReply m) {
896 h.dispatchMessage(m);
897 }
898
Ayaka Koshibee8708e32014-10-22 13:40:18 -0700899 @Override
900 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m) {
Ayaka Koshibe38594c22014-10-22 13:36:12 -0700901 h.sw.setFeaturesReply(m);
Ayaka Koshibee8708e32014-10-22 13:40:18 -0700902 h.dispatchMessage(m);
903 }
904
Yuta HIGUCHI1745e5a2017-01-15 21:43:02 -0800905 @Override
906 void processIdle(OFChannelHandler h) throws IOException {
907 log.info("{} idle", h.getSwitchInfoString());
908 }
909
tom7ef8ff92014-09-17 13:08:06 -0700910 };
911
912 private final boolean handshakeComplete;
913 ChannelState(boolean handshakeComplete) {
914 this.handshakeComplete = handshakeComplete;
915 }
916
917 /**
918 * Is this a state in which the handshake has completed?
919 * @return true if the handshake is complete
920 */
921 public boolean isHandshakeComplete() {
922 return handshakeComplete;
923 }
924
925 /**
926 * Get a string specifying the switch connection, state, and
927 * message received. To be used as message for SwitchStateException
928 * or log messages
929 * @param h The channel handler (to get switch information_
930 * @param m The OFMessage that has just been received
931 * @param details A string giving more details about the exact nature
932 * of the problem.
933 * @return display string
934 */
935 // needs to be protected because enum members are actually subclasses
936 protected String getSwitchStateMessage(OFChannelHandler h,
937 OFMessage m,
938 String details) {
939 return String.format("Switch: [%s], State: [%s], received: [%s]"
940 + ", details: %s",
941 h.getSwitchInfoString(),
942 this.toString(),
943 m.getType().toString(),
944 details);
945 }
946
947 /**
948 * We have an OFMessage we didn't expect given the current state and
949 * we want to treat this as an error.
950 * We currently throw an exception that will terminate the connection
951 * However, we could be more forgiving
952 * @param h the channel handler that received the message
953 * @param m the message
Jonathan Hart147b2ac2014-10-23 10:03:52 -0700954 * @throws SwitchStateException we always throw the exception
tom7ef8ff92014-09-17 13:08:06 -0700955 */
Jonathan Hart147b2ac2014-10-23 10:03:52 -0700956 // needs to be protected because enum members are actually subclasses
tom7ef8ff92014-09-17 13:08:06 -0700957 protected void illegalMessageReceived(OFChannelHandler h, OFMessage m)
958 throws SwitchStateException {
959 String msg = getSwitchStateMessage(h, m,
960 "Switch should never send this message in the current state");
961 throw new SwitchStateException(msg);
962
963 }
964
965 /**
966 * We have an OFMessage we didn't expect given the current state and
967 * we want to ignore the message.
968 * @param h the channel handler the received the message
969 * @param m the message
970 */
971 protected void unhandledMessageReceived(OFChannelHandler h,
972 OFMessage m) {
973 if (log.isDebugEnabled()) {
974 String msg = getSwitchStateMessage(h, m,
975 "Ignoring unexpected message");
976 log.debug(msg);
977 }
978 }
979
980 /**
981 * Log an OpenFlow error message from a switch.
982 * @param h The switch that sent the error
983 * @param error The error message
984 */
985 protected void logError(OFChannelHandler h, OFErrorMsg error) {
alshabib09d48be2014-10-03 15:43:33 -0700986 log.error("{} from switch {} in state {}",
tom7ef8ff92014-09-17 13:08:06 -0700987 error,
988 h.getSwitchInfoString(),
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800989 this);
tom7ef8ff92014-09-17 13:08:06 -0700990 }
991
992 /**
993 * Log an OpenFlow error message from a switch and disconnect the
994 * channel.
995 *
996 * @param h the IO channel for this switch.
997 * @param error The error message
998 */
999 protected void logErrorDisconnect(OFChannelHandler h, OFErrorMsg error) {
1000 logError(h, error);
HIGUCHI Yutadc5cf8a2016-04-29 15:17:06 -07001001 log.error("Disconnecting switch {}", h.getSwitchInfoString());
tom7ef8ff92014-09-17 13:08:06 -07001002 h.channel.disconnect();
1003 }
1004
1005 /**
1006 * log an error message for a duplicate dpid and disconnect this channel.
1007 * @param h the IO channel for this switch.
1008 */
1009 protected void disconnectDuplicate(OFChannelHandler h) {
1010 log.error("Duplicated dpid or incompleted cleanup - "
1011 + "disconnecting channel {}", h.getSwitchInfoString());
1012 h.duplicateDpidFound = Boolean.TRUE;
1013 h.channel.disconnect();
1014 }
1015
1016
1017
1018 /**
1019 * Handles all pending port status messages before a switch is declared
1020 * activated in MASTER or EQUAL role. Note that since this handling
1021 * precedes the activation (and therefore notification to IOFSwitchListerners)
1022 * the changes to ports will already be visible once the switch is
1023 * activated. As a result, no notifications are sent out for these
1024 * pending portStatus messages.
Thomas Vachuska4b420772014-10-30 16:46:17 -07001025 *
1026 * @param h the channel handler that received the message
tom7ef8ff92014-09-17 13:08:06 -07001027 */
1028 protected void handlePendingPortStatusMessages(OFChannelHandler h) {
1029 try {
1030 handlePendingPortStatusMessages(h, 0);
1031 } catch (SwitchStateException e) {
1032 log.error(e.getMessage());
1033 }
1034 }
1035
1036 private void handlePendingPortStatusMessages(OFChannelHandler h, int index)
1037 throws SwitchStateException {
1038 if (h.sw == null) {
1039 String msg = "State machine error: switch is null. Should never " +
1040 "happen";
1041 throw new SwitchStateException(msg);
1042 }
Thomas Vachuska39274462014-12-02 13:23:50 -08001043 log.info("Processing {} pending port status messages for {}",
1044 h.pendingPortStatusMsg.size(), h.sw.getStringId());
1045
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001046 ArrayList<OFPortStatus> temp = new ArrayList<>();
tom7ef8ff92014-09-17 13:08:06 -07001047 for (OFPortStatus ps: h.pendingPortStatusMsg) {
1048 temp.add(ps);
1049 handlePortStatusMessage(h, ps, false);
1050 }
tom7ef8ff92014-09-17 13:08:06 -07001051 // expensive but ok - we don't expect too many port-status messages
1052 // note that we cannot use clear(), because of the reasons below
1053 h.pendingPortStatusMsg.removeAll(temp);
Thomas Vachuska39274462014-12-02 13:23:50 -08001054 temp.clear();
tom7ef8ff92014-09-17 13:08:06 -07001055 // the iterator above takes a snapshot of the list - so while we were
1056 // dealing with the pending port-status messages, we could have received
1057 // newer ones. Handle them recursively, but break the recursion after
1058 // five steps to avoid an attack.
1059 if (!h.pendingPortStatusMsg.isEmpty() && ++index < 5) {
1060 handlePendingPortStatusMessages(h, index);
1061 }
1062 }
1063
1064 /**
1065 * Handle a port status message.
1066 *
1067 * Handle a port status message by updating the port maps in the
1068 * IOFSwitch instance and notifying Controller about the change so
1069 * it can dispatch a switch update.
1070 *
1071 * @param h The OFChannelHhandler that received the message
1072 * @param m The PortStatus message we received
1073 * @param doNotify if true switch port changed events will be
1074 * dispatched
Thomas Vachuskab14c77a2014-11-04 18:08:01 -08001075 * @throws SwitchStateException if the switch is not bound to the channel
tom7ef8ff92014-09-17 13:08:06 -07001076 *
1077 */
1078 protected void handlePortStatusMessage(OFChannelHandler h, OFPortStatus m,
1079 boolean doNotify) throws SwitchStateException {
1080 if (h.sw == null) {
1081 String msg = getSwitchStateMessage(h, m,
1082 "State machine error: switch is null. Should never " +
1083 "happen");
1084 throw new SwitchStateException(msg);
1085 }
Saurav Dasbd071d82018-01-09 17:38:44 -08001086 log.info("Received port status message from {}/{}: {}",
1087 h.sw.getDpid(), m.getDesc().getPortNo(), m);
tom7ef8ff92014-09-17 13:08:06 -07001088
1089 h.sw.handleMessage(m);
1090 }
1091
1092
1093 /**
1094 * Process an OF message received on the channel and
1095 * update state accordingly.
1096 *
1097 * The main "event" of the state machine. Process the received message,
1098 * send follow up message if required and update state if required.
1099 *
1100 * Switches on the message type and calls more specific event handlers
1101 * for each individual OF message type. If we receive a message that
1102 * is supposed to be sent from a controller to a switch we throw
1103 * a SwitchStateExeption.
1104 *
1105 * The more specific handlers can also throw SwitchStateExceptions
1106 *
1107 * @param h The OFChannelHandler that received the message
1108 * @param m The message we received.
Thomas Vachuskab14c77a2014-11-04 18:08:01 -08001109 * @throws SwitchStateException if the switch is not bound to the channel
1110 * @throws IOException if unable to send message back to the switch
tom7ef8ff92014-09-17 13:08:06 -07001111 */
1112 void processOFMessage(OFChannelHandler h, OFMessage m)
1113 throws IOException, SwitchStateException {
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001114 switch (m.getType()) {
tom7ef8ff92014-09-17 13:08:06 -07001115 case HELLO:
1116 processOFHello(h, (OFHello) m);
1117 break;
1118 case BARRIER_REPLY:
1119 processOFBarrierReply(h, (OFBarrierReply) m);
1120 break;
1121 case ECHO_REPLY:
1122 processOFEchoReply(h, (OFEchoReply) m);
1123 break;
1124 case ECHO_REQUEST:
1125 processOFEchoRequest(h, (OFEchoRequest) m);
1126 break;
1127 case ERROR:
1128 processOFError(h, (OFErrorMsg) m);
1129 break;
1130 case FEATURES_REPLY:
1131 processOFFeaturesReply(h, (OFFeaturesReply) m);
1132 break;
1133 case FLOW_REMOVED:
1134 processOFFlowRemoved(h, (OFFlowRemoved) m);
1135 break;
1136 case GET_CONFIG_REPLY:
1137 processOFGetConfigReply(h, (OFGetConfigReply) m);
1138 break;
1139 case PACKET_IN:
1140 processOFPacketIn(h, (OFPacketIn) m);
1141 break;
1142 case PORT_STATUS:
1143 processOFPortStatus(h, (OFPortStatus) m);
1144 break;
1145 case QUEUE_GET_CONFIG_REPLY:
1146 processOFQueueGetConfigReply(h, (OFQueueGetConfigReply) m);
1147 break;
1148 case STATS_REPLY: // multipart_reply in 1.3
1149 processOFStatisticsReply(h, (OFStatsReply) m);
1150 break;
1151 case EXPERIMENTER:
1152 processOFExperimenter(h, (OFExperimenter) m);
1153 break;
1154 case ROLE_REPLY:
1155 processOFRoleReply(h, (OFRoleReply) m);
1156 break;
1157 case GET_ASYNC_REPLY:
1158 processOFGetAsyncReply(h, (OFAsyncGetReply) m);
1159 break;
1160
1161 // The following messages are sent to switches. The controller
1162 // should never receive them
1163 case SET_CONFIG:
1164 case GET_CONFIG_REQUEST:
1165 case PACKET_OUT:
1166 case PORT_MOD:
1167 case QUEUE_GET_CONFIG_REQUEST:
1168 case BARRIER_REQUEST:
1169 case STATS_REQUEST: // multipart request in 1.3
1170 case FEATURES_REQUEST:
1171 case FLOW_MOD:
1172 case GROUP_MOD:
1173 case TABLE_MOD:
1174 case GET_ASYNC_REQUEST:
1175 case SET_ASYNC:
1176 case METER_MOD:
1177 default:
1178 illegalMessageReceived(h, m);
1179 break;
1180 }
1181 }
1182
1183 /*-----------------------------------------------------------------
1184 * Default implementation for message handlers in any state.
1185 *
1186 * Individual states must override these if they want a behavior
1187 * that differs from the default.
1188 *
1189 * In general, these handlers simply ignore the message and do
1190 * nothing.
1191 *
1192 * There are some exceptions though, since some messages really
1193 * are handled the same way in every state (e.g., ECHO_REQUST) or
1194 * that are only valid in a single state (e.g., HELLO, GET_CONFIG_REPLY
1195 -----------------------------------------------------------------*/
1196
1197 void processOFHello(OFChannelHandler h, OFHello m)
1198 throws IOException, SwitchStateException {
1199 // we only expect hello in the WAIT_HELLO state
alshabib45fd88a2015-09-24 17:34:35 -07001200 log.warn("Received Hello outside WAIT_HELLO state; switch {} is not complaint.",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001201 h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001202 }
1203
1204 void processOFBarrierReply(OFChannelHandler h, OFBarrierReply m)
1205 throws IOException {
1206 // Silently ignore.
1207 }
1208
1209 void processOFEchoRequest(OFChannelHandler h, OFEchoRequest m)
1210 throws IOException {
1211 if (h.ofVersion == null) {
1212 log.error("No OF version set for {}. Not sending Echo REPLY",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001213 h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001214 return;
1215 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001216 OFEchoReply reply = h.factory
1217 .buildEchoReply()
1218 .setXid(m.getXid())
1219 .setData(m.getData())
1220 .build();
1221 h.channel.writeAndFlush(Collections.singletonList(reply));
tom7ef8ff92014-09-17 13:08:06 -07001222 }
1223
1224 void processOFEchoReply(OFChannelHandler h, OFEchoReply m)
1225 throws IOException {
1226 // Do nothing with EchoReplies !!
1227 }
1228
1229 // no default implementation for OFError
1230 // every state must override it
1231 abstract void processOFError(OFChannelHandler h, OFErrorMsg m)
1232 throws IOException, SwitchStateException;
1233
1234
1235 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
1236 throws IOException, SwitchStateException {
1237 unhandledMessageReceived(h, m);
1238 }
1239
1240 void processOFFlowRemoved(OFChannelHandler h, OFFlowRemoved m)
1241 throws IOException {
1242 unhandledMessageReceived(h, m);
1243 }
1244
1245 void processOFGetConfigReply(OFChannelHandler h, OFGetConfigReply m)
1246 throws IOException, SwitchStateException {
1247 // we only expect config replies in the WAIT_CONFIG_REPLY state
1248 illegalMessageReceived(h, m);
1249 }
1250
1251 void processOFPacketIn(OFChannelHandler h, OFPacketIn m)
1252 throws IOException {
1253 unhandledMessageReceived(h, m);
1254 }
1255
1256 // no default implementation. Every state needs to handle it.
1257 abstract void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
1258 throws IOException, SwitchStateException;
1259
1260 void processOFQueueGetConfigReply(OFChannelHandler h,
1261 OFQueueGetConfigReply m)
1262 throws IOException {
1263 unhandledMessageReceived(h, m);
1264 }
1265
1266 void processOFStatisticsReply(OFChannelHandler h, OFStatsReply m)
1267 throws IOException, SwitchStateException {
1268 unhandledMessageReceived(h, m);
1269 }
1270
1271 void processOFExperimenter(OFChannelHandler h, OFExperimenter m)
1272 throws IOException, SwitchStateException {
1273 // TODO: it might make sense to parse the vendor message here
1274 // into the known vendor messages we support and then call more
1275 // specific event handlers
1276 unhandledMessageReceived(h, m);
1277 }
1278
1279 void processOFRoleReply(OFChannelHandler h, OFRoleReply m)
1280 throws SwitchStateException, IOException {
1281 unhandledMessageReceived(h, m);
1282 }
1283
1284 void processOFGetAsyncReply(OFChannelHandler h,
1285 OFAsyncGetReply m) {
1286 unhandledMessageReceived(h, m);
1287 }
1288
Yuta HIGUCHI1745e5a2017-01-15 21:43:02 -08001289 void processIdle(OFChannelHandler h) throws IOException {
1290 // disconnect channel which did no complete handshake
1291 log.error("{} idle in state {}, disconnecting", h.getSwitchInfoString(), this);
1292 h.channel.disconnect();
1293 }
tom7ef8ff92014-09-17 13:08:06 -07001294 }
1295
1296
1297
1298 //*************************
1299 // Channel handler methods
1300 //*************************
1301
1302 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001303 public void channelActive(ChannelHandlerContext ctx)
1304 throws Exception {
1305
1306 channel = ctx.channel();
tom7ef8ff92014-09-17 13:08:06 -07001307 log.info("New switch connection from {}",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001308 channel.remoteAddress());
1309
1310 SocketAddress address = channel.remoteAddress();
1311 if (address instanceof InetSocketAddress) {
1312 final InetSocketAddress inetAddress = (InetSocketAddress) address;
1313 final IpAddress ipAddress = IpAddress.valueOf(inetAddress.getAddress());
1314 if (ipAddress.isIp4()) {
1315 channelId = ipAddress.toString() + ':' + inetAddress.getPort();
1316 } else {
1317 channelId = '[' + ipAddress.toString() + "]:" + inetAddress.getPort();
1318 }
1319 } else {
1320 channelId = channel.toString();
1321 }
1322
1323 dispatcher = Executors.newSingleThreadExecutor(groupedThreads("onos/of/dispatcher", channelId, log));
1324
alshabib70fc7fb2015-01-06 11:04:29 -08001325 /*
1326 hack to wait for the switch to tell us what it's
1327 max version is. This is not spec compliant and should
1328 be removed as soon as switches behave better.
1329 */
1330 //sendHandshakeHelloMessage();
tom7ef8ff92014-09-17 13:08:06 -07001331 setState(ChannelState.WAIT_HELLO);
1332 }
1333
1334 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001335 public void channelInactive(ChannelHandlerContext ctx)
1336 throws Exception {
1337
tom7ef8ff92014-09-17 13:08:06 -07001338 log.info("Switch disconnected callback for sw:{}. Cleaning up ...",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001339 getSwitchInfoString());
1340
1341 if (dispatcher != null) {
Harold Huang828cd592017-11-04 10:46:04 +08001342 dispatcher.shutdownNow();
Thomas Vachuskad75684a2018-01-03 09:04:47 -08001343 dispatcher = null;
tom7ef8ff92014-09-17 13:08:06 -07001344 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001345
1346 if (thisdpid != 0) {
1347 if (!duplicateDpidFound) {
1348 // if the disconnected switch (on this ChannelHandler)
1349 // was not one with a duplicate-dpid, it is safe to remove all
1350 // state for it at the controller. Notice that if the disconnected
1351 // switch was a duplicate-dpid, calling the method below would clear
1352 // all state for the original switch (with the same dpid),
1353 // which we obviously don't want.
1354 log.info("{}:removal called", getSwitchInfoString());
1355 if (sw != null) {
1356 sw.removeConnectedSwitch();
1357 }
1358 } else {
1359 // A duplicate was disconnected on this ChannelHandler,
1360 // this is the same switch reconnecting, but the original state was
1361 // not cleaned up - XXX check liveness of original ChannelHandler
1362 log.info("{}:duplicate found", getSwitchInfoString());
1363 duplicateDpidFound = Boolean.FALSE;
1364 }
1365 } else {
1366 log.warn("no dpid in channelHandler registered for "
1367 + "disconnected switch {}", getSwitchInfoString());
1368 }
tom7ef8ff92014-09-17 13:08:06 -07001369 }
1370
1371 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001372 public void exceptionCaught(ChannelHandlerContext ctx,
1373 Throwable cause)
tom7ef8ff92014-09-17 13:08:06 -07001374 throws Exception {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001375
1376 if (cause instanceof ReadTimeoutException) {
tom7ef8ff92014-09-17 13:08:06 -07001377 // switch timeout
1378 log.error("Disconnecting switch {} due to read timeout",
1379 getSwitchInfoString());
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001380 ctx.channel().close();
1381 } else if (cause instanceof HandshakeTimeoutException) {
tom7ef8ff92014-09-17 13:08:06 -07001382 log.error("Disconnecting switch {}: failed to complete handshake",
1383 getSwitchInfoString());
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001384 ctx.channel().close();
1385 } else if (cause instanceof ClosedChannelException) {
tom7ef8ff92014-09-17 13:08:06 -07001386 log.debug("Channel for sw {} already closed", getSwitchInfoString());
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001387 } else if (cause instanceof IOException) {
1388 if (!cause.getMessage().equals(RESET_BY_PEER) &&
1389 !cause.getMessage().equals(BROKEN_PIPE)) {
Thomas Vachuskae9af1f42015-07-06 08:42:18 -07001390 log.error("Disconnecting switch {} due to IO Error: {}",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001391 getSwitchInfoString(), cause.getMessage());
Thomas Vachuskae9af1f42015-07-06 08:42:18 -07001392 if (log.isDebugEnabled()) {
1393 // still print stack trace if debug is enabled
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001394 log.debug("StackTrace for previous Exception: ", cause);
Thomas Vachuskae9af1f42015-07-06 08:42:18 -07001395 }
tom7ef8ff92014-09-17 13:08:06 -07001396 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001397 ctx.channel().close();
1398 } else if (cause instanceof SwitchStateException) {
tom7ef8ff92014-09-17 13:08:06 -07001399 log.error("Disconnecting switch {} due to switch state error: {}",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001400 getSwitchInfoString(), cause.getMessage());
tom7ef8ff92014-09-17 13:08:06 -07001401 if (log.isDebugEnabled()) {
1402 // still print stack trace if debug is enabled
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001403 log.debug("StackTrace for previous Exception: ", cause);
tom7ef8ff92014-09-17 13:08:06 -07001404 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001405 ctx.channel().close();
1406 } else if (cause instanceof OFParseError) {
tom7ef8ff92014-09-17 13:08:06 -07001407 log.error("Disconnecting switch "
1408 + getSwitchInfoString() +
1409 " due to message parse failure",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001410 cause);
1411 ctx.channel().close();
1412 } else if (cause instanceof RejectedExecutionException) {
tom7ef8ff92014-09-17 13:08:06 -07001413 log.warn("Could not process message: queue full");
1414 } else {
1415 log.error("Error while processing message from switch "
1416 + getSwitchInfoString()
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001417 + "state " + this.state, cause);
1418 ctx.channel().close();
tom7ef8ff92014-09-17 13:08:06 -07001419 }
1420 }
1421
1422 @Override
1423 public String toString() {
1424 return getSwitchInfoString();
1425 }
1426
Ray Milkey986a47a2018-01-25 11:38:51 -08001427 private void channelIdle(ChannelHandlerContext ctx,
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001428 IdleStateEvent e)
Ray Milkey986a47a2018-01-25 11:38:51 -08001429 throws IOException {
Charles Chan982d3902018-03-21 14:58:53 -07001430 // Factory can be null if the channel goes idle during initial handshake. Since the switch
1431 // is not even initialized properly, we just skip this and disconnect the channel.
1432 if (factory != null) {
1433 OFMessage m = factory.buildEchoRequest().build();
1434 log.debug("Sending Echo Request on idle channel: {}", ctx.channel());
1435 ctx.write(Collections.singletonList(m), ctx.voidPromise());
1436 // XXX S some problems here -- echo request has no transaction id, and
1437 // echo reply is not correlated to the echo request.
1438 }
Yuta HIGUCHI1745e5a2017-01-15 21:43:02 -08001439 state.processIdle(this);
tom7ef8ff92014-09-17 13:08:06 -07001440 }
1441
1442 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001443 public void userEventTriggered(ChannelHandlerContext ctx,
1444 Object evt)
tom7ef8ff92014-09-17 13:08:06 -07001445 throws Exception {
tom7ef8ff92014-09-17 13:08:06 -07001446
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001447 if (evt instanceof IdleStateEvent) {
1448 channelIdle(ctx, (IdleStateEvent) evt);
1449 }
tom7ef8ff92014-09-17 13:08:06 -07001450
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001451 super.userEventTriggered(ctx, evt);
1452 }
1453
1454 // SimpleChannelInboundHandler without dependency to TypeParameterMatcher
1455 @Override
1456 public void channelRead(ChannelHandlerContext ctx,
1457 Object msg) throws Exception {
1458
1459 boolean release = true;
1460 try {
1461 if (msg instanceof OFMessage) {
1462 // channelRead0 inlined
1463 state.processOFMessage(this, (OFMessage) msg);
1464 } else {
1465 release = false;
1466 ctx.fireChannelRead(msg);
tom7ef8ff92014-09-17 13:08:06 -07001467 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001468 } finally {
1469 if (release) {
1470 ReferenceCountUtil.release(msg);
1471 }
tom7ef8ff92014-09-17 13:08:06 -07001472 }
1473 }
1474
1475
1476
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001477
tom7ef8ff92014-09-17 13:08:06 -07001478 //*************************
1479 // Channel utility methods
1480 //*************************
1481
1482 /**
1483 * Is this a state in which the handshake has completed?
Anton Chigrin4af4f872019-01-14 17:29:56 +02001484 *
tom7ef8ff92014-09-17 13:08:06 -07001485 * @return true if the handshake is complete
1486 */
1487 public boolean isHandshakeComplete() {
1488 return this.state.isHandshakeComplete();
1489 }
1490
Anton Chigrin4af4f872019-01-14 17:29:56 +02001491 /**
1492 * Increment totalCount variable and send signal to executor.
1493 */
1494 private void incrementAndSignal() {
1495 try {
1496 totalCount.incrementAndGet();
1497 takeLock.lockInterruptibly();
1498 try {
1499 notEmpty.signal();
1500 } finally {
1501 takeLock.unlock();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001502 }
Anton Chigrin4af4f872019-01-14 17:29:56 +02001503 } catch (InterruptedException e) {
1504 e.printStackTrace();
1505 }
1506 }
1507
1508 /**
1509 * Try to push OpenFlow message to queue.
1510 *
1511 * @param message OpenFlow message
1512 * @param idQueue id of Queue
1513 * @return true if message was successful added to queue
1514 */
1515 private boolean pushMessageToQueue(OFMessage message, int idQueue) {
1516 if (!dispatchQueuesMapProducer.get(idQueue).offer(message)) {
1517 return false;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001518 } else {
Anton Chigrin4af4f872019-01-14 17:29:56 +02001519 incrementAndSignal();
1520 return true;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001521 }
Anton Chigrin4af4f872019-01-14 17:29:56 +02001522 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001523
Anton Chigrin4af4f872019-01-14 17:29:56 +02001524 /**
1525 * Process backlog - move messages from backlog to default queue.
1526 *
1527 * @return true if whole backlog was processed, otherwise false
1528 */
1529 private boolean processDispatchBacklogQueue() {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001530 while (!dispatchBacklog.isEmpty()) {
Anton Chigrin4af4f872019-01-14 17:29:56 +02001531 OFMessage msgFromBacklog = dispatchBacklog.removeFirst();
1532 if (!pushMessageToQueue(msgFromBacklog, NUM_OF_QUEUES - 1)) {
1533 dispatchBacklog.addFirst(msgFromBacklog);
1534 return false;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001535 }
1536 }
Anton Chigrin4af4f872019-01-14 17:29:56 +02001537 return true;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001538
Anton Chigrin4af4f872019-01-14 17:29:56 +02001539 }
1540
1541 /**
1542 * Parse OpenFlow message context for get Ethernet packet.
1543 *
1544 * @param message OpenFlow message
1545 * @return parsed Ethernet packet
1546 */
1547 private Ethernet parsePacketInMessage(OFMessage message) {
1548 OpenFlowPacketContext pktCtx = DefaultOpenFlowPacketContext
1549 .packetContextFromPacketIn(sw, (OFPacketIn) message);
1550 DeviceId id = DeviceId.deviceId(Dpid.uri(pktCtx.dpid().value()));
1551 DefaultInboundPacket inPkt = new DefaultInboundPacket(
1552 new ConnectPoint(id, PortNumber.portNumber(pktCtx.inPort())),
1553 pktCtx.parsed(), ByteBuffer.wrap(pktCtx.unparsed()),
1554 pktCtx.cookie());
1555 return inPkt.parsed();
1556 }
1557
1558 /**
1559 * Classify the Ethernet packet for membership on one of the queues.
1560 *
1561 * @param packet ethernet packet
1562 * @return Id of destination Queue
1563 */
1564 private int classifyEthernetPacket(Ethernet packet) {
1565 for (Set<OpenFlowClassifier> classifiers : this.messageClassifiersMapProducer) {
1566 for (OpenFlowClassifier classifier : classifiers) {
1567 if (classifier.ethernetType() == packet.getEtherType()) {
1568 return classifier.idQueue();
1569 }
1570 }
1571 }
1572 return NUM_OF_QUEUES - 1;
1573 }
1574
1575 /**
1576 * Process messages from dispatch queues.
1577 *
1578 * @param queuesSize count of messages in all queues
1579 */
1580 private void processMessages(int queuesSize) {
1581 List<OFMessage> msgs = new ArrayList<>();
1582 int processed;
1583 do {
1584 processed = 0;
1585 while (processed < queuesSize) {
1586 for (LinkedBlockingMessagesQueue<OFMessage> queue :
1587 dispatchQueuesMapProducer.values()) {
1588 processed += queue.drainTo(msgs);
1589 }
1590 }
1591
1592 msgs.forEach(sw::handleMessage);
1593 msgs.clear();
1594 /* Decrement conditional variable */
1595 queuesSize = totalCount.addAndGet(-1 * processed);
1596 } while (queuesSize > 0);
1597 }
1598
1599 private void dispatchMessage(OFMessage m) {
1600 log.debug("Begin dispatch OpenFlow Message");
1601 boolean backlogEmpty = processDispatchBacklogQueue();
1602 if (m.getType() == OFType.PACKET_IN) {
1603 Ethernet pkt = parsePacketInMessage(m);
1604 pushMessageToQueue(m, classifyEthernetPacket(pkt));
1605 } else {
1606 if (!backlogEmpty || !pushMessageToQueue(m, NUM_OF_QUEUES - 1)) {
1607 dispatchBacklog.offer(m);
1608 }
1609 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001610
1611 if (dispatcherHandle.isDone()) {
1612 // dispatcher terminated for some reason, restart
Yuta HIGUCHIfbd9ae92018-01-24 23:39:06 -08001613 dispatcherHandle = dispatcher.submit((Runnable) () -> {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001614 try {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001615 for (;;) {
Anton Chigrin4af4f872019-01-14 17:29:56 +02001616 int tc = 0;
1617 takeLock.lockInterruptibly();
1618 try {
1619 while ((tc = totalCount.get()) == 0) {
1620 notEmpty.await();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001621 }
Anton Chigrin4af4f872019-01-14 17:29:56 +02001622 } finally {
1623 takeLock.unlock();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001624 }
1625
Anton Chigrin4af4f872019-01-14 17:29:56 +02001626 processMessages(tc);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001627 }
1628 } catch (InterruptedException e) {
Anton Chigrin4af4f872019-01-14 17:29:56 +02001629 log.error("executor thread InterruptedException: {}", e);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001630 Thread.currentThread().interrupt();
1631 // interrupted. gracefully shutting down
1632 return;
1633 }
1634
1635 });
1636 }
tom7ef8ff92014-09-17 13:08:06 -07001637 }
1638
1639 /**
1640 * Return a string describing this switch based on the already available
1641 * information (DPID and/or remote socket).
1642 * @return display string
1643 */
1644 private String getSwitchInfoString() {
1645 if (sw != null) {
1646 return sw.toString();
1647 }
1648 String channelString;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001649 if (channel == null || channel.remoteAddress() == null) {
tom7ef8ff92014-09-17 13:08:06 -07001650 channelString = "?";
1651 } else {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001652 channelString = channel.remoteAddress().toString();
tom7ef8ff92014-09-17 13:08:06 -07001653 }
1654 String dpidString;
1655 if (featuresReply == null) {
1656 dpidString = "?";
1657 } else {
1658 dpidString = featuresReply.getDatapathId().toString();
1659 }
1660 return String.format("[%s DPID[%s]]", channelString, dpidString);
1661 }
1662
1663 /**
1664 * Update the channels state. Only called from the state machine.
1665 * TODO: enforce restricted state transitions
1666 * @param state
1667 */
1668 private void setState(ChannelState state) {
1669 this.state = state;
Yuta HIGUCHI10f45132017-03-01 17:09:32 -08001670 this.lastStateChange = System.currentTimeMillis();
tom7ef8ff92014-09-17 13:08:06 -07001671 }
1672
Brian O'Connorf69e3e32018-05-10 02:25:09 -07001673 private boolean setDpid(Long dpid, Channel channel) {
1674 ChannelHandlerContext sslContext = channel.pipeline().context(SslHandler.class);
1675 if (sslContext != null) {
1676 try {
1677 SslHandler sslHandler = (SslHandler) sslContext.handler();
1678 Certificate[] certs = sslHandler.engine().getSession().getPeerCertificates();
1679 Certificate cert = certs.length > 0 ? certs[0] : null;
1680 if (!controller.isValidCertificate(dpid, cert)) {
1681 return false;
1682 }
1683 } catch (SSLPeerUnverifiedException e) {
1684 log.info("Switch with dpid {} is an unverified SSL peer.", dpid, e);
1685 return false;
1686 }
1687 }
1688 this.thisdpid = dpid;
1689 return true;
1690 }
1691
tom7ef8ff92014-09-17 13:08:06 -07001692 /**
1693 * Send hello message to the switch using the handshake transactions ids.
1694 * @throws IOException
1695 */
1696 private void sendHandshakeHelloMessage() throws IOException {
1697 // The OF protocol requires us to start things off by sending the highest
1698 // version of the protocol supported.
1699
Yuta HIGUCHI6512f3e2017-05-18 17:21:24 -07001700 // bitmap represents OF1.0, OF1.3, OF1.4, and OF1.5
tom7ef8ff92014-09-17 13:08:06 -07001701 // see Sec. 7.5.1 of the OF1.3.4 spec
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001702 U32 bitmap = U32.ofRaw((0b1 << OFVersion.OF_10.getWireVersion()) |
1703 (0b1 << OFVersion.OF_13.getWireVersion()) |
Yuta HIGUCHI6512f3e2017-05-18 17:21:24 -07001704 (0b1 << OFVersion.OF_14.getWireVersion()) |
1705 (0b1 << OFVersion.OF_15.getWireVersion()));
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001706 OFVersion version = Optional.ofNullable(ofVersion).orElse(OFVersion.OF_13);
1707 OFHelloElem hem = OFFactories.getFactory(version)
1708 .buildHelloElemVersionbitmap()
tom7ef8ff92014-09-17 13:08:06 -07001709 .setBitmaps(Collections.singletonList(bitmap))
1710 .build();
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001711 OFMessage.Builder mb = OFFactories.getFactory(version)
1712 .buildHello()
tom7ef8ff92014-09-17 13:08:06 -07001713 .setXid(this.handshakeTransactionIds--)
1714 .setElements(Collections.singletonList(hem));
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001715 log.info("Sending {} Hello to {}", version, channel.remoteAddress());
1716 channel.writeAndFlush(Collections.singletonList(mb.build()));
tom7ef8ff92014-09-17 13:08:06 -07001717 }
1718
1719 /**
1720 * Send featuresRequest msg to the switch using the handshake transactions ids.
1721 * @throws IOException
1722 */
1723 private void sendHandshakeFeaturesRequestMessage() throws IOException {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001724 log.debug("Sending FEATURES_REQUEST to {}", channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001725 OFMessage m = factory.buildFeaturesRequest()
1726 .setXid(this.handshakeTransactionIds--)
1727 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001728 channel.writeAndFlush(Collections.singletonList(m));
tom7ef8ff92014-09-17 13:08:06 -07001729 }
1730
1731 /**
1732 * Send the configuration requests to tell the switch we want full
1733 * packets.
1734 * @throws IOException
1735 */
1736 private void sendHandshakeSetConfig() throws IOException {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001737 log.debug("Sending CONFIG_REQUEST to {}", channel.remoteAddress());
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001738 List<OFMessage> msglist = new ArrayList<>(3);
tom7ef8ff92014-09-17 13:08:06 -07001739
1740 // Ensure we receive the full packet via PacketIn
1741 // FIXME: We don't set the reassembly flags.
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001742 // Only send config to switches to send full packets, if they have a buffer.
Michael Jarschel7f521a32015-08-12 16:31:07 +02001743 // Saves a packet & OFSetConfig can't be handled by certain switches.
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001744 if (this.featuresReply.getNBuffers() > 0) {
Michael Jarschel7f521a32015-08-12 16:31:07 +02001745 OFSetConfig sc = factory
1746 .buildSetConfig()
1747 .setMissSendLen((short) 0xffff)
1748 .setXid(this.handshakeTransactionIds--)
1749 .build();
1750 msglist.add(sc);
1751 }
tom7ef8ff92014-09-17 13:08:06 -07001752
1753 // Barrier
1754 OFBarrierRequest br = factory
1755 .buildBarrierRequest()
1756 .setXid(this.handshakeTransactionIds--)
1757 .build();
1758 msglist.add(br);
1759
1760 // Verify (need barrier?)
1761 OFGetConfigRequest gcr = factory
1762 .buildGetConfigRequest()
1763 .setXid(this.handshakeTransactionIds--)
1764 .build();
1765 msglist.add(gcr);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001766 channel.writeAndFlush(msglist);
tom7ef8ff92014-09-17 13:08:06 -07001767 }
1768
1769 /**
1770 * send a description state request.
1771 * @throws IOException
1772 */
1773 private void sendHandshakeDescriptionStatsRequest() throws IOException {
1774 // Get Description to set switch-specific flags
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001775 log.debug("Sending DESC_STATS_REQUEST to {}", channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001776 OFDescStatsRequest dreq = factory
1777 .buildDescStatsRequest()
1778 .setXid(handshakeTransactionIds--)
1779 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001780 channel.writeAndFlush(Collections.singletonList(dreq));
tom7ef8ff92014-09-17 13:08:06 -07001781 }
1782
Jordi Ortiz91477b82016-11-29 15:22:50 +01001783 /**
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001784 * send a meter features request.
1785 *
Jordi Ortiz91477b82016-11-29 15:22:50 +01001786 * @throws IOException
1787 */
1788 private void sendMeterFeaturesRequest() throws IOException {
1789 // Get meter features including the MaxMeters value available for the device
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001790 OFFactory factory = OFFactories.getFactory(ofVersion);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001791 log.debug("Sending METER_FEATURES_REQUEST to {}", channel.remoteAddress());
Jordi Ortiz91477b82016-11-29 15:22:50 +01001792 OFMeterFeaturesStatsRequest mfreq = factory
1793 .buildMeterFeaturesStatsRequest()
1794 .setXid(handshakeTransactionIds--)
1795 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001796 channel.writeAndFlush(Collections.singletonList(mfreq));
Jordi Ortiz91477b82016-11-29 15:22:50 +01001797 }
1798
tom7ef8ff92014-09-17 13:08:06 -07001799 private void sendHandshakeOFPortDescRequest() throws IOException {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001800 log.debug("Sending OF_PORT_DESC_REQUEST to {}", channel.remoteAddress());
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001801 // Get port description for 1.3+ switch
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001802 OFPortDescStatsRequest preq = factory
tom7ef8ff92014-09-17 13:08:06 -07001803 .buildPortDescStatsRequest()
1804 .setXid(handshakeTransactionIds--)
1805 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001806 channel.writeAndFlush(Collections.singletonList(preq));
tom7ef8ff92014-09-17 13:08:06 -07001807 }
1808
1809 ChannelState getStateForTesting() {
1810 return state;
1811 }
1812
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001813
1814 @Override
1815 public boolean isActive() {
1816 if (channel != null) {
1817 return channel.isActive();
1818 }
1819 return false;
1820 }
1821
1822 @Override
1823 public void closeSession() {
1824 if (channel != null) {
1825 channel.close();
1826 }
1827 }
1828
1829 @Override
1830 public boolean sendMsg(Iterable<OFMessage> msgs) {
1831 if (channel.isActive()) {
Laszlo Pappb68fe7e2017-11-24 17:06:59 +00001832 if (log.isTraceEnabled()) {
1833 log.trace("Sending messages for switch {} via openflow channel: {}", getSwitchInfoString(), msgs);
1834 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001835 channel.writeAndFlush(msgs, channel.voidPromise());
1836 return true;
1837 } else {
1838 log.warn("Dropping messages for switch {} because channel is not connected: {}",
1839 getSwitchInfoString(), msgs);
1840 return false;
1841 }
1842 }
1843
1844 @Override
1845 public CharSequence sessionInfo() {
1846 return channelId;
1847 }
1848
Anton Chigrin4af4f872019-01-14 17:29:56 +02001849 @Override
1850 public void addClassifier(OpenFlowClassifier classifier) {
1851 if (this.deviceId.equals(classifier.deviceId())) {
1852 log.debug("Add OpenFlow Classifier for switch {} to queue {} with type {}",
1853 classifier.deviceId().toString(), classifier.idQueue(), classifier.ethernetType());
1854 this.messageClassifiersMapProducer.get(classifier.idQueue()).add(classifier);
1855 }
1856 }
1857
1858 @Override
1859 public void removeClassifier(OpenFlowClassifier classifier) {
1860 if (this.deviceId.equals(classifier.deviceId())) {
1861 log.debug("Remove OpenFlow Classifier for switch {} from queue {} with type {}",
1862 classifier.deviceId().toString(), classifier.idQueue(), classifier.ethernetType());
1863 this.messageClassifiersMapProducer.get(classifier.idQueue()).remove(classifier);
1864 }
1865 }
1866
1867 /**
1868 * Init classifier configuration for the switch. Use stored configuration if exist.
1869 * Otherwise add LLDP and BDDP classifiers for Queue N0.
1870 */
1871 private void initClassifiers() {
1872 try {
1873 openFlowManager = DefaultServiceDirectory.getService(OpenFlowService.class);
1874 DeviceId did = DeviceId.deviceId(uri(thisdpid));
1875 Set<OpenFlowClassifier> classifiers = openFlowManager.getClassifiersByDeviceId(did);
1876 if (classifiers == null) {
1877 OpenFlowClassifier classifier =
1878 new OpenFlowClassifier.Builder(did, 0).ethernetType(TYPE_LLDP).build();
1879 openFlowManager.add(classifier);
1880 classifier = new OpenFlowClassifier.Builder(did, 0).ethernetType(TYPE_BSN).build();
1881 openFlowManager.add(classifier);
1882 } else {
1883 this.messageClassifiersMapProducer.forEach((v) -> {
1884 v.clear();
1885 });
1886 classifiers.forEach((c) -> {
1887 messageClassifiersMapProducer.get(c.idQueue()).add(c);
1888 });
1889 }
1890 } catch (Exception e) {
1891 log.error("Initialize default classifier failed: {}", e.toString());
1892 e.printStackTrace();
1893 }
1894 }
tom7ef8ff92014-09-17 13:08:06 -07001895}