X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-distributed-datastore%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fdatastore%2Fshardmanager%2FShardManager.java;h=85469b27e8b4fec4fec619c8cce8126bb4ca3c8a;hb=546cd1fd100dbaa36908b22c2f422320dbd8c4b2;hp=2cc30a271cec446c3a05bbb546792d65f9511b66;hpb=9b319f491af1c65705b69e8a182aab5006a2f959;p=controller.git diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/shardmanager/ShardManager.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/shardmanager/ShardManager.java index 2cc30a271c..85469b27e8 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/shardmanager/ShardManager.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/shardmanager/ShardManager.java @@ -35,6 +35,7 @@ import akka.persistence.SnapshotOffer; import akka.persistence.SnapshotSelectionCriteria; import akka.util.Timeout; import com.google.common.annotations.VisibleForTesting; +import com.google.common.util.concurrent.SettableFuture; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.util.ArrayList; import java.util.Collection; @@ -45,7 +46,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.Consumer; @@ -53,22 +53,18 @@ import java.util.function.Supplier; import org.opendaylight.controller.cluster.access.concepts.MemberName; import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActorWithMetering; import org.opendaylight.controller.cluster.common.actor.Dispatchers; -import org.opendaylight.controller.cluster.datastore.AbstractDataStore; import org.opendaylight.controller.cluster.datastore.ClusterWrapper; import org.opendaylight.controller.cluster.datastore.DatastoreContext; -import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder; import org.opendaylight.controller.cluster.datastore.DatastoreContextFactory; import org.opendaylight.controller.cluster.datastore.Shard; import org.opendaylight.controller.cluster.datastore.config.Configuration; import org.opendaylight.controller.cluster.datastore.config.ModuleShardConfiguration; -import org.opendaylight.controller.cluster.datastore.config.PrefixShardConfiguration; import org.opendaylight.controller.cluster.datastore.exceptions.AlreadyExistsException; import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException; import org.opendaylight.controller.cluster.datastore.exceptions.NotInitializedException; import org.opendaylight.controller.cluster.datastore.exceptions.PrimaryNotFoundException; import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier; import org.opendaylight.controller.cluster.datastore.messages.ActorInitialized; -import org.opendaylight.controller.cluster.datastore.messages.AddPrefixShardReplica; import org.opendaylight.controller.cluster.datastore.messages.AddShardReplica; import org.opendaylight.controller.cluster.datastore.messages.ChangeShardMembersVotingStatus; import org.opendaylight.controller.cluster.datastore.messages.CreateShard; @@ -82,13 +78,11 @@ import org.opendaylight.controller.cluster.datastore.messages.LocalShardFound; import org.opendaylight.controller.cluster.datastore.messages.LocalShardNotFound; import org.opendaylight.controller.cluster.datastore.messages.RemoteFindPrimary; import org.opendaylight.controller.cluster.datastore.messages.RemotePrimaryShardFound; -import org.opendaylight.controller.cluster.datastore.messages.RemovePrefixShardReplica; import org.opendaylight.controller.cluster.datastore.messages.RemoveShardReplica; import org.opendaylight.controller.cluster.datastore.messages.ShardLeaderStateChanged; import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext; import org.opendaylight.controller.cluster.datastore.persisted.DatastoreSnapshot; import org.opendaylight.controller.cluster.datastore.persisted.ShardManagerSnapshot; -import org.opendaylight.controller.cluster.datastore.utils.ClusterUtils; import org.opendaylight.controller.cluster.datastore.utils.CompositeOnComplete; import org.opendaylight.controller.cluster.datastore.utils.PrimaryShardInfoFutureCache; import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener; @@ -109,14 +103,8 @@ import org.opendaylight.controller.cluster.raft.messages.ServerChangeReply; import org.opendaylight.controller.cluster.raft.messages.ServerChangeStatus; import org.opendaylight.controller.cluster.raft.messages.ServerRemoved; import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy; -import org.opendaylight.controller.cluster.sharding.PrefixedShardConfigUpdateHandler; -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.mdsal.dom.api.DOMDataTreeIdentifier; import org.opendaylight.yangtools.concepts.Registration; -import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier; -import org.opendaylight.yangtools.yang.model.api.SchemaContext; +import org.opendaylight.yangtools.yang.model.api.EffectiveModelContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import scala.concurrent.ExecutionContext; @@ -156,14 +144,14 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { private DatastoreContextFactory datastoreContextFactory; - private final CountDownLatch waitTillReadyCountdownLatch; + private final SettableFuture readinessFuture; private final PrimaryShardInfoFutureCache primaryShardInfoCache; @VisibleForTesting final ShardPeerAddressResolver peerAddressResolver; - private SchemaContext schemaContext; + private EffectiveModelContext schemaContext; private DatastoreSnapshot restoreFromSnapshot; @@ -176,9 +164,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { private final Set> shardAvailabilityCallbacks = new HashSet<>(); private final String persistenceId; - private final AbstractDataStore dataStore; - - private PrefixedShardConfigUpdateHandler configUpdateHandler; ShardManager(final AbstractShardManagerCreator builder) { this.cluster = builder.getCluster(); @@ -187,7 +172,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { this.type = datastoreContextFactory.getBaseDatastoreContext().getDataStoreName(); this.shardDispatcherPath = new Dispatchers(context().system().dispatchers()).getDispatcherPath(Dispatchers.DispatcherType.Shard); - this.waitTillReadyCountdownLatch = builder.getWaitTillReadyCountDownLatch(); + this.readinessFuture = builder.getReadinessFuture(); this.primaryShardInfoCache = builder.getPrimaryShardInfoCache(); this.restoreFromSnapshot = builder.getRestoreFromSnapshot(); @@ -203,8 +188,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { "shard-manager-" + this.type, datastoreContextFactory.getBaseDatastoreContext().getDataStoreMXBeanType()); shardManagerMBean.registerMBean(); - - dataStore = builder.getDistributedDataStore(); } @Override @@ -257,14 +240,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { onCreateShard((CreateShard)message); } else if (message instanceof AddShardReplica) { onAddShardReplica((AddShardReplica) message); - } else if (message instanceof AddPrefixShardReplica) { - onAddPrefixShardReplica((AddPrefixShardReplica) message); - } else if (message instanceof PrefixShardCreated) { - onPrefixShardCreated((PrefixShardCreated) message); - } else if (message instanceof PrefixShardRemoved) { - onPrefixShardRemoved((PrefixShardRemoved) message); - } else if (message instanceof InitConfigListener) { - onInitConfigListener(); } else if (message instanceof ForwardedAddServerReply) { ForwardedAddServerReply msg = (ForwardedAddServerReply)message; onAddServerReply(msg.shardInfo, msg.addServerReply, getSender(), msg.leaderPath, @@ -274,12 +249,10 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { onAddServerFailure(msg.shardName, msg.failureMessage, msg.failure, getSender(), msg.removeShardOnFailure); } else if (message instanceof RemoveShardReplica) { onRemoveShardReplica((RemoveShardReplica) message); - } else if (message instanceof RemovePrefixShardReplica) { - onRemovePrefixShardReplica((RemovePrefixShardReplica) message); } else if (message instanceof WrappedShardResponse) { onWrappedShardResponse((WrappedShardResponse) message); } else if (message instanceof GetSnapshot) { - onGetSnapshot(); + onGetSnapshot((GetSnapshot) message); } else if (message instanceof ServerRemoved) { onShardReplicaRemoved((ServerRemoved) message); } else if (message instanceof ChangeShardMembersVotingStatus) { @@ -342,21 +315,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { getSender().tell(new GetShardRoleReply(shardInformation.getRole()), ActorRef.noSender()); } - private void onInitConfigListener() { - LOG.debug("{}: Initializing config listener on {}", persistenceId(), cluster.getCurrentMemberName()); - - final org.opendaylight.mdsal.common.api.LogicalDatastoreType datastoreType = - org.opendaylight.mdsal.common.api.LogicalDatastoreType - .valueOf(datastoreContextFactory.getBaseDatastoreContext().getLogicalStoreType().name()); - - if (configUpdateHandler != null) { - configUpdateHandler.close(); - } - - configUpdateHandler = new PrefixedShardConfigUpdateHandler(self(), cluster.getCurrentMemberName()); - configUpdateHandler.initListener(dataStore, datastoreType); - } - void onShutDown() { List> stopFutures = new ArrayList<>(localShards.size()); for (ShardInformation info : localShards.values()) { @@ -426,49 +384,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } } - @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD", - justification = "https://github.com/spotbugs/spotbugs/issues/811") - private void removePrefixShardReplica(final RemovePrefixShardReplica contextMessage, final String shardName, - final String primaryPath, final ActorRef sender) { - if (isShardReplicaOperationInProgress(shardName, sender)) { - return; - } - - shardReplicaOperationsInProgress.add(shardName); - - final ShardIdentifier shardId = getShardIdentifier(contextMessage.getMemberName(), shardName); - - final DatastoreContext datastoreContext = newShardDatastoreContextBuilder(shardName).build(); - - //inform ShardLeader to remove this shard as a replica by sending an RemoveServer message - LOG.debug("{}: Sending RemoveServer message to peer {} for shard {}", persistenceId(), - primaryPath, shardId); - - Timeout removeServerTimeout = new Timeout(datastoreContext.getShardLeaderElectionTimeout().duration()); - Future futureObj = ask(getContext().actorSelection(primaryPath), - new RemoveServer(shardId.toString()), removeServerTimeout); - - futureObj.onComplete(new OnComplete() { - @Override - public void onComplete(final Throwable failure, final Object response) { - if (failure != null) { - shardReplicaOperationsInProgress.remove(shardName); - - LOG.debug("{}: RemoveServer request to leader {} for shard {} failed", persistenceId(), primaryPath, - shardName, failure); - - // FAILURE - 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); - } - } - }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client)); - } - @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD", justification = "https://github.com/spotbugs/spotbugs/issues/811") private void removeShardReplica(final RemoveShardReplica contextMessage, final String shardName, @@ -491,7 +406,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { Future futureObj = ask(getContext().actorSelection(primaryPath), new RemoveServer(shardId.toString()), removeServerTimeout); - futureObj.onComplete(new OnComplete() { + futureObj.onComplete(new OnComplete<>() { @Override public void onComplete(final Throwable failure, final Object response) { if (failure != null) { @@ -535,7 +450,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { final Future stopFuture = Patterns.gracefulStop(shardActor, FiniteDuration.apply(timeoutInMS, TimeUnit.MILLISECONDS), Shutdown.INSTANCE); - final CompositeOnComplete onComplete = new CompositeOnComplete() { + final CompositeOnComplete onComplete = new CompositeOnComplete<>() { @Override public void onComplete(final Throwable failure, final Boolean result) { if (failure == null) { @@ -563,7 +478,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { persistShardList(); } - private void onGetSnapshot() { + private void onGetSnapshot(final GetSnapshot getSnapshot) { LOG.debug("{}: onGetSnapshot", persistenceId()); List notInitialized = null; @@ -588,7 +503,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { datastoreContextFactory.getBaseDatastoreContext().getShardInitializationTimeout().duration())); for (ShardInformation shardInfo: localShards.values()) { - shardInfo.getActor().tell(GetSnapshot.INSTANCE, replyActor); + shardInfo.getActor().tell(getSnapshot, replyActor); } } @@ -616,32 +531,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } } - private void onPrefixShardCreated(final PrefixShardCreated message) { - LOG.debug("{}: onPrefixShardCreated: {}", persistenceId(), message); - - final PrefixShardConfiguration config = message.getConfiguration(); - final ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), - ClusterUtils.getCleanShardName(config.getPrefix().getRootIdentifier())); - final String shardName = shardId.getShardName(); - - if (isPreviousShardActorStopInProgress(shardName, message)) { - return; - } - - if (localShards.containsKey(shardName)) { - LOG.debug("{}: Received create for an already existing shard {}", persistenceId(), shardName); - final PrefixShardConfiguration existing = - configuration.getAllPrefixShardConfigurations().get(config.getPrefix()); - - if (existing != null && existing.equals(config)) { - // we don't have to do nothing here - return; - } - } - - doCreatePrefixShard(config, shardId, shardName); - } - + @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD", + justification = "https://github.com/spotbugs/spotbugs/issues/811") private boolean isPreviousShardActorStopInProgress(final String shardName, final Object messageToDefer) { final CompositeOnComplete stopOnComplete = shardActorsStopping.get(shardName); if (stopOnComplete == null) { @@ -662,43 +553,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { return true; } - private void doCreatePrefixShard(final PrefixShardConfiguration config, final ShardIdentifier shardId, - final String shardName) { - configuration.addPrefixShardConfiguration(config); - - final Builder builder = newShardDatastoreContextBuilder(shardName); - builder.logicalStoreType(config.getPrefix().getDatastoreType()) - .storeRoot(config.getPrefix().getRootIdentifier()); - DatastoreContext shardDatastoreContext = builder.build(); - - final Map peerAddresses = getPeerAddresses(shardName); - final boolean isActiveMember = true; - - LOG.debug("{} doCreatePrefixShard: shardId: {}, memberNames: {}, peerAddresses: {}, isActiveMember: {}", - persistenceId(), shardId, config.getShardMemberNames(), peerAddresses, isActiveMember); - - final ShardInformation info = new ShardInformation(shardName, shardId, peerAddresses, - shardDatastoreContext, Shard.builder(), peerAddressResolver); - info.setActiveMember(isActiveMember); - localShards.put(info.getShardName(), info); - - if (schemaContext != null) { - info.setSchemaContext(schemaContext); - info.setActor(newShardActor(info)); - } - } - - private void onPrefixShardRemoved(final PrefixShardRemoved message) { - LOG.debug("{}: onPrefixShardRemoved : {}", persistenceId(), message); - - final DOMDataTreeIdentifier prefix = message.getPrefix(); - final ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), - ClusterUtils.getCleanShardName(prefix.getRootIdentifier())); - - configuration.removePrefixShardConfiguration(prefix); - removeShard(shardId); - } - private void doCreateShard(final CreateShard createShard) { final ModuleShardConfiguration moduleShardConfig = createShard.getModuleShardConfig(); final String shardName = moduleShardConfig.getShardName(); @@ -761,10 +615,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { private void checkReady() { if (isReadyWithLeaderId()) { - LOG.info("{}: All Shards are ready - data store {} is ready, available count is {}", - persistenceId(), type, waitTillReadyCountdownLatch.getCount()); - - waitTillReadyCountdownLatch.countDown(); + LOG.info("{}: All Shards are ready - data store {} is ready", persistenceId(), type); + readinessFuture.set(null); } } @@ -989,10 +841,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { message.member().address()); peerAddressResolver.removePeerAddress(memberName); - - for (ShardInformation info : localShards.values()) { - info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf()); - } } private void memberExited(final ClusterEvent.MemberExited message) { @@ -1002,10 +850,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { message.member().address()); peerAddressResolver.removePeerAddress(memberName); - - for (ShardInformation info : localShards.values()) { - info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf()); - } } private void memberUp(final ClusterEvent.MemberUp message) { @@ -1038,8 +882,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { String shardName = info.getShardName(); String peerId = getShardIdentifier(memberName, shardName).toString(); info.updatePeerAddress(peerId, peerAddressResolver.getShardActorAddress(shardName, memberName), getSelf()); - - info.peerUp(memberName, peerId, getSelf()); } } @@ -1070,8 +912,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { notifyShardAvailabilityCallbacks(info); } - - info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf()); } } @@ -1082,8 +922,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { LOG.debug("Marking Leader {} as available.", leaderId); info.setLeaderAvailable(true); } - - info.peerUp(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf()); } } @@ -1141,7 +979,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { * @param message the message to send */ private void updateSchemaContext(final Object message) { - schemaContext = ((UpdateSchemaContext) message).getSchemaContext(); + schemaContext = ((UpdateSchemaContext) message).getEffectiveModelContext(); LOG.debug("Got updated SchemaContext: # of modules {}", schemaContext.getModules().size()); @@ -1158,7 +996,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { 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()); } @@ -1234,7 +1071,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { .getShardInitializationTimeout().duration().$times(2)); Future futureObj = ask(getSelf(), new FindPrimary(shardName, true), findPrimaryTimeout); - futureObj.onComplete(new OnComplete() { + futureObj.onComplete(new OnComplete<>() { @Override public void onComplete(final Throwable failure, final Object response) { if (failure != null) { @@ -1357,41 +1194,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { return false; } - private void onAddPrefixShardReplica(final AddPrefixShardReplica message) { - LOG.debug("{}: onAddPrefixShardReplica: {}", persistenceId(), message); - - final ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), - ClusterUtils.getCleanShardName(message.getShardPrefix())); - final String shardName = shardId.getShardName(); - - // Create the localShard - if (schemaContext == null) { - 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; - } - - findPrimary(shardName, new AutoFindPrimaryFailureResponseHandler(getSender(), shardName, persistenceId(), - getSelf()) { - @Override - public void onRemotePrimaryShardFound(final RemotePrimaryShardFound response) { - final RunnableMessage runnable = (RunnableMessage) () -> addPrefixShard(getShardName(), - message.getShardPrefix(), response, getSender()); - if (!isPreviousShardActorStopInProgress(getShardName(), runnable)) { - getSelf().tell(runnable, getTargetActor()); - } - } - - @Override - public void onLocalPrimaryFound(final LocalPrimaryShardFound message) { - sendLocalReplicaAlreadyExistsReply(getShardName(), getTargetActor()); - } - }); - } - private void onAddShardReplica(final AddShardReplica shardReplicaMsg) { final String shardName = shardReplicaMsg.getShardName(); @@ -1441,42 +1243,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { String.format("Local shard %s already exists", shardName))), getSelf()); } - @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD", - justification = "https://github.com/spotbugs/spotbugs/issues/811") - private void addPrefixShard(final String shardName, final YangInstanceIdentifier shardPrefix, - final RemotePrimaryShardFound response, final ActorRef sender) { - if (isShardReplicaOperationInProgress(shardName, sender)) { - return; - } - - shardReplicaOperationsInProgress.add(shardName); - - final ShardInformation shardInfo; - final boolean removeShardOnFailure; - ShardInformation existingShardInfo = localShards.get(shardName); - if (existingShardInfo == null) { - removeShardOnFailure = true; - ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), shardName); - - final Builder builder = newShardDatastoreContextBuilder(shardName); - builder.storeRoot(shardPrefix).customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()); - - DatastoreContext datastoreContext = builder.build(); - - shardInfo = new ShardInformation(shardName, shardId, getPeerAddresses(shardName), datastoreContext, - Shard.builder(), peerAddressResolver); - shardInfo.setActiveMember(false); - shardInfo.setSchemaContext(schemaContext); - localShards.put(shardName, shardInfo); - shardInfo.setActor(newShardActor(shardInfo)); - } else { - removeShardOnFailure = false; - shardInfo = existingShardInfo; - } - - execAddShard(shardName, shardInfo, response, removeShardOnFailure, sender); - } - @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD", justification = "https://github.com/spotbugs/spotbugs/issues/811") private void addShard(final String shardName, final RemotePrimaryShardFound response, final ActorRef sender) { @@ -1528,7 +1294,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { final Future futureObj = ask(getContext().actorSelection(response.getPrimaryPath()), new AddServer(shardInfo.getShardId().toString(), localShardAddress, true), addServerTimeout); - futureObj.onComplete(new OnComplete() { + futureObj.onComplete(new OnComplete<>() { @Override public void onComplete(final Throwable failure, final Object addServerResponse) { if (failure != null) { @@ -1631,32 +1397,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { }); } - private void onRemovePrefixShardReplica(final RemovePrefixShardReplica message) { - LOG.debug("{}: onRemovePrefixShardReplica: {}", persistenceId(), message); - - final ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), - ClusterUtils.getCleanShardName(message.getShardPrefix())); - final String shardName = shardId.getShardName(); - - findPrimary(shardName, new AutoFindPrimaryFailureResponseHandler(getSender(), - shardName, persistenceId(), getSelf()) { - @Override - public void onRemotePrimaryShardFound(final RemotePrimaryShardFound response) { - doRemoveShardReplicaAsync(response.getPrimaryPath()); - } - - @Override - public void onLocalPrimaryFound(final LocalPrimaryShardFound response) { - doRemoveShardReplicaAsync(response.getPrimaryPath()); - } - - private void doRemoveShardReplicaAsync(final String primaryPath) { - getSelf().tell((RunnableMessage) () -> removePrefixShardReplica(message, getShardName(), - primaryPath, getSender()), getTargetActor()); - } - }); - } - private void persistShardList() { List shardList = new ArrayList<>(localShards.keySet()); for (ShardInformation shardInfo : localShards.values()) { @@ -1665,13 +1405,11 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } } LOG.debug("{}: persisting the shard list {}", persistenceId(), shardList); - saveSnapshot(updateShardManagerSnapshot(shardList, configuration.getAllPrefixShardConfigurations())); + saveSnapshot(updateShardManagerSnapshot(shardList)); } - private ShardManagerSnapshot updateShardManagerSnapshot( - final List shardList, - final Map allPrefixShardConfigurations) { - currentSnapshot = new ShardManagerSnapshot(shardList, allPrefixShardConfigurations); + private ShardManagerSnapshot updateShardManagerSnapshot(final List shardList) { + currentSnapshot = new ShardManagerSnapshot(shardList); return currentSnapshot; } @@ -1732,7 +1470,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { Future future = ask(localShardFound.getPath(), GetOnDemandRaftState.INSTANCE, Timeout.apply(30, TimeUnit.SECONDS)); - future.onComplete(new OnComplete() { + future.onComplete(new OnComplete<>() { @Override public void onComplete(final Throwable failure, final Object response) { if (failure != null) { @@ -1781,7 +1519,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { .getShardInitializationTimeout().duration().$times(2)); Future futureObj = ask(getSelf(), new FindLocalShard(shardName, true), findLocalTimeout); - futureObj.onComplete(new OnComplete() { + futureObj.onComplete(new OnComplete<>() { @Override public void onComplete(final Throwable failure, final Object response) { if (failure != null) { @@ -1827,7 +1565,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { Timeout timeout = new Timeout(datastoreContext.getShardLeaderElectionTimeout().duration().$times(2)); Future futureObj = ask(shardActorRef, changeServersVotingStatus, timeout); - futureObj.onComplete(new OnComplete() { + futureObj.onComplete(new OnComplete<>() { @Override public void onComplete(final Throwable failure, final Object response) { shardReplicaOperationsInProgress.remove(shardName);