blob: 8bfab64104f213422b6eadc5d7ea271137434895 [file] [log] [blame]
Madan Jampani86940d92015-05-06 11:47:57 -07001 /*
2 * Copyright 2014-2015 Open Networking Laboratory
3 *
4 * 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
7 *
8 * 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.
15 */
16package org.onosproject.store.flow.impl;
17
18import com.google.common.base.Objects;
19import com.google.common.collect.Iterables;
20import com.google.common.collect.Maps;
21import com.google.common.collect.Sets;
22import com.google.common.util.concurrent.Futures;
23
24import org.apache.felix.scr.annotations.Activate;
25import org.apache.felix.scr.annotations.Component;
26import org.apache.felix.scr.annotations.Deactivate;
27import org.apache.felix.scr.annotations.Modified;
28import org.apache.felix.scr.annotations.Property;
29import org.apache.felix.scr.annotations.Reference;
30import org.apache.felix.scr.annotations.ReferenceCardinality;
31import org.apache.felix.scr.annotations.Service;
32import org.onlab.util.KryoNamespace;
33import org.onlab.util.NewConcurrentHashMap;
34import org.onlab.util.Tools;
35import org.onosproject.cfg.ComponentConfigService;
36import org.onosproject.cluster.ClusterService;
37import org.onosproject.cluster.NodeId;
38import org.onosproject.core.CoreService;
39import org.onosproject.core.IdGenerator;
40import org.onosproject.mastership.MastershipService;
41import org.onosproject.net.DeviceId;
42import org.onosproject.net.device.DeviceService;
43import org.onosproject.net.flow.CompletedBatchOperation;
44import org.onosproject.net.flow.DefaultFlowEntry;
45import org.onosproject.net.flow.FlowEntry;
46import org.onosproject.net.flow.FlowEntry.FlowEntryState;
47import org.onosproject.net.flow.FlowId;
48import org.onosproject.net.flow.FlowRule;
49import org.onosproject.net.flow.FlowRuleBatchEntry;
50import org.onosproject.net.flow.FlowRuleBatchEntry.FlowRuleOperation;
51import org.onosproject.net.flow.FlowRuleBatchEvent;
52import org.onosproject.net.flow.FlowRuleBatchOperation;
53import org.onosproject.net.flow.FlowRuleBatchRequest;
54import org.onosproject.net.flow.FlowRuleEvent;
55import org.onosproject.net.flow.FlowRuleEvent.Type;
56import org.onosproject.net.flow.FlowRuleService;
57import org.onosproject.net.flow.FlowRuleStore;
58import org.onosproject.net.flow.FlowRuleStoreDelegate;
59import org.onosproject.net.flow.StoredFlowEntry;
60import org.onosproject.store.AbstractStore;
61import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
62import org.onosproject.store.cluster.messaging.ClusterMessage;
63import org.onosproject.store.cluster.messaging.ClusterMessageHandler;
64import org.onosproject.store.flow.ReplicaInfo;
Madan Jampanif7536ab2015-05-07 23:23:23 -070065import org.onosproject.store.flow.ReplicaInfoEvent;
66import org.onosproject.store.flow.ReplicaInfoEventListener;
Madan Jampani86940d92015-05-06 11:47:57 -070067import org.onosproject.store.flow.ReplicaInfoService;
68import org.onosproject.store.serializers.KryoSerializer;
69import org.onosproject.store.serializers.StoreSerializer;
70import org.onosproject.store.serializers.impl.DistributedStoreSerializers;
71import org.osgi.service.component.ComponentContext;
72import org.slf4j.Logger;
73
74import java.util.Arrays;
75import java.util.Collections;
76import java.util.Dictionary;
77import java.util.HashSet;
78import java.util.List;
79import java.util.Map;
80import java.util.Set;
81import java.util.concurrent.ConcurrentHashMap;
82import java.util.concurrent.ConcurrentMap;
83import java.util.concurrent.ExecutorService;
84import java.util.concurrent.Executors;
85import java.util.concurrent.ScheduledExecutorService;
Madan Jampani08bf17b2015-05-06 16:25:26 -070086import java.util.concurrent.ScheduledFuture;
Madan Jampani86940d92015-05-06 11:47:57 -070087import java.util.concurrent.TimeUnit;
88import java.util.concurrent.atomic.AtomicInteger;
89import java.util.stream.Collectors;
90
91import static org.apache.commons.lang3.concurrent.ConcurrentUtils.createIfAbsentUnchecked;
92import static com.google.common.base.Strings.isNullOrEmpty;
93import static org.onlab.util.Tools.get;
94import static org.onlab.util.Tools.groupedThreads;
95import static org.onosproject.net.flow.FlowRuleEvent.Type.RULE_REMOVED;
96import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.*;
97import static org.slf4j.LoggerFactory.getLogger;
98
99/**
100 * Manages inventory of flow rules using a distributed state management protocol.
101 */
102@Component(immediate = true, enabled = true)
103@Service
104public class NewDistributedFlowRuleStore
105 extends AbstractStore<FlowRuleBatchEvent, FlowRuleStoreDelegate>
106 implements FlowRuleStore {
107
108 private final Logger log = getLogger(getClass());
109
110 private static final int MESSAGE_HANDLER_THREAD_POOL_SIZE = 8;
111 private static final boolean DEFAULT_BACKUP_ENABLED = true;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700112 private static final int DEFAULT_BACKUP_PERIOD_MILLIS = 2000;
Madan Jampani86940d92015-05-06 11:47:57 -0700113 private static final long FLOW_RULE_STORE_TIMEOUT_MILLIS = 5000;
114
115 @Property(name = "msgHandlerPoolSize", intValue = MESSAGE_HANDLER_THREAD_POOL_SIZE,
116 label = "Number of threads in the message handler pool")
117 private int msgHandlerPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
118
119 @Property(name = "backupEnabled", boolValue = DEFAULT_BACKUP_ENABLED,
120 label = "Indicates whether backups are enabled or not")
121 private boolean backupEnabled = DEFAULT_BACKUP_ENABLED;
122
Madan Jampani08bf17b2015-05-06 16:25:26 -0700123 @Property(name = "backupPeriod", intValue = DEFAULT_BACKUP_PERIOD_MILLIS,
124 label = "Delay in ms between successive backup runs")
125 private int backupPeriod = DEFAULT_BACKUP_PERIOD_MILLIS;
126
Madan Jampani86940d92015-05-06 11:47:57 -0700127 private InternalFlowTable flowTable = new InternalFlowTable();
128
129 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
130 protected ReplicaInfoService replicaInfoManager;
131
132 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
133 protected ClusterCommunicationService clusterCommunicator;
134
135 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
136 protected ClusterService clusterService;
137
138 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
139 protected DeviceService deviceService;
140
141 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
142 protected CoreService coreService;
143
144 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
145 protected ComponentConfigService configService;
146
147 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
148 protected MastershipService mastershipService;
149
150 private Map<Long, NodeId> pendingResponses = Maps.newConcurrentMap();
151 private ExecutorService messageHandlingExecutor;
152
Madan Jampani08bf17b2015-05-06 16:25:26 -0700153 private ScheduledFuture<?> backupTask;
Madan Jampani86940d92015-05-06 11:47:57 -0700154 private final ScheduledExecutorService backupSenderExecutor =
155 Executors.newSingleThreadScheduledExecutor(groupedThreads("onos/flow", "backup-sender"));
156
157 protected static final StoreSerializer SERIALIZER = new KryoSerializer() {
158 @Override
159 protected void setupKryoPool() {
160 serializerPool = KryoNamespace.newBuilder()
161 .register(DistributedStoreSerializers.STORE_COMMON)
162 .nextId(DistributedStoreSerializers.STORE_CUSTOM_BEGIN)
163 .register(FlowRuleEvent.class)
164 .register(FlowRuleEvent.Type.class)
165 .build();
166 }
167 };
168
169 private IdGenerator idGenerator;
170 private NodeId local;
171
172 @Activate
173 public void activate(ComponentContext context) {
174 configService.registerProperties(getClass());
175
176 idGenerator = coreService.getIdGenerator(FlowRuleService.FLOW_OP_TOPIC);
177
178 local = clusterService.getLocalNode().id();
179
180 messageHandlingExecutor = Executors.newFixedThreadPool(
181 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers"));
182
183 registerMessageHandlers(messageHandlingExecutor);
184
Madan Jampani08bf17b2015-05-06 16:25:26 -0700185 if (backupEnabled) {
Madan Jampanif7536ab2015-05-07 23:23:23 -0700186 replicaInfoManager.addListener(flowTable);
Madan Jampani08bf17b2015-05-06 16:25:26 -0700187 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
188 flowTable::backup,
189 0,
190 backupPeriod,
191 TimeUnit.MILLISECONDS);
192 }
Madan Jampani86940d92015-05-06 11:47:57 -0700193
194 logConfig("Started");
195 }
196
197 @Deactivate
198 public void deactivate(ComponentContext context) {
Madan Jampanif7536ab2015-05-07 23:23:23 -0700199 if (backupEnabled) {
200 replicaInfoManager.removeListener(flowTable);
201 backupTask.cancel(true);
202 }
Madan Jampani86940d92015-05-06 11:47:57 -0700203 configService.unregisterProperties(getClass(), false);
204 unregisterMessageHandlers();
205 messageHandlingExecutor.shutdownNow();
206 backupSenderExecutor.shutdownNow();
207 log.info("Stopped");
208 }
209
210 @SuppressWarnings("rawtypes")
211 @Modified
212 public void modified(ComponentContext context) {
213 if (context == null) {
214 backupEnabled = DEFAULT_BACKUP_ENABLED;
215 logConfig("Default config");
216 return;
217 }
218
219 Dictionary properties = context.getProperties();
220 int newPoolSize;
221 boolean newBackupEnabled;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700222 int newBackupPeriod;
Madan Jampani86940d92015-05-06 11:47:57 -0700223 try {
224 String s = get(properties, "msgHandlerPoolSize");
225 newPoolSize = isNullOrEmpty(s) ? msgHandlerPoolSize : Integer.parseInt(s.trim());
226
227 s = get(properties, "backupEnabled");
228 newBackupEnabled = isNullOrEmpty(s) ? backupEnabled : Boolean.parseBoolean(s.trim());
229
Madan Jampani08bf17b2015-05-06 16:25:26 -0700230 s = get(properties, "backupPeriod");
231 newBackupPeriod = isNullOrEmpty(s) ? backupPeriod : Integer.parseInt(s.trim());
232
Madan Jampani86940d92015-05-06 11:47:57 -0700233 } catch (NumberFormatException | ClassCastException e) {
234 newPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
235 newBackupEnabled = DEFAULT_BACKUP_ENABLED;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700236 newBackupPeriod = DEFAULT_BACKUP_PERIOD_MILLIS;
Madan Jampani86940d92015-05-06 11:47:57 -0700237 }
238
Madan Jampani08bf17b2015-05-06 16:25:26 -0700239 boolean restartBackupTask = false;
Madan Jampani86940d92015-05-06 11:47:57 -0700240 if (newBackupEnabled != backupEnabled) {
241 backupEnabled = newBackupEnabled;
Madan Jampanif7536ab2015-05-07 23:23:23 -0700242 if (!backupEnabled) {
243 replicaInfoManager.removeListener(flowTable);
244 if (backupTask != null) {
245 backupTask.cancel(false);
246 backupTask = null;
247 }
248 } else {
249 replicaInfoManager.addListener(flowTable);
Madan Jampani08bf17b2015-05-06 16:25:26 -0700250 }
251 restartBackupTask = backupEnabled;
252 }
253 if (newBackupPeriod != backupPeriod) {
254 backupPeriod = newBackupPeriod;
255 restartBackupTask = backupEnabled;
256 }
257 if (restartBackupTask) {
258 if (backupTask != null) {
259 // cancel previously running task
260 backupTask.cancel(false);
261 }
262 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
263 flowTable::backup,
264 0,
265 backupPeriod,
266 TimeUnit.MILLISECONDS);
Madan Jampani86940d92015-05-06 11:47:57 -0700267 }
268 if (newPoolSize != msgHandlerPoolSize) {
269 msgHandlerPoolSize = newPoolSize;
270 ExecutorService oldMsgHandler = messageHandlingExecutor;
271 messageHandlingExecutor = Executors.newFixedThreadPool(
272 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers"));
273
274 // replace previously registered handlers.
275 registerMessageHandlers(messageHandlingExecutor);
276 oldMsgHandler.shutdown();
277 }
278 logConfig("Reconfigured");
279 }
280
281 private void registerMessageHandlers(ExecutorService executor) {
282
283 clusterCommunicator.addSubscriber(APPLY_BATCH_FLOWS, new OnStoreBatch(), executor);
284 clusterCommunicator.<FlowRuleBatchEvent>addSubscriber(
285 REMOTE_APPLY_COMPLETED, SERIALIZER::decode, this::notifyDelegate, executor);
286 clusterCommunicator.addSubscriber(
287 GET_FLOW_ENTRY, SERIALIZER::decode, flowTable::getFlowEntry, SERIALIZER::encode, executor);
288 clusterCommunicator.addSubscriber(
289 GET_DEVICE_FLOW_ENTRIES, SERIALIZER::decode, flowTable::getFlowEntries, SERIALIZER::encode, executor);
290 clusterCommunicator.addSubscriber(
291 REMOVE_FLOW_ENTRY, SERIALIZER::decode, this::removeFlowRuleInternal, SERIALIZER::encode, executor);
292 clusterCommunicator.addSubscriber(
293 REMOVE_FLOW_ENTRY, SERIALIZER::decode, this::removeFlowRuleInternal, SERIALIZER::encode, executor);
294 clusterCommunicator.addSubscriber(
295 FLOW_TABLE_BACKUP, SERIALIZER::decode, flowTable::onBackupReceipt, executor);
296 }
297
298 private void unregisterMessageHandlers() {
299 clusterCommunicator.removeSubscriber(REMOVE_FLOW_ENTRY);
300 clusterCommunicator.removeSubscriber(GET_DEVICE_FLOW_ENTRIES);
301 clusterCommunicator.removeSubscriber(GET_FLOW_ENTRY);
302 clusterCommunicator.removeSubscriber(APPLY_BATCH_FLOWS);
303 clusterCommunicator.removeSubscriber(REMOTE_APPLY_COMPLETED);
304 clusterCommunicator.removeSubscriber(FLOW_TABLE_BACKUP);
305 }
306
307 private void logConfig(String prefix) {
Madan Jampani08bf17b2015-05-06 16:25:26 -0700308 log.info("{} with msgHandlerPoolSize = {}; backupEnabled = {}, backupPeriod = {}",
309 prefix, msgHandlerPoolSize, backupEnabled, backupPeriod);
Madan Jampani86940d92015-05-06 11:47:57 -0700310 }
311
312 // This is not a efficient operation on a distributed sharded
313 // flow store. We need to revisit the need for this operation or at least
314 // make it device specific.
315 @Override
316 public int getFlowRuleCount() {
317 AtomicInteger sum = new AtomicInteger(0);
318 deviceService.getDevices().forEach(device -> sum.addAndGet(Iterables.size(getFlowEntries(device.id()))));
319 return sum.get();
320 }
321
322 @Override
323 public FlowEntry getFlowEntry(FlowRule rule) {
324
325 ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(rule.deviceId());
326 NodeId master = replicaInfo.master().orNull();
327
328 if (master == null) {
329 log.warn("Failed to getFlowEntry: No master for {}", rule.deviceId());
330 return null;
331 }
332
333 if (Objects.equal(local, master)) {
334 return flowTable.getFlowEntry(rule);
335 }
336
337 log.trace("Forwarding getFlowEntry to {}, which is the primary (master) for device {}",
338 master, rule.deviceId());
339
340 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(rule,
341 FlowStoreMessageSubjects.GET_FLOW_ENTRY,
342 SERIALIZER::encode,
343 SERIALIZER::decode,
344 master),
345 FLOW_RULE_STORE_TIMEOUT_MILLIS,
346 TimeUnit.MILLISECONDS,
347 null);
348 }
349
350 @Override
351 public Iterable<FlowEntry> getFlowEntries(DeviceId deviceId) {
352
353 ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(deviceId);
354 NodeId master = replicaInfo.master().orNull();
355
356 if (master == null) {
357 log.warn("Failed to getFlowEntries: No master for {}", deviceId);
358 return Collections.emptyList();
359 }
360
361 if (Objects.equal(local, master)) {
362 return flowTable.getFlowEntries(deviceId);
363 }
364
365 log.trace("Forwarding getFlowEntries to {}, which is the primary (master) for device {}",
366 master, deviceId);
367
368 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(deviceId,
369 FlowStoreMessageSubjects.GET_DEVICE_FLOW_ENTRIES,
370 SERIALIZER::encode,
371 SERIALIZER::decode,
372 master),
373 FLOW_RULE_STORE_TIMEOUT_MILLIS,
374 TimeUnit.MILLISECONDS,
375 Collections.emptyList());
376 }
377
378 @Override
379 public void storeFlowRule(FlowRule rule) {
380 storeBatch(new FlowRuleBatchOperation(
381 Arrays.asList(new FlowRuleBatchEntry(FlowRuleOperation.ADD, rule)),
382 rule.deviceId(), idGenerator.getNewId()));
383 }
384
385 @Override
386 public void storeBatch(FlowRuleBatchOperation operation) {
387 if (operation.getOperations().isEmpty()) {
388 notifyDelegate(FlowRuleBatchEvent.completed(
389 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
390 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
391 return;
392 }
393
394 DeviceId deviceId = operation.deviceId();
395
396 ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(deviceId);
397 NodeId master = replicaInfo.master().orNull();
398
399 if (master == null) {
400 log.warn("No master for {} : flows will be marked for removal", deviceId);
401
402 updateStoreInternal(operation);
403
404 notifyDelegate(FlowRuleBatchEvent.completed(
405 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
406 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
407 return;
408 }
409
410 if (Objects.equal(local, master)) {
411 storeBatchInternal(operation);
412 return;
413 }
414
415 log.trace("Forwarding storeBatch to {}, which is the primary (master) for device {}",
416 master, deviceId);
417
418 if (!clusterCommunicator.unicast(operation,
419 APPLY_BATCH_FLOWS,
420 SERIALIZER::encode,
421 master)) {
422 log.warn("Failed to storeBatch: {} to {}", operation, replicaInfo.master());
423
424 Set<FlowRule> allFailures = operation.getOperations().stream()
425 .map(op -> op.target())
426 .collect(Collectors.toSet());
427
428 notifyDelegate(FlowRuleBatchEvent.completed(
429 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
430 new CompletedBatchOperation(false, allFailures, deviceId)));
431 return;
432 }
433 }
434
435 private void storeBatchInternal(FlowRuleBatchOperation operation) {
436
437 final DeviceId did = operation.deviceId();
438 //final Collection<FlowEntry> ft = flowTable.getFlowEntries(did);
439 Set<FlowRuleBatchEntry> currentOps = updateStoreInternal(operation);
440 if (currentOps.isEmpty()) {
441 batchOperationComplete(FlowRuleBatchEvent.completed(
442 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
443 new CompletedBatchOperation(true, Collections.emptySet(), did)));
444 return;
445 }
446
447 notifyDelegate(FlowRuleBatchEvent.requested(new
448 FlowRuleBatchRequest(operation.id(),
449 currentOps), operation.deviceId()));
450 }
451
452 private Set<FlowRuleBatchEntry> updateStoreInternal(FlowRuleBatchOperation operation) {
453 return operation.getOperations().stream().map(
454 op -> {
455 StoredFlowEntry entry;
456 switch (op.operator()) {
457 case ADD:
458 entry = new DefaultFlowEntry(op.target());
459 // always add requested FlowRule
460 // Note: 2 equal FlowEntry may have different treatment
461 flowTable.remove(entry.deviceId(), entry);
462 flowTable.add(entry);
463
464 return op;
465 case REMOVE:
466 entry = flowTable.getFlowEntry(op.target());
467 if (entry != null) {
468 entry.setState(FlowEntryState.PENDING_REMOVE);
469 return op;
470 }
471 break;
472 case MODIFY:
473 //TODO: figure this out at some point
474 break;
475 default:
476 log.warn("Unknown flow operation operator: {}", op.operator());
477 }
478 return null;
479 }
480 ).filter(op -> op != null).collect(Collectors.toSet());
481 }
482
483 @Override
484 public void deleteFlowRule(FlowRule rule) {
485 storeBatch(
486 new FlowRuleBatchOperation(
487 Arrays.asList(
488 new FlowRuleBatchEntry(
489 FlowRuleOperation.REMOVE,
490 rule)), rule.deviceId(), idGenerator.getNewId()));
491 }
492
493 @Override
494 public FlowRuleEvent addOrUpdateFlowRule(FlowEntry rule) {
495 ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(rule.deviceId());
496 if (Objects.equal(local, replicaInfo.master().orNull())) {
497 return addOrUpdateFlowRuleInternal(rule);
498 }
499
500 log.warn("Tried to update FlowRule {} state,"
501 + " while the Node was not the master.", rule);
502 return null;
503 }
504
505 private FlowRuleEvent addOrUpdateFlowRuleInternal(FlowEntry rule) {
506 // check if this new rule is an update to an existing entry
507 StoredFlowEntry stored = flowTable.getFlowEntry(rule);
508 if (stored != null) {
509 stored.setBytes(rule.bytes());
510 stored.setLife(rule.life());
511 stored.setPackets(rule.packets());
512 if (stored.state() == FlowEntryState.PENDING_ADD) {
513 stored.setState(FlowEntryState.ADDED);
514 return new FlowRuleEvent(Type.RULE_ADDED, rule);
515 }
516 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
517 }
518
519 // TODO: Confirm if this behavior is correct. See SimpleFlowRuleStore
520 // TODO: also update backup if the behavior is correct.
521 flowTable.add(rule);
522 return null;
523 }
524
525 @Override
526 public FlowRuleEvent removeFlowRule(FlowEntry rule) {
527 final DeviceId deviceId = rule.deviceId();
528 ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(deviceId);
529 NodeId master = replicaInfo.master().orNull();
530
531 if (Objects.equal(local, master)) {
532 // bypass and handle it locally
533 return removeFlowRuleInternal(rule);
534 }
535
536 if (master == null) {
537 log.warn("Failed to removeFlowRule: No master for {}", deviceId);
538 // TODO: revisit if this should be null (="no-op") or Exception
539 return null;
540 }
541
542 log.trace("Forwarding removeFlowRule to {}, which is the master for device {}",
543 master, deviceId);
544
545 return Futures.get(clusterCommunicator.sendAndReceive(
546 rule,
547 REMOVE_FLOW_ENTRY,
548 SERIALIZER::encode,
549 SERIALIZER::decode,
550 master),
551 FLOW_RULE_STORE_TIMEOUT_MILLIS,
552 TimeUnit.MILLISECONDS,
553 RuntimeException.class);
554 }
555
556 private FlowRuleEvent removeFlowRuleInternal(FlowEntry rule) {
557 final DeviceId deviceId = rule.deviceId();
558 // This is where one could mark a rule as removed and still keep it in the store.
559 final boolean removed = flowTable.remove(deviceId, rule); //flowEntries.remove(deviceId, rule);
560 return removed ? new FlowRuleEvent(RULE_REMOVED, rule) : null;
561 }
562
563 @Override
564 public void batchOperationComplete(FlowRuleBatchEvent event) {
565 //FIXME: need a per device pending response
566 NodeId nodeId = pendingResponses.remove(event.subject().batchId());
567 if (nodeId == null) {
568 notifyDelegate(event);
569 } else {
570 // TODO check unicast return value
571 clusterCommunicator.unicast(event, REMOTE_APPLY_COMPLETED, SERIALIZER::encode, nodeId);
572 //error log: log.warn("Failed to respond to peer for batch operation result");
573 }
574 }
575
576 private final class OnStoreBatch implements ClusterMessageHandler {
577
578 @Override
579 public void handle(final ClusterMessage message) {
580 FlowRuleBatchOperation operation = SERIALIZER.decode(message.payload());
581 log.debug("received batch request {}", operation);
582
583 final DeviceId deviceId = operation.deviceId();
584 ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(deviceId);
585 if (!local.equals(replicaInfo.master().orNull())) {
586
587 Set<FlowRule> failures = new HashSet<>(operation.size());
588 for (FlowRuleBatchEntry op : operation.getOperations()) {
589 failures.add(op.target());
590 }
591 CompletedBatchOperation allFailed = new CompletedBatchOperation(false, failures, deviceId);
592 // This node is no longer the master, respond as all failed.
593 // TODO: we might want to wrap response in envelope
594 // to distinguish sw programming failure and hand over
595 // it make sense in the latter case to retry immediately.
596 message.respond(SERIALIZER.encode(allFailed));
597 return;
598 }
599
600 pendingResponses.put(operation.id(), message.sender());
601 storeBatchInternal(operation);
602 }
603 }
604
Madan Jampanif7536ab2015-05-07 23:23:23 -0700605 private class InternalFlowTable implements ReplicaInfoEventListener {
Madan Jampani86940d92015-05-06 11:47:57 -0700606
607 private final ConcurrentMap<DeviceId, ConcurrentMap<FlowId, Set<StoredFlowEntry>>>
608 flowEntries = new ConcurrentHashMap<>();
609
610 private final Map<DeviceId, Long> lastBackupTimes = Maps.newConcurrentMap();
611 private final Map<DeviceId, Long> lastUpdateTimes = Maps.newConcurrentMap();
612 private final Map<DeviceId, NodeId> lastBackupNodes = Maps.newConcurrentMap();
613
614 private NewConcurrentHashMap<FlowId, Set<StoredFlowEntry>> lazyEmptyFlowTable() {
615 return NewConcurrentHashMap.<FlowId, Set<StoredFlowEntry>>ifNeeded();
616 }
617
Madan Jampanif7536ab2015-05-07 23:23:23 -0700618 @Override
619 public void event(ReplicaInfoEvent event) {
620 if (event.type() == ReplicaInfoEvent.Type.BACKUPS_CHANGED) {
621 DeviceId deviceId = event.subject();
622 if (!Objects.equal(local, replicaInfoManager.getReplicaInfoFor(deviceId).master())) {
623 // ignore since this event is for a device this node does not manage.
624 return;
625 }
626 NodeId latestBackupNode = getBackupNode(deviceId);
627 NodeId existingBackupNode = lastBackupNodes.get(deviceId);
628 if (Objects.equal(latestBackupNode, existingBackupNode)) {
629 // ignore since backup location hasn't changed.
630 return;
631 }
632 backupFlowEntries(latestBackupNode, Sets.newHashSet(deviceId));
633 }
634 }
635
636 private void backupFlowEntries(NodeId nodeId, Set<DeviceId> deviceIds) {
637 log.debug("Sending flowEntries for devices {} to {} as backup.", deviceIds, nodeId);
638 Map<DeviceId, ConcurrentMap<FlowId, Set<StoredFlowEntry>>> deviceFlowEntries =
639 Maps.newConcurrentMap();
640 flowEntries.forEach((key, value) -> {
641 if (deviceIds.contains(key)) {
642 deviceFlowEntries.put(key, value);
643 }
644 });
645 clusterCommunicator.unicast(deviceFlowEntries,
646 FLOW_TABLE_BACKUP,
647 SERIALIZER::encode,
648 nodeId);
649 deviceIds.forEach(id -> {
650 lastBackupTimes.put(id, System.currentTimeMillis());
651 lastBackupNodes.put(id, nodeId);
652 });
653 }
654
Madan Jampani86940d92015-05-06 11:47:57 -0700655 /**
656 * Returns the flow table for specified device.
657 *
658 * @param deviceId identifier of the device
659 * @return Map representing Flow Table of given device.
660 */
661 private ConcurrentMap<FlowId, Set<StoredFlowEntry>> getFlowTable(DeviceId deviceId) {
662 return createIfAbsentUnchecked(flowEntries, deviceId, lazyEmptyFlowTable());
663 }
664
665 private Set<StoredFlowEntry> getFlowEntriesInternal(DeviceId deviceId, FlowId flowId) {
666 return getFlowTable(deviceId).computeIfAbsent(flowId, id -> Sets.newCopyOnWriteArraySet());
667 }
668
669 private StoredFlowEntry getFlowEntryInternal(FlowRule rule) {
670 Set<StoredFlowEntry> flowEntries = getFlowEntriesInternal(rule.deviceId(), rule.id());
671 return flowEntries.stream()
672 .filter(entry -> Objects.equal(entry, rule))
673 .findAny()
674 .orElse(null);
675 }
676
677 private Set<FlowEntry> getFlowEntriesInternal(DeviceId deviceId) {
678 Set<FlowEntry> result = Sets.newHashSet();
679 getFlowTable(deviceId).values().forEach(result::addAll);
680 return result;
681 }
682
683 public StoredFlowEntry getFlowEntry(FlowRule rule) {
684 return getFlowEntryInternal(rule);
685 }
686
687 public Set<FlowEntry> getFlowEntries(DeviceId deviceId) {
688 return getFlowEntriesInternal(deviceId);
689 }
690
691 public void add(FlowEntry rule) {
692 getFlowEntriesInternal(rule.deviceId(), rule.id()).add((StoredFlowEntry) rule);
693 lastUpdateTimes.put(rule.deviceId(), System.currentTimeMillis());
694 }
695
696 public boolean remove(DeviceId deviceId, FlowEntry rule) {
697 try {
698 return getFlowEntriesInternal(deviceId, rule.id()).remove(rule);
699 } finally {
700 lastUpdateTimes.put(deviceId, System.currentTimeMillis());
701 }
702 }
703
704 private NodeId getBackupNode(DeviceId deviceId) {
705 List<NodeId> deviceStandbys = replicaInfoManager.getReplicaInfoFor(deviceId).backups();
706 // pick the standby which is most likely to become next master
707 return deviceStandbys.isEmpty() ? null : deviceStandbys.get(0);
708 }
709
710 private void backup() {
Madan Jampani08bf17b2015-05-06 16:25:26 -0700711 if (!backupEnabled) {
712 return;
713 }
Madan Jampani86940d92015-05-06 11:47:57 -0700714 try {
715 // determine the set of devices that we need to backup during this run.
716 Set<DeviceId> devicesToBackup = mastershipService.getDevicesOf(local)
717 .stream()
718 .filter(deviceId -> {
719 Long lastBackupTime = lastBackupTimes.get(deviceId);
720 Long lastUpdateTime = lastUpdateTimes.get(deviceId);
721 NodeId lastBackupNode = lastBackupNodes.get(deviceId);
722 return lastBackupTime == null
723 || !Objects.equal(lastBackupNode, getBackupNode(deviceId))
724 || (lastUpdateTime != null && lastUpdateTime > lastBackupTime);
725 })
726 .collect(Collectors.toSet());
727
728 // compute a mapping from node to the set of devices whose flow entries it should backup
729 Map<NodeId, Set<DeviceId>> devicesToBackupByNode = Maps.newHashMap();
730 devicesToBackup.forEach(deviceId -> {
731 NodeId backupLocation = getBackupNode(deviceId);
732 if (backupLocation != null) {
733 devicesToBackupByNode.computeIfAbsent(backupLocation, nodeId -> Sets.newHashSet())
734 .add(deviceId);
735 }
736 });
Madan Jampani86940d92015-05-06 11:47:57 -0700737 // send the device flow entries to their respective backup nodes
Madan Jampanif7536ab2015-05-07 23:23:23 -0700738 devicesToBackupByNode.forEach(this::backupFlowEntries);
Madan Jampani86940d92015-05-06 11:47:57 -0700739 } catch (Exception e) {
740 log.error("Backup failed.", e);
741 }
742 }
743
744 private void onBackupReceipt(Map<DeviceId, Map<FlowId, Set<StoredFlowEntry>>> flowTables) {
Madan Jampanif7536ab2015-05-07 23:23:23 -0700745 log.debug("Received flows for {} to backup", flowTables.keySet());
Madan Jampani86940d92015-05-06 11:47:57 -0700746 Set<DeviceId> managedDevices = mastershipService.getDevicesOf(local);
Madan Jampani08bf17b2015-05-06 16:25:26 -0700747 // Only process those devices are that not managed by the local node.
748 Maps.filterKeys(flowTables, deviceId -> !managedDevices.contains(deviceId))
749 .forEach((deviceId, flowTable) -> {
750 Map<FlowId, Set<StoredFlowEntry>> deviceFlowTable = getFlowTable(deviceId);
751 deviceFlowTable.clear();
752 deviceFlowTable.putAll(flowTable);
753 });
Madan Jampani86940d92015-05-06 11:47:57 -0700754 }
755 }
756}