Merge pull request #17696 from akka/wip-15410-singleton-name-patriknw

!clt #15410 Change path in ClusterSingletonProxy.props
This commit is contained in:
Patrik Nordwall 2015-06-18 12:45:22 +02:00
commit b7297e2887
17 changed files with 73 additions and 121 deletions

View file

@ -136,6 +136,9 @@ akka.cluster.singleton {
} }
akka.cluster.singleton-proxy { akka.cluster.singleton-proxy {
# The actor name of the singleton actor that is started by the ClusterSingletonManager
singleton-name = ${akka.cluster.singleton.singleton-name}
# The role of the cluster nodes where the singleton can be deployed. # The role of the cluster nodes where the singleton can be deployed.
# If the role is not specified then any node will do. # If the role is not specified then any node will do.
role = "" role = ""

View file

@ -353,10 +353,7 @@ class ClusterSingletonManagerIsStuck(message: String) extends AkkaException(mess
* but more than one active singletons is prevented by all reasonable means. Some * but more than one active singletons is prevented by all reasonable means. Some
* corner cases are eventually resolved by configurable timeouts. * corner cases are eventually resolved by configurable timeouts.
* *
* You access the singleton actor with `actorSelection` using the names you have * You access the singleton actor with [[ClusterSingletonProxy]].
* specified when creating the ClusterSingletonManager. You can subscribe to
* [[akka.cluster.ClusterEvent.MemberEvent]] and sort the members by age
* ([[akka.cluster.Member#isOlderThan]]) to keep track of oldest member.
* Alternatively the singleton actor may broadcast its existence when it is started. * Alternatively the singleton actor may broadcast its existence when it is started.
* *
* Use factory method [[ClusterSingletonManager#props]] to create the * Use factory method [[ClusterSingletonManager#props]] to create the
@ -547,8 +544,8 @@ class ClusterSingletonManager(
} }
def gotoOldest(): State = { def gotoOldest(): State = {
logInfo("Singleton manager [{}] starting singleton actor", cluster.selfAddress)
val singleton = context watch context.actorOf(singletonProps, singletonName) val singleton = context watch context.actorOf(singletonProps, singletonName)
logInfo("Singleton manager starting singleton actor [{}]", singleton.path)
goto(Oldest) using OldestData(singleton) goto(Oldest) using OldestData(singleton)
} }

View file

@ -17,6 +17,7 @@ import scala.concurrent.duration._
import scala.language.postfixOps import scala.language.postfixOps
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.actor.NoSerializationVerificationNeeded import akka.actor.NoSerializationVerificationNeeded
import akka.event.Logging
object ClusterSingletonProxySettings { object ClusterSingletonProxySettings {
@ -33,6 +34,7 @@ object ClusterSingletonProxySettings {
*/ */
def apply(config: Config): ClusterSingletonProxySettings = def apply(config: Config): ClusterSingletonProxySettings =
new ClusterSingletonProxySettings( new ClusterSingletonProxySettings(
singletonName = config.getString("singleton-name"),
role = roleOption(config.getString("role")), role = roleOption(config.getString("role")),
singletonIdentificationInterval = config.getDuration("singleton-identification-interval", MILLISECONDS).millis, singletonIdentificationInterval = config.getDuration("singleton-identification-interval", MILLISECONDS).millis,
bufferSize = config.getInt("buffer-size")) bufferSize = config.getInt("buffer-size"))
@ -58,6 +60,7 @@ object ClusterSingletonProxySettings {
} }
/** /**
* @param singletonName The actor name of the singleton actor that is started by the [[ClusterSingletonManager]].
* @param role The role of the cluster nodes where the singleton can be deployed. If None, then any node will do. * @param role The role of the cluster nodes where the singleton can be deployed. If None, then any node will do.
* @param singletonIdentificationInterval Interval at which the proxy will try to resolve the singleton instance. * @param singletonIdentificationInterval Interval at which the proxy will try to resolve the singleton instance.
* @param bufferSize If the location of the singleton is unknown the proxy will buffer this number of messages * @param bufferSize If the location of the singleton is unknown the proxy will buffer this number of messages
@ -66,12 +69,15 @@ object ClusterSingletonProxySettings {
* immediately if the location of the singleton is unknown. * immediately if the location of the singleton is unknown.
*/ */
final class ClusterSingletonProxySettings( final class ClusterSingletonProxySettings(
val singletonName: String,
val role: Option[String], val role: Option[String],
val singletonIdentificationInterval: FiniteDuration, val singletonIdentificationInterval: FiniteDuration,
val bufferSize: Int) extends NoSerializationVerificationNeeded { val bufferSize: Int) extends NoSerializationVerificationNeeded {
require(bufferSize >= 0 && bufferSize <= 10000, "bufferSize must be >= 0 and <= 10000") require(bufferSize >= 0 && bufferSize <= 10000, "bufferSize must be >= 0 and <= 10000")
def withSingletonName(name: String): ClusterSingletonProxySettings = copy(singletonName = name)
def withRole(role: String): ClusterSingletonProxySettings = copy(role = ClusterSingletonProxySettings.roleOption(role)) def withRole(role: String): ClusterSingletonProxySettings = copy(role = ClusterSingletonProxySettings.roleOption(role))
def withRole(role: Option[String]): ClusterSingletonProxySettings = copy(role = role) def withRole(role: Option[String]): ClusterSingletonProxySettings = copy(role = role)
@ -82,21 +88,23 @@ final class ClusterSingletonProxySettings(
def withBufferSize(bufferSize: Int): ClusterSingletonProxySettings = def withBufferSize(bufferSize: Int): ClusterSingletonProxySettings =
copy(bufferSize = bufferSize) copy(bufferSize = bufferSize)
private def copy(role: Option[String] = role, private def copy(singletonName: String = singletonName,
role: Option[String] = role,
singletonIdentificationInterval: FiniteDuration = singletonIdentificationInterval, singletonIdentificationInterval: FiniteDuration = singletonIdentificationInterval,
bufferSize: Int = bufferSize): ClusterSingletonProxySettings = bufferSize: Int = bufferSize): ClusterSingletonProxySettings =
new ClusterSingletonProxySettings(role, singletonIdentificationInterval, bufferSize) new ClusterSingletonProxySettings(singletonName, role, singletonIdentificationInterval, bufferSize)
} }
object ClusterSingletonProxy { object ClusterSingletonProxy {
/** /**
* Scala API: Factory method for `ClusterSingletonProxy` [[akka.actor.Props]]. * Scala API: Factory method for `ClusterSingletonProxy` [[akka.actor.Props]].
* *
* @param singletonPath The logical path of the singleton, i.e., /user/singletonManager/singleton. * @param singletonManagerPath The logical path of the singleton manager, e.g. `/user/singletonManager`,
* which ends with the name you defined in `actorOf` when creating the [[ClusterSingletonManager]].
* @param settings see [[ClusterSingletonProxySettings]] * @param settings see [[ClusterSingletonProxySettings]]
*/ */
def props(singletonPath: String, settings: ClusterSingletonProxySettings): Props = def props(singletonManagerPath: String, settings: ClusterSingletonProxySettings): Props =
Props(new ClusterSingletonProxy(singletonPath, settings)).withDeploy(Deploy.local) Props(new ClusterSingletonProxy(singletonManagerPath, settings)).withDeploy(Deploy.local)
private case object TryToIdentifySingleton private case object TryToIdentifySingleton
@ -120,12 +128,12 @@ object ClusterSingletonProxy {
* Note that this is a best effort implementation: messages can always be lost due to the distributed nature of the * Note that this is a best effort implementation: messages can always be lost due to the distributed nature of the
* actors involved. * actors involved.
*/ */
final class ClusterSingletonProxy(singletonPathString: String, settings: ClusterSingletonProxySettings) extends Actor with ActorLogging { final class ClusterSingletonProxy(singletonManagerPath: String, settings: ClusterSingletonProxySettings) extends Actor with ActorLogging {
import settings._ import settings._
val singletonPath = singletonPathString.split("/") val singletonPath = (singletonManagerPath + "/" + settings.singletonName).split("/")
var identifyCounter = 0 var identifyCounter = 0
var identifyId = createIdentifyId(identifyCounter) var identifyId = createIdentifyId(identifyCounter)
def createIdentifyId(i: Int) = "identify-singleton-" + singletonPath mkString "/" + i def createIdentifyId(i: Int) = "identify-singleton-" + singletonPath.mkString("/") + i
var identifyTimer: Option[Cancellable] = None var identifyTimer: Option[Cancellable] = None
val cluster = Cluster(context.system) val cluster = Cluster(context.system)
@ -221,7 +229,7 @@ final class ClusterSingletonProxy(singletonPathString: String, settings: Cluster
// singleton identification logic // singleton identification logic
case ActorIdentity(identifyId, Some(s)) case ActorIdentity(identifyId, Some(s))
// if the new singleton is defined, deliver all buffered messages // if the new singleton is defined, deliver all buffered messages
log.info("Singleton identified: {}", s.path) log.info("Singleton identified at [{}]", s.path)
singleton = Some(s) singleton = Some(s)
context.watch(s) context.watch(s)
cancelTimer() cancelTimer()
@ -231,7 +239,7 @@ final class ClusterSingletonProxy(singletonPathString: String, settings: Cluster
membersByAge.headOption.foreach { membersByAge.headOption.foreach {
oldest oldest
val singletonAddress = RootActorPath(oldest.address) / singletonPath val singletonAddress = RootActorPath(oldest.address) / singletonPath
log.debug("Trying to identify singleton at {}", singletonAddress) log.debug("Trying to identify singleton at [{}]", singletonAddress)
context.actorSelection(singletonAddress) ! Identify(identifyId) context.actorSelection(singletonAddress) ! Identify(identifyId)
} }
case Terminated(ref) case Terminated(ref)
@ -244,7 +252,9 @@ final class ClusterSingletonProxy(singletonPathString: String, settings: Cluster
case msg: Any case msg: Any
singleton match { singleton match {
case Some(s) case Some(s)
log.debug("Forwarding message type [{}] to current singleton instance", msg.getClass.getName) if (log.isDebugEnabled)
log.debug("Forwarding message of type [{}] to current singleton instance at [{}]: {}",
Logging.simpleName(msg.getClass.getName), s.path)
s forward msg s forward msg
case None case None
buffer(msg) buffer(msg)

View file

@ -80,8 +80,8 @@ class ClusterSingletonManagerChaosSpec extends MultiNodeSpec(ClusterSingletonMan
system.actorOf(ClusterSingletonManager.props( system.actorOf(ClusterSingletonManager.props(
singletonProps = Props(classOf[Echo], testActor), singletonProps = Props(classOf[Echo], testActor),
terminationMessage = PoisonPill, terminationMessage = PoisonPill,
settings = ClusterSingletonManagerSettings(system).withSingletonName("echo")), settings = ClusterSingletonManagerSettings(system)),
name = "singleton") name = "echo")
} }
def crash(roles: RoleName*): Unit = { def crash(roles: RoleName*): Unit = {
@ -94,7 +94,7 @@ class ClusterSingletonManagerChaosSpec extends MultiNodeSpec(ClusterSingletonMan
} }
def echo(oldest: RoleName): ActorSelection = def echo(oldest: RoleName): ActorSelection =
system.actorSelection(RootActorPath(node(oldest).address) / "user" / "singleton" / "echo") system.actorSelection(RootActorPath(node(oldest).address) / "user" / "echo" / "singleton")
def awaitMemberUp(memberProbe: TestProbe, nodes: RoleName*): Unit = { def awaitMemberUp(memberProbe: TestProbe, nodes: RoleName*): Unit = {
runOn(nodes.filterNot(_ == nodes.head): _*) { runOn(nodes.filterNot(_ == nodes.head): _*) {

View file

@ -78,13 +78,13 @@ class ClusterSingletonManagerLeaveSpec extends MultiNodeSpec(ClusterSingletonMan
system.actorOf(ClusterSingletonManager.props( system.actorOf(ClusterSingletonManager.props(
singletonProps = Props(classOf[Echo], testActor), singletonProps = Props(classOf[Echo], testActor),
terminationMessage = PoisonPill, terminationMessage = PoisonPill,
settings = ClusterSingletonManagerSettings(system).withSingletonName("echo")), settings = ClusterSingletonManagerSettings(system)),
name = "singleton") name = "echo")
} }
lazy val echoProxy: ActorRef = { lazy val echoProxy: ActorRef = {
system.actorOf(ClusterSingletonProxy.props( system.actorOf(ClusterSingletonProxy.props(
singletonPath = "/user/singleton/echo", singletonManagerPath = "/user/echo",
settings = ClusterSingletonProxySettings(system)), settings = ClusterSingletonProxySettings(system)),
name = "echoProxy") name = "echoProxy")
} }

View file

@ -50,23 +50,14 @@ object ClusterSingletonManagerSpec extends MultiNodeConfig {
ConfigFactory.parseString("akka.cluster.roles =[worker]")) ConfigFactory.parseString("akka.cluster.roles =[worker]"))
object PointToPointChannel { object PointToPointChannel {
case object RegisterConsumer case object RegisterConsumer
case object UnregisterConsumer case object UnregisterConsumer
case object RegistrationOk case object RegistrationOk
case object UnexpectedRegistration case object UnexpectedRegistration
case object UnregistrationOk case object UnregistrationOk
case object UnexpectedUnregistration case object UnexpectedUnregistration
case object Reset case object Reset
case object ResetOk case object ResetOk
} }
/** /**
@ -154,59 +145,15 @@ object ClusterSingletonManagerSpec extends MultiNodeConfig {
} }
} }
// documentation of how to keep track of the oldest member in user land
//#singleton-proxy
class ConsumerProxy extends Actor {
// subscribe to MemberEvent, re-subscribe when restart
override def preStart(): Unit =
Cluster(context.system).subscribe(self, classOf[MemberEvent])
override def postStop(): Unit =
Cluster(context.system).unsubscribe(self)
val role = "worker"
// sort by age, oldest first
val ageOrdering = Ordering.fromLessThan[Member] {
(a, b) a.isOlderThan(b)
}
var membersByAge: immutable.SortedSet[Member] =
immutable.SortedSet.empty(ageOrdering)
def receive = {
case state: CurrentClusterState
membersByAge = immutable.SortedSet.empty(ageOrdering) ++ state.members.filter(m
m.status == MemberStatus.Up && m.hasRole(role))
case MemberUp(m) if (m.hasRole(role)) membersByAge += m
case MemberRemoved(m, _) if (m.hasRole(role)) membersByAge -= m
case other consumer foreach {
_.tell(other, sender())
}
}
def consumer: Option[ActorSelection] =
membersByAge.headOption map (m context.actorSelection(
RootActorPath(m.address) / "user" / "singleton" / "consumer"))
}
//#singleton-proxy
} }
class ClusterSingletonManagerMultiJvmNode1 extends ClusterSingletonManagerSpec class ClusterSingletonManagerMultiJvmNode1 extends ClusterSingletonManagerSpec
class ClusterSingletonManagerMultiJvmNode2 extends ClusterSingletonManagerSpec class ClusterSingletonManagerMultiJvmNode2 extends ClusterSingletonManagerSpec
class ClusterSingletonManagerMultiJvmNode3 extends ClusterSingletonManagerSpec class ClusterSingletonManagerMultiJvmNode3 extends ClusterSingletonManagerSpec
class ClusterSingletonManagerMultiJvmNode4 extends ClusterSingletonManagerSpec class ClusterSingletonManagerMultiJvmNode4 extends ClusterSingletonManagerSpec
class ClusterSingletonManagerMultiJvmNode5 extends ClusterSingletonManagerSpec class ClusterSingletonManagerMultiJvmNode5 extends ClusterSingletonManagerSpec
class ClusterSingletonManagerMultiJvmNode6 extends ClusterSingletonManagerSpec class ClusterSingletonManagerMultiJvmNode6 extends ClusterSingletonManagerSpec
class ClusterSingletonManagerMultiJvmNode7 extends ClusterSingletonManagerSpec class ClusterSingletonManagerMultiJvmNode7 extends ClusterSingletonManagerSpec
class ClusterSingletonManagerMultiJvmNode8 extends ClusterSingletonManagerSpec class ClusterSingletonManagerMultiJvmNode8 extends ClusterSingletonManagerSpec
class ClusterSingletonManagerSpec extends MultiNodeSpec(ClusterSingletonManagerSpec) with STMultiNodeSpec with ImplicitSender { class ClusterSingletonManagerSpec extends MultiNodeSpec(ClusterSingletonManagerSpec) with STMultiNodeSpec with ImplicitSender {
@ -260,16 +207,15 @@ class ClusterSingletonManagerSpec extends MultiNodeSpec(ClusterSingletonManagerS
system.actorOf(ClusterSingletonManager.props( system.actorOf(ClusterSingletonManager.props(
singletonProps = Props(classOf[Consumer], queue, testActor), singletonProps = Props(classOf[Consumer], queue, testActor),
terminationMessage = End, terminationMessage = End,
settings = ClusterSingletonManagerSettings(system) settings = ClusterSingletonManagerSettings(system).withRole("worker")),
.withSingletonName("consumer").withRole("worker")), name = "consumer")
name = "singleton")
//#create-singleton-manager //#create-singleton-manager
} }
def createSingletonProxy(): ActorRef = { def createSingletonProxy(): ActorRef = {
//#create-singleton-proxy //#create-singleton-proxy
system.actorOf(ClusterSingletonProxy.props( system.actorOf(ClusterSingletonProxy.props(
singletonPath = "/user/singleton/consumer", singletonManagerPath = "/user/consumer",
settings = ClusterSingletonProxySettings(system).withRole("worker")), settings = ClusterSingletonProxySettings(system).withRole("worker")),
name = "consumerProxy") name = "consumerProxy")
//#create-singleton-proxy //#create-singleton-proxy
@ -302,7 +248,7 @@ class ClusterSingletonManagerSpec extends MultiNodeSpec(ClusterSingletonManagerS
} }
def consumer(oldest: RoleName): ActorSelection = def consumer(oldest: RoleName): ActorSelection =
system.actorSelection(RootActorPath(node(oldest).address) / "user" / "singleton" / "consumer") system.actorSelection(RootActorPath(node(oldest).address) / "user" / "consumer" / "singleton")
def verifyRegistration(oldest: RoleName): Unit = { def verifyRegistration(oldest: RoleName): Unit = {
enterBarrier("before-" + oldest.name + "-registration-verified") enterBarrier("before-" + oldest.name + "-registration-verified")
@ -420,7 +366,7 @@ class ClusterSingletonManagerSpec extends MultiNodeSpec(ClusterSingletonManagerS
verifyProxyMsg(second, sixth, msg = msg()) verifyProxyMsg(second, sixth, msg = msg())
runOn(leaveRole) { runOn(leaveRole) {
system.actorSelection("/user/singleton").tell(Identify("singleton"), identifyProbe.ref) system.actorSelection("/user/consumer").tell(Identify("singleton"), identifyProbe.ref)
identifyProbe.expectMsgPF() { identifyProbe.expectMsgPF() {
case ActorIdentity("singleton", None) // already terminated case ActorIdentity("singleton", None) // already terminated
case ActorIdentity("singleton", Some(singleton)) case ActorIdentity("singleton", Some(singleton))

View file

@ -72,13 +72,13 @@ class ClusterSingletonManagerStartupSpec extends MultiNodeSpec(ClusterSingletonM
system.actorOf(ClusterSingletonManager.props( system.actorOf(ClusterSingletonManager.props(
singletonProps = Props(classOf[Echo], testActor), singletonProps = Props(classOf[Echo], testActor),
terminationMessage = PoisonPill, terminationMessage = PoisonPill,
settings = ClusterSingletonManagerSettings(system).withSingletonName("echo")), settings = ClusterSingletonManagerSettings(system)),
name = "singleton") name = "echo")
} }
lazy val echoProxy: ActorRef = { lazy val echoProxy: ActorRef = {
system.actorOf(ClusterSingletonProxy.props( system.actorOf(ClusterSingletonProxy.props(
singletonPath = "/user/singleton/echo", singletonManagerPath = "/user/echo",
settings = ClusterSingletonProxySettings(system)), settings = ClusterSingletonProxySettings(system)),
name = "echoProxy") name = "echoProxy")
} }

View file

@ -31,16 +31,18 @@ public class ClusterSingletonManagerTest {
final ActorRef testActor = null; final ActorRef testActor = null;
//#create-singleton-manager //#create-singleton-manager
final ClusterSingletonManagerSettings settings = ClusterSingletonManagerSettings.create(system) final ClusterSingletonManagerSettings settings =
.withSingletonName("consumer").withRole("worker"); ClusterSingletonManagerSettings.create(system).withRole("worker");
system.actorOf(ClusterSingletonManager.props(Props.create(Consumer.class, queue, testActor), system.actorOf(ClusterSingletonManager.props(
new End(), settings), "singleton"); Props.create(Consumer.class, queue, testActor),
new End(), settings), "consumer");
//#create-singleton-manager //#create-singleton-manager
//#create-singleton-proxy //#create-singleton-proxy
ClusterSingletonProxySettings proxySettings = ClusterSingletonProxySettings proxySettings =
ClusterSingletonProxySettings.create(system).withRole("worker"); ClusterSingletonProxySettings.create(system).withRole("worker");
system.actorOf(ClusterSingletonProxy.props("user/singleton/consumer", proxySettings), "consumerProxy"); system.actorOf(ClusterSingletonProxy.props("/user/consumer", proxySettings),
"consumerProxy");
//#create-singleton-proxy //#create-singleton-proxy
} }

View file

@ -46,7 +46,7 @@ object ClusterSingletonProxySpec {
name = "singletonManager") name = "singletonManager")
} }
val proxy = system.actorOf(ClusterSingletonProxy.props("user/singletonManager/singleton", val proxy = system.actorOf(ClusterSingletonProxy.props("user/singletonManager",
settings = ClusterSingletonProxySettings(system)), s"singletonProxy-${cluster.selfAddress.port.getOrElse(0)}") settings = ClusterSingletonProxySettings(system)), s"singletonProxy-${cluster.selfAddress.port.getOrElse(0)}")
def testProxy(msg: String) { def testProxy(msg: String) {

View file

@ -262,6 +262,12 @@ The buffer size of the ``ClusterSingletonProxy`` can be defined in the ``Cluster
instead of defining ``stash-capacity`` of the mailbox. Buffering can be disabled by using a instead of defining ``stash-capacity`` of the mailbox. Buffering can be disabled by using a
buffer size of 0. buffer size of 0.
The ``singletonPath`` parameter of ``ClusterSingletonProxy.props`` has changed. It is now named
``singletonManagerPath`` and is the logical path of the singleton manager, e.g. ``/user/singletonManager``,
which ends with the name you defined in ``actorOf`` when creating the ``ClusterSingletonManager``.
In 2.3.x it was the path to singleton instance, which was error-prone because one had to provide both
the name of the singleton manager and the singleton actor.
DistributedPubSub construction DistributedPubSub construction
============================== ==============================

View file

@ -89,22 +89,12 @@ In Scala:
.. includecode:: ../../../akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerSpec.scala#create-singleton-manager .. includecode:: ../../../akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerSpec.scala#create-singleton-manager
Here we limit the singleton to nodes tagged with the ``"worker"`` role, but all nodes, independent of
role, can be used by specifying ``None`` as ``role`` parameter.
The corresponding Java API for the ``singletonProps`` function is ``akka.cluster.singleton.ClusterSingletonPropsFactory``.
The Java API takes a plain String for the role parameter and ``null`` means that all nodes, independent of
role, are used.
In Java: In Java:
.. includecode:: ../../../akka-cluster-tools/src/test/java/akka/cluster/singleton/ClusterSingletonManagerTest.java#create-singleton-manager .. includecode:: ../../../akka-cluster-tools/src/test/java/akka/cluster/singleton/ClusterSingletonManagerTest.java#create-singleton-manager
.. note:: Here we limit the singleton to nodes tagged with the ``"worker"`` role, but all nodes, independent of
role, can be used by not specifying ``withRole``.
The ``singletonProps``/``singletonPropsFactory`` is invoked when creating
the singleton actor and it must not use members that are not thread safe, e.g.
mutable state in enclosing actor.
Here we use an application specific ``terminationMessage`` to be able to close the Here we use an application specific ``terminationMessage`` to be able to close the
resources before actually stopping the singleton actor. Note that ``PoisonPill`` is a resources before actually stopping the singleton actor. Note that ``PoisonPill`` is a

View file

@ -35,16 +35,16 @@ public class StatsSampleOneMasterMain {
//#create-singleton-manager //#create-singleton-manager
ClusterSingletonManagerSettings settings = ClusterSingletonManagerSettings.create(system) ClusterSingletonManagerSettings settings = ClusterSingletonManagerSettings.create(system)
.withSingletonName("statsService").withRole("compute"); .withRole("compute");
system.actorOf(ClusterSingletonManager.props( system.actorOf(ClusterSingletonManager.props(
Props.create(StatsService.class), PoisonPill.getInstance(), settings), Props.create(StatsService.class), PoisonPill.getInstance(), settings),
"singleton"); "statsService");
//#create-singleton-manager //#create-singleton-manager
//#singleton-proxy //#singleton-proxy
ClusterSingletonProxySettings proxySettings = ClusterSingletonProxySettings proxySettings =
ClusterSingletonProxySettings.create(system).withRole("compute"); ClusterSingletonProxySettings.create(system).withRole("compute");
system.actorOf(ClusterSingletonProxy.props("/user/singleton/statsService", system.actorOf(ClusterSingletonProxy.props("/user/statsService",
proxySettings), "statsServiceProxy"); proxySettings), "statsServiceProxy");
//#singleton-proxy //#singleton-proxy
} }

View file

@ -2,7 +2,7 @@ include "application"
# //#config-router-deploy # //#config-router-deploy
akka.actor.deployment { akka.actor.deployment {
/singleton/statsService/workerRouter { /statsService/singleton/workerRouter {
router = consistent-hashing-pool router = consistent-hashing-pool
nr-of-instances = 100 nr-of-instances = 100
cluster { cluster {

View file

@ -54,7 +54,7 @@ object StatsSampleSingleMasterSpecConfig extends MultiNodeConfig {
akka.cluster.roles = [compute] akka.cluster.roles = [compute]
#//#router-deploy-config #//#router-deploy-config
akka.actor.deployment { akka.actor.deployment {
/singleton/statsService/workerRouter { /statsService/singleton/workerRouter {
router = consistent-hashing-pool router = consistent-hashing-pool
nr-of-instances = 100 nr-of-instances = 100
cluster { cluster {
@ -105,10 +105,10 @@ abstract class StatsSampleSingleMasterSpec extends MultiNodeSpec(StatsSampleSing
system.actorOf(ClusterSingletonManager.props( system.actorOf(ClusterSingletonManager.props(
Props[StatsService], Props[StatsService],
terminationMessage = PoisonPill, terminationMessage = PoisonPill,
settings = ClusterSingletonManagerSettings(system).withSingletonName("statsService")), settings = ClusterSingletonManagerSettings(system)),
name = "singleton") name = "statsService")
system.actorOf(ClusterSingletonProxy.props("/user/singleton/statsService", system.actorOf(ClusterSingletonProxy.props("/user/statsService",
ClusterSingletonProxySettings(system).withRole("compute")), "statsServiceProxy") ClusterSingletonProxySettings(system).withRole("compute")), "statsServiceProxy")
testConductor.enter("all-up") testConductor.enter("all-up")

View file

@ -2,7 +2,7 @@ include "application"
# //#config-router-deploy # //#config-router-deploy
akka.actor.deployment { akka.actor.deployment {
/singleton/statsService/workerRouter { /statsService/singleton/workerRouter {
router = consistent-hashing-pool router = consistent-hashing-pool
nr-of-instances = 100 nr-of-instances = 100
cluster { cluster {

View file

@ -33,13 +33,12 @@ object StatsSampleOneMaster {
system.actorOf(ClusterSingletonManager.props( system.actorOf(ClusterSingletonManager.props(
singletonProps = Props[StatsService], singletonProps = Props[StatsService],
terminationMessage = PoisonPill, terminationMessage = PoisonPill,
settings = ClusterSingletonManagerSettings(system) settings = ClusterSingletonManagerSettings(system).withRole("compute")),
.withSingletonName("statsService").withRole("compute")), name = "statsService")
name = "singleton")
//#create-singleton-manager //#create-singleton-manager
//#singleton-proxy //#singleton-proxy
system.actorOf(ClusterSingletonProxy.props(singletonPath = "/user/singleton/statsService", system.actorOf(ClusterSingletonProxy.props(singletonManagerPath = "/user/statsService",
settings = ClusterSingletonProxySettings(system).withRole("compute")), settings = ClusterSingletonProxySettings(system).withRole("compute")),
name = "statsServiceProxy") name = "statsServiceProxy")
//#singleton-proxy //#singleton-proxy

View file

@ -53,7 +53,7 @@ object StatsSampleSingleMasterSpecConfig extends MultiNodeConfig {
akka.cluster.roles = [compute] akka.cluster.roles = [compute]
#//#router-deploy-config #//#router-deploy-config
akka.actor.deployment { akka.actor.deployment {
/singleton/statsService/workerRouter { /statsService/singleton/workerRouter {
router = consistent-hashing-pool router = consistent-hashing-pool
nr-of-instances = 100 nr-of-instances = 100
cluster { cluster {
@ -103,11 +103,10 @@ abstract class StatsSampleSingleMasterSpec extends MultiNodeSpec(StatsSampleSing
system.actorOf(ClusterSingletonManager.props( system.actorOf(ClusterSingletonManager.props(
singletonProps = Props[StatsService], terminationMessage = PoisonPill, singletonProps = Props[StatsService], terminationMessage = PoisonPill,
settings = ClusterSingletonManagerSettings(system) settings = ClusterSingletonManagerSettings(system).withRole("compute")),
.withSingletonName("statsService").withRole("compute")), name = "statsService")
name = "singleton")
system.actorOf(ClusterSingletonProxy.props(singletonPath = "/user/singleton/statsService", system.actorOf(ClusterSingletonProxy.props(singletonManagerPath = "/user/statsService",
ClusterSingletonProxySettings(system).withRole("compute")), ClusterSingletonProxySettings(system).withRole("compute")),
name = "statsServiceProxy") name = "statsServiceProxy")