Now doing a 'reply(..)' to remote sender after receiving a remote message through '!' works. Added tests.
Also removed the Logging trait from Actor for lower memory footprint.
This commit is contained in:
parent
85cb032964
commit
9c57c3ba97
11 changed files with 191 additions and 147 deletions
|
|
@ -76,7 +76,7 @@ object AMQP {
|
|||
/**
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class AMQPSupervisor extends Actor {
|
||||
class AMQPSupervisor extends Actor with Logging {
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
private val connections = new ConcurrentHashMap[FaultTolerantConnectionActor, FaultTolerantConnectionActor]
|
||||
|
|
@ -516,7 +516,7 @@ object AMQP {
|
|||
/**
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
trait FaultTolerantConnectionActor extends Actor {
|
||||
trait FaultTolerantConnectionActor extends Actor with Logging {
|
||||
val reconnectionTimer = new Timer
|
||||
|
||||
var connection: Connection = _
|
||||
|
|
|
|||
|
|
@ -12,7 +12,7 @@ import se.scalablesolutions.akka.stm.Transaction.Global._
|
|||
import se.scalablesolutions.akka.stm.TransactionManagement._
|
||||
import se.scalablesolutions.akka.stm.TransactionManagement
|
||||
import se.scalablesolutions.akka.remote.protobuf.RemoteProtocol.RemoteRequest
|
||||
import se.scalablesolutions.akka.remote.{RemoteProtocolBuilder, RemoteClient, RemoteRequestIdFactory}
|
||||
import se.scalablesolutions.akka.remote.{RemoteNode, RemoteServer, RemoteClient, RemoteProtocolBuilder, RemoteRequestIdFactory}
|
||||
import se.scalablesolutions.akka.serialization.Serializer
|
||||
import se.scalablesolutions.akka.util.{HashCode, Logging, UUID}
|
||||
|
||||
|
|
@ -242,7 +242,7 @@ object Actor extends Logging {
|
|||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
trait Actor extends TransactionManagement with Logging {
|
||||
trait Actor extends TransactionManagement {
|
||||
implicit protected val self: Option[Actor] = Some(this)
|
||||
// Only mutable for RemoteServer in order to maintain identity across nodes
|
||||
private[akka] var _uuid = UUID.newUuid.toString
|
||||
|
|
@ -834,6 +834,12 @@ trait Actor extends TransactionManagement with Logging {
|
|||
|
||||
requestBuilder.setSourceHostname(host)
|
||||
requestBuilder.setSourcePort(port)
|
||||
|
||||
if (RemoteServer.serverFor(host, port).isEmpty) {
|
||||
val server = new RemoteServer
|
||||
server.start(host, port)
|
||||
}
|
||||
RemoteServer.actorsFor(RemoteServer.Address(host, port)).actors.put(sender.get.getId, sender.get)
|
||||
}
|
||||
RemoteProtocolBuilder.setMessage(message, requestBuilder)
|
||||
RemoteClient.clientFor(_remoteAddress.get).send(requestBuilder.build, None)
|
||||
|
|
|
|||
|
|
@ -95,7 +95,7 @@ private[akka] object ClusterActor {
|
|||
* Provides most of the behavior out of the box
|
||||
* only needs to be gives hooks into the underlaying cluster impl.
|
||||
*/
|
||||
abstract class BasicClusterActor extends ClusterActor {
|
||||
abstract class BasicClusterActor extends ClusterActor with Logging {
|
||||
import ClusterActor._
|
||||
type ADDR_T
|
||||
|
||||
|
|
|
|||
|
|
@ -106,7 +106,7 @@ object RemoteServer {
|
|||
}
|
||||
}
|
||||
|
||||
private[remote] def serverFor(hostname: String, port: Int): Option[RemoteServer] = {
|
||||
private[akka] def serverFor(hostname: String, port: Int): Option[RemoteServer] = {
|
||||
val server = remoteServers.get(Address(hostname, port))
|
||||
if (server eq null) None
|
||||
else Some(server)
|
||||
|
|
@ -141,8 +141,7 @@ class RemoteServer extends Logging {
|
|||
private var hostname = RemoteServer.HOSTNAME
|
||||
private var port = RemoteServer.PORT
|
||||
|
||||
@volatile private var isRunning = false
|
||||
@volatile private var isConfigured = false
|
||||
@volatile private var _isRunning = false
|
||||
|
||||
private val factory = new NioServerSocketChannelFactory(
|
||||
Executors.newCachedThreadPool,
|
||||
|
|
@ -153,6 +152,8 @@ class RemoteServer extends Logging {
|
|||
// group of open channels, used for clean-up
|
||||
private val openChannels: ChannelGroup = new DefaultChannelGroup("akka-remote-server")
|
||||
|
||||
def isRunning = _isRunning
|
||||
|
||||
def start: Unit = start(None)
|
||||
|
||||
def start(loader: Option[ClassLoader]): Unit = start(hostname, port, loader)
|
||||
|
|
@ -161,7 +162,7 @@ class RemoteServer extends Logging {
|
|||
|
||||
def start(_hostname: String, _port: Int, loader: Option[ClassLoader]): Unit = synchronized {
|
||||
try {
|
||||
if (!isRunning) {
|
||||
if (!_isRunning) {
|
||||
hostname = _hostname
|
||||
port = _port
|
||||
log.info("Starting remote server at [%s:%s]", hostname, port)
|
||||
|
|
@ -174,7 +175,7 @@ class RemoteServer extends Logging {
|
|||
bootstrap.setOption("child.reuseAddress", true)
|
||||
bootstrap.setOption("child.connectTimeoutMillis", RemoteServer.CONNECTION_TIMEOUT_MILLIS)
|
||||
openChannels.add(bootstrap.bind(new InetSocketAddress(hostname, port)))
|
||||
isRunning = true
|
||||
_isRunning = true
|
||||
Cluster.registerLocalNode(hostname, port)
|
||||
}
|
||||
} catch {
|
||||
|
|
@ -182,31 +183,37 @@ class RemoteServer extends Logging {
|
|||
}
|
||||
}
|
||||
|
||||
def shutdown = if (isRunning) {
|
||||
def shutdown = synchronized {
|
||||
if (_isRunning) {
|
||||
RemoteServer.unregister(hostname, port)
|
||||
openChannels.disconnect
|
||||
openChannels.close.awaitUninterruptibly
|
||||
bootstrap.releaseExternalResources
|
||||
Cluster.deregisterLocalNode(hostname, port)
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: register active object in RemoteServer as well
|
||||
|
||||
/**
|
||||
* Register Remote Actor by the Actor's 'id' field.
|
||||
*/
|
||||
def register(actor: Actor) = if (isRunning) {
|
||||
def register(actor: Actor) = synchronized {
|
||||
if (_isRunning) {
|
||||
log.info("Registering server side remote actor [%s] with id [%s]", actor.getClass.getName, actor.getId)
|
||||
RemoteServer.actorsFor(RemoteServer.Address(hostname, port)).actors.put(actor.getId, actor)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Register Remote Actor by a specific 'id' passed as argument.
|
||||
*/
|
||||
def register(id: String, actor: Actor) = if (isRunning) {
|
||||
def register(id: String, actor: Actor) = synchronized {
|
||||
if (_isRunning) {
|
||||
log.info("Registering server side remote actor [%s] with id [%s]", actor.getClass.getName, id)
|
||||
RemoteServer.actorsFor(RemoteServer.Address(hostname, port)).actors.put(id, actor)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
case class Codec(encoder : ChannelHandler, decoder : ChannelHandler)
|
||||
|
|
@ -294,6 +301,7 @@ class RemoteServerHandler(
|
|||
private def dispatchToActor(request: RemoteRequest, channel: Channel) = {
|
||||
log.debug("Dispatching to remote actor [%s]", request.getTarget)
|
||||
val actor = createActor(request.getTarget, request.getUuid, request.getTimeout)
|
||||
actor.start
|
||||
|
||||
val message = RemoteProtocolBuilder.getMessage(request)
|
||||
if (request.getIsOneWay) {
|
||||
|
|
@ -389,19 +397,6 @@ class RemoteServerHandler(
|
|||
}
|
||||
}
|
||||
|
||||
/*
|
||||
private def continueTransaction(request: RemoteRequest) = {
|
||||
val tx = request.tx
|
||||
if (tx.isDefined) {
|
||||
tx.get.reinit
|
||||
TransactionManagement.threadBoundTx.set(tx)
|
||||
setThreadLocalTransaction(tx.transaction)
|
||||
} else {
|
||||
TransactionManagement.threadBoundTx.set(None)
|
||||
setThreadLocalTransaction(null)
|
||||
}
|
||||
}
|
||||
*/
|
||||
private def unescapeArgs(args: scala.List[AnyRef], argClasses: scala.List[Class[_]], timeout: Long) = {
|
||||
val unescapedArgs = new Array[AnyRef](args.size)
|
||||
val unescapedArgClasses = new Array[Class[_]](args.size)
|
||||
|
|
@ -410,7 +405,7 @@ class RemoteServerHandler(
|
|||
val arg = args(i)
|
||||
if (arg.isInstanceOf[String] && arg.asInstanceOf[String].startsWith(AW_PROXY_PREFIX)) {
|
||||
val argString = arg.asInstanceOf[String]
|
||||
val proxyName = argString.replace(AW_PROXY_PREFIX, "") //argString.substring(argString.indexOf("$$ProxiedByAW"), argString.length)
|
||||
val proxyName = argString.replace(AW_PROXY_PREFIX, "")
|
||||
val activeObject = createActiveObject(proxyName, timeout)
|
||||
unescapedArgs(i) = activeObject
|
||||
unescapedArgClasses(i) = Class.forName(proxyName)
|
||||
|
|
@ -440,6 +435,11 @@ class RemoteServerHandler(
|
|||
} else activeObjectOrNull
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new instance of the actor with name, uuid and timeout specified as arguments.
|
||||
* If actor already created then just return it from the registry.
|
||||
* Does not start the actor.
|
||||
*/
|
||||
private def createActor(name: String, uuid: String, timeout: Long): Actor = {
|
||||
val actorOrNull = actors.get(uuid)
|
||||
if (actorOrNull eq null) {
|
||||
|
|
@ -452,7 +452,6 @@ class RemoteServerHandler(
|
|||
newInstance.timeout = timeout
|
||||
newInstance._remoteAddress = None
|
||||
actors.put(uuid, newInstance)
|
||||
newInstance.start
|
||||
newInstance
|
||||
} catch {
|
||||
case e =>
|
||||
|
|
|
|||
|
|
@ -51,10 +51,9 @@ object TransactionManagement extends TransactionManagement {
|
|||
}
|
||||
}
|
||||
|
||||
trait TransactionManagement extends Logging {
|
||||
trait TransactionManagement {
|
||||
|
||||
private[akka] def createNewTransactionSet: CountDownCommitBarrier = {
|
||||
log.trace("Creating new transaction set")
|
||||
val txSet = new CountDownCommitBarrier(1, TransactionManagement.FAIR_TRANSACTIONS)
|
||||
TransactionManagement.transactionSet.set(Some(txSet))
|
||||
txSet
|
||||
|
|
@ -67,12 +66,10 @@ trait TransactionManagement extends Logging {
|
|||
if (tx.isDefined) TransactionManagement.transaction.set(tx)
|
||||
|
||||
private[akka] def clearTransactionSet = {
|
||||
log.trace("Clearing transaction set")
|
||||
TransactionManagement.transactionSet.set(None)
|
||||
}
|
||||
|
||||
private[akka] def clearTransaction = {
|
||||
log.trace("Clearing transaction")
|
||||
TransactionManagement.transaction.set(None)
|
||||
setThreadLocalTransaction(null)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -46,6 +46,27 @@ class RemoteActorSpecActorAsyncSender extends Actor {
|
|||
}
|
||||
}
|
||||
|
||||
class SendOneWayAndReplyReceiverActor extends Actor {
|
||||
def receive = {
|
||||
case "Hello" =>
|
||||
reply("World")
|
||||
}
|
||||
}
|
||||
|
||||
class SendOneWayAndReplySenderActor extends Actor {
|
||||
var state: Option[AnyRef] = None
|
||||
var sendTo: Actor = _
|
||||
var latch: CountDownLatch = _
|
||||
|
||||
def sendOff = sendTo ! "Hello"
|
||||
|
||||
def receive = {
|
||||
case msg: AnyRef =>
|
||||
state = Some(msg)
|
||||
latch.countDown
|
||||
}
|
||||
}
|
||||
|
||||
class ClientInitiatedRemoteActorSpec extends JUnitSuite {
|
||||
se.scalablesolutions.akka.config.Config.config
|
||||
|
||||
|
|
@ -82,11 +103,30 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite {
|
|||
val actor = new RemoteActorSpecActorUnidirectional
|
||||
actor.makeRemote(HOSTNAME, PORT1)
|
||||
actor.start
|
||||
val result = actor ! "OneWay"
|
||||
actor ! "OneWay"
|
||||
assert(Global.oneWay.await(1, TimeUnit.SECONDS))
|
||||
actor.stop
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldSendOneWayAndReceiveReply = {
|
||||
val actor = new SendOneWayAndReplyReceiverActor
|
||||
actor.makeRemote(HOSTNAME, PORT1)
|
||||
actor.start
|
||||
val latch = new CountDownLatch(1)
|
||||
val sender = new SendOneWayAndReplySenderActor
|
||||
sender.setReplyToAddress(HOSTNAME, PORT2)
|
||||
sender.sendTo = actor
|
||||
sender.latch = latch
|
||||
sender.start
|
||||
sender.sendOff
|
||||
assert(latch.await(1, TimeUnit.SECONDS))
|
||||
assert(sender.state.isDefined === true)
|
||||
assert("World" === sender.state.get.asInstanceOf[String])
|
||||
actor.stop
|
||||
sender.stop
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldSendReplyAsync = {
|
||||
val actor = new RemoteActorSpecActorBidirectional
|
||||
|
|
@ -101,6 +141,7 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite {
|
|||
def shouldSendRemoteReply = {
|
||||
implicit val timeout = 500000000L
|
||||
val actor = new RemoteActorSpecActorBidirectional
|
||||
actor.setReplyToAddress(HOSTNAME, PORT2)
|
||||
actor.makeRemote(HOSTNAME, PORT2)
|
||||
actor.start
|
||||
|
||||
|
|
@ -128,3 +169,4 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite {
|
|||
actor.stop
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -77,7 +77,7 @@ class RemoteSupervisorSpec extends JUnitSuite {
|
|||
|
||||
new Thread(new Runnable() {
|
||||
def run = {
|
||||
RemoteNode.start
|
||||
RemoteNode.start(RemoteServer.HOSTNAME, 9988)
|
||||
}
|
||||
}).start
|
||||
Thread.sleep(1000)
|
||||
|
|
@ -335,7 +335,7 @@ class RemoteSupervisorSpec extends JUnitSuite {
|
|||
// implementation of the Actors we want to use.
|
||||
|
||||
pingpong1 = new RemotePingPong1Actor
|
||||
pingpong1.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
|
||||
pingpong1.makeRemote(RemoteServer.HOSTNAME, 9988)
|
||||
|
||||
val factory = SupervisorFactory(
|
||||
SupervisorConfig(
|
||||
|
|
@ -350,7 +350,7 @@ class RemoteSupervisorSpec extends JUnitSuite {
|
|||
|
||||
def getSingleActorOneForOneSupervisor: Supervisor = {
|
||||
pingpong1 = new RemotePingPong1Actor
|
||||
pingpong1.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
|
||||
pingpong1.makeRemote(RemoteServer.HOSTNAME, 9988)
|
||||
|
||||
val factory = SupervisorFactory(
|
||||
SupervisorConfig(
|
||||
|
|
@ -364,11 +364,11 @@ class RemoteSupervisorSpec extends JUnitSuite {
|
|||
|
||||
def getMultipleActorsAllForOneConf: Supervisor = {
|
||||
pingpong1 = new RemotePingPong1Actor
|
||||
pingpong1.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
|
||||
pingpong1.makeRemote(RemoteServer.HOSTNAME, 9988)
|
||||
pingpong2 = new RemotePingPong2Actor
|
||||
pingpong2.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
|
||||
pingpong2.makeRemote(RemoteServer.HOSTNAME, 9988)
|
||||
pingpong3 = new RemotePingPong3Actor
|
||||
pingpong3.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
|
||||
pingpong3.makeRemote(RemoteServer.HOSTNAME, 9988)
|
||||
|
||||
val factory = SupervisorFactory(
|
||||
SupervisorConfig(
|
||||
|
|
@ -390,11 +390,11 @@ class RemoteSupervisorSpec extends JUnitSuite {
|
|||
|
||||
def getMultipleActorsOneForOneConf: Supervisor = {
|
||||
pingpong1 = new RemotePingPong1Actor
|
||||
pingpong1.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
|
||||
pingpong1.makeRemote(RemoteServer.HOSTNAME, 9988)
|
||||
pingpong2 = new RemotePingPong2Actor
|
||||
pingpong2.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
|
||||
pingpong2.makeRemote(RemoteServer.HOSTNAME, 9988)
|
||||
pingpong3 = new RemotePingPong3Actor
|
||||
pingpong3.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
|
||||
pingpong3.makeRemote(RemoteServer.HOSTNAME, 9988)
|
||||
|
||||
val factory = SupervisorFactory(
|
||||
SupervisorConfig(
|
||||
|
|
@ -416,11 +416,11 @@ class RemoteSupervisorSpec extends JUnitSuite {
|
|||
|
||||
def getNestedSupervisorsAllForOneConf: Supervisor = {
|
||||
pingpong1 = new RemotePingPong1Actor
|
||||
pingpong1.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
|
||||
pingpong1.makeRemote(RemoteServer.HOSTNAME, 9988)
|
||||
pingpong2 = new RemotePingPong2Actor
|
||||
pingpong2.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
|
||||
pingpong2.makeRemote(RemoteServer.HOSTNAME, 9988)
|
||||
pingpong3 = new RemotePingPong3Actor
|
||||
pingpong3.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
|
||||
pingpong3.makeRemote(RemoteServer.HOSTNAME, 9988)
|
||||
|
||||
val factory = SupervisorFactory(
|
||||
SupervisorConfig(
|
||||
|
|
|
|||
|
|
@ -65,7 +65,7 @@ class Transformer(producer: Actor) extends Actor {
|
|||
}
|
||||
}
|
||||
|
||||
class Subscriber(name:String, uri: String) extends Actor with Consumer {
|
||||
class Subscriber(name:String, uri: String) extends Actor with Consumer with Logging {
|
||||
def endpointUri = uri
|
||||
|
||||
protected def receive = {
|
||||
|
|
|
|||
|
|
@ -127,7 +127,7 @@ class PersistentSimpleService extends Transactor {
|
|||
}
|
||||
|
||||
@Path("/chat")
|
||||
class Chat extends Actor {
|
||||
class Chat extends Actor with Logging {
|
||||
case class Chat(val who: String, val what: String, val msg: String)
|
||||
|
||||
@Suspend
|
||||
|
|
|
|||
|
|
@ -245,7 +245,7 @@ trait BasicAuthenticationActor extends AuthenticationActor[BasicCredentials] {
|
|||
* class to create an authenticator. Don't forget to set the authenticator FQN in the
|
||||
* rest-part of the akka config
|
||||
*/
|
||||
trait DigestAuthenticationActor extends AuthenticationActor[DigestCredentials] {
|
||||
trait DigestAuthenticationActor extends AuthenticationActor[DigestCredentials] with Logging {
|
||||
import Enc._
|
||||
|
||||
private object InvalidateNonces
|
||||
|
|
@ -346,7 +346,7 @@ import org.ietf.jgss.GSSContext
|
|||
import org.ietf.jgss.GSSCredential
|
||||
import org.ietf.jgss.GSSManager
|
||||
|
||||
trait SpnegoAuthenticationActor extends AuthenticationActor[SpnegoCredentials] {
|
||||
trait SpnegoAuthenticationActor extends AuthenticationActor[SpnegoCredentials] with Logging {
|
||||
override def unauthorized =
|
||||
Response.status(401).header("WWW-Authenticate", "Negotiate").build
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue