2011-09-15 10:20:18 +02:00
/* *
* Copyright ( C ) 2009 - 2011 Typesafe Inc . < http : //www.typesafe.com>
*/
2011-09-20 21:44:50 +02:00
package akka.remote
2011-09-15 10:20:18 +02:00
2011-11-10 20:08:00 +01:00
import akka.AkkaException
2011-09-15 10:20:18 +02:00
import akka.actor._
2011-10-07 15:42:55 +02:00
import akka.actor.Actor._
import akka.actor.Status._
2011-10-20 15:11:34 +02:00
import akka.routing._
2011-10-13 17:42:26 +02:00
import akka.dispatch._
2011-09-19 14:43:28 +02:00
import akka.util.duration._
import akka.config.ConfigurationException
2011-10-27 12:23:01 +02:00
import akka.event. { DeathWatch , Logging }
2011-10-20 15:11:34 +02:00
import akka.serialization.Compression.LZF
import akka.remote.RemoteProtocol._
import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._
2011-09-19 14:43:28 +02:00
import com.google.protobuf.ByteString
2011-11-03 14:53:38 +01:00
import java.util.concurrent.atomic.AtomicBoolean
2011-11-14 14:21:53 +01:00
import akka.event.EventStream
2011-11-18 11:16:23 +01:00
import java.util.concurrent.ConcurrentHashMap
import akka.dispatch.Promise
2011-11-22 13:04:10 +01:00
import java.net.InetAddress
import akka.serialization.SerializationExtension
2011-12-08 14:44:05 +01:00
import akka.serialization.Serialization
2011-09-19 14:43:28 +02:00
2011-09-15 10:20:18 +02:00
/* *
2011-09-19 14:43:28 +02:00
* Remote ActorRefProvider . Starts up actor on remote node and creates a RemoteActorRef representing it .
*
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
2011-09-15 10:20:18 +02:00
*/
2011-11-14 14:21:53 +01:00
class RemoteActorRefProvider (
2011-11-29 16:32:50 +01:00
val systemName : String ,
2011-11-17 11:51:14 +01:00
val settings : ActorSystem . Settings ,
2011-11-14 14:21:53 +01:00
val eventStream : EventStream ,
2011-11-29 16:32:50 +01:00
val scheduler : Scheduler ,
2011-12-02 14:41:13 +01:00
_deadLetters : InternalActorRef ) extends ActorRefProvider {
2011-09-15 10:20:18 +02:00
2011-11-18 11:59:43 +01:00
val log = Logging ( eventStream , "RemoteActorRefProvider" )
2011-12-08 14:44:05 +01:00
val remoteSettings = new RemoteSettings ( settings . config , systemName )
2011-11-14 14:21:53 +01:00
def deathWatch = local . deathWatch
2011-12-03 11:06:38 +01:00
def rootGuardian = local . rootGuardian
2011-11-14 14:21:53 +01:00
def guardian = local . guardian
def systemGuardian = local . systemGuardian
2011-12-08 14:44:05 +01:00
def nodename = remoteSettings . NodeName
def clustername = remoteSettings . ClusterName
2011-11-14 14:21:53 +01:00
2011-12-08 14:44:05 +01:00
val rootPath : ActorPath = RootActorPath ( RemoteAddress ( systemName , remoteSettings . serverSettings . Hostname , remoteSettings . serverSettings . Port ) )
private val local = new LocalActorRefProvider ( systemName , settings , eventStream , scheduler , _deadLetters , rootPath )
private var serialization : Serialization = _
private var _remote : Remote = _
def remote = _remote
2011-11-22 13:04:10 +01:00
2011-12-08 14:44:05 +01:00
def init ( system : ActorSystemImpl ) {
local . init ( system )
serialization = SerializationExtension ( system )
_remote = new Remote ( system , nodename , remoteSettings )
2011-12-09 00:02:27 +01:00
local . registerExtraNames ( Map ( ( "remote" , remote . remoteDaemon ) ) )
2011-12-08 14:44:05 +01:00
terminationFuture . onComplete ( _ ⇒ remote . server . shutdown ( ) )
2011-11-16 17:18:36 +01:00
}
2011-09-19 14:43:28 +02:00
2011-10-28 16:00:06 +02:00
private [ akka ] def terminationFuture = local . terminationFuture
2011-10-26 16:12:48 +02:00
2011-12-07 16:29:12 +01:00
private [ akka ] def deployer : Deployer = new RemoteDeployer ( settings , eventStream , nodename )
2011-11-04 10:11:07 +01:00
2011-11-22 13:04:10 +01:00
def dispatcher = local . dispatcher
2011-11-17 11:51:14 +01:00
def defaultTimeout = settings . ActorTimeout
2011-10-13 17:42:26 +02:00
2011-12-02 14:41:13 +01:00
def actorOf ( system : ActorSystemImpl , props : Props , supervisor : InternalActorRef , name : String , systemService : Boolean ) : InternalActorRef =
2011-11-29 16:32:50 +01:00
if ( systemService ) local . actorOf ( system , props , supervisor , name , systemService )
2011-10-18 14:21:48 +02:00
else {
2011-11-29 16:32:50 +01:00
val path = supervisor . path / name
2011-10-18 11:26:35 +02:00
2011-12-09 00:02:27 +01:00
deployer . lookupDeploymentFor ( path . elements . mkString ( "/" , "/" , "" ) ) match {
case Some ( DeploymentConfig . Deploy ( _ , _ , routerType , nrOfInstances , RemoteDeploymentConfig . RemoteScope ( remoteAddresses ) ) ) ⇒
2011-09-27 10:40:27 +02:00
2011-12-09 00:02:27 +01:00
// FIXME RK deployer shall only concern itself with placement of actors on remote nodes
val address = remoteAddresses . head
if ( address == rootPath . address ) local . actorOf ( system , props , supervisor , name , true ) // FIXME RK make non-system
else {
val rpath = RootActorPath ( address ) / "remote" / rootPath . address . hostPort / path . elements
useActorOnNode ( rpath , props . creator , supervisor )
new RemoteActorRef ( this , remote . server , rpath , supervisor , None )
}
2011-12-09 14:52:11 +01:00
// def isReplicaNode: Boolean = remoteAddresses exists { _ == remote.remoteAddress }
//
// //system.eventHandler.debug(this, "%s: Deploy Remote Actor with address [%s] connected to [%s]: isReplica(%s)".format(system.defaultAddress, address, remoteAddresses.mkString, isReplicaNode))
//
// if (isReplicaNode) {
// // we are on one of the replica node for this remote actor
// local.actorOf(system, props, supervisor, name, true) //FIXME systemService = true here to bypass Deploy, should be fixed when create-or-get is replaced by get-or-create (is this fixed now?)
// } else {
//
// implicit val dispatcher = if (props.dispatcher == Props.defaultDispatcher) system.dispatcher else props.dispatcher
// implicit val timeout = system.settings.ActorTimeout
//
// // we are on the single "reference" node uses the remote actors on the replica nodes
// val routerFactory: () ⇒ Router = DeploymentConfig.routerTypeFor(routerType) match {
// case RouterType.Direct ⇒
// if (remoteAddresses.size != 1) throw new ConfigurationException(
// "Actor [%s] configured with Direct router must have exactly 1 remote node configured. Found [%s]"
// .format(name, remoteAddresses.mkString(", ")))
// () ⇒ new DirectRouter
//
// case RouterType.Broadcast ⇒
// if (remoteAddresses.size != 1) throw new ConfigurationException(
// "Actor [%s] configured with Broadcast router must have exactly 1 remote node configured. Found [%s]"
// .format(name, remoteAddresses.mkString(", ")))
// () ⇒ new BroadcastRouter
//
// case RouterType.Random ⇒
// if (remoteAddresses.size < 1) throw new ConfigurationException(
// "Actor [%s] configured with Random router must have at least 1 remote node configured. Found [%s]"
// .format(name, remoteAddresses.mkString(", ")))
// () ⇒ new RandomRouter
//
// case RouterType.RoundRobin ⇒
// if (remoteAddresses.size < 1) throw new ConfigurationException(
// "Actor [%s] configured with RoundRobin router must have at least 1 remote node configured. Found [%s]"
// .format(name, remoteAddresses.mkString(", ")))
// () ⇒ new RoundRobinRouter
//
// case RouterType.ScatterGather ⇒
// if (remoteAddresses.size < 1) throw new ConfigurationException(
// "Actor [%s] configured with ScatterGather router must have at least 1 remote node configured. Found [%s]"
// .format(name, remoteAddresses.mkString(", ")))
// () ⇒ new ScatterGatherFirstCompletedRouter()(dispatcher, defaultTimeout)
//
// case RouterType.LeastCPU ⇒ sys.error("Router LeastCPU not supported yet")
// case RouterType.LeastRAM ⇒ sys.error("Router LeastRAM not supported yet")
// case RouterType.LeastMessages ⇒ sys.error("Router LeastMessages not supported yet")
// case RouterType.Custom(implClass) ⇒ () ⇒ Routing.createCustomRouter(implClass)
// }
//
// val connections = (Map.empty[RemoteAddress, ActorRef] /: remoteAddresses) { (conns, a) ⇒
// conns + (a -> new RemoteActorRef(this, remote.server, path, None)) // FIXME RK correct path must be put in here
// }
//
// val connectionManager = new RemoteConnectionManager(system, remote, connections)
//
// connections.keys foreach { useActorOnNode(system, _, path.toString, props.creator) }
//
// actorOf(system, RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), supervisor, name)
// }
2011-12-09 00:02:27 +01:00
case deploy ⇒ local . actorOf ( system , props , supervisor , name , systemService )
2011-09-27 16:52:33 +02:00
}
2011-09-15 10:20:18 +02:00
}
2011-10-13 13:41:44 +02:00
/* *
* Copied from LocalActorRefProvider . . .
*/
2011-11-30 10:20:57 +01:00
// FIXME: implement supervision, ticket #1408
2011-12-09 15:13:35 +01:00
// def actorOf(system: ActorSystem, props: RoutedProps, supervisor: InternalActorRef, name: String): InternalActorRef = {
// if (props.connectionManager.isEmpty) throw new ConfigurationException("RoutedProps used for creating actor [" + name + "] has zero connections configured; can't create a router")
// new RoutedActorRef(system, props, supervisor, name)
// }
2011-10-13 13:41:44 +02:00
2011-12-07 16:29:12 +01:00
def actorFor ( path : ActorPath ) : InternalActorRef = path . root match {
case `rootPath` ⇒ actorFor ( rootGuardian , path . elements )
2011-12-09 00:02:27 +01:00
case RootActorPath ( _ : RemoteAddress , _ ) ⇒ new RemoteActorRef ( this , remote . server , path , Nobody , None )
2011-12-07 16:29:12 +01:00
case _ ⇒ local . actorFor ( path )
}
def actorFor ( ref : InternalActorRef , path : String ) : InternalActorRef = path match {
case RemoteActorPath ( address , elems ) ⇒
if ( address == rootPath . address ) actorFor ( rootGuardian , elems )
2011-12-09 00:02:27 +01:00
else new RemoteActorRef ( this , remote . server , new RootActorPath ( address ) / elems , Nobody , None )
2011-12-07 16:29:12 +01:00
case _ ⇒ local . actorFor ( ref , path )
}
2011-12-03 11:06:38 +01:00
def actorFor ( ref : InternalActorRef , path : Iterable [ String ] ) : InternalActorRef = local . actorFor ( ref , path )
2011-09-15 10:20:18 +02:00
2011-11-16 17:18:36 +01:00
// TODO remove me
2011-11-03 14:53:38 +01:00
val optimizeLocal = new AtomicBoolean ( true )
def optimizeLocalScoped_? ( ) = optimizeLocal . get
2011-09-19 14:43:28 +02:00
/* *
* Using ( checking out ) actor on a specific node .
*/
2011-12-09 00:02:27 +01:00
def useActorOnNode ( path : ActorPath , actorFactory : ( ) ⇒ Actor , supervisor : ActorRef ) {
log . debug ( "[{}] Instantiating Remote Actor [{}]" , rootPath , path )
2011-09-19 14:43:28 +02:00
val actorFactoryBytes =
2011-11-24 18:53:18 +01:00
serialization . serialize ( actorFactory ) match {
2011-10-28 23:11:35 +02:00
case Left ( error ) ⇒ throw error
2011-12-08 14:44:05 +01:00
case Right ( bytes ) ⇒ if ( remoteSettings . ShouldCompressData ) LZF . compress ( bytes ) else bytes
2011-09-19 14:43:28 +02:00
}
2011-10-19 12:25:16 +02:00
val command = RemoteSystemDaemonMessageProtocol . newBuilder
2011-09-19 14:43:28 +02:00
. setMessageType ( USE )
2011-12-09 00:02:27 +01:00
. setActorPath ( path . toString )
2011-09-19 14:43:28 +02:00
. setPayload ( ByteString . copyFrom ( actorFactoryBytes ) )
2011-12-09 00:02:27 +01:00
. setSupervisor ( supervisor . path . toString )
2011-09-19 14:43:28 +02:00
. build ( )
2011-12-09 00:02:27 +01:00
// we don’ t wait for the ACK, because the remote end will process this command before any other message to the new actor
actorFor ( RootActorPath ( path . address ) / "remote" ) ! command
2011-09-15 10:20:18 +02:00
}
2011-10-20 15:11:34 +02:00
private def sendCommandToRemoteNode ( connection : ActorRef , command : RemoteSystemDaemonMessageProtocol , withACK : Boolean ) {
2011-09-22 03:36:59 +02:00
if ( withACK ) {
2011-09-15 10:20:18 +02:00
try {
2011-12-08 14:44:05 +01:00
val f = connection ? ( command , remoteSettings . RemoteSystemDaemonAckTimeout )
2011-10-28 23:11:35 +02:00
( try f . await . value catch { case _ : FutureTimeoutException ⇒ None } ) match {
case Some ( Right ( receiver ) ) ⇒
2011-10-27 12:23:01 +02:00
log . debug ( "Remote system command sent to [{}] successfully received" , receiver )
2011-09-15 10:20:18 +02:00
2011-10-28 23:11:35 +02:00
case Some ( Left ( cause ) ) ⇒
2011-10-27 12:23:01 +02:00
log . error ( cause , cause . toString )
2011-09-15 10:20:18 +02:00
throw cause
case None ⇒
2011-11-30 23:30:56 +01:00
val error = new RemoteException ( "Remote system command to [%s] timed out" . format ( connection . path ) )
2011-10-27 12:23:01 +02:00
log . error ( error , error . toString )
2011-09-15 10:20:18 +02:00
throw error
}
} catch {
case e : Exception ⇒
2011-11-30 23:30:56 +01:00
log . error ( e , "Could not send remote system command to [{}] due to: {}" , connection . path , e . toString )
2011-09-15 10:20:18 +02:00
throw e
}
2011-09-22 03:36:59 +02:00
} else {
connection ! command
2011-09-15 10:20:18 +02:00
}
}
2011-10-17 14:32:31 +02:00
2011-11-30 10:20:57 +01:00
private [ akka ] def createDeathWatch ( ) : DeathWatch = local . createDeathWatch ( ) //FIXME Implement Remote DeathWatch, ticket ##1190
2011-10-21 15:51:38 +02:00
private [ akka ] def ask ( message : Any , recipient : ActorRef , within : Timeout ) : Future [ Any ] = local . ask ( message , recipient , within )
2011-09-15 10:20:18 +02:00
}
2011-10-13 17:42:26 +02:00
/* *
* Remote ActorRef that is used when referencing the Actor on a different node than its "home" node .
* This reference is network - aware ( remembers its origin ) and immutable .
*
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
2011-12-07 16:29:12 +01:00
private [ akka ] class RemoteActorRef private [ akka ] (
2011-11-16 17:18:36 +01:00
provider : ActorRefProvider ,
2011-10-22 16:06:20 +02:00
remote : RemoteSupport ,
2011-12-07 16:29:12 +01:00
val path : ActorPath ,
2011-12-09 00:02:27 +01:00
val getParent : InternalActorRef ,
2011-10-13 17:42:26 +02:00
loader : Option [ ClassLoader ] )
2011-12-02 14:41:13 +01:00
extends InternalActorRef {
2011-11-08 11:56:46 +01:00
2011-12-09 00:02:27 +01:00
def getChild ( name : Iterator [ String ] ) : InternalActorRef = {
new RemoteActorRef ( provider , remote , path / name . toStream , Nobody , loader )
}
2011-12-03 11:06:38 +01:00
2011-10-13 17:42:26 +02:00
@volatile
private var running : Boolean = true
2011-11-23 19:03:56 +01:00
def isTerminated : Boolean = ! running
2011-10-13 17:42:26 +02:00
2011-12-08 14:44:05 +01:00
def sendSystemMessage ( message : SystemMessage ) : Unit = remote . send ( message , None , this , loader )
2011-10-18 15:39:26 +02:00
2011-12-07 16:29:12 +01:00
override def ! ( message : Any ) ( implicit sender : ActorRef = null ) : Unit = remote . send ( message , Option ( sender ) , this , loader )
2011-10-13 17:42:26 +02:00
2011-11-17 14:01:57 +01:00
override def ? ( message : Any ) ( implicit timeout : Timeout ) : Future [ Any ] = provider . ask ( message , this , timeout )
2011-10-13 17:42:26 +02:00
2011-12-08 14:44:05 +01:00
def suspend ( ) : Unit = sendSystemMessage ( Suspend ( ) )
2011-10-13 17:42:26 +02:00
2011-12-08 14:44:05 +01:00
def resume ( ) : Unit = sendSystemMessage ( Resume ( ) )
2011-10-13 17:42:26 +02:00
2011-12-08 14:44:05 +01:00
def stop ( ) : Unit = sendSystemMessage ( Terminate ( ) )
def restart ( cause : Throwable ) : Unit = sendSystemMessage ( Recreate ( cause ) )
2011-10-13 17:42:26 +02:00
@throws ( classOf [ java . io . ObjectStreamException ] )
2011-12-01 14:29:33 +01:00
private def writeReplace ( ) : AnyRef = SerializedActorRef ( path . toString )
2011-10-13 17:42:26 +02:00
}