make EventHandler non-global

- add Logging trait for nicer interface
- add EventHandlerLogging class for offering the nice interface from
  app.log
- add eventHandler instance to app and use that for all internal logging
  (this means that some places (dispatchers, remoting) were infiltrated
  by app just to do logging, but I think we'll need app in there soon
  enough for other reasons)
This commit is contained in:
Roland 2011-10-13 13:16:41 +02:00
parent e25ee9f0e2
commit 85b7accfd4
61 changed files with 590 additions and 542 deletions

View file

@ -61,7 +61,7 @@ object FSMActorSpec {
whenUnhandled {
case Ev(msg) {
EventHandler.info(this, "unhandled event " + msg + " in state " + stateName + " with data " + stateData)
app.eventHandler.info(this, "unhandled event " + msg + " in state " + stateName + " with data " + stateData)
unhandledLatch.open
stay
}
@ -102,13 +102,13 @@ object FSMActorSpec {
class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true)) with BeforeAndAfterAll with BeforeAndAfterEach with ImplicitSender {
import FSMActorSpec._
val eh_level = EventHandler.level
val eh_level = app.eventHandler.level
var logger: ActorRef = _
override def afterEach {
EventHandler.level = eh_level
app.eventHandler.level = eh_level
if (logger ne null) {
EventHandler.removeListener(logger)
app.eventHandler.removeListener(logger)
logger = null
}
}
@ -178,7 +178,7 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true
case x testActor forward x
}
})
EventHandler.addListener(logger)
app.eventHandler.addListener(logger)
fsm ! "go"
expectMsgPF(1 second) {
case EventHandler.Error(_: EventHandler.EventHandlerException, ref, "Next state 2 does not exist") if ref eq fsm.underlyingActor true
@ -224,8 +224,8 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true
case x testActor forward x
}
})
EventHandler.addListener(logger)
EventHandler.level = EventHandler.DebugLevel
app.eventHandler.addListener(logger)
app.eventHandler.level = EventHandler.DebugLevel
fsmref ! "go"
expectMsgPF(1 second) {
case EventHandler.Debug(`fsm`, s: String) if s.startsWith("processing Event(go,null) from Actor[testActor") true

View file

@ -5,7 +5,6 @@ package akka.actor
import akka.testkit._
import akka.util.duration._
import akka.event.EventHandler
import FSM._

View file

@ -5,13 +5,13 @@ package akka.actor
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
import akka.testkit.{ TestKit, TestActorRef, EventFilter, TestEvent, ImplicitSender }
import akka.event.EventHandler
import akka.util.duration._
import akka.testkit.AkkaSpec
import org.scalatest.WordSpec
import akka.AkkaApplication
import akka.AkkaApplication.defaultConfig
import akka.config.Configuration
import akka.event.EventHandler
object LoggingReceiveSpec {
class TestLogActor extends Actor {
@ -29,9 +29,9 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
val appLifecycle = AkkaApplication("lifecycle", config ++ Configuration("akka.actor.debug.lifecycle" -> true))
// override def beforeAll {
// EventHandler.notify(TestEvent.Mute(EventFilter[UnhandledMessageException],
// app.eventHandler.notify(TestEvent.Mute(EventFilter[UnhandledMessageException],
// EventFilter[ActorKilledException], EventFilter.custom {
// case d: EventHandler.Debug true
// case d: app.eventHandler.Debug true
// case _ false
// }))
// }
@ -43,14 +43,14 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
// !s.startsWith("now monitoring") && !s.startsWith("stopped monitoring")
// case EventHandler.Debug(_, _) true
// case EventHandler.Error(_: UnhandledMessageException, _, _) false
// case _: EventHandler.Error true
// case _: app.eventHandler.Error true
// }
"A LoggingReceive" ignore {
"decorate a Receive" in {
new TestKit(appLogging) {
EventHandler.addListener(testActor)
app.eventHandler.addListener(testActor)
val r: Actor.Receive = {
case null
}
@ -62,7 +62,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
"be added on Actor if requested" in {
new TestKit(appLogging) with ImplicitSender {
EventHandler.addListener(testActor)
app.eventHandler.addListener(testActor)
val actor = TestActorRef(new Actor {
def receive = loggable(this) {
case _ reply("x")
@ -89,7 +89,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
"not duplicate logging" in {
new TestKit(appLogging) with ImplicitSender {
EventHandler.addListener(testActor)
app.eventHandler.addListener(testActor)
val actor = TestActorRef(new Actor {
def receive = loggable(this)(loggable(this) {
case _ reply("x")
@ -109,7 +109,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
"log AutoReceiveMessages if requested" in {
new TestKit(appAuto) {
EventHandler.addListener(testActor)
app.eventHandler.addListener(testActor)
val actor = TestActorRef(new Actor {
def receive = {
case _
@ -123,7 +123,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
"log LifeCycle changes if requested" in {
new TestKit(appLifecycle) {
EventHandler.addListener(testActor)
app.eventHandler.addListener(testActor)
within(2 seconds) {
val supervisor = TestActorRef[TestLogActor](Props[TestLogActor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000)))

View file

@ -6,7 +6,6 @@ package akka.actor
import java.lang.Thread.sleep
import org.scalatest.BeforeAndAfterAll
import akka.event.EventHandler
import akka.testkit.TestEvent._
import akka.testkit.EventFilter
import java.util.concurrent.{ TimeUnit, CountDownLatch }
@ -16,11 +15,11 @@ import akka.testkit.AkkaSpec
class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll {
override def beforeAll() {
EventHandler.notify(Mute(EventFilter[Exception]("Crashing...")))
app.eventHandler.notify(Mute(EventFilter[Exception]("Crashing...")))
}
override def afterAll() {
EventHandler.notify(UnMuteAll)
app.eventHandler.notify(UnMuteAll)
}
object Ping

View file

@ -1,7 +1,6 @@
package akka.actor
import org.scalatest.BeforeAndAfterEach
import akka.event.EventHandler
import akka.testkit.TestEvent._
import akka.testkit.EventFilter
import org.multiverse.api.latches.StandardLatch
@ -18,13 +17,13 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
}
override def beforeEach {
EventHandler.notify(Mute(EventFilter[Exception]("CRASH")))
app.eventHandler.notify(Mute(EventFilter[Exception]("CRASH")))
}
override def afterEach {
while (futures.peek() ne null) { Option(futures.poll()).foreach(_.cancel(true)) }
app.registry.local.shutdownAll
EventHandler.start()
app.eventHandler.start()
}
"A Scheduler" must {

View file

@ -12,7 +12,6 @@ import akka.testkit.Testing.sleepFor
import akka.util.duration._
import akka.{ Die, Ping }
import akka.actor.Actor._
import akka.event.EventHandler
import akka.testkit.TestEvent._
import akka.testkit.EventFilter
import java.util.concurrent.atomic.AtomicInteger
@ -123,13 +122,13 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
}
override def beforeAll() = {
EventHandler notify Mute(EventFilter[Exception]("Die"),
app.eventHandler notify Mute(EventFilter[Exception]("Die"),
EventFilter[IllegalStateException]("Don't wanna!"),
EventFilter[RuntimeException]("Expected"))
}
override def afterAll() = {
EventHandler notify UnMuteAll
app.eventHandler notify UnMuteAll
}
override def beforeEach() = {

View file

@ -17,7 +17,6 @@ class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender
override def afterAll = {
app.registry.local.shutdownAll
akka.event.EventHandler.start()
}
"A supervised actor with lifecycle PERMANENT" should {

View file

@ -3,7 +3,6 @@
*/
package akka.actor.dispatch
import akka.event.EventHandler
import org.scalatest.Assertions._
import akka.testkit.{ Testing, filterEvents, EventFilter, AkkaSpec }
import akka.dispatch._
@ -153,14 +152,14 @@ object ActorModelSpec {
def assertDispatcher(dispatcher: MessageDispatcherInterceptor)(
starts: Long = dispatcher.starts.get(),
stops: Long = dispatcher.stops.get()) {
stops: Long = dispatcher.stops.get())(implicit app: AkkaApplication) {
val deadline = System.currentTimeMillis + dispatcher.timeoutMs * 5
try {
await(deadline)(starts == dispatcher.starts.get)
await(deadline)(stops == dispatcher.stops.get)
} catch {
case e
EventHandler.error(e, dispatcher, "actual: starts=" + dispatcher.starts.get + ",stops=" + dispatcher.stops.get +
app.eventHandler.error(e, dispatcher, "actual: starts=" + dispatcher.starts.get + ",stops=" + dispatcher.stops.get +
" required: starts=" + starts + ",stops=" + stops)
throw e
}
@ -190,7 +189,7 @@ object ActorModelSpec {
unregisters: Long = 0,
msgsReceived: Long = 0,
msgsProcessed: Long = 0,
restarts: Long = 0) {
restarts: Long = 0)(implicit app: AkkaApplication) {
assertRef(actorRef, dispatcher)(
suspensions,
resumes,
@ -208,7 +207,7 @@ object ActorModelSpec {
unregisters: Long = statsFor(actorRef).unregisters.get(),
msgsReceived: Long = statsFor(actorRef).msgsReceived.get(),
msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(),
restarts: Long = statsFor(actorRef).restarts.get()) {
restarts: Long = statsFor(actorRef).restarts.get())(implicit app: AkkaApplication) {
val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher))
val deadline = System.currentTimeMillis + 1000
try {
@ -221,7 +220,7 @@ object ActorModelSpec {
await(deadline)(stats.restarts.get() == restarts)
} catch {
case e
EventHandler.error(e, dispatcher, "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions +
app.eventHandler.error(e, dispatcher, "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions +
",res=" + resumes + ",reg=" + registers + ",unreg=" + unregisters +
",recv=" + msgsReceived + ",proc=" + msgsProcessed + ",restart=" + restarts)
throw e
@ -324,7 +323,7 @@ abstract class ActorModelSpec extends AkkaSpec {
try {
f
} catch {
case e EventHandler.error(e, this, "error in spawned thread")
case e app.eventHandler.error(e, this, "error in spawned thread")
}
}
}
@ -399,7 +398,7 @@ abstract class ActorModelSpec extends AkkaSpec {
} catch {
case e
System.err.println("Error: " + e.getMessage + " missing count downs == " + cachedMessage.latch.getCount() + " out of " + num)
//EventHandler.error(new Exception with NoStackTrace, null, cachedMessage.latch.getCount())
//app.eventHandler.error(new Exception with NoStackTrace, null, cachedMessage.latch.getCount())
}
}
for (run 1 to 3) {
@ -480,10 +479,10 @@ class DispatcherModelSpec extends ActorModelSpec {
import ActorModelSpec._
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config
new Dispatcher("foo", app.AkkaConfig.DispatcherThroughput,
new Dispatcher(app, "foo", app.AkkaConfig.DispatcherThroughput,
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType,
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig()).build.asInstanceOf[MessageDispatcherInterceptor]
ThreadPoolConfig(app)).build.asInstanceOf[MessageDispatcherInterceptor]
def dispatcherType = "Dispatcher"
@ -509,10 +508,10 @@ class BalancingDispatcherModelSpec extends ActorModelSpec {
import ActorModelSpec._
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher("foo", 1, // TODO check why 1 here? (came from old test)
new BalancingDispatcher(app, "foo", 1, // TODO check why 1 here? (came from old test)
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType,
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig()).build.asInstanceOf[MessageDispatcherInterceptor]
ThreadPoolConfig(app)).build.asInstanceOf[MessageDispatcherInterceptor]
def dispatcherType = "Balancing Dispatcher"
}

View file

@ -2,7 +2,6 @@ package akka.actor.dispatch
import java.util.concurrent.{ CountDownLatch, TimeUnit }
import akka.event.EventHandler
import akka.testkit.TestEvent._
import akka.testkit.EventFilter
import akka.dispatch.{ PinnedDispatcher, Dispatchers }
@ -25,11 +24,11 @@ class PinnedActorSpec extends AkkaSpec with BeforeAndAfterEach {
private val unit = TimeUnit.MILLISECONDS
override def beforeEach {
EventHandler.notify(Mute(EventFilter[RuntimeException]("Failure")))
app.eventHandler.notify(Mute(EventFilter[RuntimeException]("Failure")))
}
override def afterEach {
EventHandler.notify(UnMuteAll)
app.eventHandler.notify(UnMuteAll)
}
"A PinnedActor" must {
@ -51,7 +50,7 @@ class PinnedActorSpec extends AkkaSpec with BeforeAndAfterEach {
"support ask/exception" in {
val actor = createActor(Props[TestActor].withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test")))
EventHandler.notify(Mute(EventFilter[RuntimeException]("Expected exception; to test fault-tolerance")))
app.eventHandler.notify(Mute(EventFilter[RuntimeException]("Expected exception; to test fault-tolerance")))
try {
(actor ? "Failure").get
fail("Should have thrown an exception")

View file

@ -4,7 +4,6 @@ import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import akka.performance.trading.domain._
import akka.performance.trading.common._
import akka.event.EventHandler
import akka.actor.{ Props, ActorRef, Actor, PoisonPill }
import akka.AkkaApplication
@ -62,7 +61,7 @@ abstract class AkkaPerformanceTest(val app: AkkaApplication) extends BenchmarkSc
val duration = System.nanoTime - t0
stat.addValue(duration)
if (!rsp.status) {
EventHandler.error(this, "Invalid rsp")
app.eventHandler.error(this, "Invalid rsp")
}
delay(delayMs)
}

View file

@ -5,7 +5,6 @@ import akka.actor._
import akka.dispatch.Future
import akka.dispatch.FutureTimeoutException
import akka.dispatch.MessageDispatcher
import akka.event.EventHandler
trait MatchingEngine {
val meId: String
@ -27,7 +26,7 @@ class AkkaMatchingEngine(val meId: String, val orderbooks: List[Orderbook])
case order: Order
handleOrder(order)
case unknown
EventHandler.warning(this, "Received unknown message: " + unknown)
app.eventHandler.warning(this, "Received unknown message: " + unknown)
}
def handleOrder(order: Order) {
@ -42,7 +41,7 @@ class AkkaMatchingEngine(val meId: String, val orderbooks: List[Orderbook])
pendingStandbyReply.foreach(waitForStandby(_))
done(true)
case None
EventHandler.warning(this, "Orderbook not handled by this MatchingEngine: " + order.orderbookSymbol)
app.eventHandler.warning(this, "Orderbook not handled by this MatchingEngine: " + order.orderbookSymbol)
done(false)
}
}
@ -56,7 +55,7 @@ class AkkaMatchingEngine(val meId: String, val orderbooks: List[Orderbook])
pendingStandbyFuture.await
} catch {
case e: FutureTimeoutException
EventHandler.error(this, "Standby timeout: " + e)
app.eventHandler.error(this, "Standby timeout: " + e)
}
}

View file

@ -3,7 +3,6 @@ package akka.performance.trading.common
import akka.performance.trading.domain._
import akka.actor._
import akka.dispatch.MessageDispatcher
import akka.event.EventHandler
trait OrderReceiver {
type ME
@ -32,7 +31,7 @@ class AkkaOrderReceiver extends Actor with OrderReceiver {
case routing @ MatchingEngineRouting(mapping)
refreshMatchingEnginePartitions(routing.asInstanceOf[MatchingEngineRouting[ActorRef]])
case order: Order placeOrder(order)
case unknown EventHandler.warning(this, "Received unknown message: " + unknown)
case unknown app.eventHandler.warning(this, "Received unknown message: " + unknown)
}
def placeOrder(order: Order) = {
@ -41,7 +40,7 @@ class AkkaOrderReceiver extends Actor with OrderReceiver {
case Some(m)
m.forward(order)
case None
EventHandler.warning(this, "Unknown orderbook: " + order.orderbookSymbol)
app.eventHandler.warning(this, "Unknown orderbook: " + order.orderbookSymbol)
channel ! new Rsp(false)
}
}

View file

@ -51,7 +51,7 @@ trait PerformanceTest extends JUnitSuite {
var stat: DescriptiveStatistics = _
val resultRepository = BenchResultRepository()
val resultRepository = BenchResultRepository(app)
lazy val report = new Report(app, resultRepository, compareResultWith)
type TS <: TradingSystem

View file

@ -2,7 +2,6 @@ package akka.performance.trading.oneway
import akka.actor._
import akka.dispatch.MessageDispatcher
import akka.event.EventHandler
import akka.performance.trading.domain.Order
import akka.performance.trading.domain.Orderbook
import akka.performance.trading.common.AkkaMatchingEngine
@ -18,7 +17,7 @@ class OneWayMatchingEngine(meId: String, orderbooks: List[Orderbook]) extends Ak
orderbook.matchOrders()
case None
EventHandler.warning(this, "Orderbook not handled by this MatchingEngine: " + order.orderbookSymbol)
app.eventHandler.warning(this, "Orderbook not handled by this MatchingEngine: " + order.orderbookSymbol)
}
}

View file

@ -2,7 +2,6 @@ package akka.performance.trading.oneway
import akka.actor._
import akka.dispatch.MessageDispatcher
import akka.event.EventHandler
import akka.performance.trading.domain._
import akka.performance.trading.common.AkkaOrderReceiver
@ -14,7 +13,7 @@ class OneWayOrderReceiver extends AkkaOrderReceiver {
case Some(m)
m ! order
case None
EventHandler.warning(this, "Unknown orderbook: " + order.orderbookSymbol)
app.eventHandler.warning(this, "Unknown orderbook: " + order.orderbookSymbol)
}
}
}

View file

@ -11,10 +11,9 @@ import java.io.ObjectOutputStream
import java.io.PrintWriter
import java.text.SimpleDateFormat
import java.util.Date
import scala.collection.mutable.{ Map MutableMap }
import akka.AkkaApplication
import akka.event.EventHandler
trait BenchResultRepository {
def add(stats: Stats)
@ -32,11 +31,10 @@ trait BenchResultRepository {
}
object BenchResultRepository {
private val repository = new FileBenchResultRepository
def apply(): BenchResultRepository = repository
def apply(app: AkkaApplication): BenchResultRepository = new FileBenchResultRepository(app)
}
class FileBenchResultRepository extends BenchResultRepository {
class FileBenchResultRepository(val app: AkkaApplication) extends BenchResultRepository {
private val statsByName = MutableMap[String, Seq[Stats]]()
private val baselineStats = MutableMap[Key, Stats]()
private val historicalStats = MutableMap[Key, Seq[Stats]]()
@ -105,7 +103,7 @@ class FileBenchResultRepository extends BenchResultRepository {
out.writeObject(stats)
} catch {
case e: Exception
EventHandler.error(this, "Failed to save [%s] to [%s], due to [%s]".
app.eventHandler.error(this, "Failed to save [%s] to [%s], due to [%s]".
format(stats, f.getAbsolutePath, e.getMessage))
} finally {
if (out ne null) try { out.close() } catch { case ignore: Exception }
@ -122,7 +120,7 @@ class FileBenchResultRepository extends BenchResultRepository {
Some(stats)
} catch {
case e: Throwable
EventHandler.error(this, "Failed to load from [%s], due to [%s]".
app.eventHandler.error(this, "Failed to load from [%s], due to [%s]".
format(f.getAbsolutePath, e.getMessage))
None
} finally {
@ -146,7 +144,7 @@ class FileBenchResultRepository extends BenchResultRepository {
writer.flush()
} catch {
case e: Exception
EventHandler.error(this, "Failed to save report to [%s], due to [%s]".
app.eventHandler.error(this, "Failed to save report to [%s], due to [%s]".
format(f.getAbsolutePath, e.getMessage))
} finally {
if (writer ne null) try { writer.close() } catch { case ignore: Exception }

View file

@ -5,7 +5,6 @@ import java.text.SimpleDateFormat
import java.util.Date
import scala.collection.JavaConversions.asScalaBuffer
import scala.collection.JavaConversions.enumerationAsScalaIterator
import akka.event.EventHandler
import akka.AkkaApplication
class Report(app: AkkaApplication,
@ -53,7 +52,7 @@ class Report(app: AkkaApplication,
resultRepository.saveHtmlReport(sb.toString, reportName)
if (log) {
EventHandler.info(this, resultTable + "Charts in html report: " + resultRepository.htmlReportUrl(reportName))
app.eventHandler.info(this, resultTable + "Charts in html report: " + resultRepository.htmlReportUrl(reportName))
}
}

View file

@ -1,37 +0,0 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.event;
import akka.actor.ActorRef;
/**
* Java API for Akka EventHandler
*/
public class JavaEventHandler {
public static void notify(Object message) {
EventHandler$.MODULE$.notify(message);
}
public static void debug(ActorRef instance, Object message) {
EventHandler$.MODULE$.debug(instance, message);
}
public static void info(ActorRef instance, Object message) {
EventHandler$.MODULE$.info(instance,message);
}
public static void warning(ActorRef instance, Object message) {
EventHandler$.MODULE$.warning(instance,message);
}
public static void error(ActorRef instance, Object message) {
EventHandler$.MODULE$.debug(instance,message);
}
}

View file

@ -16,6 +16,9 @@ import akka.dispatch.UnboundedMailbox
import akka.routing.Routing
import remote.RemoteSupport
import akka.serialization.Serialization
import akka.event.EventHandler
import akka.event.EventHandlerLogging
import akka.event.Logging
object AkkaApplication {
@ -124,6 +127,14 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor
import AkkaConfig._
if (ConfigVersion != Version)
throw new ConfigurationException("Akka JAR version [" + Version +
"] does not match the provided config version [" + ConfigVersion + "]")
val eventHandler = new EventHandler(this)
val log: Logging = new EventHandlerLogging(eventHandler, this)
val startTime = System.currentTimeMillis
def uptime = (System.currentTimeMillis - startTime) / 1000
@ -137,10 +148,6 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor
case value value
}
if (ConfigVersion != Version)
throw new ConfigurationException("Akka JAR version [" + Version +
"] does not match the provided config version [" + ConfigVersion + "]")
// TODO correctly pull its config from the config
val dispatcherFactory = new Dispatchers(this)

View file

@ -149,17 +149,17 @@ object Actor {
/**
* This decorator adds invocation logging to a Receive function.
*/
class LoggingReceive(source: AnyRef, r: Receive) extends Receive {
class LoggingReceive(source: AnyRef, r: Receive)(implicit app: AkkaApplication) extends Receive {
def isDefinedAt(o: Any) = {
val handled = r.isDefinedAt(o)
EventHandler.debug(source, "received " + (if (handled) "handled" else "unhandled") + " message " + o)
app.eventHandler.debug(source, "received " + (if (handled) "handled" else "unhandled") + " message " + o)
handled
}
def apply(o: Any): Unit = r(o)
}
object LoggingReceive {
def apply(source: AnyRef, r: Receive): Receive = r match {
def apply(source: AnyRef, r: Receive)(implicit app: AkkaApplication): Receive = r match {
case _: LoggingReceive r
case _ new LoggingReceive(source, r)
}
@ -211,9 +211,9 @@ trait Actor {
context
}
implicit def app = context.application
implicit def app = context.app
private def config = context.application.AkkaConfig
private def config = context.app.AkkaConfig
/**
* The default timeout, based on the config setting 'akka.actor.timeout'
@ -423,7 +423,7 @@ trait Actor {
throw new InvalidMessageException("Message from [" + channel + "] to [" + self.toString + "] is null")
def autoReceiveMessage(msg: AutoReceivedMessage) {
if (config.DebugAutoReceive) EventHandler.debug(this, "received AutoReceiveMessage " + msg)
if (config.DebugAutoReceive) app.eventHandler.debug(this, "received AutoReceiveMessage " + msg)
msg match {
case HotSwap(code, discardOld) become(code(self), discardOld)
@ -468,7 +468,6 @@ object Address {
def validate(address: String) {
if (!validAddressPattern.matcher(address).matches) {
val e = new IllegalArgumentException("Address [" + address + "] is not valid, need to follow pattern: " + validAddressPattern.pattern)
EventHandler.error(e, this, e.getMessage)
throw e
}
}

View file

@ -9,7 +9,7 @@ import akka.util._
import scala.annotation.tailrec
import scala.collection.immutable.Stack
import scala.collection.JavaConverters
import akka.event.{ InVMMonitoring, EventHandler }
import akka.event.InVMMonitoring
import java.util.concurrent.{ ScheduledFuture, TimeUnit }
import java.util.{ Collection JCollection, Collections JCollections }
import akka.AkkaApplication
@ -49,7 +49,8 @@ private[akka] trait ActorContext extends ActorRefFactory {
def handleChildTerminated(child: ActorRef): Unit
def application: AkkaApplication
def app: AkkaApplication
}
case class ChildRestartStats(val child: ActorRef, var maxNrOfRetriesCount: Int = 0, var restartTimeWindowStartNanos: Long = 0L) {
@ -195,7 +196,7 @@ case class OneForOneStrategy(trapExit: List[Class[_ <: Throwable]],
fail.actor.restart(fail.cause)
else
fail.actor.stop() //TODO optimization to drop child here already?
case None EventHandler.warning(this, "Got Failure from non-child: " + fail)
case None throw new AssertionError("Got Failure from non-child: " + fail)
}
}
}
@ -207,7 +208,7 @@ private[akka] object ActorCell {
}
private[akka] class ActorCell(
val application: AkkaApplication,
val app: AkkaApplication,
val self: ActorRef with ScalaActorRef,
val props: Props,
@volatile var receiveTimeout: Option[Long],
@ -215,7 +216,7 @@ private[akka] class ActorCell(
import ActorCell._
def provider = application.provider
def provider = app.provider
@volatile
var futureTimeout: Option[ScheduledFuture[AnyRef]] = None //FIXME TODO Doesn't need to be volatile either, since it will only ever be accessed when a message is processed
@ -251,7 +252,7 @@ private[akka] class ActorCell(
}
}
application.registry.register(self)
app.registry.register(self)
dispatcher.attach(this)
}
@ -333,10 +334,10 @@ private[akka] class ActorCell(
actor = created
created.preStart()
checkReceiveTimeout
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(created, "started")
if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "started")
} catch {
case e try {
EventHandler.error(e, this, "error while creating actor")
app.eventHandler.error(e, self, "error while creating actor")
// prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this)
envelope.channel.sendException(e)
@ -347,7 +348,7 @@ private[akka] class ActorCell(
def recreate(cause: Throwable): Unit = try {
val failedActor = actor
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(failedActor, "restarting")
if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "restarting")
val freshActor = newActor()
if (failedActor ne null) {
val c = currentMessage //One read only plz
@ -361,14 +362,14 @@ private[akka] class ActorCell(
}
actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call
freshActor.postRestart(cause)
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(freshActor, "restarted")
if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "restarted")
dispatcher.resume(this) //FIXME should this be moved down?
props.faultHandler.handleSupervisorRestarted(cause, self, _children)
} catch {
case e try {
EventHandler.error(e, this, "error while creating actor")
app.eventHandler.error(e, self, "error while creating actor")
// prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this)
envelope.channel.sendException(e)
@ -384,13 +385,13 @@ private[akka] class ActorCell(
def terminate() {
receiveTimeout = None
cancelReceiveTimeout
application.provider.evict(self.address)
application.registry.unregister(self)
app.provider.evict(self.address)
app.registry.unregister(self)
dispatcher.detach(this)
try {
val a = actor
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(a, "stopping")
if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "stopping")
if (a ne null) a.postStop()
//Stop supervised actors
@ -416,8 +417,8 @@ private[akka] class ActorCell(
val links = _children
if (!links.exists(_.child == child)) {
_children = links :+ ChildRestartStats(child)
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(actor, "now supervising " + child)
} else EventHandler.warning(actor, "Already supervising " + child)
if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "now supervising " + child)
} else app.eventHandler.warning(self, "Already supervising " + child)
}
try {
@ -428,10 +429,10 @@ private[akka] class ActorCell(
case Recreate(cause) recreate(cause)
case Link(subject)
akka.event.InVMMonitoring.link(self, subject)
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(actor, "now monitoring " + subject)
if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "now monitoring " + subject)
case Unlink(subject)
akka.event.InVMMonitoring.unlink(self, subject)
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(actor, "stopped monitoring " + subject)
if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "stopped monitoring " + subject)
case Suspend suspend()
case Resume resume()
case Terminate terminate()
@ -440,7 +441,7 @@ private[akka] class ActorCell(
}
} catch {
case e //Should we really catch everything here?
EventHandler.error(e, actor, "error while processing " + envelope.message)
app.eventHandler.error(e, self, "error while processing " + envelope.message)
//TODO FIXME How should problems here be handled?
throw e
} finally {
@ -461,7 +462,7 @@ private[akka] class ActorCell(
currentMessage = null // reset current message after successful invocation
} catch {
case e
EventHandler.error(e, self, e.getMessage)
app.eventHandler.error(e, self, e.getMessage)
// prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this)
@ -480,7 +481,7 @@ private[akka] class ActorCell(
}
} catch {
case e
EventHandler.error(e, actor, e.getMessage)
app.eventHandler.error(e, self, e.getMessage)
throw e
}
} else {

View file

@ -26,15 +26,15 @@ trait ActorDeployer {
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class Deployer(val application: AkkaApplication) extends ActorDeployer {
class Deployer(val app: AkkaApplication) extends ActorDeployer {
val deploymentConfig = new DeploymentConfig(application)
val deploymentConfig = new DeploymentConfig(app)
// val defaultAddress = Node(Config.nodename)
lazy val instance: ActorDeployer = {
val deployer = if (application.reflective.ClusterModule.isEnabled) {
application.reflective.ClusterModule.clusterDeployer
val deployer = if (app.reflective.ClusterModule.isEnabled) {
app.reflective.ClusterModule.clusterDeployer
} else {
LocalDeployer
}
@ -80,14 +80,14 @@ class Deployer(val application: AkkaApplication) extends ActorDeployer {
lookupInConfig(address)
} catch {
case e: ConfigurationException
EventHandler.error(e, this, e.getMessage)
app.eventHandler.error(e, this, e.getMessage)
throw e
}
newDeployment foreach { d
if (d eq null) {
val e = new IllegalStateException("Deployment for address [" + address + "] is null")
EventHandler.error(e, this, e.getMessage)
app.eventHandler.error(e, this, e.getMessage)
throw e
}
deploy(d) // deploy and cache it
@ -106,7 +106,7 @@ class Deployer(val application: AkkaApplication) extends ActorDeployer {
private[akka] def addressesInConfig: List[String] = {
val deploymentPath = "akka.actor.deployment"
application.config.getSection(deploymentPath) match {
app.config.getSection(deploymentPath) match {
case None Nil
case Some(addressConfig)
addressConfig.map.keySet
@ -118,7 +118,7 @@ class Deployer(val application: AkkaApplication) extends ActorDeployer {
/**
* Lookup deployment in 'akka.conf' configuration file.
*/
private[akka] def lookupInConfig(address: String, configuration: Configuration = application.config): Option[Deploy] = {
private[akka] def lookupInConfig(address: String, configuration: Configuration = app.config): Option[Deploy] = {
import akka.util.ReflectiveAccess.{ createInstance, emptyArguments, emptyParams, getClassFor }
// --------------------------------
@ -332,13 +332,13 @@ class Deployer(val application: AkkaApplication) extends ActorDeployer {
private[akka] def throwDeploymentBoundException(deployment: Deploy): Nothing = {
val e = new DeploymentAlreadyBoundException("Address [" + deployment.address + "] already bound to [" + deployment + "]")
EventHandler.error(e, this, e.getMessage)
app.eventHandler.error(e, this, e.getMessage)
throw e
}
private[akka] def thrownNoDeploymentBoundException(address: String): Nothing = {
val e = new NoDeploymentBoundException("Address [" + address + "] is not bound to a deployment")
EventHandler.error(e, this, e.getMessage)
app.eventHandler.error(e, this, e.getMessage)
throw e
}
}

View file

@ -420,7 +420,7 @@ trait FSM[S, D] extends ListenerManagement {
*/
private val handleEventDefault: StateFunction = {
case Event(value, stateData)
EventHandler.warning(this, "unhandled event " + value + " in state " + stateName)
app.eventHandler.warning(context.self, "unhandled event " + value + " in state " + stateName)
stay
}
private var handleEvent: StateFunction = handleEventDefault
@ -471,7 +471,7 @@ trait FSM[S, D] extends ListenerManagement {
actorRef ! CurrentState(self, currentState.stateName)
} catch {
case e: ActorInitializationException
EventHandler.warning(this, "trying to register not running listener")
app.eventHandler.warning(context.self, "trying to register not running listener")
}
case UnsubscribeTransitionCallBack(actorRef)
removeListener(actorRef)
@ -537,8 +537,8 @@ trait FSM[S, D] extends ListenerManagement {
if (!currentState.stopReason.isDefined) {
val reason = nextState.stopReason.get
reason match {
case Failure(ex: Throwable) EventHandler.error(ex, this, "terminating due to Failure")
case Failure(msg) EventHandler.error(this, msg)
case Failure(ex: Throwable) app.eventHandler.error(ex, context.self, "terminating due to Failure")
case Failure(msg) app.eventHandler.error(context.self, msg)
case _
}
val stopEvent = StopEvent(reason, currentState.stateName, currentState.stateData)
@ -568,7 +568,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒
def logDepth: Int = 0
private val debugEvent = context.application.AkkaConfig.FsmDebugEvent
private val debugEvent = context.app.AkkaConfig.FsmDebugEvent
private val events = new Array[Event](logDepth)
private val states = new Array[AnyRef](logDepth)
@ -587,13 +587,13 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒
protected[akka] abstract override def setTimer(name: String, msg: Any, timeout: Duration, repeat: Boolean): State = {
if (debugEvent)
EventHandler.debug(this, "setting " + (if (repeat) "repeating " else "") + "timer '" + name + "'/" + timeout + ": " + msg)
app.eventHandler.debug(context.self, "setting " + (if (repeat) "repeating " else "") + "timer '" + name + "'/" + timeout + ": " + msg)
super.setTimer(name, msg, timeout, repeat)
}
protected[akka] abstract override def cancelTimer(name: String) = {
if (debugEvent)
EventHandler.debug(this, "canceling timer '" + name + "'")
app.eventHandler.debug(context.self, "canceling timer '" + name + "'")
super.cancelTimer(name)
}
@ -605,7 +605,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒
case c: UntypedChannel c.toString
case _ "unknown"
}
EventHandler.debug(this, "processing " + event + " from " + srcstr)
app.eventHandler.debug(context.self, "processing " + event + " from " + srcstr)
}
if (logDepth > 0) {
@ -619,7 +619,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒
val newState = stateName
if (debugEvent && oldState != newState)
EventHandler.debug(this, "transition " + oldState + " -> " + newState)
app.eventHandler.debug(context.self, "transition " + oldState + " -> " + newState)
}
/**

View file

@ -6,7 +6,6 @@ package akka.actor
import akka.util.ByteString
import akka.dispatch.Envelope
import akka.event.EventHandler
import java.net.InetSocketAddress
import java.io.IOException
import java.util.concurrent.atomic.AtomicReference
@ -21,13 +20,12 @@ import java.nio.channels.{
SelectionKey,
CancelledKeyException
}
import scala.collection.mutable
import scala.collection.immutable.Queue
import scala.annotation.tailrec
import scala.util.continuations._
import com.eaio.uuid.UUID
import akka.AkkaApplication
object IO {
@ -257,7 +255,7 @@ class IOManager(bufferSize: Int = 8192) extends Actor {
var worker: IOWorker = _
override def preStart {
worker = new IOWorker(self, bufferSize)
worker = new IOWorker(app, self, bufferSize)
worker.start()
}
@ -294,7 +292,7 @@ private[akka] object IOWorker {
case object Shutdown extends Request
}
private[akka] class IOWorker(ioManager: ActorRef, val bufferSize: Int) {
private[akka] class IOWorker(app: AkkaApplication, ioManager: ActorRef, val bufferSize: Int) {
import SelectionKey.{ OP_READ, OP_WRITE, OP_ACCEPT, OP_CONNECT }
import IOWorker._
@ -400,7 +398,7 @@ private[akka] class IOWorker(ioManager: ActorRef, val bufferSize: Int) {
handle.owner ! IO.Closed(handle, cause)
} catch {
case e: ActorInitializationException
EventHandler debug (this, "IO.Handle's owner not running")
app.eventHandler debug (ioManager, "IO.Handle's owner not running")
}
case None
}

View file

@ -48,7 +48,6 @@ object Scheduler {
} catch {
case e: Exception
val error = SchedulerException(message + " could not be scheduled on " + receiver, e)
EventHandler.error(error, this, "%s @ %s".format(receiver, message))
throw error
}
}
@ -74,7 +73,6 @@ object Scheduler {
} catch {
case e: Exception
val error = SchedulerException("Failed to schedule a Runnable", e)
EventHandler.error(error, this, error.getMessage)
throw error
}
}
@ -90,7 +88,6 @@ object Scheduler {
} catch {
case e: Exception
val error = SchedulerException(message + " could not be scheduleOnce'd on " + receiver, e)
EventHandler.error(e, this, receiver + " @ " + message)
throw error
}
}
@ -116,7 +113,6 @@ object Scheduler {
} catch {
case e: Exception
val error = SchedulerException("Failed to scheduleOnce a Runnable", e)
EventHandler.error(e, this, error.getMessage)
throw error
}
}

View file

@ -44,12 +44,12 @@ final case class SystemEnvelope(val receiver: ActorCell, val message: SystemMess
}
}
final case class TaskInvocation(function: () Unit, cleanup: () Unit) extends Runnable {
final case class TaskInvocation(app: AkkaApplication, function: () Unit, cleanup: () Unit) extends Runnable {
def run() {
try {
function()
} catch {
case e EventHandler.error(e, this, e.getMessage)
case e app.eventHandler.error(e, this, e.getMessage)
} finally {
cleanup()
}
@ -67,7 +67,7 @@ object MessageDispatcher {
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
abstract class MessageDispatcher extends Serializable {
abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable {
import MessageDispatcher._
protected val uuids = new ConcurrentSkipListSet[Uuid]
@ -144,7 +144,7 @@ abstract class MessageDispatcher extends Serializable {
_tasks.getAndIncrement()
try {
startIfUnstarted()
executeTask(TaskInvocation(block, taskCleanup))
executeTask(TaskInvocation(app, block, taskCleanup))
} catch {
case e
_tasks.decrementAndGet
@ -331,7 +331,7 @@ abstract class MessageDispatcherConfigurator(val application: AkkaApplication) {
import ThreadPoolConfigDispatcherBuilder.conf_?
//Apply the following options to the config if they are present in the config
ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig()).configure(
ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig(application)).configure(
conf_?(config getInt "keep-alive-time")(time _.setKeepAliveTime(Duration(time, application.AkkaConfig.DefaultTimeUnit))),
conf_?(config getDouble "core-pool-size-factor")(factor _.setCorePoolSizeFromFactor(factor)),
conf_?(config getDouble "max-pool-size-factor")(factor _.setMaxPoolSizeFromFactor(factor)),

View file

@ -9,6 +9,7 @@ import akka.actor.{ ActorCell, Actor, IllegalActorStateException }
import java.util.concurrent.{ LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet }
import java.util.{ Comparator, Queue }
import annotation.tailrec
import akka.AkkaApplication
/**
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
@ -27,13 +28,14 @@ import annotation.tailrec
* @author Viktor Klang
*/
class BalancingDispatcher(
_app: AkkaApplication,
_name: String,
throughput: Int,
throughputDeadlineTime: Int,
mailboxType: MailboxType,
config: ThreadPoolConfig,
_timeoutMs: Long)
extends Dispatcher(_name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) {
extends Dispatcher(_app, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) {
private val buddies = new ConcurrentSkipListSet[ActorCell](new Comparator[ActorCell] { def compare(a: ActorCell, b: ActorCell) = a.uuid.compareTo(b.uuid) }) //new ConcurrentLinkedQueue[ActorCell]()

View file

@ -8,6 +8,7 @@ import akka.event.EventHandler
import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue }
import akka.actor.{ ActorCell, ActorKilledException }
import akka.AkkaApplication
/**
* Default settings are:
@ -63,13 +64,14 @@ import akka.actor.{ ActorCell, ActorKilledException }
* Larger values (or zero or negative) increase throughput, smaller values increase fairness
*/
class Dispatcher(
_app: AkkaApplication,
_name: String,
val throughput: Int,
val throughputDeadlineTime: Int,
val mailboxType: MailboxType,
executorServiceFactoryProvider: ExecutorServiceFactoryProvider,
val timeoutMs: Long)
extends MessageDispatcher {
extends MessageDispatcher(_app) {
val name = "akka:event-driven:dispatcher:" + _name
@ -93,7 +95,7 @@ class Dispatcher(
executorService.get() execute invocation
} catch {
case e: RejectedExecutionException
EventHandler.warning(this, e.toString)
app.eventHandler.warning(this, e.toString)
throw e
}
}
@ -119,7 +121,7 @@ class Dispatcher(
true
} catch {
case e: RejectedExecutionException
EventHandler.warning(this, e.toString)
app.eventHandler.warning(this, e.toString)
mbox.setAsIdle()
throw e
}

View file

@ -43,15 +43,15 @@ import akka.AkkaApplication
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class Dispatchers(val application: AkkaApplication) {
val ThroughputDeadlineTimeMillis = application.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt
class Dispatchers(val app: AkkaApplication) {
val ThroughputDeadlineTimeMillis = app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt
val MailboxType: MailboxType =
if (application.AkkaConfig.MailboxCapacity < 1) UnboundedMailbox()
else BoundedMailbox(application.AkkaConfig.MailboxCapacity, application.AkkaConfig.MailboxPushTimeout)
val DispatcherShutdownMillis = application.AkkaConfig.DispatcherDefaultShutdown.toMillis
if (app.AkkaConfig.MailboxCapacity < 1) UnboundedMailbox()
else BoundedMailbox(app.AkkaConfig.MailboxCapacity, app.AkkaConfig.MailboxPushTimeout)
val DispatcherShutdownMillis = app.AkkaConfig.DispatcherDefaultShutdown.toMillis
lazy val defaultGlobalDispatcher =
application.config.getSection("akka.actor.default-dispatcher").flatMap(from) getOrElse newDispatcher("AkkaDefaultGlobalDispatcher", 1, MailboxType).build
app.config.getSection("akka.actor.default-dispatcher").flatMap(from) getOrElse newDispatcher("AkkaDefaultGlobalDispatcher", 1, MailboxType).build
/**
* Creates an thread based dispatcher serving a single actor through the same single thread.
@ -60,8 +60,8 @@ class Dispatchers(val application: AkkaApplication) {
* E.g. each actor consumes its own thread.
*/
def newPinnedDispatcher(actor: LocalActorRef) = actor match {
case null new PinnedDispatcher(null, "anon", MailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(some.underlying, some.underlying.uuid.toString, MailboxType, DispatcherShutdownMillis)
case null new PinnedDispatcher(app, null, "anon", MailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(app, some.underlying, some.underlying.uuid.toString, MailboxType, DispatcherShutdownMillis)
}
/**
@ -71,8 +71,8 @@ class Dispatchers(val application: AkkaApplication) {
* E.g. each actor consumes its own thread.
*/
def newPinnedDispatcher(actor: LocalActorRef, mailboxType: MailboxType) = actor match {
case null new PinnedDispatcher(null, "anon", mailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(some.underlying, some.underlying.uuid.toString, mailboxType, DispatcherShutdownMillis)
case null new PinnedDispatcher(app, null, "anon", mailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(app, some.underlying, some.underlying.uuid.toString, mailboxType, DispatcherShutdownMillis)
}
/**
@ -81,7 +81,7 @@ class Dispatchers(val application: AkkaApplication) {
* E.g. each actor consumes its own thread.
*/
def newPinnedDispatcher(name: String, mailboxType: MailboxType) =
new PinnedDispatcher(null, name, mailboxType, DispatcherShutdownMillis)
new PinnedDispatcher(app, null, name, mailboxType, DispatcherShutdownMillis)
/**
* Creates an thread based dispatcher serving a single actor through the same single thread.
@ -89,7 +89,7 @@ class Dispatchers(val application: AkkaApplication) {
* E.g. each actor consumes its own thread.
*/
def newPinnedDispatcher(name: String) =
new PinnedDispatcher(null, name, MailboxType, DispatcherShutdownMillis)
new PinnedDispatcher(app, null, name, MailboxType, DispatcherShutdownMillis)
/**
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
@ -97,8 +97,8 @@ class Dispatchers(val application: AkkaApplication) {
* Has a fluent builder interface for configuring its semantics.
*/
def newDispatcher(name: String) =
ThreadPoolConfigDispatcherBuilder(config new Dispatcher(name, application.AkkaConfig.DispatcherThroughput,
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
ThreadPoolConfigDispatcherBuilder(config new Dispatcher(app, name, app.AkkaConfig.DispatcherThroughput,
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
/**
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
@ -107,7 +107,7 @@ class Dispatchers(val application: AkkaApplication) {
*/
def newDispatcher(name: String, throughput: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config
new Dispatcher(name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
new Dispatcher(app, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
/**
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
@ -116,7 +116,7 @@ class Dispatchers(val application: AkkaApplication) {
*/
def newDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config
new Dispatcher(name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
new Dispatcher(app, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
/**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -124,8 +124,8 @@ class Dispatchers(val application: AkkaApplication) {
* Has a fluent builder interface for configuring its semantics.
*/
def newBalancingDispatcher(name: String) =
ThreadPoolConfigDispatcherBuilder(config new BalancingDispatcher(name, application.AkkaConfig.DispatcherThroughput,
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
ThreadPoolConfigDispatcherBuilder(config new BalancingDispatcher(app, name, app.AkkaConfig.DispatcherThroughput,
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
/**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -134,7 +134,7 @@ class Dispatchers(val application: AkkaApplication) {
*/
def newBalancingDispatcher(name: String, throughput: Int) =
ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(name, throughput, ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
new BalancingDispatcher(app, name, throughput, ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
/**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -143,7 +143,7 @@ class Dispatchers(val application: AkkaApplication) {
*/
def newBalancingDispatcher(name: String, throughput: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
new BalancingDispatcher(app, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
/**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -152,13 +152,13 @@ class Dispatchers(val application: AkkaApplication) {
*/
def newBalancingDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
new BalancingDispatcher(app, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
/**
* Utility function that tries to load the specified dispatcher config from the akka.conf
* or else use the supplied default dispatcher
*/
def fromConfig(key: String, default: MessageDispatcher = defaultGlobalDispatcher): MessageDispatcher =
application.config getSection key flatMap from getOrElse default
app.config getSection key flatMap from getOrElse default
/*
* Creates of obtains a dispatcher from a ConfigMap according to the format below
@ -185,8 +185,8 @@ class Dispatchers(val application: AkkaApplication) {
*/
def from(cfg: Configuration): Option[MessageDispatcher] = {
cfg.getString("type") flatMap {
case "Dispatcher" Some(new DispatcherConfigurator(application))
case "BalancingDispatcher" Some(new BalancingDispatcherConfigurator(application))
case "Dispatcher" Some(new DispatcherConfigurator(app))
case "BalancingDispatcher" Some(new BalancingDispatcherConfigurator(app))
case "GlobalDispatcher" None //TODO FIXME remove this
case fqn
ReflectiveAccess.getClassFor[MessageDispatcherConfigurator](fqn) match {
@ -206,26 +206,26 @@ class Dispatchers(val application: AkkaApplication) {
}
}
class DispatcherConfigurator(application: AkkaApplication) extends MessageDispatcherConfigurator(application) {
class DispatcherConfigurator(app: AkkaApplication) extends MessageDispatcherConfigurator(app) {
def configure(config: Configuration): MessageDispatcher = {
configureThreadPool(config, threadPoolConfig new Dispatcher(
configureThreadPool(config, threadPoolConfig new Dispatcher(app,
config.getString("name", newUuid.toString),
config.getInt("throughput", application.AkkaConfig.DispatcherThroughput),
config.getInt("throughput-deadline-time", application.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt),
config.getInt("throughput", app.AkkaConfig.DispatcherThroughput),
config.getInt("throughput-deadline-time", app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt),
mailboxType(config),
threadPoolConfig,
application.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build
app.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build
}
}
class BalancingDispatcherConfigurator(application: AkkaApplication) extends MessageDispatcherConfigurator(application) {
class BalancingDispatcherConfigurator(app: AkkaApplication) extends MessageDispatcherConfigurator(app) {
def configure(config: Configuration): MessageDispatcher = {
configureThreadPool(config, threadPoolConfig new BalancingDispatcher(
configureThreadPool(config, threadPoolConfig new BalancingDispatcher(app,
config.getString("name", newUuid.toString),
config.getInt("throughput", application.AkkaConfig.DispatcherThroughput),
config.getInt("throughput-deadline-time", application.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt),
config.getInt("throughput", app.AkkaConfig.DispatcherThroughput),
config.getInt("throughput-deadline-time", app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt),
mailboxType(config),
threadPoolConfig,
application.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build
app.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build
}
}

View file

@ -262,7 +262,7 @@ object Future {
result completeWithResult currentValue
} catch {
case e: Exception
EventHandler.error(e, this, e.getMessage)
dispatcher.app.eventHandler.error(e, this, e.getMessage)
result completeWithException e
} finally {
results.clear
@ -596,7 +596,7 @@ sealed trait Future[+T] extends japi.Future[T] {
Right(f(res))
} catch {
case e: Exception
EventHandler.error(e, this, e.getMessage)
dispatcher.app.eventHandler.error(e, this, e.getMessage)
Left(e)
})
}
@ -648,7 +648,7 @@ sealed trait Future[+T] extends japi.Future[T] {
future.completeWith(f(r))
} catch {
case e: Exception
EventHandler.error(e, this, e.getMessage)
dispatcher.app.eventHandler.error(e, this, e.getMessage)
future complete Left(e)
}
}
@ -681,7 +681,7 @@ sealed trait Future[+T] extends japi.Future[T] {
if (p(res)) r else Left(new MatchError(res))
} catch {
case e: Exception
EventHandler.error(e, this, e.getMessage)
dispatcher.app.eventHandler.error(e, this, e.getMessage)
Left(e)
})
}
@ -781,7 +781,7 @@ trait Promise[T] extends Future[T] {
fr completeWith cont(f)
} catch {
case e: Exception
EventHandler.error(e, this, e.getMessage)
dispatcher.app.eventHandler.error(e, this, e.getMessage)
fr completeWithException e
}
}
@ -795,7 +795,7 @@ trait Promise[T] extends Future[T] {
fr completeWith cont(f)
} catch {
case e: Exception
EventHandler.error(e, this, e.getMessage)
dispatcher.app.eventHandler.error(e, this, e.getMessage)
fr completeWithException e
}
}
@ -957,7 +957,7 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi
} else this
private def notifyCompleted(func: Future[T] Unit) {
try { func(this) } catch { case e EventHandler.error(e, this, "Future onComplete-callback raised an exception") } //TODO catch, everything? Really?
try { func(this) } catch { case e dispatcher.app.eventHandler.error(e, this, "Future onComplete-callback raised an exception") } //TODO catch, everything? Really?
}
@inline

View file

@ -6,14 +6,15 @@ package akka.dispatch
import java.util.concurrent.atomic.AtomicReference
import akka.actor.ActorCell
import akka.AkkaApplication
/**
* Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class PinnedDispatcher(_actor: ActorCell, _name: String, _mailboxType: MailboxType, _timeoutMs: Long)
extends Dispatcher(_name, Int.MaxValue, -1, _mailboxType, PinnedDispatcher.oneThread, _timeoutMs) {
class PinnedDispatcher(_app: AkkaApplication, _actor: ActorCell, _name: String, _mailboxType: MailboxType, _timeoutMs: Long)
extends Dispatcher(_app, _name, Int.MaxValue, -1, _mailboxType, PinnedDispatcher.oneThread(_app), _timeoutMs) {
protected[akka] val owner = new AtomicReference[ActorCell](_actor)
@ -32,6 +33,6 @@ class PinnedDispatcher(_actor: ActorCell, _name: String, _mailboxType: MailboxTy
}
object PinnedDispatcher {
val oneThread: ThreadPoolConfig = ThreadPoolConfig(allowCorePoolTimeout = true, corePoolSize = 1, maxPoolSize = 1)
def oneThread(app: AkkaApplication): ThreadPoolConfig = ThreadPoolConfig(app, allowCorePoolTimeout = true, corePoolSize = 1, maxPoolSize = 1)
}

View file

@ -8,9 +8,9 @@ import java.util.Collection
import java.util.concurrent._
import atomic.{ AtomicLong, AtomicInteger }
import ThreadPoolExecutor.CallerRunsPolicy
import akka.util.Duration
import akka.event.EventHandler
import akka.AkkaApplication
object ThreadPoolConfig {
type Bounds = Int
@ -59,7 +59,8 @@ trait ExecutorServiceFactoryProvider {
def createExecutorServiceFactory(name: String): ExecutorServiceFactory
}
case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout,
case class ThreadPoolConfig(app: AkkaApplication,
allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout,
corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize,
maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize,
threadTimeout: Duration = ThreadPoolConfig.defaultTimeout,
@ -76,7 +77,7 @@ case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.def
case Right(bounds)
val service = new ThreadPoolExecutor(corePoolSize, maxPoolSize, threadTimeout.length, threadTimeout.unit, queueFactory(), threadFactory)
service.allowCoreThreadTimeOut(allowCorePoolTimeout)
new BoundedExecutorDecorator(service, bounds)
new BoundedExecutorDecorator(app, service, bounds)
}
}
}
@ -197,7 +198,7 @@ class MonitorableThread(runnable: Runnable, name: String)
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extends ExecutorServiceDelegate {
class BoundedExecutorDecorator(val app: AkkaApplication, val executor: ExecutorService, bound: Int) extends ExecutorServiceDelegate {
protected val semaphore = new Semaphore(bound)
override def execute(command: Runnable) = {
@ -214,10 +215,10 @@ class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extend
})
} catch {
case e: RejectedExecutionException
EventHandler.warning(this, e.toString)
app.eventHandler.warning(this, e.toString)
semaphore.release
case e: Throwable
EventHandler.error(e, this, e.getMessage)
app.eventHandler.error(e, this, e.getMessage)
throw e
}
}

View file

@ -12,6 +12,124 @@ import akka.serialization._
import akka.AkkaException
import akka.AkkaApplication
object EventHandler {
val ErrorLevel = 1
val WarningLevel = 2
val InfoLevel = 3
val DebugLevel = 4
val errorFormat = "[ERROR] [%s] [%s] [%s] %s\n%s".intern
val warningFormat = "[WARN] [%s] [%s] [%s] %s".intern
val infoFormat = "[INFO] [%s] [%s] [%s] %s".intern
val debugFormat = "[DEBUG] [%s] [%s] [%s] %s".intern
val genericFormat = "[GENERIC] [%s] [%s]".intern
class EventHandlerException extends AkkaException
lazy val StandardOutLogger = new StandardOutLogger {}
sealed trait Event {
@transient
val thread: Thread = Thread.currentThread
def level: Int
}
case class Error(cause: Throwable, instance: AnyRef, message: Any = "") extends Event {
def level = ErrorLevel
}
case class Warning(instance: AnyRef, message: Any = "") extends Event {
def level = WarningLevel
}
case class Info(instance: AnyRef, message: Any = "") extends Event {
def level = InfoLevel
}
case class Debug(instance: AnyRef, message: Any = "") extends Event {
def level = DebugLevel
}
trait StandardOutLogger {
import java.text.SimpleDateFormat
import java.util.Date
val dateFormat = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss.S")
def timestamp = dateFormat.format(new Date)
def print(event: Any) {
event match {
case e: Error error(e)
case e: Warning warning(e)
case e: Info info(e)
case e: Debug debug(e)
case e generic(e)
}
}
def error(event: Error) =
println(errorFormat.format(
timestamp,
event.thread.getName,
instanceName(event.instance),
event.message,
stackTraceFor(event.cause)))
def warning(event: Warning) =
println(warningFormat.format(
timestamp,
event.thread.getName,
instanceName(event.instance),
event.message))
def info(event: Info) =
println(infoFormat.format(
timestamp,
event.thread.getName,
instanceName(event.instance),
event.message))
def debug(event: Debug) =
println(debugFormat.format(
timestamp,
event.thread.getName,
instanceName(event.instance),
event.message))
def generic(event: Any) =
println(genericFormat.format(timestamp, event.toString))
def instanceName(instance: AnyRef): String = instance match {
case null "NULL"
case a: ActorRef a.address
case _ instance.getClass.getSimpleName
}
}
class DefaultListener extends Actor with StandardOutLogger {
def receive = { case event print(event) }
}
def stackTraceFor(e: Throwable) = {
import java.io.{ StringWriter, PrintWriter }
val sw = new StringWriter
val pw = new PrintWriter(sw)
e.printStackTrace(pw)
sw.toString
}
private def levelFor(eventClass: Class[_ <: Event]) = {
if (classOf[Error].isAssignableFrom(eventClass)) ErrorLevel
else if (classOf[Warning].isAssignableFrom(eventClass)) WarningLevel
else if (classOf[Info].isAssignableFrom(eventClass)) InfoLevel
else if (classOf[Debug].isAssignableFrom(eventClass)) DebugLevel
else DebugLevel
}
}
/**
* Event handler.
* <p/>
@ -53,60 +171,22 @@ import akka.AkkaApplication
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object EventHandler extends ListenerManagement {
class EventHandler(app: AkkaApplication) extends ListenerManagement {
// TODO remove this EVIL thing!
private val appl = AkkaApplication("akka-reference.conf")
import EventHandler._
val synchronousLogging: Boolean = System.getProperty("akka.event.force-sync") match {
case null | "" false
case _ true
}
val ErrorLevel = 1
val WarningLevel = 2
val InfoLevel = 3
val DebugLevel = 4
sealed trait Event {
@transient
val thread: Thread = Thread.currentThread
def level: Int
}
case class Error(cause: Throwable, instance: AnyRef, message: Any = "") extends Event {
def level = ErrorLevel
}
case class Warning(instance: AnyRef, message: Any = "") extends Event {
def level = WarningLevel
}
case class Info(instance: AnyRef, message: Any = "") extends Event {
def level = InfoLevel
}
case class Debug(instance: AnyRef, message: Any = "") extends Event {
def level = DebugLevel
}
val errorFormat = "[ERROR] [%s] [%s] [%s] %s\n%s".intern
val warningFormat = "[WARN] [%s] [%s] [%s] %s".intern
val infoFormat = "[INFO] [%s] [%s] [%s] %s".intern
val debugFormat = "[DEBUG] [%s] [%s] [%s] %s".intern
val genericFormat = "[GENERIC] [%s] [%s]".intern
class EventHandlerException extends AkkaException
lazy val StandardOutLogger = new StandardOutLogger {}
lazy val EventHandlerDispatcher =
appl.dispatcherFactory.fromConfig("akka.event-handler-dispatcher", appl.dispatcherFactory.newDispatcher("event-handler-dispatcher").setCorePoolSize(2).build)
app.dispatcherFactory.fromConfig("akka.event-handler-dispatcher", app.dispatcherFactory.newDispatcher("event-handler-dispatcher").setCorePoolSize(2).build)
implicit object defaultListenerFormat extends StatelessActorFormat[DefaultListener]
@volatile
var level: Int = appl.AkkaConfig.LogLevel match {
var level: Int = app.AkkaConfig.LogLevel match {
case "ERROR" | "error" ErrorLevel
case "WARNING" | "warning" WarningLevel
case "INFO" | "info" InfoLevel
@ -117,14 +197,14 @@ object EventHandler extends ListenerManagement {
def start() {
try {
val defaultListeners = appl.AkkaConfig.EventHandlers match {
val defaultListeners = app.AkkaConfig.EventHandlers match {
case Nil "akka.event.EventHandler$DefaultListener" :: Nil
case listeners listeners
}
defaultListeners foreach { listenerName
try {
ReflectiveAccess.getClassFor[Actor](listenerName) match {
case Right(actorClass) addListener(new LocalActorRef(appl, Props(actorClass).withDispatcher(EventHandlerDispatcher), newUuid.toString, systemService = true))
case Right(actorClass) addListener(new LocalActorRef(app, Props(actorClass).withDispatcher(EventHandlerDispatcher), newUuid.toString, systemService = true))
case Left(exception) throw exception
}
} catch {
@ -207,87 +287,10 @@ object EventHandler extends ListenerManagement {
def isDebugEnabled = level >= DebugLevel
def stackTraceFor(e: Throwable) = {
import java.io.{ StringWriter, PrintWriter }
val sw = new StringWriter
val pw = new PrintWriter(sw)
e.printStackTrace(pw)
sw.toString
}
private def levelFor(eventClass: Class[_ <: Event]) = {
if (classOf[Error].isAssignableFrom(eventClass)) ErrorLevel
else if (classOf[Warning].isAssignableFrom(eventClass)) WarningLevel
else if (classOf[Info].isAssignableFrom(eventClass)) InfoLevel
else if (classOf[Debug].isAssignableFrom(eventClass)) DebugLevel
else DebugLevel
}
private def log(event: Any) {
if (synchronousLogging) StandardOutLogger.print(event)
else notifyListeners(event)
}
trait StandardOutLogger {
import java.text.SimpleDateFormat
import java.util.Date
val dateFormat = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss.S")
def timestamp = dateFormat.format(new Date)
def print(event: Any) {
event match {
case e: Error error(e)
case e: Warning warning(e)
case e: Info info(e)
case e: Debug debug(e)
case e generic(e)
}
}
def error(event: Error) =
println(errorFormat.format(
timestamp,
event.thread.getName,
instanceName(event.instance),
event.message,
stackTraceFor(event.cause)))
def warning(event: Warning) =
println(warningFormat.format(
timestamp,
event.thread.getName,
instanceName(event.instance),
event.message))
def info(event: Info) =
println(infoFormat.format(
timestamp,
event.thread.getName,
instanceName(event.instance),
event.message))
def debug(event: Debug) =
println(debugFormat.format(
timestamp,
event.thread.getName,
instanceName(event.instance),
event.message))
def generic(event: Any) =
println(genericFormat.format(timestamp, event.toString))
def instanceName(instance: AnyRef): String = instance match {
case null "NULL"
case a: ActorRef a.address
case _ instance.getClass.getSimpleName
}
}
class DefaultListener extends Actor with StandardOutLogger {
def receive = { case event print(event) }
}
start()
}

View file

@ -0,0 +1,108 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.event
import akka.actor.Actor
/**
* Logging wrapper to make nicer and optimize: provide template versions which
* evaluate .toString only if the log level is actually enabled.
*/
trait Logging {
/*
* implement these as precisely as needed/possible: always returning true
* just makes the notify... methods be called every time.
*/
def isErrorEnabled: Boolean
def isWarningEnabled: Boolean
def isInfoEnabled: Boolean
def isDebugEnabled: Boolean
/*
* These actually implement the passing on of the messages to be logged.
* Will not be called if is...Enabled returned false.
*/
protected def notifyError(cause: Throwable, message: String)
protected def notifyWarning(message: String)
protected def notifyInfo(message: String)
protected def notifyDebug(message: String)
/*
* The rest is just the widening of the API for the user's convenience.
*/
def error(cause: Throwable, message: String) { if (isErrorEnabled) notifyError(cause, message) }
def error(cause: Throwable, template: String, arg1: Any) { if (isErrorEnabled) error(cause, format(template, arg1)) }
def error(cause: Throwable, template: String, arg1: Any, arg2: Any) { if (isErrorEnabled) error(cause, format(template, arg1, arg2)) }
def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any) { if (isErrorEnabled) error(cause, format(template, arg1, arg2, arg3)) }
def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isErrorEnabled) error(cause, format(template, arg1, arg2, arg3, arg4)) }
def error(message: String) { if (isErrorEnabled) error(null: Throwable, message) }
def error(template: String, arg1: Any) { if (isErrorEnabled) error(format(template, arg1)) }
def error(template: String, arg1: Any, arg2: Any) { if (isErrorEnabled) error(format(template, arg1, arg2)) }
def error(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isErrorEnabled) error(format(template, arg1, arg2, arg3)) }
def error(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isErrorEnabled) error(format(template, arg1, arg2, arg3, arg4)) }
def warning(message: String) { if (isWarningEnabled) notifyWarning(message) }
def warning(template: String, arg1: Any) { if (isWarningEnabled) warning(format(template, arg1)) }
def warning(template: String, arg1: Any, arg2: Any) { if (isWarningEnabled) warning(format(template, arg1, arg2)) }
def warning(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isWarningEnabled) warning(format(template, arg1, arg2, arg3)) }
def warning(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isWarningEnabled) warning(format(template, arg1, arg2, arg3, arg4)) }
def info(message: String) { if (isInfoEnabled) notifyInfo(message) }
def info(template: String, arg1: Any) { if (isInfoEnabled) info(format(template, arg1)) }
def info(template: String, arg1: Any, arg2: Any) { if (isInfoEnabled) info(format(template, arg1, arg2)) }
def info(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isInfoEnabled) info(format(template, arg1, arg2, arg3)) }
def info(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isInfoEnabled) info(format(template, arg1, arg2, arg3, arg4)) }
def debug(message: String) { if (isDebugEnabled) notifyDebug(message) }
def debug(template: String, arg1: Any) { if (isDebugEnabled) debug(format(template, arg1)) }
def debug(template: String, arg1: Any, arg2: Any) { if (isDebugEnabled) debug(format(template, arg1, arg2)) }
def debug(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isDebugEnabled) debug(format(template, arg1, arg2, arg3)) }
def debug(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isDebugEnabled) debug(format(template, arg1, arg2, arg3, arg4)) }
def format(t: String, arg1: Any) = t.replaceFirst("{}", arg1.asInstanceOf[AnyRef].toString)
def format(t: String, arg1: Any, arg2: Any) = t.replaceFirst("{}", arg1.asInstanceOf[AnyRef].toString).replaceFirst("{}", arg2.asInstanceOf[AnyRef].toString)
def format(t: String, arg1: Any, arg2: Any, arg3: Any) = t.replaceFirst("{}", arg1.asInstanceOf[AnyRef].toString).replaceFirst("{}", arg2.asInstanceOf[AnyRef].toString).replaceFirst("{}", arg3.asInstanceOf[AnyRef].toString)
def format(t: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) = t.replaceFirst("{}", arg1.asInstanceOf[AnyRef].toString).replaceFirst("{}", arg2.asInstanceOf[AnyRef].toString).replaceFirst("{}", arg3.asInstanceOf[AnyRef].toString).replaceFirst("{}", arg4.asInstanceOf[AnyRef].toString)
}
trait ActorLogging extends Logging { this: Actor =>
import EventHandler._
def isErrorEnabled = app.eventHandler.level >= ErrorLevel
def isWarningEnabled = app.eventHandler.level >= WarningLevel
def isInfoEnabled = app.eventHandler.level >= InfoLevel
def isDebugEnabled = app.eventHandler.level >= DebugLevel
protected def notifyError(cause: Throwable, message: String) { app.eventHandler.notifyListeners(Error(cause, context.self, message)) }
protected def notifyWarning(message: String) { app.eventHandler.notifyListeners(Warning(context.self, message)) }
protected def notifyInfo(message: String) { app.eventHandler.notifyListeners(Info(context.self, message)) }
protected def notifyDebug(message: String) { app.eventHandler.notifyListeners(Debug(context.self, message)) }
}
class EventHandlerLogging(val eventHandler: EventHandler, val loggingInstance: AnyRef) extends Logging {
import EventHandler._
def isErrorEnabled = eventHandler.level >= ErrorLevel
def isWarningEnabled = eventHandler.level >= WarningLevel
def isInfoEnabled = eventHandler.level >= InfoLevel
def isDebugEnabled = eventHandler.level >= DebugLevel
protected def notifyError(cause: Throwable, message: String) { eventHandler.notifyListeners(Error(cause, loggingInstance, message)) }
protected def notifyWarning(message: String) { eventHandler.notifyListeners(Warning(loggingInstance, message)) }
protected def notifyInfo(message: String) { eventHandler.notifyListeners(Info(loggingInstance, message)) }
protected def notifyDebug(message: String) { eventHandler.notifyListeners(Debug(loggingInstance, message)) }
}

View file

@ -5,7 +5,6 @@
package akka.remote
import akka.actor.Actor
import akka.event.EventHandler
/**
* RemoteModule client and server event listener that pipes the events to the standard Akka EventHander.
@ -18,33 +17,33 @@ class RemoteEventHandler extends Actor {
// client
case RemoteClientError(cause, client, address)
EventHandler.error(cause, client, "RemoteClientError - Address[%s]" format address.toString)
app.eventHandler.error(cause, client, "RemoteClientError - Address[%s]" format address.toString)
case RemoteClientWriteFailed(request, cause, client, address)
EventHandler.error(cause, client, "RemoteClientWriteFailed - Request[%s] Address[%s]".format(request, address.toString))
app.eventHandler.error(cause, client, "RemoteClientWriteFailed - Request[%s] Address[%s]".format(request, address.toString))
case RemoteClientDisconnected(client, address)
EventHandler.info(client, "RemoteClientDisconnected - Address[%s]" format address.toString)
app.eventHandler.info(client, "RemoteClientDisconnected - Address[%s]" format address.toString)
case RemoteClientConnected(client, address)
EventHandler.info(client, "RemoteClientConnected - Address[%s]" format address.toString)
app.eventHandler.info(client, "RemoteClientConnected - Address[%s]" format address.toString)
case RemoteClientStarted(client, address)
EventHandler.info(client, "RemoteClientStarted - Address[%s]" format address.toString)
app.eventHandler.info(client, "RemoteClientStarted - Address[%s]" format address.toString)
case RemoteClientShutdown(client, address)
EventHandler.info(client, "RemoteClientShutdown - Address[%s]" format address.toString)
app.eventHandler.info(client, "RemoteClientShutdown - Address[%s]" format address.toString)
// server
case RemoteServerError(cause, server)
EventHandler.error(cause, server, "RemoteServerError")
app.eventHandler.error(cause, server, "RemoteServerError")
case RemoteServerWriteFailed(request, cause, server, clientAddress)
EventHandler.error(cause, server, "RemoteServerWriteFailed - Request[%s] Address[%s]" format (request, clientAddress.toString))
app.eventHandler.error(cause, server, "RemoteServerWriteFailed - Request[%s] Address[%s]" format (request, clientAddress.toString))
case RemoteServerStarted(server)
EventHandler.info(server, "RemoteServerStarted")
app.eventHandler.info(server, "RemoteServerStarted")
case RemoteServerShutdown(server)
EventHandler.info(server, "RemoteServerShutdown")
app.eventHandler.info(server, "RemoteServerShutdown")
case RemoteServerClientConnected(server, clientAddress)
EventHandler.info(server, "RemoteServerClientConnected - Address[%s]" format clientAddress.toString)
app.eventHandler.info(server, "RemoteServerClientConnected - Address[%s]" format clientAddress.toString)
case RemoteServerClientDisconnected(server, clientAddress)
EventHandler.info(server, "RemoteServerClientDisconnected - Address[%s]" format clientAddress.toString)
app.eventHandler.info(server, "RemoteServerClientDisconnected - Address[%s]" format clientAddress.toString)
case RemoteServerClientClosed(server, clientAddress)
EventHandler.info(server, "RemoteServerClientClosed - Address[%s]" format clientAddress.toString)
app.eventHandler.info(server, "RemoteServerClientClosed - Address[%s]" format clientAddress.toString)
case _ //ignore other
}

View file

@ -315,7 +315,6 @@ trait BasicRouter extends Router {
private def throwNoConnectionsError = {
val error = new RoutingException("No replica connections for router")
EventHandler.error(error, this, error.toString)
throw error
}
}

View file

@ -45,7 +45,6 @@ object Helpers {
while (root.getCause ne null) root = e.getCause
root.printStackTrace(new java.io.PrintWriter(sw))
System.err.println(sw.toString)
EventHandler.error(e, this, e.toString)
throw e
}
}

View file

@ -5,9 +5,9 @@
package akka.util
import akka.event.EventHandler
import java.lang.management.ManagementFactory
import javax.management.{ ObjectInstance, ObjectName, InstanceAlreadyExistsException, InstanceNotFoundException }
import akka.AkkaApplication
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
@ -18,20 +18,20 @@ object JMX {
def nameFor(hostname: String, service: String, bean: String): ObjectName =
new ObjectName("akka.%s:type=%s,name=%s".format(hostname, service, bean.replace(":", "_")))
def register(name: ObjectName, mbean: AnyRef): Option[ObjectInstance] = try {
def register(name: ObjectName, mbean: AnyRef)(implicit app: AkkaApplication): Option[ObjectInstance] = try {
Some(mbeanServer.registerMBean(mbean, name))
} catch {
case e: InstanceAlreadyExistsException
Some(mbeanServer.getObjectInstance(name))
case e: Exception
EventHandler.error(e, this, "Error when registering mbean [%s]".format(mbean))
app.eventHandler.error(e, this, "Error when registering mbean [%s]".format(mbean))
None
}
def unregister(mbean: ObjectName) = try {
def unregister(mbean: ObjectName)(implicit app: AkkaApplication) = try {
mbeanServer.unregisterMBean(mbean)
} catch {
case e: InstanceNotFoundException {}
case e: Exception EventHandler.error(e, this, "Error while unregistering mbean [%s]".format(mbean))
case e: Exception app.eventHandler.error(e, this, "Error while unregistering mbean [%s]".format(mbean))
}
}

View file

@ -106,11 +106,8 @@ class Switch(startAsOn: Boolean = false) {
if (switch.compareAndSet(from, !from)) {
try {
action
} catch {
case e: Throwable
EventHandler.error(e, this, e.getMessage)
switch.compareAndSet(!from, from) // revert status
throw e
} finally {
switch.compareAndSet(!from, from) // revert status
}
true
} else false

View file

@ -34,11 +34,7 @@ object ReflectiveAccess {
ctor.setAccessible(true)
Right(ctor.newInstance(args: _*).asInstanceOf[T])
} catch {
case e: java.lang.reflect.InvocationTargetException
EventHandler.debug(this, e.getCause.toString)
Left(e)
case e: Exception
EventHandler.debug(this, e.toString)
Left(e)
}
@ -154,7 +150,7 @@ class ReflectiveAccess(val application: AkkaApplication) {
if (!isEnabled) {
val e = new ModuleNotAvailableException(
"Can't load the cluster module, make sure it is enabled in the config ('akka.enabled-modules = [\"cluster\"])' and that akka-cluster.jar is on the classpath")
EventHandler.debug(this, e.toString)
application.eventHandler.debug(this, e.toString)
throw e
}
}
@ -162,21 +158,21 @@ class ReflectiveAccess(val application: AkkaApplication) {
lazy val clusterInstance: Option[Cluster] = getObjectFor("akka.cluster.Cluster$") match {
case Right(value) Some(value)
case Left(exception)
EventHandler.debug(this, exception.toString)
application.eventHandler.debug(this, exception.toString)
None
}
lazy val clusterDeployerInstance: Option[ActorDeployer] = getObjectFor("akka.cluster.ClusterDeployer$") match {
case Right(value) Some(value)
case Left(exception)
EventHandler.debug(this, exception.toString)
application.eventHandler.debug(this, exception.toString)
None
}
lazy val transactionLogInstance: Option[TransactionLogObject] = getObjectFor("akka.cluster.TransactionLog$") match {
case Right(value) Some(value)
case Left(exception)
EventHandler.debug(this, exception.toString)
application.eventHandler.debug(this, exception.toString)
None
}
@ -249,7 +245,7 @@ class ReflectiveAccess(val application: AkkaApplication) {
if (!isEnabled) {
val e = new ModuleNotAvailableException(
"Can't load the remote module, make sure it is enabled in the config ('akka.enabled-modules = [\"remote\"])' and that akka-remote.jar is on the classpath")
EventHandler.debug(this, e.toString)
application.eventHandler.debug(this, e.toString)
throw e
}
}
@ -257,7 +253,7 @@ class ReflectiveAccess(val application: AkkaApplication) {
lazy val remoteInstance: Option[RemoteService] = getObjectFor("akka.remote.Remote$") match {
case Right(value) Some(value)
case Left(exception)
EventHandler.debug(this, exception.toString)
application.eventHandler.debug(this, exception.toString)
None
}
@ -269,7 +265,7 @@ class ReflectiveAccess(val application: AkkaApplication) {
val remoteSupportClass = getClassFor[RemoteSupport](TRANSPORT) match {
case Right(value) Some(value)
case Left(exception)
EventHandler.debug(this, exception.toString)
application.eventHandler.debug(this, exception.toString)
None
}
@ -283,7 +279,7 @@ class ReflectiveAccess(val application: AkkaApplication) {
case Left(exception)
val e = new ModuleNotAvailableException(
"Can't instantiate [%s] - make sure that akka-remote.jar is on the classpath".format(remoteClass.getName), exception)
EventHandler.debug(this, e.toString)
application.eventHandler.debug(this, e.toString)
throw e
}
}

View file

@ -16,8 +16,8 @@ import akka.AkkaApplication
*/
trait JettyContinuation extends ContinuationListener {
import javax.servlet.http.HttpServletResponse
protected def application: AkkaApplication
def app: AkkaApplication
val builder: () tAsyncRequestContext
val context: Option[tAsyncRequestContext] = Some(builder())
@ -35,7 +35,7 @@ trait JettyContinuation extends ContinuationListener {
// the fresh continuation (coming through getAsyncContinuation)
//
case (true, false, false) {
continuation.setTimeout(application.MistSettings.DefaultTimeout)
continuation.setTimeout(app.MistSettings.DefaultTimeout)
continuation.addContinuationListener(this)
continuation.suspend
@ -47,7 +47,7 @@ trait JettyContinuation extends ContinuationListener {
//
case (true, true, false) {
continuation.setTimeout(application.MistSettings.DefaultTimeout)
continuation.setTimeout(app.MistSettings.DefaultTimeout)
continuation.addContinuationListener(this)
Some(continuation)
@ -58,9 +58,9 @@ trait JettyContinuation extends ContinuationListener {
//
case (false, false, false) {
continuation.setTimeout(continuation.getAttribute(application.MistSettings.TimeoutAttribute).asInstanceOf[Long])
continuation.setTimeout(continuation.getAttribute(app.MistSettings.TimeoutAttribute).asInstanceOf[Long])
continuation.suspend
continuation.removeAttribute(application.MistSettings.TimeoutAttribute)
continuation.removeAttribute(app.MistSettings.TimeoutAttribute)
None
}
@ -70,8 +70,8 @@ trait JettyContinuation extends ContinuationListener {
//
case (false, true, false) {
continuation.setTimeout(continuation.getAttribute(application.MistSettings.TimeoutAttribute).asInstanceOf[Long])
continuation.removeAttribute(application.MistSettings.TimeoutAttribute)
continuation.setTimeout(continuation.getAttribute(app.MistSettings.TimeoutAttribute).asInstanceOf[Long])
continuation.removeAttribute(app.MistSettings.TimeoutAttribute)
None
}
@ -87,13 +87,13 @@ trait JettyContinuation extends ContinuationListener {
def suspended: Boolean = _continuation match {
case None false
case Some(continuation) (continuation.isSuspended || (continuation.getAttribute(application.MistSettings.TimeoutAttribute) ne null))
case Some(continuation) (continuation.isSuspended || (continuation.getAttribute(app.MistSettings.TimeoutAttribute) ne null))
}
def timeout(ms: Long): Boolean = _continuation match {
case None false
case Some(continuation)
continuation.setAttribute(application.MistSettings.TimeoutAttribute, ms)
continuation.setAttribute(app.MistSettings.TimeoutAttribute, ms)
continuation.resume
true
}
@ -103,21 +103,19 @@ trait JettyContinuation extends ContinuationListener {
//
def onComplete(c: Continuation) = {}
def onTimeout(c: Continuation) = {
c.getServletResponse.asInstanceOf[HttpServletResponse].addHeader(application.MistSettings.ExpiredHeaderName, application.MistSettings.ExpiredHeaderValue)
c.getServletResponse.asInstanceOf[HttpServletResponse].addHeader(app.MistSettings.ExpiredHeaderName, app.MistSettings.ExpiredHeaderValue)
c.complete
}
}
class JettyContinuationMethodFactory(val _application: AkkaApplication) extends RequestMethodFactory {
trait App {
def application = _application
}
def Delete(f: () tAsyncRequestContext): RequestMethod = new Delete(f) with JettyContinuation with App
def Get(f: () tAsyncRequestContext): RequestMethod = new Get(f) with JettyContinuation with App
def Head(f: () tAsyncRequestContext): RequestMethod = new Head(f) with JettyContinuation with App
def Options(f: () tAsyncRequestContext): RequestMethod = new Options(f) with JettyContinuation with App
def Post(f: () tAsyncRequestContext): RequestMethod = new Post(f) with JettyContinuation with App
def Put(f: () tAsyncRequestContext): RequestMethod = new Put(f) with JettyContinuation with App
def Trace(f: () tAsyncRequestContext): RequestMethod = new Trace(f) with JettyContinuation with App
class JettyContinuationMethodFactory(_app: AkkaApplication) extends RequestMethodFactory {
implicit val app = _app
def Delete(f: () tAsyncRequestContext): RequestMethod = new Delete(f) with JettyContinuation
def Get(f: () tAsyncRequestContext): RequestMethod = new Get(f) with JettyContinuation
def Head(f: () tAsyncRequestContext): RequestMethod = new Head(f) with JettyContinuation
def Options(f: () tAsyncRequestContext): RequestMethod = new Options(f) with JettyContinuation
def Post(f: () tAsyncRequestContext): RequestMethod = new Post(f) with JettyContinuation
def Put(f: () tAsyncRequestContext): RequestMethod = new Put(f) with JettyContinuation
def Trace(f: () tAsyncRequestContext): RequestMethod = new Trace(f) with JettyContinuation
}

View file

@ -53,7 +53,7 @@ object Types {
trait Mist {
import javax.servlet.ServletContext
protected def application: AkkaApplication
protected def app: AkkaApplication
/**
* The root endpoint actor
@ -99,7 +99,7 @@ trait Mist {
// shoot the message to the root endpoint for processing
// IMPORTANT: the suspend method is invoked on the server thread not in the actor
val method = builder(() suspend(application.MistSettings.ConnectionClose))
val method = builder(() suspend(app.MistSettings.ConnectionClose))
if (method.go) root ! method
}
@ -111,9 +111,9 @@ trait Mist {
val server = context.getServerInfo
val (major, minor) = (context.getMajorVersion, context.getMinorVersion)
factory = if (major >= 3) {
Some(new Servlet30ContextMethodFactory(application))
} else if (server.toLowerCase startsWith application.MistSettings.JettyServer) {
Some(new JettyContinuationMethodFactory(application))
Some(new Servlet30ContextMethodFactory(app))
} else if (server.toLowerCase startsWith app.MistSettings.JettyServer) {
Some(new JettyContinuationMethodFactory(app))
} else {
None
}
@ -123,14 +123,14 @@ trait Mist {
trait RootEndpointLocator {
var root: ActorRef = null
protected def application: AkkaApplication
protected def app: AkkaApplication
def configureRoot(address: String) {
def findRoot(address: String): ActorRef =
application.registry.actorFor(address).getOrElse(
app.registry.actorFor(address).getOrElse(
throw new ConfigurationException("akka.http.root-actor-id configuration option does not have a valid actor address [" + address + "]"))
root = if ((address eq null) || address == "") findRoot(application.MistSettings.RootActorID) else findRoot(address)
root = if ((address eq null) || address == "") findRoot(app.MistSettings.RootActorID) else findRoot(address)
}
}
@ -138,7 +138,7 @@ trait RootEndpointLocator {
* AkkaMistServlet adds support to bridge Http and Actors in an asynchronous fashion
* Async impls currently supported: Servlet3.0, Jetty Continuations
*/
class AkkaMistServlet(val application: AkkaApplication) extends HttpServlet with Mist with RootEndpointLocator {
class AkkaMistServlet(val app: AkkaApplication) extends HttpServlet with Mist with RootEndpointLocator {
import javax.servlet.{ ServletConfig }
/**
@ -157,7 +157,7 @@ class AkkaMistServlet(val application: AkkaApplication) extends HttpServlet with
* Proof-of-concept, use at own risk
* Will be officially supported in a later release
*/
class AkkaMistFilter(val application: AkkaApplication) extends Filter with Mist with RootEndpointLocator {
class AkkaMistFilter(val app: AkkaApplication) extends Filter with Mist with RootEndpointLocator {
import javax.servlet.{ ServletRequest, ServletResponse, FilterConfig, FilterChain }
/**
@ -294,6 +294,8 @@ class RootEndpoint extends Actor with Endpoint {
trait RequestMethod {
import java.io.IOException
import javax.servlet.http.{ HttpServletResponse, HttpServletRequest }
def app: AkkaApplication
// required implementations
val builder: () tAsyncRequestContext
@ -358,7 +360,7 @@ trait RequestMethod {
}
} catch {
case io: Exception
EventHandler.error(io, this, io.getMessage)
app.eventHandler.error(io, this, io.getMessage)
false
}
case None false
@ -374,7 +376,7 @@ trait RequestMethod {
}
} catch {
case io: IOException
EventHandler.error(io, this, io.getMessage)
app.eventHandler.error(io, this, io.getMessage)
}
case None {}
}
@ -401,13 +403,13 @@ trait RequestMethod {
def Unavailable(body: String, retry: Int): Boolean = complete(HttpServletResponse.SC_SERVICE_UNAVAILABLE, body, List(("Retry-After", retry.toString)))
}
abstract class Delete(val builder: () tAsyncRequestContext) extends RequestMethod
abstract class Get(val builder: () tAsyncRequestContext) extends RequestMethod
abstract class Head(val builder: () tAsyncRequestContext) extends RequestMethod
abstract class Options(val builder: () tAsyncRequestContext) extends RequestMethod
abstract class Post(val builder: () tAsyncRequestContext) extends RequestMethod
abstract class Put(val builder: () tAsyncRequestContext) extends RequestMethod
abstract class Trace(val builder: () tAsyncRequestContext) extends RequestMethod
abstract class Delete(val builder: () tAsyncRequestContext)(implicit val app: AkkaApplication) extends RequestMethod
abstract class Get(val builder: () tAsyncRequestContext)(implicit val app: AkkaApplication) extends RequestMethod
abstract class Head(val builder: () tAsyncRequestContext)(implicit val app: AkkaApplication) extends RequestMethod
abstract class Options(val builder: () tAsyncRequestContext)(implicit val app: AkkaApplication) extends RequestMethod
abstract class Post(val builder: () tAsyncRequestContext)(implicit val app: AkkaApplication) extends RequestMethod
abstract class Put(val builder: () tAsyncRequestContext)(implicit val app: AkkaApplication) extends RequestMethod
abstract class Trace(val builder: () tAsyncRequestContext)(implicit val app: AkkaApplication) extends RequestMethod
trait RequestMethodFactory {
def Delete(f: () tAsyncRequestContext): RequestMethod

View file

@ -6,7 +6,6 @@ package akka.http
import javax.servlet.{ AsyncContext, AsyncListener, AsyncEvent }
import Types._
import akka.event.EventHandler
import akka.AkkaApplication
/**
@ -15,7 +14,7 @@ import akka.AkkaApplication
trait Servlet30Context extends AsyncListener {
import javax.servlet.http.HttpServletResponse
protected def application: AkkaApplication
def app: AkkaApplication
val builder: () tAsyncRequestContext
val context: Option[tAsyncRequestContext] = Some(builder())
@ -23,7 +22,7 @@ trait Servlet30Context extends AsyncListener {
protected val _ac: AsyncContext = {
val ac = context.get.asInstanceOf[AsyncContext]
ac setTimeout application.MistSettings.DefaultTimeout
ac setTimeout app.MistSettings.DefaultTimeout
ac addListener this
ac
}
@ -36,7 +35,7 @@ trait Servlet30Context extends AsyncListener {
true
} catch {
case e: IllegalStateException
EventHandler.error(e, this, e.getMessage)
app.eventHandler.error(e, this, e.getMessage)
false
}
}
@ -47,25 +46,23 @@ trait Servlet30Context extends AsyncListener {
def onComplete(e: AsyncEvent) {}
def onError(e: AsyncEvent) = e.getThrowable match {
case null
case t EventHandler.error(t, this, t.getMessage)
case t app.eventHandler.error(t, this, t.getMessage)
}
def onStartAsync(e: AsyncEvent) {}
def onTimeout(e: AsyncEvent) = {
e.getSuppliedResponse.asInstanceOf[HttpServletResponse].addHeader(application.MistSettings.ExpiredHeaderName, application.MistSettings.ExpiredHeaderValue)
e.getSuppliedResponse.asInstanceOf[HttpServletResponse].addHeader(app.MistSettings.ExpiredHeaderName, app.MistSettings.ExpiredHeaderValue)
e.getAsyncContext.complete
}
}
class Servlet30ContextMethodFactory(val _application: AkkaApplication) extends RequestMethodFactory {
trait App {
def application = _application
}
def Delete(f: () tAsyncRequestContext): RequestMethod = new Delete(f) with Servlet30Context with App
def Get(f: () tAsyncRequestContext): RequestMethod = new Get(f) with Servlet30Context with App
def Head(f: () tAsyncRequestContext): RequestMethod = new Head(f) with Servlet30Context with App
def Options(f: () tAsyncRequestContext): RequestMethod = new Options(f) with Servlet30Context with App
def Post(f: () tAsyncRequestContext): RequestMethod = new Post(f) with Servlet30Context with App
def Put(f: () tAsyncRequestContext): RequestMethod = new Put(f) with Servlet30Context with App
def Trace(f: () tAsyncRequestContext): RequestMethod = new Trace(f) with Servlet30Context with App
class Servlet30ContextMethodFactory(_app: AkkaApplication) extends RequestMethodFactory {
implicit val app = _app
def Delete(f: () tAsyncRequestContext): RequestMethod = new Delete(f) with Servlet30Context
def Get(f: () tAsyncRequestContext): RequestMethod = new Get(f) with Servlet30Context
def Head(f: () tAsyncRequestContext): RequestMethod = new Head(f) with Servlet30Context
def Options(f: () tAsyncRequestContext): RequestMethod = new Options(f) with Servlet30Context
def Post(f: () tAsyncRequestContext): RequestMethod = new Post(f) with Servlet30Context
def Put(f: () tAsyncRequestContext): RequestMethod = new Put(f) with Servlet30Context
def Trace(f: () tAsyncRequestContext): RequestMethod = new Trace(f) with Servlet30Context
}

View file

@ -26,19 +26,19 @@ trait BootableRemoteActorService extends Bootable {
abstract override def onLoad() {
if (app.reflective.ClusterModule.isEnabled && settings.isRemotingEnabled) {
EventHandler.info(this, "Initializing Remote Actors Service...")
app.eventHandler.info(this, "Initializing Remote Actors Service...")
startRemoteService()
EventHandler.info(this, "Remote Actors Service initialized")
app.eventHandler.info(this, "Remote Actors Service initialized")
}
super.onLoad()
}
abstract override def onUnload() {
EventHandler.info(this, "Shutting down Remote Actors Service")
app.eventHandler.info(this, "Shutting down Remote Actors Service")
app.remote.shutdown()
if (remoteServerThread.isAlive) remoteServerThread.join(1000)
EventHandler.info(this, "Remote Actors Service has been shut down")
app.eventHandler.info(this, "Remote Actors Service has been shut down")
super.onUnload()
}
}

View file

@ -35,7 +35,7 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
private val actors = new ConcurrentHashMap[String, Promise[Option[ActorRef]]]
private val failureDetector = new BannagePeriodFailureDetector(remote, timeToBan = 60 seconds) // FIXME make timeToBan configurable
private val failureDetector = new BannagePeriodFailureDetector(app, remote, timeToBan = 60 seconds) // FIXME make timeToBan configurable
def actorOf(props: Props, address: String): Option[ActorRef] = {
Address.validate(address)
@ -129,7 +129,7 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
* Using (checking out) actor on a specific node.
*/
def useActorOnNode(remoteAddress: InetSocketAddress, actorAddress: String, actorFactory: () Actor) {
EventHandler.debug(this, "Instantiating Actor [%s] on node [%s]".format(actorAddress, remoteAddress))
app.eventHandler.debug(this, "Instantiating Actor [%s] on node [%s]".format(actorAddress, remoteAddress))
val actorFactoryBytes =
app.serialization.serialize(actorFactory) match {
@ -164,20 +164,20 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
try {
(connection ? (command, remote.remoteDaemonAckTimeout)).as[Status] match {
case Some(Success(receiver))
EventHandler.debug(this, "Remote command sent to [%s] successfully received".format(receiver))
app.eventHandler.debug(this, "Remote command sent to [%s] successfully received".format(receiver))
case Some(Failure(cause))
EventHandler.error(cause, this, cause.toString)
app.eventHandler.error(cause, this, cause.toString)
throw cause
case None
val error = new RemoteException("Remote command to [%s] timed out".format(connection.address))
EventHandler.error(error, this, error.toString)
app.eventHandler.error(error, this, error.toString)
throw error
}
} catch {
case e: Exception
EventHandler.error(e, this, "Could not send remote command to [%s] due to: %s".format(connection.address, e.toString))
app.eventHandler.error(e, this, "Could not send remote command to [%s] due to: %s".format(connection.address, e.toString))
throw e
}
} else {

View file

@ -76,7 +76,7 @@ class Remote(val app: AkkaApplication) extends RemoteService {
def start() {
val triggerLazyServerVal = address.toString
EventHandler.info(this, "Starting remote server on [%s]".format(triggerLazyServerVal))
app.eventHandler.info(this, "Starting remote server on [%s]".format(triggerLazyServerVal))
}
def uuidProtocolToUuid(uuid: UuidProtocol): UUID = new UUID(uuid.getHigh, uuid.getLow)
@ -100,12 +100,12 @@ class RemoteDaemon(val remote: Remote) extends Actor {
import remote._
override def preRestart(reason: Throwable, msg: Option[Any]) {
EventHandler.debug(this, "RemoteDaemon failed due to [%s] restarting...".format(reason))
app.eventHandler.debug(this, "RemoteDaemon failed due to [%s] restarting...".format(reason))
}
def receive: Actor.Receive = {
case message: RemoteDaemonMessageProtocol
EventHandler.debug(this,
app.eventHandler.debug(this,
"Received command [\n%s] to RemoteDaemon on [%s]".format(message, app.nodename))
message.getMessageType match {
@ -123,7 +123,7 @@ class RemoteDaemon(val remote: Remote) extends Actor {
//TODO: should we not deal with unrecognized message types?
}
case unknown EventHandler.warning(this, "Unknown message [%s]".format(unknown))
case unknown app.eventHandler.warning(this, "Unknown message [%s]".format(unknown))
}
def handleUse(message: RemoteProtocol.RemoteDaemonMessageProtocol) {
@ -146,7 +146,7 @@ class RemoteDaemon(val remote: Remote) extends Actor {
remote.server.register(actorAddress, newActorRef)
} else {
EventHandler.error(this, "Actor 'address' is not defined, ignoring remote daemon command [%s]".format(message))
app.eventHandler.error(this, "Actor 'address' is not defined, ignoring remote daemon command [%s]".format(message))
}
reply(Success(address.toString))

View file

@ -16,13 +16,14 @@ import scala.annotation.tailrec
import java.net.InetSocketAddress
import java.util.concurrent.atomic.AtomicReference
import System.{ currentTimeMillis newTimestamp }
import akka.AkkaApplication
/**
* Base class for remote failure detection management.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
abstract class RemoteFailureDetectorBase(remote: Remote, initialConnections: Map[InetSocketAddress, ActorRef])
abstract class RemoteFailureDetectorBase(app: AkkaApplication, remote: Remote, initialConnections: Map[InetSocketAddress, ActorRef])
extends FailureDetector
with NetworkEventStream.Listener {
@ -81,7 +82,7 @@ abstract class RemoteFailureDetectorBase(remote: Remote, initialConnections: Map
@tailrec
final def failOver(from: InetSocketAddress, to: InetSocketAddress) {
EventHandler.debug(this, "RemoteFailureDetector failover from [%s] to [%s]".format(from, to))
app.eventHandler.debug(this, "RemoteFailureDetector failover from [%s] to [%s]".format(from, to))
val oldState = state.get
var changed = false
@ -132,7 +133,7 @@ abstract class RemoteFailureDetectorBase(remote: Remote, initialConnections: Map
if (!state.compareAndSet(oldState, newState)) {
remove(faultyConnection) // recur
} else {
EventHandler.debug(this, "Removing connection [%s]".format(faultyAddress))
app.eventHandler.debug(this, "Removing connection [%s]".format(faultyAddress))
remote.eventStream.unregister(this, faultyAddress) // unregister the connections - e.g stop listen to events from it
}
}
@ -160,7 +161,7 @@ abstract class RemoteFailureDetectorBase(remote: Remote, initialConnections: Map
putIfAbsent(address, newConnectionFactory) // recur
} else {
// we succeeded
EventHandler.debug(this, "Adding connection [%s]".format(address))
app.eventHandler.debug(this, "Adding connection [%s]".format(address))
remote.eventStream.register(this, address) // register the connection - e.g listen to events from it
newConnection // return new connection actor
}
@ -175,9 +176,9 @@ abstract class RemoteFailureDetectorBase(remote: Remote, initialConnections: Map
/**
* Simple failure detector that removes the failing connection permanently on first error.
*/
class RemoveConnectionOnFirstFailureRemoteFailureDetector(_remote: Remote,
class RemoveConnectionOnFirstFailureRemoteFailureDetector(_app: AkkaApplication, _remote: Remote,
initialConnections: Map[InetSocketAddress, ActorRef] = Map.empty[InetSocketAddress, ActorRef])
extends RemoteFailureDetectorBase(_remote, initialConnections) {
extends RemoteFailureDetectorBase(_app, _remote, initialConnections) {
protected def newState() = State(Long.MinValue, initialConnections)
@ -213,10 +214,10 @@ class RemoveConnectionOnFirstFailureRemoteFailureDetector(_remote: Remote,
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class BannagePeriodFailureDetector(_remote: Remote,
class BannagePeriodFailureDetector(_app: AkkaApplication, _remote: Remote,
initialConnections: Map[InetSocketAddress, ActorRef] = Map.empty[InetSocketAddress, ActorRef],
timeToBan: Duration)
extends RemoteFailureDetectorBase(_remote, initialConnections) {
extends RemoteFailureDetectorBase(_app, _remote, initialConnections) {
// FIXME considering adding a Scheduler event to notify the BannagePeriodFailureDetector unban the banned connection after the timeToBan have exprired

View file

@ -9,7 +9,6 @@ import akka.dispatch.{ ActorPromise, DefaultPromise, Promise }
import akka.remote._
import RemoteProtocol._
import akka.util._
import akka.event.EventHandler
import org.jboss.netty.channel._
import org.jboss.netty.channel.group.{ DefaultChannelGroup, ChannelGroup, ChannelGroupFuture }
import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory
@ -209,7 +208,7 @@ abstract class RemoteClient private[akka] (
senderFuture: Option[Promise[T]]): Option[Promise[T]] = {
if (isRunning) {
EventHandler.debug(this, "Sending to connection [%s] message [\n%s]".format(remoteAddress, request))
app.eventHandler.debug(this, "Sending to connection [%s] message [\n%s]".format(remoteAddress, request))
// tell
if (request.getOneWay) {
@ -279,7 +278,7 @@ abstract class RemoteClient private[akka] (
private[remote] def sendPendingRequests() = pendingRequests synchronized {
// ensure only one thread at a time can flush the log
val nrOfMessages = pendingRequests.size
if (nrOfMessages > 0) EventHandler.info(this, "Resending [%s] previously failed messages after remote client reconnect" format nrOfMessages)
if (nrOfMessages > 0) app.eventHandler.info(this, "Resending [%s] previously failed messages after remote client reconnect" format nrOfMessages)
var pendingRequest = pendingRequests.peek
while (pendingRequest ne null) {
@ -362,14 +361,14 @@ class ActiveRemoteClient private[akka] (
}
def attemptReconnect(): Boolean = {
EventHandler.debug(this, "Remote client reconnecting to [%s]".format(remoteAddress))
app.eventHandler.debug(this, "Remote client reconnecting to [%s]".format(remoteAddress))
val connection = bootstrap.connect(remoteAddress)
openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails.
if (!connection.isSuccess) {
notifyListeners(RemoteClientError(connection.getCause, module, remoteAddress))
EventHandler.error(connection.getCause, this, "Reconnection to [%s] has failed".format(remoteAddress))
app.eventHandler.error(connection.getCause, this, "Reconnection to [%s] has failed".format(remoteAddress))
false
} else {
@ -387,7 +386,7 @@ class ActiveRemoteClient private[akka] (
bootstrap.setOption("tcpNoDelay", true)
bootstrap.setOption("keepAlive", true)
EventHandler.debug(this, "Starting remote client connection to [%s]".format(remoteAddress))
app.eventHandler.debug(this, "Starting remote client connection to [%s]".format(remoteAddress))
connection = bootstrap.connect(remoteAddress)
@ -396,7 +395,7 @@ class ActiveRemoteClient private[akka] (
if (!connection.isSuccess) {
notifyListeners(RemoteClientError(connection.getCause, module, remoteAddress))
EventHandler.error(connection.getCause, this, "Remote client connection to [%s] has failed".format(remoteAddress))
app.eventHandler.error(connection.getCause, this, "Remote client connection to [%s] has failed".format(remoteAddress))
false
} else {
@ -423,7 +422,7 @@ class ActiveRemoteClient private[akka] (
case false if reconnectIfAlreadyConnected
closeChannel(connection)
EventHandler.debug(this, "Remote client reconnecting to [%s]".format(remoteAddress))
app.eventHandler.debug(this, "Remote client reconnecting to [%s]".format(remoteAddress))
attemptReconnect()
case false false
@ -432,7 +431,7 @@ class ActiveRemoteClient private[akka] (
// Please note that this method does _not_ remove the ARC from the NettyRemoteClientModule's map of clients
def shutdown() = runSwitch switchOff {
EventHandler.info(this, "Shutting down remote client [%s]".format(name))
app.eventHandler.info(this, "Shutting down remote client [%s]".format(name))
notifyListeners(RemoteClientShutdown(module, remoteAddress))
timer.stop()
@ -444,7 +443,7 @@ class ActiveRemoteClient private[akka] (
connection = null
pendingRequests.clear()
EventHandler.info(this, "[%s] has been shut down".format(name))
app.eventHandler.info(this, "[%s] has been shut down".format(name))
}
private[akka] def isWithinReconnectionTimeWindow: Boolean = {
@ -454,7 +453,7 @@ class ActiveRemoteClient private[akka] (
} else {
val timeLeft = (RECONNECTION_TIME_WINDOW - (System.currentTimeMillis - reconnectionTimeWindowStart)) > 0
if (timeLeft) {
EventHandler.info(this, "Will try to reconnect to remote server for another [%s] milliseconds".format(timeLeft))
app.eventHandler.info(this, "Will try to reconnect to remote server for another [%s] milliseconds".format(timeLeft))
}
timeLeft
}
@ -521,7 +520,7 @@ class ActiveRemoteClientHandler(
case arp: AkkaRemoteProtocol if arp.hasMessage
val reply = arp.getMessage
val replyUuid = uuidFrom(reply.getActorInfo.getUuid.getHigh, reply.getActorInfo.getUuid.getLow)
EventHandler.debug(this, "Remote client received RemoteMessageProtocol[\n%s]\nTrying to map back to future [%s]".format(reply, replyUuid))
app.eventHandler.debug(this, "Remote client received RemoteMessageProtocol[\n%s]\nTrying to map back to future [%s]".format(reply, replyUuid))
futures.remove(replyUuid).asInstanceOf[Promise[Any]] match {
case null
@ -543,7 +542,7 @@ class ActiveRemoteClientHandler(
}
} catch {
case e: Exception
EventHandler.error(e, this, e.getMessage)
app.eventHandler.error(e, this, e.getMessage)
client.notifyListeners(RemoteClientError(e, client.module, client.remoteAddress))
}
}
@ -567,24 +566,24 @@ class ActiveRemoteClientHandler(
try {
if (client.useTransactionLog) client.sendPendingRequests() // try to send pending requests (still there after client/server crash ard reconnect
client.notifyListeners(RemoteClientConnected(client.module, client.remoteAddress))
EventHandler.debug(this, "Remote client connected to [%s]".format(ctx.getChannel.getRemoteAddress))
app.eventHandler.debug(this, "Remote client connected to [%s]".format(ctx.getChannel.getRemoteAddress))
client.resetReconnectionTimeWindow
} catch {
case e: Exception
EventHandler.error(e, this, e.getMessage)
app.eventHandler.error(e, this, e.getMessage)
client.notifyListeners(RemoteClientError(e, client.module, client.remoteAddress))
}
}
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
client.notifyListeners(RemoteClientDisconnected(client.module, client.remoteAddress))
EventHandler.debug(this, "Remote client disconnected from [%s]".format(ctx.getChannel.getRemoteAddress))
app.eventHandler.debug(this, "Remote client disconnected from [%s]".format(ctx.getChannel.getRemoteAddress))
}
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
val cause = event.getCause
if (cause ne null) {
EventHandler.error(event.getCause, this, "Unexpected exception [%s] from downstream in remote client [%s]".format(event.getCause, event))
app.eventHandler.error(event.getCause, this, "Unexpected exception [%s] from downstream in remote client [%s]".format(event.getCause, event))
cause match {
case e: ReadTimeoutException
@ -596,7 +595,7 @@ class ActiveRemoteClientHandler(
event.getChannel.close //FIXME Is this the correct behavior?
}
} else EventHandler.error(this, "Unexpected exception from downstream in remote client [%s]".format(event))
} else app.eventHandler.error(this, "Unexpected exception from downstream in remote client [%s]".format(event))
}
private def parseException(reply: RemoteMessageProtocol, loader: Option[ClassLoader]): Throwable = {
@ -611,7 +610,7 @@ class ActiveRemoteClientHandler(
.newInstance(exception.getMessage).asInstanceOf[Throwable]
} catch {
case problem: Exception
EventHandler.error(problem, this, problem.getMessage)
app.eventHandler.error(problem, this, problem.getMessage)
CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorException(problem, classname, exception.getMessage)
}
}
@ -646,7 +645,7 @@ class NettyRemoteSupport(_app: AkkaApplication) extends RemoteSupport(_app) with
}
val remoteInetSocketAddress = new InetSocketAddress(host, port)
EventHandler.debug(this,
app.eventHandler.debug(this,
"Creating RemoteActorRef with address [%s] connected to [%s]"
.format(actorAddress, remoteInetSocketAddress))
RemoteActorRef(app, app.remote, remoteInetSocketAddress, actorAddress, loader)
@ -689,7 +688,7 @@ class NettyRemoteServer(app: AkkaApplication, serverModule: NettyRemoteServerMod
serverModule.notifyListeners(RemoteServerStarted(serverModule))
def shutdown() {
EventHandler.info(this, "Shutting down remote server [%s]".format(name))
app.eventHandler.info(this, "Shutting down remote server [%s]".format(name))
try {
val shutdownSignal = {
val b = RemoteControlProtocol.newBuilder.setCommandType(CommandType.SHUTDOWN)
@ -705,7 +704,7 @@ class NettyRemoteServer(app: AkkaApplication, serverModule: NettyRemoteServerMod
serverModule.notifyListeners(RemoteServerShutdown(serverModule))
} catch {
case e: Exception
EventHandler.error(e, this, e.getMessage)
app.eventHandler.error(e, this, e.getMessage)
}
}
}
@ -736,13 +735,13 @@ trait NettyRemoteServerModule extends RemoteServerModule {
def start(_hostname: String, _port: Int, loader: Option[ClassLoader] = None): RemoteServerModule = guard withGuard {
try {
_isRunning switchOn {
EventHandler.debug(this, "Starting up remote server on %s:s".format(_hostname, _port))
app.eventHandler.debug(this, "Starting up remote server on %s:s".format(_hostname, _port))
currentServer.set(Some(new NettyRemoteServer(app, this, _hostname, _port, loader)))
}
} catch {
case e: Exception
EventHandler.error(e, this, e.getMessage)
app.eventHandler.error(e, this, e.getMessage)
notifyListeners(RemoteServerError(e, this))
}
this
@ -751,7 +750,7 @@ trait NettyRemoteServerModule extends RemoteServerModule {
def shutdownServerModule() = guard withGuard {
_isRunning switchOff {
currentServer.getAndSet(None) foreach { instance
EventHandler.debug(this, "Shutting down remote server on %s:%s".format(instance.host, instance.port))
app.eventHandler.debug(this, "Shutting down remote server on %s:%s".format(instance.host, instance.port))
instance.shutdown()
}
}
@ -796,7 +795,7 @@ trait NettyRemoteServerModule extends RemoteServerModule {
def unregister(actorRef: ActorRef): Unit = guard withGuard {
if (_isRunning.isOn) {
EventHandler.debug(this, "Unregister server side remote actor with id [%s]".format(actorRef.uuid))
app.eventHandler.debug(this, "Unregister server side remote actor with id [%s]".format(actorRef.uuid))
actors.remove(actorRef.address, actorRef)
actorsByUuid.remove(actorRef.uuid.toString, actorRef)
@ -811,7 +810,7 @@ trait NettyRemoteServerModule extends RemoteServerModule {
def unregister(id: String): Unit = guard withGuard {
if (_isRunning.isOn) {
EventHandler.debug(this, "Unregister server side remote actor with id [%s]".format(id))
app.eventHandler.debug(this, "Unregister server side remote actor with id [%s]".format(id))
if (id.startsWith(UUID_PREFIX)) actorsByUuid.remove(id.substring(UUID_PREFIX.length))
else {
@ -830,7 +829,7 @@ trait NettyRemoteServerModule extends RemoteServerModule {
def unregisterPerSession(id: String) {
if (_isRunning.isOn) {
EventHandler.info(this, "Unregistering server side remote actor with id [%s]".format(id))
app.eventHandler.info(this, "Unregistering server side remote actor with id [%s]".format(id))
actorsFactories.remove(id)
}
@ -936,7 +935,7 @@ class RemoteServerHandler(
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
val clientAddress = getClientAddress(ctx)
EventHandler.debug(this, "Remote client [%s] connected to [%s]".format(clientAddress, server.name))
app.eventHandler.debug(this, "Remote client [%s] connected to [%s]".format(clientAddress, server.name))
sessionActors.set(event.getChannel(), new ConcurrentHashMap[String, ActorRef]())
server.notifyListeners(RemoteServerClientConnected(server, clientAddress))
@ -945,7 +944,7 @@ class RemoteServerHandler(
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
val clientAddress = getClientAddress(ctx)
EventHandler.debug(this, "Remote client [%s] disconnected from [%s]".format(clientAddress, server.name))
app.eventHandler.debug(this, "Remote client [%s] disconnected from [%s]".format(clientAddress, server.name))
// stop all session actors
for (
@ -955,7 +954,7 @@ class RemoteServerHandler(
try {
actor ! PoisonPill
} catch {
case e: Exception EventHandler.error(e, this, "Couldn't stop %s".format(actor))
case e: Exception app.eventHandler.error(e, this, "Couldn't stop %s".format(actor))
}
}
@ -964,7 +963,7 @@ class RemoteServerHandler(
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
val clientAddress = getClientAddress(ctx)
EventHandler.debug("Remote client [%s] channel closed from [%s]".format(clientAddress, server.name), this)
app.eventHandler.debug("Remote client [%s] channel closed from [%s]".format(clientAddress, server.name), this)
server.notifyListeners(RemoteServerClientClosed(server, clientAddress))
}
@ -984,7 +983,7 @@ class RemoteServerHandler(
}
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
EventHandler.error(event.getCause, this, "Unexpected exception from remote downstream")
app.eventHandler.error(event.getCause, this, "Unexpected exception from remote downstream")
event.getChannel.close
server.notifyListeners(RemoteServerError(event.getCause, server))
@ -997,25 +996,25 @@ class RemoteServerHandler(
}
private def handleRemoteMessageProtocol(request: RemoteMessageProtocol, channel: Channel) = try {
EventHandler.debug(this, "Received remote message [%s]".format(request))
app.eventHandler.debug(this, "Received remote message [%s]".format(request))
dispatchToActor(request, channel)
} catch {
case e: Exception
server.notifyListeners(RemoteServerError(e, server))
EventHandler.error(e, this, e.getMessage)
app.eventHandler.error(e, this, e.getMessage)
}
private def dispatchToActor(request: RemoteMessageProtocol, channel: Channel) {
val actorInfo = request.getActorInfo
EventHandler.debug(this, "Dispatching to remote actor [%s]".format(actorInfo.getUuid))
app.eventHandler.debug(this, "Dispatching to remote actor [%s]".format(actorInfo.getUuid))
val actorRef =
try {
createActor(actorInfo, channel)
} catch {
case e: SecurityException
EventHandler.error(e, this, e.getMessage)
app.eventHandler.error(e, this, e.getMessage)
write(channel, createErrorReplyMessage(e, request))
server.notifyListeners(RemoteServerError(e, server))
return
@ -1078,7 +1077,7 @@ class RemoteServerHandler(
// else addr
// }
EventHandler.debug(this,
app.eventHandler.debug(this,
"Looking up a remotely available actor for address [%s] on node [%s]"
.format(address, app.nodename))

View file

@ -144,7 +144,7 @@ class ActorSerialization(val app: AkkaApplication) {
overriddenUuid: Option[UUID],
loader: Option[ClassLoader]): ActorRef = {
EventHandler.debug(this, "Deserializing SerializedActorRefProtocol to LocalActorRef:\n%s".format(protocol))
app.eventHandler.debug(this, "Deserializing SerializedActorRefProtocol to LocalActorRef:\n%s".format(protocol))
// import ReplicationStorageType._
// import ReplicationStrategyType._
@ -240,7 +240,7 @@ class RemoteActorSerialization(val app: AkkaApplication) {
* Deserializes a RemoteActorRefProtocol Protocol Buffers (protobuf) Message into an RemoteActorRef instance.
*/
private[akka] def fromProtobufToRemoteActorRef(protocol: RemoteActorRefProtocol, loader: Option[ClassLoader]): ActorRef = {
EventHandler.debug(this, "Deserializing RemoteActorRefProtocol to RemoteActorRef:\n %s".format(protocol))
app.eventHandler.debug(this, "Deserializing RemoteActorRefProtocol to RemoteActorRef:\n %s".format(protocol))
val ref = RemoteActorRef(
app, app.remote,
@ -248,7 +248,7 @@ class RemoteActorSerialization(val app: AkkaApplication) {
protocol.getAddress,
loader)
EventHandler.debug(this, "Newly deserialized RemoteActorRef has uuid: %s".format(ref.uuid))
app.eventHandler.debug(this, "Newly deserialized RemoteActorRef has uuid: %s".format(ref.uuid))
ref
}
@ -267,7 +267,7 @@ class RemoteActorSerialization(val app: AkkaApplication) {
app.reflective.RemoteModule.configDefaultAddress
}
EventHandler.debug(this, "Register serialized Actor [%s] as remote @ [%s]".format(actor.uuid, remoteAddress))
app.eventHandler.debug(this, "Register serialized Actor [%s] as remote @ [%s]".format(actor.uuid, remoteAddress))
RemoteActorRefProtocol.newBuilder
.setInetSocketAddress(ByteString.copyFrom(JavaSerializer.toBinary(remoteAddress)))

View file

@ -153,7 +153,7 @@ class Agent[T](initialValue: T, application: AkkaApplication) {
def sendOff(f: T T): Unit = {
send((value: T) {
suspend()
val pinnedDispatcher = new PinnedDispatcher(null, "agent-send-off", UnboundedMailbox(), application.AkkaConfig.ActorTimeoutMillis)
val pinnedDispatcher = new PinnedDispatcher(application, null, "agent-send-off", UnboundedMailbox(), application.AkkaConfig.ActorTimeoutMillis)
val threadBased = application.createActor(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher))
threadBased ! Update(f)
value
@ -171,7 +171,7 @@ class Agent[T](initialValue: T, application: AkkaApplication) {
val result = new DefaultPromise[T](timeout)(application.dispatcher)
send((value: T) {
suspend()
val pinnedDispatcher = new PinnedDispatcher(null, "agent-alter-off", UnboundedMailbox(), application.AkkaConfig.ActorTimeoutMillis)
val pinnedDispatcher = new PinnedDispatcher(application, null, "agent-alter-off", UnboundedMailbox(), application.AkkaConfig.ActorTimeoutMillis)
val threadBased = application.createActor(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher))
result completeWith threadBased.?(Update(f), timeout).asInstanceOf[Future[T]]
value

View file

@ -77,7 +77,7 @@ public class UntypedCoordinatedIncrementTest {
EventFilter expectedFailureFilter = (EventFilter) new ErrorFilter(ExpectedFailureException.class);
EventFilter coordinatedFilter = (EventFilter) new ErrorFilter(CoordinatedTransactionException.class);
Seq<EventFilter> ignoreExceptions = seq(expectedFailureFilter, coordinatedFilter);
EventHandler.notify(new TestEvent.Mute(ignoreExceptions));
application.eventHandler().notify(new TestEvent.Mute(ignoreExceptions));
CountDownLatch incrementLatch = new CountDownLatch(numCounters);
List<ActorRef> actors = new ArrayList<ActorRef>(counters);
actors.add(failer);
@ -98,7 +98,7 @@ public class UntypedCoordinatedIncrementTest {
}
}
}
EventHandler.notify(new TestEvent.UnMute(ignoreExceptions));
application.eventHandler().notify(new TestEvent.UnMute(ignoreExceptions));
}
public <A> Seq<A> seq(A... args) {

View file

@ -76,7 +76,7 @@ public class UntypedTransactorTest {
EventFilter expectedFailureFilter = (EventFilter) new ErrorFilter(ExpectedFailureException.class);
EventFilter coordinatedFilter = (EventFilter) new ErrorFilter(CoordinatedTransactionException.class);
Seq<EventFilter> ignoreExceptions = seq(expectedFailureFilter, coordinatedFilter);
EventHandler.notify(new TestEvent.Mute(ignoreExceptions));
application.eventHandler().notify(new TestEvent.Mute(ignoreExceptions));
CountDownLatch incrementLatch = new CountDownLatch(numCounters);
List<ActorRef> actors = new ArrayList<ActorRef>(counters);
actors.add(failer);
@ -97,7 +97,7 @@ public class UntypedTransactorTest {
}
}
}
EventHandler.notify(new TestEvent.UnMute(ignoreExceptions));
application.eventHandler().notify(new TestEvent.UnMute(ignoreExceptions));
}
public <A> Seq<A> seq(A... args) {

View file

@ -1,7 +1,5 @@
package akka.transactor.test
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import org.scalatest.BeforeAndAfterAll
import akka.AkkaApplication
@ -55,7 +53,7 @@ object CoordinatedIncrement {
}
}
class CoordinatedIncrementSpec extends WordSpec with MustMatchers with BeforeAndAfterAll {
class CoordinatedIncrementSpec extends AkkaSpec with BeforeAndAfterAll {
import CoordinatedIncrement._
val application = AkkaApplication("CoordinatedIncrementSpec")
@ -88,7 +86,7 @@ class CoordinatedIncrementSpec extends WordSpec with MustMatchers with BeforeAnd
EventFilter[ExpectedFailureException],
EventFilter[CoordinatedTransactionException],
EventFilter[ActorTimeoutException])
EventHandler.notify(TestEvent.Mute(ignoreExceptions))
app.eventHandler.notify(TestEvent.Mute(ignoreExceptions))
val (counters, failer) = createActors
val coordinated = Coordinated()
counters(0) ! Coordinated(Increment(counters.tail :+ failer))
@ -98,7 +96,7 @@ class CoordinatedIncrementSpec extends WordSpec with MustMatchers with BeforeAnd
}
counters foreach (_.stop())
failer.stop()
EventHandler.notify(TestEvent.UnMute(ignoreExceptions))
app.eventHandler.notify(TestEvent.UnMute(ignoreExceptions))
}
}
}

View file

@ -98,7 +98,7 @@ object FickleFriends {
}
}
class FickleFriendsSpec extends WordSpec with MustMatchers with BeforeAndAfterAll {
class FickleFriendsSpec extends AkkaSpec with BeforeAndAfterAll {
import FickleFriends._
val application = AkkaApplication("FickleFriendsSpec")
@ -119,7 +119,7 @@ class FickleFriendsSpec extends WordSpec with MustMatchers with BeforeAndAfterAl
EventFilter[ExpectedFailureException],
EventFilter[CoordinatedTransactionException],
EventFilter[ActorTimeoutException])
EventHandler.notify(TestEvent.Mute(ignoreExceptions))
app.eventHandler.notify(TestEvent.Mute(ignoreExceptions))
val (counters, coordinator) = createActors
val latch = new CountDownLatch(1)
coordinator ! FriendlyIncrement(counters, latch)
@ -130,7 +130,7 @@ class FickleFriendsSpec extends WordSpec with MustMatchers with BeforeAndAfterAl
}
counters foreach (_.stop())
coordinator.stop()
EventHandler.notify(TestEvent.UnMute(ignoreExceptions))
app.eventHandler.notify(TestEvent.UnMute(ignoreExceptions))
}
}
}

View file

@ -75,7 +75,7 @@ object SimpleTransactor {
}
}
class TransactorSpec extends WordSpec with MustMatchers {
class TransactorSpec extends AkkaSpec {
import TransactorIncrement._
import SimpleTransactor._
@ -109,7 +109,7 @@ class TransactorSpec extends WordSpec with MustMatchers {
EventFilter[ExpectedFailureException],
EventFilter[CoordinatedTransactionException],
EventFilter[ActorTimeoutException])
EventHandler.notify(TestEvent.Mute(ignoreExceptions))
app.eventHandler.notify(TestEvent.Mute(ignoreExceptions))
val (counters, failer) = createTransactors
val failLatch = TestLatch(numCounters)
counters(0) ! Increment(counters.tail :+ failer, failLatch)
@ -119,7 +119,7 @@ class TransactorSpec extends WordSpec with MustMatchers {
}
counters foreach (_.stop())
failer.stop()
EventHandler.notify(TestEvent.UnMute(ignoreExceptions))
app.eventHandler.notify(TestEvent.UnMute(ignoreExceptions))
}
}

View file

@ -12,6 +12,7 @@ import java.lang.ref.WeakReference
import scala.annotation.tailrec
import akka.actor.ActorCell
import akka.dispatch._
import akka.AkkaApplication
/*
* Locking rules:
@ -32,8 +33,6 @@ import akka.dispatch._
object CallingThreadDispatcher {
lazy val global = new CallingThreadDispatcher("global-calling-thread")
// PRIVATE DATA
private var queues = Map[CallingThreadMailbox, Set[WeakReference[NestingQueue]]]()
@ -104,7 +103,7 @@ object CallingThreadDispatcher {
* @author Roland Kuhn
* @since 1.1
*/
class CallingThreadDispatcher(val name: String = "calling-thread", val warnings: Boolean = true) extends MessageDispatcher {
class CallingThreadDispatcher(_app: AkkaApplication, val name: String = "calling-thread", val warnings: Boolean = true) extends MessageDispatcher(_app) {
import CallingThreadDispatcher._
protected[akka] override def createMailbox(actor: ActorCell) = new CallingThreadMailbox(this)
@ -157,14 +156,14 @@ class CallingThreadDispatcher(val name: String = "calling-thread", val warnings:
val execute = mbox.suspendSwitch.fold {
queue.push(handle)
if (warnings && handle.channel.isInstanceOf[Promise[_]]) {
EventHandler.warning(this, "suspendSwitch, creating Future could deadlock; target: %s" format handle.receiver)
app.eventHandler.warning(this, "suspendSwitch, creating Future could deadlock; target: %s" format handle.receiver)
}
false
} {
queue.push(handle)
if (queue.isActive) {
if (warnings && handle.channel.isInstanceOf[Promise[_]]) {
EventHandler.warning(this, "blocked on this thread, creating Future could deadlock; target: %s" format handle.receiver)
app.eventHandler.warning(this, "blocked on this thread, creating Future could deadlock; target: %s" format handle.receiver)
}
false
} else {
@ -204,13 +203,13 @@ class CallingThreadDispatcher(val name: String = "calling-thread", val warnings:
handle.invoke
if (warnings) handle.channel match {
case f: ActorPromise if !f.isCompleted
EventHandler.warning(this, "calling %s with message %s did not reply as expected, might deadlock" format (handle.receiver, handle.message))
app.eventHandler.warning(this, "calling %s with message %s did not reply as expected, might deadlock" format (handle.receiver, handle.message))
case _
}
true
} catch {
case e
EventHandler.error(this, e)
app.eventHandler.error(this, e)
queue.leave
false
}

View file

@ -19,8 +19,8 @@ import akka.AkkaApplication
* @author Roland Kuhn
* @since 1.1
*/
class TestActorRef[T <: Actor](application: AkkaApplication, props: Props, address: String)
extends LocalActorRef(application, props.withDispatcher(CallingThreadDispatcher.global), address, false) {
class TestActorRef[T <: Actor](_app: AkkaApplication, props: Props, address: String)
extends LocalActorRef(_app, props.withDispatcher(new CallingThreadDispatcher(_app)), address, false) {
/**
* Directly inject messages into actor receive behavior. Any exceptions
* thrown will be available to you, while still being able to use

View file

@ -99,7 +99,7 @@ class TestKit(_app: AkkaApplication) {
* ActorRef of the test actor. Access is provided to enable e.g.
* registration as message target.
*/
val testActor: ActorRef = new LocalActorRef(app, Props(new TestActor(queue)).copy(dispatcher = CallingThreadDispatcher.global), "testActor" + TestKit.testActorId.incrementAndGet(), true)
val testActor: ActorRef = new LocalActorRef(app, Props(new TestActor(queue)).copy(dispatcher = new CallingThreadDispatcher(app)), "testActor" + TestKit.testActorId.incrementAndGet(), true)
private var end: Duration = Duration.Inf

View file

@ -3,16 +3,16 @@ package akka
import akka.event.EventHandler
package object testkit {
def filterEvents[T](eventFilters: Iterable[EventFilter])(block: T): T = {
EventHandler.notify(TestEvent.Mute(eventFilters.toSeq))
def filterEvents[T](eventFilters: Iterable[EventFilter])(block: T)(implicit app: AkkaApplication): T = {
app.eventHandler.notify(TestEvent.Mute(eventFilters.toSeq))
try {
block
} finally {
EventHandler.notify(TestEvent.UnMute(eventFilters.toSeq))
app.eventHandler.notify(TestEvent.UnMute(eventFilters.toSeq))
}
}
def filterEvents[T](eventFilters: EventFilter*)(block: T): T = filterEvents(eventFilters.toSeq)(block)
def filterEvents[T](eventFilters: EventFilter*)(block: T)(implicit app: AkkaApplication): T = filterEvents(eventFilters.toSeq)(block)
def filterException[T <: Throwable: Manifest](block: Unit): Unit = filterEvents(Seq(EventFilter[T]))(block)
def filterException[T <: Throwable](block: Unit)(implicit app: AkkaApplication, m: Manifest[T]): Unit = filterEvents(Seq(EventFilter[T]))(block)
}

View file

@ -94,8 +94,6 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach {
import TestActorRefSpec._
EventHandler.start()
override def beforeEach {
otherthread = null
}

View file

@ -108,9 +108,9 @@ object Pi extends App {
master.?(Calculate, Timeout(60000)).
await.resultOrException match { //wait for the result, with a 60 seconds timeout
case Some(pi)
EventHandler.info(this, "\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis".format(pi, (now - start)))
app.eventHandler.info(this, "\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis".format(pi, (now - start)))
case None
EventHandler.error(this, "Pi calculation did not complete within the timeout.")
app.eventHandler.error(this, "Pi calculation did not complete within the timeout.")
}
}
}