blob: 0e3c8ea758426217951a9bf68ba52a514382e713 [file] [log] [blame]
Madan Jampani86940d92015-05-06 11:47:57 -07001 /*
Brian O'Connora09fe5b2017-08-03 21:12:30 -07002 * Copyright 2014-present Open Networking Foundation
Madan Jampani86940d92015-05-06 11:47:57 -07003 *
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
Yuta HIGUCHIbaaf8dc2017-06-21 17:44:09 -070018import 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;
25import java.util.concurrent.ExecutorService;
26import java.util.concurrent.Executors;
27import java.util.concurrent.ScheduledExecutorService;
28import java.util.concurrent.ScheduledFuture;
29import java.util.concurrent.TimeUnit;
30import java.util.concurrent.atomic.AtomicReference;
31import java.util.stream.Collectors;
Ray Milkey2b6ff422016-08-26 13:03:15 -070032
Yuta HIGUCHIbaaf8dc2017-06-21 17:44:09 -070033import com.google.common.collect.Streams;
34import org.apache.felix.scr.annotations.Activate;
35import org.apache.felix.scr.annotations.Component;
36import org.apache.felix.scr.annotations.Deactivate;
37import org.apache.felix.scr.annotations.Modified;
38import org.apache.felix.scr.annotations.Property;
39import org.apache.felix.scr.annotations.Reference;
40import org.apache.felix.scr.annotations.ReferenceCardinality;
41import org.apache.felix.scr.annotations.Service;
42import org.onlab.util.KryoNamespace;
43import org.onlab.util.Tools;
44import org.onosproject.cfg.ComponentConfigService;
45import org.onosproject.cluster.ClusterService;
46import org.onosproject.cluster.NodeId;
47import org.onosproject.core.CoreService;
48import org.onosproject.core.IdGenerator;
49import org.onosproject.mastership.MastershipService;
50import org.onosproject.net.DeviceId;
51import org.onosproject.net.device.DeviceService;
52import org.onosproject.net.flow.CompletedBatchOperation;
53import org.onosproject.net.flow.DefaultFlowEntry;
54import org.onosproject.net.flow.FlowEntry;
55import org.onosproject.net.flow.FlowEntry.FlowEntryState;
56import org.onosproject.net.flow.FlowId;
57import org.onosproject.net.flow.FlowRule;
58import org.onosproject.net.flow.FlowRuleBatchEntry;
59import org.onosproject.net.flow.FlowRuleBatchEntry.FlowRuleOperation;
60import org.onosproject.net.flow.FlowRuleBatchEvent;
61import org.onosproject.net.flow.FlowRuleBatchOperation;
62import org.onosproject.net.flow.FlowRuleBatchRequest;
63import 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;
70import org.onosproject.persistence.PersistenceService;
71import org.onosproject.store.AbstractStore;
72import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
73import org.onosproject.store.cluster.messaging.ClusterMessage;
74import org.onosproject.store.cluster.messaging.ClusterMessageHandler;
75import org.onosproject.store.flow.ReplicaInfoEvent;
76import org.onosproject.store.flow.ReplicaInfoEventListener;
77import org.onosproject.store.flow.ReplicaInfoService;
78import org.onosproject.store.impl.MastershipBasedTimestamp;
79import org.onosproject.store.serializers.KryoNamespaces;
80import org.onosproject.store.service.EventuallyConsistentMap;
81import org.onosproject.store.service.EventuallyConsistentMapEvent;
82import org.onosproject.store.service.EventuallyConsistentMapListener;
83import org.onosproject.store.service.Serializer;
84import org.onosproject.store.service.StorageService;
85import org.onosproject.store.service.WallClockTimestamp;
86import org.osgi.service.component.ComponentContext;
87import org.slf4j.Logger;
Ray Milkey2b6ff422016-08-26 13:03:15 -070088
Yuta HIGUCHIbaaf8dc2017-06-21 17:44:09 -070089import com.google.common.collect.ImmutableList;
90import com.google.common.collect.Iterables;
91import com.google.common.collect.Maps;
92import com.google.common.collect.Sets;
Yuta HIGUCHI10e91fb2017-06-15 13:30:50 -070093import com.google.common.util.concurrent.Futures;
Madan Jampani86940d92015-05-06 11:47:57 -070094
Yuta HIGUCHIbaaf8dc2017-06-21 17:44:09 -070095import 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.FlowStoreMessageSubjects.APPLY_BATCH_FLOWS;
101import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.FLOW_TABLE_BACKUP;
102import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.GET_DEVICE_FLOW_ENTRIES;
103import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.GET_FLOW_ENTRY;
104import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.REMOTE_APPLY_COMPLETED;
105import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.REMOVE_FLOW_ENTRY;
106import static org.slf4j.LoggerFactory.getLogger;
Madan Jampani86940d92015-05-06 11:47:57 -0700107
108/**
109 * Manages inventory of flow rules using a distributed state management protocol.
110 */
Sho SHIMIZU5c396e32016-08-12 15:19:12 -0700111@Component(immediate = true)
Madan Jampani86940d92015-05-06 11:47:57 -0700112@Service
Madan Jampani37d04c62016-04-25 15:53:55 -0700113public class DistributedFlowRuleStore
Madan Jampani86940d92015-05-06 11:47:57 -0700114 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;
Madan Jampanic6d69f72016-07-15 15:47:12 -0700120 private static final int DEFAULT_MAX_BACKUP_COUNT = 2;
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700121 private static final boolean DEFAULT_PERSISTENCE_ENABLED = false;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700122 private static final int DEFAULT_BACKUP_PERIOD_MILLIS = 2000;
Madan Jampani86940d92015-05-06 11:47:57 -0700123 private static final long FLOW_RULE_STORE_TIMEOUT_MILLIS = 5000;
Madan Jampaniadea8902015-06-04 17:39:45 -0700124 // number of devices whose flow entries will be backed up in one communication round
125 private static final int FLOW_TABLE_BACKUP_BATCH_SIZE = 1;
Madan Jampani86940d92015-05-06 11:47:57 -0700126
127 @Property(name = "msgHandlerPoolSize", intValue = MESSAGE_HANDLER_THREAD_POOL_SIZE,
128 label = "Number of threads in the message handler pool")
129 private int msgHandlerPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
130
Madan Jampani08bf17b2015-05-06 16:25:26 -0700131 @Property(name = "backupPeriod", intValue = DEFAULT_BACKUP_PERIOD_MILLIS,
132 label = "Delay in ms between successive backup runs")
133 private int backupPeriod = DEFAULT_BACKUP_PERIOD_MILLIS;
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700134 @Property(name = "persistenceEnabled", boolValue = false,
135 label = "Indicates whether or not changes in the flow table should be persisted to disk.")
136 private boolean persistenceEnabled = DEFAULT_PERSISTENCE_ENABLED;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700137
Madan Jampanic6d69f72016-07-15 15:47:12 -0700138 @Property(name = "backupCount", intValue = DEFAULT_MAX_BACKUP_COUNT,
139 label = "Max number of backup copies for each device")
140 private volatile int backupCount = DEFAULT_MAX_BACKUP_COUNT;
141
Madan Jampani86940d92015-05-06 11:47:57 -0700142 private InternalFlowTable flowTable = new InternalFlowTable();
143
144 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
145 protected ReplicaInfoService replicaInfoManager;
146
147 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
148 protected ClusterCommunicationService clusterCommunicator;
149
150 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
151 protected ClusterService clusterService;
152
153 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
154 protected DeviceService deviceService;
155
156 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
157 protected CoreService coreService;
158
159 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
160 protected ComponentConfigService configService;
161
162 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
163 protected MastershipService mastershipService;
164
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700165 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
166 protected PersistenceService persistenceService;
167
Madan Jampani86940d92015-05-06 11:47:57 -0700168 private Map<Long, NodeId> pendingResponses = Maps.newConcurrentMap();
169 private ExecutorService messageHandlingExecutor;
Madan Jampani71c32ca2016-06-22 08:23:18 -0700170 private ExecutorService eventHandler;
Madan Jampani86940d92015-05-06 11:47:57 -0700171
Madan Jampani08bf17b2015-05-06 16:25:26 -0700172 private ScheduledFuture<?> backupTask;
Madan Jampani86940d92015-05-06 11:47:57 -0700173 private final ScheduledExecutorService backupSenderExecutor =
HIGUCHI Yutad9e01052016-04-14 09:31:42 -0700174 Executors.newSingleThreadScheduledExecutor(groupedThreads("onos/flow", "backup-sender", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700175
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700176 private EventuallyConsistentMap<DeviceId, List<TableStatisticsEntry>> deviceTableStats;
177 private final EventuallyConsistentMapListener<DeviceId, List<TableStatisticsEntry>> tableStatsListener =
178 new InternalTableStatsListener();
179
180 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
181 protected StorageService storageService;
182
Madan Jampani884d4432016-08-23 10:46:55 -0700183 protected final Serializer serializer = Serializer.using(KryoNamespaces.API);
Madan Jampani86940d92015-05-06 11:47:57 -0700184
Madan Jampani884d4432016-08-23 10:46:55 -0700185 protected final KryoNamespace.Builder serializerBuilder = KryoNamespace.newBuilder()
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700186 .register(KryoNamespaces.API)
187 .register(MastershipBasedTimestamp.class);
188
189
Madan Jampani86940d92015-05-06 11:47:57 -0700190 private IdGenerator idGenerator;
191 private NodeId local;
192
193 @Activate
194 public void activate(ComponentContext context) {
195 configService.registerProperties(getClass());
196
197 idGenerator = coreService.getIdGenerator(FlowRuleService.FLOW_OP_TOPIC);
198
199 local = clusterService.getLocalNode().id();
200
Madan Jampani71c32ca2016-06-22 08:23:18 -0700201 eventHandler = Executors.newSingleThreadExecutor(
202 groupedThreads("onos/flow", "event-handler", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700203 messageHandlingExecutor = Executors.newFixedThreadPool(
HIGUCHI Yutad9e01052016-04-14 09:31:42 -0700204 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700205
206 registerMessageHandlers(messageHandlingExecutor);
207
sivachidambaram subramanian0b33c082017-04-06 13:26:44 +0530208 replicaInfoManager.addListener(flowTable);
209 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
210 flowTable::backup,
211 0,
212 backupPeriod,
213 TimeUnit.MILLISECONDS);
Madan Jampani86940d92015-05-06 11:47:57 -0700214
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700215 deviceTableStats = storageService.<DeviceId, List<TableStatisticsEntry>>eventuallyConsistentMapBuilder()
216 .withName("onos-flow-table-stats")
Madan Jampani884d4432016-08-23 10:46:55 -0700217 .withSerializer(serializerBuilder)
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700218 .withAntiEntropyPeriod(5, TimeUnit.SECONDS)
219 .withTimestampProvider((k, v) -> new WallClockTimestamp())
220 .withTombstonesDisabled()
221 .build();
222 deviceTableStats.addListener(tableStatsListener);
223
Madan Jampani86940d92015-05-06 11:47:57 -0700224 logConfig("Started");
225 }
226
227 @Deactivate
228 public void deactivate(ComponentContext context) {
sivachidambaram subramanian0b33c082017-04-06 13:26:44 +0530229 replicaInfoManager.removeListener(flowTable);
230 backupTask.cancel(true);
Madan Jampani86940d92015-05-06 11:47:57 -0700231 configService.unregisterProperties(getClass(), false);
232 unregisterMessageHandlers();
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700233 deviceTableStats.removeListener(tableStatsListener);
234 deviceTableStats.destroy();
Madan Jampani71c32ca2016-06-22 08:23:18 -0700235 eventHandler.shutdownNow();
Madan Jampani86940d92015-05-06 11:47:57 -0700236 messageHandlingExecutor.shutdownNow();
237 backupSenderExecutor.shutdownNow();
238 log.info("Stopped");
239 }
240
241 @SuppressWarnings("rawtypes")
242 @Modified
243 public void modified(ComponentContext context) {
244 if (context == null) {
Madan Jampani86940d92015-05-06 11:47:57 -0700245 logConfig("Default config");
246 return;
247 }
248
249 Dictionary properties = context.getProperties();
250 int newPoolSize;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700251 int newBackupPeriod;
Madan Jampanic6d69f72016-07-15 15:47:12 -0700252 int newBackupCount;
Madan Jampani86940d92015-05-06 11:47:57 -0700253 try {
254 String s = get(properties, "msgHandlerPoolSize");
255 newPoolSize = isNullOrEmpty(s) ? msgHandlerPoolSize : Integer.parseInt(s.trim());
256
Madan Jampani08bf17b2015-05-06 16:25:26 -0700257 s = get(properties, "backupPeriod");
258 newBackupPeriod = isNullOrEmpty(s) ? backupPeriod : Integer.parseInt(s.trim());
259
Madan Jampanic6d69f72016-07-15 15:47:12 -0700260 s = get(properties, "backupCount");
261 newBackupCount = isNullOrEmpty(s) ? backupCount : Integer.parseInt(s.trim());
Madan Jampani86940d92015-05-06 11:47:57 -0700262 } catch (NumberFormatException | ClassCastException e) {
263 newPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700264 newBackupPeriod = DEFAULT_BACKUP_PERIOD_MILLIS;
Madan Jampanic6d69f72016-07-15 15:47:12 -0700265 newBackupCount = DEFAULT_MAX_BACKUP_COUNT;
Madan Jampani86940d92015-05-06 11:47:57 -0700266 }
267
Madan Jampani08bf17b2015-05-06 16:25:26 -0700268 boolean restartBackupTask = false;
sivachidambaram subramanian0b33c082017-04-06 13:26:44 +0530269
Madan Jampani08bf17b2015-05-06 16:25:26 -0700270 if (newBackupPeriod != backupPeriod) {
271 backupPeriod = newBackupPeriod;
sivachidambaram subramanian0b33c082017-04-06 13:26:44 +0530272 restartBackupTask = true;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700273 }
274 if (restartBackupTask) {
275 if (backupTask != null) {
276 // cancel previously running task
277 backupTask.cancel(false);
278 }
279 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
280 flowTable::backup,
281 0,
282 backupPeriod,
283 TimeUnit.MILLISECONDS);
Madan Jampani86940d92015-05-06 11:47:57 -0700284 }
285 if (newPoolSize != msgHandlerPoolSize) {
286 msgHandlerPoolSize = newPoolSize;
287 ExecutorService oldMsgHandler = messageHandlingExecutor;
288 messageHandlingExecutor = Executors.newFixedThreadPool(
HIGUCHI Yuta060da9a2016-03-11 19:16:35 -0800289 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700290
291 // replace previously registered handlers.
292 registerMessageHandlers(messageHandlingExecutor);
293 oldMsgHandler.shutdown();
294 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700295 if (backupCount != newBackupCount) {
296 backupCount = newBackupCount;
297 }
Madan Jampani86940d92015-05-06 11:47:57 -0700298 logConfig("Reconfigured");
299 }
300
301 private void registerMessageHandlers(ExecutorService executor) {
302
303 clusterCommunicator.addSubscriber(APPLY_BATCH_FLOWS, new OnStoreBatch(), executor);
304 clusterCommunicator.<FlowRuleBatchEvent>addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700305 REMOTE_APPLY_COMPLETED, serializer::decode, this::notifyDelegate, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700306 clusterCommunicator.addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700307 GET_FLOW_ENTRY, serializer::decode, flowTable::getFlowEntry, serializer::encode, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700308 clusterCommunicator.addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700309 GET_DEVICE_FLOW_ENTRIES, serializer::decode, flowTable::getFlowEntries, serializer::encode, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700310 clusterCommunicator.addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700311 REMOVE_FLOW_ENTRY, serializer::decode, this::removeFlowRuleInternal, serializer::encode, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700312 clusterCommunicator.addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700313 FLOW_TABLE_BACKUP, serializer::decode, flowTable::onBackupReceipt, serializer::encode, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700314 }
315
316 private void unregisterMessageHandlers() {
317 clusterCommunicator.removeSubscriber(REMOVE_FLOW_ENTRY);
318 clusterCommunicator.removeSubscriber(GET_DEVICE_FLOW_ENTRIES);
319 clusterCommunicator.removeSubscriber(GET_FLOW_ENTRY);
320 clusterCommunicator.removeSubscriber(APPLY_BATCH_FLOWS);
321 clusterCommunicator.removeSubscriber(REMOTE_APPLY_COMPLETED);
322 clusterCommunicator.removeSubscriber(FLOW_TABLE_BACKUP);
323 }
324
325 private void logConfig(String prefix) {
sivachidambaram subramanian0b33c082017-04-06 13:26:44 +0530326 log.info("{} with msgHandlerPoolSize = {}; backupPeriod = {}, backupCount = {}",
327 prefix, msgHandlerPoolSize, backupPeriod, backupCount);
Madan Jampani86940d92015-05-06 11:47:57 -0700328 }
329
330 // This is not a efficient operation on a distributed sharded
331 // flow store. We need to revisit the need for this operation or at least
332 // make it device specific.
333 @Override
334 public int getFlowRuleCount() {
Yuta HIGUCHI10e91fb2017-06-15 13:30:50 -0700335 return Streams.stream(deviceService.getDevices()).parallel()
336 .mapToInt(device -> Iterables.size(getFlowEntries(device.id())))
337 .sum();
Madan Jampani86940d92015-05-06 11:47:57 -0700338 }
339
340 @Override
341 public FlowEntry getFlowEntry(FlowRule rule) {
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700342 NodeId master = mastershipService.getMasterFor(rule.deviceId());
Madan Jampani86940d92015-05-06 11:47:57 -0700343
344 if (master == null) {
Thomas Vachuska88fd6902015-08-04 10:08:34 -0700345 log.debug("Failed to getFlowEntry: No master for {}", rule.deviceId());
Madan Jampani86940d92015-05-06 11:47:57 -0700346 return null;
347 }
348
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800349 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700350 return flowTable.getFlowEntry(rule);
351 }
352
353 log.trace("Forwarding getFlowEntry to {}, which is the primary (master) for device {}",
354 master, rule.deviceId());
355
356 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(rule,
357 FlowStoreMessageSubjects.GET_FLOW_ENTRY,
Madan Jampani884d4432016-08-23 10:46:55 -0700358 serializer::encode,
359 serializer::decode,
Madan Jampani86940d92015-05-06 11:47:57 -0700360 master),
361 FLOW_RULE_STORE_TIMEOUT_MILLIS,
362 TimeUnit.MILLISECONDS,
363 null);
364 }
365
366 @Override
367 public Iterable<FlowEntry> getFlowEntries(DeviceId deviceId) {
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700368 NodeId master = mastershipService.getMasterFor(deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700369
370 if (master == null) {
Thomas Vachuska88fd6902015-08-04 10:08:34 -0700371 log.debug("Failed to getFlowEntries: No master for {}", deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700372 return Collections.emptyList();
373 }
374
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800375 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700376 return flowTable.getFlowEntries(deviceId);
377 }
378
379 log.trace("Forwarding getFlowEntries to {}, which is the primary (master) for device {}",
380 master, deviceId);
381
382 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(deviceId,
383 FlowStoreMessageSubjects.GET_DEVICE_FLOW_ENTRIES,
Madan Jampani884d4432016-08-23 10:46:55 -0700384 serializer::encode,
385 serializer::decode,
Madan Jampani86940d92015-05-06 11:47:57 -0700386 master),
387 FLOW_RULE_STORE_TIMEOUT_MILLIS,
388 TimeUnit.MILLISECONDS,
389 Collections.emptyList());
390 }
391
392 @Override
393 public void storeFlowRule(FlowRule rule) {
394 storeBatch(new FlowRuleBatchOperation(
Sho SHIMIZU98ffca82015-05-11 08:39:24 -0700395 Collections.singletonList(new FlowRuleBatchEntry(FlowRuleOperation.ADD, rule)),
Madan Jampani86940d92015-05-06 11:47:57 -0700396 rule.deviceId(), idGenerator.getNewId()));
397 }
398
399 @Override
400 public void storeBatch(FlowRuleBatchOperation operation) {
401 if (operation.getOperations().isEmpty()) {
402 notifyDelegate(FlowRuleBatchEvent.completed(
403 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
404 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
405 return;
406 }
407
408 DeviceId deviceId = operation.deviceId();
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700409 NodeId master = mastershipService.getMasterFor(deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700410
411 if (master == null) {
Sivachidambaram Subramanian605104e2017-06-21 07:40:04 +0530412 log.warn("No master for {} ", deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700413
414 updateStoreInternal(operation);
415
416 notifyDelegate(FlowRuleBatchEvent.completed(
417 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
418 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
419 return;
420 }
421
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800422 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700423 storeBatchInternal(operation);
424 return;
425 }
426
427 log.trace("Forwarding storeBatch to {}, which is the primary (master) for device {}",
428 master, deviceId);
429
Madan Jampani175e8fd2015-05-20 14:10:45 -0700430 clusterCommunicator.unicast(operation,
431 APPLY_BATCH_FLOWS,
Madan Jampani884d4432016-08-23 10:46:55 -0700432 serializer::encode,
Madan Jampani175e8fd2015-05-20 14:10:45 -0700433 master)
434 .whenComplete((result, error) -> {
Thomas Vachuskaa267ce42015-05-27 16:14:23 -0700435 if (error != null) {
Madan Jampani15f1bc42015-05-28 10:51:52 -0700436 log.warn("Failed to storeBatch: {} to {}", operation, master, error);
Madan Jampani86940d92015-05-06 11:47:57 -0700437
Thomas Vachuskaa267ce42015-05-27 16:14:23 -0700438 Set<FlowRule> allFailures = operation.getOperations()
439 .stream()
440 .map(op -> op.target())
441 .collect(Collectors.toSet());
Madan Jampani86940d92015-05-06 11:47:57 -0700442
Thomas Vachuskaa267ce42015-05-27 16:14:23 -0700443 notifyDelegate(FlowRuleBatchEvent.completed(
444 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
445 new CompletedBatchOperation(false, allFailures, deviceId)));
446 }
Madan Jampani175e8fd2015-05-20 14:10:45 -0700447 });
Madan Jampani86940d92015-05-06 11:47:57 -0700448 }
449
450 private void storeBatchInternal(FlowRuleBatchOperation operation) {
451
452 final DeviceId did = operation.deviceId();
453 //final Collection<FlowEntry> ft = flowTable.getFlowEntries(did);
454 Set<FlowRuleBatchEntry> currentOps = updateStoreInternal(operation);
455 if (currentOps.isEmpty()) {
456 batchOperationComplete(FlowRuleBatchEvent.completed(
457 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
458 new CompletedBatchOperation(true, Collections.emptySet(), did)));
459 return;
460 }
461
462 notifyDelegate(FlowRuleBatchEvent.requested(new
463 FlowRuleBatchRequest(operation.id(),
464 currentOps), operation.deviceId()));
465 }
466
467 private Set<FlowRuleBatchEntry> updateStoreInternal(FlowRuleBatchOperation operation) {
468 return operation.getOperations().stream().map(
469 op -> {
470 StoredFlowEntry entry;
471 switch (op.operator()) {
472 case ADD:
473 entry = new DefaultFlowEntry(op.target());
474 // always add requested FlowRule
475 // Note: 2 equal FlowEntry may have different treatment
476 flowTable.remove(entry.deviceId(), entry);
477 flowTable.add(entry);
478
479 return op;
480 case REMOVE:
481 entry = flowTable.getFlowEntry(op.target());
482 if (entry != null) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800483 //FIXME modification of "stored" flow entry outside of flow table
Madan Jampani86940d92015-05-06 11:47:57 -0700484 entry.setState(FlowEntryState.PENDING_REMOVE);
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800485 log.debug("Setting state of rule to pending remove: {}", entry);
Madan Jampani86940d92015-05-06 11:47:57 -0700486 return op;
487 }
488 break;
489 case MODIFY:
490 //TODO: figure this out at some point
491 break;
492 default:
493 log.warn("Unknown flow operation operator: {}", op.operator());
494 }
495 return null;
496 }
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800497 ).filter(Objects::nonNull).collect(Collectors.toSet());
Madan Jampani86940d92015-05-06 11:47:57 -0700498 }
499
500 @Override
501 public void deleteFlowRule(FlowRule rule) {
502 storeBatch(
503 new FlowRuleBatchOperation(
Sho SHIMIZU98ffca82015-05-11 08:39:24 -0700504 Collections.singletonList(
Madan Jampani86940d92015-05-06 11:47:57 -0700505 new FlowRuleBatchEntry(
506 FlowRuleOperation.REMOVE,
507 rule)), rule.deviceId(), idGenerator.getNewId()));
508 }
509
510 @Override
Charles Chan93fa7272016-01-26 22:27:02 -0800511 public FlowRuleEvent pendingFlowRule(FlowEntry rule) {
512 if (mastershipService.isLocalMaster(rule.deviceId())) {
513 StoredFlowEntry stored = flowTable.getFlowEntry(rule);
514 if (stored != null &&
515 stored.state() != FlowEntryState.PENDING_ADD) {
516 stored.setState(FlowEntryState.PENDING_ADD);
517 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
518 }
519 }
520 return null;
521 }
522
523 @Override
Madan Jampani86940d92015-05-06 11:47:57 -0700524 public FlowRuleEvent addOrUpdateFlowRule(FlowEntry rule) {
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700525 NodeId master = mastershipService.getMasterFor(rule.deviceId());
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800526 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700527 return addOrUpdateFlowRuleInternal(rule);
528 }
529
530 log.warn("Tried to update FlowRule {} state,"
531 + " while the Node was not the master.", rule);
532 return null;
533 }
534
535 private FlowRuleEvent addOrUpdateFlowRuleInternal(FlowEntry rule) {
536 // check if this new rule is an update to an existing entry
537 StoredFlowEntry stored = flowTable.getFlowEntry(rule);
538 if (stored != null) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800539 //FIXME modification of "stored" flow entry outside of flow table
Madan Jampani86940d92015-05-06 11:47:57 -0700540 stored.setBytes(rule.bytes());
Thiago Santos877914d2016-07-20 18:29:29 -0300541 stored.setLife(rule.life(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
Sangsik Yoonb1b823f2016-05-16 18:55:39 +0900542 stored.setLiveType(rule.liveType());
Madan Jampani86940d92015-05-06 11:47:57 -0700543 stored.setPackets(rule.packets());
Jonathan Hart89e981f2016-01-04 13:59:55 -0800544 stored.setLastSeen();
Madan Jampani86940d92015-05-06 11:47:57 -0700545 if (stored.state() == FlowEntryState.PENDING_ADD) {
546 stored.setState(FlowEntryState.ADDED);
547 return new FlowRuleEvent(Type.RULE_ADDED, rule);
548 }
549 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
550 }
551
552 // TODO: Confirm if this behavior is correct. See SimpleFlowRuleStore
553 // TODO: also update backup if the behavior is correct.
554 flowTable.add(rule);
555 return null;
556 }
557
558 @Override
559 public FlowRuleEvent removeFlowRule(FlowEntry rule) {
560 final DeviceId deviceId = rule.deviceId();
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700561 NodeId master = mastershipService.getMasterFor(deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700562
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800563 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700564 // bypass and handle it locally
565 return removeFlowRuleInternal(rule);
566 }
567
568 if (master == null) {
569 log.warn("Failed to removeFlowRule: No master for {}", deviceId);
570 // TODO: revisit if this should be null (="no-op") or Exception
571 return null;
572 }
573
574 log.trace("Forwarding removeFlowRule to {}, which is the master for device {}",
575 master, deviceId);
576
Madan Jampani222229e2016-07-14 10:43:25 -0700577 return Futures.getUnchecked(clusterCommunicator.sendAndReceive(
Madan Jampani86940d92015-05-06 11:47:57 -0700578 rule,
579 REMOVE_FLOW_ENTRY,
Madan Jampani884d4432016-08-23 10:46:55 -0700580 serializer::encode,
581 serializer::decode,
Madan Jampani222229e2016-07-14 10:43:25 -0700582 master));
Madan Jampani86940d92015-05-06 11:47:57 -0700583 }
584
585 private FlowRuleEvent removeFlowRuleInternal(FlowEntry rule) {
586 final DeviceId deviceId = rule.deviceId();
587 // This is where one could mark a rule as removed and still keep it in the store.
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800588 final FlowEntry removed = flowTable.remove(deviceId, rule);
589 // rule may be partial rule that is missing treatment, we should use rule from store instead
590 return removed != null ? new FlowRuleEvent(RULE_REMOVED, removed) : null;
Madan Jampani86940d92015-05-06 11:47:57 -0700591 }
592
593 @Override
Charles Chan0c7c43b2016-01-14 17:39:20 -0800594 public void purgeFlowRule(DeviceId deviceId) {
595 flowTable.purgeFlowRule(deviceId);
596 }
597
598 @Override
Victor Silva139bca42016-08-18 11:46:35 -0300599 public void purgeFlowRules() {
600 flowTable.purgeFlowRules();
601 }
602
603 @Override
Madan Jampani86940d92015-05-06 11:47:57 -0700604 public void batchOperationComplete(FlowRuleBatchEvent event) {
605 //FIXME: need a per device pending response
606 NodeId nodeId = pendingResponses.remove(event.subject().batchId());
607 if (nodeId == null) {
608 notifyDelegate(event);
609 } else {
610 // TODO check unicast return value
Madan Jampani884d4432016-08-23 10:46:55 -0700611 clusterCommunicator.unicast(event, REMOTE_APPLY_COMPLETED, serializer::encode, nodeId);
Madan Jampani86940d92015-05-06 11:47:57 -0700612 //error log: log.warn("Failed to respond to peer for batch operation result");
613 }
614 }
615
616 private final class OnStoreBatch implements ClusterMessageHandler {
617
618 @Override
619 public void handle(final ClusterMessage message) {
Madan Jampani884d4432016-08-23 10:46:55 -0700620 FlowRuleBatchOperation operation = serializer.decode(message.payload());
Madan Jampani86940d92015-05-06 11:47:57 -0700621 log.debug("received batch request {}", operation);
622
623 final DeviceId deviceId = operation.deviceId();
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700624 NodeId master = mastershipService.getMasterFor(deviceId);
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800625 if (!Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700626 Set<FlowRule> failures = new HashSet<>(operation.size());
627 for (FlowRuleBatchEntry op : operation.getOperations()) {
628 failures.add(op.target());
629 }
630 CompletedBatchOperation allFailed = new CompletedBatchOperation(false, failures, deviceId);
631 // This node is no longer the master, respond as all failed.
632 // TODO: we might want to wrap response in envelope
633 // to distinguish sw programming failure and hand over
634 // it make sense in the latter case to retry immediately.
Madan Jampani884d4432016-08-23 10:46:55 -0700635 message.respond(serializer.encode(allFailed));
Madan Jampani86940d92015-05-06 11:47:57 -0700636 return;
637 }
638
639 pendingResponses.put(operation.id(), message.sender());
640 storeBatchInternal(operation);
641 }
642 }
643
Madan Jampanic6d69f72016-07-15 15:47:12 -0700644 private class BackupOperation {
645 private final NodeId nodeId;
646 private final DeviceId deviceId;
647
648 public BackupOperation(NodeId nodeId, DeviceId deviceId) {
649 this.nodeId = nodeId;
650 this.deviceId = deviceId;
651 }
652
653 @Override
654 public int hashCode() {
655 return Objects.hash(nodeId, deviceId);
656 }
657
658 @Override
659 public boolean equals(Object other) {
660 if (other != null && other instanceof BackupOperation) {
661 BackupOperation that = (BackupOperation) other;
662 return this.nodeId.equals(that.nodeId) &&
663 this.deviceId.equals(that.deviceId);
664 } else {
665 return false;
666 }
667 }
668 }
669
Madan Jampanif7536ab2015-05-07 23:23:23 -0700670 private class InternalFlowTable implements ReplicaInfoEventListener {
Madan Jampani86940d92015-05-06 11:47:57 -0700671
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800672 //TODO replace the Map<V,V> with ExtendedSet
673 private final Map<DeviceId, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>
Madan Jampani5c3766c2015-06-02 15:54:41 -0700674 flowEntries = Maps.newConcurrentMap();
Madan Jampani86940d92015-05-06 11:47:57 -0700675
Madan Jampanic6d69f72016-07-15 15:47:12 -0700676 private final Map<BackupOperation, Long> lastBackupTimes = Maps.newConcurrentMap();
Madan Jampani86940d92015-05-06 11:47:57 -0700677 private final Map<DeviceId, Long> lastUpdateTimes = Maps.newConcurrentMap();
Madan Jampani86940d92015-05-06 11:47:57 -0700678
Madan Jampanif7536ab2015-05-07 23:23:23 -0700679 @Override
680 public void event(ReplicaInfoEvent event) {
Madan Jampani71c32ca2016-06-22 08:23:18 -0700681 eventHandler.execute(() -> handleEvent(event));
682 }
683
684 private void handleEvent(ReplicaInfoEvent event) {
Madan Jampanic6d69f72016-07-15 15:47:12 -0700685 DeviceId deviceId = event.subject();
sivachidambaram subramanian0b33c082017-04-06 13:26:44 +0530686 if (!mastershipService.isLocalMaster(deviceId)) {
Madan Jampania98bf932015-06-02 12:01:36 -0700687 return;
688 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700689 if (event.type() == MASTER_CHANGED) {
690 lastUpdateTimes.put(deviceId, System.currentTimeMillis());
Madan Jampanif7536ab2015-05-07 23:23:23 -0700691 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700692 backupSenderExecutor.schedule(this::backup, 0, TimeUnit.SECONDS);
Madan Jampanif7536ab2015-05-07 23:23:23 -0700693 }
694
Madan Jampaniadea8902015-06-04 17:39:45 -0700695 private void sendBackups(NodeId nodeId, Set<DeviceId> deviceIds) {
696 // split up the devices into smaller batches and send them separately.
697 Iterables.partition(deviceIds, FLOW_TABLE_BACKUP_BATCH_SIZE)
698 .forEach(ids -> backupFlowEntries(nodeId, Sets.newHashSet(ids)));
699 }
700
Madan Jampanif7536ab2015-05-07 23:23:23 -0700701 private void backupFlowEntries(NodeId nodeId, Set<DeviceId> deviceIds) {
Madan Jampaniadea8902015-06-04 17:39:45 -0700702 if (deviceIds.isEmpty()) {
703 return;
704 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700705 log.debug("Sending flowEntries for devices {} to {} for backup.", deviceIds, nodeId);
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800706 Map<DeviceId, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>
707 deviceFlowEntries = Maps.newConcurrentMap();
Ray Milkey2b6ff422016-08-26 13:03:15 -0700708 deviceIds.forEach(id -> deviceFlowEntries.put(id, getFlowTableCopy(id)));
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800709 clusterCommunicator.<Map<DeviceId,
710 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>,
711 Set<DeviceId>>
712 sendAndReceive(deviceFlowEntries,
713 FLOW_TABLE_BACKUP,
Madan Jampani884d4432016-08-23 10:46:55 -0700714 serializer::encode,
715 serializer::decode,
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800716 nodeId)
717 .whenComplete((backedupDevices, error) -> {
718 Set<DeviceId> devicesNotBackedup = error != null ?
719 deviceFlowEntries.keySet() :
720 Sets.difference(deviceFlowEntries.keySet(), backedupDevices);
721 if (devicesNotBackedup.size() > 0) {
Ray Milkey2b6ff422016-08-26 13:03:15 -0700722 log.warn("Failed to backup devices: {}. Reason: {}, Node: {}",
723 devicesNotBackedup, error != null ? error.getMessage() : "none",
724 nodeId);
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800725 }
726 if (backedupDevices != null) {
727 backedupDevices.forEach(id -> {
Madan Jampanic6d69f72016-07-15 15:47:12 -0700728 lastBackupTimes.put(new BackupOperation(nodeId, id), System.currentTimeMillis());
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800729 });
730 }
731 });
Madan Jampanif7536ab2015-05-07 23:23:23 -0700732 }
733
Madan Jampani86940d92015-05-06 11:47:57 -0700734 /**
735 * Returns the flow table for specified device.
736 *
737 * @param deviceId identifier of the device
738 * @return Map representing Flow Table of given device.
739 */
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800740 private Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> getFlowTable(DeviceId deviceId) {
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700741 if (persistenceEnabled) {
742 return flowEntries.computeIfAbsent(deviceId, id -> persistenceService
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800743 .<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>persistentMapBuilder()
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700744 .withName("FlowTable:" + deviceId.toString())
745 .withSerializer(new Serializer() {
746 @Override
747 public <T> byte[] encode(T object) {
Madan Jampani884d4432016-08-23 10:46:55 -0700748 return serializer.encode(object);
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700749 }
750
751 @Override
752 public <T> T decode(byte[] bytes) {
Madan Jampani884d4432016-08-23 10:46:55 -0700753 return serializer.decode(bytes);
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700754 }
Jordan Halterman2c83a102017-08-20 17:11:41 -0700755
756 @Override
757 public <T> T copy(T object) {
758 return serializer.copy(object);
759 }
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700760 })
761 .build());
762 } else {
763 return flowEntries.computeIfAbsent(deviceId, id -> Maps.newConcurrentMap());
764 }
Madan Jampani86940d92015-05-06 11:47:57 -0700765 }
766
Ray Milkey2b6ff422016-08-26 13:03:15 -0700767 private Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> getFlowTableCopy(DeviceId deviceId) {
768 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> copy = Maps.newHashMap();
769 if (persistenceEnabled) {
770 return flowEntries.computeIfAbsent(deviceId, id -> persistenceService
771 .<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>persistentMapBuilder()
772 .withName("FlowTable:" + deviceId.toString())
773 .withSerializer(new Serializer() {
774 @Override
775 public <T> byte[] encode(T object) {
776 return serializer.encode(object);
777 }
778
779 @Override
780 public <T> T decode(byte[] bytes) {
781 return serializer.decode(bytes);
782 }
Jordan Halterman2c83a102017-08-20 17:11:41 -0700783
784 @Override
785 public <T> T copy(T object) {
786 return serializer.copy(object);
787 }
Ray Milkey2b6ff422016-08-26 13:03:15 -0700788 })
789 .build());
790 } else {
791 flowEntries.computeIfAbsent(deviceId, id -> Maps.newConcurrentMap()).forEach((k, v) -> {
792 copy.put(k, Maps.newHashMap(v));
793 });
794 return copy;
795 }
796 }
797
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800798 private Map<StoredFlowEntry, StoredFlowEntry> getFlowEntriesInternal(DeviceId deviceId, FlowId flowId) {
799 return getFlowTable(deviceId).computeIfAbsent(flowId, id -> Maps.newConcurrentMap());
Madan Jampani86940d92015-05-06 11:47:57 -0700800 }
801
802 private StoredFlowEntry getFlowEntryInternal(FlowRule rule) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800803 return getFlowEntriesInternal(rule.deviceId(), rule.id()).get(rule);
Madan Jampani86940d92015-05-06 11:47:57 -0700804 }
805
806 private Set<FlowEntry> getFlowEntriesInternal(DeviceId deviceId) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800807 return getFlowTable(deviceId).values().stream()
808 .flatMap(m -> m.values().stream())
809 .collect(Collectors.toSet());
Madan Jampani86940d92015-05-06 11:47:57 -0700810 }
811
812 public StoredFlowEntry getFlowEntry(FlowRule rule) {
813 return getFlowEntryInternal(rule);
814 }
815
816 public Set<FlowEntry> getFlowEntries(DeviceId deviceId) {
817 return getFlowEntriesInternal(deviceId);
818 }
819
820 public void add(FlowEntry rule) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800821 getFlowEntriesInternal(rule.deviceId(), rule.id())
822 .compute((StoredFlowEntry) rule, (k, stored) -> {
823 //TODO compare stored and rule timestamps
824 //TODO the key is not updated
825 return (StoredFlowEntry) rule;
826 });
Madan Jampani86940d92015-05-06 11:47:57 -0700827 lastUpdateTimes.put(rule.deviceId(), System.currentTimeMillis());
828 }
829
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800830 public FlowEntry remove(DeviceId deviceId, FlowEntry rule) {
831 final AtomicReference<FlowEntry> removedRule = new AtomicReference<>();
832 getFlowEntriesInternal(rule.deviceId(), rule.id())
833 .computeIfPresent((StoredFlowEntry) rule, (k, stored) -> {
834 if (rule instanceof DefaultFlowEntry) {
835 DefaultFlowEntry toRemove = (DefaultFlowEntry) rule;
836 if (stored instanceof DefaultFlowEntry) {
837 DefaultFlowEntry storedEntry = (DefaultFlowEntry) stored;
838 if (toRemove.created() < storedEntry.created()) {
839 log.debug("Trying to remove more recent flow entry {} (stored: {})",
840 toRemove, stored);
841 // the key is not updated, removedRule remains null
842 return stored;
843 }
844 }
845 }
846 removedRule.set(stored);
847 return null;
848 });
849
850 if (removedRule.get() != null) {
Madan Jampani86940d92015-05-06 11:47:57 -0700851 lastUpdateTimes.put(deviceId, System.currentTimeMillis());
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800852 return removedRule.get();
853 } else {
854 return null;
Madan Jampani86940d92015-05-06 11:47:57 -0700855 }
856 }
857
Charles Chan0c7c43b2016-01-14 17:39:20 -0800858 public void purgeFlowRule(DeviceId deviceId) {
859 flowEntries.remove(deviceId);
860 }
861
Victor Silva139bca42016-08-18 11:46:35 -0300862 public void purgeFlowRules() {
863 flowEntries.clear();
864 }
865
Madan Jampanic6d69f72016-07-15 15:47:12 -0700866 private List<NodeId> getBackupNodes(DeviceId deviceId) {
867 // The returned backup node list is in the order of preference i.e. next likely master first.
868 List<NodeId> allPossibleBackupNodes = replicaInfoManager.getReplicaInfoFor(deviceId).backups();
869 return ImmutableList.copyOf(allPossibleBackupNodes)
870 .subList(0, Math.min(allPossibleBackupNodes.size(), backupCount));
Madan Jampani86940d92015-05-06 11:47:57 -0700871 }
872
873 private void backup() {
Madan Jampani86940d92015-05-06 11:47:57 -0700874 try {
Madan Jampani86940d92015-05-06 11:47:57 -0700875 // compute a mapping from node to the set of devices whose flow entries it should backup
876 Map<NodeId, Set<DeviceId>> devicesToBackupByNode = Maps.newHashMap();
Sho SHIMIZUa09e1bb2016-08-01 14:25:25 -0700877 flowEntries.keySet().forEach(deviceId -> {
Madan Jampanic6d69f72016-07-15 15:47:12 -0700878 List<NodeId> backupNodes = getBackupNodes(deviceId);
879 backupNodes.forEach(backupNode -> {
880 if (lastBackupTimes.getOrDefault(new BackupOperation(backupNode, deviceId), 0L)
881 < lastUpdateTimes.getOrDefault(deviceId, 0L)) {
882 devicesToBackupByNode.computeIfAbsent(backupNode,
883 nodeId -> Sets.newHashSet()).add(deviceId);
884 }
885 });
Madan Jampani86940d92015-05-06 11:47:57 -0700886 });
Madan Jampani86940d92015-05-06 11:47:57 -0700887 // send the device flow entries to their respective backup nodes
Madan Jampaniadea8902015-06-04 17:39:45 -0700888 devicesToBackupByNode.forEach(this::sendBackups);
Madan Jampani86940d92015-05-06 11:47:57 -0700889 } catch (Exception e) {
890 log.error("Backup failed.", e);
891 }
892 }
893
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800894 private Set<DeviceId> onBackupReceipt(Map<DeviceId,
895 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>> flowTables) {
Madan Jampani7267c552015-05-20 22:39:17 -0700896 log.debug("Received flowEntries for {} to backup", flowTables.keySet());
Madan Jampani654b58a2015-05-22 11:28:11 -0700897 Set<DeviceId> backedupDevices = Sets.newHashSet();
898 try {
Madan Jampania98bf932015-06-02 12:01:36 -0700899 flowTables.forEach((deviceId, deviceFlowTable) -> {
900 // Only process those devices are that not managed by the local node.
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800901 if (!Objects.equals(local, mastershipService.getMasterFor(deviceId))) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800902 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> backupFlowTable =
903 getFlowTable(deviceId);
Madan Jampania98bf932015-06-02 12:01:36 -0700904 backupFlowTable.clear();
905 backupFlowTable.putAll(deviceFlowTable);
906 backedupDevices.add(deviceId);
907 }
Madan Jampani08bf17b2015-05-06 16:25:26 -0700908 });
Madan Jampani654b58a2015-05-22 11:28:11 -0700909 } catch (Exception e) {
910 log.warn("Failure processing backup request", e);
911 }
912 return backedupDevices;
Madan Jampani86940d92015-05-06 11:47:57 -0700913 }
914 }
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700915
916 @Override
917 public FlowRuleEvent updateTableStatistics(DeviceId deviceId,
918 List<TableStatisticsEntry> tableStats) {
919 deviceTableStats.put(deviceId, tableStats);
920 return null;
921 }
922
923 @Override
924 public Iterable<TableStatisticsEntry> getTableStatistics(DeviceId deviceId) {
925 NodeId master = mastershipService.getMasterFor(deviceId);
926
927 if (master == null) {
928 log.debug("Failed to getTableStats: No master for {}", deviceId);
929 return Collections.emptyList();
930 }
931
932 List<TableStatisticsEntry> tableStats = deviceTableStats.get(deviceId);
933 if (tableStats == null) {
934 return Collections.emptyList();
935 }
936 return ImmutableList.copyOf(tableStats);
937 }
938
Patryk Konopka7e40c012017-06-06 13:38:06 +0200939 @Override
940 public long getActiveFlowRuleCount(DeviceId deviceId) {
941 return Streams.stream(getTableStatistics(deviceId))
942 .mapToLong(TableStatisticsEntry::activeFlowEntries)
943 .sum();
944 }
945
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700946 private class InternalTableStatsListener
947 implements EventuallyConsistentMapListener<DeviceId, List<TableStatisticsEntry>> {
948 @Override
949 public void event(EventuallyConsistentMapEvent<DeviceId,
950 List<TableStatisticsEntry>> event) {
951 //TODO: Generate an event to listeners (do we need?)
952 }
953 }
Madan Jampani86940d92015-05-06 11:47:57 -0700954}