blob: 30806151bd4d1908a74121e6d5f1f0de4cc21031 [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 Haltermanb2f57952018-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 Haltermanb2f57952018-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 Haltermanb2f57952018-03-21 12:52:37 -0700182 protected final Serializer serializer = Serializer.using(KryoNamespace.newBuilder()
183 .register(KryoNamespaces.API)
184 .register(FlowBucket.class)
185 .build());
Jon Hallfa132292017-10-24 11:11:24 -0700186
187 protected final KryoNamespace.Builder serializerBuilder = KryoNamespace.newBuilder()
188 .register(KryoNamespaces.API)
189 .register(MastershipBasedTimestamp.class);
190
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700191 private EventuallyConsistentMap<DeviceId, Map<Integer, Integer>> flowCounts;
Jon Hallfa132292017-10-24 11:11:24 -0700192
193 private IdGenerator idGenerator;
194 private NodeId local;
195
196 @Activate
197 public void activate(ComponentContext context) {
198 configService.registerProperties(getClass());
199
200 idGenerator = coreService.getIdGenerator(FlowRuleService.FLOW_OP_TOPIC);
201
202 local = clusterService.getLocalNode().id();
203
204 eventHandler = Executors.newSingleThreadExecutor(
205 groupedThreads("onos/flow", "event-handler", log));
206 messageHandlingExecutor = Executors.newFixedThreadPool(
207 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers", log));
208
209 registerMessageHandlers(messageHandlingExecutor);
210
211 replicaInfoManager.addListener(flowTable);
212 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
213 flowTable::backup,
214 0,
215 backupPeriod,
216 TimeUnit.MILLISECONDS);
217
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700218 flowCounts = storageService.<DeviceId, Map<Integer, Integer>>eventuallyConsistentMapBuilder()
Thomas Vachuskaa8e74772018-02-26 11:33:35 -0800219 .withName("onos-flow-counts")
220 .withSerializer(serializerBuilder)
221 .withAntiEntropyPeriod(5, TimeUnit.SECONDS)
222 .withTimestampProvider((k, v) -> new WallClockTimestamp())
223 .withTombstonesDisabled()
224 .build();
225
Jon Hallfa132292017-10-24 11:11:24 -0700226 deviceTableStats = storageService.<DeviceId, List<TableStatisticsEntry>>eventuallyConsistentMapBuilder()
227 .withName("onos-flow-table-stats")
228 .withSerializer(serializerBuilder)
229 .withAntiEntropyPeriod(5, TimeUnit.SECONDS)
230 .withTimestampProvider((k, v) -> new WallClockTimestamp())
231 .withTombstonesDisabled()
232 .build();
233 deviceTableStats.addListener(tableStatsListener);
234
235 logConfig("Started");
236 }
237
238 @Deactivate
239 public void deactivate(ComponentContext context) {
240 replicaInfoManager.removeListener(flowTable);
241 backupTask.cancel(true);
242 configService.unregisterProperties(getClass(), false);
243 unregisterMessageHandlers();
244 deviceTableStats.removeListener(tableStatsListener);
245 deviceTableStats.destroy();
246 eventHandler.shutdownNow();
247 messageHandlingExecutor.shutdownNow();
248 backupSenderExecutor.shutdownNow();
249 log.info("Stopped");
250 }
251
252 @SuppressWarnings("rawtypes")
253 @Modified
254 public void modified(ComponentContext context) {
255 if (context == null) {
256 logConfig("Default config");
257 return;
258 }
259
260 Dictionary properties = context.getProperties();
261 int newPoolSize;
262 int newBackupPeriod;
263 int newBackupCount;
264 try {
265 String s = get(properties, "msgHandlerPoolSize");
266 newPoolSize = isNullOrEmpty(s) ? msgHandlerPoolSize : Integer.parseInt(s.trim());
267
268 s = get(properties, "backupPeriod");
269 newBackupPeriod = isNullOrEmpty(s) ? backupPeriod : Integer.parseInt(s.trim());
270
271 s = get(properties, "backupCount");
272 newBackupCount = isNullOrEmpty(s) ? backupCount : Integer.parseInt(s.trim());
273 } catch (NumberFormatException | ClassCastException e) {
274 newPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
275 newBackupPeriod = DEFAULT_BACKUP_PERIOD_MILLIS;
276 newBackupCount = DEFAULT_MAX_BACKUP_COUNT;
277 }
278
279 boolean restartBackupTask = false;
280
281 if (newBackupPeriod != backupPeriod) {
282 backupPeriod = newBackupPeriod;
283 restartBackupTask = true;
284 }
285 if (restartBackupTask) {
286 if (backupTask != null) {
287 // cancel previously running task
288 backupTask.cancel(false);
289 }
290 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
291 flowTable::backup,
292 0,
293 backupPeriod,
294 TimeUnit.MILLISECONDS);
295 }
296 if (newPoolSize != msgHandlerPoolSize) {
297 msgHandlerPoolSize = newPoolSize;
298 ExecutorService oldMsgHandler = messageHandlingExecutor;
299 messageHandlingExecutor = Executors.newFixedThreadPool(
300 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers", log));
301
302 // replace previously registered handlers.
303 registerMessageHandlers(messageHandlingExecutor);
304 oldMsgHandler.shutdown();
305 }
306 if (backupCount != newBackupCount) {
307 backupCount = newBackupCount;
308 }
309 logConfig("Reconfigured");
310 }
311
312 private void registerMessageHandlers(ExecutorService executor) {
313
314 clusterCommunicator.addSubscriber(APPLY_BATCH_FLOWS, new OnStoreBatch(), executor);
315 clusterCommunicator.<FlowRuleBatchEvent>addSubscriber(
316 REMOTE_APPLY_COMPLETED, serializer::decode, this::notifyDelegate, executor);
317 clusterCommunicator.addSubscriber(
318 GET_FLOW_ENTRY, serializer::decode, flowTable::getFlowEntry, serializer::encode, executor);
319 clusterCommunicator.addSubscriber(
320 GET_DEVICE_FLOW_ENTRIES, serializer::decode, flowTable::getFlowEntries, serializer::encode, executor);
321 clusterCommunicator.addSubscriber(
322 REMOVE_FLOW_ENTRY, serializer::decode, this::removeFlowRuleInternal, serializer::encode, executor);
323 clusterCommunicator.addSubscriber(
324 FLOW_TABLE_BACKUP, serializer::decode, flowTable::onBackupReceipt, serializer::encode, executor);
325 }
326
327 private void unregisterMessageHandlers() {
328 clusterCommunicator.removeSubscriber(REMOVE_FLOW_ENTRY);
329 clusterCommunicator.removeSubscriber(GET_DEVICE_FLOW_ENTRIES);
330 clusterCommunicator.removeSubscriber(GET_FLOW_ENTRY);
331 clusterCommunicator.removeSubscriber(APPLY_BATCH_FLOWS);
332 clusterCommunicator.removeSubscriber(REMOTE_APPLY_COMPLETED);
333 clusterCommunicator.removeSubscriber(FLOW_TABLE_BACKUP);
334 }
335
336 private void logConfig(String prefix) {
337 log.info("{} with msgHandlerPoolSize = {}; backupPeriod = {}, backupCount = {}",
338 prefix, msgHandlerPoolSize, backupPeriod, backupCount);
339 }
340
341 // This is not a efficient operation on a distributed sharded
342 // flow store. We need to revisit the need for this operation or at least
343 // make it device specific.
344 @Override
345 public int getFlowRuleCount() {
346 return Streams.stream(deviceService.getDevices()).parallel()
Thomas Vachuskaa8e74772018-02-26 11:33:35 -0800347 .mapToInt(device -> getFlowRuleCount(device.id()))
348 .sum();
349 }
350
351 @Override
352 public int getFlowRuleCount(DeviceId deviceId) {
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700353 Map<Integer, Integer> counts = flowCounts.get(deviceId);
354 return counts != null
355 ? counts.values().stream().mapToInt(v -> v).sum()
356 : flowTable.flowEntries.get(deviceId) != null
357 ? flowTable.flowEntries.get(deviceId).keySet().size() : 0;
Jon Hallfa132292017-10-24 11:11:24 -0700358 }
359
360 @Override
361 public FlowEntry getFlowEntry(FlowRule rule) {
362 NodeId master = mastershipService.getMasterFor(rule.deviceId());
363
364 if (master == null) {
365 log.debug("Failed to getFlowEntry: No master for {}", rule.deviceId());
366 return null;
367 }
368
369 if (Objects.equals(local, master)) {
370 return flowTable.getFlowEntry(rule);
371 }
372
373 log.trace("Forwarding getFlowEntry to {}, which is the primary (master) for device {}",
374 master, rule.deviceId());
375
376 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(rule,
377 ECFlowRuleStoreMessageSubjects.GET_FLOW_ENTRY,
378 serializer::encode,
379 serializer::decode,
380 master),
381 FLOW_RULE_STORE_TIMEOUT_MILLIS,
382 TimeUnit.MILLISECONDS,
383 null);
384 }
385
386 @Override
387 public Iterable<FlowEntry> getFlowEntries(DeviceId deviceId) {
388 NodeId master = mastershipService.getMasterFor(deviceId);
389
390 if (master == null) {
391 log.debug("Failed to getFlowEntries: No master for {}", deviceId);
392 return Collections.emptyList();
393 }
394
395 if (Objects.equals(local, master)) {
396 return flowTable.getFlowEntries(deviceId);
397 }
398
399 log.trace("Forwarding getFlowEntries to {}, which is the primary (master) for device {}",
400 master, deviceId);
401
402 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(deviceId,
403 ECFlowRuleStoreMessageSubjects.GET_DEVICE_FLOW_ENTRIES,
404 serializer::encode,
405 serializer::decode,
406 master),
407 FLOW_RULE_STORE_TIMEOUT_MILLIS,
408 TimeUnit.MILLISECONDS,
409 Collections.emptyList());
410 }
411
412 @Override
413 public void storeFlowRule(FlowRule rule) {
414 storeBatch(new FlowRuleBatchOperation(
415 Collections.singletonList(new FlowRuleBatchEntry(FlowRuleOperation.ADD, rule)),
416 rule.deviceId(), idGenerator.getNewId()));
417 }
418
419 @Override
420 public void storeBatch(FlowRuleBatchOperation operation) {
421 if (operation.getOperations().isEmpty()) {
422 notifyDelegate(FlowRuleBatchEvent.completed(
423 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
424 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
425 return;
426 }
427
428 DeviceId deviceId = operation.deviceId();
429 NodeId master = mastershipService.getMasterFor(deviceId);
430
431 if (master == null) {
432 log.warn("No master for {} ", deviceId);
433
434 updateStoreInternal(operation);
435
436 notifyDelegate(FlowRuleBatchEvent.completed(
437 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
438 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
439 return;
440 }
441
442 if (Objects.equals(local, master)) {
443 storeBatchInternal(operation);
444 return;
445 }
446
447 log.trace("Forwarding storeBatch to {}, which is the primary (master) for device {}",
448 master, deviceId);
449
450 clusterCommunicator.unicast(operation,
451 APPLY_BATCH_FLOWS,
452 serializer::encode,
453 master)
454 .whenComplete((result, error) -> {
455 if (error != null) {
456 log.warn("Failed to storeBatch: {} to {}", operation, master, error);
457
458 Set<FlowRule> allFailures = operation.getOperations()
459 .stream()
460 .map(op -> op.target())
461 .collect(Collectors.toSet());
462
463 notifyDelegate(FlowRuleBatchEvent.completed(
464 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
465 new CompletedBatchOperation(false, allFailures, deviceId)));
466 }
467 });
468 }
469
470 private void storeBatchInternal(FlowRuleBatchOperation operation) {
471
472 final DeviceId did = operation.deviceId();
473 //final Collection<FlowEntry> ft = flowTable.getFlowEntries(did);
474 Set<FlowRuleBatchEntry> currentOps = updateStoreInternal(operation);
475 if (currentOps.isEmpty()) {
476 batchOperationComplete(FlowRuleBatchEvent.completed(
477 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
478 new CompletedBatchOperation(true, Collections.emptySet(), did)));
479 return;
480 }
481
482 notifyDelegate(FlowRuleBatchEvent.requested(new
483 FlowRuleBatchRequest(operation.id(),
484 currentOps), operation.deviceId()));
485 }
486
487 private Set<FlowRuleBatchEntry> updateStoreInternal(FlowRuleBatchOperation operation) {
488 return operation.getOperations().stream().map(
489 op -> {
490 StoredFlowEntry entry;
491 switch (op.operator()) {
492 case ADD:
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800493 entry = new DefaultFlowEntry(op.target());
494 flowTable.add(entry);
495 return op;
Thomas Vachuska914b0b12018-01-09 11:54:52 -0800496 case MODIFY:
Jon Hallfa132292017-10-24 11:11:24 -0700497 entry = new DefaultFlowEntry(op.target());
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800498 flowTable.update(entry);
Jon Hallfa132292017-10-24 11:11:24 -0700499 return op;
500 case REMOVE:
501 entry = flowTable.getFlowEntry(op.target());
502 if (entry != null) {
Jon Hallfa132292017-10-24 11:11:24 -0700503 entry.setState(FlowEntryState.PENDING_REMOVE);
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800504 flowTable.update(entry);
Jon Hallfa132292017-10-24 11:11:24 -0700505 log.debug("Setting state of rule to pending remove: {}", entry);
506 return op;
507 }
508 break;
Jon Hallfa132292017-10-24 11:11:24 -0700509 default:
510 log.warn("Unknown flow operation operator: {}", op.operator());
511 }
512 return null;
513 }
514 ).filter(Objects::nonNull).collect(Collectors.toSet());
515 }
516
517 @Override
518 public void deleteFlowRule(FlowRule rule) {
519 storeBatch(
520 new FlowRuleBatchOperation(
521 Collections.singletonList(
522 new FlowRuleBatchEntry(
523 FlowRuleOperation.REMOVE,
524 rule)), rule.deviceId(), idGenerator.getNewId()));
525 }
526
527 @Override
528 public FlowRuleEvent pendingFlowRule(FlowEntry rule) {
529 if (mastershipService.isLocalMaster(rule.deviceId())) {
530 StoredFlowEntry stored = flowTable.getFlowEntry(rule);
531 if (stored != null &&
532 stored.state() != FlowEntryState.PENDING_ADD) {
533 stored.setState(FlowEntryState.PENDING_ADD);
534 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
535 }
536 }
537 return null;
538 }
539
540 @Override
541 public FlowRuleEvent addOrUpdateFlowRule(FlowEntry rule) {
542 NodeId master = mastershipService.getMasterFor(rule.deviceId());
543 if (Objects.equals(local, master)) {
544 return addOrUpdateFlowRuleInternal(rule);
545 }
546
547 log.warn("Tried to update FlowRule {} state,"
548 + " while the Node was not the master.", rule);
549 return null;
550 }
551
552 private FlowRuleEvent addOrUpdateFlowRuleInternal(FlowEntry rule) {
553 // check if this new rule is an update to an existing entry
554 StoredFlowEntry stored = flowTable.getFlowEntry(rule);
555 if (stored != null) {
Jon Hallfa132292017-10-24 11:11:24 -0700556 stored.setBytes(rule.bytes());
557 stored.setLife(rule.life(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
558 stored.setLiveType(rule.liveType());
559 stored.setPackets(rule.packets());
560 stored.setLastSeen();
561 if (stored.state() == FlowEntryState.PENDING_ADD) {
562 stored.setState(FlowEntryState.ADDED);
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800563 // Update the flow table to ensure the changes are replicated
564 flowTable.update(stored);
Jon Hallfa132292017-10-24 11:11:24 -0700565 return new FlowRuleEvent(Type.RULE_ADDED, rule);
566 }
567 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
568 }
569
570 // TODO: Confirm if this behavior is correct. See SimpleFlowRuleStore
571 // TODO: also update backup if the behavior is correct.
572 flowTable.add(rule);
573 return null;
574 }
575
576 @Override
577 public FlowRuleEvent removeFlowRule(FlowEntry rule) {
578 final DeviceId deviceId = rule.deviceId();
579 NodeId master = mastershipService.getMasterFor(deviceId);
580
581 if (Objects.equals(local, master)) {
582 // bypass and handle it locally
583 return removeFlowRuleInternal(rule);
584 }
585
586 if (master == null) {
587 log.warn("Failed to removeFlowRule: No master for {}", deviceId);
588 // TODO: revisit if this should be null (="no-op") or Exception
589 return null;
590 }
591
592 log.trace("Forwarding removeFlowRule to {}, which is the master for device {}",
593 master, deviceId);
594
595 return Futures.getUnchecked(clusterCommunicator.sendAndReceive(
596 rule,
597 REMOVE_FLOW_ENTRY,
598 serializer::encode,
599 serializer::decode,
600 master));
601 }
602
603 private FlowRuleEvent removeFlowRuleInternal(FlowEntry rule) {
Jon Hallfa132292017-10-24 11:11:24 -0700604 // This is where one could mark a rule as removed and still keep it in the store.
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800605 final FlowEntry removed = flowTable.remove(rule);
Jon Hallfa132292017-10-24 11:11:24 -0700606 // rule may be partial rule that is missing treatment, we should use rule from store instead
607 return removed != null ? new FlowRuleEvent(RULE_REMOVED, removed) : null;
608 }
609
610 @Override
611 public void purgeFlowRule(DeviceId deviceId) {
612 flowTable.purgeFlowRule(deviceId);
613 }
614
615 @Override
616 public void purgeFlowRules() {
617 flowTable.purgeFlowRules();
618 }
619
620 @Override
621 public void batchOperationComplete(FlowRuleBatchEvent event) {
622 //FIXME: need a per device pending response
623 NodeId nodeId = pendingResponses.remove(event.subject().batchId());
624 if (nodeId == null) {
625 notifyDelegate(event);
626 } else {
627 // TODO check unicast return value
628 clusterCommunicator.unicast(event, REMOTE_APPLY_COMPLETED, serializer::encode, nodeId);
629 //error log: log.warn("Failed to respond to peer for batch operation result");
630 }
631 }
632
633 private final class OnStoreBatch implements ClusterMessageHandler {
634
635 @Override
636 public void handle(final ClusterMessage message) {
637 FlowRuleBatchOperation operation = serializer.decode(message.payload());
638 log.debug("received batch request {}", operation);
639
640 final DeviceId deviceId = operation.deviceId();
641 NodeId master = mastershipService.getMasterFor(deviceId);
642 if (!Objects.equals(local, master)) {
643 Set<FlowRule> failures = new HashSet<>(operation.size());
644 for (FlowRuleBatchEntry op : operation.getOperations()) {
645 failures.add(op.target());
646 }
647 CompletedBatchOperation allFailed = new CompletedBatchOperation(false, failures, deviceId);
648 // This node is no longer the master, respond as all failed.
649 // TODO: we might want to wrap response in envelope
650 // to distinguish sw programming failure and hand over
651 // it make sense in the latter case to retry immediately.
652 message.respond(serializer.encode(allFailed));
653 return;
654 }
655
656 pendingResponses.put(operation.id(), message.sender());
657 storeBatchInternal(operation);
658 }
659 }
660
661 private class BackupOperation {
662 private final NodeId nodeId;
663 private final DeviceId deviceId;
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700664 private final int bucket;
Jon Hallfa132292017-10-24 11:11:24 -0700665
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700666 public BackupOperation(NodeId nodeId, DeviceId deviceId, int bucket) {
Jon Hallfa132292017-10-24 11:11:24 -0700667 this.nodeId = nodeId;
668 this.deviceId = deviceId;
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700669 this.bucket = bucket;
Jon Hallfa132292017-10-24 11:11:24 -0700670 }
671
672 @Override
673 public int hashCode() {
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700674 return Objects.hash(nodeId, deviceId, bucket);
Jon Hallfa132292017-10-24 11:11:24 -0700675 }
676
677 @Override
678 public boolean equals(Object other) {
679 if (other != null && other instanceof BackupOperation) {
680 BackupOperation that = (BackupOperation) other;
681 return this.nodeId.equals(that.nodeId) &&
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700682 this.deviceId.equals(that.deviceId) &&
683 this.bucket == that.bucket;
Jon Hallfa132292017-10-24 11:11:24 -0700684 } else {
685 return false;
686 }
687 }
688 }
689
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700690 private class FlowBucket {
691 private final DeviceId deviceId;
692 private final int bucket;
693 private final Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> table;
694
695 FlowBucket(DeviceId deviceId, int bucket, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> table) {
696 this.deviceId = deviceId;
697 this.bucket = bucket;
698 this.table = table;
699 }
700 }
701
Jon Hallfa132292017-10-24 11:11:24 -0700702 private class InternalFlowTable implements ReplicaInfoEventListener {
703
704 //TODO replace the Map<V,V> with ExtendedSet
705 private final Map<DeviceId, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>
706 flowEntries = Maps.newConcurrentMap();
707
708 private final Map<BackupOperation, Long> lastBackupTimes = Maps.newConcurrentMap();
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700709 private final Map<DeviceId, Map<Integer, Long>> lastUpdateTimes = Maps.newConcurrentMap();
710 private final Map<NodeId, Set<DeviceId>> inFlightUpdates = Maps.newConcurrentMap();
Jon Hallfa132292017-10-24 11:11:24 -0700711
712 @Override
713 public void event(ReplicaInfoEvent event) {
714 eventHandler.execute(() -> handleEvent(event));
715 }
716
717 private void handleEvent(ReplicaInfoEvent event) {
718 DeviceId deviceId = event.subject();
719 if (!mastershipService.isLocalMaster(deviceId)) {
720 return;
721 }
722 if (event.type() == MASTER_CHANGED) {
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700723 for (int bucket = 0; bucket < NUM_BUCKETS; bucket++) {
724 recordUpdate(deviceId, bucket);
725 }
Jon Hallfa132292017-10-24 11:11:24 -0700726 }
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700727 backupSenderExecutor.execute(this::backup);
Jon Hallfa132292017-10-24 11:11:24 -0700728 }
729
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700730 private CompletableFuture<Void> backupFlowEntries(
731 NodeId nodeId, DeviceId deviceId, int bucket, long timestamp) {
732 log.debug("Sending flowEntries in bucket {} for device {} to {} for backup.", bucket, deviceId, nodeId);
733 FlowBucket flowBucket = getFlowBucket(deviceId, bucket);
734 int flowCount = flowBucket.table.entrySet().stream()
735 .mapToInt(e -> e.getValue().values().size()).sum();
736 flowCounts.compute(deviceId, (id, counts) -> {
737 if (counts == null) {
738 counts = Maps.newConcurrentMap();
739 }
740 counts.put(bucket, flowCount);
741 return counts;
Thomas Vachuskaa8e74772018-02-26 11:33:35 -0800742 });
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700743
744 CompletableFuture<Void> future = new CompletableFuture<>();
745 clusterCommunicator.<FlowBucket, Set<FlowId>>
746 sendAndReceive(flowBucket,
Jon Hallfa132292017-10-24 11:11:24 -0700747 FLOW_TABLE_BACKUP,
748 serializer::encode,
749 serializer::decode,
750 nodeId)
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700751 .whenComplete((backedupFlows, error) -> {
752 Set<FlowId> flowsNotBackedUp = error != null ?
753 flowBucket.table.keySet() :
754 Sets.difference(flowBucket.table.keySet(), backedupFlows);
755 if (flowsNotBackedUp.size() > 0) {
756 log.warn("Failed to backup flows: {}. Reason: {}, Node: {}",
757 flowsNotBackedUp, error != null ? error.getMessage() : "none", nodeId);
Jon Hallfa132292017-10-24 11:11:24 -0700758 }
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700759 if (backedupFlows != null) {
760 lastBackupTimes.put(new BackupOperation(nodeId, deviceId, bucket), timestamp);
Jon Hallfa132292017-10-24 11:11:24 -0700761 }
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700762 future.complete(null);
Jon Hallfa132292017-10-24 11:11:24 -0700763 });
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700764 return future;
Jon Hallfa132292017-10-24 11:11:24 -0700765 }
766
767 /**
768 * Returns the flow table for specified device.
769 *
770 * @param deviceId identifier of the device
771 * @return Map representing Flow Table of given device.
772 */
773 private Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> getFlowTable(DeviceId deviceId) {
774 if (persistenceEnabled) {
775 return flowEntries.computeIfAbsent(deviceId, id -> persistenceService
776 .<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>persistentMapBuilder()
777 .withName("FlowTable:" + deviceId.toString())
778 .withSerializer(new Serializer() {
779 @Override
780 public <T> byte[] encode(T object) {
781 return serializer.encode(object);
782 }
783
784 @Override
785 public <T> T decode(byte[] bytes) {
786 return serializer.decode(bytes);
787 }
788
789 @Override
790 public <T> T copy(T object) {
791 return serializer.copy(object);
792 }
793 })
794 .build());
795 } else {
796 return flowEntries.computeIfAbsent(deviceId, id -> Maps.newConcurrentMap());
797 }
798 }
799
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700800 private FlowBucket getFlowBucket(DeviceId deviceId, int bucket) {
Jon Hallfa132292017-10-24 11:11:24 -0700801 if (persistenceEnabled) {
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700802 return new FlowBucket(deviceId, bucket, flowEntries.computeIfAbsent(deviceId, id ->
803 persistenceService.<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>persistentMapBuilder()
Jon Hallfa132292017-10-24 11:11:24 -0700804 .withName("FlowTable:" + deviceId.toString())
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700805 .withSerializer(serializer)
806 .build())
807 .entrySet()
808 .stream()
809 .filter(entry -> isBucket(entry.getKey(), bucket))
810 .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
Jon Hallfa132292017-10-24 11:11:24 -0700811 } else {
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700812 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> copy = Maps.newHashMap();
813 flowEntries.computeIfAbsent(deviceId, id -> Maps.newConcurrentMap())
814 .entrySet()
815 .stream()
816 .filter(entry -> isBucket(entry.getKey(), bucket))
817 .forEach(entry -> {
818 copy.put(entry.getKey(), Maps.newHashMap(entry.getValue()));
819 });
820 return new FlowBucket(deviceId, bucket, copy);
Jon Hallfa132292017-10-24 11:11:24 -0700821 }
822 }
823
824 private Map<StoredFlowEntry, StoredFlowEntry> getFlowEntriesInternal(DeviceId deviceId, FlowId flowId) {
825 return getFlowTable(deviceId).computeIfAbsent(flowId, id -> Maps.newConcurrentMap());
826 }
827
828 private StoredFlowEntry getFlowEntryInternal(FlowRule rule) {
829 return getFlowEntriesInternal(rule.deviceId(), rule.id()).get(rule);
830 }
831
832 private Set<FlowEntry> getFlowEntriesInternal(DeviceId deviceId) {
833 return getFlowTable(deviceId).values().stream()
834 .flatMap(m -> m.values().stream())
835 .collect(Collectors.toSet());
836 }
837
838 public StoredFlowEntry getFlowEntry(FlowRule rule) {
839 return getFlowEntryInternal(rule);
840 }
841
842 public Set<FlowEntry> getFlowEntries(DeviceId deviceId) {
843 return getFlowEntriesInternal(deviceId);
844 }
845
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700846 private boolean isBucket(FlowId flowId, int bucket) {
847 return bucket(flowId) == bucket;
848 }
849
850 private int bucket(FlowId flowId) {
851 return (int) (flowId.id() % NUM_BUCKETS);
852 }
853
854 private void recordUpdate(DeviceId deviceId, int bucket) {
855 lastUpdateTimes.computeIfAbsent(deviceId, id -> Maps.newConcurrentMap())
856 .put(bucket, System.currentTimeMillis());
857 }
858
Jon Hallfa132292017-10-24 11:11:24 -0700859 public void add(FlowEntry rule) {
Devin Limcdca1952018-03-28 18:13:33 -0700860 getFlowEntriesInternal(rule.deviceId(), rule.id())
861 .compute((StoredFlowEntry) rule, (k, stored) -> {
862 return (StoredFlowEntry) rule;
863 });
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700864 recordUpdate(rule.deviceId(), bucket(rule.id()));
Jon Hallfa132292017-10-24 11:11:24 -0700865 }
866
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800867 public void update(FlowEntry rule) {
868 getFlowEntriesInternal(rule.deviceId(), rule.id())
869 .computeIfPresent((StoredFlowEntry) rule, (k, stored) -> {
870 if (rule instanceof DefaultFlowEntry) {
871 DefaultFlowEntry updated = (DefaultFlowEntry) rule;
872 if (stored instanceof DefaultFlowEntry) {
873 DefaultFlowEntry storedEntry = (DefaultFlowEntry) stored;
874 if (updated.created() >= storedEntry.created()) {
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700875 recordUpdate(rule.deviceId(), bucket(rule.id()));
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800876 return updated;
877 } else {
878 log.debug("Trying to update more recent flow entry {} (stored: {})", updated, stored);
879 return stored;
880 }
881 }
882 }
883 return stored;
884 });
885 }
886
887 public FlowEntry remove(FlowEntry rule) {
Jon Hallfa132292017-10-24 11:11:24 -0700888 final AtomicReference<FlowEntry> removedRule = new AtomicReference<>();
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800889 final Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> flowTable = getFlowTable(rule.deviceId());
Jordan Haltermance336f72018-01-16 17:08:09 -0800890 flowTable.computeIfPresent(rule.id(), (flowId, flowEntries) -> {
891 flowEntries.computeIfPresent((StoredFlowEntry) rule, (k, stored) -> {
Jon Hallfa132292017-10-24 11:11:24 -0700892 if (rule instanceof DefaultFlowEntry) {
893 DefaultFlowEntry toRemove = (DefaultFlowEntry) rule;
894 if (stored instanceof DefaultFlowEntry) {
895 DefaultFlowEntry storedEntry = (DefaultFlowEntry) stored;
896 if (toRemove.created() < storedEntry.created()) {
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800897 log.debug("Trying to remove more recent flow entry {} (stored: {})", toRemove, stored);
Jon Hallfa132292017-10-24 11:11:24 -0700898 // the key is not updated, removedRule remains null
899 return stored;
900 }
901 }
902 }
903 removedRule.set(stored);
904 return null;
905 });
Jordan Haltermance336f72018-01-16 17:08:09 -0800906 return flowEntries.isEmpty() ? null : flowEntries;
907 });
Jon Hallfa132292017-10-24 11:11:24 -0700908
909 if (removedRule.get() != null) {
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700910 recordUpdate(rule.deviceId(), bucket(rule.id()));
Jon Hallfa132292017-10-24 11:11:24 -0700911 return removedRule.get();
912 } else {
913 return null;
914 }
915 }
916
917 public void purgeFlowRule(DeviceId deviceId) {
918 flowEntries.remove(deviceId);
919 }
920
921 public void purgeFlowRules() {
922 flowEntries.clear();
923 }
924
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700925 private boolean isMasterNode(DeviceId deviceId) {
926 NodeId master = replicaInfoManager.getReplicaInfoFor(deviceId).master().orElse(null);
927 return Objects.equals(master, clusterService.getLocalNode().id());
928 }
929
930 private boolean isBackupNode(NodeId nodeId, DeviceId deviceId) {
Jon Hallfa132292017-10-24 11:11:24 -0700931 List<NodeId> allPossibleBackupNodes = replicaInfoManager.getReplicaInfoFor(deviceId).backups();
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700932 return allPossibleBackupNodes.indexOf(nodeId) < backupCount;
Jon Hallfa132292017-10-24 11:11:24 -0700933 }
934
935 private void backup() {
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700936 clusterService.getNodes().stream()
937 .filter(node -> !node.id().equals(clusterService.getLocalNode().id()))
938 .forEach(node -> {
939 try {
940 backup(node.id());
941 } catch (Exception e) {
942 log.error("Backup failed.", e);
943 }
Jon Hallfa132292017-10-24 11:11:24 -0700944 });
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700945 }
946
947 private void backup(NodeId nodeId) {
948 for (DeviceId deviceId : flowEntries.keySet()) {
949 if (isMasterNode(deviceId) && isBackupNode(nodeId, deviceId)) {
950 backup(nodeId, deviceId);
951 }
Jon Hallfa132292017-10-24 11:11:24 -0700952 }
953 }
954
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700955 private void backup(NodeId nodeId, DeviceId deviceId) {
956 final long timestamp = System.currentTimeMillis();
957 for (int bucket = 0; bucket < NUM_BUCKETS; bucket++) {
958 long lastBackupTime = lastBackupTimes.getOrDefault(new BackupOperation(nodeId, deviceId, bucket), 0L);
959 long lastUpdateTime = lastUpdateTimes.computeIfAbsent(deviceId, id -> Maps.newConcurrentMap())
960 .getOrDefault(bucket, 0L);
961 if (lastBackupTime < lastUpdateTime && startBackup(nodeId, deviceId)) {
962 backupFlowEntries(nodeId, deviceId, bucket, timestamp)
963 .thenRunAsync(() -> {
964 finishBackup(nodeId, deviceId);
965 backup(nodeId);
966 }, backupSenderExecutor);
967 return;
968 }
969 }
970 }
971
972 private boolean startBackup(NodeId nodeId, DeviceId deviceId) {
973 return inFlightUpdates.computeIfAbsent(nodeId, id -> Sets.newConcurrentHashSet()).add(deviceId);
974 }
975
976 private void finishBackup(NodeId nodeId, DeviceId deviceId) {
977 inFlightUpdates.computeIfAbsent(nodeId, id -> Sets.newConcurrentHashSet()).remove(deviceId);
978 }
979
980 private Set<FlowId> onBackupReceipt(FlowBucket bucket) {
981 log.debug("Received flowEntries for {} bucket {} to backup", bucket.deviceId, bucket.bucket);
982 Set<FlowId> backedupFlows = Sets.newHashSet();
Jon Hallfa132292017-10-24 11:11:24 -0700983 try {
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700984 // Only process those devices are that not managed by the local node.
985 NodeId master = replicaInfoManager.getReplicaInfoFor(bucket.deviceId).master().orElse(null);
986 if (!Objects.equals(local, master)) {
987 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> backupFlowTable = getFlowTable(bucket.deviceId);
988 backupFlowTable.putAll(bucket.table);
989 backupFlowTable.entrySet()
990 .removeIf(entry -> isBucket(entry.getKey(), bucket.bucket)
991 && !bucket.table.containsKey(entry.getKey()));
992 backedupFlows.addAll(bucket.table.keySet());
993 }
Jon Hallfa132292017-10-24 11:11:24 -0700994 } catch (Exception e) {
995 log.warn("Failure processing backup request", e);
996 }
Jordan Haltermanb2f57952018-03-21 12:52:37 -0700997 return backedupFlows;
Jon Hallfa132292017-10-24 11:11:24 -0700998 }
999 }
1000
1001 @Override
1002 public FlowRuleEvent updateTableStatistics(DeviceId deviceId,
1003 List<TableStatisticsEntry> tableStats) {
1004 deviceTableStats.put(deviceId, tableStats);
1005 return null;
1006 }
1007
1008 @Override
1009 public Iterable<TableStatisticsEntry> getTableStatistics(DeviceId deviceId) {
1010 NodeId master = mastershipService.getMasterFor(deviceId);
1011
1012 if (master == null) {
1013 log.debug("Failed to getTableStats: No master for {}", deviceId);
1014 return Collections.emptyList();
1015 }
1016
1017 List<TableStatisticsEntry> tableStats = deviceTableStats.get(deviceId);
1018 if (tableStats == null) {
1019 return Collections.emptyList();
1020 }
1021 return ImmutableList.copyOf(tableStats);
1022 }
1023
1024 @Override
1025 public long getActiveFlowRuleCount(DeviceId deviceId) {
1026 return Streams.stream(getTableStatistics(deviceId))
1027 .mapToLong(TableStatisticsEntry::activeFlowEntries)
1028 .sum();
1029 }
1030
1031 private class InternalTableStatsListener
1032 implements EventuallyConsistentMapListener<DeviceId, List<TableStatisticsEntry>> {
1033 @Override
1034 public void event(EventuallyConsistentMapEvent<DeviceId,
1035 List<TableStatisticsEntry>> event) {
1036 //TODO: Generate an event to listeners (do we need?)
1037 }
1038 }
1039}