blob: 204f98f403220ddbe10414cd7a52c2b69ce85d50 [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
Jordan Halterman7ae81b82018-06-12 11:23:33 -070034import com.google.common.collect.ImmutableList;
35import com.google.common.collect.Maps;
36import com.google.common.collect.Sets;
Jon Hallfa132292017-10-24 11:11:24 -070037import com.google.common.collect.Streams;
Jordan Halterman7ae81b82018-06-12 11:23:33 -070038import com.google.common.util.concurrent.Futures;
Jon Hallfa132292017-10-24 11:11:24 -070039import org.apache.felix.scr.annotations.Activate;
40import org.apache.felix.scr.annotations.Component;
41import org.apache.felix.scr.annotations.Deactivate;
42import org.apache.felix.scr.annotations.Modified;
43import org.apache.felix.scr.annotations.Property;
44import org.apache.felix.scr.annotations.Reference;
45import org.apache.felix.scr.annotations.ReferenceCardinality;
46import org.apache.felix.scr.annotations.Service;
47import org.onlab.util.KryoNamespace;
48import org.onlab.util.Tools;
49import org.onosproject.cfg.ComponentConfigService;
50import org.onosproject.cluster.ClusterService;
51import org.onosproject.cluster.NodeId;
52import org.onosproject.core.CoreService;
53import org.onosproject.core.IdGenerator;
54import org.onosproject.mastership.MastershipService;
55import org.onosproject.net.DeviceId;
56import org.onosproject.net.device.DeviceService;
57import org.onosproject.net.flow.CompletedBatchOperation;
58import org.onosproject.net.flow.DefaultFlowEntry;
59import org.onosproject.net.flow.FlowEntry;
60import org.onosproject.net.flow.FlowEntry.FlowEntryState;
61import org.onosproject.net.flow.FlowId;
62import org.onosproject.net.flow.FlowRule;
Jon Hallfa132292017-10-24 11:11:24 -070063import org.onosproject.net.flow.FlowRuleEvent;
64import org.onosproject.net.flow.FlowRuleEvent.Type;
65import org.onosproject.net.flow.FlowRuleService;
66import org.onosproject.net.flow.FlowRuleStore;
67import org.onosproject.net.flow.FlowRuleStoreDelegate;
68import org.onosproject.net.flow.StoredFlowEntry;
69import org.onosproject.net.flow.TableStatisticsEntry;
Jordan Halterman7ae81b82018-06-12 11:23:33 -070070import org.onosproject.net.flow.oldbatch.FlowRuleBatchEntry;
71import org.onosproject.net.flow.oldbatch.FlowRuleBatchEntry.FlowRuleOperation;
72import org.onosproject.net.flow.oldbatch.FlowRuleBatchEvent;
73import org.onosproject.net.flow.oldbatch.FlowRuleBatchOperation;
74import org.onosproject.net.flow.oldbatch.FlowRuleBatchRequest;
Jon Hallfa132292017-10-24 11:11:24 -070075import org.onosproject.persistence.PersistenceService;
76import org.onosproject.store.AbstractStore;
77import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
78import org.onosproject.store.cluster.messaging.ClusterMessage;
79import org.onosproject.store.cluster.messaging.ClusterMessageHandler;
80import org.onosproject.store.flow.ReplicaInfoEvent;
81import org.onosproject.store.flow.ReplicaInfoEventListener;
82import org.onosproject.store.flow.ReplicaInfoService;
83import org.onosproject.store.impl.MastershipBasedTimestamp;
84import org.onosproject.store.serializers.KryoNamespaces;
85import org.onosproject.store.service.EventuallyConsistentMap;
86import org.onosproject.store.service.EventuallyConsistentMapEvent;
87import org.onosproject.store.service.EventuallyConsistentMapListener;
88import org.onosproject.store.service.Serializer;
89import org.onosproject.store.service.StorageService;
90import org.onosproject.store.service.WallClockTimestamp;
91import org.osgi.service.component.ComponentContext;
92import org.slf4j.Logger;
93
Jon Hallfa132292017-10-24 11:11:24 -070094import static com.google.common.base.Strings.isNullOrEmpty;
95import static org.onlab.util.Tools.get;
96import static org.onlab.util.Tools.groupedThreads;
97import static org.onosproject.net.flow.FlowRuleEvent.Type.RULE_REMOVED;
98import static org.onosproject.store.flow.ReplicaInfoEvent.Type.MASTER_CHANGED;
99import static org.onosproject.store.flow.impl.ECFlowRuleStoreMessageSubjects.APPLY_BATCH_FLOWS;
100import static org.onosproject.store.flow.impl.ECFlowRuleStoreMessageSubjects.FLOW_TABLE_BACKUP;
101import static org.onosproject.store.flow.impl.ECFlowRuleStoreMessageSubjects.GET_DEVICE_FLOW_ENTRIES;
102import static org.onosproject.store.flow.impl.ECFlowRuleStoreMessageSubjects.GET_FLOW_ENTRY;
103import static org.onosproject.store.flow.impl.ECFlowRuleStoreMessageSubjects.REMOTE_APPLY_COMPLETED;
104import static org.onosproject.store.flow.impl.ECFlowRuleStoreMessageSubjects.REMOVE_FLOW_ENTRY;
105import static org.slf4j.LoggerFactory.getLogger;
106
107/**
108 * Manages inventory of flow rules using a distributed state management protocol.
109 */
Thomas Vachuska71026b22018-01-05 16:01:44 -0800110@Component(immediate = true)
Jon Hallfa132292017-10-24 11:11:24 -0700111@Service
112public class ECFlowRuleStore
113 extends AbstractStore<FlowRuleBatchEvent, FlowRuleStoreDelegate>
114 implements FlowRuleStore {
115
116 private final Logger log = getLogger(getClass());
117
118 private static final int MESSAGE_HANDLER_THREAD_POOL_SIZE = 8;
119 private static final int DEFAULT_MAX_BACKUP_COUNT = 2;
120 private static final boolean DEFAULT_PERSISTENCE_ENABLED = false;
121 private static final int DEFAULT_BACKUP_PERIOD_MILLIS = 2000;
122 private static final long FLOW_RULE_STORE_TIMEOUT_MILLIS = 5000;
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700123 private static final int NUM_BUCKETS = 1024;
Jon Hallfa132292017-10-24 11:11:24 -0700124
125 @Property(name = "msgHandlerPoolSize", intValue = MESSAGE_HANDLER_THREAD_POOL_SIZE,
126 label = "Number of threads in the message handler pool")
127 private int msgHandlerPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
128
129 @Property(name = "backupPeriod", intValue = DEFAULT_BACKUP_PERIOD_MILLIS,
130 label = "Delay in ms between successive backup runs")
131 private int backupPeriod = DEFAULT_BACKUP_PERIOD_MILLIS;
132 @Property(name = "persistenceEnabled", boolValue = false,
133 label = "Indicates whether or not changes in the flow table should be persisted to disk.")
134 private boolean persistenceEnabled = DEFAULT_PERSISTENCE_ENABLED;
135
136 @Property(name = "backupCount", intValue = DEFAULT_MAX_BACKUP_COUNT,
137 label = "Max number of backup copies for each device")
138 private volatile int backupCount = DEFAULT_MAX_BACKUP_COUNT;
139
140 private InternalFlowTable flowTable = new InternalFlowTable();
141
142 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
143 protected ReplicaInfoService replicaInfoManager;
144
145 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
146 protected ClusterCommunicationService clusterCommunicator;
147
148 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
149 protected ClusterService clusterService;
150
151 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
152 protected DeviceService deviceService;
153
154 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
155 protected CoreService coreService;
156
157 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
158 protected ComponentConfigService configService;
159
160 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
161 protected MastershipService mastershipService;
162
163 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
164 protected PersistenceService persistenceService;
165
166 private Map<Long, NodeId> pendingResponses = Maps.newConcurrentMap();
167 private ExecutorService messageHandlingExecutor;
168 private ExecutorService eventHandler;
169
170 private ScheduledFuture<?> backupTask;
171 private final ScheduledExecutorService backupSenderExecutor =
172 Executors.newSingleThreadScheduledExecutor(groupedThreads("onos/flow", "backup-sender", log));
173
174 private EventuallyConsistentMap<DeviceId, List<TableStatisticsEntry>> deviceTableStats;
175 private final EventuallyConsistentMapListener<DeviceId, List<TableStatisticsEntry>> tableStatsListener =
176 new InternalTableStatsListener();
177
178 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
179 protected StorageService storageService;
180
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700181 protected final Serializer serializer = Serializer.using(KryoNamespace.newBuilder()
182 .register(KryoNamespaces.API)
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700183 .register(BucketId.class)
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700184 .register(FlowBucket.class)
185 .build());
Jon Hallfa132292017-10-24 11:11:24 -0700186
187 protected final KryoNamespace.Builder serializerBuilder = KryoNamespace.newBuilder()
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700188 .register(KryoNamespaces.API)
189 .register(BucketId.class)
190 .register(MastershipBasedTimestamp.class);
Jon Hallfa132292017-10-24 11:11:24 -0700191
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700192 private EventuallyConsistentMap<BucketId, Integer> flowCounts;
Jon Hallfa132292017-10-24 11:11:24 -0700193
194 private IdGenerator idGenerator;
195 private NodeId local;
196
197 @Activate
198 public void activate(ComponentContext context) {
199 configService.registerProperties(getClass());
200
201 idGenerator = coreService.getIdGenerator(FlowRuleService.FLOW_OP_TOPIC);
202
203 local = clusterService.getLocalNode().id();
204
205 eventHandler = Executors.newSingleThreadExecutor(
206 groupedThreads("onos/flow", "event-handler", log));
207 messageHandlingExecutor = Executors.newFixedThreadPool(
208 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers", log));
209
210 registerMessageHandlers(messageHandlingExecutor);
211
212 replicaInfoManager.addListener(flowTable);
213 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
214 flowTable::backup,
215 0,
216 backupPeriod,
217 TimeUnit.MILLISECONDS);
218
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700219 flowCounts = storageService.<BucketId, Integer>eventuallyConsistentMapBuilder()
Thomas Vachuskaa8e74772018-02-26 11:33:35 -0800220 .withName("onos-flow-counts")
221 .withSerializer(serializerBuilder)
222 .withAntiEntropyPeriod(5, TimeUnit.SECONDS)
223 .withTimestampProvider((k, v) -> new WallClockTimestamp())
224 .withTombstonesDisabled()
225 .build();
226
Jon Hallfa132292017-10-24 11:11:24 -0700227 deviceTableStats = storageService.<DeviceId, List<TableStatisticsEntry>>eventuallyConsistentMapBuilder()
228 .withName("onos-flow-table-stats")
229 .withSerializer(serializerBuilder)
230 .withAntiEntropyPeriod(5, TimeUnit.SECONDS)
231 .withTimestampProvider((k, v) -> new WallClockTimestamp())
232 .withTombstonesDisabled()
233 .build();
234 deviceTableStats.addListener(tableStatsListener);
235
236 logConfig("Started");
237 }
238
239 @Deactivate
240 public void deactivate(ComponentContext context) {
241 replicaInfoManager.removeListener(flowTable);
242 backupTask.cancel(true);
243 configService.unregisterProperties(getClass(), false);
244 unregisterMessageHandlers();
245 deviceTableStats.removeListener(tableStatsListener);
246 deviceTableStats.destroy();
247 eventHandler.shutdownNow();
248 messageHandlingExecutor.shutdownNow();
249 backupSenderExecutor.shutdownNow();
250 log.info("Stopped");
251 }
252
253 @SuppressWarnings("rawtypes")
254 @Modified
255 public void modified(ComponentContext context) {
256 if (context == null) {
257 logConfig("Default config");
258 return;
259 }
260
261 Dictionary properties = context.getProperties();
262 int newPoolSize;
263 int newBackupPeriod;
264 int newBackupCount;
265 try {
266 String s = get(properties, "msgHandlerPoolSize");
267 newPoolSize = isNullOrEmpty(s) ? msgHandlerPoolSize : Integer.parseInt(s.trim());
268
269 s = get(properties, "backupPeriod");
270 newBackupPeriod = isNullOrEmpty(s) ? backupPeriod : Integer.parseInt(s.trim());
271
272 s = get(properties, "backupCount");
273 newBackupCount = isNullOrEmpty(s) ? backupCount : Integer.parseInt(s.trim());
274 } catch (NumberFormatException | ClassCastException e) {
275 newPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
276 newBackupPeriod = DEFAULT_BACKUP_PERIOD_MILLIS;
277 newBackupCount = DEFAULT_MAX_BACKUP_COUNT;
278 }
279
280 boolean restartBackupTask = false;
281
282 if (newBackupPeriod != backupPeriod) {
283 backupPeriod = newBackupPeriod;
284 restartBackupTask = true;
285 }
286 if (restartBackupTask) {
287 if (backupTask != null) {
288 // cancel previously running task
289 backupTask.cancel(false);
290 }
291 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
292 flowTable::backup,
293 0,
294 backupPeriod,
295 TimeUnit.MILLISECONDS);
296 }
297 if (newPoolSize != msgHandlerPoolSize) {
298 msgHandlerPoolSize = newPoolSize;
299 ExecutorService oldMsgHandler = messageHandlingExecutor;
300 messageHandlingExecutor = Executors.newFixedThreadPool(
301 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers", log));
302
303 // replace previously registered handlers.
304 registerMessageHandlers(messageHandlingExecutor);
305 oldMsgHandler.shutdown();
306 }
307 if (backupCount != newBackupCount) {
308 backupCount = newBackupCount;
309 }
310 logConfig("Reconfigured");
311 }
312
313 private void registerMessageHandlers(ExecutorService executor) {
314
315 clusterCommunicator.addSubscriber(APPLY_BATCH_FLOWS, new OnStoreBatch(), executor);
316 clusterCommunicator.<FlowRuleBatchEvent>addSubscriber(
317 REMOTE_APPLY_COMPLETED, serializer::decode, this::notifyDelegate, executor);
318 clusterCommunicator.addSubscriber(
319 GET_FLOW_ENTRY, serializer::decode, flowTable::getFlowEntry, serializer::encode, executor);
320 clusterCommunicator.addSubscriber(
321 GET_DEVICE_FLOW_ENTRIES, serializer::decode, flowTable::getFlowEntries, serializer::encode, executor);
322 clusterCommunicator.addSubscriber(
323 REMOVE_FLOW_ENTRY, serializer::decode, this::removeFlowRuleInternal, serializer::encode, executor);
324 clusterCommunicator.addSubscriber(
Jordan Halterman7ae81b82018-06-12 11:23:33 -0700325 FLOW_TABLE_BACKUP, serializer::decode, flowTable::onBackup, serializer::encode, executor);
Jon Hallfa132292017-10-24 11:11:24 -0700326 }
327
328 private void unregisterMessageHandlers() {
329 clusterCommunicator.removeSubscriber(REMOVE_FLOW_ENTRY);
330 clusterCommunicator.removeSubscriber(GET_DEVICE_FLOW_ENTRIES);
331 clusterCommunicator.removeSubscriber(GET_FLOW_ENTRY);
332 clusterCommunicator.removeSubscriber(APPLY_BATCH_FLOWS);
333 clusterCommunicator.removeSubscriber(REMOTE_APPLY_COMPLETED);
334 clusterCommunicator.removeSubscriber(FLOW_TABLE_BACKUP);
335 }
336
337 private void logConfig(String prefix) {
338 log.info("{} with msgHandlerPoolSize = {}; backupPeriod = {}, backupCount = {}",
339 prefix, msgHandlerPoolSize, backupPeriod, backupCount);
340 }
341
Jon Hallfa132292017-10-24 11:11:24 -0700342 @Override
343 public int getFlowRuleCount() {
344 return Streams.stream(deviceService.getDevices()).parallel()
Thomas Vachuskaa8e74772018-02-26 11:33:35 -0800345 .mapToInt(device -> getFlowRuleCount(device.id()))
346 .sum();
347 }
348
349 @Override
350 public int getFlowRuleCount(DeviceId deviceId) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700351 return flowCounts.entrySet().stream()
352 .filter(entry -> entry.getKey().deviceId().equals(deviceId))
353 .mapToInt(entry -> entry.getValue())
354 .sum();
Jon Hallfa132292017-10-24 11:11:24 -0700355 }
356
357 @Override
358 public FlowEntry getFlowEntry(FlowRule rule) {
359 NodeId master = mastershipService.getMasterFor(rule.deviceId());
360
361 if (master == null) {
362 log.debug("Failed to getFlowEntry: No master for {}", rule.deviceId());
363 return null;
364 }
365
366 if (Objects.equals(local, master)) {
367 return flowTable.getFlowEntry(rule);
368 }
369
370 log.trace("Forwarding getFlowEntry to {}, which is the primary (master) for device {}",
371 master, rule.deviceId());
372
373 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(rule,
374 ECFlowRuleStoreMessageSubjects.GET_FLOW_ENTRY,
375 serializer::encode,
376 serializer::decode,
377 master),
378 FLOW_RULE_STORE_TIMEOUT_MILLIS,
379 TimeUnit.MILLISECONDS,
380 null);
381 }
382
383 @Override
384 public Iterable<FlowEntry> getFlowEntries(DeviceId deviceId) {
385 NodeId master = mastershipService.getMasterFor(deviceId);
386
387 if (master == null) {
388 log.debug("Failed to getFlowEntries: No master for {}", deviceId);
389 return Collections.emptyList();
390 }
391
392 if (Objects.equals(local, master)) {
393 return flowTable.getFlowEntries(deviceId);
394 }
395
396 log.trace("Forwarding getFlowEntries to {}, which is the primary (master) for device {}",
397 master, deviceId);
398
399 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(deviceId,
400 ECFlowRuleStoreMessageSubjects.GET_DEVICE_FLOW_ENTRIES,
401 serializer::encode,
402 serializer::decode,
403 master),
404 FLOW_RULE_STORE_TIMEOUT_MILLIS,
405 TimeUnit.MILLISECONDS,
406 Collections.emptyList());
407 }
408
409 @Override
410 public void storeFlowRule(FlowRule rule) {
411 storeBatch(new FlowRuleBatchOperation(
412 Collections.singletonList(new FlowRuleBatchEntry(FlowRuleOperation.ADD, rule)),
413 rule.deviceId(), idGenerator.getNewId()));
414 }
415
416 @Override
417 public void storeBatch(FlowRuleBatchOperation operation) {
418 if (operation.getOperations().isEmpty()) {
419 notifyDelegate(FlowRuleBatchEvent.completed(
420 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
421 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
422 return;
423 }
424
425 DeviceId deviceId = operation.deviceId();
426 NodeId master = mastershipService.getMasterFor(deviceId);
427
428 if (master == null) {
429 log.warn("No master for {} ", deviceId);
430
431 updateStoreInternal(operation);
432
433 notifyDelegate(FlowRuleBatchEvent.completed(
434 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
435 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
436 return;
437 }
438
439 if (Objects.equals(local, master)) {
440 storeBatchInternal(operation);
441 return;
442 }
443
444 log.trace("Forwarding storeBatch to {}, which is the primary (master) for device {}",
445 master, deviceId);
446
447 clusterCommunicator.unicast(operation,
448 APPLY_BATCH_FLOWS,
449 serializer::encode,
450 master)
451 .whenComplete((result, error) -> {
452 if (error != null) {
453 log.warn("Failed to storeBatch: {} to {}", operation, master, error);
454
455 Set<FlowRule> allFailures = operation.getOperations()
456 .stream()
457 .map(op -> op.target())
458 .collect(Collectors.toSet());
459
460 notifyDelegate(FlowRuleBatchEvent.completed(
461 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
462 new CompletedBatchOperation(false, allFailures, deviceId)));
463 }
464 });
465 }
466
467 private void storeBatchInternal(FlowRuleBatchOperation operation) {
468
469 final DeviceId did = operation.deviceId();
470 //final Collection<FlowEntry> ft = flowTable.getFlowEntries(did);
471 Set<FlowRuleBatchEntry> currentOps = updateStoreInternal(operation);
472 if (currentOps.isEmpty()) {
473 batchOperationComplete(FlowRuleBatchEvent.completed(
474 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
475 new CompletedBatchOperation(true, Collections.emptySet(), did)));
476 return;
477 }
478
479 notifyDelegate(FlowRuleBatchEvent.requested(new
480 FlowRuleBatchRequest(operation.id(),
481 currentOps), operation.deviceId()));
482 }
483
484 private Set<FlowRuleBatchEntry> updateStoreInternal(FlowRuleBatchOperation operation) {
485 return operation.getOperations().stream().map(
486 op -> {
487 StoredFlowEntry entry;
488 switch (op.operator()) {
489 case ADD:
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800490 entry = new DefaultFlowEntry(op.target());
491 flowTable.add(entry);
492 return op;
Thomas Vachuska914b0b12018-01-09 11:54:52 -0800493 case MODIFY:
Jon Hallfa132292017-10-24 11:11:24 -0700494 entry = new DefaultFlowEntry(op.target());
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800495 flowTable.update(entry);
Jon Hallfa132292017-10-24 11:11:24 -0700496 return op;
497 case REMOVE:
498 entry = flowTable.getFlowEntry(op.target());
499 if (entry != null) {
Jon Hallfa132292017-10-24 11:11:24 -0700500 entry.setState(FlowEntryState.PENDING_REMOVE);
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800501 flowTable.update(entry);
Jon Hallfa132292017-10-24 11:11:24 -0700502 log.debug("Setting state of rule to pending remove: {}", entry);
503 return op;
504 }
505 break;
Jon Hallfa132292017-10-24 11:11:24 -0700506 default:
507 log.warn("Unknown flow operation operator: {}", op.operator());
508 }
509 return null;
510 }
511 ).filter(Objects::nonNull).collect(Collectors.toSet());
512 }
513
514 @Override
515 public void deleteFlowRule(FlowRule rule) {
516 storeBatch(
517 new FlowRuleBatchOperation(
518 Collections.singletonList(
519 new FlowRuleBatchEntry(
520 FlowRuleOperation.REMOVE,
521 rule)), rule.deviceId(), idGenerator.getNewId()));
522 }
523
524 @Override
525 public FlowRuleEvent pendingFlowRule(FlowEntry rule) {
526 if (mastershipService.isLocalMaster(rule.deviceId())) {
527 StoredFlowEntry stored = flowTable.getFlowEntry(rule);
528 if (stored != null &&
529 stored.state() != FlowEntryState.PENDING_ADD) {
530 stored.setState(FlowEntryState.PENDING_ADD);
531 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
532 }
533 }
534 return null;
535 }
536
537 @Override
538 public FlowRuleEvent addOrUpdateFlowRule(FlowEntry rule) {
539 NodeId master = mastershipService.getMasterFor(rule.deviceId());
540 if (Objects.equals(local, master)) {
541 return addOrUpdateFlowRuleInternal(rule);
542 }
543
544 log.warn("Tried to update FlowRule {} state,"
545 + " while the Node was not the master.", rule);
546 return null;
547 }
548
549 private FlowRuleEvent addOrUpdateFlowRuleInternal(FlowEntry rule) {
550 // check if this new rule is an update to an existing entry
551 StoredFlowEntry stored = flowTable.getFlowEntry(rule);
552 if (stored != null) {
Jon Hallfa132292017-10-24 11:11:24 -0700553 stored.setBytes(rule.bytes());
554 stored.setLife(rule.life(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
555 stored.setLiveType(rule.liveType());
556 stored.setPackets(rule.packets());
557 stored.setLastSeen();
558 if (stored.state() == FlowEntryState.PENDING_ADD) {
559 stored.setState(FlowEntryState.ADDED);
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800560 // Update the flow table to ensure the changes are replicated
561 flowTable.update(stored);
Jon Hallfa132292017-10-24 11:11:24 -0700562 return new FlowRuleEvent(Type.RULE_ADDED, rule);
563 }
564 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
565 }
566
567 // TODO: Confirm if this behavior is correct. See SimpleFlowRuleStore
568 // TODO: also update backup if the behavior is correct.
569 flowTable.add(rule);
570 return null;
571 }
572
573 @Override
574 public FlowRuleEvent removeFlowRule(FlowEntry rule) {
575 final DeviceId deviceId = rule.deviceId();
576 NodeId master = mastershipService.getMasterFor(deviceId);
577
578 if (Objects.equals(local, master)) {
579 // bypass and handle it locally
580 return removeFlowRuleInternal(rule);
581 }
582
583 if (master == null) {
584 log.warn("Failed to removeFlowRule: No master for {}", deviceId);
585 // TODO: revisit if this should be null (="no-op") or Exception
586 return null;
587 }
588
589 log.trace("Forwarding removeFlowRule to {}, which is the master for device {}",
590 master, deviceId);
591
592 return Futures.getUnchecked(clusterCommunicator.sendAndReceive(
593 rule,
594 REMOVE_FLOW_ENTRY,
595 serializer::encode,
596 serializer::decode,
597 master));
598 }
599
600 private FlowRuleEvent removeFlowRuleInternal(FlowEntry rule) {
Jon Hallfa132292017-10-24 11:11:24 -0700601 // This is where one could mark a rule as removed and still keep it in the store.
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800602 final FlowEntry removed = flowTable.remove(rule);
Jon Hallfa132292017-10-24 11:11:24 -0700603 // rule may be partial rule that is missing treatment, we should use rule from store instead
604 return removed != null ? new FlowRuleEvent(RULE_REMOVED, removed) : null;
605 }
606
607 @Override
608 public void purgeFlowRule(DeviceId deviceId) {
609 flowTable.purgeFlowRule(deviceId);
610 }
611
612 @Override
613 public void purgeFlowRules() {
614 flowTable.purgeFlowRules();
615 }
616
617 @Override
618 public void batchOperationComplete(FlowRuleBatchEvent event) {
619 //FIXME: need a per device pending response
620 NodeId nodeId = pendingResponses.remove(event.subject().batchId());
621 if (nodeId == null) {
622 notifyDelegate(event);
623 } else {
624 // TODO check unicast return value
625 clusterCommunicator.unicast(event, REMOTE_APPLY_COMPLETED, serializer::encode, nodeId);
626 //error log: log.warn("Failed to respond to peer for batch operation result");
627 }
628 }
629
630 private final class OnStoreBatch implements ClusterMessageHandler {
631
632 @Override
633 public void handle(final ClusterMessage message) {
634 FlowRuleBatchOperation operation = serializer.decode(message.payload());
635 log.debug("received batch request {}", operation);
636
637 final DeviceId deviceId = operation.deviceId();
638 NodeId master = mastershipService.getMasterFor(deviceId);
639 if (!Objects.equals(local, master)) {
640 Set<FlowRule> failures = new HashSet<>(operation.size());
641 for (FlowRuleBatchEntry op : operation.getOperations()) {
642 failures.add(op.target());
643 }
644 CompletedBatchOperation allFailed = new CompletedBatchOperation(false, failures, deviceId);
645 // This node is no longer the master, respond as all failed.
646 // TODO: we might want to wrap response in envelope
647 // to distinguish sw programming failure and hand over
648 // it make sense in the latter case to retry immediately.
649 message.respond(serializer.encode(allFailed));
650 return;
651 }
652
653 pendingResponses.put(operation.id(), message.sender());
654 storeBatchInternal(operation);
655 }
656 }
657
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700658 /**
659 * Represents a backup to a of a distinct bucket to a distinct node.
660 */
Jon Hallfa132292017-10-24 11:11:24 -0700661 private class BackupOperation {
662 private final NodeId nodeId;
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700663 private final BucketId bucketId;
Jon Hallfa132292017-10-24 11:11:24 -0700664
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700665 BackupOperation(NodeId nodeId, BucketId bucketId) {
Jon Hallfa132292017-10-24 11:11:24 -0700666 this.nodeId = nodeId;
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700667 this.bucketId = bucketId;
668 }
669
670 NodeId nodeId() {
671 return nodeId;
672 }
673
674 BucketId bucketId() {
675 return bucketId;
Jon Hallfa132292017-10-24 11:11:24 -0700676 }
677
678 @Override
679 public int hashCode() {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700680 return Objects.hash(nodeId, bucketId);
Jon Hallfa132292017-10-24 11:11:24 -0700681 }
682
683 @Override
684 public boolean equals(Object other) {
685 if (other != null && other instanceof BackupOperation) {
686 BackupOperation that = (BackupOperation) other;
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700687 return this.nodeId.equals(that.nodeId)
688 && this.bucketId.equals(that.bucketId);
Jon Hallfa132292017-10-24 11:11:24 -0700689 }
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700690 return false;
Jon Hallfa132292017-10-24 11:11:24 -0700691 }
692 }
693
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700694 /**
695 * Represents a distinct device flow bucket.
696 */
697 private class BucketId {
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700698 private final DeviceId deviceId;
699 private final int bucket;
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700700
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700701 BucketId(DeviceId deviceId, int bucket) {
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700702 this.deviceId = deviceId;
703 this.bucket = bucket;
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700704 }
705
706 DeviceId deviceId() {
707 return deviceId;
708 }
709
710 int bucket() {
711 return bucket;
712 }
713
714 @Override
715 public int hashCode() {
716 return Objects.hash(deviceId, bucket);
717 }
718
719 @Override
720 public boolean equals(Object other) {
721 if (other != null && other instanceof BucketId) {
722 BucketId that = (BucketId) other;
723 return this.deviceId.equals(that.deviceId)
724 && this.bucket == that.bucket;
725 }
726 return false;
727 }
728 }
729
730 /**
731 * Container for flows in a specific bucket.
732 */
733 private class FlowBucket {
734 private final BucketId bucketId;
735 private final Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> table;
736
737 BucketId bucketId() {
738 return bucketId;
739 }
740
741 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> table() {
742 return table;
743 }
744
745 FlowBucket(BucketId bucketId, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> table) {
746 this.bucketId = bucketId;
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700747 this.table = table;
748 }
749 }
750
Jon Hallfa132292017-10-24 11:11:24 -0700751 private class InternalFlowTable implements ReplicaInfoEventListener {
752
753 //TODO replace the Map<V,V> with ExtendedSet
754 private final Map<DeviceId, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>
755 flowEntries = Maps.newConcurrentMap();
756
757 private final Map<BackupOperation, Long> lastBackupTimes = Maps.newConcurrentMap();
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700758 private final Map<BucketId, Long> lastUpdateTimes = Maps.newConcurrentMap();
759 private final Set<BackupOperation> inFlightUpdates = Sets.newConcurrentHashSet();
Jon Hallfa132292017-10-24 11:11:24 -0700760
761 @Override
762 public void event(ReplicaInfoEvent event) {
763 eventHandler.execute(() -> handleEvent(event));
764 }
765
766 private void handleEvent(ReplicaInfoEvent event) {
767 DeviceId deviceId = event.subject();
768 if (!mastershipService.isLocalMaster(deviceId)) {
769 return;
770 }
771 if (event.type() == MASTER_CHANGED) {
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700772 for (int bucket = 0; bucket < NUM_BUCKETS; bucket++) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700773 recordUpdate(new BucketId(deviceId, bucket));
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700774 }
Jon Hallfa132292017-10-24 11:11:24 -0700775 }
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700776 backupSenderExecutor.execute(this::backup);
Jon Hallfa132292017-10-24 11:11:24 -0700777 }
778
Jon Hallfa132292017-10-24 11:11:24 -0700779 /**
780 * Returns the flow table for specified device.
781 *
782 * @param deviceId identifier of the device
783 * @return Map representing Flow Table of given device.
784 */
785 private Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> getFlowTable(DeviceId deviceId) {
786 if (persistenceEnabled) {
787 return flowEntries.computeIfAbsent(deviceId, id -> persistenceService
788 .<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>persistentMapBuilder()
789 .withName("FlowTable:" + deviceId.toString())
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700790 .withSerializer(serializer)
Jon Hallfa132292017-10-24 11:11:24 -0700791 .build());
792 } else {
793 return flowEntries.computeIfAbsent(deviceId, id -> Maps.newConcurrentMap());
794 }
795 }
796
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700797 private FlowBucket getFlowBucket(BucketId bucketId) {
Jon Hallfa132292017-10-24 11:11:24 -0700798 if (persistenceEnabled) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700799 return new FlowBucket(bucketId, flowEntries.computeIfAbsent(bucketId.deviceId(), id ->
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700800 persistenceService.<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>persistentMapBuilder()
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700801 .withName("FlowTable:" + bucketId.deviceId().toString())
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700802 .withSerializer(serializer)
803 .build())
804 .entrySet()
805 .stream()
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700806 .filter(entry -> isInBucket(entry.getKey(), bucketId.bucket()))
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700807 .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
Jon Hallfa132292017-10-24 11:11:24 -0700808 } else {
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700809 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> copy = Maps.newHashMap();
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700810 flowEntries.computeIfAbsent(bucketId.deviceId(), id -> Maps.newConcurrentMap())
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700811 .entrySet()
812 .stream()
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700813 .filter(entry -> isInBucket(entry.getKey(), bucketId.bucket()))
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700814 .forEach(entry -> {
815 copy.put(entry.getKey(), Maps.newHashMap(entry.getValue()));
816 });
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700817 return new FlowBucket(bucketId, copy);
Jon Hallfa132292017-10-24 11:11:24 -0700818 }
819 }
820
821 private Map<StoredFlowEntry, StoredFlowEntry> getFlowEntriesInternal(DeviceId deviceId, FlowId flowId) {
822 return getFlowTable(deviceId).computeIfAbsent(flowId, id -> Maps.newConcurrentMap());
823 }
824
825 private StoredFlowEntry getFlowEntryInternal(FlowRule rule) {
826 return getFlowEntriesInternal(rule.deviceId(), rule.id()).get(rule);
827 }
828
829 private Set<FlowEntry> getFlowEntriesInternal(DeviceId deviceId) {
830 return getFlowTable(deviceId).values().stream()
831 .flatMap(m -> m.values().stream())
832 .collect(Collectors.toSet());
833 }
834
835 public StoredFlowEntry getFlowEntry(FlowRule rule) {
836 return getFlowEntryInternal(rule);
837 }
838
839 public Set<FlowEntry> getFlowEntries(DeviceId deviceId) {
840 return getFlowEntriesInternal(deviceId);
841 }
842
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700843 private boolean isInBucket(FlowId flowId, int bucket) {
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700844 return bucket(flowId) == bucket;
845 }
846
847 private int bucket(FlowId flowId) {
848 return (int) (flowId.id() % NUM_BUCKETS);
849 }
850
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700851 private void recordUpdate(BucketId bucketId) {
852 lastUpdateTimes.put(bucketId, System.currentTimeMillis());
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700853 }
854
Jon Hallfa132292017-10-24 11:11:24 -0700855 public void add(FlowEntry rule) {
Devin Limcdca1952018-03-28 18:13:33 -0700856 getFlowEntriesInternal(rule.deviceId(), rule.id())
857 .compute((StoredFlowEntry) rule, (k, stored) -> {
858 return (StoredFlowEntry) rule;
859 });
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700860 recordUpdate(new BucketId(rule.deviceId(), bucket(rule.id())));
Jon Hallfa132292017-10-24 11:11:24 -0700861 }
862
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800863 public void update(FlowEntry rule) {
864 getFlowEntriesInternal(rule.deviceId(), rule.id())
865 .computeIfPresent((StoredFlowEntry) rule, (k, stored) -> {
866 if (rule instanceof DefaultFlowEntry) {
867 DefaultFlowEntry updated = (DefaultFlowEntry) rule;
868 if (stored instanceof DefaultFlowEntry) {
869 DefaultFlowEntry storedEntry = (DefaultFlowEntry) stored;
870 if (updated.created() >= storedEntry.created()) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700871 recordUpdate(new BucketId(rule.deviceId(), bucket(rule.id())));
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800872 return updated;
873 } else {
874 log.debug("Trying to update more recent flow entry {} (stored: {})", updated, stored);
875 return stored;
876 }
877 }
878 }
879 return stored;
880 });
881 }
882
883 public FlowEntry remove(FlowEntry rule) {
Jon Hallfa132292017-10-24 11:11:24 -0700884 final AtomicReference<FlowEntry> removedRule = new AtomicReference<>();
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800885 final Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> flowTable = getFlowTable(rule.deviceId());
Jordan Haltermance336f72018-01-16 17:08:09 -0800886 flowTable.computeIfPresent(rule.id(), (flowId, flowEntries) -> {
887 flowEntries.computeIfPresent((StoredFlowEntry) rule, (k, stored) -> {
Jon Hallfa132292017-10-24 11:11:24 -0700888 if (rule instanceof DefaultFlowEntry) {
889 DefaultFlowEntry toRemove = (DefaultFlowEntry) rule;
890 if (stored instanceof DefaultFlowEntry) {
891 DefaultFlowEntry storedEntry = (DefaultFlowEntry) stored;
892 if (toRemove.created() < storedEntry.created()) {
Jordan Halterman2edfeef2018-01-16 14:59:49 -0800893 log.debug("Trying to remove more recent flow entry {} (stored: {})", toRemove, stored);
Jon Hallfa132292017-10-24 11:11:24 -0700894 // the key is not updated, removedRule remains null
895 return stored;
896 }
897 }
898 }
899 removedRule.set(stored);
900 return null;
901 });
Jordan Haltermance336f72018-01-16 17:08:09 -0800902 return flowEntries.isEmpty() ? null : flowEntries;
903 });
Jon Hallfa132292017-10-24 11:11:24 -0700904
905 if (removedRule.get() != null) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700906 recordUpdate(new BucketId(rule.deviceId(), bucket(rule.id())));
Jon Hallfa132292017-10-24 11:11:24 -0700907 return removedRule.get();
908 } else {
909 return null;
910 }
911 }
912
913 public void purgeFlowRule(DeviceId deviceId) {
914 flowEntries.remove(deviceId);
915 }
916
917 public void purgeFlowRules() {
918 flowEntries.clear();
919 }
920
Jordan Halterman7ae81b82018-06-12 11:23:33 -0700921 /**
922 * Returns a boolean indicating whether the local node is the current master for the given device.
923 *
924 * @param deviceId the device for which to indicate whether the local node is the current master
925 * @return indicates whether the local node is the current master for the given device
926 */
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700927 private boolean isMasterNode(DeviceId deviceId) {
928 NodeId master = replicaInfoManager.getReplicaInfoFor(deviceId).master().orElse(null);
929 return Objects.equals(master, clusterService.getLocalNode().id());
930 }
931
Jordan Halterman7ae81b82018-06-12 11:23:33 -0700932 /**
933 * Backs up all devices to all backup nodes.
934 */
Jon Hallfa132292017-10-24 11:11:24 -0700935 private void backup() {
Jordan Halterman7ae81b82018-06-12 11:23:33 -0700936 for (DeviceId deviceId : flowEntries.keySet()) {
937 backup(deviceId);
938 }
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700939 }
940
Jordan Halterman7ae81b82018-06-12 11:23:33 -0700941 /**
942 * Backs up all buckets in the given device to the given node.
943 *
944 * @param deviceId the device to back up
945 */
946 private void backup(DeviceId deviceId) {
947 if (!isMasterNode(deviceId)) {
948 return;
949 }
950
951 // Get a list of backup nodes for the device.
952 List<NodeId> backupNodes = replicaInfoManager.getReplicaInfoFor(deviceId).backups();
953 int availableBackupCount = Math.min(backupCount, backupNodes.size());
954
955 // If the list of backup nodes is empty, update the flow count.
956 if (availableBackupCount == 0) {
957 updateDeviceFlowCounts(deviceId);
958 } else {
959 // Otherwise, iterate through backup nodes and backup the device.
960 for (int index = 0; index < availableBackupCount; index++) {
961 NodeId backupNode = backupNodes.get(index);
962 try {
963 backup(deviceId, backupNode);
964 } catch (Exception e) {
965 log.error("Backup of " + deviceId + " to " + backupNode + " failed", e);
966 }
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700967 }
Jon Hallfa132292017-10-24 11:11:24 -0700968 }
969 }
970
Jordan Halterman7ae81b82018-06-12 11:23:33 -0700971 /**
972 * Backs up all buckets for the given device to the given node.
973 *
974 * @param deviceId the device to back up
975 * @param nodeId the node to which to back up the device
976 */
977 private void backup(DeviceId deviceId, NodeId nodeId) {
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700978 final long timestamp = System.currentTimeMillis();
979 for (int bucket = 0; bucket < NUM_BUCKETS; bucket++) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700980 BucketId bucketId = new BucketId(deviceId, bucket);
981 BackupOperation operation = new BackupOperation(nodeId, bucketId);
982 if (startBackup(operation)) {
983 backup(operation).whenCompleteAsync((succeeded, error) -> {
984 if (error == null && succeeded) {
985 succeedBackup(operation, timestamp);
986 } else {
987 failBackup(operation);
988 }
Jordan Halterman7ae81b82018-06-12 11:23:33 -0700989 backup(deviceId, nodeId);
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -0700990 }, backupSenderExecutor);
Jordan Halterman3664e8e2018-03-21 12:52:37 -0700991 }
992 }
993 }
994
Jordan Halterman7ae81b82018-06-12 11:23:33 -0700995 /**
996 * Returns a boolean indicating whether the given {@link BackupOperation} can be started.
997 * <p>
998 * The backup can be started if no backup for the same device/bucket/node is already in progress and changes
999 * are pending replication for the backup operation.
1000 *
1001 * @param operation the operation to start
1002 * @return indicates whether the given backup operation should be started
1003 */
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -07001004 private boolean startBackup(BackupOperation operation) {
1005 long lastBackupTime = lastBackupTimes.getOrDefault(operation, 0L);
1006 long lastUpdateTime = lastUpdateTimes.getOrDefault(operation.bucketId(), 0L);
1007 return lastUpdateTime > 0 && lastBackupTime <= lastUpdateTime && inFlightUpdates.add(operation);
Jordan Halterman3664e8e2018-03-21 12:52:37 -07001008 }
1009
Jordan Halterman7ae81b82018-06-12 11:23:33 -07001010 /**
1011 * Fails the given backup operation.
1012 *
1013 * @param operation the backup operation to fail
1014 */
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -07001015 private void failBackup(BackupOperation operation) {
1016 inFlightUpdates.remove(operation);
Jordan Halterman3664e8e2018-03-21 12:52:37 -07001017 }
1018
Jordan Halterman7ae81b82018-06-12 11:23:33 -07001019 /**
1020 * Succeeds the given backup operation.
1021 * <p>
1022 * The last backup time for the operation will be updated and the operation will be removed from
1023 * in-flight updates.
1024 *
1025 * @param operation the operation to succeed
1026 * @param timestamp the timestamp at which the operation was <em>started</em>
1027 */
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -07001028 private void succeedBackup(BackupOperation operation, long timestamp) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -07001029 lastBackupTimes.put(operation, timestamp);
Jordan Halterman7ae81b82018-06-12 11:23:33 -07001030 inFlightUpdates.remove(operation);
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -07001031 }
1032
Jordan Halterman7ae81b82018-06-12 11:23:33 -07001033 /**
1034 * Performs the given backup operation.
1035 *
1036 * @param operation the operation to perform
1037 * @return a future to be completed with a boolean indicating whether the backup operation was successful
1038 */
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -07001039 private CompletableFuture<Boolean> backup(BackupOperation operation) {
1040 log.debug("Sending flowEntries in bucket {} for device {} to {} for backup.",
1041 operation.bucketId().bucket(), operation.bucketId().deviceId(), operation.nodeId());
1042 FlowBucket flowBucket = getFlowBucket(operation.bucketId());
1043
1044 CompletableFuture<Boolean> future = new CompletableFuture<>();
1045 clusterCommunicator.<FlowBucket, Set<FlowId>>
1046 sendAndReceive(flowBucket,
1047 FLOW_TABLE_BACKUP,
1048 serializer::encode,
1049 serializer::decode,
1050 operation.nodeId())
1051 .whenComplete((backedupFlows, error) -> {
1052 Set<FlowId> flowsNotBackedUp = error != null ?
1053 flowBucket.table().keySet() :
1054 Sets.difference(flowBucket.table().keySet(), backedupFlows);
1055 if (flowsNotBackedUp.size() > 0) {
1056 log.warn("Failed to backup flows: {}. Reason: {}, Node: {}",
1057 flowsNotBackedUp, error != null ? error.getMessage() : "none", operation.nodeId());
1058 }
1059 future.complete(backedupFlows != null);
1060 });
Jordan Halterman7ae81b82018-06-12 11:23:33 -07001061
1062 updateFlowCounts(flowBucket);
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -07001063 return future;
1064 }
1065
Jordan Halterman7ae81b82018-06-12 11:23:33 -07001066 /**
1067 * Handles a flow bucket backup from a remote peer.
1068 *
1069 * @param flowBucket the flow bucket to back up
1070 * @return the set of flows that could not be backed up
1071 */
1072 private Set<FlowId> onBackup(FlowBucket flowBucket) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -07001073 log.debug("Received flowEntries for {} bucket {} to backup",
1074 flowBucket.bucketId().deviceId(), flowBucket.bucketId);
Jordan Halterman3664e8e2018-03-21 12:52:37 -07001075 Set<FlowId> backedupFlows = Sets.newHashSet();
Jon Hallfa132292017-10-24 11:11:24 -07001076 try {
Jordan Halterman3664e8e2018-03-21 12:52:37 -07001077 // Only process those devices are that not managed by the local node.
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -07001078 NodeId master = replicaInfoManager.getReplicaInfoFor(flowBucket.bucketId().deviceId())
1079 .master()
1080 .orElse(null);
Jordan Halterman3664e8e2018-03-21 12:52:37 -07001081 if (!Objects.equals(local, master)) {
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -07001082 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> backupFlowTable =
1083 getFlowTable(flowBucket.bucketId().deviceId());
1084 backupFlowTable.putAll(flowBucket.table());
Jordan Halterman3664e8e2018-03-21 12:52:37 -07001085 backupFlowTable.entrySet()
Jordan Haltermanfd73c6e2018-06-01 00:40:56 -07001086 .removeIf(entry -> isInBucket(entry.getKey(), flowBucket.bucketId().bucket())
1087 && !flowBucket.table().containsKey(entry.getKey()));
1088 backedupFlows.addAll(flowBucket.table().keySet());
Jordan Halterman3664e8e2018-03-21 12:52:37 -07001089 }
Jon Hallfa132292017-10-24 11:11:24 -07001090 } catch (Exception e) {
1091 log.warn("Failure processing backup request", e);
1092 }
Jordan Halterman3664e8e2018-03-21 12:52:37 -07001093 return backedupFlows;
Jon Hallfa132292017-10-24 11:11:24 -07001094 }
Jordan Halterman7ae81b82018-06-12 11:23:33 -07001095
1096 /**
1097 * Updates all flow counts for the given device.
1098 *
1099 * @param deviceId the device for which to update flow counts
1100 */
1101 private void updateDeviceFlowCounts(DeviceId deviceId) {
1102 for (int bucket = 0; bucket < NUM_BUCKETS; bucket++) {
1103 BucketId bucketId = new BucketId(deviceId, bucket);
1104 FlowBucket flowBucket = getFlowBucket(bucketId);
1105 updateFlowCounts(flowBucket);
1106 }
1107 }
1108
1109 /**
1110 * Updates the eventually consistent flow count for the given bucket.
1111 *
1112 * @param flowBucket the flow bucket for which to update flow counts
1113 */
1114 private void updateFlowCounts(FlowBucket flowBucket) {
1115 int flowCount = flowBucket.table().entrySet()
1116 .stream()
1117 .mapToInt(e -> e.getValue().values().size())
1118 .sum();
1119 flowCounts.put(flowBucket.bucketId(), flowCount);
1120 }
Jon Hallfa132292017-10-24 11:11:24 -07001121 }
1122
1123 @Override
1124 public FlowRuleEvent updateTableStatistics(DeviceId deviceId,
1125 List<TableStatisticsEntry> tableStats) {
1126 deviceTableStats.put(deviceId, tableStats);
1127 return null;
1128 }
1129
1130 @Override
1131 public Iterable<TableStatisticsEntry> getTableStatistics(DeviceId deviceId) {
1132 NodeId master = mastershipService.getMasterFor(deviceId);
1133
1134 if (master == null) {
1135 log.debug("Failed to getTableStats: No master for {}", deviceId);
1136 return Collections.emptyList();
1137 }
1138
1139 List<TableStatisticsEntry> tableStats = deviceTableStats.get(deviceId);
1140 if (tableStats == null) {
1141 return Collections.emptyList();
1142 }
1143 return ImmutableList.copyOf(tableStats);
1144 }
1145
1146 @Override
1147 public long getActiveFlowRuleCount(DeviceId deviceId) {
1148 return Streams.stream(getTableStatistics(deviceId))
1149 .mapToLong(TableStatisticsEntry::activeFlowEntries)
1150 .sum();
1151 }
1152
1153 private class InternalTableStatsListener
1154 implements EventuallyConsistentMapListener<DeviceId, List<TableStatisticsEntry>> {
1155 @Override
1156 public void event(EventuallyConsistentMapEvent<DeviceId,
1157 List<TableStatisticsEntry>> event) {
1158 //TODO: Generate an event to listeners (do we need?)
1159 }
1160 }
1161}