blob: b8d384121c297047c40c779541074996e9d58462 [file] [log] [blame]
Thomas Vachuska4f1a60c2014-10-28 13:39:07 -07001/*
2 * Copyright 2014 Open Networking Laboratory
3 *
4 * Licensed under the Apache License, Version 2.0 (the "License");
5 * you may not use this file except in compliance with the License.
6 * You may obtain a copy of the License at
7 *
8 * http://www.apache.org/licenses/LICENSE-2.0
9 *
10 * Unless required by applicable law or agreed to in writing, software
11 * distributed under the License is distributed on an "AS IS" BASIS,
12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 * See the License for the specific language governing permissions and
14 * limitations under the License.
15 */
Brian O'Connorabafb502014-12-02 22:26:20 -080016package org.onosproject.store.flow.impl;
alshabib339a3d92014-09-26 17:54:32 -070017
Yuta HIGUCHI92891d12014-10-27 20:04:38 -070018import static com.google.common.base.Preconditions.checkNotNull;
Yuta HIGUCHI9def0472014-10-23 15:51:10 -070019import static org.onlab.util.Tools.namedThreads;
Jonathan Hart4fb5cde2014-12-22 12:09:07 -080020import static org.onosproject.net.flow.FlowRuleEvent.Type.RULE_REMOVED;
21import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.APPLY_BATCH_FLOWS;
22import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.GET_DEVICE_FLOW_ENTRIES;
23import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.GET_FLOW_ENTRY;
24import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.REMOVE_FLOW_ENTRY;
25import static org.slf4j.LoggerFactory.getLogger;
alshabib339a3d92014-09-26 17:54:32 -070026
Madan Jampani38b250d2014-10-17 11:02:38 -070027import java.io.IOException;
Madan Jampani117aaae2014-10-23 10:04:05 -070028import java.util.ArrayList;
29import java.util.Arrays;
alshabib339a3d92014-09-26 17:54:32 -070030import java.util.Collection;
31import java.util.Collections;
Yuta HIGUCHI4b524442014-10-28 22:23:57 -070032import java.util.HashSet;
Jonathan Hart4fb5cde2014-12-22 12:09:07 -080033import java.util.List;
Yuta HIGUCHI92891d12014-10-27 20:04:38 -070034import java.util.Map.Entry;
Madan Jampanif5fdef02014-10-23 21:58:10 -070035import java.util.Set;
Madan Jampani24f9efb2014-10-24 18:56:23 -070036import java.util.concurrent.ExecutionException;
Yuta HIGUCHI9def0472014-10-23 15:51:10 -070037import java.util.concurrent.ExecutorService;
38import java.util.concurrent.Executors;
Madan Jampani117aaae2014-10-23 10:04:05 -070039import java.util.concurrent.Future;
Madan Jampani38b250d2014-10-17 11:02:38 -070040import java.util.concurrent.TimeUnit;
41import java.util.concurrent.TimeoutException;
Yuta HIGUCHI9def0472014-10-23 15:51:10 -070042import java.util.concurrent.atomic.AtomicInteger;
Yuta HIGUCHI885868f2014-11-13 19:12:07 -080043import java.util.concurrent.locks.ReentrantReadWriteLock;
alshabib339a3d92014-09-26 17:54:32 -070044
45import org.apache.felix.scr.annotations.Activate;
46import org.apache.felix.scr.annotations.Component;
47import org.apache.felix.scr.annotations.Deactivate;
Madan Jampani38b250d2014-10-17 11:02:38 -070048import org.apache.felix.scr.annotations.Reference;
49import org.apache.felix.scr.annotations.ReferenceCardinality;
alshabib339a3d92014-09-26 17:54:32 -070050import org.apache.felix.scr.annotations.Service;
Jonathan Hart4fb5cde2014-12-22 12:09:07 -080051import org.onlab.util.KryoNamespace;
Brian O'Connorabafb502014-12-02 22:26:20 -080052import org.onosproject.cluster.ClusterService;
53import org.onosproject.cluster.NodeId;
54import org.onosproject.net.Device;
55import org.onosproject.net.DeviceId;
56import org.onosproject.net.device.DeviceService;
57import org.onosproject.net.flow.CompletedBatchOperation;
58import org.onosproject.net.flow.DefaultFlowEntry;
59import org.onosproject.net.flow.FlowEntry;
60import org.onosproject.net.flow.FlowEntry.FlowEntryState;
61import org.onosproject.net.flow.FlowId;
62import org.onosproject.net.flow.FlowRule;
63import org.onosproject.net.flow.FlowRuleBatchEntry;
Jonathan Hart4fb5cde2014-12-22 12:09:07 -080064import org.onosproject.net.flow.FlowRuleBatchEntry.FlowRuleOperation;
Brian O'Connorabafb502014-12-02 22:26:20 -080065import org.onosproject.net.flow.FlowRuleBatchEvent;
66import org.onosproject.net.flow.FlowRuleBatchOperation;
67import org.onosproject.net.flow.FlowRuleBatchRequest;
68import org.onosproject.net.flow.FlowRuleEvent;
Brian O'Connorabafb502014-12-02 22:26:20 -080069import org.onosproject.net.flow.FlowRuleEvent.Type;
70import org.onosproject.net.flow.FlowRuleStore;
71import org.onosproject.net.flow.FlowRuleStoreDelegate;
72import org.onosproject.net.flow.StoredFlowEntry;
73import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
74import org.onosproject.store.cluster.messaging.ClusterMessage;
75import org.onosproject.store.cluster.messaging.ClusterMessageHandler;
76import org.onosproject.store.flow.ReplicaInfo;
77import org.onosproject.store.flow.ReplicaInfoEvent;
78import org.onosproject.store.flow.ReplicaInfoEventListener;
79import org.onosproject.store.flow.ReplicaInfoService;
80import org.onosproject.store.hz.AbstractHazelcastStore;
81import org.onosproject.store.hz.SMap;
82import org.onosproject.store.serializers.DecodeTo;
83import org.onosproject.store.serializers.KryoSerializer;
84import org.onosproject.store.serializers.StoreSerializer;
85import org.onosproject.store.serializers.impl.DistributedStoreSerializers;
alshabib339a3d92014-09-26 17:54:32 -070086import org.slf4j.Logger;
87
Yuta HIGUCHIbf89c742014-10-27 15:10:02 -070088import com.google.common.cache.Cache;
89import com.google.common.cache.CacheBuilder;
Yuta HIGUCHI92891d12014-10-27 20:04:38 -070090import com.google.common.cache.CacheLoader;
91import com.google.common.cache.LoadingCache;
Yuta HIGUCHIf1ccee82014-11-11 20:39:58 -080092import com.google.common.cache.RemovalListener;
93import com.google.common.cache.RemovalNotification;
alshabib339a3d92014-09-26 17:54:32 -070094import com.google.common.collect.ArrayListMultimap;
Yuta HIGUCHI92891d12014-10-27 20:04:38 -070095import com.google.common.collect.ImmutableList;
alshabib339a3d92014-09-26 17:54:32 -070096import com.google.common.collect.ImmutableSet;
Yuta HIGUCHI9def0472014-10-23 15:51:10 -070097import com.google.common.collect.Iterables;
alshabib339a3d92014-09-26 17:54:32 -070098import com.google.common.collect.Multimap;
Madan Jampani117aaae2014-10-23 10:04:05 -070099import com.google.common.util.concurrent.Futures;
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700100import com.google.common.util.concurrent.ListenableFuture;
101import com.google.common.util.concurrent.SettableFuture;
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700102import com.hazelcast.core.IMap;
alshabib339a3d92014-09-26 17:54:32 -0700103
104/**
Madan Jampani38b250d2014-10-17 11:02:38 -0700105 * Manages inventory of flow rules using a distributed state management protocol.
alshabib339a3d92014-09-26 17:54:32 -0700106 */
alshabib339a3d92014-09-26 17:54:32 -0700107@Component(immediate = true)
108@Service
109public class DistributedFlowRuleStore
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700110 extends AbstractHazelcastStore<FlowRuleBatchEvent, FlowRuleStoreDelegate>
alshabib1c319ff2014-10-04 20:29:09 -0700111 implements FlowRuleStore {
alshabib339a3d92014-09-26 17:54:32 -0700112
113 private final Logger log = getLogger(getClass());
114
Yuta HIGUCHIe9b2b002014-11-04 12:25:47 -0800115 // primary data:
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800116 // read/write needs to be locked
117 private final ReentrantReadWriteLock flowEntriesLock = new ReentrantReadWriteLock();
alshabib339a3d92014-09-26 17:54:32 -0700118 // store entries as a pile of rules, no info about device tables
Yuta HIGUCHIe9b2b002014-11-04 12:25:47 -0800119 private final Multimap<DeviceId, StoredFlowEntry> flowEntries
120 = ArrayListMultimap.<DeviceId, StoredFlowEntry>create();
alshabib339a3d92014-09-26 17:54:32 -0700121
Madan Jampani38b250d2014-10-17 11:02:38 -0700122 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700123 protected ReplicaInfoService replicaInfoManager;
Madan Jampani38b250d2014-10-17 11:02:38 -0700124
125 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700126 protected ClusterCommunicationService clusterCommunicator;
Madan Jampani38b250d2014-10-17 11:02:38 -0700127
128 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700129 protected ClusterService clusterService;
130
131 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
132 protected DeviceService deviceService;
133
134 private final AtomicInteger localBatchIdGen = new AtomicInteger();
135
Yuta HIGUCHIbf89c742014-10-27 15:10:02 -0700136 private int pendingFutureTimeoutMinutes = 5;
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700137
Yuta HIGUCHIbf89c742014-10-27 15:10:02 -0700138 private Cache<Integer, SettableFuture<CompletedBatchOperation>> pendingFutures =
139 CacheBuilder.newBuilder()
140 .expireAfterWrite(pendingFutureTimeoutMinutes, TimeUnit.MINUTES)
Yuta HIGUCHIf1ccee82014-11-11 20:39:58 -0800141 .removalListener(new TimeoutFuture())
Yuta HIGUCHIbf89c742014-10-27 15:10:02 -0700142 .build();
143
Yuta HIGUCHIe9b2b002014-11-04 12:25:47 -0800144 // Cache of SMaps used for backup data. each SMap contain device flow table
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700145 private LoadingCache<DeviceId, SMap<FlowId, ImmutableList<StoredFlowEntry>>> smaps;
146
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700147
148 private final ExecutorService futureListeners =
Thomas Vachuska9ea3e6f2015-01-23 16:34:22 -0800149 Executors.newCachedThreadPool(namedThreads("onos-flowstore-peer-responders"));
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700150
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700151 private final ExecutorService backupExecutors =
Thomas Vachuska9ea3e6f2015-01-23 16:34:22 -0800152 Executors.newSingleThreadExecutor(namedThreads("onos-async-backups"));
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700153
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700154 private boolean syncBackup = false;
Madan Jampani38b250d2014-10-17 11:02:38 -0700155
Yuta HIGUCHIea150152014-10-28 22:55:14 -0700156 protected static final StoreSerializer SERIALIZER = new KryoSerializer() {
Madan Jampani38b250d2014-10-17 11:02:38 -0700157 @Override
158 protected void setupKryoPool() {
Yuta HIGUCHI8d143d22014-10-19 23:15:09 -0700159 serializerPool = KryoNamespace.newBuilder()
Yuta HIGUCHI91768e32014-11-22 05:06:35 -0800160 .register(DistributedStoreSerializers.STORE_COMMON)
161 .nextId(DistributedStoreSerializers.STORE_CUSTOM_BEGIN)
Yuta HIGUCHI2f158332014-11-25 13:32:06 -0800162 .register(FlowRuleEvent.class)
Jonathan Hart4fb5cde2014-12-22 12:09:07 -0800163 .register(FlowRuleEvent.Type.class)
Yuta HIGUCHI91768e32014-11-22 05:06:35 -0800164 .build();
Madan Jampani38b250d2014-10-17 11:02:38 -0700165 }
166 };
167
Yuta HIGUCHIf3d51bd2014-10-21 01:05:33 -0700168 private static final long FLOW_RULE_STORE_TIMEOUT_MILLIS = 5000;
Madan Jampani38b250d2014-10-17 11:02:38 -0700169
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700170 private ReplicaInfoEventListener replicaInfoEventListener;
171
172 @Override
alshabib339a3d92014-09-26 17:54:32 -0700173 @Activate
174 public void activate() {
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700175
176 super.serializer = SERIALIZER;
177 super.theInstance = storeService.getHazelcastInstance();
178
179 // Cache to create SMap on demand
180 smaps = CacheBuilder.newBuilder()
181 .softValues()
182 .build(new SMapLoader());
183
Yuta HIGUCHI4b524442014-10-28 22:23:57 -0700184 final NodeId local = clusterService.getLocalNode().id();
185
Yuta HIGUCHIea150152014-10-28 22:55:14 -0700186 clusterCommunicator.addSubscriber(APPLY_BATCH_FLOWS, new OnStoreBatch(local));
Madan Jampani117aaae2014-10-23 10:04:05 -0700187
188 clusterCommunicator.addSubscriber(GET_FLOW_ENTRY, new ClusterMessageHandler() {
189
190 @Override
191 public void handle(ClusterMessage message) {
192 FlowRule rule = SERIALIZER.decode(message.payload());
Yuta HIGUCHIfaf9e1c2014-11-20 00:31:29 -0800193 log.trace("received get flow entry request for {}", rule);
Madan Jampani117aaae2014-10-23 10:04:05 -0700194 FlowEntry flowEntry = getFlowEntryInternal(rule);
195 try {
196 message.respond(SERIALIZER.encode(flowEntry));
197 } catch (IOException e) {
198 log.error("Failed to respond back", e);
199 }
200 }
201 });
202
Madan Jampanif5fdef02014-10-23 21:58:10 -0700203 clusterCommunicator.addSubscriber(GET_DEVICE_FLOW_ENTRIES, new ClusterMessageHandler() {
204
205 @Override
206 public void handle(ClusterMessage message) {
207 DeviceId deviceId = SERIALIZER.decode(message.payload());
Yuta HIGUCHIfaf9e1c2014-11-20 00:31:29 -0800208 log.trace("Received get flow entries request for {} from {}", deviceId, message.sender());
Madan Jampanif5fdef02014-10-23 21:58:10 -0700209 Set<FlowEntry> flowEntries = getFlowEntriesInternal(deviceId);
210 try {
211 message.respond(SERIALIZER.encode(flowEntries));
212 } catch (IOException e) {
213 log.error("Failed to respond to peer's getFlowEntries request", e);
214 }
215 }
216 });
217
Yuta HIGUCHIdfa45c12014-11-24 18:38:53 -0800218 clusterCommunicator.addSubscriber(REMOVE_FLOW_ENTRY, new ClusterMessageHandler() {
219
220 @Override
221 public void handle(ClusterMessage message) {
222 FlowEntry rule = SERIALIZER.decode(message.payload());
223 log.trace("received get flow entry request for {}", rule);
224 FlowRuleEvent event = removeFlowRuleInternal(rule);
225 try {
226 message.respond(SERIALIZER.encode(event));
227 } catch (IOException e) {
228 log.error("Failed to respond back", e);
229 }
230 }
231 });
232
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700233 replicaInfoEventListener = new InternalReplicaInfoEventListener();
234
235 replicaInfoManager.addListener(replicaInfoEventListener);
236
alshabib339a3d92014-09-26 17:54:32 -0700237 log.info("Started");
238 }
239
240 @Deactivate
241 public void deactivate() {
Yuta HIGUCHIdfa45c12014-11-24 18:38:53 -0800242 clusterCommunicator.removeSubscriber(REMOVE_FLOW_ENTRY);
243 clusterCommunicator.removeSubscriber(GET_DEVICE_FLOW_ENTRIES);
244 clusterCommunicator.removeSubscriber(GET_FLOW_ENTRY);
245 clusterCommunicator.removeSubscriber(APPLY_BATCH_FLOWS);
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700246 replicaInfoManager.removeListener(replicaInfoEventListener);
alshabib339a3d92014-09-26 17:54:32 -0700247 log.info("Stopped");
248 }
249
250
Brian O'Connor44008532014-12-04 16:41:36 -0800251 // This is not a efficient operation on a distributed sharded
Madan Jampani117aaae2014-10-23 10:04:05 -0700252 // flow store. We need to revisit the need for this operation or at least
253 // make it device specific.
alshabib339a3d92014-09-26 17:54:32 -0700254 @Override
tom9b4030d2014-10-06 10:39:03 -0700255 public int getFlowRuleCount() {
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700256 // implementing in-efficient operation for debugging purpose.
257 int sum = 0;
258 for (Device device : deviceService.getDevices()) {
259 final DeviceId did = device.id();
260 sum += Iterables.size(getFlowEntries(did));
261 }
262 return sum;
tom9b4030d2014-10-06 10:39:03 -0700263 }
264
265 @Override
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800266 public FlowEntry getFlowEntry(FlowRule rule) {
Madan Jampani117aaae2014-10-23 10:04:05 -0700267 ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(rule.deviceId());
Yuta HIGUCHI4b524442014-10-28 22:23:57 -0700268
269 if (!replicaInfo.master().isPresent()) {
Yuta HIGUCHI6b98ab62014-12-03 16:08:08 -0800270 log.warn("Failed to getFlowEntry: No master for {}", rule.deviceId());
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800271 return null;
Yuta HIGUCHI4b524442014-10-28 22:23:57 -0700272 }
273
Madan Jampani117aaae2014-10-23 10:04:05 -0700274 if (replicaInfo.master().get().equals(clusterService.getLocalNode().id())) {
275 return getFlowEntryInternal(rule);
276 }
277
Yuta HIGUCHIfaf9e1c2014-11-20 00:31:29 -0800278 log.trace("Forwarding getFlowEntry to {}, which is the primary (master) for device {}",
Madan Jampani117aaae2014-10-23 10:04:05 -0700279 replicaInfo.master().orNull(), rule.deviceId());
280
281 ClusterMessage message = new ClusterMessage(
282 clusterService.getLocalNode().id(),
283 FlowStoreMessageSubjects.GET_FLOW_ENTRY,
284 SERIALIZER.encode(rule));
285
286 try {
Madan Jampani24f9efb2014-10-24 18:56:23 -0700287 Future<byte[]> responseFuture = clusterCommunicator.sendAndReceive(message, replicaInfo.master().get());
288 return SERIALIZER.decode(responseFuture.get(FLOW_RULE_STORE_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS));
289 } catch (IOException | TimeoutException | ExecutionException | InterruptedException e) {
Brian O'Connor44008532014-12-04 16:41:36 -0800290 log.warn("Unable to fetch flow store contents from {}", replicaInfo.master().get());
Madan Jampani117aaae2014-10-23 10:04:05 -0700291 }
Brian O'Connor44008532014-12-04 16:41:36 -0800292 return null;
Yuta HIGUCHIf6f50a62014-10-19 15:58:49 -0700293 }
294
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800295 private StoredFlowEntry getFlowEntryInternal(FlowRule rule) {
296 flowEntriesLock.readLock().lock();
297 try {
298 for (StoredFlowEntry f : flowEntries.get(rule.deviceId())) {
299 if (f.equals(rule)) {
300 return f;
301 }
alshabib339a3d92014-09-26 17:54:32 -0700302 }
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800303 } finally {
304 flowEntriesLock.readLock().unlock();
alshabib339a3d92014-09-26 17:54:32 -0700305 }
306 return null;
307 }
308
309 @Override
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800310 public Iterable<FlowEntry> getFlowEntries(DeviceId deviceId) {
Madan Jampanif5fdef02014-10-23 21:58:10 -0700311
312 ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(deviceId);
Yuta HIGUCHI4b524442014-10-28 22:23:57 -0700313
314 if (!replicaInfo.master().isPresent()) {
Yuta HIGUCHI6b98ab62014-12-03 16:08:08 -0800315 log.warn("Failed to getFlowEntries: No master for {}", deviceId);
Yuta HIGUCHI2c1d8472014-10-31 14:13:38 -0700316 return Collections.emptyList();
Yuta HIGUCHI4b524442014-10-28 22:23:57 -0700317 }
318
Madan Jampanif5fdef02014-10-23 21:58:10 -0700319 if (replicaInfo.master().get().equals(clusterService.getLocalNode().id())) {
320 return getFlowEntriesInternal(deviceId);
321 }
322
Yuta HIGUCHIfaf9e1c2014-11-20 00:31:29 -0800323 log.trace("Forwarding getFlowEntries to {}, which is the primary (master) for device {}",
Madan Jampanif5fdef02014-10-23 21:58:10 -0700324 replicaInfo.master().orNull(), deviceId);
325
326 ClusterMessage message = new ClusterMessage(
327 clusterService.getLocalNode().id(),
328 GET_DEVICE_FLOW_ENTRIES,
329 SERIALIZER.encode(deviceId));
330
331 try {
Madan Jampani24f9efb2014-10-24 18:56:23 -0700332 Future<byte[]> responseFuture = clusterCommunicator.sendAndReceive(message, replicaInfo.master().get());
333 return SERIALIZER.decode(responseFuture.get(FLOW_RULE_STORE_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS));
334 } catch (IOException | TimeoutException | ExecutionException | InterruptedException e) {
Brian O'Connor44008532014-12-04 16:41:36 -0800335 log.warn("Unable to fetch flow store contents from {}", replicaInfo.master().get());
Madan Jampanif5fdef02014-10-23 21:58:10 -0700336 }
Yuta HIGUCHI24f79eb2014-12-12 15:46:43 -0800337 return Collections.emptyList();
Madan Jampanif5fdef02014-10-23 21:58:10 -0700338 }
339
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800340 private Set<FlowEntry> getFlowEntriesInternal(DeviceId deviceId) {
341 flowEntriesLock.readLock().lock();
342 try {
343 Collection<? extends FlowEntry> rules = flowEntries.get(deviceId);
344 if (rules == null) {
345 return Collections.emptySet();
346 }
347 return ImmutableSet.copyOf(rules);
348 } finally {
349 flowEntriesLock.readLock().unlock();
alshabib339a3d92014-09-26 17:54:32 -0700350 }
alshabib339a3d92014-09-26 17:54:32 -0700351 }
352
353 @Override
Madan Jampani117aaae2014-10-23 10:04:05 -0700354 public void storeFlowRule(FlowRule rule) {
355 storeBatch(new FlowRuleBatchOperation(Arrays.asList(new FlowRuleBatchEntry(FlowRuleOperation.ADD, rule))));
356 }
357
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700358 @Override
Madan Jampani117aaae2014-10-23 10:04:05 -0700359 public Future<CompletedBatchOperation> storeBatch(FlowRuleBatchOperation operation) {
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700360
Madan Jampani117aaae2014-10-23 10:04:05 -0700361 if (operation.getOperations().isEmpty()) {
Brian O'Connor427a1762014-11-19 18:40:32 -0800362 return Futures.immediateFuture(new CompletedBatchOperation(true,
363 Collections.<FlowRule>emptySet()));
alshabib339a3d92014-09-26 17:54:32 -0700364 }
Madan Jampani38b250d2014-10-17 11:02:38 -0700365
Madan Jampani117aaae2014-10-23 10:04:05 -0700366 DeviceId deviceId = operation.getOperations().get(0).getTarget().deviceId();
367
368 ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(deviceId);
369
Yuta HIGUCHI4b524442014-10-28 22:23:57 -0700370 if (!replicaInfo.master().isPresent()) {
Yuta HIGUCHI6b98ab62014-12-03 16:08:08 -0800371 log.warn("Failed to storeBatch: No master for {}", deviceId);
Yuta HIGUCHI4b524442014-10-28 22:23:57 -0700372 // TODO: revisit if this should be "success" from Future point of view
373 // with every FlowEntry failed
Yuta HIGUCHI6b98ab62014-12-03 16:08:08 -0800374 return Futures.immediateFailedFuture(new IOException("Failed to storeBatch: No master for " + deviceId));
Yuta HIGUCHI4b524442014-10-28 22:23:57 -0700375 }
376
377 final NodeId local = clusterService.getLocalNode().id();
378 if (replicaInfo.master().get().equals(local)) {
Madan Jampani117aaae2014-10-23 10:04:05 -0700379 return storeBatchInternal(operation);
380 }
381
Yuta HIGUCHIfaf9e1c2014-11-20 00:31:29 -0800382 log.trace("Forwarding storeBatch to {}, which is the primary (master) for device {}",
Madan Jampani117aaae2014-10-23 10:04:05 -0700383 replicaInfo.master().orNull(), deviceId);
Yuta HIGUCHIf3d51bd2014-10-21 01:05:33 -0700384
Madan Jampani38b250d2014-10-17 11:02:38 -0700385 ClusterMessage message = new ClusterMessage(
Yuta HIGUCHI4b524442014-10-28 22:23:57 -0700386 local,
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700387 APPLY_BATCH_FLOWS,
Madan Jampani117aaae2014-10-23 10:04:05 -0700388 SERIALIZER.encode(operation));
Madan Jampani38b250d2014-10-17 11:02:38 -0700389
390 try {
Madan Jampani24f9efb2014-10-24 18:56:23 -0700391 ListenableFuture<byte[]> responseFuture =
392 clusterCommunicator.sendAndReceive(message, replicaInfo.master().get());
Yuta HIGUCHIea150152014-10-28 22:55:14 -0700393 return Futures.transform(responseFuture, new DecodeTo<CompletedBatchOperation>(SERIALIZER));
Madan Jampani24f9efb2014-10-24 18:56:23 -0700394 } catch (IOException e) {
395 return Futures.immediateFailedFuture(e);
Madan Jampani38b250d2014-10-17 11:02:38 -0700396 }
397 }
398
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800399 private ListenableFuture<CompletedBatchOperation>
Yuta HIGUCHIe9b2b002014-11-04 12:25:47 -0800400 storeBatchInternal(FlowRuleBatchOperation operation) {
401
Brian O'Connor427a1762014-11-19 18:40:32 -0800402 final List<FlowRuleBatchEntry> toRemove = new ArrayList<>();
403 final List<FlowRuleBatchEntry> toAdd = new ArrayList<>();
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700404 DeviceId did = null;
405
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700406
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800407 flowEntriesLock.writeLock().lock();
408 try {
409 for (FlowRuleBatchEntry batchEntry : operation.getOperations()) {
410 FlowRule flowRule = batchEntry.getTarget();
411 FlowRuleOperation op = batchEntry.getOperator();
412 if (did == null) {
413 did = flowRule.deviceId();
414 }
415 if (op.equals(FlowRuleOperation.REMOVE)) {
416 StoredFlowEntry entry = getFlowEntryInternal(flowRule);
417 if (entry != null) {
418 entry.setState(FlowEntryState.PENDING_REMOVE);
Brian O'Connor427a1762014-11-19 18:40:32 -0800419 toRemove.add(batchEntry);
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800420 }
421 } else if (op.equals(FlowRuleOperation.ADD)) {
422 StoredFlowEntry flowEntry = new DefaultFlowEntry(flowRule);
423 DeviceId deviceId = flowRule.deviceId();
Yuta HIGUCHI08f156b2014-12-12 13:34:05 -0800424 Collection<StoredFlowEntry> ft = flowEntries.get(deviceId);
425
426 // always add requested FlowRule
427 // Note: 2 equal FlowEntry may have different treatment
428 ft.remove(flowEntry);
429 ft.add(flowEntry);
430 toAdd.add(batchEntry);
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800431 }
432 }
433 if (toAdd.isEmpty() && toRemove.isEmpty()) {
434 return Futures.immediateFuture(new CompletedBatchOperation(true, Collections.<FlowRule>emptySet()));
435 }
436
437 // create remote backup copies
438 updateBackup(did, toAdd, toRemove);
439 } finally {
440 flowEntriesLock.writeLock().unlock();
441 }
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700442
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700443 SettableFuture<CompletedBatchOperation> r = SettableFuture.create();
444 final int batchId = localBatchIdGen.incrementAndGet();
445
446 pendingFutures.put(batchId, r);
447 notifyDelegate(FlowRuleBatchEvent.requested(new FlowRuleBatchRequest(batchId, toAdd, toRemove)));
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700448
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700449 return r;
alshabib339a3d92014-09-26 17:54:32 -0700450 }
451
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700452 private void updateBackup(final DeviceId deviceId,
Brian O'Connor427a1762014-11-19 18:40:32 -0800453 final List<FlowRuleBatchEntry> toAdd,
454 final List<FlowRuleBatchEntry> list) {
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700455
456 Future<?> submit = backupExecutors.submit(new UpdateBackup(deviceId, toAdd, list));
457
458 if (syncBackup) {
459 // wait for backup to complete
460 try {
461 submit.get();
462 } catch (InterruptedException | ExecutionException e) {
463 log.error("Failed to create backups", e);
464 }
465 }
466 }
467
Brian O'Connor427a1762014-11-19 18:40:32 -0800468 private void updateBackup(DeviceId deviceId, List<FlowRuleBatchEntry> toAdd) {
469
470 updateBackup(deviceId, toAdd, Collections.<FlowRuleBatchEntry>emptyList());
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700471 }
472
alshabib339a3d92014-09-26 17:54:32 -0700473 @Override
Madan Jampani117aaae2014-10-23 10:04:05 -0700474 public void deleteFlowRule(FlowRule rule) {
475 storeBatch(new FlowRuleBatchOperation(Arrays.asList(new FlowRuleBatchEntry(FlowRuleOperation.REMOVE, rule))));
alshabib339a3d92014-09-26 17:54:32 -0700476 }
477
478 @Override
Madan Jampani38b250d2014-10-17 11:02:38 -0700479 public FlowRuleEvent addOrUpdateFlowRule(FlowEntry rule) {
480 ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(rule.deviceId());
Yuta HIGUCHI4b524442014-10-28 22:23:57 -0700481 final NodeId localId = clusterService.getLocalNode().id();
482 if (localId.equals(replicaInfo.master().orNull())) {
Madan Jampani38b250d2014-10-17 11:02:38 -0700483 return addOrUpdateFlowRuleInternal(rule);
484 }
485
Yuta HIGUCHIfaf9e1c2014-11-20 00:31:29 -0800486 log.warn("Tried to update FlowRule {} state,"
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700487 + " while the Node was not the master.", rule);
488 return null;
Madan Jampani38b250d2014-10-17 11:02:38 -0700489 }
490
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800491 private FlowRuleEvent addOrUpdateFlowRuleInternal(FlowEntry rule) {
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700492 final DeviceId did = rule.deviceId();
alshabib339a3d92014-09-26 17:54:32 -0700493
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800494 flowEntriesLock.writeLock().lock();
495 try {
496 // check if this new rule is an update to an existing entry
497 StoredFlowEntry stored = getFlowEntryInternal(rule);
498 if (stored != null) {
499 stored.setBytes(rule.bytes());
500 stored.setLife(rule.life());
501 stored.setPackets(rule.packets());
502 if (stored.state() == FlowEntryState.PENDING_ADD) {
503 stored.setState(FlowEntryState.ADDED);
Brian O'Connor427a1762014-11-19 18:40:32 -0800504 FlowRuleBatchEntry entry =
505 new FlowRuleBatchEntry(FlowRuleOperation.ADD, stored);
506 updateBackup(did, Arrays.asList(entry));
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800507 return new FlowRuleEvent(Type.RULE_ADDED, rule);
508 }
509 return new FlowRuleEvent(Type.RULE_UPDATED, rule);
alshabib1c319ff2014-10-04 20:29:09 -0700510 }
alshabib339a3d92014-09-26 17:54:32 -0700511
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800512 // TODO: Confirm if this behavior is correct. See SimpleFlowRuleStore
Yuta HIGUCHI65934892014-12-04 17:47:44 -0800513 // TODO: also update backup if the behavior is correct.
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800514 flowEntries.put(did, new DefaultFlowEntry(rule));
515 } finally {
516 flowEntriesLock.writeLock().unlock();
517 }
alshabib1c319ff2014-10-04 20:29:09 -0700518 return null;
Madan Jampani38b250d2014-10-17 11:02:38 -0700519
alshabib339a3d92014-09-26 17:54:32 -0700520 }
521
522 @Override
Madan Jampani38b250d2014-10-17 11:02:38 -0700523 public FlowRuleEvent removeFlowRule(FlowEntry rule) {
Yuta HIGUCHIb6eb9142014-11-26 16:18:13 -0800524 final DeviceId deviceId = rule.deviceId();
525 ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(deviceId);
Yuta HIGUCHI4b524442014-10-28 22:23:57 -0700526
527 final NodeId localId = clusterService.getLocalNode().id();
528 if (localId.equals(replicaInfo.master().orNull())) {
Madan Jampani38b250d2014-10-17 11:02:38 -0700529 // bypass and handle it locally
530 return removeFlowRuleInternal(rule);
531 }
532
Yuta HIGUCHIb6eb9142014-11-26 16:18:13 -0800533 if (!replicaInfo.master().isPresent()) {
Yuta HIGUCHI6b98ab62014-12-03 16:08:08 -0800534 log.warn("Failed to removeFlowRule: No master for {}", deviceId);
Yuta HIGUCHIb6eb9142014-11-26 16:18:13 -0800535 // TODO: revisit if this should be null (="no-op") or Exception
536 return null;
537 }
538
Yuta HIGUCHIdfa45c12014-11-24 18:38:53 -0800539 log.trace("Forwarding removeFlowRule to {}, which is the primary (master) for device {}",
Yuta HIGUCHIb6eb9142014-11-26 16:18:13 -0800540 replicaInfo.master().orNull(), deviceId);
Yuta HIGUCHIdfa45c12014-11-24 18:38:53 -0800541
542 ClusterMessage message = new ClusterMessage(
543 clusterService.getLocalNode().id(),
544 REMOVE_FLOW_ENTRY,
545 SERIALIZER.encode(rule));
546
547 try {
548 Future<byte[]> responseFuture = clusterCommunicator.sendAndReceive(message, replicaInfo.master().get());
549 return SERIALIZER.decode(responseFuture.get(FLOW_RULE_STORE_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS));
550 } catch (IOException | TimeoutException | ExecutionException | InterruptedException e) {
Yuta HIGUCHI65934892014-12-04 17:47:44 -0800551 // TODO: Retry against latest master or throw a FlowStoreException
Yuta HIGUCHIdfa45c12014-11-24 18:38:53 -0800552 throw new RuntimeException(e);
553 }
Madan Jampani38b250d2014-10-17 11:02:38 -0700554 }
555
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800556 private FlowRuleEvent removeFlowRuleInternal(FlowEntry rule) {
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700557 final DeviceId deviceId = rule.deviceId();
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800558 flowEntriesLock.writeLock().lock();
559 try {
560 // This is where one could mark a rule as removed and still keep it in the store.
561 final boolean removed = flowEntries.remove(deviceId, rule);
Brian O'Connor427a1762014-11-19 18:40:32 -0800562 FlowRuleBatchEntry entry =
563 new FlowRuleBatchEntry(FlowRuleOperation.REMOVE, rule);
564 updateBackup(deviceId, Collections.<FlowRuleBatchEntry>emptyList(), Arrays.asList(entry));
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800565 if (removed) {
566 return new FlowRuleEvent(RULE_REMOVED, rule);
567 } else {
568 return null;
569 }
570 } finally {
571 flowEntriesLock.writeLock().unlock();
alshabib339a3d92014-09-26 17:54:32 -0700572 }
alshabib339a3d92014-09-26 17:54:32 -0700573 }
Madan Jampani117aaae2014-10-23 10:04:05 -0700574
575 @Override
576 public void batchOperationComplete(FlowRuleBatchEvent event) {
Yuta HIGUCHIbf89c742014-10-27 15:10:02 -0700577 final Integer batchId = event.subject().batchId();
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700578 SettableFuture<CompletedBatchOperation> future
Yuta HIGUCHIbf89c742014-10-27 15:10:02 -0700579 = pendingFutures.getIfPresent(batchId);
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700580 if (future != null) {
581 future.set(event.result());
Yuta HIGUCHIbf89c742014-10-27 15:10:02 -0700582 pendingFutures.invalidate(batchId);
Yuta HIGUCHI9def0472014-10-23 15:51:10 -0700583 }
Madan Jampani117aaae2014-10-23 10:04:05 -0700584 notifyDelegate(event);
585 }
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700586
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800587 private void loadFromBackup(final DeviceId did) {
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700588
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800589 flowEntriesLock.writeLock().lock();
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700590 try {
Yuta HIGUCHIfaf9e1c2014-11-20 00:31:29 -0800591 log.debug("Loading FlowRules for {} from backups", did);
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700592 SMap<FlowId, ImmutableList<StoredFlowEntry>> backupFlowTable = smaps.get(did);
593 for (Entry<FlowId, ImmutableList<StoredFlowEntry>> e
594 : backupFlowTable.entrySet()) {
595
Yuta HIGUCHIfaf9e1c2014-11-20 00:31:29 -0800596 log.trace("loading {}", e.getValue());
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700597 for (StoredFlowEntry entry : e.getValue()) {
598 flowEntries.remove(did, entry);
599 flowEntries.put(did, entry);
600 }
601 }
602 } catch (ExecutionException e) {
603 log.error("Failed to load backup flowtable for {}", did, e);
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800604 } finally {
605 flowEntriesLock.writeLock().unlock();
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700606 }
607 }
608
Yuta HIGUCHI885868f2014-11-13 19:12:07 -0800609 private void removeFromPrimary(final DeviceId did) {
610 Collection<StoredFlowEntry> removed = null;
611 flowEntriesLock.writeLock().lock();
612 try {
613 removed = flowEntries.removeAll(did);
614 } finally {
615 flowEntriesLock.writeLock().unlock();
616 }
Yuta HIGUCHIfaf9e1c2014-11-20 00:31:29 -0800617 log.trace("removedFromPrimary {}", removed);
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700618 }
619
Yuta HIGUCHIf1ccee82014-11-11 20:39:58 -0800620 private static final class TimeoutFuture
621 implements RemovalListener<Integer, SettableFuture<CompletedBatchOperation>> {
622 @Override
623 public void onRemoval(RemovalNotification<Integer, SettableFuture<CompletedBatchOperation>> notification) {
624 // wrapping in ExecutionException to support Future.get
625 notification.getValue()
626 .setException(new ExecutionException("Timed out",
627 new TimeoutException()));
628 }
629 }
630
Yuta HIGUCHIea150152014-10-28 22:55:14 -0700631 private final class OnStoreBatch implements ClusterMessageHandler {
632 private final NodeId local;
633
634 private OnStoreBatch(NodeId local) {
635 this.local = local;
636 }
637
638 @Override
639 public void handle(final ClusterMessage message) {
640 FlowRuleBatchOperation operation = SERIALIZER.decode(message.payload());
Yuta HIGUCHIfaf9e1c2014-11-20 00:31:29 -0800641 log.debug("received batch request {}", operation);
Yuta HIGUCHIea150152014-10-28 22:55:14 -0700642
643 final DeviceId deviceId = operation.getOperations().get(0).getTarget().deviceId();
644 ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(deviceId);
645 if (!local.equals(replicaInfo.master().orNull())) {
646
647 Set<FlowRule> failures = new HashSet<>(operation.size());
648 for (FlowRuleBatchEntry op : operation.getOperations()) {
649 failures.add(op.getTarget());
650 }
651 CompletedBatchOperation allFailed = new CompletedBatchOperation(false, failures);
652 // This node is no longer the master, respond as all failed.
653 // TODO: we might want to wrap response in envelope
654 // to distinguish sw programming failure and hand over
655 // it make sense in the latter case to retry immediately.
656 try {
657 message.respond(SERIALIZER.encode(allFailed));
658 } catch (IOException e) {
659 log.error("Failed to respond back", e);
660 }
661 return;
662 }
663
664 final ListenableFuture<CompletedBatchOperation> f = storeBatchInternal(operation);
665
666 f.addListener(new Runnable() {
667
668 @Override
669 public void run() {
Yuta HIGUCHIf1ccee82014-11-11 20:39:58 -0800670 CompletedBatchOperation result;
671 try {
672 result = f.get();
673 } catch (InterruptedException | ExecutionException e) {
674 log.error("Batch operation failed", e);
675 // create everything failed response
676 Set<FlowRule> failures = new HashSet<>(operation.size());
677 for (FlowRuleBatchEntry op : operation.getOperations()) {
678 failures.add(op.getTarget());
679 }
680 result = new CompletedBatchOperation(false, failures);
681 }
Yuta HIGUCHIea150152014-10-28 22:55:14 -0700682 try {
683 message.respond(SERIALIZER.encode(result));
684 } catch (IOException e) {
685 log.error("Failed to respond back", e);
686 }
687 }
688 }, futureListeners);
689 }
690 }
691
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700692 private final class SMapLoader
693 extends CacheLoader<DeviceId, SMap<FlowId, ImmutableList<StoredFlowEntry>>> {
694
695 @Override
696 public SMap<FlowId, ImmutableList<StoredFlowEntry>> load(DeviceId id)
697 throws Exception {
698 IMap<byte[], byte[]> map = theInstance.getMap("flowtable_" + id.toString());
699 return new SMap<FlowId, ImmutableList<StoredFlowEntry>>(map, SERIALIZER);
700 }
701 }
702
703 private final class InternalReplicaInfoEventListener
704 implements ReplicaInfoEventListener {
705
706 @Override
707 public void event(ReplicaInfoEvent event) {
708 final NodeId local = clusterService.getLocalNode().id();
709 final DeviceId did = event.subject();
710 final ReplicaInfo rInfo = event.replicaInfo();
711
712 switch (event.type()) {
713 case MASTER_CHANGED:
714 if (local.equals(rInfo.master().orNull())) {
715 // This node is the new master, populate local structure
716 // from backup
717 loadFromBackup(did);
718 } else {
719 // This node is no longer the master holder,
720 // clean local structure
721 removeFromPrimary(did);
Yuta HIGUCHI65934892014-12-04 17:47:44 -0800722 // TODO: probably should stop pending backup activities in
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700723 // executors to avoid overwriting with old value
724 }
725 break;
726 default:
727 break;
728
729 }
730 }
731 }
732
733 // Task to update FlowEntries in backup HZ store
Brian O'Connor427a1762014-11-19 18:40:32 -0800734 // TODO: Should be refactored to contain only one list and not
735 // toAdd and toRemove
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700736 private final class UpdateBackup implements Runnable {
737
738 private final DeviceId deviceId;
Brian O'Connor427a1762014-11-19 18:40:32 -0800739 private final List<FlowRuleBatchEntry> toAdd;
740 private final List<FlowRuleBatchEntry> toRemove;
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700741
742 public UpdateBackup(DeviceId deviceId,
Brian O'Connor427a1762014-11-19 18:40:32 -0800743 List<FlowRuleBatchEntry> toAdd,
744 List<FlowRuleBatchEntry> list) {
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700745 this.deviceId = checkNotNull(deviceId);
746 this.toAdd = checkNotNull(toAdd);
747 this.toRemove = checkNotNull(list);
748 }
749
750 @Override
751 public void run() {
752 try {
Yuta HIGUCHIfaf9e1c2014-11-20 00:31:29 -0800753 log.trace("update backup {} +{} -{}", deviceId, toAdd, toRemove);
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700754 final SMap<FlowId, ImmutableList<StoredFlowEntry>> backupFlowTable = smaps.get(deviceId);
755 // Following should be rewritten using async APIs
Brian O'Connor427a1762014-11-19 18:40:32 -0800756 for (FlowRuleBatchEntry bEntry : toAdd) {
757 final FlowRule entry = bEntry.getTarget();
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700758 final FlowId id = entry.id();
759 ImmutableList<StoredFlowEntry> original = backupFlowTable.get(id);
760 List<StoredFlowEntry> list = new ArrayList<>();
761 if (original != null) {
762 list.addAll(original);
763 }
764
Brian O'Connor427a1762014-11-19 18:40:32 -0800765 list.remove(bEntry.getTarget());
766 list.add((StoredFlowEntry) entry);
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700767
768 ImmutableList<StoredFlowEntry> newValue = ImmutableList.copyOf(list);
769 boolean success;
770 if (original == null) {
771 success = (backupFlowTable.putIfAbsent(id, newValue) == null);
772 } else {
773 success = backupFlowTable.replace(id, original, newValue);
774 }
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700775 if (!success) {
776 log.error("Updating backup failed.");
777 }
778 }
Brian O'Connor427a1762014-11-19 18:40:32 -0800779 for (FlowRuleBatchEntry bEntry : toRemove) {
780 final FlowRule entry = bEntry.getTarget();
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700781 final FlowId id = entry.id();
782 ImmutableList<StoredFlowEntry> original = backupFlowTable.get(id);
783 List<StoredFlowEntry> list = new ArrayList<>();
784 if (original != null) {
785 list.addAll(original);
786 }
787
Brian O'Connor427a1762014-11-19 18:40:32 -0800788 list.remove(bEntry.getTarget());
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700789
790 ImmutableList<StoredFlowEntry> newValue = ImmutableList.copyOf(list);
791 boolean success;
792 if (original == null) {
793 success = (backupFlowTable.putIfAbsent(id, newValue) == null);
794 } else {
795 success = backupFlowTable.replace(id, original, newValue);
796 }
Yuta HIGUCHI92891d12014-10-27 20:04:38 -0700797 if (!success) {
798 log.error("Updating backup failed.");
799 }
800 }
801 } catch (ExecutionException e) {
802 log.error("Failed to write to backups", e);
803 }
804
805 }
806 }
alshabib339a3d92014-09-26 17:54:32 -0700807}