blob: 0f7e9806f10801b22636dd22e5abb43474fb5119 [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
Patryk Konopka7e40c012017-06-06 13:38:06 +020034 import com.google.common.collect.Streams;
Ray Milkey2b6ff422016-08-26 13:03:15 -070035 import org.apache.felix.scr.annotations.Activate;
36 import org.apache.felix.scr.annotations.Component;
37 import org.apache.felix.scr.annotations.Deactivate;
38 import org.apache.felix.scr.annotations.Modified;
39 import org.apache.felix.scr.annotations.Property;
40 import org.apache.felix.scr.annotations.Reference;
41 import org.apache.felix.scr.annotations.ReferenceCardinality;
42 import org.apache.felix.scr.annotations.Service;
43 import org.onlab.util.KryoNamespace;
44 import org.onlab.util.Tools;
45 import org.onosproject.cfg.ComponentConfigService;
46 import org.onosproject.cluster.ClusterService;
47 import org.onosproject.cluster.NodeId;
48 import org.onosproject.core.CoreService;
49 import org.onosproject.core.IdGenerator;
50 import org.onosproject.mastership.MastershipService;
51 import org.onosproject.net.DeviceId;
52 import org.onosproject.net.device.DeviceService;
53 import org.onosproject.net.flow.CompletedBatchOperation;
54 import org.onosproject.net.flow.DefaultFlowEntry;
55 import org.onosproject.net.flow.FlowEntry;
56 import org.onosproject.net.flow.FlowEntry.FlowEntryState;
57 import org.onosproject.net.flow.FlowId;
58 import org.onosproject.net.flow.FlowRule;
59 import org.onosproject.net.flow.FlowRuleBatchEntry;
60 import org.onosproject.net.flow.FlowRuleBatchEntry.FlowRuleOperation;
61 import org.onosproject.net.flow.FlowRuleBatchEvent;
62 import org.onosproject.net.flow.FlowRuleBatchOperation;
63 import org.onosproject.net.flow.FlowRuleBatchRequest;
64 import org.onosproject.net.flow.FlowRuleEvent;
65 import org.onosproject.net.flow.FlowRuleEvent.Type;
66 import org.onosproject.net.flow.FlowRuleService;
67 import org.onosproject.net.flow.FlowRuleStore;
68 import org.onosproject.net.flow.FlowRuleStoreDelegate;
69 import org.onosproject.net.flow.StoredFlowEntry;
70 import org.onosproject.net.flow.TableStatisticsEntry;
71 import org.onosproject.persistence.PersistenceService;
72 import org.onosproject.store.AbstractStore;
73 import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
74 import org.onosproject.store.cluster.messaging.ClusterMessage;
75 import org.onosproject.store.cluster.messaging.ClusterMessageHandler;
76 import org.onosproject.store.flow.ReplicaInfoEvent;
77 import org.onosproject.store.flow.ReplicaInfoEventListener;
78 import org.onosproject.store.flow.ReplicaInfoService;
79 import org.onosproject.store.impl.MastershipBasedTimestamp;
80 import org.onosproject.store.serializers.KryoNamespaces;
81 import org.onosproject.store.service.EventuallyConsistentMap;
82 import org.onosproject.store.service.EventuallyConsistentMapEvent;
83 import org.onosproject.store.service.EventuallyConsistentMapListener;
84 import org.onosproject.store.service.Serializer;
85 import org.onosproject.store.service.StorageService;
86 import org.onosproject.store.service.WallClockTimestamp;
87 import org.osgi.service.component.ComponentContext;
88 import org.slf4j.Logger;
89
90 import com.google.common.collect.ImmutableList;
91 import com.google.common.collect.Iterables;
92 import com.google.common.collect.Maps;
93 import com.google.common.collect.Sets;
94 import com.google.common.util.concurrent.Futures;
Madan Jampani86940d92015-05-06 11:47:57 -070095
Brian O'Connora3e5cd52015-12-05 15:59:19 -080096 import static com.google.common.base.Strings.isNullOrEmpty;
Ray Milkey2b6ff422016-08-26 13:03:15 -070097 import static org.onlab.util.Tools.get;
98 import static org.onlab.util.Tools.groupedThreads;
99 import static org.onosproject.net.flow.FlowRuleEvent.Type.RULE_REMOVED;
100 import static org.onosproject.store.flow.ReplicaInfoEvent.Type.MASTER_CHANGED;
101 import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.APPLY_BATCH_FLOWS;
102 import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.FLOW_TABLE_BACKUP;
103 import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.GET_DEVICE_FLOW_ENTRIES;
104 import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.GET_FLOW_ENTRY;
105 import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.REMOTE_APPLY_COMPLETED;
106 import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.REMOVE_FLOW_ENTRY;
107 import static org.slf4j.LoggerFactory.getLogger;
Madan Jampani86940d92015-05-06 11:47:57 -0700108
109/**
110 * Manages inventory of flow rules using a distributed state management protocol.
111 */
Sho SHIMIZU5c396e32016-08-12 15:19:12 -0700112@Component(immediate = true)
Madan Jampani86940d92015-05-06 11:47:57 -0700113@Service
Madan Jampani37d04c62016-04-25 15:53:55 -0700114public class DistributedFlowRuleStore
Madan Jampani86940d92015-05-06 11:47:57 -0700115 extends AbstractStore<FlowRuleBatchEvent, FlowRuleStoreDelegate>
116 implements FlowRuleStore {
117
118 private final Logger log = getLogger(getClass());
119
120 private static final int MESSAGE_HANDLER_THREAD_POOL_SIZE = 8;
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
Madan Jampani08bf17b2015-05-06 16:25:26 -0700132 @Property(name = "backupPeriod", intValue = DEFAULT_BACKUP_PERIOD_MILLIS,
133 label = "Delay in ms between successive backup runs")
134 private int backupPeriod = DEFAULT_BACKUP_PERIOD_MILLIS;
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700135 @Property(name = "persistenceEnabled", boolValue = false,
136 label = "Indicates whether or not changes in the flow table should be persisted to disk.")
137 private boolean persistenceEnabled = DEFAULT_PERSISTENCE_ENABLED;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700138
Madan Jampanic6d69f72016-07-15 15:47:12 -0700139 @Property(name = "backupCount", intValue = DEFAULT_MAX_BACKUP_COUNT,
140 label = "Max number of backup copies for each device")
141 private volatile int backupCount = DEFAULT_MAX_BACKUP_COUNT;
142
Madan Jampani86940d92015-05-06 11:47:57 -0700143 private InternalFlowTable flowTable = new InternalFlowTable();
144
145 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
146 protected ReplicaInfoService replicaInfoManager;
147
148 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
149 protected ClusterCommunicationService clusterCommunicator;
150
151 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
152 protected ClusterService clusterService;
153
154 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
155 protected DeviceService deviceService;
156
157 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
158 protected CoreService coreService;
159
160 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
161 protected ComponentConfigService configService;
162
163 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
164 protected MastershipService mastershipService;
165
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700166 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
167 protected PersistenceService persistenceService;
168
Madan Jampani86940d92015-05-06 11:47:57 -0700169 private Map<Long, NodeId> pendingResponses = Maps.newConcurrentMap();
170 private ExecutorService messageHandlingExecutor;
Madan Jampani71c32ca2016-06-22 08:23:18 -0700171 private ExecutorService eventHandler;
Madan Jampani86940d92015-05-06 11:47:57 -0700172
Madan Jampani08bf17b2015-05-06 16:25:26 -0700173 private ScheduledFuture<?> backupTask;
Madan Jampani86940d92015-05-06 11:47:57 -0700174 private final ScheduledExecutorService backupSenderExecutor =
HIGUCHI Yutad9e01052016-04-14 09:31:42 -0700175 Executors.newSingleThreadScheduledExecutor(groupedThreads("onos/flow", "backup-sender", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700176
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700177 private EventuallyConsistentMap<DeviceId, List<TableStatisticsEntry>> deviceTableStats;
178 private final EventuallyConsistentMapListener<DeviceId, List<TableStatisticsEntry>> tableStatsListener =
179 new InternalTableStatsListener();
180
181 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
182 protected StorageService storageService;
183
Madan Jampani884d4432016-08-23 10:46:55 -0700184 protected final Serializer serializer = Serializer.using(KryoNamespaces.API);
Madan Jampani86940d92015-05-06 11:47:57 -0700185
Madan Jampani884d4432016-08-23 10:46:55 -0700186 protected final KryoNamespace.Builder serializerBuilder = KryoNamespace.newBuilder()
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700187 .register(KryoNamespaces.API)
188 .register(MastershipBasedTimestamp.class);
189
190
Madan Jampani86940d92015-05-06 11:47:57 -0700191 private IdGenerator idGenerator;
192 private NodeId local;
193
194 @Activate
195 public void activate(ComponentContext context) {
196 configService.registerProperties(getClass());
197
198 idGenerator = coreService.getIdGenerator(FlowRuleService.FLOW_OP_TOPIC);
199
200 local = clusterService.getLocalNode().id();
201
Madan Jampani71c32ca2016-06-22 08:23:18 -0700202 eventHandler = Executors.newSingleThreadExecutor(
203 groupedThreads("onos/flow", "event-handler", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700204 messageHandlingExecutor = Executors.newFixedThreadPool(
HIGUCHI Yutad9e01052016-04-14 09:31:42 -0700205 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700206
207 registerMessageHandlers(messageHandlingExecutor);
208
sivachidambaram subramanian0b33c082017-04-06 13:26:44 +0530209 replicaInfoManager.addListener(flowTable);
210 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
211 flowTable::backup,
212 0,
213 backupPeriod,
214 TimeUnit.MILLISECONDS);
Madan Jampani86940d92015-05-06 11:47:57 -0700215
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700216 deviceTableStats = storageService.<DeviceId, List<TableStatisticsEntry>>eventuallyConsistentMapBuilder()
217 .withName("onos-flow-table-stats")
Madan Jampani884d4432016-08-23 10:46:55 -0700218 .withSerializer(serializerBuilder)
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700219 .withAntiEntropyPeriod(5, TimeUnit.SECONDS)
220 .withTimestampProvider((k, v) -> new WallClockTimestamp())
221 .withTombstonesDisabled()
222 .build();
223 deviceTableStats.addListener(tableStatsListener);
224
Madan Jampani86940d92015-05-06 11:47:57 -0700225 logConfig("Started");
226 }
227
228 @Deactivate
229 public void deactivate(ComponentContext context) {
sivachidambaram subramanian0b33c082017-04-06 13:26:44 +0530230 replicaInfoManager.removeListener(flowTable);
231 backupTask.cancel(true);
Madan Jampani86940d92015-05-06 11:47:57 -0700232 configService.unregisterProperties(getClass(), false);
233 unregisterMessageHandlers();
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700234 deviceTableStats.removeListener(tableStatsListener);
235 deviceTableStats.destroy();
Madan Jampani71c32ca2016-06-22 08:23:18 -0700236 eventHandler.shutdownNow();
Madan Jampani86940d92015-05-06 11:47:57 -0700237 messageHandlingExecutor.shutdownNow();
238 backupSenderExecutor.shutdownNow();
239 log.info("Stopped");
240 }
241
242 @SuppressWarnings("rawtypes")
243 @Modified
244 public void modified(ComponentContext context) {
245 if (context == null) {
Madan Jampani86940d92015-05-06 11:47:57 -0700246 logConfig("Default config");
247 return;
248 }
249
250 Dictionary properties = context.getProperties();
251 int newPoolSize;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700252 int newBackupPeriod;
Madan Jampanic6d69f72016-07-15 15:47:12 -0700253 int newBackupCount;
Madan Jampani86940d92015-05-06 11:47:57 -0700254 try {
255 String s = get(properties, "msgHandlerPoolSize");
256 newPoolSize = isNullOrEmpty(s) ? msgHandlerPoolSize : Integer.parseInt(s.trim());
257
Madan Jampani08bf17b2015-05-06 16:25:26 -0700258 s = get(properties, "backupPeriod");
259 newBackupPeriod = isNullOrEmpty(s) ? backupPeriod : Integer.parseInt(s.trim());
260
Madan Jampanic6d69f72016-07-15 15:47:12 -0700261 s = get(properties, "backupCount");
262 newBackupCount = isNullOrEmpty(s) ? backupCount : Integer.parseInt(s.trim());
Madan Jampani86940d92015-05-06 11:47:57 -0700263 } catch (NumberFormatException | ClassCastException e) {
264 newPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700265 newBackupPeriod = DEFAULT_BACKUP_PERIOD_MILLIS;
Madan Jampanic6d69f72016-07-15 15:47:12 -0700266 newBackupCount = DEFAULT_MAX_BACKUP_COUNT;
Madan Jampani86940d92015-05-06 11:47:57 -0700267 }
268
Madan Jampani08bf17b2015-05-06 16:25:26 -0700269 boolean restartBackupTask = false;
sivachidambaram subramanian0b33c082017-04-06 13:26:44 +0530270
Madan Jampani08bf17b2015-05-06 16:25:26 -0700271 if (newBackupPeriod != backupPeriod) {
272 backupPeriod = newBackupPeriod;
sivachidambaram subramanian0b33c082017-04-06 13:26:44 +0530273 restartBackupTask = true;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700274 }
275 if (restartBackupTask) {
276 if (backupTask != null) {
277 // cancel previously running task
278 backupTask.cancel(false);
279 }
280 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
281 flowTable::backup,
282 0,
283 backupPeriod,
284 TimeUnit.MILLISECONDS);
Madan Jampani86940d92015-05-06 11:47:57 -0700285 }
286 if (newPoolSize != msgHandlerPoolSize) {
287 msgHandlerPoolSize = newPoolSize;
288 ExecutorService oldMsgHandler = messageHandlingExecutor;
289 messageHandlingExecutor = Executors.newFixedThreadPool(
HIGUCHI Yuta060da9a2016-03-11 19:16:35 -0800290 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700291
292 // replace previously registered handlers.
293 registerMessageHandlers(messageHandlingExecutor);
294 oldMsgHandler.shutdown();
295 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700296 if (backupCount != newBackupCount) {
297 backupCount = newBackupCount;
298 }
Madan Jampani86940d92015-05-06 11:47:57 -0700299 logConfig("Reconfigured");
300 }
301
302 private void registerMessageHandlers(ExecutorService executor) {
303
304 clusterCommunicator.addSubscriber(APPLY_BATCH_FLOWS, new OnStoreBatch(), executor);
305 clusterCommunicator.<FlowRuleBatchEvent>addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700306 REMOTE_APPLY_COMPLETED, serializer::decode, this::notifyDelegate, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700307 clusterCommunicator.addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700308 GET_FLOW_ENTRY, serializer::decode, flowTable::getFlowEntry, serializer::encode, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700309 clusterCommunicator.addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700310 GET_DEVICE_FLOW_ENTRIES, serializer::decode, flowTable::getFlowEntries, serializer::encode, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700311 clusterCommunicator.addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700312 REMOVE_FLOW_ENTRY, serializer::decode, this::removeFlowRuleInternal, serializer::encode, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700313 clusterCommunicator.addSubscriber(
Madan Jampani884d4432016-08-23 10:46:55 -0700314 FLOW_TABLE_BACKUP, serializer::decode, flowTable::onBackupReceipt, serializer::encode, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700315 }
316
317 private void unregisterMessageHandlers() {
318 clusterCommunicator.removeSubscriber(REMOVE_FLOW_ENTRY);
319 clusterCommunicator.removeSubscriber(GET_DEVICE_FLOW_ENTRIES);
320 clusterCommunicator.removeSubscriber(GET_FLOW_ENTRY);
321 clusterCommunicator.removeSubscriber(APPLY_BATCH_FLOWS);
322 clusterCommunicator.removeSubscriber(REMOTE_APPLY_COMPLETED);
323 clusterCommunicator.removeSubscriber(FLOW_TABLE_BACKUP);
324 }
325
326 private void logConfig(String prefix) {
sivachidambaram subramanian0b33c082017-04-06 13:26:44 +0530327 log.info("{} with msgHandlerPoolSize = {}; backupPeriod = {}, backupCount = {}",
328 prefix, msgHandlerPoolSize, backupPeriod, backupCount);
Madan Jampani86940d92015-05-06 11:47:57 -0700329 }
330
331 // This is not a efficient operation on a distributed sharded
332 // flow store. We need to revisit the need for this operation or at least
333 // make it device specific.
334 @Override
335 public int getFlowRuleCount() {
336 AtomicInteger sum = new AtomicInteger(0);
337 deviceService.getDevices().forEach(device -> sum.addAndGet(Iterables.size(getFlowEntries(device.id()))));
338 return sum.get();
339 }
340
341 @Override
342 public FlowEntry getFlowEntry(FlowRule rule) {
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700343 NodeId master = mastershipService.getMasterFor(rule.deviceId());
Madan Jampani86940d92015-05-06 11:47:57 -0700344
345 if (master == null) {
Thomas Vachuska88fd6902015-08-04 10:08:34 -0700346 log.debug("Failed to getFlowEntry: No master for {}", rule.deviceId());
Madan Jampani86940d92015-05-06 11:47:57 -0700347 return null;
348 }
349
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800350 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700351 return flowTable.getFlowEntry(rule);
352 }
353
354 log.trace("Forwarding getFlowEntry to {}, which is the primary (master) for device {}",
355 master, rule.deviceId());
356
357 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(rule,
358 FlowStoreMessageSubjects.GET_FLOW_ENTRY,
Madan Jampani884d4432016-08-23 10:46:55 -0700359 serializer::encode,
360 serializer::decode,
Madan Jampani86940d92015-05-06 11:47:57 -0700361 master),
362 FLOW_RULE_STORE_TIMEOUT_MILLIS,
363 TimeUnit.MILLISECONDS,
364 null);
365 }
366
367 @Override
368 public Iterable<FlowEntry> getFlowEntries(DeviceId deviceId) {
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700369 NodeId master = mastershipService.getMasterFor(deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700370
371 if (master == null) {
Thomas Vachuska88fd6902015-08-04 10:08:34 -0700372 log.debug("Failed to getFlowEntries: No master for {}", deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700373 return Collections.emptyList();
374 }
375
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800376 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700377 return flowTable.getFlowEntries(deviceId);
378 }
379
380 log.trace("Forwarding getFlowEntries to {}, which is the primary (master) for device {}",
381 master, deviceId);
382
383 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(deviceId,
384 FlowStoreMessageSubjects.GET_DEVICE_FLOW_ENTRIES,
Madan Jampani884d4432016-08-23 10:46:55 -0700385 serializer::encode,
386 serializer::decode,
Madan Jampani86940d92015-05-06 11:47:57 -0700387 master),
388 FLOW_RULE_STORE_TIMEOUT_MILLIS,
389 TimeUnit.MILLISECONDS,
390 Collections.emptyList());
391 }
392
393 @Override
394 public void storeFlowRule(FlowRule rule) {
395 storeBatch(new FlowRuleBatchOperation(
Sho SHIMIZU98ffca82015-05-11 08:39:24 -0700396 Collections.singletonList(new FlowRuleBatchEntry(FlowRuleOperation.ADD, rule)),
Madan Jampani86940d92015-05-06 11:47:57 -0700397 rule.deviceId(), idGenerator.getNewId()));
398 }
399
400 @Override
401 public void storeBatch(FlowRuleBatchOperation operation) {
402 if (operation.getOperations().isEmpty()) {
403 notifyDelegate(FlowRuleBatchEvent.completed(
404 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
405 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
406 return;
407 }
408
409 DeviceId deviceId = operation.deviceId();
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700410 NodeId master = mastershipService.getMasterFor(deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700411
412 if (master == null) {
413 log.warn("No master for {} : flows will be marked for removal", deviceId);
414
415 updateStoreInternal(operation);
416
417 notifyDelegate(FlowRuleBatchEvent.completed(
418 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
419 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
420 return;
421 }
422
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800423 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700424 storeBatchInternal(operation);
425 return;
426 }
427
428 log.trace("Forwarding storeBatch to {}, which is the primary (master) for device {}",
429 master, deviceId);
430
Madan Jampani175e8fd2015-05-20 14:10:45 -0700431 clusterCommunicator.unicast(operation,
432 APPLY_BATCH_FLOWS,
Madan Jampani884d4432016-08-23 10:46:55 -0700433 serializer::encode,
Madan Jampani175e8fd2015-05-20 14:10:45 -0700434 master)
435 .whenComplete((result, error) -> {
Thomas Vachuskaa267ce42015-05-27 16:14:23 -0700436 if (error != null) {
Madan Jampani15f1bc42015-05-28 10:51:52 -0700437 log.warn("Failed to storeBatch: {} to {}", operation, master, error);
Madan Jampani86940d92015-05-06 11:47:57 -0700438
Thomas Vachuskaa267ce42015-05-27 16:14:23 -0700439 Set<FlowRule> allFailures = operation.getOperations()
440 .stream()
441 .map(op -> op.target())
442 .collect(Collectors.toSet());
Madan Jampani86940d92015-05-06 11:47:57 -0700443
Thomas Vachuskaa267ce42015-05-27 16:14:23 -0700444 notifyDelegate(FlowRuleBatchEvent.completed(
445 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
446 new CompletedBatchOperation(false, allFailures, deviceId)));
447 }
Madan Jampani175e8fd2015-05-20 14:10:45 -0700448 });
Madan Jampani86940d92015-05-06 11:47:57 -0700449 }
450
451 private void storeBatchInternal(FlowRuleBatchOperation operation) {
452
453 final DeviceId did = operation.deviceId();
454 //final Collection<FlowEntry> ft = flowTable.getFlowEntries(did);
455 Set<FlowRuleBatchEntry> currentOps = updateStoreInternal(operation);
456 if (currentOps.isEmpty()) {
457 batchOperationComplete(FlowRuleBatchEvent.completed(
458 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
459 new CompletedBatchOperation(true, Collections.emptySet(), did)));
460 return;
461 }
462
463 notifyDelegate(FlowRuleBatchEvent.requested(new
464 FlowRuleBatchRequest(operation.id(),
465 currentOps), operation.deviceId()));
466 }
467
468 private Set<FlowRuleBatchEntry> updateStoreInternal(FlowRuleBatchOperation operation) {
469 return operation.getOperations().stream().map(
470 op -> {
471 StoredFlowEntry entry;
472 switch (op.operator()) {
473 case ADD:
474 entry = new DefaultFlowEntry(op.target());
475 // always add requested FlowRule
476 // Note: 2 equal FlowEntry may have different treatment
477 flowTable.remove(entry.deviceId(), entry);
478 flowTable.add(entry);
479
480 return op;
481 case REMOVE:
482 entry = flowTable.getFlowEntry(op.target());
483 if (entry != null) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800484 //FIXME modification of "stored" flow entry outside of flow table
Madan Jampani86940d92015-05-06 11:47:57 -0700485 entry.setState(FlowEntryState.PENDING_REMOVE);
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800486 log.debug("Setting state of rule to pending remove: {}", entry);
Madan Jampani86940d92015-05-06 11:47:57 -0700487 return op;
488 }
489 break;
490 case MODIFY:
491 //TODO: figure this out at some point
492 break;
493 default:
494 log.warn("Unknown flow operation operator: {}", op.operator());
495 }
496 return null;
497 }
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800498 ).filter(Objects::nonNull).collect(Collectors.toSet());
Madan Jampani86940d92015-05-06 11:47:57 -0700499 }
500
501 @Override
502 public void deleteFlowRule(FlowRule rule) {
503 storeBatch(
504 new FlowRuleBatchOperation(
Sho SHIMIZU98ffca82015-05-11 08:39:24 -0700505 Collections.singletonList(
Madan Jampani86940d92015-05-06 11:47:57 -0700506 new FlowRuleBatchEntry(
507 FlowRuleOperation.REMOVE,
508 rule)), rule.deviceId(), idGenerator.getNewId()));
509 }
510
511 @Override
Charles Chan93fa7272016-01-26 22:27:02 -0800512 public FlowRuleEvent pendingFlowRule(FlowEntry rule) {
513 if (mastershipService.isLocalMaster(rule.deviceId())) {
514 StoredFlowEntry stored = flowTable.getFlowEntry(rule);
515 if (stored != null &&
516 stored.state() != FlowEntryState.PENDING_ADD) {
517 stored.setState(FlowEntryState.PENDING_ADD);
518 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
519 }
520 }
521 return null;
522 }
523
524 @Override
Madan Jampani86940d92015-05-06 11:47:57 -0700525 public FlowRuleEvent addOrUpdateFlowRule(FlowEntry rule) {
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700526 NodeId master = mastershipService.getMasterFor(rule.deviceId());
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800527 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700528 return addOrUpdateFlowRuleInternal(rule);
529 }
530
531 log.warn("Tried to update FlowRule {} state,"
532 + " while the Node was not the master.", rule);
533 return null;
534 }
535
536 private FlowRuleEvent addOrUpdateFlowRuleInternal(FlowEntry rule) {
537 // check if this new rule is an update to an existing entry
538 StoredFlowEntry stored = flowTable.getFlowEntry(rule);
539 if (stored != null) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800540 //FIXME modification of "stored" flow entry outside of flow table
Madan Jampani86940d92015-05-06 11:47:57 -0700541 stored.setBytes(rule.bytes());
Thiago Santos877914d2016-07-20 18:29:29 -0300542 stored.setLife(rule.life(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
Sangsik Yoonb1b823f2016-05-16 18:55:39 +0900543 stored.setLiveType(rule.liveType());
Madan Jampani86940d92015-05-06 11:47:57 -0700544 stored.setPackets(rule.packets());
Jonathan Hart89e981f2016-01-04 13:59:55 -0800545 stored.setLastSeen();
Madan Jampani86940d92015-05-06 11:47:57 -0700546 if (stored.state() == FlowEntryState.PENDING_ADD) {
547 stored.setState(FlowEntryState.ADDED);
548 return new FlowRuleEvent(Type.RULE_ADDED, rule);
549 }
550 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
551 }
552
553 // TODO: Confirm if this behavior is correct. See SimpleFlowRuleStore
554 // TODO: also update backup if the behavior is correct.
555 flowTable.add(rule);
556 return null;
557 }
558
559 @Override
560 public FlowRuleEvent removeFlowRule(FlowEntry rule) {
561 final DeviceId deviceId = rule.deviceId();
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700562 NodeId master = mastershipService.getMasterFor(deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700563
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800564 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700565 // bypass and handle it locally
566 return removeFlowRuleInternal(rule);
567 }
568
569 if (master == null) {
570 log.warn("Failed to removeFlowRule: No master for {}", deviceId);
571 // TODO: revisit if this should be null (="no-op") or Exception
572 return null;
573 }
574
575 log.trace("Forwarding removeFlowRule to {}, which is the master for device {}",
576 master, deviceId);
577
Madan Jampani222229e2016-07-14 10:43:25 -0700578 return Futures.getUnchecked(clusterCommunicator.sendAndReceive(
Madan Jampani86940d92015-05-06 11:47:57 -0700579 rule,
580 REMOVE_FLOW_ENTRY,
Madan Jampani884d4432016-08-23 10:46:55 -0700581 serializer::encode,
582 serializer::decode,
Madan Jampani222229e2016-07-14 10:43:25 -0700583 master));
Madan Jampani86940d92015-05-06 11:47:57 -0700584 }
585
586 private FlowRuleEvent removeFlowRuleInternal(FlowEntry rule) {
587 final DeviceId deviceId = rule.deviceId();
588 // 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 -0800589 final FlowEntry removed = flowTable.remove(deviceId, rule);
590 // rule may be partial rule that is missing treatment, we should use rule from store instead
591 return removed != null ? new FlowRuleEvent(RULE_REMOVED, removed) : null;
Madan Jampani86940d92015-05-06 11:47:57 -0700592 }
593
594 @Override
Charles Chan0c7c43b2016-01-14 17:39:20 -0800595 public void purgeFlowRule(DeviceId deviceId) {
596 flowTable.purgeFlowRule(deviceId);
597 }
598
599 @Override
Victor Silva139bca42016-08-18 11:46:35 -0300600 public void purgeFlowRules() {
601 flowTable.purgeFlowRules();
602 }
603
604 @Override
Madan Jampani86940d92015-05-06 11:47:57 -0700605 public void batchOperationComplete(FlowRuleBatchEvent event) {
606 //FIXME: need a per device pending response
607 NodeId nodeId = pendingResponses.remove(event.subject().batchId());
608 if (nodeId == null) {
609 notifyDelegate(event);
610 } else {
611 // TODO check unicast return value
Madan Jampani884d4432016-08-23 10:46:55 -0700612 clusterCommunicator.unicast(event, REMOTE_APPLY_COMPLETED, serializer::encode, nodeId);
Madan Jampani86940d92015-05-06 11:47:57 -0700613 //error log: log.warn("Failed to respond to peer for batch operation result");
614 }
615 }
616
617 private final class OnStoreBatch implements ClusterMessageHandler {
618
619 @Override
620 public void handle(final ClusterMessage message) {
Madan Jampani884d4432016-08-23 10:46:55 -0700621 FlowRuleBatchOperation operation = serializer.decode(message.payload());
Madan Jampani86940d92015-05-06 11:47:57 -0700622 log.debug("received batch request {}", operation);
623
624 final DeviceId deviceId = operation.deviceId();
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700625 NodeId master = mastershipService.getMasterFor(deviceId);
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800626 if (!Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700627 Set<FlowRule> failures = new HashSet<>(operation.size());
628 for (FlowRuleBatchEntry op : operation.getOperations()) {
629 failures.add(op.target());
630 }
631 CompletedBatchOperation allFailed = new CompletedBatchOperation(false, failures, deviceId);
632 // This node is no longer the master, respond as all failed.
633 // TODO: we might want to wrap response in envelope
634 // to distinguish sw programming failure and hand over
635 // it make sense in the latter case to retry immediately.
Madan Jampani884d4432016-08-23 10:46:55 -0700636 message.respond(serializer.encode(allFailed));
Madan Jampani86940d92015-05-06 11:47:57 -0700637 return;
638 }
639
640 pendingResponses.put(operation.id(), message.sender());
641 storeBatchInternal(operation);
642 }
643 }
644
Madan Jampanic6d69f72016-07-15 15:47:12 -0700645 private class BackupOperation {
646 private final NodeId nodeId;
647 private final DeviceId deviceId;
648
649 public BackupOperation(NodeId nodeId, DeviceId deviceId) {
650 this.nodeId = nodeId;
651 this.deviceId = deviceId;
652 }
653
654 @Override
655 public int hashCode() {
656 return Objects.hash(nodeId, deviceId);
657 }
658
659 @Override
660 public boolean equals(Object other) {
661 if (other != null && other instanceof BackupOperation) {
662 BackupOperation that = (BackupOperation) other;
663 return this.nodeId.equals(that.nodeId) &&
664 this.deviceId.equals(that.deviceId);
665 } else {
666 return false;
667 }
668 }
669 }
670
Madan Jampanif7536ab2015-05-07 23:23:23 -0700671 private class InternalFlowTable implements ReplicaInfoEventListener {
Madan Jampani86940d92015-05-06 11:47:57 -0700672
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800673 //TODO replace the Map<V,V> with ExtendedSet
674 private final Map<DeviceId, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>
Madan Jampani5c3766c2015-06-02 15:54:41 -0700675 flowEntries = Maps.newConcurrentMap();
Madan Jampani86940d92015-05-06 11:47:57 -0700676
Madan Jampanic6d69f72016-07-15 15:47:12 -0700677 private final Map<BackupOperation, Long> lastBackupTimes = Maps.newConcurrentMap();
Madan Jampani86940d92015-05-06 11:47:57 -0700678 private final Map<DeviceId, Long> lastUpdateTimes = Maps.newConcurrentMap();
Madan Jampani86940d92015-05-06 11:47:57 -0700679
Madan Jampanif7536ab2015-05-07 23:23:23 -0700680 @Override
681 public void event(ReplicaInfoEvent event) {
Madan Jampani71c32ca2016-06-22 08:23:18 -0700682 eventHandler.execute(() -> handleEvent(event));
683 }
684
685 private void handleEvent(ReplicaInfoEvent event) {
Madan Jampanic6d69f72016-07-15 15:47:12 -0700686 DeviceId deviceId = event.subject();
sivachidambaram subramanian0b33c082017-04-06 13:26:44 +0530687 if (!mastershipService.isLocalMaster(deviceId)) {
Madan Jampania98bf932015-06-02 12:01:36 -0700688 return;
689 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700690 if (event.type() == MASTER_CHANGED) {
691 lastUpdateTimes.put(deviceId, System.currentTimeMillis());
Madan Jampanif7536ab2015-05-07 23:23:23 -0700692 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700693 backupSenderExecutor.schedule(this::backup, 0, TimeUnit.SECONDS);
Madan Jampanif7536ab2015-05-07 23:23:23 -0700694 }
695
Madan Jampaniadea8902015-06-04 17:39:45 -0700696 private void sendBackups(NodeId nodeId, Set<DeviceId> deviceIds) {
697 // split up the devices into smaller batches and send them separately.
698 Iterables.partition(deviceIds, FLOW_TABLE_BACKUP_BATCH_SIZE)
699 .forEach(ids -> backupFlowEntries(nodeId, Sets.newHashSet(ids)));
700 }
701
Madan Jampanif7536ab2015-05-07 23:23:23 -0700702 private void backupFlowEntries(NodeId nodeId, Set<DeviceId> deviceIds) {
Madan Jampaniadea8902015-06-04 17:39:45 -0700703 if (deviceIds.isEmpty()) {
704 return;
705 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700706 log.debug("Sending flowEntries for devices {} to {} for backup.", deviceIds, nodeId);
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800707 Map<DeviceId, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>
708 deviceFlowEntries = Maps.newConcurrentMap();
Ray Milkey2b6ff422016-08-26 13:03:15 -0700709 deviceIds.forEach(id -> deviceFlowEntries.put(id, getFlowTableCopy(id)));
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800710 clusterCommunicator.<Map<DeviceId,
711 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>,
712 Set<DeviceId>>
713 sendAndReceive(deviceFlowEntries,
714 FLOW_TABLE_BACKUP,
Madan Jampani884d4432016-08-23 10:46:55 -0700715 serializer::encode,
716 serializer::decode,
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800717 nodeId)
718 .whenComplete((backedupDevices, error) -> {
719 Set<DeviceId> devicesNotBackedup = error != null ?
720 deviceFlowEntries.keySet() :
721 Sets.difference(deviceFlowEntries.keySet(), backedupDevices);
722 if (devicesNotBackedup.size() > 0) {
Ray Milkey2b6ff422016-08-26 13:03:15 -0700723 log.warn("Failed to backup devices: {}. Reason: {}, Node: {}",
724 devicesNotBackedup, error != null ? error.getMessage() : "none",
725 nodeId);
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800726 }
727 if (backedupDevices != null) {
728 backedupDevices.forEach(id -> {
Madan Jampanic6d69f72016-07-15 15:47:12 -0700729 lastBackupTimes.put(new BackupOperation(nodeId, id), System.currentTimeMillis());
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800730 });
731 }
732 });
Madan Jampanif7536ab2015-05-07 23:23:23 -0700733 }
734
Madan Jampani86940d92015-05-06 11:47:57 -0700735 /**
736 * Returns the flow table for specified device.
737 *
738 * @param deviceId identifier of the device
739 * @return Map representing Flow Table of given device.
740 */
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800741 private Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> getFlowTable(DeviceId deviceId) {
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700742 if (persistenceEnabled) {
743 return flowEntries.computeIfAbsent(deviceId, id -> persistenceService
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800744 .<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>persistentMapBuilder()
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700745 .withName("FlowTable:" + deviceId.toString())
746 .withSerializer(new Serializer() {
747 @Override
748 public <T> byte[] encode(T object) {
Madan Jampani884d4432016-08-23 10:46:55 -0700749 return serializer.encode(object);
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700750 }
751
752 @Override
753 public <T> T decode(byte[] bytes) {
Madan Jampani884d4432016-08-23 10:46:55 -0700754 return serializer.decode(bytes);
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700755 }
756 })
757 .build());
758 } else {
759 return flowEntries.computeIfAbsent(deviceId, id -> Maps.newConcurrentMap());
760 }
Madan Jampani86940d92015-05-06 11:47:57 -0700761 }
762
Ray Milkey2b6ff422016-08-26 13:03:15 -0700763 private Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> getFlowTableCopy(DeviceId deviceId) {
764 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> copy = Maps.newHashMap();
765 if (persistenceEnabled) {
766 return flowEntries.computeIfAbsent(deviceId, id -> persistenceService
767 .<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>persistentMapBuilder()
768 .withName("FlowTable:" + deviceId.toString())
769 .withSerializer(new Serializer() {
770 @Override
771 public <T> byte[] encode(T object) {
772 return serializer.encode(object);
773 }
774
775 @Override
776 public <T> T decode(byte[] bytes) {
777 return serializer.decode(bytes);
778 }
779 })
780 .build());
781 } else {
782 flowEntries.computeIfAbsent(deviceId, id -> Maps.newConcurrentMap()).forEach((k, v) -> {
783 copy.put(k, Maps.newHashMap(v));
784 });
785 return copy;
786 }
787 }
788
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800789 private Map<StoredFlowEntry, StoredFlowEntry> getFlowEntriesInternal(DeviceId deviceId, FlowId flowId) {
790 return getFlowTable(deviceId).computeIfAbsent(flowId, id -> Maps.newConcurrentMap());
Madan Jampani86940d92015-05-06 11:47:57 -0700791 }
792
793 private StoredFlowEntry getFlowEntryInternal(FlowRule rule) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800794 return getFlowEntriesInternal(rule.deviceId(), rule.id()).get(rule);
Madan Jampani86940d92015-05-06 11:47:57 -0700795 }
796
797 private Set<FlowEntry> getFlowEntriesInternal(DeviceId deviceId) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800798 return getFlowTable(deviceId).values().stream()
799 .flatMap(m -> m.values().stream())
800 .collect(Collectors.toSet());
Madan Jampani86940d92015-05-06 11:47:57 -0700801 }
802
803 public StoredFlowEntry getFlowEntry(FlowRule rule) {
804 return getFlowEntryInternal(rule);
805 }
806
807 public Set<FlowEntry> getFlowEntries(DeviceId deviceId) {
808 return getFlowEntriesInternal(deviceId);
809 }
810
811 public void add(FlowEntry rule) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800812 getFlowEntriesInternal(rule.deviceId(), rule.id())
813 .compute((StoredFlowEntry) rule, (k, stored) -> {
814 //TODO compare stored and rule timestamps
815 //TODO the key is not updated
816 return (StoredFlowEntry) rule;
817 });
Madan Jampani86940d92015-05-06 11:47:57 -0700818 lastUpdateTimes.put(rule.deviceId(), System.currentTimeMillis());
819 }
820
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800821 public FlowEntry remove(DeviceId deviceId, FlowEntry rule) {
822 final AtomicReference<FlowEntry> removedRule = new AtomicReference<>();
823 getFlowEntriesInternal(rule.deviceId(), rule.id())
824 .computeIfPresent((StoredFlowEntry) rule, (k, stored) -> {
825 if (rule instanceof DefaultFlowEntry) {
826 DefaultFlowEntry toRemove = (DefaultFlowEntry) rule;
827 if (stored instanceof DefaultFlowEntry) {
828 DefaultFlowEntry storedEntry = (DefaultFlowEntry) stored;
829 if (toRemove.created() < storedEntry.created()) {
830 log.debug("Trying to remove more recent flow entry {} (stored: {})",
831 toRemove, stored);
832 // the key is not updated, removedRule remains null
833 return stored;
834 }
835 }
836 }
837 removedRule.set(stored);
838 return null;
839 });
840
841 if (removedRule.get() != null) {
Madan Jampani86940d92015-05-06 11:47:57 -0700842 lastUpdateTimes.put(deviceId, System.currentTimeMillis());
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800843 return removedRule.get();
844 } else {
845 return null;
Madan Jampani86940d92015-05-06 11:47:57 -0700846 }
847 }
848
Charles Chan0c7c43b2016-01-14 17:39:20 -0800849 public void purgeFlowRule(DeviceId deviceId) {
850 flowEntries.remove(deviceId);
851 }
852
Victor Silva139bca42016-08-18 11:46:35 -0300853 public void purgeFlowRules() {
854 flowEntries.clear();
855 }
856
Madan Jampanic6d69f72016-07-15 15:47:12 -0700857 private List<NodeId> getBackupNodes(DeviceId deviceId) {
858 // The returned backup node list is in the order of preference i.e. next likely master first.
859 List<NodeId> allPossibleBackupNodes = replicaInfoManager.getReplicaInfoFor(deviceId).backups();
860 return ImmutableList.copyOf(allPossibleBackupNodes)
861 .subList(0, Math.min(allPossibleBackupNodes.size(), backupCount));
Madan Jampani86940d92015-05-06 11:47:57 -0700862 }
863
864 private void backup() {
Madan Jampani86940d92015-05-06 11:47:57 -0700865 try {
Madan Jampani86940d92015-05-06 11:47:57 -0700866 // compute a mapping from node to the set of devices whose flow entries it should backup
867 Map<NodeId, Set<DeviceId>> devicesToBackupByNode = Maps.newHashMap();
Sho SHIMIZUa09e1bb2016-08-01 14:25:25 -0700868 flowEntries.keySet().forEach(deviceId -> {
Madan Jampanic6d69f72016-07-15 15:47:12 -0700869 List<NodeId> backupNodes = getBackupNodes(deviceId);
870 backupNodes.forEach(backupNode -> {
871 if (lastBackupTimes.getOrDefault(new BackupOperation(backupNode, deviceId), 0L)
872 < lastUpdateTimes.getOrDefault(deviceId, 0L)) {
873 devicesToBackupByNode.computeIfAbsent(backupNode,
874 nodeId -> Sets.newHashSet()).add(deviceId);
875 }
876 });
Madan Jampani86940d92015-05-06 11:47:57 -0700877 });
Madan Jampani86940d92015-05-06 11:47:57 -0700878 // send the device flow entries to their respective backup nodes
Madan Jampaniadea8902015-06-04 17:39:45 -0700879 devicesToBackupByNode.forEach(this::sendBackups);
Madan Jampani86940d92015-05-06 11:47:57 -0700880 } catch (Exception e) {
881 log.error("Backup failed.", e);
882 }
883 }
884
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800885 private Set<DeviceId> onBackupReceipt(Map<DeviceId,
886 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>> flowTables) {
Madan Jampani7267c552015-05-20 22:39:17 -0700887 log.debug("Received flowEntries for {} to backup", flowTables.keySet());
Madan Jampani654b58a2015-05-22 11:28:11 -0700888 Set<DeviceId> backedupDevices = Sets.newHashSet();
889 try {
Madan Jampania98bf932015-06-02 12:01:36 -0700890 flowTables.forEach((deviceId, deviceFlowTable) -> {
891 // Only process those devices are that not managed by the local node.
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800892 if (!Objects.equals(local, mastershipService.getMasterFor(deviceId))) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800893 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> backupFlowTable =
894 getFlowTable(deviceId);
Madan Jampania98bf932015-06-02 12:01:36 -0700895 backupFlowTable.clear();
896 backupFlowTable.putAll(deviceFlowTable);
897 backedupDevices.add(deviceId);
898 }
Madan Jampani08bf17b2015-05-06 16:25:26 -0700899 });
Madan Jampani654b58a2015-05-22 11:28:11 -0700900 } catch (Exception e) {
901 log.warn("Failure processing backup request", e);
902 }
903 return backedupDevices;
Madan Jampani86940d92015-05-06 11:47:57 -0700904 }
905 }
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700906
907 @Override
908 public FlowRuleEvent updateTableStatistics(DeviceId deviceId,
909 List<TableStatisticsEntry> tableStats) {
910 deviceTableStats.put(deviceId, tableStats);
911 return null;
912 }
913
914 @Override
915 public Iterable<TableStatisticsEntry> getTableStatistics(DeviceId deviceId) {
916 NodeId master = mastershipService.getMasterFor(deviceId);
917
918 if (master == null) {
919 log.debug("Failed to getTableStats: No master for {}", deviceId);
920 return Collections.emptyList();
921 }
922
923 List<TableStatisticsEntry> tableStats = deviceTableStats.get(deviceId);
924 if (tableStats == null) {
925 return Collections.emptyList();
926 }
927 return ImmutableList.copyOf(tableStats);
928 }
929
Patryk Konopka7e40c012017-06-06 13:38:06 +0200930 @Override
931 public long getActiveFlowRuleCount(DeviceId deviceId) {
932 return Streams.stream(getTableStatistics(deviceId))
933 .mapToLong(TableStatisticsEntry::activeFlowEntries)
934 .sum();
935 }
936
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700937 private class InternalTableStatsListener
938 implements EventuallyConsistentMapListener<DeviceId, List<TableStatisticsEntry>> {
939 @Override
940 public void event(EventuallyConsistentMapEvent<DeviceId,
941 List<TableStatisticsEntry>> event) {
942 //TODO: Generate an event to listeners (do we need?)
943 }
944 }
Madan Jampani86940d92015-05-06 11:47:57 -0700945}