blob: dc8ac584547ede2f00f56788e32e05dba0f1ae93 [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;
alshabib10580802015-02-18 18:30:33 -080031import org.onlab.util.NewConcurrentHashMap;
alshabibb0285992016-03-28 23:30:37 -070032import org.onosproject.cfg.ComponentConfigService;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070033import org.onosproject.cluster.ClusterService;
Charles Chanf4838a72015-12-07 18:13:45 -080034import org.onosproject.cluster.NodeId;
alshabib10580802015-02-18 18:30:33 -080035import org.onosproject.core.DefaultGroupId;
36import org.onosproject.core.GroupId;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070037import org.onosproject.mastership.MastershipService;
alshabib10580802015-02-18 18:30:33 -080038import org.onosproject.net.DeviceId;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070039import org.onosproject.net.MastershipRole;
alshabib10580802015-02-18 18:30:33 -080040import org.onosproject.net.group.DefaultGroup;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070041import org.onosproject.net.group.DefaultGroupBucket;
alshabib10580802015-02-18 18:30:33 -080042import org.onosproject.net.group.DefaultGroupDescription;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070043import org.onosproject.net.group.DefaultGroupKey;
alshabib10580802015-02-18 18:30:33 -080044import org.onosproject.net.group.Group;
45import org.onosproject.net.group.Group.GroupState;
46import org.onosproject.net.group.GroupBucket;
47import org.onosproject.net.group.GroupBuckets;
48import org.onosproject.net.group.GroupDescription;
49import org.onosproject.net.group.GroupEvent;
50import org.onosproject.net.group.GroupEvent.Type;
51import org.onosproject.net.group.GroupKey;
52import org.onosproject.net.group.GroupOperation;
53import org.onosproject.net.group.GroupStore;
54import org.onosproject.net.group.GroupStoreDelegate;
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -070055import org.onosproject.net.group.StoredGroupBucketEntry;
alshabib10580802015-02-18 18:30:33 -080056import org.onosproject.net.group.StoredGroupEntry;
57import org.onosproject.store.AbstractStore;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070058import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070059import org.onosproject.store.serializers.KryoNamespaces;
Madan Jampani0b847532016-03-03 13:44:15 -080060import org.onosproject.store.service.ConsistentMap;
61import org.onosproject.store.service.MapEvent;
62import org.onosproject.store.service.MapEventListener;
alshabibb0285992016-03-28 23:30:37 -070063import org.onosproject.store.service.MultiValuedTimestamp;
Madan Jampani0b847532016-03-03 13:44:15 -080064import org.onosproject.store.service.Serializer;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070065import org.onosproject.store.service.StorageService;
Madan Jampani0b847532016-03-03 13:44:15 -080066import org.onosproject.store.service.Versioned;
alshabibb0285992016-03-28 23:30:37 -070067import org.osgi.service.component.ComponentContext;
alshabib10580802015-02-18 18:30:33 -080068import org.slf4j.Logger;
69
Jonathan Hart6ec029a2015-03-24 17:12:35 -070070import java.util.ArrayList;
Srikanth Vavilapalli23181912015-05-04 09:48:09 -070071import java.util.Collection;
Charles Chanf4838a72015-12-07 18:13:45 -080072import java.util.Collections;
alshabibb0285992016-03-28 23:30:37 -070073import java.util.Dictionary;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070074import java.util.HashMap;
Charles Chan0c7c43b2016-01-14 17:39:20 -080075import java.util.HashSet;
Srikanth Vavilapalli23181912015-05-04 09:48:09 -070076import java.util.Iterator;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070077import java.util.List;
Madan Jampani0b847532016-03-03 13:44:15 -080078import java.util.Map;
Charles Chan0c7c43b2016-01-14 17:39:20 -080079import java.util.Map.Entry;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070080import java.util.Objects;
Sho SHIMIZU30d639b2015-05-05 09:30:35 -070081import java.util.Optional;
alshabibb0285992016-03-28 23:30:37 -070082import java.util.Properties;
Srikanth Vavilapalli23181912015-05-04 09:48:09 -070083import java.util.Set;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070084import java.util.concurrent.ConcurrentHashMap;
85import java.util.concurrent.ConcurrentMap;
86import java.util.concurrent.ExecutorService;
87import java.util.concurrent.Executors;
88import java.util.concurrent.atomic.AtomicInteger;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070089import java.util.stream.Collectors;
90
alshabibb0285992016-03-28 23:30:37 -070091import static com.google.common.base.Strings.isNullOrEmpty;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070092import static org.apache.commons.lang3.concurrent.ConcurrentUtils.createIfAbsentUnchecked;
alshabibb0285992016-03-28 23:30:37 -070093import static org.onlab.util.Tools.get;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070094import static org.onlab.util.Tools.groupedThreads;
95import static org.slf4j.LoggerFactory.getLogger;
alshabib10580802015-02-18 18:30:33 -080096
97/**
Saurav Das0fd79d92016-03-07 10:58:36 -080098 * Manages inventory of group entries using distributed group stores from the
99 * storage service.
alshabib10580802015-02-18 18:30:33 -0800100 */
101@Component(immediate = true)
102@Service
103public class DistributedGroupStore
104 extends AbstractStore<GroupEvent, GroupStoreDelegate>
105 implements GroupStore {
106
107 private final Logger log = getLogger(getClass());
108
alshabibb0285992016-03-28 23:30:37 -0700109 private static final boolean GARBAGE_COLLECT = false;
110 private static final int GC_THRESH = 6;
111
alshabib10580802015-02-18 18:30:33 -0800112 private final int dummyId = 0xffffffff;
113 private final GroupId dummyGroupId = new DefaultGroupId(dummyId);
114
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700115 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
116 protected ClusterCommunicationService clusterCommunicator;
117
118 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
119 protected ClusterService clusterService;
120
121 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700122 protected StorageService storageService;
123
124 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700125 protected MastershipService mastershipService;
126
alshabibb0285992016-03-28 23:30:37 -0700127 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
128 protected ComponentConfigService cfgService;
129
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700130 // Per device group table with (device id + app cookie) as key
Madan Jampani0b847532016-03-03 13:44:15 -0800131 private ConsistentMap<GroupStoreKeyMapKey,
alshabibb0285992016-03-28 23:30:37 -0700132 StoredGroupEntry> groupStoreEntriesByKey = null;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700133 // Per device group table with (device id + group id) as key
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700134 private final ConcurrentMap<DeviceId, ConcurrentMap<GroupId, StoredGroupEntry>>
alshabibb0285992016-03-28 23:30:37 -0700135 groupEntriesById = new ConcurrentHashMap<>();
Madan Jampani0b847532016-03-03 13:44:15 -0800136 private ConsistentMap<GroupStoreKeyMapKey,
alshabibb0285992016-03-28 23:30:37 -0700137 StoredGroupEntry> auditPendingReqQueue = null;
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;
alshabib10580802015-02-18 18:30:33 -0800142
Sho SHIMIZU7a4087b2015-09-10 09:23:16 -0700143 private final HashMap<DeviceId, Boolean> deviceAuditStatus = new HashMap<>();
alshabib10580802015-02-18 18:30:33 -0800144
145 private final AtomicInteger groupIdGen = new AtomicInteger();
146
HIGUCHI Yuta180d70f2015-10-01 16:13:56 -0700147 private KryoNamespace clusterMsgSerializer;
148
alshabibb0285992016-03-28 23:30:37 -0700149 @Property(name = "garbageCollect", boolValue = GARBAGE_COLLECT,
150 label = "Enable group garbage collection")
151 private boolean garbageCollect = GARBAGE_COLLECT;
152
153 @Property(name = "gcThresh", intValue = GC_THRESH,
154 label = "Number of rounds for group garbage collection")
155 private int gcThresh = GC_THRESH;
156
157
alshabib10580802015-02-18 18:30:33 -0800158 @Activate
159 public void activate() {
alshabibb0285992016-03-28 23:30:37 -0700160 cfgService.registerProperties(getClass());
HIGUCHI Yuta3a84b322016-05-18 13:38:07 -0700161 KryoNamespace.Builder kryoBuilder = new KryoNamespace.Builder()
alshabibb0285992016-03-28 23:30:37 -0700162 .register(KryoNamespaces.API)
HIGUCHI Yuta3a84b322016-05-18 13:38:07 -0700163 .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
alshabibb0285992016-03-28 23:30:37 -0700164 .register(DefaultGroup.class,
165 DefaultGroupBucket.class,
166 DefaultGroupDescription.class,
167 DefaultGroupKey.class,
168 GroupDescription.Type.class,
169 Group.GroupState.class,
170 GroupBuckets.class,
alshabibb0285992016-03-28 23:30:37 -0700171 GroupStoreMessage.class,
172 GroupStoreMessage.Type.class,
173 UpdateType.class,
174 GroupStoreMessageSubjects.class,
175 MultiValuedTimestamp.class,
176 GroupStoreKeyMapKey.class,
177 GroupStoreIdMapKey.class,
178 GroupStoreMapKey.class
179 );
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700180
HIGUCHI Yuta3a84b322016-05-18 13:38:07 -0700181 clusterMsgSerializer = kryoBuilder.build("GroupStore");
182 Serializer serializer = Serializer.using(clusterMsgSerializer);
HIGUCHI Yuta180d70f2015-10-01 16:13:56 -0700183
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700184 messageHandlingExecutor = Executors.
185 newFixedThreadPool(MESSAGE_HANDLER_THREAD_POOL_SIZE,
186 groupedThreads("onos/store/group",
HIGUCHI Yutad9e01052016-04-14 09:31:42 -0700187 "message-handlers",
188 log));
Madan Jampani01e05fb2015-08-13 13:29:36 -0700189
190 clusterCommunicator.addSubscriber(GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
alshabibb0285992016-03-28 23:30:37 -0700191 clusterMsgSerializer::deserialize,
192 this::process,
193 messageHandlingExecutor);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700194
Madan Jampani0b847532016-03-03 13:44:15 -0800195 log.debug("Creating Consistent map onos-group-store-keymap");
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700196
Madan Jampani0b847532016-03-03 13:44:15 -0800197 groupStoreEntriesByKey = storageService.<GroupStoreKeyMapKey, StoredGroupEntry>consistentMapBuilder()
198 .withName("onos-group-store-keymap")
HIGUCHI Yuta3a84b322016-05-18 13:38:07 -0700199 .withSerializer(serializer)
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700200 .build();
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700201 groupStoreEntriesByKey.addListener(new GroupStoreKeyMapListener());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700202 log.debug("Current size of groupstorekeymap:{}",
203 groupStoreEntriesByKey.size());
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700204
Madan Jampani0b847532016-03-03 13:44:15 -0800205 log.debug("Creating Consistent map pendinggroupkeymap");
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700206
Madan Jampani0b847532016-03-03 13:44:15 -0800207 auditPendingReqQueue = storageService.<GroupStoreKeyMapKey, StoredGroupEntry>consistentMapBuilder()
208 .withName("onos-pending-group-keymap")
HIGUCHI Yuta3a84b322016-05-18 13:38:07 -0700209 .withSerializer(serializer)
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700210 .build();
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700211 log.debug("Current size of pendinggroupkeymap:{}",
212 auditPendingReqQueue.size());
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700213
alshabib10580802015-02-18 18:30:33 -0800214 log.info("Started");
215 }
216
217 @Deactivate
218 public void deactivate() {
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 private static NewConcurrentHashMap<GroupId, Group>
alshabibb0285992016-03-28 23:30:37 -0700241 lazyEmptyExtraneousGroupIdTable() {
alshabib10580802015-02-18 18:30:33 -0800242 return NewConcurrentHashMap.<GroupId, Group>ifNeeded();
243 }
244
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700245 private static NewConcurrentHashMap<GroupId, StoredGroupEntry>
alshabibb0285992016-03-28 23:30:37 -0700246 lazyEmptyGroupIdTable() {
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700247 return NewConcurrentHashMap.<GroupId, StoredGroupEntry>ifNeeded();
248 }
249
alshabib10580802015-02-18 18:30:33 -0800250 /**
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700251 * Returns the group store eventual consistent key map.
alshabib10580802015-02-18 18:30:33 -0800252 *
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700253 * @return Map representing group key table.
alshabib10580802015-02-18 18:30:33 -0800254 */
Madan Jampani0b847532016-03-03 13:44:15 -0800255 private Map<GroupStoreKeyMapKey, StoredGroupEntry>
alshabibb0285992016-03-28 23:30:37 -0700256 getGroupStoreKeyMap() {
Madan Jampani0b847532016-03-03 13:44:15 -0800257 return groupStoreEntriesByKey.asJavaMap();
alshabib10580802015-02-18 18:30:33 -0800258 }
259
260 /**
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700261 * Returns the group id table for specified device.
alshabib10580802015-02-18 18:30:33 -0800262 *
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700263 * @param deviceId identifier of the device
264 * @return Map representing group key table of given device.
alshabib10580802015-02-18 18:30:33 -0800265 */
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700266 private ConcurrentMap<GroupId, StoredGroupEntry> getGroupIdTable(DeviceId deviceId) {
267 return createIfAbsentUnchecked(groupEntriesById,
268 deviceId, lazyEmptyGroupIdTable());
alshabib10580802015-02-18 18:30:33 -0800269 }
270
271 /**
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700272 * Returns the pending group request table.
alshabib10580802015-02-18 18:30:33 -0800273 *
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700274 * @return Map representing group key table.
alshabib10580802015-02-18 18:30:33 -0800275 */
Madan Jampani0b847532016-03-03 13:44:15 -0800276 private Map<GroupStoreKeyMapKey, StoredGroupEntry>
alshabibb0285992016-03-28 23:30:37 -0700277 getPendingGroupKeyTable() {
Madan Jampani0b847532016-03-03 13:44:15 -0800278 return auditPendingReqQueue.asJavaMap();
alshabib10580802015-02-18 18:30:33 -0800279 }
280
281 /**
282 * Returns the extraneous group id table for specified device.
283 *
284 * @param deviceId identifier of the device
285 * @return Map representing group key table of given device.
286 */
287 private ConcurrentMap<GroupId, Group>
288 getExtraneousGroupIdTable(DeviceId deviceId) {
289 return createIfAbsentUnchecked(extraneousGroupEntriesById,
290 deviceId,
291 lazyEmptyExtraneousGroupIdTable());
292 }
293
294 /**
295 * Returns the number of groups for the specified device in the store.
296 *
297 * @return number of groups for the specified device
298 */
299 @Override
300 public int getGroupCount(DeviceId deviceId) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700301 return (getGroups(deviceId) != null) ?
alshabibb0285992016-03-28 23:30:37 -0700302 Iterables.size(getGroups(deviceId)) : 0;
alshabib10580802015-02-18 18:30:33 -0800303 }
304
305 /**
306 * Returns the groups associated with a device.
307 *
308 * @param deviceId the device ID
alshabib10580802015-02-18 18:30:33 -0800309 * @return the group entries
310 */
311 @Override
312 public Iterable<Group> getGroups(DeviceId deviceId) {
Charles Chanf4838a72015-12-07 18:13:45 -0800313 // Let ImmutableSet.copyOf do the type conversion
314 return ImmutableSet.copyOf(getStoredGroups(deviceId));
alshabib10580802015-02-18 18:30:33 -0800315 }
316
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700317 private Iterable<StoredGroupEntry> getStoredGroups(DeviceId deviceId) {
Charles Chanf4838a72015-12-07 18:13:45 -0800318 NodeId master = mastershipService.getMasterFor(deviceId);
319 if (master == null) {
320 log.debug("Failed to getGroups: No master for {}", deviceId);
321 return Collections.emptySet();
322 }
323
324 Set<StoredGroupEntry> storedGroups = getGroupStoreKeyMap().values()
325 .stream()
326 .filter(input -> input.deviceId().equals(deviceId))
327 .collect(Collectors.toSet());
328 return ImmutableSet.copyOf(storedGroups);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700329 }
330
alshabib10580802015-02-18 18:30:33 -0800331 /**
332 * Returns the stored group entry.
333 *
alshabibb0285992016-03-28 23:30:37 -0700334 * @param deviceId the device ID
alshabib10580802015-02-18 18:30:33 -0800335 * @param appCookie the group key
alshabib10580802015-02-18 18:30:33 -0800336 * @return a group associated with the key
337 */
338 @Override
339 public Group getGroup(DeviceId deviceId, GroupKey appCookie) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700340 return getStoredGroupEntry(deviceId, appCookie);
341 }
342
343 private StoredGroupEntry getStoredGroupEntry(DeviceId deviceId,
344 GroupKey appCookie) {
345 return getGroupStoreKeyMap().get(new GroupStoreKeyMapKey(deviceId,
346 appCookie));
347 }
348
349 @Override
350 public Group getGroup(DeviceId deviceId, GroupId groupId) {
351 return getStoredGroupEntry(deviceId, groupId);
352 }
353
354 private StoredGroupEntry getStoredGroupEntry(DeviceId deviceId,
355 GroupId groupId) {
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700356 return getGroupIdTable(deviceId).get(groupId);
alshabib10580802015-02-18 18:30:33 -0800357 }
358
359 private int getFreeGroupIdValue(DeviceId deviceId) {
360 int freeId = groupIdGen.incrementAndGet();
361
362 while (true) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700363 Group existing = getGroup(deviceId, new DefaultGroupId(freeId));
alshabib10580802015-02-18 18:30:33 -0800364 if (existing == null) {
365 existing = (
366 extraneousGroupEntriesById.get(deviceId) != null) ?
367 extraneousGroupEntriesById.get(deviceId).
368 get(new DefaultGroupId(freeId)) :
369 null;
370 }
371 if (existing != null) {
372 freeId = groupIdGen.incrementAndGet();
373 } else {
374 break;
375 }
376 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700377 log.debug("getFreeGroupIdValue: Next Free ID is {}", freeId);
alshabib10580802015-02-18 18:30:33 -0800378 return freeId;
379 }
380
381 /**
382 * Stores a new group entry using the information from group description.
383 *
384 * @param groupDesc group description to be used to create group entry
385 */
386 @Override
387 public void storeGroupDescription(GroupDescription groupDesc) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700388 log.debug("In storeGroupDescription");
alshabib10580802015-02-18 18:30:33 -0800389 // Check if a group is existing with the same key
Saurav Das8a0732e2015-11-20 15:27:53 -0800390 Group existingGroup = getGroup(groupDesc.deviceId(), groupDesc.appCookie());
391 if (existingGroup != null) {
Charles Chan216e3c82016-04-23 14:48:16 -0700392 log.info("Group already exists with the same key {} in dev:{} with id:0x{}",
Saurav Das8a0732e2015-11-20 15:27:53 -0800393 groupDesc.appCookie(), groupDesc.deviceId(),
394 Integer.toHexString(existingGroup.id().id()));
alshabib10580802015-02-18 18:30:33 -0800395 return;
396 }
397
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700398 // Check if group to be created by a remote instance
Madan Jampani175e8fd2015-05-20 14:10:45 -0700399 if (mastershipService.getLocalRole(groupDesc.deviceId()) != MastershipRole.MASTER) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700400 log.debug("storeGroupDescription: Device {} local role is not MASTER",
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700401 groupDesc.deviceId());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700402 if (mastershipService.getMasterFor(groupDesc.deviceId()) == null) {
403 log.error("No Master for device {}..."
alshabibb0285992016-03-28 23:30:37 -0700404 + "Can not perform add group operation",
405 groupDesc.deviceId());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700406 //TODO: Send Group operation failure event
407 return;
408 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700409 GroupStoreMessage groupOp = GroupStoreMessage.
410 createGroupAddRequestMsg(groupDesc.deviceId(),
411 groupDesc);
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700412
Madan Jampani175e8fd2015-05-20 14:10:45 -0700413 clusterCommunicator.unicast(groupOp,
alshabibb0285992016-03-28 23:30:37 -0700414 GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
415 clusterMsgSerializer::serialize,
416 mastershipService.getMasterFor(groupDesc.deviceId()))
417 .whenComplete((result, error) -> {
Madan Jampani175e8fd2015-05-20 14:10:45 -0700418 if (error != null) {
419 log.warn("Failed to send request to master: {} to {}",
alshabibb0285992016-03-28 23:30:37 -0700420 groupOp,
421 mastershipService.getMasterFor(groupDesc.deviceId()));
Madan Jampani175e8fd2015-05-20 14:10:45 -0700422 //TODO: Send Group operation failure event
423 } else {
424 log.debug("Sent Group operation request for device {} "
alshabibb0285992016-03-28 23:30:37 -0700425 + "to remote MASTER {}",
426 groupDesc.deviceId(),
427 mastershipService.getMasterFor(groupDesc.deviceId()));
Madan Jampani175e8fd2015-05-20 14:10:45 -0700428 }
429 });
alshabib10580802015-02-18 18:30:33 -0800430 return;
431 }
432
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700433 log.debug("Store group for device {} is getting handled locally",
434 groupDesc.deviceId());
alshabib10580802015-02-18 18:30:33 -0800435 storeGroupDescriptionInternal(groupDesc);
436 }
437
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700438 private Group getMatchingExtraneousGroupbyId(DeviceId deviceId, Integer groupId) {
439 ConcurrentMap<GroupId, Group> extraneousMap =
440 extraneousGroupEntriesById.get(deviceId);
441 if (extraneousMap == null) {
442 return null;
443 }
444 return extraneousMap.get(new DefaultGroupId(groupId));
445 }
446
447 private Group getMatchingExtraneousGroupbyBuckets(DeviceId deviceId,
448 GroupBuckets buckets) {
449 ConcurrentMap<GroupId, Group> extraneousMap =
450 extraneousGroupEntriesById.get(deviceId);
451 if (extraneousMap == null) {
452 return null;
453 }
454
alshabibb0285992016-03-28 23:30:37 -0700455 for (Group extraneousGroup : extraneousMap.values()) {
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700456 if (extraneousGroup.buckets().equals(buckets)) {
457 return extraneousGroup;
458 }
459 }
460 return null;
461 }
462
alshabib10580802015-02-18 18:30:33 -0800463 private void storeGroupDescriptionInternal(GroupDescription groupDesc) {
464 // Check if a group is existing with the same key
465 if (getGroup(groupDesc.deviceId(), groupDesc.appCookie()) != null) {
466 return;
467 }
468
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700469 if (deviceAuditStatus.get(groupDesc.deviceId()) == null) {
470 // Device group audit has not completed yet
471 // Add this group description to pending group key table
472 // Create a group entry object with Dummy Group ID
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700473 log.debug("storeGroupDescriptionInternal: Device {} AUDIT pending...Queuing Group ADD request",
alshabibb0285992016-03-28 23:30:37 -0700474 groupDesc.deviceId());
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700475 StoredGroupEntry group = new DefaultGroup(dummyGroupId, groupDesc);
476 group.setState(GroupState.WAITING_AUDIT_COMPLETE);
Madan Jampani0b847532016-03-03 13:44:15 -0800477 Map<GroupStoreKeyMapKey, StoredGroupEntry> pendingKeyTable =
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700478 getPendingGroupKeyTable();
479 pendingKeyTable.put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
480 groupDesc.appCookie()),
481 group);
482 return;
483 }
484
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700485 Group matchingExtraneousGroup = null;
486 if (groupDesc.givenGroupId() != null) {
487 //Check if there is a extraneous group existing with the same Id
488 matchingExtraneousGroup = getMatchingExtraneousGroupbyId(
alshabibb0285992016-03-28 23:30:37 -0700489 groupDesc.deviceId(), groupDesc.givenGroupId());
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700490 if (matchingExtraneousGroup != null) {
Saurav Das0fd79d92016-03-07 10:58:36 -0800491 log.debug("storeGroupDescriptionInternal: Matching extraneous group "
alshabibb0285992016-03-28 23:30:37 -0700492 + "found in Device {} for group id 0x{}",
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700493 groupDesc.deviceId(),
Saurav Das0fd79d92016-03-07 10:58:36 -0800494 Integer.toHexString(groupDesc.givenGroupId()));
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700495 //Check if the group buckets matches with user provided buckets
496 if (matchingExtraneousGroup.buckets().equals(groupDesc.buckets())) {
497 //Group is already existing with the same buckets and Id
498 // Create a group entry object
Saurav Das0fd79d92016-03-07 10:58:36 -0800499 log.debug("storeGroupDescriptionInternal: Buckets also matching "
alshabibb0285992016-03-28 23:30:37 -0700500 + "in Device {} for group id 0x{}",
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700501 groupDesc.deviceId(),
Saurav Das0fd79d92016-03-07 10:58:36 -0800502 Integer.toHexString(groupDesc.givenGroupId()));
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700503 StoredGroupEntry group = new DefaultGroup(
alshabibb0285992016-03-28 23:30:37 -0700504 matchingExtraneousGroup.id(), groupDesc);
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700505 // Insert the newly created group entry into key and id maps
506 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -0700507 put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
508 groupDesc.appCookie()), group);
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700509 // Ensure it also inserted into group id based table to
510 // avoid any chances of duplication in group id generation
511 getGroupIdTable(groupDesc.deviceId()).
alshabibb0285992016-03-28 23:30:37 -0700512 put(matchingExtraneousGroup.id(), group);
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700513 addOrUpdateGroupEntry(matchingExtraneousGroup);
514 removeExtraneousGroupEntry(matchingExtraneousGroup);
515 return;
516 } else {
517 //Group buckets are not matching. Update group
518 //with user provided buckets.
Saurav Das0fd79d92016-03-07 10:58:36 -0800519 log.debug("storeGroupDescriptionInternal: Buckets are not "
alshabibb0285992016-03-28 23:30:37 -0700520 + "matching in Device {} for group id 0x{}",
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700521 groupDesc.deviceId(),
Saurav Das0fd79d92016-03-07 10:58:36 -0800522 Integer.toHexString(groupDesc.givenGroupId()));
523 StoredGroupEntry modifiedGroup = new DefaultGroup(
alshabibb0285992016-03-28 23:30:37 -0700524 matchingExtraneousGroup.id(), groupDesc);
Saurav Das0fd79d92016-03-07 10:58:36 -0800525 modifiedGroup.setState(GroupState.PENDING_UPDATE);
526 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -0700527 put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
528 groupDesc.appCookie()), modifiedGroup);
Saurav Das0fd79d92016-03-07 10:58:36 -0800529 // Ensure it also inserted into group id based table to
530 // avoid any chances of duplication in group id generation
531 getGroupIdTable(groupDesc.deviceId()).
alshabibb0285992016-03-28 23:30:37 -0700532 put(matchingExtraneousGroup.id(), modifiedGroup);
Saurav Das0fd79d92016-03-07 10:58:36 -0800533 removeExtraneousGroupEntry(matchingExtraneousGroup);
534 log.debug("storeGroupDescriptionInternal: Triggering Group "
alshabibb0285992016-03-28 23:30:37 -0700535 + "UPDATE request for {} in device {}",
Saurav Das0fd79d92016-03-07 10:58:36 -0800536 matchingExtraneousGroup.id(),
537 groupDesc.deviceId());
538 notifyDelegate(new GroupEvent(Type.GROUP_UPDATE_REQUESTED, modifiedGroup));
539 return;
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700540 }
541 }
542 } else {
543 //Check if there is an extraneous group with user provided buckets
544 matchingExtraneousGroup = getMatchingExtraneousGroupbyBuckets(
alshabibb0285992016-03-28 23:30:37 -0700545 groupDesc.deviceId(), groupDesc.buckets());
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700546 if (matchingExtraneousGroup != null) {
547 //Group is already existing with the same buckets.
548 //So reuse this group.
549 log.debug("storeGroupDescriptionInternal: Matching extraneous group found in Device {}",
550 groupDesc.deviceId());
551 //Create a group entry object
552 StoredGroupEntry group = new DefaultGroup(
alshabibb0285992016-03-28 23:30:37 -0700553 matchingExtraneousGroup.id(), groupDesc);
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700554 // Insert the newly created group entry into key and id maps
555 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -0700556 put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
557 groupDesc.appCookie()), group);
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700558 // Ensure it also inserted into group id based table to
559 // avoid any chances of duplication in group id generation
560 getGroupIdTable(groupDesc.deviceId()).
alshabibb0285992016-03-28 23:30:37 -0700561 put(matchingExtraneousGroup.id(), group);
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -0700562 addOrUpdateGroupEntry(matchingExtraneousGroup);
563 removeExtraneousGroupEntry(matchingExtraneousGroup);
564 return;
565 } else {
566 //TODO: Check if there are any empty groups that can be used here
567 log.debug("storeGroupDescriptionInternal: No matching extraneous groups found in Device {}",
568 groupDesc.deviceId());
569 }
570 }
571
Saurav Das100e3b82015-04-30 11:12:10 -0700572 GroupId id = null;
573 if (groupDesc.givenGroupId() == null) {
574 // Get a new group identifier
575 id = new DefaultGroupId(getFreeGroupIdValue(groupDesc.deviceId()));
576 } else {
Saurav Das8be4e3a2016-03-11 17:19:07 -0800577 // we need to use the identifier passed in by caller, but check if
578 // already used
579 Group existing = getGroup(groupDesc.deviceId(),
580 new DefaultGroupId(groupDesc.givenGroupId()));
581 if (existing != null) {
582 log.warn("Group already exists with the same id: 0x{} in dev:{} "
alshabibb0285992016-03-28 23:30:37 -0700583 + "but with different key: {} (request gkey: {})",
584 Integer.toHexString(groupDesc.givenGroupId()),
585 groupDesc.deviceId(),
586 existing.appCookie(),
587 groupDesc.appCookie());
Saurav Das8be4e3a2016-03-11 17:19:07 -0800588 return;
589 }
Saurav Das100e3b82015-04-30 11:12:10 -0700590 id = new DefaultGroupId(groupDesc.givenGroupId());
591 }
alshabib10580802015-02-18 18:30:33 -0800592 // Create a group entry object
593 StoredGroupEntry group = new DefaultGroup(id, groupDesc);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700594 // Insert the newly created group entry into key and id maps
595 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -0700596 put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
597 groupDesc.appCookie()), group);
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700598 // Ensure it also inserted into group id based table to
599 // avoid any chances of duplication in group id generation
600 getGroupIdTable(groupDesc.deviceId()).
alshabibb0285992016-03-28 23:30:37 -0700601 put(id, group);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700602 log.debug("storeGroupDescriptionInternal: Processing Group ADD request for Id {} in device {}",
alshabibb0285992016-03-28 23:30:37 -0700603 id,
604 groupDesc.deviceId());
alshabib10580802015-02-18 18:30:33 -0800605 notifyDelegate(new GroupEvent(GroupEvent.Type.GROUP_ADD_REQUESTED,
606 group));
607 }
608
609 /**
610 * Updates the existing group entry with the information
611 * from group description.
612 *
alshabibb0285992016-03-28 23:30:37 -0700613 * @param deviceId the device ID
alshabib10580802015-02-18 18:30:33 -0800614 * @param oldAppCookie the current group key
alshabibb0285992016-03-28 23:30:37 -0700615 * @param type update type
616 * @param newBuckets group buckets for updates
alshabib10580802015-02-18 18:30:33 -0800617 * @param newAppCookie optional new group key
618 */
619 @Override
620 public void updateGroupDescription(DeviceId deviceId,
621 GroupKey oldAppCookie,
622 UpdateType type,
623 GroupBuckets newBuckets,
624 GroupKey newAppCookie) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700625 // Check if group update to be done by a remote instance
sangho52abe3a2015-05-05 14:13:34 -0700626 if (mastershipService.getMasterFor(deviceId) != null &&
627 mastershipService.getLocalRole(deviceId) != MastershipRole.MASTER) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700628 log.debug("updateGroupDescription: Device {} local role is not MASTER",
629 deviceId);
630 if (mastershipService.getMasterFor(deviceId) == null) {
631 log.error("No Master for device {}..."
alshabibb0285992016-03-28 23:30:37 -0700632 + "Can not perform update group operation",
633 deviceId);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700634 //TODO: Send Group operation failure event
635 return;
636 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700637 GroupStoreMessage groupOp = GroupStoreMessage.
638 createGroupUpdateRequestMsg(deviceId,
639 oldAppCookie,
640 type,
641 newBuckets,
642 newAppCookie);
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700643
Madan Jampani175e8fd2015-05-20 14:10:45 -0700644 clusterCommunicator.unicast(groupOp,
alshabibb0285992016-03-28 23:30:37 -0700645 GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
646 clusterMsgSerializer::serialize,
647 mastershipService.getMasterFor(deviceId)).whenComplete((result, error) -> {
648 if (error != null) {
649 log.warn("Failed to send request to master: {} to {}",
650 groupOp,
651 mastershipService.getMasterFor(deviceId), error);
652 }
653 //TODO: Send Group operation failure event
654 });
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700655 return;
656 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700657 log.debug("updateGroupDescription for device {} is getting handled locally",
658 deviceId);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700659 updateGroupDescriptionInternal(deviceId,
660 oldAppCookie,
661 type,
662 newBuckets,
663 newAppCookie);
664 }
665
666 private void updateGroupDescriptionInternal(DeviceId deviceId,
alshabibb0285992016-03-28 23:30:37 -0700667 GroupKey oldAppCookie,
668 UpdateType type,
669 GroupBuckets newBuckets,
670 GroupKey newAppCookie) {
alshabib10580802015-02-18 18:30:33 -0800671 // Check if a group is existing with the provided key
672 Group oldGroup = getGroup(deviceId, oldAppCookie);
673 if (oldGroup == null) {
Saurav Das8be4e3a2016-03-11 17:19:07 -0800674 log.warn("updateGroupDescriptionInternal: Group not found...strange. "
alshabibb0285992016-03-28 23:30:37 -0700675 + "GroupKey:{} DeviceId:{}", oldAppCookie, deviceId);
alshabib10580802015-02-18 18:30:33 -0800676 return;
677 }
678
679 List<GroupBucket> newBucketList = getUpdatedBucketList(oldGroup,
680 type,
681 newBuckets);
682 if (newBucketList != null) {
683 // Create a new group object from the old group
684 GroupBuckets updatedBuckets = new GroupBuckets(newBucketList);
685 GroupKey newCookie = (newAppCookie != null) ? newAppCookie : oldAppCookie;
686 GroupDescription updatedGroupDesc = new DefaultGroupDescription(
687 oldGroup.deviceId(),
688 oldGroup.type(),
689 updatedBuckets,
690 newCookie,
Saurav Das100e3b82015-04-30 11:12:10 -0700691 oldGroup.givenGroupId(),
alshabib10580802015-02-18 18:30:33 -0800692 oldGroup.appId());
693 StoredGroupEntry newGroup = new DefaultGroup(oldGroup.id(),
694 updatedGroupDesc);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700695 log.debug("updateGroupDescriptionInternal: group entry {} in device {} moving from {} to PENDING_UPDATE",
alshabibb0285992016-03-28 23:30:37 -0700696 oldGroup.id(),
697 oldGroup.deviceId(),
698 oldGroup.state());
alshabib10580802015-02-18 18:30:33 -0800699 newGroup.setState(GroupState.PENDING_UPDATE);
700 newGroup.setLife(oldGroup.life());
701 newGroup.setPackets(oldGroup.packets());
702 newGroup.setBytes(oldGroup.bytes());
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700703 //Update the group entry in groupkey based map.
704 //Update to groupid based map will happen in the
705 //groupkey based map update listener
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700706 log.debug("updateGroupDescriptionInternal with type {}: Group updated with buckets",
707 type);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700708 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -0700709 put(new GroupStoreKeyMapKey(newGroup.deviceId(),
710 newGroup.appCookie()), newGroup);
alshabib10580802015-02-18 18:30:33 -0800711 notifyDelegate(new GroupEvent(Type.GROUP_UPDATE_REQUESTED, newGroup));
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700712 } else {
713 log.warn("updateGroupDescriptionInternal with type {}: No "
alshabibb0285992016-03-28 23:30:37 -0700714 + "change in the buckets in update", type);
alshabib10580802015-02-18 18:30:33 -0800715 }
716 }
717
718 private List<GroupBucket> getUpdatedBucketList(Group oldGroup,
719 UpdateType type,
720 GroupBuckets buckets) {
721 GroupBuckets oldBuckets = oldGroup.buckets();
Sho SHIMIZU7a4087b2015-09-10 09:23:16 -0700722 List<GroupBucket> newBucketList = new ArrayList<>(oldBuckets.buckets());
alshabib10580802015-02-18 18:30:33 -0800723 boolean groupDescUpdated = false;
724
725 if (type == UpdateType.ADD) {
726 // Check if the any of the new buckets are part of
727 // the old bucket list
alshabibb0285992016-03-28 23:30:37 -0700728 for (GroupBucket addBucket : buckets.buckets()) {
alshabib10580802015-02-18 18:30:33 -0800729 if (!newBucketList.contains(addBucket)) {
730 newBucketList.add(addBucket);
731 groupDescUpdated = true;
732 }
733 }
734 } else if (type == UpdateType.REMOVE) {
735 // Check if the to be removed buckets are part of the
736 // old bucket list
alshabibb0285992016-03-28 23:30:37 -0700737 for (GroupBucket removeBucket : buckets.buckets()) {
alshabib10580802015-02-18 18:30:33 -0800738 if (newBucketList.contains(removeBucket)) {
739 newBucketList.remove(removeBucket);
740 groupDescUpdated = true;
741 }
742 }
743 }
744
745 if (groupDescUpdated) {
746 return newBucketList;
747 } else {
748 return null;
749 }
750 }
751
752 /**
753 * Triggers deleting the existing group entry.
754 *
alshabibb0285992016-03-28 23:30:37 -0700755 * @param deviceId the device ID
alshabib10580802015-02-18 18:30:33 -0800756 * @param appCookie the group key
757 */
758 @Override
759 public void deleteGroupDescription(DeviceId deviceId,
760 GroupKey appCookie) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700761 // Check if group to be deleted by a remote instance
762 if (mastershipService.
763 getLocalRole(deviceId) != MastershipRole.MASTER) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700764 log.debug("deleteGroupDescription: Device {} local role is not MASTER",
765 deviceId);
766 if (mastershipService.getMasterFor(deviceId) == null) {
767 log.error("No Master for device {}..."
alshabibb0285992016-03-28 23:30:37 -0700768 + "Can not perform delete group operation",
769 deviceId);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700770 //TODO: Send Group operation failure event
771 return;
772 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700773 GroupStoreMessage groupOp = GroupStoreMessage.
774 createGroupDeleteRequestMsg(deviceId,
775 appCookie);
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700776
Madan Jampani175e8fd2015-05-20 14:10:45 -0700777 clusterCommunicator.unicast(groupOp,
alshabibb0285992016-03-28 23:30:37 -0700778 GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
779 clusterMsgSerializer::serialize,
780 mastershipService.getMasterFor(deviceId)).whenComplete((result, error) -> {
781 if (error != null) {
782 log.warn("Failed to send request to master: {} to {}",
783 groupOp,
784 mastershipService.getMasterFor(deviceId), error);
785 }
786 //TODO: Send Group operation failure event
787 });
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700788 return;
789 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700790 log.debug("deleteGroupDescription in device {} is getting handled locally",
791 deviceId);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700792 deleteGroupDescriptionInternal(deviceId, appCookie);
793 }
794
795 private void deleteGroupDescriptionInternal(DeviceId deviceId,
796 GroupKey appCookie) {
alshabib10580802015-02-18 18:30:33 -0800797 // Check if a group is existing with the provided key
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700798 StoredGroupEntry existing = getStoredGroupEntry(deviceId, appCookie);
alshabib10580802015-02-18 18:30:33 -0800799 if (existing == null) {
800 return;
801 }
802
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700803 log.debug("deleteGroupDescriptionInternal: group entry {} in device {} moving from {} to PENDING_DELETE",
alshabibb0285992016-03-28 23:30:37 -0700804 existing.id(),
805 existing.deviceId(),
806 existing.state());
alshabib10580802015-02-18 18:30:33 -0800807 synchronized (existing) {
808 existing.setState(GroupState.PENDING_DELETE);
Saurav Das80980c72016-03-23 11:22:49 -0700809 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -0700810 put(new GroupStoreKeyMapKey(existing.deviceId(), existing.appCookie()),
811 existing);
alshabib10580802015-02-18 18:30:33 -0800812 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700813 log.debug("deleteGroupDescriptionInternal: in device {} issuing GROUP_REMOVE_REQUESTED",
814 deviceId);
alshabib10580802015-02-18 18:30:33 -0800815 notifyDelegate(new GroupEvent(Type.GROUP_REMOVE_REQUESTED, existing));
816 }
817
818 /**
819 * Stores a new group entry, or updates an existing entry.
820 *
821 * @param group group entry
822 */
823 @Override
824 public void addOrUpdateGroupEntry(Group group) {
825 // check if this new entry is an update to an existing entry
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700826 StoredGroupEntry existing = getStoredGroupEntry(group.deviceId(),
827 group.id());
alshabib10580802015-02-18 18:30:33 -0800828 GroupEvent event = null;
829
830 if (existing != null) {
Saurav Das0fd79d92016-03-07 10:58:36 -0800831 log.trace("addOrUpdateGroupEntry: updating group entry {} in device {}",
alshabibb0285992016-03-28 23:30:37 -0700832 group.id(),
833 group.deviceId());
alshabib10580802015-02-18 18:30:33 -0800834 synchronized (existing) {
alshabibb0285992016-03-28 23:30:37 -0700835 for (GroupBucket bucket : group.buckets().buckets()) {
Sho SHIMIZU30d639b2015-05-05 09:30:35 -0700836 Optional<GroupBucket> matchingBucket =
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -0700837 existing.buckets().buckets()
alshabibb0285992016-03-28 23:30:37 -0700838 .stream()
839 .filter((existingBucket) -> (existingBucket.equals(bucket)))
840 .findFirst();
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -0700841 if (matchingBucket.isPresent()) {
842 ((StoredGroupBucketEntry) matchingBucket.
843 get()).setPackets(bucket.packets());
844 ((StoredGroupBucketEntry) matchingBucket.
845 get()).setBytes(bucket.bytes());
846 } else {
847 log.warn("addOrUpdateGroupEntry: No matching "
alshabibb0285992016-03-28 23:30:37 -0700848 + "buckets to update stats");
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -0700849 }
850 }
alshabib10580802015-02-18 18:30:33 -0800851 existing.setLife(group.life());
852 existing.setPackets(group.packets());
853 existing.setBytes(group.bytes());
alshabibb0285992016-03-28 23:30:37 -0700854 existing.setReferenceCount(group.referenceCount());
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -0700855 if ((existing.state() == GroupState.PENDING_ADD) ||
alshabibb0285992016-03-28 23:30:37 -0700856 (existing.state() == GroupState.PENDING_ADD_RETRY)) {
Saurav Das0fd79d92016-03-07 10:58:36 -0800857 log.trace("addOrUpdateGroupEntry: group entry {} in device {} moving from {} to ADDED",
alshabibb0285992016-03-28 23:30:37 -0700858 existing.id(),
859 existing.deviceId(),
860 existing.state());
alshabib10580802015-02-18 18:30:33 -0800861 existing.setState(GroupState.ADDED);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700862 existing.setIsGroupStateAddedFirstTime(true);
alshabib10580802015-02-18 18:30:33 -0800863 event = new GroupEvent(Type.GROUP_ADDED, existing);
864 } else {
Saurav Das0fd79d92016-03-07 10:58:36 -0800865 log.trace("addOrUpdateGroupEntry: group entry {} in device {} moving from {} to ADDED",
alshabibb0285992016-03-28 23:30:37 -0700866 existing.id(),
867 existing.deviceId(),
868 GroupState.PENDING_UPDATE);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700869 existing.setState(GroupState.ADDED);
870 existing.setIsGroupStateAddedFirstTime(false);
alshabib10580802015-02-18 18:30:33 -0800871 event = new GroupEvent(Type.GROUP_UPDATED, existing);
872 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700873 //Re-PUT map entries to trigger map update events
874 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -0700875 put(new GroupStoreKeyMapKey(existing.deviceId(),
876 existing.appCookie()), existing);
alshabib10580802015-02-18 18:30:33 -0800877 }
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700878 } else {
879 log.warn("addOrUpdateGroupEntry: Group update "
alshabibb0285992016-03-28 23:30:37 -0700880 + "happening for a non-existing entry in the map");
alshabib10580802015-02-18 18:30:33 -0800881 }
882
883 if (event != null) {
884 notifyDelegate(event);
885 }
886 }
887
888 /**
889 * Removes the group entry from store.
890 *
891 * @param group group entry
892 */
893 @Override
894 public void removeGroupEntry(Group group) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700895 StoredGroupEntry existing = getStoredGroupEntry(group.deviceId(),
896 group.id());
alshabib10580802015-02-18 18:30:33 -0800897
898 if (existing != null) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700899 log.debug("removeGroupEntry: removing group entry {} in device {}",
alshabibb0285992016-03-28 23:30:37 -0700900 group.id(),
901 group.deviceId());
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700902 //Removal from groupid based map will happen in the
903 //map update listener
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700904 getGroupStoreKeyMap().remove(new GroupStoreKeyMapKey(existing.deviceId(),
905 existing.appCookie()));
alshabib10580802015-02-18 18:30:33 -0800906 notifyDelegate(new GroupEvent(Type.GROUP_REMOVED, existing));
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700907 } else {
908 log.warn("removeGroupEntry for {} in device{} is "
alshabibb0285992016-03-28 23:30:37 -0700909 + "not existing in our maps",
910 group.id(),
911 group.deviceId());
alshabib10580802015-02-18 18:30:33 -0800912 }
913 }
914
915 @Override
Charles Chan0c7c43b2016-01-14 17:39:20 -0800916 public void purgeGroupEntry(DeviceId deviceId) {
917 Set<Entry<GroupStoreKeyMapKey, StoredGroupEntry>> entryPendingRemove =
918 new HashSet<>();
919
Madan Jampani0b847532016-03-03 13:44:15 -0800920 getGroupStoreKeyMap().entrySet().stream()
Charles Chan0c7c43b2016-01-14 17:39:20 -0800921 .filter(entry -> entry.getKey().deviceId().equals(deviceId))
922 .forEach(entryPendingRemove::add);
923
924 entryPendingRemove.forEach(entry -> {
925 groupStoreEntriesByKey.remove(entry.getKey());
926 notifyDelegate(new GroupEvent(Type.GROUP_REMOVED, entry.getValue()));
927 });
928 }
929
930 @Override
alshabib10580802015-02-18 18:30:33 -0800931 public void deviceInitialAuditCompleted(DeviceId deviceId,
932 boolean completed) {
933 synchronized (deviceAuditStatus) {
934 if (completed) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700935 log.debug("AUDIT completed for device {}",
936 deviceId);
alshabib10580802015-02-18 18:30:33 -0800937 deviceAuditStatus.put(deviceId, true);
938 // Execute all pending group requests
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700939 List<StoredGroupEntry> pendingGroupRequests =
940 getPendingGroupKeyTable().values()
alshabibb0285992016-03-28 23:30:37 -0700941 .stream()
942 .filter(g -> g.deviceId().equals(deviceId))
943 .collect(Collectors.toList());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700944 log.debug("processing pending group add requests for device {} and number of pending requests {}",
alshabibb0285992016-03-28 23:30:37 -0700945 deviceId,
946 pendingGroupRequests.size());
947 for (Group group : pendingGroupRequests) {
alshabib10580802015-02-18 18:30:33 -0800948 GroupDescription tmp = new DefaultGroupDescription(
949 group.deviceId(),
950 group.type(),
951 group.buckets(),
952 group.appCookie(),
Saurav Das100e3b82015-04-30 11:12:10 -0700953 group.givenGroupId(),
alshabib10580802015-02-18 18:30:33 -0800954 group.appId());
955 storeGroupDescriptionInternal(tmp);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700956 getPendingGroupKeyTable().
alshabibb0285992016-03-28 23:30:37 -0700957 remove(new GroupStoreKeyMapKey(deviceId, group.appCookie()));
alshabib10580802015-02-18 18:30:33 -0800958 }
alshabib10580802015-02-18 18:30:33 -0800959 } else {
Thomas Vachuskac40d4632015-04-09 16:55:03 -0700960 Boolean audited = deviceAuditStatus.get(deviceId);
961 if (audited != null && audited) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700962 log.debug("Clearing AUDIT status for device {}", deviceId);
alshabib10580802015-02-18 18:30:33 -0800963 deviceAuditStatus.put(deviceId, false);
964 }
965 }
966 }
967 }
968
969 @Override
970 public boolean deviceInitialAuditStatus(DeviceId deviceId) {
971 synchronized (deviceAuditStatus) {
Thomas Vachuskac40d4632015-04-09 16:55:03 -0700972 Boolean audited = deviceAuditStatus.get(deviceId);
973 return audited != null && audited;
alshabib10580802015-02-18 18:30:33 -0800974 }
975 }
976
977 @Override
978 public void groupOperationFailed(DeviceId deviceId, GroupOperation operation) {
979
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700980 StoredGroupEntry existing = getStoredGroupEntry(deviceId,
981 operation.groupId());
alshabib10580802015-02-18 18:30:33 -0800982
983 if (existing == null) {
984 log.warn("No group entry with ID {} found ", operation.groupId());
985 return;
986 }
987
Srikanth Vavilapalli23181912015-05-04 09:48:09 -0700988 log.warn("groupOperationFailed: group operation {} failed"
alshabibb0285992016-03-28 23:30:37 -0700989 + "for group {} in device {} with code {}",
990 operation.opType(),
991 existing.id(),
992 existing.deviceId(),
993 operation.failureCode());
Saurav Das0fd79d92016-03-07 10:58:36 -0800994 if (operation.failureCode() == GroupOperation.GroupMsgErrorCode.GROUP_EXISTS) {
995 log.warn("Current extraneous groups in device:{} are: {}",
996 deviceId,
997 getExtraneousGroups(deviceId));
Saurav Das8be4e3a2016-03-11 17:19:07 -0800998 if (operation.buckets().equals(existing.buckets())) {
999 if (existing.state() == GroupState.PENDING_ADD) {
1000 log.info("GROUP_EXISTS: GroupID and Buckets match for group in pending "
alshabibb0285992016-03-28 23:30:37 -07001001 + "add state - moving to ADDED for group {} in device {}",
1002 existing.id(), deviceId);
Saurav Das8be4e3a2016-03-11 17:19:07 -08001003 addOrUpdateGroupEntry(existing);
1004 return;
1005 } else {
1006 log.warn("GROUP EXISTS: Group ID matched but buckets did not. "
alshabibb0285992016-03-28 23:30:37 -07001007 + "Operation: {} Existing: {}", operation.buckets(),
1008 existing.buckets());
Saurav Das8be4e3a2016-03-11 17:19:07 -08001009 }
1010 }
Saurav Das0fd79d92016-03-07 10:58:36 -08001011 }
alshabib10580802015-02-18 18:30:33 -08001012 switch (operation.opType()) {
1013 case ADD:
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -07001014 if (existing.state() == GroupState.PENDING_ADD) {
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -07001015 notifyDelegate(new GroupEvent(Type.GROUP_ADD_FAILED, existing));
1016 log.warn("groupOperationFailed: cleaningup "
alshabibb0285992016-03-28 23:30:37 -07001017 + "group {} from store in device {}....",
1018 existing.id(),
1019 existing.deviceId());
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -07001020 //Removal from groupid based map will happen in the
1021 //map update listener
1022 getGroupStoreKeyMap().remove(new GroupStoreKeyMapKey(existing.deviceId(),
1023 existing.appCookie()));
1024 }
alshabib10580802015-02-18 18:30:33 -08001025 break;
1026 case MODIFY:
1027 notifyDelegate(new GroupEvent(Type.GROUP_UPDATE_FAILED, existing));
1028 break;
1029 case DELETE:
1030 notifyDelegate(new GroupEvent(Type.GROUP_REMOVE_FAILED, existing));
1031 break;
1032 default:
1033 log.warn("Unknown group operation type {}", operation.opType());
1034 }
alshabib10580802015-02-18 18:30:33 -08001035 }
1036
1037 @Override
1038 public void addOrUpdateExtraneousGroupEntry(Group group) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001039 log.debug("add/update extraneous group entry {} in device {}",
alshabibb0285992016-03-28 23:30:37 -07001040 group.id(),
1041 group.deviceId());
alshabib10580802015-02-18 18:30:33 -08001042 ConcurrentMap<GroupId, Group> extraneousIdTable =
1043 getExtraneousGroupIdTable(group.deviceId());
1044 extraneousIdTable.put(group.id(), group);
Srikanth Vavilapallie48b3cf2015-07-06 11:43:07 -07001045 // Don't remove the extraneous groups, instead re-use it when
1046 // a group request comes with the same set of buckets
alshabib10580802015-02-18 18:30:33 -08001047 }
1048
1049 @Override
1050 public void removeExtraneousGroupEntry(Group group) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001051 log.debug("remove extraneous group entry {} of device {} from store",
alshabibb0285992016-03-28 23:30:37 -07001052 group.id(),
1053 group.deviceId());
alshabib10580802015-02-18 18:30:33 -08001054 ConcurrentMap<GroupId, Group> extraneousIdTable =
1055 getExtraneousGroupIdTable(group.deviceId());
1056 extraneousIdTable.remove(group.id());
1057 }
1058
1059 @Override
1060 public Iterable<Group> getExtraneousGroups(DeviceId deviceId) {
1061 // flatten and make iterator unmodifiable
1062 return FluentIterable.from(
1063 getExtraneousGroupIdTable(deviceId).values());
1064 }
1065
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001066 /**
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -07001067 * Map handler to receive any events when the group key map is updated.
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001068 */
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -07001069 private class GroupStoreKeyMapListener implements
Madan Jampani0b847532016-03-03 13:44:15 -08001070 MapEventListener<GroupStoreKeyMapKey, StoredGroupEntry> {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001071
1072 @Override
Madan Jampani0b847532016-03-03 13:44:15 -08001073 public void event(MapEvent<GroupStoreKeyMapKey, StoredGroupEntry> mapEvent) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001074 GroupEvent groupEvent = null;
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001075 GroupStoreKeyMapKey key = mapEvent.key();
Madan Jampani0b847532016-03-03 13:44:15 -08001076 StoredGroupEntry group = Versioned.valueOrNull(mapEvent.newValue());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001077 if ((key == null) && (group == null)) {
1078 log.error("GroupStoreKeyMapListener: Received "
alshabibb0285992016-03-28 23:30:37 -07001079 + "event {} with null entry", mapEvent.type());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001080 return;
1081 } else if (group == null) {
1082 group = getGroupIdTable(key.deviceId()).values()
1083 .stream()
1084 .filter((storedGroup) -> (storedGroup.appCookie().equals(key.appCookie)))
1085 .findFirst().get();
1086 if (group == null) {
1087 log.error("GroupStoreKeyMapListener: Received "
alshabibb0285992016-03-28 23:30:37 -07001088 + "event {} with null entry... can not process", mapEvent.type());
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001089 return;
1090 }
1091 }
1092 log.trace("received groupid map event {} for id {} in device {}",
1093 mapEvent.type(),
1094 group.id(),
1095 key.deviceId());
Madan Jampani0b847532016-03-03 13:44:15 -08001096 if (mapEvent.type() == MapEvent.Type.INSERT || mapEvent.type() == MapEvent.Type.UPDATE) {
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -07001097 // Update the group ID table
1098 getGroupIdTable(group.deviceId()).put(group.id(), group);
Madan Jampani0b847532016-03-03 13:44:15 -08001099 StoredGroupEntry value = Versioned.valueOrNull(mapEvent.newValue());
1100 if (value.state() == Group.GroupState.ADDED) {
1101 if (value.isGroupStateAddedFirstTime()) {
1102 groupEvent = new GroupEvent(Type.GROUP_ADDED, value);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001103 log.trace("Received first time GROUP_ADDED state update for id {} in device {}",
alshabibb0285992016-03-28 23:30:37 -07001104 group.id(),
1105 group.deviceId());
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001106 } else {
Madan Jampani0b847532016-03-03 13:44:15 -08001107 groupEvent = new GroupEvent(Type.GROUP_UPDATED, value);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001108 log.trace("Received following GROUP_ADDED state update for id {} in device {}",
alshabibb0285992016-03-28 23:30:37 -07001109 group.id(),
1110 group.deviceId());
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001111 }
1112 }
Madan Jampani0b847532016-03-03 13:44:15 -08001113 } else if (mapEvent.type() == MapEvent.Type.REMOVE) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001114 groupEvent = new GroupEvent(Type.GROUP_REMOVED, group);
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -07001115 // Remove the entry from the group ID table
1116 getGroupIdTable(group.deviceId()).remove(group.id(), group);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001117 }
1118
1119 if (groupEvent != null) {
1120 notifyDelegate(groupEvent);
1121 }
1122 }
1123 }
Madan Jampani01e05fb2015-08-13 13:29:36 -07001124
1125 private void process(GroupStoreMessage groupOp) {
1126 log.debug("Received remote group operation {} request for device {}",
alshabibb0285992016-03-28 23:30:37 -07001127 groupOp.type(),
1128 groupOp.deviceId());
1129 if (!mastershipService.isLocalMaster(groupOp.deviceId())) {
1130 log.warn("This node is not MASTER for device {}", groupOp.deviceId());
1131 return;
1132 }
1133 if (groupOp.type() == GroupStoreMessage.Type.ADD) {
1134 storeGroupDescriptionInternal(groupOp.groupDesc());
1135 } else if (groupOp.type() == GroupStoreMessage.Type.UPDATE) {
1136 updateGroupDescriptionInternal(groupOp.deviceId(),
1137 groupOp.appCookie(),
1138 groupOp.updateType(),
1139 groupOp.updateBuckets(),
1140 groupOp.newAppCookie());
1141 } else if (groupOp.type() == GroupStoreMessage.Type.DELETE) {
1142 deleteGroupDescriptionInternal(groupOp.deviceId(),
1143 groupOp.appCookie());
1144 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001145 }
1146
1147 /**
1148 * Flattened map key to be used to store group entries.
1149 */
Ray Milkeyb3c5ce22015-08-10 09:07:36 -07001150 protected static class GroupStoreMapKey {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001151 private final DeviceId deviceId;
1152
1153 public GroupStoreMapKey(DeviceId deviceId) {
1154 this.deviceId = deviceId;
1155 }
1156
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001157 public DeviceId deviceId() {
1158 return deviceId;
1159 }
1160
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001161 @Override
1162 public boolean equals(Object o) {
1163 if (this == o) {
1164 return true;
1165 }
1166 if (!(o instanceof GroupStoreMapKey)) {
1167 return false;
1168 }
1169 GroupStoreMapKey that = (GroupStoreMapKey) o;
1170 return this.deviceId.equals(that.deviceId);
1171 }
1172
1173 @Override
1174 public int hashCode() {
1175 int result = 17;
1176
1177 result = 31 * result + Objects.hash(this.deviceId);
1178
1179 return result;
1180 }
1181 }
1182
Ray Milkeyb3c5ce22015-08-10 09:07:36 -07001183 protected static class GroupStoreKeyMapKey extends GroupStoreMapKey {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001184 private final GroupKey appCookie;
alshabibb0285992016-03-28 23:30:37 -07001185
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001186 public GroupStoreKeyMapKey(DeviceId deviceId,
1187 GroupKey appCookie) {
1188 super(deviceId);
1189 this.appCookie = appCookie;
1190 }
1191
1192 @Override
1193 public boolean equals(Object o) {
1194 if (this == o) {
1195 return true;
1196 }
1197 if (!(o instanceof GroupStoreKeyMapKey)) {
1198 return false;
1199 }
1200 GroupStoreKeyMapKey that = (GroupStoreKeyMapKey) o;
1201 return (super.equals(that) &&
1202 this.appCookie.equals(that.appCookie));
1203 }
1204
1205 @Override
1206 public int hashCode() {
1207 int result = 17;
1208
1209 result = 31 * result + super.hashCode() + Objects.hash(this.appCookie);
1210
1211 return result;
1212 }
1213 }
1214
Ray Milkeyb3c5ce22015-08-10 09:07:36 -07001215 protected static class GroupStoreIdMapKey extends GroupStoreMapKey {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001216 private final GroupId groupId;
alshabibb0285992016-03-28 23:30:37 -07001217
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -07001218 public GroupStoreIdMapKey(DeviceId deviceId,
1219 GroupId groupId) {
1220 super(deviceId);
1221 this.groupId = groupId;
1222 }
1223
1224 @Override
1225 public boolean equals(Object o) {
1226 if (this == o) {
1227 return true;
1228 }
1229 if (!(o instanceof GroupStoreIdMapKey)) {
1230 return false;
1231 }
1232 GroupStoreIdMapKey that = (GroupStoreIdMapKey) o;
1233 return (super.equals(that) &&
1234 this.groupId.equals(that.groupId));
1235 }
1236
1237 @Override
1238 public int hashCode() {
1239 int result = 17;
1240
1241 result = 31 * result + super.hashCode() + Objects.hash(this.groupId);
1242
1243 return result;
1244 }
1245 }
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001246
1247 @Override
1248 public void pushGroupMetrics(DeviceId deviceId,
1249 Collection<Group> groupEntries) {
1250 boolean deviceInitialAuditStatus =
1251 deviceInitialAuditStatus(deviceId);
1252 Set<Group> southboundGroupEntries =
1253 Sets.newHashSet(groupEntries);
1254 Set<StoredGroupEntry> storedGroupEntries =
1255 Sets.newHashSet(getStoredGroups(deviceId));
1256 Set<Group> extraneousStoredEntries =
1257 Sets.newHashSet(getExtraneousGroups(deviceId));
1258
1259 log.trace("pushGroupMetrics: Displaying all ({}) southboundGroupEntries for device {}",
1260 southboundGroupEntries.size(),
1261 deviceId);
1262 for (Iterator<Group> it = southboundGroupEntries.iterator(); it.hasNext();) {
1263 Group group = it.next();
1264 log.trace("Group {} in device {}", group, deviceId);
1265 }
1266
1267 log.trace("Displaying all ({}) stored group entries for device {}",
1268 storedGroupEntries.size(),
1269 deviceId);
1270 for (Iterator<StoredGroupEntry> it1 = storedGroupEntries.iterator();
alshabibb0285992016-03-28 23:30:37 -07001271 it1.hasNext();) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001272 Group group = it1.next();
1273 log.trace("Stored Group {} for device {}", group, deviceId);
1274 }
1275
alshabibb0285992016-03-28 23:30:37 -07001276 garbageCollect(deviceId, southboundGroupEntries, storedGroupEntries);
1277
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001278 for (Iterator<Group> it2 = southboundGroupEntries.iterator(); it2.hasNext();) {
1279 Group group = it2.next();
1280 if (storedGroupEntries.remove(group)) {
1281 // we both have the group, let's update some info then.
1282 log.trace("Group AUDIT: group {} exists in both planes for device {}",
alshabibb0285992016-03-28 23:30:37 -07001283 group.id(), deviceId);
1284
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001285 groupAdded(group);
1286 it2.remove();
1287 }
1288 }
1289 for (Group group : southboundGroupEntries) {
1290 if (getGroup(group.deviceId(), group.id()) != null) {
1291 // There is a group existing with the same id
1292 // It is possible that group update is
1293 // in progress while we got a stale info from switch
1294 if (!storedGroupEntries.remove(getGroup(
alshabibb0285992016-03-28 23:30:37 -07001295 group.deviceId(), group.id()))) {
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001296 log.warn("Group AUDIT: Inconsistent state:"
alshabibb0285992016-03-28 23:30:37 -07001297 + "Group exists in ID based table while "
1298 + "not present in key based table");
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001299 }
1300 } else {
1301 // there are groups in the switch that aren't in the store
1302 log.debug("Group AUDIT: extraneous group {} exists in data plane for device {}",
alshabibb0285992016-03-28 23:30:37 -07001303 group.id(), deviceId);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001304 extraneousStoredEntries.remove(group);
1305 extraneousGroup(group);
1306 }
1307 }
1308 for (Group group : storedGroupEntries) {
1309 // there are groups in the store that aren't in the switch
1310 log.debug("Group AUDIT: group {} missing in data plane for device {}",
alshabibb0285992016-03-28 23:30:37 -07001311 group.id(), deviceId);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001312 groupMissing(group);
1313 }
1314 for (Group group : extraneousStoredEntries) {
1315 // there are groups in the extraneous store that
1316 // aren't in the switch
Saurav Das0fd79d92016-03-07 10:58:36 -08001317 log.debug("Group AUDIT: clearing extraneous group {} from store for device {}",
alshabibb0285992016-03-28 23:30:37 -07001318 group.id(), deviceId);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001319 removeExtraneousGroupEntry(group);
1320 }
1321
1322 if (!deviceInitialAuditStatus) {
Saurav Das0fd79d92016-03-07 10:58:36 -08001323 log.info("Group AUDIT: Setting device {} initial AUDIT completed",
alshabibb0285992016-03-28 23:30:37 -07001324 deviceId);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001325 deviceInitialAuditCompleted(deviceId, true);
1326 }
1327 }
1328
alshabibb0285992016-03-28 23:30:37 -07001329 private void garbageCollect(DeviceId deviceId,
1330 Set<Group> southboundGroupEntries,
1331 Set<StoredGroupEntry> storedGroupEntries) {
1332 if (!garbageCollect) {
1333 return;
1334 }
1335
1336 Iterator<StoredGroupEntry> it = storedGroupEntries.iterator();
1337 while (it.hasNext()) {
1338 StoredGroupEntry group = it.next();
1339 if (group.state() != GroupState.PENDING_DELETE && checkGroupRefCount(group)) {
1340 log.debug("Garbage collecting group {} on {}", group, deviceId);
1341 deleteGroupDescription(deviceId, group.appCookie());
1342 southboundGroupEntries.remove(group);
1343 it.remove();
1344 }
1345 }
1346 }
1347
1348 private boolean checkGroupRefCount(Group group) {
1349 return (group.referenceCount() == 0 && group.age() >= gcThresh);
1350 }
1351
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001352 private void groupMissing(Group group) {
1353 switch (group.state()) {
1354 case PENDING_DELETE:
1355 log.debug("Group {} delete confirmation from device {}",
1356 group, group.deviceId());
1357 removeGroupEntry(group);
1358 break;
1359 case ADDED:
1360 case PENDING_ADD:
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -07001361 case PENDING_ADD_RETRY:
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001362 case PENDING_UPDATE:
1363 log.debug("Group {} is in store but not on device {}",
1364 group, group.deviceId());
1365 StoredGroupEntry existing =
1366 getStoredGroupEntry(group.deviceId(), group.id());
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -07001367 log.debug("groupMissing: group entry {} in device {} moving from {} to PENDING_ADD_RETRY",
alshabibb0285992016-03-28 23:30:37 -07001368 existing.id(),
1369 existing.deviceId(),
1370 existing.state());
Srikanth Vavilapalli5428b6c2015-05-14 20:22:47 -07001371 existing.setState(Group.GroupState.PENDING_ADD_RETRY);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001372 //Re-PUT map entries to trigger map update events
1373 getGroupStoreKeyMap().
alshabibb0285992016-03-28 23:30:37 -07001374 put(new GroupStoreKeyMapKey(existing.deviceId(),
1375 existing.appCookie()), existing);
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001376 notifyDelegate(new GroupEvent(GroupEvent.Type.GROUP_ADD_REQUESTED,
1377 group));
1378 break;
1379 default:
1380 log.debug("Group {} has not been installed.", group);
1381 break;
1382 }
1383 }
1384
1385 private void extraneousGroup(Group group) {
Saurav Das0fd79d92016-03-07 10:58:36 -08001386 log.trace("Group {} is on device {} but not in store.",
Srikanth Vavilapalli23181912015-05-04 09:48:09 -07001387 group, group.deviceId());
1388 addOrUpdateExtraneousGroupEntry(group);
1389 }
1390
1391 private void groupAdded(Group group) {
1392 log.trace("Group {} Added or Updated in device {}",
1393 group, group.deviceId());
1394 addOrUpdateGroupEntry(group);
1395 }
alshabib10580802015-02-18 18:30:33 -08001396}