blob: 699a4d3ae6314749eaa09d14a62e6df8c2418d90 [file] [log] [blame]
Madan Jampani86940d92015-05-06 11:47:57 -07001 /*
Brian O'Connor0a4e6742016-09-15 23:03:10 -07002 * Copyright 2014-present Open Networking Laboratory
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
Brian O'Connora3e5cd52015-12-05 15:59:19 -080018 import java.util.Collections;
Ray Milkey2b6ff422016-08-26 13:03:15 -070019 import java.util.Dictionary;
20 import java.util.HashSet;
21 import java.util.List;
22 import java.util.Map;
23 import java.util.Objects;
24 import java.util.Set;
25 import java.util.concurrent.ExecutorService;
26 import java.util.concurrent.Executors;
27 import java.util.concurrent.ScheduledExecutorService;
28 import java.util.concurrent.ScheduledFuture;
29 import java.util.concurrent.TimeUnit;
30 import java.util.concurrent.atomic.AtomicInteger;
31 import java.util.concurrent.atomic.AtomicReference;
32 import java.util.stream.Collectors;
33
34 import org.apache.felix.scr.annotations.Activate;
35 import org.apache.felix.scr.annotations.Component;
36 import org.apache.felix.scr.annotations.Deactivate;
37 import org.apache.felix.scr.annotations.Modified;
38 import org.apache.felix.scr.annotations.Property;
39 import org.apache.felix.scr.annotations.Reference;
40 import org.apache.felix.scr.annotations.ReferenceCardinality;
41 import org.apache.felix.scr.annotations.Service;
42 import org.onlab.util.KryoNamespace;
43 import org.onlab.util.Tools;
44 import org.onosproject.cfg.ComponentConfigService;
45 import org.onosproject.cluster.ClusterService;
46 import org.onosproject.cluster.NodeId;
47 import org.onosproject.core.CoreService;
48 import org.onosproject.core.IdGenerator;
49 import org.onosproject.mastership.MastershipService;
50 import org.onosproject.net.DeviceId;
51 import org.onosproject.net.device.DeviceService;
52 import org.onosproject.net.flow.CompletedBatchOperation;
53 import org.onosproject.net.flow.DefaultFlowEntry;
54 import org.onosproject.net.flow.FlowEntry;
55 import org.onosproject.net.flow.FlowEntry.FlowEntryState;
56 import org.onosproject.net.flow.FlowId;
57 import org.onosproject.net.flow.FlowRule;
58 import org.onosproject.net.flow.FlowRuleBatchEntry;
59 import org.onosproject.net.flow.FlowRuleBatchEntry.FlowRuleOperation;
60 import org.onosproject.net.flow.FlowRuleBatchEvent;
61 import org.onosproject.net.flow.FlowRuleBatchOperation;
62 import org.onosproject.net.flow.FlowRuleBatchRequest;
63 import org.onosproject.net.flow.FlowRuleEvent;
64 import org.onosproject.net.flow.FlowRuleEvent.Type;
65 import org.onosproject.net.flow.FlowRuleService;
66 import org.onosproject.net.flow.FlowRuleStore;
67 import org.onosproject.net.flow.FlowRuleStoreDelegate;
68 import org.onosproject.net.flow.StoredFlowEntry;
69 import org.onosproject.net.flow.TableStatisticsEntry;
70 import org.onosproject.persistence.PersistenceService;
71 import org.onosproject.store.AbstractStore;
72 import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
73 import org.onosproject.store.cluster.messaging.ClusterMessage;
74 import org.onosproject.store.cluster.messaging.ClusterMessageHandler;
75 import org.onosproject.store.flow.ReplicaInfoEvent;
76 import org.onosproject.store.flow.ReplicaInfoEventListener;
77 import org.onosproject.store.flow.ReplicaInfoService;
78 import org.onosproject.store.impl.MastershipBasedTimestamp;
79 import org.onosproject.store.serializers.KryoNamespaces;
80 import org.onosproject.store.service.EventuallyConsistentMap;
81 import org.onosproject.store.service.EventuallyConsistentMapEvent;
82 import org.onosproject.store.service.EventuallyConsistentMapListener;
83 import org.onosproject.store.service.Serializer;
84 import org.onosproject.store.service.StorageService;
85 import org.onosproject.store.service.WallClockTimestamp;
86 import org.osgi.service.component.ComponentContext;
87 import org.slf4j.Logger;
88
89 import com.google.common.collect.ImmutableList;
90 import com.google.common.collect.Iterables;
91 import com.google.common.collect.Maps;
92 import com.google.common.collect.Sets;
93 import com.google.common.util.concurrent.Futures;
Madan Jampani86940d92015-05-06 11:47:57 -070094
Brian O'Connora3e5cd52015-12-05 15:59:19 -080095 import static com.google.common.base.Strings.isNullOrEmpty;
Ray Milkey2b6ff422016-08-26 13:03:15 -070096 import static org.onlab.util.Tools.get;
97 import static org.onlab.util.Tools.groupedThreads;
98 import static org.onosproject.net.flow.FlowRuleEvent.Type.RULE_REMOVED;
99 import static org.onosproject.store.flow.ReplicaInfoEvent.Type.MASTER_CHANGED;
100 import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.APPLY_BATCH_FLOWS;
101 import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.FLOW_TABLE_BACKUP;
102 import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.GET_DEVICE_FLOW_ENTRIES;
103 import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.GET_FLOW_ENTRY;
104 import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.REMOTE_APPLY_COMPLETED;
105 import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.REMOVE_FLOW_ENTRY;
106 import 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;
120 private static final boolean DEFAULT_BACKUP_ENABLED = true;
Madan Jampanic6d69f72016-07-15 15:47:12 -0700121 private static final int DEFAULT_MAX_BACKUP_COUNT = 2;
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700122 private static final boolean DEFAULT_PERSISTENCE_ENABLED = false;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700123 private static final int DEFAULT_BACKUP_PERIOD_MILLIS = 2000;
Madan Jampani86940d92015-05-06 11:47:57 -0700124 private static final long FLOW_RULE_STORE_TIMEOUT_MILLIS = 5000;
Madan Jampaniadea8902015-06-04 17:39:45 -0700125 // number of devices whose flow entries will be backed up in one communication round
126 private static final int FLOW_TABLE_BACKUP_BATCH_SIZE = 1;
Madan Jampani86940d92015-05-06 11:47:57 -0700127
128 @Property(name = "msgHandlerPoolSize", intValue = MESSAGE_HANDLER_THREAD_POOL_SIZE,
129 label = "Number of threads in the message handler pool")
130 private int msgHandlerPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
131
132 @Property(name = "backupEnabled", boolValue = DEFAULT_BACKUP_ENABLED,
133 label = "Indicates whether backups are enabled or not")
Madan Jampanic6d69f72016-07-15 15:47:12 -0700134 private volatile boolean backupEnabled = DEFAULT_BACKUP_ENABLED;
Madan Jampani86940d92015-05-06 11:47:57 -0700135
Madan Jampani08bf17b2015-05-06 16:25:26 -0700136 @Property(name = "backupPeriod", intValue = DEFAULT_BACKUP_PERIOD_MILLIS,
137 label = "Delay in ms between successive backup runs")
138 private int backupPeriod = DEFAULT_BACKUP_PERIOD_MILLIS;
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700139 @Property(name = "persistenceEnabled", boolValue = false,
140 label = "Indicates whether or not changes in the flow table should be persisted to disk.")
141 private boolean persistenceEnabled = DEFAULT_PERSISTENCE_ENABLED;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700142
Madan Jampanic6d69f72016-07-15 15:47:12 -0700143 @Property(name = "backupCount", intValue = DEFAULT_MAX_BACKUP_COUNT,
144 label = "Max number of backup copies for each device")
145 private volatile int backupCount = DEFAULT_MAX_BACKUP_COUNT;
146
Madan Jampani86940d92015-05-06 11:47:57 -0700147 private InternalFlowTable flowTable = new InternalFlowTable();
148
149 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
150 protected ReplicaInfoService replicaInfoManager;
151
152 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
153 protected ClusterCommunicationService clusterCommunicator;
154
155 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
156 protected ClusterService clusterService;
157
158 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
159 protected DeviceService deviceService;
160
161 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
162 protected CoreService coreService;
163
164 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
165 protected ComponentConfigService configService;
166
167 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
168 protected MastershipService mastershipService;
169
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700170 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
171 protected PersistenceService persistenceService;
172
Madan Jampani86940d92015-05-06 11:47:57 -0700173 private Map<Long, NodeId> pendingResponses = Maps.newConcurrentMap();
174 private ExecutorService messageHandlingExecutor;
Madan Jampani71c32ca2016-06-22 08:23:18 -0700175 private ExecutorService eventHandler;
Madan Jampani86940d92015-05-06 11:47:57 -0700176
Madan Jampani08bf17b2015-05-06 16:25:26 -0700177 private ScheduledFuture<?> backupTask;
Madan Jampani86940d92015-05-06 11:47:57 -0700178 private final ScheduledExecutorService backupSenderExecutor =
HIGUCHI Yutad9e01052016-04-14 09:31:42 -0700179 Executors.newSingleThreadScheduledExecutor(groupedThreads("onos/flow", "backup-sender", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700180
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700181 private EventuallyConsistentMap<DeviceId, List<TableStatisticsEntry>> deviceTableStats;
182 private final EventuallyConsistentMapListener<DeviceId, List<TableStatisticsEntry>> tableStatsListener =
183 new InternalTableStatsListener();
184
185 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
186 protected StorageService storageService;
187
Madan Jampani884d4432016-08-23 10:46:55 -0700188 protected final Serializer serializer = Serializer.using(KryoNamespaces.API);
Madan Jampani86940d92015-05-06 11:47:57 -0700189
Madan Jampani884d4432016-08-23 10:46:55 -0700190 protected final KryoNamespace.Builder serializerBuilder = KryoNamespace.newBuilder()
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700191 .register(KryoNamespaces.API)
192 .register(MastershipBasedTimestamp.class);
193
194
Madan Jampani86940d92015-05-06 11:47:57 -0700195 private IdGenerator idGenerator;
196 private NodeId local;
197
198 @Activate
199 public void activate(ComponentContext context) {
200 configService.registerProperties(getClass());
201
202 idGenerator = coreService.getIdGenerator(FlowRuleService.FLOW_OP_TOPIC);
203
204 local = clusterService.getLocalNode().id();
205
Madan Jampani71c32ca2016-06-22 08:23:18 -0700206 eventHandler = Executors.newSingleThreadExecutor(
207 groupedThreads("onos/flow", "event-handler", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700208 messageHandlingExecutor = Executors.newFixedThreadPool(
HIGUCHI Yutad9e01052016-04-14 09:31:42 -0700209 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700210
211 registerMessageHandlers(messageHandlingExecutor);
212
Madan Jampani08bf17b2015-05-06 16:25:26 -0700213 if (backupEnabled) {
Madan Jampanif7536ab2015-05-07 23:23:23 -0700214 replicaInfoManager.addListener(flowTable);
Madan Jampani08bf17b2015-05-06 16:25:26 -0700215 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
216 flowTable::backup,
217 0,
218 backupPeriod,
219 TimeUnit.MILLISECONDS);
220 }
Madan Jampani86940d92015-05-06 11:47:57 -0700221
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700222 deviceTableStats = storageService.<DeviceId, List<TableStatisticsEntry>>eventuallyConsistentMapBuilder()
223 .withName("onos-flow-table-stats")
Madan Jampani884d4432016-08-23 10:46:55 -0700224 .withSerializer(serializerBuilder)
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700225 .withAntiEntropyPeriod(5, TimeUnit.SECONDS)
226 .withTimestampProvider((k, v) -> new WallClockTimestamp())
227 .withTombstonesDisabled()
228 .build();
229 deviceTableStats.addListener(tableStatsListener);
230
Madan Jampani86940d92015-05-06 11:47:57 -0700231 logConfig("Started");
232 }
233
234 @Deactivate
235 public void deactivate(ComponentContext context) {
Madan Jampanif7536ab2015-05-07 23:23:23 -0700236 if (backupEnabled) {
237 replicaInfoManager.removeListener(flowTable);
238 backupTask.cancel(true);
239 }
Madan Jampani86940d92015-05-06 11:47:57 -0700240 configService.unregisterProperties(getClass(), false);
241 unregisterMessageHandlers();
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700242 deviceTableStats.removeListener(tableStatsListener);
243 deviceTableStats.destroy();
Madan Jampani71c32ca2016-06-22 08:23:18 -0700244 eventHandler.shutdownNow();
Madan Jampani86940d92015-05-06 11:47:57 -0700245 messageHandlingExecutor.shutdownNow();
246 backupSenderExecutor.shutdownNow();
247 log.info("Stopped");
248 }
249
250 @SuppressWarnings("rawtypes")
251 @Modified
252 public void modified(ComponentContext context) {
253 if (context == null) {
254 backupEnabled = DEFAULT_BACKUP_ENABLED;
255 logConfig("Default config");
256 return;
257 }
258
259 Dictionary properties = context.getProperties();
260 int newPoolSize;
261 boolean newBackupEnabled;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700262 int newBackupPeriod;
Madan Jampanic6d69f72016-07-15 15:47:12 -0700263 int newBackupCount;
Madan Jampani86940d92015-05-06 11:47:57 -0700264 try {
265 String s = get(properties, "msgHandlerPoolSize");
266 newPoolSize = isNullOrEmpty(s) ? msgHandlerPoolSize : Integer.parseInt(s.trim());
267
268 s = get(properties, "backupEnabled");
269 newBackupEnabled = isNullOrEmpty(s) ? backupEnabled : Boolean.parseBoolean(s.trim());
270
Madan Jampani08bf17b2015-05-06 16:25:26 -0700271 s = get(properties, "backupPeriod");
272 newBackupPeriod = isNullOrEmpty(s) ? backupPeriod : Integer.parseInt(s.trim());
273
Madan Jampanic6d69f72016-07-15 15:47:12 -0700274 s = get(properties, "backupCount");
275 newBackupCount = isNullOrEmpty(s) ? backupCount : Integer.parseInt(s.trim());
Madan Jampani86940d92015-05-06 11:47:57 -0700276 } catch (NumberFormatException | ClassCastException e) {
277 newPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
278 newBackupEnabled = DEFAULT_BACKUP_ENABLED;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700279 newBackupPeriod = DEFAULT_BACKUP_PERIOD_MILLIS;
Madan Jampanic6d69f72016-07-15 15:47:12 -0700280 newBackupCount = DEFAULT_MAX_BACKUP_COUNT;
Madan Jampani86940d92015-05-06 11:47:57 -0700281 }
282
Madan Jampani08bf17b2015-05-06 16:25:26 -0700283 boolean restartBackupTask = false;
Madan Jampani86940d92015-05-06 11:47:57 -0700284 if (newBackupEnabled != backupEnabled) {
285 backupEnabled = newBackupEnabled;
Madan Jampanif7536ab2015-05-07 23:23:23 -0700286 if (!backupEnabled) {
287 replicaInfoManager.removeListener(flowTable);
288 if (backupTask != null) {
289 backupTask.cancel(false);
290 backupTask = null;
291 }
292 } else {
293 replicaInfoManager.addListener(flowTable);
Madan Jampani08bf17b2015-05-06 16:25:26 -0700294 }
295 restartBackupTask = backupEnabled;
296 }
297 if (newBackupPeriod != backupPeriod) {
298 backupPeriod = newBackupPeriod;
299 restartBackupTask = backupEnabled;
300 }
301 if (restartBackupTask) {
302 if (backupTask != null) {
303 // cancel previously running task
304 backupTask.cancel(false);
305 }
306 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
307 flowTable::backup,
308 0,
309 backupPeriod,
310 TimeUnit.MILLISECONDS);
Madan Jampani86940d92015-05-06 11:47:57 -0700311 }
312 if (newPoolSize != msgHandlerPoolSize) {
313 msgHandlerPoolSize = newPoolSize;
314 ExecutorService oldMsgHandler = messageHandlingExecutor;
315 messageHandlingExecutor = Executors.newFixedThreadPool(
HIGUCHI Yuta060da9a2016-03-11 19:16:35 -0800316 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700317
318 // replace previously registered handlers.
319 registerMessageHandlers(messageHandlingExecutor);
320 oldMsgHandler.shutdown();
321 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700322 if (backupCount != newBackupCount) {
323 backupCount = newBackupCount;
324 }
Madan Jampani86940d92015-05-06 11:47:57 -0700325 logConfig("Reconfigured");
326 }
327
328 private void registerMessageHandlers(ExecutorService executor) {
329
330 clusterCommunicator.addSubscriber(APPLY_BATCH_FLOWS, new OnStoreBatch(), executor);
331 clusterCommunicator.<FlowRuleBatchEvent>addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700332 REMOTE_APPLY_COMPLETED, serializer::decode, this::notifyDelegate, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700333 clusterCommunicator.addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700334 GET_FLOW_ENTRY, serializer::decode, flowTable::getFlowEntry, serializer::encode, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700335 clusterCommunicator.addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700336 GET_DEVICE_FLOW_ENTRIES, serializer::decode, flowTable::getFlowEntries, serializer::encode, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700337 clusterCommunicator.addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700338 REMOVE_FLOW_ENTRY, serializer::decode, this::removeFlowRuleInternal, serializer::encode, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700339 clusterCommunicator.addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700340 REMOVE_FLOW_ENTRY, serializer::decode, this::removeFlowRuleInternal, serializer::encode, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700341 clusterCommunicator.addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700342 FLOW_TABLE_BACKUP, serializer::decode, flowTable::onBackupReceipt, serializer::encode, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700343 }
344
345 private void unregisterMessageHandlers() {
346 clusterCommunicator.removeSubscriber(REMOVE_FLOW_ENTRY);
347 clusterCommunicator.removeSubscriber(GET_DEVICE_FLOW_ENTRIES);
348 clusterCommunicator.removeSubscriber(GET_FLOW_ENTRY);
349 clusterCommunicator.removeSubscriber(APPLY_BATCH_FLOWS);
350 clusterCommunicator.removeSubscriber(REMOTE_APPLY_COMPLETED);
351 clusterCommunicator.removeSubscriber(FLOW_TABLE_BACKUP);
352 }
353
354 private void logConfig(String prefix) {
Madan Jampanic6d69f72016-07-15 15:47:12 -0700355 log.info("{} with msgHandlerPoolSize = {}; backupEnabled = {}, backupPeriod = {}, backupCount = {}",
356 prefix, msgHandlerPoolSize, backupEnabled, backupPeriod, backupCount);
Madan Jampani86940d92015-05-06 11:47:57 -0700357 }
358
359 // This is not a efficient operation on a distributed sharded
360 // flow store. We need to revisit the need for this operation or at least
361 // make it device specific.
362 @Override
363 public int getFlowRuleCount() {
364 AtomicInteger sum = new AtomicInteger(0);
365 deviceService.getDevices().forEach(device -> sum.addAndGet(Iterables.size(getFlowEntries(device.id()))));
366 return sum.get();
367 }
368
369 @Override
370 public FlowEntry getFlowEntry(FlowRule rule) {
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700371 NodeId master = mastershipService.getMasterFor(rule.deviceId());
Madan Jampani86940d92015-05-06 11:47:57 -0700372
373 if (master == null) {
Thomas Vachuska88fd6902015-08-04 10:08:34 -0700374 log.debug("Failed to getFlowEntry: No master for {}", rule.deviceId());
Madan Jampani86940d92015-05-06 11:47:57 -0700375 return null;
376 }
377
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800378 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700379 return flowTable.getFlowEntry(rule);
380 }
381
382 log.trace("Forwarding getFlowEntry to {}, which is the primary (master) for device {}",
383 master, rule.deviceId());
384
385 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(rule,
386 FlowStoreMessageSubjects.GET_FLOW_ENTRY,
Madan Jampani884d4432016-08-23 10:46:55 -0700387 serializer::encode,
388 serializer::decode,
Madan Jampani86940d92015-05-06 11:47:57 -0700389 master),
390 FLOW_RULE_STORE_TIMEOUT_MILLIS,
391 TimeUnit.MILLISECONDS,
392 null);
393 }
394
395 @Override
396 public Iterable<FlowEntry> getFlowEntries(DeviceId deviceId) {
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700397 NodeId master = mastershipService.getMasterFor(deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700398
399 if (master == null) {
Thomas Vachuska88fd6902015-08-04 10:08:34 -0700400 log.debug("Failed to getFlowEntries: No master for {}", deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700401 return Collections.emptyList();
402 }
403
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800404 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700405 return flowTable.getFlowEntries(deviceId);
406 }
407
408 log.trace("Forwarding getFlowEntries to {}, which is the primary (master) for device {}",
409 master, deviceId);
410
411 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(deviceId,
412 FlowStoreMessageSubjects.GET_DEVICE_FLOW_ENTRIES,
Madan Jampani884d4432016-08-23 10:46:55 -0700413 serializer::encode,
414 serializer::decode,
Madan Jampani86940d92015-05-06 11:47:57 -0700415 master),
416 FLOW_RULE_STORE_TIMEOUT_MILLIS,
417 TimeUnit.MILLISECONDS,
418 Collections.emptyList());
419 }
420
421 @Override
422 public void storeFlowRule(FlowRule rule) {
423 storeBatch(new FlowRuleBatchOperation(
Sho SHIMIZU98ffca82015-05-11 08:39:24 -0700424 Collections.singletonList(new FlowRuleBatchEntry(FlowRuleOperation.ADD, rule)),
Madan Jampani86940d92015-05-06 11:47:57 -0700425 rule.deviceId(), idGenerator.getNewId()));
426 }
427
428 @Override
429 public void storeBatch(FlowRuleBatchOperation operation) {
430 if (operation.getOperations().isEmpty()) {
431 notifyDelegate(FlowRuleBatchEvent.completed(
432 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
433 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
434 return;
435 }
436
437 DeviceId deviceId = operation.deviceId();
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700438 NodeId master = mastershipService.getMasterFor(deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700439
440 if (master == null) {
441 log.warn("No master for {} : flows will be marked for removal", deviceId);
442
443 updateStoreInternal(operation);
444
445 notifyDelegate(FlowRuleBatchEvent.completed(
446 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
447 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
448 return;
449 }
450
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800451 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700452 storeBatchInternal(operation);
453 return;
454 }
455
456 log.trace("Forwarding storeBatch to {}, which is the primary (master) for device {}",
457 master, deviceId);
458
Madan Jampani175e8fd2015-05-20 14:10:45 -0700459 clusterCommunicator.unicast(operation,
460 APPLY_BATCH_FLOWS,
Madan Jampani884d4432016-08-23 10:46:55 -0700461 serializer::encode,
Madan Jampani175e8fd2015-05-20 14:10:45 -0700462 master)
463 .whenComplete((result, error) -> {
Thomas Vachuskaa267ce42015-05-27 16:14:23 -0700464 if (error != null) {
Madan Jampani15f1bc42015-05-28 10:51:52 -0700465 log.warn("Failed to storeBatch: {} to {}", operation, master, error);
Madan Jampani86940d92015-05-06 11:47:57 -0700466
Thomas Vachuskaa267ce42015-05-27 16:14:23 -0700467 Set<FlowRule> allFailures = operation.getOperations()
468 .stream()
469 .map(op -> op.target())
470 .collect(Collectors.toSet());
Madan Jampani86940d92015-05-06 11:47:57 -0700471
Thomas Vachuskaa267ce42015-05-27 16:14:23 -0700472 notifyDelegate(FlowRuleBatchEvent.completed(
473 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
474 new CompletedBatchOperation(false, allFailures, deviceId)));
475 }
Madan Jampani175e8fd2015-05-20 14:10:45 -0700476 });
Madan Jampani86940d92015-05-06 11:47:57 -0700477 }
478
479 private void storeBatchInternal(FlowRuleBatchOperation operation) {
480
481 final DeviceId did = operation.deviceId();
482 //final Collection<FlowEntry> ft = flowTable.getFlowEntries(did);
483 Set<FlowRuleBatchEntry> currentOps = updateStoreInternal(operation);
484 if (currentOps.isEmpty()) {
485 batchOperationComplete(FlowRuleBatchEvent.completed(
486 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
487 new CompletedBatchOperation(true, Collections.emptySet(), did)));
488 return;
489 }
490
491 notifyDelegate(FlowRuleBatchEvent.requested(new
492 FlowRuleBatchRequest(operation.id(),
493 currentOps), operation.deviceId()));
494 }
495
496 private Set<FlowRuleBatchEntry> updateStoreInternal(FlowRuleBatchOperation operation) {
497 return operation.getOperations().stream().map(
498 op -> {
499 StoredFlowEntry entry;
500 switch (op.operator()) {
501 case ADD:
502 entry = new DefaultFlowEntry(op.target());
503 // always add requested FlowRule
504 // Note: 2 equal FlowEntry may have different treatment
505 flowTable.remove(entry.deviceId(), entry);
506 flowTable.add(entry);
507
508 return op;
509 case REMOVE:
510 entry = flowTable.getFlowEntry(op.target());
511 if (entry != null) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800512 //FIXME modification of "stored" flow entry outside of flow table
Madan Jampani86940d92015-05-06 11:47:57 -0700513 entry.setState(FlowEntryState.PENDING_REMOVE);
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800514 log.debug("Setting state of rule to pending remove: {}", entry);
Madan Jampani86940d92015-05-06 11:47:57 -0700515 return op;
516 }
517 break;
518 case MODIFY:
519 //TODO: figure this out at some point
520 break;
521 default:
522 log.warn("Unknown flow operation operator: {}", op.operator());
523 }
524 return null;
525 }
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800526 ).filter(Objects::nonNull).collect(Collectors.toSet());
Madan Jampani86940d92015-05-06 11:47:57 -0700527 }
528
529 @Override
530 public void deleteFlowRule(FlowRule rule) {
531 storeBatch(
532 new FlowRuleBatchOperation(
Sho SHIMIZU98ffca82015-05-11 08:39:24 -0700533 Collections.singletonList(
Madan Jampani86940d92015-05-06 11:47:57 -0700534 new FlowRuleBatchEntry(
535 FlowRuleOperation.REMOVE,
536 rule)), rule.deviceId(), idGenerator.getNewId()));
537 }
538
539 @Override
Charles Chan93fa7272016-01-26 22:27:02 -0800540 public FlowRuleEvent pendingFlowRule(FlowEntry rule) {
541 if (mastershipService.isLocalMaster(rule.deviceId())) {
542 StoredFlowEntry stored = flowTable.getFlowEntry(rule);
543 if (stored != null &&
544 stored.state() != FlowEntryState.PENDING_ADD) {
545 stored.setState(FlowEntryState.PENDING_ADD);
546 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
547 }
548 }
549 return null;
550 }
551
552 @Override
Madan Jampani86940d92015-05-06 11:47:57 -0700553 public FlowRuleEvent addOrUpdateFlowRule(FlowEntry rule) {
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700554 NodeId master = mastershipService.getMasterFor(rule.deviceId());
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800555 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700556 return addOrUpdateFlowRuleInternal(rule);
557 }
558
559 log.warn("Tried to update FlowRule {} state,"
560 + " while the Node was not the master.", rule);
561 return null;
562 }
563
564 private FlowRuleEvent addOrUpdateFlowRuleInternal(FlowEntry rule) {
565 // check if this new rule is an update to an existing entry
566 StoredFlowEntry stored = flowTable.getFlowEntry(rule);
567 if (stored != null) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800568 //FIXME modification of "stored" flow entry outside of flow table
Madan Jampani86940d92015-05-06 11:47:57 -0700569 stored.setBytes(rule.bytes());
Thiago Santos877914d2016-07-20 18:29:29 -0300570 stored.setLife(rule.life(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
Sangsik Yoonb1b823f2016-05-16 18:55:39 +0900571 stored.setLiveType(rule.liveType());
Madan Jampani86940d92015-05-06 11:47:57 -0700572 stored.setPackets(rule.packets());
Jonathan Hart89e981f2016-01-04 13:59:55 -0800573 stored.setLastSeen();
Madan Jampani86940d92015-05-06 11:47:57 -0700574 if (stored.state() == FlowEntryState.PENDING_ADD) {
575 stored.setState(FlowEntryState.ADDED);
576 return new FlowRuleEvent(Type.RULE_ADDED, rule);
577 }
578 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
579 }
580
581 // TODO: Confirm if this behavior is correct. See SimpleFlowRuleStore
582 // TODO: also update backup if the behavior is correct.
583 flowTable.add(rule);
584 return null;
585 }
586
587 @Override
588 public FlowRuleEvent removeFlowRule(FlowEntry rule) {
589 final DeviceId deviceId = rule.deviceId();
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700590 NodeId master = mastershipService.getMasterFor(deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700591
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800592 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700593 // bypass and handle it locally
594 return removeFlowRuleInternal(rule);
595 }
596
597 if (master == null) {
598 log.warn("Failed to removeFlowRule: No master for {}", deviceId);
599 // TODO: revisit if this should be null (="no-op") or Exception
600 return null;
601 }
602
603 log.trace("Forwarding removeFlowRule to {}, which is the master for device {}",
604 master, deviceId);
605
Madan Jampani222229e2016-07-14 10:43:25 -0700606 return Futures.getUnchecked(clusterCommunicator.sendAndReceive(
Madan Jampani86940d92015-05-06 11:47:57 -0700607 rule,
608 REMOVE_FLOW_ENTRY,
Madan Jampani884d4432016-08-23 10:46:55 -0700609 serializer::encode,
610 serializer::decode,
Madan Jampani222229e2016-07-14 10:43:25 -0700611 master));
Madan Jampani86940d92015-05-06 11:47:57 -0700612 }
613
614 private FlowRuleEvent removeFlowRuleInternal(FlowEntry rule) {
615 final DeviceId deviceId = rule.deviceId();
616 // 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 -0800617 final FlowEntry removed = flowTable.remove(deviceId, rule);
618 // rule may be partial rule that is missing treatment, we should use rule from store instead
619 return removed != null ? new FlowRuleEvent(RULE_REMOVED, removed) : null;
Madan Jampani86940d92015-05-06 11:47:57 -0700620 }
621
622 @Override
Charles Chan0c7c43b2016-01-14 17:39:20 -0800623 public void purgeFlowRule(DeviceId deviceId) {
624 flowTable.purgeFlowRule(deviceId);
625 }
626
627 @Override
Victor Silva139bca42016-08-18 11:46:35 -0300628 public void purgeFlowRules() {
629 flowTable.purgeFlowRules();
630 }
631
632 @Override
Madan Jampani86940d92015-05-06 11:47:57 -0700633 public void batchOperationComplete(FlowRuleBatchEvent event) {
634 //FIXME: need a per device pending response
635 NodeId nodeId = pendingResponses.remove(event.subject().batchId());
636 if (nodeId == null) {
637 notifyDelegate(event);
638 } else {
639 // TODO check unicast return value
Madan Jampani884d4432016-08-23 10:46:55 -0700640 clusterCommunicator.unicast(event, REMOTE_APPLY_COMPLETED, serializer::encode, nodeId);
Madan Jampani86940d92015-05-06 11:47:57 -0700641 //error log: log.warn("Failed to respond to peer for batch operation result");
642 }
643 }
644
645 private final class OnStoreBatch implements ClusterMessageHandler {
646
647 @Override
648 public void handle(final ClusterMessage message) {
Madan Jampani884d4432016-08-23 10:46:55 -0700649 FlowRuleBatchOperation operation = serializer.decode(message.payload());
Madan Jampani86940d92015-05-06 11:47:57 -0700650 log.debug("received batch request {}", operation);
651
652 final DeviceId deviceId = operation.deviceId();
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700653 NodeId master = mastershipService.getMasterFor(deviceId);
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800654 if (!Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700655 Set<FlowRule> failures = new HashSet<>(operation.size());
656 for (FlowRuleBatchEntry op : operation.getOperations()) {
657 failures.add(op.target());
658 }
659 CompletedBatchOperation allFailed = new CompletedBatchOperation(false, failures, deviceId);
660 // This node is no longer the master, respond as all failed.
661 // TODO: we might want to wrap response in envelope
662 // to distinguish sw programming failure and hand over
663 // it make sense in the latter case to retry immediately.
Madan Jampani884d4432016-08-23 10:46:55 -0700664 message.respond(serializer.encode(allFailed));
Madan Jampani86940d92015-05-06 11:47:57 -0700665 return;
666 }
667
668 pendingResponses.put(operation.id(), message.sender());
669 storeBatchInternal(operation);
670 }
671 }
672
Madan Jampanic6d69f72016-07-15 15:47:12 -0700673 private class BackupOperation {
674 private final NodeId nodeId;
675 private final DeviceId deviceId;
676
677 public BackupOperation(NodeId nodeId, DeviceId deviceId) {
678 this.nodeId = nodeId;
679 this.deviceId = deviceId;
680 }
681
682 @Override
683 public int hashCode() {
684 return Objects.hash(nodeId, deviceId);
685 }
686
687 @Override
688 public boolean equals(Object other) {
689 if (other != null && other instanceof BackupOperation) {
690 BackupOperation that = (BackupOperation) other;
691 return this.nodeId.equals(that.nodeId) &&
692 this.deviceId.equals(that.deviceId);
693 } else {
694 return false;
695 }
696 }
697 }
698
Madan Jampanif7536ab2015-05-07 23:23:23 -0700699 private class InternalFlowTable implements ReplicaInfoEventListener {
Madan Jampani86940d92015-05-06 11:47:57 -0700700
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800701 //TODO replace the Map<V,V> with ExtendedSet
702 private final Map<DeviceId, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>
Madan Jampani5c3766c2015-06-02 15:54:41 -0700703 flowEntries = Maps.newConcurrentMap();
Madan Jampani86940d92015-05-06 11:47:57 -0700704
Madan Jampanic6d69f72016-07-15 15:47:12 -0700705 private final Map<BackupOperation, Long> lastBackupTimes = Maps.newConcurrentMap();
Madan Jampani86940d92015-05-06 11:47:57 -0700706 private final Map<DeviceId, Long> lastUpdateTimes = Maps.newConcurrentMap();
Madan Jampani86940d92015-05-06 11:47:57 -0700707
Madan Jampanif7536ab2015-05-07 23:23:23 -0700708 @Override
709 public void event(ReplicaInfoEvent event) {
Madan Jampani71c32ca2016-06-22 08:23:18 -0700710 eventHandler.execute(() -> handleEvent(event));
711 }
712
713 private void handleEvent(ReplicaInfoEvent event) {
Madan Jampanic6d69f72016-07-15 15:47:12 -0700714 DeviceId deviceId = event.subject();
715 if (!backupEnabled || !mastershipService.isLocalMaster(deviceId)) {
Madan Jampania98bf932015-06-02 12:01:36 -0700716 return;
717 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700718 if (event.type() == MASTER_CHANGED) {
719 lastUpdateTimes.put(deviceId, System.currentTimeMillis());
Madan Jampanif7536ab2015-05-07 23:23:23 -0700720 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700721 backupSenderExecutor.schedule(this::backup, 0, TimeUnit.SECONDS);
Madan Jampanif7536ab2015-05-07 23:23:23 -0700722 }
723
Madan Jampaniadea8902015-06-04 17:39:45 -0700724 private void sendBackups(NodeId nodeId, Set<DeviceId> deviceIds) {
725 // split up the devices into smaller batches and send them separately.
726 Iterables.partition(deviceIds, FLOW_TABLE_BACKUP_BATCH_SIZE)
727 .forEach(ids -> backupFlowEntries(nodeId, Sets.newHashSet(ids)));
728 }
729
Madan Jampanif7536ab2015-05-07 23:23:23 -0700730 private void backupFlowEntries(NodeId nodeId, Set<DeviceId> deviceIds) {
Madan Jampaniadea8902015-06-04 17:39:45 -0700731 if (deviceIds.isEmpty()) {
732 return;
733 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700734 log.debug("Sending flowEntries for devices {} to {} for backup.", deviceIds, nodeId);
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800735 Map<DeviceId, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>
736 deviceFlowEntries = Maps.newConcurrentMap();
Ray Milkey2b6ff422016-08-26 13:03:15 -0700737 deviceIds.forEach(id -> deviceFlowEntries.put(id, getFlowTableCopy(id)));
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800738 clusterCommunicator.<Map<DeviceId,
739 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>,
740 Set<DeviceId>>
741 sendAndReceive(deviceFlowEntries,
742 FLOW_TABLE_BACKUP,
Madan Jampani884d4432016-08-23 10:46:55 -0700743 serializer::encode,
744 serializer::decode,
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800745 nodeId)
746 .whenComplete((backedupDevices, error) -> {
747 Set<DeviceId> devicesNotBackedup = error != null ?
748 deviceFlowEntries.keySet() :
749 Sets.difference(deviceFlowEntries.keySet(), backedupDevices);
750 if (devicesNotBackedup.size() > 0) {
Ray Milkey2b6ff422016-08-26 13:03:15 -0700751 log.warn("Failed to backup devices: {}. Reason: {}, Node: {}",
752 devicesNotBackedup, error != null ? error.getMessage() : "none",
753 nodeId);
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800754 }
755 if (backedupDevices != null) {
756 backedupDevices.forEach(id -> {
Madan Jampanic6d69f72016-07-15 15:47:12 -0700757 lastBackupTimes.put(new BackupOperation(nodeId, id), System.currentTimeMillis());
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800758 });
759 }
760 });
Madan Jampanif7536ab2015-05-07 23:23:23 -0700761 }
762
Madan Jampani86940d92015-05-06 11:47:57 -0700763 /**
764 * Returns the flow table for specified device.
765 *
766 * @param deviceId identifier of the device
767 * @return Map representing Flow Table of given device.
768 */
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800769 private Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> getFlowTable(DeviceId deviceId) {
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700770 if (persistenceEnabled) {
771 return flowEntries.computeIfAbsent(deviceId, id -> persistenceService
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800772 .<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>persistentMapBuilder()
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700773 .withName("FlowTable:" + deviceId.toString())
774 .withSerializer(new Serializer() {
775 @Override
776 public <T> byte[] encode(T object) {
Madan Jampani884d4432016-08-23 10:46:55 -0700777 return serializer.encode(object);
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700778 }
779
780 @Override
781 public <T> T decode(byte[] bytes) {
Madan Jampani884d4432016-08-23 10:46:55 -0700782 return serializer.decode(bytes);
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700783 }
784 })
785 .build());
786 } else {
787 return flowEntries.computeIfAbsent(deviceId, id -> Maps.newConcurrentMap());
788 }
Madan Jampani86940d92015-05-06 11:47:57 -0700789 }
790
Ray Milkey2b6ff422016-08-26 13:03:15 -0700791 private Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> getFlowTableCopy(DeviceId deviceId) {
792 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> copy = Maps.newHashMap();
793 if (persistenceEnabled) {
794 return flowEntries.computeIfAbsent(deviceId, id -> persistenceService
795 .<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>persistentMapBuilder()
796 .withName("FlowTable:" + deviceId.toString())
797 .withSerializer(new Serializer() {
798 @Override
799 public <T> byte[] encode(T object) {
800 return serializer.encode(object);
801 }
802
803 @Override
804 public <T> T decode(byte[] bytes) {
805 return serializer.decode(bytes);
806 }
807 })
808 .build());
809 } else {
810 flowEntries.computeIfAbsent(deviceId, id -> Maps.newConcurrentMap()).forEach((k, v) -> {
811 copy.put(k, Maps.newHashMap(v));
812 });
813 return copy;
814 }
815 }
816
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800817 private Map<StoredFlowEntry, StoredFlowEntry> getFlowEntriesInternal(DeviceId deviceId, FlowId flowId) {
818 return getFlowTable(deviceId).computeIfAbsent(flowId, id -> Maps.newConcurrentMap());
Madan Jampani86940d92015-05-06 11:47:57 -0700819 }
820
821 private StoredFlowEntry getFlowEntryInternal(FlowRule rule) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800822 return getFlowEntriesInternal(rule.deviceId(), rule.id()).get(rule);
Madan Jampani86940d92015-05-06 11:47:57 -0700823 }
824
825 private Set<FlowEntry> getFlowEntriesInternal(DeviceId deviceId) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800826 return getFlowTable(deviceId).values().stream()
827 .flatMap(m -> m.values().stream())
828 .collect(Collectors.toSet());
Madan Jampani86940d92015-05-06 11:47:57 -0700829 }
830
831 public StoredFlowEntry getFlowEntry(FlowRule rule) {
832 return getFlowEntryInternal(rule);
833 }
834
835 public Set<FlowEntry> getFlowEntries(DeviceId deviceId) {
836 return getFlowEntriesInternal(deviceId);
837 }
838
839 public void add(FlowEntry rule) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800840 getFlowEntriesInternal(rule.deviceId(), rule.id())
841 .compute((StoredFlowEntry) rule, (k, stored) -> {
842 //TODO compare stored and rule timestamps
843 //TODO the key is not updated
844 return (StoredFlowEntry) rule;
845 });
Madan Jampani86940d92015-05-06 11:47:57 -0700846 lastUpdateTimes.put(rule.deviceId(), System.currentTimeMillis());
847 }
848
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800849 public FlowEntry remove(DeviceId deviceId, FlowEntry rule) {
850 final AtomicReference<FlowEntry> removedRule = new AtomicReference<>();
851 getFlowEntriesInternal(rule.deviceId(), rule.id())
852 .computeIfPresent((StoredFlowEntry) rule, (k, stored) -> {
853 if (rule instanceof DefaultFlowEntry) {
854 DefaultFlowEntry toRemove = (DefaultFlowEntry) rule;
855 if (stored instanceof DefaultFlowEntry) {
856 DefaultFlowEntry storedEntry = (DefaultFlowEntry) stored;
857 if (toRemove.created() < storedEntry.created()) {
858 log.debug("Trying to remove more recent flow entry {} (stored: {})",
859 toRemove, stored);
860 // the key is not updated, removedRule remains null
861 return stored;
862 }
863 }
864 }
865 removedRule.set(stored);
866 return null;
867 });
868
869 if (removedRule.get() != null) {
Madan Jampani86940d92015-05-06 11:47:57 -0700870 lastUpdateTimes.put(deviceId, System.currentTimeMillis());
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800871 return removedRule.get();
872 } else {
873 return null;
Madan Jampani86940d92015-05-06 11:47:57 -0700874 }
875 }
876
Charles Chan0c7c43b2016-01-14 17:39:20 -0800877 public void purgeFlowRule(DeviceId deviceId) {
878 flowEntries.remove(deviceId);
879 }
880
Victor Silva139bca42016-08-18 11:46:35 -0300881 public void purgeFlowRules() {
882 flowEntries.clear();
883 }
884
Madan Jampanic6d69f72016-07-15 15:47:12 -0700885 private List<NodeId> getBackupNodes(DeviceId deviceId) {
886 // The returned backup node list is in the order of preference i.e. next likely master first.
887 List<NodeId> allPossibleBackupNodes = replicaInfoManager.getReplicaInfoFor(deviceId).backups();
888 return ImmutableList.copyOf(allPossibleBackupNodes)
889 .subList(0, Math.min(allPossibleBackupNodes.size(), backupCount));
Madan Jampani86940d92015-05-06 11:47:57 -0700890 }
891
892 private void backup() {
Madan Jampani08bf17b2015-05-06 16:25:26 -0700893 if (!backupEnabled) {
894 return;
895 }
Madan Jampani86940d92015-05-06 11:47:57 -0700896 try {
Madan Jampani86940d92015-05-06 11:47:57 -0700897 // compute a mapping from node to the set of devices whose flow entries it should backup
898 Map<NodeId, Set<DeviceId>> devicesToBackupByNode = Maps.newHashMap();
Sho SHIMIZUa09e1bb2016-08-01 14:25:25 -0700899 flowEntries.keySet().forEach(deviceId -> {
Madan Jampanic6d69f72016-07-15 15:47:12 -0700900 List<NodeId> backupNodes = getBackupNodes(deviceId);
901 backupNodes.forEach(backupNode -> {
902 if (lastBackupTimes.getOrDefault(new BackupOperation(backupNode, deviceId), 0L)
903 < lastUpdateTimes.getOrDefault(deviceId, 0L)) {
904 devicesToBackupByNode.computeIfAbsent(backupNode,
905 nodeId -> Sets.newHashSet()).add(deviceId);
906 }
907 });
Madan Jampani86940d92015-05-06 11:47:57 -0700908 });
Madan Jampani86940d92015-05-06 11:47:57 -0700909 // send the device flow entries to their respective backup nodes
Madan Jampaniadea8902015-06-04 17:39:45 -0700910 devicesToBackupByNode.forEach(this::sendBackups);
Madan Jampani86940d92015-05-06 11:47:57 -0700911 } catch (Exception e) {
912 log.error("Backup failed.", e);
913 }
914 }
915
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800916 private Set<DeviceId> onBackupReceipt(Map<DeviceId,
917 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>> flowTables) {
Madan Jampani7267c552015-05-20 22:39:17 -0700918 log.debug("Received flowEntries for {} to backup", flowTables.keySet());
Madan Jampani654b58a2015-05-22 11:28:11 -0700919 Set<DeviceId> backedupDevices = Sets.newHashSet();
920 try {
Madan Jampania98bf932015-06-02 12:01:36 -0700921 flowTables.forEach((deviceId, deviceFlowTable) -> {
922 // Only process those devices are that not managed by the local node.
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800923 if (!Objects.equals(local, mastershipService.getMasterFor(deviceId))) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800924 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> backupFlowTable =
925 getFlowTable(deviceId);
Madan Jampania98bf932015-06-02 12:01:36 -0700926 backupFlowTable.clear();
927 backupFlowTable.putAll(deviceFlowTable);
928 backedupDevices.add(deviceId);
929 }
Madan Jampani08bf17b2015-05-06 16:25:26 -0700930 });
Madan Jampani654b58a2015-05-22 11:28:11 -0700931 } catch (Exception e) {
932 log.warn("Failure processing backup request", e);
933 }
934 return backedupDevices;
Madan Jampani86940d92015-05-06 11:47:57 -0700935 }
936 }
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700937
938 @Override
939 public FlowRuleEvent updateTableStatistics(DeviceId deviceId,
940 List<TableStatisticsEntry> tableStats) {
941 deviceTableStats.put(deviceId, tableStats);
942 return null;
943 }
944
945 @Override
946 public Iterable<TableStatisticsEntry> getTableStatistics(DeviceId deviceId) {
947 NodeId master = mastershipService.getMasterFor(deviceId);
948
949 if (master == null) {
950 log.debug("Failed to getTableStats: No master for {}", deviceId);
951 return Collections.emptyList();
952 }
953
954 List<TableStatisticsEntry> tableStats = deviceTableStats.get(deviceId);
955 if (tableStats == null) {
956 return Collections.emptyList();
957 }
958 return ImmutableList.copyOf(tableStats);
959 }
960
961 private class InternalTableStatsListener
962 implements EventuallyConsistentMapListener<DeviceId, List<TableStatisticsEntry>> {
963 @Override
964 public void event(EventuallyConsistentMapEvent<DeviceId,
965 List<TableStatisticsEntry>> event) {
966 //TODO: Generate an event to listeners (do we need?)
967 }
968 }
Madan Jampani86940d92015-05-06 11:47:57 -0700969}