blob: 068582c1a2054c619c03a5aa204f66cdf29aa076 [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;
19import com.google.common.collect.Iterables;
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -070020
alshabib10580802015-02-18 18:30:33 -080021import org.apache.felix.scr.annotations.Activate;
22import org.apache.felix.scr.annotations.Component;
23import org.apache.felix.scr.annotations.Deactivate;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070024import org.apache.felix.scr.annotations.Reference;
25import org.apache.felix.scr.annotations.ReferenceCardinality;
alshabib10580802015-02-18 18:30:33 -080026import org.apache.felix.scr.annotations.Service;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070027import org.onlab.util.KryoNamespace;
alshabib10580802015-02-18 18:30:33 -080028import org.onlab.util.NewConcurrentHashMap;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070029import org.onosproject.cluster.ClusterService;
30import org.onosproject.core.DefaultApplicationId;
alshabib10580802015-02-18 18:30:33 -080031import org.onosproject.core.DefaultGroupId;
32import org.onosproject.core.GroupId;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070033import org.onosproject.mastership.MastershipService;
alshabib10580802015-02-18 18:30:33 -080034import org.onosproject.net.DeviceId;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070035import org.onosproject.net.MastershipRole;
36import org.onosproject.net.PortNumber;
37import org.onosproject.net.flow.DefaultTrafficTreatment;
38import org.onosproject.net.flow.FlowRule;
39import org.onosproject.net.flow.instructions.Instructions;
40import org.onosproject.net.flow.instructions.L0ModificationInstruction;
41import org.onosproject.net.flow.instructions.L2ModificationInstruction;
42import org.onosproject.net.flow.instructions.L3ModificationInstruction;
alshabib10580802015-02-18 18:30:33 -080043import org.onosproject.net.group.DefaultGroup;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070044import org.onosproject.net.group.DefaultGroupBucket;
alshabib10580802015-02-18 18:30:33 -080045import org.onosproject.net.group.DefaultGroupDescription;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070046import org.onosproject.net.group.DefaultGroupKey;
alshabib10580802015-02-18 18:30:33 -080047import org.onosproject.net.group.Group;
48import org.onosproject.net.group.Group.GroupState;
49import org.onosproject.net.group.GroupBucket;
50import org.onosproject.net.group.GroupBuckets;
51import org.onosproject.net.group.GroupDescription;
52import org.onosproject.net.group.GroupEvent;
53import org.onosproject.net.group.GroupEvent.Type;
54import org.onosproject.net.group.GroupKey;
55import org.onosproject.net.group.GroupOperation;
56import org.onosproject.net.group.GroupStore;
57import org.onosproject.net.group.GroupStoreDelegate;
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -070058import org.onosproject.net.group.StoredGroupBucketEntry;
alshabib10580802015-02-18 18:30:33 -080059import org.onosproject.net.group.StoredGroupEntry;
60import org.onosproject.store.AbstractStore;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070061import org.onosproject.store.Timestamp;
62import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
63import org.onosproject.store.cluster.messaging.ClusterMessage;
64import org.onosproject.store.cluster.messaging.ClusterMessageHandler;
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -070065import org.onosproject.store.impl.MultiValuedTimestamp;
66import org.onosproject.store.serializers.KryoNamespaces;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070067import org.onosproject.store.service.ClockService;
68import org.onosproject.store.service.EventuallyConsistentMap;
69import org.onosproject.store.service.EventuallyConsistentMapBuilder;
70import org.onosproject.store.service.EventuallyConsistentMapEvent;
71import org.onosproject.store.service.EventuallyConsistentMapListener;
72import org.onosproject.store.service.StorageService;
alshabib10580802015-02-18 18:30:33 -080073import org.slf4j.Logger;
74
Jonathan Hart6ec029a2015-03-24 17:12:35 -070075import java.net.URI;
76import java.util.ArrayList;
77import java.util.HashMap;
78import java.util.List;
79import java.util.Objects;
Sho SHIMIZU30d639b2015-05-05 09:30:35 -070080import java.util.Optional;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070081import java.util.concurrent.ConcurrentHashMap;
82import java.util.concurrent.ConcurrentMap;
83import java.util.concurrent.ExecutorService;
84import java.util.concurrent.Executors;
85import java.util.concurrent.atomic.AtomicInteger;
86import java.util.concurrent.atomic.AtomicLong;
87import java.util.stream.Collectors;
88
89import static org.apache.commons.lang3.concurrent.ConcurrentUtils.createIfAbsentUnchecked;
90import static org.onlab.util.Tools.groupedThreads;
91import static org.slf4j.LoggerFactory.getLogger;
alshabib10580802015-02-18 18:30:33 -080092
93/**
94 * Manages inventory of group entries using trivial in-memory implementation.
95 */
96@Component(immediate = true)
97@Service
98public class DistributedGroupStore
99 extends AbstractStore<GroupEvent, GroupStoreDelegate>
100 implements GroupStore {
101
102 private final Logger log = getLogger(getClass());
103
104 private final int dummyId = 0xffffffff;
105 private final GroupId dummyGroupId = new DefaultGroupId(dummyId);
106
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700107 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
108 protected ClusterCommunicationService clusterCommunicator;
109
110 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
111 protected ClusterService clusterService;
112
113 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700114 protected StorageService storageService;
115
116 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700117 protected MastershipService mastershipService;
118
119 // Per device group table with (device id + app cookie) as key
120 private EventuallyConsistentMap<GroupStoreKeyMapKey,
121 StoredGroupEntry> groupStoreEntriesByKey = null;
122 // Per device group table with (device id + group id) as key
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700123 private final ConcurrentMap<DeviceId, ConcurrentMap<GroupId, StoredGroupEntry>>
124 groupEntriesById = new ConcurrentHashMap<>();
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700125 private EventuallyConsistentMap<GroupStoreKeyMapKey,
126 StoredGroupEntry> auditPendingReqQueue = null;
alshabib10580802015-02-18 18:30:33 -0800127 private final ConcurrentMap<DeviceId, ConcurrentMap<GroupId, Group>>
128 extraneousGroupEntriesById = new ConcurrentHashMap<>();
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700129 private ExecutorService messageHandlingExecutor;
130 private static final int MESSAGE_HANDLER_THREAD_POOL_SIZE = 1;
alshabib10580802015-02-18 18:30:33 -0800131
132 private final HashMap<DeviceId, Boolean> deviceAuditStatus =
133 new HashMap<DeviceId, Boolean>();
134
135 private final AtomicInteger groupIdGen = new AtomicInteger();
136
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700137 private KryoNamespace.Builder kryoBuilder = null;
138
alshabib10580802015-02-18 18:30:33 -0800139 @Activate
140 public void activate() {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700141 kryoBuilder = new KryoNamespace.Builder()
142 .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
158 )
159 .register(URI.class)
160 .register(DeviceId.class)
161 .register(PortNumber.class)
162 .register(DefaultApplicationId.class)
163 .register(DefaultTrafficTreatment.class,
164 Instructions.DropInstruction.class,
165 Instructions.OutputInstruction.class,
166 Instructions.GroupInstruction.class,
167 Instructions.TableTypeTransition.class,
168 FlowRule.Type.class,
169 L0ModificationInstruction.class,
170 L0ModificationInstruction.L0SubType.class,
171 L0ModificationInstruction.ModLambdaInstruction.class,
172 L2ModificationInstruction.class,
173 L2ModificationInstruction.L2SubType.class,
174 L2ModificationInstruction.ModEtherInstruction.class,
175 L2ModificationInstruction.PushHeaderInstructions.class,
176 L2ModificationInstruction.ModVlanIdInstruction.class,
177 L2ModificationInstruction.ModVlanPcpInstruction.class,
178 L2ModificationInstruction.ModMplsLabelInstruction.class,
179 L2ModificationInstruction.ModMplsTtlInstruction.class,
180 L3ModificationInstruction.class,
181 L3ModificationInstruction.L3SubType.class,
182 L3ModificationInstruction.ModIPInstruction.class,
183 L3ModificationInstruction.ModIPv6FlowLabelInstruction.class,
184 L3ModificationInstruction.ModTtlInstruction.class,
185 org.onlab.packet.MplsLabel.class
186 )
187 .register(org.onosproject.cluster.NodeId.class)
188 .register(KryoNamespaces.BASIC)
189 .register(KryoNamespaces.MISC);
190
191 messageHandlingExecutor = Executors.
192 newFixedThreadPool(MESSAGE_HANDLER_THREAD_POOL_SIZE,
193 groupedThreads("onos/store/group",
194 "message-handlers"));
195 clusterCommunicator.
196 addSubscriber(GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
197 new ClusterGroupMsgHandler(),
198 messageHandlingExecutor);
199
200 log.debug("Creating EC map groupstorekeymap");
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700201 EventuallyConsistentMapBuilder<GroupStoreKeyMapKey, StoredGroupEntry>
202 keyMapBuilder = storageService.eventuallyConsistentMapBuilder();
203
204 groupStoreEntriesByKey = keyMapBuilder
205 .withName("groupstorekeymap")
206 .withSerializer(kryoBuilder)
207 .withClockService(new GroupStoreLogicalClockManager<>())
208 .build();
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700209 groupStoreEntriesByKey.addListener(new GroupStoreKeyMapListener());
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700210 log.trace("Current size {}", groupStoreEntriesByKey.size());
211
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700212 log.debug("Creating EC map pendinggroupkeymap");
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700213 EventuallyConsistentMapBuilder<GroupStoreKeyMapKey, StoredGroupEntry>
214 auditMapBuilder = storageService.eventuallyConsistentMapBuilder();
215
216 auditPendingReqQueue = auditMapBuilder
217 .withName("pendinggroupkeymap")
218 .withSerializer(kryoBuilder)
219 .withClockService(new GroupStoreLogicalClockManager<>())
220 .build();
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700221 log.trace("Current size {}", auditPendingReqQueue.size());
222
alshabib10580802015-02-18 18:30:33 -0800223 log.info("Started");
224 }
225
226 @Deactivate
227 public void deactivate() {
Thomas Vachuska152f9fd2015-04-02 16:28:13 -0700228 groupStoreEntriesByKey.destroy();
Thomas Vachuska152f9fd2015-04-02 16:28:13 -0700229 auditPendingReqQueue.destroy();
alshabib10580802015-02-18 18:30:33 -0800230 log.info("Stopped");
231 }
232
alshabib10580802015-02-18 18:30:33 -0800233 private static NewConcurrentHashMap<GroupId, Group>
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700234 lazyEmptyExtraneousGroupIdTable() {
alshabib10580802015-02-18 18:30:33 -0800235 return NewConcurrentHashMap.<GroupId, Group>ifNeeded();
236 }
237
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700238 private static NewConcurrentHashMap<GroupId, StoredGroupEntry>
239 lazyEmptyGroupIdTable() {
240 return NewConcurrentHashMap.<GroupId, StoredGroupEntry>ifNeeded();
241 }
242
alshabib10580802015-02-18 18:30:33 -0800243 /**
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700244 * Returns the group store eventual consistent key map.
alshabib10580802015-02-18 18:30:33 -0800245 *
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700246 * @return Map representing group key table.
alshabib10580802015-02-18 18:30:33 -0800247 */
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700248 private EventuallyConsistentMap<GroupStoreKeyMapKey, StoredGroupEntry>
249 getGroupStoreKeyMap() {
250 return groupStoreEntriesByKey;
alshabib10580802015-02-18 18:30:33 -0800251 }
252
253 /**
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700254 * Returns the group id table for specified device.
alshabib10580802015-02-18 18:30:33 -0800255 *
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700256 * @param deviceId identifier of the device
257 * @return Map representing group key table of given device.
alshabib10580802015-02-18 18:30:33 -0800258 */
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700259 private ConcurrentMap<GroupId, StoredGroupEntry> getGroupIdTable(DeviceId deviceId) {
260 return createIfAbsentUnchecked(groupEntriesById,
261 deviceId, lazyEmptyGroupIdTable());
alshabib10580802015-02-18 18:30:33 -0800262 }
263
264 /**
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700265 * Returns the pending group request table.
alshabib10580802015-02-18 18:30:33 -0800266 *
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700267 * @return Map representing group key table.
alshabib10580802015-02-18 18:30:33 -0800268 */
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700269 private EventuallyConsistentMap<GroupStoreKeyMapKey, StoredGroupEntry>
270 getPendingGroupKeyTable() {
271 return auditPendingReqQueue;
alshabib10580802015-02-18 18:30:33 -0800272 }
273
274 /**
275 * Returns the extraneous group id table for specified device.
276 *
277 * @param deviceId identifier of the device
278 * @return Map representing group key table of given device.
279 */
280 private ConcurrentMap<GroupId, Group>
281 getExtraneousGroupIdTable(DeviceId deviceId) {
282 return createIfAbsentUnchecked(extraneousGroupEntriesById,
283 deviceId,
284 lazyEmptyExtraneousGroupIdTable());
285 }
286
287 /**
288 * Returns the number of groups for the specified device in the store.
289 *
290 * @return number of groups for the specified device
291 */
292 @Override
293 public int getGroupCount(DeviceId deviceId) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700294 return (getGroups(deviceId) != null) ?
295 Iterables.size(getGroups(deviceId)) : 0;
alshabib10580802015-02-18 18:30:33 -0800296 }
297
298 /**
299 * Returns the groups associated with a device.
300 *
301 * @param deviceId the device ID
302 *
303 * @return the group entries
304 */
305 @Override
306 public Iterable<Group> getGroups(DeviceId deviceId) {
307 // flatten and make iterator unmodifiable
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700308 log.trace("getGroups: for device {} total number of groups {}",
309 deviceId, getGroupStoreKeyMap().values().size());
310 return FluentIterable.from(getGroupStoreKeyMap().values())
311 .filter(input -> input.deviceId().equals(deviceId))
312 .transform(input -> input);
alshabib10580802015-02-18 18:30:33 -0800313 }
314
315 /**
316 * Returns the stored group entry.
317 *
318 * @param deviceId the device ID
319 * @param appCookie the group key
320 *
321 * @return a group associated with the key
322 */
323 @Override
324 public Group getGroup(DeviceId deviceId, GroupKey appCookie) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700325 return getStoredGroupEntry(deviceId, appCookie);
326 }
327
328 private StoredGroupEntry getStoredGroupEntry(DeviceId deviceId,
329 GroupKey appCookie) {
330 return getGroupStoreKeyMap().get(new GroupStoreKeyMapKey(deviceId,
331 appCookie));
332 }
333
334 @Override
335 public Group getGroup(DeviceId deviceId, GroupId groupId) {
336 return getStoredGroupEntry(deviceId, groupId);
337 }
338
339 private StoredGroupEntry getStoredGroupEntry(DeviceId deviceId,
340 GroupId groupId) {
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700341 return getGroupIdTable(deviceId).get(groupId);
alshabib10580802015-02-18 18:30:33 -0800342 }
343
344 private int getFreeGroupIdValue(DeviceId deviceId) {
345 int freeId = groupIdGen.incrementAndGet();
346
347 while (true) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700348 Group existing = getGroup(deviceId, new DefaultGroupId(freeId));
alshabib10580802015-02-18 18:30:33 -0800349 if (existing == null) {
350 existing = (
351 extraneousGroupEntriesById.get(deviceId) != null) ?
352 extraneousGroupEntriesById.get(deviceId).
353 get(new DefaultGroupId(freeId)) :
354 null;
355 }
356 if (existing != null) {
357 freeId = groupIdGen.incrementAndGet();
358 } else {
359 break;
360 }
361 }
362 return freeId;
363 }
364
365 /**
366 * Stores a new group entry using the information from group description.
367 *
368 * @param groupDesc group description to be used to create group entry
369 */
370 @Override
371 public void storeGroupDescription(GroupDescription groupDesc) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700372 log.trace("In storeGroupDescription");
alshabib10580802015-02-18 18:30:33 -0800373 // Check if a group is existing with the same key
374 if (getGroup(groupDesc.deviceId(), groupDesc.appCookie()) != null) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700375 log.warn("Group already exists with the same key {}",
376 groupDesc.appCookie());
alshabib10580802015-02-18 18:30:33 -0800377 return;
378 }
379
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700380 // Check if group to be created by a remote instance
381 if (mastershipService.getLocalRole(
382 groupDesc.deviceId()) != MastershipRole.MASTER) {
383 log.debug("Device {} local role is not MASTER",
384 groupDesc.deviceId());
385 GroupStoreMessage groupOp = GroupStoreMessage.
386 createGroupAddRequestMsg(groupDesc.deviceId(),
387 groupDesc);
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700388
389 if (!clusterCommunicator.unicast(groupOp,
390 GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
391 m -> kryoBuilder.build().serialize(m),
392 mastershipService.getMasterFor(groupDesc.deviceId()))) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700393 log.warn("Failed to send request to master: {} to {}",
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700394 groupOp,
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700395 mastershipService.getMasterFor(groupDesc.deviceId()));
396 //TODO: Send Group operation failure event
397 }
398 log.debug("Sent Group operation request for device {} "
399 + "to remote MASTER {}",
400 groupDesc.deviceId(),
401 mastershipService.getMasterFor(groupDesc.deviceId()));
alshabib10580802015-02-18 18:30:33 -0800402 return;
403 }
404
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700405 log.debug("Store group for device {} is getting handled locally",
406 groupDesc.deviceId());
alshabib10580802015-02-18 18:30:33 -0800407 storeGroupDescriptionInternal(groupDesc);
408 }
409
410 private void storeGroupDescriptionInternal(GroupDescription groupDesc) {
411 // Check if a group is existing with the same key
412 if (getGroup(groupDesc.deviceId(), groupDesc.appCookie()) != null) {
413 return;
414 }
415
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700416 if (deviceAuditStatus.get(groupDesc.deviceId()) == null) {
417 // Device group audit has not completed yet
418 // Add this group description to pending group key table
419 // Create a group entry object with Dummy Group ID
420 log.debug("storeGroupDescriptionInternal: Device {} AUDIT "
421 + "pending...Queuing Group ADD request",
422 groupDesc.deviceId());
423 StoredGroupEntry group = new DefaultGroup(dummyGroupId, groupDesc);
424 group.setState(GroupState.WAITING_AUDIT_COMPLETE);
425 EventuallyConsistentMap<GroupStoreKeyMapKey, StoredGroupEntry> pendingKeyTable =
426 getPendingGroupKeyTable();
427 pendingKeyTable.put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
428 groupDesc.appCookie()),
429 group);
430 return;
431 }
432
alshabib10580802015-02-18 18:30:33 -0800433 // Get a new group identifier
434 GroupId id = new DefaultGroupId(getFreeGroupIdValue(groupDesc.deviceId()));
435 // Create a group entry object
436 StoredGroupEntry group = new DefaultGroup(id, groupDesc);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700437 // Insert the newly created group entry into key and id maps
438 getGroupStoreKeyMap().
439 put(new GroupStoreKeyMapKey(groupDesc.deviceId(),
440 groupDesc.appCookie()), group);
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700441 // Ensure it also inserted into group id based table to
442 // avoid any chances of duplication in group id generation
443 getGroupIdTable(groupDesc.deviceId()).
444 put(id, group);
alshabib10580802015-02-18 18:30:33 -0800445 notifyDelegate(new GroupEvent(GroupEvent.Type.GROUP_ADD_REQUESTED,
446 group));
447 }
448
449 /**
450 * Updates the existing group entry with the information
451 * from group description.
452 *
453 * @param deviceId the device ID
454 * @param oldAppCookie the current group key
455 * @param type update type
456 * @param newBuckets group buckets for updates
457 * @param newAppCookie optional new group key
458 */
459 @Override
460 public void updateGroupDescription(DeviceId deviceId,
461 GroupKey oldAppCookie,
462 UpdateType type,
463 GroupBuckets newBuckets,
464 GroupKey newAppCookie) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700465 // Check if group update to be done by a remote instance
466 if (mastershipService.
467 getLocalRole(deviceId) != MastershipRole.MASTER) {
468 GroupStoreMessage groupOp = GroupStoreMessage.
469 createGroupUpdateRequestMsg(deviceId,
470 oldAppCookie,
471 type,
472 newBuckets,
473 newAppCookie);
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700474
475 if (!clusterCommunicator.unicast(groupOp,
476 GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
477 m -> kryoBuilder.build().serialize(m),
478 mastershipService.getMasterFor(deviceId))) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700479 log.warn("Failed to send request to master: {} to {}",
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700480 groupOp,
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700481 mastershipService.getMasterFor(deviceId));
482 //TODO: Send Group operation failure event
483 }
484 return;
485 }
486 updateGroupDescriptionInternal(deviceId,
487 oldAppCookie,
488 type,
489 newBuckets,
490 newAppCookie);
491 }
492
493 private void updateGroupDescriptionInternal(DeviceId deviceId,
494 GroupKey oldAppCookie,
495 UpdateType type,
496 GroupBuckets newBuckets,
497 GroupKey newAppCookie) {
alshabib10580802015-02-18 18:30:33 -0800498 // Check if a group is existing with the provided key
499 Group oldGroup = getGroup(deviceId, oldAppCookie);
500 if (oldGroup == null) {
501 return;
502 }
503
504 List<GroupBucket> newBucketList = getUpdatedBucketList(oldGroup,
505 type,
506 newBuckets);
507 if (newBucketList != null) {
508 // Create a new group object from the old group
509 GroupBuckets updatedBuckets = new GroupBuckets(newBucketList);
510 GroupKey newCookie = (newAppCookie != null) ? newAppCookie : oldAppCookie;
511 GroupDescription updatedGroupDesc = new DefaultGroupDescription(
512 oldGroup.deviceId(),
513 oldGroup.type(),
514 updatedBuckets,
515 newCookie,
516 oldGroup.appId());
517 StoredGroupEntry newGroup = new DefaultGroup(oldGroup.id(),
518 updatedGroupDesc);
519 newGroup.setState(GroupState.PENDING_UPDATE);
520 newGroup.setLife(oldGroup.life());
521 newGroup.setPackets(oldGroup.packets());
522 newGroup.setBytes(oldGroup.bytes());
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700523 //Update the group entry in groupkey based map.
524 //Update to groupid based map will happen in the
525 //groupkey based map update listener
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700526 getGroupStoreKeyMap().
527 put(new GroupStoreKeyMapKey(newGroup.deviceId(),
528 newGroup.appCookie()), newGroup);
alshabib10580802015-02-18 18:30:33 -0800529 notifyDelegate(new GroupEvent(Type.GROUP_UPDATE_REQUESTED, newGroup));
530 }
531 }
532
533 private List<GroupBucket> getUpdatedBucketList(Group oldGroup,
534 UpdateType type,
535 GroupBuckets buckets) {
536 GroupBuckets oldBuckets = oldGroup.buckets();
537 List<GroupBucket> newBucketList = new ArrayList<GroupBucket>(
538 oldBuckets.buckets());
539 boolean groupDescUpdated = false;
540
541 if (type == UpdateType.ADD) {
542 // Check if the any of the new buckets are part of
543 // the old bucket list
544 for (GroupBucket addBucket:buckets.buckets()) {
545 if (!newBucketList.contains(addBucket)) {
546 newBucketList.add(addBucket);
547 groupDescUpdated = true;
548 }
549 }
550 } else if (type == UpdateType.REMOVE) {
551 // Check if the to be removed buckets are part of the
552 // old bucket list
553 for (GroupBucket removeBucket:buckets.buckets()) {
554 if (newBucketList.contains(removeBucket)) {
555 newBucketList.remove(removeBucket);
556 groupDescUpdated = true;
557 }
558 }
559 }
560
561 if (groupDescUpdated) {
562 return newBucketList;
563 } else {
564 return null;
565 }
566 }
567
568 /**
569 * Triggers deleting the existing group entry.
570 *
571 * @param deviceId the device ID
572 * @param appCookie the group key
573 */
574 @Override
575 public void deleteGroupDescription(DeviceId deviceId,
576 GroupKey appCookie) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700577 // Check if group to be deleted by a remote instance
578 if (mastershipService.
579 getLocalRole(deviceId) != MastershipRole.MASTER) {
580 GroupStoreMessage groupOp = GroupStoreMessage.
581 createGroupDeleteRequestMsg(deviceId,
582 appCookie);
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700583
584 if (!clusterCommunicator.unicast(groupOp,
585 GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST,
586 m -> kryoBuilder.build().serialize(m),
587 mastershipService.getMasterFor(deviceId))) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700588 log.warn("Failed to send request to master: {} to {}",
Madan Jampani2bfa94c2015-04-11 05:03:49 -0700589 groupOp,
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700590 mastershipService.getMasterFor(deviceId));
591 //TODO: Send Group operation failure event
592 }
593 return;
594 }
595 deleteGroupDescriptionInternal(deviceId, appCookie);
596 }
597
598 private void deleteGroupDescriptionInternal(DeviceId deviceId,
599 GroupKey appCookie) {
alshabib10580802015-02-18 18:30:33 -0800600 // Check if a group is existing with the provided key
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700601 StoredGroupEntry existing = getStoredGroupEntry(deviceId, appCookie);
alshabib10580802015-02-18 18:30:33 -0800602 if (existing == null) {
603 return;
604 }
605
606 synchronized (existing) {
607 existing.setState(GroupState.PENDING_DELETE);
608 }
609 notifyDelegate(new GroupEvent(Type.GROUP_REMOVE_REQUESTED, existing));
610 }
611
612 /**
613 * Stores a new group entry, or updates an existing entry.
614 *
615 * @param group group entry
616 */
617 @Override
618 public void addOrUpdateGroupEntry(Group group) {
619 // check if this new entry is an update to an existing entry
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700620 StoredGroupEntry existing = getStoredGroupEntry(group.deviceId(),
621 group.id());
alshabib10580802015-02-18 18:30:33 -0800622 GroupEvent event = null;
623
624 if (existing != null) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700625 log.trace("addOrUpdateGroupEntry: updating group "
626 + "entry {} in device {}",
627 group.id(),
628 group.deviceId());
alshabib10580802015-02-18 18:30:33 -0800629 synchronized (existing) {
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -0700630 for (GroupBucket bucket:group.buckets().buckets()) {
Sho SHIMIZU30d639b2015-05-05 09:30:35 -0700631 Optional<GroupBucket> matchingBucket =
Srikanth Vavilapalli10e75cd2015-04-13 16:21:24 -0700632 existing.buckets().buckets()
633 .stream()
634 .filter((existingBucket)->(existingBucket.equals(bucket)))
635 .findFirst();
636 if (matchingBucket.isPresent()) {
637 ((StoredGroupBucketEntry) matchingBucket.
638 get()).setPackets(bucket.packets());
639 ((StoredGroupBucketEntry) matchingBucket.
640 get()).setBytes(bucket.bytes());
641 } else {
642 log.warn("addOrUpdateGroupEntry: No matching "
643 + "buckets to update stats");
644 }
645 }
alshabib10580802015-02-18 18:30:33 -0800646 existing.setLife(group.life());
647 existing.setPackets(group.packets());
648 existing.setBytes(group.bytes());
649 if (existing.state() == GroupState.PENDING_ADD) {
650 existing.setState(GroupState.ADDED);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700651 existing.setIsGroupStateAddedFirstTime(true);
alshabib10580802015-02-18 18:30:33 -0800652 event = new GroupEvent(Type.GROUP_ADDED, existing);
653 } else {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700654 existing.setState(GroupState.ADDED);
655 existing.setIsGroupStateAddedFirstTime(false);
alshabib10580802015-02-18 18:30:33 -0800656 event = new GroupEvent(Type.GROUP_UPDATED, existing);
657 }
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700658 //Re-PUT map entries to trigger map update events
659 getGroupStoreKeyMap().
660 put(new GroupStoreKeyMapKey(existing.deviceId(),
661 existing.appCookie()), existing);
alshabib10580802015-02-18 18:30:33 -0800662 }
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700663 } else {
664 log.warn("addOrUpdateGroupEntry: Group update "
665 + "happening for a non-existing entry in the map");
alshabib10580802015-02-18 18:30:33 -0800666 }
667
668 if (event != null) {
669 notifyDelegate(event);
670 }
671 }
672
673 /**
674 * Removes the group entry from store.
675 *
676 * @param group group entry
677 */
678 @Override
679 public void removeGroupEntry(Group group) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700680 StoredGroupEntry existing = getStoredGroupEntry(group.deviceId(),
681 group.id());
alshabib10580802015-02-18 18:30:33 -0800682
683 if (existing != null) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700684 log.trace("removeGroupEntry: removing group "
685 + "entry {} in device {}",
686 group.id(),
687 group.deviceId());
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700688 //Removal from groupid based map will happen in the
689 //map update listener
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700690 getGroupStoreKeyMap().remove(new GroupStoreKeyMapKey(existing.deviceId(),
691 existing.appCookie()));
alshabib10580802015-02-18 18:30:33 -0800692 notifyDelegate(new GroupEvent(Type.GROUP_REMOVED, existing));
693 }
694 }
695
696 @Override
697 public void deviceInitialAuditCompleted(DeviceId deviceId,
698 boolean completed) {
699 synchronized (deviceAuditStatus) {
700 if (completed) {
701 log.debug("deviceInitialAuditCompleted: AUDIT "
702 + "completed for device {}", deviceId);
703 deviceAuditStatus.put(deviceId, true);
704 // Execute all pending group requests
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700705 List<StoredGroupEntry> pendingGroupRequests =
706 getPendingGroupKeyTable().values()
707 .stream()
708 .filter(g-> g.deviceId().equals(deviceId))
709 .collect(Collectors.toList());
710 log.trace("deviceInitialAuditCompleted: processing "
711 + "pending group add requests for device {} and "
712 + "number of pending requests {}",
713 deviceId,
714 pendingGroupRequests.size());
715 for (Group group:pendingGroupRequests) {
alshabib10580802015-02-18 18:30:33 -0800716 GroupDescription tmp = new DefaultGroupDescription(
717 group.deviceId(),
718 group.type(),
719 group.buckets(),
720 group.appCookie(),
721 group.appId());
722 storeGroupDescriptionInternal(tmp);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700723 getPendingGroupKeyTable().
724 remove(new GroupStoreKeyMapKey(deviceId, group.appCookie()));
alshabib10580802015-02-18 18:30:33 -0800725 }
alshabib10580802015-02-18 18:30:33 -0800726 } else {
Thomas Vachuskac40d4632015-04-09 16:55:03 -0700727 Boolean audited = deviceAuditStatus.get(deviceId);
728 if (audited != null && audited) {
alshabib10580802015-02-18 18:30:33 -0800729 log.debug("deviceInitialAuditCompleted: Clearing AUDIT "
730 + "status for device {}", deviceId);
731 deviceAuditStatus.put(deviceId, false);
732 }
733 }
734 }
735 }
736
737 @Override
738 public boolean deviceInitialAuditStatus(DeviceId deviceId) {
739 synchronized (deviceAuditStatus) {
Thomas Vachuskac40d4632015-04-09 16:55:03 -0700740 Boolean audited = deviceAuditStatus.get(deviceId);
741 return audited != null && audited;
alshabib10580802015-02-18 18:30:33 -0800742 }
743 }
744
745 @Override
746 public void groupOperationFailed(DeviceId deviceId, GroupOperation operation) {
747
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700748 StoredGroupEntry existing = getStoredGroupEntry(deviceId,
749 operation.groupId());
alshabib10580802015-02-18 18:30:33 -0800750
751 if (existing == null) {
752 log.warn("No group entry with ID {} found ", operation.groupId());
753 return;
754 }
755
756 switch (operation.opType()) {
757 case ADD:
758 notifyDelegate(new GroupEvent(Type.GROUP_ADD_FAILED, existing));
759 break;
760 case MODIFY:
761 notifyDelegate(new GroupEvent(Type.GROUP_UPDATE_FAILED, existing));
762 break;
763 case DELETE:
764 notifyDelegate(new GroupEvent(Type.GROUP_REMOVE_FAILED, existing));
765 break;
766 default:
767 log.warn("Unknown group operation type {}", operation.opType());
768 }
769
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700770 //Removal from groupid based map will happen in the
771 //map update listener
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700772 getGroupStoreKeyMap().remove(new GroupStoreKeyMapKey(existing.deviceId(),
773 existing.appCookie()));
alshabib10580802015-02-18 18:30:33 -0800774 }
775
776 @Override
777 public void addOrUpdateExtraneousGroupEntry(Group group) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700778 log.trace("addOrUpdateExtraneousGroupEntry: add/update extraneous "
779 + "group entry {} in device {}",
780 group.id(),
781 group.deviceId());
alshabib10580802015-02-18 18:30:33 -0800782 ConcurrentMap<GroupId, Group> extraneousIdTable =
783 getExtraneousGroupIdTable(group.deviceId());
784 extraneousIdTable.put(group.id(), group);
785 // Check the reference counter
786 if (group.referenceCount() == 0) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700787 log.trace("addOrUpdateExtraneousGroupEntry: Flow reference "
788 + "counter is zero and triggering remove",
789 group.id(),
790 group.deviceId());
alshabib10580802015-02-18 18:30:33 -0800791 notifyDelegate(new GroupEvent(Type.GROUP_REMOVE_REQUESTED, group));
792 }
793 }
794
795 @Override
796 public void removeExtraneousGroupEntry(Group group) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700797 log.trace("removeExtraneousGroupEntry: remove extraneous "
798 + "group entry {} of device {} from store",
799 group.id(),
800 group.deviceId());
alshabib10580802015-02-18 18:30:33 -0800801 ConcurrentMap<GroupId, Group> extraneousIdTable =
802 getExtraneousGroupIdTable(group.deviceId());
803 extraneousIdTable.remove(group.id());
804 }
805
806 @Override
807 public Iterable<Group> getExtraneousGroups(DeviceId deviceId) {
808 // flatten and make iterator unmodifiable
809 return FluentIterable.from(
810 getExtraneousGroupIdTable(deviceId).values());
811 }
812
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700813 /**
814 * ClockService that generates wallclock based timestamps.
815 */
816 private class GroupStoreLogicalClockManager<T, U>
817 implements ClockService<T, U> {
alshabib10580802015-02-18 18:30:33 -0800818
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700819 private final AtomicLong sequenceNumber = new AtomicLong(0);
820
821 @Override
822 public Timestamp getTimestamp(T t1, U u1) {
823 return new MultiValuedTimestamp<>(System.currentTimeMillis(),
824 sequenceNumber.getAndIncrement());
825 }
826 }
827
828 /**
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700829 * Map handler to receive any events when the group key map is updated.
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700830 */
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700831 private class GroupStoreKeyMapListener implements
832 EventuallyConsistentMapListener<GroupStoreKeyMapKey, StoredGroupEntry> {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700833
834 @Override
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700835 public void event(EventuallyConsistentMapEvent<GroupStoreKeyMapKey,
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700836 StoredGroupEntry> mapEvent) {
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700837 GroupEvent groupEvent = null;
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700838 StoredGroupEntry group = mapEvent.value();
839 log.trace("GroupStoreKeyMapListener: received groupid map event {}",
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700840 mapEvent.type());
841 if (mapEvent.type() == EventuallyConsistentMapEvent.Type.PUT) {
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700842 log.trace("GroupStoreKeyMapListener: Received PUT event");
843 // Update the group ID table
844 getGroupIdTable(group.deviceId()).put(group.id(), group);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700845 if (mapEvent.value().state() == Group.GroupState.ADDED) {
846 if (mapEvent.value().isGroupStateAddedFirstTime()) {
847 groupEvent = new GroupEvent(Type.GROUP_ADDED,
848 mapEvent.value());
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700849 log.trace("GroupStoreKeyMapListener: Received first time "
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700850 + "GROUP_ADDED state update");
851 } else {
852 groupEvent = new GroupEvent(Type.GROUP_UPDATED,
853 mapEvent.value());
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700854 log.trace("GroupStoreKeyMapListener: Received following "
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700855 + "GROUP_ADDED state update");
856 }
857 }
858 } else if (mapEvent.type() == EventuallyConsistentMapEvent.Type.REMOVE) {
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700859 log.trace("GroupStoreKeyMapListener: Received REMOVE event");
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700860 groupEvent = new GroupEvent(Type.GROUP_REMOVED, mapEvent.value());
Srikanth Vavilapalli6a9d4e42015-03-30 19:41:56 -0700861 // Remove the entry from the group ID table
862 getGroupIdTable(group.deviceId()).remove(group.id(), group);
Srikanth Vavilapalli717361f2015-03-16 12:06:04 -0700863 }
864
865 if (groupEvent != null) {
866 notifyDelegate(groupEvent);
867 }
868 }
869 }
870 /**
871 * Message handler to receive messages from group subsystems of
872 * other cluster members.
873 */
874 private final class ClusterGroupMsgHandler
875 implements ClusterMessageHandler {
876 @Override
877 public void handle(ClusterMessage message) {
878 log.trace("ClusterGroupMsgHandler: received remote group message");
879 if (message.subject() ==
880 GroupStoreMessageSubjects.REMOTE_GROUP_OP_REQUEST) {
881 GroupStoreMessage groupOp = kryoBuilder.
882 build().deserialize(message.payload());
883 log.trace("received remote group operation request");
884 if (!(mastershipService.
885 getLocalRole(groupOp.deviceId()) !=
886 MastershipRole.MASTER)) {
887 log.warn("ClusterGroupMsgHandler: This node is not "
888 + "MASTER for device {}", groupOp.deviceId());
889 return;
890 }
891 if (groupOp.type() == GroupStoreMessage.Type.ADD) {
892 log.trace("processing remote group "
893 + "add operation request");
894 storeGroupDescriptionInternal(groupOp.groupDesc());
895 } else if (groupOp.type() == GroupStoreMessage.Type.UPDATE) {
896 log.trace("processing remote group "
897 + "update operation request");
898 updateGroupDescriptionInternal(groupOp.deviceId(),
899 groupOp.appCookie(),
900 groupOp.updateType(),
901 groupOp.updateBuckets(),
902 groupOp.newAppCookie());
903 } else if (groupOp.type() == GroupStoreMessage.Type.DELETE) {
904 log.trace("processing remote group "
905 + "delete operation request");
906 deleteGroupDescriptionInternal(groupOp.deviceId(),
907 groupOp.appCookie());
908 }
909 }
910 }
911 }
912
913 /**
914 * Flattened map key to be used to store group entries.
915 */
916 private class GroupStoreMapKey {
917 private final DeviceId deviceId;
918
919 public GroupStoreMapKey(DeviceId deviceId) {
920 this.deviceId = deviceId;
921 }
922
923 @Override
924 public boolean equals(Object o) {
925 if (this == o) {
926 return true;
927 }
928 if (!(o instanceof GroupStoreMapKey)) {
929 return false;
930 }
931 GroupStoreMapKey that = (GroupStoreMapKey) o;
932 return this.deviceId.equals(that.deviceId);
933 }
934
935 @Override
936 public int hashCode() {
937 int result = 17;
938
939 result = 31 * result + Objects.hash(this.deviceId);
940
941 return result;
942 }
943 }
944
945 private class GroupStoreKeyMapKey extends GroupStoreMapKey {
946 private final GroupKey appCookie;
947 public GroupStoreKeyMapKey(DeviceId deviceId,
948 GroupKey appCookie) {
949 super(deviceId);
950 this.appCookie = appCookie;
951 }
952
953 @Override
954 public boolean equals(Object o) {
955 if (this == o) {
956 return true;
957 }
958 if (!(o instanceof GroupStoreKeyMapKey)) {
959 return false;
960 }
961 GroupStoreKeyMapKey that = (GroupStoreKeyMapKey) o;
962 return (super.equals(that) &&
963 this.appCookie.equals(that.appCookie));
964 }
965
966 @Override
967 public int hashCode() {
968 int result = 17;
969
970 result = 31 * result + super.hashCode() + Objects.hash(this.appCookie);
971
972 return result;
973 }
974 }
975
976 private class GroupStoreIdMapKey extends GroupStoreMapKey {
977 private final GroupId groupId;
978 public GroupStoreIdMapKey(DeviceId deviceId,
979 GroupId groupId) {
980 super(deviceId);
981 this.groupId = groupId;
982 }
983
984 @Override
985 public boolean equals(Object o) {
986 if (this == o) {
987 return true;
988 }
989 if (!(o instanceof GroupStoreIdMapKey)) {
990 return false;
991 }
992 GroupStoreIdMapKey that = (GroupStoreIdMapKey) o;
993 return (super.equals(that) &&
994 this.groupId.equals(that.groupId));
995 }
996
997 @Override
998 public int hashCode() {
999 int result = 17;
1000
1001 result = 31 * result + super.hashCode() + Objects.hash(this.groupId);
1002
1003 return result;
1004 }
1005 }
alshabib10580802015-02-18 18:30:33 -08001006}