blob: 8efb40a4d09e5265ccd15d18c99329bc296b2d16 [file] [log] [blame]
Jon Hallfa132292017-10-24 11:11:24 -07001 /*
2 * Copyright 2014-present Open Networking Foundation
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 java.util.Collections;
19import java.util.Dictionary;
20import java.util.HashSet;
21import java.util.List;
22import java.util.Map;
23import java.util.Objects;
24import java.util.Set;
Jordan Halterman3664e8e2018-03-21 12:52:37 -070025import java.util.concurrent.CompletableFuture;
Jon Hallfa132292017-10-24 11:11:24 -070026import java.util.concurrent.ExecutorService;
27import java.util.concurrent.Executors;
28import java.util.concurrent.ScheduledExecutorService;
29import java.util.concurrent.ScheduledFuture;
30import java.util.concurrent.TimeUnit;
31import java.util.concurrent.atomic.AtomicReference;
32import java.util.stream.Collectors;
33
34import com.google.common.collect.Streams;
35import org.apache.felix.scr.annotations.Activate;
36import org.apache.felix.scr.annotations.Component;
37import org.apache.felix.scr.annotations.Deactivate;
38import org.apache.felix.scr.annotations.Modified;
39import org.apache.felix.scr.annotations.Property;
40import org.apache.felix.scr.annotations.Reference;
41import org.apache.felix.scr.annotations.ReferenceCardinality;
42import org.apache.felix.scr.annotations.Service;
43import org.onlab.util.KryoNamespace;
44import org.onlab.util.Tools;
45import org.onosproject.cfg.ComponentConfigService;
46import org.onosproject.cluster.ClusterService;
47import org.onosproject.cluster.NodeId;
48import org.onosproject.core.CoreService;
49import org.onosproject.core.IdGenerator;
50import org.onosproject.mastership.MastershipService;
51import org.onosproject.net.DeviceId;
52import org.onosproject.net.device.DeviceService;
53import org.onosproject.net.flow.CompletedBatchOperation;
54import org.onosproject.net.flow.DefaultFlowEntry;
55import org.onosproject.net.flow.FlowEntry;
56import org.onosproject.net.flow.FlowEntry.FlowEntryState;
57import org.onosproject.net.flow.FlowId;
58import org.onosproject.net.flow.FlowRule;
59import org.onosproject.net.flow.oldbatch.FlowRuleBatchEntry;
60import org.onosproject.net.flow.oldbatch.FlowRuleBatchEntry.FlowRuleOperation;
61import org.onosproject.net.flow.oldbatch.FlowRuleBatchEvent;
62import org.onosproject.net.flow.oldbatch.FlowRuleBatchOperation;
63import org.onosproject.net.flow.oldbatch.FlowRuleBatchRequest;
64import org.onosproject.net.flow.FlowRuleEvent;
65import org.onosproject.net.flow.FlowRuleEvent.Type;
66import org.onosproject.net.flow.FlowRuleService;
67import org.onosproject.net.flow.FlowRuleStore;
68import org.onosproject.net.flow.FlowRuleStoreDelegate;
69import org.onosproject.net.flow.StoredFlowEntry;
70import org.onosproject.net.flow.TableStatisticsEntry;
71import org.onosproject.persistence.PersistenceService;
72import org.onosproject.store.AbstractStore;
73import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
74import org.onosproject.store.cluster.messaging.ClusterMessage;
75import org.onosproject.store.cluster.messaging.ClusterMessageHandler;
76import org.onosproject.store.flow.ReplicaInfoEvent;
77import org.onosproject.store.flow.ReplicaInfoEventListener;
78import org.onosproject.store.flow.ReplicaInfoService;
79import org.onosproject.store.impl.MastershipBasedTimestamp;
80import org.onosproject.store.serializers.KryoNamespaces;
81import org.onosproject.store.service.EventuallyConsistentMap;
82import org.onosproject.store.service.EventuallyConsistentMapEvent;
83import org.onosproject.store.service.EventuallyConsistentMapListener;
84import org.onosproject.store.service.Serializer;
85import org.onosproject.store.service.StorageService;
86import org.onosproject.store.service.WallClockTimestamp;
87import org.osgi.service.component.ComponentContext;
88import org.slf4j.Logger;
89
90import com.google.common.collect.ImmutableList;
Jon Hallfa132292017-10-24 11:11:24 -070091import com.google.common.collect.Maps;
92import com.google.common.collect.Sets;
93import com.google.common.util.concurrent.Futures;
94
95import static com.google.common.base.Strings.isNullOrEmpty;
96import static org.onlab.util.Tools.get;
97import static org.onlab.util.Tools.groupedThreads;
98import static org.onosproject.net.flow.FlowRuleEvent.Type.RULE_REMOVED;
99import static org.onosproject.store.flow.ReplicaInfoEvent.Type.MASTER_CHANGED;
100import static org.onosproject.store.flow.impl.ECFlowRuleStoreMessageSubjects.APPLY_BATCH_FLOWS;
101import static org.onosproject.store.flow.impl.ECFlowRuleStoreMessageSubjects.FLOW_TABLE_BACKUP;
102import static org.onosproject.store.flow.impl.ECFlowRuleStoreMessageSubjects.GET_DEVICE_FLOW_ENTRIES;
103import static org.onosproject.store.flow.impl.ECFlowRuleStoreMessageSubjects.GET_FLOW_ENTRY;
104import static org.onosproject.store.flow.impl.ECFlowRuleStoreMessageSubjects.REMOTE_APPLY_COMPLETED;
105import static org.onosproject.store.flow.impl.ECFlowRuleStoreMessageSubjects.REMOVE_FLOW_ENTRY;
106import static org.slf4j.LoggerFactory.getLogger;
107
108/**
109 * Manages inventory of flow rules using a distributed state management protocol.
110 */
Thomas Vachuska71026b22018-01-05 16:01:44 -0800111@Component(immediate = true)
Jon Hallfa132292017-10-24 11:11:24 -0700112@Service
113public class ECFlowRuleStore
114 extends AbstractStore<FlowRuleBatchEvent, FlowRuleStoreDelegate>
115 implements FlowRuleStore {
116
117 private final Logger log = getLogger(getClass());
118
119 private static final int MESSAGE_HANDLER_THREAD_POOL_SIZE = 8;
120 private static final int DEFAULT_MAX_BACKUP_COUNT = 2;
121 private static final boolean DEFAULT_PERSISTENCE_ENABLED = false;
122 private static final int DEFAULT_BACKUP_PERIOD_MILLIS = 2000;
123 private static final long FLOW_RULE_STORE_TIMEOUT_MILLIS = 5000;
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700124 private static final int NUM_BUCKETS = 1024;
Jon Hallfa132292017-10-24 11:11:24 -0700125
126 @Property(name = "msgHandlerPoolSize", intValue = MESSAGE_HANDLER_THREAD_POOL_SIZE,
127 label = "Number of threads in the message handler pool")
128 private int msgHandlerPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
129
130 @Property(name = "backupPeriod", intValue = DEFAULT_BACKUP_PERIOD_MILLIS,
131 label = "Delay in ms between successive backup runs")
132 private int backupPeriod = DEFAULT_BACKUP_PERIOD_MILLIS;
133 @Property(name = "persistenceEnabled", boolValue = false,
134 label = "Indicates whether or not changes in the flow table should be persisted to disk.")
135 private boolean persistenceEnabled = DEFAULT_PERSISTENCE_ENABLED;
136
137 @Property(name = "backupCount", intValue = DEFAULT_MAX_BACKUP_COUNT,
138 label = "Max number of backup copies for each device")
139 private volatile int backupCount = DEFAULT_MAX_BACKUP_COUNT;
140
141 private InternalFlowTable flowTable = new InternalFlowTable();
142
143 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
144 protected ReplicaInfoService replicaInfoManager;
145
146 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
147 protected ClusterCommunicationService clusterCommunicator;
148
149 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
150 protected ClusterService clusterService;
151
152 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
153 protected DeviceService deviceService;
154
155 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
156 protected CoreService coreService;
157
158 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
159 protected ComponentConfigService configService;
160
161 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
162 protected MastershipService mastershipService;
163
164 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
165 protected PersistenceService persistenceService;
166
167 private Map<Long, NodeId> pendingResponses = Maps.newConcurrentMap();
168 private ExecutorService messageHandlingExecutor;
169 private ExecutorService eventHandler;
170
171 private ScheduledFuture<?> backupTask;
172 private final ScheduledExecutorService backupSenderExecutor =
173 Executors.newSingleThreadScheduledExecutor(groupedThreads("onos/flow", "backup-sender", log));
174
175 private EventuallyConsistentMap<DeviceId, List<TableStatisticsEntry>> deviceTableStats;
176 private final EventuallyConsistentMapListener<DeviceId, List<TableStatisticsEntry>> tableStatsListener =
177 new InternalTableStatsListener();
178
179 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
180 protected StorageService storageService;
181
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700182 protected final Serializer serializer = Serializer.using(KryoNamespace.newBuilder()
183 .register(KryoNamespaces.API)
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700184 .register(BucketId.class)
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700185 .register(FlowBucket.class)
186 .build());
Jon Hallfa132292017-10-24 11:11:24 -0700187
188 protected final KryoNamespace.Builder serializerBuilder = KryoNamespace.newBuilder()
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700189 .register(KryoNamespaces.API)
190 .register(BucketId.class)
191 .register(MastershipBasedTimestamp.class);
Jon Hallfa132292017-10-24 11:11:24 -0700192
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700193 private EventuallyConsistentMap<BucketId, Integer> flowCounts;
Jon Hallfa132292017-10-24 11:11:24 -0700194
195 private IdGenerator idGenerator;
196 private NodeId local;
197
198 @Activate
199 public void activate(ComponentContext context) {
200 configService.registerProperties(getClass());
201
202 idGenerator = coreService.getIdGenerator(FlowRuleService.FLOW_OP_TOPIC);
203
204 local = clusterService.getLocalNode().id();
205
206 eventHandler = Executors.newSingleThreadExecutor(
207 groupedThreads("onos/flow", "event-handler", log));
208 messageHandlingExecutor = Executors.newFixedThreadPool(
209 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers", log));
210
211 registerMessageHandlers(messageHandlingExecutor);
212
213 replicaInfoManager.addListener(flowTable);
214 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
215 flowTable::backup,
216 0,
217 backupPeriod,
218 TimeUnit.MILLISECONDS);
219
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700220 flowCounts = storageService.<BucketId, Integer>eventuallyConsistentMapBuilder()
Thomas Vachuskaa8e74772018-02-26 11:33:35 -0800221 .withName("onos-flow-counts")
222 .withSerializer(serializerBuilder)
223 .withAntiEntropyPeriod(5, TimeUnit.SECONDS)
224 .withTimestampProvider((k, v) -> new WallClockTimestamp())
225 .withTombstonesDisabled()
226 .build();
227
Jon Hallfa132292017-10-24 11:11:24 -0700228 deviceTableStats = storageService.<DeviceId, List<TableStatisticsEntry>>eventuallyConsistentMapBuilder()
229 .withName("onos-flow-table-stats")
230 .withSerializer(serializerBuilder)
231 .withAntiEntropyPeriod(5, TimeUnit.SECONDS)
232 .withTimestampProvider((k, v) -> new WallClockTimestamp())
233 .withTombstonesDisabled()
234 .build();
235 deviceTableStats.addListener(tableStatsListener);
236
237 logConfig("Started");
238 }
239
240 @Deactivate
241 public void deactivate(ComponentContext context) {
242 replicaInfoManager.removeListener(flowTable);
243 backupTask.cancel(true);
244 configService.unregisterProperties(getClass(), false);
245 unregisterMessageHandlers();
246 deviceTableStats.removeListener(tableStatsListener);
247 deviceTableStats.destroy();
248 eventHandler.shutdownNow();
249 messageHandlingExecutor.shutdownNow();
250 backupSenderExecutor.shutdownNow();
251 log.info("Stopped");
252 }
253
254 @SuppressWarnings("rawtypes")
255 @Modified
256 public void modified(ComponentContext context) {
257 if (context == null) {
258 logConfig("Default config");
259 return;
260 }
261
262 Dictionary properties = context.getProperties();
263 int newPoolSize;
264 int newBackupPeriod;
265 int newBackupCount;
266 try {
267 String s = get(properties, "msgHandlerPoolSize");
268 newPoolSize = isNullOrEmpty(s) ? msgHandlerPoolSize : Integer.parseInt(s.trim());
269
270 s = get(properties, "backupPeriod");
271 newBackupPeriod = isNullOrEmpty(s) ? backupPeriod : Integer.parseInt(s.trim());
272
273 s = get(properties, "backupCount");
274 newBackupCount = isNullOrEmpty(s) ? backupCount : Integer.parseInt(s.trim());
275 } catch (NumberFormatException | ClassCastException e) {
276 newPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
277 newBackupPeriod = DEFAULT_BACKUP_PERIOD_MILLIS;
278 newBackupCount = DEFAULT_MAX_BACKUP_COUNT;
279 }
280
281 boolean restartBackupTask = false;
282
283 if (newBackupPeriod != backupPeriod) {
284 backupPeriod = newBackupPeriod;
285 restartBackupTask = true;
286 }
287 if (restartBackupTask) {
288 if (backupTask != null) {
289 // cancel previously running task
290 backupTask.cancel(false);
291 }
292 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
293 flowTable::backup,
294 0,
295 backupPeriod,
296 TimeUnit.MILLISECONDS);
297 }
298 if (newPoolSize != msgHandlerPoolSize) {
299 msgHandlerPoolSize = newPoolSize;
300 ExecutorService oldMsgHandler = messageHandlingExecutor;
301 messageHandlingExecutor = Executors.newFixedThreadPool(
302 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers", log));
303
304 // replace previously registered handlers.
305 registerMessageHandlers(messageHandlingExecutor);
306 oldMsgHandler.shutdown();
307 }
308 if (backupCount != newBackupCount) {
309 backupCount = newBackupCount;
310 }
311 logConfig("Reconfigured");
312 }
313
314 private void registerMessageHandlers(ExecutorService executor) {
315
316 clusterCommunicator.addSubscriber(APPLY_BATCH_FLOWS, new OnStoreBatch(), executor);
317 clusterCommunicator.<FlowRuleBatchEvent>addSubscriber(
318 REMOTE_APPLY_COMPLETED, serializer::decode, this::notifyDelegate, executor);
319 clusterCommunicator.addSubscriber(
320 GET_FLOW_ENTRY, serializer::decode, flowTable::getFlowEntry, serializer::encode, executor);
321 clusterCommunicator.addSubscriber(
322 GET_DEVICE_FLOW_ENTRIES, serializer::decode, flowTable::getFlowEntries, serializer::encode, executor);
323 clusterCommunicator.addSubscriber(
324 REMOVE_FLOW_ENTRY, serializer::decode, this::removeFlowRuleInternal, serializer::encode, executor);
325 clusterCommunicator.addSubscriber(
326 FLOW_TABLE_BACKUP, serializer::decode, flowTable::onBackupReceipt, serializer::encode, executor);
327 }
328
329 private void unregisterMessageHandlers() {
330 clusterCommunicator.removeSubscriber(REMOVE_FLOW_ENTRY);
331 clusterCommunicator.removeSubscriber(GET_DEVICE_FLOW_ENTRIES);
332 clusterCommunicator.removeSubscriber(GET_FLOW_ENTRY);
333 clusterCommunicator.removeSubscriber(APPLY_BATCH_FLOWS);
334 clusterCommunicator.removeSubscriber(REMOTE_APPLY_COMPLETED);
335 clusterCommunicator.removeSubscriber(FLOW_TABLE_BACKUP);
336 }
337
338 private void logConfig(String prefix) {
339 log.info("{} with msgHandlerPoolSize = {}; backupPeriod = {}, backupCount = {}",
340 prefix, msgHandlerPoolSize, backupPeriod, backupCount);
341 }
342
Jon Hallfa132292017-10-24 11:11:24 -0700343 @Override
344 public int getFlowRuleCount() {
345 return Streams.stream(deviceService.getDevices()).parallel()
Thomas Vachuskaa8e74772018-02-26 11:33:35 -0800346 .mapToInt(device -> getFlowRuleCount(device.id()))
347 .sum();
348 }
349
350 @Override
351 public int getFlowRuleCount(DeviceId deviceId) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700352 return flowCounts.entrySet().stream()
353 .filter(entry -> entry.getKey().deviceId().equals(deviceId))
354 .mapToInt(entry -> entry.getValue())
355 .sum();
Jon Hallfa132292017-10-24 11:11:24 -0700356 }
357
358 @Override
359 public FlowEntry getFlowEntry(FlowRule rule) {
360 NodeId master = mastershipService.getMasterFor(rule.deviceId());
361
362 if (master == null) {
363 log.debug("Failed to getFlowEntry: No master for {}", rule.deviceId());
364 return null;
365 }
366
367 if (Objects.equals(local, master)) {
368 return flowTable.getFlowEntry(rule);
369 }
370
371 log.trace("Forwarding getFlowEntry to {}, which is the primary (master) for device {}",
372 master, rule.deviceId());
373
374 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(rule,
375 ECFlowRuleStoreMessageSubjects.GET_FLOW_ENTRY,
376 serializer::encode,
377 serializer::decode,
378 master),
379 FLOW_RULE_STORE_TIMEOUT_MILLIS,
380 TimeUnit.MILLISECONDS,
381 null);
382 }
383
384 @Override
385 public Iterable<FlowEntry> getFlowEntries(DeviceId deviceId) {
386 NodeId master = mastershipService.getMasterFor(deviceId);
387
388 if (master == null) {
389 log.debug("Failed to getFlowEntries: No master for {}", deviceId);
390 return Collections.emptyList();
391 }
392
393 if (Objects.equals(local, master)) {
394 return flowTable.getFlowEntries(deviceId);
395 }
396
397 log.trace("Forwarding getFlowEntries to {}, which is the primary (master) for device {}",
398 master, deviceId);
399
400 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(deviceId,
401 ECFlowRuleStoreMessageSubjects.GET_DEVICE_FLOW_ENTRIES,
402 serializer::encode,
403 serializer::decode,
404 master),
405 FLOW_RULE_STORE_TIMEOUT_MILLIS,
406 TimeUnit.MILLISECONDS,
407 Collections.emptyList());
408 }
409
410 @Override
411 public void storeFlowRule(FlowRule rule) {
412 storeBatch(new FlowRuleBatchOperation(
413 Collections.singletonList(new FlowRuleBatchEntry(FlowRuleOperation.ADD, rule)),
414 rule.deviceId(), idGenerator.getNewId()));
415 }
416
417 @Override
418 public void storeBatch(FlowRuleBatchOperation operation) {
419 if (operation.getOperations().isEmpty()) {
420 notifyDelegate(FlowRuleBatchEvent.completed(
421 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
422 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
423 return;
424 }
425
426 DeviceId deviceId = operation.deviceId();
427 NodeId master = mastershipService.getMasterFor(deviceId);
428
429 if (master == null) {
430 log.warn("No master for {} ", deviceId);
431
432 updateStoreInternal(operation);
433
434 notifyDelegate(FlowRuleBatchEvent.completed(
435 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
436 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
437 return;
438 }
439
440 if (Objects.equals(local, master)) {
441 storeBatchInternal(operation);
442 return;
443 }
444
445 log.trace("Forwarding storeBatch to {}, which is the primary (master) for device {}",
446 master, deviceId);
447
448 clusterCommunicator.unicast(operation,
449 APPLY_BATCH_FLOWS,
450 serializer::encode,
451 master)
452 .whenComplete((result, error) -> {
453 if (error != null) {
454 log.warn("Failed to storeBatch: {} to {}", operation, master, error);
455
456 Set<FlowRule> allFailures = operation.getOperations()
457 .stream()
458 .map(op -> op.target())
459 .collect(Collectors.toSet());
460
461 notifyDelegate(FlowRuleBatchEvent.completed(
462 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
463 new CompletedBatchOperation(false, allFailures, deviceId)));
464 }
465 });
466 }
467
468 private void storeBatchInternal(FlowRuleBatchOperation operation) {
469
470 final DeviceId did = operation.deviceId();
471 //final Collection<FlowEntry> ft = flowTable.getFlowEntries(did);
472 Set<FlowRuleBatchEntry> currentOps = updateStoreInternal(operation);
473 if (currentOps.isEmpty()) {
474 batchOperationComplete(FlowRuleBatchEvent.completed(
475 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
476 new CompletedBatchOperation(true, Collections.emptySet(), did)));
477 return;
478 }
479
480 notifyDelegate(FlowRuleBatchEvent.requested(new
481 FlowRuleBatchRequest(operation.id(),
482 currentOps), operation.deviceId()));
483 }
484
485 private Set<FlowRuleBatchEntry> updateStoreInternal(FlowRuleBatchOperation operation) {
486 return operation.getOperations().stream().map(
487 op -> {
488 StoredFlowEntry entry;
489 switch (op.operator()) {
490 case ADD:
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800491 entry = new DefaultFlowEntry(op.target());
492 flowTable.add(entry);
493 return op;
Thomas Vachuska914b0b12018-01-09 11:54:52 -0800494 case MODIFY:
Jon Hallfa132292017-10-24 11:11:24 -0700495 entry = new DefaultFlowEntry(op.target());
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800496 flowTable.update(entry);
Jon Hallfa132292017-10-24 11:11:24 -0700497 return op;
498 case REMOVE:
499 entry = flowTable.getFlowEntry(op.target());
500 if (entry != null) {
Jon Hallfa132292017-10-24 11:11:24 -0700501 entry.setState(FlowEntryState.PENDING_REMOVE);
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800502 flowTable.update(entry);
Jon Hallfa132292017-10-24 11:11:24 -0700503 log.debug("Setting state of rule to pending remove: {}", entry);
504 return op;
505 }
506 break;
Jon Hallfa132292017-10-24 11:11:24 -0700507 default:
508 log.warn("Unknown flow operation operator: {}", op.operator());
509 }
510 return null;
511 }
512 ).filter(Objects::nonNull).collect(Collectors.toSet());
513 }
514
515 @Override
516 public void deleteFlowRule(FlowRule rule) {
517 storeBatch(
518 new FlowRuleBatchOperation(
519 Collections.singletonList(
520 new FlowRuleBatchEntry(
521 FlowRuleOperation.REMOVE,
522 rule)), rule.deviceId(), idGenerator.getNewId()));
523 }
524
525 @Override
526 public FlowRuleEvent pendingFlowRule(FlowEntry rule) {
527 if (mastershipService.isLocalMaster(rule.deviceId())) {
528 StoredFlowEntry stored = flowTable.getFlowEntry(rule);
529 if (stored != null &&
530 stored.state() != FlowEntryState.PENDING_ADD) {
531 stored.setState(FlowEntryState.PENDING_ADD);
532 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
533 }
534 }
535 return null;
536 }
537
538 @Override
539 public FlowRuleEvent addOrUpdateFlowRule(FlowEntry rule) {
540 NodeId master = mastershipService.getMasterFor(rule.deviceId());
541 if (Objects.equals(local, master)) {
542 return addOrUpdateFlowRuleInternal(rule);
543 }
544
545 log.warn("Tried to update FlowRule {} state,"
546 + " while the Node was not the master.", rule);
547 return null;
548 }
549
550 private FlowRuleEvent addOrUpdateFlowRuleInternal(FlowEntry rule) {
551 // check if this new rule is an update to an existing entry
552 StoredFlowEntry stored = flowTable.getFlowEntry(rule);
553 if (stored != null) {
Jon Hallfa132292017-10-24 11:11:24 -0700554 stored.setBytes(rule.bytes());
555 stored.setLife(rule.life(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
556 stored.setLiveType(rule.liveType());
557 stored.setPackets(rule.packets());
558 stored.setLastSeen();
559 if (stored.state() == FlowEntryState.PENDING_ADD) {
560 stored.setState(FlowEntryState.ADDED);
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800561 // Update the flow table to ensure the changes are replicated
562 flowTable.update(stored);
Jon Hallfa132292017-10-24 11:11:24 -0700563 return new FlowRuleEvent(Type.RULE_ADDED, rule);
564 }
565 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
566 }
567
568 // TODO: Confirm if this behavior is correct. See SimpleFlowRuleStore
569 // TODO: also update backup if the behavior is correct.
570 flowTable.add(rule);
571 return null;
572 }
573
574 @Override
575 public FlowRuleEvent removeFlowRule(FlowEntry rule) {
576 final DeviceId deviceId = rule.deviceId();
577 NodeId master = mastershipService.getMasterFor(deviceId);
578
579 if (Objects.equals(local, master)) {
580 // bypass and handle it locally
581 return removeFlowRuleInternal(rule);
582 }
583
584 if (master == null) {
585 log.warn("Failed to removeFlowRule: No master for {}", deviceId);
586 // TODO: revisit if this should be null (="no-op") or Exception
587 return null;
588 }
589
590 log.trace("Forwarding removeFlowRule to {}, which is the master for device {}",
591 master, deviceId);
592
593 return Futures.getUnchecked(clusterCommunicator.sendAndReceive(
594 rule,
595 REMOVE_FLOW_ENTRY,
596 serializer::encode,
597 serializer::decode,
598 master));
599 }
600
601 private FlowRuleEvent removeFlowRuleInternal(FlowEntry rule) {
Jon Hallfa132292017-10-24 11:11:24 -0700602 // This is where one could mark a rule as removed and still keep it in the store.
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800603 final FlowEntry removed = flowTable.remove(rule);
Jon Hallfa132292017-10-24 11:11:24 -0700604 // rule may be partial rule that is missing treatment, we should use rule from store instead
605 return removed != null ? new FlowRuleEvent(RULE_REMOVED, removed) : null;
606 }
607
608 @Override
609 public void purgeFlowRule(DeviceId deviceId) {
610 flowTable.purgeFlowRule(deviceId);
611 }
612
613 @Override
614 public void purgeFlowRules() {
615 flowTable.purgeFlowRules();
616 }
617
618 @Override
619 public void batchOperationComplete(FlowRuleBatchEvent event) {
620 //FIXME: need a per device pending response
621 NodeId nodeId = pendingResponses.remove(event.subject().batchId());
622 if (nodeId == null) {
623 notifyDelegate(event);
624 } else {
625 // TODO check unicast return value
626 clusterCommunicator.unicast(event, REMOTE_APPLY_COMPLETED, serializer::encode, nodeId);
627 //error log: log.warn("Failed to respond to peer for batch operation result");
628 }
629 }
630
631 private final class OnStoreBatch implements ClusterMessageHandler {
632
633 @Override
634 public void handle(final ClusterMessage message) {
635 FlowRuleBatchOperation operation = serializer.decode(message.payload());
636 log.debug("received batch request {}", operation);
637
638 final DeviceId deviceId = operation.deviceId();
639 NodeId master = mastershipService.getMasterFor(deviceId);
640 if (!Objects.equals(local, master)) {
641 Set<FlowRule> failures = new HashSet<>(operation.size());
642 for (FlowRuleBatchEntry op : operation.getOperations()) {
643 failures.add(op.target());
644 }
645 CompletedBatchOperation allFailed = new CompletedBatchOperation(false, failures, deviceId);
646 // This node is no longer the master, respond as all failed.
647 // TODO: we might want to wrap response in envelope
648 // to distinguish sw programming failure and hand over
649 // it make sense in the latter case to retry immediately.
650 message.respond(serializer.encode(allFailed));
651 return;
652 }
653
654 pendingResponses.put(operation.id(), message.sender());
655 storeBatchInternal(operation);
656 }
657 }
658
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700659 /**
660 * Represents a backup to a of a distinct bucket to a distinct node.
661 */
Jon Hallfa132292017-10-24 11:11:24 -0700662 private class BackupOperation {
663 private final NodeId nodeId;
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700664 private final BucketId bucketId;
Jon Hallfa132292017-10-24 11:11:24 -0700665
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700666 BackupOperation(NodeId nodeId, BucketId bucketId) {
Jon Hallfa132292017-10-24 11:11:24 -0700667 this.nodeId = nodeId;
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700668 this.bucketId = bucketId;
669 }
670
671 NodeId nodeId() {
672 return nodeId;
673 }
674
675 BucketId bucketId() {
676 return bucketId;
Jon Hallfa132292017-10-24 11:11:24 -0700677 }
678
679 @Override
680 public int hashCode() {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700681 return Objects.hash(nodeId, bucketId);
Jon Hallfa132292017-10-24 11:11:24 -0700682 }
683
684 @Override
685 public boolean equals(Object other) {
686 if (other != null && other instanceof BackupOperation) {
687 BackupOperation that = (BackupOperation) other;
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700688 return this.nodeId.equals(that.nodeId)
689 && this.bucketId.equals(that.bucketId);
Jon Hallfa132292017-10-24 11:11:24 -0700690 }
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700691 return false;
Jon Hallfa132292017-10-24 11:11:24 -0700692 }
693 }
694
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700695 /**
696 * Represents a distinct device flow bucket.
697 */
698 private class BucketId {
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700699 private final DeviceId deviceId;
700 private final int bucket;
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700701
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700702 BucketId(DeviceId deviceId, int bucket) {
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700703 this.deviceId = deviceId;
704 this.bucket = bucket;
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700705 }
706
707 DeviceId deviceId() {
708 return deviceId;
709 }
710
711 int bucket() {
712 return bucket;
713 }
714
715 @Override
716 public int hashCode() {
717 return Objects.hash(deviceId, bucket);
718 }
719
720 @Override
721 public boolean equals(Object other) {
722 if (other != null && other instanceof BucketId) {
723 BucketId that = (BucketId) other;
724 return this.deviceId.equals(that.deviceId)
725 && this.bucket == that.bucket;
726 }
727 return false;
728 }
729 }
730
731 /**
732 * Container for flows in a specific bucket.
733 */
734 private class FlowBucket {
735 private final BucketId bucketId;
736 private final Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> table;
737
738 BucketId bucketId() {
739 return bucketId;
740 }
741
742 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> table() {
743 return table;
744 }
745
746 FlowBucket(BucketId bucketId, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> table) {
747 this.bucketId = bucketId;
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700748 this.table = table;
749 }
750 }
751
Jon Hallfa132292017-10-24 11:11:24 -0700752 private class InternalFlowTable implements ReplicaInfoEventListener {
753
754 //TODO replace the Map<V,V> with ExtendedSet
755 private final Map<DeviceId, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>
756 flowEntries = Maps.newConcurrentMap();
757
758 private final Map<BackupOperation, Long> lastBackupTimes = Maps.newConcurrentMap();
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700759 private final Map<BucketId, Long> lastUpdateTimes = Maps.newConcurrentMap();
760 private final Set<BackupOperation> inFlightUpdates = Sets.newConcurrentHashSet();
Jon Hallfa132292017-10-24 11:11:24 -0700761
762 @Override
763 public void event(ReplicaInfoEvent event) {
764 eventHandler.execute(() -> handleEvent(event));
765 }
766
767 private void handleEvent(ReplicaInfoEvent event) {
768 DeviceId deviceId = event.subject();
769 if (!mastershipService.isLocalMaster(deviceId)) {
770 return;
771 }
772 if (event.type() == MASTER_CHANGED) {
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700773 for (int bucket = 0; bucket < NUM_BUCKETS; bucket++) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700774 recordUpdate(new BucketId(deviceId, bucket));
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700775 }
Jon Hallfa132292017-10-24 11:11:24 -0700776 }
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700777 backupSenderExecutor.execute(this::backup);
Jon Hallfa132292017-10-24 11:11:24 -0700778 }
779
Jon Hallfa132292017-10-24 11:11:24 -0700780 /**
781 * Returns the flow table for specified device.
782 *
783 * @param deviceId identifier of the device
784 * @return Map representing Flow Table of given device.
785 */
786 private Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> getFlowTable(DeviceId deviceId) {
787 if (persistenceEnabled) {
788 return flowEntries.computeIfAbsent(deviceId, id -> persistenceService
789 .<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>persistentMapBuilder()
790 .withName("FlowTable:" + deviceId.toString())
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700791 .withSerializer(serializer)
Jon Hallfa132292017-10-24 11:11:24 -0700792 .build());
793 } else {
794 return flowEntries.computeIfAbsent(deviceId, id -> Maps.newConcurrentMap());
795 }
796 }
797
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700798 private FlowBucket getFlowBucket(BucketId bucketId) {
Jon Hallfa132292017-10-24 11:11:24 -0700799 if (persistenceEnabled) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700800 return new FlowBucket(bucketId, flowEntries.computeIfAbsent(bucketId.deviceId(), id ->
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700801 persistenceService.<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>persistentMapBuilder()
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700802 .withName("FlowTable:" + bucketId.deviceId().toString())
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700803 .withSerializer(serializer)
804 .build())
805 .entrySet()
806 .stream()
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700807 .filter(entry -> isInBucket(entry.getKey(), bucketId.bucket()))
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700808 .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
Jon Hallfa132292017-10-24 11:11:24 -0700809 } else {
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700810 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> copy = Maps.newHashMap();
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700811 flowEntries.computeIfAbsent(bucketId.deviceId(), id -> Maps.newConcurrentMap())
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700812 .entrySet()
813 .stream()
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700814 .filter(entry -> isInBucket(entry.getKey(), bucketId.bucket()))
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700815 .forEach(entry -> {
816 copy.put(entry.getKey(), Maps.newHashMap(entry.getValue()));
817 });
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700818 return new FlowBucket(bucketId, copy);
Jon Hallfa132292017-10-24 11:11:24 -0700819 }
820 }
821
822 private Map<StoredFlowEntry, StoredFlowEntry> getFlowEntriesInternal(DeviceId deviceId, FlowId flowId) {
823 return getFlowTable(deviceId).computeIfAbsent(flowId, id -> Maps.newConcurrentMap());
824 }
825
826 private StoredFlowEntry getFlowEntryInternal(FlowRule rule) {
827 return getFlowEntriesInternal(rule.deviceId(), rule.id()).get(rule);
828 }
829
830 private Set<FlowEntry> getFlowEntriesInternal(DeviceId deviceId) {
831 return getFlowTable(deviceId).values().stream()
832 .flatMap(m -> m.values().stream())
833 .collect(Collectors.toSet());
834 }
835
836 public StoredFlowEntry getFlowEntry(FlowRule rule) {
837 return getFlowEntryInternal(rule);
838 }
839
840 public Set<FlowEntry> getFlowEntries(DeviceId deviceId) {
841 return getFlowEntriesInternal(deviceId);
842 }
843
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700844 private boolean isInBucket(FlowId flowId, int bucket) {
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700845 return bucket(flowId) == bucket;
846 }
847
848 private int bucket(FlowId flowId) {
849 return (int) (flowId.id() % NUM_BUCKETS);
850 }
851
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700852 private void recordUpdate(BucketId bucketId) {
853 lastUpdateTimes.put(bucketId, System.currentTimeMillis());
854 FlowBucket flowBucket = getFlowBucket(bucketId);
855 int flowCount = flowBucket.table().entrySet()
856 .stream()
857 .mapToInt(e -> e.getValue().values().size())
858 .sum();
859 flowCounts.put(bucketId, flowCount);
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700860 }
861
Jon Hallfa132292017-10-24 11:11:24 -0700862 public void add(FlowEntry rule) {
Devin Limcdca1952018-03-28 18:13:33 -0700863 getFlowEntriesInternal(rule.deviceId(), rule.id())
864 .compute((StoredFlowEntry) rule, (k, stored) -> {
865 return (StoredFlowEntry) rule;
866 });
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700867 recordUpdate(new BucketId(rule.deviceId(), bucket(rule.id())));
Jon Hallfa132292017-10-24 11:11:24 -0700868 }
869
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800870 public void update(FlowEntry rule) {
871 getFlowEntriesInternal(rule.deviceId(), rule.id())
872 .computeIfPresent((StoredFlowEntry) rule, (k, stored) -> {
873 if (rule instanceof DefaultFlowEntry) {
874 DefaultFlowEntry updated = (DefaultFlowEntry) rule;
875 if (stored instanceof DefaultFlowEntry) {
876 DefaultFlowEntry storedEntry = (DefaultFlowEntry) stored;
877 if (updated.created() >= storedEntry.created()) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700878 recordUpdate(new BucketId(rule.deviceId(), bucket(rule.id())));
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800879 return updated;
880 } else {
881 log.debug("Trying to update more recent flow entry {} (stored: {})", updated, stored);
882 return stored;
883 }
884 }
885 }
886 return stored;
887 });
888 }
889
890 public FlowEntry remove(FlowEntry rule) {
Jon Hallfa132292017-10-24 11:11:24 -0700891 final AtomicReference<FlowEntry> removedRule = new AtomicReference<>();
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800892 final Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> flowTable = getFlowTable(rule.deviceId());
Jordan Haltermance336f72018-01-16 17:08:09 -0800893 flowTable.computeIfPresent(rule.id(), (flowId, flowEntries) -> {
894 flowEntries.computeIfPresent((StoredFlowEntry) rule, (k, stored) -> {
Jon Hallfa132292017-10-24 11:11:24 -0700895 if (rule instanceof DefaultFlowEntry) {
896 DefaultFlowEntry toRemove = (DefaultFlowEntry) rule;
897 if (stored instanceof DefaultFlowEntry) {
898 DefaultFlowEntry storedEntry = (DefaultFlowEntry) stored;
899 if (toRemove.created() < storedEntry.created()) {
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800900 log.debug("Trying to remove more recent flow entry {} (stored: {})", toRemove, stored);
Jon Hallfa132292017-10-24 11:11:24 -0700901 // the key is not updated, removedRule remains null
902 return stored;
903 }
904 }
905 }
906 removedRule.set(stored);
907 return null;
908 });
Jordan Haltermance336f72018-01-16 17:08:09 -0800909 return flowEntries.isEmpty() ? null : flowEntries;
910 });
Jon Hallfa132292017-10-24 11:11:24 -0700911
912 if (removedRule.get() != null) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700913 recordUpdate(new BucketId(rule.deviceId(), bucket(rule.id())));
Jon Hallfa132292017-10-24 11:11:24 -0700914 return removedRule.get();
915 } else {
916 return null;
917 }
918 }
919
920 public void purgeFlowRule(DeviceId deviceId) {
921 flowEntries.remove(deviceId);
922 }
923
924 public void purgeFlowRules() {
925 flowEntries.clear();
926 }
927
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700928 private boolean isMasterNode(DeviceId deviceId) {
929 NodeId master = replicaInfoManager.getReplicaInfoFor(deviceId).master().orElse(null);
930 return Objects.equals(master, clusterService.getLocalNode().id());
931 }
932
933 private boolean isBackupNode(NodeId nodeId, DeviceId deviceId) {
Jon Hallfa132292017-10-24 11:11:24 -0700934 List<NodeId> allPossibleBackupNodes = replicaInfoManager.getReplicaInfoFor(deviceId).backups();
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700935 return allPossibleBackupNodes.indexOf(nodeId) < backupCount;
Jon Hallfa132292017-10-24 11:11:24 -0700936 }
937
938 private void backup() {
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700939 clusterService.getNodes().stream()
940 .filter(node -> !node.id().equals(clusterService.getLocalNode().id()))
941 .forEach(node -> {
942 try {
943 backup(node.id());
944 } catch (Exception e) {
945 log.error("Backup failed.", e);
946 }
Jon Hallfa132292017-10-24 11:11:24 -0700947 });
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700948 }
949
950 private void backup(NodeId nodeId) {
951 for (DeviceId deviceId : flowEntries.keySet()) {
952 if (isMasterNode(deviceId) && isBackupNode(nodeId, deviceId)) {
953 backup(nodeId, deviceId);
954 }
Jon Hallfa132292017-10-24 11:11:24 -0700955 }
956 }
957
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700958 private void backup(NodeId nodeId, DeviceId deviceId) {
959 final long timestamp = System.currentTimeMillis();
960 for (int bucket = 0; bucket < NUM_BUCKETS; bucket++) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700961 BucketId bucketId = new BucketId(deviceId, bucket);
962 BackupOperation operation = new BackupOperation(nodeId, bucketId);
963 if (startBackup(operation)) {
964 backup(operation).whenCompleteAsync((succeeded, error) -> {
965 if (error == null && succeeded) {
966 succeedBackup(operation, timestamp);
967 } else {
968 failBackup(operation);
969 }
970 backup(nodeId);
971 }, backupSenderExecutor);
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700972 }
973 }
974 }
975
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700976 private boolean startBackup(BackupOperation operation) {
977 long lastBackupTime = lastBackupTimes.getOrDefault(operation, 0L);
978 long lastUpdateTime = lastUpdateTimes.getOrDefault(operation.bucketId(), 0L);
979 return lastUpdateTime > 0 && lastBackupTime <= lastUpdateTime && inFlightUpdates.add(operation);
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700980 }
981
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700982 private void failBackup(BackupOperation operation) {
983 inFlightUpdates.remove(operation);
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700984 }
985
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700986 private void succeedBackup(BackupOperation operation, long timestamp) {
987 inFlightUpdates.remove(operation);
988 lastBackupTimes.put(operation, timestamp);
989 }
990
991 private CompletableFuture<Boolean> backup(BackupOperation operation) {
992 log.debug("Sending flowEntries in bucket {} for device {} to {} for backup.",
993 operation.bucketId().bucket(), operation.bucketId().deviceId(), operation.nodeId());
994 FlowBucket flowBucket = getFlowBucket(operation.bucketId());
995
996 CompletableFuture<Boolean> future = new CompletableFuture<>();
997 clusterCommunicator.<FlowBucket, Set<FlowId>>
998 sendAndReceive(flowBucket,
999 FLOW_TABLE_BACKUP,
1000 serializer::encode,
1001 serializer::decode,
1002 operation.nodeId())
1003 .whenComplete((backedupFlows, error) -> {
1004 Set<FlowId> flowsNotBackedUp = error != null ?
1005 flowBucket.table().keySet() :
1006 Sets.difference(flowBucket.table().keySet(), backedupFlows);
1007 if (flowsNotBackedUp.size() > 0) {
1008 log.warn("Failed to backup flows: {}. Reason: {}, Node: {}",
1009 flowsNotBackedUp, error != null ? error.getMessage() : "none", operation.nodeId());
1010 }
1011 future.complete(backedupFlows != null);
1012 });
1013 return future;
1014 }
1015
1016 private Set<FlowId> onBackupReceipt(FlowBucket flowBucket) {
1017 log.debug("Received flowEntries for {} bucket {} to backup",
1018 flowBucket.bucketId().deviceId(), flowBucket.bucketId);
Jordan Halterman3664e8e2018-03-21 12:52:37 -07001019 Set<FlowId> backedupFlows = Sets.newHashSet();
Jon Hallfa132292017-10-24 11:11:24 -07001020 try {
Jordan Halterman3664e8e2018-03-21 12:52:37 -07001021 // Only process those devices are that not managed by the local node.
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -07001022 NodeId master = replicaInfoManager.getReplicaInfoFor(flowBucket.bucketId().deviceId())
1023 .master()
1024 .orElse(null);
Jordan Halterman3664e8e2018-03-21 12:52:37 -07001025 if (!Objects.equals(local, master)) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -07001026 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> backupFlowTable =
1027 getFlowTable(flowBucket.bucketId().deviceId());
1028 backupFlowTable.putAll(flowBucket.table());
Jordan Halterman3664e8e2018-03-21 12:52:37 -07001029 backupFlowTable.entrySet()
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -07001030 .removeIf(entry -> isInBucket(entry.getKey(), flowBucket.bucketId().bucket())
1031 && !flowBucket.table().containsKey(entry.getKey()));
1032 backedupFlows.addAll(flowBucket.table().keySet());
Jordan Halterman3664e8e2018-03-21 12:52:37 -07001033 }
Jon Hallfa132292017-10-24 11:11:24 -07001034 } catch (Exception e) {
1035 log.warn("Failure processing backup request", e);
1036 }
Jordan Halterman3664e8e2018-03-21 12:52:37 -07001037 return backedupFlows;
Jon Hallfa132292017-10-24 11:11:24 -07001038 }
1039 }
1040
1041 @Override
1042 public FlowRuleEvent updateTableStatistics(DeviceId deviceId,
1043 List<TableStatisticsEntry> tableStats) {
1044 deviceTableStats.put(deviceId, tableStats);
1045 return null;
1046 }
1047
1048 @Override
1049 public Iterable<TableStatisticsEntry> getTableStatistics(DeviceId deviceId) {
1050 NodeId master = mastershipService.getMasterFor(deviceId);
1051
1052 if (master == null) {
1053 log.debug("Failed to getTableStats: No master for {}", deviceId);
1054 return Collections.emptyList();
1055 }
1056
1057 List<TableStatisticsEntry> tableStats = deviceTableStats.get(deviceId);
1058 if (tableStats == null) {
1059 return Collections.emptyList();
1060 }
1061 return ImmutableList.copyOf(tableStats);
1062 }
1063
1064 @Override
1065 public long getActiveFlowRuleCount(DeviceId deviceId) {
1066 return Streams.stream(getTableStatistics(deviceId))
1067 .mapToLong(TableStatisticsEntry::activeFlowEntries)
1068 .sum();
1069 }
1070
1071 private class InternalTableStatsListener
1072 implements EventuallyConsistentMapListener<DeviceId, List<TableStatisticsEntry>> {
1073 @Override
1074 public void event(EventuallyConsistentMapEvent<DeviceId,
1075 List<TableStatisticsEntry>> event) {
1076 //TODO: Generate an event to listeners (do we need?)
1077 }
1078 }
1079}