blob: 0cf77b36bd686706a9fa1ac3e1295ef5b0ac8417 [file] [log] [blame]
alshabib10580802015-02-18 18:30:33 -08001/*
2 * Copyright 2015 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 */
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;
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -070022
alshabib10580802015-02-18 18:30:33 -080023import org.apache.felix.scr.annotations.Activate;
24import org.apache.felix.scr.annotations.Component;
25import org.apache.felix.scr.annotations.Deactivate;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070026import org.apache.felix.scr.annotations.Reference;
27import org.apache.felix.scr.annotations.ReferenceCardinality;
alshabib10580802015-02-18 18:30:33 -080028import org.apache.felix.scr.annotations.Service;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070029import org.onlab.util.KryoNamespace;
alshabib10580802015-02-18 18:30:33 -080030import org.onlab.util.NewConcurrentHashMap;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070031import org.onosproject.cluster.ClusterService;
Charles Chanf4838a72015-12-07 18:13:45 -080032import org.onosproject.cluster.NodeId;
alshabib10580802015-02-18 18:30:33 -080033import org.onosproject.core.DefaultGroupId;
34import org.onosproject.core.GroupId;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070035import org.onosproject.mastership.MastershipService;
alshabib10580802015-02-18 18:30:33 -080036import org.onosproject.net.DeviceId;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070037import org.onosproject.net.MastershipRole;
alshabib10580802015-02-18 18:30:33 -080038import org.onosproject.net.group.DefaultGroup;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070039import org.onosproject.net.group.DefaultGroupBucket;
alshabib10580802015-02-18 18:30:33 -080040import org.onosproject.net.group.DefaultGroupDescription;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070041import org.onosproject.net.group.DefaultGroupKey;
alshabib10580802015-02-18 18:30:33 -080042import org.onosproject.net.group.Group;
43import org.onosproject.net.group.Group.GroupState;
44import org.onosproject.net.group.GroupBucket;
45import org.onosproject.net.group.GroupBuckets;
46import org.onosproject.net.group.GroupDescription;
47import org.onosproject.net.group.GroupEvent;
48import org.onosproject.net.group.GroupEvent.Type;
49import org.onosproject.net.group.GroupKey;
50import org.onosproject.net.group.GroupOperation;
51import org.onosproject.net.group.GroupStore;
52import org.onosproject.net.group.GroupStoreDelegate;
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -070053import org.onosproject.net.group.StoredGroupBucketEntry;
alshabib10580802015-02-18 18:30:33 -080054import org.onosproject.net.group.StoredGroupEntry;
55import org.onosproject.store.AbstractStore;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070056import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
Jonathan Hart63939a32015-05-08 11:57:03 -070057import org.onosproject.store.service.MultiValuedTimestamp;
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;
62import org.onosproject.store.service.Serializer;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070063import org.onosproject.store.service.StorageService;
Madan Jampani0b847532016-03-03 13:44:15 -080064import org.onosproject.store.service.Versioned;
alshabib10580802015-02-18 18:30:33 -080065import org.slf4j.Logger;
66
Jonathan Hart6ec029a2015-03-24 17:12:35 -070067import java.util.ArrayList;
Srikanth Vavilapalli23181912015-05-04 09:48:09 -070068import java.util.Collection;
Charles Chanf4838a72015-12-07 18:13:45 -080069import java.util.Collections;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070070import java.util.HashMap;
Charles Chan0c7c43b2016-01-14 17:39:20 -080071import java.util.HashSet;
Srikanth Vavilapalli23181912015-05-04 09:48:09 -070072import java.util.Iterator;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070073import java.util.List;
Madan Jampani0b847532016-03-03 13:44:15 -080074import java.util.Map;
Charles Chan0c7c43b2016-01-14 17:39:20 -080075import java.util.Map.Entry;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070076import java.util.Objects;
Sho SHIMIZU30d639b2015-05-05 09:30:35 -070077import java.util.Optional;
Srikanth Vavilapalli23181912015-05-04 09:48:09 -070078import java.util.Set;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070079import java.util.concurrent.ConcurrentHashMap;
80import java.util.concurrent.ConcurrentMap;
81import java.util.concurrent.ExecutorService;
82import java.util.concurrent.Executors;
83import java.util.concurrent.atomic.AtomicInteger;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070084import java.util.stream.Collectors;
85
86import static org.apache.commons.lang3.concurrent.ConcurrentUtils.createIfAbsentUnchecked;
87import static org.onlab.util.Tools.groupedThreads;
88import static org.slf4j.LoggerFactory.getLogger;
alshabib10580802015-02-18 18:30:33 -080089
90/**
Saurav Das0fd79d92016-03-07 10:58:36 -080091 * Manages inventory of group entries using distributed group stores from the
92 * storage service.
alshabib10580802015-02-18 18:30:33 -080093 */
94@Component(immediate = true)
95@Service
96public class DistributedGroupStore
97 extends AbstractStore<GroupEvent, GroupStoreDelegate>
98 implements GroupStore {
99
100 private final Logger log = getLogger(getClass());
101
102 private final int dummyId = 0xffffffff;
103 private final GroupId dummyGroupId = new DefaultGroupId(dummyId);
104
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700105 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
106 protected ClusterCommunicationService clusterCommunicator;
107
108 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
109 protected ClusterService clusterService;
110
111 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700112 protected StorageService storageService;
113
114 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700115 protected MastershipService mastershipService;
116
117 // Per device group table with (device id + app cookie) as key
Madan Jampani0b847532016-03-03 13:44:15 -0800118 private ConsistentMap<GroupStoreKeyMapKey,
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700119 StoredGroupEntry> groupStoreEntriesByKey = null;
120 // Per device group table with (device id + group id) as key
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700121 private final ConcurrentMap<DeviceId, ConcurrentMap<GroupId, StoredGroupEntry>>
122 groupEntriesById = new ConcurrentHashMap<>();
Madan Jampani0b847532016-03-03 13:44:15 -0800123 private ConsistentMap<GroupStoreKeyMapKey,
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700124 StoredGroupEntry> auditPendingReqQueue = null;
alshabib10580802015-02-18 18:30:33 -0800125 private final ConcurrentMap<DeviceId, ConcurrentMap<GroupId, Group>>
126 extraneousGroupEntriesById = new ConcurrentHashMap<>();
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700127 private ExecutorService messageHandlingExecutor;
128 private static final int MESSAGE_HANDLER_THREAD_POOL_SIZE = 1;
alshabib10580802015-02-18 18:30:33 -0800129
Sho SHIMIZU7a4087b2015-09-10 09:23:16 -0700130 private final HashMap<DeviceId, Boolean> deviceAuditStatus = new HashMap<>();
alshabib10580802015-02-18 18:30:33 -0800131
132 private final AtomicInteger groupIdGen = new AtomicInteger();
133
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700134 private KryoNamespace.Builder kryoBuilder = null;
135
HIGUCHI Yuta180d70f2015-10-01 16:13:56 -0700136 private KryoNamespace clusterMsgSerializer;
137
alshabib10580802015-02-18 18:30:33 -0800138 @Activate
139 public void activate() {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700140 kryoBuilder = new KryoNamespace.Builder()
Charles Chan138cd5a2015-09-29 16:57:41 -0700141 .register(KryoNamespaces.API)
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700142 .register(DefaultGroup.class,
143 DefaultGroupBucket.class,
144 DefaultGroupDescription.class,
145 DefaultGroupKey.class,
146 GroupDescription.Type.class,
147 Group.GroupState.class,
148 GroupBuckets.class,
149 DefaultGroupId.class,
150 GroupStoreMessage.class,
151 GroupStoreMessage.Type.class,
152 UpdateType.class,
153 GroupStoreMessageSubjects.class,
154 MultiValuedTimestamp.class,
155 GroupStoreKeyMapKey.class,
156 GroupStoreIdMapKey.class,
157 GroupStoreMapKey.class
Charles Chan138cd5a2015-09-29 16:57:41 -0700158 );
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700159
HIGUCHI Yuta180d70f2015-10-01 16:13:56 -0700160 clusterMsgSerializer = kryoBuilder.build();
161
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700162 messageHandlingExecutor = Executors.
163 newFixedThreadPool(MESSAGE_HANDLER_THREAD_POOL_SIZE,
164 groupedThreads("onos/store/group",
165 "message-handlers"));
Madan Jampani01e05fb2015-08-13 13:29:36 -0700166
167 clusterCommunicator.addSubscriber(GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
HIGUCHI Yuta180d70f2015-10-01 16:13:56 -0700168 clusterMsgSerializer::deserialize,
Madan Jampani01e05fb2015-08-13 13:29:36 -0700169 this::process,
170 messageHandlingExecutor);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700171
Madan Jampani0b847532016-03-03 13:44:15 -0800172 log.debug("Creating Consistent map onos-group-store-keymap");
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700173
Madan Jampani0b847532016-03-03 13:44:15 -0800174 groupStoreEntriesByKey = storageService.<GroupStoreKeyMapKey, StoredGroupEntry>consistentMapBuilder()
175 .withName("onos-group-store-keymap")
176 .withSerializer(Serializer.using(kryoBuilder.build()))
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700177 .build();
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700178 groupStoreEntriesByKey.addListener(new GroupStoreKeyMapListener());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700179 log.debug("Current size of groupstorekeymap:{}",
180 groupStoreEntriesByKey.size());
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700181
Madan Jampani0b847532016-03-03 13:44:15 -0800182 log.debug("Creating Consistent map pendinggroupkeymap");
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700183
Madan Jampani0b847532016-03-03 13:44:15 -0800184 auditPendingReqQueue = storageService.<GroupStoreKeyMapKey, StoredGroupEntry>consistentMapBuilder()
185 .withName("onos-pending-group-keymap")
186 .withSerializer(Serializer.using(kryoBuilder.build()))
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700187 .build();
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700188 log.debug("Current size of pendinggroupkeymap:{}",
189 auditPendingReqQueue.size());
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700190
alshabib10580802015-02-18 18:30:33 -0800191 log.info("Started");
192 }
193
194 @Deactivate
195 public void deactivate() {
HIGUCHI Yuta180d70f2015-10-01 16:13:56 -0700196 clusterCommunicator.removeSubscriber(GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST);
Thomas Vachuska152f9fd2015-04-02 16:28:13 -0700197 groupStoreEntriesByKey.destroy();
Thomas Vachuska152f9fd2015-04-02 16:28:13 -0700198 auditPendingReqQueue.destroy();
alshabib10580802015-02-18 18:30:33 -0800199 log.info("Stopped");
200 }
201
alshabib10580802015-02-18 18:30:33 -0800202 private static NewConcurrentHashMap<GroupId, Group>
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700203 lazyEmptyExtraneousGroupIdTable() {
alshabib10580802015-02-18 18:30:33 -0800204 return NewConcurrentHashMap.<GroupId, Group>ifNeeded();
205 }
206
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700207 private static NewConcurrentHashMap<GroupId, StoredGroupEntry>
208 lazyEmptyGroupIdTable() {
209 return NewConcurrentHashMap.<GroupId, StoredGroupEntry>ifNeeded();
210 }
211
alshabib10580802015-02-18 18:30:33 -0800212 /**
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700213 * Returns the group store eventual consistent key map.
alshabib10580802015-02-18 18:30:33 -0800214 *
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700215 * @return Map representing group key table.
alshabib10580802015-02-18 18:30:33 -0800216 */
Madan Jampani0b847532016-03-03 13:44:15 -0800217 private Map<GroupStoreKeyMapKey, StoredGroupEntry>
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700218 getGroupStoreKeyMap() {
Madan Jampani0b847532016-03-03 13:44:15 -0800219 return groupStoreEntriesByKey.asJavaMap();
alshabib10580802015-02-18 18:30:33 -0800220 }
221
222 /**
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700223 * Returns the group id table for specified device.
alshabib10580802015-02-18 18:30:33 -0800224 *
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700225 * @param deviceId identifier of the device
226 * @return Map representing group key table of given device.
alshabib10580802015-02-18 18:30:33 -0800227 */
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700228 private ConcurrentMap<GroupId, StoredGroupEntry> getGroupIdTable(DeviceId deviceId) {
229 return createIfAbsentUnchecked(groupEntriesById,
230 deviceId, lazyEmptyGroupIdTable());
alshabib10580802015-02-18 18:30:33 -0800231 }
232
233 /**
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700234 * Returns the pending group request table.
alshabib10580802015-02-18 18:30:33 -0800235 *
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700236 * @return Map representing group key table.
alshabib10580802015-02-18 18:30:33 -0800237 */
Madan Jampani0b847532016-03-03 13:44:15 -0800238 private Map<GroupStoreKeyMapKey, StoredGroupEntry>
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700239 getPendingGroupKeyTable() {
Madan Jampani0b847532016-03-03 13:44:15 -0800240 return auditPendingReqQueue.asJavaMap();
alshabib10580802015-02-18 18:30:33 -0800241 }
242
243 /**
244 * Returns the extraneous group id table for specified device.
245 *
246 * @param deviceId identifier of the device
247 * @return Map representing group key table of given device.
248 */
249 private ConcurrentMap<GroupId, Group>
250 getExtraneousGroupIdTable(DeviceId deviceId) {
251 return createIfAbsentUnchecked(extraneousGroupEntriesById,
252 deviceId,
253 lazyEmptyExtraneousGroupIdTable());
254 }
255
256 /**
257 * Returns the number of groups for the specified device in the store.
258 *
259 * @return number of groups for the specified device
260 */
261 @Override
262 public int getGroupCount(DeviceId deviceId) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700263 return (getGroups(deviceId) != null) ?
264 Iterables.size(getGroups(deviceId)) : 0;
alshabib10580802015-02-18 18:30:33 -0800265 }
266
267 /**
268 * Returns the groups associated with a device.
269 *
270 * @param deviceId the device ID
271 *
272 * @return the group entries
273 */
274 @Override
275 public Iterable<Group> getGroups(DeviceId deviceId) {
Charles Chanf4838a72015-12-07 18:13:45 -0800276 // Let ImmutableSet.copyOf do the type conversion
277 return ImmutableSet.copyOf(getStoredGroups(deviceId));
alshabib10580802015-02-18 18:30:33 -0800278 }
279
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700280 private Iterable<StoredGroupEntry> getStoredGroups(DeviceId deviceId) {
Charles Chanf4838a72015-12-07 18:13:45 -0800281 NodeId master = mastershipService.getMasterFor(deviceId);
282 if (master == null) {
283 log.debug("Failed to getGroups: No master for {}", deviceId);
284 return Collections.emptySet();
285 }
286
287 Set<StoredGroupEntry> storedGroups = getGroupStoreKeyMap().values()
288 .stream()
289 .filter(input -> input.deviceId().equals(deviceId))
290 .collect(Collectors.toSet());
291 return ImmutableSet.copyOf(storedGroups);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700292 }
293
alshabib10580802015-02-18 18:30:33 -0800294 /**
295 * Returns the stored group entry.
296 *
297 * @param deviceId the device ID
298 * @param appCookie the group key
299 *
300 * @return a group associated with the key
301 */
302 @Override
303 public Group getGroup(DeviceId deviceId, GroupKey appCookie) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700304 return getStoredGroupEntry(deviceId, appCookie);
305 }
306
307 private StoredGroupEntry getStoredGroupEntry(DeviceId deviceId,
308 GroupKey appCookie) {
309 return getGroupStoreKeyMap().get(new GroupStoreKeyMapKey(deviceId,
310 appCookie));
311 }
312
313 @Override
314 public Group getGroup(DeviceId deviceId, GroupId groupId) {
315 return getStoredGroupEntry(deviceId, groupId);
316 }
317
318 private StoredGroupEntry getStoredGroupEntry(DeviceId deviceId,
319 GroupId groupId) {
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700320 return getGroupIdTable(deviceId).get(groupId);
alshabib10580802015-02-18 18:30:33 -0800321 }
322
323 private int getFreeGroupIdValue(DeviceId deviceId) {
324 int freeId = groupIdGen.incrementAndGet();
325
326 while (true) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700327 Group existing = getGroup(deviceId, new DefaultGroupId(freeId));
alshabib10580802015-02-18 18:30:33 -0800328 if (existing == null) {
329 existing = (
330 extraneousGroupEntriesById.get(deviceId) != null) ?
331 extraneousGroupEntriesById.get(deviceId).
332 get(new DefaultGroupId(freeId)) :
333 null;
334 }
335 if (existing != null) {
336 freeId = groupIdGen.incrementAndGet();
337 } else {
338 break;
339 }
340 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700341 log.debug("getFreeGroupIdValue: Next Free ID is {}", freeId);
alshabib10580802015-02-18 18:30:33 -0800342 return freeId;
343 }
344
345 /**
346 * Stores a new group entry using the information from group description.
347 *
348 * @param groupDesc group description to be used to create group entry
349 */
350 @Override
351 public void storeGroupDescription(GroupDescription groupDesc) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700352 log.debug("In storeGroupDescription");
alshabib10580802015-02-18 18:30:33 -0800353 // Check if a group is existing with the same key
Saurav Das8a0732e2015-11-20 15:27:53 -0800354 Group existingGroup = getGroup(groupDesc.deviceId(), groupDesc.appCookie());
355 if (existingGroup != null) {
Saurav Das4ce45962015-11-24 23:21:05 -0800356 log.warn("Group already exists with the same key {} in dev:{} with id:0x{}",
Saurav Das8a0732e2015-11-20 15:27:53 -0800357 groupDesc.appCookie(), groupDesc.deviceId(),
358 Integer.toHexString(existingGroup.id().id()));
alshabib10580802015-02-18 18:30:33 -0800359 return;
360 }
361
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700362 // Check if group to be created by a remote instance
Madan Jampani175e8fd2015-05-20 14:10:45 -0700363 if (mastershipService.getLocalRole(groupDesc.deviceId()) != MastershipRole.MASTER) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700364 log.debug("storeGroupDescription: Device {} local role is not MASTER",
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700365 groupDesc.deviceId());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700366 if (mastershipService.getMasterFor(groupDesc.deviceId()) == null) {
367 log.error("No Master for device {}..."
368 + "Can not perform add group operation",
369 groupDesc.deviceId());
370 //TODO: Send Group operation failure event
371 return;
372 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700373 GroupStoreMessage groupOp = GroupStoreMessage.
374 createGroupAddRequestMsg(groupDesc.deviceId(),
375 groupDesc);
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700376
Madan Jampani175e8fd2015-05-20 14:10:45 -0700377 clusterCommunicator.unicast(groupOp,
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700378 GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
HIGUCHI Yuta180d70f2015-10-01 16:13:56 -0700379 clusterMsgSerializer::serialize,
Madan Jampani175e8fd2015-05-20 14:10:45 -0700380 mastershipService.getMasterFor(groupDesc.deviceId())).whenComplete((result, error) -> {
381 if (error != null) {
382 log.warn("Failed to send request to master: {} to {}",
383 groupOp,
384 mastershipService.getMasterFor(groupDesc.deviceId()));
385 //TODO: Send Group operation failure event
386 } else {
387 log.debug("Sent Group operation request for device {} "
388 + "to remote MASTER {}",
389 groupDesc.deviceId(),
390 mastershipService.getMasterFor(groupDesc.deviceId()));
391 }
392 });
alshabib10580802015-02-18 18:30:33 -0800393 return;
394 }
395
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700396 log.debug("Store group for device {} is getting handled locally",
397 groupDesc.deviceId());
alshabib10580802015-02-18 18:30:33 -0800398 storeGroupDescriptionInternal(groupDesc);
399 }
400
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700401 private Group getMatchingExtraneousGroupbyId(DeviceId deviceId, Integer groupId) {
402 ConcurrentMap<GroupId, Group> extraneousMap =
403 extraneousGroupEntriesById.get(deviceId);
404 if (extraneousMap == null) {
405 return null;
406 }
407 return extraneousMap.get(new DefaultGroupId(groupId));
408 }
409
410 private Group getMatchingExtraneousGroupbyBuckets(DeviceId deviceId,
411 GroupBuckets buckets) {
412 ConcurrentMap<GroupId, Group> extraneousMap =
413 extraneousGroupEntriesById.get(deviceId);
414 if (extraneousMap == null) {
415 return null;
416 }
417
418 for (Group extraneousGroup:extraneousMap.values()) {
419 if (extraneousGroup.buckets().equals(buckets)) {
420 return extraneousGroup;
421 }
422 }
423 return null;
424 }
425
alshabib10580802015-02-18 18:30:33 -0800426 private void storeGroupDescriptionInternal(GroupDescription groupDesc) {
427 // Check if a group is existing with the same key
428 if (getGroup(groupDesc.deviceId(), groupDesc.appCookie()) != null) {
429 return;
430 }
431
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700432 if (deviceAuditStatus.get(groupDesc.deviceId()) == null) {
433 // Device group audit has not completed yet
434 // Add this group description to pending group key table
435 // Create a group entry object with Dummy Group ID
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700436 log.debug("storeGroupDescriptionInternal: Device {} AUDIT pending...Queuing Group ADD request",
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700437 groupDesc.deviceId());
438 StoredGroupEntry group = new DefaultGroup(dummyGroupId, groupDesc);
439 group.setState(GroupState.WAITING_AUDIT_COMPLETE);
Madan Jampani0b847532016-03-03 13:44:15 -0800440 Map<GroupStoreKeyMapKey, StoredGroupEntry> pendingKeyTable =
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700441 getPendingGroupKeyTable();
442 pendingKeyTable.put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
443 groupDesc.appCookie()),
444 group);
445 return;
446 }
447
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700448 Group matchingExtraneousGroup = null;
449 if (groupDesc.givenGroupId() != null) {
450 //Check if there is a extraneous group existing with the same Id
451 matchingExtraneousGroup = getMatchingExtraneousGroupbyId(
452 groupDesc.deviceId(), groupDesc.givenGroupId());
453 if (matchingExtraneousGroup != null) {
Saurav Das0fd79d92016-03-07 10:58:36 -0800454 log.debug("storeGroupDescriptionInternal: Matching extraneous group "
455 + "found in Device {} for group id 0x{}",
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700456 groupDesc.deviceId(),
Saurav Das0fd79d92016-03-07 10:58:36 -0800457 Integer.toHexString(groupDesc.givenGroupId()));
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700458 //Check if the group buckets matches with user provided buckets
459 if (matchingExtraneousGroup.buckets().equals(groupDesc.buckets())) {
460 //Group is already existing with the same buckets and Id
461 // Create a group entry object
Saurav Das0fd79d92016-03-07 10:58:36 -0800462 log.debug("storeGroupDescriptionInternal: Buckets also matching "
463 + "in Device {} for group id 0x{}",
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700464 groupDesc.deviceId(),
Saurav Das0fd79d92016-03-07 10:58:36 -0800465 Integer.toHexString(groupDesc.givenGroupId()));
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700466 StoredGroupEntry group = new DefaultGroup(
467 matchingExtraneousGroup.id(), groupDesc);
468 // Insert the newly created group entry into key and id maps
469 getGroupStoreKeyMap().
470 put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
471 groupDesc.appCookie()), group);
472 // Ensure it also inserted into group id based table to
473 // avoid any chances of duplication in group id generation
474 getGroupIdTable(groupDesc.deviceId()).
475 put(matchingExtraneousGroup.id(), group);
476 addOrUpdateGroupEntry(matchingExtraneousGroup);
477 removeExtraneousGroupEntry(matchingExtraneousGroup);
478 return;
479 } else {
480 //Group buckets are not matching. Update group
481 //with user provided buckets.
Saurav Das0fd79d92016-03-07 10:58:36 -0800482 log.debug("storeGroupDescriptionInternal: Buckets are not "
483 + "matching in Device {} for group id 0x{}",
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700484 groupDesc.deviceId(),
Saurav Das0fd79d92016-03-07 10:58:36 -0800485 Integer.toHexString(groupDesc.givenGroupId()));
486 StoredGroupEntry modifiedGroup = new DefaultGroup(
487 matchingExtraneousGroup.id(), groupDesc);
488 modifiedGroup.setState(GroupState.PENDING_UPDATE);
489 getGroupStoreKeyMap().
490 put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
491 groupDesc.appCookie()), modifiedGroup);
492 // Ensure it also inserted into group id based table to
493 // avoid any chances of duplication in group id generation
494 getGroupIdTable(groupDesc.deviceId()).
495 put(matchingExtraneousGroup.id(), modifiedGroup);
496 removeExtraneousGroupEntry(matchingExtraneousGroup);
497 log.debug("storeGroupDescriptionInternal: Triggering Group "
498 + "UPDATE request for {} in device {}",
499 matchingExtraneousGroup.id(),
500 groupDesc.deviceId());
501 notifyDelegate(new GroupEvent(Type.GROUP_UPDATE_REQUESTED, modifiedGroup));
502 return;
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700503 }
504 }
505 } else {
506 //Check if there is an extraneous group with user provided buckets
507 matchingExtraneousGroup = getMatchingExtraneousGroupbyBuckets(
508 groupDesc.deviceId(), groupDesc.buckets());
509 if (matchingExtraneousGroup != null) {
510 //Group is already existing with the same buckets.
511 //So reuse this group.
512 log.debug("storeGroupDescriptionInternal: Matching extraneous group found in Device {}",
513 groupDesc.deviceId());
514 //Create a group entry object
515 StoredGroupEntry group = new DefaultGroup(
516 matchingExtraneousGroup.id(), groupDesc);
517 // Insert the newly created group entry into key and id maps
518 getGroupStoreKeyMap().
519 put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
520 groupDesc.appCookie()), group);
521 // Ensure it also inserted into group id based table to
522 // avoid any chances of duplication in group id generation
523 getGroupIdTable(groupDesc.deviceId()).
524 put(matchingExtraneousGroup.id(), group);
525 addOrUpdateGroupEntry(matchingExtraneousGroup);
526 removeExtraneousGroupEntry(matchingExtraneousGroup);
527 return;
528 } else {
529 //TODO: Check if there are any empty groups that can be used here
530 log.debug("storeGroupDescriptionInternal: No matching extraneous groups found in Device {}",
531 groupDesc.deviceId());
532 }
533 }
534
Saurav Das100e3b82015-04-30 11:12:10 -0700535 GroupId id = null;
536 if (groupDesc.givenGroupId() == null) {
537 // Get a new group identifier
538 id = new DefaultGroupId(getFreeGroupIdValue(groupDesc.deviceId()));
539 } else {
Saurav Das8be4e3a2016-03-11 17:19:07 -0800540 // we need to use the identifier passed in by caller, but check if
541 // already used
542 Group existing = getGroup(groupDesc.deviceId(),
543 new DefaultGroupId(groupDesc.givenGroupId()));
544 if (existing != null) {
545 log.warn("Group already exists with the same id: 0x{} in dev:{} "
546 + "but with different key: {} (request gkey: {})",
547 Integer.toHexString(groupDesc.givenGroupId()),
548 groupDesc.deviceId(),
549 existing.appCookie(),
550 groupDesc.appCookie());
551 return;
552 }
Saurav Das100e3b82015-04-30 11:12:10 -0700553 id = new DefaultGroupId(groupDesc.givenGroupId());
554 }
alshabib10580802015-02-18 18:30:33 -0800555 // Create a group entry object
556 StoredGroupEntry group = new DefaultGroup(id, groupDesc);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700557 // Insert the newly created group entry into key and id maps
558 getGroupStoreKeyMap().
559 put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
560 groupDesc.appCookie()), group);
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700561 // Ensure it also inserted into group id based table to
562 // avoid any chances of duplication in group id generation
563 getGroupIdTable(groupDesc.deviceId()).
564 put(id, group);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700565 log.debug("storeGroupDescriptionInternal: Processing Group ADD request for Id {} in device {}",
566 id,
567 groupDesc.deviceId());
alshabib10580802015-02-18 18:30:33 -0800568 notifyDelegate(new GroupEvent(GroupEvent.Type.GROUP_ADD_REQUESTED,
569 group));
570 }
571
572 /**
573 * Updates the existing group entry with the information
574 * from group description.
575 *
576 * @param deviceId the device ID
577 * @param oldAppCookie the current group key
578 * @param type update type
579 * @param newBuckets group buckets for updates
580 * @param newAppCookie optional new group key
581 */
582 @Override
583 public void updateGroupDescription(DeviceId deviceId,
584 GroupKey oldAppCookie,
585 UpdateType type,
586 GroupBuckets newBuckets,
587 GroupKey newAppCookie) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700588 // Check if group update to be done by a remote instance
sangho52abe3a2015-05-05 14:13:34 -0700589 if (mastershipService.getMasterFor(deviceId) != null &&
590 mastershipService.getLocalRole(deviceId) != MastershipRole.MASTER) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700591 log.debug("updateGroupDescription: Device {} local role is not MASTER",
592 deviceId);
593 if (mastershipService.getMasterFor(deviceId) == null) {
594 log.error("No Master for device {}..."
595 + "Can not perform update group operation",
596 deviceId);
597 //TODO: Send Group operation failure event
598 return;
599 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700600 GroupStoreMessage groupOp = GroupStoreMessage.
601 createGroupUpdateRequestMsg(deviceId,
602 oldAppCookie,
603 type,
604 newBuckets,
605 newAppCookie);
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700606
Madan Jampani175e8fd2015-05-20 14:10:45 -0700607 clusterCommunicator.unicast(groupOp,
608 GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
HIGUCHI Yuta180d70f2015-10-01 16:13:56 -0700609 clusterMsgSerializer::serialize,
Madan Jampani175e8fd2015-05-20 14:10:45 -0700610 mastershipService.getMasterFor(deviceId)).whenComplete((result, error) -> {
611 if (error != null) {
612 log.warn("Failed to send request to master: {} to {}",
613 groupOp,
614 mastershipService.getMasterFor(deviceId), error);
615 }
616 //TODO: Send Group operation failure event
617 });
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700618 return;
619 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700620 log.debug("updateGroupDescription for device {} is getting handled locally",
621 deviceId);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700622 updateGroupDescriptionInternal(deviceId,
623 oldAppCookie,
624 type,
625 newBuckets,
626 newAppCookie);
627 }
628
629 private void updateGroupDescriptionInternal(DeviceId deviceId,
630 GroupKey oldAppCookie,
631 UpdateType type,
632 GroupBuckets newBuckets,
633 GroupKey newAppCookie) {
alshabib10580802015-02-18 18:30:33 -0800634 // Check if a group is existing with the provided key
635 Group oldGroup = getGroup(deviceId, oldAppCookie);
636 if (oldGroup == null) {
Saurav Das8be4e3a2016-03-11 17:19:07 -0800637 log.warn("updateGroupDescriptionInternal: Group not found...strange. "
638 + "GroupKey:{} DeviceId:{}", oldAppCookie, deviceId);
alshabib10580802015-02-18 18:30:33 -0800639 return;
640 }
641
642 List<GroupBucket> newBucketList = getUpdatedBucketList(oldGroup,
643 type,
644 newBuckets);
645 if (newBucketList != null) {
646 // Create a new group object from the old group
647 GroupBuckets updatedBuckets = new GroupBuckets(newBucketList);
648 GroupKey newCookie = (newAppCookie != null) ? newAppCookie : oldAppCookie;
649 GroupDescription updatedGroupDesc = new DefaultGroupDescription(
650 oldGroup.deviceId(),
651 oldGroup.type(),
652 updatedBuckets,
653 newCookie,
Saurav Das100e3b82015-04-30 11:12:10 -0700654 oldGroup.givenGroupId(),
alshabib10580802015-02-18 18:30:33 -0800655 oldGroup.appId());
656 StoredGroupEntry newGroup = new DefaultGroup(oldGroup.id(),
657 updatedGroupDesc);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700658 log.debug("updateGroupDescriptionInternal: group entry {} in device {} moving from {} to PENDING_UPDATE",
659 oldGroup.id(),
660 oldGroup.deviceId(),
661 oldGroup.state());
alshabib10580802015-02-18 18:30:33 -0800662 newGroup.setState(GroupState.PENDING_UPDATE);
663 newGroup.setLife(oldGroup.life());
664 newGroup.setPackets(oldGroup.packets());
665 newGroup.setBytes(oldGroup.bytes());
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700666 //Update the group entry in groupkey based map.
667 //Update to groupid based map will happen in the
668 //groupkey based map update listener
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700669 log.debug("updateGroupDescriptionInternal with type {}: Group updated with buckets",
670 type);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700671 getGroupStoreKeyMap().
672 put(new GroupStoreKeyMapKey(newGroup.deviceId(),
673 newGroup.appCookie()), newGroup);
alshabib10580802015-02-18 18:30:33 -0800674 notifyDelegate(new GroupEvent(Type.GROUP_UPDATE_REQUESTED, newGroup));
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700675 } else {
676 log.warn("updateGroupDescriptionInternal with type {}: No "
677 + "change in the buckets in update", type);
alshabib10580802015-02-18 18:30:33 -0800678 }
679 }
680
681 private List<GroupBucket> getUpdatedBucketList(Group oldGroup,
682 UpdateType type,
683 GroupBuckets buckets) {
684 GroupBuckets oldBuckets = oldGroup.buckets();
Sho SHIMIZU7a4087b2015-09-10 09:23:16 -0700685 List<GroupBucket> newBucketList = new ArrayList<>(oldBuckets.buckets());
alshabib10580802015-02-18 18:30:33 -0800686 boolean groupDescUpdated = false;
687
688 if (type == UpdateType.ADD) {
689 // Check if the any of the new buckets are part of
690 // the old bucket list
691 for (GroupBucket addBucket:buckets.buckets()) {
692 if (!newBucketList.contains(addBucket)) {
693 newBucketList.add(addBucket);
694 groupDescUpdated = true;
695 }
696 }
697 } else if (type == UpdateType.REMOVE) {
698 // Check if the to be removed buckets are part of the
699 // old bucket list
700 for (GroupBucket removeBucket:buckets.buckets()) {
701 if (newBucketList.contains(removeBucket)) {
702 newBucketList.remove(removeBucket);
703 groupDescUpdated = true;
704 }
705 }
706 }
707
708 if (groupDescUpdated) {
709 return newBucketList;
710 } else {
711 return null;
712 }
713 }
714
715 /**
716 * Triggers deleting the existing group entry.
717 *
718 * @param deviceId the device ID
719 * @param appCookie the group key
720 */
721 @Override
722 public void deleteGroupDescription(DeviceId deviceId,
723 GroupKey appCookie) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700724 // Check if group to be deleted by a remote instance
725 if (mastershipService.
726 getLocalRole(deviceId) != MastershipRole.MASTER) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700727 log.debug("deleteGroupDescription: Device {} local role is not MASTER",
728 deviceId);
729 if (mastershipService.getMasterFor(deviceId) == null) {
730 log.error("No Master for device {}..."
731 + "Can not perform delete group operation",
732 deviceId);
733 //TODO: Send Group operation failure event
734 return;
735 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700736 GroupStoreMessage groupOp = GroupStoreMessage.
737 createGroupDeleteRequestMsg(deviceId,
738 appCookie);
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700739
Madan Jampani175e8fd2015-05-20 14:10:45 -0700740 clusterCommunicator.unicast(groupOp,
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700741 GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
HIGUCHI Yuta180d70f2015-10-01 16:13:56 -0700742 clusterMsgSerializer::serialize,
Madan Jampani175e8fd2015-05-20 14:10:45 -0700743 mastershipService.getMasterFor(deviceId)).whenComplete((result, error) -> {
744 if (error != null) {
745 log.warn("Failed to send request to master: {} to {}",
746 groupOp,
747 mastershipService.getMasterFor(deviceId), error);
748 }
749 //TODO: Send Group operation failure event
750 });
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700751 return;
752 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700753 log.debug("deleteGroupDescription in device {} is getting handled locally",
754 deviceId);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700755 deleteGroupDescriptionInternal(deviceId, appCookie);
756 }
757
758 private void deleteGroupDescriptionInternal(DeviceId deviceId,
759 GroupKey appCookie) {
alshabib10580802015-02-18 18:30:33 -0800760 // Check if a group is existing with the provided key
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700761 StoredGroupEntry existing = getStoredGroupEntry(deviceId, appCookie);
alshabib10580802015-02-18 18:30:33 -0800762 if (existing == null) {
763 return;
764 }
765
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700766 log.debug("deleteGroupDescriptionInternal: group entry {} in device {} moving from {} to PENDING_DELETE",
767 existing.id(),
768 existing.deviceId(),
769 existing.state());
alshabib10580802015-02-18 18:30:33 -0800770 synchronized (existing) {
771 existing.setState(GroupState.PENDING_DELETE);
Saurav Das80980c72016-03-23 11:22:49 -0700772 getGroupStoreKeyMap().
773 put(new GroupStoreKeyMapKey(existing.deviceId(), existing.appCookie()),
774 existing);
alshabib10580802015-02-18 18:30:33 -0800775 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700776 log.debug("deleteGroupDescriptionInternal: in device {} issuing GROUP_REMOVE_REQUESTED",
777 deviceId);
alshabib10580802015-02-18 18:30:33 -0800778 notifyDelegate(new GroupEvent(Type.GROUP_REMOVE_REQUESTED, existing));
779 }
780
781 /**
782 * Stores a new group entry, or updates an existing entry.
783 *
784 * @param group group entry
785 */
786 @Override
787 public void addOrUpdateGroupEntry(Group group) {
788 // check if this new entry is an update to an existing entry
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700789 StoredGroupEntry existing = getStoredGroupEntry(group.deviceId(),
790 group.id());
alshabib10580802015-02-18 18:30:33 -0800791 GroupEvent event = null;
792
793 if (existing != null) {
Saurav Das0fd79d92016-03-07 10:58:36 -0800794 log.trace("addOrUpdateGroupEntry: updating group entry {} in device {}",
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700795 group.id(),
796 group.deviceId());
alshabib10580802015-02-18 18:30:33 -0800797 synchronized (existing) {
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -0700798 for (GroupBucket bucket:group.buckets().buckets()) {
Sho SHIMIZU30d639b2015-05-05 09:30:35 -0700799 Optional<GroupBucket> matchingBucket =
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -0700800 existing.buckets().buckets()
801 .stream()
802 .filter((existingBucket)->(existingBucket.equals(bucket)))
803 .findFirst();
804 if (matchingBucket.isPresent()) {
805 ((StoredGroupBucketEntry) matchingBucket.
806 get()).setPackets(bucket.packets());
807 ((StoredGroupBucketEntry) matchingBucket.
808 get()).setBytes(bucket.bytes());
809 } else {
810 log.warn("addOrUpdateGroupEntry: No matching "
811 + "buckets to update stats");
812 }
813 }
alshabib10580802015-02-18 18:30:33 -0800814 existing.setLife(group.life());
815 existing.setPackets(group.packets());
816 existing.setBytes(group.bytes());
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -0700817 if ((existing.state() == GroupState.PENDING_ADD) ||
818 (existing.state() == GroupState.PENDING_ADD_RETRY)) {
Saurav Das0fd79d92016-03-07 10:58:36 -0800819 log.trace("addOrUpdateGroupEntry: group entry {} in device {} moving from {} to ADDED",
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700820 existing.id(),
821 existing.deviceId(),
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -0700822 existing.state());
alshabib10580802015-02-18 18:30:33 -0800823 existing.setState(GroupState.ADDED);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700824 existing.setIsGroupStateAddedFirstTime(true);
alshabib10580802015-02-18 18:30:33 -0800825 event = new GroupEvent(Type.GROUP_ADDED, existing);
826 } else {
Saurav Das0fd79d92016-03-07 10:58:36 -0800827 log.trace("addOrUpdateGroupEntry: group entry {} in device {} moving from {} to ADDED",
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700828 existing.id(),
829 existing.deviceId(),
830 GroupState.PENDING_UPDATE);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700831 existing.setState(GroupState.ADDED);
832 existing.setIsGroupStateAddedFirstTime(false);
alshabib10580802015-02-18 18:30:33 -0800833 event = new GroupEvent(Type.GROUP_UPDATED, existing);
834 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700835 //Re-PUT map entries to trigger map update events
836 getGroupStoreKeyMap().
837 put(new GroupStoreKeyMapKey(existing.deviceId(),
838 existing.appCookie()), existing);
alshabib10580802015-02-18 18:30:33 -0800839 }
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700840 } else {
841 log.warn("addOrUpdateGroupEntry: Group update "
842 + "happening for a non-existing entry in the map");
alshabib10580802015-02-18 18:30:33 -0800843 }
844
845 if (event != null) {
846 notifyDelegate(event);
847 }
848 }
849
850 /**
851 * Removes the group entry from store.
852 *
853 * @param group group entry
854 */
855 @Override
856 public void removeGroupEntry(Group group) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700857 StoredGroupEntry existing = getStoredGroupEntry(group.deviceId(),
858 group.id());
alshabib10580802015-02-18 18:30:33 -0800859
860 if (existing != null) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700861 log.debug("removeGroupEntry: removing group entry {} in device {}",
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700862 group.id(),
863 group.deviceId());
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700864 //Removal from groupid based map will happen in the
865 //map update listener
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700866 getGroupStoreKeyMap().remove(new GroupStoreKeyMapKey(existing.deviceId(),
867 existing.appCookie()));
alshabib10580802015-02-18 18:30:33 -0800868 notifyDelegate(new GroupEvent(Type.GROUP_REMOVED, existing));
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700869 } else {
870 log.warn("removeGroupEntry for {} in device{} is "
871 + "not existing in our maps",
872 group.id(),
873 group.deviceId());
alshabib10580802015-02-18 18:30:33 -0800874 }
875 }
876
877 @Override
Charles Chan0c7c43b2016-01-14 17:39:20 -0800878 public void purgeGroupEntry(DeviceId deviceId) {
879 Set<Entry<GroupStoreKeyMapKey, StoredGroupEntry>> entryPendingRemove =
880 new HashSet<>();
881
Madan Jampani0b847532016-03-03 13:44:15 -0800882 getGroupStoreKeyMap().entrySet().stream()
Charles Chan0c7c43b2016-01-14 17:39:20 -0800883 .filter(entry -> entry.getKey().deviceId().equals(deviceId))
884 .forEach(entryPendingRemove::add);
885
886 entryPendingRemove.forEach(entry -> {
887 groupStoreEntriesByKey.remove(entry.getKey());
888 notifyDelegate(new GroupEvent(Type.GROUP_REMOVED, entry.getValue()));
889 });
890 }
891
892 @Override
alshabib10580802015-02-18 18:30:33 -0800893 public void deviceInitialAuditCompleted(DeviceId deviceId,
894 boolean completed) {
895 synchronized (deviceAuditStatus) {
896 if (completed) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700897 log.debug("AUDIT completed for device {}",
898 deviceId);
alshabib10580802015-02-18 18:30:33 -0800899 deviceAuditStatus.put(deviceId, true);
900 // Execute all pending group requests
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700901 List<StoredGroupEntry> pendingGroupRequests =
902 getPendingGroupKeyTable().values()
903 .stream()
904 .filter(g-> g.deviceId().equals(deviceId))
905 .collect(Collectors.toList());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700906 log.debug("processing pending group add requests for device {} and number of pending requests {}",
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700907 deviceId,
908 pendingGroupRequests.size());
909 for (Group group:pendingGroupRequests) {
alshabib10580802015-02-18 18:30:33 -0800910 GroupDescription tmp = new DefaultGroupDescription(
911 group.deviceId(),
912 group.type(),
913 group.buckets(),
914 group.appCookie(),
Saurav Das100e3b82015-04-30 11:12:10 -0700915 group.givenGroupId(),
alshabib10580802015-02-18 18:30:33 -0800916 group.appId());
917 storeGroupDescriptionInternal(tmp);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700918 getPendingGroupKeyTable().
919 remove(new GroupStoreKeyMapKey(deviceId, group.appCookie()));
alshabib10580802015-02-18 18:30:33 -0800920 }
alshabib10580802015-02-18 18:30:33 -0800921 } else {
Thomas Vachuskac40d4632015-04-09 16:55:03 -0700922 Boolean audited = deviceAuditStatus.get(deviceId);
923 if (audited != null && audited) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700924 log.debug("Clearing AUDIT status for device {}", deviceId);
alshabib10580802015-02-18 18:30:33 -0800925 deviceAuditStatus.put(deviceId, false);
926 }
927 }
928 }
929 }
930
931 @Override
932 public boolean deviceInitialAuditStatus(DeviceId deviceId) {
933 synchronized (deviceAuditStatus) {
Thomas Vachuskac40d4632015-04-09 16:55:03 -0700934 Boolean audited = deviceAuditStatus.get(deviceId);
935 return audited != null && audited;
alshabib10580802015-02-18 18:30:33 -0800936 }
937 }
938
939 @Override
940 public void groupOperationFailed(DeviceId deviceId, GroupOperation operation) {
941
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700942 StoredGroupEntry existing = getStoredGroupEntry(deviceId,
943 operation.groupId());
alshabib10580802015-02-18 18:30:33 -0800944
945 if (existing == null) {
946 log.warn("No group entry with ID {} found ", operation.groupId());
947 return;
948 }
949
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700950 log.warn("groupOperationFailed: group operation {} failed"
Saurav Das0fd79d92016-03-07 10:58:36 -0800951 + "for group {} in device {} with code {}",
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700952 operation.opType(),
953 existing.id(),
Saurav Das0fd79d92016-03-07 10:58:36 -0800954 existing.deviceId(),
955 operation.failureCode());
956 if (operation.failureCode() == GroupOperation.GroupMsgErrorCode.GROUP_EXISTS) {
957 log.warn("Current extraneous groups in device:{} are: {}",
958 deviceId,
959 getExtraneousGroups(deviceId));
Saurav Das8be4e3a2016-03-11 17:19:07 -0800960 if (operation.buckets().equals(existing.buckets())) {
961 if (existing.state() == GroupState.PENDING_ADD) {
962 log.info("GROUP_EXISTS: GroupID and Buckets match for group in pending "
963 + "add state - moving to ADDED for group {} in device {}",
964 existing.id(), deviceId);
965 addOrUpdateGroupEntry(existing);
966 return;
967 } else {
968 log.warn("GROUP EXISTS: Group ID matched but buckets did not. "
969 + "Operation: {} Existing: {}", operation.buckets(),
970 existing.buckets());
971 }
972 }
Saurav Das0fd79d92016-03-07 10:58:36 -0800973 }
alshabib10580802015-02-18 18:30:33 -0800974 switch (operation.opType()) {
975 case ADD:
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -0700976 if (existing.state() == GroupState.PENDING_ADD) {
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -0700977 notifyDelegate(new GroupEvent(Type.GROUP_ADD_FAILED, existing));
978 log.warn("groupOperationFailed: cleaningup "
979 + "group {} from store in device {}....",
980 existing.id(),
981 existing.deviceId());
982 //Removal from groupid based map will happen in the
983 //map update listener
984 getGroupStoreKeyMap().remove(new GroupStoreKeyMapKey(existing.deviceId(),
985 existing.appCookie()));
986 }
alshabib10580802015-02-18 18:30:33 -0800987 break;
988 case MODIFY:
989 notifyDelegate(new GroupEvent(Type.GROUP_UPDATE_FAILED, existing));
990 break;
991 case DELETE:
992 notifyDelegate(new GroupEvent(Type.GROUP_REMOVE_FAILED, existing));
993 break;
994 default:
995 log.warn("Unknown group operation type {}", operation.opType());
996 }
alshabib10580802015-02-18 18:30:33 -0800997 }
998
999 @Override
1000 public void addOrUpdateExtraneousGroupEntry(Group group) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001001 log.debug("add/update extraneous group entry {} in device {}",
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001002 group.id(),
1003 group.deviceId());
alshabib10580802015-02-18 18:30:33 -08001004 ConcurrentMap<GroupId, Group> extraneousIdTable =
1005 getExtraneousGroupIdTable(group.deviceId());
1006 extraneousIdTable.put(group.id(), group);
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -07001007 // Don't remove the extraneous groups, instead re-use it when
1008 // a group request comes with the same set of buckets
alshabib10580802015-02-18 18:30:33 -08001009 }
1010
1011 @Override
1012 public void removeExtraneousGroupEntry(Group group) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001013 log.debug("remove extraneous group entry {} of device {} from store",
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001014 group.id(),
1015 group.deviceId());
alshabib10580802015-02-18 18:30:33 -08001016 ConcurrentMap<GroupId, Group> extraneousIdTable =
1017 getExtraneousGroupIdTable(group.deviceId());
1018 extraneousIdTable.remove(group.id());
1019 }
1020
1021 @Override
1022 public Iterable<Group> getExtraneousGroups(DeviceId deviceId) {
1023 // flatten and make iterator unmodifiable
1024 return FluentIterable.from(
1025 getExtraneousGroupIdTable(deviceId).values());
1026 }
1027
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001028 /**
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -07001029 * Map handler to receive any events when the group key map is updated.
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001030 */
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -07001031 private class GroupStoreKeyMapListener implements
Madan Jampani0b847532016-03-03 13:44:15 -08001032 MapEventListener<GroupStoreKeyMapKey, StoredGroupEntry> {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001033
1034 @Override
Madan Jampani0b847532016-03-03 13:44:15 -08001035 public void event(MapEvent<GroupStoreKeyMapKey, StoredGroupEntry> mapEvent) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001036 GroupEvent groupEvent = null;
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001037 GroupStoreKeyMapKey key = mapEvent.key();
Madan Jampani0b847532016-03-03 13:44:15 -08001038 StoredGroupEntry group = Versioned.valueOrNull(mapEvent.newValue());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001039 if ((key == null) && (group == null)) {
1040 log.error("GroupStoreKeyMapListener: Received "
1041 + "event {} with null entry", mapEvent.type());
1042 return;
1043 } else if (group == null) {
1044 group = getGroupIdTable(key.deviceId()).values()
1045 .stream()
1046 .filter((storedGroup) -> (storedGroup.appCookie().equals(key.appCookie)))
1047 .findFirst().get();
1048 if (group == null) {
1049 log.error("GroupStoreKeyMapListener: Received "
1050 + "event {} with null entry... can not process", mapEvent.type());
1051 return;
1052 }
1053 }
1054 log.trace("received groupid map event {} for id {} in device {}",
1055 mapEvent.type(),
1056 group.id(),
1057 key.deviceId());
Madan Jampani0b847532016-03-03 13:44:15 -08001058 if (mapEvent.type() == MapEvent.Type.INSERT || mapEvent.type() == MapEvent.Type.UPDATE) {
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -07001059 // Update the group ID table
1060 getGroupIdTable(group.deviceId()).put(group.id(), group);
Madan Jampani0b847532016-03-03 13:44:15 -08001061 StoredGroupEntry value = Versioned.valueOrNull(mapEvent.newValue());
1062 if (value.state() == Group.GroupState.ADDED) {
1063 if (value.isGroupStateAddedFirstTime()) {
1064 groupEvent = new GroupEvent(Type.GROUP_ADDED, value);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001065 log.trace("Received first time GROUP_ADDED state update for id {} in device {}",
1066 group.id(),
1067 group.deviceId());
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001068 } else {
Madan Jampani0b847532016-03-03 13:44:15 -08001069 groupEvent = new GroupEvent(Type.GROUP_UPDATED, value);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001070 log.trace("Received following GROUP_ADDED state update for id {} in device {}",
1071 group.id(),
1072 group.deviceId());
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001073 }
1074 }
Madan Jampani0b847532016-03-03 13:44:15 -08001075 } else if (mapEvent.type() == MapEvent.Type.REMOVE) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001076 groupEvent = new GroupEvent(Type.GROUP_REMOVED, group);
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -07001077 // Remove the entry from the group ID table
1078 getGroupIdTable(group.deviceId()).remove(group.id(), group);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001079 }
1080
1081 if (groupEvent != null) {
1082 notifyDelegate(groupEvent);
1083 }
1084 }
1085 }
Madan Jampani01e05fb2015-08-13 13:29:36 -07001086
1087 private void process(GroupStoreMessage groupOp) {
1088 log.debug("Received remote group operation {} request for device {}",
1089 groupOp.type(),
1090 groupOp.deviceId());
1091 if (!mastershipService.isLocalMaster(groupOp.deviceId())) {
1092 log.warn("This node is not MASTER for device {}", groupOp.deviceId());
1093 return;
1094 }
1095 if (groupOp.type() == GroupStoreMessage.Type.ADD) {
1096 storeGroupDescriptionInternal(groupOp.groupDesc());
1097 } else if (groupOp.type() == GroupStoreMessage.Type.UPDATE) {
1098 updateGroupDescriptionInternal(groupOp.deviceId(),
1099 groupOp.appCookie(),
1100 groupOp.updateType(),
1101 groupOp.updateBuckets(),
1102 groupOp.newAppCookie());
1103 } else if (groupOp.type() == GroupStoreMessage.Type.DELETE) {
1104 deleteGroupDescriptionInternal(groupOp.deviceId(),
1105 groupOp.appCookie());
1106 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001107 }
1108
1109 /**
1110 * Flattened map key to be used to store group entries.
1111 */
Ray Milkeyb3c5ce22015-08-10 09:07:36 -07001112 protected static class GroupStoreMapKey {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001113 private final DeviceId deviceId;
1114
1115 public GroupStoreMapKey(DeviceId deviceId) {
1116 this.deviceId = deviceId;
1117 }
1118
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001119 public DeviceId deviceId() {
1120 return deviceId;
1121 }
1122
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001123 @Override
1124 public boolean equals(Object o) {
1125 if (this == o) {
1126 return true;
1127 }
1128 if (!(o instanceof GroupStoreMapKey)) {
1129 return false;
1130 }
1131 GroupStoreMapKey that = (GroupStoreMapKey) o;
1132 return this.deviceId.equals(that.deviceId);
1133 }
1134
1135 @Override
1136 public int hashCode() {
1137 int result = 17;
1138
1139 result = 31 * result + Objects.hash(this.deviceId);
1140
1141 return result;
1142 }
1143 }
1144
Ray Milkeyb3c5ce22015-08-10 09:07:36 -07001145 protected static class GroupStoreKeyMapKey extends GroupStoreMapKey {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001146 private final GroupKey appCookie;
1147 public GroupStoreKeyMapKey(DeviceId deviceId,
1148 GroupKey appCookie) {
1149 super(deviceId);
1150 this.appCookie = appCookie;
1151 }
1152
1153 @Override
1154 public boolean equals(Object o) {
1155 if (this == o) {
1156 return true;
1157 }
1158 if (!(o instanceof GroupStoreKeyMapKey)) {
1159 return false;
1160 }
1161 GroupStoreKeyMapKey that = (GroupStoreKeyMapKey) o;
1162 return (super.equals(that) &&
1163 this.appCookie.equals(that.appCookie));
1164 }
1165
1166 @Override
1167 public int hashCode() {
1168 int result = 17;
1169
1170 result = 31 * result + super.hashCode() + Objects.hash(this.appCookie);
1171
1172 return result;
1173 }
1174 }
1175
Ray Milkeyb3c5ce22015-08-10 09:07:36 -07001176 protected static class GroupStoreIdMapKey extends GroupStoreMapKey {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001177 private final GroupId groupId;
1178 public GroupStoreIdMapKey(DeviceId deviceId,
1179 GroupId groupId) {
1180 super(deviceId);
1181 this.groupId = groupId;
1182 }
1183
1184 @Override
1185 public boolean equals(Object o) {
1186 if (this == o) {
1187 return true;
1188 }
1189 if (!(o instanceof GroupStoreIdMapKey)) {
1190 return false;
1191 }
1192 GroupStoreIdMapKey that = (GroupStoreIdMapKey) o;
1193 return (super.equals(that) &&
1194 this.groupId.equals(that.groupId));
1195 }
1196
1197 @Override
1198 public int hashCode() {
1199 int result = 17;
1200
1201 result = 31 * result + super.hashCode() + Objects.hash(this.groupId);
1202
1203 return result;
1204 }
1205 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001206
1207 @Override
1208 public void pushGroupMetrics(DeviceId deviceId,
1209 Collection<Group> groupEntries) {
1210 boolean deviceInitialAuditStatus =
1211 deviceInitialAuditStatus(deviceId);
1212 Set<Group> southboundGroupEntries =
1213 Sets.newHashSet(groupEntries);
1214 Set<StoredGroupEntry> storedGroupEntries =
1215 Sets.newHashSet(getStoredGroups(deviceId));
1216 Set<Group> extraneousStoredEntries =
1217 Sets.newHashSet(getExtraneousGroups(deviceId));
1218
1219 log.trace("pushGroupMetrics: Displaying all ({}) southboundGroupEntries for device {}",
1220 southboundGroupEntries.size(),
1221 deviceId);
1222 for (Iterator<Group> it = southboundGroupEntries.iterator(); it.hasNext();) {
1223 Group group = it.next();
1224 log.trace("Group {} in device {}", group, deviceId);
1225 }
1226
1227 log.trace("Displaying all ({}) stored group entries for device {}",
1228 storedGroupEntries.size(),
1229 deviceId);
1230 for (Iterator<StoredGroupEntry> it1 = storedGroupEntries.iterator();
1231 it1.hasNext();) {
1232 Group group = it1.next();
1233 log.trace("Stored Group {} for device {}", group, deviceId);
1234 }
1235
1236 for (Iterator<Group> it2 = southboundGroupEntries.iterator(); it2.hasNext();) {
1237 Group group = it2.next();
1238 if (storedGroupEntries.remove(group)) {
1239 // we both have the group, let's update some info then.
1240 log.trace("Group AUDIT: group {} exists in both planes for device {}",
1241 group.id(), deviceId);
1242 groupAdded(group);
1243 it2.remove();
1244 }
1245 }
1246 for (Group group : southboundGroupEntries) {
1247 if (getGroup(group.deviceId(), group.id()) != null) {
1248 // There is a group existing with the same id
1249 // It is possible that group update is
1250 // in progress while we got a stale info from switch
1251 if (!storedGroupEntries.remove(getGroup(
1252 group.deviceId(), group.id()))) {
1253 log.warn("Group AUDIT: Inconsistent state:"
1254 + "Group exists in ID based table while "
1255 + "not present in key based table");
1256 }
1257 } else {
1258 // there are groups in the switch that aren't in the store
1259 log.debug("Group AUDIT: extraneous group {} exists in data plane for device {}",
1260 group.id(), deviceId);
1261 extraneousStoredEntries.remove(group);
1262 extraneousGroup(group);
1263 }
1264 }
1265 for (Group group : storedGroupEntries) {
1266 // there are groups in the store that aren't in the switch
1267 log.debug("Group AUDIT: group {} missing in data plane for device {}",
1268 group.id(), deviceId);
1269 groupMissing(group);
1270 }
1271 for (Group group : extraneousStoredEntries) {
1272 // there are groups in the extraneous store that
1273 // aren't in the switch
Saurav Das0fd79d92016-03-07 10:58:36 -08001274 log.debug("Group AUDIT: clearing extraneous group {} from store for device {}",
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001275 group.id(), deviceId);
1276 removeExtraneousGroupEntry(group);
1277 }
1278
1279 if (!deviceInitialAuditStatus) {
Saurav Das0fd79d92016-03-07 10:58:36 -08001280 log.info("Group AUDIT: Setting device {} initial AUDIT completed",
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001281 deviceId);
1282 deviceInitialAuditCompleted(deviceId, true);
1283 }
1284 }
1285
1286 private void groupMissing(Group group) {
1287 switch (group.state()) {
1288 case PENDING_DELETE:
1289 log.debug("Group {} delete confirmation from device {}",
1290 group, group.deviceId());
1291 removeGroupEntry(group);
1292 break;
1293 case ADDED:
1294 case PENDING_ADD:
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -07001295 case PENDING_ADD_RETRY:
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001296 case PENDING_UPDATE:
1297 log.debug("Group {} is in store but not on device {}",
1298 group, group.deviceId());
1299 StoredGroupEntry existing =
1300 getStoredGroupEntry(group.deviceId(), group.id());
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -07001301 log.debug("groupMissing: group entry {} in device {} moving from {} to PENDING_ADD_RETRY",
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001302 existing.id(),
1303 existing.deviceId(),
1304 existing.state());
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -07001305 existing.setState(Group.GroupState.PENDING_ADD_RETRY);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001306 //Re-PUT map entries to trigger map update events
1307 getGroupStoreKeyMap().
1308 put(new GroupStoreKeyMapKey(existing.deviceId(),
1309 existing.appCookie()), existing);
1310 notifyDelegate(new GroupEvent(GroupEvent.Type.GROUP_ADD_REQUESTED,
1311 group));
1312 break;
1313 default:
1314 log.debug("Group {} has not been installed.", group);
1315 break;
1316 }
1317 }
1318
1319 private void extraneousGroup(Group group) {
Saurav Das0fd79d92016-03-07 10:58:36 -08001320 log.trace("Group {} is on device {} but not in store.",
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001321 group, group.deviceId());
1322 addOrUpdateExtraneousGroupEntry(group);
1323 }
1324
1325 private void groupAdded(Group group) {
1326 log.trace("Group {} Added or Updated in device {}",
1327 group, group.deviceId());
1328 addOrUpdateGroupEntry(group);
1329 }
alshabib10580802015-02-18 18:30:33 -08001330}