Madan Jampani
Committed by Gerrit Code Review

ONOS-1883: Fix for lost flow rules on CLI directed mastership changes.

- Made all mastership role change operations asynchronous, which they are.
- In flowrule store we now check to see if any new backups need to be made when a device backup location (standby) changes
- In device mastership store we now wait briefly before we step down from mastership after promoting a new candidate as highest standy

Change-Id: Icb76cf4d0d23403053a3fd5a458a940b847da49f
...@@ -16,6 +16,7 @@ ...@@ -16,6 +16,7 @@
16 package org.onosproject.mastership; 16 package org.onosproject.mastership;
17 17
18 import java.util.Set; 18 import java.util.Set;
19 +import java.util.concurrent.CompletableFuture;
19 20
20 import org.onosproject.cluster.NodeId; 21 import org.onosproject.cluster.NodeId;
21 import org.onosproject.cluster.RoleInfo; 22 import org.onosproject.cluster.RoleInfo;
...@@ -80,7 +81,7 @@ public interface MastershipStore extends Store<MastershipEvent, MastershipStoreD ...@@ -80,7 +81,7 @@ public interface MastershipStore extends Store<MastershipEvent, MastershipStoreD
80 * @param deviceId device identifier 81 * @param deviceId device identifier
81 * @return a mastership event 82 * @return a mastership event
82 */ 83 */
83 - MastershipEvent setMaster(NodeId nodeId, DeviceId deviceId); 84 + CompletableFuture<MastershipEvent> setMaster(NodeId nodeId, DeviceId deviceId);
84 85
85 /** 86 /**
86 * Returns the current master and number of past mastership hand-offs 87 * Returns the current master and number of past mastership hand-offs
...@@ -100,7 +101,7 @@ public interface MastershipStore extends Store<MastershipEvent, MastershipStoreD ...@@ -100,7 +101,7 @@ public interface MastershipStore extends Store<MastershipEvent, MastershipStoreD
100 * @param deviceId device to revoke mastership role for 101 * @param deviceId device to revoke mastership role for
101 * @return a mastership event 102 * @return a mastership event
102 */ 103 */
103 - MastershipEvent setStandby(NodeId nodeId, DeviceId deviceId); 104 + CompletableFuture<MastershipEvent> setStandby(NodeId nodeId, DeviceId deviceId);
104 105
105 /** 106 /**
106 * Allows a controller instance to give up its current role for a device. 107 * Allows a controller instance to give up its current role for a device.
...@@ -111,7 +112,7 @@ public interface MastershipStore extends Store<MastershipEvent, MastershipStoreD ...@@ -111,7 +112,7 @@ public interface MastershipStore extends Store<MastershipEvent, MastershipStoreD
111 * @param deviceId device to revoke mastership role for 112 * @param deviceId device to revoke mastership role for
112 * @return a mastership event 113 * @return a mastership event
113 */ 114 */
114 - MastershipEvent relinquishRole(NodeId nodeId, DeviceId deviceId); 115 + CompletableFuture<MastershipEvent> relinquishRole(NodeId nodeId, DeviceId deviceId);
115 116
116 /** 117 /**
117 * Removes all the roles for the specified controller instance. 118 * Removes all the roles for the specified controller instance.
......
...@@ -17,6 +17,7 @@ package org.onosproject.cluster.impl; ...@@ -17,6 +17,7 @@ package org.onosproject.cluster.impl;
17 17
18 import com.codahale.metrics.Timer; 18 import com.codahale.metrics.Timer;
19 import com.codahale.metrics.Timer.Context; 19 import com.codahale.metrics.Timer.Context;
20 +
20 import org.apache.felix.scr.annotations.Activate; 21 import org.apache.felix.scr.annotations.Activate;
21 import org.apache.felix.scr.annotations.Component; 22 import org.apache.felix.scr.annotations.Component;
22 import org.apache.felix.scr.annotations.Deactivate; 23 import org.apache.felix.scr.annotations.Deactivate;
...@@ -50,6 +51,8 @@ import java.util.Iterator; ...@@ -50,6 +51,8 @@ import java.util.Iterator;
50 import java.util.List; 51 import java.util.List;
51 import java.util.Map; 52 import java.util.Map;
52 import java.util.Set; 53 import java.util.Set;
54 +import java.util.concurrent.CompletableFuture;
55 +
53 import static com.google.common.base.Preconditions.checkNotNull; 56 import static com.google.common.base.Preconditions.checkNotNull;
54 import static com.google.common.collect.Lists.newArrayList; 57 import static com.google.common.collect.Lists.newArrayList;
55 import static org.onlab.metrics.MetricsUtil.startTimer; 58 import static org.onlab.metrics.MetricsUtil.startTimer;
...@@ -111,26 +114,28 @@ public class MastershipManager ...@@ -111,26 +114,28 @@ public class MastershipManager
111 checkNotNull(deviceId, DEVICE_ID_NULL); 114 checkNotNull(deviceId, DEVICE_ID_NULL);
112 checkNotNull(role, ROLE_NULL); 115 checkNotNull(role, ROLE_NULL);
113 116
114 - MastershipEvent event = null; 117 + CompletableFuture<MastershipEvent> eventFuture = null;
115 118
116 switch (role) { 119 switch (role) {
117 case MASTER: 120 case MASTER:
118 - event = store.setMaster(nodeId, deviceId); 121 + eventFuture = store.setMaster(nodeId, deviceId);
119 break; 122 break;
120 case STANDBY: 123 case STANDBY:
121 - event = store.setStandby(nodeId, deviceId); 124 + eventFuture = store.setStandby(nodeId, deviceId);
122 break; 125 break;
123 case NONE: 126 case NONE:
124 - event = store.relinquishRole(nodeId, deviceId); 127 + eventFuture = store.relinquishRole(nodeId, deviceId);
125 break; 128 break;
126 default: 129 default:
127 log.info("Unknown role; ignoring"); 130 log.info("Unknown role; ignoring");
128 return; 131 return;
129 } 132 }
130 133
134 + eventFuture.whenComplete((event, error) -> {
131 if (event != null) { 135 if (event != null) {
132 post(event); 136 post(event);
133 } 137 }
138 + });
134 } 139 }
135 140
136 @Override 141 @Override
...@@ -141,12 +146,12 @@ public class MastershipManager ...@@ -141,12 +146,12 @@ public class MastershipManager
141 146
142 @Override 147 @Override
143 public void relinquishMastership(DeviceId deviceId) { 148 public void relinquishMastership(DeviceId deviceId) {
144 - MastershipEvent event = null; 149 + store.relinquishRole(clusterService.getLocalNode().id(), deviceId)
145 - event = store.relinquishRole( 150 + .whenComplete((event, error) -> {
146 - clusterService.getLocalNode().id(), deviceId);
147 if (event != null) { 151 if (event != null) {
148 post(event); 152 post(event);
149 } 153 }
154 + });
150 } 155 }
151 156
152 @Override 157 @Override
......
...@@ -62,6 +62,8 @@ import org.onosproject.store.cluster.messaging.ClusterCommunicationService; ...@@ -62,6 +62,8 @@ import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
62 import org.onosproject.store.cluster.messaging.ClusterMessage; 62 import org.onosproject.store.cluster.messaging.ClusterMessage;
63 import org.onosproject.store.cluster.messaging.ClusterMessageHandler; 63 import org.onosproject.store.cluster.messaging.ClusterMessageHandler;
64 import org.onosproject.store.flow.ReplicaInfo; 64 import org.onosproject.store.flow.ReplicaInfo;
65 +import org.onosproject.store.flow.ReplicaInfoEvent;
66 +import org.onosproject.store.flow.ReplicaInfoEventListener;
65 import org.onosproject.store.flow.ReplicaInfoService; 67 import org.onosproject.store.flow.ReplicaInfoService;
66 import org.onosproject.store.serializers.KryoSerializer; 68 import org.onosproject.store.serializers.KryoSerializer;
67 import org.onosproject.store.serializers.StoreSerializer; 69 import org.onosproject.store.serializers.StoreSerializer;
...@@ -181,6 +183,7 @@ public class NewDistributedFlowRuleStore ...@@ -181,6 +183,7 @@ public class NewDistributedFlowRuleStore
181 registerMessageHandlers(messageHandlingExecutor); 183 registerMessageHandlers(messageHandlingExecutor);
182 184
183 if (backupEnabled) { 185 if (backupEnabled) {
186 + replicaInfoManager.addListener(flowTable);
184 backupTask = backupSenderExecutor.scheduleWithFixedDelay( 187 backupTask = backupSenderExecutor.scheduleWithFixedDelay(
185 flowTable::backup, 188 flowTable::backup,
186 0, 189 0,
...@@ -193,6 +196,10 @@ public class NewDistributedFlowRuleStore ...@@ -193,6 +196,10 @@ public class NewDistributedFlowRuleStore
193 196
194 @Deactivate 197 @Deactivate
195 public void deactivate(ComponentContext context) { 198 public void deactivate(ComponentContext context) {
199 + if (backupEnabled) {
200 + replicaInfoManager.removeListener(flowTable);
201 + backupTask.cancel(true);
202 + }
196 configService.unregisterProperties(getClass(), false); 203 configService.unregisterProperties(getClass(), false);
197 unregisterMessageHandlers(); 204 unregisterMessageHandlers();
198 messageHandlingExecutor.shutdownNow(); 205 messageHandlingExecutor.shutdownNow();
...@@ -232,10 +239,15 @@ public class NewDistributedFlowRuleStore ...@@ -232,10 +239,15 @@ public class NewDistributedFlowRuleStore
232 boolean restartBackupTask = false; 239 boolean restartBackupTask = false;
233 if (newBackupEnabled != backupEnabled) { 240 if (newBackupEnabled != backupEnabled) {
234 backupEnabled = newBackupEnabled; 241 backupEnabled = newBackupEnabled;
235 - if (!backupEnabled && backupTask != null) { 242 + if (!backupEnabled) {
243 + replicaInfoManager.removeListener(flowTable);
244 + if (backupTask != null) {
236 backupTask.cancel(false); 245 backupTask.cancel(false);
237 backupTask = null; 246 backupTask = null;
238 } 247 }
248 + } else {
249 + replicaInfoManager.addListener(flowTable);
250 + }
239 restartBackupTask = backupEnabled; 251 restartBackupTask = backupEnabled;
240 } 252 }
241 if (newBackupPeriod != backupPeriod) { 253 if (newBackupPeriod != backupPeriod) {
...@@ -590,7 +602,7 @@ public class NewDistributedFlowRuleStore ...@@ -590,7 +602,7 @@ public class NewDistributedFlowRuleStore
590 } 602 }
591 } 603 }
592 604
593 - private class InternalFlowTable { 605 + private class InternalFlowTable implements ReplicaInfoEventListener {
594 606
595 private final ConcurrentMap<DeviceId, ConcurrentMap<FlowId, Set<StoredFlowEntry>>> 607 private final ConcurrentMap<DeviceId, ConcurrentMap<FlowId, Set<StoredFlowEntry>>>
596 flowEntries = new ConcurrentHashMap<>(); 608 flowEntries = new ConcurrentHashMap<>();
...@@ -603,6 +615,43 @@ public class NewDistributedFlowRuleStore ...@@ -603,6 +615,43 @@ public class NewDistributedFlowRuleStore
603 return NewConcurrentHashMap.<FlowId, Set<StoredFlowEntry>>ifNeeded(); 615 return NewConcurrentHashMap.<FlowId, Set<StoredFlowEntry>>ifNeeded();
604 } 616 }
605 617
618 + @Override
619 + public void event(ReplicaInfoEvent event) {
620 + if (event.type() == ReplicaInfoEvent.Type.BACKUPS_CHANGED) {
621 + DeviceId deviceId = event.subject();
622 + if (!Objects.equal(local, replicaInfoManager.getReplicaInfoFor(deviceId).master())) {
623 + // ignore since this event is for a device this node does not manage.
624 + return;
625 + }
626 + NodeId latestBackupNode = getBackupNode(deviceId);
627 + NodeId existingBackupNode = lastBackupNodes.get(deviceId);
628 + if (Objects.equal(latestBackupNode, existingBackupNode)) {
629 + // ignore since backup location hasn't changed.
630 + return;
631 + }
632 + backupFlowEntries(latestBackupNode, Sets.newHashSet(deviceId));
633 + }
634 + }
635 +
636 + private void backupFlowEntries(NodeId nodeId, Set<DeviceId> deviceIds) {
637 + log.debug("Sending flowEntries for devices {} to {} as backup.", deviceIds, nodeId);
638 + Map<DeviceId, ConcurrentMap<FlowId, Set<StoredFlowEntry>>> deviceFlowEntries =
639 + Maps.newConcurrentMap();
640 + flowEntries.forEach((key, value) -> {
641 + if (deviceIds.contains(key)) {
642 + deviceFlowEntries.put(key, value);
643 + }
644 + });
645 + clusterCommunicator.unicast(deviceFlowEntries,
646 + FLOW_TABLE_BACKUP,
647 + SERIALIZER::encode,
648 + nodeId);
649 + deviceIds.forEach(id -> {
650 + lastBackupTimes.put(id, System.currentTimeMillis());
651 + lastBackupNodes.put(id, nodeId);
652 + });
653 + }
654 +
606 /** 655 /**
607 * Returns the flow table for specified device. 656 * Returns the flow table for specified device.
608 * 657 *
...@@ -662,7 +711,6 @@ public class NewDistributedFlowRuleStore ...@@ -662,7 +711,6 @@ public class NewDistributedFlowRuleStore
662 if (!backupEnabled) { 711 if (!backupEnabled) {
663 return; 712 return;
664 } 713 }
665 - //TODO: Force backup when backups change.
666 try { 714 try {
667 // determine the set of devices that we need to backup during this run. 715 // determine the set of devices that we need to backup during this run.
668 Set<DeviceId> devicesToBackup = mastershipService.getDevicesOf(local) 716 Set<DeviceId> devicesToBackup = mastershipService.getDevicesOf(local)
...@@ -686,35 +734,15 @@ public class NewDistributedFlowRuleStore ...@@ -686,35 +734,15 @@ public class NewDistributedFlowRuleStore
686 .add(deviceId); 734 .add(deviceId);
687 } 735 }
688 }); 736 });
689 -
690 // send the device flow entries to their respective backup nodes 737 // send the device flow entries to their respective backup nodes
691 - devicesToBackupByNode.forEach((nodeId, deviceIds) -> { 738 + devicesToBackupByNode.forEach(this::backupFlowEntries);
692 - Map<DeviceId, ConcurrentMap<FlowId, Set<StoredFlowEntry>>> deviceFlowEntries =
693 - Maps.newConcurrentMap();
694 - flowEntries.forEach((key, value) -> {
695 - if (deviceIds.contains(key)) {
696 - deviceFlowEntries.put(key, value);
697 - }
698 - });
699 - clusterCommunicator.unicast(deviceFlowEntries,
700 - FLOW_TABLE_BACKUP,
701 - SERIALIZER::encode,
702 - nodeId);
703 - });
704 -
705 - // update state for use in subsequent run.
706 - devicesToBackupByNode.forEach((node, devices) -> {
707 - devices.forEach(id -> {
708 - lastBackupTimes.put(id, System.currentTimeMillis());
709 - lastBackupNodes.put(id, node);
710 - });
711 - });
712 } catch (Exception e) { 739 } catch (Exception e) {
713 log.error("Backup failed.", e); 740 log.error("Backup failed.", e);
714 } 741 }
715 } 742 }
716 743
717 private void onBackupReceipt(Map<DeviceId, Map<FlowId, Set<StoredFlowEntry>>> flowTables) { 744 private void onBackupReceipt(Map<DeviceId, Map<FlowId, Set<StoredFlowEntry>>> flowTables) {
745 + log.debug("Received flows for {} to backup", flowTables.keySet());
718 Set<DeviceId> managedDevices = mastershipService.getDevicesOf(local); 746 Set<DeviceId> managedDevices = mastershipService.getDevicesOf(local);
719 // Only process those devices are that not managed by the local node. 747 // Only process those devices are that not managed by the local node.
720 Maps.filterKeys(flowTables, deviceId -> !managedDevices.contains(deviceId)) 748 Maps.filterKeys(flowTables, deviceId -> !managedDevices.contains(deviceId))
......
...@@ -25,8 +25,11 @@ import static com.google.common.base.Preconditions.checkArgument; ...@@ -25,8 +25,11 @@ import static com.google.common.base.Preconditions.checkArgument;
25 import java.util.List; 25 import java.util.List;
26 import java.util.Map; 26 import java.util.Map;
27 import java.util.Set; 27 import java.util.Set;
28 +import java.util.concurrent.CompletableFuture;
28 import java.util.concurrent.ExecutorService; 29 import java.util.concurrent.ExecutorService;
29 import java.util.concurrent.Executors; 30 import java.util.concurrent.Executors;
31 +import java.util.concurrent.ScheduledExecutorService;
32 +import java.util.concurrent.TimeUnit;
30 import java.util.regex.Matcher; 33 import java.util.regex.Matcher;
31 import java.util.regex.Pattern; 34 import java.util.regex.Pattern;
32 import java.util.stream.Collectors; 35 import java.util.stream.Collectors;
...@@ -98,11 +101,13 @@ public class ConsistentDeviceMastershipStore ...@@ -98,11 +101,13 @@ public class ConsistentDeviceMastershipStore
98 Pattern.compile("device:(.*)"); 101 Pattern.compile("device:(.*)");
99 102
100 private ExecutorService messageHandlingExecutor; 103 private ExecutorService messageHandlingExecutor;
104 + private ScheduledExecutorService transferExecutor;
101 private final LeadershipEventListener leadershipEventListener = 105 private final LeadershipEventListener leadershipEventListener =
102 new InternalDeviceMastershipEventListener(); 106 new InternalDeviceMastershipEventListener();
103 107
104 private static final String NODE_ID_NULL = "Node ID cannot be null"; 108 private static final String NODE_ID_NULL = "Node ID cannot be null";
105 - private static final String DEVICE_ID_NULL = "Device ID cannot be null";; 109 + private static final String DEVICE_ID_NULL = "Device ID cannot be null";
110 + private static final int WAIT_BEFORE_MASTERSHIP_HANDOFF_MILLIS = 3000;
106 111
107 public static final StoreSerializer SERIALIZER = new KryoSerializer() { 112 public static final StoreSerializer SERIALIZER = new KryoSerializer() {
108 @Override 113 @Override
...@@ -119,7 +124,11 @@ public class ConsistentDeviceMastershipStore ...@@ -119,7 +124,11 @@ public class ConsistentDeviceMastershipStore
119 @Activate 124 @Activate
120 public void activate() { 125 public void activate() {
121 messageHandlingExecutor = 126 messageHandlingExecutor =
122 - Executors.newSingleThreadExecutor(groupedThreads("onos/store/device/mastership", "message-handler")); 127 + Executors.newSingleThreadExecutor(
128 + groupedThreads("onos/store/device/mastership", "message-handler"));
129 + transferExecutor =
130 + Executors.newSingleThreadScheduledExecutor(
131 + groupedThreads("onos/store/device/mastership", "mastership-transfer-executor"));
123 clusterCommunicator.<DeviceId, MastershipRole>addSubscriber(ROLE_QUERY_SUBJECT, 132 clusterCommunicator.<DeviceId, MastershipRole>addSubscriber(ROLE_QUERY_SUBJECT,
124 SERIALIZER::decode, 133 SERIALIZER::decode,
125 deviceId -> getRole(localNodeId, deviceId), 134 deviceId -> getRole(localNodeId, deviceId),
...@@ -127,7 +136,7 @@ public class ConsistentDeviceMastershipStore ...@@ -127,7 +136,7 @@ public class ConsistentDeviceMastershipStore
127 messageHandlingExecutor); 136 messageHandlingExecutor);
128 clusterCommunicator.<DeviceId, MastershipEvent>addSubscriber(ROLE_RELINQUISH_SUBJECT, 137 clusterCommunicator.<DeviceId, MastershipEvent>addSubscriber(ROLE_RELINQUISH_SUBJECT,
129 SERIALIZER::decode, 138 SERIALIZER::decode,
130 - deviceId -> relinquishRole(localNodeId, deviceId), 139 + this::relinquishLocalRole,
131 SERIALIZER::encode, 140 SERIALIZER::encode,
132 messageHandlingExecutor); 141 messageHandlingExecutor);
133 clusterCommunicator.addSubscriber(TRANSITION_FROM_MASTER_TO_STANDBY_SUBJECT, 142 clusterCommunicator.addSubscriber(TRANSITION_FROM_MASTER_TO_STANDBY_SUBJECT,
...@@ -147,6 +156,7 @@ public class ConsistentDeviceMastershipStore ...@@ -147,6 +156,7 @@ public class ConsistentDeviceMastershipStore
147 clusterCommunicator.removeSubscriber(ROLE_RELINQUISH_SUBJECT); 156 clusterCommunicator.removeSubscriber(ROLE_RELINQUISH_SUBJECT);
148 clusterCommunicator.removeSubscriber(TRANSITION_FROM_MASTER_TO_STANDBY_SUBJECT); 157 clusterCommunicator.removeSubscriber(TRANSITION_FROM_MASTER_TO_STANDBY_SUBJECT);
149 messageHandlingExecutor.shutdown(); 158 messageHandlingExecutor.shutdown();
159 + transferExecutor.shutdown();
150 leadershipService.removeListener(leadershipEventListener); 160 leadershipService.removeListener(leadershipEventListener);
151 161
152 log.info("Stoppped."); 162 log.info("Stoppped.");
...@@ -246,26 +256,36 @@ public class ConsistentDeviceMastershipStore ...@@ -246,26 +256,36 @@ public class ConsistentDeviceMastershipStore
246 } 256 }
247 257
248 @Override 258 @Override
249 - public MastershipEvent setMaster(NodeId nodeId, DeviceId deviceId) { 259 + public CompletableFuture<MastershipEvent> setMaster(NodeId nodeId, DeviceId deviceId) {
250 checkArgument(nodeId != null, NODE_ID_NULL); 260 checkArgument(nodeId != null, NODE_ID_NULL);
251 checkArgument(deviceId != null, DEVICE_ID_NULL); 261 checkArgument(deviceId != null, DEVICE_ID_NULL);
252 262
253 NodeId currentMaster = getMaster(deviceId); 263 NodeId currentMaster = getMaster(deviceId);
254 if (nodeId.equals(currentMaster)) { 264 if (nodeId.equals(currentMaster)) {
255 - return null; 265 + return CompletableFuture.completedFuture(null);
256 } else { 266 } else {
257 String leadershipTopic = createDeviceMastershipTopic(deviceId); 267 String leadershipTopic = createDeviceMastershipTopic(deviceId);
258 List<NodeId> candidates = leadershipService.getCandidates(leadershipTopic); 268 List<NodeId> candidates = leadershipService.getCandidates(leadershipTopic);
259 if (candidates.isEmpty()) { 269 if (candidates.isEmpty()) {
260 - return null; 270 + return CompletableFuture.completedFuture(null);
261 } 271 }
262 if (leadershipService.makeTopCandidate(leadershipTopic, nodeId)) { 272 if (leadershipService.makeTopCandidate(leadershipTopic, nodeId)) {
263 - return transitionFromMasterToStandby(deviceId); 273 + CompletableFuture<MastershipEvent> result = new CompletableFuture<>();
274 + // There is brief wait before we step down from mastership.
275 + // This is to ensure any work that happens when standby preference
276 + // order changes can complete. For example: flow entries need to be backed
277 + // to the new top standby (ONOS-1883)
278 + // FIXME: This potentially introduces a race-condition.
279 + // Right now role changes are only forced via CLI.
280 + transferExecutor.schedule(() -> {
281 + result.complete(transitionFromMasterToStandby(deviceId));
282 + }, WAIT_BEFORE_MASTERSHIP_HANDOFF_MILLIS, TimeUnit.MILLISECONDS);
283 + return result;
264 } else { 284 } else {
265 log.warn("Failed to promote {} to mastership for {}", nodeId, deviceId); 285 log.warn("Failed to promote {} to mastership for {}", nodeId, deviceId);
266 } 286 }
267 } 287 }
268 - return null; 288 + return CompletableFuture.completedFuture(null);
269 } 289 }
270 290
271 @Override 291 @Override
...@@ -278,13 +298,13 @@ public class ConsistentDeviceMastershipStore ...@@ -278,13 +298,13 @@ public class ConsistentDeviceMastershipStore
278 } 298 }
279 299
280 @Override 300 @Override
281 - public MastershipEvent setStandby(NodeId nodeId, DeviceId deviceId) { 301 + public CompletableFuture<MastershipEvent> setStandby(NodeId nodeId, DeviceId deviceId) {
282 checkArgument(nodeId != null, NODE_ID_NULL); 302 checkArgument(nodeId != null, NODE_ID_NULL);
283 checkArgument(deviceId != null, DEVICE_ID_NULL); 303 checkArgument(deviceId != null, DEVICE_ID_NULL);
284 304
285 NodeId currentMaster = getMaster(deviceId); 305 NodeId currentMaster = getMaster(deviceId);
286 if (!nodeId.equals(currentMaster)) { 306 if (!nodeId.equals(currentMaster)) {
287 - return null; 307 + return CompletableFuture.completedFuture(null);
288 } 308 }
289 309
290 String leadershipTopic = createDeviceMastershipTopic(deviceId); 310 String leadershipTopic = createDeviceMastershipTopic(deviceId);
...@@ -304,21 +324,26 @@ public class ConsistentDeviceMastershipStore ...@@ -304,21 +324,26 @@ public class ConsistentDeviceMastershipStore
304 } 324 }
305 325
306 @Override 326 @Override
307 - public MastershipEvent relinquishRole(NodeId nodeId, DeviceId deviceId) { 327 + public CompletableFuture<MastershipEvent> relinquishRole(NodeId nodeId, DeviceId deviceId) {
308 checkArgument(nodeId != null, NODE_ID_NULL); 328 checkArgument(nodeId != null, NODE_ID_NULL);
309 checkArgument(deviceId != null, DEVICE_ID_NULL); 329 checkArgument(deviceId != null, DEVICE_ID_NULL);
310 330
311 if (!nodeId.equals(localNodeId)) { 331 if (!nodeId.equals(localNodeId)) {
312 log.debug("Forwarding request to relinquish " 332 log.debug("Forwarding request to relinquish "
313 + "role for device {} to {}", deviceId, nodeId); 333 + "role for device {} to {}", deviceId, nodeId);
314 - return futureGetOrElse(clusterCommunicator.sendAndReceive( 334 + return clusterCommunicator.sendAndReceive(
315 deviceId, 335 deviceId,
316 ROLE_RELINQUISH_SUBJECT, 336 ROLE_RELINQUISH_SUBJECT,
317 SERIALIZER::encode, 337 SERIALIZER::encode,
318 SERIALIZER::decode, 338 SERIALIZER::decode,
319 - nodeId), null); 339 + nodeId);
340 + }
341 + return CompletableFuture.completedFuture(relinquishLocalRole(deviceId));
320 } 342 }
321 343
344 + private MastershipEvent relinquishLocalRole(DeviceId deviceId) {
345 + checkArgument(deviceId != null, DEVICE_ID_NULL);
346 +
322 // Check if this node is can be managed by this node. 347 // Check if this node is can be managed by this node.
323 if (!connectedDevices.contains(deviceId)) { 348 if (!connectedDevices.contains(deviceId)) {
324 return null; 349 return null;
......
...@@ -25,6 +25,7 @@ import java.util.List; ...@@ -25,6 +25,7 @@ import java.util.List;
25 import java.util.Map; 25 import java.util.Map;
26 import java.util.Map.Entry; 26 import java.util.Map.Entry;
27 import java.util.Set; 27 import java.util.Set;
28 +import java.util.concurrent.CompletableFuture;
28 29
29 import org.apache.felix.scr.annotations.Activate; 30 import org.apache.felix.scr.annotations.Activate;
30 import org.apache.felix.scr.annotations.Component; 31 import org.apache.felix.scr.annotations.Component;
...@@ -130,7 +131,7 @@ public class DistributedMastershipStore ...@@ -130,7 +131,7 @@ public class DistributedMastershipStore
130 } 131 }
131 132
132 @Override 133 @Override
133 - public MastershipEvent setMaster(NodeId newMaster, DeviceId deviceId) { 134 + public CompletableFuture<MastershipEvent> setMaster(NodeId newMaster, DeviceId deviceId) {
134 135
135 roleMap.lock(deviceId); 136 roleMap.lock(deviceId);
136 try { 137 try {
...@@ -147,7 +148,7 @@ public class DistributedMastershipStore ...@@ -147,7 +148,7 @@ public class DistributedMastershipStore
147 log.warn("{} was in both MASTER and STANDBY for {}", newMaster, deviceId); 148 log.warn("{} was in both MASTER and STANDBY for {}", newMaster, deviceId);
148 // trigger BACKUPS_CHANGED? 149 // trigger BACKUPS_CHANGED?
149 } 150 }
150 - return null; 151 + return CompletableFuture.completedFuture(null);
151 case STANDBY: 152 case STANDBY:
152 case NONE: 153 case NONE:
153 final NodeId currentMaster = rv.get(MASTER); 154 final NodeId currentMaster = rv.get(MASTER);
...@@ -163,10 +164,11 @@ public class DistributedMastershipStore ...@@ -163,10 +164,11 @@ public class DistributedMastershipStore
163 rv.reassign(newMaster, STANDBY, NONE); 164 rv.reassign(newMaster, STANDBY, NONE);
164 updateTerm(deviceId); 165 updateTerm(deviceId);
165 roleMap.put(deviceId, rv); 166 roleMap.put(deviceId, rv);
166 - return new MastershipEvent(MASTER_CHANGED, deviceId, rv.roleInfo()); 167 + return CompletableFuture.completedFuture(
168 + new MastershipEvent(MASTER_CHANGED, deviceId, rv.roleInfo()));
167 default: 169 default:
168 log.warn("unknown Mastership Role {}", currentRole); 170 log.warn("unknown Mastership Role {}", currentRole);
169 - return null; 171 + return CompletableFuture.completedFuture(null);
170 } 172 }
171 } finally { 173 } finally {
172 roleMap.unlock(deviceId); 174 roleMap.unlock(deviceId);
...@@ -282,7 +284,7 @@ public class DistributedMastershipStore ...@@ -282,7 +284,7 @@ public class DistributedMastershipStore
282 } 284 }
283 285
284 @Override 286 @Override
285 - public MastershipEvent setStandby(NodeId nodeId, DeviceId deviceId) { 287 + public CompletableFuture<MastershipEvent> setStandby(NodeId nodeId, DeviceId deviceId) {
286 // if nodeId was MASTER, rotate STANDBY 288 // if nodeId was MASTER, rotate STANDBY
287 // if nodeId was STANDBY no-op 289 // if nodeId was STANDBY no-op
288 // if nodeId was NONE, add to STANDBY 290 // if nodeId was NONE, add to STANDBY
...@@ -298,30 +300,33 @@ public class DistributedMastershipStore ...@@ -298,30 +300,33 @@ public class DistributedMastershipStore
298 updateTerm(deviceId); 300 updateTerm(deviceId);
299 if (newMaster != null) { 301 if (newMaster != null) {
300 roleMap.put(deviceId, rv); 302 roleMap.put(deviceId, rv);
301 - return new MastershipEvent(MASTER_CHANGED, deviceId, rv.roleInfo()); 303 + return CompletableFuture.completedFuture(
304 + new MastershipEvent(MASTER_CHANGED, deviceId, rv.roleInfo()));
302 } else { 305 } else {
303 // no master candidate 306 // no master candidate
304 roleMap.put(deviceId, rv); 307 roleMap.put(deviceId, rv);
305 // TBD: Should there be new event type for no MASTER? 308 // TBD: Should there be new event type for no MASTER?
306 - return new MastershipEvent(MASTER_CHANGED, deviceId, rv.roleInfo()); 309 + return CompletableFuture.completedFuture(
310 + new MastershipEvent(MASTER_CHANGED, deviceId, rv.roleInfo()));
307 } 311 }
308 case STANDBY: 312 case STANDBY:
309 - return null; 313 + return CompletableFuture.completedFuture(null);
310 case NONE: 314 case NONE:
311 rv.reassign(nodeId, NONE, STANDBY); 315 rv.reassign(nodeId, NONE, STANDBY);
312 roleMap.put(deviceId, rv); 316 roleMap.put(deviceId, rv);
313 - return new MastershipEvent(BACKUPS_CHANGED, deviceId, rv.roleInfo()); 317 + return CompletableFuture.completedFuture(
318 + new MastershipEvent(BACKUPS_CHANGED, deviceId, rv.roleInfo()));
314 default: 319 default:
315 log.warn("unknown Mastership Role {}", currentRole); 320 log.warn("unknown Mastership Role {}", currentRole);
316 } 321 }
317 - return null; 322 + return CompletableFuture.completedFuture(null);
318 } finally { 323 } finally {
319 roleMap.unlock(deviceId); 324 roleMap.unlock(deviceId);
320 } 325 }
321 } 326 }
322 327
323 @Override 328 @Override
324 - public MastershipEvent relinquishRole(NodeId nodeId, DeviceId deviceId) { 329 + public CompletableFuture<MastershipEvent> relinquishRole(NodeId nodeId, DeviceId deviceId) {
325 // relinquishRole is basically set to None 330 // relinquishRole is basically set to None
326 331
327 // If nodeId was master reelect next and remove nodeId 332 // If nodeId was master reelect next and remove nodeId
...@@ -337,13 +342,14 @@ public class DistributedMastershipStore ...@@ -337,13 +342,14 @@ public class DistributedMastershipStore
337 if (newMaster != null) { 342 if (newMaster != null) {
338 updateTerm(deviceId); 343 updateTerm(deviceId);
339 roleMap.put(deviceId, rv); 344 roleMap.put(deviceId, rv);
340 - return new MastershipEvent(MASTER_CHANGED, deviceId, rv.roleInfo()); 345 + return CompletableFuture.completedFuture(
346 + new MastershipEvent(MASTER_CHANGED, deviceId, rv.roleInfo()));
341 } else { 347 } else {
342 // No master candidate - no more backups, device is likely 348 // No master candidate - no more backups, device is likely
343 // fully disconnected 349 // fully disconnected
344 roleMap.put(deviceId, rv); 350 roleMap.put(deviceId, rv);
345 // Should there be new event type? 351 // Should there be new event type?
346 - return null; 352 + return CompletableFuture.completedFuture(null);
347 } 353 }
348 case STANDBY: 354 case STANDBY:
349 //fall through to reinforce relinquishment 355 //fall through to reinforce relinquishment
...@@ -351,13 +357,14 @@ public class DistributedMastershipStore ...@@ -351,13 +357,14 @@ public class DistributedMastershipStore
351 boolean modified = rv.reassign(nodeId, STANDBY, NONE); 357 boolean modified = rv.reassign(nodeId, STANDBY, NONE);
352 if (modified) { 358 if (modified) {
353 roleMap.put(deviceId, rv); 359 roleMap.put(deviceId, rv);
354 - return new MastershipEvent(BACKUPS_CHANGED, deviceId, rv.roleInfo()); 360 + return CompletableFuture.completedFuture(
361 + new MastershipEvent(BACKUPS_CHANGED, deviceId, rv.roleInfo()));
355 } 362 }
356 - return null; 363 + return CompletableFuture.completedFuture(null);
357 default: 364 default:
358 log.warn("unknown Mastership Role {}", currentRole); 365 log.warn("unknown Mastership Role {}", currentRole);
359 } 366 }
360 - return null; 367 + return CompletableFuture.completedFuture(null);
361 } finally { 368 } finally {
362 roleMap.unlock(deviceId); 369 roleMap.unlock(deviceId);
363 } 370 }
...@@ -374,10 +381,11 @@ public class DistributedMastershipStore ...@@ -374,10 +381,11 @@ public class DistributedMastershipStore
374 if (roleValue.contains(MASTER, nodeId) || 381 if (roleValue.contains(MASTER, nodeId) ||
375 roleValue.contains(STANDBY, nodeId)) { 382 roleValue.contains(STANDBY, nodeId)) {
376 383
377 - MastershipEvent event = relinquishRole(nodeId, deviceId); 384 + relinquishRole(nodeId, deviceId).whenComplete((event, error) -> {
378 if (event != null) { 385 if (event != null) {
379 events.add(event); 386 events.add(event);
380 } 387 }
388 + });
381 } 389 }
382 } 390 }
383 notifyDelegate(events); 391 notifyDelegate(events);
......
...@@ -44,6 +44,7 @@ import org.onosproject.store.hz.TestStoreManager; ...@@ -44,6 +44,7 @@ import org.onosproject.store.hz.TestStoreManager;
44 import org.onosproject.store.serializers.KryoSerializer; 44 import org.onosproject.store.serializers.KryoSerializer;
45 45
46 import com.google.common.collect.Sets; 46 import com.google.common.collect.Sets;
47 +import com.google.common.util.concurrent.Futures;
47 48
48 import static org.junit.Assert.assertEquals; 49 import static org.junit.Assert.assertEquals;
49 import static org.junit.Assert.assertNull; 50 import static org.junit.Assert.assertNull;
...@@ -168,15 +169,15 @@ public class DistributedMastershipStoreTest { ...@@ -168,15 +169,15 @@ public class DistributedMastershipStoreTest {
168 //populate maps with DID1, N1 as MASTER thru NONE case 169 //populate maps with DID1, N1 as MASTER thru NONE case
169 testStore.setCurrent(CN1); 170 testStore.setCurrent(CN1);
170 assertEquals("wrong role for NONE:", MASTER, dms.requestRole(DID1)); 171 assertEquals("wrong role for NONE:", MASTER, dms.requestRole(DID1));
171 - assertNull("wrong event:", dms.setMaster(N1, DID1)); 172 + assertNull("wrong event:", Futures.getUnchecked(dms.setMaster(N1, DID1)));
172 173
173 //switch over to N2 174 //switch over to N2
174 - assertEquals("wrong event:", Type.MASTER_CHANGED, dms.setMaster(N2, DID1).type()); 175 + assertEquals("wrong event:", Type.MASTER_CHANGED, Futures.getUnchecked(dms.setMaster(N2, DID1)).type());
175 System.out.println(dms.getTermFor(DID1).master() + ":" + dms.getTermFor(DID1).termNumber()); 176 System.out.println(dms.getTermFor(DID1).master() + ":" + dms.getTermFor(DID1).termNumber());
176 assertEquals("wrong term", MastershipTerm.of(N2, 2), dms.getTermFor(DID1)); 177 assertEquals("wrong term", MastershipTerm.of(N2, 2), dms.getTermFor(DID1));
177 178
178 //orphan switch - should be rare case 179 //orphan switch - should be rare case
179 - assertEquals("wrong event:", Type.MASTER_CHANGED, dms.setMaster(N2, DID2).type()); 180 + assertEquals("wrong event:", Type.MASTER_CHANGED, Futures.getUnchecked(dms.setMaster(N2, DID2)).type());
180 assertEquals("wrong term", MastershipTerm.of(N2, 1), dms.getTermFor(DID2)); 181 assertEquals("wrong term", MastershipTerm.of(N2, 1), dms.getTermFor(DID2));
181 //disconnect and reconnect - sign of failing re-election or single-instance channel 182 //disconnect and reconnect - sign of failing re-election or single-instance channel
182 dms.roleMap.clear(); 183 dms.roleMap.clear();
...@@ -190,18 +191,18 @@ public class DistributedMastershipStoreTest { ...@@ -190,18 +191,18 @@ public class DistributedMastershipStoreTest {
190 testStore.setCurrent(CN1); 191 testStore.setCurrent(CN1);
191 assertEquals("wrong role for NONE:", MASTER, dms.requestRole(DID1)); 192 assertEquals("wrong role for NONE:", MASTER, dms.requestRole(DID1));
192 //no backup, no new MASTER/event 193 //no backup, no new MASTER/event
193 - assertNull("wrong event:", dms.relinquishRole(N1, DID1)); 194 + assertNull("wrong event:", Futures.getUnchecked(dms.relinquishRole(N1, DID1)));
194 195
195 dms.requestRole(DID1); 196 dms.requestRole(DID1);
196 197
197 //add backup CN2, get it elected MASTER by relinquishing 198 //add backup CN2, get it elected MASTER by relinquishing
198 testStore.setCurrent(CN2); 199 testStore.setCurrent(CN2);
199 assertEquals("wrong role for NONE:", STANDBY, dms.requestRole(DID1)); 200 assertEquals("wrong role for NONE:", STANDBY, dms.requestRole(DID1));
200 - assertEquals("wrong event:", Type.MASTER_CHANGED, dms.relinquishRole(N1, DID1).type()); 201 + assertEquals("wrong event:", Type.MASTER_CHANGED, Futures.getUnchecked(dms.relinquishRole(N1, DID1)).type());
201 assertEquals("wrong master", N2, dms.getMaster(DID1)); 202 assertEquals("wrong master", N2, dms.getMaster(DID1));
202 203
203 //all nodes "give up" on device, which goes back to NONE. 204 //all nodes "give up" on device, which goes back to NONE.
204 - assertNull("wrong event:", dms.relinquishRole(N2, DID1)); 205 + assertNull("wrong event:", Futures.getUnchecked(dms.relinquishRole(N2, DID1)));
205 assertEquals("wrong role for node:", NONE, dms.getRole(N2, DID1)); 206 assertEquals("wrong role for node:", NONE, dms.getRole(N2, DID1));
206 207
207 assertEquals("wrong number of retired nodes", 2, 208 assertEquals("wrong number of retired nodes", 2,
...@@ -215,11 +216,11 @@ public class DistributedMastershipStoreTest { ...@@ -215,11 +216,11 @@ public class DistributedMastershipStoreTest {
215 dms.roleMap.get(DID1).nodesOfRole(STANDBY).size()); 216 dms.roleMap.get(DID1).nodesOfRole(STANDBY).size());
216 217
217 //If STANDBY, should drop to NONE 218 //If STANDBY, should drop to NONE
218 - assertEquals("wrong event:", Type.BACKUPS_CHANGED, dms.relinquishRole(N1, DID1).type()); 219 + assertEquals("wrong event:", Type.BACKUPS_CHANGED, Futures.getUnchecked(dms.relinquishRole(N1, DID1)).type());
219 assertEquals("wrong role for node:", NONE, dms.getRole(N1, DID1)); 220 assertEquals("wrong role for node:", NONE, dms.getRole(N1, DID1));
220 221
221 //NONE - nothing happens 222 //NONE - nothing happens
222 - assertEquals("wrong event:", Type.BACKUPS_CHANGED, dms.relinquishRole(N1, DID2).type()); 223 + assertEquals("wrong event:", Type.BACKUPS_CHANGED, Futures.getUnchecked(dms.relinquishRole(N1, DID2)).type());
223 assertEquals("wrong role for node:", NONE, dms.getRole(N1, DID2)); 224 assertEquals("wrong role for node:", NONE, dms.getRole(N1, DID2));
224 225
225 } 226 }
......
...@@ -27,6 +27,7 @@ import java.util.List; ...@@ -27,6 +27,7 @@ import java.util.List;
27 import java.util.Map; 27 import java.util.Map;
28 import java.util.Objects; 28 import java.util.Objects;
29 import java.util.Set; 29 import java.util.Set;
30 +import java.util.concurrent.CompletableFuture;
30 import java.util.concurrent.atomic.AtomicInteger; 31 import java.util.concurrent.atomic.AtomicInteger;
31 32
32 import org.apache.felix.scr.annotations.Activate; 33 import org.apache.felix.scr.annotations.Activate;
...@@ -143,13 +144,13 @@ public class SimpleMastershipStore ...@@ -143,13 +144,13 @@ public class SimpleMastershipStore
143 } 144 }
144 145
145 @Override 146 @Override
146 - public synchronized MastershipEvent setMaster(NodeId nodeId, DeviceId deviceId) { 147 + public synchronized CompletableFuture<MastershipEvent> setMaster(NodeId nodeId, DeviceId deviceId) {
147 148
148 MastershipRole role = getRole(nodeId, deviceId); 149 MastershipRole role = getRole(nodeId, deviceId);
149 switch (role) { 150 switch (role) {
150 case MASTER: 151 case MASTER:
151 // no-op 152 // no-op
152 - return null; 153 + return CompletableFuture.completedFuture(null);
153 case STANDBY: 154 case STANDBY:
154 case NONE: 155 case NONE:
155 NodeId prevMaster = masterMap.put(deviceId, nodeId); 156 NodeId prevMaster = masterMap.put(deviceId, nodeId);
...@@ -162,8 +163,8 @@ public class SimpleMastershipStore ...@@ -162,8 +163,8 @@ public class SimpleMastershipStore
162 return null; 163 return null;
163 } 164 }
164 165
165 - return new MastershipEvent(MASTER_CHANGED, deviceId, 166 + return CompletableFuture.completedFuture(
166 - getNodes(deviceId)); 167 + new MastershipEvent(MASTER_CHANGED, deviceId, getNodes(deviceId)));
167 } 168 }
168 169
169 @Override 170 @Override
...@@ -285,7 +286,7 @@ public class SimpleMastershipStore ...@@ -285,7 +286,7 @@ public class SimpleMastershipStore
285 } 286 }
286 287
287 @Override 288 @Override
288 - public synchronized MastershipEvent setStandby(NodeId nodeId, DeviceId deviceId) { 289 + public synchronized CompletableFuture<MastershipEvent> setStandby(NodeId nodeId, DeviceId deviceId) {
289 MastershipRole role = getRole(nodeId, deviceId); 290 MastershipRole role = getRole(nodeId, deviceId);
290 switch (role) { 291 switch (role) {
291 case MASTER: 292 case MASTER:
...@@ -294,22 +295,22 @@ public class SimpleMastershipStore ...@@ -294,22 +295,22 @@ public class SimpleMastershipStore
294 // no master alternative 295 // no master alternative
295 masterMap.remove(deviceId); 296 masterMap.remove(deviceId);
296 // TODO: Should there be new event type for no MASTER? 297 // TODO: Should there be new event type for no MASTER?
297 - return new MastershipEvent(MASTER_CHANGED, deviceId, 298 + return CompletableFuture.completedFuture(
298 - getNodes(deviceId)); 299 + new MastershipEvent(MASTER_CHANGED, deviceId, getNodes(deviceId)));
299 } else { 300 } else {
300 NodeId prevMaster = masterMap.put(deviceId, backup); 301 NodeId prevMaster = masterMap.put(deviceId, backup);
301 incrementTerm(deviceId); 302 incrementTerm(deviceId);
302 addToBackup(deviceId, prevMaster); 303 addToBackup(deviceId, prevMaster);
303 - return new MastershipEvent(MASTER_CHANGED, deviceId, 304 + return CompletableFuture.completedFuture(
304 - getNodes(deviceId)); 305 + new MastershipEvent(MASTER_CHANGED, deviceId, getNodes(deviceId)));
305 } 306 }
306 307
307 case STANDBY: 308 case STANDBY:
308 case NONE: 309 case NONE:
309 boolean modified = addToBackup(deviceId, nodeId); 310 boolean modified = addToBackup(deviceId, nodeId);
310 if (modified) { 311 if (modified) {
311 - return new MastershipEvent(BACKUPS_CHANGED, deviceId, 312 + return CompletableFuture.completedFuture(
312 - getNodes(deviceId)); 313 + new MastershipEvent(BACKUPS_CHANGED, deviceId, getNodes(deviceId)));
313 } 314 }
314 break; 315 break;
315 316
...@@ -335,20 +336,20 @@ public class SimpleMastershipStore ...@@ -335,20 +336,20 @@ public class SimpleMastershipStore
335 } 336 }
336 337
337 @Override 338 @Override
338 - public synchronized MastershipEvent relinquishRole(NodeId nodeId, DeviceId deviceId) { 339 + public synchronized CompletableFuture<MastershipEvent> relinquishRole(NodeId nodeId, DeviceId deviceId) {
339 MastershipRole role = getRole(nodeId, deviceId); 340 MastershipRole role = getRole(nodeId, deviceId);
340 switch (role) { 341 switch (role) {
341 case MASTER: 342 case MASTER:
342 NodeId backup = reelect(deviceId, nodeId); 343 NodeId backup = reelect(deviceId, nodeId);
343 masterMap.put(deviceId, backup); 344 masterMap.put(deviceId, backup);
344 incrementTerm(deviceId); 345 incrementTerm(deviceId);
345 - return new MastershipEvent(MASTER_CHANGED, deviceId, 346 + return CompletableFuture.completedFuture(
346 - getNodes(deviceId)); 347 + new MastershipEvent(MASTER_CHANGED, deviceId, getNodes(deviceId)));
347 348
348 case STANDBY: 349 case STANDBY:
349 if (removeFromBackups(deviceId, nodeId)) { 350 if (removeFromBackups(deviceId, nodeId)) {
350 - return new MastershipEvent(BACKUPS_CHANGED, deviceId, 351 + return CompletableFuture.completedFuture(
351 - getNodes(deviceId)); 352 + new MastershipEvent(BACKUPS_CHANGED, deviceId, getNodes(deviceId)));
352 } 353 }
353 break; 354 break;
354 355
...@@ -358,12 +359,12 @@ public class SimpleMastershipStore ...@@ -358,12 +359,12 @@ public class SimpleMastershipStore
358 default: 359 default:
359 log.warn("unknown Mastership Role {}", role); 360 log.warn("unknown Mastership Role {}", role);
360 } 361 }
361 - return null; 362 + return CompletableFuture.completedFuture(null);
362 } 363 }
363 364
364 @Override 365 @Override
365 public synchronized void relinquishAllRole(NodeId nodeId) { 366 public synchronized void relinquishAllRole(NodeId nodeId) {
366 - List<MastershipEvent> events = new ArrayList<>(); 367 + List<CompletableFuture<MastershipEvent>> eventFutures = new ArrayList<>();
367 Set<DeviceId> toRelinquish = new HashSet<>(); 368 Set<DeviceId> toRelinquish = new HashSet<>();
368 369
369 masterMap.entrySet().stream() 370 masterMap.entrySet().stream()
...@@ -375,12 +376,13 @@ public class SimpleMastershipStore ...@@ -375,12 +376,13 @@ public class SimpleMastershipStore
375 .forEach(entry -> toRelinquish.add(entry.getKey())); 376 .forEach(entry -> toRelinquish.add(entry.getKey()));
376 377
377 toRelinquish.forEach(deviceId -> { 378 toRelinquish.forEach(deviceId -> {
378 - MastershipEvent event = relinquishRole(nodeId, deviceId); 379 + eventFutures.add(relinquishRole(nodeId, deviceId));
379 - if (event != null) {
380 - events.add(event);
381 - }
382 }); 380 });
383 381
384 - notifyDelegate(events); 382 + eventFutures.forEach(future -> {
383 + future.whenComplete((event, error) -> {
384 + notifyDelegate(event);
385 + });
386 + });
385 } 387 }
386 } 388 }
......
...@@ -29,6 +29,7 @@ import org.onosproject.mastership.MastershipTerm; ...@@ -29,6 +29,7 @@ import org.onosproject.mastership.MastershipTerm;
29 import org.onosproject.net.DeviceId; 29 import org.onosproject.net.DeviceId;
30 30
31 import com.google.common.collect.Sets; 31 import com.google.common.collect.Sets;
32 +import com.google.common.util.concurrent.Futures;
32 33
33 import static org.junit.Assert.assertEquals; 34 import static org.junit.Assert.assertEquals;
34 import static org.junit.Assert.assertNull; 35 import static org.junit.Assert.assertNull;
...@@ -92,15 +93,15 @@ public class SimpleMastershipStoreTest { ...@@ -92,15 +93,15 @@ public class SimpleMastershipStoreTest {
92 @Test 93 @Test
93 public void setMaster() { 94 public void setMaster() {
94 put(DID1, N1, false, false); 95 put(DID1, N1, false, false);
95 - assertEquals("wrong event", MASTER_CHANGED, sms.setMaster(N1, DID1).type()); 96 + assertEquals("wrong event", MASTER_CHANGED, Futures.getUnchecked(sms.setMaster(N1, DID1)).type());
96 assertEquals("wrong role", MASTER, sms.getRole(N1, DID1)); 97 assertEquals("wrong role", MASTER, sms.getRole(N1, DID1));
97 //set node that's already master - should be ignored 98 //set node that's already master - should be ignored
98 - assertNull("wrong event", sms.setMaster(N1, DID1)); 99 + assertNull("wrong event", Futures.getUnchecked(sms.setMaster(N1, DID1)));
99 100
100 //set STANDBY to MASTER 101 //set STANDBY to MASTER
101 put(DID2, N1, false, true); 102 put(DID2, N1, false, true);
102 assertEquals("wrong role", STANDBY, sms.getRole(N1, DID2)); 103 assertEquals("wrong role", STANDBY, sms.getRole(N1, DID2));
103 - assertEquals("wrong event", MASTER_CHANGED, sms.setMaster(N1, DID2).type()); 104 + assertEquals("wrong event", MASTER_CHANGED, Futures.getUnchecked(sms.setMaster(N1, DID2)).type());
104 assertEquals("wrong role", MASTER, sms.getRole(N1, DID2)); 105 assertEquals("wrong role", MASTER, sms.getRole(N1, DID2));
105 } 106 }
106 107
...@@ -156,7 +157,7 @@ public class SimpleMastershipStoreTest { ...@@ -156,7 +157,7 @@ public class SimpleMastershipStoreTest {
156 157
157 //no backup, MASTER 158 //no backup, MASTER
158 put(DID1, N1, true, false); 159 put(DID1, N1, true, false);
159 - assertNull("expect no MASTER event", sms.setStandby(N1, DID1).roleInfo().master()); 160 + assertNull("expect no MASTER event", Futures.getUnchecked(sms.setStandby(N1, DID1)).roleInfo().master());
160 assertNull("wrong node", sms.masterMap.get(DID1)); 161 assertNull("wrong node", sms.masterMap.get(DID1));
161 162
162 //backup, switch 163 //backup, switch
...@@ -164,7 +165,7 @@ public class SimpleMastershipStoreTest { ...@@ -164,7 +165,7 @@ public class SimpleMastershipStoreTest {
164 put(DID1, N1, true, true); 165 put(DID1, N1, true, true);
165 put(DID1, N2, false, true); 166 put(DID1, N2, false, true);
166 put(DID2, N2, true, true); 167 put(DID2, N2, true, true);
167 - MastershipEvent event = sms.setStandby(N1, DID1); 168 + MastershipEvent event = Futures.getUnchecked(sms.setStandby(N1, DID1));
168 assertEquals("wrong event", MASTER_CHANGED, event.type()); 169 assertEquals("wrong event", MASTER_CHANGED, event.type());
169 assertEquals("wrong master", N2, event.roleInfo().master()); 170 assertEquals("wrong master", N2, event.roleInfo().master());
170 } 171 }
......