Remove ShardManager.configListenerReg
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / shardmanager / ShardManager.java
index 6d26b8419c3561dd070623821e62cfca116674f2..e1752cc57d5c8347948887c86c06fb9162e93ca8 100644 (file)
@@ -112,17 +112,14 @@ import org.opendaylight.controller.cluster.sharding.PrefixedShardConfigUpdateHan
 import org.opendaylight.controller.cluster.sharding.messages.InitConfigListener;
 import org.opendaylight.controller.cluster.sharding.messages.PrefixShardCreated;
 import org.opendaylight.controller.cluster.sharding.messages.PrefixShardRemoved;
-import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
-import org.opendaylight.controller.md.sal.dom.api.DOMDataTreeChangeListener;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
-import org.opendaylight.yangtools.concepts.ListenerRegistration;
+import org.opendaylight.yangtools.concepts.Registration;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.concurrent.ExecutionContext;
 import scala.concurrent.Future;
-import scala.concurrent.duration.Duration;
 import scala.concurrent.duration.FiniteDuration;
 
 /**
@@ -172,10 +169,11 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
     private final Map<String, CompositeOnComplete<Boolean>> shardActorsStopping = new HashMap<>();
 
+    private final Set<Consumer<String>> shardAvailabilityCallbacks = new HashSet<>();
+
     private final String persistenceId;
     private final AbstractDataStore dataStore;
 
-    private ListenerRegistration<DOMDataTreeChangeListener> configListenerReg = null;
     private PrefixedShardConfigUpdateHandler configUpdateHandler;
 
     ShardManager(final AbstractShardManagerCreator<?> builder) {
@@ -215,11 +213,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         LOG.info("Stopping ShardManager {}", persistenceId());
 
         shardManagerMBean.unregisterMBean();
-
-        if (configListenerReg != null) {
-            configListenerReg.close();
-            configListenerReg = null;
-        }
     }
 
     @Override
@@ -302,11 +295,13 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             onGetShardRole((GetShardRole) message);
         } else if (message instanceof RunnableMessage) {
             ((RunnableMessage)message).run();
+        } else if (message instanceof RegisterForShardAvailabilityChanges) {
+            onRegisterForShardAvailabilityChanges((RegisterForShardAvailabilityChanges)message);
         } else if (message instanceof DeleteSnapshotsFailure) {
             LOG.warn("{}: Failed to delete prior snapshots", persistenceId(),
                     ((DeleteSnapshotsFailure) message).cause());
         } else if (message instanceof DeleteSnapshotsSuccess) {
-            LOG.debug("{}: Successfully deleted prior snapshots", persistenceId(), message);
+            LOG.debug("{}: Successfully deleted prior snapshots", persistenceId());
         } else if (message instanceof RegisterRoleChangeListenerReply) {
             LOG.trace("{}: Received RegisterRoleChangeListenerReply", persistenceId());
         } else if (message instanceof ClusterEvent.MemberEvent) {
@@ -316,6 +311,16 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
     }
 
+    private void onRegisterForShardAvailabilityChanges(final RegisterForShardAvailabilityChanges message) {
+        LOG.debug("{}: onRegisterForShardAvailabilityChanges: {}", persistenceId(), message);
+
+        final Consumer<String> callback = message.getCallback();
+        shardAvailabilityCallbacks.add(callback);
+
+        getSender().tell(new Status.Success((Registration)
+            () -> executeInSelf(() -> shardAvailabilityCallbacks.remove(callback))), self());
+    }
+
     private void onGetShardRole(final GetShardRole message) {
         LOG.debug("{}: onGetShardRole for shard: {}", persistenceId(), message.getName());
 
@@ -336,7 +341,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     private void onInitConfigListener() {
         LOG.debug("{}: Initializing config listener on {}", persistenceId(), cluster.getCurrentMemberName());
 
-        final org.opendaylight.mdsal.common.api.LogicalDatastoreType type =
+        final org.opendaylight.mdsal.common.api.LogicalDatastoreType datastoreType =
                 org.opendaylight.mdsal.common.api.LogicalDatastoreType
                         .valueOf(datastoreContextFactory.getBaseDatastoreContext().getLogicalStoreType().name());
 
@@ -345,7 +350,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
 
         configUpdateHandler = new PrefixedShardConfigUpdateHandler(self(), cluster.getCurrentMemberName());
-        configUpdateHandler.initListener(dataStore, type);
+        configUpdateHandler.initListener(dataStore, datastoreType);
     }
 
     private void onShutDown() {
@@ -442,13 +447,14 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             public void onComplete(final Throwable failure, final Object response) {
                 if (failure != null) {
                     shardReplicaOperationsInProgress.remove(shardName);
-                    String msg = String.format("RemoveServer request to leader %s for shard %s failed",
-                            primaryPath, shardName);
 
-                    LOG.debug("{}: {}", persistenceId(), msg, failure);
+                    LOG.debug("{}: RemoveServer request to leader {} for shard {} failed", persistenceId(), primaryPath,
+                        shardName, failure);
 
                     // FAILURE
-                    sender.tell(new Status.Failure(new RuntimeException(msg, failure)), self());
+                    sender.tell(new Status.Failure(new RuntimeException(
+                        String.format("RemoveServer request to leader %s for shard %s failed", primaryPath, shardName),
+                        failure)), self());
                 } else {
                     // SUCCESS
                     self().tell(new WrappedShardResponse(shardId, response, primaryPath), sender);
@@ -482,13 +488,13 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             public void onComplete(final Throwable failure, final Object response) {
                 if (failure != null) {
                     shardReplicaOperationsInProgress.remove(shardName);
-                    String msg = String.format("RemoveServer request to leader %s for shard %s failed",
-                            primaryPath, shardName);
-
-                    LOG.debug("{}: {}", persistenceId(), msg, failure);
+                    LOG.debug("{}: RemoveServer request to leader {} for shard {} failed", persistenceId(), primaryPath,
+                        shardName, failure);
 
                     // FAILURE
-                    sender.tell(new Status.Failure(new RuntimeException(msg, failure)), self());
+                    sender.tell(new Status.Failure(new RuntimeException(
+                        String.format("RemoveServer request to leader %s for shard %s failed", primaryPath, shardName),
+                        failure)), self());
                 } else {
                     // SUCCESS
                     self().tell(new WrappedShardResponse(shardId, response, primaryPath), sender);
@@ -531,6 +537,9 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                     }
 
                     self().tell((RunnableMessage) () -> {
+                        // At any rate, invalidate primaryShardInfo cache
+                        primaryShardInfoCache.remove(shardName);
+
                         shardActorsStopping.remove(shardName);
                         notifyOnCompleteTasks(failure, result);
                     }, ActorRef.noSender());
@@ -650,7 +659,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         configuration.addPrefixShardConfiguration(config);
 
         final Builder builder = newShardDatastoreContextBuilder(shardName);
-        builder.logicalStoreType(LogicalDatastoreType.valueOf(config.getPrefix().getDatastoreType().name()))
+        builder.logicalStoreType(config.getPrefix().getDatastoreType())
                 .storeRoot(config.getPrefix().getRootIdentifier());
         DatastoreContext shardDatastoreContext = builder.build();
 
@@ -760,6 +769,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             shardInformation.setLeaderVersion(leaderStateChanged.getLeaderPayloadVersion());
             if (shardInformation.setLeaderId(leaderStateChanged.getLeaderId())) {
                 primaryShardInfoCache.remove(shardInformation.getShardName());
+
+                notifyShardAvailabilityCallbacks(shardInformation);
             }
 
             checkReady();
@@ -768,6 +779,10 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
     }
 
+    private void notifyShardAvailabilityCallbacks(final ShardInformation shardInformation) {
+        shardAvailabilityCallbacks.forEach(callback -> callback.accept(shardInformation.getShardName()));
+    }
+
     private void onShardNotInitializedTimeout(final ShardNotInitializedTimeout message) {
         ShardInformation shardInfo = message.getShardInfo();
 
@@ -781,7 +796,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             message.getSender().tell(createNotInitializedException(shardInfo.getShardId()), getSelf());
         } else {
             LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(), shardInfo.getShardName());
-            message.getSender().tell(createNoShardLeaderException(shardInfo.getShardId()), getSelf());
+            message.getSender().tell(new NoShardLeaderException(shardInfo.getShardId()), getSelf());
         }
     }
 
@@ -846,7 +861,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         final ActorRef sender = getSender();
 
         if (sender == null) {
-            return; //why is a non-actor sending this message? Just ignore.
+            // why is a non-actor sending this message? Just ignore.
+            return;
         }
 
         String actorName = sender.path().name();
@@ -856,7 +872,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         try {
             shardId = ShardIdentifier.fromShardIdString(actorName);
         } catch (IllegalArgumentException e) {
-            LOG.debug("{}: ignoring actor {}", actorName, e);
+            LOG.debug("{}: ignoring actor {}", persistenceId, actorName, e);
             return;
         }
 
@@ -923,7 +939,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                 }
 
                 LOG.debug("{}: Scheduling {} ms timer to wait for shard {}", persistenceId(), timeout.toMillis(),
-                        shardInformation.getShardName());
+                        shardInformation);
 
                 Cancellable timeoutSchedule = getContext().system().scheduler().scheduleOnce(
                         timeout, getSelf(),
@@ -939,7 +955,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             } else {
                 LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(),
                         shardInformation.getShardName());
-                getSender().tell(createNoShardLeaderException(shardInformation.getShardId()), getSelf());
+                getSender().tell(new NoShardLeaderException(shardInformation.getShardId()), getSelf());
             }
 
             return;
@@ -948,10 +964,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         getSender().tell(messageSupplier.get(), getSelf());
     }
 
-    private static NoShardLeaderException createNoShardLeaderException(final ShardIdentifier shardId) {
-        return new NoShardLeaderException(null, shardId.toString());
-    }
-
     private static NotInitializedException createNotInitializedException(final ShardIdentifier shardId) {
         return new NotInitializedException(String.format(
                 "Found primary shard %s but it's not initialized yet. Please try again later", shardId));
@@ -1040,15 +1052,15 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     }
 
     private void markMemberUnavailable(final MemberName memberName) {
-        final String memberStr = memberName.getName();
         for (ShardInformation info : localShards.values()) {
             String leaderId = info.getLeaderId();
-            // XXX: why are we using String#contains() here?
-            if (leaderId != null && leaderId.contains(memberStr)) {
+            if (leaderId != null && ShardIdentifier.fromShardIdString(leaderId).getMemberName().equals(memberName)) {
                 LOG.debug("Marking Leader {} as unavailable.", leaderId);
                 info.setLeaderAvailable(false);
 
                 primaryShardInfoCache.remove(info.getShardName());
+
+                notifyShardAvailabilityCallbacks(info);
             }
 
             info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
@@ -1056,11 +1068,9 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     }
 
     private void markMemberAvailable(final MemberName memberName) {
-        final String memberStr = memberName.getName();
         for (ShardInformation info : localShards.values()) {
             String leaderId = info.getLeaderId();
-            // XXX: why are we using String#contains() here?
-            if (leaderId != null && leaderId.contains(memberStr)) {
+            if (leaderId != null && ShardIdentifier.fromShardIdString(leaderId).getMemberName().equals(memberName)) {
                 LOG.debug("Marking Leader {} as available.", leaderId);
                 info.setLeaderAvailable(true);
             }
@@ -1125,7 +1135,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     private void updateSchemaContext(final Object message) {
         schemaContext = ((UpdateSchemaContext) message).getSchemaContext();
 
-        LOG.debug("Got updated SchemaContext: # of modules {}", schemaContext.getAllModuleIdentifiers().size());
+        LOG.debug("Got updated SchemaContext: # of modules {}", schemaContext.getModules().size());
 
         for (ShardInformation info : localShards.values()) {
             info.setSchemaContext(schemaContext);
@@ -1133,6 +1143,17 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             if (info.getActor() == null) {
                 LOG.debug("Creating Shard {}", info.getShardId());
                 info.setActor(newShardActor(info));
+                // Update peer address for every existing peer memeber to avoid missing sending
+                // PeerAddressResolved and PeerUp to this shard while UpdateSchemaContext comes after MemberUp.
+                String shardName = info.getShardName();
+                for (MemberName memberName : peerAddressResolver.getPeerMembers()) {
+                    String peerId = getShardIdentifier(memberName, shardName).toString() ;
+                    String peerAddress = peerAddressResolver.getShardActorAddress(shardName, memberName);
+                    info.updatePeerAddress(peerId, peerAddress, getSelf());
+                    info.peerUp(memberName, peerId, getSelf());
+                    LOG.debug("{}: updated peer {} on member {} with address {} on shard {} whose actor address is {}",
+                            persistenceId(), peerId, memberName, peerAddress, info.getShardId(), info.getActor());
+                }
             } else {
                 info.getActor().tell(message, getSelf());
             }
@@ -1249,7 +1270,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             }
         }
 
-        restoreFromSnapshot = null; // null out to GC
+        // null out to GC
+        restoreFromSnapshot = null;
 
         for (String shardName : memberShardNames) {
             ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
@@ -1290,7 +1312,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     @Override
     public SupervisorStrategy supervisorStrategy() {
 
-        return new OneForOneStrategy(10, Duration.create("1 minute"),
+        return new OneForOneStrategy(10, FiniteDuration.create(1, TimeUnit.MINUTES),
                 (Function<Throwable, Directive>) t -> {
                     LOG.warn("Supervisor Strategy caught unexpected exception - resuming", t);
                     return SupervisorStrategy.resume();
@@ -1309,9 +1331,9 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
     private boolean isShardReplicaOperationInProgress(final String shardName, final ActorRef sender) {
         if (shardReplicaOperationsInProgress.contains(shardName)) {
-            String msg = String.format("A shard replica operation for %s is already in progress", shardName);
-            LOG.debug("{}: {}", persistenceId(), msg);
-            sender.tell(new Status.Failure(new IllegalStateException(msg)), getSelf());
+            LOG.debug("{}: A shard replica operation for {} is already in progress", persistenceId(), shardName);
+            sender.tell(new Status.Failure(new IllegalStateException(
+                String.format("A shard replica operation for %s is already in progress", shardName))), getSelf());
             return true;
         }
 
@@ -1327,10 +1349,11 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
         // Create the localShard
         if (schemaContext == null) {
-            String msg = String.format(
-                    "No SchemaContext is available in order to create a local shard instance for %s", shardName);
-            LOG.debug("{}: {}", persistenceId(), msg);
-            getSender().tell(new Status.Failure(new IllegalStateException(msg)), getSelf());
+            LOG.debug("{}: No SchemaContext is available in order to create a local shard instance for {}",
+                persistenceId(), shardName);
+            getSender().tell(new Status.Failure(new IllegalStateException(
+                "No SchemaContext is available in order to create a local shard instance for " + shardName)),
+                getSelf());
             return;
         }
 
@@ -1359,18 +1382,19 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
         // verify the shard with the specified name is present in the cluster configuration
         if (!this.configuration.isShardConfigured(shardName)) {
-            String msg = String.format("No module configuration exists for shard %s", shardName);
-            LOG.debug("{}: {}", persistenceId(), msg);
-            getSender().tell(new Status.Failure(new IllegalArgumentException(msg)), getSelf());
+            LOG.debug("{}: No module configuration exists for shard {}", persistenceId(), shardName);
+            getSender().tell(new Status.Failure(new IllegalArgumentException(
+                "No module configuration exists for shard " + shardName)), getSelf());
             return;
         }
 
         // Create the localShard
         if (schemaContext == null) {
-            String msg = String.format(
-                  "No SchemaContext is available in order to create a local shard instance for %s", shardName);
-            LOG.debug("{}: {}", persistenceId(), msg);
-            getSender().tell(new Status.Failure(new IllegalStateException(msg)), getSelf());
+            LOG.debug("{}: No SchemaContext is available in order to create a local shard instance for {}",
+                persistenceId(), shardName);
+            getSender().tell(new Status.Failure(new IllegalStateException(
+                "No SchemaContext is available in order to create a local shard instance for " + shardName)),
+                getSelf());
             return;
         }
 
@@ -1393,9 +1417,9 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     }
 
     private void sendLocalReplicaAlreadyExistsReply(final String shardName, final ActorRef sender) {
-        String msg = String.format("Local shard %s already exists", shardName);
-        LOG.debug("{}: {}", persistenceId(), msg);
-        sender.tell(new Status.Failure(new AlreadyExistsException(msg)), getSelf());
+        LOG.debug("{}: Local shard {} already exists", persistenceId(), shardName);
+        sender.tell(new Status.Failure(new AlreadyExistsException(
+            String.format("Local shard %s already exists", shardName))), getSelf());
     }
 
     private void addPrefixShard(final String shardName, final YangInstanceIdentifier shardPrefix,
@@ -1552,7 +1576,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                         + "Possible causes - there was a problem replicating the data or shard leadership changed "
                         + "while replicating the shard data", leaderPath, shardId.getShardName()));
             case NO_LEADER:
-                return createNoShardLeaderException(shardId);
+                return new NoShardLeaderException(shardId);
             case NOT_SUPPORTED:
                 return new UnsupportedOperationException(String.format("%s request is not supported for shard %s",
                         serverChange.getSimpleName(), shardId.getShardName()));
@@ -1747,15 +1771,16 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                         getSelf().tell((RunnableMessage) () -> onLocalShardFound.accept((LocalShardFound) response),
                                 sender);
                     } else if (response instanceof LocalShardNotFound) {
-                        String msg = String.format("Local shard %s does not exist", shardName);
-                        LOG.debug("{}: {}", persistenceId, msg);
-                        sender.tell(new Status.Failure(new IllegalArgumentException(msg)), self());
+                        LOG.debug("{}: Local shard {} does not exist", persistenceId, shardName);
+                        sender.tell(new Status.Failure(new IllegalArgumentException(
+                            String.format("Local shard %s does not exist", shardName))), self());
                     } else {
-                        String msg = String.format("Failed to find local shard %s: received response: %s",
-                                shardName, response);
-                        LOG.debug("{}: {}", persistenceId, msg);
-                        sender.tell(new Status.Failure(response instanceof Throwable ? (Throwable) response :
-                                new RuntimeException(msg)), self());
+                        LOG.debug("{}: Failed to find local shard {}: received response: {}", persistenceId, shardName,
+                            response);
+                        sender.tell(new Status.Failure(response instanceof Throwable ? (Throwable) response
+                                : new RuntimeException(
+                                    String.format("Failed to find local shard %s: received response: %s", shardName,
+                                        response))), self());
                     }
                 }
             }
@@ -1784,10 +1809,11 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             public void onComplete(final Throwable failure, final Object response) {
                 shardReplicaOperationsInProgress.remove(shardName);
                 if (failure != null) {
-                    String msg = String.format("ChangeServersVotingStatus request to local shard %s failed",
-                            shardActorRef.path());
-                    LOG.debug("{}: {}", persistenceId(), msg, failure);
-                    sender.tell(new Status.Failure(new RuntimeException(msg, failure)), self());
+                    LOG.debug("{}: ChangeServersVotingStatus request to local shard {} failed", persistenceId(),
+                        shardActorRef.path(), failure);
+                    sender.tell(new Status.Failure(new RuntimeException(
+                        String.format("ChangeServersVotingStatus request to local shard %s failed",
+                            shardActorRef.path()), failure)), self());
                 } else {
                     LOG.debug("{}: Received {} from local shard {}", persistenceId(), response, shardActorRef.path());
 
@@ -1949,11 +1975,11 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
         @Override
         public void onUnknownResponse(final Object response) {
-            String msg = String.format("Failed to find leader for shard %s: received response: %s",
-                    shardName, response);
-            LOG.debug("{}: {}", persistenceId, msg);
-            targetActor.tell(new Status.Failure(response instanceof Throwable ? (Throwable) response :
-                    new RuntimeException(msg)), shardManagerActor);
+            LOG.debug("{}: Failed to find leader for shard {}: received response: {}", persistenceId, shardName,
+                response);
+            targetActor.tell(new Status.Failure(response instanceof Throwable ? (Throwable) response
+                    : new RuntimeException(String.format("Failed to find leader for shard %s: received response: %s",
+                        shardName, response))), shardManagerActor);
         }
     }