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:
Jonas Bonér 2010-04-06 12:45:09 +02:00
parent 85cb032964
commit 9c57c3ba97
11 changed files with 191 additions and 147 deletions

View file

@ -53,7 +53,7 @@ object AMQP {
initReconnectDelay: Long) = initReconnectDelay: Long) =
supervisor.newProducer( supervisor.newProducer(
config, hostname, port, exchangeName, returnListener, shutdownListener, initReconnectDelay) config, hostname, port, exchangeName, returnListener, shutdownListener, initReconnectDelay)
def newConsumer( def newConsumer(
config: ConnectionParameters, config: ConnectionParameters,
hostname: String, hostname: String,
@ -76,9 +76,9 @@ object AMQP {
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class AMQPSupervisor extends Actor { class AMQPSupervisor extends Actor with Logging {
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
private val connections = new ConcurrentHashMap[FaultTolerantConnectionActor, FaultTolerantConnectionActor] private val connections = new ConcurrentHashMap[FaultTolerantConnectionActor, FaultTolerantConnectionActor]
faultHandler = Some(OneForOneStrategy(5, 5000)) faultHandler = Some(OneForOneStrategy(5, 5000))
@ -114,7 +114,7 @@ object AMQP {
initReconnectDelay: Long, initReconnectDelay: Long,
passive: Boolean, passive: Boolean,
durable: Boolean, durable: Boolean,
autoDelete: Boolean, autoDelete: Boolean,
configurationArguments: Map[String, AnyRef]): Consumer = { configurationArguments: Map[String, AnyRef]): Consumer = {
val consumer = new Consumer( val consumer = new Consumer(
new ConnectionFactory(config), new ConnectionFactory(config),
@ -185,10 +185,10 @@ object AMQP {
*/ */
class MessageConsumerListener(val queueName: String, class MessageConsumerListener(val queueName: String,
val routingKey: String, val routingKey: String,
val exclusive: Boolean, val exclusive: Boolean,
val autoDelete: Boolean, val autoDelete: Boolean,
val isUsingExistingQueue: Boolean, val isUsingExistingQueue: Boolean,
val actor: Actor) extends AMQPMessage { val actor: Actor) extends AMQPMessage {
/** /**
* Creates a non-exclusive, non-autodelete message listener. * Creates a non-exclusive, non-autodelete message listener.
*/ */
@ -238,8 +238,8 @@ object AMQP {
object MessageConsumerListener { object MessageConsumerListener {
def apply(queueName: String, def apply(queueName: String,
routingKey: String, routingKey: String,
exclusive: Boolean, exclusive: Boolean,
autoDelete: Boolean, autoDelete: Boolean,
isUsingExistingQueue: Boolean, isUsingExistingQueue: Boolean,
actor: Actor) = actor: Actor) =
new MessageConsumerListener(queueName, routingKey, exclusive, autoDelete, isUsingExistingQueue, actor) new MessageConsumerListener(queueName, routingKey, exclusive, autoDelete, isUsingExistingQueue, actor)
@ -356,7 +356,7 @@ object AMQP {
val initReconnectDelay: Long, val initReconnectDelay: Long,
val passive: Boolean, val passive: Boolean,
val durable: Boolean, val durable: Boolean,
val autoDelete: Boolean, val autoDelete: Boolean,
val configurationArguments: Map[java.lang.String, Object]) val configurationArguments: Map[java.lang.String, Object])
extends FaultTolerantConnectionActor { extends FaultTolerantConnectionActor {
consumer: Consumer => consumer: Consumer =>
@ -365,7 +365,7 @@ object AMQP {
faultHandler = Some(OneForOneStrategy(5, 5000)) faultHandler = Some(OneForOneStrategy(5, 5000))
trapExit = List(classOf[Throwable]) trapExit = List(classOf[Throwable])
//FIXME use better strategy to convert scala.immutable.Map to java.util.Map //FIXME use better strategy to convert scala.immutable.Map to java.util.Map
private val jConfigMap = configurationArguments.foldLeft(new java.util.HashMap[String,Object]){ (m,kv) => { m.put(kv._1,kv._2); m } } private val jConfigMap = configurationArguments.foldLeft(new java.util.HashMap[String,Object]){ (m,kv) => { m.put(kv._1,kv._2); m } }
@ -424,13 +424,13 @@ object AMQP {
private def registerListener(listener: MessageConsumerListener) = { private def registerListener(listener: MessageConsumerListener) = {
log.debug("Register MessageConsumerListener %s", listener.toString(exchangeName)) log.debug("Register MessageConsumerListener %s", listener.toString(exchangeName))
listeners.put(listener, listener) listeners.put(listener, listener)
if (!listener.isUsingExistingQueue) { if (!listener.isUsingExistingQueue) {
log.debug("Declaring new queue for MessageConsumerListener [%s]", listener.queueName) log.debug("Declaring new queue for MessageConsumerListener [%s]", listener.queueName)
channel.queueDeclare( channel.queueDeclare(
listener.queueName, listener.queueName,
passive, durable, passive, durable,
listener.exclusive, listener.autoDelete, listener.exclusive, listener.autoDelete,
jConfigMap) jConfigMap)
} }
@ -516,7 +516,7 @@ object AMQP {
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
trait FaultTolerantConnectionActor extends Actor { trait FaultTolerantConnectionActor extends Actor with Logging {
val reconnectionTimer = new Timer val reconnectionTimer = new Timer
var connection: Connection = _ var connection: Connection = _
@ -548,7 +548,7 @@ object AMQP {
def bindQueue(name: String) { def bindQueue(name: String) {
channel.queueBind(name, exchangeName, name) channel.queueBind(name, exchangeName, name)
} }
def createBindQueue: String = { def createBindQueue: String = {
val name = createQueue val name = createQueue
channel.queueBind(name, exchangeName, name) channel.queueBind(name, exchangeName, name)

View file

@ -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.stm.TransactionManagement import se.scalablesolutions.akka.stm.TransactionManagement
import se.scalablesolutions.akka.remote.protobuf.RemoteProtocol.RemoteRequest 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.serialization.Serializer
import se.scalablesolutions.akka.util.{HashCode, Logging, UUID} import se.scalablesolutions.akka.util.{HashCode, Logging, UUID}
@ -29,7 +29,7 @@ import java.util.concurrent.locks.{Lock, ReentrantLock}
* Implements the Transactor abstraction. E.g. a transactional actor. * Implements the Transactor abstraction. E.g. a transactional actor.
* <p/> * <p/>
* Equivalent to invoking the <code>makeTransactionRequired</code> method in the body of the <code>Actor</code * Equivalent to invoking the <code>makeTransactionRequired</code> method in the body of the <code>Actor</code
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
trait Transactor extends Actor { trait Transactor extends Actor {
@ -40,7 +40,7 @@ trait Transactor extends Actor {
* Extend this abstract class to create a remote actor. * Extend this abstract class to create a remote actor.
* <p/> * <p/>
* Equivalent to invoking the <code>makeRemote(..)</code> method in the body of the <code>Actor</code * Equivalent to invoking the <code>makeRemote(..)</code> method in the body of the <code>Actor</code
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
abstract class RemoteActor(hostname: String, port: Int) extends Actor { abstract class RemoteActor(hostname: String, port: Int) extends Actor {
@ -72,7 +72,7 @@ class ActorMessageInvoker(val actor: Actor) extends MessageInvoker {
/** /**
* Utility class with factory methods for creating Actors. * Utility class with factory methods for creating Actors.
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
object Actor extends Logging { object Actor extends Logging {
@ -82,14 +82,14 @@ object Actor extends Logging {
val PORT = config.getInt("akka.remote.server.port", 9999) val PORT = config.getInt("akka.remote.server.port", 9999)
object Sender { object Sender {
@deprecated("import Actor.Sender.Self is not needed anymore, just use 'actor ! msg'") @deprecated("import Actor.Sender.Self is not needed anymore, just use 'actor ! msg'")
object Self object Self
} }
/** /**
* Use to create an anonymous event-driven actor. * Use to create an anonymous event-driven actor.
* <p/> * <p/>
* The actor is created with a 'permanent' life-cycle configuration, which means that * The actor is created with a 'permanent' life-cycle configuration, which means that
* if the actor is supervised and dies it will be restarted. * if the actor is supervised and dies it will be restarted.
* <p/> * <p/>
* The actor is started when created. * The actor is started when created.
@ -111,7 +111,7 @@ object Actor extends Logging {
/** /**
* Use to create an anonymous transactional event-driven actor. * Use to create an anonymous transactional event-driven actor.
* <p/> * <p/>
* The actor is created with a 'permanent' life-cycle configuration, which means that * The actor is created with a 'permanent' life-cycle configuration, which means that
* if the actor is supervised and dies it will be restarted. * if the actor is supervised and dies it will be restarted.
* <p/> * <p/>
* The actor is started when created. * The actor is started when created.
@ -131,7 +131,7 @@ object Actor extends Logging {
} }
/** /**
* Use to create an anonymous event-driven actor with a 'temporary' life-cycle configuration, * Use to create an anonymous event-driven actor with a 'temporary' life-cycle configuration,
* which means that if the actor is supervised and dies it will *not* be restarted. * which means that if the actor is supervised and dies it will *not* be restarted.
* <p/> * <p/>
* The actor is started when created. * The actor is started when created.
@ -153,7 +153,7 @@ object Actor extends Logging {
/** /**
* Use to create an anonymous event-driven remote actor. * Use to create an anonymous event-driven remote actor.
* <p/> * <p/>
* The actor is created with a 'permanent' life-cycle configuration, which means that * The actor is created with a 'permanent' life-cycle configuration, which means that
* if the actor is supervised and dies it will be restarted. * if the actor is supervised and dies it will be restarted.
* <p/> * <p/>
* The actor is started when created. * The actor is started when created.
@ -176,7 +176,7 @@ object Actor extends Logging {
/** /**
* Use to create an anonymous event-driven actor with both an init block and a message loop block. * Use to create an anonymous event-driven actor with both an init block and a message loop block.
* <p/> * <p/>
* The actor is created with a 'permanent' life-cycle configuration, which means that * The actor is created with a 'permanent' life-cycle configuration, which means that
* if the actor is supervised and dies it will be restarted. * if the actor is supervised and dies it will be restarted.
* <p/> * <p/>
* The actor is started when created. * The actor is started when created.
@ -203,10 +203,10 @@ object Actor extends Logging {
} }
/** /**
* Use to spawn out a block of code in an event-driven actor. Will shut actor down when * Use to spawn out a block of code in an event-driven actor. Will shut actor down when
* the block has been executed. * the block has been executed.
* <p/> * <p/>
* NOTE: If used from within an Actor then has to be qualified with 'Actor.spawn' since * NOTE: If used from within an Actor then has to be qualified with 'Actor.spawn' since
* there is a method 'spawn[ActorType]' in the Actor trait already. * there is a method 'spawn[ActorType]' in the Actor trait already.
* Example: * Example:
* <pre> * <pre>
@ -242,7 +242,7 @@ object Actor extends Logging {
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
trait Actor extends TransactionManagement with Logging { trait Actor extends TransactionManagement {
implicit protected val self: Option[Actor] = Some(this) implicit protected val self: Option[Actor] = Some(this)
// Only mutable for RemoteServer in order to maintain identity across nodes // Only mutable for RemoteServer in order to maintain identity across nodes
private[akka] var _uuid = UUID.newUuid.toString private[akka] var _uuid = UUID.newUuid.toString
@ -264,7 +264,7 @@ trait Actor extends TransactionManagement with Logging {
private[akka] val _mailbox: Deque[MessageInvocation] = new ConcurrentLinkedDeque[MessageInvocation] private[akka] val _mailbox: Deque[MessageInvocation] = new ConcurrentLinkedDeque[MessageInvocation]
/** /**
* This lock ensures thread safety in the dispatching: only one message can * This lock ensures thread safety in the dispatching: only one message can
* be dispatched at once on the actor. * be dispatched at once on the actor.
*/ */
private[akka] val _dispatcherLock: Lock = new ReentrantLock private[akka] val _dispatcherLock: Lock = new ReentrantLock
@ -495,7 +495,7 @@ trait Actor extends TransactionManagement with Logging {
* If invoked from within an actor then the actor reference is implicitly passed on as the implicit 'sender' argument. * If invoked from within an actor then the actor reference is implicitly passed on as the implicit 'sender' argument.
* <p/> * <p/>
* *
* This actor 'sender' reference is then available in the receiving actor in the 'sender' member variable, * This actor 'sender' reference is then available in the receiving actor in the 'sender' member variable,
* if invoked from within an Actor. If not then no sender is available. * if invoked from within an Actor. If not then no sender is available.
* <pre> * <pre>
* actor ! message * actor ! message
@ -567,7 +567,7 @@ trait Actor extends TransactionManagement with Logging {
/** /**
* Forwards the message and passes the original sender actor as the sender. * Forwards the message and passes the original sender actor as the sender.
* <p/> * <p/>
* Works with both '!' and '!!'. * Works with both '!' and '!!'.
*/ */
def forward(message: Any)(implicit sender: Option[Actor] = None) = { def forward(message: Any)(implicit sender: Option[Actor] = None) = {
if (_isKilled) throw new ActorKilledException("Actor [" + toString + "] has been killed, can't respond to messages") if (_isKilled) throw new ActorKilledException("Actor [" + toString + "] has been killed, can't respond to messages")
@ -807,7 +807,7 @@ trait Actor extends TransactionManagement with Logging {
protected[akka] def postMessageToMailbox(message: Any, sender: Option[Actor]): Unit = { protected[akka] def postMessageToMailbox(message: Any, sender: Option[Actor]): Unit = {
joinTransaction(message) joinTransaction(message)
if (_remoteAddress.isDefined) { if (_remoteAddress.isDefined) {
val requestBuilder = RemoteRequest.newBuilder val requestBuilder = RemoteRequest.newBuilder
.setId(RemoteRequestIdFactory.nextId) .setId(RemoteRequestIdFactory.nextId)
@ -829,11 +829,17 @@ trait Actor extends TransactionManagement with Logging {
requestBuilder.setSourceUuid(s.uuid) requestBuilder.setSourceUuid(s.uuid)
val (host, port) = s._replyToAddress.map(a => (a.getHostName,a.getPort)).getOrElse((Actor.HOSTNAME, Actor.PORT)) val (host, port) = s._replyToAddress.map(a => (a.getHostName,a.getPort)).getOrElse((Actor.HOSTNAME, Actor.PORT))
Actor.log.debug("Setting sending actor as %s @ %s:%s", s.getClass.getName, host, port) Actor.log.debug("Setting sending actor as %s @ %s:%s", s.getClass.getName, host, port)
requestBuilder.setSourceHostname(host) requestBuilder.setSourceHostname(host)
requestBuilder.setSourcePort(port) 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) RemoteProtocolBuilder.setMessage(message, requestBuilder)
RemoteClient.clientFor(_remoteAddress.get).send(requestBuilder.build, None) RemoteClient.clientFor(_remoteAddress.get).send(requestBuilder.build, None)
@ -846,13 +852,13 @@ trait Actor extends TransactionManagement with Logging {
else invocation.send else invocation.send
} }
} }
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout( protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout(
message: Any, message: Any,
timeout: Long, timeout: Long,
senderFuture: Option[CompletableFuture]): CompletableFuture = { senderFuture: Option[CompletableFuture]): CompletableFuture = {
joinTransaction(message) joinTransaction(message)
if (_remoteAddress.isDefined) { if (_remoteAddress.isDefined) {
val requestBuilder = RemoteRequest.newBuilder val requestBuilder = RemoteRequest.newBuilder
.setId(RemoteRequestIdFactory.nextId) .setId(RemoteRequestIdFactory.nextId)
@ -882,7 +888,7 @@ trait Actor extends TransactionManagement with Logging {
private def joinTransaction(message: Any) = if (isTransactionSetInScope) { private def joinTransaction(message: Any) = if (isTransactionSetInScope) {
// FIXME test to run bench without this trace call // FIXME test to run bench without this trace call
Actor.log.trace("Joining transaction set [%s];\n\tactor %s\n\twith message [%s]", Actor.log.trace("Joining transaction set [%s];\n\tactor %s\n\twith message [%s]",
getTransactionSetInScope, toString, message) getTransactionSetInScope, toString, message)
getTransactionSetInScope.incParties getTransactionSetInScope.incParties
} }
@ -930,7 +936,7 @@ trait Actor extends TransactionManagement with Logging {
else { else {
topLevelTransaction = true // FIXME create a new internal atomic block that can wait for X seconds if top level tx topLevelTransaction = true // FIXME create a new internal atomic block that can wait for X seconds if top level tx
if (isTransactor) { if (isTransactor) {
Actor.log.trace("Creating a new transaction set (top-level transaction)\n\tfor actor %s\n\twith message %s", Actor.log.trace("Creating a new transaction set (top-level transaction)\n\tfor actor %s\n\twith message %s",
toString, messageHandle) toString, messageHandle)
Some(createNewTransactionSet) Some(createNewTransactionSet)
} else None } else None
@ -959,13 +965,13 @@ trait Actor extends TransactionManagement with Logging {
case e: IllegalStateException => {} case e: IllegalStateException => {}
case e => case e =>
// abort transaction set // abort transaction set
if (isTransactionSetInScope) try { if (isTransactionSetInScope) try {
getTransactionSetInScope.abort getTransactionSetInScope.abort
} catch { case e: IllegalStateException => {} } } catch { case e: IllegalStateException => {} }
Actor.log.error(e, "Exception when invoking \n\tactor [%s] \n\twith message [%s]", this, message) Actor.log.error(e, "Exception when invoking \n\tactor [%s] \n\twith message [%s]", this, message)
if (senderFuture.isDefined) senderFuture.get.completeWithException(this, e) if (senderFuture.isDefined) senderFuture.get.completeWithException(this, e)
clearTransaction clearTransaction
if (topLevelTransaction) clearTransactionSet if (topLevelTransaction) clearTransactionSet

View file

@ -63,9 +63,9 @@ trait Cluster {
*/ */
trait ClusterActor extends Actor with Cluster { trait ClusterActor extends Actor with Cluster {
val name = config.getString("akka.remote.cluster.name") getOrElse "default" val name = config.getString("akka.remote.cluster.name") getOrElse "default"
@volatile protected var serializer : Serializer = _ @volatile protected var serializer : Serializer = _
private[remote] def setSerializer(s : Serializer) : Unit = serializer = s private[remote] def setSerializer(s : Serializer) : Unit = serializer = s
} }
@ -95,7 +95,7 @@ private[akka] object ClusterActor {
* Provides most of the behavior out of the box * Provides most of the behavior out of the box
* only needs to be gives hooks into the underlaying cluster impl. * 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._ import ClusterActor._
type ADDR_T type ADDR_T
@ -235,13 +235,13 @@ abstract class BasicClusterActor extends ClusterActor {
object Cluster extends Cluster with Logging { object Cluster extends Cluster with Logging {
lazy val DEFAULT_SERIALIZER_CLASS_NAME = Serializer.Java.getClass.getName lazy val DEFAULT_SERIALIZER_CLASS_NAME = Serializer.Java.getClass.getName
@volatile private[remote] var clusterActor: Option[ClusterActor] = None @volatile private[remote] var clusterActor: Option[ClusterActor] = None
private[remote] def createClusterActor(loader : ClassLoader): Option[ClusterActor] = { private[remote] def createClusterActor(loader : ClassLoader): Option[ClusterActor] = {
val name = config.getString("akka.remote.cluster.actor") val name = config.getString("akka.remote.cluster.actor")
if (name.isEmpty) throw new IllegalArgumentException( if (name.isEmpty) throw new IllegalArgumentException(
"Can't start cluster since the 'akka.remote.cluster.actor' configuration option is not defined") "Can't start cluster since the 'akka.remote.cluster.actor' configuration option is not defined")
val serializer = Class.forName(config.getString("akka.remote.cluster.serializer", DEFAULT_SERIALIZER_CLASS_NAME)).newInstance.asInstanceOf[Serializer] val serializer = Class.forName(config.getString("akka.remote.cluster.serializer", DEFAULT_SERIALIZER_CLASS_NAME)).newInstance.asInstanceOf[Serializer]
serializer.classLoader = Some(loader) serializer.classLoader = Some(loader)
try { try {

View file

@ -33,7 +33,7 @@ import org.jboss.netty.handler.codec.compression.{ZlibEncoder, ZlibDecoder}
* <pre> * <pre>
* RemoteNode.start(hostname, port) * RemoteNode.start(hostname, port)
* </pre> * </pre>
* *
* You can specify the class loader to use to load the remote actors. * You can specify the class loader to use to load the remote actors.
* <pre> * <pre>
* RemoteNode.start(hostname, port, classLoader) * RemoteNode.start(hostname, port, classLoader)
@ -87,15 +87,15 @@ object RemoteServer {
that.asInstanceOf[Address].port == port that.asInstanceOf[Address].port == port
} }
} }
class RemoteActorSet { class RemoteActorSet {
val actors = new ConcurrentHashMap[String, Actor] val actors = new ConcurrentHashMap[String, Actor]
val activeObjects = new ConcurrentHashMap[String, AnyRef] val activeObjects = new ConcurrentHashMap[String, AnyRef]
} }
private val remoteActorSets = new ConcurrentHashMap[Address, RemoteActorSet] private val remoteActorSets = new ConcurrentHashMap[Address, RemoteActorSet]
private val remoteServers = new ConcurrentHashMap[Address, RemoteServer] private val remoteServers = new ConcurrentHashMap[Address, RemoteServer]
private[akka] def actorsFor(remoteServerAddress: RemoteServer.Address): RemoteActorSet = { private[akka] def actorsFor(remoteServerAddress: RemoteServer.Address): RemoteActorSet = {
val set = remoteActorSets.get(remoteServerAddress) val set = remoteActorSets.get(remoteServerAddress)
if (set ne null) set if (set ne null) set
@ -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)) val server = remoteServers.get(Address(hostname, port))
if (server eq null) None if (server eq null) None
else Some(server) else Some(server)
@ -114,7 +114,7 @@ object RemoteServer {
private[remote] def register(hostname: String, port: Int, server: RemoteServer) = private[remote] def register(hostname: String, port: Int, server: RemoteServer) =
remoteServers.put(Address(hostname, port), server) remoteServers.put(Address(hostname, port), server)
private[remote] def unregister(hostname: String, port: Int) = private[remote] def unregister(hostname: String, port: Int) =
remoteServers.remove(Address(hostname, port)) remoteServers.remove(Address(hostname, port))
} }
@ -141,8 +141,7 @@ class RemoteServer extends Logging {
private var hostname = RemoteServer.HOSTNAME private var hostname = RemoteServer.HOSTNAME
private var port = RemoteServer.PORT private var port = RemoteServer.PORT
@volatile private var isRunning = false @volatile private var _isRunning = false
@volatile private var isConfigured = false
private val factory = new NioServerSocketChannelFactory( private val factory = new NioServerSocketChannelFactory(
Executors.newCachedThreadPool, Executors.newCachedThreadPool,
@ -153,6 +152,8 @@ class RemoteServer extends Logging {
// group of open channels, used for clean-up // group of open channels, used for clean-up
private val openChannels: ChannelGroup = new DefaultChannelGroup("akka-remote-server") private val openChannels: ChannelGroup = new DefaultChannelGroup("akka-remote-server")
def isRunning = _isRunning
def start: Unit = start(None) def start: Unit = start(None)
def start(loader: Option[ClassLoader]): Unit = start(hostname, port, loader) 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 { def start(_hostname: String, _port: Int, loader: Option[ClassLoader]): Unit = synchronized {
try { try {
if (!isRunning) { if (!_isRunning) {
hostname = _hostname hostname = _hostname
port = _port port = _port
log.info("Starting remote server at [%s:%s]", hostname, port) log.info("Starting remote server at [%s:%s]", hostname, port)
@ -174,20 +175,22 @@ class RemoteServer extends Logging {
bootstrap.setOption("child.reuseAddress", true) bootstrap.setOption("child.reuseAddress", true)
bootstrap.setOption("child.connectTimeoutMillis", RemoteServer.CONNECTION_TIMEOUT_MILLIS) bootstrap.setOption("child.connectTimeoutMillis", RemoteServer.CONNECTION_TIMEOUT_MILLIS)
openChannels.add(bootstrap.bind(new InetSocketAddress(hostname, port))) openChannels.add(bootstrap.bind(new InetSocketAddress(hostname, port)))
isRunning = true _isRunning = true
Cluster.registerLocalNode(hostname, port) Cluster.registerLocalNode(hostname, port)
} }
} catch { } catch {
case e => log.error(e, "Could not start up remote server") case e => log.error(e, "Could not start up remote server")
} }
} }
def shutdown = if (isRunning) { def shutdown = synchronized {
RemoteServer.unregister(hostname, port) if (_isRunning) {
openChannels.disconnect RemoteServer.unregister(hostname, port)
openChannels.close.awaitUninterruptibly openChannels.disconnect
bootstrap.releaseExternalResources openChannels.close.awaitUninterruptibly
Cluster.deregisterLocalNode(hostname, port) bootstrap.releaseExternalResources
Cluster.deregisterLocalNode(hostname, port)
}
} }
// TODO: register active object in RemoteServer as well // TODO: register active object in RemoteServer as well
@ -195,17 +198,21 @@ class RemoteServer extends Logging {
/** /**
* Register Remote Actor by the Actor's 'id' field. * Register Remote Actor by the Actor's 'id' field.
*/ */
def register(actor: Actor) = if (isRunning) { def register(actor: Actor) = synchronized {
log.info("Registering server side remote actor [%s] with id [%s]", actor.getClass.getName, actor.getId) if (_isRunning) {
RemoteServer.actorsFor(RemoteServer.Address(hostname, port)).actors.put(actor.getId, actor) 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. * 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 {
log.info("Registering server side remote actor [%s] with id [%s]", actor.getClass.getName, id) if (_isRunning) {
RemoteServer.actorsFor(RemoteServer.Address(hostname, port)).actors.put(id, actor) 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)
}
} }
} }
@ -232,9 +239,9 @@ class RemoteServerPipelineFactory(
//case "lzf" => Some(Codec(new LzfEncoder, new LzfDecoder)) //case "lzf" => Some(Codec(new LzfEncoder, new LzfDecoder))
case _ => None case _ => None
} }
val remoteServer = new RemoteServerHandler(name, openChannels, loader, actors, activeObjects) val remoteServer = new RemoteServerHandler(name, openChannels, loader, actors, activeObjects)
val stages: Array[ChannelHandler] = val stages: Array[ChannelHandler] =
zipCodec.map(codec => Array(codec.decoder, lenDec, protobufDec, codec.encoder, lenPrep, protobufEnc, remoteServer)) zipCodec.map(codec => Array(codec.decoder, lenDec, protobufDec, codec.encoder, lenPrep, protobufEnc, remoteServer))
.getOrElse(Array(lenDec, protobufDec, lenPrep, protobufEnc, remoteServer)) .getOrElse(Array(lenDec, protobufDec, lenPrep, protobufEnc, remoteServer))
new StaticChannelPipeline(stages: _*) new StaticChannelPipeline(stages: _*)
@ -294,7 +301,8 @@ class RemoteServerHandler(
private def dispatchToActor(request: RemoteRequest, channel: Channel) = { private def dispatchToActor(request: RemoteRequest, channel: Channel) = {
log.debug("Dispatching to remote actor [%s]", request.getTarget) log.debug("Dispatching to remote actor [%s]", request.getTarget)
val actor = createActor(request.getTarget, request.getUuid, request.getTimeout) val actor = createActor(request.getTarget, request.getUuid, request.getTimeout)
actor.start
val message = RemoteProtocolBuilder.getMessage(request) val message = RemoteProtocolBuilder.getMessage(request)
if (request.getIsOneWay) { if (request.getIsOneWay) {
if (request.hasSourceHostname && request.hasSourcePort) { if (request.hasSourceHostname && request.hasSourcePort) {
@ -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) = { private def unescapeArgs(args: scala.List[AnyRef], argClasses: scala.List[Class[_]], timeout: Long) = {
val unescapedArgs = new Array[AnyRef](args.size) val unescapedArgs = new Array[AnyRef](args.size)
val unescapedArgClasses = new Array[Class[_]](args.size) val unescapedArgClasses = new Array[Class[_]](args.size)
@ -410,7 +405,7 @@ class RemoteServerHandler(
val arg = args(i) val arg = args(i)
if (arg.isInstanceOf[String] && arg.asInstanceOf[String].startsWith(AW_PROXY_PREFIX)) { if (arg.isInstanceOf[String] && arg.asInstanceOf[String].startsWith(AW_PROXY_PREFIX)) {
val argString = arg.asInstanceOf[String] 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) val activeObject = createActiveObject(proxyName, timeout)
unescapedArgs(i) = activeObject unescapedArgs(i) = activeObject
unescapedArgClasses(i) = Class.forName(proxyName) unescapedArgClasses(i) = Class.forName(proxyName)
@ -440,6 +435,11 @@ class RemoteServerHandler(
} else activeObjectOrNull } 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 = { private def createActor(name: String, uuid: String, timeout: Long): Actor = {
val actorOrNull = actors.get(uuid) val actorOrNull = actors.get(uuid)
if (actorOrNull eq null) { if (actorOrNull eq null) {
@ -452,7 +452,6 @@ class RemoteServerHandler(
newInstance.timeout = timeout newInstance.timeout = timeout
newInstance._remoteAddress = None newInstance._remoteAddress = None
actors.put(uuid, newInstance) actors.put(uuid, newInstance)
newInstance.start
newInstance newInstance
} catch { } catch {
case e => case e =>

View file

@ -51,10 +51,9 @@ object TransactionManagement extends TransactionManagement {
} }
} }
trait TransactionManagement extends Logging { trait TransactionManagement {
private[akka] def createNewTransactionSet: CountDownCommitBarrier = { private[akka] def createNewTransactionSet: CountDownCommitBarrier = {
log.trace("Creating new transaction set")
val txSet = new CountDownCommitBarrier(1, TransactionManagement.FAIR_TRANSACTIONS) val txSet = new CountDownCommitBarrier(1, TransactionManagement.FAIR_TRANSACTIONS)
TransactionManagement.transactionSet.set(Some(txSet)) TransactionManagement.transactionSet.set(Some(txSet))
txSet txSet
@ -67,12 +66,10 @@ trait TransactionManagement extends Logging {
if (tx.isDefined) TransactionManagement.transaction.set(tx) if (tx.isDefined) TransactionManagement.transaction.set(tx)
private[akka] def clearTransactionSet = { private[akka] def clearTransactionSet = {
log.trace("Clearing transaction set")
TransactionManagement.transactionSet.set(None) TransactionManagement.transactionSet.set(None)
} }
private[akka] def clearTransaction = { private[akka] def clearTransaction = {
log.trace("Clearing transaction")
TransactionManagement.transaction.set(None) TransactionManagement.transaction.set(None)
setThreadLocalTransaction(null) setThreadLocalTransaction(null)
} }
@ -90,4 +87,4 @@ trait TransactionManagement extends Logging {
val option = TransactionManagement.transaction.get val option = TransactionManagement.transaction.get
(option ne null) && option.isDefined (option ne null) && option.isDefined
} }
} }

View file

@ -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 { class ClientInitiatedRemoteActorSpec extends JUnitSuite {
se.scalablesolutions.akka.config.Config.config se.scalablesolutions.akka.config.Config.config
@ -82,11 +103,30 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite {
val actor = new RemoteActorSpecActorUnidirectional val actor = new RemoteActorSpecActorUnidirectional
actor.makeRemote(HOSTNAME, PORT1) actor.makeRemote(HOSTNAME, PORT1)
actor.start actor.start
val result = actor ! "OneWay" actor ! "OneWay"
assert(Global.oneWay.await(1, TimeUnit.SECONDS)) assert(Global.oneWay.await(1, TimeUnit.SECONDS))
actor.stop 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 @Test
def shouldSendReplyAsync = { def shouldSendReplyAsync = {
val actor = new RemoteActorSpecActorBidirectional val actor = new RemoteActorSpecActorBidirectional
@ -101,6 +141,7 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite {
def shouldSendRemoteReply = { def shouldSendRemoteReply = {
implicit val timeout = 500000000L implicit val timeout = 500000000L
val actor = new RemoteActorSpecActorBidirectional val actor = new RemoteActorSpecActorBidirectional
actor.setReplyToAddress(HOSTNAME, PORT2)
actor.makeRemote(HOSTNAME, PORT2) actor.makeRemote(HOSTNAME, PORT2)
actor.start actor.start
@ -128,3 +169,4 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite {
actor.stop actor.stop
} }
} }

View file

@ -77,7 +77,7 @@ class RemoteSupervisorSpec extends JUnitSuite {
new Thread(new Runnable() { new Thread(new Runnable() {
def run = { def run = {
RemoteNode.start RemoteNode.start(RemoteServer.HOSTNAME, 9988)
} }
}).start }).start
Thread.sleep(1000) Thread.sleep(1000)
@ -335,7 +335,7 @@ class RemoteSupervisorSpec extends JUnitSuite {
// implementation of the Actors we want to use. // implementation of the Actors we want to use.
pingpong1 = new RemotePingPong1Actor pingpong1 = new RemotePingPong1Actor
pingpong1.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT) pingpong1.makeRemote(RemoteServer.HOSTNAME, 9988)
val factory = SupervisorFactory( val factory = SupervisorFactory(
SupervisorConfig( SupervisorConfig(
@ -350,7 +350,7 @@ class RemoteSupervisorSpec extends JUnitSuite {
def getSingleActorOneForOneSupervisor: Supervisor = { def getSingleActorOneForOneSupervisor: Supervisor = {
pingpong1 = new RemotePingPong1Actor pingpong1 = new RemotePingPong1Actor
pingpong1.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT) pingpong1.makeRemote(RemoteServer.HOSTNAME, 9988)
val factory = SupervisorFactory( val factory = SupervisorFactory(
SupervisorConfig( SupervisorConfig(
@ -364,11 +364,11 @@ class RemoteSupervisorSpec extends JUnitSuite {
def getMultipleActorsAllForOneConf: Supervisor = { def getMultipleActorsAllForOneConf: Supervisor = {
pingpong1 = new RemotePingPong1Actor pingpong1 = new RemotePingPong1Actor
pingpong1.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT) pingpong1.makeRemote(RemoteServer.HOSTNAME, 9988)
pingpong2 = new RemotePingPong2Actor pingpong2 = new RemotePingPong2Actor
pingpong2.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT) pingpong2.makeRemote(RemoteServer.HOSTNAME, 9988)
pingpong3 = new RemotePingPong3Actor pingpong3 = new RemotePingPong3Actor
pingpong3.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT) pingpong3.makeRemote(RemoteServer.HOSTNAME, 9988)
val factory = SupervisorFactory( val factory = SupervisorFactory(
SupervisorConfig( SupervisorConfig(
@ -390,11 +390,11 @@ class RemoteSupervisorSpec extends JUnitSuite {
def getMultipleActorsOneForOneConf: Supervisor = { def getMultipleActorsOneForOneConf: Supervisor = {
pingpong1 = new RemotePingPong1Actor pingpong1 = new RemotePingPong1Actor
pingpong1.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT) pingpong1.makeRemote(RemoteServer.HOSTNAME, 9988)
pingpong2 = new RemotePingPong2Actor pingpong2 = new RemotePingPong2Actor
pingpong2.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT) pingpong2.makeRemote(RemoteServer.HOSTNAME, 9988)
pingpong3 = new RemotePingPong3Actor pingpong3 = new RemotePingPong3Actor
pingpong3.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT) pingpong3.makeRemote(RemoteServer.HOSTNAME, 9988)
val factory = SupervisorFactory( val factory = SupervisorFactory(
SupervisorConfig( SupervisorConfig(
@ -416,11 +416,11 @@ class RemoteSupervisorSpec extends JUnitSuite {
def getNestedSupervisorsAllForOneConf: Supervisor = { def getNestedSupervisorsAllForOneConf: Supervisor = {
pingpong1 = new RemotePingPong1Actor pingpong1 = new RemotePingPong1Actor
pingpong1.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT) pingpong1.makeRemote(RemoteServer.HOSTNAME, 9988)
pingpong2 = new RemotePingPong2Actor pingpong2 = new RemotePingPong2Actor
pingpong2.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT) pingpong2.makeRemote(RemoteServer.HOSTNAME, 9988)
pingpong3 = new RemotePingPong3Actor pingpong3 = new RemotePingPong3Actor
pingpong3.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT) pingpong3.makeRemote(RemoteServer.HOSTNAME, 9988)
val factory = SupervisorFactory( val factory = SupervisorFactory(
SupervisorConfig( SupervisorConfig(

View file

@ -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 def endpointUri = uri
protected def receive = { protected def receive = {
@ -89,4 +89,4 @@ class PublisherBridge(uri: String, publisher: Actor) extends Actor with Consumer
reply("message published") reply("message published")
} }
} }
} }

View file

@ -50,7 +50,7 @@ class Boot {
*/ */
@Path("/scalacount") @Path("/scalacount")
class SimpleService extends Transactor { class SimpleService extends Transactor {
case object Tick case object Tick
private val KEY = "COUNTER" private val KEY = "COUNTER"
private var hasStartedTicking = false private var hasStartedTicking = false
@ -127,7 +127,7 @@ class PersistentSimpleService extends Transactor {
} }
@Path("/chat") @Path("/chat")
class Chat extends Actor { class Chat extends Actor with Logging {
case class Chat(val who: String, val what: String, val msg: String) case class Chat(val who: String, val what: String, val msg: String)
@Suspend @Suspend

View file

@ -87,7 +87,7 @@ class AkkaSecurityFilterFactory extends ResourceFilterFactory with Logging {
override def filter(request: ContainerRequest): ContainerRequest = override def filter(request: ContainerRequest): ContainerRequest =
rolesAllowed match { rolesAllowed match {
case Some(roles) => { case Some(roles) => {
val result : Option[AnyRef] = authenticator !! Authenticate(request, roles) val result : Option[AnyRef] = authenticator !! Authenticate(request, roles)
result match { result match {
case Some(OK) => request case Some(OK) => request
case Some(r) if r.isInstanceOf[Response] => case Some(r) if r.isInstanceOf[Response] =>
@ -245,7 +245,7 @@ trait BasicAuthenticationActor extends AuthenticationActor[BasicCredentials] {
* class to create an authenticator. Don't forget to set the authenticator FQN in the * class to create an authenticator. Don't forget to set the authenticator FQN in the
* rest-part of the akka config * rest-part of the akka config
*/ */
trait DigestAuthenticationActor extends AuthenticationActor[DigestCredentials] { trait DigestAuthenticationActor extends AuthenticationActor[DigestCredentials] with Logging {
import Enc._ import Enc._
private object InvalidateNonces private object InvalidateNonces
@ -346,7 +346,7 @@ import org.ietf.jgss.GSSContext
import org.ietf.jgss.GSSCredential import org.ietf.jgss.GSSCredential
import org.ietf.jgss.GSSManager import org.ietf.jgss.GSSManager
trait SpnegoAuthenticationActor extends AuthenticationActor[SpnegoCredentials] { trait SpnegoAuthenticationActor extends AuthenticationActor[SpnegoCredentials] with Logging {
override def unauthorized = override def unauthorized =
Response.status(401).header("WWW-Authenticate", "Negotiate").build Response.status(401).header("WWW-Authenticate", "Negotiate").build

View file

@ -10,12 +10,12 @@
* Simple and high-level abstractions for concurrency and parallelism. * Simple and high-level abstractions for concurrency and parallelism.
* Asynchronous, non-blocking and highly performant event-driven programming model. * Asynchronous, non-blocking and highly performant event-driven programming model.
* Very lightweight event-driven processes (create ~6.5 million actors on 4 G RAM). * Very lightweight event-driven processes (create ~6.5 million actors on 4 G RAM).
* Supervision hierarchies with let-it-crash semantics. For writing highly * Supervision hierarchies with let-it-crash semantics. For writing highly
fault-tolerant systems that never stop, systems that self-heal. fault-tolerant systems that never stop, systems that self-heal.
* Software Transactional Memory (STM). (Distributed transactions coming soon). * Software Transactional Memory (STM). (Distributed transactions coming soon).
* Transactors: combine actors and STM into transactional actors. Allows you to * Transactors: combine actors and STM into transactional actors. Allows you to
compose atomic message flows with automatic rollback and retry. compose atomic message flows with automatic rollback and retry.
* Remoting: highly performant distributed actors with remote supervision and * Remoting: highly performant distributed actors with remote supervision and
error management. error management.
* Cluster membership management. * Cluster membership management.
@ -31,7 +31,7 @@
* Spring: Spring integration * Spring: Spring integration
* Guice: Guice integration * Guice: Guice integration
* Microkernel: Run Akka as a stand-alone kernel. * Microkernel: Run Akka as a stand-alone kernel.
-------------------------------------------------------------------------------*/ -------------------------------------------------------------------------------*/
import sbt._ import sbt._
@ -50,7 +50,7 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
val CASSANDRA_VERSION = "0.5.0" val CASSANDRA_VERSION = "0.5.0"
val LIFT_VERSION = "2.0-scala280-SNAPSHOT" val LIFT_VERSION = "2.0-scala280-SNAPSHOT"
val SCALATEST_VERSION = "1.0.1-for-scala-2.8.0.Beta1-with-test-interfaces-0.3-SNAPSHOT" val SCALATEST_VERSION = "1.0.1-for-scala-2.8.0.Beta1-with-test-interfaces-0.3-SNAPSHOT"
// ------------------------------------------------------------ // ------------------------------------------------------------
lazy val akkaHome = { lazy val akkaHome = {
val home = System.getenv("AKKA_HOME") val home = System.getenv("AKKA_HOME")
@ -97,10 +97,10 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
lazy val akka_persistence = project("akka-persistence", "akka-persistence", new AkkaPersistenceParentProject(_)) lazy val akka_persistence = project("akka-persistence", "akka-persistence", new AkkaPersistenceParentProject(_))
lazy val akka_cluster = project("akka-cluster", "akka-cluster", new AkkaClusterParentProject(_)) lazy val akka_cluster = project("akka-cluster", "akka-cluster", new AkkaClusterParentProject(_))
lazy val akka_spring = project("akka-spring", "akka-spring", new AkkaSpringProject(_), akka_core) lazy val akka_spring = project("akka-spring", "akka-spring", new AkkaSpringProject(_), akka_core)
lazy val akka_servlet = project("akka-servlet", "akka-servlet", new AkkaServletProject(_), lazy val akka_servlet = project("akka-servlet", "akka-servlet", new AkkaServletProject(_),
akka_core, akka_rest, akka_camel) akka_core, akka_rest, akka_camel)
lazy val akka_kernel = project("akka-kernel", "akka-kernel", new AkkaKernelProject(_), lazy val akka_kernel = project("akka-kernel", "akka-kernel", new AkkaKernelProject(_),
akka_core, akka_rest, akka_spring, akka_camel, akka_persistence, akka_core, akka_rest, akka_spring, akka_camel, akka_persistence,
akka_cluster, akka_amqp, akka_security, akka_comet, akka_patterns, akka_servlet) akka_cluster, akka_amqp, akka_security, akka_comet, akka_patterns, akka_servlet)
// functional tests in java // functional tests in java
@ -113,11 +113,11 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
// create executable jar // create executable jar
override def mainClass = Some("se.scalablesolutions.akka.kernel.Main") override def mainClass = Some("se.scalablesolutions.akka.kernel.Main")
override def packageOptions = override def packageOptions =
manifestClassPath.map(cp => ManifestAttributes( manifestClassPath.map(cp => ManifestAttributes(
(Attributes.Name.CLASS_PATH, cp), (Attributes.Name.CLASS_PATH, cp),
(IMPLEMENTATION_TITLE, "Akka"), (IMPLEMENTATION_TITLE, "Akka"),
(IMPLEMENTATION_URL, "http://akkasource.org"), (IMPLEMENTATION_URL, "http://akkasource.org"),
(IMPLEMENTATION_VENDOR, "The Akka Project") (IMPLEMENTATION_VENDOR, "The Akka Project")
)).toList ::: )).toList :::
getMainClass(false).map(MainClass(_)).toList getMainClass(false).map(MainClass(_)).toList
@ -126,7 +126,7 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
override def manifestClassPath = Some(allArtifacts.getFiles override def manifestClassPath = Some(allArtifacts.getFiles
.filter(_.getName.endsWith(".jar")) .filter(_.getName.endsWith(".jar"))
.map("lib_managed/scala_%s/compile/".format(buildScalaVersion) + _.getName) .map("lib_managed/scala_%s/compile/".format(buildScalaVersion) + _.getName)
.mkString(" ") + .mkString(" ") +
" scala-library.jar" + " scala-library.jar" +
" dist/akka-util_%s-%s.jar".format(buildScalaVersion, version) + " dist/akka-util_%s-%s.jar".format(buildScalaVersion, version) +
" dist/akka-util-java_%s-%s.jar".format(buildScalaVersion, version) + " dist/akka-util-java_%s-%s.jar".format(buildScalaVersion, version) +
@ -146,7 +146,7 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
" dist/akka-servlet_%s-%s.jar".format(buildScalaVersion, version) + " dist/akka-servlet_%s-%s.jar".format(buildScalaVersion, version) +
" dist/akka-kernel_%s-%s.jar".format(buildScalaVersion, version) + " dist/akka-kernel_%s-%s.jar".format(buildScalaVersion, version) +
" dist/akka-spring_%s-%s.jar".format(buildScalaVersion, version) " dist/akka-spring_%s-%s.jar".format(buildScalaVersion, version)
) )
// ------------------------------------------------------------ // ------------------------------------------------------------
// publishing // publishing
@ -159,7 +159,7 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
// Credentials(Path.userHome / ".akka_publish_credentials", log) // Credentials(Path.userHome / ".akka_publish_credentials", log)
//override def documentOptions = encodingUtf8.map(SimpleDocOption(_)) //override def documentOptions = encodingUtf8.map(SimpleDocOption(_))
override def packageDocsJar = defaultJarPath("-doc.jar") override def packageDocsJar = defaultJarPath("-doc.jar")
override def packageSrcJar= defaultJarPath("-src.jar") override def packageSrcJar= defaultJarPath("-src.jar")
override def packageToPublishActions = super.packageToPublishActions ++ Seq(packageDocs, packageSrc) override def packageToPublishActions = super.packageToPublishActions ++ Seq(packageDocs, packageSrc)
@ -291,19 +291,19 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
val google_coll = "com.google.collections" % "google-collections" % "1.0" % "test" val google_coll = "com.google.collections" % "google-collections" % "1.0" % "test"
val slf4j = "org.slf4j" % "slf4j-api" % "1.5.8" % "test" val slf4j = "org.slf4j" % "slf4j-api" % "1.5.8" % "test"
val slf4j_log4j = "org.slf4j" % "slf4j-log4j12" % "1.5.8" % "test" val slf4j_log4j = "org.slf4j" % "slf4j-log4j12" % "1.5.8" % "test"
val log4j = "log4j" % "log4j" % "1.2.15" % "test" val log4j = "log4j" % "log4j" % "1.2.15" % "test"
override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying") lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
} }
class AkkaPersistenceParentProject(info: ProjectInfo) extends ParentProject(info) { class AkkaPersistenceParentProject(info: ProjectInfo) extends ParentProject(info) {
lazy val akka_persistence_common = project("akka-persistence-common", "akka-persistence-common", lazy val akka_persistence_common = project("akka-persistence-common", "akka-persistence-common",
new AkkaPersistenceCommonProject(_), akka_core) new AkkaPersistenceCommonProject(_), akka_core)
lazy val akka_persistence_redis = project("akka-persistence-redis", "akka-persistence-redis", lazy val akka_persistence_redis = project("akka-persistence-redis", "akka-persistence-redis",
new AkkaRedisProject(_), akka_persistence_common) new AkkaRedisProject(_), akka_persistence_common)
lazy val akka_persistence_mongo = project("akka-persistence-mongo", "akka-persistence-mongo", lazy val akka_persistence_mongo = project("akka-persistence-mongo", "akka-persistence-mongo",
new AkkaMongoProject(_), akka_persistence_common) new AkkaMongoProject(_), akka_persistence_common)
lazy val akka_persistence_cassandra = project("akka-persistence-cassandra", "akka-persistence-cassandra", lazy val akka_persistence_cassandra = project("akka-persistence-cassandra", "akka-persistence-cassandra",
new AkkaCassandraProject(_), akka_persistence_common) new AkkaCassandraProject(_), akka_persistence_common)
} }
@ -319,9 +319,9 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
} }
class AkkaClusterParentProject(info: ProjectInfo) extends ParentProject(info) { class AkkaClusterParentProject(info: ProjectInfo) extends ParentProject(info) {
lazy val akka_cluster_jgroups = project("akka-cluster-jgroups", "akka-cluster-jgroups", lazy val akka_cluster_jgroups = project("akka-cluster-jgroups", "akka-cluster-jgroups",
new AkkaJgroupsProject(_), akka_core) new AkkaJgroupsProject(_), akka_core)
lazy val akka_cluster_shoal = project("akka-cluster-shoal", "akka-cluster-shoal", lazy val akka_cluster_shoal = project("akka-cluster-shoal", "akka-cluster-shoal",
new AkkaShoalProject(_), akka_core) new AkkaShoalProject(_), akka_core)
} }
@ -398,18 +398,18 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
} }
class AkkaSamplesParentProject(info: ProjectInfo) extends ParentProject(info) { class AkkaSamplesParentProject(info: ProjectInfo) extends ParentProject(info) {
lazy val akka_sample_chat = project("akka-sample-chat", "akka-sample-chat", lazy val akka_sample_chat = project("akka-sample-chat", "akka-sample-chat",
new AkkaSampleChatProject(_), akka_kernel) new AkkaSampleChatProject(_), akka_kernel)
lazy val akka_sample_lift = project("akka-sample-lift", "akka-sample-lift", lazy val akka_sample_lift = project("akka-sample-lift", "akka-sample-lift",
new AkkaSampleLiftProject(_), akka_kernel) new AkkaSampleLiftProject(_), akka_kernel)
lazy val akka_sample_rest_java = project("akka-sample-rest-java", "akka-sample-rest-java", lazy val akka_sample_rest_java = project("akka-sample-rest-java", "akka-sample-rest-java",
new AkkaSampleRestJavaProject(_), akka_kernel) new AkkaSampleRestJavaProject(_), akka_kernel)
lazy val akka_sample_rest_scala = project("akka-sample-rest-scala", "akka-sample-rest-scala", lazy val akka_sample_rest_scala = project("akka-sample-rest-scala", "akka-sample-rest-scala",
new AkkaSampleRestScalaProject(_), akka_kernel) new AkkaSampleRestScalaProject(_), akka_kernel)
lazy val akka_sample_camel = project("akka-sample-camel", "akka-sample-camel", lazy val akka_sample_camel = project("akka-sample-camel", "akka-sample-camel",
new AkkaSampleCamelProject(_), akka_kernel) new AkkaSampleCamelProject(_), akka_kernel)
lazy val akka_sample_security = project("akka-sample-security", "akka-sample-security", lazy val akka_sample_security = project("akka-sample-security", "akka-sample-security",
new AkkaSampleSecurityProject(_), akka_kernel) new AkkaSampleSecurityProject(_), akka_kernel)
} }
// ------------------------------------------------------------ // ------------------------------------------------------------
@ -432,9 +432,9 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
descendents(path("lib") ##, "*.jar") +++ descendents(path("lib") ##, "*.jar") +++
descendents(configurationPath(Configurations.Compile) ##, "*.jar")) descendents(configurationPath(Configurations.Compile) ##, "*.jar"))
.filter(jar => // remove redundant libs .filter(jar => // remove redundant libs
!jar.toString.endsWith("stax-api-1.0.1.jar") || !jar.toString.endsWith("stax-api-1.0.1.jar") ||
!jar.toString.endsWith("scala-library-2.7.7.jar") !jar.toString.endsWith("scala-library-2.7.7.jar")
) )
} }
def deployTask(info: ProjectInfo, toDir: Path, genJar: Boolean, genDocs: Boolean, genSource: Boolean) = task { def deployTask(info: ProjectInfo, toDir: Path, genJar: Boolean, genDocs: Boolean, genSource: Boolean) = task {
@ -444,7 +444,7 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
// FIXME need to find out a way to grab these paths from the sbt system // FIXME need to find out a way to grab these paths from the sbt system
// binary // binary
if (genJar) { if (genJar) {
val JAR_FILE_NAME = moduleName + "_%s-%s.jar".format(buildScalaVersion, version) val JAR_FILE_NAME = moduleName + "_%s-%s.jar".format(buildScalaVersion, version)
val JAR_FILE_PATH = projectPath + "/target/scala_%s/".format(buildScalaVersion) + JAR_FILE_NAME val JAR_FILE_PATH = projectPath + "/target/scala_%s/".format(buildScalaVersion) + JAR_FILE_NAME
@ -463,7 +463,7 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
log.info("Deploying docs " + toDoc) log.info("Deploying docs " + toDoc)
FileUtilities.copyFile(fromDoc, toDoc, log) FileUtilities.copyFile(fromDoc, toDoc, log)
} }
// sources // sources
if (genSource) { if (genSource) {
val SRC_FILE_NAME = moduleName + "_%s-%s-%s.jar".format(buildScalaVersion, version, "src") val SRC_FILE_NAME = moduleName + "_%s-%s-%s.jar".format(buildScalaVersion, version, "src")