blob: 84d9deba1eeed0ad222e8b0b207ae843ceaec405 [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
Andrea Campanelladad65ac2020-11-06 21:41:01 +010019import static java.util.concurrent.Executors.newSingleThreadExecutor;
Anton Chigrin4af4f872019-01-14 17:29:56 +020020import static org.onlab.packet.Ethernet.TYPE_BSN;
21import static org.onlab.packet.Ethernet.TYPE_LLDP;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070022import static org.onlab.util.Tools.groupedThreads;
Anton Chigrin4af4f872019-01-14 17:29:56 +020023import static org.onosproject.openflow.controller.Dpid.uri;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070024
tom7ef8ff92014-09-17 13:08:06 -070025import java.io.IOException;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070026import java.net.InetSocketAddress;
27import java.net.SocketAddress;
Anton Chigrin4af4f872019-01-14 17:29:56 +020028import java.nio.ByteBuffer;
tom7ef8ff92014-09-17 13:08:06 -070029import java.nio.channels.ClosedChannelException;
Brian O'Connorf69e3e32018-05-10 02:25:09 -070030import java.security.cert.Certificate;
tom7ef8ff92014-09-17 13:08:06 -070031import java.util.ArrayList;
32import java.util.Collections;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070033import java.util.Deque;
tom7ef8ff92014-09-17 13:08:06 -070034import java.util.List;
Anton Chigrin4af4f872019-01-14 17:29:56 +020035import java.util.Map;
Yuta HIGUCHI2341e602017-03-08 20:10:08 -080036import java.util.Optional;
Anton Chigrin4af4f872019-01-14 17:29:56 +020037import java.util.Set;
38import java.util.concurrent.LinkedBlockingDeque;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070039import java.util.concurrent.CompletableFuture;
Anton Chigrin4af4f872019-01-14 17:29:56 +020040import java.util.concurrent.ConcurrentHashMap;
tom7ef8ff92014-09-17 13:08:06 -070041import java.util.concurrent.CopyOnWriteArrayList;
Anton Chigrin4af4f872019-01-14 17:29:56 +020042import java.util.concurrent.CopyOnWriteArraySet;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070043import java.util.concurrent.ExecutorService;
44import java.util.concurrent.Executors;
45import java.util.concurrent.Future;
tom7ef8ff92014-09-17 13:08:06 -070046import java.util.concurrent.RejectedExecutionException;
Anton Chigrin4af4f872019-01-14 17:29:56 +020047import java.util.concurrent.atomic.AtomicInteger;
48import java.util.concurrent.locks.Condition;
49import java.util.concurrent.locks.ReentrantLock;
Brian O'Connorf69e3e32018-05-10 02:25:09 -070050
Anton Chigrin4af4f872019-01-14 17:29:56 +020051import org.osgi.service.component.annotations.Reference;
52import org.osgi.service.component.annotations.ReferenceCardinality;
53import org.onlab.osgi.DefaultServiceDirectory;
54import org.onlab.packet.Ethernet;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070055import org.onlab.packet.IpAddress;
Anton Chigrin4af4f872019-01-14 17:29:56 +020056import org.onosproject.net.ConnectPoint;
57import org.onosproject.net.DeviceId;
58import org.onosproject.net.PortNumber;
59import org.onosproject.net.packet.DefaultInboundPacket;
60import org.onosproject.openflow.controller.DefaultOpenFlowPacketContext;
Charles Chan34155e52016-11-30 18:28:11 -080061import org.onosproject.openflow.controller.Dpid;
Anton Chigrin4af4f872019-01-14 17:29:56 +020062import org.onosproject.openflow.controller.OpenFlowClassifier;
63import org.onosproject.openflow.controller.OpenFlowPacketContext;
64import org.onosproject.openflow.controller.OpenFlowService;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -070065import org.onosproject.openflow.controller.OpenFlowSession;
Brian O'Connorabafb502014-12-02 22:26:20 -080066import org.onosproject.openflow.controller.driver.OpenFlowSwitchDriver;
67import org.onosproject.openflow.controller.driver.SwitchStateException;
tom7ef8ff92014-09-17 13:08:06 -070068import org.projectfloodlight.openflow.exceptions.OFParseError;
69import org.projectfloodlight.openflow.protocol.OFAsyncGetReply;
70import org.projectfloodlight.openflow.protocol.OFBadRequestCode;
71import org.projectfloodlight.openflow.protocol.OFBarrierReply;
72import org.projectfloodlight.openflow.protocol.OFBarrierRequest;
73import org.projectfloodlight.openflow.protocol.OFDescStatsReply;
74import org.projectfloodlight.openflow.protocol.OFDescStatsRequest;
75import org.projectfloodlight.openflow.protocol.OFEchoReply;
76import org.projectfloodlight.openflow.protocol.OFEchoRequest;
77import org.projectfloodlight.openflow.protocol.OFErrorMsg;
78import org.projectfloodlight.openflow.protocol.OFErrorType;
79import org.projectfloodlight.openflow.protocol.OFExperimenter;
Yuta HIGUCHI2341e602017-03-08 20:10:08 -080080import org.projectfloodlight.openflow.protocol.OFFactories;
tom7ef8ff92014-09-17 13:08:06 -070081import org.projectfloodlight.openflow.protocol.OFFactory;
82import org.projectfloodlight.openflow.protocol.OFFeaturesReply;
83import org.projectfloodlight.openflow.protocol.OFFlowModFailedCode;
84import org.projectfloodlight.openflow.protocol.OFFlowRemoved;
85import org.projectfloodlight.openflow.protocol.OFGetConfigReply;
86import org.projectfloodlight.openflow.protocol.OFGetConfigRequest;
87import org.projectfloodlight.openflow.protocol.OFHello;
88import org.projectfloodlight.openflow.protocol.OFHelloElem;
89import org.projectfloodlight.openflow.protocol.OFMessage;
Jordi Ortiz91477b82016-11-29 15:22:50 +010090import org.projectfloodlight.openflow.protocol.OFMeterFeaturesStatsReply;
91import org.projectfloodlight.openflow.protocol.OFMeterFeaturesStatsRequest;
tom7ef8ff92014-09-17 13:08:06 -070092import org.projectfloodlight.openflow.protocol.OFPacketIn;
93import org.projectfloodlight.openflow.protocol.OFPortDescStatsReply;
94import org.projectfloodlight.openflow.protocol.OFPortDescStatsRequest;
95import org.projectfloodlight.openflow.protocol.OFPortStatus;
96import org.projectfloodlight.openflow.protocol.OFQueueGetConfigReply;
97import org.projectfloodlight.openflow.protocol.OFRoleReply;
98import org.projectfloodlight.openflow.protocol.OFSetConfig;
99import org.projectfloodlight.openflow.protocol.OFStatsReply;
100import org.projectfloodlight.openflow.protocol.OFStatsReplyFlags;
101import org.projectfloodlight.openflow.protocol.OFStatsType;
102import org.projectfloodlight.openflow.protocol.OFType;
103import org.projectfloodlight.openflow.protocol.OFVersion;
104import org.projectfloodlight.openflow.protocol.errormsg.OFBadRequestErrorMsg;
105import org.projectfloodlight.openflow.protocol.errormsg.OFFlowModFailedErrorMsg;
106import org.projectfloodlight.openflow.types.U32;
107import org.slf4j.Logger;
108import org.slf4j.LoggerFactory;
109
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700110import io.netty.channel.Channel;
111import io.netty.channel.ChannelHandlerContext;
112import io.netty.channel.ChannelInboundHandlerAdapter;
Brian O'Connorf69e3e32018-05-10 02:25:09 -0700113import io.netty.handler.ssl.SslHandler;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700114import io.netty.handler.timeout.IdleStateEvent;
115import io.netty.handler.timeout.ReadTimeoutException;
116import io.netty.util.ReferenceCountUtil;
117
Brian O'Connorf69e3e32018-05-10 02:25:09 -0700118import javax.net.ssl.SSLPeerUnverifiedException;
119
tom7ef8ff92014-09-17 13:08:06 -0700120/**
121 * Channel handler deals with the switch connection and dispatches
122 * switch messages to the appropriate locations.
123 */
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700124class OFChannelHandler extends ChannelInboundHandlerAdapter
125 implements OpenFlowSession {
126
tom7ef8ff92014-09-17 13:08:06 -0700127 private static final Logger log = LoggerFactory.getLogger(OFChannelHandler.class);
Thomas Vachuskae9af1f42015-07-06 08:42:18 -0700128
129 private static final String RESET_BY_PEER = "Connection reset by peer";
130 private static final String BROKEN_PIPE = "Broken pipe";
pieref3d74c2019-07-16 15:52:50 +0200131 static final int NUM_OF_QUEUES = 8;
Thomas Vachuskae9af1f42015-07-06 08:42:18 -0700132
tom7ef8ff92014-09-17 13:08:06 -0700133 private final Controller controller;
134 private OpenFlowSwitchDriver sw;
135 private long thisdpid; // channelHandler cached value of connected switch id
Anton Chigrin4af4f872019-01-14 17:29:56 +0200136 private DeviceId deviceId;
tom7ef8ff92014-09-17 13:08:06 -0700137 private Channel channel;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700138 private String channelId;
139
140
tom7ef8ff92014-09-17 13:08:06 -0700141 // State needs to be volatile because the HandshakeTimeoutHandler
142 // needs to check if the handshake is complete
143 private volatile ChannelState state;
144
Yuta HIGUCHI10f45132017-03-01 17:09:32 -0800145 /**
146 * Timeout in ms to wait for meter feature reply.
147 */
148 private static final long METER_TIMEOUT = 60_000;
149
150 private volatile long lastStateChange = System.currentTimeMillis();
151
tom7ef8ff92014-09-17 13:08:06 -0700152 // When a switch with a duplicate dpid is found (i.e we already have a
153 // connected switch with the same dpid), the new switch is immediately
154 // disconnected. At that point netty callsback channelDisconnected() which
155 // proceeds to cleaup switch state - we need to ensure that it does not cleanup
156 // switch state for the older (still connected) switch
157 private volatile Boolean duplicateDpidFound;
158
159 // Temporary storage for switch-features and port-description
160 private OFFeaturesReply featuresReply;
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700161 private List<OFPortDescStatsReply> portDescReplies;
Jordi Ortiz91477b82016-11-29 15:22:50 +0100162 private OFMeterFeaturesStatsReply meterFeaturesReply;
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700163 //private OFPortDescStatsReply portDescReply;
tom7ef8ff92014-09-17 13:08:06 -0700164 // a concurrent ArrayList to temporarily store port status messages
165 // before we are ready to deal with them
166 private final CopyOnWriteArrayList<OFPortStatus> pendingPortStatusMsg;
167
168 //Indicates the openflow version used by this switch
169 protected OFVersion ofVersion;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700170 protected OFFactory factory;
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800171
tom7ef8ff92014-09-17 13:08:06 -0700172 /** transaction Ids to use during handshake. Since only one thread
173 * calls into an OFChannelHandler instance, we don't need atomic.
174 * We will count down
175 */
176 private int handshakeTransactionIds = -1;
177
Anton Chigrin4af4f872019-01-14 17:29:56 +0200178 @Reference(cardinality = ReferenceCardinality.MANDATORY)
179 private OpenFlowService openFlowManager;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700180
pier063e9032019-11-22 20:51:26 +0100181 // Each IDLE_INTERVAL (see OFChannelInitializer) we will perform a sanity check
182 // Idle timeout actions will be performed each MAX_IDLE_RETRY * IDLE_INTERVAL
183 private static final int MAX_IDLE_RETRY = 4;
184 private int maxIdleRetry = MAX_IDLE_RETRY;
185
186 // Dispatcher buffer/read size
Anton Chigrin4af4f872019-01-14 17:29:56 +0200187 private static final int BACKLOG_READ_BUFFER_DEFAULT = 1000;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700188
189 /**
Anton Chigrin4af4f872019-01-14 17:29:56 +0200190 * Map with all LinkedBlockingMessagesQueue queues which contains OFMessages.
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700191 */
Anton Chigrin4af4f872019-01-14 17:29:56 +0200192 private Map<Integer, LinkedBlockingMessagesQueue<OFMessage>> dispatchQueuesMapProducer = new ConcurrentHashMap<>();
193
194 /**
195 * OFMessage classifiers map.
196 */
197 private List<Set<OpenFlowClassifier>> messageClassifiersMapProducer =
198 new CopyOnWriteArrayList<Set<OpenFlowClassifier>>();
199
200
201 /**
202 * Lock held by take, poll, etc.
203 */
204 private final ReentrantLock takeLock = new ReentrantLock();
205
206 /**
207 * Wait queue for waiting takes.
208 */
209 private final Condition notEmpty = takeLock.newCondition();
210
211 /**
212 * Current number of elements in enabled sub-queues.
213 */
214 private final AtomicInteger totalCount = new AtomicInteger();
215
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700216
217 /**
218 * Single thread executor for OFMessage dispatching.
219 *
220 * Gets initialized on channelActive, shutdown on channelInactive.
221 */
222 private ExecutorService dispatcher;
223
224 /**
225 * Handle for dispatcher thread.
226 * <p>
227 * Should only be touched from the Channel I/O thread
228 */
229 private Future<?> dispatcherHandle = CompletableFuture.completedFuture(null);
230
231 /**
232 * Dispatch backlog.
233 * <p>
234 * Should only be touched from the Channel I/O thread
235 */
Anton Chigrin4af4f872019-01-14 17:29:56 +0200236 private final Deque<OFMessage> dispatchBacklog;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700237
tom7ef8ff92014-09-17 13:08:06 -0700238 /**
Andrea Campanelladad65ac2020-11-06 21:41:01 +0100239 * Port Status executor to offload from the main thread the processing of port
240 * status OF messages.
241 */
242 protected ExecutorService portStatusExecutor;
243
244 /**
tom7ef8ff92014-09-17 13:08:06 -0700245 * Create a new unconnected OFChannelHandler.
Thomas Vachuskab14c77a2014-11-04 18:08:01 -0800246 * @param controller parent controller
tom7ef8ff92014-09-17 13:08:06 -0700247 */
248 OFChannelHandler(Controller controller) {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700249
tom7ef8ff92014-09-17 13:08:06 -0700250 this.controller = controller;
251 this.state = ChannelState.INIT;
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800252 this.pendingPortStatusMsg = new CopyOnWriteArrayList<>();
253 this.portDescReplies = new ArrayList<>();
tom7ef8ff92014-09-17 13:08:06 -0700254 duplicateDpidFound = Boolean.FALSE;
Andrea Campanelladad65ac2020-11-06 21:41:01 +0100255 portStatusExecutor = newSingleThreadExecutor(
256 groupedThreads("onos/of-channel-handler", "port-status-%d", log));
Anton Chigrin4af4f872019-01-14 17:29:56 +0200257 //Initialize queues and classifiers
258 dispatchBacklog = new LinkedBlockingDeque<>(BACKLOG_READ_BUFFER_DEFAULT);
259 for (int i = 0; i < NUM_OF_QUEUES; i++) {
260 if (controller.getQueueSize(i) > 0) {
261 dispatchQueuesMapProducer.put(i,
262 new LinkedBlockingMessagesQueue<>(i, controller.getQueueSize(i), controller.getBulkSize(i)));
263 }
264 if (i != NUM_OF_QUEUES) {
265 messageClassifiersMapProducer.add(i, new CopyOnWriteArraySet<>());
266 }
267 }
tom7ef8ff92014-09-17 13:08:06 -0700268 }
269
270
271
272 // XXX S consider if necessary
273 public void disconnectSwitch() {
274 sw.disconnectSwitch();
275 }
276
277
278
279 //*************************
280 // Channel State Machine
281 //*************************
282
283 /**
284 * The state machine for handling the switch/channel state. All state
285 * transitions should happen from within the state machine (and not from other
286 * parts of the code)
287 */
288 enum ChannelState {
289 /**
290 * Initial state before channel is connected.
291 */
292 INIT(false) {
293 @Override
294 void processOFMessage(OFChannelHandler h, OFMessage m)
295 throws IOException, SwitchStateException {
296 illegalMessageReceived(h, m);
297 }
298
299 @Override
300 void processOFError(OFChannelHandler h, OFErrorMsg m)
301 throws IOException {
302 // need to implement since its abstract but it will never
303 // be called
304 }
305
306 @Override
307 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
308 throws IOException {
309 unhandledMessageReceived(h, m);
310 }
311 },
312
313 /**
314 * We send a OF 1.3 HELLO to the switch and wait for a Hello from the switch.
315 * Once we receive the reply, we decide on OF 1.3 or 1.0 switch - no other
316 * protocol version is accepted.
317 * We send an OFFeaturesRequest depending on the protocol version selected
318 * Next state is WAIT_FEATURES_REPLY
319 */
320 WAIT_HELLO(false) {
321 @Override
322 void processOFHello(OFChannelHandler h, OFHello m)
323 throws IOException {
324 // TODO We could check for the optional bitmap, but for now
325 // we are just checking the version number.
Chip Boling68bc6562015-07-06 10:00:01 -0500326 if (m.getVersion().getWireVersion() >= OFVersion.OF_13.getWireVersion()) {
327 log.debug("Received {} Hello from {} - switching to OF "
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800328 + "version 1.3+", m.getVersion(),
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700329 h.channel.remoteAddress());
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800330 h.ofVersion = m.getVersion();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700331 h.factory = OFFactories.getFactory(h.ofVersion);
alshabib70fc7fb2015-01-06 11:04:29 -0800332 h.sendHandshakeHelloMessage();
Chip Boling68bc6562015-07-06 10:00:01 -0500333 } else if (m.getVersion().getWireVersion() >= OFVersion.OF_10.getWireVersion()) {
alshabib09d48be2014-10-03 15:43:33 -0700334 log.debug("Received {} Hello from {} - switching to OF "
tom7ef8ff92014-09-17 13:08:06 -0700335 + "version 1.0", m.getVersion(),
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700336 h.channel.remoteAddress());
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800337 h.ofVersion = m.getVersion();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700338 h.factory = OFFactories.getFactory(h.ofVersion);
alshabib70fc7fb2015-01-06 11:04:29 -0800339 OFHello hi =
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700340 h.factory.buildHello()
alshabib70fc7fb2015-01-06 11:04:29 -0800341 .setXid(h.handshakeTransactionIds--)
342 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700343 h.channel.writeAndFlush(Collections.singletonList(hi));
tom7ef8ff92014-09-17 13:08:06 -0700344 } else {
345 log.error("Received Hello of version {} from switch at {}. "
346 + "This controller works with OF1.0 and OF1.3 "
347 + "switches. Disconnecting switch ...",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700348 m.getVersion(), h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -0700349 h.channel.disconnect();
350 return;
351 }
352 h.sendHandshakeFeaturesRequestMessage();
353 h.setState(WAIT_FEATURES_REPLY);
354 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700355
tom7ef8ff92014-09-17 13:08:06 -0700356 @Override
357 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
358 throws IOException, SwitchStateException {
359 illegalMessageReceived(h, m);
360 }
361 @Override
362 void processOFStatisticsReply(OFChannelHandler h,
363 OFStatsReply m)
364 throws IOException, SwitchStateException {
365 illegalMessageReceived(h, m);
366 }
367 @Override
368 void processOFError(OFChannelHandler h, OFErrorMsg m) {
369 logErrorDisconnect(h, m);
370 }
371
372 @Override
373 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
374 throws IOException {
375 unhandledMessageReceived(h, m);
376 }
377 },
378
379
380 /**
381 * We are waiting for a features reply message. Once we receive it, the
382 * behavior depends on whether this is a 1.0 or 1.3 switch. For 1.0,
383 * we send a SetConfig request, barrier, and GetConfig request and the
384 * next state is WAIT_CONFIG_REPLY. For 1.3, we send a Port description
385 * request and the next state is WAIT_PORT_DESC_REPLY.
386 */
387 WAIT_FEATURES_REPLY(false) {
388 @Override
Brian O'Connorf69e3e32018-05-10 02:25:09 -0700389 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
tom7ef8ff92014-09-17 13:08:06 -0700390 throws IOException {
Brian O'Connorf69e3e32018-05-10 02:25:09 -0700391 Long dpid = m.getDatapathId().getLong();
392 if (!h.setDpid(dpid, h.channel)) {
393 log.error("Switch presented invalid certificate for dpid {}. Disconnecting",
394 dpid);
395 h.channel.disconnect();
396 return;
397 }
Anton Chigrin4af4f872019-01-14 17:29:56 +0200398 h.deviceId = DeviceId.deviceId(uri(h.thisdpid));
alshabib09d48be2014-10-03 15:43:33 -0700399 log.debug("Received features reply for switch at {} with dpid {}",
tom7ef8ff92014-09-17 13:08:06 -0700400 h.getSwitchInfoString(), h.thisdpid);
401
402 h.featuresReply = m; //temp store
403 if (h.ofVersion == OFVersion.OF_10) {
404 h.sendHandshakeSetConfig();
405 h.setState(WAIT_CONFIG_REPLY);
406 } else {
407 //version is 1.3, must get switchport information
408 h.sendHandshakeOFPortDescRequest();
409 h.setState(WAIT_PORT_DESC_REPLY);
410 }
411 }
412 @Override
413 void processOFStatisticsReply(OFChannelHandler h,
414 OFStatsReply m)
415 throws IOException, SwitchStateException {
416 illegalMessageReceived(h, m);
417 }
418 @Override
419 void processOFError(OFChannelHandler h, OFErrorMsg m) {
420 logErrorDisconnect(h, m);
421 }
422
423 @Override
424 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
425 throws IOException {
Thomas Vachuska39274462014-12-02 13:23:50 -0800426 h.pendingPortStatusMsg.add(m);
tom7ef8ff92014-09-17 13:08:06 -0700427 }
428 },
429
430 /**
431 * We are waiting for a description of the 1.3 switch ports.
432 * Once received, we send a SetConfig request
433 * Next State is WAIT_CONFIG_REPLY
434 */
435 WAIT_PORT_DESC_REPLY(false) {
436
437 @Override
438 void processOFStatisticsReply(OFChannelHandler h, OFStatsReply m)
439 throws SwitchStateException {
440 // Read port description
441 if (m.getStatsType() != OFStatsType.PORT_DESC) {
442 log.warn("Expecting port description stats but received stats "
443 + "type {} from {}. Ignoring ...", m.getStatsType(),
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700444 h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -0700445 return;
446 }
447 if (m.getFlags().contains(OFStatsReplyFlags.REPLY_MORE)) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700448 log.debug("Stats reply indicates more stats from sw {} for "
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700449 + "port description",
tom7ef8ff92014-09-17 13:08:06 -0700450 h.getSwitchInfoString());
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800451 h.portDescReplies.add((OFPortDescStatsReply) m);
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700452 return;
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800453 } else {
454 h.portDescReplies.add((OFPortDescStatsReply) m);
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700455 }
456 //h.portDescReply = (OFPortDescStatsReply) m; // temp store
Andrea Campanellaf9a5ecb2020-05-22 12:42:12 +0200457 if (log.isDebugEnabled()) {
458 log.debug("Received port desc reply for switch at {}: {}",
459 h.getSwitchInfoString(),
460 ((OFPortDescStatsReply) m).getEntries());
461 }
tom7ef8ff92014-09-17 13:08:06 -0700462 try {
463 h.sendHandshakeSetConfig();
464 } catch (IOException e) {
465 log.error("Unable to send setConfig after PortDescReply. "
466 + "Error: {}", e.getMessage());
467 }
468 h.setState(WAIT_CONFIG_REPLY);
469 }
470
471 @Override
472 void processOFError(OFChannelHandler h, OFErrorMsg m)
473 throws IOException, SwitchStateException {
474 logErrorDisconnect(h, m);
475
476 }
477
478 @Override
479 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
480 throws IOException, SwitchStateException {
Thomas Vachuska39274462014-12-02 13:23:50 -0800481 h.pendingPortStatusMsg.add(m);
tom7ef8ff92014-09-17 13:08:06 -0700482
483 }
484 },
485
486 /**
487 * We are waiting for a config reply message. Once we receive it
488 * we send a DescriptionStatsRequest to the switch.
489 * Next state: WAIT_DESCRIPTION_STAT_REPLY
490 */
491 WAIT_CONFIG_REPLY(false) {
492 @Override
493 void processOFGetConfigReply(OFChannelHandler h, OFGetConfigReply m)
494 throws IOException {
495 if (m.getMissSendLen() == 0xffff) {
496 log.trace("Config Reply from switch {} confirms "
497 + "miss length set to 0xffff",
498 h.getSwitchInfoString());
499 } else {
500 // FIXME: we can't really deal with switches that don't send
501 // full packets. Shouldn't we drop the connection here?
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800502 log.warn("Config Reply from switch {} has "
tom7ef8ff92014-09-17 13:08:06 -0700503 + "miss length set to {}",
504 h.getSwitchInfoString(),
505 m.getMissSendLen());
506 }
Jordi Ortiz91477b82016-11-29 15:22:50 +0100507
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800508 nextState(h);
509 }
510
511 /**
512 * Transition to next state based on OF version.
513 *
514 * @param h current channel handler
515 * @throws IOException
516 */
517 private void nextState(OFChannelHandler h) throws IOException {
518 if (h.ofVersion.getWireVersion() >= OFVersion.OF_13.getWireVersion()) {
Jordi Ortiz91477b82016-11-29 15:22:50 +0100519 // Meters were introduced in OpenFlow 1.3
520 h.sendMeterFeaturesRequest();
521 h.setState(WAIT_METER_FEATURES_REPLY);
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800522 } else {
Jordi Ortiz91477b82016-11-29 15:22:50 +0100523 h.sendHandshakeDescriptionStatsRequest();
524 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
525 }
tom7ef8ff92014-09-17 13:08:06 -0700526 }
527
528 @Override
529 void processOFBarrierReply(OFChannelHandler h, OFBarrierReply m) {
530 // do nothing;
531 }
532
533 @Override
534 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
535 throws IOException, SwitchStateException {
536 illegalMessageReceived(h, m);
537 }
Anton Chigrin4af4f872019-01-14 17:29:56 +0200538
tom7ef8ff92014-09-17 13:08:06 -0700539 @Override
540 void processOFStatisticsReply(OFChannelHandler h,
541 OFStatsReply m)
542 throws IOException, SwitchStateException {
543 log.error("Received multipart(stats) message sub-type {}",
544 m.getStatsType());
545 illegalMessageReceived(h, m);
546 }
547
548 @Override
549 void processOFError(OFChannelHandler h, OFErrorMsg m) {
Yuta HIGUCHI2341e602017-03-08 20:10:08 -0800550 if (m.getErrType() == OFErrorType.BAD_REQUEST) {
551 OFBadRequestErrorMsg badRequest = (OFBadRequestErrorMsg) m;
552 if (badRequest.getCode() == OFBadRequestCode.BAD_TYPE) {
553 log.debug("{} does not support GetConfig, moving on", h.getSwitchInfoString());
554 try {
555 nextState(h);
556 return;
557 } catch (IOException e) {
558 log.error("Exception thrown transitioning to next", e);
559 logErrorDisconnect(h, m);
560 }
561 }
562 }
tom7ef8ff92014-09-17 13:08:06 -0700563 logErrorDisconnect(h, m);
564 }
565
566 @Override
567 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
568 throws IOException {
569 h.pendingPortStatusMsg.add(m);
570 }
571 },
572
573
574 /**
575 * We are waiting for a OFDescriptionStat message from the switch.
576 * Once we receive any stat message we try to parse it. If it's not
577 * a description stats message we disconnect. If its the expected
578 * description stats message, we:
579 * - use the switch driver to bind the switch and get an IOFSwitch instance
580 * - setup the IOFSwitch instance
581 * - add switch controller and send the initial role
582 * request to the switch.
583 * Next state: WAIT_INITIAL_ROLE
584 * In the typical case, where switches support role request messages
585 * the next state is where we expect the role reply message.
586 * In the special case that where the switch does not support any kind
587 * of role request messages, we don't send a role message, but we do
588 * request mastership from the registry service. This controller
589 * should become master once we hear back from the registry service.
590 * All following states will have a h.sw instance!
591 */
592 WAIT_DESCRIPTION_STAT_REPLY(false) {
593 @Override
594 void processOFStatisticsReply(OFChannelHandler h, OFStatsReply m)
595 throws SwitchStateException {
596 // Read description, if it has been updated
597 if (m.getStatsType() != OFStatsType.DESC) {
598 log.warn("Expecting Description stats but received stats "
599 + "type {} from {}. Ignoring ...", m.getStatsType(),
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700600 h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -0700601 return;
602 }
tom7ef8ff92014-09-17 13:08:06 -0700603 OFDescStatsReply drep = (OFDescStatsReply) m;
Saurav Dasf9ba4222015-05-07 17:13:59 -0700604 log.info("Received switch description reply {} from switch at {}",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700605 drep, h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -0700606 // Here is where we differentiate between different kinds of switches
607 h.sw = h.controller.getOFSwitchInstance(h.thisdpid, drep, h.ofVersion);
608
Ray Milkey31b00482019-02-07 08:06:28 -0800609 if (h.sw == null) {
610 log.info("Switch not found for {}", h.thisdpid);
611 return;
612 }
613
Andrea Campanellaf9a5ecb2020-05-22 12:42:12 +0200614 // set switch information
tom7ef8ff92014-09-17 13:08:06 -0700615 h.sw.setOFVersion(h.ofVersion);
616 h.sw.setFeaturesReply(h.featuresReply);
Srikanth Vavilapallif5b234a2015-04-21 13:04:13 -0700617 h.sw.setPortDescReplies(h.portDescReplies);
Jordi Ortiz91477b82016-11-29 15:22:50 +0100618 h.sw.setMeterFeaturesReply(h.meterFeaturesReply);
tom7ef8ff92014-09-17 13:08:06 -0700619 h.sw.setConnected(true);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -0700620 h.sw.setChannel(h);
Andrea Campanellaf9a5ecb2020-05-22 12:42:12 +0200621
622 //Port Description List has served its purpose, clearing.
623 h.portDescReplies.clear();
624
Praseed Balakrishnana22eadf2014-10-20 14:21:45 -0700625// boolean success = h.sw.connectSwitch();
626//
627// if (!success) {
628// disconnectDuplicate(h);
629// return;
630// }
tom7ef8ff92014-09-17 13:08:06 -0700631
alshabib09d48be2014-10-03 15:43:33 -0700632 log.debug("Switch {} bound to class {}, description {}",
Ray Milkey6bc43c22015-11-06 13:22:38 -0800633 h.sw, h.sw.getClass(), drep);
tom7ef8ff92014-09-17 13:08:06 -0700634 //Put switch in EQUAL mode until we hear back from the global registry
635 //log.debug("Setting new switch {} to EQUAL and sending Role request",
636 // h.sw.getStringId());
637 //h.sw.activateEqualSwitch();
638 //h.setSwitchRole(RoleState.EQUAL);
639
640 h.sw.startDriverHandshake();
alshabib9eab22f2014-10-20 17:17:31 -0700641 if (h.sw.isDriverHandshakeComplete()) {
pieref3d74c2019-07-16 15:52:50 +0200642 // We are not able to complete the connection for a dpid collision.
643 // Same device reconnecting or different device configured with
644 // the same dpid.
alshabib9eab22f2014-10-20 17:17:31 -0700645 if (!h.sw.connectSwitch()) {
pieref3d74c2019-07-16 15:52:50 +0200646 // Disconnect from the device and return
alshabib9eab22f2014-10-20 17:17:31 -0700647 disconnectDuplicate(h);
pieref3d74c2019-07-16 15:52:50 +0200648 return;
Anton Chigrin4af4f872019-01-14 17:29:56 +0200649 } else {
650 h.initClassifiers();
alshabib9eab22f2014-10-20 17:17:31 -0700651 }
Thomas Vachuska39274462014-12-02 13:23:50 -0800652 handlePendingPortStatusMessages(h);
alshabib9eab22f2014-10-20 17:17:31 -0700653 h.setState(ACTIVE);
654 } else {
655 h.setState(WAIT_SWITCH_DRIVER_SUB_HANDSHAKE);
656 }
tom7ef8ff92014-09-17 13:08:06 -0700657
658 }
659
660 @Override
661 void processOFError(OFChannelHandler h, OFErrorMsg m) {
662 logErrorDisconnect(h, m);
663 }
664
665 @Override
666 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
667 throws IOException, SwitchStateException {
668 illegalMessageReceived(h, m);
669 }
670
671 @Override
672 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
673 throws IOException {
674 h.pendingPortStatusMsg.add(m);
675 }
676 },
677
678
679 /**
680 * We are waiting for the respective switch driver to complete its
681 * configuration. Notice that we do not consider this to be part of the main
682 * switch-controller handshake. But we do consider it as a step that comes
683 * before we declare the switch as available to the controller.
684 * Next State: depends on the role of this controller for this switch - either
685 * MASTER or EQUAL.
686 */
687 WAIT_SWITCH_DRIVER_SUB_HANDSHAKE(true) {
688
689 @Override
690 void processOFError(OFChannelHandler h, OFErrorMsg m)
691 throws IOException {
692 // will never be called. We override processOFMessage
693 }
694
alshabibd7963912014-10-20 14:52:04 -0700695
696
tom7ef8ff92014-09-17 13:08:06 -0700697 @Override
698 void processOFMessage(OFChannelHandler h, OFMessage m)
699 throws IOException, SwitchStateException {
alshabibd7963912014-10-20 14:52:04 -0700700
701 if (h.sw.isDriverHandshakeComplete()) {
702 moveToActive(h);
alshabib9eab22f2014-10-20 17:17:31 -0700703 h.state.processOFMessage(h, m);
704 return;
alshabibd7963912014-10-20 14:52:04 -0700705
706 }
707
tom7ef8ff92014-09-17 13:08:06 -0700708 if (m.getType() == OFType.ECHO_REQUEST) {
709 processOFEchoRequest(h, (OFEchoRequest) m);
Praseed Balakrishnana22eadf2014-10-20 14:21:45 -0700710 } else if (m.getType() == OFType.ECHO_REPLY) {
711 processOFEchoReply(h, (OFEchoReply) m);
tom7ef8ff92014-09-17 13:08:06 -0700712 } else if (m.getType() == OFType.ROLE_REPLY) {
713 h.sw.handleRole(m);
714 } else if (m.getType() == OFType.ERROR) {
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800715 if (!h.sw.handleRoleError((OFErrorMsg) m)) {
tom7ef8ff92014-09-17 13:08:06 -0700716 h.sw.processDriverHandshakeMessage(m);
717 if (h.sw.isDriverHandshakeComplete()) {
alshabibd7963912014-10-20 14:52:04 -0700718 moveToActive(h);
tom7ef8ff92014-09-17 13:08:06 -0700719 }
720 }
721 } else {
722 if (m.getType() == OFType.EXPERIMENTER &&
723 ((OFExperimenter) m).getExperimenter() ==
724 RoleManager.NICIRA_EXPERIMENTER) {
725 h.sw.handleNiciraRole(m);
726 } else {
727 h.sw.processDriverHandshakeMessage(m);
728 if (h.sw.isDriverHandshakeComplete()) {
alshabibd7963912014-10-20 14:52:04 -0700729 moveToActive(h);
tom7ef8ff92014-09-17 13:08:06 -0700730 }
731 }
732 }
733 }
734
735 @Override
736 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
737 throws IOException, SwitchStateException {
738 h.pendingPortStatusMsg.add(m);
739 }
alshabibd7963912014-10-20 14:52:04 -0700740
741 private void moveToActive(OFChannelHandler h) {
742 boolean success = h.sw.connectSwitch();
pieref3d74c2019-07-16 15:52:50 +0200743 // Disconnect from the device and return
alshabibd7963912014-10-20 14:52:04 -0700744 if (!success) {
745 disconnectDuplicate(h);
pieref3d74c2019-07-16 15:52:50 +0200746 return;
alshabibd7963912014-10-20 14:52:04 -0700747 }
pieref3d74c2019-07-16 15:52:50 +0200748 handlePendingPortStatusMessages(h);
749 h.setState(ACTIVE);
alshabibd7963912014-10-20 14:52:04 -0700750 }
751
tom7ef8ff92014-09-17 13:08:06 -0700752 },
753
Jordi Ortiz91477b82016-11-29 15:22:50 +0100754 /**
755 * We are expecting a OF Multi Part Meter Features Stats Reply.
756 * Notice that this information is only available for switches running
757 * OpenFlow 1.3
758 */
759 WAIT_METER_FEATURES_REPLY(true) {
Yuta HIGUCHI10f45132017-03-01 17:09:32 -0800760
761 @Override
762 void processOFEchoRequest(OFChannelHandler h, OFEchoRequest m)
763 throws IOException {
764 super.processOFEchoRequest(h, m);
765 if (System.currentTimeMillis() - h.lastStateChange > METER_TIMEOUT) {
766 log.info("{} did not respond to MeterFeaturesRequest on time, " +
767 "moving on without it.",
768 h.getSwitchInfoString());
769 h.sendHandshakeDescriptionStatsRequest();
770 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
771 }
772 }
773
Jordi Ortiz91477b82016-11-29 15:22:50 +0100774 @Override
775 void processOFError(OFChannelHandler h, OFErrorMsg m)
776 throws IOException {
Charles Chan34155e52016-11-30 18:28:11 -0800777 // Hardware switches may reply OFErrorMsg if meter is not supported
778 log.warn("Received OFError {}. It seems {} does not support Meter.",
779 m.getErrType().name(), Dpid.uri(h.thisdpid));
780 log.debug("{}", m);
781 h.sendHandshakeDescriptionStatsRequest();
782 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
Jordi Ortiz91477b82016-11-29 15:22:50 +0100783 }
784
785 @Override
786 void processOFStatisticsReply(OFChannelHandler h,
787 OFStatsReply m)
788 throws IOException, SwitchStateException {
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800789 switch (m.getStatsType()) {
Jordi Ortiz91477b82016-11-29 15:22:50 +0100790 case METER_FEATURES:
791
792 log.debug("Received Meter Features");
Yuta HIGUCHI605758e2017-01-13 20:26:44 -0800793 OFMeterFeaturesStatsReply ofmfsr = (OFMeterFeaturesStatsReply) m;
Jordi Ortiz91477b82016-11-29 15:22:50 +0100794 log.info("Received meter features from {} with max meters: {}",
795 h.getSwitchInfoString(),
796 ofmfsr.getFeatures().getMaxMeter());
797 h.meterFeaturesReply = ofmfsr;
798 h.sendHandshakeDescriptionStatsRequest();
799 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
800 break;
801 default:
802 log.error("Unexpected OF Multi Part stats reply");
803 illegalMessageReceived(h, m);
804 break;
805 }
806 }
807
808 @Override
809 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
810 throws IOException, SwitchStateException {
811 illegalMessageReceived(h, m);
812 }
813
814 @Override
815 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
816 throws IOException {
817 h.pendingPortStatusMsg.add(m);
818 }
Yuta HIGUCHI1745e5a2017-01-15 21:43:02 -0800819
820 @Override
821 void processIdle(OFChannelHandler h) throws IOException {
822 log.info("{} did not respond to MeterFeaturesRequest, " +
823 "moving on without it.",
824 h.getSwitchInfoString());
825 h.sendHandshakeDescriptionStatsRequest();
826 h.setState(WAIT_DESCRIPTION_STAT_REPLY);
827 }
Jordi Ortiz91477b82016-11-29 15:22:50 +0100828 },
829
tom7ef8ff92014-09-17 13:08:06 -0700830
831 /**
832 * This controller is in MASTER role for this switch. We enter this state
833 * after requesting and winning control from the global registry.
834 * The main handshake as well as the switch-driver sub-handshake
835 * is complete at this point.
836 * // XXX S reconsider below
837 * In the (near) future we may deterministically assign controllers to
838 * switches at startup.
839 * We only leave this state if the switch disconnects or
840 * if we send a role request for SLAVE /and/ receive the role reply for
841 * SLAVE.
842 */
843 ACTIVE(true) {
844 @Override
845 void processOFError(OFChannelHandler h, OFErrorMsg m)
846 throws IOException, SwitchStateException {
847 // if we get here, then the error message is for something else
848 if (m.getErrType() == OFErrorType.BAD_REQUEST &&
Ray Milkey30d19652016-09-06 12:09:46 -0700849 (((OFBadRequestErrorMsg) m).getCode() ==
Charles Chan9d7465e2016-09-09 19:02:34 -0700850 OFBadRequestCode.EPERM ||
tom7ef8ff92014-09-17 13:08:06 -0700851 ((OFBadRequestErrorMsg) m).getCode() ==
Charles Chan9d7465e2016-09-09 19:02:34 -0700852 OFBadRequestCode.IS_SLAVE)) {
tom7ef8ff92014-09-17 13:08:06 -0700853 // We are the master controller and the switch returned
854 // a permission error. This is a likely indicator that
855 // the switch thinks we are slave. Reassert our
856 // role
857 // FIXME: this could be really bad during role transitions
858 // if two controllers are master (even if its only for
859 // a brief period). We might need to see if these errors
860 // persist before we reassert
alshabib339a3d92014-09-26 17:54:32 -0700861
tom7ef8ff92014-09-17 13:08:06 -0700862 h.sw.reassertRole();
863 } else if (m.getErrType() == OFErrorType.FLOW_MOD_FAILED &&
864 ((OFFlowModFailedErrorMsg) m).getCode() ==
865 OFFlowModFailedCode.ALL_TABLES_FULL) {
866 h.sw.setTableFull(true);
867 } else {
868 logError(h, m);
869 }
870 h.dispatchMessage(m);
871 }
872
873 @Override
874 void processOFStatisticsReply(OFChannelHandler h,
875 OFStatsReply m) {
Ayaka Koshibe38594c22014-10-22 13:36:12 -0700876 if (m.getStatsType().equals(OFStatsType.PORT_DESC)) {
Andrea Campanellaf9a5ecb2020-05-22 12:42:12 +0200877 if (log.isDebugEnabled()) {
878 log.debug("Received port desc message from {}: {}",
879 h.sw.getDpid(),
880 ((OFPortDescStatsReply) m).getEntries());
881 }
882 if (m.getFlags().contains(OFStatsReplyFlags.REPLY_MORE)) {
883 log.debug("Active Stats reply indicates more stats from sw {} for "
884 + "port description",
885 h.getSwitchInfoString());
886 h.portDescReplies.add((OFPortDescStatsReply) m);
887 h.dispatchMessage(m);
888 return;
889 }
890
891 h.portDescReplies.add((OFPortDescStatsReply) m);
892 if (log.isDebugEnabled()) {
893 log.debug("Adding all Port Desc Active Replies to {}: {}",
894 h.sw.getDpid(),
895 h.portDescReplies);
896 }
897 h.sw.setPortDescReplies(h.portDescReplies);
898 //clearing to wait for next full response
899 h.portDescReplies.clear();
Ayaka Koshibe38594c22014-10-22 13:36:12 -0700900 }
tom7ef8ff92014-09-17 13:08:06 -0700901 h.dispatchMessage(m);
902 }
903
904 @Override
905 void processOFExperimenter(OFChannelHandler h, OFExperimenter m)
906 throws SwitchStateException {
907 h.sw.handleNiciraRole(m);
908 }
909
910 @Override
911 void processOFRoleReply(OFChannelHandler h, OFRoleReply m)
912 throws SwitchStateException {
913 h.sw.handleRole(m);
914 }
915
916 @Override
917 void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
918 throws SwitchStateException {
Andrea Campanelladad65ac2020-11-06 21:41:01 +0100919 // Handing over processing of port status messages to a thread to avoid
920 // getting blocked on the main thread and resulting other OF
921 // message being delayed.
922 // Ordering of the port status messages is guaranteed by portStatsExecutor
923 // being a single threaded executor.
924 // This executor will execute concurrently to the netty thread;
925 // meaning that the order is no more guaranteed like it was in the
926 // past between port status handling and the other events handled
927 // inline to the netty thread.
928 // This also remove guarantees of ordered processing of ROLE_CHANGED
929 // during active state, this should have no effect given that mastership
930 // is ignored here: https://github.com/opennetworkinglab/onos/blob/master/
931 // protocols/openflow/api/src/main/java/org/onosproject/openflow/controller/
932 // driver/AbstractOpenFlowSwitch.java#L279
933 h.portStatusExecutor.submit(() -> {
934 try {
935 handlePortStatusMessage(h, m, true);
936 } catch (SwitchStateException e) {
937 log.error("SwitchStateException while processing " +
938 "port status message {}", m, e);
939 }
940 });
Thomas Vachuska39274462014-12-02 13:23:50 -0800941 //h.dispatchMessage(m);
tom7ef8ff92014-09-17 13:08:06 -0700942 }
943
944 @Override
945 void processOFPacketIn(OFChannelHandler h, OFPacketIn m) {
alshabib9eab22f2014-10-20 17:17:31 -0700946// OFPacketOut out =
947// h.sw.factory().buildPacketOut()
948// .setXid(m.getXid())
949// .setBufferId(m.getBufferId()).build();
950// h.sw.sendMsg(out);
tom7ef8ff92014-09-17 13:08:06 -0700951 h.dispatchMessage(m);
952 }
953
954 @Override
955 void processOFFlowRemoved(OFChannelHandler h,
956 OFFlowRemoved m) {
957 h.dispatchMessage(m);
958 }
959
960 @Override
961 void processOFBarrierReply(OFChannelHandler h, OFBarrierReply m) {
962 h.dispatchMessage(m);
963 }
964
Ayaka Koshibee8708e32014-10-22 13:40:18 -0700965 @Override
966 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m) {
Ayaka Koshibe38594c22014-10-22 13:36:12 -0700967 h.sw.setFeaturesReply(m);
Ayaka Koshibee8708e32014-10-22 13:40:18 -0700968 h.dispatchMessage(m);
969 }
970
Yuta HIGUCHI1745e5a2017-01-15 21:43:02 -0800971 @Override
972 void processIdle(OFChannelHandler h) throws IOException {
973 log.info("{} idle", h.getSwitchInfoString());
974 }
975
tom7ef8ff92014-09-17 13:08:06 -0700976 };
977
978 private final boolean handshakeComplete;
979 ChannelState(boolean handshakeComplete) {
980 this.handshakeComplete = handshakeComplete;
981 }
982
983 /**
984 * Is this a state in which the handshake has completed?
985 * @return true if the handshake is complete
986 */
987 public boolean isHandshakeComplete() {
988 return handshakeComplete;
989 }
990
991 /**
992 * Get a string specifying the switch connection, state, and
993 * message received. To be used as message for SwitchStateException
994 * or log messages
995 * @param h The channel handler (to get switch information_
996 * @param m The OFMessage that has just been received
997 * @param details A string giving more details about the exact nature
998 * of the problem.
999 * @return display string
1000 */
1001 // needs to be protected because enum members are actually subclasses
1002 protected String getSwitchStateMessage(OFChannelHandler h,
1003 OFMessage m,
1004 String details) {
1005 return String.format("Switch: [%s], State: [%s], received: [%s]"
1006 + ", details: %s",
1007 h.getSwitchInfoString(),
1008 this.toString(),
1009 m.getType().toString(),
1010 details);
1011 }
1012
1013 /**
1014 * We have an OFMessage we didn't expect given the current state and
1015 * we want to treat this as an error.
1016 * We currently throw an exception that will terminate the connection
1017 * However, we could be more forgiving
1018 * @param h the channel handler that received the message
1019 * @param m the message
Jonathan Hart147b2ac2014-10-23 10:03:52 -07001020 * @throws SwitchStateException we always throw the exception
tom7ef8ff92014-09-17 13:08:06 -07001021 */
Jonathan Hart147b2ac2014-10-23 10:03:52 -07001022 // needs to be protected because enum members are actually subclasses
tom7ef8ff92014-09-17 13:08:06 -07001023 protected void illegalMessageReceived(OFChannelHandler h, OFMessage m)
1024 throws SwitchStateException {
1025 String msg = getSwitchStateMessage(h, m,
1026 "Switch should never send this message in the current state");
1027 throw new SwitchStateException(msg);
1028
1029 }
1030
1031 /**
1032 * We have an OFMessage we didn't expect given the current state and
1033 * we want to ignore the message.
1034 * @param h the channel handler the received the message
1035 * @param m the message
1036 */
1037 protected void unhandledMessageReceived(OFChannelHandler h,
1038 OFMessage m) {
1039 if (log.isDebugEnabled()) {
1040 String msg = getSwitchStateMessage(h, m,
1041 "Ignoring unexpected message");
1042 log.debug(msg);
1043 }
1044 }
1045
1046 /**
1047 * Log an OpenFlow error message from a switch.
1048 * @param h The switch that sent the error
1049 * @param error The error message
1050 */
1051 protected void logError(OFChannelHandler h, OFErrorMsg error) {
alshabib09d48be2014-10-03 15:43:33 -07001052 log.error("{} from switch {} in state {}",
tom7ef8ff92014-09-17 13:08:06 -07001053 error,
1054 h.getSwitchInfoString(),
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001055 this);
tom7ef8ff92014-09-17 13:08:06 -07001056 }
1057
1058 /**
1059 * Log an OpenFlow error message from a switch and disconnect the
1060 * channel.
1061 *
1062 * @param h the IO channel for this switch.
1063 * @param error The error message
1064 */
1065 protected void logErrorDisconnect(OFChannelHandler h, OFErrorMsg error) {
1066 logError(h, error);
HIGUCHI Yutadc5cf8a2016-04-29 15:17:06 -07001067 log.error("Disconnecting switch {}", h.getSwitchInfoString());
tom7ef8ff92014-09-17 13:08:06 -07001068 h.channel.disconnect();
1069 }
1070
1071 /**
1072 * log an error message for a duplicate dpid and disconnect this channel.
1073 * @param h the IO channel for this switch.
1074 */
1075 protected void disconnectDuplicate(OFChannelHandler h) {
1076 log.error("Duplicated dpid or incompleted cleanup - "
1077 + "disconnecting channel {}", h.getSwitchInfoString());
1078 h.duplicateDpidFound = Boolean.TRUE;
1079 h.channel.disconnect();
1080 }
1081
1082
1083
1084 /**
1085 * Handles all pending port status messages before a switch is declared
1086 * activated in MASTER or EQUAL role. Note that since this handling
1087 * precedes the activation (and therefore notification to IOFSwitchListerners)
1088 * the changes to ports will already be visible once the switch is
1089 * activated. As a result, no notifications are sent out for these
1090 * pending portStatus messages.
Thomas Vachuska4b420772014-10-30 16:46:17 -07001091 *
1092 * @param h the channel handler that received the message
tom7ef8ff92014-09-17 13:08:06 -07001093 */
1094 protected void handlePendingPortStatusMessages(OFChannelHandler h) {
1095 try {
1096 handlePendingPortStatusMessages(h, 0);
1097 } catch (SwitchStateException e) {
1098 log.error(e.getMessage());
1099 }
1100 }
1101
1102 private void handlePendingPortStatusMessages(OFChannelHandler h, int index)
1103 throws SwitchStateException {
1104 if (h.sw == null) {
1105 String msg = "State machine error: switch is null. Should never " +
1106 "happen";
1107 throw new SwitchStateException(msg);
1108 }
Thomas Vachuska39274462014-12-02 13:23:50 -08001109 log.info("Processing {} pending port status messages for {}",
1110 h.pendingPortStatusMsg.size(), h.sw.getStringId());
1111
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001112 ArrayList<OFPortStatus> temp = new ArrayList<>();
tom7ef8ff92014-09-17 13:08:06 -07001113 for (OFPortStatus ps: h.pendingPortStatusMsg) {
1114 temp.add(ps);
1115 handlePortStatusMessage(h, ps, false);
1116 }
tom7ef8ff92014-09-17 13:08:06 -07001117 // expensive but ok - we don't expect too many port-status messages
1118 // note that we cannot use clear(), because of the reasons below
1119 h.pendingPortStatusMsg.removeAll(temp);
Thomas Vachuska39274462014-12-02 13:23:50 -08001120 temp.clear();
tom7ef8ff92014-09-17 13:08:06 -07001121 // the iterator above takes a snapshot of the list - so while we were
1122 // dealing with the pending port-status messages, we could have received
1123 // newer ones. Handle them recursively, but break the recursion after
1124 // five steps to avoid an attack.
1125 if (!h.pendingPortStatusMsg.isEmpty() && ++index < 5) {
1126 handlePendingPortStatusMessages(h, index);
1127 }
1128 }
1129
1130 /**
1131 * Handle a port status message.
1132 *
1133 * Handle a port status message by updating the port maps in the
1134 * IOFSwitch instance and notifying Controller about the change so
1135 * it can dispatch a switch update.
1136 *
1137 * @param h The OFChannelHhandler that received the message
1138 * @param m The PortStatus message we received
1139 * @param doNotify if true switch port changed events will be
1140 * dispatched
Thomas Vachuskab14c77a2014-11-04 18:08:01 -08001141 * @throws SwitchStateException if the switch is not bound to the channel
tom7ef8ff92014-09-17 13:08:06 -07001142 *
1143 */
1144 protected void handlePortStatusMessage(OFChannelHandler h, OFPortStatus m,
1145 boolean doNotify) throws SwitchStateException {
1146 if (h.sw == null) {
1147 String msg = getSwitchStateMessage(h, m,
1148 "State machine error: switch is null. Should never " +
1149 "happen");
1150 throw new SwitchStateException(msg);
1151 }
Saurav Dasbd071d82018-01-09 17:38:44 -08001152 log.info("Received port status message from {}/{}: {}",
1153 h.sw.getDpid(), m.getDesc().getPortNo(), m);
tom7ef8ff92014-09-17 13:08:06 -07001154
1155 h.sw.handleMessage(m);
1156 }
1157
1158
1159 /**
1160 * Process an OF message received on the channel and
1161 * update state accordingly.
1162 *
1163 * The main "event" of the state machine. Process the received message,
1164 * send follow up message if required and update state if required.
1165 *
1166 * Switches on the message type and calls more specific event handlers
1167 * for each individual OF message type. If we receive a message that
1168 * is supposed to be sent from a controller to a switch we throw
1169 * a SwitchStateExeption.
1170 *
1171 * The more specific handlers can also throw SwitchStateExceptions
1172 *
1173 * @param h The OFChannelHandler that received the message
1174 * @param m The message we received.
Thomas Vachuskab14c77a2014-11-04 18:08:01 -08001175 * @throws SwitchStateException if the switch is not bound to the channel
1176 * @throws IOException if unable to send message back to the switch
tom7ef8ff92014-09-17 13:08:06 -07001177 */
1178 void processOFMessage(OFChannelHandler h, OFMessage m)
1179 throws IOException, SwitchStateException {
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001180 switch (m.getType()) {
tom7ef8ff92014-09-17 13:08:06 -07001181 case HELLO:
1182 processOFHello(h, (OFHello) m);
1183 break;
1184 case BARRIER_REPLY:
1185 processOFBarrierReply(h, (OFBarrierReply) m);
1186 break;
1187 case ECHO_REPLY:
1188 processOFEchoReply(h, (OFEchoReply) m);
1189 break;
1190 case ECHO_REQUEST:
1191 processOFEchoRequest(h, (OFEchoRequest) m);
1192 break;
1193 case ERROR:
1194 processOFError(h, (OFErrorMsg) m);
1195 break;
1196 case FEATURES_REPLY:
1197 processOFFeaturesReply(h, (OFFeaturesReply) m);
1198 break;
1199 case FLOW_REMOVED:
1200 processOFFlowRemoved(h, (OFFlowRemoved) m);
1201 break;
1202 case GET_CONFIG_REPLY:
1203 processOFGetConfigReply(h, (OFGetConfigReply) m);
1204 break;
1205 case PACKET_IN:
1206 processOFPacketIn(h, (OFPacketIn) m);
1207 break;
1208 case PORT_STATUS:
1209 processOFPortStatus(h, (OFPortStatus) m);
1210 break;
1211 case QUEUE_GET_CONFIG_REPLY:
1212 processOFQueueGetConfigReply(h, (OFQueueGetConfigReply) m);
1213 break;
1214 case STATS_REPLY: // multipart_reply in 1.3
1215 processOFStatisticsReply(h, (OFStatsReply) m);
1216 break;
1217 case EXPERIMENTER:
1218 processOFExperimenter(h, (OFExperimenter) m);
1219 break;
1220 case ROLE_REPLY:
1221 processOFRoleReply(h, (OFRoleReply) m);
1222 break;
1223 case GET_ASYNC_REPLY:
1224 processOFGetAsyncReply(h, (OFAsyncGetReply) m);
1225 break;
1226
1227 // The following messages are sent to switches. The controller
1228 // should never receive them
1229 case SET_CONFIG:
1230 case GET_CONFIG_REQUEST:
1231 case PACKET_OUT:
1232 case PORT_MOD:
1233 case QUEUE_GET_CONFIG_REQUEST:
1234 case BARRIER_REQUEST:
1235 case STATS_REQUEST: // multipart request in 1.3
1236 case FEATURES_REQUEST:
1237 case FLOW_MOD:
1238 case GROUP_MOD:
1239 case TABLE_MOD:
1240 case GET_ASYNC_REQUEST:
1241 case SET_ASYNC:
1242 case METER_MOD:
1243 default:
1244 illegalMessageReceived(h, m);
1245 break;
1246 }
1247 }
1248
1249 /*-----------------------------------------------------------------
1250 * Default implementation for message handlers in any state.
1251 *
1252 * Individual states must override these if they want a behavior
1253 * that differs from the default.
1254 *
1255 * In general, these handlers simply ignore the message and do
1256 * nothing.
1257 *
1258 * There are some exceptions though, since some messages really
1259 * are handled the same way in every state (e.g., ECHO_REQUST) or
1260 * that are only valid in a single state (e.g., HELLO, GET_CONFIG_REPLY
1261 -----------------------------------------------------------------*/
1262
1263 void processOFHello(OFChannelHandler h, OFHello m)
1264 throws IOException, SwitchStateException {
1265 // we only expect hello in the WAIT_HELLO state
alshabib45fd88a2015-09-24 17:34:35 -07001266 log.warn("Received Hello outside WAIT_HELLO state; switch {} is not complaint.",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001267 h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001268 }
1269
1270 void processOFBarrierReply(OFChannelHandler h, OFBarrierReply m)
1271 throws IOException {
1272 // Silently ignore.
1273 }
1274
1275 void processOFEchoRequest(OFChannelHandler h, OFEchoRequest m)
1276 throws IOException {
1277 if (h.ofVersion == null) {
1278 log.error("No OF version set for {}. Not sending Echo REPLY",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001279 h.channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001280 return;
1281 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001282 OFEchoReply reply = h.factory
1283 .buildEchoReply()
1284 .setXid(m.getXid())
1285 .setData(m.getData())
1286 .build();
1287 h.channel.writeAndFlush(Collections.singletonList(reply));
tom7ef8ff92014-09-17 13:08:06 -07001288 }
1289
1290 void processOFEchoReply(OFChannelHandler h, OFEchoReply m)
1291 throws IOException {
1292 // Do nothing with EchoReplies !!
1293 }
1294
1295 // no default implementation for OFError
1296 // every state must override it
1297 abstract void processOFError(OFChannelHandler h, OFErrorMsg m)
1298 throws IOException, SwitchStateException;
1299
1300
1301 void processOFFeaturesReply(OFChannelHandler h, OFFeaturesReply m)
1302 throws IOException, SwitchStateException {
1303 unhandledMessageReceived(h, m);
1304 }
1305
1306 void processOFFlowRemoved(OFChannelHandler h, OFFlowRemoved m)
1307 throws IOException {
1308 unhandledMessageReceived(h, m);
1309 }
1310
1311 void processOFGetConfigReply(OFChannelHandler h, OFGetConfigReply m)
1312 throws IOException, SwitchStateException {
1313 // we only expect config replies in the WAIT_CONFIG_REPLY state
1314 illegalMessageReceived(h, m);
1315 }
1316
1317 void processOFPacketIn(OFChannelHandler h, OFPacketIn m)
1318 throws IOException {
1319 unhandledMessageReceived(h, m);
1320 }
1321
1322 // no default implementation. Every state needs to handle it.
1323 abstract void processOFPortStatus(OFChannelHandler h, OFPortStatus m)
1324 throws IOException, SwitchStateException;
1325
1326 void processOFQueueGetConfigReply(OFChannelHandler h,
1327 OFQueueGetConfigReply m)
1328 throws IOException {
1329 unhandledMessageReceived(h, m);
1330 }
1331
1332 void processOFStatisticsReply(OFChannelHandler h, OFStatsReply m)
1333 throws IOException, SwitchStateException {
1334 unhandledMessageReceived(h, m);
1335 }
1336
1337 void processOFExperimenter(OFChannelHandler h, OFExperimenter m)
1338 throws IOException, SwitchStateException {
1339 // TODO: it might make sense to parse the vendor message here
1340 // into the known vendor messages we support and then call more
1341 // specific event handlers
1342 unhandledMessageReceived(h, m);
1343 }
1344
1345 void processOFRoleReply(OFChannelHandler h, OFRoleReply m)
1346 throws SwitchStateException, IOException {
1347 unhandledMessageReceived(h, m);
1348 }
1349
1350 void processOFGetAsyncReply(OFChannelHandler h,
1351 OFAsyncGetReply m) {
1352 unhandledMessageReceived(h, m);
1353 }
1354
Yuta HIGUCHI1745e5a2017-01-15 21:43:02 -08001355 void processIdle(OFChannelHandler h) throws IOException {
1356 // disconnect channel which did no complete handshake
1357 log.error("{} idle in state {}, disconnecting", h.getSwitchInfoString(), this);
1358 h.channel.disconnect();
1359 }
tom7ef8ff92014-09-17 13:08:06 -07001360 }
1361
1362
1363
1364 //*************************
1365 // Channel handler methods
1366 //*************************
1367
1368 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001369 public void channelActive(ChannelHandlerContext ctx)
1370 throws Exception {
1371
1372 channel = ctx.channel();
tom7ef8ff92014-09-17 13:08:06 -07001373 log.info("New switch connection from {}",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001374 channel.remoteAddress());
1375
1376 SocketAddress address = channel.remoteAddress();
1377 if (address instanceof InetSocketAddress) {
1378 final InetSocketAddress inetAddress = (InetSocketAddress) address;
1379 final IpAddress ipAddress = IpAddress.valueOf(inetAddress.getAddress());
1380 if (ipAddress.isIp4()) {
1381 channelId = ipAddress.toString() + ':' + inetAddress.getPort();
1382 } else {
1383 channelId = '[' + ipAddress.toString() + "]:" + inetAddress.getPort();
1384 }
1385 } else {
1386 channelId = channel.toString();
1387 }
1388
1389 dispatcher = Executors.newSingleThreadExecutor(groupedThreads("onos/of/dispatcher", channelId, log));
1390
alshabib70fc7fb2015-01-06 11:04:29 -08001391 /*
1392 hack to wait for the switch to tell us what it's
1393 max version is. This is not spec compliant and should
1394 be removed as soon as switches behave better.
1395 */
1396 //sendHandshakeHelloMessage();
tom7ef8ff92014-09-17 13:08:06 -07001397 setState(ChannelState.WAIT_HELLO);
1398 }
1399
1400 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001401 public void channelInactive(ChannelHandlerContext ctx)
1402 throws Exception {
1403
tom7ef8ff92014-09-17 13:08:06 -07001404 log.info("Switch disconnected callback for sw:{}. Cleaning up ...",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001405 getSwitchInfoString());
1406
1407 if (dispatcher != null) {
Harold Huang828cd592017-11-04 10:46:04 +08001408 dispatcher.shutdownNow();
Thomas Vachuskad75684a2018-01-03 09:04:47 -08001409 dispatcher = null;
tom7ef8ff92014-09-17 13:08:06 -07001410 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001411
1412 if (thisdpid != 0) {
1413 if (!duplicateDpidFound) {
1414 // if the disconnected switch (on this ChannelHandler)
1415 // was not one with a duplicate-dpid, it is safe to remove all
1416 // state for it at the controller. Notice that if the disconnected
1417 // switch was a duplicate-dpid, calling the method below would clear
1418 // all state for the original switch (with the same dpid),
1419 // which we obviously don't want.
1420 log.info("{}:removal called", getSwitchInfoString());
1421 if (sw != null) {
1422 sw.removeConnectedSwitch();
1423 }
1424 } else {
1425 // A duplicate was disconnected on this ChannelHandler,
1426 // this is the same switch reconnecting, but the original state was
1427 // not cleaned up - XXX check liveness of original ChannelHandler
1428 log.info("{}:duplicate found", getSwitchInfoString());
1429 duplicateDpidFound = Boolean.FALSE;
1430 }
1431 } else {
1432 log.warn("no dpid in channelHandler registered for "
1433 + "disconnected switch {}", getSwitchInfoString());
1434 }
tom7ef8ff92014-09-17 13:08:06 -07001435 }
1436
1437 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001438 public void exceptionCaught(ChannelHandlerContext ctx,
1439 Throwable cause)
tom7ef8ff92014-09-17 13:08:06 -07001440 throws Exception {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001441
1442 if (cause instanceof ReadTimeoutException) {
tom7ef8ff92014-09-17 13:08:06 -07001443 // switch timeout
1444 log.error("Disconnecting switch {} due to read timeout",
1445 getSwitchInfoString());
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001446 ctx.channel().close();
1447 } else if (cause instanceof HandshakeTimeoutException) {
tom7ef8ff92014-09-17 13:08:06 -07001448 log.error("Disconnecting switch {}: failed to complete handshake",
1449 getSwitchInfoString());
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001450 ctx.channel().close();
1451 } else if (cause instanceof ClosedChannelException) {
tom7ef8ff92014-09-17 13:08:06 -07001452 log.debug("Channel for sw {} already closed", getSwitchInfoString());
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001453 } else if (cause instanceof IOException) {
1454 if (!cause.getMessage().equals(RESET_BY_PEER) &&
1455 !cause.getMessage().equals(BROKEN_PIPE)) {
Thomas Vachuskae9af1f42015-07-06 08:42:18 -07001456 log.error("Disconnecting switch {} due to IO Error: {}",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001457 getSwitchInfoString(), cause.getMessage());
Thomas Vachuskae9af1f42015-07-06 08:42:18 -07001458 if (log.isDebugEnabled()) {
1459 // still print stack trace if debug is enabled
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001460 log.debug("StackTrace for previous Exception: ", cause);
Thomas Vachuskae9af1f42015-07-06 08:42:18 -07001461 }
tom7ef8ff92014-09-17 13:08:06 -07001462 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001463 ctx.channel().close();
1464 } else if (cause instanceof SwitchStateException) {
tom7ef8ff92014-09-17 13:08:06 -07001465 log.error("Disconnecting switch {} due to switch state error: {}",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001466 getSwitchInfoString(), cause.getMessage());
tom7ef8ff92014-09-17 13:08:06 -07001467 if (log.isDebugEnabled()) {
1468 // still print stack trace if debug is enabled
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001469 log.debug("StackTrace for previous Exception: ", cause);
tom7ef8ff92014-09-17 13:08:06 -07001470 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001471 ctx.channel().close();
1472 } else if (cause instanceof OFParseError) {
tom7ef8ff92014-09-17 13:08:06 -07001473 log.error("Disconnecting switch "
1474 + getSwitchInfoString() +
1475 " due to message parse failure",
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001476 cause);
1477 ctx.channel().close();
1478 } else if (cause instanceof RejectedExecutionException) {
tom7ef8ff92014-09-17 13:08:06 -07001479 log.warn("Could not process message: queue full");
1480 } else {
1481 log.error("Error while processing message from switch "
1482 + getSwitchInfoString()
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001483 + "state " + this.state, cause);
1484 ctx.channel().close();
tom7ef8ff92014-09-17 13:08:06 -07001485 }
1486 }
1487
1488 @Override
1489 public String toString() {
1490 return getSwitchInfoString();
1491 }
1492
pier063e9032019-11-22 20:51:26 +01001493 // We have reduced the idle period, the idea is to use
1494 // the IdleHandler to perform also some sanity checks.
1495 // Previous code is still executed with the same frequency
1496 // which is IDLE_INTERVAL * MAX_IDLE_RETRY of inactivity
Ray Milkey986a47a2018-01-25 11:38:51 -08001497 private void channelIdle(ChannelHandlerContext ctx,
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001498 IdleStateEvent e)
Ray Milkey986a47a2018-01-25 11:38:51 -08001499 throws IOException {
pier063e9032019-11-22 20:51:26 +01001500 // dispatcher terminated for some reason, restart
1501 if (dispatcherHandle.isDone()) {
1502 dispatcherHandle = dispatcher.submit(new Dispatcher());
Charles Chan982d3902018-03-21 14:58:53 -07001503 }
pier063e9032019-11-22 20:51:26 +01001504 // drain the backlog
1505 processDispatchBacklogQueue();
1506 // Original timeout reached
1507 if (--maxIdleRetry == 0) {
1508 maxIdleRetry = MAX_IDLE_RETRY;
1509 // Factory can be null if the channel goes idle during initial handshake. Since the switch
1510 // is not even initialized properly, we just skip this and disconnect the channel.
1511 if (factory != null) {
1512 // send an echo request each time idle_timeout * TICK
1513 OFMessage m = factory.buildEchoRequest().build();
1514 log.info("Sending Echo Request on idle channel: {}", ctx.channel());
1515 // XXX S some problems here -- echo request has no transaction id, and
1516 // echo reply is not correlated to the echo request.
1517 ctx.writeAndFlush(Collections.singletonList(m), ctx.voidPromise());
1518 }
1519 state.processIdle(this);
1520 }
tom7ef8ff92014-09-17 13:08:06 -07001521 }
1522
1523 @Override
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001524 public void userEventTriggered(ChannelHandlerContext ctx,
1525 Object evt)
tom7ef8ff92014-09-17 13:08:06 -07001526 throws Exception {
pier63295402019-10-24 16:53:25 +02001527 // If the connection is READER/WRITER idle try to send an echo request
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001528 if (evt instanceof IdleStateEvent) {
pier063e9032019-11-22 20:51:26 +01001529 log.debug("Channel {} is {}", ctx.channel(), ((IdleStateEvent) evt).state());
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001530 channelIdle(ctx, (IdleStateEvent) evt);
pier63295402019-10-24 16:53:25 +02001531 } else {
1532 super.userEventTriggered(ctx, evt);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001533 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001534 }
1535
1536 // SimpleChannelInboundHandler without dependency to TypeParameterMatcher
1537 @Override
1538 public void channelRead(ChannelHandlerContext ctx,
1539 Object msg) throws Exception {
1540
1541 boolean release = true;
pier063e9032019-11-22 20:51:26 +01001542 maxIdleRetry = MAX_IDLE_RETRY;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001543 try {
1544 if (msg instanceof OFMessage) {
1545 // channelRead0 inlined
1546 state.processOFMessage(this, (OFMessage) msg);
1547 } else {
1548 release = false;
1549 ctx.fireChannelRead(msg);
tom7ef8ff92014-09-17 13:08:06 -07001550 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001551 } finally {
1552 if (release) {
1553 ReferenceCountUtil.release(msg);
1554 }
tom7ef8ff92014-09-17 13:08:06 -07001555 }
1556 }
1557
1558
1559
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001560
tom7ef8ff92014-09-17 13:08:06 -07001561 //*************************
1562 // Channel utility methods
1563 //*************************
1564
1565 /**
1566 * Is this a state in which the handshake has completed?
Anton Chigrin4af4f872019-01-14 17:29:56 +02001567 *
tom7ef8ff92014-09-17 13:08:06 -07001568 * @return true if the handshake is complete
1569 */
1570 public boolean isHandshakeComplete() {
1571 return this.state.isHandshakeComplete();
1572 }
1573
Anton Chigrin4af4f872019-01-14 17:29:56 +02001574 /**
1575 * Increment totalCount variable and send signal to executor.
1576 */
1577 private void incrementAndSignal() {
1578 try {
1579 totalCount.incrementAndGet();
1580 takeLock.lockInterruptibly();
1581 try {
1582 notEmpty.signal();
1583 } finally {
1584 takeLock.unlock();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001585 }
Anton Chigrin4af4f872019-01-14 17:29:56 +02001586 } catch (InterruptedException e) {
1587 e.printStackTrace();
1588 }
1589 }
1590
1591 /**
1592 * Try to push OpenFlow message to queue.
1593 *
1594 * @param message OpenFlow message
1595 * @param idQueue id of Queue
1596 * @return true if message was successful added to queue
1597 */
1598 private boolean pushMessageToQueue(OFMessage message, int idQueue) {
1599 if (!dispatchQueuesMapProducer.get(idQueue).offer(message)) {
1600 return false;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001601 } else {
Anton Chigrin4af4f872019-01-14 17:29:56 +02001602 incrementAndSignal();
1603 return true;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001604 }
Anton Chigrin4af4f872019-01-14 17:29:56 +02001605 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001606
Anton Chigrin4af4f872019-01-14 17:29:56 +02001607 /**
1608 * Process backlog - move messages from backlog to default queue.
1609 *
1610 * @return true if whole backlog was processed, otherwise false
1611 */
1612 private boolean processDispatchBacklogQueue() {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001613 while (!dispatchBacklog.isEmpty()) {
Anton Chigrin4af4f872019-01-14 17:29:56 +02001614 OFMessage msgFromBacklog = dispatchBacklog.removeFirst();
1615 if (!pushMessageToQueue(msgFromBacklog, NUM_OF_QUEUES - 1)) {
1616 dispatchBacklog.addFirst(msgFromBacklog);
1617 return false;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001618 }
1619 }
Anton Chigrin4af4f872019-01-14 17:29:56 +02001620 return true;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001621
Anton Chigrin4af4f872019-01-14 17:29:56 +02001622 }
1623
1624 /**
1625 * Parse OpenFlow message context for get Ethernet packet.
1626 *
1627 * @param message OpenFlow message
1628 * @return parsed Ethernet packet
1629 */
1630 private Ethernet parsePacketInMessage(OFMessage message) {
1631 OpenFlowPacketContext pktCtx = DefaultOpenFlowPacketContext
1632 .packetContextFromPacketIn(sw, (OFPacketIn) message);
1633 DeviceId id = DeviceId.deviceId(Dpid.uri(pktCtx.dpid().value()));
1634 DefaultInboundPacket inPkt = new DefaultInboundPacket(
1635 new ConnectPoint(id, PortNumber.portNumber(pktCtx.inPort())),
1636 pktCtx.parsed(), ByteBuffer.wrap(pktCtx.unparsed()),
1637 pktCtx.cookie());
1638 return inPkt.parsed();
1639 }
1640
1641 /**
1642 * Classify the Ethernet packet for membership on one of the queues.
1643 *
1644 * @param packet ethernet packet
1645 * @return Id of destination Queue
1646 */
1647 private int classifyEthernetPacket(Ethernet packet) {
1648 for (Set<OpenFlowClassifier> classifiers : this.messageClassifiersMapProducer) {
1649 for (OpenFlowClassifier classifier : classifiers) {
1650 if (classifier.ethernetType() == packet.getEtherType()) {
1651 return classifier.idQueue();
1652 }
1653 }
1654 }
1655 return NUM_OF_QUEUES - 1;
1656 }
1657
1658 /**
1659 * Process messages from dispatch queues.
1660 *
1661 * @param queuesSize count of messages in all queues
1662 */
1663 private void processMessages(int queuesSize) {
1664 List<OFMessage> msgs = new ArrayList<>();
1665 int processed;
1666 do {
1667 processed = 0;
1668 while (processed < queuesSize) {
1669 for (LinkedBlockingMessagesQueue<OFMessage> queue :
1670 dispatchQueuesMapProducer.values()) {
1671 processed += queue.drainTo(msgs);
1672 }
1673 }
1674
1675 msgs.forEach(sw::handleMessage);
1676 msgs.clear();
1677 /* Decrement conditional variable */
1678 queuesSize = totalCount.addAndGet(-1 * processed);
1679 } while (queuesSize > 0);
1680 }
1681
1682 private void dispatchMessage(OFMessage m) {
1683 log.debug("Begin dispatch OpenFlow Message");
1684 boolean backlogEmpty = processDispatchBacklogQueue();
1685 if (m.getType() == OFType.PACKET_IN) {
1686 Ethernet pkt = parsePacketInMessage(m);
1687 pushMessageToQueue(m, classifyEthernetPacket(pkt));
1688 } else {
1689 if (!backlogEmpty || !pushMessageToQueue(m, NUM_OF_QUEUES - 1)) {
1690 dispatchBacklog.offer(m);
1691 }
1692 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001693
1694 if (dispatcherHandle.isDone()) {
1695 // dispatcher terminated for some reason, restart
pier063e9032019-11-22 20:51:26 +01001696 dispatcherHandle = dispatcher.submit(new Dispatcher());
1697 }
1698 }
1699
1700 private final class Dispatcher implements Runnable {
1701 // dispatch loop
1702 @Override
1703 public void run() {
1704 try {
1705 for (;;) {
1706 int tc = 0;
1707 takeLock.lockInterruptibly();
1708 try {
1709 while ((tc = totalCount.get()) == 0) {
1710 notEmpty.await();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001711 }
pier063e9032019-11-22 20:51:26 +01001712 } finally {
1713 takeLock.unlock();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001714 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001715
pier063e9032019-11-22 20:51:26 +01001716 processMessages(tc);
1717 }
1718 } catch (InterruptedException e) {
1719 log.warn("Dispatcher interrupted");
1720 Thread.currentThread().interrupt();
1721 // interrupted. gracefully shutting down
1722 return;
1723 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001724 }
tom7ef8ff92014-09-17 13:08:06 -07001725 }
1726
1727 /**
1728 * Return a string describing this switch based on the already available
1729 * information (DPID and/or remote socket).
1730 * @return display string
1731 */
1732 private String getSwitchInfoString() {
1733 if (sw != null) {
1734 return sw.toString();
1735 }
1736 String channelString;
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001737 if (channel == null || channel.remoteAddress() == null) {
tom7ef8ff92014-09-17 13:08:06 -07001738 channelString = "?";
1739 } else {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001740 channelString = channel.remoteAddress().toString();
tom7ef8ff92014-09-17 13:08:06 -07001741 }
1742 String dpidString;
1743 if (featuresReply == null) {
1744 dpidString = "?";
1745 } else {
1746 dpidString = featuresReply.getDatapathId().toString();
1747 }
1748 return String.format("[%s DPID[%s]]", channelString, dpidString);
1749 }
1750
1751 /**
1752 * Update the channels state. Only called from the state machine.
1753 * TODO: enforce restricted state transitions
pieref3d74c2019-07-16 15:52:50 +02001754 * @param state new state
tom7ef8ff92014-09-17 13:08:06 -07001755 */
pieref3d74c2019-07-16 15:52:50 +02001756 void setState(ChannelState state) {
tom7ef8ff92014-09-17 13:08:06 -07001757 this.state = state;
Yuta HIGUCHI10f45132017-03-01 17:09:32 -08001758 this.lastStateChange = System.currentTimeMillis();
tom7ef8ff92014-09-17 13:08:06 -07001759 }
1760
Brian O'Connorf69e3e32018-05-10 02:25:09 -07001761 private boolean setDpid(Long dpid, Channel channel) {
1762 ChannelHandlerContext sslContext = channel.pipeline().context(SslHandler.class);
1763 if (sslContext != null) {
1764 try {
1765 SslHandler sslHandler = (SslHandler) sslContext.handler();
1766 Certificate[] certs = sslHandler.engine().getSession().getPeerCertificates();
1767 Certificate cert = certs.length > 0 ? certs[0] : null;
1768 if (!controller.isValidCertificate(dpid, cert)) {
1769 return false;
1770 }
1771 } catch (SSLPeerUnverifiedException e) {
1772 log.info("Switch with dpid {} is an unverified SSL peer.", dpid, e);
1773 return false;
1774 }
1775 }
1776 this.thisdpid = dpid;
1777 return true;
1778 }
1779
tom7ef8ff92014-09-17 13:08:06 -07001780 /**
1781 * Send hello message to the switch using the handshake transactions ids.
1782 * @throws IOException
1783 */
1784 private void sendHandshakeHelloMessage() throws IOException {
1785 // The OF protocol requires us to start things off by sending the highest
1786 // version of the protocol supported.
1787
Yuta HIGUCHI6512f3e2017-05-18 17:21:24 -07001788 // bitmap represents OF1.0, OF1.3, OF1.4, and OF1.5
tom7ef8ff92014-09-17 13:08:06 -07001789 // see Sec. 7.5.1 of the OF1.3.4 spec
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001790 U32 bitmap = U32.ofRaw((0b1 << OFVersion.OF_10.getWireVersion()) |
1791 (0b1 << OFVersion.OF_13.getWireVersion()) |
Yuta HIGUCHI6512f3e2017-05-18 17:21:24 -07001792 (0b1 << OFVersion.OF_14.getWireVersion()) |
1793 (0b1 << OFVersion.OF_15.getWireVersion()));
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001794 OFVersion version = Optional.ofNullable(ofVersion).orElse(OFVersion.OF_13);
1795 OFHelloElem hem = OFFactories.getFactory(version)
1796 .buildHelloElemVersionbitmap()
tom7ef8ff92014-09-17 13:08:06 -07001797 .setBitmaps(Collections.singletonList(bitmap))
1798 .build();
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001799 OFMessage.Builder mb = OFFactories.getFactory(version)
1800 .buildHello()
tom7ef8ff92014-09-17 13:08:06 -07001801 .setXid(this.handshakeTransactionIds--)
1802 .setElements(Collections.singletonList(hem));
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001803 log.info("Sending {} Hello to {}", version, channel.remoteAddress());
1804 channel.writeAndFlush(Collections.singletonList(mb.build()));
tom7ef8ff92014-09-17 13:08:06 -07001805 }
1806
1807 /**
1808 * Send featuresRequest msg to the switch using the handshake transactions ids.
1809 * @throws IOException
1810 */
1811 private void sendHandshakeFeaturesRequestMessage() throws IOException {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001812 log.debug("Sending FEATURES_REQUEST to {}", channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001813 OFMessage m = factory.buildFeaturesRequest()
1814 .setXid(this.handshakeTransactionIds--)
1815 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001816 channel.writeAndFlush(Collections.singletonList(m));
tom7ef8ff92014-09-17 13:08:06 -07001817 }
1818
1819 /**
1820 * Send the configuration requests to tell the switch we want full
1821 * packets.
1822 * @throws IOException
1823 */
1824 private void sendHandshakeSetConfig() throws IOException {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001825 log.debug("Sending CONFIG_REQUEST to {}", channel.remoteAddress());
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001826 List<OFMessage> msglist = new ArrayList<>(3);
tom7ef8ff92014-09-17 13:08:06 -07001827
1828 // Ensure we receive the full packet via PacketIn
1829 // FIXME: We don't set the reassembly flags.
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001830 // Only send config to switches to send full packets, if they have a buffer.
Michael Jarschel7f521a32015-08-12 16:31:07 +02001831 // Saves a packet & OFSetConfig can't be handled by certain switches.
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001832 if (this.featuresReply.getNBuffers() > 0) {
Michael Jarschel7f521a32015-08-12 16:31:07 +02001833 OFSetConfig sc = factory
1834 .buildSetConfig()
1835 .setMissSendLen((short) 0xffff)
1836 .setXid(this.handshakeTransactionIds--)
1837 .build();
1838 msglist.add(sc);
1839 }
tom7ef8ff92014-09-17 13:08:06 -07001840
1841 // Barrier
1842 OFBarrierRequest br = factory
1843 .buildBarrierRequest()
1844 .setXid(this.handshakeTransactionIds--)
1845 .build();
1846 msglist.add(br);
1847
1848 // Verify (need barrier?)
1849 OFGetConfigRequest gcr = factory
1850 .buildGetConfigRequest()
1851 .setXid(this.handshakeTransactionIds--)
1852 .build();
1853 msglist.add(gcr);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001854 channel.writeAndFlush(msglist);
tom7ef8ff92014-09-17 13:08:06 -07001855 }
1856
1857 /**
1858 * send a description state request.
1859 * @throws IOException
1860 */
1861 private void sendHandshakeDescriptionStatsRequest() throws IOException {
1862 // Get Description to set switch-specific flags
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001863 log.debug("Sending DESC_STATS_REQUEST to {}", channel.remoteAddress());
tom7ef8ff92014-09-17 13:08:06 -07001864 OFDescStatsRequest dreq = factory
1865 .buildDescStatsRequest()
1866 .setXid(handshakeTransactionIds--)
1867 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001868 channel.writeAndFlush(Collections.singletonList(dreq));
tom7ef8ff92014-09-17 13:08:06 -07001869 }
1870
Jordi Ortiz91477b82016-11-29 15:22:50 +01001871 /**
Yuta HIGUCHI605758e2017-01-13 20:26:44 -08001872 * send a meter features request.
1873 *
Jordi Ortiz91477b82016-11-29 15:22:50 +01001874 * @throws IOException
1875 */
1876 private void sendMeterFeaturesRequest() throws IOException {
1877 // Get meter features including the MaxMeters value available for the device
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001878 OFFactory factory = OFFactories.getFactory(ofVersion);
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001879 log.debug("Sending METER_FEATURES_REQUEST to {}", channel.remoteAddress());
Jordi Ortiz91477b82016-11-29 15:22:50 +01001880 OFMeterFeaturesStatsRequest mfreq = factory
1881 .buildMeterFeaturesStatsRequest()
1882 .setXid(handshakeTransactionIds--)
1883 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001884 channel.writeAndFlush(Collections.singletonList(mfreq));
Jordi Ortiz91477b82016-11-29 15:22:50 +01001885 }
1886
tom7ef8ff92014-09-17 13:08:06 -07001887 private void sendHandshakeOFPortDescRequest() throws IOException {
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001888 log.debug("Sending OF_PORT_DESC_REQUEST to {}", channel.remoteAddress());
Yuta HIGUCHI2341e602017-03-08 20:10:08 -08001889 // Get port description for 1.3+ switch
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001890 OFPortDescStatsRequest preq = factory
tom7ef8ff92014-09-17 13:08:06 -07001891 .buildPortDescStatsRequest()
1892 .setXid(handshakeTransactionIds--)
1893 .build();
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001894 channel.writeAndFlush(Collections.singletonList(preq));
tom7ef8ff92014-09-17 13:08:06 -07001895 }
1896
1897 ChannelState getStateForTesting() {
1898 return state;
1899 }
1900
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001901
1902 @Override
1903 public boolean isActive() {
1904 if (channel != null) {
1905 return channel.isActive();
1906 }
1907 return false;
1908 }
1909
1910 @Override
1911 public void closeSession() {
1912 if (channel != null) {
1913 channel.close();
1914 }
1915 }
1916
1917 @Override
1918 public boolean sendMsg(Iterable<OFMessage> msgs) {
1919 if (channel.isActive()) {
Laszlo Pappb68fe7e2017-11-24 17:06:59 +00001920 if (log.isTraceEnabled()) {
1921 log.trace("Sending messages for switch {} via openflow channel: {}", getSwitchInfoString(), msgs);
1922 }
Yuta HIGUCHI6ee6b8c2017-05-09 14:44:30 -07001923 channel.writeAndFlush(msgs, channel.voidPromise());
1924 return true;
1925 } else {
1926 log.warn("Dropping messages for switch {} because channel is not connected: {}",
1927 getSwitchInfoString(), msgs);
1928 return false;
1929 }
1930 }
1931
1932 @Override
1933 public CharSequence sessionInfo() {
1934 return channelId;
1935 }
1936
Anton Chigrin4af4f872019-01-14 17:29:56 +02001937 @Override
1938 public void addClassifier(OpenFlowClassifier classifier) {
1939 if (this.deviceId.equals(classifier.deviceId())) {
1940 log.debug("Add OpenFlow Classifier for switch {} to queue {} with type {}",
1941 classifier.deviceId().toString(), classifier.idQueue(), classifier.ethernetType());
1942 this.messageClassifiersMapProducer.get(classifier.idQueue()).add(classifier);
1943 }
1944 }
1945
1946 @Override
1947 public void removeClassifier(OpenFlowClassifier classifier) {
1948 if (this.deviceId.equals(classifier.deviceId())) {
1949 log.debug("Remove OpenFlow Classifier for switch {} from queue {} with type {}",
1950 classifier.deviceId().toString(), classifier.idQueue(), classifier.ethernetType());
1951 this.messageClassifiersMapProducer.get(classifier.idQueue()).remove(classifier);
1952 }
1953 }
1954
1955 /**
1956 * Init classifier configuration for the switch. Use stored configuration if exist.
1957 * Otherwise add LLDP and BDDP classifiers for Queue N0.
1958 */
1959 private void initClassifiers() {
1960 try {
1961 openFlowManager = DefaultServiceDirectory.getService(OpenFlowService.class);
1962 DeviceId did = DeviceId.deviceId(uri(thisdpid));
1963 Set<OpenFlowClassifier> classifiers = openFlowManager.getClassifiersByDeviceId(did);
1964 if (classifiers == null) {
1965 OpenFlowClassifier classifier =
1966 new OpenFlowClassifier.Builder(did, 0).ethernetType(TYPE_LLDP).build();
1967 openFlowManager.add(classifier);
1968 classifier = new OpenFlowClassifier.Builder(did, 0).ethernetType(TYPE_BSN).build();
1969 openFlowManager.add(classifier);
1970 } else {
1971 this.messageClassifiersMapProducer.forEach((v) -> {
1972 v.clear();
1973 });
1974 classifiers.forEach((c) -> {
1975 messageClassifiersMapProducer.get(c.idQueue()).add(c);
1976 });
1977 }
1978 } catch (Exception e) {
1979 log.error("Initialize default classifier failed: {}", e.toString());
1980 e.printStackTrace();
1981 }
1982 }
tom7ef8ff92014-09-17 13:08:06 -07001983}