Merge pull request #17696 from akka/wip-15410-singleton-name-patriknw
!clt #15410 Change path in ClusterSingletonProxy.props
This commit is contained in:
commit
b7297e2887
17 changed files with 73 additions and 121 deletions
|
|
@ -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 = ""
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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): _*) {
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)) ⇒
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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) {
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
==============================
|
==============================
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
|
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue