2011-10-26 08:48:16 +02:00
|
|
|
|
/**
|
2017-01-04 17:37:10 +01:00
|
|
|
|
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
|
2011-10-26 08:48:16 +02:00
|
|
|
|
*/
|
|
|
|
|
|
|
2012-01-31 13:33:04 +01:00
|
|
|
|
package akka.cluster
|
2011-10-26 08:48:16 +02:00
|
|
|
|
|
2015-04-17 17:28:37 +08:00
|
|
|
|
import java.io.Closeable
|
|
|
|
|
|
import java.util.concurrent.ThreadFactory
|
|
|
|
|
|
import java.util.concurrent.atomic.AtomicBoolean
|
|
|
|
|
|
|
2012-06-16 00:00:19 +02:00
|
|
|
|
import akka.ConfigurationException
|
2015-04-17 17:28:37 +08:00
|
|
|
|
import akka.actor._
|
2012-06-29 16:40:36 +02:00
|
|
|
|
import akka.dispatch.MonitorableThreadFactory
|
2015-04-17 17:28:37 +08:00
|
|
|
|
import akka.event.{ Logging, LoggingAdapter }
|
2015-07-01 10:38:13 +02:00
|
|
|
|
import akka.japi.Util
|
2012-06-21 10:58:35 +02:00
|
|
|
|
import akka.pattern._
|
2015-04-17 17:28:37 +08:00
|
|
|
|
import akka.remote.{ DefaultFailureDetectorRegistry, FailureDetector, _ }
|
|
|
|
|
|
import com.typesafe.config.{ Config, ConfigFactory }
|
|
|
|
|
|
|
|
|
|
|
|
import scala.annotation.varargs
|
2012-11-15 12:33:11 +01:00
|
|
|
|
import scala.collection.immutable
|
2015-04-17 17:28:37 +08:00
|
|
|
|
import scala.concurrent.duration._
|
|
|
|
|
|
import scala.concurrent.{ Await, ExecutionContext }
|
2013-03-19 17:36:36 +01:00
|
|
|
|
import scala.util.control.NonFatal
|
2012-01-24 12:09:32 +01:00
|
|
|
|
|
2012-02-22 18:40:16 +01:00
|
|
|
|
/**
|
2012-03-22 23:04:04 +01:00
|
|
|
|
* Cluster Extension Id and factory for creating Cluster extension.
|
2012-02-22 18:40:16 +01:00
|
|
|
|
*/
|
2012-03-22 23:04:04 +01:00
|
|
|
|
object Cluster extends ExtensionId[Cluster] with ExtensionIdProvider {
|
|
|
|
|
|
override def get(system: ActorSystem): Cluster = super.get(system)
|
2012-02-22 18:40:16 +01:00
|
|
|
|
|
2012-03-22 23:04:04 +01:00
|
|
|
|
override def lookup = Cluster
|
2012-02-22 18:40:16 +01:00
|
|
|
|
|
2012-09-06 21:48:40 +02:00
|
|
|
|
override def createExtension(system: ExtendedActorSystem): Cluster = new Cluster(system)
|
2013-09-11 13:38:56 +02:00
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
|
*/
|
|
|
|
|
|
private[cluster] final val isAssertInvariantsEnabled: Boolean =
|
|
|
|
|
|
System.getProperty("akka.cluster.assert", "off").toLowerCase match {
|
|
|
|
|
|
case "on" | "true" ⇒ true
|
|
|
|
|
|
case _ ⇒ false
|
|
|
|
|
|
}
|
2012-02-22 18:40:16 +01:00
|
|
|
|
}
|
2012-02-14 20:50:12 +01:00
|
|
|
|
|
2011-10-26 08:48:16 +02:00
|
|
|
|
/**
|
2013-07-08 13:42:44 +02:00
|
|
|
|
* This module is responsible cluster membership information. Changes to the cluster
|
|
|
|
|
|
* information is retrieved through [[#subscribe]]. Commands to operate the cluster is
|
|
|
|
|
|
* available through methods in this class, such as [[#join]], [[#down]] and [[#leave]].
|
|
|
|
|
|
*
|
|
|
|
|
|
* Each cluster [[Member]] is identified by its [[akka.actor.Address]], and
|
|
|
|
|
|
* the cluster address of this actor system is [[#selfAddress]]. A member also has a status;
|
2015-05-15 16:53:24 +02:00
|
|
|
|
* initially [[MemberStatus]] `Joining` followed by [[MemberStatus]] `Up`.
|
2011-10-26 08:48:16 +02:00
|
|
|
|
*/
|
2012-09-06 21:48:40 +02:00
|
|
|
|
class Cluster(val system: ExtendedActorSystem) extends Extension {
|
2012-02-22 18:40:16 +01:00
|
|
|
|
|
2012-08-14 10:58:30 +02:00
|
|
|
|
import ClusterEvent._
|
2011-10-26 08:48:16 +02:00
|
|
|
|
|
2012-07-04 11:37:56 +02:00
|
|
|
|
val settings = new ClusterSettings(system.settings.config, system.name)
|
2013-05-23 13:36:35 +02:00
|
|
|
|
import InfoLogger._
|
2015-04-17 17:28:37 +08:00
|
|
|
|
import settings._
|
2012-02-07 16:53:49 +01:00
|
|
|
|
|
2013-04-11 09:18:12 +02:00
|
|
|
|
/**
|
2014-06-05 13:25:42 +02:00
|
|
|
|
* The address including a `uid` of this cluster member.
|
|
|
|
|
|
* The `uid` is needed to be able to distinguish different
|
|
|
|
|
|
* incarnations of a member with same hostname and port.
|
2013-04-11 09:18:12 +02:00
|
|
|
|
*/
|
2014-06-05 13:25:42 +02:00
|
|
|
|
val selfUniqueAddress: UniqueAddress = system.provider match {
|
2013-04-11 09:18:12 +02:00
|
|
|
|
case c: ClusterActorRefProvider ⇒
|
2016-09-26 15:34:59 +02:00
|
|
|
|
UniqueAddress(c.transport.defaultAddress, AddressUidExtension(system).longAddressUid)
|
2012-09-11 19:11:20 +02:00
|
|
|
|
case other ⇒ throw new ConfigurationException(
|
2013-04-11 09:18:12 +02:00
|
|
|
|
s"ActorSystem [${system}] needs to have a 'ClusterActorRefProvider' enabled in the configuration, currently uses [${other.getClass.getName}]")
|
2012-09-11 19:11:20 +02:00
|
|
|
|
}
|
2012-01-31 15:00:46 +01:00
|
|
|
|
|
2013-04-11 09:18:12 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* The address of this cluster member.
|
|
|
|
|
|
*/
|
|
|
|
|
|
def selfAddress: Address = selfUniqueAddress.address
|
|
|
|
|
|
|
2013-03-14 20:32:43 +01:00
|
|
|
|
/**
|
|
|
|
|
|
* roles that this member has
|
|
|
|
|
|
*/
|
|
|
|
|
|
def selfRoles: Set[String] = settings.Roles
|
|
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* Java API: roles that this member has
|
|
|
|
|
|
*/
|
|
|
|
|
|
def getSelfRoles: java.util.Set[String] =
|
|
|
|
|
|
scala.collection.JavaConverters.setAsJavaSetConverter(selfRoles).asJava
|
|
|
|
|
|
|
2012-12-06 15:26:57 +01:00
|
|
|
|
private val _isTerminated = new AtomicBoolean(false)
|
2014-11-17 17:17:52 -06:00
|
|
|
|
private val log = Logging(system, getClass.getName)
|
2013-03-19 17:36:36 +01:00
|
|
|
|
// ClusterJmx is initialized as the last thing in the constructor
|
|
|
|
|
|
private var clusterJmx: Option[ClusterJmx] = None
|
2012-04-14 20:06:03 +02:00
|
|
|
|
|
2013-05-23 13:36:35 +02:00
|
|
|
|
logInfo("Starting up...")
|
2012-03-09 12:56:56 +01:00
|
|
|
|
|
2013-01-29 11:55:33 +01:00
|
|
|
|
val failureDetector: FailureDetectorRegistry[Address] = {
|
2013-05-17 14:16:26 +02:00
|
|
|
|
def createFailureDetector(): FailureDetector =
|
|
|
|
|
|
FailureDetectorLoader.load(settings.FailureDetectorImplementationClass, settings.FailureDetectorConfig, system)
|
2013-01-29 11:55:33 +01:00
|
|
|
|
|
|
|
|
|
|
new DefaultFailureDetectorRegistry(() ⇒ createFailureDetector())
|
2012-09-06 21:48:40 +02:00
|
|
|
|
}
|
|
|
|
|
|
|
2016-04-11 10:33:02 +02:00
|
|
|
|
// needs to be lazy to allow downing provider impls to access Cluster (if not we get deadlock)
|
|
|
|
|
|
lazy val downingProvider: DowningProvider =
|
|
|
|
|
|
DowningProvider.load(settings.DowningProviderClassName, system)
|
|
|
|
|
|
|
2012-03-09 12:56:56 +01:00
|
|
|
|
// ========================================================
|
|
|
|
|
|
// ===================== WORK DAEMONS =====================
|
|
|
|
|
|
// ========================================================
|
|
|
|
|
|
|
|
|
|
|
|
/**
|
2012-07-04 11:37:56 +02:00
|
|
|
|
* INTERNAL API
|
2012-03-09 12:56:56 +01:00
|
|
|
|
*/
|
2013-02-13 15:37:22 +01:00
|
|
|
|
private[cluster] val scheduler: Scheduler = {
|
2013-01-14 23:21:51 +01:00
|
|
|
|
if (system.scheduler.maxFrequency < 1.second / SchedulerTickDuration) {
|
2016-06-02 14:06:57 +02:00
|
|
|
|
logInfo(
|
|
|
|
|
|
"Using a dedicated scheduler for cluster. Default scheduler can be used if configured " +
|
|
|
|
|
|
"with 'akka.scheduler.tick-duration' [{} ms] <= 'akka.cluster.scheduler.tick-duration' [{} ms].",
|
2013-01-24 13:43:40 +01:00
|
|
|
|
(1000 / system.scheduler.maxFrequency).toInt, SchedulerTickDuration.toMillis)
|
2013-02-13 15:37:22 +01:00
|
|
|
|
|
|
|
|
|
|
val cfg = ConfigFactory.parseString(
|
|
|
|
|
|
s"akka.scheduler.tick-duration=${SchedulerTickDuration.toMillis}ms").withFallback(
|
|
|
|
|
|
system.settings.config)
|
|
|
|
|
|
val threadFactory = system.threadFactory match {
|
|
|
|
|
|
case tf: MonitorableThreadFactory ⇒ tf.withName(tf.name + "-cluster-scheduler")
|
|
|
|
|
|
case tf ⇒ tf
|
|
|
|
|
|
}
|
|
|
|
|
|
system.dynamicAccess.createInstanceFor[Scheduler](system.settings.SchedulerClass, immutable.Seq(
|
2016-06-02 14:06:57 +02:00
|
|
|
|
classOf[Config] → cfg,
|
|
|
|
|
|
classOf[LoggingAdapter] → log,
|
|
|
|
|
|
classOf[ThreadFactory] → threadFactory)).get
|
2012-06-12 14:16:30 +02:00
|
|
|
|
} else {
|
2012-08-08 15:57:30 +02:00
|
|
|
|
// delegate to system.scheduler, but don't close over system
|
2012-06-12 14:16:30 +02:00
|
|
|
|
val systemScheduler = system.scheduler
|
|
|
|
|
|
new Scheduler with Closeable {
|
2012-08-08 15:57:30 +02:00
|
|
|
|
override def close(): Unit = () // we are using system.scheduler, which we are not responsible for closing
|
|
|
|
|
|
|
2013-01-14 23:21:51 +01:00
|
|
|
|
override def maxFrequency: Double = systemScheduler.maxFrequency
|
2012-08-08 15:57:30 +02:00
|
|
|
|
|
2012-09-18 18:17:44 +02:00
|
|
|
|
override def schedule(initialDelay: FiniteDuration, interval: FiniteDuration,
|
2012-08-08 15:57:30 +02:00
|
|
|
|
runnable: Runnable)(implicit executor: ExecutionContext): Cancellable =
|
2012-09-18 18:17:44 +02:00
|
|
|
|
systemScheduler.schedule(initialDelay, interval, runnable)
|
2012-08-08 15:57:30 +02:00
|
|
|
|
|
2016-06-02 14:06:57 +02:00
|
|
|
|
override def scheduleOnce(
|
|
|
|
|
|
delay: FiniteDuration,
|
|
|
|
|
|
runnable: Runnable)(implicit executor: ExecutionContext): Cancellable =
|
2012-06-12 14:16:30 +02:00
|
|
|
|
systemScheduler.scheduleOnce(delay, runnable)
|
|
|
|
|
|
}
|
|
|
|
|
|
}
|
2012-06-12 13:34:59 +02:00
|
|
|
|
}
|
2012-06-11 22:12:45 +02:00
|
|
|
|
|
2012-07-04 11:37:56 +02:00
|
|
|
|
// create supervisor for daemons under path "/system/cluster"
|
|
|
|
|
|
private val clusterDaemons: ActorRef = {
|
2014-03-03 12:00:25 +01:00
|
|
|
|
system.systemActorOf(Props(classOf[ClusterDaemon], settings).
|
2013-05-29 16:13:10 +02:00
|
|
|
|
withDispatcher(UseDispatcher).withDeploy(Deploy.local), name = "cluster")
|
2012-07-04 11:37:56 +02:00
|
|
|
|
}
|
2012-03-09 12:56:56 +01:00
|
|
|
|
|
2012-06-05 22:16:15 +02:00
|
|
|
|
/**
|
2012-07-04 11:37:56 +02:00
|
|
|
|
* INTERNAL API
|
2012-06-05 22:16:15 +02:00
|
|
|
|
*/
|
2012-07-05 07:56:40 +02:00
|
|
|
|
private[cluster] val clusterCore: ActorRef = {
|
|
|
|
|
|
implicit val timeout = system.settings.CreationTimeout
|
2013-03-19 17:36:36 +01:00
|
|
|
|
try {
|
|
|
|
|
|
Await.result((clusterDaemons ? InternalClusterAction.GetClusterCoreRef).mapTo[ActorRef], timeout.duration)
|
|
|
|
|
|
} catch {
|
|
|
|
|
|
case NonFatal(e) ⇒
|
2013-09-17 16:52:08 +02:00
|
|
|
|
log.error(e, "Failed to startup Cluster. You can try to increase 'akka.actor.creation-timeout'.")
|
2013-03-19 17:36:36 +01:00
|
|
|
|
shutdown()
|
2013-09-17 16:52:08 +02:00
|
|
|
|
// don't re-throw, that would cause the extension to be re-recreated
|
2015-07-01 14:39:20 +02:00
|
|
|
|
// from shutdown() or other places, which may result in
|
2013-09-17 16:52:08 +02:00
|
|
|
|
// InvalidActorNameException: actor name [cluster] is not unique
|
|
|
|
|
|
system.deadLetters
|
2013-03-19 17:36:36 +01:00
|
|
|
|
}
|
2012-07-05 07:56:40 +02:00
|
|
|
|
}
|
2012-04-14 20:06:03 +02:00
|
|
|
|
|
2014-01-08 14:14:48 +01:00
|
|
|
|
private[cluster] val readView: ClusterReadView = new ClusterReadView(this)
|
2012-08-15 16:47:34 +02:00
|
|
|
|
|
2012-06-11 22:12:45 +02:00
|
|
|
|
system.registerOnTermination(shutdown())
|
|
|
|
|
|
|
2013-03-19 17:36:36 +01:00
|
|
|
|
if (JmxEnabled)
|
|
|
|
|
|
clusterJmx = {
|
|
|
|
|
|
val jmx = new ClusterJmx(this, log)
|
|
|
|
|
|
jmx.createMBean()
|
|
|
|
|
|
Some(jmx)
|
|
|
|
|
|
}
|
2012-07-05 11:56:54 +02:00
|
|
|
|
|
2013-05-23 13:36:35 +02:00
|
|
|
|
logInfo("Started up successfully")
|
2012-03-09 12:56:56 +01:00
|
|
|
|
|
2012-02-14 20:50:12 +01:00
|
|
|
|
// ======================================================
|
|
|
|
|
|
// ===================== PUBLIC API =====================
|
|
|
|
|
|
// ======================================================
|
|
|
|
|
|
|
2012-06-16 00:00:19 +02:00
|
|
|
|
/**
|
2012-12-06 15:26:57 +01:00
|
|
|
|
* Returns true if this cluster instance has be shutdown.
|
2012-06-16 00:00:19 +02:00
|
|
|
|
*/
|
2012-12-06 15:26:57 +01:00
|
|
|
|
def isTerminated: Boolean = _isTerminated.get
|
2012-06-16 00:00:19 +02:00
|
|
|
|
|
2011-01-01 01:50:33 +01:00
|
|
|
|
/**
|
2014-01-08 14:14:48 +01:00
|
|
|
|
* Current snapshot state of the cluster.
|
|
|
|
|
|
*/
|
|
|
|
|
|
def state: CurrentClusterState = readView.state
|
|
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* Subscribe to one or more cluster domain events.
|
|
|
|
|
|
* The `to` classes can be [[akka.cluster.ClusterEvent.ClusterDomainEvent]]
|
|
|
|
|
|
* or subclasses.
|
2013-01-15 09:35:07 +01:00
|
|
|
|
*
|
|
|
|
|
|
* A snapshot of [[akka.cluster.ClusterEvent.CurrentClusterState]]
|
2014-01-08 14:14:48 +01:00
|
|
|
|
* will be sent to the subscriber as the first message.
|
|
|
|
|
|
*/
|
|
|
|
|
|
@varargs def subscribe(subscriber: ActorRef, to: Class[_]*): Unit =
|
2015-07-01 15:09:16 +02:00
|
|
|
|
subscribe(subscriber, initialStateMode = InitialStateAsSnapshot, to: _*)
|
2014-01-08 14:14:48 +01:00
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* Subscribe to one or more cluster domain events.
|
|
|
|
|
|
* The `to` classes can be [[akka.cluster.ClusterEvent.ClusterDomainEvent]]
|
|
|
|
|
|
* or subclasses.
|
|
|
|
|
|
*
|
2015-05-15 16:53:24 +02:00
|
|
|
|
* If `initialStateMode` is `ClusterEvent.InitialStateAsEvents` the events corresponding
|
2014-01-08 14:14:48 +01:00
|
|
|
|
* to the current state will be sent to the subscriber to mimic what you would
|
|
|
|
|
|
* have seen if you were listening to the events when they occurred in the past.
|
|
|
|
|
|
*
|
2015-05-15 16:53:24 +02:00
|
|
|
|
* If `initialStateMode` is `ClusterEvent.InitialStateAsSnapshot` a snapshot of
|
2014-01-08 14:14:48 +01:00
|
|
|
|
* [[akka.cluster.ClusterEvent.CurrentClusterState]] will be sent to the subscriber as the
|
|
|
|
|
|
* first message.
|
|
|
|
|
|
*
|
|
|
|
|
|
* Note that for large clusters it is more efficient to use `InitialStateAsSnapshot`.
|
2011-01-01 01:50:33 +01:00
|
|
|
|
*/
|
2015-07-01 15:09:16 +02:00
|
|
|
|
@varargs def subscribe(subscriber: ActorRef, initialStateMode: SubscriptionInitialStateMode, to: Class[_]*): Unit = {
|
|
|
|
|
|
require(to.length > 0, "at least one `ClusterDomainEvent` class is required")
|
2016-06-02 14:06:57 +02:00
|
|
|
|
require(
|
|
|
|
|
|
to.forall(classOf[ClusterDomainEvent].isAssignableFrom),
|
2015-07-01 15:09:16 +02:00
|
|
|
|
s"subscribe to `akka.cluster.ClusterEvent.ClusterDomainEvent` or subclasses, was [${to.map(_.getName).mkString(", ")}]")
|
2014-01-08 14:14:48 +01:00
|
|
|
|
clusterCore ! InternalClusterAction.Subscribe(subscriber, initialStateMode, to.toSet)
|
2015-07-01 15:09:16 +02:00
|
|
|
|
}
|
2012-08-14 15:33:34 +02:00
|
|
|
|
|
|
|
|
|
|
/**
|
2012-09-28 13:09:36 +02:00
|
|
|
|
* Unsubscribe to all cluster domain events.
|
2012-08-14 15:33:34 +02:00
|
|
|
|
*/
|
2012-08-15 16:47:34 +02:00
|
|
|
|
def unsubscribe(subscriber: ActorRef): Unit =
|
2012-09-28 13:09:36 +02:00
|
|
|
|
clusterCore ! InternalClusterAction.Unsubscribe(subscriber, None)
|
|
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* Unsubscribe to a specific type of cluster domain events,
|
|
|
|
|
|
* matching previous `subscribe` registration.
|
|
|
|
|
|
*/
|
|
|
|
|
|
def unsubscribe(subscriber: ActorRef, to: Class[_]): Unit =
|
|
|
|
|
|
clusterCore ! InternalClusterAction.Unsubscribe(subscriber, Some(to))
|
2011-01-01 01:50:33 +01:00
|
|
|
|
|
2012-09-12 09:23:02 +02:00
|
|
|
|
/**
|
2014-03-12 11:38:09 +01:00
|
|
|
|
* Send current (full) state of the cluster to the specified
|
2012-09-12 09:23:02 +02:00
|
|
|
|
* receiver. If you want this to happen periodically you need to schedule
|
2014-01-08 14:14:48 +01:00
|
|
|
|
* a call to this method yourself. Note that you can also retrieve the current
|
|
|
|
|
|
* state with [[#state]].
|
2012-09-12 09:23:02 +02:00
|
|
|
|
*/
|
|
|
|
|
|
def sendCurrentClusterState(receiver: ActorRef): Unit =
|
2014-03-12 11:38:09 +01:00
|
|
|
|
clusterCore ! InternalClusterAction.SendCurrentClusterState(receiver)
|
2012-09-12 09:23:02 +02:00
|
|
|
|
|
2012-02-29 10:02:00 +01:00
|
|
|
|
/**
|
2012-04-12 22:50:50 +02:00
|
|
|
|
* Try to join this cluster node with the node specified by 'address'.
|
2013-04-11 09:18:12 +02:00
|
|
|
|
* A 'Join(selfAddress)' command is sent to the node to join.
|
2013-06-24 11:46:29 +02:00
|
|
|
|
*
|
|
|
|
|
|
* An actor system can only join a cluster once. Additional attempts will be ignored.
|
|
|
|
|
|
* When it has successfully joined it must be restarted to be able to join another
|
|
|
|
|
|
* cluster or to join the same cluster again.
|
2016-04-04 12:37:12 +02:00
|
|
|
|
*
|
|
|
|
|
|
* The name of the [[akka.actor.ActorSystem]] must be the same for all members of a
|
|
|
|
|
|
* cluster.
|
2012-02-29 10:02:00 +01:00
|
|
|
|
*/
|
2012-07-04 11:37:56 +02:00
|
|
|
|
def join(address: Address): Unit =
|
2015-09-04 08:53:36 +02:00
|
|
|
|
clusterCore ! ClusterUserAction.JoinTo(fillLocal(address))
|
|
|
|
|
|
|
|
|
|
|
|
private def fillLocal(address: Address): Address = {
|
|
|
|
|
|
// local address might be used if grabbed from actorRef.path.address
|
|
|
|
|
|
if (address.hasLocalScope && address.system == selfAddress.system) selfAddress
|
|
|
|
|
|
else address
|
|
|
|
|
|
}
|
2012-02-29 10:02:00 +01:00
|
|
|
|
|
2013-06-24 11:46:29 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* Join the specified seed nodes without defining them in config.
|
|
|
|
|
|
* Especially useful from tests when Addresses are unknown before startup time.
|
|
|
|
|
|
*
|
|
|
|
|
|
* An actor system can only join a cluster once. Additional attempts will be ignored.
|
|
|
|
|
|
* When it has successfully joined it must be restarted to be able to join another
|
|
|
|
|
|
* cluster or to join the same cluster again.
|
|
|
|
|
|
*/
|
|
|
|
|
|
def joinSeedNodes(seedNodes: immutable.Seq[Address]): Unit =
|
2015-09-04 08:53:36 +02:00
|
|
|
|
clusterCore ! InternalClusterAction.JoinSeedNodes(seedNodes.toVector.map(fillLocal))
|
2013-06-24 11:46:29 +02:00
|
|
|
|
|
2015-07-01 10:38:13 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* Java API
|
|
|
|
|
|
*
|
|
|
|
|
|
* Join the specified seed nodes without defining them in config.
|
|
|
|
|
|
* Especially useful from tests when Addresses are unknown before startup time.
|
|
|
|
|
|
*
|
|
|
|
|
|
* An actor system can only join a cluster once. Additional attempts will be ignored.
|
|
|
|
|
|
* When it has successfully joined it must be restarted to be able to join another
|
|
|
|
|
|
* cluster or to join the same cluster again.
|
|
|
|
|
|
*/
|
|
|
|
|
|
def joinSeedNodes(seedNodes: java.util.List[Address]): Unit =
|
|
|
|
|
|
joinSeedNodes(Util.immutableSeq(seedNodes))
|
|
|
|
|
|
|
2012-02-29 10:02:00 +01:00
|
|
|
|
/**
|
2012-04-12 22:50:50 +02:00
|
|
|
|
* Send command to issue state transition to LEAVING for the node specified by 'address'.
|
2015-05-15 16:53:24 +02:00
|
|
|
|
* The member will go through the status changes [[MemberStatus]] `Leaving` (not published to
|
|
|
|
|
|
* subscribers) followed by [[MemberStatus]] `Exiting` and finally [[MemberStatus]] `Removed`.
|
2013-07-08 13:42:44 +02:00
|
|
|
|
*
|
|
|
|
|
|
* Note that this command can be issued to any member in the cluster, not necessarily the
|
|
|
|
|
|
* one that is leaving. The cluster extension, but not the actor system or JVM, of the
|
|
|
|
|
|
* leaving member will be shutdown after the leader has changed status of the member to
|
|
|
|
|
|
* Exiting. Thereafter the member will be removed from the cluster. Normally this is
|
|
|
|
|
|
* handled automatically, but in case of network failures during this process it might
|
|
|
|
|
|
* still be necessary to set the node’s status to Down in order to complete the removal.
|
2012-02-29 10:02:00 +01:00
|
|
|
|
*/
|
2012-07-04 11:37:56 +02:00
|
|
|
|
def leave(address: Address): Unit =
|
2015-09-04 08:53:36 +02:00
|
|
|
|
clusterCore ! ClusterUserAction.Leave(fillLocal(address))
|
2012-02-29 10:02:00 +01:00
|
|
|
|
|
|
|
|
|
|
/**
|
2012-06-08 11:51:34 +02:00
|
|
|
|
* Send command to DOWN the node specified by 'address'.
|
2013-07-08 13:42:44 +02:00
|
|
|
|
*
|
|
|
|
|
|
* When a member is considered by the failure detector to be unreachable the leader is not
|
|
|
|
|
|
* allowed to perform its duties, such as changing status of new joining members to 'Up'.
|
|
|
|
|
|
* The status of the unreachable member must be changed to 'Down', which can be done with
|
|
|
|
|
|
* this method.
|
2012-02-29 10:02:00 +01:00
|
|
|
|
*/
|
2012-07-04 11:37:56 +02:00
|
|
|
|
def down(address: Address): Unit =
|
2015-09-04 08:53:36 +02:00
|
|
|
|
clusterCore ! ClusterUserAction.Down(fillLocal(address))
|
2012-02-29 10:02:00 +01:00
|
|
|
|
|
2012-12-10 08:46:25 +01:00
|
|
|
|
/**
|
|
|
|
|
|
* The supplied thunk will be run, once, when current cluster member is `Up`.
|
2015-05-15 16:53:24 +02:00
|
|
|
|
* Typically used together with configuration option `akka.cluster.min-nr-of-members`
|
2012-12-10 08:46:25 +01:00
|
|
|
|
* to defer some action, such as starting actors, until the cluster has reached
|
|
|
|
|
|
* a certain size.
|
|
|
|
|
|
*/
|
|
|
|
|
|
def registerOnMemberUp[T](code: ⇒ T): Unit =
|
2015-04-17 17:28:37 +08:00
|
|
|
|
registerOnMemberUp(new Runnable { def run() = code })
|
2012-12-10 08:46:25 +01:00
|
|
|
|
|
|
|
|
|
|
/**
|
2013-03-07 09:05:55 +01:00
|
|
|
|
* Java API: The supplied callback will be run, once, when current cluster member is `Up`.
|
2015-05-15 16:53:24 +02:00
|
|
|
|
* Typically used together with configuration option `akka.cluster.min-nr-of-members`
|
2012-12-10 08:46:25 +01:00
|
|
|
|
* to defer some action, such as starting actors, until the cluster has reached
|
|
|
|
|
|
* a certain size.
|
|
|
|
|
|
*/
|
2015-04-17 17:28:37 +08:00
|
|
|
|
def registerOnMemberUp(callback: Runnable): Unit =
|
|
|
|
|
|
clusterDaemons ! InternalClusterAction.AddOnMemberUpListener(callback)
|
2015-02-11 22:16:27 +01:00
|
|
|
|
|
2015-04-17 17:28:37 +08:00
|
|
|
|
/**
|
|
|
|
|
|
* The supplied thunk will be run, once, when current cluster member is `Removed`.
|
2015-09-10 08:48:50 +02:00
|
|
|
|
* If the cluster has already been shutdown the thunk will run on the caller thread immediately.
|
2016-01-17 15:48:52 +01:00
|
|
|
|
* Typically used together `cluster.leave(cluster.selfAddress)` and then `system.terminate()`.
|
2015-04-17 17:28:37 +08:00
|
|
|
|
*/
|
|
|
|
|
|
def registerOnMemberRemoved[T](code: ⇒ T): Unit =
|
|
|
|
|
|
registerOnMemberRemoved(new Runnable { override def run(): Unit = code })
|
2012-12-10 08:46:25 +01:00
|
|
|
|
|
2015-04-17 17:28:37 +08:00
|
|
|
|
/**
|
|
|
|
|
|
* Java API: The supplied thunk will be run, once, when current cluster member is `Removed`.
|
2015-09-10 08:48:50 +02:00
|
|
|
|
* If the cluster has already been shutdown the thunk will run on the caller thread immediately.
|
2016-01-17 15:48:52 +01:00
|
|
|
|
* Typically used together `cluster.leave(cluster.selfAddress)` and then `system.terminate()`.
|
2015-04-17 17:28:37 +08:00
|
|
|
|
*/
|
|
|
|
|
|
def registerOnMemberRemoved(callback: Runnable): Unit = {
|
|
|
|
|
|
if (_isTerminated.get())
|
|
|
|
|
|
callback.run()
|
|
|
|
|
|
else
|
|
|
|
|
|
clusterDaemons ! InternalClusterAction.AddOnMemberRemovedListener(callback)
|
|
|
|
|
|
}
|
2015-12-15 22:31:50 +01:00
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* Generate the remote actor path by replacing the Address in the RootActor Path for the given
|
|
|
|
|
|
* ActorRef with the cluster's `selfAddress`, unless address' host is already defined
|
|
|
|
|
|
*/
|
|
|
|
|
|
def remotePathOf(actorRef: ActorRef): ActorPath = {
|
|
|
|
|
|
val path = actorRef.path
|
|
|
|
|
|
if (path.address.host.isDefined) {
|
|
|
|
|
|
path
|
|
|
|
|
|
} else {
|
|
|
|
|
|
path.root.copy(selfAddress) / path.elements withUid path.uid
|
|
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2011-01-01 01:50:33 +01:00
|
|
|
|
// ========================================================
|
|
|
|
|
|
// ===================== INTERNAL API =====================
|
|
|
|
|
|
// ========================================================
|
|
|
|
|
|
|
2012-02-08 14:14:01 +01:00
|
|
|
|
/**
|
2012-06-16 00:00:19 +02:00
|
|
|
|
* INTERNAL API.
|
|
|
|
|
|
*
|
|
|
|
|
|
* Shuts down all connections to other members, the cluster daemon and the periodic gossip and cleanup tasks.
|
|
|
|
|
|
*
|
|
|
|
|
|
* Should not called by the user. The user can issue a LEAVE command which will tell the node
|
2015-05-15 16:53:24 +02:00
|
|
|
|
* to go through graceful handoff process `LEAVE -> EXITING -> REMOVED -> SHUTDOWN`.
|
2012-06-16 00:00:19 +02:00
|
|
|
|
*/
|
|
|
|
|
|
private[cluster] def shutdown(): Unit = {
|
2012-12-06 15:26:57 +01:00
|
|
|
|
if (_isTerminated.compareAndSet(false, true)) {
|
2013-05-23 13:36:35 +02:00
|
|
|
|
logInfo("Shutting down...")
|
2012-06-16 00:00:19 +02:00
|
|
|
|
|
2012-08-15 16:47:34 +02:00
|
|
|
|
system.stop(clusterDaemons)
|
2015-07-01 14:39:20 +02:00
|
|
|
|
|
|
|
|
|
|
// readView might be null if init fails before it is created
|
|
|
|
|
|
if (readView != null)
|
|
|
|
|
|
readView.close()
|
2012-06-16 00:00:19 +02:00
|
|
|
|
|
2013-02-13 15:37:22 +01:00
|
|
|
|
closeScheduler()
|
2012-03-09 12:56:56 +01:00
|
|
|
|
|
2012-09-20 08:09:01 +02:00
|
|
|
|
clusterJmx foreach { _.unregisterMBean() }
|
2012-03-09 12:56:56 +01:00
|
|
|
|
|
2013-05-23 13:36:35 +02:00
|
|
|
|
logInfo("Successfully shut down")
|
2012-03-09 12:56:56 +01:00
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2013-02-13 15:37:22 +01:00
|
|
|
|
private def closeScheduler(): Unit = scheduler match {
|
|
|
|
|
|
case x: Closeable ⇒ x.close()
|
|
|
|
|
|
case _ ⇒
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2013-05-23 13:36:35 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
|
*/
|
|
|
|
|
|
private[cluster] object InfoLogger {
|
|
|
|
|
|
|
|
|
|
|
|
def logInfo(message: String): Unit =
|
|
|
|
|
|
if (LogInfo) log.info("Cluster Node [{}] - {}", selfAddress, message)
|
|
|
|
|
|
|
|
|
|
|
|
def logInfo(template: String, arg1: Any): Unit =
|
|
|
|
|
|
if (LogInfo) log.info("Cluster Node [{}] - " + template, selfAddress, arg1)
|
|
|
|
|
|
|
|
|
|
|
|
def logInfo(template: String, arg1: Any, arg2: Any): Unit =
|
|
|
|
|
|
if (LogInfo) log.info("Cluster Node [{}] - " + template, selfAddress, arg1, arg2)
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2011-10-26 08:48:16 +02:00
|
|
|
|
}
|