Allow passing of delay to the EntityOwnerElectionStrategy 01/29401/4
authorMoiz Raja <moraja@cisco.com>
Fri, 9 Oct 2015 01:52:02 +0000 (18:52 -0700)
committerTom Pantelis <tpanteli@brocade.com>
Thu, 12 Nov 2015 19:13:46 +0000 (19:13 +0000)
Change-Id: If745443585e68a26c10622a7888ec52dbee0059c
Signed-off-by: Moiz Raja <moraja@cisco.com>
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/EntityOwnersModel.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/EntityOwnershipShard.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/selectionstrategy/AbstractEntityOwnerSelectionStrategy.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/selectionstrategy/EntityOwnerSelectionStrategy.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/selectionstrategy/EntityOwnerSelectionStrategyConfig.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/selectionstrategy/EntityOwnerSelectionStrategyWrapper.java [deleted file]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/selectionstrategy/FirstCandidateSelectionStrategy.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/entityownership/EntityOwnershipShardTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/entityownership/selectionstrategy/LastCandidateSelectionStrategy.java

index aa8d654b65d86b37d2730a957e698365004cb68b..205763a00060b9b903c1d95b879fd0c4e22e25d9 100644 (file)
@@ -30,7 +30,7 @@ import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableOr
  *
  * @author Thomas Pantelis
  */
-final class EntityOwnersModel {
+public final class EntityOwnersModel {
     static final  QName ENTITY_QNAME = org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.
             md.sal.clustering.entity.owners.rev150804.entity.owners.entity.type.Entity.QNAME;
     static final QName CANDIDATE_NAME_QNAME = QName.create(Candidate.QNAME, "name");
@@ -111,7 +111,7 @@ final class EntityOwnersModel {
                 ImmutableNodes.leafNode(ENTITY_OWNER_QNAME, owner)).build();
     }
 
-    static String entityTypeFromEntityPath(YangInstanceIdentifier entityPath){
+    public static String entityTypeFromEntityPath(YangInstanceIdentifier entityPath){
         YangInstanceIdentifier parent = entityPath;
         while(!parent.isEmpty()) {
             if (EntityType.QNAME.equals(parent.getLastPathArgument().getNodeType())) {
index 334e093d7f4babfafef4174275d7c32092decbe1..2f3b51e1d1d7ae4c8c2e73cde2ef49788c0fe3c0 100644 (file)
@@ -25,6 +25,7 @@ import static org.opendaylight.controller.cluster.datastore.entityownership.Enti
 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.entityOwnersWithCandidate;
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
+import akka.actor.Cancellable;
 import akka.pattern.Patterns;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
@@ -47,8 +48,7 @@ import org.opendaylight.controller.cluster.datastore.entityownership.messages.Se
 import org.opendaylight.controller.cluster.datastore.entityownership.messages.UnregisterCandidateLocal;
 import org.opendaylight.controller.cluster.datastore.entityownership.messages.UnregisterListenerLocal;
 import org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy.EntityOwnerSelectionStrategy;
-import org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy.EntityOwnerSelectionStrategyWrapper;
-import org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy.FirstCandidateSelectionStrategy;
+import org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy.EntityOwnerSelectionStrategyConfig;
 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
 import org.opendaylight.controller.cluster.datastore.messages.PeerDown;
@@ -66,6 +66,7 @@ import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
 import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
 
 /**
  * Special Shard for EntityOwnership.
@@ -78,8 +79,8 @@ class EntityOwnershipShard extends Shard {
     private final EntityOwnershipListenerSupport listenerSupport;
     private final Set<String> downPeerMemberNames = new HashSet<>();
     private final Map<String, String> peerIdToMemberNames = new HashMap<>();
-    private final Map<String, EntityOwnerSelectionStrategyWrapper> ownerSelectionStrategies = new HashMap<>();
-    private final EntityOwnerSelectionStrategyWrapper defaultEntityOwnerSelectionStrategy;
+    private EntityOwnerSelectionStrategyConfig strategyConfig;
+    private Map<YangInstanceIdentifier, Cancellable> entityToScheduledOwnershipTask = new HashMap<>();
 
     private static DatastoreContext noPersistenceDatastoreContext(DatastoreContext datastoreContext) {
         return DatastoreContext.newBuilderFrom(datastoreContext).persistent(false).build();
@@ -90,8 +91,7 @@ class EntityOwnershipShard extends Shard {
         this.localMemberName = builder.localMemberName;
         this.commitCoordinator = new EntityOwnershipShardCommitCoordinator(builder.localMemberName, LOG);
         this.listenerSupport = new EntityOwnershipListenerSupport(getContext(), persistenceId());
-        this.defaultEntityOwnerSelectionStrategy =
-                createEntityOwnerSelectionStrategyWrapper(FirstCandidateSelectionStrategy.INSTANCE);
+        this.strategyConfig = EntityOwnerSelectionStrategyConfig.newBuilder().build();
 
         for(String peerId: getRaftActorContext().getPeerIds()) {
             ShardIdentifier shardId = ShardIdentifier.builder().fromShardIdString(peerId).build();
@@ -142,6 +142,14 @@ class EntityOwnershipShard extends Shard {
         if(Strings.isNullOrEmpty(currentOwner)) {
             writeNewOwner(selectOwner.getEntityPath(), newOwner(selectOwner.getAllCandidates(),
                     selectOwner.getOwnerSelectionStrategy()));
+
+            Cancellable cancellable = entityToScheduledOwnershipTask.get(selectOwner.getEntityPath());
+            if(cancellable != null){
+                if(!cancellable.isCancelled()){
+                    cancellable.cancel();
+                }
+                entityToScheduledOwnershipTask.remove(selectOwner.getEntityPath());
+            }
         }
     }
 
@@ -263,7 +271,7 @@ class EntityOwnershipShard extends Shard {
             String currentOwner = getCurrentOwner(message.getEntityPath());
             if(message.getRemovedCandidate().equals(currentOwner)){
                 writeNewOwner(message.getEntityPath(),
-                        newOwner(message.getRemainingCandidates(), getEntityOwnerElectionStrategyWrapper(message.getEntityPath())));
+                        newOwner(message.getRemainingCandidates(), getEntityOwnerElectionStrategy(message.getEntityPath())));
             }
         } else {
             // We're not the leader. If the removed candidate is our local member then check if we actually
@@ -283,16 +291,9 @@ class EntityOwnershipShard extends Shard {
         }
     }
 
-    private EntityOwnerSelectionStrategyWrapper getEntityOwnerElectionStrategyWrapper(YangInstanceIdentifier entityPath) {
-        String entityType = EntityOwnersModel.entityTypeFromEntityPath(entityPath);
-        EntityOwnerSelectionStrategyWrapper entityOwnerSelectionStrategy = ownerSelectionStrategies.get(entityType);
-
-        if(entityOwnerSelectionStrategy == null){
-            entityOwnerSelectionStrategy = defaultEntityOwnerSelectionStrategy;
-            ownerSelectionStrategies.put(entityType, entityOwnerSelectionStrategy);
-        }
-
-        return entityOwnerSelectionStrategy;
+    private EntityOwnerSelectionStrategy getEntityOwnerElectionStrategy(YangInstanceIdentifier entityPath) {
+        final String entityType = EntityOwnersModel.entityTypeFromEntityPath(entityPath);
+        return strategyConfig.createStrategy(entityType);
     }
 
     private void onCandidateAdded(CandidateAdded message) {
@@ -307,12 +308,12 @@ class EntityOwnershipShard extends Shard {
         downPeerMemberNames.remove(message.getNewCandidate());
 
         String currentOwner = getCurrentOwner(message.getEntityPath());
+        EntityOwnerSelectionStrategy strategy = getEntityOwnerElectionStrategy(message.getEntityPath());
         if(Strings.isNullOrEmpty(currentOwner)){
-            EntityOwnerSelectionStrategyWrapper strategy = getEntityOwnerElectionStrategyWrapper(message.getEntityPath());
-            if(strategy.selectionDelayInMillis() == 0L) {
+            if(strategy.getSelectionDelayInMillis() == 0L) {
                 writeNewOwner(message.getEntityPath(), newOwner(message.getAllCandidates(), strategy));
             } else {
-                strategy.scheduleOwnerSelection(message.getEntityPath(), message.getAllCandidates());
+                scheduleOwnerSelection(message.getEntityPath(), message.getAllCandidates(), strategy);
             }
         }
     }
@@ -344,7 +345,7 @@ class EntityOwnershipShard extends Shard {
                         node(entityTypeNode.getIdentifier()).node(ENTITY_NODE_ID).node(entityNode.getIdentifier()).
                         node(ENTITY_OWNER_NODE_ID).build();
 
-                Object newOwner = newOwner(getCandidateNames(entityNode), getEntityOwnerElectionStrategyWrapper(entityPath));
+                Object newOwner = newOwner(getCandidateNames(entityNode), getEntityOwnerElectionStrategy(entityPath));
 
                 LOG.debug("{}: Found entity {}, writing new owner {}", persistenceId(), entityPath, newOwner);
 
@@ -422,8 +423,8 @@ class EntityOwnershipShard extends Shard {
         }
     }
 
-    private static Collection<String> getCandidateNames(MapEntryNode entity) {
-        Collection<MapEntryNode> candidates = ((MapNode)entity.getChild(CANDIDATE_NODE_ID).get()).getValue();
+    private Collection<String> getCandidateNames(MapEntryNode entity) {
+        Collection<MapEntryNode> candidates = ((MapNode) entity.getChild(CANDIDATE_NODE_ID).get()).getValue();
         Collection<String> candidateNames = new ArrayList<>(candidates.size());
         for(MapEntryNode candidate: candidates) {
             candidateNames.add(candidate.getChild(CANDIDATE_NAME_NODE_ID).get().getValue().toString());
@@ -439,6 +440,26 @@ class EntityOwnershipShard extends Shard {
                 ImmutableNodes.leafNode(ENTITY_OWNER_NODE_ID, newOwner)), this);
     }
 
+    /**
+     * Schedule a new owner selection job. Cancelling any outstanding job if it has not been cancelled.
+     *
+     * @param entityPath
+     * @param allCandidates
+     */
+    public void scheduleOwnerSelection(YangInstanceIdentifier entityPath, Collection<String> allCandidates,
+                                       EntityOwnerSelectionStrategy strategy){
+        Cancellable lastScheduledTask = entityToScheduledOwnershipTask.get(entityPath);
+        if(lastScheduledTask != null && !lastScheduledTask.isCancelled()){
+            lastScheduledTask.cancel();
+        }
+        lastScheduledTask = context().system().scheduler().scheduleOnce(
+                FiniteDuration.apply(strategy.getSelectionDelayInMillis(), TimeUnit.MILLISECONDS)
+                , self(), new SelectOwner(entityPath, allCandidates, strategy)
+                , context().system().dispatcher(), self());
+
+        entityToScheduledOwnershipTask.put(entityPath, lastScheduledTask);
+    }
+
     private String newOwner(Collection<String> candidates, EntityOwnerSelectionStrategy ownerSelectionStrategy) {
         Collection<String> viableCandidates = getViableCandidates(candidates);
         if(viableCandidates.size() == 0){
@@ -470,20 +491,13 @@ class EntityOwnershipShard extends Shard {
         void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode);
     }
 
-    private EntityOwnerSelectionStrategyWrapper createEntityOwnerSelectionStrategyWrapper(EntityOwnerSelectionStrategy entityOwnerSelectionStrategy){
-        return new EntityOwnerSelectionStrategyWrapper(context().system().scheduler(), self(),
-                context().system().dispatcher(), entityOwnerSelectionStrategy);
-    }
-
     @VisibleForTesting
-    void addEntityOwnerSelectionStrategy(String entityType, Class<? extends EntityOwnerSelectionStrategy> ownerSelectionStrategyClass){
-        try {
-            EntityOwnerSelectionStrategyWrapper strategy =
-                    createEntityOwnerSelectionStrategyWrapper(ownerSelectionStrategyClass.newInstance());
-            ownerSelectionStrategies.put(entityType, strategy);
-        } catch (InstantiationException | IllegalAccessException e) {
-            LOG.error("Exception occurred when adding election strategy", e);
-        }
+    void addEntityOwnerSelectionStrategy(String entityType,
+                                         Class<? extends EntityOwnerSelectionStrategy> clazz,
+                                         long delay){
+        EntityOwnerSelectionStrategyConfig config = EntityOwnerSelectionStrategyConfig.newBuilder()
+                .addStrategy(entityType, clazz, delay).build();
+        this.strategyConfig = config;
     }
 
     public static Builder newBuilder() {
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/selectionstrategy/AbstractEntityOwnerSelectionStrategy.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/selectionstrategy/AbstractEntityOwnerSelectionStrategy.java
new file mode 100644 (file)
index 0000000..df709c1
--- /dev/null
@@ -0,0 +1,23 @@
+/*
+ * Copyright (c) 2015 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+
+package org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy;
+
+public abstract class AbstractEntityOwnerSelectionStrategy implements EntityOwnerSelectionStrategy {
+
+    private final long selectionDelayInMillis;
+
+    protected AbstractEntityOwnerSelectionStrategy(long selectionDelayInMillis) {
+        this.selectionDelayInMillis = selectionDelayInMillis;
+    }
+
+    @Override
+    public long getSelectionDelayInMillis() {
+        return selectionDelayInMillis;
+    }
+}
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/selectionstrategy/EntityOwnerSelectionStrategyConfig.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/selectionstrategy/EntityOwnerSelectionStrategyConfig.java
new file mode 100644 (file)
index 0000000..db2377d
--- /dev/null
@@ -0,0 +1,84 @@
+/*
+ * Copyright (c) 2015 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+
+package org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy;
+
+import java.lang.reflect.InvocationTargetException;
+import java.util.HashMap;
+import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class EntityOwnerSelectionStrategyConfig {
+    private static final Logger LOG = LoggerFactory.getLogger(EntityOwnerSelectionStrategyConfig.class);
+    private final Map<String, StrategyInfo> entityTypeToStrategyInfo = new HashMap<>();
+    private final Map<String, EntityOwnerSelectionStrategy> entityTypeToOwnerSelectionStrategy = new HashMap<>();
+
+    private EntityOwnerSelectionStrategyConfig(){
+
+    }
+
+    public EntityOwnerSelectionStrategy createStrategy(String entityType){
+        final EntityOwnerSelectionStrategy strategy;
+        final EntityOwnerSelectionStrategy existingStrategy = entityTypeToOwnerSelectionStrategy.get(entityType);
+        if(existingStrategy != null){
+            strategy = existingStrategy;
+        } else {
+            EntityOwnerSelectionStrategyConfig.StrategyInfo strategyInfo = entityTypeToStrategyInfo.get(entityType);
+            if(strategyInfo == null){
+                strategy = FirstCandidateSelectionStrategy.INSTANCE;
+            } else {
+                strategy = strategyInfo.createStrategy();
+            }
+            entityTypeToOwnerSelectionStrategy.put(entityType, strategy);
+        }
+        return strategy;
+
+    }
+
+    private static final class StrategyInfo {
+        private final Class<? extends EntityOwnerSelectionStrategy> strategyClass;
+        private final long delay;
+
+        private StrategyInfo(Class<? extends EntityOwnerSelectionStrategy> strategyClass, long delay) {
+            this.strategyClass = strategyClass;
+            this.delay = delay;
+        }
+
+        public EntityOwnerSelectionStrategy createStrategy(){
+            try {
+                return strategyClass.getDeclaredConstructor(long.class).newInstance(delay);
+            } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) {
+                LOG.warn("could not create custom strategy", e);
+            }
+            return FirstCandidateSelectionStrategy.INSTANCE;
+        }
+    }
+
+    public static Builder newBuilder(){
+        return new Builder(new EntityOwnerSelectionStrategyConfig());
+    }
+
+    public static class Builder {
+        private final EntityOwnerSelectionStrategyConfig config;
+
+        private Builder(EntityOwnerSelectionStrategyConfig config){
+            this.config = config;
+        }
+
+        public Builder addStrategy(String entityType, Class<? extends EntityOwnerSelectionStrategy> strategy, long delay){
+            config.entityTypeToStrategyInfo.put(entityType, new StrategyInfo(strategy, delay));
+            return this;
+        }
+
+        public EntityOwnerSelectionStrategyConfig build(){
+            return this.config;
+        }
+    }
+
+}
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/selectionstrategy/EntityOwnerSelectionStrategyWrapper.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/selectionstrategy/EntityOwnerSelectionStrategyWrapper.java
deleted file mode 100644 (file)
index da3a4e0..0000000
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Copyright (c) 2015 Cisco Systems, Inc. and others.  All rights reserved.
- *
- * This program and the accompanying materials are made available under the
- * terms of the Eclipse Public License v1.0 which accompanies this distribution,
- * and is available at http://www.eclipse.org/legal/epl-v10.html
- */
-
-package org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy;
-
-import akka.actor.ActorRef;
-import akka.actor.Cancellable;
-import akka.actor.Scheduler;
-import com.google.common.base.Preconditions;
-import java.util.Collection;
-import java.util.concurrent.TimeUnit;
-import org.opendaylight.controller.cluster.datastore.entityownership.messages.SelectOwner;
-import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
-import scala.concurrent.ExecutionContextExecutor;
-import scala.concurrent.duration.FiniteDuration;
-
-/**
- * The EntityOwnerSelectionStrategyWrapper is an EntityOwnerSelectionStrategy decorator that adds the ability to
- * schedule an owner selection job.
- */
-public class EntityOwnerSelectionStrategyWrapper implements EntityOwnerSelectionStrategy {
-    private final Scheduler scheduler;
-    private final ActorRef shard;
-    private final ExecutionContextExecutor dispatcher;
-    private final EntityOwnerSelectionStrategy strategy;
-
-    private Cancellable lastScheduledTask;
-
-    public EntityOwnerSelectionStrategyWrapper(Scheduler scheduler,
-                                                ActorRef shard,
-                                                ExecutionContextExecutor dispatcher,
-                                                EntityOwnerSelectionStrategy strategy) {
-        this.scheduler = Preconditions.checkNotNull(scheduler);
-        this.shard = Preconditions.checkNotNull(shard);
-        this.dispatcher = Preconditions.checkNotNull(dispatcher);
-        this.strategy = Preconditions.checkNotNull(strategy);
-    }
-
-    /**
-     * Schedule a new owner selection job. Cancelling any outstanding job if it has not been cancelled.
-     *
-     * @param entityPath
-     * @param allCandidates
-     */
-    public void scheduleOwnerSelection(YangInstanceIdentifier entityPath, Collection<String> allCandidates){
-        if(lastScheduledTask != null && !lastScheduledTask.isCancelled()){
-            lastScheduledTask.cancel();
-        }
-        lastScheduledTask = scheduler.scheduleOnce(
-                FiniteDuration.apply(strategy.selectionDelayInMillis(), TimeUnit.MILLISECONDS)
-                , shard, new SelectOwner(entityPath, allCandidates, strategy)
-                , dispatcher, shard);
-    }
-
-    @Override
-    public long selectionDelayInMillis(){
-        return strategy.selectionDelayInMillis();
-    }
-
-    @Override
-    public String newOwner(Collection<String> viableCandidates){
-        return strategy.newOwner(viableCandidates);
-    }
-}
\ No newline at end of file
index a619035f35f9fc6963ab89e65579391bb7fc94c5..b009c3aff429e29693a7de30abdbb427f92652b2 100644 (file)
@@ -14,13 +14,12 @@ import java.util.Collection;
 /**
  * The FirstCandidateSelectionStrategy always selects the first viable candidate from the list of candidates
  */
-public class FirstCandidateSelectionStrategy implements EntityOwnerSelectionStrategy {
+public class FirstCandidateSelectionStrategy extends AbstractEntityOwnerSelectionStrategy {
 
-    public static final FirstCandidateSelectionStrategy INSTANCE = new FirstCandidateSelectionStrategy();
+    public static final FirstCandidateSelectionStrategy INSTANCE = new FirstCandidateSelectionStrategy(0L);
 
-    @Override
-    public long selectionDelayInMillis() {
-        return 0;
+    public FirstCandidateSelectionStrategy(long selectionDelayInMillis) {
+        super(selectionDelayInMillis);
     }
 
     @Override
index cb8dade4cbf7e5ba04c230d6646265ced734e22c..63ec7cf55e3e9bba4d7c0420d63da39cc237c46b 100644 (file)
@@ -878,7 +878,7 @@ public class EntityOwnershipShardTest extends AbstractEntityOwnershipTest {
     public void testDelayedEntityOwnerSelection() throws Exception {
         ShardTestKit kit = new ShardTestKit(getSystem());
         TestActorRef<EntityOwnershipShard> shard = actorFactory.createTestActor(newShardProps());
-        shard.underlyingActor().addEntityOwnerSelectionStrategy(ENTITY_TYPE, LastCandidateSelectionStrategy.class);
+        shard.underlyingActor().addEntityOwnerSelectionStrategy(ENTITY_TYPE, LastCandidateSelectionStrategy.class, 500);
         kit.waitUntilLeader(shard);
 
         Entity entity = new Entity(ENTITY_TYPE, ENTITY_ID1);
index db5d56b9dcb72002d20fa513dae8ec15446b4d71..30c627735730d668b600015b369e606d74c9d2c9 100644 (file)
@@ -12,10 +12,9 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
-public class LastCandidateSelectionStrategy implements EntityOwnerSelectionStrategy {
-    @Override
-    public long selectionDelayInMillis() {
-        return 500;
+public class LastCandidateSelectionStrategy extends AbstractEntityOwnerSelectionStrategy {
+    public LastCandidateSelectionStrategy(long selectionDelayInMillis) {
+        super(selectionDelayInMillis);
     }
 
     @Override