!act #3583 Timer based auto-down
* Replace (deprecate) akka.cluster.auto-down config setting with akka.cluster.auto-down-unreachable-after * AutoDown actor that keeps track of unreachable members and performs down from the leader node when they have been unreachable for the specified duration * Migration guide
This commit is contained in:
parent
c55189f615
commit
d5b25cbbc6
27 changed files with 355 additions and 450 deletions
|
|
@ -23,9 +23,13 @@ akka {
|
|||
retry-unsuccessful-join-after = 10s
|
||||
|
||||
# Should the 'leader' in the cluster be allowed to automatically mark
|
||||
# unreachable nodes as DOWN?
|
||||
# unreachable nodes as DOWN after a configured time of unreachability?
|
||||
# Using auto-down implies that two separate clusters will automatically be
|
||||
# formed in case of network partition.
|
||||
# Disable with "off" or specify a duration to enable auto-down.
|
||||
auto-down-unreachable-after = off
|
||||
|
||||
# deprecated in 2.3, use 'auto-down-unreachable-after' instead
|
||||
auto-down = off
|
||||
|
||||
# The roles of this member. List of strings, e.g. roles = ["A", "B"].
|
||||
|
|
|
|||
148
akka-cluster/src/main/scala/akka/cluster/AutoDown.scala
Normal file
148
akka-cluster/src/main/scala/akka/cluster/AutoDown.scala
Normal file
|
|
@ -0,0 +1,148 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster
|
||||
|
||||
import akka.actor.Actor
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import akka.actor.Props
|
||||
import akka.cluster.ClusterEvent._
|
||||
import akka.actor.Cancellable
|
||||
import scala.concurrent.duration.Duration
|
||||
import akka.actor.Address
|
||||
import akka.actor.Scheduler
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[cluster] object AutoDown {
|
||||
|
||||
def props(autoDownUnreachableAfter: FiniteDuration): Props =
|
||||
Props(classOf[AutoDown], autoDownUnreachableAfter)
|
||||
|
||||
case class UnreachableTimeout(node: UniqueAddress)
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* An unreachable member will be downed by this actor if it remains unreachable
|
||||
* for the specified duration and this actor is running on the leader node in the
|
||||
* cluster.
|
||||
*
|
||||
* The implementation is split into two classes AutoDown and AutoDownBase to be
|
||||
* able to unit test the logic without running cluster.
|
||||
*/
|
||||
private[cluster] class AutoDown(autoDownUnreachableAfter: FiniteDuration)
|
||||
extends AutoDownBase(autoDownUnreachableAfter) {
|
||||
|
||||
val cluster = Cluster(context.system)
|
||||
import cluster.InfoLogger._
|
||||
|
||||
override def selfAddress = cluster.selfAddress
|
||||
|
||||
override def scheduler: Scheduler = cluster.scheduler
|
||||
|
||||
// re-subscribe when restart
|
||||
override def preStart(): Unit = {
|
||||
cluster.subscribe(self, classOf[ClusterDomainEvent])
|
||||
super.preStart()
|
||||
}
|
||||
override def postStop(): Unit = {
|
||||
cluster.unsubscribe(self)
|
||||
super.postStop()
|
||||
}
|
||||
|
||||
override def down(node: Address): Unit = {
|
||||
require(leader)
|
||||
logInfo("Leader is auto-downing unreachable node [{}]", node)
|
||||
cluster.down(node)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* The implementation is split into two classes AutoDown and AutoDownBase to be
|
||||
* able to unit test the logic without running cluster.
|
||||
*/
|
||||
private[cluster] abstract class AutoDownBase(autoDownUnreachableAfter: FiniteDuration) extends Actor {
|
||||
|
||||
import AutoDown._
|
||||
|
||||
def selfAddress: Address
|
||||
|
||||
def down(node: Address): Unit
|
||||
|
||||
def scheduler: Scheduler
|
||||
|
||||
import context.dispatcher
|
||||
|
||||
val skipMemberStatus = Gossip.convergenceSkipUnreachableWithMemberStatus
|
||||
|
||||
var scheduledUnreachable: Map[UniqueAddress, Cancellable] = Map.empty
|
||||
var pendingUnreachable: Set[UniqueAddress] = Set.empty
|
||||
var leader = false
|
||||
|
||||
override def postStop(): Unit = {
|
||||
scheduledUnreachable.values foreach { _.cancel }
|
||||
}
|
||||
|
||||
def receive = {
|
||||
case state: CurrentClusterState ⇒
|
||||
leader = state.leader.exists(_ == selfAddress)
|
||||
state.unreachable foreach unreachableMember
|
||||
|
||||
case UnreachableMember(m) ⇒ unreachableMember(m)
|
||||
|
||||
case ReachableMember(m) ⇒ remove(m.uniqueAddress)
|
||||
case MemberRemoved(m, _) ⇒ remove(m.uniqueAddress)
|
||||
|
||||
case LeaderChanged(leaderOption) ⇒
|
||||
leader = leaderOption.exists(_ == selfAddress)
|
||||
if (leader) {
|
||||
pendingUnreachable.foreach(node ⇒ down(node.address))
|
||||
pendingUnreachable = Set.empty
|
||||
}
|
||||
|
||||
case UnreachableTimeout(node) ⇒
|
||||
if (scheduledUnreachable contains node) {
|
||||
scheduledUnreachable -= node
|
||||
downOrAddPending(node)
|
||||
}
|
||||
|
||||
case _: ClusterDomainEvent ⇒ // not interested in other events
|
||||
|
||||
}
|
||||
|
||||
def unreachableMember(m: Member): Unit =
|
||||
if (!skipMemberStatus(m.status) && !scheduledUnreachable.contains(m.uniqueAddress))
|
||||
scheduleUnreachable(m.uniqueAddress)
|
||||
|
||||
def scheduleUnreachable(node: UniqueAddress): Unit = {
|
||||
if (autoDownUnreachableAfter == Duration.Zero) {
|
||||
downOrAddPending(node)
|
||||
} else {
|
||||
val task = scheduler.scheduleOnce(autoDownUnreachableAfter, self, UnreachableTimeout(node))
|
||||
scheduledUnreachable += (node -> task)
|
||||
}
|
||||
}
|
||||
|
||||
def downOrAddPending(node: UniqueAddress): Unit = {
|
||||
if (leader) {
|
||||
down(node.address)
|
||||
} else {
|
||||
// it's supposed to be downed by another node, current leader, but if that crash
|
||||
// a new leader must pick up these
|
||||
pendingUnreachable += node
|
||||
}
|
||||
}
|
||||
|
||||
def remove(node: UniqueAddress): Unit = {
|
||||
scheduledUnreachable.get(node) foreach { _.cancel }
|
||||
scheduledUnreachable -= node
|
||||
pendingUnreachable -= node
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -100,6 +100,9 @@ class Cluster(val system: ExtendedActorSystem) extends Extension {
|
|||
|
||||
logInfo("Starting up...")
|
||||
|
||||
if (settings.AutoDown)
|
||||
log.warning("[akka.cluster.auto-down] setting is replaced by [akka.cluster.auto-down-unreachable-after]")
|
||||
|
||||
val failureDetector: FailureDetectorRegistry[Address] = {
|
||||
def createFailureDetector(): FailureDetector =
|
||||
FailureDetectorLoader.load(settings.FailureDetectorImplementationClass, settings.FailureDetectorConfig, system)
|
||||
|
|
|
|||
|
|
@ -219,7 +219,7 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
|
||||
val cluster = Cluster(context.system)
|
||||
import cluster.{ selfAddress, scheduler, failureDetector }
|
||||
import cluster.settings._
|
||||
import cluster.settings.{ AutoDown ⇒ _, _ }
|
||||
import cluster.InfoLogger._
|
||||
|
||||
protected def selfUniqueAddress = cluster.selfUniqueAddress
|
||||
|
|
@ -267,6 +267,13 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
|
||||
override def preStart(): Unit = {
|
||||
context.system.eventStream.subscribe(self, classOf[QuarantinedEvent])
|
||||
|
||||
AutoDownUnreachableAfter match {
|
||||
case d: FiniteDuration ⇒
|
||||
context.actorOf(AutoDown.props(d) withDispatcher (context.props.dispatcher), name = "autoDown")
|
||||
case _ ⇒ // auto-down is disabled
|
||||
}
|
||||
|
||||
if (SeedNodes.isEmpty)
|
||||
logInfo("No seed-nodes configured, manual cluster join required")
|
||||
else
|
||||
|
|
@ -691,16 +698,11 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
}
|
||||
|
||||
/**
|
||||
* Runs periodic leader actions, such as member status transitions, auto-downing unreachable nodes,
|
||||
* assigning partitions etc.
|
||||
* Runs periodic leader actions, such as member status transitions, assigning partitions etc.
|
||||
*/
|
||||
def leaderActions(): Unit =
|
||||
if (latestGossip.isLeader(selfUniqueAddress)) {
|
||||
// only run the leader actions if we are the LEADER
|
||||
|
||||
if (AutoDown)
|
||||
leaderAutoDownActions()
|
||||
|
||||
if (latestGossip.convergence)
|
||||
leaderActionsOnConvergence()
|
||||
}
|
||||
|
|
@ -812,44 +814,6 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* When the node is in the UNREACHABLE set it can be auto-down by leader
|
||||
*/
|
||||
def leaderAutoDownActions(): Unit = {
|
||||
val localGossip = latestGossip
|
||||
val localMembers = localGossip.members
|
||||
val localOverview = localGossip.overview
|
||||
val localSeen = localOverview.seen
|
||||
|
||||
val changedUnreachableMembers = for {
|
||||
node ← localOverview.reachability.allUnreachableOrTerminated
|
||||
m = localGossip.member(node)
|
||||
if m.status != Removed && !Gossip.convergenceSkipUnreachableWithMemberStatus(m.status)
|
||||
} yield m.copy(status = Down)
|
||||
|
||||
if (changedUnreachableMembers.nonEmpty) {
|
||||
// handle changes
|
||||
|
||||
// replace changed unreachable
|
||||
val newMembers = localMembers -- changedUnreachableMembers ++ changedUnreachableMembers
|
||||
|
||||
// removing nodes marked as Down/Exiting from the `seen` table
|
||||
val newSeen = localSeen -- changedUnreachableMembers.map(_.uniqueAddress)
|
||||
|
||||
val newOverview = localOverview copy (seen = newSeen) // update gossip overview
|
||||
val newGossip = localGossip copy (members = newMembers, overview = newOverview) // update gossip
|
||||
|
||||
updateLatestGossip(newGossip)
|
||||
|
||||
// log the auto-downing of the unreachable nodes
|
||||
changedUnreachableMembers foreach { m ⇒
|
||||
logInfo("Leader is marking unreachable node [{}] as [{}]", m.address, m.status)
|
||||
}
|
||||
|
||||
publish(latestGossip)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reaps the unreachable members according to the failure detector's verdict.
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -59,7 +59,17 @@ final class ClusterSettings(val config: Config, val systemName: String) {
|
|||
case _ ⇒ Duration(cc.getMilliseconds(key), MILLISECONDS) requiring (_ >= Duration.Zero, key + " >= 0s, or off")
|
||||
}
|
||||
}
|
||||
|
||||
@deprecated("akka.cluster.auto-down setting is replaced by akka.cluster.auto-down-unreachable-after", "2.3")
|
||||
val AutoDown: Boolean = cc.getBoolean("auto-down")
|
||||
val AutoDownUnreachableAfter: Duration = {
|
||||
val key = "auto-down-unreachable-after"
|
||||
cc.getString(key).toLowerCase match {
|
||||
case "off" ⇒ if (AutoDown) Duration.Zero else Duration.Undefined
|
||||
case _ ⇒ Duration(cc.getMilliseconds(key), MILLISECONDS) requiring (_ >= Duration.Zero, key + " >= 0s, or off")
|
||||
}
|
||||
}
|
||||
|
||||
val Roles: Set[String] = immutableSeq(cc.getStringList("roles")).toSet
|
||||
val MinNrOfMembers: Int = {
|
||||
cc.getInt("min-nr-of-members")
|
||||
|
|
|
|||
|
|
@ -20,7 +20,7 @@ case class LeaderDowningNodeThatIsUnreachableMultiNodeConfig(failureDetectorPupp
|
|||
val fourth = role("fourth")
|
||||
|
||||
commonConfig(debugConfig(on = false).
|
||||
withFallback(ConfigFactory.parseString("akka.cluster.auto-down = on")).
|
||||
withFallback(ConfigFactory.parseString("akka.cluster.auto-down-unreachable-after = 2s")).
|
||||
withFallback(MultiNodeClusterSpec.clusterConfig(failureDetectorPuppet)))
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -19,8 +19,9 @@ object LeaderLeavingMultiJvmSpec extends MultiNodeConfig {
|
|||
val second = role("second")
|
||||
val third = role("third")
|
||||
|
||||
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(
|
||||
"akka.cluster.auto-down=on")).withFallback(MultiNodeClusterSpec.clusterConfigWithFailureDetectorPuppet))
|
||||
commonConfig(debugConfig(on = false).
|
||||
withFallback(ConfigFactory.parseString("akka.cluster.auto-down-unreachable-after = 0s")).
|
||||
withFallback(MultiNodeClusterSpec.clusterConfigWithFailureDetectorPuppet))
|
||||
}
|
||||
|
||||
class LeaderLeavingMultiJvmNode1 extends LeaderLeavingSpec
|
||||
|
|
|
|||
|
|
@ -34,7 +34,6 @@ object MultiNodeClusterSpec {
|
|||
def clusterConfig: Config = ConfigFactory.parseString("""
|
||||
akka.actor.provider = akka.cluster.ClusterActorRefProvider
|
||||
akka.cluster {
|
||||
auto-down = off
|
||||
jmx.enabled = off
|
||||
gossip-interval = 200 ms
|
||||
leader-actions-interval = 200 ms
|
||||
|
|
|
|||
|
|
@ -16,7 +16,8 @@ object NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec extends MultiNodeConfig
|
|||
val third = role("third")
|
||||
|
||||
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(
|
||||
"akka.cluster.auto-down=on")).withFallback(MultiNodeClusterSpec.clusterConfigWithFailureDetectorPuppet))
|
||||
"akka.cluster.auto-down-unreachable-after = 0s")).
|
||||
withFallback(MultiNodeClusterSpec.clusterConfigWithFailureDetectorPuppet))
|
||||
}
|
||||
|
||||
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode1 extends NodeLeavingAndExitingAndBeingRemovedSpec
|
||||
|
|
|
|||
|
|
@ -25,11 +25,8 @@ object RestartFirstSeedNodeMultiJvmSpec extends MultiNodeConfig {
|
|||
val seed3 = role("seed3")
|
||||
|
||||
commonConfig(debugConfig(on = false).
|
||||
withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster {
|
||||
auto-down = on
|
||||
}
|
||||
""")).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
withFallback(ConfigFactory.parseString("akka.cluster.auto-down-unreachable-after = 0s")).
|
||||
withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class RestartFirstSeedNodeMultiJvmNode1 extends RestartFirstSeedNodeSpec
|
||||
|
|
|
|||
|
|
@ -17,7 +17,7 @@ case class SingletonClusterMultiNodeConfig(failureDetectorPuppet: Boolean) exten
|
|||
commonConfig(debugConfig(on = false).
|
||||
withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster {
|
||||
auto-down = on
|
||||
auto-down-unreachable-after = 0s
|
||||
failure-detector.threshold = 4
|
||||
}
|
||||
""")).
|
||||
|
|
|
|||
|
|
@ -26,7 +26,7 @@ case class SplitBrainMultiNodeConfig(failureDetectorPuppet: Boolean) extends Mul
|
|||
withFallback(ConfigFactory.parseString("""
|
||||
akka.remote.retry-gate-closed-for = 3 s
|
||||
akka.cluster {
|
||||
auto-down = on
|
||||
auto-down-unreachable-after = 1s
|
||||
failure-detector.threshold = 4
|
||||
}""")).
|
||||
withFallback(MultiNodeClusterSpec.clusterConfig(failureDetectorPuppet)))
|
||||
|
|
@ -80,14 +80,14 @@ abstract class SplitBrainSpec(multiNodeConfig: SplitBrainMultiNodeConfig)
|
|||
|
||||
runOn(side1: _*) {
|
||||
for (role ← side2) markNodeAsUnavailable(role)
|
||||
// auto-down = on
|
||||
// auto-down
|
||||
awaitMembersUp(side1.size, side2.toSet map address)
|
||||
assertLeader(side1: _*)
|
||||
}
|
||||
|
||||
runOn(side2: _*) {
|
||||
for (role ← side1) markNodeAsUnavailable(role)
|
||||
// auto-down = on
|
||||
// auto-down
|
||||
awaitMembersUp(side2.size, side1.toSet map address)
|
||||
assertLeader(side2: _*)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -122,7 +122,7 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
|
|||
akka.actor.serialize-creators = off
|
||||
akka.actor.provider = akka.cluster.ClusterActorRefProvider
|
||||
akka.cluster {
|
||||
auto-down = on
|
||||
auto-down-unreachable-after = 1s
|
||||
publish-stats-interval = 1s
|
||||
}
|
||||
akka.loggers = ["akka.testkit.TestEventListener"]
|
||||
|
|
|
|||
123
akka-cluster/src/test/scala/akka/cluster/AutoDownSpec.scala
Normal file
123
akka-cluster/src/test/scala/akka/cluster/AutoDownSpec.scala
Normal file
|
|
@ -0,0 +1,123 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
|
||||
import language.postfixOps
|
||||
import language.reflectiveCalls
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.Address
|
||||
import akka.actor.Scheduler
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Props
|
||||
import akka.cluster.MemberStatus._
|
||||
import akka.cluster.ClusterEvent._
|
||||
import akka.testkit.AkkaSpec
|
||||
|
||||
object AutoDownSpec {
|
||||
case class DownCalled(address: Address)
|
||||
|
||||
val memberA = TestMember(Address("akka.tcp", "sys", "a", 2552), Up)
|
||||
val memberB = TestMember(Address("akka.tcp", "sys", "b", 2552), Up)
|
||||
val memberC = TestMember(Address("akka.tcp", "sys", "c", 2552), Up)
|
||||
|
||||
class AutoDownTestActor(
|
||||
autoDownUnreachableAfter: FiniteDuration,
|
||||
probe: ActorRef)
|
||||
extends AutoDownBase(autoDownUnreachableAfter) {
|
||||
|
||||
override def selfAddress = memberA.address
|
||||
override def scheduler: Scheduler = context.system.scheduler
|
||||
|
||||
override def down(node: Address): Unit = {
|
||||
if (leader)
|
||||
probe ! DownCalled(node)
|
||||
else
|
||||
probe ! "down must only be done by leader"
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class AutoDownSpec extends AkkaSpec {
|
||||
import AutoDownSpec._
|
||||
|
||||
def autoDownActor(autoDownUnreachableAfter: FiniteDuration): ActorRef =
|
||||
system.actorOf(Props(classOf[AutoDownTestActor], autoDownUnreachableAfter, testActor))
|
||||
|
||||
"AutoDown" must {
|
||||
|
||||
"down unreachable when leader" in {
|
||||
val a = autoDownActor(Duration.Zero)
|
||||
a ! LeaderChanged(Some(memberA.address))
|
||||
a ! UnreachableMember(memberB)
|
||||
expectMsg(DownCalled(memberB.address))
|
||||
}
|
||||
|
||||
"not down unreachable when not leader" in {
|
||||
val a = autoDownActor(Duration.Zero)
|
||||
a ! LeaderChanged(Some(memberB.address))
|
||||
a ! UnreachableMember(memberC)
|
||||
expectNoMsg(1.second)
|
||||
}
|
||||
|
||||
"down unreachable when becoming leader" in {
|
||||
val a = autoDownActor(Duration.Zero)
|
||||
a ! LeaderChanged(Some(memberB.address))
|
||||
a ! UnreachableMember(memberC)
|
||||
a ! LeaderChanged(Some(memberA.address))
|
||||
expectMsg(DownCalled(memberC.address))
|
||||
}
|
||||
|
||||
"down unreachable after specified duration" in {
|
||||
val a = autoDownActor(2.seconds)
|
||||
a ! LeaderChanged(Some(memberA.address))
|
||||
a ! UnreachableMember(memberB)
|
||||
expectNoMsg(1.second)
|
||||
expectMsg(DownCalled(memberB.address))
|
||||
}
|
||||
|
||||
"down unreachable when becoming leader inbetween detection and specified duration" in {
|
||||
val a = autoDownActor(2.seconds)
|
||||
a ! LeaderChanged(Some(memberB.address))
|
||||
a ! UnreachableMember(memberC)
|
||||
a ! LeaderChanged(Some(memberA.address))
|
||||
expectNoMsg(1.second)
|
||||
expectMsg(DownCalled(memberC.address))
|
||||
}
|
||||
|
||||
"not down unreachable when loosing leadership inbetween detection and specified duration" in {
|
||||
val a = autoDownActor(2.seconds)
|
||||
a ! LeaderChanged(Some(memberA.address))
|
||||
a ! UnreachableMember(memberC)
|
||||
a ! LeaderChanged(Some(memberB.address))
|
||||
expectNoMsg(3.second)
|
||||
}
|
||||
|
||||
"not down when unreachable become reachable inbetween detection and specified duration" in {
|
||||
val a = autoDownActor(2.seconds)
|
||||
a ! LeaderChanged(Some(memberA.address))
|
||||
a ! UnreachableMember(memberB)
|
||||
a ! ReachableMember(memberB)
|
||||
expectNoMsg(3.second)
|
||||
}
|
||||
|
||||
"not down when unreachable is removed inbetween detection and specified duration" in {
|
||||
val a = autoDownActor(2.seconds)
|
||||
a ! LeaderChanged(Some(memberA.address))
|
||||
a ! UnreachableMember(memberB)
|
||||
a ! MemberRemoved(memberB.copy(Removed), previousStatus = Exiting)
|
||||
expectNoMsg(3.second)
|
||||
}
|
||||
|
||||
"not down when unreachable is already Down" in {
|
||||
val a = autoDownActor(Duration.Zero)
|
||||
a ! LeaderChanged(Some(memberA.address))
|
||||
a ! UnreachableMember(memberB.copy(Down))
|
||||
expectNoMsg(1.second)
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -37,7 +37,7 @@ class ClusterConfigSpec extends AkkaSpec {
|
|||
LeaderActionsInterval must be(1 second)
|
||||
UnreachableNodesReaperInterval must be(1 second)
|
||||
PublishStatsInterval must be(Duration.Undefined)
|
||||
AutoDown must be(false)
|
||||
AutoDownUnreachableAfter must be(Duration.Undefined)
|
||||
MinNrOfMembers must be(1)
|
||||
MinNrOfMembersOfRole must be === Map.empty
|
||||
Roles must be === Set.empty
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ import akka.actor.ActorRef
|
|||
object ClusterSpec {
|
||||
val config = """
|
||||
akka.cluster {
|
||||
auto-down = off
|
||||
auto-down-unreachable-after = 0s
|
||||
periodic-tasks-initial-delay = 120 seconds // turn off scheduled tasks
|
||||
publish-stats-interval = 0 s # always, when it happens
|
||||
failure-detector.implementation-class = akka.cluster.FailureDetectorPuppet
|
||||
|
|
|
|||
|
|
@ -29,7 +29,7 @@ object ClusterClientSpec extends MultiNodeConfig {
|
|||
akka.loglevel = INFO
|
||||
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
|
||||
akka.remote.log-remote-lifecycle-events = off
|
||||
akka.cluster.auto-down = on
|
||||
akka.cluster.auto-down-unreachable-after = 0s
|
||||
"""))
|
||||
|
||||
class TestService(testActor: ActorRef) extends Actor {
|
||||
|
|
|
|||
|
|
@ -40,7 +40,7 @@ object ClusterSingletonManagerChaosSpec extends MultiNodeConfig {
|
|||
akka.loglevel = INFO
|
||||
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
|
||||
akka.remote.log-remote-lifecycle-events = off
|
||||
akka.cluster.auto-down = on
|
||||
akka.cluster.auto-down-unreachable-after = 0s
|
||||
"""))
|
||||
|
||||
case object EchoStarted
|
||||
|
|
|
|||
|
|
@ -42,7 +42,7 @@ object ClusterSingletonManagerSpec extends MultiNodeConfig {
|
|||
akka.loglevel = INFO
|
||||
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
|
||||
akka.remote.log-remote-lifecycle-events = off
|
||||
akka.cluster.auto-down = on
|
||||
akka.cluster.auto-down-unreachable-after = 0s
|
||||
"""))
|
||||
|
||||
nodeConfig(first, second, third, fourth, fifth, sixth)(
|
||||
|
|
|
|||
|
|
@ -28,7 +28,7 @@ object DistributedPubSubMediatorSpec extends MultiNodeConfig {
|
|||
akka.loglevel = INFO
|
||||
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
|
||||
akka.remote.log-remote-lifecycle-events = off
|
||||
akka.cluster.auto-down = on
|
||||
akka.cluster.auto-down-unreachable-after = 0s
|
||||
"""))
|
||||
|
||||
object TestChatUser {
|
||||
|
|
|
|||
|
|
@ -159,7 +159,8 @@ state with gossip convergence.
|
|||
|
||||
The ``leader`` also has the power, if configured so, to "auto-down" a node that
|
||||
according to the `Failure Detector`_ is considered ``unreachable``. This means setting
|
||||
the ``unreachable`` node status to ``down`` automatically.
|
||||
the ``unreachable`` node status to ``down`` automatically after a configured time
|
||||
of unreachability.
|
||||
|
||||
|
||||
Seed Nodes
|
||||
|
|
|
|||
|
|
@ -150,12 +150,21 @@ It can also be performed programatically with ``Cluster.get(system).down(address
|
|||
|
||||
You can enable automatic downing with configuration::
|
||||
|
||||
akka.cluster.auto-down = on
|
||||
akka.cluster.auto-down-unreachable-after = 120s
|
||||
|
||||
This means that the cluster leader member will change the ``unreachable`` node
|
||||
status to ``down`` automatically after the configured time of unreachability.
|
||||
|
||||
Be aware of that using auto-down implies that two separate clusters will
|
||||
automatically be formed in case of network partition. That might be
|
||||
desired by some applications but not by others.
|
||||
|
||||
.. note:: If you have *auto-down* enabled and the failure detector triggers, you
|
||||
can over time end up with a lot of single node clusters if you don't put
|
||||
measures in place to shut down nodes that have become ``unreachable``. This
|
||||
follows from the fact that the ``unreachable`` node will likely see the rest of
|
||||
the cluster as ``unreachable``, become its own leader and form its own cluster.
|
||||
|
||||
Leaving
|
||||
^^^^^^^
|
||||
|
||||
|
|
|
|||
|
|
@ -4,381 +4,5 @@
|
|||
Migration Guide 2.1.x to 2.2.x
|
||||
################################
|
||||
|
||||
The 2.2 release contains several structural changes that require some
|
||||
simple, mechanical source-level changes in client code.
|
||||
|
||||
When migrating from 1.3.x to 2.1.x you should first follow the instructions for
|
||||
migrating :ref:`1.3.x to 2.0.x <migration-2.0>` and then :ref:`2.0.x to 2.1.x <migration-2.1>`.
|
||||
|
||||
Deprecated Closure-Taking Props
|
||||
===============================
|
||||
|
||||
:class:`Props` instances used to contain a closure which produces an
|
||||
:class:`Actor` instance when invoked. This approach is flawed in that closures
|
||||
are usually created in-line and thus carry a reference to their enclosing
|
||||
object; this is not well known among programmers, in particular it can be
|
||||
surprising that innocent-looking actor creation should not be serializable,
|
||||
e.g. if the enclosing class is an actor. Another issue which came up often
|
||||
during reviews is that these actor creators inadvertedly close over the Actor’s
|
||||
``this`` reference for calling methods on it, which is inherently unsafe.
|
||||
|
||||
Another reason for changing the underlying implementation is that Props now
|
||||
carries information about which class of actor will be created, allowing the
|
||||
extraction of mailbox type requirements (e.g. when using the Stash) before
|
||||
trying to create the actor. Being based on the actor class and a list of
|
||||
constructor arguments also allows these arguments to be serialized according to
|
||||
the configured serializer bindings instead of mandating Java serialization
|
||||
(which was used previously).
|
||||
|
||||
What changes for Java?
|
||||
----------------------
|
||||
|
||||
A new method ``Props.create`` has been introduced with two overloads::
|
||||
|
||||
Props.create(MyActor.class, arg1, arg2, ...);
|
||||
// or
|
||||
Props.create(new MyActorCreator(args ...));
|
||||
|
||||
In the first case the existence of a constructor signature matching the
|
||||
supplied arguments is verified at Props construction time. In the second case
|
||||
it is verified that ``MyActorCreator`` (which must be a ``akka.japi.Creator<?
|
||||
extends Actor>``) is a static class. In both cases failure is signaled by
|
||||
throwing a :class:`IllegalArgumentException`.
|
||||
|
||||
The constructors of :class:`Props` have been deprecated to facilitate migration.
|
||||
|
||||
The :meth:`withCreator` methods have been deprecated. The functionality is
|
||||
available by using ``Props.create(...).withDeploy(oldProps.deploy());``.
|
||||
|
||||
:class:`UntypedActorFactory` has been deprecated in favor of the more precisely
|
||||
typed :class:`Creator<T>`.
|
||||
|
||||
What changes for Scala?
|
||||
-----------------------
|
||||
|
||||
The case class signature of Props has been changed to only contain a
|
||||
:class:`Deploy`, a :class:`Class[_]` and an immutable :class:`Seq[Any]` (the
|
||||
constructor arguments for the class). The old factory and extractor methods
|
||||
have been deprecated.
|
||||
|
||||
Properly serializable :class:`Props` can now be created for actors which take
|
||||
constructor arguments by using ``Props(classOf[MyActor], arg1, arg2, ...)``.
|
||||
In a future update—possibly within the 2.2.x timeframe—we plan to introduce a
|
||||
macro which will transform the by-name argument to ``Props(new MyActor(...))``
|
||||
into a call to the former.
|
||||
|
||||
The :meth:`withCreator` methods have been deprecated. The functionality is
|
||||
available by using ``Props(...).withDeploy(oldProps.deploy)``.
|
||||
|
||||
Immutable everywhere
|
||||
====================
|
||||
|
||||
Akka has in 2.2 been refactored to require ``scala.collection.immutable`` data structures as much as possible,
|
||||
this leads to fewer bugs and more opportunity for sharing data safely.
|
||||
|
||||
==================================== ====================================
|
||||
Search Replace with
|
||||
==================================== ====================================
|
||||
``akka.japi.Util.arrayToSeq`` ``akka.japi.Util.immutableSeq``
|
||||
==================================== ====================================
|
||||
|
||||
If you need to convert from Java to ``scala.collection.immutable.Seq`` or ``scala.collection.immutable.Iterable`` you should use ``akka.japi.Util.immutableSeq(…)``,
|
||||
and if you need to convert from Scala you can simply switch to using immutable collections yourself or use the ``to[immutable.<collection-type>]`` method.
|
||||
|
||||
ActorContext & ActorRefFactory Dispatcher
|
||||
=========================================
|
||||
|
||||
The return type of ``ActorContext``'s and ``ActorRefFactory``'s ``dispatcher``-method now returns ``ExecutionContext`` instead of ``MessageDispatcher``.
|
||||
|
||||
Removed Fallback to Default Dispatcher
|
||||
======================================
|
||||
|
||||
If deploying an actor with a specific dispatcher, e.g.
|
||||
``Props(...).withDispatcher("d")``, then it would previously fall back to
|
||||
``akka.actor.default-dispatcher`` if no configuration section for ``d`` could
|
||||
be found.
|
||||
|
||||
This was beneficial for preparing later deployment choices during development
|
||||
by grouping actors on dispatcher IDs but not immediately configuring those.
|
||||
Akka 2.2 introduces the possibility to add dispatcher configuration to the
|
||||
``akka.actor.deployment`` section, making this unnecessary.
|
||||
|
||||
The fallback was removed because in many cases its application was neither
|
||||
intended nor noticed.
|
||||
|
||||
Changed Configuration Section for Dispatcher & Mailbox
|
||||
======================================================
|
||||
|
||||
The mailbox configuration defaults moved from ``akka.actor.default-dispatcher``
|
||||
to ``akka.actor.default-mailbox``. You will not have to change anything unless
|
||||
your configuration overrides a setting in the default dispatcher section.
|
||||
|
||||
The ``mailbox-type`` now requires a fully-qualified class name for the mailbox
|
||||
to use. The special words ``bounded`` and ``unbounded`` are retained for a
|
||||
migration period throughout the 2.2 series.
|
||||
|
||||
API changes to FSM and TestFSMRef
|
||||
=================================
|
||||
|
||||
The ``timerActive_?`` method has been deprecated in both the ``FSM`` trait and the ``TestFSMRef``
|
||||
class. You should now use the ``isTimerActive`` method instead. The old method will remain
|
||||
throughout 2.2.x. It will be removed in Akka 2.3.
|
||||
|
||||
|
||||
ThreadPoolConfigBuilder
|
||||
=======================
|
||||
|
||||
``akka.dispatch.ThreadPoolConfigBuilder`` companion object has been removed,
|
||||
and with it the ``conf_?`` method that was essentially only a type-inferencer aid for creation
|
||||
of optional transformations on ``ThreadPoolConfigBuilder``.
|
||||
Instead use: ``option.map(o => (t: ThreadPoolConfigBuilder) => t.op(o))``.
|
||||
|
||||
Scheduler
|
||||
=========
|
||||
|
||||
Akka's ``Scheduler`` has been augmented to also include a ``sender`` when scheduling to send messages, this should work Out-Of-The-Box for Scala users,
|
||||
but for Java Users you will need to manually provide the ``sender`` – as usual use ``null`` to designate "no sender" which will behave just as before the change.
|
||||
|
||||
ZeroMQ ByteString
|
||||
=================
|
||||
|
||||
``akka.zeromq.Frame`` and the use of ``Seq[Byte]`` in the API has been removed and is replaced by ``akka.util.ByteString``.
|
||||
|
||||
``ZMQMessage.firstFrameAsString`` has been removed, please use ``ZMQMessage.frames`` or ``ZMQMessage.frame(int)`` to access the frames.
|
||||
|
||||
Brand new Agents
|
||||
================
|
||||
|
||||
Akka's ``Agent`` has been rewritten to improve the API and to remove the need to manually ``close`` an Agent.
|
||||
It's also now an abstract class with the potential for subtyping and has a new factory method
|
||||
allowing Java to correctly infer the type of the Agent.
|
||||
The Java API has also been harmonized so both Java and Scala call the same methods.
|
||||
|
||||
======================================================= =======================================================
|
||||
Old Java API New Java API
|
||||
======================================================= =======================================================
|
||||
``new Agent<type>(value, actorSystem)`` ``Agent.create(value, executionContext)``
|
||||
``agent.update(newValue)`` ``agent.send(newValue)``
|
||||
``agent.future(Timeout)`` ``agent.future()``
|
||||
``agent.await(Timeout)`` ``Await.result(agent.future(), Timeout)``
|
||||
``agent.send(Function)`` ``agent.send(Mapper)``
|
||||
``agent.sendOff(Function, ExecutionContext)`` ``agent.sendOff(Mapper, ExecutionContext)``
|
||||
``agent.alter(Function, Timeout)`` ``agent.alter(Mapper)``
|
||||
``agent.alterOff(Function, Timeout, ExecutionContext)`` ``agent.alter(Mapper, ExecutionContext)``
|
||||
``agent.map(Function)`` ``agent.map(Mapper)``
|
||||
``agent.flatMap(Function)`` ``agent.flatMap(Mapper)``
|
||||
``agent.foreach(Procedure)`` ``agent.foreach(Foreach)``
|
||||
``agent.suspend()`` ``No replacement, pointless feature``
|
||||
``agent.resume()`` ``No replacement, pointless feature``
|
||||
``agent.close()`` ``No replacement, not needed in new implementation``
|
||||
======================================================= =======================================================
|
||||
|
||||
|
||||
======================================================== ========================================================
|
||||
Old Scala API New Scala API
|
||||
======================================================== ========================================================
|
||||
``Agent[T](value)(implicit ActorSystem)`` ``Agent[T](value)(implicit ExecutionContext)``
|
||||
``agent.update(newValue)`` ``agent.send(newValue)``
|
||||
``agent.alterOff(Function1)(Timeout, ExecutionContext)`` ``agent.alterOff(Function1)(ExecutionContext)``
|
||||
``agent.await(Timeout)`` ``Await.result(agent.future, Timeout)``
|
||||
``agent.future(Timeout)`` ``agent.future``
|
||||
``agent.suspend()`` ``No replacement, pointless feature``
|
||||
``agent.resume()`` ``No replacement, pointless feature``
|
||||
``agent.close()`` ``No replacement, not needed in new implementation``
|
||||
======================================================== ========================================================
|
||||
|
||||
|
||||
``event-handlers`` renamed to ``loggers``
|
||||
=========================================
|
||||
|
||||
If you have defined custom event handlers (loggers) in your configuration you need to change
|
||||
``akka.event-handlers`` to ``akka.loggers`` and
|
||||
``akka.event-handler-startup-timeout`` to ``akka.logger-startup-timeout``.
|
||||
|
||||
The SLF4J logger has been renamed from ``akka.event.slf4j.Slf4jEventHandler`` to
|
||||
``akka.event.slf4j.Slf4jLogger``.
|
||||
|
||||
The ``java.util.logging`` logger has been renamed from ``akka.contrib.jul.JavaLoggingEventHandler`` to
|
||||
``akka.contrib.jul.JavaLogger``.
|
||||
|
||||
Remoting
|
||||
========
|
||||
|
||||
The remoting subsystem of Akka has been replaced in favor of a more flexible, pluggable driver based implementation. This
|
||||
has required some changes to the configuration sections of ``akka.remote``, the format of Akka remote addresses
|
||||
and the Akka protocol itself.
|
||||
|
||||
The internal communication protocol of Akka has been evolved into a completely standalone entity, not tied to any
|
||||
particular transport. This change has the effect that Akka 2.2 remoting is no longer able to directly communicate with
|
||||
older versions.
|
||||
|
||||
The ``akka.remote.transport`` configuration key has been removed as the remoting system itself is no longer replaceable.
|
||||
Custom transports are now pluggable via the ``akka.remote.enabled-transpotrs`` key (see the :meth:`akka.remote.Transport` SPI
|
||||
and the documentation of remoting for more detail on drivers). The transport loaded by default is a Netty based TCP
|
||||
driver similar in functionality to the default remoting in Akka 2.1.
|
||||
|
||||
Transports are now fully pluggable through drivers, therefore transport specific settings like listening ports now live in the namespace
|
||||
of their driver configuration. In particular TCP related settings are now under ``akka.remote.netty.tcp``.
|
||||
|
||||
As a result of being able to replace the transport protocol, it is now necessary to include the protocol information
|
||||
in Akka URLs for remote addresses. Therefore a remote address of ``akka://remote-sys@remotehost:2552/user/actor``
|
||||
has to be changed to ``akka.tcp://remote-sys@remotehost:2552/user/actor`` if the remote system uses TCP as transport. If
|
||||
the other system uses SSL on top of TCP, the correct address would be ``akka.ssl.tcp://remote-sys@remotehost:2552/user/actor``.
|
||||
|
||||
Remote lifecycle events have been changed to a more coarse-grained, simplified model. All remoting events are subclasses
|
||||
of :meth:`akka.remote.RemotingLifecycle`. Events related to the lifecycle of *associations* (formerly called *connections*)
|
||||
be it inbound or outbound are subclasses of :meth:`akka.remote.AssociationEvent` (which is in turn a subclass of
|
||||
:meth:`RemotingLifecycle`). The direction of the association (inbound or outbound) triggering an ``AssociationEvent`` is
|
||||
available via the ``inbound`` boolean field of the event.
|
||||
|
||||
.. note::
|
||||
The change in terminology from "Connection" to "Association" reflects the fact that the remoting subsystem may use
|
||||
connectionless transports, but an association similar to transport layer connections is maintained between endpoints
|
||||
by the Akka protocol.
|
||||
|
||||
New configuration settings are also available, see the remoting documentation for more detail: :ref:`remoting-scala`
|
||||
|
||||
.. _migration_2.2_actorSelection:
|
||||
|
||||
Use ``actorSelection`` instead of ``actorFor``
|
||||
==============================================
|
||||
|
||||
``actorFor`` is deprecated in favor of ``actorSelection`` because actor references
|
||||
acquired with ``actorFor`` behave differently for local and remote actors.
|
||||
In the case of a local actor reference, the named actor needs to exist before the
|
||||
lookup, or else the acquired reference will be an :class:`EmptyLocalActorRef`.
|
||||
This will be true even if an actor with that exact path is created after acquiring
|
||||
the actor reference. For remote actor references acquired with `actorFor` the
|
||||
behaviour is different and sending messages to such a reference will under the hood
|
||||
look up the actor by path on the remote system for every message send.
|
||||
|
||||
Messages can be sent via the :class:`ActorSelection` and the path of the
|
||||
:class:`ActorSelection` is looked up when delivering each message. If the selection
|
||||
does not match any actors the message will be dropped.
|
||||
|
||||
To acquire an :class:`ActorRef` for an :class:`ActorSelection` you need to
|
||||
send a message to the selection and use the ``sender`` reference of the reply from
|
||||
the actor. There is a built-in ``Identify`` message that all Actors will understand
|
||||
and automatically reply to with a ``ActorIdentity`` message containing the
|
||||
:class:`ActorRef`.
|
||||
|
||||
You can also acquire an :class:`ActorRef` for an :class:`ActorSelection` with
|
||||
the ``resolveOne`` method of the :class:`ActorSelection`. It returns a ``Future``
|
||||
of the matching :class:`ActorRef` if such an actor exists. It is completed with
|
||||
failure [[akka.actor.ActorNotFound]] if no such actor exists or the identification
|
||||
didn't complete within the supplied `timeout`.
|
||||
|
||||
Read more about ``actorSelection`` in :ref:`docs for Java <actorSelection-java>` or
|
||||
:ref:`docs for Scala <actorSelection-scala>`.
|
||||
|
||||
ActorRef equality and sending to remote actors
|
||||
==============================================
|
||||
|
||||
Sending messages to an ``ActorRef`` must have the same semantics no matter if the target actor is located
|
||||
on a remote host or in the same ``ActorSystem`` in the same JVM. This was not always the case. For example
|
||||
when the target actor is terminated and created again under the same path. Sending to local references
|
||||
of the previous incarnation of the actor will not be delivered to the new incarnation, but that was the case
|
||||
for remote references. The reason was that the target actor was looked up by its path on every message
|
||||
delivery and the path didn't distinguish between the two incarnations of the actor. This has been fixed, and
|
||||
messages sent to a remote reference that points to a terminated actor will not be delivered to a new
|
||||
actor with the same path.
|
||||
|
||||
Equality of ``ActorRef`` has been changed to match the intention that an ``ActorRef`` corresponds to the target
|
||||
actor instance. Two actor references are compared equal when they have the same path and point to the same
|
||||
actor incarnation. A reference pointing to a terminated actor does not compare equal to a reference pointing
|
||||
to another (re-created) actor with the same path. Note that a restart of an actor caused by a failure still
|
||||
means that it's the same actor incarnation, i.e. a restart is not visible for the consumer of the ``ActorRef``.
|
||||
|
||||
Equality in 2.1 was only based on the path of the ``ActorRef``. If you need to keep track of actor references
|
||||
in a collection and do not care about the exact actor incarnation you can use the ``ActorPath`` as key, because
|
||||
the identifier of the target actor is not taken into account when comparing actor paths.
|
||||
|
||||
Remote actor references acquired with ``actorFor`` do not include the full information about the underlying actor
|
||||
identity and therefore such references do not compare equal to references acquired with ``actorOf``,
|
||||
``sender``, or ``context.self``. Because of this ``actorFor`` is deprecated, as explained in
|
||||
:ref:`migration_2.2_actorSelection`.
|
||||
|
||||
Note that when a parent actor is restarted its children are by default stopped and re-created, i.e. the child
|
||||
after the restart will be a different incarnation than the child before the restart. This has always been the
|
||||
case, but in some situations you might not have noticed, e.g. when comparing such actor references or sending
|
||||
messages to remote deployed children of a restarted parent.
|
||||
|
||||
This may also have implications if you compare the ``ActorRef`` received in a ``Terminated`` message
|
||||
with an expected ``ActorRef``.
|
||||
|
||||
The following will not match::
|
||||
|
||||
val ref = context.actorFor("akka.tcp://actorSystemName@10.0.0.1:2552/user/actorName")
|
||||
|
||||
def receive = {
|
||||
case Terminated(`ref`) => // ...
|
||||
}
|
||||
|
||||
Instead, use actorSelection followed by identify request, and watch the verified actor reference::
|
||||
|
||||
val selection = context.actorSelection(
|
||||
"akka.tcp://actorSystemName@10.0.0.1:2552/user/actorName")
|
||||
selection ! Identify(None)
|
||||
var ref: ActorRef = _
|
||||
|
||||
def receive = {
|
||||
case ActorIdentity(_, Some(actorRef)) =>
|
||||
ref = actorRef
|
||||
context watch ref
|
||||
case ActorIdentity(_, None) => // not alive
|
||||
case Terminated(r) if r == ref => // ...
|
||||
}
|
||||
|
||||
Use ``watch`` instead of ``isTerminated``
|
||||
=========================================
|
||||
|
||||
``ActorRef.isTerminated`` is deprecated in favor of ``ActorContext.watch`` because
|
||||
``isTerminated`` behaves differently for local and remote actors.
|
||||
|
||||
DeathWatch Semantics are Simplified
|
||||
===================================
|
||||
|
||||
DeathPactException is now Fatal
|
||||
-------------------------------
|
||||
|
||||
Previously an unhandled :class:`Terminated` message which led to a
|
||||
:class:`DeathPactException` to the thrown would be answered with a ``Restart``
|
||||
directive by the default supervisor strategy. This is not intuitive given the
|
||||
name of the exception and the Erlang linking feature by which it was inspired.
|
||||
The default strategy has thus be changed to return ``Stop`` in this case.
|
||||
|
||||
It can be argued that previously the actor would likely run into a restart loop
|
||||
because watching a terminated actor would lead to a :class:`DeathPactException`
|
||||
immediately again.
|
||||
|
||||
Unwatching now Prevents Reception of Terminated
|
||||
-----------------------------------------------
|
||||
|
||||
Previously calling :meth:`ActorContext.unwatch` would unregister lifecycle
|
||||
monitoring interest, but if the target actor had terminated already the
|
||||
:class:`Terminated` message had already been enqueued and would be received
|
||||
later—possibly leading to a :class:`DeathPactException`. This behavior has been
|
||||
modified such that the :class:`Terminated` message will be silently discarded
|
||||
if :meth:`unwatch` is called before processing the :class:`Terminated`
|
||||
message. Therefore the following is now safe::
|
||||
|
||||
context.stop(target)
|
||||
context.unwatch(target)
|
||||
|
||||
Dispatcher and Mailbox Implementation Changes
|
||||
=============================================
|
||||
|
||||
This point is only relevant if you have implemented a custom mailbox or
|
||||
dispatcher and want to migrate that to Akka 2.2. The constructor signature of
|
||||
:class:`MessageDispatcher` has changed, it now takes a
|
||||
:class:`MessageDispatcherConfigurator` instead of
|
||||
:class:`DispatcherPrerequisites`. Its :class:`createMailbox` method now
|
||||
receives one more argument of type :class:`MailboxType`, which is the mailbox
|
||||
type determined by the :class:`ActorRefProvider` for the actor based on its
|
||||
deployment. The :class:`DispatcherPrerequisites` now include a
|
||||
:class:`Mailboxes` instance which can be used for resolving mailbox references.
|
||||
The constructor signatures of the built-in dispatcher implementation have been
|
||||
adapted accordingly. The traits describing mailbox semantics have been
|
||||
separated from the implementation traits.
|
||||
|
||||
|
||||
Migration from 2.1.x to 2.2.x is described in the
|
||||
`documentation of 2.2 <http://doc.akka.io/docs/akka/2.2.1/project/migration-guide-2.0.x-2.1.x.html>`_.
|
||||
|
|
@ -19,3 +19,14 @@ in a graceful leaving scenario has been removed. Valuable state should be persis
|
|||
in durable storage instead, e.g. using akka-persistence. The constructor/props parameters
|
||||
of ``ClusterSingletonManager`` has been changed to ordinary ``Props`` parameter for the
|
||||
singleton actor instead of the factory parameter.
|
||||
|
||||
Changed cluster auto-down configuration
|
||||
=======================================
|
||||
|
||||
``akka.cluster.auto-down`` setting has been replaced by ``akka.cluster.auto-down-unreachable-after``,
|
||||
which instructs the cluster to automatically mark unreachable nodes as DOWN after this
|
||||
configured time of unreachability. This feature is disabled by default, as it also was in 2.2.x.
|
||||
|
||||
During the deprecation phase ``akka.cluster.auto-down=on`` is interpreted at as instant auto-down.
|
||||
|
||||
|
||||
|
|
|
|||
|
|
@ -9,3 +9,4 @@ Migration Guides
|
|||
migration-guide-1.3.x-2.0.x
|
||||
migration-guide-2.0.x-2.1.x
|
||||
migration-guide-2.1.x-2.2.x
|
||||
migration-guide-2.2.x-2.3.x
|
||||
|
|
|
|||
|
|
@ -143,12 +143,21 @@ It can also be performed programatically with ``Cluster(system).down(address)``.
|
|||
|
||||
You can enable automatic downing with configuration::
|
||||
|
||||
akka.cluster.auto-down = on
|
||||
akka.cluster.auto-down-unreachable-after = 120s
|
||||
|
||||
This means that the cluster leader member will change the ``unreachable`` node
|
||||
status to ``down`` automatically after the configured time of unreachability.
|
||||
|
||||
Be aware of that using auto-down implies that two separate clusters will
|
||||
automatically be formed in case of network partition. That might be
|
||||
desired by some applications but not by others.
|
||||
|
||||
.. note:: If you have *auto-down* enabled and the failure detector triggers, you
|
||||
can over time end up with a lot of single node clusters if you don't put
|
||||
measures in place to shut down nodes that have become ``unreachable``. This
|
||||
follows from the fact that the ``unreachable`` node will likely see the rest of
|
||||
the cluster as ``unreachable``, become its own leader and form its own cluster.
|
||||
|
||||
Leaving
|
||||
^^^^^^^
|
||||
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@ akka {
|
|||
"akka.tcp://ClusterSystem@127.0.0.1:2551",
|
||||
"akka.tcp://ClusterSystem@127.0.0.1:2552"]
|
||||
|
||||
auto-down = on
|
||||
auto-down-unreachable-after = 10s
|
||||
}
|
||||
}
|
||||
# //#cluster
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue