blob: ed688875dffcae88602a796fefa291f5d5879ea1 [file] [log] [blame]
Madan Jampani86940d92015-05-06 11:47:57 -07001 /*
Brian O'Connor5ab426f2016-04-09 01:19:45 -07002 * Copyright 2015-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 com.google.common.collect.ImmutableList;
Madan Jampanic6d69f72016-07-15 15:47:12 -070019import com.google.common.collect.ImmutableMap;
20import com.google.common.collect.Iterables;
21import com.google.common.collect.Maps;
22import com.google.common.collect.Sets;
23import com.google.common.util.concurrent.Futures;
24
25import org.apache.felix.scr.annotations.Activate;
26import org.apache.felix.scr.annotations.Component;
27import org.apache.felix.scr.annotations.Deactivate;
28import org.apache.felix.scr.annotations.Modified;
29import org.apache.felix.scr.annotations.Property;
30import org.apache.felix.scr.annotations.Reference;
31import org.apache.felix.scr.annotations.ReferenceCardinality;
32import org.apache.felix.scr.annotations.Service;
33import org.onlab.util.KryoNamespace;
34import org.onlab.util.Tools;
35import org.onosproject.cfg.ComponentConfigService;
36import org.onosproject.cluster.ClusterService;
37import org.onosproject.cluster.NodeId;
38import org.onosproject.core.CoreService;
39import org.onosproject.core.IdGenerator;
40import org.onosproject.mastership.MastershipService;
41import org.onosproject.net.DeviceId;
42import org.onosproject.net.device.DeviceService;
43import org.onosproject.net.flow.CompletedBatchOperation;
44import org.onosproject.net.flow.DefaultFlowEntry;
45import org.onosproject.net.flow.FlowEntry;
46import org.onosproject.net.flow.FlowEntry.FlowEntryState;
47import org.onosproject.net.flow.FlowId;
48import org.onosproject.net.flow.FlowRule;
49import org.onosproject.net.flow.FlowRuleBatchEntry;
50import org.onosproject.net.flow.FlowRuleBatchEntry.FlowRuleOperation;
51import org.onosproject.net.flow.FlowRuleBatchEvent;
52import org.onosproject.net.flow.FlowRuleBatchOperation;
53import org.onosproject.net.flow.FlowRuleBatchRequest;
54import org.onosproject.net.flow.FlowRuleEvent;
55import org.onosproject.net.flow.FlowRuleEvent.Type;
56import org.onosproject.net.flow.FlowRuleService;
57import org.onosproject.net.flow.FlowRuleStore;
58import org.onosproject.net.flow.FlowRuleStoreDelegate;
59import org.onosproject.net.flow.StoredFlowEntry;
60import org.onosproject.net.flow.TableStatisticsEntry;
61import org.onosproject.persistence.PersistenceService;
62import org.onosproject.store.AbstractStore;
63import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
64import org.onosproject.store.cluster.messaging.ClusterMessage;
65import org.onosproject.store.cluster.messaging.ClusterMessageHandler;
66import org.onosproject.store.flow.ReplicaInfoEvent;
67import org.onosproject.store.flow.ReplicaInfoEventListener;
68import org.onosproject.store.flow.ReplicaInfoService;
69import org.onosproject.store.impl.MastershipBasedTimestamp;
70import org.onosproject.store.serializers.KryoNamespaces;
71import org.onosproject.store.serializers.StoreSerializer;
72import org.onosproject.store.serializers.custom.DistributedStoreSerializers;
73import org.onosproject.store.service.EventuallyConsistentMap;
74import org.onosproject.store.service.EventuallyConsistentMapEvent;
75import org.onosproject.store.service.EventuallyConsistentMapListener;
76import org.onosproject.store.service.Serializer;
77import org.onosproject.store.service.StorageService;
78import org.onosproject.store.service.WallClockTimestamp;
79import org.osgi.service.component.ComponentContext;
80import org.slf4j.Logger;
Madan Jampani86940d92015-05-06 11:47:57 -070081
Brian O'Connora3e5cd52015-12-05 15:59:19 -080082 import java.util.Collections;
Madan Jampanic6d69f72016-07-15 15:47:12 -070083import java.util.Dictionary;
84import java.util.HashSet;
85import java.util.List;
86import java.util.Map;
87import java.util.Objects;
88import java.util.Set;
89import java.util.concurrent.ExecutorService;
90import java.util.concurrent.Executors;
91import java.util.concurrent.ScheduledExecutorService;
92import java.util.concurrent.ScheduledFuture;
93import java.util.concurrent.TimeUnit;
94import java.util.concurrent.atomic.AtomicInteger;
95import java.util.concurrent.atomic.AtomicReference;
96import java.util.stream.Collectors;
Madan Jampani86940d92015-05-06 11:47:57 -070097
Brian O'Connora3e5cd52015-12-05 15:59:19 -080098 import static com.google.common.base.Strings.isNullOrEmpty;
Madan Jampanic6d69f72016-07-15 15:47:12 -070099import static org.onlab.util.Tools.get;
100import static org.onlab.util.Tools.groupedThreads;
101import static org.onosproject.net.flow.FlowRuleEvent.Type.RULE_REMOVED;
102import static org.onosproject.store.flow.ReplicaInfoEvent.Type.MASTER_CHANGED;
103import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.*;
104import static org.slf4j.LoggerFactory.getLogger;
Madan Jampani86940d92015-05-06 11:47:57 -0700105
106/**
107 * Manages inventory of flow rules using a distributed state management protocol.
108 */
109@Component(immediate = true, enabled = true)
110@Service
Madan Jampani37d04c62016-04-25 15:53:55 -0700111public class DistributedFlowRuleStore
Madan Jampani86940d92015-05-06 11:47:57 -0700112 extends AbstractStore<FlowRuleBatchEvent, FlowRuleStoreDelegate>
113 implements FlowRuleStore {
114
115 private final Logger log = getLogger(getClass());
116
117 private static final int MESSAGE_HANDLER_THREAD_POOL_SIZE = 8;
118 private static final boolean DEFAULT_BACKUP_ENABLED = true;
Madan Jampanic6d69f72016-07-15 15:47:12 -0700119 private static final int DEFAULT_MAX_BACKUP_COUNT = 2;
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700120 private static final boolean DEFAULT_PERSISTENCE_ENABLED = false;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700121 private static final int DEFAULT_BACKUP_PERIOD_MILLIS = 2000;
Madan Jampani86940d92015-05-06 11:47:57 -0700122 private static final long FLOW_RULE_STORE_TIMEOUT_MILLIS = 5000;
Madan Jampaniadea8902015-06-04 17:39:45 -0700123 // number of devices whose flow entries will be backed up in one communication round
124 private static final int FLOW_TABLE_BACKUP_BATCH_SIZE = 1;
Madan Jampani86940d92015-05-06 11:47:57 -0700125
126 @Property(name = "msgHandlerPoolSize", intValue = MESSAGE_HANDLER_THREAD_POOL_SIZE,
127 label = "Number of threads in the message handler pool")
128 private int msgHandlerPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
129
130 @Property(name = "backupEnabled", boolValue = DEFAULT_BACKUP_ENABLED,
131 label = "Indicates whether backups are enabled or not")
Madan Jampanic6d69f72016-07-15 15:47:12 -0700132 private volatile boolean backupEnabled = DEFAULT_BACKUP_ENABLED;
Madan Jampani86940d92015-05-06 11:47:57 -0700133
Madan Jampani08bf17b2015-05-06 16:25:26 -0700134 @Property(name = "backupPeriod", intValue = DEFAULT_BACKUP_PERIOD_MILLIS,
135 label = "Delay in ms between successive backup runs")
136 private int backupPeriod = DEFAULT_BACKUP_PERIOD_MILLIS;
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700137 @Property(name = "persistenceEnabled", boolValue = false,
138 label = "Indicates whether or not changes in the flow table should be persisted to disk.")
139 private boolean persistenceEnabled = DEFAULT_PERSISTENCE_ENABLED;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700140
Madan Jampanic6d69f72016-07-15 15:47:12 -0700141 @Property(name = "backupCount", intValue = DEFAULT_MAX_BACKUP_COUNT,
142 label = "Max number of backup copies for each device")
143 private volatile int backupCount = DEFAULT_MAX_BACKUP_COUNT;
144
Madan Jampani86940d92015-05-06 11:47:57 -0700145 private InternalFlowTable flowTable = new InternalFlowTable();
146
147 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
148 protected ReplicaInfoService replicaInfoManager;
149
150 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
151 protected ClusterCommunicationService clusterCommunicator;
152
153 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
154 protected ClusterService clusterService;
155
156 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
157 protected DeviceService deviceService;
158
159 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
160 protected CoreService coreService;
161
162 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
163 protected ComponentConfigService configService;
164
165 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
166 protected MastershipService mastershipService;
167
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700168 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
169 protected PersistenceService persistenceService;
170
Madan Jampani86940d92015-05-06 11:47:57 -0700171 private Map<Long, NodeId> pendingResponses = Maps.newConcurrentMap();
172 private ExecutorService messageHandlingExecutor;
Madan Jampani71c32ca2016-06-22 08:23:18 -0700173 private ExecutorService eventHandler;
Madan Jampani86940d92015-05-06 11:47:57 -0700174
Madan Jampani08bf17b2015-05-06 16:25:26 -0700175 private ScheduledFuture<?> backupTask;
Madan Jampani86940d92015-05-06 11:47:57 -0700176 private final ScheduledExecutorService backupSenderExecutor =
HIGUCHI Yutad9e01052016-04-14 09:31:42 -0700177 Executors.newSingleThreadScheduledExecutor(groupedThreads("onos/flow", "backup-sender", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700178
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700179 private EventuallyConsistentMap<DeviceId, List<TableStatisticsEntry>> deviceTableStats;
180 private final EventuallyConsistentMapListener<DeviceId, List<TableStatisticsEntry>> tableStatsListener =
181 new InternalTableStatsListener();
182
183 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
184 protected StorageService storageService;
185
HIGUCHI Yutae7290652016-05-18 11:29:01 -0700186 protected static final StoreSerializer SERIALIZER = StoreSerializer.using(
187 KryoNamespace.newBuilder()
Madan Jampani86940d92015-05-06 11:47:57 -0700188 .register(DistributedStoreSerializers.STORE_COMMON)
189 .nextId(DistributedStoreSerializers.STORE_CUSTOM_BEGIN)
HIGUCHI Yutae7290652016-05-18 11:29:01 -0700190 .build("FlowRuleStore"));
Madan Jampani86940d92015-05-06 11:47:57 -0700191
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700192 protected static final KryoNamespace.Builder SERIALIZER_BUILDER = KryoNamespace.newBuilder()
193 .register(KryoNamespaces.API)
194 .register(MastershipBasedTimestamp.class);
195
196
Madan Jampani86940d92015-05-06 11:47:57 -0700197 private IdGenerator idGenerator;
198 private NodeId local;
199
200 @Activate
201 public void activate(ComponentContext context) {
202 configService.registerProperties(getClass());
203
204 idGenerator = coreService.getIdGenerator(FlowRuleService.FLOW_OP_TOPIC);
205
206 local = clusterService.getLocalNode().id();
207
Madan Jampani71c32ca2016-06-22 08:23:18 -0700208 eventHandler = Executors.newSingleThreadExecutor(
209 groupedThreads("onos/flow", "event-handler", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700210 messageHandlingExecutor = Executors.newFixedThreadPool(
HIGUCHI Yutad9e01052016-04-14 09:31:42 -0700211 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700212
213 registerMessageHandlers(messageHandlingExecutor);
214
Madan Jampani08bf17b2015-05-06 16:25:26 -0700215 if (backupEnabled) {
Madan Jampanif7536ab2015-05-07 23:23:23 -0700216 replicaInfoManager.addListener(flowTable);
Madan Jampani08bf17b2015-05-06 16:25:26 -0700217 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
218 flowTable::backup,
219 0,
220 backupPeriod,
221 TimeUnit.MILLISECONDS);
222 }
Madan Jampani86940d92015-05-06 11:47:57 -0700223
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700224 deviceTableStats = storageService.<DeviceId, List<TableStatisticsEntry>>eventuallyConsistentMapBuilder()
225 .withName("onos-flow-table-stats")
226 .withSerializer(SERIALIZER_BUILDER)
227 .withAntiEntropyPeriod(5, TimeUnit.SECONDS)
228 .withTimestampProvider((k, v) -> new WallClockTimestamp())
229 .withTombstonesDisabled()
230 .build();
231 deviceTableStats.addListener(tableStatsListener);
232
Madan Jampani86940d92015-05-06 11:47:57 -0700233 logConfig("Started");
234 }
235
236 @Deactivate
237 public void deactivate(ComponentContext context) {
Madan Jampanif7536ab2015-05-07 23:23:23 -0700238 if (backupEnabled) {
239 replicaInfoManager.removeListener(flowTable);
240 backupTask.cancel(true);
241 }
Madan Jampani86940d92015-05-06 11:47:57 -0700242 configService.unregisterProperties(getClass(), false);
243 unregisterMessageHandlers();
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700244 deviceTableStats.removeListener(tableStatsListener);
245 deviceTableStats.destroy();
Madan Jampani71c32ca2016-06-22 08:23:18 -0700246 eventHandler.shutdownNow();
Madan Jampani86940d92015-05-06 11:47:57 -0700247 messageHandlingExecutor.shutdownNow();
248 backupSenderExecutor.shutdownNow();
249 log.info("Stopped");
250 }
251
252 @SuppressWarnings("rawtypes")
253 @Modified
254 public void modified(ComponentContext context) {
255 if (context == null) {
256 backupEnabled = DEFAULT_BACKUP_ENABLED;
257 logConfig("Default config");
258 return;
259 }
260
261 Dictionary properties = context.getProperties();
262 int newPoolSize;
263 boolean newBackupEnabled;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700264 int newBackupPeriod;
Madan Jampanic6d69f72016-07-15 15:47:12 -0700265 int newBackupCount;
Madan Jampani86940d92015-05-06 11:47:57 -0700266 try {
267 String s = get(properties, "msgHandlerPoolSize");
268 newPoolSize = isNullOrEmpty(s) ? msgHandlerPoolSize : Integer.parseInt(s.trim());
269
270 s = get(properties, "backupEnabled");
271 newBackupEnabled = isNullOrEmpty(s) ? backupEnabled : Boolean.parseBoolean(s.trim());
272
Madan Jampani08bf17b2015-05-06 16:25:26 -0700273 s = get(properties, "backupPeriod");
274 newBackupPeriod = isNullOrEmpty(s) ? backupPeriod : Integer.parseInt(s.trim());
275
Madan Jampanic6d69f72016-07-15 15:47:12 -0700276 s = get(properties, "backupCount");
277 newBackupCount = isNullOrEmpty(s) ? backupCount : Integer.parseInt(s.trim());
Madan Jampani86940d92015-05-06 11:47:57 -0700278 } catch (NumberFormatException | ClassCastException e) {
279 newPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
280 newBackupEnabled = DEFAULT_BACKUP_ENABLED;
Madan Jampani08bf17b2015-05-06 16:25:26 -0700281 newBackupPeriod = DEFAULT_BACKUP_PERIOD_MILLIS;
Madan Jampanic6d69f72016-07-15 15:47:12 -0700282 newBackupCount = DEFAULT_MAX_BACKUP_COUNT;
Madan Jampani86940d92015-05-06 11:47:57 -0700283 }
284
Madan Jampani08bf17b2015-05-06 16:25:26 -0700285 boolean restartBackupTask = false;
Madan Jampani86940d92015-05-06 11:47:57 -0700286 if (newBackupEnabled != backupEnabled) {
287 backupEnabled = newBackupEnabled;
Madan Jampanif7536ab2015-05-07 23:23:23 -0700288 if (!backupEnabled) {
289 replicaInfoManager.removeListener(flowTable);
290 if (backupTask != null) {
291 backupTask.cancel(false);
292 backupTask = null;
293 }
294 } else {
295 replicaInfoManager.addListener(flowTable);
Madan Jampani08bf17b2015-05-06 16:25:26 -0700296 }
297 restartBackupTask = backupEnabled;
298 }
299 if (newBackupPeriod != backupPeriod) {
300 backupPeriod = newBackupPeriod;
301 restartBackupTask = backupEnabled;
302 }
303 if (restartBackupTask) {
304 if (backupTask != null) {
305 // cancel previously running task
306 backupTask.cancel(false);
307 }
308 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
309 flowTable::backup,
310 0,
311 backupPeriod,
312 TimeUnit.MILLISECONDS);
Madan Jampani86940d92015-05-06 11:47:57 -0700313 }
314 if (newPoolSize != msgHandlerPoolSize) {
315 msgHandlerPoolSize = newPoolSize;
316 ExecutorService oldMsgHandler = messageHandlingExecutor;
317 messageHandlingExecutor = Executors.newFixedThreadPool(
HIGUCHI Yuta060da9a2016-03-11 19:16:35 -0800318 msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers", log));
Madan Jampani86940d92015-05-06 11:47:57 -0700319
320 // replace previously registered handlers.
321 registerMessageHandlers(messageHandlingExecutor);
322 oldMsgHandler.shutdown();
323 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700324 if (backupCount != newBackupCount) {
325 backupCount = newBackupCount;
326 }
Madan Jampani86940d92015-05-06 11:47:57 -0700327 logConfig("Reconfigured");
328 }
329
330 private void registerMessageHandlers(ExecutorService executor) {
331
332 clusterCommunicator.addSubscriber(APPLY_BATCH_FLOWS, new OnStoreBatch(), executor);
333 clusterCommunicator.<FlowRuleBatchEvent>addSubscriber(
334 REMOTE_APPLY_COMPLETED, SERIALIZER::decode, this::notifyDelegate, executor);
335 clusterCommunicator.addSubscriber(
336 GET_FLOW_ENTRY, SERIALIZER::decode, flowTable::getFlowEntry, SERIALIZER::encode, executor);
337 clusterCommunicator.addSubscriber(
338 GET_DEVICE_FLOW_ENTRIES, SERIALIZER::decode, flowTable::getFlowEntries, SERIALIZER::encode, executor);
339 clusterCommunicator.addSubscriber(
340 REMOVE_FLOW_ENTRY, SERIALIZER::decode, this::removeFlowRuleInternal, SERIALIZER::encode, executor);
341 clusterCommunicator.addSubscriber(
342 REMOVE_FLOW_ENTRY, SERIALIZER::decode, this::removeFlowRuleInternal, SERIALIZER::encode, executor);
343 clusterCommunicator.addSubscriber(
Madan Jampani654b58a2015-05-22 11:28:11 -0700344 FLOW_TABLE_BACKUP, SERIALIZER::decode, flowTable::onBackupReceipt, SERIALIZER::encode, executor);
Madan Jampani86940d92015-05-06 11:47:57 -0700345 }
346
347 private void unregisterMessageHandlers() {
348 clusterCommunicator.removeSubscriber(REMOVE_FLOW_ENTRY);
349 clusterCommunicator.removeSubscriber(GET_DEVICE_FLOW_ENTRIES);
350 clusterCommunicator.removeSubscriber(GET_FLOW_ENTRY);
351 clusterCommunicator.removeSubscriber(APPLY_BATCH_FLOWS);
352 clusterCommunicator.removeSubscriber(REMOTE_APPLY_COMPLETED);
353 clusterCommunicator.removeSubscriber(FLOW_TABLE_BACKUP);
354 }
355
356 private void logConfig(String prefix) {
Madan Jampanic6d69f72016-07-15 15:47:12 -0700357 log.info("{} with msgHandlerPoolSize = {}; backupEnabled = {}, backupPeriod = {}, backupCount = {}",
358 prefix, msgHandlerPoolSize, backupEnabled, backupPeriod, backupCount);
Madan Jampani86940d92015-05-06 11:47:57 -0700359 }
360
361 // This is not a efficient operation on a distributed sharded
362 // flow store. We need to revisit the need for this operation or at least
363 // make it device specific.
364 @Override
365 public int getFlowRuleCount() {
366 AtomicInteger sum = new AtomicInteger(0);
367 deviceService.getDevices().forEach(device -> sum.addAndGet(Iterables.size(getFlowEntries(device.id()))));
368 return sum.get();
369 }
370
371 @Override
372 public FlowEntry getFlowEntry(FlowRule rule) {
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700373 NodeId master = mastershipService.getMasterFor(rule.deviceId());
Madan Jampani86940d92015-05-06 11:47:57 -0700374
375 if (master == null) {
Thomas Vachuska88fd6902015-08-04 10:08:34 -0700376 log.debug("Failed to getFlowEntry: No master for {}", rule.deviceId());
Madan Jampani86940d92015-05-06 11:47:57 -0700377 return null;
378 }
379
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800380 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700381 return flowTable.getFlowEntry(rule);
382 }
383
384 log.trace("Forwarding getFlowEntry to {}, which is the primary (master) for device {}",
385 master, rule.deviceId());
386
387 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(rule,
388 FlowStoreMessageSubjects.GET_FLOW_ENTRY,
389 SERIALIZER::encode,
390 SERIALIZER::decode,
391 master),
392 FLOW_RULE_STORE_TIMEOUT_MILLIS,
393 TimeUnit.MILLISECONDS,
394 null);
395 }
396
397 @Override
398 public Iterable<FlowEntry> getFlowEntries(DeviceId deviceId) {
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700399 NodeId master = mastershipService.getMasterFor(deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700400
401 if (master == null) {
Thomas Vachuska88fd6902015-08-04 10:08:34 -0700402 log.debug("Failed to getFlowEntries: No master for {}", deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700403 return Collections.emptyList();
404 }
405
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800406 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700407 return flowTable.getFlowEntries(deviceId);
408 }
409
410 log.trace("Forwarding getFlowEntries to {}, which is the primary (master) for device {}",
411 master, deviceId);
412
413 return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(deviceId,
414 FlowStoreMessageSubjects.GET_DEVICE_FLOW_ENTRIES,
415 SERIALIZER::encode,
416 SERIALIZER::decode,
417 master),
418 FLOW_RULE_STORE_TIMEOUT_MILLIS,
419 TimeUnit.MILLISECONDS,
420 Collections.emptyList());
421 }
422
423 @Override
424 public void storeFlowRule(FlowRule rule) {
425 storeBatch(new FlowRuleBatchOperation(
Sho SHIMIZU98ffca82015-05-11 08:39:24 -0700426 Collections.singletonList(new FlowRuleBatchEntry(FlowRuleOperation.ADD, rule)),
Madan Jampani86940d92015-05-06 11:47:57 -0700427 rule.deviceId(), idGenerator.getNewId()));
428 }
429
430 @Override
431 public void storeBatch(FlowRuleBatchOperation operation) {
432 if (operation.getOperations().isEmpty()) {
433 notifyDelegate(FlowRuleBatchEvent.completed(
434 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
435 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
436 return;
437 }
438
439 DeviceId deviceId = operation.deviceId();
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700440 NodeId master = mastershipService.getMasterFor(deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700441
442 if (master == null) {
443 log.warn("No master for {} : flows will be marked for removal", deviceId);
444
445 updateStoreInternal(operation);
446
447 notifyDelegate(FlowRuleBatchEvent.completed(
448 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
449 new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
450 return;
451 }
452
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800453 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700454 storeBatchInternal(operation);
455 return;
456 }
457
458 log.trace("Forwarding storeBatch to {}, which is the primary (master) for device {}",
459 master, deviceId);
460
Madan Jampani175e8fd2015-05-20 14:10:45 -0700461 clusterCommunicator.unicast(operation,
462 APPLY_BATCH_FLOWS,
463 SERIALIZER::encode,
464 master)
465 .whenComplete((result, error) -> {
Thomas Vachuskaa267ce42015-05-27 16:14:23 -0700466 if (error != null) {
Madan Jampani15f1bc42015-05-28 10:51:52 -0700467 log.warn("Failed to storeBatch: {} to {}", operation, master, error);
Madan Jampani86940d92015-05-06 11:47:57 -0700468
Thomas Vachuskaa267ce42015-05-27 16:14:23 -0700469 Set<FlowRule> allFailures = operation.getOperations()
470 .stream()
471 .map(op -> op.target())
472 .collect(Collectors.toSet());
Madan Jampani86940d92015-05-06 11:47:57 -0700473
Thomas Vachuskaa267ce42015-05-27 16:14:23 -0700474 notifyDelegate(FlowRuleBatchEvent.completed(
475 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
476 new CompletedBatchOperation(false, allFailures, deviceId)));
477 }
Madan Jampani175e8fd2015-05-20 14:10:45 -0700478 });
Madan Jampani86940d92015-05-06 11:47:57 -0700479 }
480
481 private void storeBatchInternal(FlowRuleBatchOperation operation) {
482
483 final DeviceId did = operation.deviceId();
484 //final Collection<FlowEntry> ft = flowTable.getFlowEntries(did);
485 Set<FlowRuleBatchEntry> currentOps = updateStoreInternal(operation);
486 if (currentOps.isEmpty()) {
487 batchOperationComplete(FlowRuleBatchEvent.completed(
488 new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
489 new CompletedBatchOperation(true, Collections.emptySet(), did)));
490 return;
491 }
492
493 notifyDelegate(FlowRuleBatchEvent.requested(new
494 FlowRuleBatchRequest(operation.id(),
495 currentOps), operation.deviceId()));
496 }
497
498 private Set<FlowRuleBatchEntry> updateStoreInternal(FlowRuleBatchOperation operation) {
499 return operation.getOperations().stream().map(
500 op -> {
501 StoredFlowEntry entry;
502 switch (op.operator()) {
503 case ADD:
504 entry = new DefaultFlowEntry(op.target());
505 // always add requested FlowRule
506 // Note: 2 equal FlowEntry may have different treatment
507 flowTable.remove(entry.deviceId(), entry);
508 flowTable.add(entry);
509
510 return op;
511 case REMOVE:
512 entry = flowTable.getFlowEntry(op.target());
513 if (entry != null) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800514 //FIXME modification of "stored" flow entry outside of flow table
Madan Jampani86940d92015-05-06 11:47:57 -0700515 entry.setState(FlowEntryState.PENDING_REMOVE);
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800516 log.debug("Setting state of rule to pending remove: {}", entry);
Madan Jampani86940d92015-05-06 11:47:57 -0700517 return op;
518 }
519 break;
520 case MODIFY:
521 //TODO: figure this out at some point
522 break;
523 default:
524 log.warn("Unknown flow operation operator: {}", op.operator());
525 }
526 return null;
527 }
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800528 ).filter(Objects::nonNull).collect(Collectors.toSet());
Madan Jampani86940d92015-05-06 11:47:57 -0700529 }
530
531 @Override
532 public void deleteFlowRule(FlowRule rule) {
533 storeBatch(
534 new FlowRuleBatchOperation(
Sho SHIMIZU98ffca82015-05-11 08:39:24 -0700535 Collections.singletonList(
Madan Jampani86940d92015-05-06 11:47:57 -0700536 new FlowRuleBatchEntry(
537 FlowRuleOperation.REMOVE,
538 rule)), rule.deviceId(), idGenerator.getNewId()));
539 }
540
541 @Override
Charles Chan93fa7272016-01-26 22:27:02 -0800542 public FlowRuleEvent pendingFlowRule(FlowEntry rule) {
543 if (mastershipService.isLocalMaster(rule.deviceId())) {
544 StoredFlowEntry stored = flowTable.getFlowEntry(rule);
545 if (stored != null &&
546 stored.state() != FlowEntryState.PENDING_ADD) {
547 stored.setState(FlowEntryState.PENDING_ADD);
548 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
549 }
550 }
551 return null;
552 }
553
554 @Override
Madan Jampani86940d92015-05-06 11:47:57 -0700555 public FlowRuleEvent addOrUpdateFlowRule(FlowEntry rule) {
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700556 NodeId master = mastershipService.getMasterFor(rule.deviceId());
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800557 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700558 return addOrUpdateFlowRuleInternal(rule);
559 }
560
561 log.warn("Tried to update FlowRule {} state,"
562 + " while the Node was not the master.", rule);
563 return null;
564 }
565
566 private FlowRuleEvent addOrUpdateFlowRuleInternal(FlowEntry rule) {
567 // check if this new rule is an update to an existing entry
568 StoredFlowEntry stored = flowTable.getFlowEntry(rule);
569 if (stored != null) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800570 //FIXME modification of "stored" flow entry outside of flow table
Madan Jampani86940d92015-05-06 11:47:57 -0700571 stored.setBytes(rule.bytes());
572 stored.setLife(rule.life());
573 stored.setPackets(rule.packets());
Jonathan Hart89e981f2016-01-04 13:59:55 -0800574 stored.setLastSeen();
Madan Jampani86940d92015-05-06 11:47:57 -0700575 if (stored.state() == FlowEntryState.PENDING_ADD) {
576 stored.setState(FlowEntryState.ADDED);
577 return new FlowRuleEvent(Type.RULE_ADDED, rule);
578 }
579 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
580 }
581
582 // TODO: Confirm if this behavior is correct. See SimpleFlowRuleStore
583 // TODO: also update backup if the behavior is correct.
584 flowTable.add(rule);
585 return null;
586 }
587
588 @Override
589 public FlowRuleEvent removeFlowRule(FlowEntry rule) {
590 final DeviceId deviceId = rule.deviceId();
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700591 NodeId master = mastershipService.getMasterFor(deviceId);
Madan Jampani86940d92015-05-06 11:47:57 -0700592
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800593 if (Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700594 // bypass and handle it locally
595 return removeFlowRuleInternal(rule);
596 }
597
598 if (master == null) {
599 log.warn("Failed to removeFlowRule: No master for {}", deviceId);
600 // TODO: revisit if this should be null (="no-op") or Exception
601 return null;
602 }
603
604 log.trace("Forwarding removeFlowRule to {}, which is the master for device {}",
605 master, deviceId);
606
Madan Jampani222229e2016-07-14 10:43:25 -0700607 return Futures.getUnchecked(clusterCommunicator.sendAndReceive(
Madan Jampani86940d92015-05-06 11:47:57 -0700608 rule,
609 REMOVE_FLOW_ENTRY,
610 SERIALIZER::encode,
611 SERIALIZER::decode,
Madan Jampani222229e2016-07-14 10:43:25 -0700612 master));
Madan Jampani86940d92015-05-06 11:47:57 -0700613 }
614
615 private FlowRuleEvent removeFlowRuleInternal(FlowEntry rule) {
616 final DeviceId deviceId = rule.deviceId();
617 // 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 -0800618 final FlowEntry removed = flowTable.remove(deviceId, rule);
619 // rule may be partial rule that is missing treatment, we should use rule from store instead
620 return removed != null ? new FlowRuleEvent(RULE_REMOVED, removed) : null;
Madan Jampani86940d92015-05-06 11:47:57 -0700621 }
622
623 @Override
Charles Chan0c7c43b2016-01-14 17:39:20 -0800624 public void purgeFlowRule(DeviceId deviceId) {
625 flowTable.purgeFlowRule(deviceId);
626 }
627
628 @Override
Madan Jampani86940d92015-05-06 11:47:57 -0700629 public void batchOperationComplete(FlowRuleBatchEvent event) {
630 //FIXME: need a per device pending response
631 NodeId nodeId = pendingResponses.remove(event.subject().batchId());
632 if (nodeId == null) {
633 notifyDelegate(event);
634 } else {
635 // TODO check unicast return value
636 clusterCommunicator.unicast(event, REMOTE_APPLY_COMPLETED, SERIALIZER::encode, nodeId);
637 //error log: log.warn("Failed to respond to peer for batch operation result");
638 }
639 }
640
641 private final class OnStoreBatch implements ClusterMessageHandler {
642
643 @Override
644 public void handle(final ClusterMessage message) {
645 FlowRuleBatchOperation operation = SERIALIZER.decode(message.payload());
646 log.debug("received batch request {}", operation);
647
648 final DeviceId deviceId = operation.deviceId();
Madan Jampani6bd2d9f2015-05-14 14:10:42 -0700649 NodeId master = mastershipService.getMasterFor(deviceId);
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800650 if (!Objects.equals(local, master)) {
Madan Jampani86940d92015-05-06 11:47:57 -0700651 Set<FlowRule> failures = new HashSet<>(operation.size());
652 for (FlowRuleBatchEntry op : operation.getOperations()) {
653 failures.add(op.target());
654 }
655 CompletedBatchOperation allFailed = new CompletedBatchOperation(false, failures, deviceId);
656 // This node is no longer the master, respond as all failed.
657 // TODO: we might want to wrap response in envelope
658 // to distinguish sw programming failure and hand over
659 // it make sense in the latter case to retry immediately.
660 message.respond(SERIALIZER.encode(allFailed));
661 return;
662 }
663
664 pendingResponses.put(operation.id(), message.sender());
665 storeBatchInternal(operation);
666 }
667 }
668
Madan Jampanic6d69f72016-07-15 15:47:12 -0700669 private class BackupOperation {
670 private final NodeId nodeId;
671 private final DeviceId deviceId;
672
673 public BackupOperation(NodeId nodeId, DeviceId deviceId) {
674 this.nodeId = nodeId;
675 this.deviceId = deviceId;
676 }
677
678 @Override
679 public int hashCode() {
680 return Objects.hash(nodeId, deviceId);
681 }
682
683 @Override
684 public boolean equals(Object other) {
685 if (other != null && other instanceof BackupOperation) {
686 BackupOperation that = (BackupOperation) other;
687 return this.nodeId.equals(that.nodeId) &&
688 this.deviceId.equals(that.deviceId);
689 } else {
690 return false;
691 }
692 }
693 }
694
Madan Jampanif7536ab2015-05-07 23:23:23 -0700695 private class InternalFlowTable implements ReplicaInfoEventListener {
Madan Jampani86940d92015-05-06 11:47:57 -0700696
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800697 //TODO replace the Map<V,V> with ExtendedSet
698 private final Map<DeviceId, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>
Madan Jampani5c3766c2015-06-02 15:54:41 -0700699 flowEntries = Maps.newConcurrentMap();
Madan Jampani86940d92015-05-06 11:47:57 -0700700
Madan Jampanic6d69f72016-07-15 15:47:12 -0700701 private final Map<BackupOperation, Long> lastBackupTimes = Maps.newConcurrentMap();
Madan Jampani86940d92015-05-06 11:47:57 -0700702 private final Map<DeviceId, Long> lastUpdateTimes = Maps.newConcurrentMap();
Madan Jampani86940d92015-05-06 11:47:57 -0700703
Madan Jampanif7536ab2015-05-07 23:23:23 -0700704 @Override
705 public void event(ReplicaInfoEvent event) {
Madan Jampani71c32ca2016-06-22 08:23:18 -0700706 eventHandler.execute(() -> handleEvent(event));
707 }
708
709 private void handleEvent(ReplicaInfoEvent event) {
Madan Jampanic6d69f72016-07-15 15:47:12 -0700710 DeviceId deviceId = event.subject();
711 if (!backupEnabled || !mastershipService.isLocalMaster(deviceId)) {
Madan Jampania98bf932015-06-02 12:01:36 -0700712 return;
713 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700714 if (event.type() == MASTER_CHANGED) {
715 lastUpdateTimes.put(deviceId, System.currentTimeMillis());
Madan Jampanif7536ab2015-05-07 23:23:23 -0700716 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700717 backupSenderExecutor.schedule(this::backup, 0, TimeUnit.SECONDS);
Madan Jampanif7536ab2015-05-07 23:23:23 -0700718 }
719
Madan Jampaniadea8902015-06-04 17:39:45 -0700720 private void sendBackups(NodeId nodeId, Set<DeviceId> deviceIds) {
721 // split up the devices into smaller batches and send them separately.
722 Iterables.partition(deviceIds, FLOW_TABLE_BACKUP_BATCH_SIZE)
723 .forEach(ids -> backupFlowEntries(nodeId, Sets.newHashSet(ids)));
724 }
725
Madan Jampanif7536ab2015-05-07 23:23:23 -0700726 private void backupFlowEntries(NodeId nodeId, Set<DeviceId> deviceIds) {
Madan Jampaniadea8902015-06-04 17:39:45 -0700727 if (deviceIds.isEmpty()) {
728 return;
729 }
Madan Jampanic6d69f72016-07-15 15:47:12 -0700730 log.debug("Sending flowEntries for devices {} to {} for backup.", deviceIds, nodeId);
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800731 Map<DeviceId, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>
732 deviceFlowEntries = Maps.newConcurrentMap();
Madan Jampani85a9b0d2015-06-03 17:15:44 -0700733 deviceIds.forEach(id -> deviceFlowEntries.put(id, ImmutableMap.copyOf(getFlowTable(id))));
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800734 clusterCommunicator.<Map<DeviceId,
735 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>,
736 Set<DeviceId>>
737 sendAndReceive(deviceFlowEntries,
738 FLOW_TABLE_BACKUP,
739 SERIALIZER::encode,
740 SERIALIZER::decode,
741 nodeId)
742 .whenComplete((backedupDevices, error) -> {
743 Set<DeviceId> devicesNotBackedup = error != null ?
744 deviceFlowEntries.keySet() :
745 Sets.difference(deviceFlowEntries.keySet(), backedupDevices);
746 if (devicesNotBackedup.size() > 0) {
747 log.warn("Failed to backup devices: {}. Reason: {}",
748 devicesNotBackedup, error.getMessage());
749 }
750 if (backedupDevices != null) {
751 backedupDevices.forEach(id -> {
Madan Jampanic6d69f72016-07-15 15:47:12 -0700752 lastBackupTimes.put(new BackupOperation(nodeId, id), System.currentTimeMillis());
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800753 });
754 }
755 });
Madan Jampanif7536ab2015-05-07 23:23:23 -0700756 }
757
Madan Jampani86940d92015-05-06 11:47:57 -0700758 /**
759 * Returns the flow table for specified device.
760 *
761 * @param deviceId identifier of the device
762 * @return Map representing Flow Table of given device.
763 */
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800764 private Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> getFlowTable(DeviceId deviceId) {
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700765 if (persistenceEnabled) {
766 return flowEntries.computeIfAbsent(deviceId, id -> persistenceService
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800767 .<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>persistentMapBuilder()
Aaron Kruglikova62fdbb2015-10-27 16:32:06 -0700768 .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 return flowEntries.computeIfAbsent(deviceId, id -> Maps.newConcurrentMap());
783 }
Madan Jampani86940d92015-05-06 11:47:57 -0700784 }
785
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800786 private Map<StoredFlowEntry, StoredFlowEntry> getFlowEntriesInternal(DeviceId deviceId, FlowId flowId) {
787 return getFlowTable(deviceId).computeIfAbsent(flowId, id -> Maps.newConcurrentMap());
Madan Jampani86940d92015-05-06 11:47:57 -0700788 }
789
790 private StoredFlowEntry getFlowEntryInternal(FlowRule rule) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800791 return getFlowEntriesInternal(rule.deviceId(), rule.id()).get(rule);
Madan Jampani86940d92015-05-06 11:47:57 -0700792 }
793
794 private Set<FlowEntry> getFlowEntriesInternal(DeviceId deviceId) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800795 return getFlowTable(deviceId).values().stream()
796 .flatMap(m -> m.values().stream())
797 .collect(Collectors.toSet());
Madan Jampani86940d92015-05-06 11:47:57 -0700798 }
799
800 public StoredFlowEntry getFlowEntry(FlowRule rule) {
801 return getFlowEntryInternal(rule);
802 }
803
804 public Set<FlowEntry> getFlowEntries(DeviceId deviceId) {
805 return getFlowEntriesInternal(deviceId);
806 }
807
808 public void add(FlowEntry rule) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800809 getFlowEntriesInternal(rule.deviceId(), rule.id())
810 .compute((StoredFlowEntry) rule, (k, stored) -> {
811 //TODO compare stored and rule timestamps
812 //TODO the key is not updated
813 return (StoredFlowEntry) rule;
814 });
Madan Jampani86940d92015-05-06 11:47:57 -0700815 lastUpdateTimes.put(rule.deviceId(), System.currentTimeMillis());
816 }
817
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800818 public FlowEntry remove(DeviceId deviceId, FlowEntry rule) {
819 final AtomicReference<FlowEntry> removedRule = new AtomicReference<>();
820 getFlowEntriesInternal(rule.deviceId(), rule.id())
821 .computeIfPresent((StoredFlowEntry) rule, (k, stored) -> {
822 if (rule instanceof DefaultFlowEntry) {
823 DefaultFlowEntry toRemove = (DefaultFlowEntry) rule;
824 if (stored instanceof DefaultFlowEntry) {
825 DefaultFlowEntry storedEntry = (DefaultFlowEntry) stored;
826 if (toRemove.created() < storedEntry.created()) {
827 log.debug("Trying to remove more recent flow entry {} (stored: {})",
828 toRemove, stored);
829 // the key is not updated, removedRule remains null
830 return stored;
831 }
832 }
833 }
834 removedRule.set(stored);
835 return null;
836 });
837
838 if (removedRule.get() != null) {
Madan Jampani86940d92015-05-06 11:47:57 -0700839 lastUpdateTimes.put(deviceId, System.currentTimeMillis());
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800840 return removedRule.get();
841 } else {
842 return null;
Madan Jampani86940d92015-05-06 11:47:57 -0700843 }
844 }
845
Charles Chan0c7c43b2016-01-14 17:39:20 -0800846 public void purgeFlowRule(DeviceId deviceId) {
847 flowEntries.remove(deviceId);
848 }
849
Madan Jampanic6d69f72016-07-15 15:47:12 -0700850 private List<NodeId> getBackupNodes(DeviceId deviceId) {
851 // The returned backup node list is in the order of preference i.e. next likely master first.
852 List<NodeId> allPossibleBackupNodes = replicaInfoManager.getReplicaInfoFor(deviceId).backups();
853 return ImmutableList.copyOf(allPossibleBackupNodes)
854 .subList(0, Math.min(allPossibleBackupNodes.size(), backupCount));
Madan Jampani86940d92015-05-06 11:47:57 -0700855 }
856
857 private void backup() {
Madan Jampani08bf17b2015-05-06 16:25:26 -0700858 if (!backupEnabled) {
859 return;
860 }
Madan Jampani86940d92015-05-06 11:47:57 -0700861 try {
Madan Jampani86940d92015-05-06 11:47:57 -0700862 // compute a mapping from node to the set of devices whose flow entries it should backup
863 Map<NodeId, Set<DeviceId>> devicesToBackupByNode = Maps.newHashMap();
Madan Jampanic6d69f72016-07-15 15:47:12 -0700864 flowEntries.keySet().stream().forEach(deviceId -> {
865 List<NodeId> backupNodes = getBackupNodes(deviceId);
866 backupNodes.forEach(backupNode -> {
867 if (lastBackupTimes.getOrDefault(new BackupOperation(backupNode, deviceId), 0L)
868 < lastUpdateTimes.getOrDefault(deviceId, 0L)) {
869 devicesToBackupByNode.computeIfAbsent(backupNode,
870 nodeId -> Sets.newHashSet()).add(deviceId);
871 }
872 });
Madan Jampani86940d92015-05-06 11:47:57 -0700873 });
Madan Jampani86940d92015-05-06 11:47:57 -0700874 // send the device flow entries to their respective backup nodes
Madan Jampaniadea8902015-06-04 17:39:45 -0700875 devicesToBackupByNode.forEach(this::sendBackups);
Madan Jampani86940d92015-05-06 11:47:57 -0700876 } catch (Exception e) {
877 log.error("Backup failed.", e);
878 }
879 }
880
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800881 private Set<DeviceId> onBackupReceipt(Map<DeviceId,
882 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>> flowTables) {
Madan Jampani7267c552015-05-20 22:39:17 -0700883 log.debug("Received flowEntries for {} to backup", flowTables.keySet());
Madan Jampani654b58a2015-05-22 11:28:11 -0700884 Set<DeviceId> backedupDevices = Sets.newHashSet();
885 try {
Madan Jampania98bf932015-06-02 12:01:36 -0700886 flowTables.forEach((deviceId, deviceFlowTable) -> {
887 // Only process those devices are that not managed by the local node.
Sho SHIMIZU828bc162016-01-13 23:10:43 -0800888 if (!Objects.equals(local, mastershipService.getMasterFor(deviceId))) {
Brian O'Connora3e5cd52015-12-05 15:59:19 -0800889 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> backupFlowTable =
890 getFlowTable(deviceId);
Madan Jampania98bf932015-06-02 12:01:36 -0700891 backupFlowTable.clear();
892 backupFlowTable.putAll(deviceFlowTable);
893 backedupDevices.add(deviceId);
894 }
Madan Jampani08bf17b2015-05-06 16:25:26 -0700895 });
Madan Jampani654b58a2015-05-22 11:28:11 -0700896 } catch (Exception e) {
897 log.warn("Failure processing backup request", e);
898 }
899 return backedupDevices;
Madan Jampani86940d92015-05-06 11:47:57 -0700900 }
901 }
Srikanth Vavilapalli95810f52015-09-14 15:49:56 -0700902
903 @Override
904 public FlowRuleEvent updateTableStatistics(DeviceId deviceId,
905 List<TableStatisticsEntry> tableStats) {
906 deviceTableStats.put(deviceId, tableStats);
907 return null;
908 }
909
910 @Override
911 public Iterable<TableStatisticsEntry> getTableStatistics(DeviceId deviceId) {
912 NodeId master = mastershipService.getMasterFor(deviceId);
913
914 if (master == null) {
915 log.debug("Failed to getTableStats: No master for {}", deviceId);
916 return Collections.emptyList();
917 }
918
919 List<TableStatisticsEntry> tableStats = deviceTableStats.get(deviceId);
920 if (tableStats == null) {
921 return Collections.emptyList();
922 }
923 return ImmutableList.copyOf(tableStats);
924 }
925
926 private class InternalTableStatsListener
927 implements EventuallyConsistentMapListener<DeviceId, List<TableStatisticsEntry>> {
928 @Override
929 public void event(EventuallyConsistentMapEvent<DeviceId,
930 List<TableStatisticsEntry>> event) {
931 //TODO: Generate an event to listeners (do we need?)
932 }
933 }
Madan Jampani86940d92015-05-06 11:47:57 -0700934}