Remove @(Not)ThreadSafe annotation
[mdsal.git] / singleton-service / mdsal-singleton-dom-impl / src / main / java / org / opendaylight / mdsal / singleton / dom / impl / ClusterSingletonServiceGroupImpl.java
1 /*
2  * Copyright (c) 2016 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.mdsal.singleton.dom.impl;
9
10 import static com.google.common.base.Preconditions.checkArgument;
11 import static com.google.common.base.Preconditions.checkState;
12 import static com.google.common.base.Verify.verify;
13 import static com.google.common.base.Verify.verifyNotNull;
14 import static java.util.Objects.requireNonNull;
15
16 import com.google.common.annotations.VisibleForTesting;
17 import com.google.common.base.MoreObjects;
18 import com.google.common.collect.ImmutableList;
19 import com.google.common.collect.ImmutableSet;
20 import com.google.common.util.concurrent.FutureCallback;
21 import com.google.common.util.concurrent.Futures;
22 import com.google.common.util.concurrent.ListenableFuture;
23 import com.google.common.util.concurrent.MoreExecutors;
24 import com.google.common.util.concurrent.SettableFuture;
25 import java.util.Collection;
26 import java.util.HashMap;
27 import java.util.Iterator;
28 import java.util.Map;
29 import java.util.Map.Entry;
30 import java.util.Set;
31 import java.util.concurrent.ConcurrentHashMap;
32 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
33 import java.util.concurrent.atomic.AtomicReference;
34 import javax.annotation.concurrent.GuardedBy;
35 import org.eclipse.jdt.annotation.NonNull;
36 import org.opendaylight.mdsal.eos.common.api.CandidateAlreadyRegisteredException;
37 import org.opendaylight.mdsal.eos.common.api.EntityOwnershipChangeState;
38 import org.opendaylight.mdsal.eos.common.api.GenericEntity;
39 import org.opendaylight.mdsal.eos.common.api.GenericEntityOwnershipCandidateRegistration;
40 import org.opendaylight.mdsal.eos.common.api.GenericEntityOwnershipChange;
41 import org.opendaylight.mdsal.eos.common.api.GenericEntityOwnershipListener;
42 import org.opendaylight.mdsal.eos.common.api.GenericEntityOwnershipService;
43 import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonService;
44 import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceRegistration;
45 import org.opendaylight.yangtools.concepts.Path;
46 import org.slf4j.Logger;
47 import org.slf4j.LoggerFactory;
48
49 /**
50  * Implementation of {@link ClusterSingletonServiceGroup} on top of the Entity Ownership Service. Since EOS is atomic
51  * in its operation and singleton services incur startup and most notably cleanup, we need to do something smart here.
52  *
53  * <p>
54  * The implementation takes advantage of the fact that EOS provides stable ownership, i.e. owners are not moved as
55  * a result on new candidates appearing. We use two entities:
56  * - service entity, to which all nodes register
57  * - cleanup entity, which only the service entity owner registers to
58  *
59  * <p>
60  * Once the cleanup entity ownership is acquired, services are started. As long as the cleanup entity is registered,
61  * it should remain the owner. In case a new service owner emerges, the old owner will start the cleanup process,
62  * eventually releasing the cleanup entity. The new owner registers for the cleanup entity -- but will not see it
63  * granted until the old owner finishes the cleanup.
64  *
65  * @param <P> the instance identifier path type
66  * @param <E> the GenericEntity type
67  * @param <C> the GenericEntityOwnershipChange type
68  * @param <G> the GenericEntityOwnershipListener type
69  * @param <S> the GenericEntityOwnershipService type
70  */
71 final class ClusterSingletonServiceGroupImpl<P extends Path<P>, E extends GenericEntity<P>,
72         C extends GenericEntityOwnershipChange<P, E>,  G extends GenericEntityOwnershipListener<P, C>,
73         S extends GenericEntityOwnershipService<P, E, G>> extends ClusterSingletonServiceGroup<P, E, C> {
74
75     private enum EntityState {
76         /**
77          * This entity was never registered.
78          */
79         UNREGISTERED,
80         /**
81          * Registration exists, but we are waiting for it to resolve.
82          */
83         REGISTERED,
84         /**
85          * Registration indicated we are the owner.
86          */
87         OWNED,
88         /**
89          * Registration indicated we are the owner, but global state is uncertain -- meaning there can be owners in
90          * another partition, for example.
91          */
92         OWNED_JEOPARDY,
93         /**
94          * Registration indicated we are not the owner. In this state we do not care about global state, therefore we
95          * do not need an UNOWNED_JEOPARDY state.
96          */
97         UNOWNED,
98     }
99
100     enum ServiceState {
101         /**
102          * Local service is up and running.
103          */
104         // FIXME: we should support async startup, which will require a STARTING state.
105         STARTED,
106         /**
107          * Local service is being stopped.
108          */
109         STOPPING,
110     }
111
112     private static final Logger LOG = LoggerFactory.getLogger(ClusterSingletonServiceGroupImpl.class);
113
114     private final S entityOwnershipService;
115     private final String identifier;
116
117     /* Entity instances */
118     private final E serviceEntity;
119     private final E cleanupEntity;
120
121     private final Set<ClusterSingletonServiceRegistration> members = ConcurrentHashMap.newKeySet();
122     // Guarded by lock
123     private final Map<ClusterSingletonServiceRegistration, ServiceInfo> services = new HashMap<>();
124
125     // Marker for when any state changed
126     @SuppressWarnings("rawtypes")
127     private static final AtomicIntegerFieldUpdater<ClusterSingletonServiceGroupImpl> DIRTY_UPDATER =
128             AtomicIntegerFieldUpdater.newUpdater(ClusterSingletonServiceGroupImpl.class, "dirty");
129     private volatile int dirty;
130
131     // Simplified lock: non-reentrant, support tryLock() only
132     @SuppressWarnings("rawtypes")
133     private static final AtomicIntegerFieldUpdater<ClusterSingletonServiceGroupImpl> LOCK_UPDATER =
134             AtomicIntegerFieldUpdater.newUpdater(ClusterSingletonServiceGroupImpl.class, "lock");
135     @SuppressWarnings("unused")
136     private volatile int lock;
137
138     /*
139      * State tracking is quite involved, as we are tracking up to four asynchronous sources of events:
140      * - user calling close()
141      * - service entity ownership
142      * - cleanup entity ownership
143      * - service shutdown future
144      *
145      * Absolutely correct solution would be a set of behaviors, which govern each state, remembering where we want to
146      * get to and what we are doing. That would result in ~15 classes which would quickly render this code unreadable
147      * due to boilerplate overhead.
148      *
149      * We therefore take a different approach, tracking state directly in this class and evaluate state transitions
150      * based on recorded bits -- without explicit representation of state machine state.
151      */
152     /**
153      * Group close future. In can only go from null to non-null reference. Whenever it is non-null, it indicates that
154      * the user has closed the group and we are converging to termination.
155      */
156     // We are using volatile get-and-set to support non-blocking close(). It may be more efficient to inline it here,
157     // as we perform a volatile read after unlocking -- that volatile read may easier on L1 cache.
158     // XXX: above needs a microbenchmark contention ever becomes a problem.
159     private final AtomicReference<SettableFuture<Void>> closeFuture = new AtomicReference<>();
160
161     /**
162      * Service (base) entity registration. This entity selects an owner candidate across nodes. Candidates proceed to
163      * acquire {@link #cleanupEntity}.
164      */
165     @GuardedBy("this")
166     private GenericEntityOwnershipCandidateRegistration<P, E> serviceEntityReg = null;
167     /**
168      * Service (base) entity last reported state.
169      */
170     @GuardedBy("this")
171     private EntityState serviceEntityState = EntityState.UNREGISTERED;
172
173     /**
174      * Cleanup (owner) entity registration. This entity guards access to service state and coordinates shutdown cleanup
175      * and startup.
176      */
177     @GuardedBy("this")
178     private GenericEntityOwnershipCandidateRegistration<P, E> cleanupEntityReg;
179     /**
180      * Cleanup (owner) entity last reported state.
181      */
182     @GuardedBy("this")
183     private EntityState cleanupEntityState = EntityState.UNREGISTERED;
184
185     private volatile boolean initialized;
186
187     /**
188      * Class constructor. Note: last argument is reused as-is.
189      *
190      * @param identifier non-empty string as identifier
191      * @param mainEntity as Entity instance
192      * @param closeEntity as Entity instance
193      * @param entityOwnershipService GenericEntityOwnershipService instance
194      * @param parent parent service
195      * @param services Services list
196      */
197     ClusterSingletonServiceGroupImpl(final String identifier, final S entityOwnershipService, final E mainEntity,
198             final E closeEntity, final Collection<ClusterSingletonServiceRegistration> services) {
199         checkArgument(!identifier.isEmpty(), "Identifier may not be empty");
200         this.identifier = identifier;
201         this.entityOwnershipService = requireNonNull(entityOwnershipService);
202         this.serviceEntity = requireNonNull(mainEntity);
203         this.cleanupEntity = requireNonNull(closeEntity);
204         members.addAll(services);
205
206         LOG.debug("Instantiated new service group for {}", identifier);
207     }
208
209     @VisibleForTesting
210     ClusterSingletonServiceGroupImpl(final String identifier, final E mainEntity,
211             final E closeEntity, final S entityOwnershipService) {
212         this(identifier, entityOwnershipService, mainEntity, closeEntity, ImmutableList.of());
213     }
214
215     @Override
216     public String getIdentifier() {
217         return identifier;
218     }
219
220     @Override
221     ListenableFuture<?> closeClusterSingletonGroup() {
222         final ListenableFuture<?> ret = destroyGroup();
223         members.clear();
224         markDirty();
225
226         if (tryLock()) {
227             reconcileState();
228         } else {
229             LOG.debug("Service group {} postponing sync on close", identifier);
230         }
231
232         return ret;
233     }
234
235     private boolean isClosed() {
236         return closeFuture.get() != null;
237     }
238
239     @Override
240     void initialize() throws CandidateAlreadyRegisteredException {
241         verify(tryLock());
242         try {
243             checkState(!initialized, "Singleton group %s was already initilized", identifier);
244             LOG.debug("Initializing service group {} with services {}", identifier, members);
245             synchronized (this) {
246                 serviceEntityState = EntityState.REGISTERED;
247                 serviceEntityReg = entityOwnershipService.registerCandidate(serviceEntity);
248                 initialized = true;
249             }
250         } finally {
251             unlock();
252         }
253     }
254
255     private void checkNotClosed() {
256         checkState(!isClosed(), "Service group %s has already been closed", identifier);
257     }
258
259     @Override
260     void registerService(final ClusterSingletonServiceRegistration reg) {
261         final ClusterSingletonService service = reg.getInstance();
262         verify(identifier.equals(service.getIdentifier().getValue()));
263         checkNotClosed();
264
265         checkState(initialized, "Service group %s is not initialized yet", identifier);
266
267         // First put the service
268         LOG.debug("Adding service {} to service group {}", service, identifier);
269         verify(members.add(reg));
270         markDirty();
271
272         if (!tryLock()) {
273             LOG.debug("Service group {} delayed register of {}", identifier, reg);
274             return;
275         }
276
277         reconcileState();
278     }
279
280     @Override
281     ListenableFuture<?> unregisterService(final ClusterSingletonServiceRegistration reg) {
282         final ClusterSingletonService service = reg.getInstance();
283         verify(identifier.equals(service.getIdentifier().getValue()));
284         checkNotClosed();
285
286         verify(members.remove(reg));
287         markDirty();
288         if (members.isEmpty()) {
289             // We need to let AbstractClusterSingletonServiceProviderImpl know this group is to be shutdown
290             // before we start applying state, because while we do not re-enter, the user is free to do whatever,
291             // notably including registering a service with the same ID from the service shutdown hook. That
292             // registration request needs to hit the successor of this group.
293             return destroyGroup();
294         }
295
296         if (tryLock()) {
297             reconcileState();
298         } else {
299             LOG.debug("Service group {} delayed unregister of {}", identifier, reg);
300         }
301         return null;
302     }
303
304     private synchronized @NonNull ListenableFuture<?> destroyGroup() {
305         final SettableFuture<Void> future = SettableFuture.create();
306         if (!closeFuture.compareAndSet(null, future)) {
307             return verifyNotNull(closeFuture.get());
308         }
309
310         if (serviceEntityReg != null) {
311             // We are still holding the service registration, close it now...
312             LOG.debug("Service group {} unregistering service entity {}", identifier, serviceEntity);
313             serviceEntityReg.close();
314             serviceEntityReg = null;
315         }
316
317         markDirty();
318         return future;
319     }
320
321     @Override
322     void ownershipChanged(final C ownershipChange) {
323         LOG.debug("Ownership change {} for ClusterSingletonServiceGroup {}", ownershipChange, identifier);
324
325         synchronized (this) {
326             lockedOwnershipChanged(ownershipChange);
327         }
328
329         if (isDirty()) {
330             if (!tryLock()) {
331                 LOG.debug("Service group {} postponing ownership change sync", identifier);
332                 return;
333             }
334
335             reconcileState();
336         }
337     }
338
339     /**
340      * Handle an ownership change with the lock held. Callers are expected to handle termination conditions, this method
341      * and anything it calls must not call {@link #lockedClose(SettableFuture)}.
342      *
343      * @param ownershipChange reported change
344      */
345     @GuardedBy("this")
346     private void lockedOwnershipChanged(final C ownershipChange) {
347         final E entity = ownershipChange.getEntity();
348         if (serviceEntity.equals(entity)) {
349             serviceOwnershipChanged(ownershipChange.getState(), ownershipChange.inJeopardy());
350             markDirty();
351         } else if (cleanupEntity.equals(entity)) {
352             cleanupCandidateOwnershipChanged(ownershipChange.getState(), ownershipChange.inJeopardy());
353             markDirty();
354         } else {
355             LOG.warn("Group {} received unrecognized change {}", identifier, ownershipChange);
356         }
357     }
358
359     @GuardedBy("this")
360     private void cleanupCandidateOwnershipChanged(final EntityOwnershipChangeState state, final boolean jeopardy) {
361         if (jeopardy) {
362             switch (state) {
363                 case LOCAL_OWNERSHIP_GRANTED:
364                 case LOCAL_OWNERSHIP_RETAINED_WITH_NO_CHANGE:
365                     LOG.warn("Service group {} cleanup entity owned without certainty", identifier);
366                     cleanupEntityState = EntityState.OWNED_JEOPARDY;
367                     break;
368                 case LOCAL_OWNERSHIP_LOST_NEW_OWNER:
369                 case LOCAL_OWNERSHIP_LOST_NO_OWNER:
370                 case REMOTE_OWNERSHIP_CHANGED:
371                 case REMOTE_OWNERSHIP_LOST_NO_OWNER:
372                     LOG.info("Service group {} cleanup entity ownership uncertain", identifier);
373                     cleanupEntityState = EntityState.UNOWNED;
374                     break;
375                 default:
376                     throw new IllegalStateException("Unhandled cleanup entity jeopardy change " + state);
377             }
378
379             return;
380         }
381
382         if (cleanupEntityState == EntityState.OWNED_JEOPARDY) {
383             // Pair info message with previous jeopardy
384             LOG.info("Service group {} cleanup entity ownership ascertained", identifier);
385         }
386
387         switch (state) {
388             case LOCAL_OWNERSHIP_GRANTED:
389             case LOCAL_OWNERSHIP_RETAINED_WITH_NO_CHANGE:
390                 cleanupEntityState = EntityState.OWNED;
391                 break;
392             case LOCAL_OWNERSHIP_LOST_NEW_OWNER:
393             case LOCAL_OWNERSHIP_LOST_NO_OWNER:
394             case REMOTE_OWNERSHIP_LOST_NO_OWNER:
395             case REMOTE_OWNERSHIP_CHANGED:
396                 cleanupEntityState = EntityState.UNOWNED;
397                 break;
398             default:
399                 LOG.warn("Service group {} ignoring unhandled cleanup entity change {}", identifier, state);
400                 break;
401         }
402     }
403
404     @GuardedBy("this")
405     private void serviceOwnershipChanged(final EntityOwnershipChangeState state, final boolean jeopardy) {
406         if (jeopardy) {
407             LOG.info("Service group {} service entity ownership uncertain", identifier);
408             switch (state) {
409                 case LOCAL_OWNERSHIP_GRANTED:
410                 case LOCAL_OWNERSHIP_RETAINED_WITH_NO_CHANGE:
411                     serviceEntityState = EntityState.OWNED_JEOPARDY;
412                     break;
413                 case LOCAL_OWNERSHIP_LOST_NEW_OWNER:
414                 case LOCAL_OWNERSHIP_LOST_NO_OWNER:
415                 case REMOTE_OWNERSHIP_CHANGED:
416                 case REMOTE_OWNERSHIP_LOST_NO_OWNER:
417                     serviceEntityState = EntityState.UNOWNED;
418                     break;
419                 default:
420                     throw new IllegalStateException("Unhandled cleanup entity jeopardy change " + state);
421             }
422             return;
423         }
424
425         if (serviceEntityState == EntityState.OWNED_JEOPARDY) {
426             // Pair info message with previous jeopardy
427             LOG.info("Service group {} service entity ownership ascertained", identifier);
428         }
429
430         switch (state) {
431             case LOCAL_OWNERSHIP_GRANTED:
432             case LOCAL_OWNERSHIP_RETAINED_WITH_NO_CHANGE:
433                 LOG.debug("Service group {} acquired service entity ownership", identifier);
434                 serviceEntityState = EntityState.OWNED;
435                 break;
436             case LOCAL_OWNERSHIP_LOST_NEW_OWNER:
437             case LOCAL_OWNERSHIP_LOST_NO_OWNER:
438             case REMOTE_OWNERSHIP_CHANGED:
439             case REMOTE_OWNERSHIP_LOST_NO_OWNER:
440                 LOG.debug("Service group {} lost service entity ownership", identifier);
441                 serviceEntityState = EntityState.UNOWNED;
442                 break;
443             default:
444                 LOG.warn("Service group {} ignoring unhandled cleanup entity change {}", identifier, state);
445         }
446     }
447
448     // has to be called with lock asserted, which will be released prior to returning
449     private void reconcileState() {
450         // Always check if there is any state change to be applied.
451         while (true) {
452             try {
453                 if (conditionalClean()) {
454                     tryReconcileState();
455                 }
456             } finally {
457                 // We may have ran a round of reconciliation, but the either one of may have happened asynchronously:
458                 // - registration
459                 // - unregistration
460                 // - service future completed
461                 // - entity state changed
462                 //
463                 // We are dropping the lock, but we need to recheck dirty and try to apply state again if it is found to
464                 // be dirty again. This closes the following race condition:
465                 //
466                 // A: runs these checks holding the lock
467                 // B: modifies them, fails to acquire lock
468                 // A: releases lock -> noone takes care of reconciliation
469
470                 unlock();
471             }
472
473             if (isDirty()) {
474                 if (tryLock()) {
475                     LOG.debug("Service group {} re-running reconciliation", identifier);
476                     continue;
477                 }
478
479                 LOG.debug("Service group {} will be reconciled by someone else", identifier);
480             } else {
481                 LOG.debug("Service group {} is completely reconciled", identifier);
482             }
483
484             break;
485         }
486     }
487
488     private void serviceTransitionCompleted() {
489         markDirty();
490         if (tryLock()) {
491             reconcileState();
492         }
493     }
494
495     // Has to be called with lock asserted
496     private void tryReconcileState() {
497         // First take a safe snapshot of current state on which we will base our decisions.
498         final Set<ClusterSingletonServiceRegistration> localMembers;
499         final boolean haveCleanup;
500         final boolean haveService;
501         synchronized (this) {
502             if (serviceEntityReg != null) {
503                 switch (serviceEntityState) {
504                     case OWNED:
505                     case OWNED_JEOPARDY:
506                         haveService = true;
507                         break;
508                     case REGISTERED:
509                     case UNOWNED:
510                     case UNREGISTERED:
511                         haveService = false;
512                         break;
513                     default:
514                         throw new IllegalStateException("Unhandled service entity state " + serviceEntityState);
515                 }
516             } else {
517                 haveService = false;
518             }
519
520             if (haveService && cleanupEntityReg == null) {
521                 // We have the service entity but have not registered for cleanup entity. Do that now and retry.
522                 LOG.debug("Service group {} registering cleanup entity", identifier);
523                 try {
524                     cleanupEntityState = EntityState.REGISTERED;
525                     cleanupEntityReg = entityOwnershipService.registerCandidate(cleanupEntity);
526                 } catch (CandidateAlreadyRegisteredException e) {
527                     LOG.error("Service group {} failed to take ownership, aborting", identifier, e);
528                     if (serviceEntityReg != null) {
529                         serviceEntityReg.close();
530                         serviceEntityReg = null;
531                     }
532                 }
533                 markDirty();
534                 return;
535             }
536
537             if (cleanupEntityReg != null) {
538                 switch (cleanupEntityState) {
539                     case OWNED:
540                         haveCleanup = true;
541                         break;
542                     case OWNED_JEOPARDY:
543                     case REGISTERED:
544                     case UNOWNED:
545                     case UNREGISTERED:
546                         haveCleanup = false;
547                         break;
548                     default:
549                         throw new IllegalStateException("Unhandled service entity state " + serviceEntityState);
550                 }
551             } else {
552                 haveCleanup = false;
553             }
554
555             localMembers = ImmutableSet.copyOf(members);
556         }
557
558         if (haveService && haveCleanup) {
559             ensureServicesStarting(localMembers);
560             return;
561         }
562
563         ensureServicesStopping();
564
565         if (!haveService && services.isEmpty()) {
566             LOG.debug("Service group {} has no running services", identifier);
567             final boolean canFinishClose;
568             synchronized (this) {
569                 if (cleanupEntityReg != null) {
570                     LOG.debug("Service group {} releasing cleanup entity", identifier);
571                     cleanupEntityReg.close();
572                     cleanupEntityReg = null;
573                 }
574
575                 switch (cleanupEntityState) {
576                     case OWNED:
577                     case OWNED_JEOPARDY:
578                     case REGISTERED:
579                         // When we are registered we need to wait for registration to resolve, otherwise
580                         // the notification could be routed to the next incarnation of this group -- which could be
581                         // confused by the fact it is not registered, but receives, for example, OWNED notification.
582                         canFinishClose = false;
583                         break;
584                     case UNOWNED:
585                     case UNREGISTERED:
586                         canFinishClose = true;
587                         break;
588                     default:
589                         throw new IllegalStateException("Unhandled cleanup entity state " + cleanupEntityState);
590                 }
591             }
592
593             if (canFinishClose) {
594                 final SettableFuture<Void> localFuture = closeFuture.get();
595                 if (localFuture != null && !localFuture.isDone()) {
596                     LOG.debug("Service group {} completing termination", identifier);
597                     localFuture.set(null);
598                 }
599             }
600         }
601     }
602
603     // Has to be called with lock asserted
604     @SuppressWarnings("illegalCatch")
605     private void ensureServicesStarting(final Set<ClusterSingletonServiceRegistration> localConfig) {
606         LOG.debug("Service group {} starting services", identifier);
607
608         // This may look counter-intuitive, but the localConfig may be missing some services that are started -- for
609         // example when this method is executed as part of unregisterService() call. In that case we need to ensure
610         // services in the list are stopping
611         final Iterator<Entry<ClusterSingletonServiceRegistration, ServiceInfo>> it = services.entrySet().iterator();
612         while (it.hasNext()) {
613             final Entry<ClusterSingletonServiceRegistration, ServiceInfo> entry = it.next();
614             final ClusterSingletonServiceRegistration reg = entry.getKey();
615             if (!localConfig.contains(reg)) {
616                 final ServiceInfo newInfo = ensureStopping(reg, entry.getValue());
617                 if (newInfo != null) {
618                     entry.setValue(newInfo);
619                 } else {
620                     it.remove();
621                 }
622             }
623         }
624
625         // Now make sure member services are being juggled around
626         for (ClusterSingletonServiceRegistration reg : localConfig) {
627             if (!services.containsKey(reg)) {
628                 final ClusterSingletonService service = reg.getInstance();
629                 LOG.debug("Starting service {}", service);
630
631                 try {
632                     service.instantiateServiceInstance();
633                 } catch (Exception e) {
634                     LOG.warn("Service group {} service {} failed to start, attempting to continue", identifier, service,
635                         e);
636                     continue;
637                 }
638
639                 services.put(reg, ServiceInfo.started());
640             }
641         }
642     }
643
644     // Has to be called with lock asserted
645     private void ensureServicesStopping() {
646         final Iterator<Entry<ClusterSingletonServiceRegistration, ServiceInfo>> it = services.entrySet().iterator();
647         while (it.hasNext()) {
648             final Entry<ClusterSingletonServiceRegistration, ServiceInfo> entry = it.next();
649             final ServiceInfo newInfo = ensureStopping(entry.getKey(), entry.getValue());
650             if (newInfo != null) {
651                 entry.setValue(newInfo);
652             } else {
653                 it.remove();
654             }
655         }
656     }
657
658     @SuppressWarnings("illegalCatch")
659     private ServiceInfo ensureStopping(final ClusterSingletonServiceRegistration reg, final ServiceInfo info) {
660         switch (info.getState()) {
661             case STARTED:
662                 final ClusterSingletonService service = reg.getInstance();
663
664                 LOG.debug("Service group {} stopping service {}", identifier, service);
665                 final @NonNull ListenableFuture<?> future;
666                 try {
667                     future = verifyNotNull(service.closeServiceInstance());
668                 } catch (Exception e) {
669                     LOG.warn("Service group {} service {} failed to stop, attempting to continue", identifier, service,
670                         e);
671                     return null;
672                 }
673
674                 Futures.addCallback(future, new FutureCallback<Object>() {
675                     @Override
676                     public void onSuccess(final Object result) {
677                         LOG.debug("Service group {} service {} stopped successfully", identifier, service);
678                         serviceTransitionCompleted();
679                     }
680
681                     @Override
682                     public void onFailure(final Throwable cause) {
683                         LOG.debug("Service group {} service {} stopped with error", identifier, service, cause);
684                         serviceTransitionCompleted();
685                     }
686                 }, MoreExecutors.directExecutor());
687                 return info.toState(ServiceState.STOPPING, future);
688             case STOPPING:
689                 if (info.getFuture().isDone()) {
690                     LOG.debug("Service group {} removed stopped service {}", identifier, reg.getInstance());
691                     return null;
692                 }
693                 return info;
694             default:
695                 throw new IllegalStateException("Unhandled state " + info.getState());
696         }
697     }
698
699     private void markDirty() {
700         dirty = 1;
701     }
702
703     private boolean isDirty() {
704         return dirty != 0;
705     }
706
707     private boolean conditionalClean() {
708         return DIRTY_UPDATER.compareAndSet(this, 1, 0);
709     }
710
711     private boolean tryLock() {
712         return LOCK_UPDATER.compareAndSet(this, 0, 1);
713     }
714
715     private boolean unlock() {
716         verify(LOCK_UPDATER.compareAndSet(this, 1, 0));
717         return true;
718     }
719
720     @Override
721     public String toString() {
722         return MoreObjects.toStringHelper(this).add("identifier", identifier).toString();
723     }
724 }