2011-04-27 01:10:00 +02:00
/* *
2011-07-14 16:03:08 +02:00
* Copyright ( C ) 2009 - 2011 Typesafe Inc . < http : //www.typesafe.com>
2011-04-27 01:10:00 +02:00
*/
2011-04-28 20:12:37 +02:00
package akka.cluster
2011-04-27 01:10:00 +02:00
import org.apache.zookeeper._
import org.apache.zookeeper.Watcher.Event._
import org.apache.zookeeper.data.Stat
2011-07-15 09:55:45 +03:00
import org.apache.zookeeper.recipes.lock. { WriteLock , LockListener }
2011-04-27 01:10:00 +02:00
import org.I0Itec.zkclient._
import org.I0Itec.zkclient.serialize._
import org.I0Itec.zkclient.exception._
2011-07-15 09:55:45 +03:00
import java.util. { List ⇒ JList }
import java.util.concurrent.atomic. { AtomicBoolean , AtomicReference }
2011-04-27 01:10:00 +02:00
import java.net.InetSocketAddress
import javax.management.StandardMBean
import scala.collection.mutable.ConcurrentMap
import scala.collection.JavaConversions._
import akka.util._
2011-04-28 20:12:37 +02:00
import Helpers._
2011-06-14 19:35:18 +02:00
2011-04-27 01:10:00 +02:00
import akka.actor._
2011-05-20 09:08:11 +02:00
import Actor._
2011-05-25 16:18:35 +02:00
import Status._
2011-07-04 19:10:06 +02:00
import DeploymentConfig._
2011-06-14 19:35:18 +02:00
2011-04-27 01:10:00 +02:00
import akka.event.EventHandler
2011-07-15 09:55:45 +03:00
import akka.dispatch. { Dispatchers , Future }
2011-04-27 01:10:00 +02:00
import akka.remoteinterface._
2011-05-24 19:04:25 +02:00
import akka.routing.RouterType
2011-06-14 19:35:18 +02:00
2011-07-15 09:55:45 +03:00
import akka.config. { Config , Supervision }
2011-05-30 10:53:25 +02:00
import Supervision._
2011-05-20 09:08:11 +02:00
import Config._
2011-06-14 19:35:18 +02:00
2011-07-15 09:55:45 +03:00
import akka.serialization. { Serialization , Serializer , ActorSerialization }
2011-07-08 19:35:27 +02:00
import ActorSerialization._
2011-07-15 09:55:45 +03:00
import akka.serialization.Compression.LZF
2011-04-27 01:10:00 +02:00
2011-04-28 20:12:37 +02:00
import akka.cluster.zookeeper._
2011-07-08 19:35:27 +02:00
import ChangeListener._
import ClusterProtocol._
import RemoteDaemonMessageType._
2011-04-27 01:10:00 +02:00
import com.eaio.uuid.UUID
import com.google.protobuf.ByteString
2011-07-15 09:55:45 +03:00
import java.util.concurrent. { CopyOnWriteArrayList , Callable , ConcurrentHashMap }
2011-04-27 01:10:00 +02:00
// FIXME add watch for each node that when the entry for the node is removed then the node shuts itself down
/* *
* JMX MBean for the cluster service .
*
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
trait ClusterNodeMBean {
2011-07-13 11:27:16 +03:00
2011-05-18 08:37:58 +02:00
def start ( )
2011-05-18 12:25:27 +02:00
2011-05-18 08:37:58 +02:00
def stop ( )
2011-04-27 01:10:00 +02:00
2011-05-18 08:37:58 +02:00
def disconnect ( )
2011-05-18 12:25:27 +02:00
2011-05-18 08:37:58 +02:00
def reconnect ( )
2011-05-18 12:25:27 +02:00
2011-05-18 08:37:58 +02:00
def resign ( )
2011-04-27 01:10:00 +02:00
def isConnected : Boolean
def getRemoteServerHostname : String
2011-05-18 12:25:27 +02:00
2011-04-27 01:10:00 +02:00
def getRemoteServerPort : Int
def getNodeName : String
2011-05-18 12:25:27 +02:00
2011-04-27 01:10:00 +02:00
def getClusterName : String
2011-05-18 12:25:27 +02:00
2011-04-27 01:10:00 +02:00
def getZooKeeperServerAddresses : String
def getMemberNodes : Array [ String ]
2011-05-18 12:25:27 +02:00
2011-07-15 09:55:45 +03:00
def getNodeAddres ( ) : NodeAddress
2011-07-13 11:27:16 +03:00
2011-07-13 19:22:49 +03:00
def getLeaderLockName : String
2011-04-27 01:10:00 +02:00
2011-07-13 11:27:16 +03:00
def isLeader : Boolean
2011-04-27 01:10:00 +02:00
def getUuidsForClusteredActors : Array [ String ]
2011-05-18 12:25:27 +02:00
2011-04-29 15:47:56 +02:00
def getAddressesForClusteredActors : Array [ String ]
2011-04-27 01:10:00 +02:00
def getUuidsForActorsInUse : Array [ String ]
2011-05-18 12:25:27 +02:00
2011-04-29 15:47:56 +02:00
def getAddressesForActorsInUse : Array [ String ]
2011-04-27 01:10:00 +02:00
2011-04-29 15:47:56 +02:00
def getNodesForActorInUseWithAddress ( address : String ) : Array [ String ]
2011-04-27 01:10:00 +02:00
def getUuidsForActorsInUseOnNode ( nodeName : String ) : Array [ String ]
2011-05-18 12:25:27 +02:00
2011-04-29 15:47:56 +02:00
def getAddressesForActorsInUseOnNode ( nodeName : String ) : Array [ String ]
2011-04-27 01:10:00 +02:00
2011-05-18 08:37:58 +02:00
def setConfigElement ( key : String , value : String )
2011-05-18 12:25:27 +02:00
2011-04-27 01:10:00 +02:00
def getConfigElement ( key : String ) : AnyRef
2011-05-18 12:25:27 +02:00
2011-05-18 08:37:58 +02:00
def removeConfigElement ( key : String )
2011-05-18 12:25:27 +02:00
2011-04-27 01:10:00 +02:00
def getConfigElementKeys : Array [ String ]
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
def getMemberShipPathFor ( node : String ) : String
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
def getConfigurationPathFor ( key : String ) : String
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
def getActorAddresstoNodesPathFor ( actorAddress : String ) : String
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
def getActorAddressToNodesPathForWithNodeName ( actorAddress : String , nodeName : String ) : String
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
def getNodeToUuidsPathFor ( node : String ) : String
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
def getNodeToUuidsPathFor ( node : String , uuid : UUID ) : String
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
def getActorAddressRegistryPathFor ( actorAddress : String ) : String
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
def getActorAddressRegistrySerializerPathFor ( actorAddress : String ) : String
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
def getActorAddressRegistryUuidPathFor ( actorAddress : String ) : String
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
def getActorUuidRegistryNodePathFor ( uuid : UUID ) : String
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
def getActorUuidRegistryRemoteAddressPathFor ( uuid : UUID ) : String
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
def getActorAddressToUuidsPathFor ( actorAddress : String ) : String
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
def getActorAddressToUuidsPathForWithNodeName ( actorAddress : String , uuid : UUID ) : String
2011-04-27 01:10:00 +02:00
}
/* *
2011-07-08 08:28:13 +02:00
* Module for the Cluster . Also holds global state such as configuration data etc .
2011-04-27 01:10:00 +02:00
*
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
object Cluster {
val EMPTY_STRING = "" . intern
// config options
2011-05-23 22:35:01 +02:00
val name = Config . clusterName
2011-05-18 12:25:27 +02:00
val zooKeeperServers = config . getString ( "akka.cluster.zookeeper-server-addresses" , "localhost:2181" )
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 connectionTimeout = Duration ( config . getInt ( "akka.cluster.connection-timeout" , 60 ) , TIME_UNIT ) . toMillis . toInt
2011-04-28 20:12:37 +02:00
val maxTimeToWaitUntilConnected = Duration ( config . getInt ( "akka.cluster.max-time-to-wait-until-connected" , 30 ) , TIME_UNIT ) . toMillis . toInt
2011-05-18 12:25:27 +02:00
val shouldCompressData = config . getBool ( "akka.cluster.use-compression" , false )
val enableJMX = config . getBool ( "akka.enable-jmx" , true )
2011-05-25 16:18:35 +02:00
val remoteDaemonAckTimeout = Duration ( config . getInt ( "akka.cluster.remote-daemon-ack-timeout" , 30 ) , TIME_UNIT ) . toMillis . toInt
2011-07-04 19:10:06 +02:00
val includeRefNodeInReplicaSet = config . getBool ( "akka.cluster.include-ref-node-in-replica-set" , true )
2011-07-08 08:28:13 +02:00
val clusterDirectory = config . getString ( "akka.cluster.log-directory" , "_akka_cluster" )
val clusterDataDirectory = clusterDirectory + "/data"
val clusterLogDirectory = clusterDirectory + "/log"
2011-04-27 01:10:00 +02:00
2011-05-20 09:08:11 +02:00
@volatile
private var properties = Map . empty [ String , String ]
2011-04-27 01:10:00 +02:00
2011-07-04 19:10:06 +02:00
/* *
* Use to override JVM options such as < code >- Dakka . cluster . nodename = node1 </ code > etc .
* Currently supported options are :
* < pre >
* Cluster setProperty ( "akka.cluster.nodename" , "node1" )
* Cluster setProperty ( "akka.cluster.hostname" , "darkstar.lan" )
* Cluster setProperty ( "akka.cluster.port" , "1234" )
* </ pre >
*/
2011-05-20 09:08:11 +02:00
def setProperty ( property : ( String , String ) ) {
properties = properties + property
2011-05-18 08:37:58 +02:00
}
2011-04-27 01:10:00 +02:00
2011-05-21 16:55:32 +02:00
private def nodename : String = properties . get ( "akka.cluster.nodename" ) match {
case Some ( uberride ) ⇒ uberride
2011-07-15 09:55:45 +03:00
case None ⇒ Config . nodename
2011-04-27 01:10:00 +02:00
}
2011-05-21 16:55:32 +02:00
private def hostname : String = properties . get ( "akka.cluster.hostname" ) match {
case Some ( uberride ) ⇒ uberride
2011-07-15 09:55:45 +03:00
case None ⇒ Config . hostname
2011-05-20 09:08:11 +02:00
}
2011-04-27 01:10:00 +02:00
2011-05-21 16:55:32 +02:00
private def port : Int = properties . get ( "akka.cluster.port" ) match {
case Some ( uberride ) ⇒ uberride . toInt
2011-07-15 09:55:45 +03:00
case None ⇒ Config . remoteServerPort
2011-05-20 09:08:11 +02:00
}
2011-04-27 01:10:00 +02:00
2011-07-04 19:10:06 +02:00
val defaultZooKeeperSerializer = new SerializableSerializer
2011-04-27 01:10:00 +02:00
2011-05-20 09:08:11 +02:00
private val _zkServer = new AtomicReference [ Option [ ZkServer ] ] ( None )
2011-04-27 01:10:00 +02:00
/* *
2011-05-20 09:08:11 +02:00
* The node address .
2011-04-27 01:10:00 +02:00
*/
2011-06-26 09:49:03 +02:00
val nodeAddress = NodeAddress ( name , nodename )
2011-04-27 01:10:00 +02:00
/* *
2011-05-20 09:08:11 +02:00
* The reference to the running ClusterNode .
2011-04-27 01:10:00 +02:00
*/
2011-05-23 22:35:01 +02:00
val node = {
2011-05-20 09:08:11 +02:00
if ( nodeAddress eq null ) throw new IllegalArgumentException ( "NodeAddress can't be null" )
2011-07-04 19:10:06 +02:00
new DefaultClusterNode ( nodeAddress , hostname , port , zooKeeperServers , defaultZooKeeperSerializer )
2011-05-20 09:08:11 +02:00
}
2011-04-27 01:10:00 +02:00
/* *
2011-05-20 09:08:11 +02:00
* Looks up the local hostname .
2011-04-27 01:10:00 +02:00
*/
2011-05-20 09:08:11 +02:00
def lookupLocalhostName = NetworkUtil . getLocalhostName
2011-04-27 01:10:00 +02:00
/* *
* Starts up a local ZooKeeper server . Should only be used for testing purposes .
*/
def startLocalCluster ( ) : ZkServer =
2011-07-08 08:28:13 +02:00
startLocalCluster ( clusterDataDirectory , clusterLogDirectory , 2181 , 5000 )
2011-04-27 01:10:00 +02:00
/* *
* Starts up a local ZooKeeper server . Should only be used for testing purposes .
*/
def startLocalCluster ( port : Int , tickTime : Int ) : ZkServer =
2011-07-08 08:28:13 +02:00
startLocalCluster ( clusterDataDirectory , clusterLogDirectory , port , tickTime )
2011-04-27 01:10:00 +02:00
/* *
* Starts up a local ZooKeeper server . Should only be used for testing purposes .
*/
def startLocalCluster ( tickTime : Int ) : ZkServer =
2011-07-08 08:28:13 +02:00
startLocalCluster ( clusterDataDirectory , clusterLogDirectory , 2181 , tickTime )
2011-04-27 01:10:00 +02:00
/* *
* Starts up a local ZooKeeper server . Should only be used for testing purposes .
*/
def startLocalCluster ( dataPath : String , logPath : String ) : ZkServer =
startLocalCluster ( dataPath , logPath , 2181 , 500 )
/* *
* Starts up a local ZooKeeper server . Should only be used for testing purposes .
*/
def startLocalCluster ( dataPath : String , logPath : String , port : Int , tickTime : Int ) : ZkServer = {
try {
val zkServer = AkkaZooKeeper . startLocalServer ( dataPath , logPath , port , tickTime )
_zkServer . set ( Some ( zkServer ) )
zkServer
} catch {
2011-05-18 17:25:30 +02:00
case e : Throwable ⇒
2011-04-27 01:10:00 +02:00
EventHandler . error ( e , this , "Could not start local ZooKeeper cluster" )
throw e
}
}
/* *
* Shut down the local ZooKeeper server .
*/
2011-05-18 08:37:58 +02:00
def shutdownLocalCluster ( ) {
withPrintStackTraceOnError {
EventHandler . info ( this , "Shuts down local cluster" )
_zkServer . get . foreach ( _ . shutdown ( ) )
_zkServer . set ( None )
}
2011-04-27 01:10:00 +02:00
}
/* *
* Creates a new AkkaZkClient .
*/
2011-07-04 19:10:06 +02:00
def newZkClient ( ) : AkkaZkClient = new AkkaZkClient ( zooKeeperServers , sessionTimeout , connectionTimeout , defaultZooKeeperSerializer )
2011-04-27 01:10:00 +02:00
2011-05-23 22:35:01 +02:00
def createQueue ( rootPath : String , blocking : Boolean = true ) = new ZooKeeperQueue ( node . zkClient , rootPath , blocking )
2011-06-27 22:27:40 +03:00
def barrier ( name : String , count : Int ) : ZooKeeperBarrier =
2011-05-23 22:35:01 +02:00
ZooKeeperBarrier ( node . zkClient , node . nodeAddress . clusterName , name , node . nodeAddress . nodeName , count )
2011-06-27 22:27:40 +03:00
def barrier ( name : String , count : Int , timeout : Duration ) : ZooKeeperBarrier =
2011-05-23 22:35:01 +02:00
ZooKeeperBarrier ( node . zkClient , node . nodeAddress . clusterName , name , node . nodeAddress . nodeName , count , timeout )
2011-04-27 01:10:00 +02:00
def uuidToString ( uuid : UUID ) : String = uuid . toString
def stringToUuid ( uuid : String ) : UUID = {
if ( uuid eq null ) throw new ClusterException ( "UUID is null" )
2011-05-18 12:25:27 +02:00
if ( uuid == "" ) throw new ClusterException ( "UUID is an empty string" )
try {
new UUID ( uuid )
2011-05-18 17:25:30 +02:00
} catch {
case e : StringIndexOutOfBoundsException ⇒
2011-04-27 01:10:00 +02:00
val error = new ClusterException ( "UUID not valid [" + uuid + "]" )
EventHandler . error ( error , this , "" )
throw error
}
}
2011-06-27 22:27:40 +03:00
def uuidProtocolToUuid ( uuid : UuidProtocol ) : UUID = new UUID ( uuid . getHigh , uuid . getLow )
2011-04-27 01:10:00 +02:00
2011-06-27 22:27:40 +03:00
def uuidToUuidProtocol ( uuid : UUID ) : UuidProtocol =
2011-04-27 01:10:00 +02:00
UuidProtocol . newBuilder
. setHigh ( uuid . getTime )
. setLow ( uuid . getClockSeqAndNode )
. build
}
/* *
2011-05-18 08:37:58 +02:00
* A Cluster is made up by a bunch of jvm 's, the ClusterNode .
*
2011-07-02 18:18:48 +02:00
* These are the path tree holding the cluster meta - data in ZooKeeper .
*
* Syntax : foo means a variable string , 'foo' means a symbol that does not change and "data" in foo [ data ] means the value ( in bytes ) for the node "foo"
*
* < pre >
* / clusterName / 'members' / nodeName
* / clusterName / 'config' / key [ bytes ]
*
* / clusterName / 'actor- address - to - nodes ' / actorAddress / nodeName
* / clusterName / 'actors- node - to - uuids ' / nodeName / actorUuid
*
* / clusterName / 'actor- address - registry ' / actorAddress / 'serializer' [ serializerName ]
* / clusterName / 'actor- address - registry ' / actorAddress / 'uuid' [ actorUuid ]
*
* / clusterName / 'actor- uuid - registry ' / actorUuid / 'node' [ nodeName ]
* / clusterName / 'actor- uuid - registry ' / actorUuid / 'node' / ip : port
* / clusterName / 'actor- uuid - registry ' / actorUuid / 'address' [ actorAddress ]
*
* / clusterName / 'actor- address - to - uuids ' / actorAddress / actorUuid
* </ pre >
*
2011-04-27 01:10:00 +02:00
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
2011-07-15 09:55:45 +03:00
class DefaultClusterNode private [ akka ] (
val nodeAddress : NodeAddress ,
val hostname : String = Config . hostname ,
val port : Int = Config . remoteServerPort ,
val zkServerAddresses : String ,
val serializer : ZkSerializer ) extends ErrorHandler with ClusterNode {
2011-05-18 17:25:30 +02:00
self ⇒
2011-04-27 01:10:00 +02:00
2011-06-26 09:49:03 +02:00
if ( ( hostname eq null ) || hostname == "" ) throw new NullPointerException ( "Host name must not be null or empty string" )
if ( port < 1 ) throw new NullPointerException ( "Port can not be negative" )
2011-04-27 01:10:00 +02:00
if ( nodeAddress eq null ) throw new IllegalArgumentException ( "'nodeAddress' can not be 'null'" )
2011-06-26 09:49:03 +02:00
val clusterJmxObjectName = JMX . nameFor ( hostname , "monitoring" , "cluster" )
2011-04-27 01:10:00 +02:00
2011-05-23 22:35:01 +02:00
import Cluster._
2011-04-27 01:10:00 +02:00
2011-06-27 11:53:24 +02:00
private [ cluster ] lazy val remoteClientLifeCycleListener = localActorOf ( new Actor {
2011-04-27 01:10:00 +02:00
def receive = {
2011-05-18 17:25:30 +02:00
case RemoteClientError ( cause , client , address ) ⇒ client . shutdownClientModule ( )
case RemoteClientDisconnected ( client , address ) ⇒ client . shutdownClientModule ( )
2011-07-15 09:55:45 +03:00
case _ ⇒ //ignore other
2011-04-27 01:10:00 +02:00
}
2011-05-30 10:53:25 +02:00
} , "akka.cluster.RemoteClientLifeCycleListener" ) . start ( )
2011-07-08 08:28:13 +02:00
private [ cluster ] lazy val remoteDaemon = localActorOf ( new RemoteClusterDaemon ( this ) , RemoteClusterDaemon . Address ) . start ( )
2011-05-30 10:53:25 +02:00
2011-06-27 11:53:24 +02:00
private [ cluster ] lazy val remoteDaemonSupervisor = Supervisor (
2011-05-30 10:53:25 +02:00
SupervisorConfig (
OneForOneStrategy ( List ( classOf [ Exception ] ) , Int . MaxValue , Int . MaxValue ) , // is infinite restart what we want?
Supervise (
remoteDaemon ,
Permanent )
: : Nil ) )
2011-05-19 10:58:30 +02:00
lazy val remoteService : RemoteSupport = {
2011-04-27 01:10:00 +02:00
val remote = new akka . remote . netty . NettyRemoteSupport
2011-06-26 09:49:03 +02:00
remote . start ( hostname , port )
2011-07-08 08:28:13 +02:00
remote . register ( RemoteClusterDaemon . Address , remoteDaemon )
2011-04-27 01:10:00 +02:00
remote . addListener ( remoteClientLifeCycleListener )
remote
}
2011-05-30 10:53:25 +02:00
2011-05-19 10:58:30 +02:00
lazy val remoteServerAddress : InetSocketAddress = remoteService . address
2011-04-27 01:10:00 +02:00
2011-07-15 09:55:45 +03:00
val isConnected = new Switch ( false )
2011-04-27 01:10:00 +02:00
// static nodes
2011-06-22 09:59:00 +02:00
val CLUSTER_PATH = "/" + nodeAddress . clusterName
val MEMBERSHIP_PATH = CLUSTER_PATH + "/members"
val CONFIGURATION_PATH = CLUSTER_PATH + "/config"
val PROVISIONING_PATH = CLUSTER_PATH + "/provisioning"
2011-07-02 18:18:48 +02:00
val ACTOR_ADDRESS_NODES_TO_PATH = CLUSTER_PATH + "/actor-address-to-nodes"
val ACTOR_ADDRESS_REGISTRY_PATH = CLUSTER_PATH + "/actor-address-registry"
val ACTOR_UUID_REGISTRY_PATH = CLUSTER_PATH + "/actor-uuid-registry"
2011-06-22 09:59:00 +02:00
val ACTOR_ADDRESS_TO_UUIDS_PATH = CLUSTER_PATH + "/actor-address-to-uuids"
2011-07-02 18:18:48 +02:00
val NODE_TO_ACTOR_UUIDS_PATH = CLUSTER_PATH + "/node-to-actors-uuids"
2011-06-22 09:59:00 +02:00
val basePaths = List (
CLUSTER_PATH ,
MEMBERSHIP_PATH ,
2011-07-02 18:18:48 +02:00
ACTOR_ADDRESS_REGISTRY_PATH ,
ACTOR_UUID_REGISTRY_PATH ,
ACTOR_ADDRESS_NODES_TO_PATH ,
NODE_TO_ACTOR_UUIDS_PATH ,
2011-06-22 09:59:00 +02:00
ACTOR_ADDRESS_TO_UUIDS_PATH ,
CONFIGURATION_PATH ,
PROVISIONING_PATH )
val LEADER_ELECTION_PATH = CLUSTER_PATH + "/leader" // should NOT be part of 'basePaths' only used by 'leaderLock'
2011-04-27 01:10:00 +02:00
2011-05-18 12:25:27 +02:00
private val membershipNodePath = membershipPathFor ( nodeAddress . nodeName )
2011-04-27 01:10:00 +02:00
def membershipNodes : Array [ String ] = locallyCachedMembershipNodes . toList . toArray . asInstanceOf [ Array [ String ] ]
2011-07-02 18:18:48 +02:00
private [ akka ] val nodeConnections : ConcurrentMap [ String , Tuple2 [ InetSocketAddress , ActorRef ] ] = {
val conns = new ConcurrentHashMap [ String , Tuple2 [ InetSocketAddress , ActorRef ] ]
2011-07-04 19:10:06 +02:00
if ( includeRefNodeInReplicaSet )
conns . put ( nodeAddress . nodeName , ( remoteServerAddress , remoteDaemon ) ) // add the remote connection to 'this' node as well, but as a 'local' actor
2011-07-02 18:18:48 +02:00
conns
}
2011-04-27 01:10:00 +02:00
// zookeeper listeners
2011-05-18 12:25:27 +02:00
private val stateListener = new StateListener ( this )
private val membershipListener = new MembershipChildListener ( this )
2011-04-27 01:10:00 +02:00
// cluster node listeners
2011-05-18 12:25:27 +02:00
private val changeListeners = new CopyOnWriteArrayList [ ChangeListener ] ( )
2011-04-27 01:10:00 +02:00
// Address -> ClusterActorRef
2011-05-18 12:25:27 +02:00
private val clusterActorRefs = new Index [ InetSocketAddress , ClusterActorRef ]
2011-04-27 01:10:00 +02:00
2011-07-02 18:18:48 +02:00
// ZooKeeper client
2011-05-23 22:35:01 +02:00
lazy private [ cluster ] val zkClient = new AkkaZkClient ( zkServerAddresses , sessionTimeout , connectionTimeout , serializer )
2011-04-27 01:10:00 +02:00
2011-07-02 18:18:48 +02:00
// leader election listener, registered to the 'leaderLock' below
2011-05-23 22:35:01 +02:00
lazy private [ cluster ] val leaderElectionCallback = new LockListener {
2011-05-18 08:37:58 +02:00
override def lockAcquired ( ) {
2011-04-27 01:10:00 +02:00
EventHandler . info ( this , "Node [%s] is the new leader" . format ( self . nodeAddress . nodeName ) )
2011-05-23 22:35:01 +02:00
self . publish ( NewLeader ( self . nodeAddress . nodeName ) )
2011-04-27 01:10:00 +02:00
}
2011-05-18 08:37:58 +02:00
override def lockReleased ( ) {
2011-06-22 09:59:00 +02:00
EventHandler . info ( this , "Node [%s] is *NOT* the leader anymore" . format ( self . nodeAddress . nodeName ) )
2011-04-27 01:10:00 +02:00
}
}
2011-07-02 18:18:48 +02:00
// leader election lock in ZooKeeper
2011-05-23 22:35:01 +02:00
lazy private [ cluster ] val leaderLock = new WriteLock (
2011-06-22 09:59:00 +02:00
zkClient . connection . getZookeeper ,
LEADER_ELECTION_PATH , null ,
leaderElectionCallback )
2011-04-27 01:10:00 +02:00
if ( enableJMX ) createMBean
// =======================================
// Node
// =======================================
2011-07-15 09:55:45 +03:00
def isRunning : Boolean = isConnected . isOn
2011-04-27 01:10:00 +02:00
def start ( ) : ClusterNode = {
2011-07-15 09:55:45 +03:00
isConnected . switchOn {
2011-05-18 08:37:58 +02:00
initializeNode ( )
2011-04-27 01:10:00 +02:00
}
2011-07-15 09:55:45 +03:00
2011-04-27 01:10:00 +02:00
this
}
2011-07-15 09:55:45 +03:00
private [ cluster ] def initializeNode ( ) {
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 , port , zkServerAddresses , serializer ) )
EventHandler . info ( this , "Starting up remote server [%s]" . format ( remoteServerAddress . toString ) )
createZooKeeperPathStructureIfNeeded ( )
registerListeners ( )
joinCluster ( )
joinLeaderElection ( )
fetchMembershipNodes ( )
EventHandler . info ( this , "Cluster node [%s] started successfully" . format ( nodeAddress ) )
}
2011-05-20 14:36:26 +02:00
def shutdown ( ) {
2011-07-15 10:59:29 +03:00
def shutdownNode ( ) {
ignore [ ZkNoNodeException ] ( zkClient . deleteRecursive ( membershipNodePath ) )
2011-04-27 01:10:00 +02:00
2011-07-15 10:59:29 +03:00
locallyCachedMembershipNodes . clear ( )
2011-04-27 01:10:00 +02:00
2011-07-15 10:59:29 +03:00
nodeConnections . toList . foreach ( {
case ( _ , ( address , _ ) ) ⇒
Actor . remote . shutdownClientConnection ( address ) // shut down client connections
} )
2011-04-27 01:10:00 +02:00
2011-07-15 10:59:29 +03:00
remoteService . shutdown ( ) // shutdown server
2011-04-27 01:10:00 +02:00
2011-07-15 10:59:29 +03:00
remoteClientLifeCycleListener . stop ( )
remoteDaemon . stop ( )
2011-04-27 01:10:00 +02:00
2011-07-15 10:59:29 +03:00
// for monitoring remote listener
registry . local . actors . filter ( remoteService . hasListener ) . foreach ( _ . stop ( ) )
2011-04-27 01:10:00 +02:00
2011-07-15 10:59:29 +03:00
nodeConnections . clear ( )
2011-04-27 01:10:00 +02:00
2011-07-15 10:59:29 +03:00
disconnect ( )
EventHandler . info ( this , "Cluster node shut down [%s]" . format ( nodeAddress ) )
}
2011-07-15 09:55:45 +03:00
2011-07-15 10:59:29 +03:00
isConnected . switchOff {
shutdownNode ( )
}
2011-04-27 01:10:00 +02:00
}
def disconnect ( ) : ClusterNode = {
2011-05-18 08:37:58 +02:00
zkClient . unsubscribeAll ( )
zkClient . close ( )
2011-04-27 01:10:00 +02:00
this
}
def reconnect ( ) : ClusterNode = {
2011-05-18 08:37:58 +02:00
zkClient . reconnect ( )
2011-04-27 01:10:00 +02:00
this
}
// =======================================
// Change notification
// =======================================
/* *
* Registers a cluster change listener .
*/
2011-06-17 16:07:41 +02:00
def register ( listener : ChangeListener ) : ClusterNode = {
2011-04-27 01:10:00 +02:00
changeListeners . add ( listener )
this
2011-06-17 16:07:41 +02:00
}
2011-04-27 01:10:00 +02:00
2011-05-18 08:37:58 +02:00
private [ cluster ] def publish ( change : ChangeNotification ) {
changeListeners . iterator . foreach ( _ . notify ( change , this ) )
}
2011-04-27 01:10:00 +02:00
// =======================================
// Leader
// =======================================
/* *
2011-06-22 09:59:00 +02:00
* Returns the name of the current leader lock .
*/
def leader : String = leaderLock . getId
/* *
* Returns true if 'this' node is the current leader .
2011-04-27 01:10:00 +02:00
*/
2011-06-22 09:59:00 +02:00
def isLeader : Boolean = leaderLock . isOwner
2011-04-27 01:10:00 +02:00
/* *
* Explicitly resign from being a leader . If this node is not a leader then this operation is a no - op .
*/
2011-05-18 12:25:27 +02:00
def resign ( ) {
2011-06-22 09:59:00 +02:00
if ( isLeader ) leaderLock . unlock ( )
2011-05-18 12:25:27 +02:00
}
2011-04-27 01:10:00 +02:00
// =======================================
// Actor
// =======================================
/* *
* 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 .
*/
2011-07-02 18:18:48 +02:00
def store [ T <: Actor ] ( actorAddress : String , actorClass : Class [ T ] , serializer : Serializer ) : ClusterNode =
store ( actorAddress , ( ) ⇒ Actor . actorOf ( actorClass , actorAddress ) . start , 0 , Transient , false , serializer )
2011-06-07 11:10:29 -07:00
/* *
* 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 .
*/
2011-07-02 18:18:48 +02:00
def store [ T <: Actor ] ( actorAddress : String , actorClass : Class [ T ] , replicationScheme : ReplicationScheme , serializer : Serializer ) : ClusterNode =
store ( actorAddress , ( ) ⇒ Actor . actorOf ( actorClass , actorAddress ) . start , 0 , replicationScheme , false , serializer )
2011-04-27 01:10:00 +02:00
/* *
* 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 .
*/
2011-07-02 18:18:48 +02:00
def store [ T <: Actor ] ( actorAddress : String , actorClass : Class [ T ] , replicationFactor : Int , serializer : Serializer ) : ClusterNode =
store ( actorAddress , ( ) ⇒ Actor . actorOf ( actorClass , actorAddress ) . start , replicationFactor , Transient , false , serializer )
2011-06-07 11:10:29 -07:00
/* *
* 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 .
*/
2011-07-02 18:18:48 +02:00
def store [ T <: Actor ] ( actorAddress : String , actorClass : Class [ T ] , replicationFactor : Int , replicationScheme : ReplicationScheme , serializer : Serializer ) : ClusterNode =
store ( actorAddress , ( ) ⇒ Actor . actorOf ( actorClass , actorAddress ) . start , replicationFactor , replicationScheme , false , serializer )
2011-04-27 01:10:00 +02:00
/* *
* 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 .
*/
2011-07-02 18:18:48 +02:00
def store [ T <: Actor ] ( actorAddress : String , actorClass : Class [ T ] , serializeMailbox : Boolean , serializer : Serializer ) : ClusterNode =
store ( actorAddress , ( ) ⇒ Actor . actorOf ( actorClass , actorAddress ) . start , 0 , Transient , serializeMailbox , serializer )
2011-06-07 11:10:29 -07:00
/* *
* 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 .
*/
2011-07-02 18:18:48 +02:00
def store [ T <: Actor ] ( actorAddress : String , actorClass : Class [ T ] , replicationScheme : ReplicationScheme , serializeMailbox : Boolean , serializer : Serializer ) : ClusterNode =
store ( actorAddress , ( ) ⇒ Actor . actorOf ( actorClass , actorAddress ) . start , 0 , replicationScheme , serializeMailbox , serializer )
2011-04-27 01:10:00 +02:00
/* *
* 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 .
*/
2011-07-02 18:18:48 +02:00
def store [ T <: Actor ] ( actorAddress : String , actorClass : Class [ T ] , replicationFactor : Int , serializeMailbox : Boolean , serializer : Serializer ) : ClusterNode =
store ( actorAddress , ( ) ⇒ Actor . actorOf ( actorClass , actorAddress ) . start , replicationFactor , Transient , serializeMailbox , serializer )
2011-06-07 11:10:29 -07:00
/* *
* 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 .
*/
2011-07-02 18:18:48 +02:00
def store [ T <: Actor ] ( actorAddress : String , actorClass : Class [ T ] , replicationFactor : Int , replicationScheme : ReplicationScheme , serializeMailbox : Boolean , serializer : Serializer ) : ClusterNode =
store ( actorAddress , ( ) ⇒ Actor . actorOf ( actorClass , actorAddress ) . start , replicationFactor , replicationScheme , serializeMailbox , serializer )
2011-04-27 01:10:00 +02:00
/* *
* 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 .
*/
2011-07-02 18:18:48 +02:00
def store ( actorAddress : String , actorFactory : ( ) ⇒ ActorRef , serializer : Serializer ) : ClusterNode =
store ( actorAddress , actorFactory , 0 , Transient , false , serializer )
2011-06-07 11:10:29 -07:00
/* *
* 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 .
*/
2011-07-02 18:18:48 +02:00
def store ( actorAddress : String , actorFactory : ( ) ⇒ ActorRef , serializeMailbox : Boolean , serializer : Serializer ) : ClusterNode =
store ( actorAddress , actorFactory , 0 , Transient , serializeMailbox , serializer )
2011-04-27 01:10:00 +02:00
/* *
* 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 .
*/
2011-07-02 18:18:48 +02:00
def store ( actorAddress : String , actorFactory : ( ) ⇒ ActorRef , replicationScheme : ReplicationScheme , serializer : Serializer ) : ClusterNode =
store ( actorAddress , actorFactory , 0 , replicationScheme , false , serializer )
2011-06-07 11:10:29 -07:00
/* *
* 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 .
*/
2011-07-02 18:18:48 +02:00
def store ( actorAddress : String , actorFactory : ( ) ⇒ ActorRef , replicationFactor : Int , serializer : Serializer ) : ClusterNode =
store ( actorAddress , actorFactory , replicationFactor , Transient , false , serializer )
2011-04-27 01:10:00 +02:00
/* *
* 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 .
*/
2011-07-02 18:18:48 +02:00
def store ( actorAddress : String , actorFactory : ( ) ⇒ ActorRef , replicationFactor : Int , replicationScheme : ReplicationScheme , serializer : Serializer ) : ClusterNode =
store ( actorAddress , actorFactory , replicationFactor , replicationScheme , false , serializer )
2011-06-07 11:10:29 -07:00
/* *
* 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 .
*/
2011-07-02 18:18:48 +02:00
def store ( actorAddress : String , actorFactory : ( ) ⇒ ActorRef , replicationFactor : Int , serializeMailbox : Boolean , serializer : Serializer ) : ClusterNode =
store ( actorAddress , actorFactory , replicationFactor , Transient , serializeMailbox , serializer )
2011-06-07 11:10:29 -07:00
/* *
* 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 .
*/
2011-07-02 18:18:48 +02:00
def store ( actorAddress : String , actorFactory : ( ) ⇒ ActorRef , replicationScheme : ReplicationScheme , serializeMailbox : Boolean , serializer : Serializer ) : ClusterNode =
store ( actorAddress , actorFactory , 0 , replicationScheme , serializeMailbox , serializer )
2011-06-07 11:10:29 -07:00
/* *
* Needed to have reflection through structural typing work .
*/
2011-07-02 18:18:48 +02:00
def store ( actorAddress : String , actorFactory : ( ) ⇒ ActorRef , replicationFactor : Int , replicationScheme : ReplicationScheme , serializeMailbox : Boolean , serializer : AnyRef ) : ClusterNode =
store ( actorAddress , actorFactory , replicationFactor , replicationScheme , serializeMailbox , serializer . asInstanceOf [ Serializer ] )
2011-05-20 09:08:11 +02:00
/* *
* Needed to have reflection through structural typing work .
*/
2011-07-02 18:18:48 +02:00
def store ( actorAddress : String , actorFactory : ( ) ⇒ ActorRef , replicationFactor : Int , serializeMailbox : Boolean , serializer : AnyRef ) : ClusterNode =
store ( actorAddress , actorFactory , replicationFactor , Transient , serializeMailbox , serializer )
2011-04-27 01:10:00 +02:00
/* *
2011-07-02 18:18:48 +02:00
* Clusters an actor . If the actor is already clustered then the clustered version will be updated
2011-04-27 01:10:00 +02:00
* with the actor passed in as argument . You can use this to save off snapshots of the actor to a highly
* available durable store .
*/
2011-06-07 11:10:29 -07:00
def store (
2011-07-15 09:55:45 +03:00
actorAddress : String ,
actorFactory : ( ) ⇒ ActorRef ,
replicationFactor : Int ,
replicationScheme : ReplicationScheme ,
serializeMailbox : Boolean ,
serializer : Serializer ) : ClusterNode = if ( isConnected . isOn ) {
2011-04-27 01:10:00 +02:00
2011-06-22 09:59:00 +02:00
val serializerClassName = serializer . getClass . getName
2011-04-27 01:10:00 +02:00
EventHandler . debug ( this ,
2011-07-02 18:18:48 +02:00
"Storing actor with address [%s] in cluster" . format ( actorAddress ) )
val actorFactoryBytes =
Serialization . serialize ( actorFactory ) match {
case Left ( error ) ⇒ throw error
case Right ( bytes ) ⇒
if ( shouldCompressData ) LZF . compress ( bytes )
else bytes
}
2011-06-07 11:10:29 -07:00
2011-07-02 18:18:48 +02:00
val actorAddressRegistryPath = actorAddressRegistryPathFor ( actorAddress )
2011-04-27 01:10:00 +02:00
2011-07-02 18:18:48 +02:00
// create ADDRESS -> Array[Byte] for actor registry
2011-06-17 10:25:02 +02:00
try {
2011-07-04 19:10:06 +02:00
zkClient . writeData ( actorAddressRegistryPath , actorFactoryBytes )
2011-06-17 10:25:02 +02:00
} catch {
case e : ZkNoNodeException ⇒ // if not stored yet, store the actor
zkClient . retryUntilConnected ( new Callable [ Either [ String , Exception ] ] ( ) {
def call : Either [ String , Exception ] = {
try {
2011-07-02 18:18:48 +02:00
Left ( zkClient . connection . create ( actorAddressRegistryPath , actorFactoryBytes , CreateMode . PERSISTENT ) )
2011-06-17 10:25:02 +02:00
} catch {
case e : KeeperException . NodeExistsException ⇒ Right ( e )
}
2011-04-27 01:10:00 +02:00
}
2011-06-17 10:25:02 +02:00
} ) match {
2011-07-15 09:55:45 +03:00
case Left ( path ) ⇒ path
2011-07-02 18:18:48 +02:00
case Right ( exception ) ⇒ actorAddressRegistryPath
2011-06-17 10:25:02 +02:00
}
2011-07-08 08:28:13 +02:00
}
2011-04-27 01:10:00 +02:00
2011-07-08 08:28:13 +02:00
// create ADDRESS -> SERIALIZER CLASS NAME mapping
try {
zkClient . createPersistent ( actorAddressRegistrySerializerPathFor ( actorAddress ) , serializerClassName )
} catch {
case e : ZkNodeExistsException ⇒ zkClient . writeData ( actorAddressRegistrySerializerPathFor ( actorAddress ) , serializerClassName )
}
2011-04-27 01:10:00 +02:00
2011-07-08 08:28:13 +02:00
// create ADDRESS -> NODE mapping
ignore [ ZkNodeExistsException ] ( zkClient . createPersistent ( actorAddressToNodesPathFor ( actorAddress ) ) )
2011-04-27 01:10:00 +02:00
2011-07-08 08:28:13 +02:00
// create ADDRESS -> UUIDs mapping
ignore [ ZkNodeExistsException ] ( zkClient . createPersistent ( actorAddressToUuidsPathFor ( actorAddress ) ) )
2011-04-27 01:10:00 +02:00
2011-07-04 19:10:06 +02:00
useActorOnNodes ( nodesForReplicationFactor ( replicationFactor , Some ( actorAddress ) ) . toArray , actorAddress )
2011-04-27 01:10:00 +02:00
this
} else throw new ClusterException ( "Not connected to cluster" )
2011-06-22 11:31:01 +02:00
/* *
* Removes actor from the cluster .
*/
2011-07-02 18:18:48 +02:00
// def remove(actorRef: ActorRef) {
// remove(actorRef.address)
// }
2011-06-22 11:31:01 +02:00
2011-04-27 01:10:00 +02:00
/* *
2011-04-29 15:47:56 +02:00
* Removes actor with uuid from the cluster .
2011-04-27 01:10:00 +02:00
*/
2011-07-02 18:18:48 +02:00
// def remove(actorAddress: String) {
// releaseActorOnAllNodes(actorAddress)
// // warning: ordering matters here
// // FIXME remove ADDRESS to UUID mapping?
// ignore[ZkNoNodeException](zkClient.deleteRecursive(actorAddressToUuidsPathFor(actorAddress)))
// ignore[ZkNoNodeException](zkClient.deleteRecursive(actorAddressRegistryPathFor(actorAddress)))
// ignore[ZkNoNodeException](zkClient.deleteRecursive(actorAddressToNodesPathFor(actorAddress)))
// }
2011-04-27 01:10:00 +02:00
/* *
* Is the actor with uuid clustered or not ?
*/
2011-07-15 09:55:45 +03:00
def isClustered ( actorAddress : String ) : Boolean = if ( isConnected . isOn ) {
2011-07-02 18:18:48 +02:00
zkClient . exists ( actorAddressRegistryPathFor ( actorAddress ) )
2011-04-27 01:10:00 +02:00
} else false
/* *
* Is the actor with uuid in use on 'this' node or not ?
*/
2011-04-29 15:47:56 +02:00
def isInUseOnNode ( actorAddress : String ) : Boolean = isInUseOnNode ( actorAddress , nodeAddress )
2011-04-27 01:10:00 +02:00
/* *
* Is the actor with uuid in use or not ?
*/
2011-07-15 09:55:45 +03:00
def isInUseOnNode ( actorAddress : String , node : NodeAddress ) : Boolean = if ( isConnected . isOn ) {
2011-07-02 18:18:48 +02:00
zkClient . exists ( actorAddressToNodesPathFor ( actorAddress , node . nodeName ) )
2011-04-27 01:10:00 +02:00
} else false
/* *
* 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 .
*/
2011-07-08 19:35:27 +02:00
def use [ T <: Actor ] ( actorAddress : String ) : Option [ LocalActorRef ] = use ( actorAddress , serializerForActor ( actorAddress ) )
2011-05-20 17:13:39 +02:00
/* *
* 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 .
*/
2011-07-15 09:55:45 +03:00
def use [ T <: Actor ] ( actorAddress : String , serializer : Serializer ) : Option [ LocalActorRef ] = if ( isConnected . isOn ) {
2011-07-02 18:18:48 +02:00
val nodeName = nodeAddress . nodeName
2011-04-27 01:10:00 +02:00
2011-07-02 18:18:48 +02:00
ignore [ ZkNodeExistsException ] ( zkClient . createEphemeral ( actorAddressToNodesPathFor ( actorAddress , nodeName ) ) )
2011-04-27 01:10:00 +02:00
2011-07-02 18:18:48 +02:00
val actorFactoryPath = actorAddressRegistryPathFor ( actorAddress )
2011-07-08 19:35:27 +02:00
zkClient . retryUntilConnected ( new Callable [ Either [ Exception , ( ) ⇒ LocalActorRef ] ] ( ) {
def call : Either [ Exception , ( ) ⇒ LocalActorRef ] = {
2011-07-02 18:18:48 +02:00
try {
2011-04-27 01:10:00 +02:00
2011-07-02 18:18:48 +02:00
val actorFactoryBytes =
if ( shouldCompressData ) LZF . uncompress ( zkClient . connection . readData ( actorFactoryPath , new Stat , false ) )
else zkClient . connection . readData ( actorFactoryPath , new Stat , false )
2011-04-27 01:10:00 +02:00
2011-07-02 18:18:48 +02:00
val actorFactory =
2011-07-08 19:35:27 +02:00
Serialization . deserialize ( actorFactoryBytes , classOf [ ( ) ⇒ LocalActorRef ] , None ) match {
2011-07-15 09:55:45 +03:00
case Left ( error ) ⇒ throw error
2011-07-08 19:35:27 +02:00
case Right ( instance ) ⇒ instance . asInstanceOf [ ( ) ⇒ LocalActorRef ]
2011-07-02 18:18:48 +02:00
}
Right ( actorFactory )
} catch {
case e : KeeperException . NoNodeException ⇒ Left ( e )
2011-05-18 12:25:27 +02:00
}
2011-05-18 17:25:30 +02:00
}
2011-07-02 18:18:48 +02:00
} ) match {
case Left ( exception ) ⇒ throw exception
case Right ( actorFactory ) ⇒
val actorRef = actorFactory ( )
EventHandler . debug ( this ,
"Checking out actor [%s] to be used on node [%s] as local actor"
. format ( actorAddress , nodeName ) )
val uuid = actorRef . uuid
// create UUID registry
ignore [ ZkNodeExistsException ] ( zkClient . createPersistent ( actorUuidRegistryPathFor ( uuid ) ) )
// create UUID -> NODE mapping
try {
zkClient . createEphemeral ( actorUuidRegistryNodePathFor ( uuid ) , nodeName )
} catch {
case e : ZkNodeExistsException ⇒ zkClient . writeData ( actorUuidRegistryNodePathFor ( uuid ) , nodeName )
}
// create UUID -> ADDRESS
try {
zkClient . createEphemeral ( actorUuidRegistryAddressPathFor ( uuid ) , actorAddress )
} catch {
case e : ZkNodeExistsException ⇒ zkClient . writeData ( actorUuidRegistryAddressPathFor ( uuid ) , actorAddress )
}
// create UUID -> REMOTE ADDRESS (InetSocketAddress) mapping
try {
zkClient . createEphemeral ( actorUuidRegistryRemoteAddressPathFor ( uuid ) , remoteServerAddress )
} catch {
case e : ZkNodeExistsException ⇒ zkClient . writeData ( actorUuidRegistryRemoteAddressPathFor ( uuid ) , remoteServerAddress )
}
// create ADDRESS -> UUID mapping
try {
zkClient . createPersistent ( actorAddressRegistryUuidPathFor ( actorAddress ) , uuid )
} catch {
case e : ZkNodeExistsException ⇒ zkClient . writeData ( actorAddressRegistryUuidPathFor ( actorAddress ) , uuid )
}
// create NODE -> UUID mapping
ignore [ ZkNodeExistsException ] ( zkClient . createPersistent ( nodeToUuidsPathFor ( nodeName , uuid ) , true ) )
// create ADDRESS -> UUIDs mapping
ignore [ ZkNodeExistsException ] ( zkClient . createPersistent ( actorAddressToUuidsPathFor ( actorAddress , uuid ) ) )
actorRef . start ( )
actorRef
}
2011-05-23 22:35:01 +02:00
} else None
2011-04-27 01:10:00 +02:00
/* *
2011-07-04 19:10:06 +02:00
* Using ( checking out ) actor on a specific set of nodes .
2011-04-27 01:10:00 +02:00
*/
2011-07-08 08:28:13 +02:00
def useActorOnNodes ( nodes : Array [ String ] , actorAddress : String , replicateFromUuid : Option [ UUID ] = None ) {
2011-07-04 19:10:06 +02:00
EventHandler . debug ( this ,
"Sending command to nodes [%s] for checking out actor [%s]" . format ( nodes . mkString ( ", " ) , actorAddress ) )
2011-07-15 09:55:45 +03:00
if ( isConnected . isOn ) {
2011-06-26 09:49:03 +02:00
2011-07-08 08:28:13 +02:00
val builder = RemoteDaemonMessageProtocol . newBuilder
2011-05-18 08:37:58 +02:00
. setMessageType ( USE )
2011-07-02 18:18:48 +02:00
. setActorAddress ( actorAddress )
2011-07-08 08:28:13 +02:00
// set the UUID to replicated from - if available
replicateFromUuid foreach ( uuid ⇒ builder . setReplicateActorFromUuid ( uuidToUuidProtocol ( uuid ) ) )
val command = builder . build
2011-06-07 20:10:08 -07:00
2011-07-15 09:55:45 +03:00
nodes foreach {
node ⇒
nodeConnections . get ( node ) foreach {
case ( _ , connection ) ⇒
sendCommandToNode ( connection , command , async = false )
}
2011-04-27 01:10:00 +02:00
}
}
}
2011-07-04 19:10:06 +02:00
/* *
* Using ( checking out ) actor on all nodes in the cluster .
*/
2011-07-08 08:28:13 +02:00
def useActorOnAllNodes ( actorAddress : String , replicateFromUuid : Option [ UUID ] = None ) {
useActorOnNodes ( membershipNodes , actorAddress , replicateFromUuid )
2011-07-04 19:10:06 +02:00
}
2011-04-27 01:10:00 +02:00
/* *
2011-07-02 18:18:48 +02:00
* Using ( checking out ) actor on a specific node .
2011-04-27 01:10:00 +02:00
*/
2011-07-08 08:28:13 +02:00
def useActorOnNode ( node : String , actorAddress : String , replicateFromUuid : Option [ UUID ] = None ) {
useActorOnNodes ( Array ( node ) , actorAddress , replicateFromUuid )
2011-04-27 01:10:00 +02:00
}
2011-06-22 11:31:01 +02:00
/* *
* Checks in an actor after done using it on this node .
*/
def release ( actorRef : ActorRef ) {
release ( actorRef . address )
}
2011-04-27 01:10:00 +02:00
/* *
* Checks in an actor after done using it on this node .
*/
2011-05-18 08:37:58 +02:00
def release ( actorAddress : String ) {
2011-06-22 11:31:01 +02:00
// FIXME 'Cluster.release' needs to notify all existing ClusterActorRef's that are using the instance that it is no longer available. Then what to do? Should we even remove this method?
2011-07-15 09:55:45 +03:00
if ( isConnected . isOn ) {
2011-07-02 18:18:48 +02:00
ignore [ ZkNoNodeException ] ( zkClient . delete ( actorAddressToNodesPathFor ( actorAddress , nodeAddress . nodeName ) ) )
2011-07-15 09:55:45 +03:00
uuidsForActorAddress ( actorAddress ) foreach {
uuid ⇒
EventHandler . debug ( this ,
"Releasing actor [%s] with UUID [%s] after usage" . format ( actorAddress , uuid ) )
2011-07-02 18:18:48 +02:00
2011-07-15 09:55:45 +03:00
ignore [ ZkNoNodeException ] ( zkClient . deleteRecursive ( nodeToUuidsPathFor ( nodeAddress . nodeName , uuid ) ) )
ignore [ ZkNoNodeException ] ( zkClient . delete ( actorUuidRegistryRemoteAddressPathFor ( uuid ) ) )
2011-05-18 08:37:58 +02:00
}
2011-04-27 01:10:00 +02:00
}
}
/* *
2011-07-02 18:18:48 +02:00
* Releases ( checking in ) all actors with a specific address on all nodes in the cluster where the actor is in 'use' .
2011-04-27 01:10:00 +02:00
*/
2011-07-02 18:18:48 +02:00
private [ akka ] def releaseActorOnAllNodes ( actorAddress : String ) {
2011-07-15 09:55:45 +03:00
if ( isConnected . isOn ) {
2011-05-18 08:37:58 +02:00
EventHandler . debug ( this ,
2011-07-02 18:18:48 +02:00
"Releasing (checking in) all actors with address [%s] on all nodes in cluster" . format ( actorAddress ) )
2011-06-26 09:49:03 +02:00
2011-05-18 08:37:58 +02:00
val command = RemoteDaemonMessageProtocol . newBuilder
. setMessageType ( RELEASE )
2011-07-04 19:10:06 +02:00
. setActorAddress ( actorAddress )
2011-05-18 08:37:58 +02:00
. build
2011-06-26 09:49:03 +02:00
2011-07-15 09:55:45 +03:00
nodesForActorsInUseWithAddress ( actorAddress ) foreach {
node ⇒
nodeConnections . get ( node ) foreach {
case ( _ , connection ) ⇒ sendCommandToNode ( connection , command , async = true )
}
2011-04-27 01:10:00 +02:00
}
}
}
/* *
* Creates an ActorRef with a Router to a set of clustered actors .
*/
2011-07-15 09:55:45 +03:00
def ref ( actorAddress : String , router : RouterType ) : ActorRef = if ( isConnected . isOn ) {
2011-04-27 01:10:00 +02:00
val addresses = addressesForActor ( actorAddress )
EventHandler . debug ( this ,
2011-06-07 11:10:29 -07:00
"Checking out cluster actor ref with address [%s] and router [%s] on [%s] connected to [\n\t%s]"
. format ( actorAddress , router , remoteServerAddress , addresses . map ( _ . _2 ) . mkString ( "\n\t" ) ) )
2011-04-27 01:10:00 +02:00
2011-05-25 16:18:35 +02:00
val actorRef = Router newRouter ( router , addresses , actorAddress , Actor . TIMEOUT )
2011-07-15 09:55:45 +03:00
addresses foreach {
case ( _ , address ) ⇒ clusterActorRefs . put ( address , actorRef )
}
2011-05-25 16:18:35 +02:00
actorRef . start ( )
2011-05-24 19:04:25 +02:00
2011-04-27 01:10:00 +02:00
} else throw new ClusterException ( "Not connected to cluster" )
/* *
* Returns the UUIDs of all actors checked out on this node .
*/
2011-06-27 11:53:24 +02:00
private [ akka ] def uuidsForActorsInUse : Array [ UUID ] = uuidsForActorsInUseOnNode ( nodeAddress . nodeName )
2011-04-27 01:10:00 +02:00
/* *
2011-04-29 15:47:56 +02:00
* Returns the addresses of all actors checked out on this node .
2011-04-27 01:10:00 +02:00
*/
2011-04-29 15:47:56 +02:00
def addressesForActorsInUse : Array [ String ] = actorAddressForUuids ( uuidsForActorsInUse )
2011-04-27 01:10:00 +02:00
/* *
* Returns the UUIDs of all actors registered in this cluster .
*/
2011-07-15 09:55:45 +03:00
private [ akka ] def uuidsForClusteredActors : Array [ UUID ] = if ( isConnected . isOn ) {
2011-07-02 18:18:48 +02:00
zkClient . getChildren ( ACTOR_UUID_REGISTRY_PATH ) . toList . map ( new UUID ( _ ) ) . toArray . asInstanceOf [ Array [ UUID ] ]
2011-04-27 01:10:00 +02:00
} else Array . empty [ UUID ]
/* *
2011-04-29 15:47:56 +02:00
* Returns the addresses of all actors registered in this cluster .
2011-04-27 01:10:00 +02:00
*/
2011-04-29 15:47:56 +02:00
def addressesForClusteredActors : Array [ String ] = actorAddressForUuids ( uuidsForClusteredActors )
2011-04-27 01:10:00 +02:00
/* *
* Returns the actor id for the actor with a specific UUID .
*/
2011-07-15 09:55:45 +03:00
private [ akka ] def actorAddressForUuid ( uuid : UUID ) : Option [ String ] = if ( isConnected . isOn ) {
2011-05-18 12:25:27 +02:00
try {
2011-07-02 18:18:48 +02:00
Some ( zkClient . readData ( actorUuidRegistryAddressPathFor ( uuid ) ) . asInstanceOf [ String ] )
2011-05-18 17:25:30 +02:00
} catch {
2011-06-17 10:25:02 +02:00
case e : ZkNoNodeException ⇒ None
2011-05-18 12:25:27 +02:00
}
2011-06-17 10:25:02 +02:00
} else None
2011-04-27 01:10:00 +02:00
/* *
* Returns the actor ids for all the actors with a specific UUID .
*/
2011-06-27 11:53:24 +02:00
private [ akka ] def actorAddressForUuids ( uuids : Array [ UUID ] ) : Array [ String ] =
2011-06-17 10:25:02 +02:00
uuids map ( actorAddressForUuid ( _ ) ) filter ( _ . isDefined ) map ( _ . get )
2011-04-27 01:10:00 +02:00
/* *
* Returns the actor UUIDs for actor ID .
*/
2011-07-15 09:55:45 +03:00
private [ akka ] def uuidsForActorAddress ( actorAddress : String ) : Array [ UUID ] = if ( isConnected . isOn ) {
2011-05-18 12:25:27 +02:00
try {
2011-07-02 18:18:48 +02:00
zkClient . getChildren ( actorAddressToUuidsPathFor ( actorAddress ) ) . toList . toArray map {
2011-05-21 17:30:16 +02:00
case c : CharSequence ⇒ new UUID ( c )
2011-07-02 18:18:48 +02:00
} filter ( _ ne null )
2011-05-18 17:25:30 +02:00
} catch {
case e : ZkNoNodeException ⇒ Array [ UUID ] ( )
2011-05-18 12:25:27 +02:00
}
2011-04-27 01:10:00 +02:00
} else Array . empty [ UUID ]
/* *
* Returns the node names of all actors in use with UUID .
*/
2011-07-15 09:55:45 +03:00
private [ akka ] def nodesForActorsInUseWithAddress ( actorAddress : String ) : Array [ String ] = if ( isConnected . isOn ) {
2011-05-18 12:25:27 +02:00
try {
2011-07-02 18:18:48 +02:00
zkClient . getChildren ( actorAddressToNodesPathFor ( actorAddress ) ) . toList . toArray . asInstanceOf [ Array [ String ] ]
2011-05-18 17:25:30 +02:00
} catch {
case e : ZkNoNodeException ⇒ Array [ String ] ( )
2011-05-18 12:25:27 +02:00
}
2011-04-27 01:10:00 +02:00
} else Array . empty [ String ]
/* *
* Returns the UUIDs of all actors in use registered on a specific node .
*/
2011-07-15 09:55:45 +03:00
private [ akka ] def uuidsForActorsInUseOnNode ( nodeName : String ) : Array [ UUID ] = if ( isConnected . isOn ) {
2011-05-18 12:25:27 +02:00
try {
2011-07-02 18:18:48 +02:00
zkClient . getChildren ( nodeToUuidsPathFor ( nodeName ) ) . toList . toArray map {
2011-05-21 17:30:16 +02:00
case c : CharSequence ⇒ new UUID ( c )
2011-07-02 18:18:48 +02:00
} filter ( _ ne null )
2011-05-18 17:25:30 +02:00
} catch {
case e : ZkNoNodeException ⇒ Array [ UUID ] ( )
2011-05-18 12:25:27 +02:00
}
2011-04-27 01:10:00 +02:00
} else Array . empty [ UUID ]
/* *
2011-04-29 15:47:56 +02:00
* Returns the addresses of all actors in use registered on a specific node .
2011-04-27 01:10:00 +02:00
*/
2011-07-15 09:55:45 +03:00
def addressesForActorsInUseOnNode ( nodeName : String ) : Array [ String ] = if ( isConnected . isOn ) {
2011-04-27 01:10:00 +02:00
val uuids =
2011-05-18 12:25:27 +02:00
try {
2011-07-02 18:18:48 +02:00
zkClient . getChildren ( nodeToUuidsPathFor ( nodeName ) ) . toList . toArray map {
2011-05-21 17:30:16 +02:00
case c : CharSequence ⇒ new UUID ( c )
2011-07-02 18:18:48 +02:00
} filter ( _ ne null )
2011-05-18 17:25:30 +02:00
} catch {
case e : ZkNoNodeException ⇒ Array [ UUID ] ( )
2011-05-18 12:25:27 +02:00
}
2011-04-29 15:47:56 +02:00
actorAddressForUuids ( uuids )
2011-04-27 01:10:00 +02:00
} else Array . empty [ String ]
/* *
2011-06-22 09:59:00 +02:00
* Returns Serializer for actor with specific address .
2011-04-27 01:10:00 +02:00
*/
2011-06-22 09:59:00 +02:00
def serializerForActor ( actorAddress : String ) : Serializer = {
2011-07-02 18:18:48 +02:00
val serializerClassName =
2011-06-17 10:25:02 +02:00
try {
2011-07-02 18:18:48 +02:00
zkClient . readData ( actorAddressRegistrySerializerPathFor ( actorAddress ) , new Stat ) . asInstanceOf [ String ]
2011-06-17 10:25:02 +02:00
} catch {
2011-07-02 18:18:48 +02:00
case e : ZkNoNodeException ⇒ throw new IllegalStateException ( "No serializer found for actor with address [%s]" . format ( actorAddress ) )
2011-06-17 10:25:02 +02:00
}
2011-05-21 16:55:32 +02:00
2011-07-15 09:55:45 +03:00
ReflectiveAccess . getClassFor ( serializerClassName ) match {
// FIXME need to pass in a user provide class loader? Now using default in ReflectiveAccess.
2011-06-22 09:59:00 +02:00
case Right ( clazz ) ⇒ clazz . newInstance . asInstanceOf [ Serializer ]
case Left ( error ) ⇒
EventHandler . error ( error , this , "Could not load serializer class [%s] due to: %s" . format ( serializerClassName , error . toString ) )
throw error
}
2011-04-27 01:10:00 +02:00
}
/* *
2011-07-02 18:18:48 +02:00
* Returns addresses for nodes that the clustered actor is in use on .
2011-04-27 01:10:00 +02:00
*/
2011-04-29 15:47:56 +02:00
def addressesForActor ( actorAddress : String ) : Array [ ( UUID , InetSocketAddress ) ] = {
2011-04-27 01:10:00 +02:00
try {
for {
2011-07-02 18:18:48 +02:00
uuid ← uuidsForActorAddress ( actorAddress )
2011-04-27 01:10:00 +02:00
} yield {
2011-07-02 18:18:48 +02:00
val remoteAddress = zkClient . readData ( actorUuidRegistryRemoteAddressPathFor ( uuid ) ) . asInstanceOf [ InetSocketAddress ]
( uuid , remoteAddress )
2011-04-27 01:10:00 +02:00
}
} catch {
2011-07-02 18:18:48 +02:00
case e : ZkNoNodeException ⇒
EventHandler . warning ( this ,
"Could not retrieve remote socket address for node hosting actor [%s] due to: %s"
. format ( actorAddress , e . toString ) )
Array [ ( UUID , InetSocketAddress ) ] ( )
2011-04-27 01:10:00 +02:00
}
}
// =======================================
// Compute Grid
// =======================================
/* *
* Send a function 'Function0[ Unit ] ' to be invoked on a random number of nodes ( defined by 'replicationFactor' argument ) .
*/
2011-05-18 08:37:58 +02:00
def send ( f : Function0 [ Unit ] , replicationFactor : Int ) {
2011-06-14 19:35:18 +02:00
Serialization . serialize ( f ) match {
case Left ( error ) ⇒ throw error
case Right ( bytes ) ⇒
val message = RemoteDaemonMessageProtocol . newBuilder
. setMessageType ( FUNCTION_FUN0_UNIT )
. setPayload ( ByteString . copyFrom ( bytes ) )
. build
2011-06-26 09:49:03 +02:00
nodeConnectionsForReplicationFactor ( replicationFactor ) foreach ( _ ! message )
2011-06-14 19:35:18 +02:00
}
2011-04-27 01:10:00 +02:00
}
/* *
* 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 ] ] = {
2011-06-14 19:35:18 +02:00
Serialization . serialize ( f ) match {
case Left ( error ) ⇒ throw error
case Right ( bytes ) ⇒
val message = RemoteDaemonMessageProtocol . newBuilder
. setMessageType ( FUNCTION_FUN0_ANY )
. setPayload ( ByteString . copyFrom ( bytes ) )
. build
2011-06-26 09:49:03 +02:00
val results = nodeConnectionsForReplicationFactor ( replicationFactor ) map ( _ ? message )
2011-06-14 19:35:18 +02:00
results . toList . asInstanceOf [ List [ Future [ Any ] ] ]
}
2011-04-27 01:10:00 +02:00
}
/* *
* Send a function 'Function1[ Any , Unit ] ' to be invoked on a random number of nodes ( defined by 'replicationFactor' argument )
* with the argument speficied .
*/
2011-05-18 08:37:58 +02:00
def send ( f : Function1 [ Any , Unit ] , arg : Any , replicationFactor : Int ) {
2011-06-14 19:35:18 +02:00
Serialization . serialize ( ( f , arg ) ) match {
case Left ( error ) ⇒ throw error
case Right ( bytes ) ⇒
val message = RemoteDaemonMessageProtocol . newBuilder
. setMessageType ( FUNCTION_FUN1_ARG_UNIT )
. setPayload ( ByteString . copyFrom ( bytes ) )
. build
2011-06-26 09:49:03 +02:00
nodeConnectionsForReplicationFactor ( replicationFactor ) foreach ( _ ! message )
2011-06-14 19:35:18 +02:00
}
2011-04-27 01:10:00 +02:00
}
/* *
* 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 ] ] = {
2011-06-14 19:35:18 +02:00
Serialization . serialize ( ( f , arg ) ) match {
case Left ( error ) ⇒ throw error
case Right ( bytes ) ⇒
val message = RemoteDaemonMessageProtocol . newBuilder
. setMessageType ( FUNCTION_FUN1_ARG_ANY )
. setPayload ( ByteString . copyFrom ( bytes ) )
. build
2011-06-26 09:49:03 +02:00
val results = nodeConnectionsForReplicationFactor ( replicationFactor ) map ( _ ? message )
2011-06-14 19:35:18 +02:00
results . toList . asInstanceOf [ List [ Future [ Any ] ] ]
}
2011-04-27 01:10:00 +02:00
}
// =======================================
// Config
// =======================================
2011-06-22 12:03:58 +02:00
/* *
* Stores a configuration element under a specific key .
* If the key already exists then it will be overwritten .
*/
2011-04-27 01:10:00 +02:00
def setConfigElement ( key : String , bytes : Array [ Byte ] ) {
val compressedBytes = if ( shouldCompressData ) LZF . compress ( bytes ) else bytes
EventHandler . debug ( this ,
"Adding config value [%s] under key [%s] in cluster registry" . format ( key , compressedBytes ) )
zkClient . retryUntilConnected ( new Callable [ Either [ Unit , Exception ] ] ( ) {
2011-05-18 12:25:27 +02:00
def call : Either [ Unit , Exception ] = {
try {
Left ( zkClient . connection . create ( configurationPathFor ( key ) , compressedBytes , CreateMode . PERSISTENT ) )
} catch {
2011-05-18 17:25:30 +02:00
case e : KeeperException . NodeExistsException ⇒
2011-04-27 01:10:00 +02:00
try {
2011-04-29 15:47:56 +02:00
Left ( zkClient . connection . writeData ( configurationPathFor ( key ) , compressedBytes ) )
2011-05-18 12:25:27 +02:00
} catch {
2011-05-18 17:25:30 +02:00
case e : Exception ⇒ Right ( e )
2011-05-18 12:25:27 +02:00
}
2011-04-27 01:10:00 +02:00
}
}
2011-05-18 12:25:27 +02:00
} ) match {
2011-07-15 09:55:45 +03:00
case Left ( _ ) ⇒ /* do nothing */
2011-05-18 17:25:30 +02:00
case Right ( exception ) ⇒ throw exception
2011-05-18 12:25:27 +02:00
}
2011-04-27 01:10:00 +02:00
}
/* *
* Returns the config element for the key or NULL if no element exists under the key .
2011-06-22 12:03:58 +02:00
* Returns < code > Some ( element ) </ code > if it exists else < code > None </ code >
2011-04-27 01:10:00 +02:00
*/
2011-06-17 10:25:02 +02:00
def getConfigElement ( key : String ) : Option [ Array [ Byte ] ] = try {
Some ( zkClient . connection . readData ( configurationPathFor ( key ) , new Stat , true ) )
2011-04-27 01:10:00 +02:00
} catch {
2011-06-17 10:25:02 +02:00
case e : KeeperException . NoNodeException ⇒ None
2011-04-27 01:10:00 +02:00
}
2011-06-22 12:03:58 +02:00
/* *
* Removes configuration element for a specific key .
* Does nothing if the key does not exist .
*/
2011-05-18 08:37:58 +02:00
def removeConfigElement ( key : String ) {
ignore [ ZkNoNodeException ] {
EventHandler . debug ( this ,
"Removing config element with key [%s] from cluster registry" . format ( key ) )
zkClient . deleteRecursive ( configurationPathFor ( key ) )
}
2011-04-27 01:10:00 +02:00
}
2011-06-26 09:49:03 +02:00
/* *
* Returns a list with all config element keys .
*/
2011-06-22 09:59:00 +02:00
def getConfigElementKeys : Array [ String ] = zkClient . getChildren ( CONFIGURATION_PATH ) . toList . toArray . asInstanceOf [ Array [ String ] ]
2011-04-27 01:10:00 +02:00
// =======================================
// Private
// =======================================
2011-07-02 18:18:48 +02:00
private def sendCommandToNode ( connection : ActorRef , command : RemoteDaemonMessageProtocol , async : Boolean = true ) {
2011-06-26 09:49:03 +02:00
if ( async ) {
2011-06-27 18:57:48 +02:00
connection ! command
2011-06-26 09:49:03 +02:00
} else {
( connection ? ( command , remoteDaemonAckTimeout ) ) . as [ Status ] match {
case Some ( Success ) ⇒
EventHandler . debug ( this , "Replica for [%s] successfully created" . format ( connection . address ) )
case Some ( Failure ( cause ) ) ⇒
EventHandler . error ( cause , this , cause . toString )
throw cause
case None ⇒
val error = new ClusterException (
"Operation to instantiate replicas throughout the cluster timed out" )
EventHandler . error ( error , this , error . toString )
throw error
}
}
}
2011-07-02 18:18:48 +02:00
private [ cluster ] def membershipPathFor ( node : String ) : String = "%s/%s" . format ( MEMBERSHIP_PATH , node )
2011-07-15 09:55:45 +03:00
2011-07-02 18:18:48 +02:00
private [ cluster ] def configurationPathFor ( key : String ) : String = "%s/%s" . format ( CONFIGURATION_PATH , key )
2011-04-27 01:10:00 +02:00
2011-07-02 18:18:48 +02:00
private [ cluster ] def actorAddressToNodesPathFor ( actorAddress : String ) : String = "%s/%s" . format ( ACTOR_ADDRESS_NODES_TO_PATH , actorAddress )
2011-07-15 09:55:45 +03:00
2011-07-02 18:18:48 +02:00
private [ cluster ] def actorAddressToNodesPathFor ( actorAddress : String , nodeName : String ) : String = "%s/%s" . format ( actorAddressToNodesPathFor ( actorAddress ) , nodeName )
2011-05-18 12:25:27 +02:00
2011-07-02 18:18:48 +02:00
private [ cluster ] def nodeToUuidsPathFor ( node : String ) : String = "%s/%s" . format ( NODE_TO_ACTOR_UUIDS_PATH , node )
2011-07-15 09:55:45 +03:00
2011-07-02 18:18:48 +02:00
private [ cluster ] def nodeToUuidsPathFor ( node : String , uuid : UUID ) : String = "%s/%s/%s" . format ( NODE_TO_ACTOR_UUIDS_PATH , node , uuid )
2011-05-18 12:25:27 +02:00
2011-07-02 18:18:48 +02:00
private [ cluster ] def actorAddressRegistryPathFor ( actorAddress : String ) : String = "%s/%s" . format ( ACTOR_ADDRESS_REGISTRY_PATH , actorAddress )
2011-07-15 09:55:45 +03:00
2011-07-02 18:18:48 +02:00
private [ cluster ] def actorAddressRegistrySerializerPathFor ( actorAddress : String ) : String = "%s/%s" . format ( actorAddressRegistryPathFor ( actorAddress ) , "serializer" )
2011-07-15 09:55:45 +03:00
2011-07-02 18:18:48 +02:00
private [ cluster ] def actorAddressRegistryUuidPathFor ( actorAddress : String ) : String = "%s/%s" . format ( actorAddressRegistryPathFor ( actorAddress ) , "uuid" )
2011-05-18 12:25:27 +02:00
2011-07-02 18:18:48 +02:00
private [ cluster ] def actorUuidRegistryPathFor ( uuid : UUID ) : String = "%s/%s" . format ( ACTOR_UUID_REGISTRY_PATH , uuid )
2011-07-15 09:55:45 +03:00
2011-07-02 18:18:48 +02:00
private [ cluster ] def actorUuidRegistryNodePathFor ( uuid : UUID ) : String = "%s/%s" . format ( actorUuidRegistryPathFor ( uuid ) , "node" )
2011-07-15 09:55:45 +03:00
2011-07-02 18:18:48 +02:00
private [ cluster ] def actorUuidRegistryAddressPathFor ( uuid : UUID ) : String = "%s/%s" . format ( actorUuidRegistryPathFor ( uuid ) , "address" )
2011-05-18 12:25:27 +02:00
2011-07-02 18:18:48 +02:00
private [ cluster ] def actorUuidRegistryRemoteAddressPathFor ( uuid : UUID ) : String = "%s/%s" . format ( actorUuidRegistryPathFor ( uuid ) , "remote-address" )
2011-05-18 12:25:27 +02:00
2011-07-02 18:18:48 +02:00
private [ cluster ] def actorAddressToUuidsPathFor ( actorAddress : String ) : String = "%s/%s" . format ( ACTOR_ADDRESS_TO_UUIDS_PATH , actorAddress . replace ( '.' , '_' ) )
2011-07-15 09:55:45 +03:00
2011-07-02 18:18:48 +02:00
private [ cluster ] def actorAddressToUuidsPathFor ( actorAddress : String , uuid : UUID ) : String = "%s/%s" . format ( actorAddressToUuidsPathFor ( actorAddress ) , uuid )
2011-04-27 01:10:00 +02:00
/* *
2011-07-04 19:10:06 +02:00
* Returns a random set with node names of size 'replicationFactor' .
2011-07-05 14:46:05 +02:00
* Default replicationFactor is 0 , which returns the empty Set .
2011-07-04 19:10:06 +02:00
*/
2011-07-05 14:46:05 +02:00
private def nodesForReplicationFactor ( replicationFactor : Int = 0 , actorAddress : Option [ String ] = None ) : Set [ String ] = {
var replicaNames = Set . empty [ String ]
2011-07-04 19:10:06 +02:00
val nrOfClusterNodes = nodeConnections . size
if ( replicationFactor < 1 ) return replicaNames
if ( nrOfClusterNodes < replicationFactor ) throw new IllegalArgumentException (
"Replication factor [" + replicationFactor +
"] is greater than the number of available nodeNames [" + nrOfClusterNodes + "]" )
val preferredNodes =
2011-07-15 09:55:45 +03:00
if ( actorAddress . isDefined ) {
// use 'preferred-nodes' in deployment config for the actor
2011-07-04 19:10:06 +02:00
Deployer . deploymentFor ( actorAddress . get ) match {
2011-07-08 08:28:13 +02:00
case Deploy ( _ , _ , Clustered ( nodes , _ , _ ) ) ⇒
2011-07-05 14:46:05 +02:00
nodes map ( node ⇒ DeploymentConfig . nodeNameFor ( node ) ) take replicationFactor
2011-07-04 19:10:06 +02:00
case _ ⇒
throw new ClusterException ( "Actor [" + actorAddress . get + "] is not configured as clustered" )
}
} else Vector . empty [ String ]
for {
nodeName ← preferredNodes
key ← nodeConnections . keys
if key == nodeName
2011-07-05 14:46:05 +02:00
} replicaNames = replicaNames + nodeName
2011-07-04 19:10:06 +02:00
val nrOfCurrentReplicaNames = replicaNames . size
val replicaSet =
if ( nrOfCurrentReplicaNames > replicationFactor ) throw new IllegalStateException ( "Replica set is larger than replication factor" )
else if ( nrOfCurrentReplicaNames == replicationFactor ) replicaNames
else {
val random = new java . util . Random ( System . currentTimeMillis )
while ( replicaNames . size < replicationFactor ) {
2011-07-05 14:46:05 +02:00
replicaNames = replicaNames + membershipNodes ( random . nextInt ( nrOfClusterNodes ) )
2011-07-04 19:10:06 +02:00
}
replicaNames
2011-04-27 01:10:00 +02:00
}
2011-07-04 19:10:06 +02:00
EventHandler . debug ( this ,
"Picked out replica set [%s] for actor [%s]" . format ( replicaSet . mkString ( ", " ) , actorAddress ) )
replicaSet
2011-04-27 01:10:00 +02:00
}
2011-07-04 19:10:06 +02:00
/* *
* Returns a random set with replica connections of size 'replicationFactor' .
2011-07-05 14:46:05 +02:00
* Default replicationFactor is 0 , which returns the empty Set .
2011-07-04 19:10:06 +02:00
*/
2011-07-05 14:46:05 +02:00
private def nodeConnectionsForReplicationFactor ( replicationFactor : Int = 0 , actorAddress : Option [ String ] = None ) : Set [ ActorRef ] = {
2011-07-04 19:10:06 +02:00
for {
node ← nodesForReplicationFactor ( replicationFactor , actorAddress )
connectionOption ← nodeConnections . get ( node )
connection ← connectionOption
actorRef ← connection . _2
} yield actorRef
}
private val connectToAllNewlyArrivedMembershipNodesInClusterLock = new AtomicBoolean ( false )
2011-04-27 01:10:00 +02:00
/* *
2011-07-02 18:18:48 +02:00
* Update the list of connections to other nodes in the cluster .
*
* @returns a Map with the remote socket addresses to of disconnected node connections
2011-04-27 01:10:00 +02:00
*/
2011-07-02 18:18:48 +02:00
private [ cluster ] def connectToAllNewlyArrivedMembershipNodesInCluster (
2011-07-15 09:55:45 +03:00
newlyConnectedMembershipNodes : Traversable [ String ] ,
newlyDisconnectedMembershipNodes : Traversable [ String ] ) : Map [ String , InetSocketAddress ] = {
2011-07-02 18:18:48 +02:00
// cache the disconnected connections in a map, needed for fail-over of these connections later
var disconnectedConnections = Map . empty [ String , InetSocketAddress ]
2011-07-15 09:55:45 +03:00
newlyDisconnectedMembershipNodes foreach {
node ⇒
disconnectedConnections += ( node -> ( nodeConnections ( node ) match {
case ( address , _ ) ⇒ address
} ) )
2011-07-02 18:18:48 +02:00
}
2011-07-04 19:10:06 +02:00
if ( connectToAllNewlyArrivedMembershipNodesInClusterLock . compareAndSet ( false , true ) ) {
try {
// remove connections to failed nodes
newlyDisconnectedMembershipNodes foreach ( nodeConnections . remove ( _ ) )
2011-07-02 18:18:48 +02:00
2011-07-04 19:10:06 +02:00
// add connections newly arrived nodes
2011-07-15 09:55:45 +03:00
newlyConnectedMembershipNodes foreach {
node ⇒
if ( ! nodeConnections . contains ( node ) ) {
// only connect to each replica once
remoteSocketAddressForNode ( node ) foreach {
address ⇒
EventHandler . debug ( this ,
"Setting up connection to node with nodename [%s] and address [%s]" . format ( node , address ) )
val clusterDaemon = Actor . remote . actorFor ( RemoteClusterDaemon . Address , address . getHostName , address . getPort ) . start ( )
nodeConnections . put ( node , ( address , clusterDaemon ) )
}
2011-07-04 19:10:06 +02:00
}
2011-05-25 17:41:53 +02:00
}
2011-07-04 19:10:06 +02:00
} finally {
connectToAllNewlyArrivedMembershipNodesInClusterLock . set ( false )
2011-05-18 17:25:30 +02:00
}
2011-04-27 01:10:00 +02:00
}
2011-07-04 19:10:06 +02:00
2011-07-02 18:18:48 +02:00
disconnectedConnections
2011-04-27 01:10:00 +02:00
}
2011-06-26 09:49:03 +02:00
private [ cluster ] def joinCluster ( ) {
2011-04-27 01:10:00 +02:00
try {
EventHandler . info ( this ,
"Joining cluster as membership node [%s] on [%s]" . format ( nodeAddress , membershipNodePath ) )
2011-06-26 09:49:03 +02:00
zkClient . createEphemeral ( membershipNodePath , remoteServerAddress )
2011-04-27 01:10:00 +02:00
} catch {
2011-05-18 17:25:30 +02:00
case e : ZkNodeExistsException ⇒
2011-07-02 18:18:48 +02:00
val error = new ClusterException (
"Can't join the cluster. The node name [" + nodeAddress . nodeName + "] is already in by another node" )
2011-05-30 10:53:25 +02:00
EventHandler . error ( error , this , error . toString )
2011-04-27 01:10:00 +02:00
throw error
}
2011-07-05 15:58:19 +02:00
ignore [ ZkNodeExistsException ] ( zkClient . createPersistent ( nodeToUuidsPathFor ( nodeAddress . nodeName ) ) )
2011-04-27 01:10:00 +02:00
}
2011-05-30 10:53:25 +02:00
private [ cluster ] def joinLeaderElection ( ) : Boolean = {
2011-04-27 01:10:00 +02:00
EventHandler . info ( this , "Node [%s] is joining leader election" . format ( nodeAddress . nodeName ) )
2011-06-22 09:59:00 +02:00
try {
leaderLock . lock
} catch {
case e : KeeperException . NodeExistsException ⇒ false
}
2011-04-27 01:10:00 +02:00
}
2011-06-27 18:57:48 +02:00
private [ cluster ] def remoteSocketAddressForNode ( node : String ) : Option [ InetSocketAddress ] = {
try {
Some ( zkClient . readData ( membershipPathFor ( node ) , new Stat ) . asInstanceOf [ InetSocketAddress ] )
} catch {
case e : ZkNoNodeException ⇒ None
}
2011-06-26 09:49:03 +02:00
}
2011-07-02 18:18:48 +02:00
private [ cluster ] def failOverClusterActorRefConnections ( from : InetSocketAddress , to : InetSocketAddress ) {
2011-04-27 01:10:00 +02:00
clusterActorRefs . values ( from ) foreach ( _ . failOver ( from , to ) )
}
2011-07-02 18:18:48 +02:00
private [ cluster ] def migrateActorsOnFailedNodes (
2011-07-15 09:55:45 +03:00
failedNodes : List [ String ] ,
currentClusterNodes : List [ String ] ,
oldClusterNodes : List [ String ] ,
disconnectedConnections : Map [ String , InetSocketAddress ] ) {
2011-05-18 17:25:30 +02:00
2011-07-15 09:55:45 +03:00
failedNodes . foreach {
failedNodeName ⇒
2011-07-02 18:18:48 +02:00
2011-07-15 09:55:45 +03:00
val failedNodeAddress = NodeAddress ( nodeAddress . clusterName , failedNodeName )
2011-04-27 01:10:00 +02:00
2011-07-15 09:55:45 +03:00
val myIndex = oldClusterNodes . indexWhere ( _ . endsWith ( nodeAddress . nodeName ) )
val failedNodeIndex = oldClusterNodes . indexWhere ( _ == failedNodeName )
2011-04-27 01:10:00 +02:00
2011-07-15 09:55:45 +03:00
// Migrate to the successor of the failed node (using a sorted circular list of the node names)
if ( ( failedNodeIndex == 0 && myIndex == oldClusterNodes . size - 1 ) || // No leftmost successor exists, check the tail
( failedNodeIndex == myIndex + 1 ) ) {
// Am I the leftmost successor?
2011-06-27 18:57:48 +02:00
2011-07-15 09:55:45 +03:00
// Takes the lead of migrating the actors. Not all to this node.
// All to this node except if the actor already resides here, then pick another node it is not already on.
2011-04-27 01:10:00 +02:00
2011-07-15 09:55:45 +03:00
// Yes I am the node to migrate the actor to (can only be one in the cluster)
val actorUuidsForFailedNode = zkClient . getChildren ( nodeToUuidsPathFor ( failedNodeName ) ) . toList
2011-06-17 10:25:02 +02:00
2011-07-15 09:55:45 +03:00
actorUuidsForFailedNode . foreach {
uuidAsString ⇒
EventHandler . debug ( this ,
"Cluster node [%s] has failed, migrating actor with UUID [%s] to [%s]"
. format ( failedNodeName , uuidAsString , nodeAddress . nodeName ) )
val uuid = uuidFrom ( uuidAsString )
val actorAddress = actorAddressForUuid ( uuid ) . getOrElse (
throw new IllegalStateException ( "No actor address found for UUID [" + uuidAsString + "]" ) )
val migrateToNodeAddress =
if ( isInUseOnNode ( actorAddress ) ) {
// already in use on this node, pick another node to instantiate the actor on
val replicaNodesForActor = nodesForActorsInUseWithAddress ( actorAddress )
val nodesAvailableForMigration = ( currentClusterNodes . toSet diff failedNodes . toSet ) diff replicaNodesForActor . toSet
if ( nodesAvailableForMigration . isEmpty ) throw new ClusterException (
"Can not migrate actor to new node since there are not any available nodes left. " +
"(However, the actor already has >1 replica in cluster, so we are ok)" )
NodeAddress ( nodeAddress . clusterName , nodesAvailableForMigration . head )
} else {
// actor is not in use on this node, migrate it here
nodeAddress
}
2011-07-02 18:18:48 +02:00
2011-07-15 09:55:45 +03:00
// if actor is replicated => pass along the UUID for the actor to replicate from (replay transaction log etc.)
val replicateFromUuid =
if ( isReplicated ( actorAddress ) ) Some ( uuid )
else None
2011-07-08 08:28:13 +02:00
2011-07-15 09:55:45 +03:00
migrateWithoutCheckingThatActorResidesOnItsHomeNode (
failedNodeAddress ,
migrateToNodeAddress ,
actorAddress ,
replicateFromUuid )
}
2011-04-27 01:10:00 +02:00
2011-07-15 09:55:45 +03:00
// notify all available nodes that they should fail-over all connections from 'from' to 'to'
val from = disconnectedConnections ( failedNodeName )
val to = remoteServerAddress
2011-06-27 18:57:48 +02:00
2011-07-15 09:55:45 +03:00
Serialization . serialize ( ( from , to ) ) match {
case Left ( error ) ⇒ throw error
case Right ( bytes ) ⇒
2011-06-27 18:57:48 +02:00
2011-07-15 09:55:45 +03:00
val command = RemoteDaemonMessageProtocol . newBuilder
. setMessageType ( FAIL_OVER_CONNECTIONS )
. setPayload ( ByteString . copyFrom ( bytes ) )
. build
2011-06-27 18:57:48 +02:00
2011-07-15 09:55:45 +03:00
// FIXME now we are broadcasting to ALL nodes in the cluster even though a fraction might have a reference to the actors - should that be fixed?
nodeConnections . values foreach {
case ( _ , connection ) ⇒ sendCommandToNode ( connection , command , async = true )
}
}
2011-04-27 01:10:00 +02:00
}
}
}
/* *
2011-07-02 18:18:48 +02:00
* Used when the ephemeral "home" node is already gone , so we can 't check if it is available .
2011-04-27 01:10:00 +02:00
*/
private def migrateWithoutCheckingThatActorResidesOnItsHomeNode (
2011-07-15 09:55:45 +03:00
from : NodeAddress , to : NodeAddress , actorAddress : String , replicateFromUuid : Option [ UUID ] ) {
2011-04-27 01:10:00 +02:00
2011-06-26 09:49:03 +02:00
EventHandler . debug ( this , "Migrating actor [%s] from node [%s] to node [%s]" . format ( actorAddress , from , to ) )
2011-07-02 18:18:48 +02:00
if ( ! isInUseOnNode ( actorAddress , to ) ) {
release ( actorAddress )
2011-06-26 09:49:03 +02:00
2011-07-02 18:18:48 +02:00
val remoteAddress = remoteSocketAddressForNode ( to . nodeName ) . getOrElse ( throw new ClusterException ( "No remote address registered for [" + to . nodeName + "]" ) )
2011-04-27 01:10:00 +02:00
2011-07-02 18:18:48 +02:00
ignore [ ZkNodeExistsException ] ( zkClient . createEphemeral ( actorAddressToNodesPathFor ( actorAddress , to . nodeName ) ) )
2011-04-27 01:10:00 +02:00
2011-07-02 18:18:48 +02:00
ignore [ ZkNoNodeException ] ( zkClient . delete ( actorAddressToNodesPathFor ( actorAddress , from . nodeName ) ) )
2011-06-27 18:57:48 +02:00
2011-07-02 18:18:48 +02:00
// FIXME who takes care of this line?
//ignore[ZkNoNodeException](zkClient.delete(nodeToUuidsPathFor(from.nodeName, uuid)))
2011-04-27 01:10:00 +02:00
2011-07-02 18:18:48 +02:00
// 'use' (check out) actor on the remote 'to' node
2011-07-08 08:28:13 +02:00
useActorOnNode ( to . nodeName , actorAddress , replicateFromUuid )
2011-04-27 01:10:00 +02:00
}
}
2011-07-05 15:58:19 +02:00
private def createZooKeeperPathStructureIfNeeded ( ) {
2011-05-18 08:37:58 +02:00
ignore [ ZkNodeExistsException ] {
2011-06-22 09:59:00 +02:00
zkClient . create ( CLUSTER_PATH , null , CreateMode . PERSISTENT )
EventHandler . info ( this , "Created node [%s]" . format ( CLUSTER_PATH ) )
2011-05-18 08:37:58 +02:00
}
2011-04-27 01:10:00 +02:00
2011-07-15 09:55:45 +03:00
basePaths . foreach {
path ⇒
try {
ignore [ ZkNodeExistsException ] ( zkClient . create ( path , null , CreateMode . PERSISTENT ) )
EventHandler . debug ( this , "Created node [%s]" . format ( path ) )
} catch {
case e ⇒
val error = new ClusterException ( e . toString )
EventHandler . error ( error , this )
throw error
}
2011-04-27 01:10:00 +02:00
}
}
2011-05-30 10:53:25 +02:00
private def registerListeners ( ) = {
2011-04-27 01:10:00 +02:00
zkClient . subscribeStateChanges ( stateListener )
2011-06-22 09:59:00 +02:00
zkClient . subscribeChildChanges ( MEMBERSHIP_PATH , membershipListener )
2011-04-27 01:10:00 +02:00
}
2011-06-17 10:25:02 +02:00
private def unregisterListeners ( ) = {
zkClient . unsubscribeStateChanges ( stateListener )
2011-06-22 09:59:00 +02:00
zkClient . unsubscribeChildChanges ( MEMBERSHIP_PATH , membershipListener )
2011-06-17 10:25:02 +02:00
}
2011-06-22 09:59:00 +02:00
private def fetchMembershipNodes ( ) {
val membershipChildren = zkClient . getChildren ( MEMBERSHIP_PATH )
2011-05-18 08:37:58 +02:00
locallyCachedMembershipNodes . clear ( )
2011-04-27 01:10:00 +02:00
membershipChildren . iterator . foreach ( locallyCachedMembershipNodes . add )
2011-07-02 18:18:48 +02:00
connectToAllNewlyArrivedMembershipNodesInCluster ( membershipNodes , Nil )
2011-04-27 01:10:00 +02:00
}
2011-07-08 08:28:13 +02:00
private def isReplicated ( actorAddress : String ) : Boolean = DeploymentConfig . isReplicated ( Deployer . deploymentFor ( actorAddress ) )
2011-04-27 01:10:00 +02:00
private def createMBean = {
val clusterMBean = new StandardMBean ( classOf [ ClusterNodeMBean ] ) with ClusterNodeMBean {
2011-05-18 12:25:27 +02:00
2011-07-13 11:27:16 +03:00
override def start ( ) = self . start ( )
2011-05-18 12:25:27 +02:00
2011-07-13 11:27:16 +03:00
override def stop ( ) = self . shutdown ( )
2011-04-27 01:10:00 +02:00
2011-05-18 08:37:58 +02:00
override def disconnect ( ) = self . disconnect ( )
2011-04-27 01:10:00 +02:00
2011-07-13 11:27:16 +03:00
override def reconnect ( ) = self . reconnect ( )
2011-05-18 12:25:27 +02:00
2011-07-13 11:27:16 +03:00
override def resign ( ) = self . resign ( )
2011-05-18 12:25:27 +02:00
2011-07-15 09:55:45 +03:00
override def isConnected = self . isConnected . isOn
2011-05-18 12:25:27 +02:00
2011-07-13 11:27:16 +03:00
override def getNodeAddres = self . nodeAddress
2011-06-26 09:49:03 +02:00
override def getRemoteServerHostname = self . hostname
2011-05-18 12:25:27 +02:00
2011-06-26 09:49:03 +02:00
override def getRemoteServerPort = self . port
2011-05-18 12:25:27 +02:00
override def getNodeName = self . nodeAddress . nodeName
override def getClusterName = self . nodeAddress . clusterName
override def getZooKeeperServerAddresses = self . zkServerAddresses
override def getMemberNodes = self . locallyCachedMembershipNodes . iterator . map ( _ . toString ) . toArray
2011-07-13 19:22:49 +03:00
override def getLeaderLockName = self . leader . toString
2011-04-27 01:10:00 +02:00
2011-07-13 11:27:16 +03:00
override def isLeader = self . isLeader
2011-05-18 12:25:27 +02:00
override def getUuidsForActorsInUse = self . uuidsForActorsInUse . map ( _ . toString ) . toArray
2011-04-27 01:10:00 +02:00
2011-05-18 12:25:27 +02:00
override def getAddressesForActorsInUse = self . addressesForActorsInUse . map ( _ . toString ) . toArray
2011-04-27 01:10:00 +02:00
2011-05-18 12:25:27 +02:00
override def getUuidsForClusteredActors = self . uuidsForClusteredActors . map ( _ . toString ) . toArray
2011-04-27 01:10:00 +02:00
2011-05-18 12:25:27 +02:00
override def getAddressesForClusteredActors = self . addressesForClusteredActors . map ( _ . toString ) . toArray
2011-04-27 01:10:00 +02:00
2011-07-02 18:18:48 +02:00
override def getNodesForActorInUseWithAddress ( address : String ) = self . nodesForActorsInUseWithAddress ( address )
2011-04-27 01:10:00 +02:00
2011-05-18 12:25:27 +02:00
override def getUuidsForActorsInUseOnNode ( nodeName : String ) = self . uuidsForActorsInUseOnNode ( nodeName ) . map ( _ . toString ) . toArray
2011-04-27 01:10:00 +02:00
2011-05-18 12:25:27 +02:00
override def getAddressesForActorsInUseOnNode ( nodeName : String ) = self . addressesForActorsInUseOnNode ( nodeName ) . map ( _ . toString ) . toArray
2011-05-21 16:55:32 +02:00
override def setConfigElement ( key : String , value : String ) : Unit = self . setConfigElement ( key , value . getBytes ( "UTF-8" ) )
2011-05-18 12:25:27 +02:00
2011-06-17 10:25:02 +02:00
override def getConfigElement ( key : String ) = new String ( self . getConfigElement ( key ) . getOrElse ( Array [ Byte ] ( ) ) , "UTF-8" )
2011-05-18 12:25:27 +02:00
2011-05-21 16:55:32 +02:00
override def removeConfigElement ( key : String ) : Unit = self . removeConfigElement ( key )
2011-05-18 12:25:27 +02:00
override def getConfigElementKeys = self . getConfigElementKeys . toArray
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
override def getMemberShipPathFor ( node : String ) = self . membershipPathFor ( node )
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
override def getConfigurationPathFor ( key : String ) = self . configurationPathFor ( key )
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
override def getActorAddresstoNodesPathFor ( actorAddress : String ) = self . actorAddressToNodesPathFor ( actorAddress )
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
override def getActorAddressToNodesPathForWithNodeName ( actorAddress : String , nodeName : String ) = self . actorAddressToNodesPathFor ( actorAddress , nodeName )
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
override def getNodeToUuidsPathFor ( node : String ) = self . nodeToUuidsPathFor ( node )
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
override def getNodeToUuidsPathFor ( node : String , uuid : UUID ) = self . nodeToUuidsPathFor ( node , uuid )
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
override def getActorAddressRegistryPathFor ( actorAddress : String ) = self . actorAddressRegistryPathFor ( actorAddress )
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
override def getActorAddressRegistrySerializerPathFor ( actorAddress : String ) = self . actorAddressRegistrySerializerPathFor ( actorAddress )
2011-07-13 11:27:16 +03:00
2011-07-15 09:55:45 +03:00
override def getActorAddressRegistryUuidPathFor ( actorAddress : String ) = self . actorAddressRegistryUuidPathFor ( actorAddress )
2011-07-13 11:27:16 +03:00
override def getActorUuidRegistryNodePathFor ( uuid : UUID ) = self . actorUuidRegistryNodePathFor ( uuid )
2011-07-15 09:55:45 +03:00
override def getActorUuidRegistryRemoteAddressPathFor ( uuid : UUID ) = self . actorUuidRegistryNodePathFor ( uuid )
2011-07-13 11:27:16 +03:00
override def getActorAddressToUuidsPathFor ( actorAddress : String ) = self . actorAddressToUuidsPathFor ( actorAddress )
2011-07-13 19:22:49 +03:00
override def getActorAddressToUuidsPathForWithNodeName ( actorAddress : String , uuid : UUID ) = self . actorAddressToUuidsPathFor ( actorAddress , uuid )
2011-04-27 01:10:00 +02:00
}
JMX . register ( clusterJmxObjectName , clusterMBean )
2011-04-28 20:12:37 +02:00
// FIXME need monitoring to lookup the cluster MBean dynamically
// Monitoring.registerLocalMBean(clusterJmxObjectName, clusterMBean)
2011-04-27 01:10:00 +02:00
}
}
/* *
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
class MembershipChildListener ( self : ClusterNode ) extends IZkChildListener with ErrorHandler {
2011-05-18 08:37:58 +02:00
def handleChildChange ( parentPath : String , currentChilds : JList [ String ] ) {
withErrorHandler {
if ( currentChilds ne null ) {
2011-07-02 18:18:48 +02:00
val currentClusterNodes = currentChilds . toList
if ( ! currentClusterNodes . isEmpty ) EventHandler . debug ( this ,
2011-05-18 08:37:58 +02:00
"MembershipChildListener at [%s] has children [%s]"
2011-07-02 18:18:48 +02:00
. format ( self . nodeAddress . nodeName , currentClusterNodes . mkString ( " " ) ) )
// take a snapshot of the old cluster nodes and then update the list with the current connected nodes in the cluster
val oldClusterNodes = self . locallyCachedMembershipNodes . toArray . toSet . asInstanceOf [ Set [ String ] ]
self . locallyCachedMembershipNodes . clear ( )
currentClusterNodes foreach ( self . locallyCachedMembershipNodes . add )
2011-06-27 18:57:48 +02:00
2011-07-02 18:18:48 +02:00
val newlyConnectedMembershipNodes = ( Set ( currentClusterNodes : _ * ) diff oldClusterNodes ) . toList
val newlyDisconnectedMembershipNodes = ( oldClusterNodes diff Set ( currentClusterNodes : _ * ) ) . toList
2011-06-27 18:57:48 +02:00
2011-07-02 18:18:48 +02:00
// update the connections with the new set of cluster nodes
val disconnectedConnections = self . connectToAllNewlyArrivedMembershipNodesInCluster ( newlyConnectedMembershipNodes , newlyDisconnectedMembershipNodes )
2011-04-27 01:10:00 +02:00
2011-07-02 18:18:48 +02:00
// if node(s) left cluster then migrate actors residing on the failed node
if ( ! newlyDisconnectedMembershipNodes . isEmpty )
self . migrateActorsOnFailedNodes ( newlyDisconnectedMembershipNodes , currentClusterNodes , oldClusterNodes . toList , disconnectedConnections )
2011-04-27 01:10:00 +02:00
2011-07-02 18:18:48 +02:00
// publish NodeConnected and NodeDisconnect events to the listeners
newlyConnectedMembershipNodes foreach ( node ⇒ self . publish ( NodeConnected ( node ) ) )
newlyDisconnectedMembershipNodes foreach ( node ⇒ self . publish ( NodeDisconnected ( node ) ) )
2011-05-18 08:37:58 +02:00
}
2011-04-27 01:10:00 +02:00
}
}
}
/* *
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
class StateListener ( self : ClusterNode ) extends IZkStateListener {
2011-05-18 08:37:58 +02:00
def handleStateChanged ( state : KeeperState ) {
state match {
2011-05-18 17:25:30 +02:00
case KeeperState . SyncConnected ⇒
2011-05-18 08:37:58 +02:00
EventHandler . debug ( this , "Cluster node [%s] - Connected" . format ( self . nodeAddress ) )
2011-05-23 22:35:01 +02:00
self . publish ( ThisNode . Connected )
2011-05-18 17:25:30 +02:00
case KeeperState . Disconnected ⇒
2011-05-18 08:37:58 +02:00
EventHandler . debug ( this , "Cluster node [%s] - Disconnected" . format ( self . nodeAddress ) )
2011-05-23 22:35:01 +02:00
self . publish ( ThisNode . Disconnected )
2011-05-18 17:25:30 +02:00
case KeeperState . Expired ⇒
2011-05-18 08:37:58 +02:00
EventHandler . debug ( this , "Cluster node [%s] - Expired" . format ( self . nodeAddress ) )
2011-05-23 22:35:01 +02:00
self . publish ( ThisNode . Expired )
2011-05-18 08:37:58 +02:00
}
2011-04-27 01:10:00 +02:00
}
/* *
* Re - initialize after the zookeeper session has expired and a new session has been created .
*/
2011-05-18 08:37:58 +02:00
def handleNewSession ( ) {
2011-04-27 01:10:00 +02:00
EventHandler . debug ( this , "Session expired re-initializing node [%s]" . format ( self . nodeAddress ) )
2011-05-18 08:37:58 +02:00
self . initializeNode ( )
2011-05-23 22:35:01 +02:00
self . publish ( NewSession )
2011-04-27 01:10:00 +02:00
}
}
/* *
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
trait ErrorHandler {
2011-05-18 17:25:30 +02:00
def withErrorHandler [ T ] ( body : ⇒ T ) = {
2011-04-27 01:10:00 +02:00
try {
2011-07-08 08:28:13 +02:00
ignore [ ZkInterruptedException ] ( body ) // FIXME Is it good to ignore ZkInterruptedException? If not, how should we handle it?
2011-04-27 01:10:00 +02:00
} catch {
2011-05-18 17:25:30 +02:00
case e : Throwable ⇒
2011-04-27 01:10:00 +02:00
EventHandler . error ( e , this , e . toString )
throw e
}
}
}
/* *
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
object RemoteClusterDaemon {
2011-07-08 08:28:13 +02:00
val Address = "akka-cluster-daemon" . intern
2011-04-27 01:10:00 +02:00
2011-05-30 10:53:25 +02:00
// FIXME configure computeGridDispatcher to what?
2011-07-04 19:10:06 +02:00
val computeGridDispatcher = Dispatchers . newDispatcher ( "akka:compute-grid" ) . build
2011-04-27 01:10:00 +02:00
}
/* *
2011-07-08 08:28:13 +02:00
* Internal "daemon" actor for cluster internal communication .
*
2011-04-27 01:10:00 +02:00
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
class RemoteClusterDaemon ( cluster : ClusterNode ) extends Actor {
2011-05-18 12:25:27 +02:00
2011-04-27 01:10:00 +02:00
import RemoteClusterDaemon._
import Cluster._
2011-05-20 22:41:41 +02:00
self . dispatcher = Dispatchers . newPinnedDispatcher ( self )
2011-04-27 01:10:00 +02:00
2011-05-30 10:53:25 +02:00
override def preRestart ( reason : Throwable ) {
EventHandler . debug ( this , "RemoteClusterDaemon failed due to [%s] restarting..." . format ( reason ) )
}
2011-04-27 01:10:00 +02:00
def receive : Receive = {
2011-05-18 17:25:30 +02:00
case message : RemoteDaemonMessageProtocol ⇒
2011-07-02 18:18:48 +02:00
EventHandler . debug ( this ,
"Received command [\n%s] to RemoteClusterDaemon on node [%s]"
. format ( message , cluster . nodeAddress . nodeName ) )
2011-05-25 16:18:35 +02:00
2011-04-27 01:10:00 +02:00
message . getMessageType match {
2011-05-18 17:25:30 +02:00
case USE ⇒
2011-05-25 16:18:35 +02:00
try {
2011-07-02 18:18:48 +02:00
if ( message . hasActorAddress ) {
val actorAddress = message . getActorAddress
2011-07-15 09:55:45 +03:00
cluster . serializerForActor ( actorAddress ) foreach {
serializer ⇒
cluster . use ( actorAddress , serializer ) foreach {
newActorRef ⇒
cluster . remoteService . register ( actorAddress , newActorRef )
if ( message . hasReplicateActorFromUuid ) {
// replication is used - fetch the messages and replay them
import akka.remote.protocol.RemoteProtocol._
import akka.remote.MessageSerializer
val replicateFromUuid = uuidProtocolToUuid ( message . getReplicateActorFromUuid )
val deployment = Deployer . deploymentFor ( actorAddress )
val replicationScheme = DeploymentConfig . replicationSchemeFor ( deployment ) . getOrElse (
throw new IllegalStateException (
"Actor [" + actorAddress + "] should have been configured as a replicated actor but could not find its ReplicationScheme" ) )
val isWriteBehind = DeploymentConfig . isWriteBehindReplication ( replicationScheme )
try {
// get the transaction log for the actor UUID
val txLog = TransactionLog . logFor ( replicateFromUuid . toString , isWriteBehind , replicationScheme )
// get the latest snapshot (Option[Array[Byte]]) and all the subsequent messages (Array[Byte])
val ( snapshotAsBytes , entriesAsBytes ) = txLog . latestSnapshotAndSubsequentEntries
// deserialize and restore actor snapshot
val actorRefToUseForReplay =
snapshotAsBytes match {
// we have a new actor ref - the snapshot
case Some ( bytes ) ⇒
// stop the new actor ref and use the snapshot instead
cluster . remoteService . unregister ( actorAddress )
// deserialize the snapshot actor ref and register it as remote actor
val uncompressedBytes =
if ( Cluster . shouldCompressData ) LZF . uncompress ( bytes )
else bytes
val snapshotActorRef = fromBinary ( uncompressedBytes , newActorRef . uuid ) . start ( )
cluster . remoteService . register ( actorAddress , snapshotActorRef )
// FIXME we should call 'stop()' here (to GC the actor), but can't since that will currently shut down the TransactionLog for this UUID - since both this actor and the new snapshotActorRef have the same UUID (which they should)
//newActorRef.stop()
snapshotActorRef
// we have no snapshot - use the new actor ref
case None ⇒
newActorRef
}
// deserialize the messages
val messages : Vector [ AnyRef ] = entriesAsBytes map {
bytes ⇒
val messageBytes =
if ( Cluster . shouldCompressData ) LZF . uncompress ( bytes )
else bytes
MessageSerializer . deserialize ( MessageProtocol . parseFrom ( messageBytes ) , None )
}
EventHandler . info ( this , "Replaying [%s] messages to actor [%s]" . format ( messages . size , actorAddress ) )
// replay all messages
messages foreach {
message ⇒
EventHandler . debug ( this , "Replaying message [%s] to actor [%s]" . format ( message , actorAddress ) )
// FIXME how to handle '?' messages? We can *not* replay them with the correct semantics. Should we: 1. Ignore/drop them and log warning? 2. Throw exception when about to log them? 3. Other?
actorRefToUseForReplay ! message
}
} catch {
case e : Throwable ⇒
EventHandler . error ( e , this , e . toString )
throw e
2011-07-08 19:35:27 +02:00
}
2011-07-08 08:28:13 +02:00
}
}
2011-07-02 18:18:48 +02:00
}
2011-05-25 16:18:35 +02:00
} else {
2011-07-08 08:28:13 +02:00
EventHandler . error ( this , "Actor 'address' is not defined, ignoring remote cluster daemon command [%s]" . format ( message ) )
2011-05-25 16:18:35 +02:00
}
self . reply ( Success )
2011-06-17 10:25:02 +02:00
2011-05-25 16:18:35 +02:00
} catch {
case error ⇒
self . reply ( Failure ( error ) )
throw error
}
2011-04-27 01:10:00 +02:00
2011-05-18 17:25:30 +02:00
case RELEASE ⇒
2011-05-18 12:25:27 +02:00
if ( message . hasActorUuid ) {
2011-07-15 09:55:45 +03:00
cluster . actorAddressForUuid ( uuidProtocolToUuid ( message . getActorUuid ) ) foreach {
address ⇒
cluster . release ( address )
2011-06-17 10:25:02 +02:00
}
2011-05-18 17:25:30 +02:00
} else if ( message . hasActorAddress ) {
2011-05-18 12:25:27 +02:00
cluster release message . getActorAddress
2011-05-25 16:18:35 +02:00
} else {
EventHandler . warning ( this ,
"None of 'uuid' or 'actorAddress'' is specified, ignoring remote cluster daemon command [%s]"
. format ( message ) )
}
2011-04-27 01:10:00 +02:00
2011-07-15 09:55:45 +03:00
case START ⇒ cluster . start ( )
2011-04-27 01:10:00 +02:00
2011-07-15 09:55:45 +03:00
case STOP ⇒ cluster . shutdown ( )
2011-04-27 01:10:00 +02:00
2011-05-18 17:25:30 +02:00
case DISCONNECT ⇒ cluster . disconnect ( )
2011-04-27 01:10:00 +02:00
2011-07-15 09:55:45 +03:00
case RECONNECT ⇒ cluster . reconnect ( )
2011-04-27 01:10:00 +02:00
2011-07-15 09:55:45 +03:00
case RESIGN ⇒ cluster . resign ( )
2011-04-27 01:10:00 +02:00
2011-05-18 17:25:30 +02:00
case FAIL_OVER_CONNECTIONS ⇒
2011-04-27 01:10:00 +02:00
val ( from , to ) = payloadFor ( message , classOf [ ( InetSocketAddress , InetSocketAddress ) ] )
2011-07-02 18:18:48 +02:00
cluster . failOverClusterActorRefConnections ( from , to )
2011-04-27 01:10:00 +02:00
2011-05-18 17:25:30 +02:00
case FUNCTION_FUN0_UNIT ⇒
2011-06-22 09:59:00 +02:00
localActorOf ( new Actor ( ) {
2011-05-30 10:53:25 +02:00
self . dispatcher = computeGridDispatcher
2011-05-18 12:25:27 +02:00
2011-04-27 01:10:00 +02:00
def receive = {
2011-05-18 17:25:30 +02:00
case f : Function0 [ Unit ] ⇒ try {
2011-05-18 12:25:27 +02:00
f ( )
} finally {
self . stop ( )
}
2011-04-27 01:10:00 +02:00
}
} ) . start ! payloadFor ( message , classOf [ Function0 [ Unit ] ] )
2011-05-18 17:25:30 +02:00
case FUNCTION_FUN0_ANY ⇒
2011-06-22 09:59:00 +02:00
localActorOf ( new Actor ( ) {
2011-05-30 10:53:25 +02:00
self . dispatcher = computeGridDispatcher
2011-05-18 12:25:27 +02:00
2011-04-27 01:10:00 +02:00
def receive = {
2011-05-18 17:25:30 +02:00
case f : Function0 [ Any ] ⇒ try {
2011-05-18 12:25:27 +02:00
self . reply ( f ( ) )
} finally {
self . stop ( )
}
2011-04-27 01:10:00 +02:00
}
} ) . start forward payloadFor ( message , classOf [ Function0 [ Any ] ] )
2011-05-18 17:25:30 +02:00
case FUNCTION_FUN1_ARG_UNIT ⇒
2011-06-22 09:59:00 +02:00
localActorOf ( new Actor ( ) {
2011-05-30 10:53:25 +02:00
self . dispatcher = computeGridDispatcher
2011-05-18 12:25:27 +02:00
2011-04-27 01:10:00 +02:00
def receive = {
2011-05-21 16:55:32 +02:00
case ( fun : Function [ Any , Unit ] , param : Any ) ⇒ try {
fun ( param )
2011-05-18 12:25:27 +02:00
} finally {
self . stop ( )
}
2011-04-27 01:10:00 +02:00
}
} ) . start ! payloadFor ( message , classOf [ Tuple2 [ Function1 [ Any , Unit ] , Any ] ] )
2011-05-18 17:25:30 +02:00
case FUNCTION_FUN1_ARG_ANY ⇒
2011-06-22 09:59:00 +02:00
localActorOf ( new Actor ( ) {
2011-05-30 10:53:25 +02:00
self . dispatcher = computeGridDispatcher
2011-05-18 12:25:27 +02:00
2011-04-27 01:10:00 +02:00
def receive = {
2011-05-21 16:55:32 +02:00
case ( fun : Function [ Any , Unit ] , param : Any ) ⇒ try {
self . reply ( fun ( param ) )
2011-05-18 12:25:27 +02:00
} finally {
self . stop ( )
}
2011-04-27 01:10:00 +02:00
}
} ) . start forward payloadFor ( message , classOf [ Tuple2 [ Function1 [ Any , Any ] , Any ] ] )
}
2011-05-18 17:25:30 +02:00
case unknown ⇒ EventHandler . warning ( this , "Unknown message [%s]" . format ( unknown ) )
2011-04-27 01:10:00 +02:00
}
private def payloadFor [ T ] ( message : RemoteDaemonMessageProtocol , clazz : Class [ T ] ) : T = {
2011-06-27 18:57:48 +02:00
Serialization . deserialize ( message . getPayload . toByteArray , clazz , None ) match {
2011-07-15 09:55:45 +03:00
case Left ( error ) ⇒ throw error
2011-06-27 18:57:48 +02:00
case Right ( instance ) ⇒ instance . asInstanceOf [ T ]
}
2011-04-27 01:10:00 +02:00
}
}