Moved ClusterNode interface, NodeAddress and ChangeListener into akka-actor as real Trait instead of using structural typing.
Refactored boot dependency in Cluster/Actor/Deployer. Added multi-jvm test for testing clustered actor deployment, check out as LocalActorRef and ClusterActorRef. Signed-off-by: Jonas Bonér <jonasremove@jonasboner.com>
This commit is contained in:
parent
7778c93c1c
commit
ddb2a69c19
23 changed files with 709 additions and 265 deletions
|
|
@ -20,7 +20,7 @@ class DeployerSpec extends WordSpec with MustMatchers {
|
||||||
LeastCPU,
|
LeastCPU,
|
||||||
"akka.serialization.Format$Default$",
|
"akka.serialization.Format$Default$",
|
||||||
Clustered(
|
Clustered(
|
||||||
Node("test-1"),
|
Node("node1"),
|
||||||
Replicate(3),
|
Replicate(3),
|
||||||
Stateless))))
|
Stateless))))
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -6,7 +6,7 @@ package akka.actor
|
||||||
|
|
||||||
import DeploymentConfig._
|
import DeploymentConfig._
|
||||||
import akka.dispatch._
|
import akka.dispatch._
|
||||||
import akka.config.Config
|
import akka.config._
|
||||||
import Config._
|
import Config._
|
||||||
import akka.util.{ ListenerManagement, ReflectiveAccess, Duration, Helpers }
|
import akka.util.{ ListenerManagement, ReflectiveAccess, Duration, Helpers }
|
||||||
import ReflectiveAccess._
|
import ReflectiveAccess._
|
||||||
|
|
@ -15,6 +15,7 @@ import akka.remoteinterface.RemoteSupport
|
||||||
import akka.japi.{ Creator, Procedure }
|
import akka.japi.{ Creator, Procedure }
|
||||||
import akka.AkkaException
|
import akka.AkkaException
|
||||||
import akka.serialization.{ Format, Serializer }
|
import akka.serialization.{ Format, Serializer }
|
||||||
|
import akka.cluster.ClusterNode
|
||||||
import akka.event.EventHandler
|
import akka.event.EventHandler
|
||||||
|
|
||||||
import scala.reflect.BeanProperty
|
import scala.reflect.BeanProperty
|
||||||
|
|
@ -137,7 +138,11 @@ object Actor extends ListenerManagement {
|
||||||
/**
|
/**
|
||||||
* Handle to the ClusterNode. API for the cluster client.
|
* Handle to the ClusterNode. API for the cluster client.
|
||||||
*/
|
*/
|
||||||
lazy val cluster: ClusterModule.ClusterNode = ClusterModule.node
|
lazy val cluster: ClusterNode = {
|
||||||
|
val node = ClusterModule.node
|
||||||
|
node.start()
|
||||||
|
node
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Handle to the RemoteSupport. API for the remote client/server.
|
* Handle to the RemoteSupport. API for the remote client/server.
|
||||||
|
|
@ -146,7 +151,7 @@ object Actor extends ListenerManagement {
|
||||||
private[akka] lazy val remote: RemoteSupport = cluster.remoteService
|
private[akka] lazy val remote: RemoteSupport = cluster.remoteService
|
||||||
|
|
||||||
// start up a cluster node to join the ZooKeeper cluster
|
// start up a cluster node to join the ZooKeeper cluster
|
||||||
if (ClusterModule.isEnabled) cluster.start()
|
//if (ClusterModule.isEnabled) cluster.start()
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates an ActorRef out of the Actor with type T.
|
* Creates an ActorRef out of the Actor with type T.
|
||||||
|
|
@ -388,10 +393,10 @@ object Actor extends ListenerManagement {
|
||||||
|
|
||||||
if (!Actor.remote.isRunning) throw new IllegalStateException("Remote server is not running")
|
if (!Actor.remote.isRunning) throw new IllegalStateException("Remote server is not running")
|
||||||
|
|
||||||
val hostname = home match {
|
val isHomeNode = home match {
|
||||||
case Host(hostname) ⇒ hostname
|
case Host(hostname) ⇒ hostname == Config.hostname
|
||||||
case IP(address) ⇒ address
|
case IP(address) ⇒ address == "0.0.0.0" // FIXME checking if IP address is on home node is missing
|
||||||
case Node(nodeName) ⇒ Config.hostname
|
case Node(nodename) ⇒ nodename == Config.nodename
|
||||||
}
|
}
|
||||||
|
|
||||||
val replicas = replication match {
|
val replicas = replication match {
|
||||||
|
|
@ -402,7 +407,7 @@ object Actor extends ListenerManagement {
|
||||||
case NoReplicas() ⇒ 0
|
case NoReplicas() ⇒ 0
|
||||||
}
|
}
|
||||||
|
|
||||||
if (hostname == Config.hostname) { // home node for clustered actor
|
if (isHomeNode) { // home node for clustered actor
|
||||||
|
|
||||||
def serializerErrorDueTo(reason: String) =
|
def serializerErrorDueTo(reason: String) =
|
||||||
throw new akka.config.ConfigurationException(
|
throw new akka.config.ConfigurationException(
|
||||||
|
|
@ -433,7 +438,9 @@ object Actor extends ListenerManagement {
|
||||||
|
|
||||||
if (!cluster.isClustered(address)) cluster.store(factory().start(), replicas, false, serializer) // add actor to cluster registry (if not already added)
|
if (!cluster.isClustered(address)) cluster.store(factory().start(), replicas, false, serializer) // add actor to cluster registry (if not already added)
|
||||||
|
|
||||||
cluster.use(address, serializer)
|
cluster
|
||||||
|
.use(address, serializer)
|
||||||
|
.getOrElse(throw new ConfigurationException("Could not check out actor [" + address + "] from cluster registry as a \"local\" actor"))
|
||||||
|
|
||||||
} else {
|
} else {
|
||||||
val routerType = router match {
|
val routerType = router match {
|
||||||
|
|
@ -464,7 +471,7 @@ object Actor extends ListenerManagement {
|
||||||
|
|
||||||
*/
|
*/
|
||||||
|
|
||||||
RemoteActorRef(address, Actor.TIMEOUT, None)
|
// RemoteActorRef(address, Actor.TIMEOUT, None)
|
||||||
|
|
||||||
case invalid ⇒ throw new IllegalActorStateException(
|
case invalid ⇒ throw new IllegalActorStateException(
|
||||||
"Could not create actor with address [" + address +
|
"Could not create actor with address [" + address +
|
||||||
|
|
|
||||||
|
|
@ -1006,8 +1006,10 @@ private[akka] case class RemoteActorRef private[akka] (
|
||||||
case Node(nodeName) ⇒ Config.hostname
|
case Node(nodeName) ⇒ Config.hostname
|
||||||
}
|
}
|
||||||
new InetSocketAddress(hostname, Config.remoteServerPort)
|
new InetSocketAddress(hostname, Config.remoteServerPort)
|
||||||
case _ ⇒ throw new IllegalStateException(
|
case _ ⇒
|
||||||
"Actor with Address [" + address + "] is not bound to a Clustered Deployment")
|
new InetSocketAddress(Config.hostname, Config.remoteServerPort)
|
||||||
|
//throw new IllegalStateException(
|
||||||
|
// "Actor with Address [" + address + "] is not bound to a Clustered Deployment")
|
||||||
}
|
}
|
||||||
|
|
||||||
start()
|
start()
|
||||||
|
|
|
||||||
|
|
@ -265,6 +265,7 @@ object Deployer {
|
||||||
// --------------------------------
|
// --------------------------------
|
||||||
// akka.actor.deployment.<address>.clustered.home
|
// akka.actor.deployment.<address>.clustered.home
|
||||||
// --------------------------------
|
// --------------------------------
|
||||||
|
|
||||||
val home = clusteredConfig.getString("home", "") match {
|
val home = clusteredConfig.getString("home", "") match {
|
||||||
case "" ⇒ Host("localhost")
|
case "" ⇒ Host("localhost")
|
||||||
case home ⇒
|
case home ⇒
|
||||||
|
|
|
||||||
449
akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala
Normal file
449
akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala
Normal file
|
|
@ -0,0 +1,449 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
|
||||||
|
*/
|
||||||
|
|
||||||
|
package akka.cluster
|
||||||
|
|
||||||
|
import akka.remoteinterface.RemoteSupport
|
||||||
|
import akka.serialization.Serializer
|
||||||
|
import akka.actor._
|
||||||
|
import akka.dispatch.Future
|
||||||
|
import akka.config.Config
|
||||||
|
import akka.util._
|
||||||
|
|
||||||
|
import com.eaio.uuid.UUID
|
||||||
|
|
||||||
|
import java.net.InetSocketAddress
|
||||||
|
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger }
|
||||||
|
import java.util.concurrent.{ ConcurrentSkipListSet, ConcurrentHashMap }
|
||||||
|
|
||||||
|
import scala.collection.mutable.ConcurrentMap
|
||||||
|
import scala.collection.JavaConversions._
|
||||||
|
|
||||||
|
object ChangeListener {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Cluster membership change listener.
|
||||||
|
* For Scala API.
|
||||||
|
*/
|
||||||
|
trait ChangeListener {
|
||||||
|
def notify(event: ChangeNotification, client: ClusterNode) {
|
||||||
|
event match {
|
||||||
|
case NodeConnected(name) ⇒ nodeConnected(name, client)
|
||||||
|
case NodeDisconnected(name) ⇒ nodeDisconnected(name, client)
|
||||||
|
case NewLeader(name: String) ⇒ newLeader(name, client)
|
||||||
|
case NewSession ⇒ thisNodeNewSession(client)
|
||||||
|
case ThisNode.Connected ⇒ thisNodeConnected(client)
|
||||||
|
case ThisNode.Disconnected ⇒ thisNodeDisconnected(client)
|
||||||
|
case ThisNode.Expired ⇒ thisNodeExpired(client)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def nodeConnected(node: String, client: ClusterNode) {}
|
||||||
|
|
||||||
|
def nodeDisconnected(node: String, client: ClusterNode) {}
|
||||||
|
|
||||||
|
def newLeader(name: String, client: ClusterNode) {}
|
||||||
|
|
||||||
|
def thisNodeNewSession(client: ClusterNode) {}
|
||||||
|
|
||||||
|
def thisNodeConnected(client: ClusterNode) {}
|
||||||
|
|
||||||
|
def thisNodeDisconnected(client: ClusterNode) {}
|
||||||
|
|
||||||
|
def thisNodeExpired(client: ClusterNode) {}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Cluster membership change listener.
|
||||||
|
* For Java API.
|
||||||
|
*/
|
||||||
|
abstract class ChangeListenerAdapter extends ChangeListener
|
||||||
|
|
||||||
|
sealed trait ChangeNotification
|
||||||
|
|
||||||
|
case class NodeConnected(node: String) extends ChangeNotification
|
||||||
|
|
||||||
|
case class NodeDisconnected(node: String) extends ChangeNotification
|
||||||
|
|
||||||
|
case class NewLeader(name: String) extends ChangeNotification
|
||||||
|
|
||||||
|
case object NewSession extends ChangeNotification
|
||||||
|
|
||||||
|
object ThisNode {
|
||||||
|
|
||||||
|
case object Connected extends ChangeNotification
|
||||||
|
|
||||||
|
case object Disconnected extends ChangeNotification
|
||||||
|
|
||||||
|
case object Expired extends ChangeNotification
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Node address holds the node name and the cluster name and can be used as a hash lookup key for a Node instance.
|
||||||
|
*
|
||||||
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
|
*/
|
||||||
|
class NodeAddress(
|
||||||
|
val clusterName: String,
|
||||||
|
val nodeName: String,
|
||||||
|
val hostname: String,
|
||||||
|
val port: Int) {
|
||||||
|
if ((hostname eq null) || hostname == "") throw new NullPointerException("Host name must not be null or empty string")
|
||||||
|
if ((nodeName eq null) || nodeName == "") throw new NullPointerException("Node name must not be null or empty string")
|
||||||
|
if ((clusterName eq null) || clusterName == "") throw new NullPointerException("Cluster name must not be null or empty string")
|
||||||
|
if (port < 1) throw new NullPointerException("Port can not be negative")
|
||||||
|
|
||||||
|
override def toString = "%s:%s:%s:%s".format(clusterName, nodeName, hostname, port)
|
||||||
|
|
||||||
|
override def hashCode = 0 + clusterName.## + nodeName.## + hostname.## + port.##
|
||||||
|
|
||||||
|
override def equals(other: Any) = NodeAddress.unapply(this) == NodeAddress.unapply(other)
|
||||||
|
}
|
||||||
|
|
||||||
|
object NodeAddress {
|
||||||
|
|
||||||
|
def apply(
|
||||||
|
clusterName: String = Config.clusterName,
|
||||||
|
nodeName: String = Config.nodename,
|
||||||
|
hostname: String = Config.hostname,
|
||||||
|
port: Int = Config.remoteServerPort): NodeAddress = new NodeAddress(clusterName, nodeName, hostname, port)
|
||||||
|
|
||||||
|
def unapply(other: Any) = other match {
|
||||||
|
case address: NodeAddress ⇒ Some((address.clusterName, address.nodeName, address.hostname, address.port))
|
||||||
|
case _ ⇒ None
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Interface for cluster node.
|
||||||
|
*
|
||||||
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
|
*/
|
||||||
|
trait ClusterNode {
|
||||||
|
import ChangeListener._
|
||||||
|
|
||||||
|
val nodeAddress: NodeAddress
|
||||||
|
val zkServerAddresses: String
|
||||||
|
|
||||||
|
val remoteClientLifeCycleListener: ActorRef
|
||||||
|
val remoteDaemon: ActorRef
|
||||||
|
val remoteService: RemoteSupport
|
||||||
|
val remoteServerAddress: InetSocketAddress
|
||||||
|
|
||||||
|
val isConnected = new Switch(false)
|
||||||
|
val isLeader = new AtomicBoolean(false)
|
||||||
|
val electionNumber = new AtomicInteger(Int.MaxValue)
|
||||||
|
|
||||||
|
private[cluster] val locallyCachedMembershipNodes = new ConcurrentSkipListSet[String]()
|
||||||
|
private[cluster] val nodeNameToAddress: ConcurrentMap[String, InetSocketAddress] = new ConcurrentHashMap[String, InetSocketAddress]
|
||||||
|
private[cluster] val locallyCheckedOutActors: ConcurrentMap[UUID, Array[Byte]] = new ConcurrentHashMap[UUID, Array[Byte]]
|
||||||
|
|
||||||
|
def membershipNodes: Array[String]
|
||||||
|
|
||||||
|
def isRunning: Boolean = isConnected.isOn
|
||||||
|
|
||||||
|
def start(): ClusterNode
|
||||||
|
|
||||||
|
def shutdown()
|
||||||
|
|
||||||
|
def disconnect(): ClusterNode
|
||||||
|
|
||||||
|
def reconnect(): ClusterNode
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Registers a cluster change listener.
|
||||||
|
*/
|
||||||
|
def register(listener: ChangeListener): ClusterNode
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the name of the current leader.
|
||||||
|
*/
|
||||||
|
def leader: String
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Explicitly resign from being a leader. If this node is not a leader then this operation is a no-op.
|
||||||
|
*/
|
||||||
|
def resign()
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
|
||||||
|
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
|
||||||
|
* available durable store.
|
||||||
|
*/
|
||||||
|
def store[T <: Actor](address: String, actorClass: Class[T], format: Serializer): ClusterNode
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
|
||||||
|
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
|
||||||
|
* available durable store.
|
||||||
|
*/
|
||||||
|
def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, format: Serializer): ClusterNode
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
|
||||||
|
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
|
||||||
|
* available durable store.
|
||||||
|
*/
|
||||||
|
def store[T <: Actor](address: String, actorClass: Class[T], serializeMailbox: Boolean, format: Serializer): ClusterNode
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
|
||||||
|
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
|
||||||
|
* available durable store.
|
||||||
|
*/
|
||||||
|
def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, serializeMailbox: Boolean, format: Serializer): ClusterNode
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
|
||||||
|
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
|
||||||
|
* available durable store.
|
||||||
|
*/
|
||||||
|
def store(actorRef: ActorRef, format: Serializer): ClusterNode
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
|
||||||
|
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
|
||||||
|
* available durable store.
|
||||||
|
*/
|
||||||
|
def store(actorRef: ActorRef, replicationFactor: Int, format: Serializer): ClusterNode
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
|
||||||
|
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
|
||||||
|
* available durable store.
|
||||||
|
*/
|
||||||
|
def store(actorRef: ActorRef, serializeMailbox: Boolean, format: Serializer): ClusterNode
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Needed to have reflection through structural typing work.
|
||||||
|
*/
|
||||||
|
def store(actorRef: ActorRef, replicationFactor: Int, serializeMailbox: Boolean, format: AnyRef): ClusterNode
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
|
||||||
|
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
|
||||||
|
* available durable store.
|
||||||
|
*/
|
||||||
|
def store(actorRef: ActorRef, replicationFactor: Int, serializeMailbox: Boolean, format: Serializer): ClusterNode
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Removes actor with uuid from the cluster.
|
||||||
|
*/
|
||||||
|
def remove(uuid: UUID)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Removes actor with address from the cluster.
|
||||||
|
*/
|
||||||
|
def remove(address: String): ClusterNode
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Is the actor with uuid clustered or not?
|
||||||
|
*/
|
||||||
|
def isClustered(actorAddress: String): Boolean
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Is the actor with uuid in use on 'this' node or not?
|
||||||
|
*/
|
||||||
|
def isInUseOnNode(actorAddress: String): Boolean
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Is the actor with uuid in use or not?
|
||||||
|
*/
|
||||||
|
def isInUseOnNode(actorAddress: String, node: NodeAddress): Boolean
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Checks out an actor for use on this node, e.g. checked out as a 'LocalActorRef' but it makes it available
|
||||||
|
* for remote access through lookup by its UUID.
|
||||||
|
*/
|
||||||
|
def use[T <: Actor](actorAddress: String): Option[LocalActorRef]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Checks out an actor for use on this node, e.g. checked out as a 'LocalActorRef' but it makes it available
|
||||||
|
* for remote access through lookup by its UUID.
|
||||||
|
*/
|
||||||
|
def use[T <: Actor](actorAddress: String, format: Serializer): Option[LocalActorRef]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Using (checking out) all actors with a specific UUID on all nodes in the cluster.
|
||||||
|
*/
|
||||||
|
def useActorOnAllNodes(uuid: UUID)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Using (checking out) specific UUID on a specefic node.
|
||||||
|
*/
|
||||||
|
def useActorOnNode(node: String, uuid: UUID)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Checks in an actor after done using it on this node.
|
||||||
|
*/
|
||||||
|
def release(actorAddress: String)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Releases (checking in) all actors with a specific UUID on all nodes in the cluster where the actor is in 'use'.
|
||||||
|
*/
|
||||||
|
def releaseActorOnAllNodes(uuid: UUID)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates an ActorRef with a Router to a set of clustered actors.
|
||||||
|
*/
|
||||||
|
def ref(actorAddress: String, router: RouterType): ActorRef
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Migrate the actor from 'this' node to node 'to'.
|
||||||
|
*/
|
||||||
|
def migrate(to: NodeAddress, actorAddress: String)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Migrate the actor from node 'from' to node 'to'.
|
||||||
|
*/
|
||||||
|
def migrate(from: NodeAddress, to: NodeAddress, actorAddress: String)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the UUIDs of all actors checked out on this node.
|
||||||
|
*/
|
||||||
|
def uuidsForActorsInUse: Array[UUID]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the addresses of all actors checked out on this node.
|
||||||
|
*/
|
||||||
|
def addressesForActorsInUse: Array[String]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the UUIDs of all actors registered in this cluster.
|
||||||
|
*/
|
||||||
|
def uuidsForClusteredActors: Array[UUID]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the addresses of all actors registered in this cluster.
|
||||||
|
*/
|
||||||
|
def addressesForClusteredActors: Array[String]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the actor id for the actor with a specific UUID.
|
||||||
|
*/
|
||||||
|
def actorAddressForUuid(uuid: UUID): String
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the actor ids for all the actors with a specific UUID.
|
||||||
|
*/
|
||||||
|
def actorAddressForUuids(uuids: Array[UUID]): Array[String]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the actor UUIDs for actor ID.
|
||||||
|
*/
|
||||||
|
def uuidsForActorAddress(actorAddress: String): Array[UUID]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the node names of all actors in use with UUID.
|
||||||
|
*/
|
||||||
|
def nodesForActorsInUseWithUuid(uuid: UUID): Array[String]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the UUIDs of all actors in use registered on a specific node.
|
||||||
|
*/
|
||||||
|
def uuidsForActorsInUseOnNode(nodeName: String): Array[UUID]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the addresses of all actors in use registered on a specific node.
|
||||||
|
*/
|
||||||
|
def addressesForActorsInUseOnNode(nodeName: String): Array[String]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns Format for actor with UUID.
|
||||||
|
*/
|
||||||
|
def formatForActor(actorAddress: String): Serializer
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns home address for actor with UUID.
|
||||||
|
*/
|
||||||
|
def addressesForActor(actorAddress: String): Array[(UUID, InetSocketAddress)]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Send a function 'Function0[Unit]' to be invoked on a random number of nodes (defined by 'replicationFactor' argument).
|
||||||
|
*/
|
||||||
|
def send(f: Function0[Unit], replicationFactor: Int)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Send a function 'Function0[Any]' to be invoked on a random number of nodes (defined by 'replicationFactor' argument).
|
||||||
|
* Returns an 'Array' with all the 'Future's from the computation.
|
||||||
|
*/
|
||||||
|
def send(f: Function0[Any], replicationFactor: Int): List[Future[Any]]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Send a function 'Function1[Any, Unit]' to be invoked on a random number of nodes (defined by 'replicationFactor' argument)
|
||||||
|
* with the argument speficied.
|
||||||
|
*/
|
||||||
|
def send(f: Function1[Any, Unit], arg: Any, replicationFactor: Int)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Send a function 'Function1[Any, Any]' to be invoked on a random number of nodes (defined by 'replicationFactor' argument)
|
||||||
|
* with the argument speficied.
|
||||||
|
* Returns an 'Array' with all the 'Future's from the computation.
|
||||||
|
*/
|
||||||
|
def send(f: Function1[Any, Any], arg: Any, replicationFactor: Int): List[Future[Any]]
|
||||||
|
|
||||||
|
def setConfigElement(key: String, bytes: Array[Byte])
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the config element for the key or NULL if no element exists under the key.
|
||||||
|
*/
|
||||||
|
def getConfigElement(key: String): Array[Byte]
|
||||||
|
|
||||||
|
def removeConfigElement(key: String)
|
||||||
|
|
||||||
|
def getConfigElementKeys: Array[String]
|
||||||
|
|
||||||
|
private[cluster] def initializeNode()
|
||||||
|
|
||||||
|
private[cluster] def addressForNode(node: String): InetSocketAddress
|
||||||
|
|
||||||
|
private[cluster] def publish(change: ChangeNotification)
|
||||||
|
|
||||||
|
private[cluster] def findFailedNodes(nodes: List[String]): List[String]
|
||||||
|
|
||||||
|
private[cluster] def findNewlyConnectedMembershipNodes(nodes: List[String]): List[String]
|
||||||
|
|
||||||
|
private[cluster] def findNewlyDisconnectedMembershipNodes(nodes: List[String]): List[String]
|
||||||
|
|
||||||
|
private[cluster] def findNewlyConnectedAvailableNodes(nodes: List[String]): List[String]
|
||||||
|
|
||||||
|
private[cluster] def findNewlyDisconnectedAvailableNodes(nodes: List[String]): List[String]
|
||||||
|
|
||||||
|
private[cluster] def joinMembershipNode()
|
||||||
|
|
||||||
|
private[cluster] def joinActorsAtAddressNode()
|
||||||
|
|
||||||
|
private[cluster] def joinLeaderElection: Boolean
|
||||||
|
|
||||||
|
private[cluster] def failOverConnections(from: InetSocketAddress, to: InetSocketAddress)
|
||||||
|
|
||||||
|
private[cluster] def migrateFromFailedNodes[T <: Actor](currentSetOfClusterNodes: List[String])
|
||||||
|
|
||||||
|
private[cluster] def membershipPathFor(node: String): String
|
||||||
|
|
||||||
|
private[cluster] def configurationPathFor(key: String): String
|
||||||
|
|
||||||
|
private[cluster] def actorAddressToUuidsPathFor(actorAddress: String): String
|
||||||
|
|
||||||
|
private[cluster] def actorLocationsPathFor(uuid: UUID): String
|
||||||
|
|
||||||
|
private[cluster] def actorLocationsPathFor(uuid: UUID, node: NodeAddress): String
|
||||||
|
|
||||||
|
private[cluster] def actorsAtNodePathFor(node: String): String
|
||||||
|
|
||||||
|
private[cluster] def actorAtNodePathFor(node: String, uuid: UUID): String
|
||||||
|
|
||||||
|
private[cluster] def actorRegistryPathFor(uuid: UUID): String
|
||||||
|
|
||||||
|
private[cluster] def actorRegistryFormatPathFor(uuid: UUID): String
|
||||||
|
|
||||||
|
private[cluster] def actorRegistryActorAddressPathFor(uuid: UUID): String
|
||||||
|
|
||||||
|
private[cluster] def actorRegistryNodePathFor(uuid: UUID): String
|
||||||
|
|
||||||
|
private[cluster] def actorRegistryNodePathFor(uuid: UUID, address: InetSocketAddress): String
|
||||||
|
}
|
||||||
|
|
||||||
|
|
@ -115,6 +115,8 @@ object Config {
|
||||||
case value ⇒ value.toInt
|
case value ⇒ value.toInt
|
||||||
}
|
}
|
||||||
|
|
||||||
|
val clusterName = config.getString("akka.cluster.name", "default")
|
||||||
|
|
||||||
val startTime = System.currentTimeMillis
|
val startTime = System.currentTimeMillis
|
||||||
def uptime = (System.currentTimeMillis - startTime) / 1000
|
def uptime = (System.currentTimeMillis - startTime) / 1000
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -11,6 +11,7 @@ import akka.actor._
|
||||||
import DeploymentConfig.Deploy
|
import DeploymentConfig.Deploy
|
||||||
import akka.event.EventHandler
|
import akka.event.EventHandler
|
||||||
import akka.serialization.Format
|
import akka.serialization.Format
|
||||||
|
import akka.cluster.ClusterNode
|
||||||
|
|
||||||
import java.net.InetSocketAddress
|
import java.net.InetSocketAddress
|
||||||
|
|
||||||
|
|
@ -71,24 +72,6 @@ object ReflectiveAccess {
|
||||||
clusterDeployerInstance.get
|
clusterDeployerInstance.get
|
||||||
}
|
}
|
||||||
|
|
||||||
type ClusterNode = {
|
|
||||||
def start()
|
|
||||||
def shutdown()
|
|
||||||
|
|
||||||
def remoteService: RemoteSupport
|
|
||||||
|
|
||||||
def store(address: String, actorClass: Class[_ <: Actor], replicas: Int, serializeMailbox: Boolean, format: Serializer)
|
|
||||||
def store(actorRef: ActorRef, replicas: Int, serializeMailbox: Boolean, format: Serializer)
|
|
||||||
|
|
||||||
def remove(address: String)
|
|
||||||
|
|
||||||
def use(address: String, format: Serializer): Array[ActorRef]
|
|
||||||
def ref(address: String, router: RouterType): ActorRef
|
|
||||||
|
|
||||||
def isClustered(address: String): Boolean
|
|
||||||
def nrOfActors: Int
|
|
||||||
}
|
|
||||||
|
|
||||||
type ClusterDeployer = {
|
type ClusterDeployer = {
|
||||||
def init(deployments: List[Deploy])
|
def init(deployments: List[Deploy])
|
||||||
def shutdown()
|
def shutdown()
|
||||||
|
|
|
||||||
|
|
@ -38,6 +38,7 @@ import Compression.LZF
|
||||||
import akka.AkkaException
|
import akka.AkkaException
|
||||||
|
|
||||||
import akka.cluster.zookeeper._
|
import akka.cluster.zookeeper._
|
||||||
|
import akka.cluster.ChangeListener._
|
||||||
|
|
||||||
import com.eaio.uuid.UUID
|
import com.eaio.uuid.UUID
|
||||||
|
|
||||||
|
|
@ -153,7 +154,7 @@ object Cluster {
|
||||||
val UUID_PREFIX = "uuid:".intern
|
val UUID_PREFIX = "uuid:".intern
|
||||||
|
|
||||||
// config options
|
// config options
|
||||||
val name = config.getString("akka.cluster.name", "default")
|
val name = Config.clusterName
|
||||||
val zooKeeperServers = config.getString("akka.cluster.zookeeper-server-addresses", "localhost:2181")
|
val zooKeeperServers = config.getString("akka.cluster.zookeeper-server-addresses", "localhost:2181")
|
||||||
val remoteServerPort = config.getInt("akka.cluster.remote-server-port", 2552)
|
val remoteServerPort = config.getInt("akka.cluster.remote-server-port", 2552)
|
||||||
val sessionTimeout = Duration(config.getInt("akka.cluster.session-timeout", 60), TIME_UNIT).toMillis.toInt
|
val sessionTimeout = Duration(config.getInt("akka.cluster.session-timeout", 60), TIME_UNIT).toMillis.toInt
|
||||||
|
|
@ -162,64 +163,6 @@ object Cluster {
|
||||||
val shouldCompressData = config.getBool("akka.cluster.use-compression", false)
|
val shouldCompressData = config.getBool("akka.cluster.use-compression", false)
|
||||||
val enableJMX = config.getBool("akka.enable-jmx", true)
|
val enableJMX = config.getBool("akka.enable-jmx", true)
|
||||||
|
|
||||||
/**
|
|
||||||
* Cluster membership change listener.
|
|
||||||
* For Scala API.
|
|
||||||
*/
|
|
||||||
trait ChangeListener {
|
|
||||||
def notify(event: ChangeNotification, client: ClusterNode) {
|
|
||||||
event match {
|
|
||||||
case NodeConnected(name) ⇒ nodeConnected(name, client)
|
|
||||||
case NodeDisconnected(name) ⇒ nodeDisconnected(name, client)
|
|
||||||
case NewLeader(name: String) ⇒ newLeader(name, client)
|
|
||||||
case NewSession ⇒ thisNodeNewSession(client)
|
|
||||||
case ThisNode.Connected ⇒ thisNodeConnected(client)
|
|
||||||
case ThisNode.Disconnected ⇒ thisNodeDisconnected(client)
|
|
||||||
case ThisNode.Expired ⇒ thisNodeExpired(client)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
def nodeConnected(node: String, client: ClusterNode) {}
|
|
||||||
|
|
||||||
def nodeDisconnected(node: String, client: ClusterNode) {}
|
|
||||||
|
|
||||||
def newLeader(name: String, client: ClusterNode) {}
|
|
||||||
|
|
||||||
def thisNodeNewSession(client: ClusterNode) {}
|
|
||||||
|
|
||||||
def thisNodeConnected(client: ClusterNode) {}
|
|
||||||
|
|
||||||
def thisNodeDisconnected(client: ClusterNode) {}
|
|
||||||
|
|
||||||
def thisNodeExpired(client: ClusterNode) {}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Cluster membership change listener.
|
|
||||||
* For Java API.
|
|
||||||
*/
|
|
||||||
abstract class ChangeListenerAdapter extends ChangeListener
|
|
||||||
|
|
||||||
sealed trait ChangeNotification
|
|
||||||
|
|
||||||
case class NodeConnected(node: String) extends ChangeNotification
|
|
||||||
|
|
||||||
case class NodeDisconnected(node: String) extends ChangeNotification
|
|
||||||
|
|
||||||
case class NewLeader(name: String) extends ChangeNotification
|
|
||||||
|
|
||||||
case object NewSession extends ChangeNotification
|
|
||||||
|
|
||||||
object ThisNode {
|
|
||||||
|
|
||||||
case object Connected extends ChangeNotification
|
|
||||||
|
|
||||||
case object Disconnected extends ChangeNotification
|
|
||||||
|
|
||||||
case object Expired extends ChangeNotification
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
@volatile
|
@volatile
|
||||||
private var properties = Map.empty[String, String]
|
private var properties = Map.empty[String, String]
|
||||||
|
|
||||||
|
|
@ -249,14 +192,14 @@ object Cluster {
|
||||||
/**
|
/**
|
||||||
* The node address.
|
* The node address.
|
||||||
*/
|
*/
|
||||||
lazy val nodeAddress = NodeAddress(name, nodename, hostname, port)
|
val nodeAddress = NodeAddress(name, nodename, hostname, port)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The reference to the running ClusterNode.
|
* The reference to the running ClusterNode.
|
||||||
*/
|
*/
|
||||||
lazy val node: ClusterNode = {
|
val node = {
|
||||||
if (nodeAddress eq null) throw new IllegalArgumentException("NodeAddress can't be null")
|
if (nodeAddress eq null) throw new IllegalArgumentException("NodeAddress can't be null")
|
||||||
new ClusterNode(nodeAddress, zooKeeperServers, defaultSerializer)
|
new DefaultClusterNode(nodeAddress, zooKeeperServers, defaultSerializer)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -319,6 +262,14 @@ object Cluster {
|
||||||
*/
|
*/
|
||||||
def newZkClient: AkkaZkClient = new AkkaZkClient(zooKeeperServers, sessionTimeout, connectionTimeout, defaultSerializer)
|
def newZkClient: AkkaZkClient = new AkkaZkClient(zooKeeperServers, sessionTimeout, connectionTimeout, defaultSerializer)
|
||||||
|
|
||||||
|
def createQueue(rootPath: String, blocking: Boolean = true) = new ZooKeeperQueue(node.zkClient, rootPath, blocking)
|
||||||
|
|
||||||
|
def barrier(name: String, count: Int) =
|
||||||
|
ZooKeeperBarrier(node.zkClient, node.nodeAddress.clusterName, name, node.nodeAddress.nodeName, count)
|
||||||
|
|
||||||
|
def barrier(name: String, count: Int, timeout: Duration) =
|
||||||
|
ZooKeeperBarrier(node.zkClient, node.nodeAddress.clusterName, name, node.nodeAddress.nodeName, count, timeout)
|
||||||
|
|
||||||
def uuidToString(uuid: UUID): String = uuid.toString
|
def uuidToString(uuid: UUID): String = uuid.toString
|
||||||
|
|
||||||
def stringToUuid(uuid: String): UUID = {
|
def stringToUuid(uuid: String): UUID = {
|
||||||
|
|
@ -348,35 +299,26 @@ object Cluster {
|
||||||
*
|
*
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
class ClusterNode private[akka] (
|
class DefaultClusterNode private[akka] (
|
||||||
val nodeAddress: NodeAddress,
|
val nodeAddress: NodeAddress,
|
||||||
val zkServerAddresses: String,
|
val zkServerAddresses: String,
|
||||||
val serializer: ZkSerializer) extends ErrorHandler {
|
val serializer: ZkSerializer) extends ErrorHandler with ClusterNode {
|
||||||
self ⇒
|
self ⇒
|
||||||
|
|
||||||
if (nodeAddress eq null) throw new IllegalArgumentException("'nodeAddress' can not be 'null'")
|
if (nodeAddress eq null) throw new IllegalArgumentException("'nodeAddress' can not be 'null'")
|
||||||
|
|
||||||
|
val clusterJmxObjectName = JMX.nameFor(nodeAddress.hostname, "monitoring", "cluster")
|
||||||
|
|
||||||
import Cluster._
|
import Cluster._
|
||||||
|
|
||||||
EventHandler.info(this,
|
lazy val remoteClientLifeCycleListener = actorOf(new Actor {
|
||||||
("\nCreating cluster node with" +
|
|
||||||
"\n\tcluster name = [%s]" +
|
|
||||||
"\n\tnode name = [%s]" +
|
|
||||||
"\n\tport = [%s]" +
|
|
||||||
"\n\tzookeeper server addresses = [%s]" +
|
|
||||||
"\n\tserializer = [%s]")
|
|
||||||
.format(nodeAddress.clusterName, nodeAddress.nodeName, nodeAddress.port, zkServerAddresses, serializer))
|
|
||||||
|
|
||||||
val remoteClientLifeCycleListener = actorOf(new Actor {
|
|
||||||
def receive = {
|
def receive = {
|
||||||
case RemoteClientError(cause, client, address) ⇒ client.shutdownClientModule()
|
case RemoteClientError(cause, client, address) ⇒ client.shutdownClientModule()
|
||||||
case RemoteClientDisconnected(client, address) ⇒ client.shutdownClientModule()
|
case RemoteClientDisconnected(client, address) ⇒ client.shutdownClientModule()
|
||||||
case _ ⇒ //ignore other
|
case _ ⇒ //ignore other
|
||||||
}
|
}
|
||||||
}, "akka.cluster.remoteClientLifeCycleListener").start()
|
}, "akka.cluster.remoteClientLifeCycleListener").start()
|
||||||
|
|
||||||
lazy val remoteDaemon = actorOf(new RemoteClusterDaemon(this), RemoteClusterDaemon.ADDRESS).start()
|
lazy val remoteDaemon = actorOf(new RemoteClusterDaemon(this), RemoteClusterDaemon.ADDRESS).start()
|
||||||
|
|
||||||
lazy val remoteService: RemoteSupport = {
|
lazy val remoteService: RemoteSupport = {
|
||||||
val remote = new akka.remote.netty.NettyRemoteSupport
|
val remote = new akka.remote.netty.NettyRemoteSupport
|
||||||
remote.start(nodeAddress.hostname, nodeAddress.port)
|
remote.start(nodeAddress.hostname, nodeAddress.port)
|
||||||
|
|
@ -386,8 +328,6 @@ class ClusterNode private[akka] (
|
||||||
}
|
}
|
||||||
lazy val remoteServerAddress: InetSocketAddress = remoteService.address
|
lazy val remoteServerAddress: InetSocketAddress = remoteService.address
|
||||||
|
|
||||||
val clusterJmxObjectName = JMX.nameFor(nodeAddress.hostname, "monitoring", "cluster")
|
|
||||||
|
|
||||||
// static nodes
|
// static nodes
|
||||||
val CLUSTER_NODE = "/" + nodeAddress.clusterName
|
val CLUSTER_NODE = "/" + nodeAddress.clusterName
|
||||||
val MEMBERSHIP_NODE = CLUSTER_NODE + "/members"
|
val MEMBERSHIP_NODE = CLUSTER_NODE + "/members"
|
||||||
|
|
@ -409,17 +349,8 @@ class ClusterNode private[akka] (
|
||||||
|
|
||||||
val LEADER_ELECTION_NODE = CLUSTER_NODE + "/leader" // should NOT be part of 'baseNodes' only used by 'leaderLock'
|
val LEADER_ELECTION_NODE = CLUSTER_NODE + "/leader" // should NOT be part of 'baseNodes' only used by 'leaderLock'
|
||||||
|
|
||||||
val isConnected = new Switch(false)
|
|
||||||
val isLeader = new AtomicBoolean(false)
|
|
||||||
val electionNumber = new AtomicInteger(Integer.MAX_VALUE)
|
|
||||||
|
|
||||||
private val membershipNodePath = membershipPathFor(nodeAddress.nodeName)
|
private val membershipNodePath = membershipPathFor(nodeAddress.nodeName)
|
||||||
|
|
||||||
// local caches of ZK data
|
|
||||||
private[akka] val locallyCachedMembershipNodes = new ConcurrentSkipListSet[String]()
|
|
||||||
private[akka] val nodeNameToAddress: ConcurrentMap[String, InetSocketAddress] = new ConcurrentHashMap[String, InetSocketAddress]
|
|
||||||
private[akka] val locallyCheckedOutActors: ConcurrentMap[UUID, Array[Byte]] = new ConcurrentHashMap[UUID, Array[Byte]]
|
|
||||||
|
|
||||||
def membershipNodes: Array[String] = locallyCachedMembershipNodes.toList.toArray.asInstanceOf[Array[String]]
|
def membershipNodes: Array[String] = locallyCachedMembershipNodes.toList.toArray.asInstanceOf[Array[String]]
|
||||||
|
|
||||||
private[akka] val replicaConnections: ConcurrentMap[String, Tuple2[InetSocketAddress, ActorRef]] =
|
private[akka] val replicaConnections: ConcurrentMap[String, Tuple2[InetSocketAddress, ActorRef]] =
|
||||||
|
|
@ -436,13 +367,13 @@ class ClusterNode private[akka] (
|
||||||
private val clusterActorRefs = new Index[InetSocketAddress, ClusterActorRef]
|
private val clusterActorRefs = new Index[InetSocketAddress, ClusterActorRef]
|
||||||
|
|
||||||
// resources
|
// resources
|
||||||
private[cluster] val zkClient = new AkkaZkClient(zkServerAddresses, sessionTimeout, connectionTimeout, serializer)
|
lazy private[cluster] val zkClient = new AkkaZkClient(zkServerAddresses, sessionTimeout, connectionTimeout, serializer)
|
||||||
|
|
||||||
private[cluster] val leaderElectionCallback = new LockListener {
|
lazy private[cluster] val leaderElectionCallback = new LockListener {
|
||||||
override def lockAcquired() {
|
override def lockAcquired() {
|
||||||
EventHandler.info(this, "Node [%s] is the new leader".format(self.nodeAddress.nodeName))
|
EventHandler.info(this, "Node [%s] is the new leader".format(self.nodeAddress.nodeName))
|
||||||
self.isLeader.set(true)
|
self.isLeader.set(true)
|
||||||
self.publish(Cluster.NewLeader(self.nodeAddress.nodeName))
|
self.publish(NewLeader(self.nodeAddress.nodeName))
|
||||||
}
|
}
|
||||||
|
|
||||||
override def lockReleased() {
|
override def lockReleased() {
|
||||||
|
|
@ -453,7 +384,7 @@ class ClusterNode private[akka] (
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private[cluster] val leaderLock = new WriteLock(
|
lazy private[cluster] val leaderLock = new WriteLock(
|
||||||
zkClient.connection.getZookeeper, LEADER_ELECTION_NODE, null, leaderElectionCallback) {
|
zkClient.connection.getZookeeper, LEADER_ELECTION_NODE, null, leaderElectionCallback) {
|
||||||
// ugly hack, but what do you do? <--- haha epic
|
// ugly hack, but what do you do? <--- haha epic
|
||||||
private val ownerIdField = classOf[WriteLock].getDeclaredField("ownerId")
|
private val ownerIdField = classOf[WriteLock].getDeclaredField("ownerId")
|
||||||
|
|
@ -468,8 +399,6 @@ class ClusterNode private[akka] (
|
||||||
// Node
|
// Node
|
||||||
// =======================================
|
// =======================================
|
||||||
|
|
||||||
def isRunning: Boolean = isConnected.isOn
|
|
||||||
|
|
||||||
def start(): ClusterNode = {
|
def start(): ClusterNode = {
|
||||||
isConnected switchOn {
|
isConnected switchOn {
|
||||||
initializeNode()
|
initializeNode()
|
||||||
|
|
@ -738,13 +667,13 @@ class ClusterNode private[akka] (
|
||||||
* Checks out an actor for use on this node, e.g. checked out as a 'LocalActorRef' but it makes it available
|
* Checks out an actor for use on this node, e.g. checked out as a 'LocalActorRef' but it makes it available
|
||||||
* for remote access through lookup by its UUID.
|
* for remote access through lookup by its UUID.
|
||||||
*/
|
*/
|
||||||
def use[T <: Actor](actorAddress: String): Array[LocalActorRef] = use(actorAddress, formatForActor(actorAddress))
|
def use[T <: Actor](actorAddress: String): Option[LocalActorRef] = use(actorAddress, formatForActor(actorAddress))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Checks out an actor for use on this node, e.g. checked out as a 'LocalActorRef' but it makes it available
|
* Checks out an actor for use on this node, e.g. checked out as a 'LocalActorRef' but it makes it available
|
||||||
* for remote access through lookup by its UUID.
|
* for remote access through lookup by its UUID.
|
||||||
*/
|
*/
|
||||||
def use[T <: Actor](actorAddress: String, format: Serializer): Array[LocalActorRef] = if (isConnected.isOn) {
|
def use[T <: Actor](actorAddress: String, format: Serializer): Option[LocalActorRef] = if (isConnected.isOn) {
|
||||||
|
|
||||||
import akka.serialization.ActorSerialization._
|
import akka.serialization.ActorSerialization._
|
||||||
|
|
||||||
|
|
@ -780,8 +709,8 @@ class ClusterNode private[akka] (
|
||||||
actor.asInstanceOf[LocalActorRef]
|
actor.asInstanceOf[LocalActorRef]
|
||||||
case Right(exception) ⇒ throw exception
|
case Right(exception) ⇒ throw exception
|
||||||
}
|
}
|
||||||
}
|
} headOption // FIXME should not be an array at all coming here
|
||||||
} else Array.empty[LocalActorRef]
|
} else None
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Using (checking out) all actors with a specific UUID on all nodes in the cluster.
|
* Using (checking out) all actors with a specific UUID on all nodes in the cluster.
|
||||||
|
|
@ -1151,22 +1080,6 @@ class ClusterNode private[akka] (
|
||||||
|
|
||||||
def getConfigElementKeys: Array[String] = zkClient.getChildren(CONFIGURATION_NODE).toList.toArray.asInstanceOf[Array[String]]
|
def getConfigElementKeys: Array[String] = zkClient.getChildren(CONFIGURATION_NODE).toList.toArray.asInstanceOf[Array[String]]
|
||||||
|
|
||||||
// =======================================
|
|
||||||
// Queue
|
|
||||||
// =======================================
|
|
||||||
|
|
||||||
def createQueue(rootPath: String, blocking: Boolean = true) = new ZooKeeperQueue(zkClient, rootPath, blocking)
|
|
||||||
|
|
||||||
// =======================================
|
|
||||||
// Barrier
|
|
||||||
// =======================================
|
|
||||||
|
|
||||||
def barrier(name: String, count: Int) =
|
|
||||||
ZooKeeperBarrier(zkClient, nodeAddress.clusterName, name, nodeAddress.nodeName, count)
|
|
||||||
|
|
||||||
def barrier(name: String, count: Int, timeout: Duration) =
|
|
||||||
ZooKeeperBarrier(zkClient, nodeAddress.clusterName, name, nodeAddress.nodeName, count, timeout)
|
|
||||||
|
|
||||||
// =======================================
|
// =======================================
|
||||||
// Private
|
// Private
|
||||||
// =======================================
|
// =======================================
|
||||||
|
|
@ -1198,7 +1111,14 @@ class ClusterNode private[akka] (
|
||||||
"%s/%s:%s".format(actorRegistryNodePathFor(uuid), address.getHostName, address.getPort)
|
"%s/%s:%s".format(actorRegistryNodePathFor(uuid), address.getHostName, address.getPort)
|
||||||
|
|
||||||
private[cluster] def initializeNode() {
|
private[cluster] def initializeNode() {
|
||||||
EventHandler.info(this, "Initializing cluster node [%s]".format(nodeAddress))
|
EventHandler.info(this,
|
||||||
|
("\nCreating cluster node with" +
|
||||||
|
"\n\tcluster name = [%s]" +
|
||||||
|
"\n\tnode name = [%s]" +
|
||||||
|
"\n\tport = [%s]" +
|
||||||
|
"\n\tzookeeper server addresses = [%s]" +
|
||||||
|
"\n\tserializer = [%s]")
|
||||||
|
.format(nodeAddress.clusterName, nodeAddress.nodeName, nodeAddress.port, zkServerAddresses, serializer))
|
||||||
EventHandler.info(this, "Starting up remote server [%s]".format(remoteServerAddress.toString))
|
EventHandler.info(this, "Starting up remote server [%s]".format(remoteServerAddress.toString))
|
||||||
createRootClusterNode()
|
createRootClusterNode()
|
||||||
val isLeader = joinLeaderElection
|
val isLeader = joinLeaderElection
|
||||||
|
|
@ -1499,12 +1419,12 @@ class MembershipChildListener(self: ClusterNode) extends IZkChildListener with E
|
||||||
.format(self.nodeAddress.nodeName, childList.mkString(" ")))
|
.format(self.nodeAddress.nodeName, childList.mkString(" ")))
|
||||||
self.findNewlyConnectedMembershipNodes(childList) foreach { name ⇒
|
self.findNewlyConnectedMembershipNodes(childList) foreach { name ⇒
|
||||||
self.nodeNameToAddress.put(name, self.addressForNode(name)) // update 'nodename-address' map
|
self.nodeNameToAddress.put(name, self.addressForNode(name)) // update 'nodename-address' map
|
||||||
self.publish(Cluster.NodeConnected(name))
|
self.publish(NodeConnected(name))
|
||||||
}
|
}
|
||||||
|
|
||||||
self.findNewlyDisconnectedMembershipNodes(childList) foreach { name ⇒
|
self.findNewlyDisconnectedMembershipNodes(childList) foreach { name ⇒
|
||||||
self.nodeNameToAddress.remove(name) // update 'nodename-address' map
|
self.nodeNameToAddress.remove(name) // update 'nodename-address' map
|
||||||
self.publish(Cluster.NodeDisconnected(name))
|
self.publish(NodeDisconnected(name))
|
||||||
}
|
}
|
||||||
|
|
||||||
self.locallyCachedMembershipNodes.clear()
|
self.locallyCachedMembershipNodes.clear()
|
||||||
|
|
@ -1522,13 +1442,13 @@ class StateListener(self: ClusterNode) extends IZkStateListener {
|
||||||
state match {
|
state match {
|
||||||
case KeeperState.SyncConnected ⇒
|
case KeeperState.SyncConnected ⇒
|
||||||
EventHandler.debug(this, "Cluster node [%s] - Connected".format(self.nodeAddress))
|
EventHandler.debug(this, "Cluster node [%s] - Connected".format(self.nodeAddress))
|
||||||
self.publish(Cluster.ThisNode.Connected)
|
self.publish(ThisNode.Connected)
|
||||||
case KeeperState.Disconnected ⇒
|
case KeeperState.Disconnected ⇒
|
||||||
EventHandler.debug(this, "Cluster node [%s] - Disconnected".format(self.nodeAddress))
|
EventHandler.debug(this, "Cluster node [%s] - Disconnected".format(self.nodeAddress))
|
||||||
self.publish(Cluster.ThisNode.Disconnected)
|
self.publish(ThisNode.Disconnected)
|
||||||
case KeeperState.Expired ⇒
|
case KeeperState.Expired ⇒
|
||||||
EventHandler.debug(this, "Cluster node [%s] - Expired".format(self.nodeAddress))
|
EventHandler.debug(this, "Cluster node [%s] - Expired".format(self.nodeAddress))
|
||||||
self.publish(Cluster.ThisNode.Expired)
|
self.publish(ThisNode.Expired)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -1538,7 +1458,7 @@ class StateListener(self: ClusterNode) extends IZkStateListener {
|
||||||
def handleNewSession() {
|
def handleNewSession() {
|
||||||
EventHandler.debug(this, "Session expired re-initializing node [%s]".format(self.nodeAddress))
|
EventHandler.debug(this, "Session expired re-initializing node [%s]".format(self.nodeAddress))
|
||||||
self.initializeNode()
|
self.initializeNode()
|
||||||
self.publish(Cluster.NewSession)
|
self.publish(NewSession)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -56,5 +56,5 @@ class ClusterActorRef private[akka] (
|
||||||
|
|
||||||
// clustered refs are always registered and looked up by UUID
|
// clustered refs are always registered and looked up by UUID
|
||||||
private def createRemoteActorRef(uuid: UUID, address: InetSocketAddress) =
|
private def createRemoteActorRef(uuid: UUID, address: InetSocketAddress) =
|
||||||
RemoteActorRef(UUID_PREFIX + uuidToString(uuid), Actor.TIMEOUT, None)
|
RemoteActorRef(uuidToString(uuid), Actor.TIMEOUT, None)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -4,7 +4,7 @@
|
||||||
|
|
||||||
package akka.cluster
|
package akka.cluster
|
||||||
|
|
||||||
import akka.actor.{ DeploymentConfig, Deployer, DeploymentException }
|
import akka.actor.{ DeploymentConfig, Deployer, LocalDeployer, DeploymentException }
|
||||||
import DeploymentConfig._
|
import DeploymentConfig._
|
||||||
import akka.event.EventHandler
|
import akka.event.EventHandler
|
||||||
import akka.config.Config
|
import akka.config.Config
|
||||||
|
|
@ -119,6 +119,11 @@ object ClusterDeployer {
|
||||||
|
|
||||||
private[akka] def deploy(deployment: Deploy) {
|
private[akka] def deploy(deployment: Deploy) {
|
||||||
ensureRunning {
|
ensureRunning {
|
||||||
|
deployment match {
|
||||||
|
case Deploy(_, _, _, Local) ⇒ // local deployment
|
||||||
|
LocalDeployer.deploy(deployment)
|
||||||
|
|
||||||
|
case _ ⇒ // cluster deployment
|
||||||
val path = deploymentAddressPath.format(deployment.address)
|
val path = deploymentAddressPath.format(deployment.address)
|
||||||
try {
|
try {
|
||||||
ignore[ZkNodeExistsException](zkClient.create(path, null, CreateMode.PERSISTENT))
|
ignore[ZkNodeExistsException](zkClient.create(path, null, CreateMode.PERSISTENT))
|
||||||
|
|
@ -133,6 +138,7 @@ object ClusterDeployer {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private[akka] def lookupDeploymentFor(address: String): Option[Deploy] = ensureRunning {
|
private[akka] def lookupDeploymentFor(address: String): Option[Deploy] = ensureRunning {
|
||||||
try {
|
try {
|
||||||
|
|
|
||||||
|
|
@ -39,12 +39,6 @@ class ClusterDeployerSpec extends WordSpec with MustMatchers with BeforeAndAfter
|
||||||
oldDeployment must equal(newDeployment.get)
|
oldDeployment must equal(newDeployment.get)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
"be able to create an actor deployed using ClusterDeployer, add it to ZooKeeper and then check the actor out for use" in {
|
|
||||||
val pi = Actor.actorOf[HelloWorld]("service-hello")
|
|
||||||
pi must not equal (null)
|
|
||||||
pi.address must equal("service-hello")
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
override def beforeAll() {
|
override def beforeAll() {
|
||||||
|
|
|
||||||
|
|
@ -2,7 +2,7 @@
|
||||||
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
|
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package akka.cluster.multi
|
package akka.cluster.sample
|
||||||
|
|
||||||
import org.scalatest.WordSpec
|
import org.scalatest.WordSpec
|
||||||
import org.scalatest.matchers.MustMatchers
|
import org.scalatest.matchers.MustMatchers
|
||||||
|
|
@ -10,16 +10,15 @@ import org.scalatest.BeforeAndAfterAll
|
||||||
|
|
||||||
import akka.cluster._
|
import akka.cluster._
|
||||||
|
|
||||||
object ClusterMultiJvmSpec {
|
object SampleMultiJvmSpec {
|
||||||
val NrOfNodes = 2
|
val NrOfNodes = 2
|
||||||
}
|
}
|
||||||
|
|
||||||
class ClusterMultiJvmNode1 extends WordSpec with MustMatchers with BeforeAndAfterAll {
|
class SampleMultiJvmNode1 extends WordSpec with MustMatchers with BeforeAndAfterAll {
|
||||||
import ClusterMultiJvmSpec._
|
import SampleMultiJvmSpec._
|
||||||
|
|
||||||
override def beforeAll() = {
|
override def beforeAll() = {
|
||||||
Cluster.startLocalCluster()
|
Cluster.startLocalCluster()
|
||||||
// resetCluster()
|
|
||||||
}
|
}
|
||||||
|
|
||||||
override def afterAll() = {
|
override def afterAll() = {
|
||||||
|
|
@ -45,23 +44,17 @@ class ClusterMultiJvmNode1 extends WordSpec with MustMatchers with BeforeAndAfte
|
||||||
}
|
}
|
||||||
|
|
||||||
"be able to start all nodes" in {
|
"be able to start all nodes" in {
|
||||||
Cluster.node.barrier("start", NrOfNodes) {
|
Cluster.barrier("start", NrOfNodes) {
|
||||||
// Cluster.node.start()
|
Cluster.node.start()
|
||||||
}
|
}
|
||||||
// Cluster.node.isRunning must be(true)
|
Cluster.node.isRunning must be(true)
|
||||||
}
|
Cluster.node.shutdown()
|
||||||
|
|
||||||
"be able to shutdown all nodes" in {
|
|
||||||
Cluster.node.barrier("shutdown", NrOfNodes) {
|
|
||||||
// Cluster.node.shutdown()
|
|
||||||
}
|
|
||||||
// Cluster.node.isRunning must be(false)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class ClusterMultiJvmNode2 extends WordSpec with MustMatchers {
|
class SampleMultiJvmNode2 extends WordSpec with MustMatchers {
|
||||||
import ClusterMultiJvmSpec._
|
import SampleMultiJvmSpec._
|
||||||
|
|
||||||
"A cluster" must {
|
"A cluster" must {
|
||||||
|
|
||||||
|
|
@ -72,17 +65,11 @@ class ClusterMultiJvmNode2 extends WordSpec with MustMatchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"be able to start all nodes" in {
|
"be able to start all nodes" in {
|
||||||
Cluster.node.barrier("start", NrOfNodes) {
|
Cluster.barrier("start", NrOfNodes) {
|
||||||
// Cluster.node.start()
|
Cluster.node.start()
|
||||||
}
|
}
|
||||||
// Cluster.node.isRunning must be(true)
|
Cluster.node.isRunning must be(true)
|
||||||
}
|
Cluster.node.shutdown()
|
||||||
|
|
||||||
"be able to shutdown all nodes" in {
|
|
||||||
Cluster.node.barrier("shutdown", NrOfNodes) {
|
|
||||||
// Cluster.node.shutdown()
|
|
||||||
}
|
|
||||||
// Cluster.node.isRunning must be(false)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -0,0 +1,4 @@
|
||||||
|
akka.actor.deployment.service-hello.router = "round-robin"
|
||||||
|
akka.actor.deployment.service-hello.clustered.home = "node:node1"
|
||||||
|
akka.actor.deployment.service-hello.clustered.replicas = 2
|
||||||
|
akka.actor.deployment.service-hello.clustered.stateless = on
|
||||||
|
|
@ -0,0 +1 @@
|
||||||
|
-Dakka.cluster.nodename=node1 -Dakka.cluster.port=9991
|
||||||
|
|
@ -0,0 +1,4 @@
|
||||||
|
akka.actor.deployment.service-hello.router = "round-robin"
|
||||||
|
akka.actor.deployment.service-hello.clustered.home = "node:node1"
|
||||||
|
akka.actor.deployment.service-hello.clustered.replicas = 2
|
||||||
|
akka.actor.deployment.service-hello.clustered.stateless = on
|
||||||
|
|
@ -0,0 +1 @@
|
||||||
|
-Dakka.cluster.nodename=node2 -Dakka.cluster.port=9992
|
||||||
|
|
@ -0,0 +1,89 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
|
||||||
|
*/
|
||||||
|
|
||||||
|
package akka.cluster.store_actor
|
||||||
|
|
||||||
|
import org.scalatest.WordSpec
|
||||||
|
import org.scalatest.matchers.MustMatchers
|
||||||
|
import org.scalatest.BeforeAndAfterAll
|
||||||
|
|
||||||
|
import akka.cluster._
|
||||||
|
import akka.actor._
|
||||||
|
import Actor._
|
||||||
|
|
||||||
|
object StoreActorMultiJvmSpec {
|
||||||
|
val NrOfNodes = 2
|
||||||
|
|
||||||
|
class HelloWorld extends Actor with Serializable {
|
||||||
|
def receive = {
|
||||||
|
case "Hello" ⇒ self.reply("World")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
class StoreActorMultiJvmNode1 extends WordSpec with MustMatchers with BeforeAndAfterAll {
|
||||||
|
import StoreActorMultiJvmSpec._
|
||||||
|
|
||||||
|
"A cluster" must {
|
||||||
|
|
||||||
|
"create clustered actor, get a 'local' actor on 'home' node and a 'ref' to actor on remote node" in {
|
||||||
|
System.getProperty("akka.cluster.nodename", "") must be("node1")
|
||||||
|
System.getProperty("akka.cluster.port", "") must be("9991")
|
||||||
|
|
||||||
|
Cluster.barrier("start-node1", NrOfNodes) {
|
||||||
|
Cluster.node.start()
|
||||||
|
}
|
||||||
|
|
||||||
|
Cluster.barrier("start-node2", NrOfNodes) {}
|
||||||
|
|
||||||
|
Cluster.barrier("create-clustered-actor-node1", NrOfNodes) {
|
||||||
|
val pi = Actor.actorOf[HelloWorld]("service-hello")
|
||||||
|
pi must not equal (null)
|
||||||
|
pi.address must equal("service-hello")
|
||||||
|
pi.isInstanceOf[LocalActorRef] must be(true)
|
||||||
|
}
|
||||||
|
|
||||||
|
Cluster.barrier("get-ref-to-actor-on-node2", NrOfNodes) {}
|
||||||
|
|
||||||
|
Cluster.node.shutdown()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
override def beforeAll() = {
|
||||||
|
Cluster.startLocalCluster()
|
||||||
|
}
|
||||||
|
|
||||||
|
override def afterAll() = {
|
||||||
|
Cluster.shutdownLocalCluster()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
class StoreActorMultiJvmNode2 extends WordSpec with MustMatchers {
|
||||||
|
import StoreActorMultiJvmSpec._
|
||||||
|
|
||||||
|
"A cluster" must {
|
||||||
|
|
||||||
|
"create clustered actor, get a 'local' actor on 'home' node and a 'ref' to actor on remote node" in {
|
||||||
|
System.getProperty("akka.cluster.nodename", "") must be("node2")
|
||||||
|
System.getProperty("akka.cluster.port", "") must be("9992")
|
||||||
|
|
||||||
|
Cluster.barrier("start-node1", NrOfNodes) {}
|
||||||
|
|
||||||
|
Cluster.barrier("start-node2", NrOfNodes) {
|
||||||
|
Cluster.node.start()
|
||||||
|
}
|
||||||
|
|
||||||
|
Cluster.barrier("create-clustered-actor-node1", NrOfNodes) {}
|
||||||
|
|
||||||
|
Cluster.barrier("get-ref-to-actor-on-node2", NrOfNodes) {
|
||||||
|
val pi = Actor.actorOf[HelloWorld]("service-hello")
|
||||||
|
pi must not equal (null)
|
||||||
|
pi.address must equal("service-hello")
|
||||||
|
pi.isInstanceOf[ClusterActorRef] must be(true)
|
||||||
|
}
|
||||||
|
|
||||||
|
Cluster.node.shutdown()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -41,19 +41,19 @@ like the following::
|
||||||
|
|
||||||
package example
|
package example
|
||||||
|
|
||||||
object TestMultiJvmNode1 {
|
object SampleMultiJvmNode1 {
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
println("Hello from node 1")
|
println("Hello from node 1")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
object TestMultiJvmNode2 {
|
object SampleMultiJvmNode2 {
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
println("Hello from node 2")
|
println("Hello from node 2")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
object TestMultiJvmNode3 {
|
object SampleMultiJvmNode3 {
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
println("Hello from node 3")
|
println("Hello from node 3")
|
||||||
}
|
}
|
||||||
|
|
@ -68,9 +68,9 @@ spawned, one for each node. It will look like this:
|
||||||
...
|
...
|
||||||
[info] == multi-jvm-run ==
|
[info] == multi-jvm-run ==
|
||||||
[info] == multi-jvm / Test ==
|
[info] == multi-jvm / Test ==
|
||||||
[info] Starting JVM-Node1 for example.TestMultiJvmNode1
|
[info] Starting JVM-Node1 for example.SampleMultiJvmNode1
|
||||||
[info] Starting JVM-Node2 for example.TestMultiJvmNode2
|
[info] Starting JVM-Node2 for example.SampleMultiJvmNode2
|
||||||
[info] Starting JVM-Node3 for example.TestMultiJvmNode3
|
[info] Starting JVM-Node3 for example.SampleMultiJvmNode3
|
||||||
[JVM-Node1] Hello from node 1
|
[JVM-Node1] Hello from node 1
|
||||||
[JVM-Node2] Hello from node 2
|
[JVM-Node2] Hello from node 2
|
||||||
[JVM-Node3] Hello from node 3
|
[JVM-Node3] Hello from node 3
|
||||||
|
|
@ -98,21 +98,22 @@ Setting JVM options
|
||||||
-------------------
|
-------------------
|
||||||
|
|
||||||
You can define specific JVM options for each of the spawned JVMs. You do that by creating
|
You can define specific JVM options for each of the spawned JVMs. You do that by creating
|
||||||
a file named after the node in the test with suffix ``.opts``.
|
a file named after the node in the test with suffix ``.opts`` and put them in the same
|
||||||
|
directory as the test.
|
||||||
|
|
||||||
For example, to feed the JVM options ``-Dakka.cluster.nodename=node1`` and
|
For example, to feed the JVM options ``-Dakka.cluster.nodename=node1`` and
|
||||||
``-Dakka.cluster.port=9991`` to the ``TestMultiJvmNode1`` let's create three ``*.opts`` files
|
``-Dakka.cluster.port=9991`` to the ``SampleMultiJvmNode1`` let's create three ``*.opts`` files
|
||||||
and add the options to them.
|
and add the options to them.
|
||||||
|
|
||||||
``TestMultiJvmNode1.opts``::
|
``SampleMultiJvmNode1.opts``::
|
||||||
|
|
||||||
-Dakka.cluster.nodename=node1 -Dakka.cluster.port=9991
|
-Dakka.cluster.nodename=node1 -Dakka.cluster.port=9991
|
||||||
|
|
||||||
``TestMultiJvmNode2.opts``::
|
``SampleMultiJvmNode2.opts``::
|
||||||
|
|
||||||
-Dakka.cluster.nodename=node2 -Dakka.cluster.port=9992
|
-Dakka.cluster.nodename=node2 -Dakka.cluster.port=9992
|
||||||
|
|
||||||
``TestMultiJvmNode3.opts``::
|
``SampleMultiJvmNode3.opts``::
|
||||||
|
|
||||||
-Dakka.cluster.nodename=node3 -Dakka.cluster.port=9993
|
-Dakka.cluster.nodename=node3 -Dakka.cluster.port=9993
|
||||||
|
|
||||||
|
|
@ -120,20 +121,21 @@ Overriding akka.conf options
|
||||||
----------------------------
|
----------------------------
|
||||||
|
|
||||||
You can also override the options in the ``akka.conf`` file with different options for each
|
You can also override the options in the ``akka.conf`` file with different options for each
|
||||||
spawned JVM. You do that by creating a file named after the node in the test with suffix ``.conf``.
|
spawned JVM. You do that by creating a file named after the node in the test with suffix
|
||||||
|
``.conf`` and put them in the same directory as the test .
|
||||||
|
|
||||||
For example, to override the configuration option ``akka.cluster.name`` let's create three ``*.conf`` files
|
For example, to override the configuration option ``akka.cluster.name`` let's create three
|
||||||
and add the option to them.
|
``*.conf`` files and add the option to them.
|
||||||
|
|
||||||
``TestMultiJvmNode1.conf``::
|
``SampleMultiJvmNode1.conf``::
|
||||||
|
|
||||||
akka.cluster.name = "test-cluster"
|
akka.cluster.name = "test-cluster"
|
||||||
|
|
||||||
``TestMultiJvmNode2.conf``::
|
``SampleMultiJvmNode2.conf``::
|
||||||
|
|
||||||
akka.cluster.name = "test-cluster"
|
akka.cluster.name = "test-cluster"
|
||||||
|
|
||||||
``TestMultiJvmNode3.conf``::
|
``SampleMultiJvmNode3.conf``::
|
||||||
|
|
||||||
akka.cluster.name = "test-cluster"
|
akka.cluster.name = "test-cluster"
|
||||||
|
|
||||||
|
|
@ -175,10 +177,10 @@ Zookeeper Barrier
|
||||||
~~~~~~~~~~~~~~~~~
|
~~~~~~~~~~~~~~~~~
|
||||||
|
|
||||||
When running multi-JVM tests it's common to need to coordinate timing across
|
When running multi-JVM tests it's common to need to coordinate timing across
|
||||||
nodes. To do this there is a Zookeeper-based double-barrier (there is both an
|
nodes. To do this there is a ZooKeeper-based double-barrier (there is both an
|
||||||
entry barrier and an exit barrier). ClusterNodes also have support for creating
|
entry barrier and an exit barrier). ClusterNodes also have support for creating
|
||||||
barriers easily. To wait at the entry use the ``enter`` method. To wait at the
|
barriers easily. To wait at the entry use the ``enter`` method. To wait at the
|
||||||
exit use the ``leave`` method. It's also possible to pass a block of code which
|
exit use the ``leave`` method. It's also possible t pass a block of code which
|
||||||
will be run between the barriers.
|
will be run between the barriers.
|
||||||
|
|
||||||
When creating a barrier you pass it a name and the number of nodes that are
|
When creating a barrier you pass it a name and the number of nodes that are
|
||||||
|
|
@ -188,60 +190,63 @@ timeout is 60 seconds.
|
||||||
Here is an example of coordinating the starting of two nodes and then running
|
Here is an example of coordinating the starting of two nodes and then running
|
||||||
something in coordination::
|
something in coordination::
|
||||||
|
|
||||||
package example
|
import org.scalatest.WordSpec
|
||||||
|
import org.scalatest.matchers.MustMatchers
|
||||||
|
import org.scalatest.BeforeAndAfterAll
|
||||||
|
|
||||||
import akka.cluster._
|
import akka.cluster._
|
||||||
import akka.actor._
|
|
||||||
|
|
||||||
object TestMultiJvmNode1 {
|
object SampleMultiJvmSpec {
|
||||||
val NrOfNodes = 2
|
val NrOfNodes = 2
|
||||||
|
}
|
||||||
|
|
||||||
def main(args: Array[String]) {
|
class SampleMultiJvmNode1 extends WordSpec with MustMatchers with BeforeAndAfterAll {
|
||||||
|
import SampleMultiJvmSpec._
|
||||||
|
|
||||||
|
override def beforeAll() = {
|
||||||
Cluster.startLocalCluster()
|
Cluster.startLocalCluster()
|
||||||
|
|
||||||
val node = Cluster.newNode(NodeAddress("example", "node1", port = 9991))
|
|
||||||
|
|
||||||
node.barrier("start-node1", NrOfNodes) {
|
|
||||||
node.start
|
|
||||||
}
|
}
|
||||||
|
|
||||||
node.barrier("start-node2", NrOfNodes) {
|
override def afterAll() = {
|
||||||
// wait for node 2 to start
|
Cluster.shutdownLocalCluster()
|
||||||
}
|
}
|
||||||
|
|
||||||
node.barrier("hello", NrOfNodes) {
|
"A cluster" must {
|
||||||
println("Hello from node 1")
|
|
||||||
|
"have jvm options" in {
|
||||||
|
System.getProperty("akka.cluster.nodename", "") must be("node1")
|
||||||
|
System.getProperty("akka.cluster.port", "") must be("9991")
|
||||||
|
akka.config.Config.config.getString("test.name", "") must be("node1")
|
||||||
}
|
}
|
||||||
|
|
||||||
Actor.registry.local.shutdownAll
|
"be able to start all nodes" in {
|
||||||
|
Cluster.barrier("start", NrOfNodes) {
|
||||||
node.stop
|
Cluster.node.start()
|
||||||
|
}
|
||||||
Cluster.shutdownLocalCluster
|
Cluster.node.isRunning must be(true)
|
||||||
|
Cluster.node.shutdown()
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
object TestMultiJvmNode2 {
|
class SampleMultiJvmNode2 extends WordSpec with MustMatchers {
|
||||||
val NrOfNodes = 2
|
import SampleMultiJvmSpec._
|
||||||
|
|
||||||
def main(args: Array[String]) {
|
"A cluster" must {
|
||||||
val node = Cluster.newNode(NodeAddress("example", "node2", port = 9992))
|
|
||||||
|
|
||||||
node.barrier("start-node1", NrOfNodes) {
|
"have jvm options" in {
|
||||||
// wait for node 1 to start
|
System.getProperty("akka.cluster.nodename", "") must be("node2")
|
||||||
|
System.getProperty("akka.cluster.port", "") must be("9992")
|
||||||
|
akka.config.Config.config.getString("test.name", "") must be("node2")
|
||||||
}
|
}
|
||||||
|
|
||||||
node.barrier("start-node2", NrOfNodes) {
|
"be able to start all nodes" in {
|
||||||
node.start
|
Cluster.barrier("start", NrOfNodes) {
|
||||||
|
Cluster.node.start()
|
||||||
}
|
}
|
||||||
|
Cluster.node.isRunning must be(true)
|
||||||
node.barrier("hello", NrOfNodes) {
|
Cluster.node.shutdown()
|
||||||
println("Hello from node 2")
|
|
||||||
}
|
}
|
||||||
|
|
||||||
Actor.registry.local.shutdownAll
|
|
||||||
|
|
||||||
node.stop
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -253,8 +258,8 @@ An example output from this would be:
|
||||||
...
|
...
|
||||||
[info] == multi-jvm-run ==
|
[info] == multi-jvm-run ==
|
||||||
[info] == multi-jvm / Test ==
|
[info] == multi-jvm / Test ==
|
||||||
[info] Starting JVM-Node1 for example.TestMultiJvmNode1
|
[info] Starting JVM-Node1 for example.SampleMultiJvmNode1
|
||||||
[info] Starting JVM-Node2 for example.TestMultiJvmNode2
|
[info] Starting JVM-Node2 for example.SampleMultiJvmNode2
|
||||||
[JVM-Node1] Loading config [akka.conf] from the application classpath.
|
[JVM-Node1] Loading config [akka.conf] from the application classpath.
|
||||||
[JVM-Node2] Loading config [akka.conf] from the application classpath.
|
[JVM-Node2] Loading config [akka.conf] from the application classpath.
|
||||||
...
|
...
|
||||||
|
|
|
||||||
|
|
@ -48,7 +48,7 @@ akka {
|
||||||
format = "akka.serialization.Format$Default$"
|
format = "akka.serialization.Format$Default$"
|
||||||
clustered { # makes the actor available in the cluster registry
|
clustered { # makes the actor available in the cluster registry
|
||||||
# default (if omitted) is local non-clustered actor
|
# default (if omitted) is local non-clustered actor
|
||||||
home = "node:test-1" # defines the hostname, IP-address or node name of the "home" node for clustered actor
|
home = "node:node1" # defines the hostname, IP-address or node name of the "home" node for clustered actor
|
||||||
# available: "host:<hostname>", "ip:<ip address>" and "node:<node name>"
|
# available: "host:<hostname>", "ip:<ip address>" and "node:<node name>"
|
||||||
# default is "host:localhost"
|
# default is "host:localhost"
|
||||||
replicas = 3 # number of actor replicas in the cluster
|
replicas = 3 # number of actor replicas in the cluster
|
||||||
|
|
@ -61,17 +61,6 @@ akka {
|
||||||
# default is 'off'
|
# default is 'off'
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
service-pong {} # local actor
|
|
||||||
|
|
||||||
service-hello {
|
|
||||||
router = "round-robin"
|
|
||||||
clustered {
|
|
||||||
home = "host:localhost"
|
|
||||||
replicas = 3
|
|
||||||
stateless = on
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
default-dispatcher {
|
default-dispatcher {
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue