blob: afdec73c2b66ff0b99b07fc49b4b1abdbcee2ae8 [file] [log] [blame]
alshabib10580802015-02-18 18:30:33 -08001/*
Brian O'Connor5ab426f2016-04-09 01:19:45 -07002 * Copyright 2015-present Open Networking Laboratory
alshabib10580802015-02-18 18:30:33 -08003 *
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.group.impl;
17
Jonathan Hart6ec029a2015-03-24 17:12:35 -070018import com.google.common.collect.FluentIterable;
Charles Chanf4838a72015-12-07 18:13:45 -080019import com.google.common.collect.ImmutableSet;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070020import com.google.common.collect.Iterables;
Srikanth Vavilapalli23181912015-05-04 09:48:09 -070021import com.google.common.collect.Sets;
alshabib10580802015-02-18 18:30:33 -080022import org.apache.felix.scr.annotations.Activate;
23import org.apache.felix.scr.annotations.Component;
24import org.apache.felix.scr.annotations.Deactivate;
alshabibb0285992016-03-28 23:30:37 -070025import org.apache.felix.scr.annotations.Modified;
26import org.apache.felix.scr.annotations.Property;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070027import org.apache.felix.scr.annotations.Reference;
28import org.apache.felix.scr.annotations.ReferenceCardinality;
alshabib10580802015-02-18 18:30:33 -080029import org.apache.felix.scr.annotations.Service;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070030import org.onlab.util.KryoNamespace;
alshabibb0285992016-03-28 23:30:37 -070031import org.onosproject.cfg.ComponentConfigService;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070032import org.onosproject.cluster.ClusterService;
Charles Chanf4838a72015-12-07 18:13:45 -080033import org.onosproject.cluster.NodeId;
alshabib10580802015-02-18 18:30:33 -080034import org.onosproject.core.DefaultGroupId;
35import org.onosproject.core.GroupId;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070036import org.onosproject.mastership.MastershipService;
alshabib10580802015-02-18 18:30:33 -080037import org.onosproject.net.DeviceId;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070038import org.onosproject.net.MastershipRole;
alshabib10580802015-02-18 18:30:33 -080039import org.onosproject.net.group.DefaultGroup;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070040import org.onosproject.net.group.DefaultGroupBucket;
alshabib10580802015-02-18 18:30:33 -080041import org.onosproject.net.group.DefaultGroupDescription;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070042import org.onosproject.net.group.DefaultGroupKey;
alshabib10580802015-02-18 18:30:33 -080043import org.onosproject.net.group.Group;
44import org.onosproject.net.group.Group.GroupState;
45import org.onosproject.net.group.GroupBucket;
46import org.onosproject.net.group.GroupBuckets;
47import org.onosproject.net.group.GroupDescription;
48import org.onosproject.net.group.GroupEvent;
49import org.onosproject.net.group.GroupEvent.Type;
50import org.onosproject.net.group.GroupKey;
51import org.onosproject.net.group.GroupOperation;
52import org.onosproject.net.group.GroupStore;
53import org.onosproject.net.group.GroupStoreDelegate;
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -070054import org.onosproject.net.group.StoredGroupBucketEntry;
alshabib10580802015-02-18 18:30:33 -080055import org.onosproject.net.group.StoredGroupEntry;
56import org.onosproject.store.AbstractStore;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070057import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070058import org.onosproject.store.serializers.KryoNamespaces;
Madan Jampani0b847532016-03-03 13:44:15 -080059import org.onosproject.store.service.ConsistentMap;
60import org.onosproject.store.service.MapEvent;
61import org.onosproject.store.service.MapEventListener;
alshabibb0285992016-03-28 23:30:37 -070062import org.onosproject.store.service.MultiValuedTimestamp;
Madan Jampani0b847532016-03-03 13:44:15 -080063import org.onosproject.store.service.Serializer;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070064import org.onosproject.store.service.StorageService;
Madan Jampani0b847532016-03-03 13:44:15 -080065import org.onosproject.store.service.Versioned;
alshabibb0285992016-03-28 23:30:37 -070066import org.osgi.service.component.ComponentContext;
alshabib10580802015-02-18 18:30:33 -080067import org.slf4j.Logger;
68
Jonathan Hart6ec029a2015-03-24 17:12:35 -070069import java.util.ArrayList;
Srikanth Vavilapalli23181912015-05-04 09:48:09 -070070import java.util.Collection;
Charles Chanf4838a72015-12-07 18:13:45 -080071import java.util.Collections;
alshabibb0285992016-03-28 23:30:37 -070072import java.util.Dictionary;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070073import java.util.HashMap;
Charles Chan0c7c43b2016-01-14 17:39:20 -080074import java.util.HashSet;
Srikanth Vavilapalli23181912015-05-04 09:48:09 -070075import java.util.Iterator;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070076import java.util.List;
Madan Jampani0b847532016-03-03 13:44:15 -080077import java.util.Map;
Charles Chan0c7c43b2016-01-14 17:39:20 -080078import java.util.Map.Entry;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070079import java.util.Objects;
Sho SHIMIZU30d639b2015-05-05 09:30:35 -070080import java.util.Optional;
alshabibb0285992016-03-28 23:30:37 -070081import java.util.Properties;
Srikanth Vavilapalli23181912015-05-04 09:48:09 -070082import java.util.Set;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070083import java.util.concurrent.ConcurrentHashMap;
84import java.util.concurrent.ConcurrentMap;
85import java.util.concurrent.ExecutorService;
86import java.util.concurrent.Executors;
87import java.util.concurrent.atomic.AtomicInteger;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070088import java.util.stream.Collectors;
89
alshabibb0285992016-03-28 23:30:37 -070090import static com.google.common.base.Strings.isNullOrEmpty;
alshabibb0285992016-03-28 23:30:37 -070091import static org.onlab.util.Tools.get;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070092import static org.onlab.util.Tools.groupedThreads;
93import static org.slf4j.LoggerFactory.getLogger;
alshabib10580802015-02-18 18:30:33 -080094
95/**
Saurav Das0fd79d92016-03-07 10:58:36 -080096 * Manages inventory of group entries using distributed group stores from the
97 * storage service.
alshabib10580802015-02-18 18:30:33 -080098 */
99@Component(immediate = true)
100@Service
101public class DistributedGroupStore
102 extends AbstractStore<GroupEvent, GroupStoreDelegate>
103 implements GroupStore {
104
105 private final Logger log = getLogger(getClass());
106
alshabibb0285992016-03-28 23:30:37 -0700107 private static final boolean GARBAGE_COLLECT = false;
108 private static final int GC_THRESH = 6;
109
alshabib10580802015-02-18 18:30:33 -0800110 private final int dummyId = 0xffffffff;
111 private final GroupId dummyGroupId = new DefaultGroupId(dummyId);
112
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700113 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
114 protected ClusterCommunicationService clusterCommunicator;
115
116 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
117 protected ClusterService clusterService;
118
119 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700120 protected StorageService storageService;
121
122 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700123 protected MastershipService mastershipService;
124
alshabibb0285992016-03-28 23:30:37 -0700125 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
126 protected ComponentConfigService cfgService;
127
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700128 // Per device group table with (device id + app cookie) as key
Madan Jampani0b847532016-03-03 13:44:15 -0800129 private ConsistentMap<GroupStoreKeyMapKey,
alshabibb0285992016-03-28 23:30:37 -0700130 StoredGroupEntry> groupStoreEntriesByKey = null;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700131 // Per device group table with (device id + group id) as key
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700132 private final ConcurrentMap<DeviceId, ConcurrentMap<GroupId, StoredGroupEntry>>
alshabibb0285992016-03-28 23:30:37 -0700133 groupEntriesById = new ConcurrentHashMap<>();
Madan Jampani0b847532016-03-03 13:44:15 -0800134 private ConsistentMap<GroupStoreKeyMapKey,
alshabibb0285992016-03-28 23:30:37 -0700135 StoredGroupEntry> auditPendingReqQueue = null;
Frank Wange0eb5ce2016-07-01 18:21:25 +0800136 private MapEventListener<GroupStoreKeyMapKey, StoredGroupEntry>
137 mapListener = new GroupStoreKeyMapListener();
alshabib10580802015-02-18 18:30:33 -0800138 private final ConcurrentMap<DeviceId, ConcurrentMap<GroupId, Group>>
139 extraneousGroupEntriesById = new ConcurrentHashMap<>();
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700140 private ExecutorService messageHandlingExecutor;
141 private static final int MESSAGE_HANDLER_THREAD_POOL_SIZE = 1;
Sho SHIMIZU7a4087b2015-09-10 09:23:16 -0700142 private final HashMap<DeviceId, Boolean> deviceAuditStatus = new HashMap<>();
alshabib10580802015-02-18 18:30:33 -0800143
144 private final AtomicInteger groupIdGen = new AtomicInteger();
145
HIGUCHI Yuta180d70f2015-10-01 16:13:56 -0700146 private KryoNamespace clusterMsgSerializer;
147
alshabibb0285992016-03-28 23:30:37 -0700148 @Property(name = "garbageCollect", boolValue = GARBAGE_COLLECT,
149 label = "Enable group garbage collection")
150 private boolean garbageCollect = GARBAGE_COLLECT;
151
152 @Property(name = "gcThresh", intValue = GC_THRESH,
153 label = "Number of rounds for group garbage collection")
154 private int gcThresh = GC_THRESH;
155
156
alshabib10580802015-02-18 18:30:33 -0800157 @Activate
158 public void activate() {
alshabibb0285992016-03-28 23:30:37 -0700159 cfgService.registerProperties(getClass());
HIGUCHI Yuta3a84b322016-05-18 13:38:07 -0700160 KryoNamespace.Builder kryoBuilder = new KryoNamespace.Builder()
alshabibb0285992016-03-28 23:30:37 -0700161 .register(KryoNamespaces.API)
HIGUCHI Yuta3a84b322016-05-18 13:38:07 -0700162 .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
alshabibb0285992016-03-28 23:30:37 -0700163 .register(DefaultGroup.class,
164 DefaultGroupBucket.class,
165 DefaultGroupDescription.class,
166 DefaultGroupKey.class,
167 GroupDescription.Type.class,
168 Group.GroupState.class,
169 GroupBuckets.class,
alshabibb0285992016-03-28 23:30:37 -0700170 GroupStoreMessage.class,
171 GroupStoreMessage.Type.class,
172 UpdateType.class,
173 GroupStoreMessageSubjects.class,
174 MultiValuedTimestamp.class,
175 GroupStoreKeyMapKey.class,
176 GroupStoreIdMapKey.class,
177 GroupStoreMapKey.class
178 );
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700179
HIGUCHI Yuta3a84b322016-05-18 13:38:07 -0700180 clusterMsgSerializer = kryoBuilder.build("GroupStore");
181 Serializer serializer = Serializer.using(clusterMsgSerializer);
HIGUCHI Yuta180d70f2015-10-01 16:13:56 -0700182
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700183 messageHandlingExecutor = Executors.
184 newFixedThreadPool(MESSAGE_HANDLER_THREAD_POOL_SIZE,
185 groupedThreads("onos/store/group",
HIGUCHI Yutad9e01052016-04-14 09:31:42 -0700186 "message-handlers",
187 log));
Madan Jampani01e05fb2015-08-13 13:29:36 -0700188
189 clusterCommunicator.addSubscriber(GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
alshabibb0285992016-03-28 23:30:37 -0700190 clusterMsgSerializer::deserialize,
191 this::process,
192 messageHandlingExecutor);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700193
Madan Jampani0b847532016-03-03 13:44:15 -0800194 log.debug("Creating Consistent map onos-group-store-keymap");
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700195
Madan Jampani0b847532016-03-03 13:44:15 -0800196 groupStoreEntriesByKey = storageService.<GroupStoreKeyMapKey, StoredGroupEntry>consistentMapBuilder()
197 .withName("onos-group-store-keymap")
HIGUCHI Yuta3a84b322016-05-18 13:38:07 -0700198 .withSerializer(serializer)
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700199 .build();
Frank Wange0eb5ce2016-07-01 18:21:25 +0800200 groupStoreEntriesByKey.addListener(mapListener);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700201 log.debug("Current size of groupstorekeymap:{}",
202 groupStoreEntriesByKey.size());
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700203
Madan Jampani0b847532016-03-03 13:44:15 -0800204 log.debug("Creating Consistent map pendinggroupkeymap");
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700205
Madan Jampani0b847532016-03-03 13:44:15 -0800206 auditPendingReqQueue = storageService.<GroupStoreKeyMapKey, StoredGroupEntry>consistentMapBuilder()
207 .withName("onos-pending-group-keymap")
HIGUCHI Yuta3a84b322016-05-18 13:38:07 -0700208 .withSerializer(serializer)
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700209 .build();
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700210 log.debug("Current size of pendinggroupkeymap:{}",
211 auditPendingReqQueue.size());
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700212
alshabib10580802015-02-18 18:30:33 -0800213 log.info("Started");
214 }
215
216 @Deactivate
217 public void deactivate() {
Frank Wange0eb5ce2016-07-01 18:21:25 +0800218 groupStoreEntriesByKey.removeListener(mapListener);
alshabibb0285992016-03-28 23:30:37 -0700219 cfgService.unregisterProperties(getClass(), false);
HIGUCHI Yuta180d70f2015-10-01 16:13:56 -0700220 clusterCommunicator.removeSubscriber(GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST);
alshabib10580802015-02-18 18:30:33 -0800221 log.info("Stopped");
222 }
223
alshabibb0285992016-03-28 23:30:37 -0700224 @Modified
225 public void modified(ComponentContext context) {
226 Dictionary<?, ?> properties = context != null ? context.getProperties() : new Properties();
227
228 try {
229 String s = get(properties, "garbageCollect");
230 garbageCollect = isNullOrEmpty(s) ? GARBAGE_COLLECT : Boolean.parseBoolean(s.trim());
231
232 s = get(properties, "gcThresh");
233 gcThresh = isNullOrEmpty(s) ? GC_THRESH : Integer.parseInt(s.trim());
234 } catch (Exception e) {
235 gcThresh = GC_THRESH;
236 garbageCollect = GARBAGE_COLLECT;
237 }
238 }
239
alshabib10580802015-02-18 18:30:33 -0800240 /**
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700241 * Returns the group store eventual consistent key map.
alshabib10580802015-02-18 18:30:33 -0800242 *
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700243 * @return Map representing group key table.
alshabib10580802015-02-18 18:30:33 -0800244 */
Madan Jampani0b847532016-03-03 13:44:15 -0800245 private Map<GroupStoreKeyMapKey, StoredGroupEntry>
alshabibb0285992016-03-28 23:30:37 -0700246 getGroupStoreKeyMap() {
Madan Jampani0b847532016-03-03 13:44:15 -0800247 return groupStoreEntriesByKey.asJavaMap();
alshabib10580802015-02-18 18:30:33 -0800248 }
249
250 /**
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700251 * Returns the group id table for specified device.
alshabib10580802015-02-18 18:30:33 -0800252 *
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700253 * @param deviceId identifier of the device
254 * @return Map representing group key table of given device.
alshabib10580802015-02-18 18:30:33 -0800255 */
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700256 private ConcurrentMap<GroupId, StoredGroupEntry> getGroupIdTable(DeviceId deviceId) {
Yuta HIGUCHIc2e68152016-08-16 13:47:36 -0700257 return groupEntriesById.computeIfAbsent(deviceId, k -> new ConcurrentHashMap<>());
alshabib10580802015-02-18 18:30:33 -0800258 }
259
260 /**
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700261 * Returns the pending group request table.
alshabib10580802015-02-18 18:30:33 -0800262 *
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700263 * @return Map representing group key table.
alshabib10580802015-02-18 18:30:33 -0800264 */
Madan Jampani0b847532016-03-03 13:44:15 -0800265 private Map<GroupStoreKeyMapKey, StoredGroupEntry>
alshabibb0285992016-03-28 23:30:37 -0700266 getPendingGroupKeyTable() {
Madan Jampani0b847532016-03-03 13:44:15 -0800267 return auditPendingReqQueue.asJavaMap();
alshabib10580802015-02-18 18:30:33 -0800268 }
269
270 /**
271 * Returns the extraneous group id table for specified device.
272 *
273 * @param deviceId identifier of the device
274 * @return Map representing group key table of given device.
275 */
276 private ConcurrentMap<GroupId, Group>
277 getExtraneousGroupIdTable(DeviceId deviceId) {
Yuta HIGUCHIc2e68152016-08-16 13:47:36 -0700278 return extraneousGroupEntriesById.computeIfAbsent(deviceId, k -> new ConcurrentHashMap<>());
alshabib10580802015-02-18 18:30:33 -0800279 }
280
281 /**
282 * Returns the number of groups for the specified device in the store.
283 *
284 * @return number of groups for the specified device
285 */
286 @Override
287 public int getGroupCount(DeviceId deviceId) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700288 return (getGroups(deviceId) != null) ?
alshabibb0285992016-03-28 23:30:37 -0700289 Iterables.size(getGroups(deviceId)) : 0;
alshabib10580802015-02-18 18:30:33 -0800290 }
291
292 /**
293 * Returns the groups associated with a device.
294 *
295 * @param deviceId the device ID
alshabib10580802015-02-18 18:30:33 -0800296 * @return the group entries
297 */
298 @Override
299 public Iterable<Group> getGroups(DeviceId deviceId) {
Charles Chanf4838a72015-12-07 18:13:45 -0800300 // Let ImmutableSet.copyOf do the type conversion
301 return ImmutableSet.copyOf(getStoredGroups(deviceId));
alshabib10580802015-02-18 18:30:33 -0800302 }
303
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700304 private Iterable<StoredGroupEntry> getStoredGroups(DeviceId deviceId) {
Charles Chanf4838a72015-12-07 18:13:45 -0800305 NodeId master = mastershipService.getMasterFor(deviceId);
306 if (master == null) {
307 log.debug("Failed to getGroups: No master for {}", deviceId);
308 return Collections.emptySet();
309 }
310
311 Set<StoredGroupEntry> storedGroups = getGroupStoreKeyMap().values()
312 .stream()
313 .filter(input -> input.deviceId().equals(deviceId))
314 .collect(Collectors.toSet());
315 return ImmutableSet.copyOf(storedGroups);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700316 }
317
alshabib10580802015-02-18 18:30:33 -0800318 /**
319 * Returns the stored group entry.
320 *
alshabibb0285992016-03-28 23:30:37 -0700321 * @param deviceId the device ID
alshabib10580802015-02-18 18:30:33 -0800322 * @param appCookie the group key
alshabib10580802015-02-18 18:30:33 -0800323 * @return a group associated with the key
324 */
325 @Override
326 public Group getGroup(DeviceId deviceId, GroupKey appCookie) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700327 return getStoredGroupEntry(deviceId, appCookie);
328 }
329
330 private StoredGroupEntry getStoredGroupEntry(DeviceId deviceId,
331 GroupKey appCookie) {
332 return getGroupStoreKeyMap().get(new GroupStoreKeyMapKey(deviceId,
333 appCookie));
334 }
335
336 @Override
337 public Group getGroup(DeviceId deviceId, GroupId groupId) {
338 return getStoredGroupEntry(deviceId, groupId);
339 }
340
341 private StoredGroupEntry getStoredGroupEntry(DeviceId deviceId,
342 GroupId groupId) {
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700343 return getGroupIdTable(deviceId).get(groupId);
alshabib10580802015-02-18 18:30:33 -0800344 }
345
346 private int getFreeGroupIdValue(DeviceId deviceId) {
347 int freeId = groupIdGen.incrementAndGet();
348
349 while (true) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700350 Group existing = getGroup(deviceId, new DefaultGroupId(freeId));
alshabib10580802015-02-18 18:30:33 -0800351 if (existing == null) {
352 existing = (
353 extraneousGroupEntriesById.get(deviceId) != null) ?
354 extraneousGroupEntriesById.get(deviceId).
355 get(new DefaultGroupId(freeId)) :
356 null;
357 }
358 if (existing != null) {
359 freeId = groupIdGen.incrementAndGet();
360 } else {
361 break;
362 }
363 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700364 log.debug("getFreeGroupIdValue: Next Free ID is {}", freeId);
alshabib10580802015-02-18 18:30:33 -0800365 return freeId;
366 }
367
368 /**
369 * Stores a new group entry using the information from group description.
370 *
371 * @param groupDesc group description to be used to create group entry
372 */
373 @Override
374 public void storeGroupDescription(GroupDescription groupDesc) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700375 log.debug("In storeGroupDescription");
alshabib10580802015-02-18 18:30:33 -0800376 // Check if a group is existing with the same key
Saurav Das8a0732e2015-11-20 15:27:53 -0800377 Group existingGroup = getGroup(groupDesc.deviceId(), groupDesc.appCookie());
378 if (existingGroup != null) {
Charles Chan216e3c82016-04-23 14:48:16 -0700379 log.info("Group already exists with the same key {} in dev:{} with id:0x{}",
Saurav Das8a0732e2015-11-20 15:27:53 -0800380 groupDesc.appCookie(), groupDesc.deviceId(),
381 Integer.toHexString(existingGroup.id().id()));
alshabib10580802015-02-18 18:30:33 -0800382 return;
383 }
384
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700385 // Check if group to be created by a remote instance
Madan Jampani175e8fd2015-05-20 14:10:45 -0700386 if (mastershipService.getLocalRole(groupDesc.deviceId()) != MastershipRole.MASTER) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700387 log.debug("storeGroupDescription: Device {} local role is not MASTER",
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700388 groupDesc.deviceId());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700389 if (mastershipService.getMasterFor(groupDesc.deviceId()) == null) {
390 log.error("No Master for device {}..."
alshabibb0285992016-03-28 23:30:37 -0700391 + "Can not perform add group operation",
392 groupDesc.deviceId());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700393 //TODO: Send Group operation failure event
394 return;
395 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700396 GroupStoreMessage groupOp = GroupStoreMessage.
397 createGroupAddRequestMsg(groupDesc.deviceId(),
398 groupDesc);
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700399
Madan Jampani175e8fd2015-05-20 14:10:45 -0700400 clusterCommunicator.unicast(groupOp,
alshabibb0285992016-03-28 23:30:37 -0700401 GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
402 clusterMsgSerializer::serialize,
403 mastershipService.getMasterFor(groupDesc.deviceId()))
404 .whenComplete((result, error) -> {
Madan Jampani175e8fd2015-05-20 14:10:45 -0700405 if (error != null) {
406 log.warn("Failed to send request to master: {} to {}",
alshabibb0285992016-03-28 23:30:37 -0700407 groupOp,
408 mastershipService.getMasterFor(groupDesc.deviceId()));
Madan Jampani175e8fd2015-05-20 14:10:45 -0700409 //TODO: Send Group operation failure event
410 } else {
411 log.debug("Sent Group operation request for device {} "
alshabibb0285992016-03-28 23:30:37 -0700412 + "to remote MASTER {}",
413 groupDesc.deviceId(),
414 mastershipService.getMasterFor(groupDesc.deviceId()));
Madan Jampani175e8fd2015-05-20 14:10:45 -0700415 }
416 });
alshabib10580802015-02-18 18:30:33 -0800417 return;
418 }
419
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700420 log.debug("Store group for device {} is getting handled locally",
421 groupDesc.deviceId());
alshabib10580802015-02-18 18:30:33 -0800422 storeGroupDescriptionInternal(groupDesc);
423 }
424
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700425 private Group getMatchingExtraneousGroupbyId(DeviceId deviceId, Integer groupId) {
426 ConcurrentMap<GroupId, Group> extraneousMap =
427 extraneousGroupEntriesById.get(deviceId);
428 if (extraneousMap == null) {
429 return null;
430 }
431 return extraneousMap.get(new DefaultGroupId(groupId));
432 }
433
434 private Group getMatchingExtraneousGroupbyBuckets(DeviceId deviceId,
435 GroupBuckets buckets) {
436 ConcurrentMap<GroupId, Group> extraneousMap =
437 extraneousGroupEntriesById.get(deviceId);
438 if (extraneousMap == null) {
439 return null;
440 }
441
alshabibb0285992016-03-28 23:30:37 -0700442 for (Group extraneousGroup : extraneousMap.values()) {
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700443 if (extraneousGroup.buckets().equals(buckets)) {
444 return extraneousGroup;
445 }
446 }
447 return null;
448 }
449
alshabib10580802015-02-18 18:30:33 -0800450 private void storeGroupDescriptionInternal(GroupDescription groupDesc) {
451 // Check if a group is existing with the same key
452 if (getGroup(groupDesc.deviceId(), groupDesc.appCookie()) != null) {
453 return;
454 }
455
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700456 if (deviceAuditStatus.get(groupDesc.deviceId()) == null) {
457 // Device group audit has not completed yet
458 // Add this group description to pending group key table
459 // Create a group entry object with Dummy Group ID
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700460 log.debug("storeGroupDescriptionInternal: Device {} AUDIT pending...Queuing Group ADD request",
alshabibb0285992016-03-28 23:30:37 -0700461 groupDesc.deviceId());
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700462 StoredGroupEntry group = new DefaultGroup(dummyGroupId, groupDesc);
463 group.setState(GroupState.WAITING_AUDIT_COMPLETE);
Madan Jampani0b847532016-03-03 13:44:15 -0800464 Map<GroupStoreKeyMapKey, StoredGroupEntry> pendingKeyTable =
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700465 getPendingGroupKeyTable();
466 pendingKeyTable.put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
467 groupDesc.appCookie()),
468 group);
469 return;
470 }
471
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700472 Group matchingExtraneousGroup = null;
473 if (groupDesc.givenGroupId() != null) {
474 //Check if there is a extraneous group existing with the same Id
475 matchingExtraneousGroup = getMatchingExtraneousGroupbyId(
alshabibb0285992016-03-28 23:30:37 -0700476 groupDesc.deviceId(), groupDesc.givenGroupId());
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700477 if (matchingExtraneousGroup != null) {
Saurav Das0fd79d92016-03-07 10:58:36 -0800478 log.debug("storeGroupDescriptionInternal: Matching extraneous group "
alshabibb0285992016-03-28 23:30:37 -0700479 + "found in Device {} for group id 0x{}",
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700480 groupDesc.deviceId(),
Saurav Das0fd79d92016-03-07 10:58:36 -0800481 Integer.toHexString(groupDesc.givenGroupId()));
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700482 //Check if the group buckets matches with user provided buckets
483 if (matchingExtraneousGroup.buckets().equals(groupDesc.buckets())) {
484 //Group is already existing with the same buckets and Id
485 // Create a group entry object
Saurav Das0fd79d92016-03-07 10:58:36 -0800486 log.debug("storeGroupDescriptionInternal: Buckets also matching "
alshabibb0285992016-03-28 23:30:37 -0700487 + "in Device {} for group id 0x{}",
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700488 groupDesc.deviceId(),
Saurav Das0fd79d92016-03-07 10:58:36 -0800489 Integer.toHexString(groupDesc.givenGroupId()));
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700490 StoredGroupEntry group = new DefaultGroup(
alshabibb0285992016-03-28 23:30:37 -0700491 matchingExtraneousGroup.id(), groupDesc);
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700492 // Insert the newly created group entry into key and id maps
493 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -0700494 put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
495 groupDesc.appCookie()), group);
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700496 // Ensure it also inserted into group id based table to
497 // avoid any chances of duplication in group id generation
498 getGroupIdTable(groupDesc.deviceId()).
alshabibb0285992016-03-28 23:30:37 -0700499 put(matchingExtraneousGroup.id(), group);
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700500 addOrUpdateGroupEntry(matchingExtraneousGroup);
501 removeExtraneousGroupEntry(matchingExtraneousGroup);
502 return;
503 } else {
504 //Group buckets are not matching. Update group
505 //with user provided buckets.
Saurav Das0fd79d92016-03-07 10:58:36 -0800506 log.debug("storeGroupDescriptionInternal: Buckets are not "
alshabibb0285992016-03-28 23:30:37 -0700507 + "matching in Device {} for group id 0x{}",
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700508 groupDesc.deviceId(),
Saurav Das0fd79d92016-03-07 10:58:36 -0800509 Integer.toHexString(groupDesc.givenGroupId()));
510 StoredGroupEntry modifiedGroup = new DefaultGroup(
alshabibb0285992016-03-28 23:30:37 -0700511 matchingExtraneousGroup.id(), groupDesc);
Saurav Das0fd79d92016-03-07 10:58:36 -0800512 modifiedGroup.setState(GroupState.PENDING_UPDATE);
513 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -0700514 put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
515 groupDesc.appCookie()), modifiedGroup);
Saurav Das0fd79d92016-03-07 10:58:36 -0800516 // Ensure it also inserted into group id based table to
517 // avoid any chances of duplication in group id generation
518 getGroupIdTable(groupDesc.deviceId()).
alshabibb0285992016-03-28 23:30:37 -0700519 put(matchingExtraneousGroup.id(), modifiedGroup);
Saurav Das0fd79d92016-03-07 10:58:36 -0800520 removeExtraneousGroupEntry(matchingExtraneousGroup);
521 log.debug("storeGroupDescriptionInternal: Triggering Group "
alshabibb0285992016-03-28 23:30:37 -0700522 + "UPDATE request for {} in device {}",
Saurav Das0fd79d92016-03-07 10:58:36 -0800523 matchingExtraneousGroup.id(),
524 groupDesc.deviceId());
525 notifyDelegate(new GroupEvent(Type.GROUP_UPDATE_REQUESTED, modifiedGroup));
526 return;
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700527 }
528 }
529 } else {
530 //Check if there is an extraneous group with user provided buckets
531 matchingExtraneousGroup = getMatchingExtraneousGroupbyBuckets(
alshabibb0285992016-03-28 23:30:37 -0700532 groupDesc.deviceId(), groupDesc.buckets());
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700533 if (matchingExtraneousGroup != null) {
534 //Group is already existing with the same buckets.
535 //So reuse this group.
536 log.debug("storeGroupDescriptionInternal: Matching extraneous group found in Device {}",
537 groupDesc.deviceId());
538 //Create a group entry object
539 StoredGroupEntry group = new DefaultGroup(
alshabibb0285992016-03-28 23:30:37 -0700540 matchingExtraneousGroup.id(), groupDesc);
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700541 // Insert the newly created group entry into key and id maps
542 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -0700543 put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
544 groupDesc.appCookie()), group);
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700545 // Ensure it also inserted into group id based table to
546 // avoid any chances of duplication in group id generation
547 getGroupIdTable(groupDesc.deviceId()).
alshabibb0285992016-03-28 23:30:37 -0700548 put(matchingExtraneousGroup.id(), group);
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700549 addOrUpdateGroupEntry(matchingExtraneousGroup);
550 removeExtraneousGroupEntry(matchingExtraneousGroup);
551 return;
552 } else {
553 //TODO: Check if there are any empty groups that can be used here
554 log.debug("storeGroupDescriptionInternal: No matching extraneous groups found in Device {}",
555 groupDesc.deviceId());
556 }
557 }
558
Saurav Das100e3b82015-04-30 11:12:10 -0700559 GroupId id = null;
560 if (groupDesc.givenGroupId() == null) {
561 // Get a new group identifier
562 id = new DefaultGroupId(getFreeGroupIdValue(groupDesc.deviceId()));
563 } else {
Saurav Das8be4e3a2016-03-11 17:19:07 -0800564 // we need to use the identifier passed in by caller, but check if
565 // already used
566 Group existing = getGroup(groupDesc.deviceId(),
567 new DefaultGroupId(groupDesc.givenGroupId()));
568 if (existing != null) {
569 log.warn("Group already exists with the same id: 0x{} in dev:{} "
alshabibb0285992016-03-28 23:30:37 -0700570 + "but with different key: {} (request gkey: {})",
571 Integer.toHexString(groupDesc.givenGroupId()),
572 groupDesc.deviceId(),
573 existing.appCookie(),
574 groupDesc.appCookie());
Saurav Das8be4e3a2016-03-11 17:19:07 -0800575 return;
576 }
Saurav Das100e3b82015-04-30 11:12:10 -0700577 id = new DefaultGroupId(groupDesc.givenGroupId());
578 }
alshabib10580802015-02-18 18:30:33 -0800579 // Create a group entry object
580 StoredGroupEntry group = new DefaultGroup(id, groupDesc);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700581 // Insert the newly created group entry into key and id maps
582 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -0700583 put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
584 groupDesc.appCookie()), group);
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700585 // Ensure it also inserted into group id based table to
586 // avoid any chances of duplication in group id generation
587 getGroupIdTable(groupDesc.deviceId()).
alshabibb0285992016-03-28 23:30:37 -0700588 put(id, group);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700589 log.debug("storeGroupDescriptionInternal: Processing Group ADD request for Id {} in device {}",
alshabibb0285992016-03-28 23:30:37 -0700590 id,
591 groupDesc.deviceId());
alshabib10580802015-02-18 18:30:33 -0800592 notifyDelegate(new GroupEvent(GroupEvent.Type.GROUP_ADD_REQUESTED,
593 group));
594 }
595
596 /**
597 * Updates the existing group entry with the information
598 * from group description.
599 *
alshabibb0285992016-03-28 23:30:37 -0700600 * @param deviceId the device ID
alshabib10580802015-02-18 18:30:33 -0800601 * @param oldAppCookie the current group key
alshabibb0285992016-03-28 23:30:37 -0700602 * @param type update type
603 * @param newBuckets group buckets for updates
alshabib10580802015-02-18 18:30:33 -0800604 * @param newAppCookie optional new group key
605 */
606 @Override
607 public void updateGroupDescription(DeviceId deviceId,
608 GroupKey oldAppCookie,
609 UpdateType type,
610 GroupBuckets newBuckets,
611 GroupKey newAppCookie) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700612 // Check if group update to be done by a remote instance
sangho52abe3a2015-05-05 14:13:34 -0700613 if (mastershipService.getMasterFor(deviceId) != null &&
614 mastershipService.getLocalRole(deviceId) != MastershipRole.MASTER) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700615 log.debug("updateGroupDescription: Device {} local role is not MASTER",
616 deviceId);
617 if (mastershipService.getMasterFor(deviceId) == null) {
618 log.error("No Master for device {}..."
alshabibb0285992016-03-28 23:30:37 -0700619 + "Can not perform update group operation",
620 deviceId);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700621 //TODO: Send Group operation failure event
622 return;
623 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700624 GroupStoreMessage groupOp = GroupStoreMessage.
625 createGroupUpdateRequestMsg(deviceId,
626 oldAppCookie,
627 type,
628 newBuckets,
629 newAppCookie);
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700630
Madan Jampani175e8fd2015-05-20 14:10:45 -0700631 clusterCommunicator.unicast(groupOp,
alshabibb0285992016-03-28 23:30:37 -0700632 GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
633 clusterMsgSerializer::serialize,
634 mastershipService.getMasterFor(deviceId)).whenComplete((result, error) -> {
635 if (error != null) {
636 log.warn("Failed to send request to master: {} to {}",
637 groupOp,
638 mastershipService.getMasterFor(deviceId), error);
639 }
640 //TODO: Send Group operation failure event
641 });
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700642 return;
643 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700644 log.debug("updateGroupDescription for device {} is getting handled locally",
645 deviceId);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700646 updateGroupDescriptionInternal(deviceId,
647 oldAppCookie,
648 type,
649 newBuckets,
650 newAppCookie);
651 }
652
653 private void updateGroupDescriptionInternal(DeviceId deviceId,
alshabibb0285992016-03-28 23:30:37 -0700654 GroupKey oldAppCookie,
655 UpdateType type,
656 GroupBuckets newBuckets,
657 GroupKey newAppCookie) {
alshabib10580802015-02-18 18:30:33 -0800658 // Check if a group is existing with the provided key
659 Group oldGroup = getGroup(deviceId, oldAppCookie);
660 if (oldGroup == null) {
Saurav Das8be4e3a2016-03-11 17:19:07 -0800661 log.warn("updateGroupDescriptionInternal: Group not found...strange. "
alshabibb0285992016-03-28 23:30:37 -0700662 + "GroupKey:{} DeviceId:{}", oldAppCookie, deviceId);
alshabib10580802015-02-18 18:30:33 -0800663 return;
664 }
665
666 List<GroupBucket> newBucketList = getUpdatedBucketList(oldGroup,
667 type,
668 newBuckets);
669 if (newBucketList != null) {
670 // Create a new group object from the old group
671 GroupBuckets updatedBuckets = new GroupBuckets(newBucketList);
672 GroupKey newCookie = (newAppCookie != null) ? newAppCookie : oldAppCookie;
673 GroupDescription updatedGroupDesc = new DefaultGroupDescription(
674 oldGroup.deviceId(),
675 oldGroup.type(),
676 updatedBuckets,
677 newCookie,
Saurav Das100e3b82015-04-30 11:12:10 -0700678 oldGroup.givenGroupId(),
alshabib10580802015-02-18 18:30:33 -0800679 oldGroup.appId());
680 StoredGroupEntry newGroup = new DefaultGroup(oldGroup.id(),
681 updatedGroupDesc);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700682 log.debug("updateGroupDescriptionInternal: group entry {} in device {} moving from {} to PENDING_UPDATE",
alshabibb0285992016-03-28 23:30:37 -0700683 oldGroup.id(),
684 oldGroup.deviceId(),
685 oldGroup.state());
alshabib10580802015-02-18 18:30:33 -0800686 newGroup.setState(GroupState.PENDING_UPDATE);
687 newGroup.setLife(oldGroup.life());
688 newGroup.setPackets(oldGroup.packets());
689 newGroup.setBytes(oldGroup.bytes());
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700690 //Update the group entry in groupkey based map.
691 //Update to groupid based map will happen in the
692 //groupkey based map update listener
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700693 log.debug("updateGroupDescriptionInternal with type {}: Group updated with buckets",
694 type);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700695 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -0700696 put(new GroupStoreKeyMapKey(newGroup.deviceId(),
697 newGroup.appCookie()), newGroup);
alshabib10580802015-02-18 18:30:33 -0800698 notifyDelegate(new GroupEvent(Type.GROUP_UPDATE_REQUESTED, newGroup));
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700699 } else {
700 log.warn("updateGroupDescriptionInternal with type {}: No "
alshabibb0285992016-03-28 23:30:37 -0700701 + "change in the buckets in update", type);
alshabib10580802015-02-18 18:30:33 -0800702 }
703 }
704
705 private List<GroupBucket> getUpdatedBucketList(Group oldGroup,
706 UpdateType type,
707 GroupBuckets buckets) {
708 GroupBuckets oldBuckets = oldGroup.buckets();
Sho SHIMIZU7a4087b2015-09-10 09:23:16 -0700709 List<GroupBucket> newBucketList = new ArrayList<>(oldBuckets.buckets());
alshabib10580802015-02-18 18:30:33 -0800710 boolean groupDescUpdated = false;
711
712 if (type == UpdateType.ADD) {
713 // Check if the any of the new buckets are part of
714 // the old bucket list
alshabibb0285992016-03-28 23:30:37 -0700715 for (GroupBucket addBucket : buckets.buckets()) {
alshabib10580802015-02-18 18:30:33 -0800716 if (!newBucketList.contains(addBucket)) {
717 newBucketList.add(addBucket);
718 groupDescUpdated = true;
719 }
720 }
721 } else if (type == UpdateType.REMOVE) {
722 // Check if the to be removed buckets are part of the
723 // old bucket list
alshabibb0285992016-03-28 23:30:37 -0700724 for (GroupBucket removeBucket : buckets.buckets()) {
alshabib10580802015-02-18 18:30:33 -0800725 if (newBucketList.contains(removeBucket)) {
726 newBucketList.remove(removeBucket);
727 groupDescUpdated = true;
728 }
729 }
730 }
731
732 if (groupDescUpdated) {
733 return newBucketList;
734 } else {
735 return null;
736 }
737 }
738
739 /**
740 * Triggers deleting the existing group entry.
741 *
alshabibb0285992016-03-28 23:30:37 -0700742 * @param deviceId the device ID
alshabib10580802015-02-18 18:30:33 -0800743 * @param appCookie the group key
744 */
745 @Override
746 public void deleteGroupDescription(DeviceId deviceId,
747 GroupKey appCookie) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700748 // Check if group to be deleted by a remote instance
749 if (mastershipService.
750 getLocalRole(deviceId) != MastershipRole.MASTER) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700751 log.debug("deleteGroupDescription: Device {} local role is not MASTER",
752 deviceId);
753 if (mastershipService.getMasterFor(deviceId) == null) {
754 log.error("No Master for device {}..."
alshabibb0285992016-03-28 23:30:37 -0700755 + "Can not perform delete group operation",
756 deviceId);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700757 //TODO: Send Group operation failure event
758 return;
759 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700760 GroupStoreMessage groupOp = GroupStoreMessage.
761 createGroupDeleteRequestMsg(deviceId,
762 appCookie);
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700763
Madan Jampani175e8fd2015-05-20 14:10:45 -0700764 clusterCommunicator.unicast(groupOp,
alshabibb0285992016-03-28 23:30:37 -0700765 GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
766 clusterMsgSerializer::serialize,
767 mastershipService.getMasterFor(deviceId)).whenComplete((result, error) -> {
768 if (error != null) {
769 log.warn("Failed to send request to master: {} to {}",
770 groupOp,
771 mastershipService.getMasterFor(deviceId), error);
772 }
773 //TODO: Send Group operation failure event
774 });
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700775 return;
776 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700777 log.debug("deleteGroupDescription in device {} is getting handled locally",
778 deviceId);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700779 deleteGroupDescriptionInternal(deviceId, appCookie);
780 }
781
782 private void deleteGroupDescriptionInternal(DeviceId deviceId,
783 GroupKey appCookie) {
alshabib10580802015-02-18 18:30:33 -0800784 // Check if a group is existing with the provided key
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700785 StoredGroupEntry existing = getStoredGroupEntry(deviceId, appCookie);
alshabib10580802015-02-18 18:30:33 -0800786 if (existing == null) {
787 return;
788 }
789
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700790 log.debug("deleteGroupDescriptionInternal: group entry {} in device {} moving from {} to PENDING_DELETE",
alshabibb0285992016-03-28 23:30:37 -0700791 existing.id(),
792 existing.deviceId(),
793 existing.state());
alshabib10580802015-02-18 18:30:33 -0800794 synchronized (existing) {
795 existing.setState(GroupState.PENDING_DELETE);
Saurav Das80980c72016-03-23 11:22:49 -0700796 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -0700797 put(new GroupStoreKeyMapKey(existing.deviceId(), existing.appCookie()),
798 existing);
alshabib10580802015-02-18 18:30:33 -0800799 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700800 log.debug("deleteGroupDescriptionInternal: in device {} issuing GROUP_REMOVE_REQUESTED",
801 deviceId);
alshabib10580802015-02-18 18:30:33 -0800802 notifyDelegate(new GroupEvent(Type.GROUP_REMOVE_REQUESTED, existing));
803 }
804
805 /**
806 * Stores a new group entry, or updates an existing entry.
807 *
808 * @param group group entry
809 */
810 @Override
811 public void addOrUpdateGroupEntry(Group group) {
812 // check if this new entry is an update to an existing entry
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700813 StoredGroupEntry existing = getStoredGroupEntry(group.deviceId(),
814 group.id());
alshabib10580802015-02-18 18:30:33 -0800815 GroupEvent event = null;
816
817 if (existing != null) {
Saurav Das0fd79d92016-03-07 10:58:36 -0800818 log.trace("addOrUpdateGroupEntry: updating group entry {} in device {}",
alshabibb0285992016-03-28 23:30:37 -0700819 group.id(),
820 group.deviceId());
alshabib10580802015-02-18 18:30:33 -0800821 synchronized (existing) {
alshabibb0285992016-03-28 23:30:37 -0700822 for (GroupBucket bucket : group.buckets().buckets()) {
Sho SHIMIZU30d639b2015-05-05 09:30:35 -0700823 Optional<GroupBucket> matchingBucket =
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -0700824 existing.buckets().buckets()
alshabibb0285992016-03-28 23:30:37 -0700825 .stream()
826 .filter((existingBucket) -> (existingBucket.equals(bucket)))
827 .findFirst();
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -0700828 if (matchingBucket.isPresent()) {
829 ((StoredGroupBucketEntry) matchingBucket.
830 get()).setPackets(bucket.packets());
831 ((StoredGroupBucketEntry) matchingBucket.
832 get()).setBytes(bucket.bytes());
833 } else {
834 log.warn("addOrUpdateGroupEntry: No matching "
alshabibb0285992016-03-28 23:30:37 -0700835 + "buckets to update stats");
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -0700836 }
837 }
alshabib10580802015-02-18 18:30:33 -0800838 existing.setLife(group.life());
839 existing.setPackets(group.packets());
840 existing.setBytes(group.bytes());
alshabibb0285992016-03-28 23:30:37 -0700841 existing.setReferenceCount(group.referenceCount());
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -0700842 if ((existing.state() == GroupState.PENDING_ADD) ||
alshabibb0285992016-03-28 23:30:37 -0700843 (existing.state() == GroupState.PENDING_ADD_RETRY)) {
Saurav Das0fd79d92016-03-07 10:58:36 -0800844 log.trace("addOrUpdateGroupEntry: group entry {} in device {} moving from {} to ADDED",
alshabibb0285992016-03-28 23:30:37 -0700845 existing.id(),
846 existing.deviceId(),
847 existing.state());
alshabib10580802015-02-18 18:30:33 -0800848 existing.setState(GroupState.ADDED);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700849 existing.setIsGroupStateAddedFirstTime(true);
alshabib10580802015-02-18 18:30:33 -0800850 event = new GroupEvent(Type.GROUP_ADDED, existing);
851 } else {
Saurav Das0fd79d92016-03-07 10:58:36 -0800852 log.trace("addOrUpdateGroupEntry: group entry {} in device {} moving from {} to ADDED",
alshabibb0285992016-03-28 23:30:37 -0700853 existing.id(),
854 existing.deviceId(),
855 GroupState.PENDING_UPDATE);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700856 existing.setState(GroupState.ADDED);
857 existing.setIsGroupStateAddedFirstTime(false);
alshabib10580802015-02-18 18:30:33 -0800858 event = new GroupEvent(Type.GROUP_UPDATED, existing);
859 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700860 //Re-PUT map entries to trigger map update events
861 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -0700862 put(new GroupStoreKeyMapKey(existing.deviceId(),
863 existing.appCookie()), existing);
alshabib10580802015-02-18 18:30:33 -0800864 }
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700865 } else {
866 log.warn("addOrUpdateGroupEntry: Group update "
alshabibb0285992016-03-28 23:30:37 -0700867 + "happening for a non-existing entry in the map");
alshabib10580802015-02-18 18:30:33 -0800868 }
869
870 if (event != null) {
871 notifyDelegate(event);
872 }
873 }
874
875 /**
876 * Removes the group entry from store.
877 *
878 * @param group group entry
879 */
880 @Override
881 public void removeGroupEntry(Group group) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700882 StoredGroupEntry existing = getStoredGroupEntry(group.deviceId(),
883 group.id());
alshabib10580802015-02-18 18:30:33 -0800884
885 if (existing != null) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700886 log.debug("removeGroupEntry: removing group entry {} in device {}",
alshabibb0285992016-03-28 23:30:37 -0700887 group.id(),
888 group.deviceId());
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700889 //Removal from groupid based map will happen in the
890 //map update listener
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700891 getGroupStoreKeyMap().remove(new GroupStoreKeyMapKey(existing.deviceId(),
892 existing.appCookie()));
alshabib10580802015-02-18 18:30:33 -0800893 notifyDelegate(new GroupEvent(Type.GROUP_REMOVED, existing));
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700894 } else {
895 log.warn("removeGroupEntry for {} in device{} is "
alshabibb0285992016-03-28 23:30:37 -0700896 + "not existing in our maps",
897 group.id(),
898 group.deviceId());
alshabib10580802015-02-18 18:30:33 -0800899 }
900 }
901
902 @Override
Charles Chan0c7c43b2016-01-14 17:39:20 -0800903 public void purgeGroupEntry(DeviceId deviceId) {
904 Set<Entry<GroupStoreKeyMapKey, StoredGroupEntry>> entryPendingRemove =
905 new HashSet<>();
906
Madan Jampani0b847532016-03-03 13:44:15 -0800907 getGroupStoreKeyMap().entrySet().stream()
Charles Chan0c7c43b2016-01-14 17:39:20 -0800908 .filter(entry -> entry.getKey().deviceId().equals(deviceId))
909 .forEach(entryPendingRemove::add);
910
911 entryPendingRemove.forEach(entry -> {
912 groupStoreEntriesByKey.remove(entry.getKey());
913 notifyDelegate(new GroupEvent(Type.GROUP_REMOVED, entry.getValue()));
914 });
915 }
916
917 @Override
alshabib10580802015-02-18 18:30:33 -0800918 public void deviceInitialAuditCompleted(DeviceId deviceId,
919 boolean completed) {
920 synchronized (deviceAuditStatus) {
921 if (completed) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700922 log.debug("AUDIT completed for device {}",
923 deviceId);
alshabib10580802015-02-18 18:30:33 -0800924 deviceAuditStatus.put(deviceId, true);
925 // Execute all pending group requests
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700926 List<StoredGroupEntry> pendingGroupRequests =
927 getPendingGroupKeyTable().values()
alshabibb0285992016-03-28 23:30:37 -0700928 .stream()
929 .filter(g -> g.deviceId().equals(deviceId))
930 .collect(Collectors.toList());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700931 log.debug("processing pending group add requests for device {} and number of pending requests {}",
alshabibb0285992016-03-28 23:30:37 -0700932 deviceId,
933 pendingGroupRequests.size());
934 for (Group group : pendingGroupRequests) {
alshabib10580802015-02-18 18:30:33 -0800935 GroupDescription tmp = new DefaultGroupDescription(
936 group.deviceId(),
937 group.type(),
938 group.buckets(),
939 group.appCookie(),
Saurav Das100e3b82015-04-30 11:12:10 -0700940 group.givenGroupId(),
alshabib10580802015-02-18 18:30:33 -0800941 group.appId());
942 storeGroupDescriptionInternal(tmp);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700943 getPendingGroupKeyTable().
alshabibb0285992016-03-28 23:30:37 -0700944 remove(new GroupStoreKeyMapKey(deviceId, group.appCookie()));
alshabib10580802015-02-18 18:30:33 -0800945 }
alshabib10580802015-02-18 18:30:33 -0800946 } else {
Thomas Vachuskac40d4632015-04-09 16:55:03 -0700947 Boolean audited = deviceAuditStatus.get(deviceId);
948 if (audited != null && audited) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700949 log.debug("Clearing AUDIT status for device {}", deviceId);
alshabib10580802015-02-18 18:30:33 -0800950 deviceAuditStatus.put(deviceId, false);
951 }
952 }
953 }
954 }
955
956 @Override
957 public boolean deviceInitialAuditStatus(DeviceId deviceId) {
958 synchronized (deviceAuditStatus) {
Thomas Vachuskac40d4632015-04-09 16:55:03 -0700959 Boolean audited = deviceAuditStatus.get(deviceId);
960 return audited != null && audited;
alshabib10580802015-02-18 18:30:33 -0800961 }
962 }
963
964 @Override
965 public void groupOperationFailed(DeviceId deviceId, GroupOperation operation) {
966
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700967 StoredGroupEntry existing = getStoredGroupEntry(deviceId,
968 operation.groupId());
alshabib10580802015-02-18 18:30:33 -0800969
970 if (existing == null) {
971 log.warn("No group entry with ID {} found ", operation.groupId());
972 return;
973 }
974
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700975 log.warn("groupOperationFailed: group operation {} failed"
alshabibb0285992016-03-28 23:30:37 -0700976 + "for group {} in device {} with code {}",
977 operation.opType(),
978 existing.id(),
979 existing.deviceId(),
980 operation.failureCode());
Saurav Das0fd79d92016-03-07 10:58:36 -0800981 if (operation.failureCode() == GroupOperation.GroupMsgErrorCode.GROUP_EXISTS) {
982 log.warn("Current extraneous groups in device:{} are: {}",
983 deviceId,
984 getExtraneousGroups(deviceId));
Saurav Das8be4e3a2016-03-11 17:19:07 -0800985 if (operation.buckets().equals(existing.buckets())) {
986 if (existing.state() == GroupState.PENDING_ADD) {
987 log.info("GROUP_EXISTS: GroupID and Buckets match for group in pending "
alshabibb0285992016-03-28 23:30:37 -0700988 + "add state - moving to ADDED for group {} in device {}",
989 existing.id(), deviceId);
Saurav Das8be4e3a2016-03-11 17:19:07 -0800990 addOrUpdateGroupEntry(existing);
991 return;
992 } else {
993 log.warn("GROUP EXISTS: Group ID matched but buckets did not. "
alshabibb0285992016-03-28 23:30:37 -0700994 + "Operation: {} Existing: {}", operation.buckets(),
995 existing.buckets());
Saurav Das8be4e3a2016-03-11 17:19:07 -0800996 }
997 }
Saurav Das0fd79d92016-03-07 10:58:36 -0800998 }
alshabib10580802015-02-18 18:30:33 -0800999 switch (operation.opType()) {
1000 case ADD:
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -07001001 if (existing.state() == GroupState.PENDING_ADD) {
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -07001002 notifyDelegate(new GroupEvent(Type.GROUP_ADD_FAILED, existing));
1003 log.warn("groupOperationFailed: cleaningup "
alshabibb0285992016-03-28 23:30:37 -07001004 + "group {} from store in device {}....",
1005 existing.id(),
1006 existing.deviceId());
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -07001007 //Removal from groupid based map will happen in the
1008 //map update listener
1009 getGroupStoreKeyMap().remove(new GroupStoreKeyMapKey(existing.deviceId(),
1010 existing.appCookie()));
1011 }
alshabib10580802015-02-18 18:30:33 -08001012 break;
1013 case MODIFY:
1014 notifyDelegate(new GroupEvent(Type.GROUP_UPDATE_FAILED, existing));
1015 break;
1016 case DELETE:
1017 notifyDelegate(new GroupEvent(Type.GROUP_REMOVE_FAILED, existing));
1018 break;
1019 default:
1020 log.warn("Unknown group operation type {}", operation.opType());
1021 }
alshabib10580802015-02-18 18:30:33 -08001022 }
1023
1024 @Override
1025 public void addOrUpdateExtraneousGroupEntry(Group group) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001026 log.debug("add/update extraneous group entry {} in device {}",
alshabibb0285992016-03-28 23:30:37 -07001027 group.id(),
1028 group.deviceId());
alshabib10580802015-02-18 18:30:33 -08001029 ConcurrentMap<GroupId, Group> extraneousIdTable =
1030 getExtraneousGroupIdTable(group.deviceId());
1031 extraneousIdTable.put(group.id(), group);
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -07001032 // Don't remove the extraneous groups, instead re-use it when
1033 // a group request comes with the same set of buckets
alshabib10580802015-02-18 18:30:33 -08001034 }
1035
1036 @Override
1037 public void removeExtraneousGroupEntry(Group group) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001038 log.debug("remove extraneous group entry {} of device {} from store",
alshabibb0285992016-03-28 23:30:37 -07001039 group.id(),
1040 group.deviceId());
alshabib10580802015-02-18 18:30:33 -08001041 ConcurrentMap<GroupId, Group> extraneousIdTable =
1042 getExtraneousGroupIdTable(group.deviceId());
1043 extraneousIdTable.remove(group.id());
1044 }
1045
1046 @Override
1047 public Iterable<Group> getExtraneousGroups(DeviceId deviceId) {
1048 // flatten and make iterator unmodifiable
1049 return FluentIterable.from(
1050 getExtraneousGroupIdTable(deviceId).values());
1051 }
1052
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001053 /**
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -07001054 * Map handler to receive any events when the group key map is updated.
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001055 */
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -07001056 private class GroupStoreKeyMapListener implements
Madan Jampani0b847532016-03-03 13:44:15 -08001057 MapEventListener<GroupStoreKeyMapKey, StoredGroupEntry> {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001058
1059 @Override
Madan Jampani0b847532016-03-03 13:44:15 -08001060 public void event(MapEvent<GroupStoreKeyMapKey, StoredGroupEntry> mapEvent) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001061 GroupEvent groupEvent = null;
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001062 GroupStoreKeyMapKey key = mapEvent.key();
Madan Jampani0b847532016-03-03 13:44:15 -08001063 StoredGroupEntry group = Versioned.valueOrNull(mapEvent.newValue());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001064 if ((key == null) && (group == null)) {
1065 log.error("GroupStoreKeyMapListener: Received "
alshabibb0285992016-03-28 23:30:37 -07001066 + "event {} with null entry", mapEvent.type());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001067 return;
1068 } else if (group == null) {
1069 group = getGroupIdTable(key.deviceId()).values()
1070 .stream()
1071 .filter((storedGroup) -> (storedGroup.appCookie().equals(key.appCookie)))
1072 .findFirst().get();
1073 if (group == null) {
1074 log.error("GroupStoreKeyMapListener: Received "
alshabibb0285992016-03-28 23:30:37 -07001075 + "event {} with null entry... can not process", mapEvent.type());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001076 return;
1077 }
1078 }
1079 log.trace("received groupid map event {} for id {} in device {}",
1080 mapEvent.type(),
1081 group.id(),
1082 key.deviceId());
Madan Jampani0b847532016-03-03 13:44:15 -08001083 if (mapEvent.type() == MapEvent.Type.INSERT || mapEvent.type() == MapEvent.Type.UPDATE) {
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -07001084 // Update the group ID table
1085 getGroupIdTable(group.deviceId()).put(group.id(), group);
Madan Jampani0b847532016-03-03 13:44:15 -08001086 StoredGroupEntry value = Versioned.valueOrNull(mapEvent.newValue());
1087 if (value.state() == Group.GroupState.ADDED) {
1088 if (value.isGroupStateAddedFirstTime()) {
1089 groupEvent = new GroupEvent(Type.GROUP_ADDED, value);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001090 log.trace("Received first time GROUP_ADDED state update for id {} in device {}",
alshabibb0285992016-03-28 23:30:37 -07001091 group.id(),
1092 group.deviceId());
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001093 } else {
Madan Jampani0b847532016-03-03 13:44:15 -08001094 groupEvent = new GroupEvent(Type.GROUP_UPDATED, value);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001095 log.trace("Received following GROUP_ADDED state update for id {} in device {}",
alshabibb0285992016-03-28 23:30:37 -07001096 group.id(),
1097 group.deviceId());
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001098 }
1099 }
Madan Jampani0b847532016-03-03 13:44:15 -08001100 } else if (mapEvent.type() == MapEvent.Type.REMOVE) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001101 groupEvent = new GroupEvent(Type.GROUP_REMOVED, group);
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -07001102 // Remove the entry from the group ID table
1103 getGroupIdTable(group.deviceId()).remove(group.id(), group);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001104 }
1105
1106 if (groupEvent != null) {
1107 notifyDelegate(groupEvent);
1108 }
1109 }
1110 }
Madan Jampani01e05fb2015-08-13 13:29:36 -07001111
1112 private void process(GroupStoreMessage groupOp) {
1113 log.debug("Received remote group operation {} request for device {}",
alshabibb0285992016-03-28 23:30:37 -07001114 groupOp.type(),
1115 groupOp.deviceId());
1116 if (!mastershipService.isLocalMaster(groupOp.deviceId())) {
1117 log.warn("This node is not MASTER for device {}", groupOp.deviceId());
1118 return;
1119 }
1120 if (groupOp.type() == GroupStoreMessage.Type.ADD) {
1121 storeGroupDescriptionInternal(groupOp.groupDesc());
1122 } else if (groupOp.type() == GroupStoreMessage.Type.UPDATE) {
1123 updateGroupDescriptionInternal(groupOp.deviceId(),
1124 groupOp.appCookie(),
1125 groupOp.updateType(),
1126 groupOp.updateBuckets(),
1127 groupOp.newAppCookie());
1128 } else if (groupOp.type() == GroupStoreMessage.Type.DELETE) {
1129 deleteGroupDescriptionInternal(groupOp.deviceId(),
1130 groupOp.appCookie());
1131 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001132 }
1133
1134 /**
1135 * Flattened map key to be used to store group entries.
1136 */
Ray Milkeyb3c5ce22015-08-10 09:07:36 -07001137 protected static class GroupStoreMapKey {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001138 private final DeviceId deviceId;
1139
1140 public GroupStoreMapKey(DeviceId deviceId) {
1141 this.deviceId = deviceId;
1142 }
1143
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001144 public DeviceId deviceId() {
1145 return deviceId;
1146 }
1147
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001148 @Override
1149 public boolean equals(Object o) {
1150 if (this == o) {
1151 return true;
1152 }
1153 if (!(o instanceof GroupStoreMapKey)) {
1154 return false;
1155 }
1156 GroupStoreMapKey that = (GroupStoreMapKey) o;
1157 return this.deviceId.equals(that.deviceId);
1158 }
1159
1160 @Override
1161 public int hashCode() {
1162 int result = 17;
1163
1164 result = 31 * result + Objects.hash(this.deviceId);
1165
1166 return result;
1167 }
1168 }
1169
Ray Milkeyb3c5ce22015-08-10 09:07:36 -07001170 protected static class GroupStoreKeyMapKey extends GroupStoreMapKey {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001171 private final GroupKey appCookie;
alshabibb0285992016-03-28 23:30:37 -07001172
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001173 public GroupStoreKeyMapKey(DeviceId deviceId,
1174 GroupKey appCookie) {
1175 super(deviceId);
1176 this.appCookie = appCookie;
1177 }
1178
1179 @Override
1180 public boolean equals(Object o) {
1181 if (this == o) {
1182 return true;
1183 }
1184 if (!(o instanceof GroupStoreKeyMapKey)) {
1185 return false;
1186 }
1187 GroupStoreKeyMapKey that = (GroupStoreKeyMapKey) o;
1188 return (super.equals(that) &&
1189 this.appCookie.equals(that.appCookie));
1190 }
1191
1192 @Override
1193 public int hashCode() {
1194 int result = 17;
1195
1196 result = 31 * result + super.hashCode() + Objects.hash(this.appCookie);
1197
1198 return result;
1199 }
1200 }
1201
Ray Milkeyb3c5ce22015-08-10 09:07:36 -07001202 protected static class GroupStoreIdMapKey extends GroupStoreMapKey {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001203 private final GroupId groupId;
alshabibb0285992016-03-28 23:30:37 -07001204
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001205 public GroupStoreIdMapKey(DeviceId deviceId,
1206 GroupId groupId) {
1207 super(deviceId);
1208 this.groupId = groupId;
1209 }
1210
1211 @Override
1212 public boolean equals(Object o) {
1213 if (this == o) {
1214 return true;
1215 }
1216 if (!(o instanceof GroupStoreIdMapKey)) {
1217 return false;
1218 }
1219 GroupStoreIdMapKey that = (GroupStoreIdMapKey) o;
1220 return (super.equals(that) &&
1221 this.groupId.equals(that.groupId));
1222 }
1223
1224 @Override
1225 public int hashCode() {
1226 int result = 17;
1227
1228 result = 31 * result + super.hashCode() + Objects.hash(this.groupId);
1229
1230 return result;
1231 }
1232 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001233
1234 @Override
1235 public void pushGroupMetrics(DeviceId deviceId,
1236 Collection<Group> groupEntries) {
1237 boolean deviceInitialAuditStatus =
1238 deviceInitialAuditStatus(deviceId);
1239 Set<Group> southboundGroupEntries =
1240 Sets.newHashSet(groupEntries);
1241 Set<StoredGroupEntry> storedGroupEntries =
1242 Sets.newHashSet(getStoredGroups(deviceId));
1243 Set<Group> extraneousStoredEntries =
1244 Sets.newHashSet(getExtraneousGroups(deviceId));
1245
Sho SHIMIZU695bac62016-08-15 12:41:59 -07001246 if (log.isTraceEnabled()) {
1247 log.trace("pushGroupMetrics: Displaying all ({}) southboundGroupEntries for device {}",
1248 southboundGroupEntries.size(),
1249 deviceId);
1250 for (Group group : southboundGroupEntries) {
1251 log.trace("Group {} in device {}", group, deviceId);
1252 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001253
Sho SHIMIZU695bac62016-08-15 12:41:59 -07001254 log.trace("Displaying all ({}) stored group entries for device {}",
1255 storedGroupEntries.size(),
1256 deviceId);
1257 for (StoredGroupEntry group : storedGroupEntries) {
1258 log.trace("Stored Group {} for device {}", group, deviceId);
1259 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001260 }
1261
alshabibb0285992016-03-28 23:30:37 -07001262 garbageCollect(deviceId, southboundGroupEntries, storedGroupEntries);
1263
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001264 for (Iterator<Group> it2 = southboundGroupEntries.iterator(); it2.hasNext();) {
1265 Group group = it2.next();
1266 if (storedGroupEntries.remove(group)) {
1267 // we both have the group, let's update some info then.
1268 log.trace("Group AUDIT: group {} exists in both planes for device {}",
alshabibb0285992016-03-28 23:30:37 -07001269 group.id(), deviceId);
1270
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001271 groupAdded(group);
1272 it2.remove();
1273 }
1274 }
1275 for (Group group : southboundGroupEntries) {
1276 if (getGroup(group.deviceId(), group.id()) != null) {
1277 // There is a group existing with the same id
1278 // It is possible that group update is
1279 // in progress while we got a stale info from switch
1280 if (!storedGroupEntries.remove(getGroup(
alshabibb0285992016-03-28 23:30:37 -07001281 group.deviceId(), group.id()))) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001282 log.warn("Group AUDIT: Inconsistent state:"
alshabibb0285992016-03-28 23:30:37 -07001283 + "Group exists in ID based table while "
1284 + "not present in key based table");
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001285 }
1286 } else {
1287 // there are groups in the switch that aren't in the store
1288 log.debug("Group AUDIT: extraneous group {} exists in data plane for device {}",
alshabibb0285992016-03-28 23:30:37 -07001289 group.id(), deviceId);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001290 extraneousStoredEntries.remove(group);
1291 extraneousGroup(group);
1292 }
1293 }
1294 for (Group group : storedGroupEntries) {
1295 // there are groups in the store that aren't in the switch
1296 log.debug("Group AUDIT: group {} missing in data plane for device {}",
alshabibb0285992016-03-28 23:30:37 -07001297 group.id(), deviceId);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001298 groupMissing(group);
1299 }
1300 for (Group group : extraneousStoredEntries) {
1301 // there are groups in the extraneous store that
1302 // aren't in the switch
Saurav Das0fd79d92016-03-07 10:58:36 -08001303 log.debug("Group AUDIT: clearing extraneous group {} from store for device {}",
alshabibb0285992016-03-28 23:30:37 -07001304 group.id(), deviceId);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001305 removeExtraneousGroupEntry(group);
1306 }
1307
1308 if (!deviceInitialAuditStatus) {
Saurav Das0fd79d92016-03-07 10:58:36 -08001309 log.info("Group AUDIT: Setting device {} initial AUDIT completed",
alshabibb0285992016-03-28 23:30:37 -07001310 deviceId);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001311 deviceInitialAuditCompleted(deviceId, true);
1312 }
1313 }
1314
alshabibb0285992016-03-28 23:30:37 -07001315 private void garbageCollect(DeviceId deviceId,
1316 Set<Group> southboundGroupEntries,
1317 Set<StoredGroupEntry> storedGroupEntries) {
1318 if (!garbageCollect) {
1319 return;
1320 }
1321
1322 Iterator<StoredGroupEntry> it = storedGroupEntries.iterator();
1323 while (it.hasNext()) {
1324 StoredGroupEntry group = it.next();
1325 if (group.state() != GroupState.PENDING_DELETE && checkGroupRefCount(group)) {
1326 log.debug("Garbage collecting group {} on {}", group, deviceId);
1327 deleteGroupDescription(deviceId, group.appCookie());
1328 southboundGroupEntries.remove(group);
1329 it.remove();
1330 }
1331 }
1332 }
1333
1334 private boolean checkGroupRefCount(Group group) {
1335 return (group.referenceCount() == 0 && group.age() >= gcThresh);
1336 }
1337
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001338 private void groupMissing(Group group) {
1339 switch (group.state()) {
1340 case PENDING_DELETE:
1341 log.debug("Group {} delete confirmation from device {}",
1342 group, group.deviceId());
1343 removeGroupEntry(group);
1344 break;
1345 case ADDED:
1346 case PENDING_ADD:
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -07001347 case PENDING_ADD_RETRY:
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001348 case PENDING_UPDATE:
1349 log.debug("Group {} is in store but not on device {}",
1350 group, group.deviceId());
1351 StoredGroupEntry existing =
1352 getStoredGroupEntry(group.deviceId(), group.id());
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -07001353 log.debug("groupMissing: group entry {} in device {} moving from {} to PENDING_ADD_RETRY",
alshabibb0285992016-03-28 23:30:37 -07001354 existing.id(),
1355 existing.deviceId(),
1356 existing.state());
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -07001357 existing.setState(Group.GroupState.PENDING_ADD_RETRY);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001358 //Re-PUT map entries to trigger map update events
1359 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -07001360 put(new GroupStoreKeyMapKey(existing.deviceId(),
1361 existing.appCookie()), existing);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001362 notifyDelegate(new GroupEvent(GroupEvent.Type.GROUP_ADD_REQUESTED,
1363 group));
1364 break;
1365 default:
1366 log.debug("Group {} has not been installed.", group);
1367 break;
1368 }
1369 }
1370
1371 private void extraneousGroup(Group group) {
Saurav Das0fd79d92016-03-07 10:58:36 -08001372 log.trace("Group {} is on device {} but not in store.",
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001373 group, group.deviceId());
1374 addOrUpdateExtraneousGroupEntry(group);
1375 }
1376
1377 private void groupAdded(Group group) {
1378 log.trace("Group {} Added or Updated in device {}",
1379 group, group.deviceId());
1380 addOrUpdateGroupEntry(group);
1381 }
alshabib10580802015-02-18 18:30:33 -08001382}