pekko/kernel/src/main/scala/ActiveObject.scala

353 lines
12 KiB
Scala
Raw Normal View History

2009-02-17 21:05:07 +01:00
/**
* Copyright (C) 2009 Scalable Solutions.
*/
2009-03-23 19:17:49 +01:00
package se.scalablesolutions.akka.kernel
2009-02-17 21:05:07 +01:00
import kernel.camel.{MessageDriven, ActiveObjectProducer}
2009-05-09 19:55:42 +02:00
import config.ActiveObjectGuiceConfigurator
import config.ScalaConfig._
2009-02-17 21:05:07 +01:00
import java.util.{List => JList, ArrayList}
2009-05-20 16:54:42 +02:00
import java.lang.reflect.{Method, Field}
import java.lang.annotation.Annotation
2009-05-20 16:54:42 +02:00
import org.codehaus.aspectwerkz.intercept.{Advisable, AroundAdvice}
import org.codehaus.aspectwerkz.joinpoint.{MethodRtti, JoinPoint}
import org.codehaus.aspectwerkz.proxy.Proxy
2009-05-09 19:55:42 +02:00
import org.apache.camel.{Processor, Exchange}
2009-05-09 19:55:42 +02:00
import scala.collection.mutable.HashMap
2009-03-15 08:35:37 +01:00
2009-02-17 21:05:07 +01:00
sealed class ActiveObjectException(msg: String) extends RuntimeException(msg)
class ActiveObjectInvocationTimeoutException(msg: String) extends ActiveObjectException(msg)
object Annotations {
import se.scalablesolutions.akka.annotation._
val transactional = classOf[transactional]
val oneway = classOf[oneway]
val immutable = classOf[immutable]
val state = classOf[state]
}
2009-02-17 21:05:07 +01:00
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class ActiveObjectFactory {
2009-05-20 16:54:42 +02:00
def newInstance[T](target: Class[T], server: GenericServerContainer): T = {
ActiveObject.newInstance(target, server)
}
def newInstance[T](intf: Class[T], target: AnyRef, server: GenericServerContainer): T = {
ActiveObject.newInstance(intf, target, server)
2009-05-20 16:54:42 +02:00
}
2009-02-17 21:05:07 +01:00
def supervise(restartStrategy: RestartStrategy, components: List[Worker]): Supervisor =
ActiveObject.supervise(restartStrategy, components)
2009-02-17 21:05:07 +01:00
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object ActiveObject {
2009-05-09 19:55:42 +02:00
val AKKA_CAMEL_ROUTING_SCHEME = "akka"
private[kernel] val threadBoundTx: ThreadLocal[Option[Transaction]] = {
val tl = new ThreadLocal[Option[Transaction]]
tl.set(None)
tl
}
2009-05-20 16:54:42 +02:00
def newInstance[T](target: Class[T], server: GenericServerContainer): T = {
val proxy = Proxy.newInstance(target, false, true)
// FIXME switch to weaving in the aspect at compile time
proxy.asInstanceOf[Advisable].aw_addAdvice("execution(* *.*(..))", new TransactionalAroundAdvice(target, proxy, server))
2009-05-20 16:54:42 +02:00
proxy.asInstanceOf[T]
2009-05-09 19:55:42 +02:00
}
2009-05-20 16:54:42 +02:00
def newInstance[T](intf: Class[T], target: AnyRef, server: GenericServerContainer): T = {
val proxy = Proxy.newInstance(Array(intf), Array(target), false, true)
proxy.asInstanceOf[Advisable].aw_addAdvice("execution(* *.*(..))", new TransactionalAroundAdvice(intf, target, server))
2009-05-20 16:54:42 +02:00
proxy.asInstanceOf[T]
}
2009-02-17 21:05:07 +01:00
def supervise(restartStrategy: RestartStrategy, components: List[Worker]): Supervisor = {
2009-03-23 19:17:49 +01:00
object factory extends SupervisorFactory {
2009-02-17 21:05:07 +01:00
override def getSupervisorConfig = SupervisorConfig(restartStrategy, components)
}
val supervisor = factory.newSupervisor
2009-03-23 19:17:49 +01:00
supervisor ! se.scalablesolutions.akka.kernel.Start
2009-02-17 21:05:07 +01:00
supervisor
}
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
2009-05-18 08:19:30 +02:00
// FIXME: STM that allows concurrent updates, detects collision, rolls back and restarts
sealed class TransactionalAroundAdvice(target: Class[_],
targetInstance: AnyRef,
server: GenericServerContainer) extends AroundAdvice {
private val (maps, vectors, refs) = getTransactionalItemsFor(targetInstance)
2009-05-20 16:54:42 +02:00
server.transactionalRefs = refs
server.transactionalMaps = maps
server.transactionalVectors = vectors
2009-05-09 19:55:42 +02:00
import kernel.reactor._
private[this] var dispatcher = new ProxyMessageDispatcher
private[this] var mailbox = dispatcher.messageQueue
dispatcher.start
2009-05-20 16:54:42 +02:00
import ActiveObject.threadBoundTx
2009-03-23 19:17:49 +01:00
private[this] var activeTx: Option[Transaction] = None
// FIXME: switch to using PCD annotation matching, break out into its own aspect + switch to StaticJoinPoint
2009-05-20 16:54:42 +02:00
def invoke(joinpoint: JoinPoint): AnyRef = {
val rtti = joinpoint.getRtti.asInstanceOf[MethodRtti]
val method = rtti.getMethod
2009-05-20 16:54:42 +02:00
if (method.isAnnotationPresent(Annotations.transactional)) {
tryToCommitTransaction
startNewTransaction
}
joinExistingTransaction
incrementTransaction
val result: AnyRef = try {
if (rtti.getMethod.isAnnotationPresent(Annotations.oneway)) sendOneWay(joinpoint)
else handleResult(sendAndReceiveEventually(joinpoint))
} finally {
decrementTransaction
if (isTransactionAborted) removeTransactionIfTopLevel
else tryToPrecommitTransaction
}
result
}
private def isTransactionAborted = activeTx.isDefined && activeTx.get.isAborted
private def incrementTransaction = if (activeTx.isDefined) activeTx.get.increment
private def decrementTransaction = if (activeTx.isDefined) activeTx.get.decrement
private def removeTransactionIfTopLevel =
if (activeTx.isDefined && activeTx.get.topLevel_?) {
activeTx = None
threadBoundTx.set(None)
}
private def startNewTransaction = {
val newTx = new Transaction
newTx.begin(server)
threadBoundTx.set(Some(newTx))
}
private def joinExistingTransaction = {
val cflowTx = threadBoundTx.get
if (!activeTx.isDefined && cflowTx.isDefined) {
val currentTx = cflowTx.get
currentTx.join(server)
activeTx = Some(currentTx)
2009-03-23 19:17:49 +01:00
}
activeTx = threadBoundTx.get
}
2009-02-17 21:05:07 +01:00
private def tryToPrecommitTransaction = if (activeTx.isDefined) activeTx.get.precommit(server)
private def tryToCommitTransaction = if (activeTx.isDefined) {
val tx = activeTx.get
tx.commit(server)
removeTransactionIfTopLevel
}
private def handleResult(result: ResultOrFailure[AnyRef]): AnyRef = {
try {
result()
} catch {
case e =>
rollback(result.tx)
throw e
}
2009-02-17 21:05:07 +01:00
}
private def rollback(tx: Option[Transaction]) = tx match {
case None => {} // no tx; nothing to do
case Some(tx) =>
tx.rollback(server)
}
private def sendOneWay(joinpoint: JoinPoint) =
mailbox.append(new MessageHandle(this, Invocation(joinpoint, activeTx), new NullFutureResult))
private def sendAndReceiveEventually(joinpoint: JoinPoint): ResultOrFailure[AnyRef] = {
val future = postMessageToMailboxAndCreateFutureResultWithTimeout(Invocation(joinpoint, activeTx), 1000)
future.await_?
getResultOrThrowException(future)
}
private def postMessageToMailboxAndCreateFutureResultWithTimeout(
message: AnyRef, timeout: Long): CompletableFutureResult = {
val future = new DefaultCompletableFutureResult(timeout)
mailbox.append(new MessageHandle(this, message, future))
future
}
private def getResultOrThrowException[T](future: FutureResult): ResultOrFailure[AnyRef] =
if (future.exception.isDefined) {
var resultOrFailure = ResultOrFailure(activeTx)
resultOrFailure() = throw future.exception.get
resultOrFailure
} else ResultOrFailure(future.result.get, activeTx)
/**
* Search for transactional items for a specific target instance, crawl the class hierarchy recursively up to the top.
*/
private def getTransactionalItemsFor(targetInstance: AnyRef):
Tuple3[List[TransactionalMap[_, _]], List[TransactionalVector[_]], List[TransactionalRef[_]]] = {
require(targetInstance != null)
var maps: List[TransactionalMap[_, _]] = Nil
var refs: List[TransactionalRef[_]] = Nil
var vectors: List[TransactionalVector[_]] = Nil
def getTransactionalItemsFor(target: Class[_]):
Tuple3[List[TransactionalMap[_, _]], List[TransactionalVector[_]], List[TransactionalRef[_]]] = {
target.getDeclaredFields.toArray.toList.asInstanceOf[List[Field]].foreach(println)
for {
field <- target.getDeclaredFields.toArray.toList.asInstanceOf[List[Field]]
fieldType = field.getType
if (fieldType == classOf[TransactionalMap[_, _]]) ||
(fieldType == classOf[TransactionalVector[_]]) ||
(fieldType == classOf[TransactionalRef[_]])
txItem = {
field.setAccessible(true)
field.get(targetInstance)
}
if txItem != null
} {
if (txItem.isInstanceOf[TransactionalMap[_, _]]) maps ::= txItem.asInstanceOf[TransactionalMap[_, _]]
else if (txItem.isInstanceOf[TransactionalRef[_]]) refs ::= txItem.asInstanceOf[TransactionalRef[_]]
else if (txItem.isInstanceOf[TransactionalVector[_]]) vectors ::= txItem.asInstanceOf[TransactionalVector[_]]
}
val parent = target.getSuperclass
if (parent == classOf[Object]) (maps, vectors, refs)
else getTransactionalItemsFor(parent)
}
// start the search for transactional items, crawl the class hierarchy up until we reach 'null'
getTransactionalItemsFor(targetInstance.getClass)
}
2009-02-17 21:05:07 +01:00
}
/**
* Generic GenericServer managing Invocation dispatch, transaction and error management.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
private[kernel] class Dispatcher(val targetName: String) extends GenericServer {
override def body: PartialFunction[Any, Unit] = {
2009-05-09 19:55:42 +02:00
case Invocation(joinpoint: JoinPoint, tx: Option[Transaction]) =>
ActiveObject.threadBoundTx.set(tx)
try {
reply(ResultOrFailure(joinpoint.proceed, tx))
} catch {
case e =>
val resultOrFailure = ResultOrFailure(tx)
resultOrFailure() = throw e
reply(resultOrFailure)
}
2009-05-09 19:55:42 +02:00
case 'exit =>
exit
2009-05-09 19:55:42 +02:00
2009-05-09 20:40:36 +02:00
/* case exchange: Exchange =>
2009-05-09 19:55:42 +02:00
println("=============> Exchange From Actor: " + exchange)
val invocation = exchange.getIn.getBody.asInstanceOf[Invocation]
invocation.invoke
2009-05-09 20:40:36 +02:00
*/
case unexpected =>
throw new ActiveObjectException("Unexpected message [" + unexpected + "] to [" + this + "] from [" + sender + "]")
}
override def toString(): String = "GenericServer[" + targetName + "]"
}
2009-05-09 19:55:42 +02:00
/**
* Represents a snapshot of the current invocation.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
private[kernel] case class Invocation(val joinpoint: JoinPoint, val transaction: Option[Transaction]) {
override def toString: String = synchronized {
"Invocation [joinpoint: " + joinpoint.toString+ " | transaction: " + transaction.toString + "]"
}
override def hashCode(): Int = synchronized {
var result = HashCode.SEED
result = HashCode.hash(result, joinpoint)
if (transaction.isDefined) result = HashCode.hash(result, transaction.get)
result
}
override def equals(that: Any): Boolean = synchronized {
that != null &&
that.isInstanceOf[Invocation] &&
that.asInstanceOf[Invocation].joinpoint == joinpoint &&
that.asInstanceOf[Invocation].transaction.getOrElse(false) == transaction.getOrElse(false)
}
}
2009-05-09 19:55:42 +02:00
/*
ublic class CamelInvocationHandler implements InvocationHandler {
private final Endpoint endpoint;
private final Producer producer;
private final MethodInfoCache methodInfoCache;
public CamelInvocationHandler(Endpoint endpoint, Producer producer, MethodInfoCache methodInfoCache) {
this.endpoint = endpoint;
this.producer = producer;
this.methodInfoCache = methodInfoCache;
}
public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
BeanInvocation invocation = new BeanInvocation(method, args);
ExchangePattern pattern = ExchangePattern.InOut;
MethodInfo methodInfo = methodInfoCache.getMethodInfo(method);
if (methodInfo != null) {
pattern = methodInfo.getPattern();
}
Exchange exchange = new DefaultExchange(endpoint, pattern);
exchange.getIn().setBody(invocation);
producer.process(exchange);
Throwable fault = exchange.getException();
if (fault != null) {
throw new InvocationTargetException(fault);
}
if (pattern.isOutCapable()) {
return exchange.getOut().getBody();
} else {
return null;
}
}
}
if (joinpoint.target.isInstanceOf[MessageDriven] &&
joinpoint.method.getName == "onMessage") {
val m = joinpoint.method
val endpointName = m.getDeclaringClass.getName + "." + m.getName
val activeObjectName = m.getDeclaringClass.getName
val endpoint = conf.getRoutingEndpoint(conf.lookupUriFor(m))
val producer = endpoint.createProducer
val exchange = endpoint.createExchange
exchange.getIn().setBody(joinpoint)
producer.process(exchange)
val fault = exchange.getException();
if (fault != null) throw new InvocationTargetException(fault)
// FIXME: need some timeout and future here...
exchange.getOut.getBody
} else
2009-05-09 19:55:42 +02:00
*/