Removing hostname and port for AkkaApplication, renaming defaultAddress to address, removing Deployer.RemoteAddress and use the normal akka.remote.RemoteAddress instead
This commit is contained in:
parent
c75a8db0de
commit
c6e44ffef7
13 changed files with 33 additions and 37 deletions
|
|
@ -284,7 +284,7 @@ class ActorRefSpec extends AkkaSpec {
|
||||||
val baos = new ByteArrayOutputStream(8192 * 32)
|
val baos = new ByteArrayOutputStream(8192 * 32)
|
||||||
val out = new ObjectOutputStream(baos)
|
val out = new ObjectOutputStream(baos)
|
||||||
|
|
||||||
val serialized = SerializedActorRef(app.hostname, app.port, "/this/path/does/not/exist")
|
val serialized = SerializedActorRef(app.address.hostname, app.address.port, "/this/path/does/not/exist")
|
||||||
|
|
||||||
out.writeObject(serialized)
|
out.writeObject(serialized)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -7,6 +7,7 @@ package akka.actor
|
||||||
import akka.testkit.AkkaSpec
|
import akka.testkit.AkkaSpec
|
||||||
import akka.util.duration._
|
import akka.util.duration._
|
||||||
import DeploymentConfig._
|
import DeploymentConfig._
|
||||||
|
import akka.remote.RemoteAddress
|
||||||
|
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class DeployerSpec extends AkkaSpec {
|
class DeployerSpec extends AkkaSpec {
|
||||||
|
|
|
||||||
|
|
@ -217,13 +217,13 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true
|
||||||
app.mainbus.subscribe(testActor, classOf[Logging.Debug])
|
app.mainbus.subscribe(testActor, classOf[Logging.Debug])
|
||||||
fsm ! "go"
|
fsm ! "go"
|
||||||
expectMsgPF(1 second, hint = "processing Event(go,null)") {
|
expectMsgPF(1 second, hint = "processing Event(go,null)") {
|
||||||
case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(go,null) from Actor[" + app.defaultAddress + "/sys/testActor") ⇒ true
|
case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(go,null) from Actor[" + app.address + "/sys/testActor") ⇒ true
|
||||||
}
|
}
|
||||||
expectMsg(1 second, Logging.Debug(fsm, "setting timer 't'/1500 milliseconds: Shutdown"))
|
expectMsg(1 second, Logging.Debug(fsm, "setting timer 't'/1500 milliseconds: Shutdown"))
|
||||||
expectMsg(1 second, Logging.Debug(fsm, "transition 1 -> 2"))
|
expectMsg(1 second, Logging.Debug(fsm, "transition 1 -> 2"))
|
||||||
fsm ! "stop"
|
fsm ! "stop"
|
||||||
expectMsgPF(1 second, hint = "processing Event(stop,null)") {
|
expectMsgPF(1 second, hint = "processing Event(stop,null)") {
|
||||||
case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(stop,null) from Actor[" + app.defaultAddress + "/sys/testActor") ⇒ true
|
case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(stop,null) from Actor[" + app.address + "/sys/testActor") ⇒ true
|
||||||
}
|
}
|
||||||
expectMsgAllOf(1 second, Logging.Debug(fsm, "canceling timer 't'"), Normal)
|
expectMsgAllOf(1 second, Logging.Debug(fsm, "canceling timer 't'"), Normal)
|
||||||
expectNoMsg(1 second)
|
expectNoMsg(1 second)
|
||||||
|
|
|
||||||
|
|
@ -152,7 +152,7 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor
|
||||||
case value ⇒ value
|
case value ⇒ value
|
||||||
}
|
}
|
||||||
|
|
||||||
val defaultAddress = RemoteAddress(System.getProperty("akka.remote.hostname") match {
|
val address = RemoteAddress(System.getProperty("akka.remote.hostname") match {
|
||||||
case null | "" ⇒ InetAddress.getLocalHost.getHostAddress
|
case null | "" ⇒ InetAddress.getLocalHost.getHostAddress
|
||||||
case value ⇒ value
|
case value ⇒ value
|
||||||
}, System.getProperty("akka.remote.port") match {
|
}, System.getProperty("akka.remote.port") match {
|
||||||
|
|
@ -160,10 +160,6 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor
|
||||||
case value ⇒ value.toInt
|
case value ⇒ value.toInt
|
||||||
})
|
})
|
||||||
|
|
||||||
def hostname: String = defaultAddress.hostname
|
|
||||||
|
|
||||||
def port: Int = defaultAddress.port
|
|
||||||
|
|
||||||
// this provides basic logging (to stdout) until .start() is called below
|
// this provides basic logging (to stdout) until .start() is called below
|
||||||
val mainbus = new MainBus(DebugMainBus)
|
val mainbus = new MainBus(DebugMainBus)
|
||||||
mainbus.startStdoutLogger(AkkaConfig)
|
mainbus.startStdoutLogger(AkkaConfig)
|
||||||
|
|
|
||||||
|
|
@ -172,7 +172,7 @@ class LocalActorRef private[akka] (
|
||||||
|
|
||||||
def name = path.name
|
def name = path.name
|
||||||
|
|
||||||
def address: String = _app.defaultAddress + path.toString
|
def address: String = _app.address + path.toString
|
||||||
|
|
||||||
private[this] val actorCell = new ActorCell(_app, this, props, _supervisor, receiveTimeout, hotswap)
|
private[this] val actorCell = new ActorCell(_app, this, props, _supervisor, receiveTimeout, hotswap)
|
||||||
actorCell.start()
|
actorCell.start()
|
||||||
|
|
@ -380,7 +380,7 @@ class DeadLetterActorRef(val app: AkkaApplication) extends MinimalActorRef {
|
||||||
// FIXME (actor path): put this under the sys guardian supervisor
|
// FIXME (actor path): put this under the sys guardian supervisor
|
||||||
val path: ActorPath = app.root / "sys" / name
|
val path: ActorPath = app.root / "sys" / name
|
||||||
|
|
||||||
def address: String = app.defaultAddress + path.toString
|
def address: String = app.address + path.toString
|
||||||
|
|
||||||
override def isShutdown(): Boolean = true
|
override def isShutdown(): Boolean = true
|
||||||
|
|
||||||
|
|
@ -402,7 +402,7 @@ abstract class AskActorRef(protected val app: AkkaApplication)(timeout: Timeout
|
||||||
// FIXME (actor path): put this under the tmp guardian supervisor
|
// FIXME (actor path): put this under the tmp guardian supervisor
|
||||||
val path: ActorPath = app.root / "tmp" / name
|
val path: ActorPath = app.root / "tmp" / name
|
||||||
|
|
||||||
def address: String = app.defaultAddress + path.toString
|
def address: String = app.address + path.toString
|
||||||
|
|
||||||
{
|
{
|
||||||
val callback: Future[Any] ⇒ Unit = { _ ⇒ app.deathWatch.publish(Terminated(AskActorRef.this)); whenDone() }
|
val callback: Future[Any] ⇒ Unit = { _ ⇒ app.deathWatch.publish(Terminated(AskActorRef.this)); whenDone() }
|
||||||
|
|
|
||||||
|
|
@ -129,7 +129,7 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
|
||||||
// FIXME (actor path): move the root path to the new root guardian
|
// FIXME (actor path): move the root path to the new root guardian
|
||||||
val path = app.root
|
val path = app.root
|
||||||
|
|
||||||
val address = app.defaultAddress + path.toString
|
val address = app.address + path.toString
|
||||||
|
|
||||||
override def toString = name
|
override def toString = name
|
||||||
|
|
||||||
|
|
@ -271,7 +271,7 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
|
||||||
}
|
}
|
||||||
|
|
||||||
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = actorFor(ActorPath.split(actor.path))
|
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = actorFor(ActorPath.split(actor.path))
|
||||||
private[akka] def serialize(actor: ActorRef): SerializedActorRef = new SerializedActorRef(app.defaultAddress, actor.path.toString)
|
private[akka] def serialize(actor: ActorRef): SerializedActorRef = new SerializedActorRef(app.address, actor.path.toString)
|
||||||
|
|
||||||
private[akka] def createDeathWatch(): DeathWatch = new LocalDeathWatch
|
private[akka] def createDeathWatch(): DeathWatch = new LocalDeathWatch
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -14,6 +14,7 @@ import akka.{ AkkaException, AkkaApplication }
|
||||||
import akka.config.{ Configuration, ConfigurationException }
|
import akka.config.{ Configuration, ConfigurationException }
|
||||||
import akka.util.Duration
|
import akka.util.Duration
|
||||||
import java.net.InetSocketAddress
|
import java.net.InetSocketAddress
|
||||||
|
import akka.remote.RemoteAddress
|
||||||
|
|
||||||
trait ActorDeployer {
|
trait ActorDeployer {
|
||||||
private[akka] def init(deployments: Seq[Deploy]): Unit
|
private[akka] def init(deployments: Seq[Deploy]): Unit
|
||||||
|
|
@ -187,8 +188,8 @@ class Deployer(val app: AkkaApplication) extends ActorDeployer {
|
||||||
case e: Exception ⇒ raiseRemoteNodeParsingError()
|
case e: Exception ⇒ raiseRemoteNodeParsingError()
|
||||||
}
|
}
|
||||||
if (port == 0) raiseRemoteNodeParsingError()
|
if (port == 0) raiseRemoteNodeParsingError()
|
||||||
val inet = new InetSocketAddress(hostname, port) //FIXME switch to non-ip-tied
|
|
||||||
RemoteAddress(Option(inet.getAddress).map(_.getHostAddress).getOrElse(hostname), inet.getPort)
|
RemoteAddress(new InetSocketAddress(hostname, port))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -8,6 +8,7 @@ import akka.AkkaApplication
|
||||||
import akka.util.Duration
|
import akka.util.Duration
|
||||||
import akka.routing.{ RouterType, FailureDetectorType }
|
import akka.routing.{ RouterType, FailureDetectorType }
|
||||||
import akka.routing.FailureDetectorType._
|
import akka.routing.FailureDetectorType._
|
||||||
|
import akka.remote.RemoteAddress
|
||||||
|
|
||||||
object DeploymentConfig {
|
object DeploymentConfig {
|
||||||
|
|
||||||
|
|
@ -63,8 +64,6 @@ object DeploymentConfig {
|
||||||
|
|
||||||
case class RemoteScope(nodes: Iterable[RemoteAddress]) extends Scope
|
case class RemoteScope(nodes: Iterable[RemoteAddress]) extends Scope
|
||||||
|
|
||||||
case class RemoteAddress(hostname: String, port: Int)
|
|
||||||
|
|
||||||
// --------------------------------
|
// --------------------------------
|
||||||
// --- Home
|
// --- Home
|
||||||
// --------------------------------
|
// --------------------------------
|
||||||
|
|
|
||||||
|
|
@ -17,20 +17,19 @@ object RemoteAddress {
|
||||||
def apply(inetAddress: InetSocketAddress): RemoteAddress = inetAddress match {
|
def apply(inetAddress: InetSocketAddress): RemoteAddress = inetAddress match {
|
||||||
case null ⇒ null
|
case null ⇒ null
|
||||||
case inet ⇒
|
case inet ⇒
|
||||||
val host = inet.getAddress.getHostAddress
|
val host = inet.getAddress match {
|
||||||
val portNo = inet.getPort
|
case null ⇒ inet.getHostName //Fall back to given name
|
||||||
new RemoteAddress {
|
case other ⇒ other.getHostAddress
|
||||||
def hostname = host
|
|
||||||
def port = portNo
|
|
||||||
}
|
}
|
||||||
|
val portNo = inet.getPort
|
||||||
|
RemoteAddress(portNo, host)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
trait RemoteAddress extends Serializable {
|
case class RemoteAddress private[akka] (port: Int, hostname: String) {
|
||||||
def hostname: String
|
|
||||||
def port: Int
|
|
||||||
@transient
|
@transient
|
||||||
override lazy val toString = "" + hostname + ":" + port
|
override lazy val toString = "" + hostname + ":" + port
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class RemoteException(message: String) extends AkkaException(message)
|
class RemoteException(message: String) extends AkkaException(message)
|
||||||
|
|
|
||||||
|
|
@ -107,7 +107,7 @@ class Gossiper(remote: Remote) {
|
||||||
private val connectionManager = new RemoteConnectionManager(app, remote, Map.empty[RemoteAddress, ActorRef])
|
private val connectionManager = new RemoteConnectionManager(app, remote, Map.empty[RemoteAddress, ActorRef])
|
||||||
private val seeds = Set(address) // FIXME read in list of seeds from config
|
private val seeds = Set(address) // FIXME read in list of seeds from config
|
||||||
|
|
||||||
private val address = app.defaultAddress
|
private val address = app.address
|
||||||
private val nodeFingerprint = address.##
|
private val nodeFingerprint = address.##
|
||||||
|
|
||||||
private val random = SecureRandom.getInstance("SHA1PRNG")
|
private val random = SecureRandom.getInstance("SHA1PRNG")
|
||||||
|
|
|
||||||
|
|
@ -83,7 +83,7 @@ class Remote(val app: AkkaApplication) {
|
||||||
}
|
}
|
||||||
|
|
||||||
def start(): Unit = {
|
def start(): Unit = {
|
||||||
val serverAddress = server.app.defaultAddress //Force init of server
|
val serverAddress = server.app.address //Force init of server
|
||||||
val daemonAddress = remoteDaemon.address //Force init of daemon
|
val daemonAddress = remoteDaemon.address //Force init of daemon
|
||||||
log.info("Starting remote server on [{}] and starting remoteDaemon with address [{}]", serverAddress, daemonAddress)
|
log.info("Starting remote server on [{}] and starting remoteDaemon with address [{}]", serverAddress, daemonAddress)
|
||||||
}
|
}
|
||||||
|
|
@ -153,7 +153,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
|
||||||
log.error("Actor 'address' for actor to instantiate is not defined, ignoring remote system daemon command [{}]", message)
|
log.error("Actor 'address' for actor to instantiate is not defined, ignoring remote system daemon command [{}]", message)
|
||||||
}
|
}
|
||||||
|
|
||||||
sender ! Success(app.defaultAddress)
|
sender ! Success(app.address)
|
||||||
} catch {
|
} catch {
|
||||||
case error: Throwable ⇒ //FIXME doesn't seem sensible
|
case error: Throwable ⇒ //FIXME doesn't seem sensible
|
||||||
sender ! Failure(error)
|
sender ! Failure(error)
|
||||||
|
|
|
||||||
|
|
@ -76,7 +76,7 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider
|
||||||
// case FailureDetectorType.Custom(implClass) ⇒ FailureDetector.createCustomFailureDetector(implClass)
|
// case FailureDetectorType.Custom(implClass) ⇒ FailureDetector.createCustomFailureDetector(implClass)
|
||||||
// }
|
// }
|
||||||
|
|
||||||
def isReplicaNode: Boolean = remoteAddresses exists { some ⇒ some.port == app.port && some.hostname == app.hostname }
|
def isReplicaNode: Boolean = remoteAddresses exists { _ == app.address }
|
||||||
|
|
||||||
//app.eventHandler.debug(this, "%s: Deploy Remote Actor with address [%s] connected to [%s]: isReplica(%s)".format(app.defaultAddress, address, remoteAddresses.mkString, isReplicaNode))
|
//app.eventHandler.debug(this, "%s: Deploy Remote Actor with address [%s] connected to [%s]: isReplica(%s)".format(app.defaultAddress, address, remoteAddresses.mkString, isReplicaNode))
|
||||||
|
|
||||||
|
|
@ -177,10 +177,10 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider
|
||||||
|
|
||||||
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = {
|
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = {
|
||||||
val remoteAddress = RemoteAddress(actor.hostname, actor.port)
|
val remoteAddress = RemoteAddress(actor.hostname, actor.port)
|
||||||
if (optimizeLocalScoped_? && remoteAddress == app.defaultAddress) {
|
if (optimizeLocalScoped_? && remoteAddress == app.address) {
|
||||||
local.actorFor(ActorPath.split(actor.path))
|
local.actorFor(ActorPath.split(actor.path))
|
||||||
} else {
|
} else {
|
||||||
log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", app.defaultAddress, actor.path, remoteAddress)
|
log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", app.address, actor.path, remoteAddress)
|
||||||
Some(RemoteActorRef(remote.server, remoteAddress, ActorPath(app, actor.path), None)) //Should it be None here
|
Some(RemoteActorRef(remote.server, remoteAddress, ActorPath(app, actor.path), None)) //Should it be None here
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -189,7 +189,7 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider
|
||||||
* Using (checking out) actor on a specific node.
|
* Using (checking out) actor on a specific node.
|
||||||
*/
|
*/
|
||||||
def useActorOnNode(remoteAddress: RemoteAddress, actorPath: String, actorFactory: () ⇒ Actor) {
|
def useActorOnNode(remoteAddress: RemoteAddress, actorPath: String, actorFactory: () ⇒ Actor) {
|
||||||
log.debug("[{}] Instantiating Actor [{}] on node [{}]", app.defaultAddress, actorPath, remoteAddress)
|
log.debug("[{}] Instantiating Actor [{}] on node [{}]", app.address, actorPath, remoteAddress)
|
||||||
|
|
||||||
val actorFactoryBytes =
|
val actorFactoryBytes =
|
||||||
app.serialization.serialize(actorFactory) match {
|
app.serialization.serialize(actorFactory) match {
|
||||||
|
|
|
||||||
|
|
@ -147,7 +147,7 @@ class ActiveRemoteClient private[akka] (
|
||||||
def sendSecureCookie(connection: ChannelFuture) {
|
def sendSecureCookie(connection: ChannelFuture) {
|
||||||
val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT)
|
val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT)
|
||||||
if (SECURE_COOKIE.nonEmpty) handshake.setCookie(SECURE_COOKIE.get)
|
if (SECURE_COOKIE.nonEmpty) handshake.setCookie(SECURE_COOKIE.get)
|
||||||
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder().setHostname(remoteSupport.app.hostname).setPort(remoteSupport.app.port).build)
|
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder.setHostname(remoteSupport.app.address.hostname).setPort(remoteSupport.app.address.port).build)
|
||||||
connection.getChannel.write(remoteSupport.createControlEnvelope(handshake.build))
|
connection.getChannel.write(remoteSupport.createControlEnvelope(handshake.build))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -428,7 +428,7 @@ class NettyRemoteSupport(_app: AkkaApplication) extends RemoteSupport(_app) with
|
||||||
|
|
||||||
def name = currentServer.get match {
|
def name = currentServer.get match {
|
||||||
case Some(server) ⇒ server.name
|
case Some(server) ⇒ server.name
|
||||||
case None ⇒ "Non-running NettyRemoteServer@" + app.hostname + ":" + app.port
|
case None ⇒ "Non-running NettyRemoteServer@" + app.address
|
||||||
}
|
}
|
||||||
|
|
||||||
private val _isRunning = new Switch(false)
|
private val _isRunning = new Switch(false)
|
||||||
|
|
@ -459,9 +459,9 @@ class NettyRemoteSupport(_app: AkkaApplication) extends RemoteSupport(_app) with
|
||||||
class NettyRemoteServer(val remoteSupport: NettyRemoteSupport, val loader: Option[ClassLoader]) {
|
class NettyRemoteServer(val remoteSupport: NettyRemoteSupport, val loader: Option[ClassLoader]) {
|
||||||
val log = Logging(remoteSupport.app, this)
|
val log = Logging(remoteSupport.app, this)
|
||||||
import remoteSupport.serverSettings._
|
import remoteSupport.serverSettings._
|
||||||
import remoteSupport.app.defaultAddress
|
import remoteSupport.app.address
|
||||||
|
|
||||||
val name = "NettyRemoteServer@" + defaultAddress
|
val name = "NettyRemoteServer@" + address
|
||||||
|
|
||||||
private val factory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool, Executors.newCachedThreadPool)
|
private val factory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool, Executors.newCachedThreadPool)
|
||||||
|
|
||||||
|
|
@ -478,7 +478,7 @@ class NettyRemoteServer(val remoteSupport: NettyRemoteSupport, val loader: Optio
|
||||||
bootstrap.setOption("child.reuseAddress", true)
|
bootstrap.setOption("child.reuseAddress", true)
|
||||||
bootstrap.setOption("child.connectTimeoutMillis", CONNECTION_TIMEOUT.toMillis)
|
bootstrap.setOption("child.connectTimeoutMillis", CONNECTION_TIMEOUT.toMillis)
|
||||||
|
|
||||||
openChannels.add(bootstrap.bind(new InetSocketAddress(defaultAddress.hostname, defaultAddress.port)))
|
openChannels.add(bootstrap.bind(new InetSocketAddress(address.hostname, address.port)))
|
||||||
remoteSupport.notifyListeners(RemoteServerStarted(remoteSupport))
|
remoteSupport.notifyListeners(RemoteServerStarted(remoteSupport))
|
||||||
|
|
||||||
def shutdown() {
|
def shutdown() {
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue