From f46c6dc533331a112ac68f8300b59995a9900b90 Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 27 Oct 2011 12:23:01 +0200 Subject: [PATCH 01/19] introducing: MainBus feat. LoggingBus most tests passing, everything compiling, but docs not updated and nasty thread-leak preventing me from running the whole test-suite (which is the reason for this commit: I want to chase down that one first). - the app.mainbus is classified by Class[_] (currently lookup, will possibly change to sub-class-aware) and accepts AnyRef messages - LoggingBus handles akka.event-handlers from config specially: + start them as system services, supervised by SystemGuardian + keep their subscriptions in sync when logLevel_= is called + send them InitializeLogger(bus) message before subscribing them (so they can register for extras like Mute/UnMute) - two-phased start-up: first phase with actor-less stdout logging, then subscription of config loggers, then remove stdout logger (logLevels configurable separately) - MainBusReaper watches registered receivers and unsubscribes them upon death (started in phase 2) - logger factory on Logging object, needs app/bus and log source; default instance in app.log --- .../test/scala/akka/actor/FSMActorSpec.scala | 52 ++-- .../scala/akka/actor/LoggingReceiveSpec.scala | 63 ++-- .../akka/actor/RestartStrategySpec.scala | 8 - .../test/scala/akka/actor/SchedulerSpec.scala | 4 - .../scala/akka/actor/SupervisorSpec.scala | 10 +- .../akka/actor/dispatch/ActorModelSpec.scala | 13 +- .../akka/actor/dispatch/PinnedActorSpec.scala | 26 +- .../test/scala/akka/event/EventBusSpec.scala | 1 - .../test/scala/akka/event/MainBusSpec.scala | 32 ++ .../trading/common/AkkaPerformanceTest.scala | 5 +- .../trading/common/MatchingEngine.scala | 8 +- .../trading/common/OrderReceiver.scala | 6 +- .../trading/oneway/OneWayMatchingEngine.scala | 2 +- .../trading/oneway/OneWayOrderReceiver.scala | 2 +- .../workbench/BenchResultRepository.scala | 12 +- .../akka/performance/workbench/Report.scala | 8 +- .../src/main/scala/akka/AkkaApplication.scala | 43 ++- .../src/main/scala/akka/actor/Actor.scala | 10 +- .../src/main/scala/akka/actor/ActorCell.scala | 29 +- .../src/main/scala/akka/actor/ActorRef.scala | 4 +- .../scala/akka/actor/ActorRefProvider.scala | 7 +- .../src/main/scala/akka/actor/Deployer.scala | 9 +- .../src/main/scala/akka/actor/FSM.scala | 18 +- akka-actor/src/main/scala/akka/actor/IO.scala | 1 - .../akka/dispatch/AbstractDispatcher.scala | 4 +- .../main/scala/akka/dispatch/Dispatcher.scala | 6 +- .../src/main/scala/akka/dispatch/Future.scala | 16 +- .../main/scala/akka/dispatch/Mailbox.scala | 3 +- .../akka/dispatch/ThreadPoolBuilder.scala | 6 +- .../src/main/scala/akka/event/EventBus.scala | 7 +- .../main/scala/akka/event/EventHandler.scala | 241 --------------- .../src/main/scala/akka/event/Logging.scala | 292 ++++++++++++++++-- .../src/main/scala/akka/event/MainBus.scala | 56 ++++ .../scala/akka/remote/RemoteInterface.scala | 4 +- .../src/main/scala/akka/routing/Routing.scala | 1 - .../src/main/scala/akka/util/Index.scala | 19 ++ akka-actor/src/main/scala/akka/util/JMX.scala | 6 +- .../scala/akka/util/ReflectiveAccess.scala | 11 +- akka-docs/scala/code/ActorDocSpec.scala | 29 +- akka-http/src/main/scala/akka/http/Mist.scala | 6 +- .../scala/akka/http/Servlet30Context.scala | 5 +- .../src/main/scala/akka/remote/Remote.scala | 24 +- .../akka/remote/RemoteActorRefProvider.scala | 14 +- .../akka/remote/RemoteConnectionManager.scala | 9 +- .../remote/netty/NettyRemoteSupport.scala | 93 +++--- .../serialization/SerializationProtocol.scala | 14 +- .../main/scala/akka/event/slf4j/SLF4J.scala | 3 +- .../test/UntypedCoordinatedIncrementTest.java | 5 +- .../test/UntypedTransactorTest.java | 5 +- .../transactor/CoordinatedIncrementSpec.scala | 21 +- .../scala/transactor/FickleFriendsSpec.scala | 23 +- .../scala/transactor/TransactorSpec.scala | 21 +- .../testkit/CallingThreadDispatcher.scala | 12 +- .../scala/akka/testkit/TestActorRef.scala | 1 - .../akka/testkit/TestEventListener.scala | 32 +- .../src/main/scala/akka/testkit/TestKit.scala | 3 +- .../src/main/scala/akka/testkit/package.scala | 6 +- .../test/scala/akka/testkit/AkkaSpec.scala | 3 + .../scala/akka/testkit/TestActorRefSpec.scala | 3 +- .../scala/akka/testkit/TestProbeSpec.scala | 1 - .../src/main/scala/Pi.scala | 7 +- config/akka-reference.conf | 5 +- config/akka.test.conf | 2 +- 63 files changed, 749 insertions(+), 643 deletions(-) create mode 100644 akka-actor-tests/src/test/scala/akka/event/MainBusSpec.scala create mode 100644 akka-actor/src/main/scala/akka/event/MainBus.scala diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala index 6d85c6997c..d1cda03f9d 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala @@ -7,7 +7,7 @@ package akka.actor import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } import akka.testkit._ -import TestEvent.{ Mute, UnMuteAll } +import TestEvent.Mute import FSM._ import akka.util.Duration import akka.util.duration._ @@ -61,7 +61,7 @@ object FSMActorSpec { whenUnhandled { case Ev(msg) ⇒ { - app.eventHandler.info(this, "unhandled event " + msg + " in state " + stateName + " with data " + stateData) + log.info("unhandled event " + msg + " in state " + stateName + " with data " + stateData) unhandledLatch.open stay } @@ -132,8 +132,8 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true lockedLatch.await filterEvents(EventFilter.custom { - case EventHandler.Info(_: Lock, _) ⇒ true - case _ ⇒ false + case Logging.Info(_: Lock, _) ⇒ true + case _ ⇒ false }) { lock ! "not_handled" unhandledLatch.await @@ -163,18 +163,13 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true case Ev("go") ⇒ goto(2) } }) - val logger = actorOf(new Actor { - def receive = { - case x ⇒ testActor forward x - } - }) - filterException[EventHandler.EventHandlerException] { - app.eventHandler.addListener(logger) + filterException[Logging.EventHandlerException] { + app.mainbus.subscribe(testActor, classOf[Logging.Error]) fsm ! "go" - expectMsgPF(1 second) { - case EventHandler.Error(_: EventHandler.EventHandlerException, `fsm`, "Next state 2 does not exist") ⇒ true + expectMsgPF(1 second, hint = "Next state 2 does not exist") { + case Logging.Error(_, `fsm`, "Next state 2 does not exist") ⇒ true } - app.eventHandler.removeListener(logger) + app.mainbus.unsubscribe(testActor) } } @@ -198,9 +193,9 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true new TestKit(AkkaApplication("fsm event", AkkaApplication.defaultConfig ++ Configuration("akka.event-handler-level" -> "DEBUG", "akka.actor.debug.fsm" -> true))) { - app.eventHandler.notify(TestEvent.Mute(EventFilter.custom { - case _: EventHandler.Debug ⇒ true - case _ ⇒ false + app.mainbus.publish(TestEvent.Mute(EventFilter.custom { + case _: Logging.Debug ⇒ true + case _ ⇒ false })) val fsm = TestActorRef(new Actor with LoggingFSM[Int, Null] { startWith(1, null) @@ -218,25 +213,20 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true case StopEvent(r, _, _) ⇒ testActor ! r } }) - val logger = actorOf(new Actor { - def receive = { - case x ⇒ testActor forward x - } - }) - app.eventHandler.addListener(logger) + app.mainbus.subscribe(testActor, classOf[Logging.Debug]) fsm ! "go" - expectMsgPF(1 second) { - case EventHandler.Debug(`fsm`, s: String) if s.startsWith("processing Event(go,null) from Actor[testActor") ⇒ true + expectMsgPF(1 second, hint = "processing Event(go,null)") { + case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(go,null) from Actor[testActor") ⇒ true } - expectMsg(1 second, EventHandler.Debug(fsm, "setting timer 't'/1500 milliseconds: Shutdown")) - expectMsg(1 second, EventHandler.Debug(fsm, "transition 1 -> 2")) + expectMsg(1 second, Logging.Debug(fsm, "setting timer 't'/1500 milliseconds: Shutdown")) + expectMsg(1 second, Logging.Debug(fsm, "transition 1 -> 2")) fsm ! "stop" - expectMsgPF(1 second) { - case EventHandler.Debug(`fsm`, s: String) if s.startsWith("processing Event(stop,null) from Actor[testActor") ⇒ true + expectMsgPF(1 second, hint = "processing Event(stop,null)") { + case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(stop,null) from Actor[testActor") ⇒ true } - expectMsgAllOf(1 second, EventHandler.Debug(fsm, "canceling timer 't'"), Normal) + expectMsgAllOf(1 second, Logging.Debug(fsm, "canceling timer 't'"), Normal) expectNoMsg(1 second) - app.eventHandler.removeListener(logger) + app.mainbus.unsubscribe(testActor) } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala index 5ea68924d1..e708b4771e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala @@ -10,7 +10,7 @@ import org.scalatest.WordSpec import akka.AkkaApplication import akka.AkkaApplication.defaultConfig import akka.config.Configuration -import akka.event.EventHandler +import akka.event.Logging object LoggingReceiveSpec { class TestLogActor extends Actor { @@ -28,13 +28,13 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd val appLifecycle = AkkaApplication("lifecycle", config ++ Configuration("akka.actor.debug.lifecycle" -> true)) val filter = TestEvent.Mute(EventFilter.custom { - case _: EventHandler.Debug ⇒ true - case _: EventHandler.Info ⇒ true - case _ ⇒ false + case _: Logging.Debug ⇒ true + case _: Logging.Info ⇒ true + case _ ⇒ false }) - appLogging.eventHandler.notify(filter) - appAuto.eventHandler.notify(filter) - appLifecycle.eventHandler.notify(filter) + appLogging.mainbus.publish(filter) + appAuto.mainbus.publish(filter) + appLifecycle.mainbus.publish(filter) def ignoreMute(t: TestKit) { t.ignoreMsg { @@ -46,20 +46,20 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd "decorate a Receive" in { new TestKit(appLogging) { - app.eventHandler.addListener(testActor) + app.mainbus.subscribe(testActor, classOf[Logging.Debug]) val r: Actor.Receive = { case null ⇒ } val log = Actor.LoggingReceive(this, r) log.isDefinedAt("hallo") - expectMsg(1 second, EventHandler.Debug(this, "received unhandled message hallo")) - } + expectMsg(1 second, Logging.Debug(this, "received unhandled message hallo")) + }.app.stop() } "be added on Actor if requested" in { new TestKit(appLogging) with ImplicitSender { ignoreMute(this) - app.eventHandler.addListener(testActor) + app.mainbus.subscribe(testActor, classOf[Logging.Debug]) val actor = TestActorRef(new Actor { def receive = loggable(this) { case _ ⇒ channel ! "x" @@ -67,7 +67,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd }) actor ! "buh" within(1 second) { - expectMsg(EventHandler.Debug(actor.underlyingActor, "received handled message buh")) + expectMsg(Logging.Debug(actor.underlyingActor, "received handled message buh")) expectMsg("x") } val r: Actor.Receive = { @@ -78,17 +78,16 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd within(300 millis) { actor ! "bah" expectMsgPF() { - case EventHandler.Error(_: UnhandledMessageException, `actor`, _) ⇒ true + case Logging.Error(_: UnhandledMessageException, `actor`, _) ⇒ true } } } - actor.stop() - } + }.app.stop() } "not duplicate logging" in { new TestKit(appLogging) with ImplicitSender { - app.eventHandler.addListener(testActor) + app.mainbus.subscribe(testActor, classOf[Logging.Debug]) val actor = TestActorRef(new Actor { def receive = loggable(this)(loggable(this) { case _ ⇒ channel ! "x" @@ -96,10 +95,10 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd }) actor ! "buh" within(1 second) { - expectMsg(EventHandler.Debug(actor.underlyingActor, "received handled message buh")) + expectMsg(Logging.Debug(actor.underlyingActor, "received handled message buh")) expectMsg("x") } - } + }.app.stop() } } @@ -108,65 +107,65 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd "log AutoReceiveMessages if requested" in { new TestKit(appAuto) { - app.eventHandler.addListener(testActor) + app.mainbus.subscribe(testActor, classOf[Logging.Debug]) val actor = TestActorRef(new Actor { def receive = { case _ ⇒ } }) actor ! PoisonPill - expectMsg(300 millis, EventHandler.Debug(actor.underlyingActor, "received AutoReceiveMessage PoisonPill")) + expectMsg(300 millis, Logging.Debug(actor.underlyingActor, "received AutoReceiveMessage PoisonPill")) awaitCond(actor.isShutdown, 100 millis) - } + }.app.stop() } // TODO remove ignore as soon as logging is working properly during start-up again "log LifeCycle changes if requested" ignore { new TestKit(appLifecycle) { ignoreMute(this) - app.eventHandler.addListener(testActor) + app.mainbus.subscribe(testActor, classOf[Logging.Debug]) within(2 seconds) { val supervisor = TestActorRef[TestLogActor](Props[TestLogActor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000))) - expectMsg(EventHandler.Debug(supervisor, "started")) + expectMsg(Logging.Debug(supervisor, "started")) val actor = new TestActorRef[TestLogActor](app, Props[TestLogActor], supervisor, "none") expectMsgPF() { - case EventHandler.Debug(ref, msg: String) ⇒ ref == supervisor && msg.startsWith("now supervising") + case Logging.Debug(ref, msg: String) ⇒ ref == supervisor && msg.startsWith("now supervising") } - expectMsg(EventHandler.Debug(actor, "started")) + expectMsg(Logging.Debug(actor, "started")) supervisor startsMonitoring actor expectMsgPF(hint = "now monitoring") { - case EventHandler.Debug(ref, msg: String) ⇒ + case Logging.Debug(ref, msg: String) ⇒ ref == supervisor.underlyingActor && msg.startsWith("now monitoring") } supervisor stopsMonitoring actor expectMsgPF(hint = "stopped monitoring") { - case EventHandler.Debug(ref, msg: String) ⇒ + case Logging.Debug(ref, msg: String) ⇒ ref == supervisor.underlyingActor && msg.startsWith("stopped monitoring") } filterException[ActorKilledException] { actor ! Kill expectMsgPF() { - case EventHandler.Error(_: ActorKilledException, `actor`, "Kill") ⇒ true + case Logging.Error(_: ActorKilledException, `actor`, "Kill") ⇒ true } - expectMsg(EventHandler.Debug(actor, "restarting")) + expectMsg(Logging.Debug(actor, "restarting")) } awaitCond(msgAvailable) expectMsgPF(hint = "restarted") { - case EventHandler.Debug(`actor`, "restarted") ⇒ true + case Logging.Debug(`actor`, "restarted") ⇒ true } actor.stop() - expectMsg(EventHandler.Debug(actor, "stopping")) + expectMsg(Logging.Debug(actor, "stopping")) supervisor.stop() } - } + }.app.stop() } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala index 1fa42ac61b..6021ecf8cc 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala @@ -15,14 +15,6 @@ import akka.testkit.AkkaSpec @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class RestartStrategySpec extends AkkaSpec { - override def atStartup() { - app.eventHandler.notify(Mute(EventFilter[Exception]("Crashing..."))) - } - - override def atTermination() { - app.eventHandler.notify(UnMuteAll) - } - object Ping object Crash diff --git a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala index 9d4f0ebd4e..7ab547cf1f 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala @@ -17,10 +17,6 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach { future } - override def beforeEach { - app.eventHandler.notify(Mute(EventFilter[Exception]("CRASH"))) - } - override def afterEach { while (futures.peek() ne null) { Option(futures.poll()).foreach(_.cancel(true)) } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala index 20b1f92aea..0f90840564 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala @@ -118,14 +118,8 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende (pingpong1, pingpong2, pingpong3, topSupervisor) } - override def atStartup() = { - app.eventHandler notify Mute(EventFilter[Exception]("Die"), - EventFilter[IllegalStateException]("Don't wanna!"), - EventFilter[RuntimeException]("Expected")) - } - - override def atTermination() = { - app.eventHandler notify UnMuteAll + override def atStartup() { + app.mainbus.publish(Mute(EventFilter[RuntimeException](ExceptionMessage))) } override def beforeEach() = { diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index 318120a2a4..54a6d3bd0e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -15,6 +15,7 @@ import akka.actor._ import util.control.NoStackTrace import akka.AkkaApplication import akka.util.duration._ +import akka.event.Logging.Error object ActorModelSpec { @@ -154,8 +155,8 @@ object ActorModelSpec { await(deadline)(stops == dispatcher.stops.get) } catch { case e ⇒ - app.eventHandler.error(e, dispatcher, "actual: starts=" + dispatcher.starts.get + ",stops=" + dispatcher.stops.get + - " required: starts=" + starts + ",stops=" + stops) + app.mainbus.publish(Error(e, dispatcher, "actual: starts=" + dispatcher.starts.get + ",stops=" + dispatcher.stops.get + + " required: starts=" + starts + ",stops=" + stops)) throw e } } @@ -215,9 +216,9 @@ object ActorModelSpec { await(deadline)(stats.restarts.get() == restarts) } catch { case e ⇒ - app.eventHandler.error(e, dispatcher, "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions + + app.mainbus.publish(Error(e, dispatcher, "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions + ",res=" + resumes + ",reg=" + registers + ",unreg=" + unregisters + - ",recv=" + msgsReceived + ",proc=" + msgsProcessed + ",restart=" + restarts) + ",recv=" + msgsReceived + ",proc=" + msgsProcessed + ",restart=" + restarts)) throw e } } @@ -321,7 +322,7 @@ abstract class ActorModelSpec extends AkkaSpec { try { f } catch { - case e ⇒ app.eventHandler.error(e, this, "error in spawned thread") + case e ⇒ app.mainbus.publish(Error(e, this, "error in spawned thread")) } } } @@ -373,7 +374,7 @@ abstract class ActorModelSpec extends AkkaSpec { } "continue to process messages when a thread gets interrupted" in { - filterEvents(EventFilter[InterruptedException], EventFilter[akka.event.EventHandler.EventHandlerException]) { + filterEvents(EventFilter[InterruptedException], EventFilter[akka.event.Logging.EventHandlerException]) { implicit val dispatcher = newInterceptedDispatcher implicit val timeout = Timeout(5 seconds) val a = newTestActor(dispatcher) diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala index a2f0a785de..f337c445b6 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala @@ -2,8 +2,7 @@ package akka.actor.dispatch import java.util.concurrent.{ CountDownLatch, TimeUnit } -import akka.testkit.TestEvent._ -import akka.testkit.EventFilter +import akka.testkit._ import akka.dispatch.{ PinnedDispatcher, Dispatchers } import akka.actor.{ Props, Actor } import akka.testkit.AkkaSpec @@ -24,14 +23,6 @@ class PinnedActorSpec extends AkkaSpec with BeforeAndAfterEach { private val unit = TimeUnit.MILLISECONDS - override def beforeEach { - app.eventHandler.notify(Mute(EventFilter[RuntimeException]("Failure"))) - } - - override def afterEach { - app.eventHandler.notify(UnMuteAll) - } - "A PinnedActor" must { "support tell" in { @@ -51,13 +42,14 @@ class PinnedActorSpec extends AkkaSpec with BeforeAndAfterEach { "support ask/exception" in { val actor = actorOf(Props[TestActor].withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test"))) - app.eventHandler.notify(Mute(EventFilter[RuntimeException]("Expected exception; to test fault-tolerance"))) - try { - (actor ? "Failure").get - fail("Should have thrown an exception") - } catch { - case e ⇒ - assert("Expected exception; to test fault-tolerance" === e.getMessage()) + filterException[RuntimeException] { + try { + (actor ? "Failure").get + fail("Should have thrown an exception") + } catch { + case e ⇒ + assert("Expected exception; to test fault-tolerance" === e.getMessage()) + } } actor.stop() } diff --git a/akka-actor-tests/src/test/scala/akka/event/EventBusSpec.scala b/akka-actor-tests/src/test/scala/akka/event/EventBusSpec.scala index 4861dd9ea5..afd255a27c 100644 --- a/akka-actor-tests/src/test/scala/akka/event/EventBusSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/event/EventBusSpec.scala @@ -147,7 +147,6 @@ object ActorEventBusSpec { type Classifier = String def classify(event: Event) = event.toString - protected def compareSubscribers(a: Subscriber, b: Subscriber): Int = a compareTo b protected def mapSize = 32 def publish(event: Event, subscriber: Subscriber) = subscriber ! event } diff --git a/akka-actor-tests/src/test/scala/akka/event/MainBusSpec.scala b/akka-actor-tests/src/test/scala/akka/event/MainBusSpec.scala new file mode 100644 index 0000000000..a132ae32d6 --- /dev/null +++ b/akka-actor-tests/src/test/scala/akka/event/MainBusSpec.scala @@ -0,0 +1,32 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.event + +import akka.testkit.AkkaSpec +import akka.config.Configuration +import akka.util.duration._ + +object MainBusSpec { + case class M(i: Int) +} + +class MainBusSpec extends AkkaSpec(Configuration( + "akka.actor.debug.lifecycle" -> true, + "akka.actor.debug.mainbus" -> true)) { + + import MainBusSpec._ + + "A MainBus" must { + + "allow subscriptions" in { + val bus = new MainBus(true) + bus.start(app) + bus.subscribe(testActor, classOf[M]) + bus.publish(M(42)) + expectMsg(1 second, M(42)) + } + + } + +} \ No newline at end of file diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/common/AkkaPerformanceTest.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/common/AkkaPerformanceTest.scala index 3bd7ea96d9..0ba7c65be6 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/common/AkkaPerformanceTest.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/common/AkkaPerformanceTest.scala @@ -6,9 +6,12 @@ import akka.performance.trading.domain._ import akka.performance.trading.common._ import akka.actor.{ Props, ActorRef, Actor, PoisonPill } import akka.AkkaApplication +import akka.event.Logging abstract class AkkaPerformanceTest(val app: AkkaApplication) extends BenchmarkScenarios { + val log = Logging(app, this) + type TS = AkkaTradingSystem val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher") @@ -70,7 +73,7 @@ abstract class AkkaPerformanceTest(val app: AkkaApplication) extends BenchmarkSc placeOrder(orderReceiver, o) } if (!rsp.status) { - app.eventHandler.error(this, "Invalid rsp") + log.error("Invalid rsp") } delay(delayMs) } diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/common/MatchingEngine.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/common/MatchingEngine.scala index 5f2989fa97..f13fd20dba 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/common/MatchingEngine.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/common/MatchingEngine.scala @@ -16,7 +16,7 @@ trait MatchingEngine { } class AkkaMatchingEngine(val meId: String, val orderbooks: List[Orderbook]) - extends Actor with MatchingEngine { + extends Actor with MatchingEngine with ActorLogging { var standby: Option[ActorRef] = None @@ -26,7 +26,7 @@ class AkkaMatchingEngine(val meId: String, val orderbooks: List[Orderbook]) case order: Order ⇒ handleOrder(order) case unknown ⇒ - app.eventHandler.warning(this, "Received unknown message: " + unknown) + log.warning("Received unknown message: " + unknown) } def handleOrder(order: Order) { @@ -41,7 +41,7 @@ class AkkaMatchingEngine(val meId: String, val orderbooks: List[Orderbook]) pendingStandbyReply.foreach(waitForStandby(_)) done(true) case None ⇒ - app.eventHandler.warning(this, "Orderbook not handled by this MatchingEngine: " + order.orderbookSymbol) + log.warning("Orderbook not handled by this MatchingEngine: " + order.orderbookSymbol) done(false) } } @@ -55,7 +55,7 @@ class AkkaMatchingEngine(val meId: String, val orderbooks: List[Orderbook]) pendingStandbyFuture.await } catch { case e: FutureTimeoutException ⇒ - app.eventHandler.error(this, "Standby timeout: " + e) + log.error("Standby timeout: " + e) } } diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/common/OrderReceiver.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/common/OrderReceiver.scala index c1f811b425..f477cd43ca 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/common/OrderReceiver.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/common/OrderReceiver.scala @@ -24,14 +24,14 @@ trait OrderReceiver { } -class AkkaOrderReceiver extends Actor with OrderReceiver { +class AkkaOrderReceiver extends Actor with OrderReceiver with ActorLogging { type ME = ActorRef def receive = { case routing @ MatchingEngineRouting(mapping) ⇒ refreshMatchingEnginePartitions(routing.asInstanceOf[MatchingEngineRouting[ActorRef]]) case order: Order ⇒ placeOrder(order) - case unknown ⇒ app.eventHandler.warning(this, "Received unknown message: " + unknown) + case unknown ⇒ log.warning("Received unknown message: " + unknown) } def placeOrder(order: Order) = { @@ -40,7 +40,7 @@ class AkkaOrderReceiver extends Actor with OrderReceiver { case Some(m) ⇒ m.forward(order) case None ⇒ - app.eventHandler.warning(this, "Unknown orderbook: " + order.orderbookSymbol) + log.warning("Unknown orderbook: " + order.orderbookSymbol) channel ! new Rsp(false) } } diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayMatchingEngine.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayMatchingEngine.scala index 44ce92a92d..20c33551e9 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayMatchingEngine.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayMatchingEngine.scala @@ -17,7 +17,7 @@ class OneWayMatchingEngine(meId: String, orderbooks: List[Orderbook]) extends Ak orderbook.matchOrders() case None ⇒ - app.eventHandler.warning(this, "Orderbook not handled by this MatchingEngine: " + order.orderbookSymbol) + log.warning("Orderbook not handled by this MatchingEngine: " + order.orderbookSymbol) } } diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayOrderReceiver.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayOrderReceiver.scala index daeabfb36b..46dd57c111 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayOrderReceiver.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayOrderReceiver.scala @@ -13,7 +13,7 @@ class OneWayOrderReceiver extends AkkaOrderReceiver { case Some(m) ⇒ m ! order case None ⇒ - app.eventHandler.warning(this, "Unknown orderbook: " + order.orderbookSymbol) + log.warning("Unknown orderbook: " + order.orderbookSymbol) } } } diff --git a/akka-actor-tests/src/test/scala/akka/performance/workbench/BenchResultRepository.scala b/akka-actor-tests/src/test/scala/akka/performance/workbench/BenchResultRepository.scala index bc9d6593f3..5713f91762 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/workbench/BenchResultRepository.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/workbench/BenchResultRepository.scala @@ -13,6 +13,7 @@ import java.text.SimpleDateFormat import java.util.Date import scala.collection.mutable.{ Map ⇒ MutableMap } import akka.AkkaApplication +import akka.event.Logging trait BenchResultRepository { def add(stats: Stats) @@ -43,6 +44,8 @@ class FileBenchResultRepository(val app: AkkaApplication) extends BenchResultRep private def htmlDirExists: Boolean = new File(htmlDir).exists protected val maxHistorical = 7 + val log = Logging(app, this) + case class Key(name: String, load: Int) def add(stats: Stats) { @@ -102,8 +105,7 @@ class FileBenchResultRepository(val app: AkkaApplication) extends BenchResultRep out.writeObject(stats) } catch { case e: Exception ⇒ - app.eventHandler.error(this, "Failed to save [%s] to [%s], due to [%s]". - format(stats, f.getAbsolutePath, e.getMessage)) + log.error("Failed to save [{}] to [{}], due to [{}]", stats, f.getAbsolutePath, e.getMessage) } finally { if (out ne null) try { out.close() } catch { case ignore: Exception ⇒ } } @@ -119,8 +121,7 @@ class FileBenchResultRepository(val app: AkkaApplication) extends BenchResultRep Some(stats) } catch { case e: Throwable ⇒ - app.eventHandler.error(this, "Failed to load from [%s], due to [%s]". - format(f.getAbsolutePath, e.getMessage)) + log.error("Failed to load from [{}], due to [{}]", f.getAbsolutePath, e.getMessage) None } finally { if (in ne null) try { in.close() } catch { case ignore: Exception ⇒ } @@ -143,8 +144,7 @@ class FileBenchResultRepository(val app: AkkaApplication) extends BenchResultRep writer.flush() } catch { case e: Exception ⇒ - app.eventHandler.error(this, "Failed to save report to [%s], due to [%s]". - format(f.getAbsolutePath, e.getMessage)) + log.error("Failed to save report to [{}], due to [{}]", f.getAbsolutePath, e.getMessage) } finally { if (writer ne null) try { writer.close() } catch { case ignore: Exception ⇒ } } diff --git a/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala b/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala index d3a5f020d0..0383b04e8c 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala @@ -6,12 +6,14 @@ import java.util.Date import scala.collection.JavaConversions.asScalaBuffer import scala.collection.JavaConversions.enumerationAsScalaIterator import akka.AkkaApplication +import akka.event.Logging class Report(app: AkkaApplication, resultRepository: BenchResultRepository, compareResultWith: Option[String] = None) { - private def log = System.getProperty("benchmark.logResult", "true").toBoolean + private def doLog = System.getProperty("benchmark.logResult", "true").toBoolean + val log = Logging(app, this) val dateTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm") val legendTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm") @@ -51,8 +53,8 @@ class Report(app: AkkaApplication, val reportName = current.name + "--" + timestamp + ".html" resultRepository.saveHtmlReport(sb.toString, reportName) - if (log) { - app.eventHandler.info(this, resultTable + "Charts in html report: " + resultRepository.htmlReportUrl(reportName)) + if (doLog) { + log.info(resultTable + "Charts in html report: " + resultRepository.htmlReportUrl(reportName)) } } diff --git a/akka-actor/src/main/scala/akka/AkkaApplication.scala b/akka-actor/src/main/scala/akka/AkkaApplication.scala index b2bcccb0b5..837960229d 100644 --- a/akka-actor/src/main/scala/akka/AkkaApplication.scala +++ b/akka-actor/src/main/scala/akka/AkkaApplication.scala @@ -20,6 +20,8 @@ import java.net.InetSocketAddress object AkkaApplication { + type AkkaConfig = a.AkkaConfig.type forSome { val a: AkkaApplication } + val Version = "2.0-SNAPSHOT" val envHome = System.getenv("AKKA_HOME") match { @@ -92,12 +94,14 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor val ActorTimeoutMillis = ActorTimeout.duration.toMillis val SerializeAllMessages = getBool("akka.actor.serialize-messages", false) - val LogLevel = getString("akka.event-handler-level", "INFO") + val LogLevel = getString("akka.loglevel", "INFO") + val StdoutLogLevel = getString("akka.stdout-loglevel", LogLevel) val EventHandlers = getList("akka.event-handlers") val AddLoggingReceive = getBool("akka.actor.debug.receive", false) val DebugAutoReceive = getBool("akka.actor.debug.autoreceive", false) val DebugLifecycle = getBool("akka.actor.debug.lifecycle", false) val FsmDebugEvent = getBool("akka.actor.debug.fsm", false) + val DebugMainBus = getBool("akka.actor.debug.mainbus", false) val DispatcherThroughput = getInt("akka.actor.throughput", 5) val DispatcherDefaultShutdown = getLong("akka.actor.dispatcher-shutdown-timeout"). @@ -133,6 +137,8 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor val ExpiredHeaderValue = config.getString("akka.http.expired-header-value", "expired") } + private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(props, systemGuardian, address, true) + import AkkaConfig._ if (ConfigVersion != Version) @@ -159,11 +165,14 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor val defaultAddress = new InetSocketAddress(hostname, AkkaConfig.RemoteServerPort) + // this provides basic logging (to stdout) until .start() is called below + val mainbus = new MainBus(DebugMainBus) + mainbus.startStdoutLogger(AkkaConfig) + val log = new MainBusLogging(mainbus, this) + // TODO correctly pull its config from the config val dispatcherFactory = new Dispatchers(this) - implicit val dispatcher = dispatcherFactory.defaultGlobalDispatcher - def terminationFuture: Future[ExitStatus] = provider.terminationFuture // TODO think about memory consistency effects when doing funky stuff inside constructor @@ -175,7 +184,7 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor // TODO make this configurable protected[akka] val guardian: ActorRef = { import akka.actor.FaultHandlingStrategy._ - new LocalActorRef(this, + provider.actorOf( Props(context ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy { case _: ActorKilledException ⇒ Stop case _: ActorInitializationException ⇒ Stop @@ -186,20 +195,31 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor true) } - // TODO think about memory consistency effects when doing funky stuff inside constructor - val eventHandler = new EventHandler(this) - - // TODO think about memory consistency effects when doing funky stuff inside constructor - val log: Logging = new EventHandlerLogging(eventHandler, this) + protected[akka] val systemGuardian: ActorRef = { + import akka.actor.FaultHandlingStrategy._ + provider.actorOf( + Props(context ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy { + case _: ActorKilledException ⇒ Stop + case _: ActorInitializationException ⇒ Stop + case _: Exception ⇒ Restart + }).withDispatcher(dispatcher), + provider.theOneWhoWalksTheBubblesOfSpaceTime, + "SystemSupervisor", + true) + } // TODO think about memory consistency effects when doing funky stuff inside constructor val deadLetters = new DeadLetterActorRef(this) + val deathWatch = provider.createDeathWatch() + + // this starts the reaper actor and the user-configured logging subscribers, which are also actors + mainbus.start(this) + mainbus.startDefaultLoggers(this, AkkaConfig) + // TODO think about memory consistency effects when doing funky stuff inside an ActorRefProvider's constructor val deployer = new Deployer(this) - val deathWatch = provider.createDeathWatch() - // TODO think about memory consistency effects when doing funky stuff inside constructor val typedActor = new TypedActor(this) @@ -212,6 +232,7 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor // TODO shutdown all that other stuff, whatever that may be def stop(): Unit = { guardian.stop() + systemGuardian.stop() } terminationFuture.onComplete(_ ⇒ dispatcher.shutdown()) diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index d55c330840..f8ad786341 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -13,7 +13,7 @@ import akka.remote.RemoteSupport import akka.cluster.ClusterNode import akka.japi.{ Creator, Procedure } import akka.serialization.{ Serializer, Serialization } -import akka.event.EventHandler +import akka.event.Logging.Debug import akka.experimental import akka.{ AkkaApplication, AkkaException } @@ -153,6 +153,10 @@ object Timeout { implicit def defaultTimeout(implicit app: AkkaApplication) = app.AkkaConfig.ActorTimeout } +trait ActorLogging { this: Actor ⇒ + val log = akka.event.Logging(app.mainbus, context.self) +} + object Actor { type Receive = PartialFunction[Any, Unit] @@ -163,7 +167,7 @@ object Actor { class LoggingReceive(source: AnyRef, r: Receive)(implicit app: AkkaApplication) extends Receive { def isDefinedAt(o: Any) = { val handled = r.isDefinedAt(o) - app.eventHandler.debug(source, "received " + (if (handled) "handled" else "unhandled") + " message " + o) + app.mainbus.publish(Debug(source, "received " + (if (handled) "handled" else "unhandled") + " message " + o)) handled } def apply(o: Any): Unit = r(o) @@ -418,7 +422,7 @@ trait Actor { private[akka] final def apply(msg: Any) = { def autoReceiveMessage(msg: AutoReceivedMessage) { - if (app.AkkaConfig.DebugAutoReceive) app.eventHandler.debug(this, "received AutoReceiveMessage " + msg) + if (app.AkkaConfig.DebugAutoReceive) app.mainbus.publish(Debug(this, "received AutoReceiveMessage " + msg)) msg match { case HotSwap(code, discardOld) ⇒ become(code(self), discardOld) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 75c7d1f02e..180cbe5df7 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -12,6 +12,7 @@ import scala.collection.JavaConverters import java.util.concurrent.{ ScheduledFuture, TimeUnit } import java.util.{ Collection ⇒ JCollection, Collections ⇒ JCollections } import akka.AkkaApplication +import akka.event.Logging.{ Debug, Warning, Error } /** * The actor context - the view of the actor cell from the actor. @@ -176,11 +177,11 @@ private[akka] class ActorCell( actor = created created.preStart() checkReceiveTimeout - if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "started") + if (app.AkkaConfig.DebugLifecycle) app.mainbus.publish(Debug(self, "started (" + actor + ")")) } catch { case e ⇒ try { - app.eventHandler.error(e, self, "error while creating actor") + app.mainbus.publish(Error(e, self, "error while creating actor")) // prevent any further messages to be processed until the actor has been restarted dispatcher.suspend(this) } finally { @@ -190,7 +191,7 @@ private[akka] class ActorCell( def recreate(cause: Throwable): Unit = try { val failedActor = actor - if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "restarting") + if (app.AkkaConfig.DebugLifecycle) app.mainbus.publish(Debug(self, "restarting")) val freshActor = newActor() if (failedActor ne null) { val c = currentMessage //One read only plz @@ -204,14 +205,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 (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "restarted") + if (app.AkkaConfig.DebugLifecycle) app.mainbus.publish(Debug(self, "restarted")) dispatcher.resume(this) //FIXME should this be moved down? props.faultHandler.handleSupervisorRestarted(cause, self, children) } catch { case e ⇒ try { - app.eventHandler.error(e, self, "error while creating actor") + app.mainbus.publish(Error(e, self, "error while creating actor")) // prevent any further messages to be processed until the actor has been restarted dispatcher.suspend(this) } finally { @@ -232,7 +233,7 @@ private[akka] class ActorCell( try { try { val a = actor - if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "stopping") + if (app.AkkaConfig.DebugLifecycle) app.mainbus.publish(Debug(self, "stopping")) if (a ne null) a.postStop() } finally { //Stop supervised actors @@ -257,8 +258,8 @@ private[akka] class ActorCell( val links = _children if (!links.contains(child)) { _children = _children.updated(child, ChildRestartStats()) - if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "now supervising " + child) - } else app.eventHandler.warning(self, "Already supervising " + child) + if (app.AkkaConfig.DebugLifecycle) app.mainbus.publish(Debug(self, "now supervising " + child)) + } else app.mainbus.publish(Warning(self, "Already supervising " + child)) } try { @@ -269,10 +270,10 @@ private[akka] class ActorCell( case Recreate(cause) ⇒ recreate(cause) case Link(subject) ⇒ app.deathWatch.subscribe(self, subject) - if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "now monitoring " + subject) + if (app.AkkaConfig.DebugLifecycle) app.mainbus.publish(Debug(self, "now monitoring " + subject)) case Unlink(subject) ⇒ app.deathWatch.unsubscribe(self, subject) - if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "stopped monitoring " + subject) + if (app.AkkaConfig.DebugLifecycle) app.mainbus.publish(Debug(self, "stopped monitoring " + subject)) case Suspend() ⇒ suspend() case Resume() ⇒ resume() case Terminate() ⇒ terminate() @@ -281,7 +282,7 @@ private[akka] class ActorCell( } } catch { case e ⇒ //Should we really catch everything here? - app.eventHandler.error(e, self, "error while processing " + message) + app.mainbus.publish(Error(e, self, "error while processing " + message)) //TODO FIXME How should problems here be handled? throw e } @@ -300,7 +301,7 @@ private[akka] class ActorCell( currentMessage = null // reset current message after successful invocation } catch { case e ⇒ - app.eventHandler.error(e, self, e.getMessage) + app.mainbus.publish(Error(e, self, e.getMessage)) // prevent any further messages to be processed until the actor has been restarted dispatcher.suspend(this) @@ -322,7 +323,7 @@ private[akka] class ActorCell( } } catch { case e ⇒ - app.eventHandler.error(e, self, e.getMessage) + app.mainbus.publish(Error(e, self, e.getMessage)) throw e } } else { @@ -334,7 +335,7 @@ private[akka] class ActorCell( def handleFailure(fail: Failed): Unit = _children.get(fail.actor) match { case Some(stats) ⇒ if (!props.faultHandler.handleFailure(fail, stats, _children)) throw fail.cause - case None ⇒ app.eventHandler.warning(self, "dropping " + fail + " from unknown child") + case None ⇒ app.mainbus.publish(Warning(self, "dropping " + fail + " from unknown child")) } def handleChildTerminated(child: ActorRef): Unit = { diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 43fb856afb..1df0dc64da 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -383,12 +383,12 @@ class DeadLetterActorRef(app: AkkaApplication) extends MinimalActorRef { override def isShutdown(): Boolean = true - protected[akka] override def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = app.eventHandler.notify(DeadLetter(message, channel)) + protected[akka] override def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = app.mainbus.publish(DeadLetter(message, channel)) protected[akka] override def postMessageToMailboxAndCreateFutureResultWithTimeout( message: Any, timeout: Timeout, - channel: UntypedChannel): Future[Any] = { app.eventHandler.notify(DeadLetter(message, channel)); brokenPromise } + channel: UntypedChannel): Future[Any] = { app.mainbus.publish(DeadLetter(message, channel)); brokenPromise } } abstract class AskActorRef(promise: Promise[Any], app: AkkaApplication) extends MinimalActorRef { diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index aa84ef2711..f8f701c4d2 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -11,7 +11,7 @@ import akka.AkkaApplication import java.util.concurrent.ConcurrentHashMap import com.eaio.uuid.UUID import akka.AkkaException -import akka.event.{ ActorClassification, DeathWatch, EventHandler } +import akka.event.{ ActorClassification, DeathWatch, MainBusLogging } import akka.dispatch._ /** @@ -91,6 +91,7 @@ class ActorRefProviderException(message: String) extends AkkaException(message) class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider { val terminationFuture = new DefaultPromise[AkkaApplication.ExitStatus](Timeout.never)(app.dispatcher) + val log = new MainBusLogging(app.mainbus, this) /** * Top-level anchor for the supervision hierarchy of this actor system. Will @@ -105,14 +106,14 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider { msg match { case Failed(child, ex) ⇒ child.stop() case ChildTerminated(child) ⇒ terminationFuture.completeWithResult(AkkaApplication.Stopped) - case _ ⇒ app.eventHandler.error(this, this + " received unexpected message " + msg) + case _ ⇒ log.error(this + " received unexpected message " + msg) } } protected[akka] override def sendSystemMessage(message: SystemMessage) { message match { case Supervise(child) ⇒ // TODO register child in some map to keep track of it and enable shutdown after all dead - case _ ⇒ app.eventHandler.error(this, this + " received unexpected system message " + message) + case _ ⇒ log.error(this + " received unexpected system message " + message) } } } diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index 82af1d9c56..fecdc801fa 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -8,7 +8,7 @@ import collection.immutable.Seq import java.util.concurrent.ConcurrentHashMap -import akka.event.EventHandler +import akka.event.MainBusLogging import akka.actor.DeploymentConfig._ import akka.{ AkkaException, AkkaApplication } import akka.config.{ Configuration, ConfigurationException } @@ -34,6 +34,7 @@ trait ActorDeployer { class Deployer(val app: AkkaApplication) extends ActorDeployer { val deploymentConfig = new DeploymentConfig(app) + val log = new MainBusLogging(app.mainbus, this) // val defaultAddress = Node(Config.nodename) @@ -81,7 +82,7 @@ class Deployer(val app: AkkaApplication) extends ActorDeployer { lookupInConfig(address) } catch { case e: ConfigurationException ⇒ - app.eventHandler.error(e, this, e.getMessage) //TODO FIXME I do not condone log AND rethrow + log.error(e, e.getMessage) //TODO FIXME I do not condone log AND rethrow throw e } @@ -324,13 +325,13 @@ class Deployer(val app: AkkaApplication) extends ActorDeployer { private[akka] def throwDeploymentBoundException(deployment: Deploy): Nothing = { val e = new DeploymentAlreadyBoundException("Address [" + deployment.address + "] already bound to [" + deployment + "]") - app.eventHandler.error(e, this, e.getMessage) + log.error(e, e.getMessage) throw e } private[akka] def thrownNoDeploymentBoundException(address: String): Nothing = { val e = new NoDeploymentBoundException("Address [" + address + "] is not bound to a deployment") - app.eventHandler.error(e, this, e.getMessage) + log.error(e, e.getMessage) throw e } } diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index 3ea53364ac..dee625d92c 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -4,7 +4,7 @@ package akka.actor import akka.util._ -import akka.event.EventHandler +import akka.event.MainBusLogging import scala.collection.mutable import java.util.concurrent.ScheduledFuture @@ -190,6 +190,8 @@ trait FSM[S, D] extends ListenerManagement { type Timeout = Option[Duration] type TransitionHandler = PartialFunction[(S, S), Unit] + val log = new MainBusLogging(app.mainbus, context.self) + /** * **************************************** * DSL @@ -421,7 +423,7 @@ trait FSM[S, D] extends ListenerManagement { */ private val handleEventDefault: StateFunction = { case Event(value, stateData) ⇒ - app.eventHandler.warning(context.self, "unhandled event " + value + " in state " + stateName) + log.warning("unhandled event " + value + " in state " + stateName) stay } private var handleEvent: StateFunction = handleEventDefault @@ -534,8 +536,8 @@ trait FSM[S, D] extends ListenerManagement { if (!currentState.stopReason.isDefined) { val reason = nextState.stopReason.get reason match { - case Failure(ex: Throwable) ⇒ app.eventHandler.error(ex, context.self, "terminating due to Failure") - case Failure(msg) ⇒ app.eventHandler.error(context.self, msg) + case Failure(ex: Throwable) ⇒ log.error(ex, "terminating due to Failure") + case Failure(msg: AnyRef) ⇒ log.error(msg.toString) case _ ⇒ } val stopEvent = StopEvent(reason, currentState.stateName, currentState.stateData) @@ -584,13 +586,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) - app.eventHandler.debug(context.self, "setting " + (if (repeat) "repeating " else "") + "timer '" + name + "'/" + timeout + ": " + msg) + log.debug("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) - app.eventHandler.debug(context.self, "canceling timer '" + name + "'") + log.debug("canceling timer '" + name + "'") super.cancelTimer(name) } @@ -602,7 +604,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒ case c: UntypedChannel ⇒ c.toString case _ ⇒ "unknown" } - app.eventHandler.debug(context.self, "processing " + event + " from " + srcstr) + log.debug("processing " + event + " from " + srcstr) } if (logDepth > 0) { @@ -616,7 +618,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒ val newState = stateName if (debugEvent && oldState != newState) - app.eventHandler.debug(context.self, "transition " + oldState + " -> " + newState) + log.debug("transition " + oldState + " -> " + newState) } /** diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index 600d3a334a..164321ade9 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -5,7 +5,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 diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index f89451b962..f3d41e138e 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -6,7 +6,7 @@ package akka.dispatch import java.util.concurrent._ import java.util.concurrent.atomic.AtomicLong -import akka.event.EventHandler +import akka.event.Logging.Error import akka.config.Configuration import akka.util.{ Duration, Switch, ReentrantGuard } import java.util.concurrent.ThreadPoolExecutor.{ AbortPolicy, CallerRunsPolicy, DiscardOldestPolicy, DiscardPolicy } @@ -66,7 +66,7 @@ final case class TaskInvocation(app: AkkaApplication, function: () ⇒ Unit, cle try { function() } catch { - case e ⇒ app.eventHandler.error(e, this, e.getMessage) + case e ⇒ app.mainbus.publish(Error(e, this, e.getMessage)) } finally { cleanup() } diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index f9819c3cd3..3e9411d593 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -4,7 +4,7 @@ package akka.dispatch -import akka.event.EventHandler +import akka.event.Logging.Warning import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue } import akka.actor.{ ActorCell, ActorKilledException } @@ -95,7 +95,7 @@ class Dispatcher( executorService.get() execute invocation } catch { case e: RejectedExecutionException ⇒ - app.eventHandler.warning(this, e.toString) + app.mainbus.publish(Warning(this, e.toString)) throw e } } @@ -122,7 +122,7 @@ class Dispatcher( } catch { case e: RejectedExecutionException ⇒ try { - app.eventHandler.warning(this, e.toString) + app.mainbus.publish(Warning(this, e.toString)) } finally { mbox.setAsIdle() } diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 922aa9cf5c..f232762687 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -6,7 +6,7 @@ package akka.dispatch import akka.AkkaException -import akka.event.EventHandler +import akka.event.Logging.Error import akka.actor.{ UntypedChannel, Timeout, ExceptionChannel } import scala.Option import akka.japi.{ Procedure, Function ⇒ JFunc, Option ⇒ JOption } @@ -262,7 +262,7 @@ object Future { result completeWithResult currentValue } catch { case e: Exception ⇒ - dispatcher.app.eventHandler.error(e, this, e.getMessage) + dispatcher.app.mainbus.publish(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 ⇒ - dispatcher.app.eventHandler.error(e, this, e.getMessage) + dispatcher.app.mainbus.publish(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 ⇒ - dispatcher.app.eventHandler.error(e, this, e.getMessage) + dispatcher.app.mainbus.publish(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 ⇒ - dispatcher.app.eventHandler.error(e, this, e.getMessage) + dispatcher.app.mainbus.publish(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 ⇒ - dispatcher.app.eventHandler.error(e, this, e.getMessage) + dispatcher.app.mainbus.publish(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 ⇒ - dispatcher.app.eventHandler.error(e, this, e.getMessage) + dispatcher.app.mainbus.publish(Error(e, this, e.getMessage)) fr completeWithException e } } @@ -985,7 +985,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 ⇒ dispatcher.app.eventHandler.error(e, this, "Future onComplete-callback raised an exception") } //TODO catch, everything? Really? + try { func(this) } catch { case e ⇒ dispatcher.app.mainbus.publish(Error(e, this, "Future onComplete-callback raised an exception")) } //TODO catch, everything? Really? } @inline diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index d03655489b..4222c6fd77 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -12,6 +12,7 @@ import akka.actor.{ ActorContext, ActorCell } import java.util.concurrent._ import atomic.{ AtomicInteger, AtomicReferenceFieldUpdater } import annotation.tailrec +import akka.event.Logging.Error class MessageQueueAppendFailedException(message: String, cause: Throwable = null) extends AkkaException(message, cause) @@ -205,7 +206,7 @@ abstract class Mailbox(val actor: ActorCell) extends AbstractMailbox with Messag } } catch { case e ⇒ - actor.app.eventHandler.error(e, this, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!") + actor.app.mainbus.publish(Error(e, this, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!")) throw e } } diff --git a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala index ac529342bd..0d579e6e1e 100644 --- a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala @@ -9,7 +9,7 @@ import java.util.concurrent._ import atomic.{ AtomicLong, AtomicInteger } import ThreadPoolExecutor.CallerRunsPolicy import akka.util.Duration -import akka.event.EventHandler +import akka.event.Logging.{ Warning, Error } import akka.AkkaApplication object ThreadPoolConfig { @@ -227,10 +227,10 @@ class BoundedExecutorDecorator(val app: AkkaApplication, val executor: ExecutorS }) } catch { case e: RejectedExecutionException ⇒ - app.eventHandler.warning(this, e.toString) + app.mainbus.publish(Warning(this, e.toString)) semaphore.release case e: Throwable ⇒ - app.eventHandler.error(e, this, e.getMessage) + app.mainbus.publish(Error(e, this, e.getMessage)) throw e } } diff --git a/akka-actor/src/main/scala/akka/event/EventBus.scala b/akka-actor/src/main/scala/akka/event/EventBus.scala index 9dd76f5344..c95b91e9ce 100644 --- a/akka-actor/src/main/scala/akka/event/EventBus.scala +++ b/akka-actor/src/main/scala/akka/event/EventBus.scala @@ -48,6 +48,7 @@ trait EventBus { */ trait ActorEventBus extends EventBus { type Subscriber = ActorRef + protected def compareSubscribers(a: ActorRef, b: ActorRef) = a compareTo b } /** @@ -254,9 +255,9 @@ trait ActorClassification { self: ActorEventBus with ActorClassifier ⇒ */ protected def mapSize: Int - def publish(event: Event): Unit = mappings.get(classify(event)) match { - case null ⇒ - case raw: Vector[_] ⇒ raw.asInstanceOf[Vector[ActorRef]] foreach { _ ! event } + def publish(event: Event): Unit = { + val receivers = mappings.get(classify(event)) + if (receivers ne null) receivers foreach { _ ! event } } def subscribe(subscriber: Subscriber, to: Classifier): Boolean = associate(to, subscriber) diff --git a/akka-actor/src/main/scala/akka/event/EventHandler.scala b/akka-actor/src/main/scala/akka/event/EventHandler.scala index 3acdf52a0b..7a4a5fe0ad 100644 --- a/akka-actor/src/main/scala/akka/event/EventHandler.scala +++ b/akka-actor/src/main/scala/akka/event/EventHandler.scala @@ -12,124 +12,6 @@ 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 _ ⇒ simpleName(instance) - } - } - - 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. *

@@ -171,126 +53,3 @@ object EventHandler { * * @author Jonas Bonér */ -class EventHandler(app: AkkaApplication) extends ListenerManagement { - - import EventHandler._ - - val synchronousLogging: Boolean = System.getProperty("akka.event.force-sync") match { - case null | "" ⇒ false - case _ ⇒ true - } - - lazy val EventHandlerDispatcher = - 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 = app.AkkaConfig.LogLevel match { - case "ERROR" | "error" ⇒ ErrorLevel - case "WARNING" | "warning" ⇒ WarningLevel - case "INFO" | "info" ⇒ InfoLevel - case "DEBUG" | "debug" ⇒ DebugLevel - case unknown ⇒ throw new ConfigurationException( - "Configuration option 'akka.event-handler-level' is invalid [" + unknown + "]") - } - - def start() { - try { - 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(app, Props(actorClass).withDispatcher(EventHandlerDispatcher), app.guardian, Props.randomAddress, systemService = true)) - case Left(exception) ⇒ throw exception - } - } catch { - case e: Exception ⇒ - throw new ConfigurationException( - "Event Handler specified in config can't be loaded [" + listenerName + - "] due to [" + e.toString + "]", e) - } - } - info(this, "Starting up EventHandler") - } catch { - case e: Exception ⇒ - System.err.println("error while starting up EventHandler") - e.printStackTrace() - throw new ConfigurationException("Could not start Event Handler due to [" + e.toString + "]") - } - } - - /** - * Shuts down all event handler listeners including the event handle dispatcher. - */ - def shutdown() { - foreachListener { l ⇒ - removeListener(l) - l.stop() - } - } - - def notify(event: Any) { - if (event.isInstanceOf[Event]) { - if (level >= event.asInstanceOf[Event].level) log(event) - } else log(event) - } - - def notify[T <: Event: ClassManifest](event: ⇒ T) { - if (level >= levelFor(classManifest[T].erasure.asInstanceOf[Class[_ <: Event]])) log(event) - } - - def error(cause: Throwable, instance: AnyRef, message: ⇒ String) { - if (level >= ErrorLevel) log(Error(cause, instance, message)) - } - - def error(cause: Throwable, instance: AnyRef, message: Any) { - if (level >= ErrorLevel) log(Error(cause, instance, message)) - } - - def error(instance: AnyRef, message: ⇒ String) { - if (level >= ErrorLevel) log(Error(new EventHandlerException, instance, message)) - } - - def error(instance: AnyRef, message: Any) { - if (level >= ErrorLevel) log(Error(new EventHandlerException, instance, message)) - } - - def warning(instance: AnyRef, message: ⇒ String) { - if (level >= WarningLevel) log(Warning(instance, message)) - } - - def warning(instance: AnyRef, message: Any) { - if (level >= WarningLevel) log(Warning(instance, message)) - } - - def info(instance: AnyRef, message: ⇒ String) { - if (level >= InfoLevel) log(Info(instance, message)) - } - - def info(instance: AnyRef, message: Any) { - if (level >= InfoLevel) log(Info(instance, message)) - } - - def debug(instance: AnyRef, message: ⇒ String) { - if (level >= DebugLevel) log(Debug(instance, message)) - } - - def debug(instance: AnyRef, message: Any) { - if (level >= DebugLevel) log(Debug(instance, message)) - } - - def isInfoEnabled = level >= InfoLevel - - def isDebugEnabled = level >= DebugLevel - - private def log(event: Any) { - if (synchronousLogging) StandardOutLogger.print(event) - else notifyListeners(event) - } - - start() -} diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index 12ead94967..59932879f1 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -2,7 +2,258 @@ * Copyright (C) 2009-2011 Typesafe Inc. */ package akka.event -import akka.actor.Actor + +import akka.actor.{ Actor, ActorRef, MinimalActorRef, LocalActorRef, Props, UntypedChannel } +import akka.{ AkkaException, AkkaApplication } +import akka.AkkaApplication.AkkaConfig +import akka.util.ReflectiveAccess +import akka.config.ConfigurationException + +/** + * This trait brings log level handling to the MainBus: it reads the log + * levels for the initial logging (StandardOutLogger) and the loggers&level + * for after-init logging, possibly keeping the StandardOutLogger enabled if + * it is part of the configured loggers. All configured loggers are treated as + * system services and managed by this trait, i.e. subscribed/unsubscribed in + * response to changes of LoggingBus.logLevel. + */ +trait LoggingBus extends ActorEventBus { + + type Event >: Logging.LogEvent + type Classifier >: Class[_] + + import Logging._ + + private var loggers = Seq.empty[ActorRef] + @volatile + private var _logLevel: LogLevel = _ + + def logLevel = _logLevel + + def logLevel_=(level: LogLevel) { + for { l ← AllLogLevels if l > _logLevel && l <= level; log ← loggers } subscribe(log, classFor(l)) + for { l ← AllLogLevels if l <= _logLevel && l > level; log ← loggers } unsubscribe(log, classFor(l)) + _logLevel = level + } + + def startStdoutLogger(config: AkkaConfig) { + val level = levelFor(config.StdoutLogLevel) getOrElse { + StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.StdoutLogLevel)) + ErrorLevel + } + AllLogLevels filter (level >= _) foreach (l ⇒ subscribe(StandardOutLogger, classFor(l))) + loggers = Seq(StandardOutLogger) + _logLevel = level + publish(Info(this, "StandardOutLogger started")) + } + + def startDefaultLoggers(app: AkkaApplication, config: AkkaConfig) { + val level = levelFor(config.LogLevel) getOrElse { + StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.LogLevel)) + ErrorLevel + } + try { + val defaultLoggers = config.EventHandlers match { + case Nil ⇒ "akka.event.Logging$DefaultLogger" :: Nil + case loggers ⇒ loggers + } + loggers = for { + loggerName ← defaultLoggers + if loggerName != DefaultLoggerName + } yield { + try { + ReflectiveAccess.getClassFor[Actor](loggerName) match { + case Right(actorClass) ⇒ addLogger(app, actorClass, level) + case Left(exception) ⇒ throw exception + } + } catch { + case e: Exception ⇒ + throw new ConfigurationException( + "Event Handler specified in config can't be loaded [" + loggerName + + "] due to [" + e.toString + "]", e) + } + } + publish(Info(this, "Default Loggers started")) + if (defaultLoggers contains DefaultLoggerName) { + loggers :+= StandardOutLogger + } else { + unsubscribe(StandardOutLogger) + } + _logLevel = level + } catch { + case e: Exception ⇒ + System.err.println("error while starting up EventHandler") + e.printStackTrace() + throw new ConfigurationException("Could not start Event Handler due to [" + e.toString + "]") + } + } + + private def addLogger(app: AkkaApplication, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = { + val actor = app.systemActorOf(Props(clazz), Props.randomAddress) + actor ! InitializeLogger(this) + AllLogLevels filter (level >= _) foreach (l ⇒ subscribe(actor, classFor(l))) + publish(Info(this, "logger " + clazz.getName + " started")) + actor + } + +} + +object Logging { + + trait LogLevelType + type LogLevel = Int with LogLevelType + final val ErrorLevel = 1.asInstanceOf[Int with LogLevelType] + final val WarningLevel = 2.asInstanceOf[Int with LogLevelType] + final val InfoLevel = 3.asInstanceOf[Int with LogLevelType] + final val DebugLevel = 4.asInstanceOf[Int with LogLevelType] + + def levelFor(s: String): Option[LogLevel] = s match { + case "ERROR" | "error" ⇒ Some(ErrorLevel) + case "WARNING" | "warning" ⇒ Some(WarningLevel) + case "INFO" | "info" ⇒ Some(InfoLevel) + case "DEBUG" | "debug" ⇒ Some(DebugLevel) + case unknown ⇒ None + } + + def levelFor(eventClass: Class[_ <: LogEvent]) = { + 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 + } + + def classFor(level: LogLevel): Class[_ <: LogEvent] = level match { + case ErrorLevel ⇒ classOf[Error] + case WarningLevel ⇒ classOf[Warning] + case InfoLevel ⇒ classOf[Info] + case DebugLevel ⇒ classOf[Debug] + } + + val AllLogLevels = Seq(ErrorLevel: AnyRef, WarningLevel, InfoLevel, DebugLevel).asInstanceOf[Seq[LogLevel]] + + 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 + + def apply(app: AkkaApplication, instance: AnyRef): Logging = new MainBusLogging(app.mainbus, instance) + def apply(bus: MainBus, instance: AnyRef): Logging = new MainBusLogging(bus, instance) + + class EventHandlerException extends AkkaException + + sealed trait LogEvent { + @transient + val thread: Thread = Thread.currentThread + def level: LogLevel + } + + case class Error(cause: Throwable, instance: AnyRef, message: Any = "") extends LogEvent { + def level = ErrorLevel + } + object Error { + def apply(instance: AnyRef, message: Any) = new Error(new EventHandlerException, instance, message) + } + + case class Warning(instance: AnyRef, message: Any = "") extends LogEvent { + def level = WarningLevel + } + + case class Info(instance: AnyRef, message: Any = "") extends LogEvent { + def level = InfoLevel + } + + case class Debug(instance: AnyRef, message: Any = "") extends LogEvent { + def level = DebugLevel + } + + case class InitializeLogger(bus: LoggingBus) + + trait StdOutLogger { + 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 StandardOutLogger extends MinimalActorRef with StdOutLogger { + override val toString = "StandardOutLogger" + override def postMessageToMailbox(obj: Any, channel: UntypedChannel) { print(obj) } + } + val StandardOutLogger = new StandardOutLogger + val DefaultLoggerName = StandardOutLogger.getClass.getName + + class DefaultLogger extends Actor with StdOutLogger { + def receive = { + case InitializeLogger(_) ⇒ + case event: LogEvent ⇒ print(event) + } + } + + def stackTraceFor(e: Throwable) = { + if (e ne null) { + import java.io.{ StringWriter, PrintWriter } + val sw = new StringWriter + val pw = new PrintWriter(sw) + e.printStackTrace(pw) + sw.toString + } else { + "[NO STACK TRACE]" + } + } + +} /** * Logging wrapper to make nicer and optimize: provide template versions which @@ -69,40 +320,21 @@ trait Logging { } -trait ActorLogging extends Logging { this: Actor ⇒ +class MainBusLogging(val mainbus: MainBus, val loggingInstance: AnyRef) extends Logging { - import EventHandler._ + import Logging._ - def isErrorEnabled = app.eventHandler.level >= ErrorLevel - def isWarningEnabled = app.eventHandler.level >= WarningLevel - def isInfoEnabled = app.eventHandler.level >= InfoLevel - def isDebugEnabled = app.eventHandler.level >= DebugLevel + def isErrorEnabled = mainbus.logLevel >= ErrorLevel + def isWarningEnabled = mainbus.logLevel >= WarningLevel + def isInfoEnabled = mainbus.logLevel >= InfoLevel + def isDebugEnabled = mainbus.logLevel >= DebugLevel - protected def notifyError(cause: Throwable, message: String) { app.eventHandler.notifyListeners(Error(cause, context.self, message)) } + protected def notifyError(cause: Throwable, message: String) { mainbus.publish(Error(cause, loggingInstance, message)) } - protected def notifyWarning(message: String) { app.eventHandler.notifyListeners(Warning(context.self, message)) } + protected def notifyWarning(message: String) { mainbus.publish(Warning(loggingInstance, message)) } - protected def notifyInfo(message: String) { app.eventHandler.notifyListeners(Info(context.self, message)) } + protected def notifyInfo(message: String) { mainbus.publish(Info(loggingInstance, 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)) } + protected def notifyDebug(message: String) { mainbus.publish(Debug(loggingInstance, message)) } } \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/event/MainBus.scala b/akka-actor/src/main/scala/akka/event/MainBus.scala new file mode 100644 index 0000000000..ffaa41ca5a --- /dev/null +++ b/akka-actor/src/main/scala/akka/event/MainBus.scala @@ -0,0 +1,56 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.event + +import akka.actor.{ ActorRef, Actor, Props } +import akka.AkkaApplication +import akka.actor.Terminated + +class MainBus(debug: Boolean = false) extends LoggingBus with LookupClassification { + + type Event = AnyRef + type Classifier = Class[_] + + @volatile + private var reaper: ActorRef = _ + + protected def mapSize = 16 + + protected def classify(event: AnyRef): Class[_] = event.getClass + + protected def publish(event: AnyRef, subscriber: ActorRef) = subscriber ! event + + override def subscribe(subscriber: ActorRef, channel: Class[_]): Boolean = { + if (debug) publish(Logging.Debug(this, "subscribing " + subscriber + " to channel " + channel)) + if (reaper ne null) reaper ! subscriber + super.subscribe(subscriber, channel) + } + + override def unsubscribe(subscriber: ActorRef, channel: Class[_]): Boolean = { + if (debug) publish(Logging.Debug(this, "unsubscribing " + subscriber + " from channel " + channel)) + super.unsubscribe(subscriber, channel) + } + + override def unsubscribe(subscriber: ActorRef) { + if (debug) publish(Logging.Debug(this, "unsubscribing " + subscriber + " from all channels")) + super.unsubscribe(subscriber) + } + + def start(app: AkkaApplication) { + reaper = app.systemActorOf(Props(new Actor { + def receive = loggable(context.self) { + case ref: ActorRef ⇒ watch(ref) + case Terminated(ref) ⇒ unsubscribe(ref) + } + }), "MainBusReaper") + subscribers.values foreach (reaper ! _) + } + + def printSubscribers: String = { + val sb = new StringBuilder + for (c ← subscribers.keys) sb.append(c + " -> " + subscribers.valueIterator(c).mkString("[", ", ", "]")) + sb.toString + } + +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/remote/RemoteInterface.scala b/akka-actor/src/main/scala/akka/remote/RemoteInterface.scala index c1050c7842..4650f945ba 100644 --- a/akka-actor/src/main/scala/akka/remote/RemoteInterface.scala +++ b/akka-actor/src/main/scala/akka/remote/RemoteInterface.scala @@ -30,7 +30,7 @@ trait RemoteModule { val UUID_PREFIX = "uuid:".intern def optimizeLocalScoped_?(): Boolean //Apply optimizations for remote operations in local scope - protected[akka] def notifyListeners(message: ⇒ Any): Unit + protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit private[akka] def actors: ConcurrentHashMap[String, ActorRef] // FIXME need to invalidate this cache on replication private[akka] def actorsByUuid: ConcurrentHashMap[String, ActorRef] // FIXME remove actorsByUuid map? @@ -193,7 +193,7 @@ abstract class RemoteSupport(val app: AkkaApplication) extends RemoteServerModul clear } - protected[akka] override def notifyListeners(message: ⇒ Any): Unit = app.eventHandler.notify(message) + protected[akka] override def notifyListeners(message: RemoteLifeCycleEvent): Unit = app.mainbus.publish(message) private[akka] val actors = new ConcurrentHashMap[String, ActorRef] private[akka] val actorsByUuid = new ConcurrentHashMap[String, ActorRef] diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index 77ce3de642..29d30ef489 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -6,7 +6,6 @@ package akka.routing import akka.AkkaException import akka.actor._ -import akka.event.EventHandler import akka.config.ConfigurationException import akka.dispatch.{ Future, MessageDispatcher } import akka.AkkaApplication diff --git a/akka-actor/src/main/scala/akka/util/Index.scala b/akka-actor/src/main/scala/akka/util/Index.scala index afbb7a2c20..f79ea921dd 100644 --- a/akka-actor/src/main/scala/akka/util/Index.scala +++ b/akka-actor/src/main/scala/akka/util/Index.scala @@ -8,6 +8,7 @@ import annotation.tailrec import java.util.concurrent.{ ConcurrentSkipListSet, ConcurrentHashMap } import java.util.{ Comparator, Set ⇒ JSet } +import scala.collection.mutable /** * An implementation of a ConcurrentMultiMap @@ -98,6 +99,24 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) { container.entrySet foreach { e ⇒ e.getValue.foreach(fun(e.getKey, _)) } } + /** + * Returns the value set. + */ + def values = { + import scala.collection.JavaConversions._ + val builder = mutable.Set.empty[V] + for { + entry ← container.entrySet + v ← entry.getValue + } builder += v + builder.toSet + } + + /** + * Returns the key set. + */ + def keys = scala.collection.JavaConversions.asScalaIterable(container.keySet) + /** * Disassociates the value of type V from the key of type K * @return true if the value was disassociated from the key and false if it wasn't previously associated with the key diff --git a/akka-actor/src/main/scala/akka/util/JMX.scala b/akka-actor/src/main/scala/akka/util/JMX.scala index d4e287c2e3..ad735ae39b 100644 --- a/akka-actor/src/main/scala/akka/util/JMX.scala +++ b/akka-actor/src/main/scala/akka/util/JMX.scala @@ -4,7 +4,7 @@ package akka.util -import akka.event.EventHandler +import akka.event.Logging.Error import java.lang.management.ManagementFactory import javax.management.{ ObjectInstance, ObjectName, InstanceAlreadyExistsException, InstanceNotFoundException } import akka.AkkaApplication @@ -24,7 +24,7 @@ object JMX { case e: InstanceAlreadyExistsException ⇒ Some(mbeanServer.getObjectInstance(name)) case e: Exception ⇒ - app.eventHandler.error(e, this, "Error when registering mbean [%s]".format(mbean)) + app.mainbus.publish(Error(e, this, "Error when registering mbean [%s]".format(mbean))) None } @@ -32,6 +32,6 @@ object JMX { mbeanServer.unregisterMBean(mbean) } catch { case e: InstanceNotFoundException ⇒ {} - case e: Exception ⇒ app.eventHandler.error(e, this, "Error while unregistering mbean [%s]".format(mbean)) + case e: Exception ⇒ app.mainbus.publish(Error(e, this, "Error while unregistering mbean [%s]".format(mbean))) } } diff --git a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala index a1e05cc819..4bacf85636 100644 --- a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala +++ b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala @@ -3,12 +3,13 @@ */ package akka.util + import akka.dispatch.Envelope import akka.config.ModuleNotAvailableException import akka.actor._ import DeploymentConfig.ReplicationScheme import akka.config.ModuleNotAvailableException -import akka.event.EventHandler +import akka.event.Logging.Debug import akka.cluster.ClusterNode import akka.remote.{ RemoteSupport, RemoteService } import akka.routing.{ RoutedProps, Router } @@ -167,7 +168,7 @@ class ReflectiveAccess(val app: 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") - app.eventHandler.debug(this, e.toString) + app.mainbus.publish(Debug(this, e.toString)) throw e } } @@ -175,21 +176,21 @@ class ReflectiveAccess(val app: AkkaApplication) { lazy val clusterInstance: Option[Cluster] = getObjectFor("akka.cluster.Cluster$") match { case Right(value) ⇒ Some(value) case Left(exception) ⇒ - app.eventHandler.debug(this, exception.toString) + app.mainbus.publish(Debug(this, exception.toString)) None } lazy val clusterDeployerInstance: Option[ActorDeployer] = getObjectFor("akka.cluster.ClusterDeployer$") match { case Right(value) ⇒ Some(value) case Left(exception) ⇒ - app.eventHandler.debug(this, exception.toString) + app.mainbus.publish(Debug(this, exception.toString)) None } lazy val transactionLogInstance: Option[TransactionLogObject] = getObjectFor("akka.cluster.TransactionLog$") match { case Right(value) ⇒ Some(value) case Left(exception) ⇒ - app.eventHandler.debug(this, exception.toString) + app.mainbus.publish(Debug(this, exception.toString)) None } diff --git a/akka-docs/scala/code/ActorDocSpec.scala b/akka-docs/scala/code/ActorDocSpec.scala index a8998d7691..72e7adad73 100644 --- a/akka-docs/scala/code/ActorDocSpec.scala +++ b/akka-docs/scala/code/ActorDocSpec.scala @@ -7,15 +7,16 @@ import akka.util.duration._ //#imports import akka.actor.Actor -import akka.event.EventHandler +import akka.event.Logging //#imports //#my-actor class MyActor extends Actor { + val log = Logging(app, this) def receive = { - case "test" ⇒ app.eventHandler.info(this, "received test") - case _ ⇒ app.eventHandler.info(this, "received unknown message") + case "test" ⇒ log.info("received test") + case _ ⇒ log.info("received unknown message") } } //#my-actor @@ -30,23 +31,21 @@ class ActorDocSpec extends AkkaSpec { // testing the actor // TODO: convert docs to AkkaSpec(Configuration(...)) - app.eventHandler.notify(TestEvent.Mute(EventFilter.custom { - case e: EventHandler.Info ⇒ true - case _ ⇒ false - })) - app.eventHandler.addListener(testActor) - val eventLevel = app.eventHandler.level - app.eventHandler.level = EventHandler.InfoLevel + val filter = EventFilter.custom { + case e: Logging.Info ⇒ true + case _ ⇒ false + } + app.mainbus.publish(TestEvent.Mute(filter)) + app.mainbus.subscribe(testActor, classOf[Logging.Info]) myActor ! "test" - expectMsgPF(1 second) { case EventHandler.Info(_, "received test") ⇒ true } + expectMsgPF(1 second) { case Logging.Info(_, "received test") ⇒ true } myActor ! "unknown" - expectMsgPF(1 second) { case EventHandler.Info(_, "received unknown message") ⇒ true } + expectMsgPF(1 second) { case Logging.Info(_, "received unknown message") ⇒ true } - app.eventHandler.level = eventLevel - app.eventHandler.removeListener(testActor) - app.eventHandler.notify(TestEvent.UnMuteAll) + app.mainbus.unsubscribe(testActor) + app.mainbus.publish(TestEvent.UnMute(filter)) myActor.stop() } diff --git a/akka-http/src/main/scala/akka/http/Mist.scala b/akka-http/src/main/scala/akka/http/Mist.scala index a079ee8ecd..7f71faa1f8 100644 --- a/akka-http/src/main/scala/akka/http/Mist.scala +++ b/akka-http/src/main/scala/akka/http/Mist.scala @@ -4,7 +4,7 @@ package akka.http -import akka.event.EventHandler +import akka.event.Logging import akka.config.ConfigurationException import javax.servlet.http.{ HttpServletResponse, HttpServletRequest } import javax.servlet.http.HttpServlet @@ -360,7 +360,7 @@ trait RequestMethod { } } catch { case io: Exception ⇒ - app.eventHandler.error(io, this, io.getMessage) + app.mainbus.publish(Logging.Error(io, this, io.getMessage)) false } case None ⇒ false @@ -376,7 +376,7 @@ trait RequestMethod { } } catch { case io: IOException ⇒ - app.eventHandler.error(io, this, io.getMessage) + app.mainbus.publish(Logging.Error(io, this, io.getMessage)) } case None ⇒ {} } diff --git a/akka-http/src/main/scala/akka/http/Servlet30Context.scala b/akka-http/src/main/scala/akka/http/Servlet30Context.scala index fd797b8a5c..e70ce0d360 100644 --- a/akka-http/src/main/scala/akka/http/Servlet30Context.scala +++ b/akka-http/src/main/scala/akka/http/Servlet30Context.scala @@ -7,6 +7,7 @@ package akka.http import javax.servlet.{ AsyncContext, AsyncListener, AsyncEvent } import Types._ import akka.AkkaApplication +import akka.event.Logging /** * @author Garrick Evans @@ -35,7 +36,7 @@ trait Servlet30Context extends AsyncListener { true } catch { case e: IllegalStateException ⇒ - app.eventHandler.error(e, this, e.getMessage) + app.mainbus.publish(Logging.Error(e, this, e.getMessage)) false } } @@ -46,7 +47,7 @@ trait Servlet30Context extends AsyncListener { def onComplete(e: AsyncEvent) {} def onError(e: AsyncEvent) = e.getThrowable match { case null ⇒ - case t ⇒ app.eventHandler.error(t, this, t.getMessage) + case t ⇒ app.mainbus.publish(Logging.Error(t, this, t.getMessage)) } def onStartAsync(e: AsyncEvent) {} def onTimeout(e: AsyncEvent) = { diff --git a/akka-remote/src/main/scala/akka/remote/Remote.scala b/akka-remote/src/main/scala/akka/remote/Remote.scala index 550b1a20c7..e21c4f2ef5 100644 --- a/akka-remote/src/main/scala/akka/remote/Remote.scala +++ b/akka-remote/src/main/scala/akka/remote/Remote.scala @@ -6,7 +6,7 @@ package akka.remote import akka.AkkaApplication import akka.actor._ -import akka.event.EventHandler +import akka.event.Logging import akka.dispatch.{ Dispatchers, Future, PinnedDispatcher } import akka.actor.Status._ import akka.util._ @@ -29,6 +29,8 @@ import com.eaio.uuid.UUID */ class Remote(val app: AkkaApplication) extends RemoteService { + val log = Logging(app, this) + import app._ import app.config import app.AkkaConfig._ @@ -75,8 +77,8 @@ class Remote(val app: AkkaApplication) extends RemoteService { val remote = new akka.remote.netty.NettyRemoteSupport(app) remote.start(hostname, port) remote.register(remoteDaemonServiceName, remoteDaemon) - app.eventHandler.addListener(eventStream.channel) - app.eventHandler.addListener(remoteClientLifeCycleHandler) + app.mainbus.subscribe(eventStream.channel, classOf[RemoteLifeCycleEvent]) + app.mainbus.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent]) // TODO actually register this provider in app in remote mode //provider.register(ActorRefProvider.RemoteProvider, new RemoteActorRefProvider) remote @@ -86,7 +88,7 @@ class Remote(val app: AkkaApplication) extends RemoteService { def start() { val triggerLazyServerVal = address.toString - eventHandler.info(this, "Starting remote server on [%s]".format(triggerLazyServerVal)) + log.info("Starting remote server on [{}]", triggerLazyServerVal) } def uuidProtocolToUuid(uuid: UuidProtocol): UUID = new UUID(uuid.getHigh, uuid.getLow) @@ -108,16 +110,14 @@ class Remote(val app: AkkaApplication) extends RemoteService { class RemoteSystemDaemon(remote: Remote) extends Actor { import remote._ - import remote.app._ override def preRestart(reason: Throwable, msg: Option[Any]) { - eventHandler.debug(this, "RemoteSystemDaemon failed due to [%s] - restarting...".format(reason)) + log.debug("RemoteSystemDaemon failed due to [{}] - restarting...", reason) } def receive: Actor.Receive = { case message: RemoteSystemDaemonMessageProtocol ⇒ - eventHandler.debug(this, - "Received command [\n%s] to RemoteSystemDaemon on [%s]".format(message, nodename)) + log.debug("Received command [\n{}] to RemoteSystemDaemon on [{}]", message, app.nodename) message.getMessageType match { case USE ⇒ handleUse(message) @@ -135,7 +135,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { //TODO: should we not deal with unrecognized message types? } - case unknown ⇒ eventHandler.warning(this, "Unknown message to RemoteSystemDaemon [%s]".format(unknown)) + case unknown ⇒ log.warning("Unknown message to RemoteSystemDaemon [{}]", unknown) } def handleUse(message: RemoteSystemDaemonMessageProtocol) { @@ -147,7 +147,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { else message.getPayload.toByteArray val actorFactory = - serialization.deserialize(actorFactoryBytes, classOf[() ⇒ Actor], None) match { + app.serialization.deserialize(actorFactoryBytes, classOf[() ⇒ Actor], None) match { case Left(error) ⇒ throw error case Right(instance) ⇒ instance.asInstanceOf[() ⇒ Actor] } @@ -158,7 +158,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { server.register(actorAddress, newActorRef) } else { - eventHandler.error(this, "Actor 'address' for actor to instantiate is not defined, ignoring remote system daemon command [%s]".format(message)) + log.error("Actor 'address' for actor to instantiate is not defined, ignoring remote system daemon command [{}]", message) } channel ! Success(address.toString) @@ -232,7 +232,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { } private def payloadFor[T](message: RemoteSystemDaemonMessageProtocol, clazz: Class[T]): T = { - serialization.deserialize(message.getPayload.toByteArray, clazz, None) match { + app.serialization.deserialize(message.getPayload.toByteArray, clazz, None) match { case Left(error) ⇒ throw error case Right(instance) ⇒ instance.asInstanceOf[T] } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index b87eae50aa..1092bd6875 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -12,7 +12,7 @@ import akka.routing._ import akka.dispatch._ import akka.util.duration._ import akka.config.ConfigurationException -import akka.event.{ DeathWatch, EventHandler } +import akka.event.{ DeathWatch, Logging } import akka.serialization.{ Serialization, Serializer, ActorSerialization, Compression } import akka.serialization.Compression.LZF import akka.remote.RemoteProtocol._ @@ -30,6 +30,8 @@ import com.google.protobuf.ByteString */ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider { + val log = Logging(app, this) + import java.util.concurrent.ConcurrentHashMap import akka.dispatch.Promise @@ -167,7 +169,7 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider * Using (checking out) actor on a specific node. */ def useActorOnNode(remoteAddress: InetSocketAddress, actorAddress: String, actorFactory: () ⇒ Actor) { - app.eventHandler.debug(this, "Instantiating Actor [%s] on node [%s]".format(actorAddress, remoteAddress)) + log.debug("Instantiating Actor [{}] on node [{}]", actorAddress, remoteAddress) val actorFactoryBytes = app.serialization.serialize(actorFactory) match { @@ -198,20 +200,20 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider try { (connection ? (command, remote.remoteSystemDaemonAckTimeout)).as[Status] match { case Some(Success(receiver)) ⇒ - app.eventHandler.debug(this, "Remote system command sent to [%s] successfully received".format(receiver)) + log.debug("Remote system command sent to [{}] successfully received", receiver) case Some(Failure(cause)) ⇒ - app.eventHandler.error(cause, this, cause.toString) + log.error(cause, cause.toString) throw cause case None ⇒ val error = new RemoteException("Remote system command to [%s] timed out".format(connection.address)) - app.eventHandler.error(error, this, error.toString) + log.error(error, error.toString) throw error } } catch { case e: Exception ⇒ - app.eventHandler.error(e, this, "Could not send remote system command to [%s] due to: %s".format(connection.address, e.toString)) + log.error(e, "Could not send remote system command to [{}] due to: {}", connection.address, e.toString) throw e } } else { diff --git a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala index f8837ec4f4..61a4e3172b 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala @@ -7,6 +7,7 @@ package akka.remote import akka.actor._ import akka.routing._ import akka.AkkaApplication +import akka.event.Logging import scala.collection.immutable.Map import scala.annotation.tailrec @@ -25,6 +26,8 @@ class RemoteConnectionManager( initialConnections: Map[InetSocketAddress, ActorRef] = Map.empty[InetSocketAddress, ActorRef]) extends ConnectionManager { + val log = Logging(app, this) + // FIXME is this VersionedIterable really needed? It is not used I think. Complicates API. See 'def connections' etc. case class State(version: Long, connections: Map[InetSocketAddress, ActorRef]) extends VersionedIterable[ActorRef] { @@ -62,7 +65,7 @@ class RemoteConnectionManager( @tailrec final def failOver(from: InetSocketAddress, to: InetSocketAddress) { - app.eventHandler.debug(this, "Failing over connection from [%s] to [%s]".format(from, to)) + log.debug("Failing over connection from [{}] to [{}]", from, to) val oldState = state.get var changed = false @@ -113,7 +116,7 @@ class RemoteConnectionManager( if (!state.compareAndSet(oldState, newState)) { remove(faultyConnection) // recur } else { - app.eventHandler.debug(this, "Removing connection [%s]".format(faultyAddress)) + log.debug("Removing connection [{}]", faultyAddress) } } } @@ -140,7 +143,7 @@ class RemoteConnectionManager( putIfAbsent(address, newConnectionFactory) // recur } else { // we succeeded - app.eventHandler.debug(this, "Adding connection [%s]".format(address)) + log.debug("Adding connection [{}]", address) newConnection // return new connection actor } } diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index d143b20013..b4194f5926 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -28,6 +28,7 @@ import akka.AkkaException import akka.AkkaApplication import akka.serialization.RemoteActorSerialization import akka.dispatch.{ Terminate, ActorPromise, DefaultPromise, Promise } +import akka.event.Logging class RemoteClientMessageBufferException(message: String, cause: Throwable = null) extends AkkaException(message, cause) { def this(msg: String) = this(msg, null); @@ -143,6 +144,8 @@ abstract class RemoteClient private[akka] ( val module: NettyRemoteClientModule, val remoteAddress: InetSocketAddress) { + val log = Logging(app, this) + val name = simpleName(this) + "@" + remoteAddress.getAddress.getHostAddress + "::" + remoteAddress.getPort @@ -155,7 +158,7 @@ abstract class RemoteClient private[akka] ( private[remote] def isRunning = runSwitch.isOn - protected def notifyListeners(msg: ⇒ Any): Unit + protected def notifyListeners(msg: RemoteLifeCycleEvent): Unit protected def currentChannel: Channel @@ -186,7 +189,7 @@ abstract class RemoteClient private[akka] ( senderFuture: Option[Promise[T]]): Option[Promise[T]] = { if (isRunning) { - app.eventHandler.debug(this, "Sending to connection [%s] message [\n%s]".format(remoteAddress, request)) + log.debug("Sending to connection [{}] message [\n{}]", remoteAddress, request) // tell if (request.getOneWay) { @@ -253,7 +256,7 @@ class ActiveRemoteClient private[akka] ( module: NettyRemoteClientModule, remoteAddress: InetSocketAddress, val loader: Option[ClassLoader] = None, - notifyListenersFun: (⇒ Any) ⇒ Unit) + notifyListenersFun: (RemoteLifeCycleEvent) ⇒ Unit) extends RemoteClient(_app, remoteSupport, module, remoteAddress) { val settings = new RemoteClientSettings(app) @@ -271,7 +274,7 @@ class ActiveRemoteClient private[akka] ( @volatile private var reconnectionTimeWindowStart = 0L - def notifyListeners(msg: ⇒ Any): Unit = notifyListenersFun(msg) + def notifyListeners(msg: RemoteLifeCycleEvent): Unit = notifyListenersFun(msg) def currentChannel = connection.getChannel @@ -293,14 +296,14 @@ class ActiveRemoteClient private[akka] ( } def attemptReconnect(): Boolean = { - app.eventHandler.debug(this, "Remote client reconnecting to [%s]".format(remoteAddress)) + log.debug("Remote client reconnecting to [{}]", 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)) - app.eventHandler.error(connection.getCause, this, "Reconnection to [%s] has failed".format(remoteAddress)) + log.error(connection.getCause, "Reconnection to [{}] has failed", remoteAddress) false } else { @@ -318,7 +321,7 @@ class ActiveRemoteClient private[akka] ( bootstrap.setOption("tcpNoDelay", true) bootstrap.setOption("keepAlive", true) - app.eventHandler.debug(this, "Starting remote client connection to [%s]".format(remoteAddress)) + log.debug("Starting remote client connection to [{}]", remoteAddress) connection = bootstrap.connect(remoteAddress) @@ -327,7 +330,7 @@ class ActiveRemoteClient private[akka] ( if (!connection.isSuccess) { notifyListeners(RemoteClientError(connection.getCause, module, remoteAddress)) - app.eventHandler.error(connection.getCause, this, "Remote client connection to [%s] has failed".format(remoteAddress)) + log.error(connection.getCause, "Remote client connection to [{}] has failed", remoteAddress) false } else { @@ -354,7 +357,7 @@ class ActiveRemoteClient private[akka] ( case false if reconnectIfAlreadyConnected ⇒ closeChannel(connection) - app.eventHandler.debug(this, "Remote client reconnecting to [%s]".format(remoteAddress)) + log.debug("Remote client reconnecting to [{}]", remoteAddress) attemptReconnect() case false ⇒ false @@ -363,7 +366,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 { - app.eventHandler.info(this, "Shutting down remote client [%s]".format(name)) + log.info("Shutting down remote client [{}]", name) notifyListeners(RemoteClientShutdown(module, remoteAddress)) timer.stop() @@ -374,7 +377,7 @@ class ActiveRemoteClient private[akka] ( bootstrap = null connection = null - app.eventHandler.info(this, "[%s] has been shut down".format(name)) + log.info("[{}] has been shut down", name) } private[akka] def isWithinReconnectionTimeWindow: Boolean = { @@ -384,7 +387,7 @@ class ActiveRemoteClient private[akka] ( } else { val timeLeft = (RECONNECTION_TIME_WINDOW - (System.currentTimeMillis - reconnectionTimeWindowStart)) > 0 if (timeLeft) { - app.eventHandler.info(this, "Will try to reconnect to remote server for another [%s] milliseconds".format(timeLeft)) + log.info("Will try to reconnect to remote server for another [{}] milliseconds", timeLeft) } timeLeft } @@ -435,6 +438,8 @@ class ActiveRemoteClientHandler( val client: ActiveRemoteClient) extends SimpleChannelUpstreamHandler { + val log = Logging(app, this) + implicit def _app = app override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) { @@ -451,7 +456,7 @@ class ActiveRemoteClientHandler( case arp: AkkaRemoteProtocol if arp.hasMessage ⇒ val reply = arp.getMessage val replyUuid = uuidFrom(reply.getActorInfo.getUuid.getHigh, reply.getActorInfo.getUuid.getLow) - app.eventHandler.debug(this, "Remote client received RemoteMessageProtocol[\n%s]\nTrying to map back to future [%s]".format(reply, replyUuid)) + log.debug("Remote client received RemoteMessageProtocol[\n{}]\nTrying to map back to future [{}]", reply, replyUuid) futures.remove(replyUuid).asInstanceOf[Promise[Any]] match { case null ⇒ @@ -473,7 +478,7 @@ class ActiveRemoteClientHandler( } } catch { case e: Exception ⇒ - app.eventHandler.error(e, this, e.getMessage) + log.error(e, e.getMessage) client.notifyListeners(RemoteClientError(e, client.module, client.remoteAddress)) } } @@ -496,24 +501,24 @@ class ActiveRemoteClientHandler( override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { try { client.notifyListeners(RemoteClientConnected(client.module, client.remoteAddress)) - app.eventHandler.debug(this, "Remote client connected to [%s]".format(ctx.getChannel.getRemoteAddress)) + log.debug("Remote client connected to [{}]", ctx.getChannel.getRemoteAddress) client.resetReconnectionTimeWindow } catch { case e: Exception ⇒ - app.eventHandler.error(e, this, e.getMessage) + log.error(e, e.getMessage) client.notifyListeners(RemoteClientError(e, client.module, client.remoteAddress)) } } override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { client.notifyListeners(RemoteClientDisconnected(client.module, client.remoteAddress)) - app.eventHandler.debug(this, "Remote client disconnected from [%s]".format(ctx.getChannel.getRemoteAddress)) + log.debug("Remote client disconnected from [{}]", ctx.getChannel.getRemoteAddress) } override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { val cause = event.getCause if (cause ne null) { - app.eventHandler.error(event.getCause, this, "Unexpected exception [%s] from downstream in remote client [%s]".format(event.getCause, event)) + log.error(event.getCause, "Unexpected exception [{}] from downstream in remote client [{}]", event.getCause, event) cause match { case e: ReadTimeoutException ⇒ @@ -525,7 +530,7 @@ class ActiveRemoteClientHandler( event.getChannel.close //FIXME Is this the correct behavior? } - } else app.eventHandler.error(this, "Unexpected exception from downstream in remote client [%s]".format(event)) + } else log.error("Unexpected exception from downstream in remote client [{}]", event) } private def parseException(reply: RemoteMessageProtocol, loader: Option[ClassLoader]): Throwable = { @@ -540,7 +545,7 @@ class ActiveRemoteClientHandler( .newInstance(exception.getMessage).asInstanceOf[Throwable] } catch { case problem: Exception ⇒ - app.eventHandler.error(problem, this, problem.getMessage) + log.error(problem, problem.getMessage) CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorException(problem, classname, exception.getMessage) } } @@ -575,15 +580,15 @@ class NettyRemoteSupport(_app: AkkaApplication) extends RemoteSupport(_app) with } val remoteInetSocketAddress = new InetSocketAddress(host, port) - app.eventHandler.debug(this, - "Creating RemoteActorRef with address [%s] connected to [%s]" - .format(actorAddress, remoteInetSocketAddress)) + log.debug("Creating RemoteActorRef with address [{}] connected to [{}]", actorAddress, remoteInetSocketAddress) RemoteActorRef(this, remoteInetSocketAddress, actorAddress, loader) } } class NettyRemoteServer(app: AkkaApplication, serverModule: NettyRemoteServerModule, val host: String, val port: Int, val loader: Option[ClassLoader]) { + val log = Logging(app, this) + val settings = new RemoteServerSettings(app) import settings._ @@ -618,7 +623,7 @@ class NettyRemoteServer(app: AkkaApplication, serverModule: NettyRemoteServerMod serverModule.notifyListeners(RemoteServerStarted(serverModule)) def shutdown() { - app.eventHandler.info(this, "Shutting down remote server [%s]".format(name)) + log.info("Shutting down remote server [{}]", name) try { val shutdownSignal = { val b = RemoteControlProtocol.newBuilder.setCommandType(CommandType.SHUTDOWN) @@ -634,7 +639,7 @@ class NettyRemoteServer(app: AkkaApplication, serverModule: NettyRemoteServerMod serverModule.notifyListeners(RemoteServerShutdown(serverModule)) } catch { case e: Exception ⇒ - app.eventHandler.error(e, this, e.getMessage) + log.error(e, e.getMessage) } } } @@ -642,6 +647,8 @@ class NettyRemoteServer(app: AkkaApplication, serverModule: NettyRemoteServerMod trait NettyRemoteServerModule extends RemoteServerModule { self: RemoteSupport ⇒ + val log = Logging(app, this) + def app: AkkaApplication def remoteSupport = self @@ -666,13 +673,13 @@ trait NettyRemoteServerModule extends RemoteServerModule { def start(_hostname: String, _port: Int, loader: Option[ClassLoader] = None): RemoteServerModule = guard withGuard { try { _isRunning switchOn { - app.eventHandler.debug(this, "Starting up remote server on %s:s".format(_hostname, _port)) + log.debug("Starting up remote server on {}:{}", _hostname, _port) currentServer.set(Some(new NettyRemoteServer(app, this, _hostname, _port, loader))) } } catch { case e: Exception ⇒ - app.eventHandler.error(e, this, e.getMessage) + log.error(e, e.getMessage) notifyListeners(RemoteServerError(e, this)) } this @@ -681,7 +688,7 @@ trait NettyRemoteServerModule extends RemoteServerModule { def shutdownServerModule() = guard withGuard { _isRunning switchOff { currentServer.getAndSet(None) foreach { instance ⇒ - app.eventHandler.debug(this, "Shutting down remote server on %s:%s".format(instance.host, instance.port)) + log.debug("Shutting down remote server on {}:{}", instance.host, instance.port) instance.shutdown() } } @@ -726,7 +733,7 @@ trait NettyRemoteServerModule extends RemoteServerModule { def unregister(actorRef: ActorRef): Unit = guard withGuard { if (_isRunning.isOn) { - app.eventHandler.debug(this, "Unregister server side remote actor with id [%s]".format(actorRef.uuid)) + log.debug("Unregister server side remote actor with id [{}]", actorRef.uuid) actors.remove(actorRef.address, actorRef) actorsByUuid.remove(actorRef.uuid.toString, actorRef) @@ -741,7 +748,7 @@ trait NettyRemoteServerModule extends RemoteServerModule { def unregister(id: String): Unit = guard withGuard { if (_isRunning.isOn) { - app.eventHandler.debug(this, "Unregister server side remote actor with id [%s]".format(id)) + log.debug("Unregister server side remote actor with id [{}]", id) if (id.startsWith(UUID_PREFIX)) actorsByUuid.remove(id.substring(UUID_PREFIX.length)) else { @@ -760,7 +767,7 @@ trait NettyRemoteServerModule extends RemoteServerModule { def unregisterPerSession(id: String) { if (_isRunning.isOn) { - app.eventHandler.info(this, "Unregistering server side remote actor with id [%s]".format(id)) + log.info("Unregistering server side remote actor with id [{}]", id) actorsFactories.remove(id) } @@ -832,6 +839,8 @@ class RemoteServerHandler( val applicationLoader: Option[ClassLoader], val server: NettyRemoteServerModule) extends SimpleChannelUpstreamHandler { + val log = Logging(app, this) + import settings._ implicit def app = server.app @@ -866,7 +875,7 @@ class RemoteServerHandler( override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { val clientAddress = getClientAddress(ctx) - app.eventHandler.debug(this, "Remote client [%s] connected to [%s]".format(clientAddress, server.name)) + log.debug("Remote client [{}] connected to [{}]", clientAddress, server.name) sessionActors.set(event.getChannel(), new ConcurrentHashMap[String, ActorRef]()) server.notifyListeners(RemoteServerClientConnected(server, clientAddress)) @@ -875,7 +884,7 @@ class RemoteServerHandler( override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { val clientAddress = getClientAddress(ctx) - app.eventHandler.debug(this, "Remote client [%s] disconnected from [%s]".format(clientAddress, server.name)) + log.debug("Remote client [{}] disconnected from [{}]", clientAddress, server.name) // stop all session actors for ( @@ -885,7 +894,7 @@ class RemoteServerHandler( try { actor ! PoisonPill } catch { - case e: Exception ⇒ app.eventHandler.error(e, this, "Couldn't stop %s".format(actor)) + case e: Exception ⇒ log.error(e, "Couldn't stop {}", actor) } } @@ -894,7 +903,7 @@ class RemoteServerHandler( override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { val clientAddress = getClientAddress(ctx) - app.eventHandler.debug("Remote client [%s] channel closed from [%s]".format(clientAddress, server.name), this) + log.debug("Remote client [{}] channel closed from [{}]", clientAddress, server.name) server.notifyListeners(RemoteServerClientClosed(server, clientAddress)) } @@ -914,7 +923,7 @@ class RemoteServerHandler( } override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { - app.eventHandler.error(event.getCause, this, "Unexpected exception from remote downstream") + log.error(event.getCause, "Unexpected exception from remote downstream") event.getChannel.close server.notifyListeners(RemoteServerError(event.getCause, server)) @@ -927,25 +936,25 @@ class RemoteServerHandler( } private def handleRemoteMessageProtocol(request: RemoteMessageProtocol, channel: Channel) = try { - app.eventHandler.debug(this, "Received remote message [%s]".format(request)) + log.debug("Received remote message [{}]", request) dispatchToActor(request, channel) } catch { case e: Exception ⇒ server.notifyListeners(RemoteServerError(e, server)) - app.eventHandler.error(e, this, e.getMessage) + log.error(e, e.getMessage) } private def dispatchToActor(request: RemoteMessageProtocol, channel: Channel) { val actorInfo = request.getActorInfo - app.eventHandler.debug(this, "Dispatching to remote actor [%s]".format(actorInfo.getUuid)) + log.debug("Dispatching to remote actor [{}]", actorInfo.getUuid) val actorRef = try { actorOf(actorInfo, channel) } catch { case e: SecurityException ⇒ - app.eventHandler.error(e, this, e.getMessage) + log.error(e, e.getMessage) write(channel, createErrorReplyMessage(e, request)) server.notifyListeners(RemoteServerError(e, server)) return @@ -1002,9 +1011,7 @@ class RemoteServerHandler( val uuid = actorInfo.getUuid val address = actorInfo.getAddress - app.eventHandler.debug(this, - "Looking up a remotely available actor for address [%s] on node [%s]" - .format(address, app.nodename)) + log.debug("Looking up a remotely available actor for address [{}] on node [{}]", address, app.nodename) val byAddress = server.actors.get(address) // try actor-by-address if (byAddress eq null) { diff --git a/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala b/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala index c4bcfce6ab..378255c199 100644 --- a/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala +++ b/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala @@ -8,7 +8,7 @@ import akka.actor._ import akka.actor.DeploymentConfig._ import akka.dispatch.Envelope import akka.util.{ ReflectiveAccess, Duration } -import akka.event.EventHandler +import akka.event.Logging import akka.remote._ import RemoteProtocol._ import akka.AkkaApplication @@ -28,6 +28,8 @@ import com.eaio.uuid.UUID class ActorSerialization(val app: AkkaApplication, remote: RemoteSupport) { implicit val defaultSerializer = akka.serialization.JavaSerializer // Format.Default + val log = Logging(app, this) + val remoteActorSerialization = new RemoteActorSerialization(app, remote) def fromBinary[T <: Actor](bytes: Array[Byte], homeAddress: InetSocketAddress): ActorRef = @@ -144,7 +146,7 @@ class ActorSerialization(val app: AkkaApplication, remote: RemoteSupport) { overriddenUuid: Option[UUID], loader: Option[ClassLoader]): ActorRef = { - app.eventHandler.debug(this, "Deserializing SerializedActorRefProtocol to LocalActorRef:\n%s".format(protocol)) + log.debug("Deserializing SerializedActorRefProtocol to LocalActorRef:\n{}", protocol) // import ReplicationStorageType._ // import ReplicationStrategyType._ @@ -223,6 +225,8 @@ class ActorSerialization(val app: AkkaApplication, remote: RemoteSupport) { class RemoteActorSerialization(val app: AkkaApplication, remote: RemoteSupport) { + val log = Logging(app, this) + /** * Deserializes a byte array (Array[Byte]) into an RemoteActorRef instance. */ @@ -239,7 +243,7 @@ class RemoteActorSerialization(val app: AkkaApplication, remote: RemoteSupport) * Deserializes a RemoteActorRefProtocol Protocol Buffers (protobuf) Message into an RemoteActorRef instance. */ private[akka] def fromProtobufToRemoteActorRef(protocol: RemoteActorRefProtocol, loader: Option[ClassLoader]): ActorRef = { - app.eventHandler.debug(this, "Deserializing RemoteActorRefProtocol to RemoteActorRef:\n %s".format(protocol)) + log.debug("Deserializing RemoteActorRefProtocol to RemoteActorRef:\n{}", protocol) val ref = RemoteActorRef( remote, @@ -247,7 +251,7 @@ class RemoteActorSerialization(val app: AkkaApplication, remote: RemoteSupport) protocol.getAddress, loader) - app.eventHandler.debug(this, "Newly deserialized RemoteActorRef has uuid: %s".format(ref.uuid)) + log.debug("Newly deserialized RemoteActorRef has uuid: {}", ref.uuid) ref } @@ -266,7 +270,7 @@ class RemoteActorSerialization(val app: AkkaApplication, remote: RemoteSupport) app.defaultAddress } - app.eventHandler.debug(this, "Register serialized Actor [%s] as remote @ [%s]".format(actor.uuid, remoteAddress)) + log.debug("Register serialized Actor [{}] as remote @ [{}]", actor.uuid, remoteAddress) RemoteActorRefProtocol.newBuilder .setInetSocketAddress(ByteString.copyFrom(JavaSerializer.toBinary(remoteAddress))) diff --git a/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala b/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala index 4db40e6c28..0ffd1b8490 100644 --- a/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala +++ b/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala @@ -6,7 +6,7 @@ package akka.event.slf4j import org.slf4j.{ Logger ⇒ SLFLogger, LoggerFactory ⇒ SLFLoggerFactory } -import akka.event.EventHandler +import akka.event.Logging._ import akka.actor._ import Actor._ @@ -32,7 +32,6 @@ object Logger { * @author Jonas Bonér */ class Slf4jEventHandler extends Actor with Logging { - import EventHandler._ def receive = { case event @ Error(cause, instance, message) ⇒ diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java index 6eeb1546b5..3a9491ed3f 100644 --- a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java +++ b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java @@ -12,7 +12,6 @@ import akka.actor.Props; import akka.actor.UntypedActor; import akka.actor.UntypedActorFactory; import akka.dispatch.Future; -import akka.event.EventHandler; import akka.testkit.EventFilter; import akka.testkit.ErrorFilter; import akka.testkit.TestEvent; @@ -77,7 +76,7 @@ public class UntypedCoordinatedIncrementTest { EventFilter expectedFailureFilter = (EventFilter) new ErrorFilter(ExpectedFailureException.class); EventFilter coordinatedFilter = (EventFilter) new ErrorFilter(CoordinatedTransactionException.class); Seq ignoreExceptions = seq(expectedFailureFilter, coordinatedFilter); - application.eventHandler().notify(new TestEvent.Mute(ignoreExceptions)); + application.mainbus().publish(new TestEvent.Mute(ignoreExceptions)); CountDownLatch incrementLatch = new CountDownLatch(numCounters); List actors = new ArrayList(counters); actors.add(failer); @@ -98,7 +97,7 @@ public class UntypedCoordinatedIncrementTest { } } } - application.eventHandler().notify(new TestEvent.UnMute(ignoreExceptions)); + application.mainbus().publish(new TestEvent.UnMute(ignoreExceptions)); } public Seq seq(A... args) { diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java b/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java index 8dc2c2beae..6d99e83e51 100644 --- a/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java +++ b/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java @@ -11,7 +11,6 @@ import akka.actor.Props; import akka.actor.UntypedActor; import akka.actor.UntypedActorFactory; import akka.dispatch.Future; -import akka.event.EventHandler; import akka.testkit.EventFilter; import akka.testkit.ErrorFilter; import akka.testkit.TestEvent; @@ -76,7 +75,7 @@ public class UntypedTransactorTest { EventFilter expectedFailureFilter = (EventFilter) new ErrorFilter(ExpectedFailureException.class); EventFilter coordinatedFilter = (EventFilter) new ErrorFilter(CoordinatedTransactionException.class); Seq ignoreExceptions = seq(expectedFailureFilter, coordinatedFilter); - application.eventHandler().notify(new TestEvent.Mute(ignoreExceptions)); + application.mainbus().publish(new TestEvent.Mute(ignoreExceptions)); CountDownLatch incrementLatch = new CountDownLatch(numCounters); List actors = new ArrayList(counters); actors.add(failer); @@ -97,7 +96,7 @@ public class UntypedTransactorTest { } } } - application.eventHandler().notify(new TestEvent.UnMute(ignoreExceptions)); + application.mainbus().publish(new TestEvent.UnMute(ignoreExceptions)); } public Seq seq(A... args) { diff --git a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala index 284786ef5a..e0f242a61b 100644 --- a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala +++ b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala @@ -7,7 +7,6 @@ import akka.transactor.Coordinated import akka.actor._ import akka.stm.{ Ref, TransactionFactory } import akka.util.duration._ -import akka.event.EventHandler import akka.transactor.CoordinatedTransactionException import akka.testkit._ @@ -86,17 +85,17 @@ class CoordinatedIncrementSpec extends AkkaSpec with BeforeAndAfterAll { EventFilter[ExpectedFailureException], EventFilter[CoordinatedTransactionException], EventFilter[ActorTimeoutException]) - app.eventHandler.notify(TestEvent.Mute(ignoreExceptions)) - val (counters, failer) = actorOfs - val coordinated = Coordinated() - counters(0) ! Coordinated(Increment(counters.tail :+ failer)) - coordinated.await - for (counter ← counters) { - (counter ? GetCount).as[Int].get must be === 0 + filterEvents(ignoreExceptions) { + val (counters, failer) = actorOfs + val coordinated = Coordinated() + counters(0) ! Coordinated(Increment(counters.tail :+ failer)) + coordinated.await + for (counter ← counters) { + (counter ? GetCount).as[Int].get must be === 0 + } + counters foreach (_.stop()) + failer.stop() } - counters foreach (_.stop()) - failer.stop() - app.eventHandler.notify(TestEvent.UnMute(ignoreExceptions)) } } } diff --git a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala index e4b0eed68e..f8d0e8e8b2 100644 --- a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala +++ b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala @@ -9,7 +9,6 @@ import akka.transactor.Coordinated import akka.actor._ import akka.stm._ import akka.util.duration._ -import akka.event.EventHandler import akka.transactor.CoordinatedTransactionException import akka.testkit._ @@ -119,18 +118,18 @@ class FickleFriendsSpec extends AkkaSpec with BeforeAndAfterAll { EventFilter[ExpectedFailureException], EventFilter[CoordinatedTransactionException], EventFilter[ActorTimeoutException]) - app.eventHandler.notify(TestEvent.Mute(ignoreExceptions)) - val (counters, coordinator) = actorOfs - val latch = new CountDownLatch(1) - coordinator ! FriendlyIncrement(counters, latch) - latch.await // this could take a while - (coordinator ? GetCount).as[Int].get must be === 1 - for (counter ← counters) { - (counter ? GetCount).as[Int].get must be === 1 + filterEvents(ignoreExceptions) { + val (counters, coordinator) = actorOfs + val latch = new CountDownLatch(1) + coordinator ! FriendlyIncrement(counters, latch) + latch.await // this could take a while + (coordinator ? GetCount).as[Int].get must be === 1 + for (counter ← counters) { + (counter ? GetCount).as[Int].get must be === 1 + } + counters foreach (_.stop()) + coordinator.stop() } - counters foreach (_.stop()) - coordinator.stop() - app.eventHandler.notify(TestEvent.UnMute(ignoreExceptions)) } } } diff --git a/akka-stm/src/test/scala/transactor/TransactorSpec.scala b/akka-stm/src/test/scala/transactor/TransactorSpec.scala index 7c8dda761a..daea5134db 100644 --- a/akka-stm/src/test/scala/transactor/TransactorSpec.scala +++ b/akka-stm/src/test/scala/transactor/TransactorSpec.scala @@ -8,7 +8,6 @@ import akka.transactor.Transactor import akka.actor._ import akka.stm._ import akka.util.duration._ -import akka.event.EventHandler import akka.transactor.CoordinatedTransactionException import akka.testkit._ @@ -109,17 +108,17 @@ class TransactorSpec extends AkkaSpec { EventFilter[ExpectedFailureException], EventFilter[CoordinatedTransactionException], EventFilter[ActorTimeoutException]) - app.eventHandler.notify(TestEvent.Mute(ignoreExceptions)) - val (counters, failer) = createTransactors - val failLatch = TestLatch(numCounters) - counters(0) ! Increment(counters.tail :+ failer, failLatch) - failLatch.await - for (counter ← counters) { - (counter ? GetCount).as[Int].get must be === 0 + filterEvents(ignoreExceptions) { + val (counters, failer) = createTransactors + val failLatch = TestLatch(numCounters) + counters(0) ! Increment(counters.tail :+ failer, failLatch) + failLatch.await + for (counter ← counters) { + (counter ? GetCount).as[Int].get must be === 0 + } + counters foreach (_.stop()) + failer.stop() } - counters foreach (_.stop()) - failer.stop() - app.eventHandler.notify(TestEvent.UnMute(ignoreExceptions)) } } diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index ea38de78a1..089e3bac18 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -3,7 +3,7 @@ */ package akka.testkit -import akka.event.EventHandler +import akka.event.Logging.{ Warning, Error } import java.util.concurrent.locks.ReentrantLock import java.util.LinkedList import java.util.concurrent.RejectedExecutionException @@ -166,14 +166,14 @@ class CallingThreadDispatcher(_app: AkkaApplication, val name: String = "calling val execute = mbox.suspendSwitch.fold { queue.push(handle) if (warnings && handle.channel.isInstanceOf[Promise[_]]) { - app.eventHandler.warning(this, "suspendSwitch, creating Future could deadlock; target: %s" format receiver) + app.mainbus.publish(Warning(this, "suspendSwitch, creating Future could deadlock; target: %s" format receiver)) } false } { queue.push(handle) if (queue.isActive) { if (warnings && handle.channel.isInstanceOf[Promise[_]]) { - app.eventHandler.warning(this, "blocked on this thread, creating Future could deadlock; target: %s" format receiver) + app.mainbus.publish(Warning(this, "blocked on this thread, creating Future could deadlock; target: %s" format receiver)) } false } else { @@ -216,18 +216,18 @@ class CallingThreadDispatcher(_app: AkkaApplication, val name: String = "calling mbox.actor.invoke(handle) if (warnings) handle.channel match { case f: ActorPromise if !f.isCompleted ⇒ - app.eventHandler.warning(this, "calling %s with message %s did not reply as expected, might deadlock" format (mbox.actor, handle.message)) + app.mainbus.publish(Warning(this, "calling %s with message %s did not reply as expected, might deadlock" format (mbox.actor, handle.message))) case _ ⇒ } true } catch { case ie: InterruptedException ⇒ - app.eventHandler.error(this, ie) + app.mainbus.publish(Error(this, ie)) Thread.currentThread().interrupt() intex = ie true case e ⇒ - app.eventHandler.error(this, e) + app.mainbus.publish(Error(this, e)) queue.leave false } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 3402429ddd..c6827a3547 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -6,7 +6,6 @@ package akka.testkit import akka.actor._ import akka.util.ReflectiveAccess -import akka.event.EventHandler import com.eaio.uuid.UUID import akka.actor.Props._ import akka.AkkaApplication diff --git a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala index c7773bdf80..ce7ef422fe 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala @@ -1,7 +1,6 @@ package akka.testkit -import akka.event.EventHandler -import akka.event.EventHandler.{ Event, Error } +import akka.event.Logging.{ LogEvent, Error, InitializeLogger } import akka.actor.Actor sealed trait TestEvent @@ -15,11 +14,10 @@ object TestEvent { def apply(filter: EventFilter, filters: EventFilter*): UnMute = new UnMute(filter +: filters.toSeq) } case class UnMute(filters: Seq[EventFilter]) extends TestEvent - case object UnMuteAll extends TestEvent } trait EventFilter { - def apply(event: Event): Boolean + def apply(event: LogEvent): Boolean } object EventFilter { @@ -36,19 +34,19 @@ object EventFilter { def apply[A <: Throwable: Manifest](source: AnyRef, message: String): EventFilter = ErrorSourceMessageFilter(manifest[A].erasure, source, message) - def custom(test: (Event) ⇒ Boolean): EventFilter = + def custom(test: (LogEvent) ⇒ Boolean): EventFilter = CustomEventFilter(test) } case class ErrorFilter(throwable: Class[_]) extends EventFilter { - def apply(event: Event) = event match { + def apply(event: LogEvent) = event match { case Error(cause, _, _) ⇒ throwable isInstance cause case _ ⇒ false } } case class ErrorMessageFilter(throwable: Class[_], message: String) extends EventFilter { - def apply(event: Event) = event match { + def apply(event: LogEvent) = event match { case Error(cause, _, _) if !(throwable isInstance cause) ⇒ false case Error(cause, _, null) if cause.getMessage eq null ⇒ cause.getStackTrace.length == 0 case Error(cause, _, null) ⇒ cause.getMessage startsWith message @@ -59,14 +57,14 @@ case class ErrorMessageFilter(throwable: Class[_], message: String) extends Even } case class ErrorSourceFilter(throwable: Class[_], source: AnyRef) extends EventFilter { - def apply(event: Event) = event match { + def apply(event: LogEvent) = event match { case Error(cause, instance, _) ⇒ (throwable isInstance cause) && (source eq instance) case _ ⇒ false } } case class ErrorSourceMessageFilter(throwable: Class[_], source: AnyRef, message: String) extends EventFilter { - def apply(event: Event) = event match { + def apply(event: LogEvent) = event match { case Error(cause, instance, _) if !((throwable isInstance cause) && (source eq instance)) ⇒ false case Error(cause, _, null) if cause.getMessage eq null ⇒ cause.getStackTrace.length == 0 case Error(cause, _, null) ⇒ cause.getMessage startsWith message @@ -76,23 +74,23 @@ case class ErrorSourceMessageFilter(throwable: Class[_], source: AnyRef, message } } -case class CustomEventFilter(test: (Event) ⇒ Boolean) extends EventFilter { - def apply(event: Event) = test(event) +case class CustomEventFilter(test: (LogEvent) ⇒ Boolean) extends EventFilter { + def apply(event: LogEvent) = test(event) } -class TestEventListener extends EventHandler.DefaultListener { +class TestEventListener extends akka.event.Logging.DefaultLogger { import TestEvent._ var filters: List[EventFilter] = Nil override def receive: Actor.Receive = ({ - case Mute(filters) ⇒ filters foreach addFilter - case UnMute(filters) ⇒ filters foreach removeFilter - case UnMuteAll ⇒ filters = Nil - case event: Event if filter(event) ⇒ + case InitializeLogger(bus) ⇒ Seq(classOf[Mute], classOf[UnMute]) foreach (bus.subscribe(context.self, _)) + case Mute(filters) ⇒ filters foreach addFilter + case UnMute(filters) ⇒ filters foreach removeFilter + case event: LogEvent if filter(event) ⇒ }: Actor.Receive) orElse super.receive - def filter(event: Event): Boolean = filters exists (f ⇒ try { f(event) } catch { case e: Exception ⇒ false }) + def filter(event: LogEvent): Boolean = filters exists (f ⇒ try { f(event) } catch { case e: Exception ⇒ false }) def addFilter(filter: EventFilter): Unit = filters ::= filter diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index c7fc73bb05..fbf8138bfe 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -99,7 +99,8 @@ 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 = new CallingThreadDispatcher(app)), app.guardian, "testActor" + TestKit.testActorId.incrementAndGet(), true) + val testActor: ActorRef = app.systemActorOf(Props(new TestActor(queue)).copy(dispatcher = new CallingThreadDispatcher(app)), + "testActor" + TestKit.testActorId.incrementAndGet) private var end: Duration = Duration.Inf diff --git a/akka-testkit/src/main/scala/akka/testkit/package.scala b/akka-testkit/src/main/scala/akka/testkit/package.scala index 54c6ed8182..384baa3d11 100644 --- a/akka-testkit/src/main/scala/akka/testkit/package.scala +++ b/akka-testkit/src/main/scala/akka/testkit/package.scala @@ -1,14 +1,12 @@ package akka -import akka.event.EventHandler - package object testkit { def filterEvents[T](eventFilters: Iterable[EventFilter])(block: ⇒ T)(implicit app: AkkaApplication): T = { - app.eventHandler.notify(TestEvent.Mute(eventFilters.toSeq)) + app.mainbus.publish(TestEvent.Mute(eventFilters.toSeq)) try { block } finally { - app.eventHandler.notify(TestEvent.UnMute(eventFilters.toSeq)) + app.mainbus.publish(TestEvent.UnMute(eventFilters.toSeq)) } } diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala index b4341e949d..22f114b109 100644 --- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala @@ -9,10 +9,13 @@ import org.scalatest.matchers.MustMatchers import akka.AkkaApplication import akka.actor.{ Actor, ActorRef, Props } import akka.dispatch.MessageDispatcher +import akka.event.{ Logging, MainBusLogging } abstract class AkkaSpec(_application: AkkaApplication = AkkaApplication()) extends TestKit(_application) with WordSpec with MustMatchers with BeforeAndAfterAll { + val log: Logging = new MainBusLogging(app.mainbus, this) + final override def beforeAll { atStartup() } diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index 3fb594a91e..6720299337 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -6,7 +6,7 @@ package akka.testkit import org.scalatest.matchers.MustMatchers import org.scalatest.{ BeforeAndAfterEach, WordSpec } import akka.actor._ -import akka.event.EventHandler +import akka.event.Logging.Warning import akka.dispatch.{ Future, Promise } import akka.AkkaApplication @@ -80,7 +80,6 @@ object TestActorRefSpec { } class Logger extends Actor { - import EventHandler._ var count = 0 var msg: String = _ def receive = { diff --git a/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala index 930e5c1454..302c461f95 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala @@ -4,7 +4,6 @@ import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers import org.scalatest.{ BeforeAndAfterEach, WordSpec } import akka.actor._ -import akka.event.EventHandler import akka.dispatch.Future import akka.util.duration._ diff --git a/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala b/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala index 67841c7a60..09b9246ad6 100644 --- a/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala +++ b/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala @@ -5,7 +5,7 @@ package akka.tutorial.second import akka.actor.Actor._ -import akka.event.EventHandler +import akka.event.Logging import System.{ currentTimeMillis ⇒ now } import akka.routing.Routing.Broadcast import akka.actor.{ Timeout, Channel, Actor, PoisonPill } @@ -15,6 +15,7 @@ import akka.AkkaApplication object Pi extends App { val app = AkkaApplication() + val log = Logging(app, this) calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000) @@ -110,9 +111,9 @@ object Pi extends App { master.?(Calculate, Timeout(60000)). await.resultOrException match { //wait for the result, with a 60 seconds timeout case Some(pi) ⇒ - app.eventHandler.info(this, "\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis".format(pi, (now - start))) + log.info("\n\tPi estimate: \t\t{}\n\tCalculation time: \t{} millis", pi, now - start) case None ⇒ - app.eventHandler.error(this, "Pi calculation did not complete within the timeout.") + log.error("Pi calculation did not complete within the timeout.") } } } diff --git a/config/akka-reference.conf b/config/akka-reference.conf index 0a5fcb9fa8..6b11df0560 100644 --- a/config/akka-reference.conf +++ b/config/akka-reference.conf @@ -14,8 +14,9 @@ akka { time-unit = "seconds" # Time unit for all timeout properties throughout the config - event-handlers = ["akka.event.EventHandler$DefaultListener"] # Event handlers to register at boot time (EventHandler$DefaultListener logs to STDOUT) - event-handler-level = "INFO" # Options: ERROR, WARNING, INFO, DEBUG + event-handlers = ["akka.event.Logging$DefaultLogger"] # Event handlers to register at boot time (Logging$DefaultLogger logs to STDOUT) + loglevel = "INFO" # Options: ERROR, WARNING, INFO, DEBUG + stdout-loglevel = "WARNING" # Loglevel for the very basic logger activated during AkkaApplication startup event-handler-dispatcher { type = "Dispatcher" # Must be one of the following diff --git a/config/akka.test.conf b/config/akka.test.conf index 4755e1f11f..8e21a7d184 100644 --- a/config/akka.test.conf +++ b/config/akka.test.conf @@ -6,7 +6,7 @@ include "akka-reference.conf" akka { event-handlers = ["akka.testkit.TestEventListener"] - event-handler-level = "WARNING" + loglevel = "WARNING" actor { default-dispatcher { core-pool-size = 4 From 897c7bde6908906165f4e88b69cf5ccfa0091cfd Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 27 Oct 2011 12:46:10 +0200 Subject: [PATCH 02/19] fix overlooked Gossiper change (from rebase) --- .../src/test/scala/akka/event/MainBusSpec.scala | 2 +- .../src/main/scala/akka/remote/Gossiper.scala | 12 +++++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/event/MainBusSpec.scala b/akka-actor-tests/src/test/scala/akka/event/MainBusSpec.scala index a132ae32d6..887628ec94 100644 --- a/akka-actor-tests/src/test/scala/akka/event/MainBusSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/event/MainBusSpec.scala @@ -14,7 +14,7 @@ object MainBusSpec { class MainBusSpec extends AkkaSpec(Configuration( "akka.actor.debug.lifecycle" -> true, "akka.actor.debug.mainbus" -> true)) { - + import MainBusSpec._ "A MainBus" must { diff --git a/akka-remote/src/main/scala/akka/remote/Gossiper.scala b/akka-remote/src/main/scala/akka/remote/Gossiper.scala index 73e975ad91..230204963b 100644 --- a/akka-remote/src/main/scala/akka/remote/Gossiper.scala +++ b/akka-remote/src/main/scala/akka/remote/Gossiper.scala @@ -7,6 +7,7 @@ package akka.remote import akka.AkkaApplication import akka.actor._ import akka.actor.Status._ +import akka.event.Logging import akka.util.duration._ import akka.remote.RemoteProtocol._ import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._ @@ -104,6 +105,7 @@ class Gossiper(remote: Remote) { nodeMembershipChangeListeners: Set[NodeMembershipChangeListener] = Set.empty[NodeMembershipChangeListener]) private val app = remote.app + private val log = Logging(app, this) private val failureDetector = remote.failureDetector private val connectionManager = new RemoteConnectionManager(app, remote, Map.empty[InetSocketAddress, ActorRef]) private val seeds = Set(address) // FIXME read in list of seeds from config @@ -243,20 +245,20 @@ class Gossiper(remote: Remote) { try { (connection ? (toRemoteMessage(newGossip), remote.remoteSystemDaemonAckTimeout)).as[Status] match { case Some(Success(receiver)) ⇒ - app.eventHandler.debug(this, "Gossip sent to [%s] was successfully received".format(receiver)) + log.debug("Gossip sent to [{}] was successfully received", receiver) case Some(Failure(cause)) ⇒ - app.eventHandler.error(cause, this, cause.toString) + log.error(cause, cause.toString) throw cause case None ⇒ - val error = new RemoteException("Gossip to [%s] timed out".format(connection.address)) - app.eventHandler.error(error, this, error.toString) + val error = new RemoteException("Gossip to [{}] timed out".format(connection.address)) + log.error(error, error.toString) throw error } } catch { case e: Exception ⇒ - app.eventHandler.error(e, this, "Could not gossip to [%s] due to: %s".format(connection.address, e.toString)) + log.error(e, "Could not gossip to [{}] due to: {}", connection.address, e.toString) throw e } From 01d8b00a766f2163ff9223ad13c68c5123947652 Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 27 Oct 2011 14:31:14 +0200 Subject: [PATCH 03/19] first time Eclipse deceived me: fix three more import statements --- akka-actor/src/main/scala/akka/util/Helpers.scala | 2 -- akka-actor/src/main/scala/akka/util/LockUtil.scala | 1 - .../src/main/scala/akka/remote/BootableRemoteActorService.scala | 1 - 3 files changed, 4 deletions(-) diff --git a/akka-actor/src/main/scala/akka/util/Helpers.scala b/akka-actor/src/main/scala/akka/util/Helpers.scala index da44abbe25..0975081d97 100644 --- a/akka-actor/src/main/scala/akka/util/Helpers.scala +++ b/akka-actor/src/main/scala/akka/util/Helpers.scala @@ -1,10 +1,8 @@ /** * Copyright (C) 2009-2011 Typesafe Inc. */ - package akka.util -import akka.event.EventHandler import java.io.{ PrintWriter, StringWriter } /** diff --git a/akka-actor/src/main/scala/akka/util/LockUtil.scala b/akka-actor/src/main/scala/akka/util/LockUtil.scala index f39eb9e237..b5009d1c3f 100644 --- a/akka-actor/src/main/scala/akka/util/LockUtil.scala +++ b/akka-actor/src/main/scala/akka/util/LockUtil.scala @@ -6,7 +6,6 @@ package akka.util import java.util.concurrent.locks.{ ReentrantReadWriteLock, ReentrantLock } import java.util.concurrent.atomic.{ AtomicBoolean } -import akka.event.EventHandler /** * @author Jonas Bonér diff --git a/akka-remote/src/main/scala/akka/remote/BootableRemoteActorService.scala b/akka-remote/src/main/scala/akka/remote/BootableRemoteActorService.scala index 6004e7c5ad..07bc74ad30 100644 --- a/akka-remote/src/main/scala/akka/remote/BootableRemoteActorService.scala +++ b/akka-remote/src/main/scala/akka/remote/BootableRemoteActorService.scala @@ -6,7 +6,6 @@ package akka.remote import akka.actor.{ Actor, BootableActorLoaderService } import akka.util.{ ReflectiveAccess, Bootable } -import akka.event.EventHandler // TODO: remove me - remoting is enabled through the RemoteActorRefProvider From d1e0f411ef943b4039b157cd6f8f67953e200e56 Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 28 Oct 2011 17:15:10 +0200 Subject: [PATCH 04/19] clean up application structure MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - introduce common parent for guardian and systemGuardian - install deathWatch chain to ensure proper shutdown upon guardian’s death - re-install stdout logger before detaching default loggers - await app termination after running AkkaSpec tests (max 5sec, with warning logged if not stopped) --- .../akka/actor/RestartStrategySpec.scala | 4 ++ .../src/main/scala/akka/AkkaApplication.scala | 65 ++++++++++--------- .../src/main/scala/akka/actor/Scheduler.scala | 2 +- .../main/scala/akka/dispatch/Dispatcher.scala | 2 +- .../src/main/scala/akka/event/Logging.scala | 13 ++++ .../src/main/scala/akka/util/Index.scala | 2 +- .../test/scala/akka/testkit/AkkaSpec.scala | 21 ++++++ 7 files changed, 77 insertions(+), 32 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala index 6021ecf8cc..459e479ebe 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala @@ -15,6 +15,10 @@ import akka.testkit.AkkaSpec @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class RestartStrategySpec extends AkkaSpec { + override def atStartup { + app.mainbus.publish(Mute(EventFilter[Exception]("Crashing..."))) + } + object Ping object Crash diff --git a/akka-actor/src/main/scala/akka/AkkaApplication.scala b/akka-actor/src/main/scala/akka/AkkaApplication.scala index 837960229d..a47a375e21 100644 --- a/akka-actor/src/main/scala/akka/AkkaApplication.scala +++ b/akka-actor/src/main/scala/akka/AkkaApplication.scala @@ -5,8 +5,9 @@ package akka import akka.config._ import akka.actor._ -import dispatch._ -import event._ +import akka.dispatch._ +import akka.event._ +import akka.util.duration._ import java.net.InetAddress import com.eaio.uuid.UUID import akka.dispatch.{ Dispatchers, Future } @@ -105,8 +106,7 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor val DispatcherThroughput = getInt("akka.actor.throughput", 5) val DispatcherDefaultShutdown = getLong("akka.actor.dispatcher-shutdown-timeout"). - map(time ⇒ Duration(time, DefaultTimeUnit)). - getOrElse(Duration(1000, TimeUnit.MILLISECONDS)) + map(time ⇒ Duration(time, DefaultTimeUnit)).getOrElse(1 second) val MailboxCapacity = getInt("akka.actor.default-dispatcher.mailbox-capacity", -1) val MailboxPushTimeout = Duration(getInt("akka.actor.default-dispatcher.mailbox-push-timeout-time", 10), DefaultTimeUnit) val DispatcherThroughputDeadlineTime = Duration(getInt("akka.actor.throughput-deadline-time", -1), DefaultTimeUnit) @@ -181,38 +181,46 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor // TODO think about memory consistency effects when doing funky stuff inside constructor val provider: ActorRefProvider = reflective.createProvider - // TODO make this configurable - protected[akka] val guardian: ActorRef = { - import akka.actor.FaultHandlingStrategy._ - provider.actorOf( - Props(context ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy { - case _: ActorKilledException ⇒ Stop - case _: ActorInitializationException ⇒ Stop - case _: Exception ⇒ Restart - }).withDispatcher(dispatcher), - provider.theOneWhoWalksTheBubblesOfSpaceTime, - "ApplicationSupervisor", - true) + private class Guardian extends Actor { + def receive = { + case Terminated(_) ⇒ context.self.stop() + } } + private class SystemGuardian extends Actor { + def receive = { + case Terminated(_) ⇒ + mainbus.stopDefaultLoggers() + context.self.stop() + } + } + private val guardianFaultHandlingStrategy = { + import akka.actor.FaultHandlingStrategy._ + OneForOneStrategy { + case _: ActorKilledException ⇒ Stop + case _: ActorInitializationException ⇒ Stop + case _: Exception ⇒ Restart + } + } + private val guardianProps = Props(new Guardian).withFaultHandler(guardianFaultHandlingStrategy) - protected[akka] val systemGuardian: ActorRef = { - import akka.actor.FaultHandlingStrategy._ - provider.actorOf( - Props(context ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy { - case _: ActorKilledException ⇒ Stop - case _: ActorInitializationException ⇒ Stop - case _: Exception ⇒ Restart - }).withDispatcher(dispatcher), - provider.theOneWhoWalksTheBubblesOfSpaceTime, - "SystemSupervisor", - true) - } + private val guardianInChief: ActorRef = + provider.actorOf(guardianProps, provider.theOneWhoWalksTheBubblesOfSpaceTime, "GuardianInChief", true) + + protected[akka] val guardian: ActorRef = + provider.actorOf(guardianProps, guardianInChief, "ApplicationSupervisor", true) + + protected[akka] val systemGuardian: ActorRef = + provider.actorOf(guardianProps.withCreator(new SystemGuardian), guardianInChief, "SystemSupervisor", true) // TODO think about memory consistency effects when doing funky stuff inside constructor val deadLetters = new DeadLetterActorRef(this) val deathWatch = provider.createDeathWatch() + // chain death watchers so that killing guardian stops the application + deathWatch.subscribe(systemGuardian, guardian) + deathWatch.subscribe(guardianInChief, systemGuardian) + // this starts the reaper actor and the user-configured logging subscribers, which are also actors mainbus.start(this) mainbus.startDefaultLoggers(this, AkkaConfig) @@ -232,7 +240,6 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor // TODO shutdown all that other stuff, whatever that may be def stop(): Unit = { guardian.stop() - systemGuardian.stop() } terminationFuture.onComplete(_ ⇒ dispatcher.shutdown()) diff --git a/akka-actor/src/main/scala/akka/actor/Scheduler.scala b/akka-actor/src/main/scala/akka/actor/Scheduler.scala index 5d64eec5f1..ae9b86695d 100644 --- a/akka-actor/src/main/scala/akka/actor/Scheduler.scala +++ b/akka-actor/src/main/scala/akka/actor/Scheduler.scala @@ -130,7 +130,7 @@ class DefaultScheduler extends Scheduler { } } - private[akka] def shutdown() { service.shutdown() } + private[akka] def shutdown() { service.shutdownNow() } } private object SchedulerThreadFactory extends ThreadFactory { diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index 3e9411d593..f288150265 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -107,7 +107,7 @@ class Dispatcher( protected[akka] def shutdown { val old = executorService.getAndSet(new LazyExecutorServiceWrapper(executorServiceFactory.createExecutorService)) if (old ne null) - old.shutdown() + old.shutdownNow() } /** diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index 59932879f1..82a4df5d00 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -88,6 +88,19 @@ trait LoggingBus extends ActorEventBus { } } + def stopDefaultLoggers() { + val level = _logLevel // volatile access before reading loggers + if (!(loggers contains StandardOutLogger)) { + AllLogLevels filter (level >= _) foreach (l ⇒ subscribe(StandardOutLogger, classFor(l))) + publish(Info(this, "shutting down: StandardOutLogger started")) + } + for { + logger ← loggers + if logger != StandardOutLogger + } logger.stop() + publish(Info(this, "all default loggers stopped")) + } + private def addLogger(app: AkkaApplication, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = { val actor = app.systemActorOf(Props(clazz), Props.randomAddress) actor ! InitializeLogger(this) diff --git a/akka-actor/src/main/scala/akka/util/Index.scala b/akka-actor/src/main/scala/akka/util/Index.scala index f79ea921dd..f91c20228c 100644 --- a/akka-actor/src/main/scala/akka/util/Index.scala +++ b/akka-actor/src/main/scala/akka/util/Index.scala @@ -115,7 +115,7 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) { /** * Returns the key set. */ - def keys = scala.collection.JavaConversions.asScalaIterable(container.keySet) + def keys = scala.collection.JavaConversions.collectionAsScalaIterable(container.keySet) /** * Disassociates the value of type V from the key of type K diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala index 22f114b109..4264427e0d 100644 --- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala @@ -10,6 +10,8 @@ import akka.AkkaApplication import akka.actor.{ Actor, ActorRef, Props } import akka.dispatch.MessageDispatcher import akka.event.{ Logging, MainBusLogging } +import akka.util.duration._ +import akka.dispatch.FutureTimeoutException abstract class AkkaSpec(_application: AkkaApplication = AkkaApplication()) extends TestKit(_application) with WordSpec with MustMatchers with BeforeAndAfterAll { @@ -22,6 +24,9 @@ abstract class AkkaSpec(_application: AkkaApplication = AkkaApplication()) final override def afterAll { app.stop() + try app.terminationFuture.await(5 seconds) catch { + case _: FutureTimeoutException ⇒ app.log.warning("failed to stop within 5 seconds") + } atTermination() } @@ -42,4 +47,20 @@ abstract class AkkaSpec(_application: AkkaApplication = AkkaApplication()) def spawn(body: ⇒ Unit)(implicit dispatcher: MessageDispatcher) { actorOf(Props(ctx ⇒ { case "go" ⇒ try body finally ctx.self.stop() }).withDispatcher(dispatcher)) ! "go" } +} + +class AkkaSpecSpec extends WordSpec with MustMatchers { + "An AkkaSpec" must { + "terminate all actors" in { + import AkkaApplication.defaultConfig + val app = AkkaApplication("test", defaultConfig ++ Configuration( + "akka.actor.debug.lifecycle" -> true, "akka.loglevel" -> "DEBUG")) + val spec = new AkkaSpec(app) { + val ref = Seq(testActor, app.actorOf(Props.empty, "name")) + } + spec.ref foreach (_ must not be 'shutdown) + app.stop() + spec.awaitCond(spec.ref forall (_.isShutdown), 2 seconds) + } + } } \ No newline at end of file From 55f896226c2502139f66ca588daba22773f6e6bb Mon Sep 17 00:00:00 2001 From: Roland Date: Sat, 29 Oct 2011 19:10:58 +0200 Subject: [PATCH 05/19] some polishing of new Logging - add scaladoc - remove (empty) EventHandler.scala - make BugLogging more of an implementation detail, i.e. use the Logging() factories to get a Logging implementation. --- .../src/main/scala/akka/AkkaApplication.scala | 2 +- .../scala/akka/actor/ActorRefProvider.scala | 4 +- .../src/main/scala/akka/actor/Deployer.scala | 4 +- .../src/main/scala/akka/actor/FSM.scala | 4 +- .../main/scala/akka/event/EventHandler.scala | 55 ----------- .../src/main/scala/akka/event/Logging.scala | 96 +++++++++++++++---- .../test/scala/akka/testkit/AkkaSpec.scala | 4 +- 7 files changed, 87 insertions(+), 82 deletions(-) delete mode 100644 akka-actor/src/main/scala/akka/event/EventHandler.scala diff --git a/akka-actor/src/main/scala/akka/AkkaApplication.scala b/akka-actor/src/main/scala/akka/AkkaApplication.scala index a47a375e21..96cc0568be 100644 --- a/akka-actor/src/main/scala/akka/AkkaApplication.scala +++ b/akka-actor/src/main/scala/akka/AkkaApplication.scala @@ -168,7 +168,7 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor // this provides basic logging (to stdout) until .start() is called below val mainbus = new MainBus(DebugMainBus) mainbus.startStdoutLogger(AkkaConfig) - val log = new MainBusLogging(mainbus, this) + val log = new BusLogging(mainbus, this) // TODO correctly pull its config from the config val dispatcherFactory = new Dispatchers(this) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index f8f701c4d2..c5aaac83df 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -11,7 +11,7 @@ import akka.AkkaApplication import java.util.concurrent.ConcurrentHashMap import com.eaio.uuid.UUID import akka.AkkaException -import akka.event.{ ActorClassification, DeathWatch, MainBusLogging } +import akka.event.{ ActorClassification, DeathWatch, Logging } import akka.dispatch._ /** @@ -91,7 +91,7 @@ class ActorRefProviderException(message: String) extends AkkaException(message) class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider { val terminationFuture = new DefaultPromise[AkkaApplication.ExitStatus](Timeout.never)(app.dispatcher) - val log = new MainBusLogging(app.mainbus, this) + val log = Logging(app.mainbus, this) /** * Top-level anchor for the supervision hierarchy of this actor system. Will diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index fecdc801fa..5522d9c978 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -8,7 +8,7 @@ import collection.immutable.Seq import java.util.concurrent.ConcurrentHashMap -import akka.event.MainBusLogging +import akka.event.Logging import akka.actor.DeploymentConfig._ import akka.{ AkkaException, AkkaApplication } import akka.config.{ Configuration, ConfigurationException } @@ -34,7 +34,7 @@ trait ActorDeployer { class Deployer(val app: AkkaApplication) extends ActorDeployer { val deploymentConfig = new DeploymentConfig(app) - val log = new MainBusLogging(app.mainbus, this) + val log = Logging(app.mainbus, this) // val defaultAddress = Node(Config.nodename) diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index dee625d92c..f2cea81c6d 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -4,7 +4,7 @@ package akka.actor import akka.util._ -import akka.event.MainBusLogging +import akka.event.Logging import scala.collection.mutable import java.util.concurrent.ScheduledFuture @@ -190,7 +190,7 @@ trait FSM[S, D] extends ListenerManagement { type Timeout = Option[Duration] type TransitionHandler = PartialFunction[(S, S), Unit] - val log = new MainBusLogging(app.mainbus, context.self) + val log = Logging(app.mainbus, context.self) /** * **************************************** diff --git a/akka-actor/src/main/scala/akka/event/EventHandler.scala b/akka-actor/src/main/scala/akka/event/EventHandler.scala deleted file mode 100644 index 7a4a5fe0ad..0000000000 --- a/akka-actor/src/main/scala/akka/event/EventHandler.scala +++ /dev/null @@ -1,55 +0,0 @@ -/** - * Copyright (C) 2009-2011 Typesafe Inc. - */ - -package akka.event - -import akka.actor._ -import akka.dispatch.Dispatchers -import akka.config.ConfigurationException -import akka.util.{ ListenerManagement, ReflectiveAccess } -import akka.serialization._ -import akka.AkkaException -import akka.AkkaApplication - -/** - * Event handler. - *

- * Create, add and remove a listener: - *

- * val eventHandlerListener = Actor.actorOf(new Actor {
- *   self.dispatcher = EventHandler.EventHandlerDispatcher
- *
- *   def receive = {
- *     case EventHandler.Error(cause, instance, message) ⇒ ...
- *     case EventHandler.Warning(instance, message)      ⇒ ...
- *     case EventHandler.Info(instance, message)         ⇒ ...
- *     case EventHandler.Debug(instance, message)        ⇒ ...
- *     case genericEvent                                 ⇒ ...
- * }
- * })
- *
- * EventHandler.addListener(eventHandlerListener)
- * ...
- * EventHandler.removeListener(eventHandlerListener)
- * 
- *

- * However best is probably to register the listener in the 'akka.conf' - * configuration file. - *

- * Log an error event: - *

- * EventHandler.notify(EventHandler.Error(exception, this, message))
- * 
- * Or use the direct methods (better performance): - *
- * EventHandler.error(exception, this, message)
- * 
- * - * Shut down the EventHandler: - *
- * EventHandler.shutdown()
- * 
- * - * @author Jonas Bonér - */ diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index 82a4df5d00..c93c763244 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -28,15 +28,24 @@ trait LoggingBus extends ActorEventBus { @volatile private var _logLevel: LogLevel = _ + /** + * Query currently set log level. See object Logging for more information. + */ def logLevel = _logLevel + /** + * Change log level: default loggers (i.e. from configuration file) are + * subscribed/unsubscribed as necessary so that they listen to all levels + * which are at least as severe as the given one. See object Logging for + * more information. + */ def logLevel_=(level: LogLevel) { for { l ← AllLogLevels if l > _logLevel && l <= level; log ← loggers } subscribe(log, classFor(l)) for { l ← AllLogLevels if l <= _logLevel && l > level; log ← loggers } unsubscribe(log, classFor(l)) _logLevel = level } - def startStdoutLogger(config: AkkaConfig) { + private[akka] def startStdoutLogger(config: AkkaConfig) { val level = levelFor(config.StdoutLogLevel) getOrElse { StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.StdoutLogLevel)) ErrorLevel @@ -47,7 +56,7 @@ trait LoggingBus extends ActorEventBus { publish(Info(this, "StandardOutLogger started")) } - def startDefaultLoggers(app: AkkaApplication, config: AkkaConfig) { + private[akka] def startDefaultLoggers(app: AkkaApplication, config: AkkaConfig) { val level = levelFor(config.LogLevel) getOrElse { StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.LogLevel)) ErrorLevel @@ -59,7 +68,7 @@ trait LoggingBus extends ActorEventBus { } loggers = for { loggerName ← defaultLoggers - if loggerName != DefaultLoggerName + if loggerName != StandardOutLoggerName } yield { try { ReflectiveAccess.getClassFor[Actor](loggerName) match { @@ -74,7 +83,7 @@ trait LoggingBus extends ActorEventBus { } } publish(Info(this, "Default Loggers started")) - if (defaultLoggers contains DefaultLoggerName) { + if (defaultLoggers contains StandardOutLoggerName) { loggers :+= StandardOutLogger } else { unsubscribe(StandardOutLogger) @@ -88,7 +97,7 @@ trait LoggingBus extends ActorEventBus { } } - def stopDefaultLoggers() { + private[akka] def stopDefaultLoggers() { val level = _logLevel // volatile access before reading loggers if (!(loggers contains StandardOutLogger)) { AllLogLevels filter (level >= _) foreach (l ⇒ subscribe(StandardOutLogger, classFor(l))) @@ -111,6 +120,24 @@ trait LoggingBus extends ActorEventBus { } +/** + * Main entry point for Akka logging: log levels and message types (aka + * channels) defined for the main transport medium, the main event bus. The + * recommended use is to obtain an implementation of the Logging trait with + * suitable and efficient methods for generating log events: + * + *

+ * val log = Logging(<bus>, <source object>)
+ * ...
+ * log.info("hello world!")
+ * 
+ * + * Loggers are attached to the level-specific channels Error, + * Warning, Info and Debug as + * appropriate for the configured (or set) log level. If you want to implement + * your own, make sure to handle these four event types plus the InitializeLogger + * message which is sent before actually attaching it to the logging bus. + */ object Logging { trait LogLevelType @@ -143,6 +170,7 @@ object Logging { case DebugLevel ⇒ classOf[Debug] } + // these type ascriptions/casts are necessary to avoid CCEs during construction while retaining correct type val AllLogLevels = Seq(ErrorLevel: AnyRef, WarningLevel, InfoLevel, DebugLevel).asInstanceOf[Seq[LogLevel]] val errorFormat = "[ERROR] [%s] [%s] [%s] %s\n%s".intern @@ -151,8 +179,8 @@ object Logging { val debugFormat = "[DEBUG] [%s] [%s] [%s] %s".intern val genericFormat = "[GENERIC] [%s] [%s]".intern - def apply(app: AkkaApplication, instance: AnyRef): Logging = new MainBusLogging(app.mainbus, instance) - def apply(bus: MainBus, instance: AnyRef): Logging = new MainBusLogging(bus, instance) + def apply(app: AkkaApplication, instance: AnyRef): Logging = new BusLogging(app.mainbus, instance) + def apply(bus: LoggingBus, instance: AnyRef): Logging = new BusLogging(bus, instance) class EventHandlerException extends AkkaException @@ -181,6 +209,11 @@ object Logging { def level = DebugLevel } + /** + * Message which is sent to each default logger (i.e. from configuration file) + * after its creation but before attaching it to the logging bus. The logger + * actor should handle this message, e.g. to register for more channels. + */ case class InitializeLogger(bus: LoggingBus) trait StdOutLogger { @@ -240,13 +273,25 @@ object Logging { } } + /** + * Actor-less logging implementation for synchronous logging to standard + * output. This logger is always attached first in order to be able to log + * failures during application start-up, even before normal logging is + * started. Its log level can be configured by setting + * akka.stdout-loglevel in akka.conf. + */ class StandardOutLogger extends MinimalActorRef with StdOutLogger { override val toString = "StandardOutLogger" override def postMessageToMailbox(obj: Any, channel: UntypedChannel) { print(obj) } } val StandardOutLogger = new StandardOutLogger - val DefaultLoggerName = StandardOutLogger.getClass.getName + val StandardOutLoggerName = StandardOutLogger.getClass.getName + /** + * Actor wrapper around the standard output logger. If + * akka.event-handlers is not set, it defaults to just this + * logger. + */ class DefaultLogger extends Actor with StdOutLogger { def receive = { case InitializeLogger(_) ⇒ @@ -270,7 +315,22 @@ object Logging { /** * Logging wrapper to make nicer and optimize: provide template versions which - * evaluate .toString only if the log level is actually enabled. + * evaluate .toString only if the log level is actually enabled. Typically used + * by obtaining an implementation from the Logging object: + * + *
+ * val log = Logging(<bus>, <source object>)
+ * ...
+ * log.info("hello world!")
+ * 
+ * + * All log-level methods support simple interpolation templates with up to four + * arguments placed by using {} within the template (first string + * argument): + * + *
+ * log.error(exception, "Exception while processing {} in state {}", msg, state)
+ * 
*/ trait Logging { @@ -333,21 +393,21 @@ trait Logging { } -class MainBusLogging(val mainbus: MainBus, val loggingInstance: AnyRef) extends Logging { +class BusLogging(val bus: LoggingBus, val loggingInstance: AnyRef) extends Logging { import Logging._ - def isErrorEnabled = mainbus.logLevel >= ErrorLevel - def isWarningEnabled = mainbus.logLevel >= WarningLevel - def isInfoEnabled = mainbus.logLevel >= InfoLevel - def isDebugEnabled = mainbus.logLevel >= DebugLevel + def isErrorEnabled = bus.logLevel >= ErrorLevel + def isWarningEnabled = bus.logLevel >= WarningLevel + def isInfoEnabled = bus.logLevel >= InfoLevel + def isDebugEnabled = bus.logLevel >= DebugLevel - protected def notifyError(cause: Throwable, message: String) { mainbus.publish(Error(cause, loggingInstance, message)) } + protected def notifyError(cause: Throwable, message: String) { bus.publish(Error(cause, loggingInstance, message)) } - protected def notifyWarning(message: String) { mainbus.publish(Warning(loggingInstance, message)) } + protected def notifyWarning(message: String) { bus.publish(Warning(loggingInstance, message)) } - protected def notifyInfo(message: String) { mainbus.publish(Info(loggingInstance, message)) } + protected def notifyInfo(message: String) { bus.publish(Info(loggingInstance, message)) } - protected def notifyDebug(message: String) { mainbus.publish(Debug(loggingInstance, message)) } + protected def notifyDebug(message: String) { bus.publish(Debug(loggingInstance, message)) } } \ No newline at end of file diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala index 4264427e0d..27874aa799 100644 --- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala @@ -9,14 +9,14 @@ import org.scalatest.matchers.MustMatchers import akka.AkkaApplication import akka.actor.{ Actor, ActorRef, Props } import akka.dispatch.MessageDispatcher -import akka.event.{ Logging, MainBusLogging } +import akka.event.Logging import akka.util.duration._ import akka.dispatch.FutureTimeoutException abstract class AkkaSpec(_application: AkkaApplication = AkkaApplication()) extends TestKit(_application) with WordSpec with MustMatchers with BeforeAndAfterAll { - val log: Logging = new MainBusLogging(app.mainbus, this) + val log: Logging = Logging(app.mainbus, this) final override def beforeAll { atStartup() From c671600a8e41203f23a80671ffa7258a5c86dfc4 Mon Sep 17 00:00:00 2001 From: Roland Date: Sun, 30 Oct 2011 12:02:11 +0100 Subject: [PATCH 06/19] fix up Slf4jEventHandler to handle InitializeLogger message --- akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala b/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala index 0ffd1b8490..8253dcd248 100644 --- a/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala +++ b/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala @@ -8,14 +8,13 @@ import org.slf4j.{ Logger ⇒ SLFLogger, LoggerFactory ⇒ SLFLoggerFactory } import akka.event.Logging._ import akka.actor._ -import Actor._ /** * Base trait for all classes that wants to be able use the SLF4J logging infrastructure. * * @author Jonas Bonér */ -trait Logging { +trait SLF4JLogging { @transient lazy val log = Logger(this.getClass.getName) } @@ -31,7 +30,7 @@ object Logger { * * @author Jonas Bonér */ -class Slf4jEventHandler extends Actor with Logging { +class Slf4jEventHandler extends Actor with SLF4JLogging { def receive = { case event @ Error(cause, instance, message) ⇒ @@ -50,10 +49,11 @@ class Slf4jEventHandler extends Actor with Logging { logger(instance).debug("[{}] [{}]", event.thread.getName, message.asInstanceOf[AnyRef]) - case event ⇒ log.debug("[{}]", event.toString) + case InitializeLogger(_) ⇒ log.info("Slf4jEventHandler started") } def logger(instance: AnyRef): SLFLogger = instance match { + // TODO make sure that this makes sense (i.e. should be the full path after Peter’s changes) case a: ActorRef ⇒ Logger(a.address) case _ ⇒ Logger(instance.getClass) } From b35f8dece0fa4ddc67d775872b7a4da924b80601 Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 3 Nov 2011 18:56:20 +0100 Subject: [PATCH 07/19] incorporate review from Viktor & Jonas - use intercept[]{} in PinnedActorSpec - make Logging.logLevel_= thread safe - add lots of scaladoc - make more friendly to Java API --- .../akka/actor/dispatch/PinnedActorSpec.scala | 8 +- .../src/main/scala/akka/event/Logging.scala | 132 +++++++++++++----- .../src/main/scala/akka/event/MainBus.scala | 2 +- .../src/main/scala/akka/util/Index.scala | 4 +- .../test/scala/akka/testkit/AkkaSpec.scala | 5 +- config/akka-reference.conf | 2 + 6 files changed, 107 insertions(+), 46 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala index f337c445b6..7c6a9f7c86 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala @@ -43,13 +43,9 @@ class PinnedActorSpec extends AkkaSpec with BeforeAndAfterEach { "support ask/exception" in { val actor = actorOf(Props[TestActor].withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test"))) filterException[RuntimeException] { - try { + intercept[RuntimeException] { (actor ? "Failure").get - fail("Should have thrown an exception") - } catch { - case e ⇒ - assert("Expected exception; to test fault-tolerance" === e.getMessage()) - } + }.getMessage must be("Expected exception; to test fault-tolerance") } actor.stop() } diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index c93c763244..ea5f63f5e1 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -8,6 +8,7 @@ import akka.{ AkkaException, AkkaApplication } import akka.AkkaApplication.AkkaConfig import akka.util.ReflectiveAccess import akka.config.ConfigurationException +import akka.util.ReentrantGuard /** * This trait brings log level handling to the MainBus: it reads the log @@ -24,24 +25,38 @@ trait LoggingBus extends ActorEventBus { import Logging._ + private val guard = new ReentrantGuard private var loggers = Seq.empty[ActorRef] - @volatile private var _logLevel: LogLevel = _ /** * Query currently set log level. See object Logging for more information. */ - def logLevel = _logLevel + def logLevel = guard.withGuard { _logLevel } /** - * Change log level: default loggers (i.e. from configuration file) are - * subscribed/unsubscribed as necessary so that they listen to all levels - * which are at least as severe as the given one. See object Logging for + * Change log level: default loggers (i.e. from configuration file) are + * subscribed/unsubscribed as necessary so that they listen to all levels + * which are at least as severe as the given one. See object Logging for * more information. + * + * NOTE: if the StandardOutLogger is configured also as normal logger, it + * will not participate in the automatic management of log level + * subscriptions! */ - def logLevel_=(level: LogLevel) { - for { l ← AllLogLevels if l > _logLevel && l <= level; log ← loggers } subscribe(log, classFor(l)) - for { l ← AllLogLevels if l <= _logLevel && l > level; log ← loggers } unsubscribe(log, classFor(l)) + def logLevel_=(level: LogLevel): Unit = guard.withGuard { + for { + l ← AllLogLevels + // subscribe if previously ignored and now requested + if l > _logLevel && l <= level + log ← loggers + } subscribe(log, classFor(l)) + for { + l ← AllLogLevels + // unsubscribe if previously registered and now ignored + if l <= _logLevel && l > level + log ← loggers + } unsubscribe(log, classFor(l)) _logLevel = level } @@ -51,8 +66,10 @@ trait LoggingBus extends ActorEventBus { ErrorLevel } AllLogLevels filter (level >= _) foreach (l ⇒ subscribe(StandardOutLogger, classFor(l))) - loggers = Seq(StandardOutLogger) - _logLevel = level + guard.withGuard { + loggers = Seq(StandardOutLogger) + _logLevel = level + } publish(Info(this, "StandardOutLogger started")) } @@ -66,7 +83,7 @@ trait LoggingBus extends ActorEventBus { case Nil ⇒ "akka.event.Logging$DefaultLogger" :: Nil case loggers ⇒ loggers } - loggers = for { + val myloggers = for { loggerName ← defaultLoggers if loggerName != StandardOutLoggerName } yield { @@ -82,13 +99,14 @@ trait LoggingBus extends ActorEventBus { "] due to [" + e.toString + "]", e) } } + guard.withGuard { + loggers = myloggers + _logLevel = level + } publish(Info(this, "Default Loggers started")) - if (defaultLoggers contains StandardOutLoggerName) { - loggers :+= StandardOutLogger - } else { + if (!(defaultLoggers contains StandardOutLoggerName)) { unsubscribe(StandardOutLogger) } - _logLevel = level } catch { case e: Exception ⇒ System.err.println("error while starting up EventHandler") @@ -121,26 +139,45 @@ trait LoggingBus extends ActorEventBus { } /** - * Main entry point for Akka logging: log levels and message types (aka - * channels) defined for the main transport medium, the main event bus. The - * recommended use is to obtain an implementation of the Logging trait with + * Main entry point for Akka logging: log levels and message types (aka + * channels) defined for the main transport medium, the main event bus. The + * recommended use is to obtain an implementation of the Logging trait with * suitable and efficient methods for generating log events: - * + * *

  * val log = Logging(<bus>, <source object>)
  * ...
  * log.info("hello world!")
  * 
- * - * Loggers are attached to the level-specific channels Error, - * Warning, Info and Debug as + * + * Loggers are attached to the level-specific channels Error, + * Warning, Info and Debug as * appropriate for the configured (or set) log level. If you want to implement * your own, make sure to handle these four event types plus the InitializeLogger * message which is sent before actually attaching it to the logging bus. + * + * Logging is configured in akka.conf by setting (some of) the following: + * + *

+ * akka {
+ *   event-handlers = ["akka.slf4j.Slf4jEventHandler"] # for example
+ *   loglevel = "INFO"        # used when normal logging ("event-handlers") has been started
+ *   stdout-loglevel = "WARN" # used during application start-up until normal logging is available
+ * }
+ * 
*/ object Logging { + /** + * Marker trait for annotating LogLevel, which must be Int after erasure. + */ trait LogLevelType + /** + * Log level in numeric form, used when deciding whether a certain log + * statement should generate a log event. Predefined levels are ErrorLevel (1) + * to DebugLevel (4). In case you want to add more levels, loggers need to + * be subscribed to their event bus channels manually. + */ type LogLevel = Int with LogLevelType final val ErrorLevel = 1.asInstanceOf[Int with LogLevelType] final val WarningLevel = 2.asInstanceOf[Int with LogLevelType] @@ -179,9 +216,31 @@ object Logging { val debugFormat = "[DEBUG] [%s] [%s] [%s] %s".intern val genericFormat = "[GENERIC] [%s] [%s]".intern - def apply(app: AkkaApplication, instance: AnyRef): Logging = new BusLogging(app.mainbus, instance) - def apply(bus: LoggingBus, instance: AnyRef): Logging = new BusLogging(bus, instance) + /** + * Obtain LoggingAdapter for the given application and source object. The + * source object is used to identify the source of this logging channel. + */ + def apply(app: AkkaApplication, source: AnyRef): LoggingAdapter = new BusLogging(app.mainbus, source) + /** + * Java API: Obtain LoggingAdapter for the given application and source object. The + * source object is used to identify the source of this logging channel. + */ + def getLogger(app: AkkaApplication, source: AnyRef): LoggingAdapter = apply(app, source) + /** + * Obtain LoggingAdapter for the given event bus and source object. The + * source object is used to identify the source of this logging channel. + */ + def apply(bus: LoggingBus, source: AnyRef): LoggingAdapter = new BusLogging(bus, source) + /** + * Java API: Obtain LoggingAdapter for the given event bus and source object. The + * source object is used to identify the source of this logging channel. + */ + def getLogger(bus: LoggingBus, source: AnyRef): LoggingAdapter = apply(bus, source) + /** + * Artificial exception injected into Error events if no Throwable is + * supplied; used for getting a stack dump of error locations. + */ class EventHandlerException extends AkkaException sealed trait LogEvent { @@ -274,10 +333,10 @@ object Logging { } /** - * Actor-less logging implementation for synchronous logging to standard + * Actor-less logging implementation for synchronous logging to standard * output. This logger is always attached first in order to be able to log - * failures during application start-up, even before normal logging is - * started. Its log level can be configured by setting + * failures during application start-up, even before normal logging is + * started. Its log level can be configured by setting * akka.stdout-loglevel in akka.conf. */ class StandardOutLogger extends MinimalActorRef with StdOutLogger { @@ -288,7 +347,7 @@ object Logging { val StandardOutLoggerName = StandardOutLogger.getClass.getName /** - * Actor wrapper around the standard output logger. If + * Actor wrapper around the standard output logger. If * akka.event-handlers is not set, it defaults to just this * logger. */ @@ -317,22 +376,22 @@ object Logging { * Logging wrapper to make nicer and optimize: provide template versions which * evaluate .toString only if the log level is actually enabled. Typically used * by obtaining an implementation from the Logging object: - * + * *
  * val log = Logging(<bus>, <source object>)
  * ...
  * log.info("hello world!")
  * 
- * + * * All log-level methods support simple interpolation templates with up to four - * arguments placed by using {} within the template (first string + * arguments placed by using {} within the template (first string * argument): - * + * *
  * log.error(exception, "Exception while processing {} in state {}", msg, state)
  * 
*/ -trait Logging { +trait LoggingAdapter { /* * implement these as precisely as needed/possible: always returning true @@ -347,6 +406,7 @@ trait Logging { * These actually implement the passing on of the messages to be logged. * Will not be called if is...Enabled returned false. */ + protected def notifyError(message: String) protected def notifyError(cause: Throwable, message: String) protected def notifyWarning(message: String) protected def notifyInfo(message: String) @@ -362,7 +422,7 @@ trait Logging { 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(message: String) { if (isErrorEnabled) notifyError(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)) } @@ -393,7 +453,7 @@ trait Logging { } -class BusLogging(val bus: LoggingBus, val loggingInstance: AnyRef) extends Logging { +class BusLogging(val bus: LoggingBus, val loggingInstance: AnyRef) extends LoggingAdapter { import Logging._ @@ -402,6 +462,8 @@ class BusLogging(val bus: LoggingBus, val loggingInstance: AnyRef) extends Loggi def isInfoEnabled = bus.logLevel >= InfoLevel def isDebugEnabled = bus.logLevel >= DebugLevel + protected def notifyError(message: String) { bus.publish(Error(loggingInstance, message)) } + protected def notifyError(cause: Throwable, message: String) { bus.publish(Error(cause, loggingInstance, message)) } protected def notifyWarning(message: String) { bus.publish(Warning(loggingInstance, message)) } diff --git a/akka-actor/src/main/scala/akka/event/MainBus.scala b/akka-actor/src/main/scala/akka/event/MainBus.scala index ffaa41ca5a..8a628e6edb 100644 --- a/akka-actor/src/main/scala/akka/event/MainBus.scala +++ b/akka-actor/src/main/scala/akka/event/MainBus.scala @@ -39,7 +39,7 @@ class MainBus(debug: Boolean = false) extends LoggingBus with LookupClassificati def start(app: AkkaApplication) { reaper = app.systemActorOf(Props(new Actor { - def receive = loggable(context.self) { + def receive = { case ref: ActorRef ⇒ watch(ref) case Terminated(ref) ⇒ unsubscribe(ref) } diff --git a/akka-actor/src/main/scala/akka/util/Index.scala b/akka-actor/src/main/scala/akka/util/Index.scala index f91c20228c..07d0d0ffba 100644 --- a/akka-actor/src/main/scala/akka/util/Index.scala +++ b/akka-actor/src/main/scala/akka/util/Index.scala @@ -100,9 +100,9 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) { } /** - * Returns the value set. + * Returns the union of all value sets. */ - def values = { + def values: Set[V] = { import scala.collection.JavaConversions._ val builder = mutable.Set.empty[V] for { diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala index 27874aa799..3ba25f303f 100644 --- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala @@ -9,14 +9,14 @@ import org.scalatest.matchers.MustMatchers import akka.AkkaApplication import akka.actor.{ Actor, ActorRef, Props } import akka.dispatch.MessageDispatcher -import akka.event.Logging +import akka.event.{ Logging, LoggingAdapter } import akka.util.duration._ import akka.dispatch.FutureTimeoutException abstract class AkkaSpec(_application: AkkaApplication = AkkaApplication()) extends TestKit(_application) with WordSpec with MustMatchers with BeforeAndAfterAll { - val log: Logging = Logging(app.mainbus, this) + val log: LoggingAdapter = Logging(app.mainbus, this) final override def beforeAll { atStartup() @@ -49,6 +49,7 @@ abstract class AkkaSpec(_application: AkkaApplication = AkkaApplication()) } } +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class AkkaSpecSpec extends WordSpec with MustMatchers { "An AkkaSpec" must { "terminate all actors" in { diff --git a/config/akka-reference.conf b/config/akka-reference.conf index 6b11df0560..3922dbd19e 100644 --- a/config/akka-reference.conf +++ b/config/akka-reference.conf @@ -16,6 +16,8 @@ akka { event-handlers = ["akka.event.Logging$DefaultLogger"] # Event handlers to register at boot time (Logging$DefaultLogger logs to STDOUT) loglevel = "INFO" # Options: ERROR, WARNING, INFO, DEBUG + # this level is used by the configured loggers (see "event-handlers") as soon + # as they have been started; before that, see "stdout-loglevel" stdout-loglevel = "WARNING" # Loglevel for the very basic logger activated during AkkaApplication startup event-handler-dispatcher { From 05b9cbc8c7837af3481acb958c89033c783018a9 Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 3 Nov 2011 19:09:17 +0100 Subject: [PATCH 08/19] fix LoggingReceiveSpec MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - make sure that needed log channels are subscribed - do not .stop() applications within test cases, that’s no good for following test cases (might change when we move to one app per case) - re-enable and fix (adapting to message format changes) the life-cycle logging spec --- .../scala/akka/actor/LoggingReceiveSpec.scala | 62 ++++++++++++------- 1 file changed, 39 insertions(+), 23 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala index e708b4771e..d4932de5ee 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala @@ -11,6 +11,7 @@ import akka.AkkaApplication import akka.AkkaApplication.defaultConfig import akka.config.Configuration import akka.event.Logging +import akka.util.Duration object LoggingReceiveSpec { class TestLogActor extends Actor { @@ -18,6 +19,7 @@ object LoggingReceiveSpec { } } +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAndAfterAll { import LoggingReceiveSpec._ @@ -42,6 +44,12 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd } } + override def afterAll { + appLogging.stop() + appAuto.stop() + appLifecycle.stop() + } + "A LoggingReceive" must { "decorate a Receive" in { @@ -53,13 +61,14 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd val log = Actor.LoggingReceive(this, r) log.isDefinedAt("hallo") expectMsg(1 second, Logging.Debug(this, "received unhandled message hallo")) - }.app.stop() + } } "be added on Actor if requested" in { new TestKit(appLogging) with ImplicitSender { ignoreMute(this) app.mainbus.subscribe(testActor, classOf[Logging.Debug]) + app.mainbus.subscribe(testActor, classOf[Logging.Error]) val actor = TestActorRef(new Actor { def receive = loggable(this) { case _ ⇒ channel ! "x" @@ -75,14 +84,14 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd } actor ! HotSwap(_ ⇒ r, false) filterException[UnhandledMessageException] { - within(300 millis) { + within(500 millis) { actor ! "bah" expectMsgPF() { case Logging.Error(_: UnhandledMessageException, `actor`, _) ⇒ true } } } - }.app.stop() + } } "not duplicate logging" in { @@ -98,7 +107,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd expectMsg(Logging.Debug(actor.underlyingActor, "received handled message buh")) expectMsg("x") } - }.app.stop() + } } } @@ -116,26 +125,34 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd actor ! PoisonPill expectMsg(300 millis, Logging.Debug(actor.underlyingActor, "received AutoReceiveMessage PoisonPill")) awaitCond(actor.isShutdown, 100 millis) - }.app.stop() + } } - // TODO remove ignore as soon as logging is working properly during start-up again - "log LifeCycle changes if requested" ignore { + "log LifeCycle changes if requested" in { new TestKit(appLifecycle) { ignoreMute(this) + ignoreMsg { + case Logging.Debug(ref, _) ⇒ + val s = ref.toString + s.contains("MainBusReaper") || s.contains("Supervisor") + } app.mainbus.subscribe(testActor, classOf[Logging.Debug]) - within(2 seconds) { + app.mainbus.subscribe(testActor, classOf[Logging.Error]) + within(3 seconds) { val supervisor = TestActorRef[TestLogActor](Props[TestLogActor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000))) - expectMsg(Logging.Debug(supervisor, "started")) + expectMsgPF() { + case Logging.Debug(`supervisor`, msg: String) if msg startsWith "started" ⇒ + } val actor = new TestActorRef[TestLogActor](app, Props[TestLogActor], supervisor, "none") - expectMsgPF() { - case Logging.Debug(ref, msg: String) ⇒ ref == supervisor && msg.startsWith("now supervising") - } - - expectMsg(Logging.Debug(actor, "started")) + val set = receiveWhile(messages = 2) { + case Logging.Debug(`supervisor`, msg: String) if msg startsWith "now supervising" ⇒ 1 + case Logging.Debug(`actor`, msg: String) if msg startsWith "started" ⇒ 2 + }.toSet + expectNoMsg(Duration.Zero) + assert(set == Set(1, 2), set + " was not Set(1, 2)") supervisor startsMonitoring actor expectMsgPF(hint = "now monitoring") { @@ -151,21 +168,20 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd filterException[ActorKilledException] { actor ! Kill - expectMsgPF() { - case Logging.Error(_: ActorKilledException, `actor`, "Kill") ⇒ true - } - expectMsg(Logging.Debug(actor, "restarting")) - } - awaitCond(msgAvailable) - expectMsgPF(hint = "restarted") { - case Logging.Debug(`actor`, "restarted") ⇒ true + val set = receiveWhile(messages = 3) { + case Logging.Error(_: ActorKilledException, `actor`, "Kill") ⇒ 1 + case Logging.Debug(`actor`, "restarting") ⇒ 2 + case Logging.Debug(`actor`, "restarted") ⇒ 3 + }.toSet + expectNoMsg(Duration.Zero) + assert(set == Set(1, 2, 3), set + " was not Set(1, 2, 3)") } actor.stop() expectMsg(Logging.Debug(actor, "stopping")) supervisor.stop() } - }.app.stop() + } } } From 91bee0370ca852287985f80756284748ec99acdb Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 3 Nov 2011 19:11:15 +0100 Subject: [PATCH 09/19] fix TestKit.receiveWhile when using implicitly discovered maximum wait time (i.e. default argument) --- akka-testkit/src/main/scala/akka/testkit/TestKit.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index fbf8138bfe..1f4f0d680e 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -477,7 +477,7 @@ class TestKit(_app: AkkaApplication) { * */ def receiveWhile[T](max: Duration = Duration.MinusInf, idle: Duration = Duration.Inf, messages: Int = Int.MaxValue)(f: PartialFunction[AnyRef, T]): Seq[T] = { - val stop = now + (if (max == Duration.MinusInf) remaining else max.dilated) + val stop = now + (if (max eq Duration.MinusInf) remaining else max.dilated) var msg: Message = NullMessage @tailrec From d4c91ef61c045e0d2d45f43e23b1571728e719d6 Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 3 Nov 2011 21:41:32 +0100 Subject: [PATCH 10/19] expand MainBusSpec wrt. logLevel setting --- .../test/scala/akka/event/MainBusSpec.scala | 51 +++++++++++++++++-- 1 file changed, 47 insertions(+), 4 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/event/MainBusSpec.scala b/akka-actor-tests/src/test/scala/akka/event/MainBusSpec.scala index 887628ec94..16e735542b 100644 --- a/akka-actor-tests/src/test/scala/akka/event/MainBusSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/event/MainBusSpec.scala @@ -6,27 +6,70 @@ package akka.event import akka.testkit.AkkaSpec import akka.config.Configuration import akka.util.duration._ +import akka.actor.{ Actor, ActorRef } object MainBusSpec { case class M(i: Int) + + case class SetTarget(ref: ActorRef) + + class MyLog extends Actor { + var dst: ActorRef = app.deadLetters + def receive = { + case Logging.InitializeLogger(bus) ⇒ bus.subscribe(context.self, classOf[SetTarget]) + case SetTarget(ref) ⇒ dst = ref + case e: Logging.LogEvent ⇒ dst ! e + } + } } class MainBusSpec extends AkkaSpec(Configuration( - "akka.actor.debug.lifecycle" -> true, - "akka.actor.debug.mainbus" -> true)) { + "akka.stdout-loglevel" -> "WARNING", + "akka.loglevel" -> "INFO", + "akka.event-handlers" -> Seq("akka.event.MainBusSpec$MyLog", Logging.StandardOutLoggerName))) { import MainBusSpec._ "A MainBus" must { - "allow subscriptions" in { + "manage subscriptions" in { val bus = new MainBus(true) bus.start(app) bus.subscribe(testActor, classOf[M]) bus.publish(M(42)) - expectMsg(1 second, M(42)) + within(1 second) { + expectMsg(M(42)) + bus.unsubscribe(testActor) + bus.publish(M(13)) + expectNoMsg + } + } + + "manage log levels" in { + val bus = new MainBus(false) + bus.start(app) + bus.startDefaultLoggers(app, app.AkkaConfig) + bus.publish(SetTarget(testActor)) + within(1 second) { + import Logging._ + verifyLevel(bus, InfoLevel) + bus.logLevel = WarningLevel + verifyLevel(bus, WarningLevel) + bus.logLevel = DebugLevel + verifyLevel(bus, DebugLevel) + bus.logLevel = ErrorLevel + verifyLevel(bus, ErrorLevel) + } } } + private def verifyLevel(bus: LoggingBus, level: Logging.LogLevel) { + import Logging._ + val allmsg = Seq(Debug(this, "debug"), Info(this, "info"), Warning(this, "warning"), Error(this, "error")) + val msg = allmsg filter (_.level <= level) + allmsg foreach bus.publish + msg foreach (x ⇒ expectMsg(x)) + } + } \ No newline at end of file From 7198dd6287020f63f1f1ae770ce26b9a56e3d225 Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 3 Nov 2011 22:42:28 +0100 Subject: [PATCH 11/19] fix FSMActorSpec (wrongly setting loglevel) --- akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala index d1cda03f9d..e94cbaaa02 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala @@ -191,7 +191,7 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true "log events and transitions if asked to do so" in { new TestKit(AkkaApplication("fsm event", AkkaApplication.defaultConfig ++ - Configuration("akka.event-handler-level" -> "DEBUG", + Configuration("akka.loglevel" -> "DEBUG", "akka.actor.debug.fsm" -> true))) { app.mainbus.publish(TestEvent.Mute(EventFilter.custom { case _: Logging.Debug ⇒ true From 3f21c8a2a6570ded8d7dcbc1a73958b7c82f8d01 Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 4 Nov 2011 09:30:33 +0100 Subject: [PATCH 12/19] fix ActorDocSpec by allowing INFO loglevel to pass through --- akka-docs/scala/code/ActorDocSpec.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/akka-docs/scala/code/ActorDocSpec.scala b/akka-docs/scala/code/ActorDocSpec.scala index 72e7adad73..47f9fe0a89 100644 --- a/akka-docs/scala/code/ActorDocSpec.scala +++ b/akka-docs/scala/code/ActorDocSpec.scala @@ -8,6 +8,7 @@ import akka.util.duration._ //#imports import akka.actor.Actor import akka.event.Logging +import akka.config.Configuration //#imports @@ -21,7 +22,7 @@ class MyActor extends Actor { } //#my-actor -class ActorDocSpec extends AkkaSpec { +class ActorDocSpec extends AkkaSpec(Configuration("akka.loglevel" -> "INFO")) { "creating actor with AkkaSpec.actorOf" in { //#creating-actorOf From b4ab6730abcbe464bad864e9379fd625c390893b Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 4 Nov 2011 19:01:08 +0100 Subject: [PATCH 13/19] fix Logging.format by implementing {} replacement directly --- .../src/main/scala/akka/event/Logging.scala | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index ea5f63f5e1..603e7bea78 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -446,11 +446,20 @@ trait LoggingAdapter { 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) - + def format(t: String, arg: Any*) = { + val sb = new StringBuilder + var p = 0 + var rest = t + while (p < arg.length) { + val index = rest.indexOf("{}") + sb.append(rest.substring(0, index)) + sb.append(arg(p)) + rest = rest.substring(index + 2) + p += 1 + } + sb.append(rest) + sb.toString + } } class BusLogging(val bus: LoggingBus, val loggingInstance: AnyRef) extends LoggingAdapter { From 4f4227acf37124edf911e824e7218fa7ebd9e92f Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 4 Nov 2011 22:20:58 +0100 Subject: [PATCH 14/19] work-around compiler bug in ActiveRemoteClient It seems that 2.9.1 has a bug when a constructor argument is passed to superclass which makes a public field out of it under the same name: scalac got confused and generated invokespecial byte-code which is rejected by the verifier. One could argue that the verifier is too strict in this case ... So, I changed the names of the constructor arguments by prepending an underscore and thus avoiding the clash. --- .../main/scala/akka/remote/netty/NettyRemoteSupport.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index b4194f5926..91485a490e 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -252,12 +252,12 @@ abstract class RemoteClient private[akka] ( */ class ActiveRemoteClient private[akka] ( _app: AkkaApplication, - remoteSupport: RemoteSupport, - module: NettyRemoteClientModule, - remoteAddress: InetSocketAddress, + _remoteSupport: RemoteSupport, + _module: NettyRemoteClientModule, + _remoteAddress: InetSocketAddress, val loader: Option[ClassLoader] = None, notifyListenersFun: (RemoteLifeCycleEvent) ⇒ Unit) - extends RemoteClient(_app, remoteSupport, module, remoteAddress) { + extends RemoteClient(_app, _remoteSupport, _module, _remoteAddress) { val settings = new RemoteClientSettings(app) import settings._ From 6559511bce44433a5aa85cc8b7ac131d350fa89e Mon Sep 17 00:00:00 2001 From: Roland Date: Sun, 6 Nov 2011 11:55:45 +0100 Subject: [PATCH 15/19] FSMTimingSpec overhaul MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - revise timings to make them more robust (granularity changed in general from 100ms to 500ms) - make one test more deterministic by awaiting context.hasMessages, which has been added for this purpose; is the same kind of “non-racy” as isShutdown - add static version of TestKit.awaitCond (checked that it does not conflict, even when “import TestKit._” is used; seems like imports are not even searched if the symbol is found locally) --- .../test/scala/akka/actor/FSMTimingSpec.scala | 102 ++++++++++-------- .../src/main/scala/akka/actor/ActorCell.scala | 4 + .../src/main/scala/akka/testkit/TestKit.scala | 30 ++++++ 3 files changed, 92 insertions(+), 44 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala index 094bd4f196..df255f619f 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala @@ -4,9 +4,10 @@ package akka.actor -import akka.testkit.{ AkkaSpec, ImplicitSender } +import akka.testkit._ import akka.util.Duration import akka.util.duration._ +import akka.event.Logging @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class FSMTimingSpec extends AkkaSpec with ImplicitSender { @@ -15,7 +16,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { val fsm = actorOf(new StateMachine(testActor)) fsm ! SubscribeTransitionCallBack(testActor) - expectMsg(200 millis, CurrentState(fsm, Initial)) + expectMsg(1 second, CurrentState(fsm, Initial)) ignoreMsg { case Transition(_, Initial, _) ⇒ true @@ -24,8 +25,20 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { "A Finite State Machine" must { "receive StateTimeout" in { - within(50 millis, 250 millis) { + within(1 second) { + within(500 millis, 1 second) { + fsm ! TestStateTimeout + expectMsg(Transition(fsm, TestStateTimeout, Initial)) + } + expectNoMsg + } + } + + "cancel a StateTimeout" in { + within(1 second) { fsm ! TestStateTimeout + fsm ! Cancel + expectMsg(Cancel) expectMsg(Transition(fsm, TestStateTimeout, Initial)) expectNoMsg } @@ -36,7 +49,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { fsm ! TestStateTimeoutOverride expectNoMsg } - within(50 millis) { + within(500 millis) { fsm ! Cancel expectMsg(Cancel) expectMsg(Transition(fsm, TestStateTimeout, Initial)) @@ -44,70 +57,70 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { } "receive single-shot timer" in { - within(50 millis, 250 millis) { - fsm ! TestSingleTimer - expectMsg(Tick) - expectMsg(Transition(fsm, TestSingleTimer, Initial)) + within(1.5 seconds) { + within(500 millis, 1 second) { + fsm ! TestSingleTimer + expectMsg(Tick) + expectMsg(Transition(fsm, TestSingleTimer, Initial)) + } expectNoMsg } } "correctly cancel a named timer" in { fsm ! TestCancelTimer - within(100 millis, 200 millis) { + within(500 millis) { fsm ! Tick expectMsg(Tick) + } + within(300 millis, 1 second) { expectMsg(Tock) } fsm ! Cancel - expectMsg(Transition(fsm, TestCancelTimer, Initial)) + expectMsg(1 second, Transition(fsm, TestCancelTimer, Initial)) } "not get confused between named and state timers" in { fsm ! TestCancelStateTimerInNamedTimerMessage fsm ! Tick - expectMsg(100 millis, Tick) - Thread.sleep(200) + expectMsg(500 millis, Tick) + Thread.sleep(200) // this is ugly: need to wait for StateTimeout to be queued resume(fsm) - expectMsg(100 millis, Transition(fsm, TestCancelStateTimerInNamedTimerMessage, TestCancelStateTimerInNamedTimerMessage2)) + expectMsg(500 millis, Transition(fsm, TestCancelStateTimerInNamedTimerMessage, TestCancelStateTimerInNamedTimerMessage2)) fsm ! Cancel - within(100 millis) { - expectMsg(Cancel) + within(500 millis) { + expectMsg(Cancel) // if this is not received, that means StateTimeout was not properly discarded expectMsg(Transition(fsm, TestCancelStateTimerInNamedTimerMessage2, Initial)) } } "receive and cancel a repeated timer" in { fsm ! TestRepeatedTimer - val seq = receiveWhile(600 millis) { + val seq = receiveWhile(1 second) { case Tick ⇒ Tick } - seq must have length (5) - within(250 millis) { + seq must have length 5 + within(500 millis) { expectMsg(Transition(fsm, TestRepeatedTimer, Initial)) - expectNoMsg } } "notify unhandled messages" in { - fsm ! TestUnhandled - within(200 millis) { - fsm ! Tick - expectNoMsg - } - within(200 millis) { - fsm ! SetHandler - fsm ! Tick - expectMsg(Unhandled(Tick)) - expectNoMsg - } - within(200 millis) { - fsm ! Unhandled("test") - expectNoMsg - } - within(200 millis) { - fsm ! Cancel - expectMsg(Transition(fsm, TestUnhandled, Initial)) + filterEvents(EventFilter.custom { + case Logging.Warning(`fsm`, "unhandled event Tick in state TestUnhandled") ⇒ true + case Logging.Warning(`fsm`, "unhandled event Unhandled(test) in state TestUnhandled") ⇒ true + case _ ⇒ false + }) { + fsm ! TestUnhandled + within(1 second) { + fsm ! Tick + fsm ! SetHandler + fsm ! Tick + expectMsg(Unhandled(Tick)) + fsm ! Unhandled("test") + fsm ! Cancel + expectMsg(Transition(fsm, TestUnhandled, Initial)) + } } } @@ -151,7 +164,7 @@ object FSMTimingSpec { startWith(Initial, 0) when(Initial) { case Ev(TestSingleTimer) ⇒ - setTimer("tester", Tick, 100 millis, false) + setTimer("tester", Tick, 500 millis, false) goto(TestSingleTimer) case Ev(TestRepeatedTimer) ⇒ setTimer("tester", Tick, 100 millis, true) @@ -160,7 +173,7 @@ object FSMTimingSpec { goto(TestStateTimeout) forMax (Duration.Inf) case Ev(x: FSMTimingSpec.State) ⇒ goto(x) } - when(TestStateTimeout, stateTimeout = 100 millis) { + when(TestStateTimeout, stateTimeout = 500 millis) { case Ev(StateTimeout) ⇒ goto(Initial) case Ev(Cancel) ⇒ goto(Initial) replying (Cancel) } @@ -173,9 +186,9 @@ object FSMTimingSpec { case Ev(Tick) ⇒ tester ! Tick setTimer("hallo", Tock, 1 milli, false) - Thread.sleep(10); + TestKit.awaitCond(context.hasMessages, 1 second) cancelTimer("hallo") - setTimer("hallo", Tock, 100 millis, false) + setTimer("hallo", Tock, 500 millis, false) stay case Ev(Tock) ⇒ tester ! Tock @@ -195,11 +208,12 @@ object FSMTimingSpec { } } when(TestCancelStateTimerInNamedTimerMessage) { - // FSM is suspended after processing this message and resumed 200ms later + // FSM is suspended after processing this message and resumed 500ms later case Ev(Tick) ⇒ suspend(self) - setTimer("named", Tock, 10 millis, false) - stay forMax (100 millis) replying Tick + setTimer("named", Tock, 1 millis, false) + TestKit.awaitCond(context.hasMessages, 1 second) + stay forMax (1 millis) replying Tick case Ev(Tock) ⇒ goto(TestCancelStateTimerInNamedTimerMessage2) } diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 180cbe5df7..b04791a6b7 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -23,6 +23,8 @@ private[akka] trait ActorContext extends ActorRefFactory with TypedActorFactory def self: ActorRef with ScalaActorRef + def hasMessages: Boolean + def receiveTimeout: Option[Long] def receiveTimeout_=(timeout: Option[Long]): Unit @@ -95,6 +97,8 @@ private[akka] class ActorCell( @volatile //This must be volatile since it isn't protected by the mailbox status var mailbox: Mailbox = _ + def hasMessages: Boolean = mailbox.hasMessages + def start(): Unit = { mailbox = dispatcher.createMailbox(this) diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index 1f4f0d680e..cb788a66d1 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -555,6 +555,36 @@ class TestKit(_app: AkkaApplication) { object TestKit { private[testkit] val testActorId = new AtomicInteger(0) + + /** + * Block until the given condition evaluates to `true` or the timeout + * expires, whichever comes first. + * + * If no timeout is given, take it from the innermost enclosing `within` + * block. + * + * Note that the timeout is scaled using Duration.timeFactor. + */ + def awaitCond(p: ⇒ Boolean, max: Duration, interval: Duration = 100.millis) { + val stop = now + max + + @tailrec + def poll(t: Duration) { + if (!p) { + assert(now < stop, "timeout " + max + " expired") + Thread.sleep(t.toMillis) + poll((stop - now) min interval) + } + } + + poll(max min interval) + } + + /** + * Obtain current timestamp as Duration for relative measurements (using System.nanoTime). + */ + def now: Duration = System.nanoTime().nanos + } /** From c1a9475015bf3c9a906fab4c9e15dd63262b63f4 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 7 Nov 2011 22:10:17 +0100 Subject: [PATCH 16/19] TestEventFilter overhaul - make normal filters available for all four log levels - allow filtering for + exact and complete message + start of message + regular expression - keep count of occurrences if requested and verify correct count at end of filterEvents/filterExceptions block - remove akka.testkit.Testing (sleepFor replaced by Duration.dilated.sleep) - remove Duration.timeFactor (needs config -> AkkaApplication) - TestLatch needs config -> AkkaApplication (was forgotten because used directly System.getProperty) - lots of scaladoc for TestEventListener infrastructure --- .../ActorFireForgetRequestReplySpec.scala | 3 +- .../test/scala/akka/actor/ActorRefSpec.scala | 10 +- .../scala/akka/actor/DeathWatchSpec.scala | 2 +- .../test/scala/akka/actor/FSMActorSpec.scala | 25 +- .../actor/LocalActorRefProviderSpec.scala | 1 - .../scala/akka/actor/SupervisorSpec.scala | 10 +- .../scala/akka/actor/SupervisorTreeSpec.scala | 1 - .../akka/actor/dispatch/ActorModelSpec.scala | 24 +- .../scala/akka/routing/ActorPoolSpec.scala | 31 +- .../src/main/scala/akka/AkkaApplication.scala | 5 +- .../main/scala/akka/dispatch/Mailbox.scala | 1 - .../src/main/scala/akka/util/Duration.scala | 13 +- .../transactor/CoordinatedIncrementSpec.scala | 6 +- .../scala/transactor/FickleFriendsSpec.scala | 6 +- .../scala/transactor/TransactorSpec.scala | 6 +- .../main/scala/akka/testkit/TestBarrier.scala | 10 +- .../akka/testkit/TestEventListener.scala | 459 ++++++++++++++++-- .../src/main/scala/akka/testkit/TestKit.scala | 6 - .../main/scala/akka/testkit/TestLatch.scala | 13 +- .../src/main/scala/akka/testkit/Testing.scala | 21 - .../src/main/scala/akka/testkit/package.scala | 16 +- .../scala/akka/testkit/TestActorRefSpec.scala | 4 +- .../scala/akka/testkit/TestTimeSpec.scala | 17 +- 23 files changed, 508 insertions(+), 182 deletions(-) delete mode 100644 akka-testkit/src/main/scala/akka/testkit/Testing.scala diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala index 2a327a35d9..a6fde4389f 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala @@ -6,7 +6,6 @@ package akka.actor import akka.testkit._ import org.scalatest.BeforeAndAfterEach -import akka.testkit.Testing.sleepFor import akka.util.duration._ import akka.dispatch.Dispatchers @@ -84,7 +83,7 @@ class ActorFireForgetRequestReplySpec extends AkkaSpec with BeforeAndAfterEach { actor.isShutdown must be(false) actor ! "Die" state.finished.await - sleepFor(1 second) + 1.second.dilated.sleep() actor.isShutdown must be(true) supervisor.stop() } diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala index 5fdf0487e5..f915ed591c 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -9,7 +9,6 @@ import org.scalatest.matchers.MustMatchers import akka.testkit._ import akka.util.duration._ -import akka.testkit.Testing.sleepFor import java.lang.IllegalStateException import akka.util.ReflectiveAccess import akka.dispatch.{ DefaultPromise, Promise, Future } @@ -20,8 +19,6 @@ object ActorRefSpec { case class ReplyTo(channel: Channel[Any]) - val latch = TestLatch(4) - class ReplyActor extends Actor { var replyTo: Channel[Any] = null @@ -53,11 +50,11 @@ object ActorRefSpec { } private def work { - sleepFor(1 second) + 1.second.dilated.sleep } } - class SenderActor(replyActor: ActorRef) extends Actor { + class SenderActor(replyActor: ActorRef, latch: TestLatch) extends Actor { def receive = { case "complex" ⇒ replyActor ! "complexRequest" @@ -343,8 +340,9 @@ class ActorRefSpec extends AkkaSpec { } "support reply via channel" in { + val latch = new TestLatch(4) val serverRef = actorOf(Props[ReplyActor]) - val clientRef = actorOf(Props(new SenderActor(serverRef))) + val clientRef = actorOf(Props(new SenderActor(serverRef, latch))) clientRef ! "complex" clientRef ! "simple" diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala index 485de60d42..c298a73ac3 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala @@ -103,7 +103,7 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende } "fail a monitor which does not handle Terminated()" in { - filterEvents(EventFilter[ActorKilledException], EventFilter[DeathPactException]) { + filterEvents(EventFilter[ActorKilledException](), EventFilter[DeathPactException]()) { case class FF(fail: Failed) val supervisor = actorOf(Props[Supervisor] .withFaultHandler(new OneForOneStrategy(FaultHandlingStrategy.makeDecider(List(classOf[Exception])), Some(0)) { diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala index e94cbaaa02..b6e972513f 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala @@ -18,19 +18,23 @@ import akka.config.Configuration object FSMActorSpec { - val unlockedLatch = TestLatch() - val lockedLatch = TestLatch() - val unhandledLatch = TestLatch() - val terminatedLatch = TestLatch() - val transitionLatch = TestLatch() - val initialStateLatch = TestLatch() - val transitionCallBackLatch = TestLatch() + class Latches(implicit app: AkkaApplication) { + val unlockedLatch = TestLatch() + val lockedLatch = TestLatch() + val unhandledLatch = TestLatch() + val terminatedLatch = TestLatch() + val transitionLatch = TestLatch() + val initialStateLatch = TestLatch() + val transitionCallBackLatch = TestLatch() + } sealed trait LockState case object Locked extends LockState case object Open extends LockState - class Lock(code: String, timeout: Duration) extends Actor with FSM[LockState, CodeState] { + class Lock(code: String, timeout: Duration, latches: Latches) extends Actor with FSM[LockState, CodeState] { + + import latches._ startWith(Locked, CodeState("", code)) @@ -107,8 +111,11 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true "unlock the lock" in { + val latches = new Latches + import latches._ + // lock that locked after being open for 1 sec - val lock = actorOf(new Lock("33221", 1 second)) + val lock = actorOf(new Lock("33221", 1 second, latches)) val transitionTester = actorOf(new Actor { def receive = { diff --git a/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala index 9f43c0232f..5e58dfdaa9 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala @@ -6,7 +6,6 @@ package akka.actor import akka.testkit._ import akka.util.duration._ -import akka.testkit.Testing.sleepFor import java.util.concurrent.{ TimeUnit, CountDownLatch } diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala index 0f90840564..81032cf78d 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala @@ -5,7 +5,6 @@ package akka.actor import org.scalatest.BeforeAndAfterEach -import akka.testkit.Testing.sleepFor import akka.util.duration._ import akka.{ Die, Ping } import akka.actor.Actor._ @@ -17,7 +16,6 @@ import akka.testkit.AkkaSpec object SupervisorSpec { val Timeout = 5 seconds - val TimeoutMillis = Timeout.dilated.toMillis.toInt // ===================================================== // Message logs @@ -67,6 +65,8 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende import SupervisorSpec._ + val TimeoutMillis = Timeout.dilated.toMillis.toInt + // ===================================================== // Creating actors and supervisors // ===================================================== @@ -144,7 +144,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende master ! Die expectMsg(3 seconds, "terminated") - sleepFor(1 second) + 1.second.dilated.sleep messageLogPoll must be(null) } @@ -155,7 +155,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende (temporaryActor.?(Die, TimeoutMillis)).get } - sleepFor(1 second) + 1.second.dilated.sleep messageLog.size must be(0) } @@ -299,7 +299,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende } // give time for restart - sleepFor(3 seconds) + 3.seconds.dilated.sleep (dyingActor.?(Ping, TimeoutMillis)).as[String].getOrElse("nil") must be === PongMessage diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala index e5b6283c36..8834689c39 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala @@ -6,7 +6,6 @@ package akka.actor import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers import akka.util.duration._ -import akka.testkit.Testing.sleepFor import akka.dispatch.Dispatchers import akka.actor.Actor._ import akka.testkit.{ TestKit, EventFilter, filterEvents, filterException } diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index 54a6d3bd0e..d1e197738e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -4,7 +4,7 @@ package akka.actor.dispatch import org.scalatest.Assertions._ -import akka.testkit.{ Testing, filterEvents, EventFilter, AkkaSpec } +import akka.testkit.{ filterEvents, EventFilter, AkkaSpec } import akka.dispatch._ import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.{ ConcurrentHashMap, CountDownLatch, TimeUnit } @@ -285,13 +285,13 @@ abstract class ActorModelSpec extends AkkaSpec { val a = newTestActor(dispatcher) a ! CountDown(start) - assertCountDown(start, Testing.testTime(3000), "Should process first message within 3 seconds") + assertCountDown(start, 3.seconds.dilated.toMillis, "Should process first message within 3 seconds") assertRefDefaultZero(a)(registers = 1, msgsReceived = 1, msgsProcessed = 1) a ! Wait(1000) a ! CountDown(oneAtATime) // in case of serialization violation, restart would happen instead of count down - assertCountDown(oneAtATime, Testing.testTime(1500), "Processed message when allowed") + assertCountDown(oneAtATime, (1.5 seconds).dilated.toMillis, "Processed message when allowed") assertRefDefaultZero(a)(registers = 1, msgsReceived = 3, msgsProcessed = 3) a.stop() @@ -310,7 +310,7 @@ abstract class ActorModelSpec extends AkkaSpec { } } } - assertCountDown(counter, Testing.testTime(3000), "Should process 200 messages") + assertCountDown(counter, 3.seconds.dilated.toMillis, "Should process 200 messages") assertRefDefaultZero(a)(registers = 1, msgsReceived = 200, msgsProcessed = 200) a.stop() @@ -339,7 +339,7 @@ abstract class ActorModelSpec extends AkkaSpec { assertRefDefaultZero(a)(registers = 1, msgsReceived = 1, suspensions = 1) a.resume - assertCountDown(done, Testing.testTime(3000), "Should resume processing of messages when resumed") + assertCountDown(done, 3.seconds.dilated.toMillis, "Should resume processing of messages when resumed") assertRefDefaultZero(a)(registers = 1, msgsReceived = 1, msgsProcessed = 1, suspensions = 1, resumes = 1) @@ -360,7 +360,7 @@ abstract class ActorModelSpec extends AkkaSpec { }).withDispatcher(wavesSupervisorDispatcher(dispatcher))) boss ! "run" try { - assertCountDown(cachedMessage.latch, Testing.testTime(10000), "Should process " + num + " countdowns") + assertCountDown(cachedMessage.latch, 10.seconds.dilated.toMillis, "Should process " + num + " countdowns") } catch { case e ⇒ System.err.println("Error: " + e.getMessage + " missing count downs == " + cachedMessage.latch.getCount() + " out of " + num) @@ -374,7 +374,7 @@ abstract class ActorModelSpec extends AkkaSpec { } "continue to process messages when a thread gets interrupted" in { - filterEvents(EventFilter[InterruptedException], EventFilter[akka.event.Logging.EventHandlerException]) { + filterEvents(EventFilter[InterruptedException](), EventFilter[akka.event.Logging.EventHandlerException]()) { implicit val dispatcher = newInterceptedDispatcher implicit val timeout = Timeout(5 seconds) val a = newTestActor(dispatcher) @@ -408,7 +408,7 @@ abstract class ActorModelSpec extends AkkaSpec { } "continue to process messages when exception is thrown" in { - filterEvents(EventFilter[IndexOutOfBoundsException], EventFilter[RemoteException]) { + filterEvents(EventFilter[IndexOutOfBoundsException](), EventFilter[RemoteException]()) { implicit val dispatcher = newInterceptedDispatcher val a = newTestActor(dispatcher) val f1 = a ? Reply("foo") @@ -467,10 +467,10 @@ class DispatcherModelSpec extends ActorModelSpec { val a, b = newTestActor(dispatcher) a ! Meet(aStart, aStop) - assertCountDown(aStart, Testing.testTime(3000), "Should process first message within 3 seconds") + assertCountDown(aStart, 3.seconds.dilated.toMillis, "Should process first message within 3 seconds") b ! CountDown(bParallel) - assertCountDown(bParallel, Testing.testTime(3000), "Should process other actors in parallel") + assertCountDown(bParallel, 3.seconds.dilated.toMillis, "Should process other actors in parallel") aStop.countDown() @@ -506,10 +506,10 @@ class BalancingDispatcherModelSpec extends ActorModelSpec { val a, b = newTestActor(dispatcher) a ! Meet(aStart, aStop) - assertCountDown(aStart, Testing.testTime(3000), "Should process first message within 3 seconds") + assertCountDown(aStart, 3.seconds.dilated.toMillis, "Should process first message within 3 seconds") b ! CountDown(bParallel) - assertCountDown(bParallel, Testing.testTime(3000), "Should process other actors in parallel") + assertCountDown(bParallel, 3.seconds.dilated.toMillis, "Should process other actors in parallel") aStop.countDown() diff --git a/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala index ad48435996..13b2a5d8af 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala @@ -2,7 +2,6 @@ package akka.routing import akka.dispatch.{ KeptPromise, Future } import akka.actor._ -import akka.testkit.Testing._ import akka.testkit.{ TestLatch, filterEvents, EventFilter, filterException } import akka.util.duration._ import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger } @@ -88,7 +87,7 @@ class ActorPoolSpec extends AkkaSpec { def instance(p: Props) = actorOf(p.withCreator(new Actor { def receive = { case req: String ⇒ { - sleepFor(10 millis) + (10 millis).dilated.sleep channel.tryTell("Response") } } @@ -116,7 +115,7 @@ class ActorPoolSpec extends AkkaSpec { def instance(p: Props) = actorOf(p.withCreator(new Actor { def receive = { case n: Int ⇒ - sleepFor(n millis) + (n millis).dilated.sleep count.incrementAndGet latch.countDown() } @@ -142,7 +141,7 @@ class ActorPoolSpec extends AkkaSpec { count.set(0) for (m ← 0 until loops) { pool ? t - sleepFor(50 millis) + (50 millis).dilated.sleep } } @@ -180,7 +179,7 @@ class ActorPoolSpec extends AkkaSpec { def instance(p: Props) = actorOf(p.withCreator(new Actor { def receive = { case n: Int ⇒ - sleepFor(n millis) + (n millis).dilated.sleep count.incrementAndGet latch.countDown() } @@ -291,7 +290,7 @@ class ActorPoolSpec extends AkkaSpec { def instance(p: Props) = actorOf(p.withCreator(new Actor { def receive = { case n: Int ⇒ - sleepFor(n millis) + (n millis).dilated.sleep latch.countDown() } })) @@ -311,7 +310,7 @@ class ActorPoolSpec extends AkkaSpec { for (m ← 0 to 10) pool ! 250 - sleepFor(5 millis) + (5 millis).dilated.sleep val z = (pool ? ActorPool.Stat).as[ActorPool.Stats].get.size @@ -321,7 +320,7 @@ class ActorPoolSpec extends AkkaSpec { for (m ← 0 to 3) { pool ! 1 - sleepFor(500 millis) + (500 millis).dilated.sleep } (pool ? ActorPool.Stat).as[ActorPool.Stats].get.size must be <= (z) @@ -414,7 +413,7 @@ class ActorPoolSpec extends AkkaSpec { pool1 ! "ping" (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) pool1 ! akka.Die - sleepFor(2 seconds) + (2 seconds).dilated.sleep (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) pingCount.get must be(1) @@ -425,7 +424,7 @@ class ActorPoolSpec extends AkkaSpec { pool1 ! "ping" (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) pool1 ! akka.Die - sleepFor(2 seconds) + (2 seconds).dilated.sleep (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(1) pool1 ! "ping" (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) @@ -438,7 +437,7 @@ class ActorPoolSpec extends AkkaSpec { pool2 ! "ping" (pool2 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) pool2 ! akka.Die - sleepFor(2 seconds) + (2 seconds).dilated.sleep (pool2 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) pingCount.get must be(1) @@ -449,7 +448,7 @@ class ActorPoolSpec extends AkkaSpec { pool2 ! "ping" (pool2 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) pool2 ! akka.Die - sleepFor(2 seconds) + (2 seconds).dilated.sleep (pool2 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(1) pool2 ! "ping" (pool2 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) @@ -461,7 +460,7 @@ class ActorPoolSpec extends AkkaSpec { pool3 ! "ping" (pool3 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) pool3 ! akka.Die - sleepFor(2 seconds) + (2 seconds).dilated.sleep (pool3 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(1) pool3 ! "ping" pool3 ! "ping" @@ -472,7 +471,7 @@ class ActorPoolSpec extends AkkaSpec { } "support customizable supervision config of pooled actors" in { - filterEvents(EventFilter[IllegalStateException], EventFilter[RuntimeException]) { + filterEvents(EventFilter[IllegalStateException](), EventFilter[RuntimeException]()) { val pingCount = new AtomicInteger(0) val deathCount = new AtomicInteger(0) var keepDying = new AtomicBoolean(false) @@ -510,7 +509,7 @@ class ActorPoolSpec extends AkkaSpec { pool1 ! "ping" (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) pool1 ! BadState - sleepFor(2 seconds) + (2 seconds).dilated.sleep (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) pingCount.get must be(1) @@ -520,7 +519,7 @@ class ActorPoolSpec extends AkkaSpec { pool1 ! "ping" (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) pool1 ! BadState - sleepFor(2 seconds) + (2 seconds).dilated.sleep (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(1) pool1 ! "ping" (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) diff --git a/akka-actor/src/main/scala/akka/AkkaApplication.scala b/akka-actor/src/main/scala/akka/AkkaApplication.scala index 96cc0568be..31bf16423b 100644 --- a/akka-actor/src/main/scala/akka/AkkaApplication.scala +++ b/akka-actor/src/main/scala/akka/AkkaApplication.scala @@ -90,11 +90,14 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor val ProviderClass = getString("akka.actor.provider", "akka.actor.LocalActorRefProvider") - val DefaultTimeUnit = getString("akka.time-unit", "seconds") + val DefaultTimeUnit = Duration.timeUnit(getString("akka.time-unit", "seconds")) val ActorTimeout = Timeout(Duration(getInt("akka.actor.timeout", 5), DefaultTimeUnit)) val ActorTimeoutMillis = ActorTimeout.duration.toMillis val SerializeAllMessages = getBool("akka.actor.serialize-messages", false) + val TestTimeFactor = getDouble("akka.test.timefactor", 1.0) + val TestEventFilterLeeway = Duration(getDouble("akka.test.filter-leeway", 0.5), DefaultTimeUnit) + val LogLevel = getString("akka.loglevel", "INFO") val StdoutLogLevel = getString("akka.stdout-loglevel", LogLevel) val EventHandlers = getList("akka.event-handlers") diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 4222c6fd77..04dd2f8f0f 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -1,7 +1,6 @@ /** * Copyright (C) 2009-2011 Typesafe Inc. */ - package akka.dispatch import akka.AkkaException diff --git a/akka-actor/src/main/scala/akka/util/Duration.scala b/akka-actor/src/main/scala/akka/util/Duration.scala index cdf4347c30..3993a8a363 100644 --- a/akka-actor/src/main/scala/akka/util/Duration.scala +++ b/akka-actor/src/main/scala/akka/util/Duration.scala @@ -7,6 +7,7 @@ package akka.util import java.util.concurrent.TimeUnit import TimeUnit._ import java.lang.{ Long ⇒ JLong, Double ⇒ JDouble } +import akka.AkkaApplication class TimerException(message: String) extends RuntimeException(message) @@ -119,15 +120,6 @@ object Duration { case "ns" | "nano" | "nanos" | "nanosecond" | "nanoseconds" ⇒ NANOSECONDS } - /* - * Testing facilities - */ - val timeFactor: Double = { - val factor = System.getProperty("akka.test.timefactor", "1.0") - try { factor.toDouble } - catch { case e: java.lang.NumberFormatException ⇒ 1.0 } - } - val Zero: Duration = new FiniteDuration(0, NANOSECONDS) trait Infinite { @@ -272,9 +264,10 @@ abstract class Duration extends Serializable { def /(other: Duration): Double def unary_- : Duration def finite_? : Boolean - def dilated: Duration = this * Duration.timeFactor + def dilated(implicit app: AkkaApplication): Duration = this * app.AkkaConfig.TestTimeFactor def min(other: Duration): Duration = if (this < other) this else other def max(other: Duration): Duration = if (this > other) this else other + def sleep(): Unit = Thread.sleep(toMillis) // Java API def lt(other: Duration) = this < other diff --git a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala index e0f242a61b..763b9add63 100644 --- a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala +++ b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala @@ -82,9 +82,9 @@ class CoordinatedIncrementSpec extends AkkaSpec with BeforeAndAfterAll { "increment no counters with a failing transaction" in { val ignoreExceptions = Seq( - EventFilter[ExpectedFailureException], - EventFilter[CoordinatedTransactionException], - EventFilter[ActorTimeoutException]) + EventFilter[ExpectedFailureException](), + EventFilter[CoordinatedTransactionException](), + EventFilter[ActorTimeoutException]()) filterEvents(ignoreExceptions) { val (counters, failer) = actorOfs val coordinated = Coordinated() diff --git a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala index f8d0e8e8b2..6f2f21cafd 100644 --- a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala +++ b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala @@ -115,9 +115,9 @@ class FickleFriendsSpec extends AkkaSpec with BeforeAndAfterAll { "Coordinated fickle friends" should { "eventually succeed to increment all counters by one" in { val ignoreExceptions = Seq( - EventFilter[ExpectedFailureException], - EventFilter[CoordinatedTransactionException], - EventFilter[ActorTimeoutException]) + EventFilter[ExpectedFailureException](), + EventFilter[CoordinatedTransactionException](), + EventFilter[ActorTimeoutException]()) filterEvents(ignoreExceptions) { val (counters, coordinator) = actorOfs val latch = new CountDownLatch(1) diff --git a/akka-stm/src/test/scala/transactor/TransactorSpec.scala b/akka-stm/src/test/scala/transactor/TransactorSpec.scala index daea5134db..99b631e5b0 100644 --- a/akka-stm/src/test/scala/transactor/TransactorSpec.scala +++ b/akka-stm/src/test/scala/transactor/TransactorSpec.scala @@ -105,9 +105,9 @@ class TransactorSpec extends AkkaSpec { "increment no counters with a failing transaction" in { val ignoreExceptions = Seq( - EventFilter[ExpectedFailureException], - EventFilter[CoordinatedTransactionException], - EventFilter[ActorTimeoutException]) + EventFilter[ExpectedFailureException](), + EventFilter[CoordinatedTransactionException](), + EventFilter[ActorTimeoutException]()) filterEvents(ignoreExceptions) { val (counters, failer) = createTransactors val failLatch = TestLatch(numCounters) diff --git a/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala b/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala index 4c6c59f5f5..8e9a6eebcd 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala @@ -6,6 +6,7 @@ package akka.testkit import akka.util.Duration import java.util.concurrent.{ CyclicBarrier, TimeUnit, TimeoutException } +import akka.AkkaApplication class TestBarrierTimeoutException(message: String) extends RuntimeException(message) @@ -24,14 +25,15 @@ object TestBarrier { class TestBarrier(count: Int) { private val barrier = new CyclicBarrier(count) - def await(): Unit = await(TestBarrier.DefaultTimeout) + def await()(implicit app: AkkaApplication): Unit = await(TestBarrier.DefaultTimeout) - def await(timeout: Duration) { + def await(timeout: Duration)(implicit app: AkkaApplication) { try { - barrier.await(Testing.testTime(timeout.toNanos), TimeUnit.NANOSECONDS) + barrier.await(timeout.dilated.toNanos, TimeUnit.NANOSECONDS) } catch { case e: TimeoutException ⇒ - throw new TestBarrierTimeoutException("Timeout of %s and time factor of %s" format (timeout.toString, Duration.timeFactor)) + throw new TestBarrierTimeoutException("Timeout of %s and time factor of %s" + format (timeout.toString, app.AkkaConfig.TestTimeFactor)) } } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala index ce7ef422fe..9c4e421fa9 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala @@ -1,10 +1,36 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ package akka.testkit -import akka.event.Logging.{ LogEvent, Error, InitializeLogger } -import akka.actor.Actor +import scala.util.matching.Regex +import akka.actor.Actor +import akka.event.Logging._ +import akka.event.Logging +import akka.util.Duration +import akka.AkkaApplication + +/** + * Implementation helpers of the EventFilter facilities: send `Mute` + * to the TestEventListener to install a filter, and `UnMute` to + * deinstall it. + * + * You should always prefer the filter methods in the package object + * (see [[akka.testkit]] `filterEvents` and `filterException`) or on the + * EventFilter implementations. + */ sealed trait TestEvent +/** + * Implementation helpers of the EventFilter facilities: send Mute + * to the TestEventFilter to install a filter, and UnMute to + * deinstall it. + * + * You should always prefer the filter methods in the package object + * (see [[akka.testkit]] `filterEvents` and `filterException`) or on the + * EventFilter implementations. + */ object TestEvent { object Mute { def apply(filter: EventFilter, filters: EventFilter*): Mute = new Mute(filter +: filters.toSeq) @@ -16,69 +42,398 @@ object TestEvent { case class UnMute(filters: Seq[EventFilter]) extends TestEvent } -trait EventFilter { - def apply(event: LogEvent): Boolean +/** + * Facilities for selectively filtering out expected events from logging so + * that you can keep your test run’s console output clean and do not miss real + * error messages. + * + * See the companion object for convenient factory methods. + * + * If the `occurrences` is set to Int.MaxValue, no tracking is done. + */ +abstract class EventFilter(occurrences: Int) { + + private var todo = occurrences + + /** + * This method decides whether to filter the event (true) or not + * (false). + */ + protected def matches(event: LogEvent): Boolean + + final def apply(event: LogEvent): Boolean = { + if (matches(event)) { + if (todo != Int.MaxValue) todo -= 1 + true + } else false + } + + def awaitDone(max: Duration): Boolean = { + if (todo != Int.MaxValue && todo > 0) TestKit.awaitCond(todo == 0, max) + todo == Int.MaxValue || todo == 0 + } + + /** + * Apply this filter while executing the given code block. Care is taken to + * remove the filter when the block is finished or aborted. + */ + def intercept[T](code: ⇒ T)(implicit app: AkkaApplication): T = { + app.mainbus publish TestEvent.Mute(this) + try { + val result = code + if (!awaitDone(app.AkkaConfig.TestEventFilterLeeway)) + throw new AssertionError("Timeout waiting for " + todo + " messages on " + this) + result + } finally app.mainbus publish TestEvent.UnMute(this) + } + + /* + * these default values are just there for easier subclassing + */ + protected val source: Option[AnyRef] = None + protected val message: Either[String, Regex] = Left("") + protected val complete: Boolean = false + /** + * internal implementation helper, no guaranteed API + */ + protected def doMatch(src: AnyRef, msg: Any) = { + val msgstr = if (msg != null) msg.toString else "null" + (source.isDefined && sourceMatch(src) || source.isEmpty) && + (message match { + case Left(s) ⇒ if (complete) msgstr == s else msgstr.startsWith(s) + case Right(p) ⇒ p.findFirstIn(msgstr).isDefined + }) + } + private def sourceMatch(src: AnyRef) = { + source.get match { + case c: Class[_] ⇒ c isInstance src + case s ⇒ src == s + } + } } +/** + * Facilities for selectively filtering out expected events from logging so + * that you can keep your test run’s console output clean and do not miss real + * error messages. + * + * '''Also have a look at the [[akka.testkit]] package object’s `filterEvents` and + * `filterException` methods.''' + * + * The source filters do accept `Class[_]` arguments, matching any + * object which is an instance of the given class, e.g. + * + * {{{ + * EventFilter.info(source = classOf[MyActor]) // will match Info events from any MyActor instance + * }}} + * + * The message object will be converted to a string before matching (`"null"` if it is `null`). + */ object EventFilter { - def apply[A <: Throwable: Manifest](): EventFilter = - ErrorFilter(manifest[A].erasure) + /** + * Create a filter for Error events. Give up to one of start and pattern: + * + * {{{ + * EventFilter[MyException]() // filter only on exception type + * EventFilter[MyException]("message") // filter on exactly matching message + * EventFilter[MyException](source = obj) // filter on event source + * EventFilter[MyException](start = "Expected") // filter on start of message + * EventFilter[MyException](source = obj, pattern = "weird.*message") // filter on pattern and message + * }}} + * + * ''Please note that filtering on the `source` being + * `null` does NOT work (passing `null` disables the + * source filter).'' + */ + def apply[A <: Throwable: Manifest](message: String = null, source: AnyRef = null, start: String = "", pattern: String = null, occurrences: Int = Int.MaxValue): EventFilter = + ErrorFilter(manifest[A].erasure, Option(source), + if (message ne null) Left(message) else Option(pattern) map (new Regex(_)) toRight start, + message ne null)(occurrences) - def apply[A <: Throwable: Manifest](message: String): EventFilter = - ErrorMessageFilter(manifest[A].erasure, message) + /** + * Create a filter for Warning events. Give up to one of start and pattern: + * + * {{{ + * EventFilter.warning() // filter only on exception type + * EventFilter.warning(source = obj) // filter on event source + * EventFilter.warning(start = "Expected") // filter on start of message + * EventFilter.warning(source = obj, pattern = "weird.*message") // filter on pattern and message + * }}} + * + * ''Please note that filtering on the `source` being + * `null` does NOT work (passing `null` disables the + * source filter).'' + */ + def warning(message: String = null, source: AnyRef = null, start: String = "", pattern: String = null, occurrences: Int = Int.MaxValue): EventFilter = + WarningFilter(Option(source), + if (message ne null) Left(message) else Option(pattern) map (new Regex(_)) toRight start, + message ne null)(occurrences) - def apply[A <: Throwable: Manifest](source: AnyRef): EventFilter = - ErrorSourceFilter(manifest[A].erasure, source) + /** + * Create a filter for Info events. Give up to one of start and pattern: + * + * {{{ + * EventFilter.info() // filter only on exception type + * EventFilter.info(source = obj) // filter on event source + * EventFilter.info(start = "Expected") // filter on start of message + * EventFilter.info(source = obj, pattern = "weird.*message") // filter on pattern and message + * }}} + * + * ''Please note that filtering on the `source` being + * `null` does NOT work (passing `null` disables the + * source filter).'' + */ + def info(message: String = null, source: AnyRef = null, start: String = "", pattern: String = null, occurrences: Int = Int.MaxValue): EventFilter = + InfoFilter(Option(source), + if (message ne null) Left(message) else Option(pattern) map (new Regex(_)) toRight start, + message ne null)(occurrences) - def apply[A <: Throwable: Manifest](source: AnyRef, message: String): EventFilter = - ErrorSourceMessageFilter(manifest[A].erasure, source, message) + /** + * Create a filter for Debug events. Give up to one of start and pattern: + * + * {{{ + * EventFilter.debug() // filter only on exception type + * EventFilter.debug(source = obj) // filter on event source + * EventFilter.debug(start = "Expected") // filter on start of message + * EventFilter.debug(source = obj, pattern = "weird.*message") // filter on pattern and message + * }}} + * + * ''Please note that filtering on the `source` being + * `null` does NOT work (passing `null` disables the + * source filter).'' + */ + def debug(message: String = null, source: AnyRef = null, start: String = "", pattern: String = null, occurrences: Int = Int.MaxValue): EventFilter = + DebugFilter(Option(source), + if (message ne null) Left(message) else Option(pattern) map (new Regex(_)) toRight start, + message ne null)(occurrences) - def custom(test: (LogEvent) ⇒ Boolean): EventFilter = - CustomEventFilter(test) + /** + * Create a custom event filter. The filter will affect those events for + * which the supplied partial function is defined and returns + * `true`. + * + * {{{ + * EventFilter.custom { + * case Warning(ref, "my warning") if ref == actor || ref == null => true + * } + * }}} + */ + def custom(test: PartialFunction[LogEvent, Boolean], occurrences: Int = Int.MaxValue): EventFilter = + CustomEventFilter(test)(occurrences) } -case class ErrorFilter(throwable: Class[_]) extends EventFilter { - def apply(event: LogEvent) = event match { - case Error(cause, _, _) ⇒ throwable isInstance cause - case _ ⇒ false +/** + * Filter which matches Error events, if they satisfy the given criteria: + *
    + *
  • throwable applies an upper bound on the type of exception contained in the Error event
  • + *
  • source, if given, applies a filter on the event’s origin
  • + *
  • message applies a filter on the event’s message (either + * with String.startsWith or Regex.findFirstIn().isDefined); if the message + * itself does not match, the match is retried with the contained Exception’s + * message; if both are null, the filter always matches if at + * the same time the Exception’s stack trace is empty (this catches + * JVM-omitted “fast-throw” exceptions)
  • + *
+ * If you want to match all Error events, the most efficient is to use Left(""). + */ +case class ErrorFilter( + throwable: Class[_], + override val source: Option[AnyRef], + override val message: Either[String, Regex], + override val complete: Boolean)(occurrences: Int) extends EventFilter(occurrences) { + + def matches(event: LogEvent) = { + event match { + case Error(cause, src, msg) if throwable isInstance cause ⇒ + (msg == null && cause.getMessage == null && cause.getStackTrace.length == 0) || + doMatch(src, msg) || doMatch(src, cause.getMessage) + case _ ⇒ false + } + } + + /** + * Java API + * + * @param source + * apply this filter only to events from the given source; do not filter on source if this is given as null + * @param message + * apply this filter only to events whose message matches; do not filter on message if this is given as null + * @param pattern + * if false, the message string must start with the given + * string, otherwise the message argument is treated as + * regular expression which is matched against the message (may match only + * a substring to filter) + * @param complete + * whether the event’s message must match the given message string or pattern completely + */ + def this(throwable: Class[_], source: AnyRef, message: String, pattern: Boolean, complete: Boolean, occurrences: Int) = + this(throwable, Option(source), + if (message eq null) Left("") + else if (pattern) Right(new Regex(message)) + else Left(message), + complete)(occurrences) + + /** + * Java API: filter only on the given type of exception + */ + def this(throwable: Class[_]) = this(throwable, null, null, false, false, Int.MaxValue) + +} + +/** + * Filter which matches Warning events, if they satisfy the given criteria: + *
    + *
  • source, if given, applies a filter on the event’s origin
  • + *
  • message applies a filter on the event’s message (either with String.startsWith or Regex.findFirstIn().isDefined)
  • + *
+ * If you want to match all Warning events, the most efficient is to use Left(""). + */ +case class WarningFilter( + override val source: Option[AnyRef], + override val message: Either[String, Regex], + override val complete: Boolean)(occurrences: Int) extends EventFilter(occurrences) { + + def matches(event: LogEvent) = { + event match { + case Warning(src, msg) ⇒ doMatch(src, msg) + case _ ⇒ false + } + } + + /** + * Java API + * + * @param source + * apply this filter only to events from the given source; do not filter on source if this is given as null + * @param message + * apply this filter only to events whose message matches; do not filter on message if this is given as null + * @param pattern + * if false, the message string must start with the given + * string, otherwise the message argument is treated as + * regular expression which is matched against the message (may match only + * a substring to filter) + * @param complete + * whether the event’s message must match the given message string or pattern completely + */ + def this(source: AnyRef, message: String, pattern: Boolean, complete: Boolean, occurrences: Int) = + this(Option(source), + if (message eq null) Left("") + else if (pattern) Right(new Regex(message)) + else Left(message), + complete)(occurrences) +} + +/** + * Filter which matches Info events, if they satisfy the given criteria: + *
    + *
  • source, if given, applies a filter on the event’s origin
  • + *
  • message applies a filter on the event’s message (either with String.startsWith or Regex.findFirstIn().isDefined)
  • + *
+ * If you want to match all Info events, the most efficient is to use Left(""). + */ +case class InfoFilter( + override val source: Option[AnyRef], + override val message: Either[String, Regex], + override val complete: Boolean)(occurrences: Int) extends EventFilter(occurrences) { + + def matches(event: LogEvent) = { + event match { + case Info(src, msg) ⇒ doMatch(src, msg) + case _ ⇒ false + } + } + + /** + * Java API + * + * @param source + * apply this filter only to events from the given source; do not filter on source if this is given as null + * @param message + * apply this filter only to events whose message matches; do not filter on message if this is given as null + * @param pattern + * if false, the message string must start with the given + * string, otherwise the message argument is treated as + * regular expression which is matched against the message (may match only + * a substring to filter) + * @param complete + * whether the event’s message must match the given message string or pattern completely + */ + def this(source: AnyRef, message: String, pattern: Boolean, complete: Boolean, occurrences: Int) = + this(Option(source), + if (message eq null) Left("") + else if (pattern) Right(new Regex(message)) + else Left(message), + complete)(occurrences) +} + +/** + * Filter which matches Debug events, if they satisfy the given criteria: + *
    + *
  • source, if given, applies a filter on the event’s origin
  • + *
  • message applies a filter on the event’s message (either with String.startsWith or Regex.findFirstIn().isDefined)
  • + *
+ * If you want to match all Debug events, the most efficient is to use Left(""). + */ +case class DebugFilter( + override val source: Option[AnyRef], + override val message: Either[String, Regex], + override val complete: Boolean)(occurrences: Int) extends EventFilter(occurrences) { + + def matches(event: LogEvent) = { + event match { + case Debug(src, msg) ⇒ doMatch(src, msg) + case _ ⇒ false + } + } + + /** + * Java API + * + * @param source + * apply this filter only to events from the given source; do not filter on source if this is given as null + * @param message + * apply this filter only to events whose message matches; do not filter on message if this is given as null + * @param pattern + * if false, the message string must start with the given + * string, otherwise the message argument is treated as + * regular expression which is matched against the message (may match only + * a substring to filter) + * @param complete + * whether the event’s message must match the given message string or pattern completely + */ + def this(source: AnyRef, message: String, pattern: Boolean, complete: Boolean, occurrences: Int) = + this(Option(source), + if (message eq null) Left("") + else if (pattern) Right(new Regex(message)) + else Left(message), + complete)(occurrences) +} + +/** + * Custom event filter when the others do not fit the bill. + * + * If the partial function is defined and returns true, filter the event. + */ +case class CustomEventFilter(test: PartialFunction[LogEvent, Boolean])(occurrences: Int) extends EventFilter(occurrences) { + def matches(event: LogEvent) = { + test.isDefinedAt(event) && test(event) } } -case class ErrorMessageFilter(throwable: Class[_], message: String) extends EventFilter { - def apply(event: LogEvent) = event match { - case Error(cause, _, _) if !(throwable isInstance cause) ⇒ false - case Error(cause, _, null) if cause.getMessage eq null ⇒ cause.getStackTrace.length == 0 - case Error(cause, _, null) ⇒ cause.getMessage startsWith message - case Error(cause, _, msg) ⇒ - (msg.toString startsWith message) || (cause.getMessage startsWith message) - case _ ⇒ false - } -} - -case class ErrorSourceFilter(throwable: Class[_], source: AnyRef) extends EventFilter { - def apply(event: LogEvent) = event match { - case Error(cause, instance, _) ⇒ (throwable isInstance cause) && (source eq instance) - case _ ⇒ false - } -} - -case class ErrorSourceMessageFilter(throwable: Class[_], source: AnyRef, message: String) extends EventFilter { - def apply(event: LogEvent) = event match { - case Error(cause, instance, _) if !((throwable isInstance cause) && (source eq instance)) ⇒ false - case Error(cause, _, null) if cause.getMessage eq null ⇒ cause.getStackTrace.length == 0 - case Error(cause, _, null) ⇒ cause.getMessage startsWith message - case Error(cause, _, msg) ⇒ - (msg.toString startsWith message) || (cause.getMessage startsWith message) - case _ ⇒ false - } -} - -case class CustomEventFilter(test: (LogEvent) ⇒ Boolean) extends EventFilter { - def apply(event: LogEvent) = test(event) -} - -class TestEventListener extends akka.event.Logging.DefaultLogger { +/** + * EventListener for running tests, which allows selectively filtering out + * expected messages. To use it, include something like this into + * akka.test.conf and run your tests with system property + * "akka.mode" set to "test": + * + *

+ * akka {
+ *   event-handlers = ["akka.testkit.TestEventListener"]
+ * }
+ * 
+ */ +class TestEventListener extends Logging.DefaultLogger { import TestEvent._ var filters: List[EventFilter] = Nil diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index cb788a66d1..d0cee3c740 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -452,12 +452,6 @@ class TestKit(_app: AkkaApplication) { lastWasNoMsg = true } - /** - * Same as `receiveWhile(remaining)(f)`, but correctly treating the timeFactor. - */ - @deprecated("insert empty first parameter list", "1.2") - def receiveWhile[T](f: PartialFunction[AnyRef, T]): Seq[T] = receiveWhile(remaining / Duration.timeFactor)(f) - /** * Receive a series of messages until one does not match the given partial * function or the idle timeout is met (disabled by default) or the overall diff --git a/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala b/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala index 3c852bf75f..fa32257b0e 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala @@ -6,6 +6,7 @@ package akka.testkit import akka.util.Duration import java.util.concurrent.{ CountDownLatch, TimeUnit } +import akka.AkkaApplication class TestLatchTimeoutException(message: String) extends RuntimeException(message) class TestLatchNoTimeoutException(message: String) extends RuntimeException(message) @@ -20,10 +21,10 @@ class TestLatchNoTimeoutException(message: String) extends RuntimeException(mess object TestLatch { val DefaultTimeout = Duration(5, TimeUnit.SECONDS) - def apply(count: Int = 1) = new TestLatch(count) + def apply(count: Int = 1)(implicit app: AkkaApplication) = new TestLatch(count) } -class TestLatch(count: Int = 1) { +class TestLatch(count: Int = 1)(implicit app: AkkaApplication) { private var latch = new CountDownLatch(count) def countDown() = latch.countDown() @@ -33,9 +34,9 @@ class TestLatch(count: Int = 1) { def await(): Boolean = await(TestLatch.DefaultTimeout) def await(timeout: Duration): Boolean = { - val opened = latch.await(Testing.testTime(timeout.toNanos), TimeUnit.NANOSECONDS) + val opened = latch.await(timeout.dilated.toNanos, TimeUnit.NANOSECONDS) if (!opened) throw new TestLatchTimeoutException( - "Timeout of %s with time factor of %s" format (timeout.toString, Duration.timeFactor)) + "Timeout of %s with time factor of %s" format (timeout.toString, app.AkkaConfig.TestTimeFactor)) opened } @@ -43,9 +44,9 @@ class TestLatch(count: Int = 1) { * Timeout is expected. Throws exception if latch is opened before timeout. */ def awaitTimeout(timeout: Duration = TestLatch.DefaultTimeout) = { - val opened = latch.await(Testing.testTime(timeout.toNanos), TimeUnit.NANOSECONDS) + val opened = latch.await(timeout.dilated.toNanos, TimeUnit.NANOSECONDS) if (opened) throw new TestLatchNoTimeoutException( - "Latch opened before timeout of %s with time factor of %s" format (timeout.toString, Duration.timeFactor)) + "Latch opened before timeout of %s with time factor of %s" format (timeout.toString, app.AkkaConfig.TestTimeFactor)) opened } diff --git a/akka-testkit/src/main/scala/akka/testkit/Testing.scala b/akka-testkit/src/main/scala/akka/testkit/Testing.scala deleted file mode 100644 index 70e4929b54..0000000000 --- a/akka-testkit/src/main/scala/akka/testkit/Testing.scala +++ /dev/null @@ -1,21 +0,0 @@ -/** - * Copyright (C) 2009-2011 Typesafe Inc. - */ - -package akka.testkit - -import akka.util.Duration -import Duration.timeFactor - -/** - * Multiplying numbers used in test timeouts by a factor, set by system property. - * Useful for Jenkins builds (where the machine may need more time). - */ -object Testing { - def testTime(t: Int): Int = (timeFactor * t).toInt - def testTime(t: Long): Long = (timeFactor * t).toLong - def testTime(t: Float): Float = (timeFactor * t).toFloat - def testTime(t: Double): Double = timeFactor * t - - def sleepFor(duration: Duration) = Thread.sleep(testTime(duration.toMillis)) -} diff --git a/akka-testkit/src/main/scala/akka/testkit/package.scala b/akka-testkit/src/main/scala/akka/testkit/package.scala index 384baa3d11..975b426297 100644 --- a/akka-testkit/src/main/scala/akka/testkit/package.scala +++ b/akka-testkit/src/main/scala/akka/testkit/package.scala @@ -1,10 +1,22 @@ package akka +import akka.util.Duration +import java.util.concurrent.TimeUnit.MILLISECONDS + package object testkit { def filterEvents[T](eventFilters: Iterable[EventFilter])(block: ⇒ T)(implicit app: AkkaApplication): T = { + def now = System.currentTimeMillis + app.mainbus.publish(TestEvent.Mute(eventFilters.toSeq)) try { - block + val result = block + + val stop = now + app.AkkaConfig.TestEventFilterLeeway.toMillis + val failed = eventFilters filterNot (_.awaitDone(Duration(stop - now, MILLISECONDS))) map ("Timeout waiting for " + _) + if (failed.nonEmpty) + throw new AssertionError("Filter completion error:\n" + failed.mkString("\n")) + + result } finally { app.mainbus.publish(TestEvent.UnMute(eventFilters.toSeq)) } @@ -12,5 +24,5 @@ package object testkit { def filterEvents[T](eventFilters: EventFilter*)(block: ⇒ T)(implicit app: AkkaApplication): T = filterEvents(eventFilters.toSeq)(block) - def filterException[T <: Throwable](block: ⇒ Unit)(implicit app: AkkaApplication, m: Manifest[T]): Unit = filterEvents(Seq(EventFilter[T]))(block) + def filterException[T <: Throwable](block: ⇒ Unit)(implicit app: AkkaApplication, m: Manifest[T]): Unit = EventFilter[T]() intercept (block) } diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index 6720299337..0af6a976f5 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -156,7 +156,7 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach { } "stop when sent a poison pill" in { - filterEvents(EventFilter[ActorKilledException]) { + EventFilter[ActorKilledException]() intercept { val a = TestActorRef(Props[WorkerActor]) intercept[ActorKilledException] { (a ? PoisonPill).get @@ -167,7 +167,7 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach { } "restart when Kill:ed" in { - filterEvents(EventFilter[ActorKilledException]) { + EventFilter[ActorKilledException]() intercept { counter = 2 val boss = TestActorRef(Props(new TActor { diff --git a/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala index ab3101b827..a8b50bc43a 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala @@ -3,23 +3,10 @@ package akka.testkit import org.scalatest.matchers.MustMatchers import org.scalatest.{ BeforeAndAfterEach, WordSpec } import akka.util.Duration +import akka.config.Configuration @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class TestTimeSpec extends AkkaSpec with BeforeAndAfterEach { - - val tf = Duration.timeFactor - - override def beforeEach { - val f = Duration.getClass.getDeclaredField("timeFactor") - f.setAccessible(true) - f.setDouble(Duration, 2.0) - } - - override def afterEach { - val f = Duration.getClass.getDeclaredField("timeFactor") - f.setAccessible(true) - f.setDouble(Duration, tf) - } +class TestTimeSpec extends AkkaSpec(Configuration("akka.test.timefactor" -> 2.0)) with BeforeAndAfterEach { "A TestKit" must { From b3249e0adb424ab036fef9e32d059eb2632f70d4 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 9 Nov 2011 11:04:39 +0100 Subject: [PATCH 17/19] finish EventFilter work and apply in three places - fix memory visibility issue with occurrences counter - add non-throwing awaitCond and use that for better error reporting - move occurrence book-keeping (side-effect) out of PF guard, was evaluated multiple times (of course!) - apply in cases where EventFilter.custom was used (one legitimate use case remains) --- .../test/scala/akka/actor/FSMActorSpec.scala | 67 +++++++++---------- .../test/scala/akka/actor/FSMTimingSpec.scala | 27 ++++---- .../akka/testkit/TestEventListener.scala | 22 +++--- .../src/main/scala/akka/testkit/TestKit.scala | 19 ++++-- 4 files changed, 68 insertions(+), 67 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala index b6e972513f..05716ea04b 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala @@ -65,7 +65,7 @@ object FSMActorSpec { whenUnhandled { case Ev(msg) ⇒ { - log.info("unhandled event " + msg + " in state " + stateName + " with data " + stateData) + log.warning("unhandled event " + msg + " in state " + stateName + " with data " + stateData) unhandledLatch.open stay } @@ -138,10 +138,7 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true transitionCallBackLatch.await lockedLatch.await - filterEvents(EventFilter.custom { - case Logging.Info(_: Lock, _) ⇒ true - case _ ⇒ false - }) { + EventFilter.warning(start = "unhandled event", occurrences = 1) intercept { lock ! "not_handled" unhandledLatch.await } @@ -200,40 +197,38 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true new TestKit(AkkaApplication("fsm event", AkkaApplication.defaultConfig ++ Configuration("akka.loglevel" -> "DEBUG", "akka.actor.debug.fsm" -> true))) { - app.mainbus.publish(TestEvent.Mute(EventFilter.custom { - case _: Logging.Debug ⇒ true - case _ ⇒ false - })) - val fsm = TestActorRef(new Actor with LoggingFSM[Int, Null] { - startWith(1, null) - when(1) { - case Ev("go") ⇒ - setTimer("t", Shutdown, 1.5 seconds, false) - goto(2) + EventFilter.debug() intercept { + val fsm = TestActorRef(new Actor with LoggingFSM[Int, Null] { + startWith(1, null) + when(1) { + case Ev("go") ⇒ + setTimer("t", Shutdown, 1.5 seconds, false) + goto(2) + } + when(2) { + case Ev("stop") ⇒ + cancelTimer("t") + stop + } + onTermination { + case StopEvent(r, _, _) ⇒ testActor ! r + } + }) + app.mainbus.subscribe(testActor, classOf[Logging.Debug]) + fsm ! "go" + expectMsgPF(1 second, hint = "processing Event(go,null)") { + case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(go,null) from Actor[testActor") ⇒ true } - when(2) { - case Ev("stop") ⇒ - cancelTimer("t") - stop + expectMsg(1 second, Logging.Debug(fsm, "setting timer 't'/1500 milliseconds: Shutdown")) + expectMsg(1 second, Logging.Debug(fsm, "transition 1 -> 2")) + fsm ! "stop" + expectMsgPF(1 second, hint = "processing Event(stop,null)") { + case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(stop,null) from Actor[testActor") ⇒ true } - onTermination { - case StopEvent(r, _, _) ⇒ testActor ! r - } - }) - app.mainbus.subscribe(testActor, classOf[Logging.Debug]) - fsm ! "go" - expectMsgPF(1 second, hint = "processing Event(go,null)") { - case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(go,null) from Actor[testActor") ⇒ true + expectMsgAllOf(1 second, Logging.Debug(fsm, "canceling timer 't'"), Normal) + expectNoMsg(1 second) + app.mainbus.unsubscribe(testActor) } - expectMsg(1 second, Logging.Debug(fsm, "setting timer 't'/1500 milliseconds: Shutdown")) - expectMsg(1 second, Logging.Debug(fsm, "transition 1 -> 2")) - fsm ! "stop" - expectMsgPF(1 second, hint = "processing Event(stop,null)") { - case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(stop,null) from Actor[testActor") ⇒ true - } - expectMsgAllOf(1 second, Logging.Debug(fsm, "canceling timer 't'"), Normal) - expectNoMsg(1 second) - app.mainbus.unsubscribe(testActor) } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala index df255f619f..bfda48fba9 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala @@ -106,22 +106,19 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { } "notify unhandled messages" in { - filterEvents(EventFilter.custom { - case Logging.Warning(`fsm`, "unhandled event Tick in state TestUnhandled") ⇒ true - case Logging.Warning(`fsm`, "unhandled event Unhandled(test) in state TestUnhandled") ⇒ true - case _ ⇒ false - }) { - fsm ! TestUnhandled - within(1 second) { - fsm ! Tick - fsm ! SetHandler - fsm ! Tick - expectMsg(Unhandled(Tick)) - fsm ! Unhandled("test") - fsm ! Cancel - expectMsg(Transition(fsm, TestUnhandled, Initial)) + filterEvents(EventFilter.warning("unhandled event Tick in state TestUnhandled", source = fsm, occurrences = 1), + EventFilter.warning("unhandled event Unhandled(test) in state TestUnhandled", source = fsm, occurrences = 1)) { + fsm ! TestUnhandled + within(1 second) { + fsm ! Tick + fsm ! SetHandler + fsm ! Tick + expectMsg(Unhandled(Tick)) + fsm ! Unhandled("test") + fsm ! Cancel + expectMsg(Transition(fsm, TestUnhandled, Initial)) + } } - } } } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala index 9c4e421fa9..1180fc52f8 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala @@ -48,11 +48,12 @@ object TestEvent { * error messages. * * See the companion object for convenient factory methods. - * + * * If the `occurrences` is set to Int.MaxValue, no tracking is done. */ abstract class EventFilter(occurrences: Int) { + @volatile // JMM does not guarantee visibility for non-final fields private var todo = occurrences /** @@ -69,7 +70,7 @@ abstract class EventFilter(occurrences: Int) { } def awaitDone(max: Duration): Boolean = { - if (todo != Int.MaxValue && todo > 0) TestKit.awaitCond(todo == 0, max) + if (todo != Int.MaxValue && todo > 0) TestKit.awaitCond(todo == 0, max, noThrow = true) todo == Int.MaxValue || todo == 0 } @@ -82,7 +83,10 @@ abstract class EventFilter(occurrences: Int) { try { val result = code if (!awaitDone(app.AkkaConfig.TestEventFilterLeeway)) - throw new AssertionError("Timeout waiting for " + todo + " messages on " + this) + if (todo > 0) + throw new AssertionError("Timeout waiting for " + todo + " messages on " + this) + else + throw new AssertionError("Received " + (-todo) + " messages too many on " + this) result } finally app.mainbus publish TestEvent.UnMute(this) } @@ -438,12 +442,12 @@ class TestEventListener extends Logging.DefaultLogger { var filters: List[EventFilter] = Nil - override def receive: Actor.Receive = ({ - case InitializeLogger(bus) ⇒ Seq(classOf[Mute], classOf[UnMute]) foreach (bus.subscribe(context.self, _)) - case Mute(filters) ⇒ filters foreach addFilter - case UnMute(filters) ⇒ filters foreach removeFilter - case event: LogEvent if filter(event) ⇒ - }: Actor.Receive) orElse super.receive + override def receive = { + case InitializeLogger(bus) ⇒ Seq(classOf[Mute], classOf[UnMute]) foreach (bus.subscribe(context.self, _)) + case Mute(filters) ⇒ filters foreach addFilter + case UnMute(filters) ⇒ filters foreach removeFilter + case event: LogEvent ⇒ if (!filter(event)) print(event) + } def filter(event: LogEvent): Boolean = filters exists (f ⇒ try { f(event) } catch { case e: Exception ⇒ false }) diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index d0cee3c740..e8d82a4650 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -559,19 +559,24 @@ object TestKit { * * Note that the timeout is scaled using Duration.timeFactor. */ - def awaitCond(p: ⇒ Boolean, max: Duration, interval: Duration = 100.millis) { + def awaitCond(p: ⇒ Boolean, max: Duration, interval: Duration = 100.millis, noThrow: Boolean = false): Boolean = { val stop = now + max @tailrec - def poll(t: Duration) { + def poll(): Boolean = { if (!p) { - assert(now < stop, "timeout " + max + " expired") - Thread.sleep(t.toMillis) - poll((stop - now) min interval) - } + val toSleep = stop - now + if (toSleep <= Duration.Zero) { + if (noThrow) false + else throw new AssertionError("timeout " + max + " expired") + } else { + Thread.sleep((toSleep min interval).toMillis) + poll() + } + } else true } - poll(max min interval) + poll() } /** From 402258f5b7db85e71621a533080dc5099e090202 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 9 Nov 2011 17:25:23 +0100 Subject: [PATCH 18/19] make BalancingDispatcher.buddies comparator transitive --- .../src/main/scala/akka/dispatch/BalancingDispatcher.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index 133795cb4d..f5429b8742 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -37,7 +37,10 @@ class BalancingDispatcher( _timeoutMs: Long) extends Dispatcher(_app, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) { - private val buddies = new ConcurrentSkipListSet[ActorCell](new Comparator[ActorCell] { def compare(a: ActorCell, b: ActorCell) = System.identityHashCode(a) - System.identityHashCode(b) }) //new ConcurrentLinkedQueue[ActorCell]() + private val buddies = new ConcurrentSkipListSet[ActorCell]( + new Comparator[ActorCell] { + def compare(a: ActorCell, b: ActorCell): Int = ((System.identityHashCode(a) & 0xffffffffL) - (System.identityHashCode(b) & 0xffffffffL)).toInt + }) //new ConcurrentLinkedQueue[ActorCell]() protected val messageQueue: MessageQueue = mailboxType match { case u: UnboundedMailbox ⇒ new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { From 594f52100b63305acc1a1bc8d53f8622432fd79c Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 9 Nov 2011 17:40:04 +0100 Subject: [PATCH 19/19] actually make buddies comparator consistent now --- .../scala/akka/dispatch/BalancingDispatcher.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index f5429b8742..fe374f6b03 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -39,8 +39,16 @@ class BalancingDispatcher( private val buddies = new ConcurrentSkipListSet[ActorCell]( new Comparator[ActorCell] { - def compare(a: ActorCell, b: ActorCell): Int = ((System.identityHashCode(a) & 0xffffffffL) - (System.identityHashCode(b) & 0xffffffffL)).toInt - }) //new ConcurrentLinkedQueue[ActorCell]() + def compare(a: ActorCell, b: ActorCell): Int = { + /* + * make sure that there is no overflow or underflow in comparisons, so + * that the ordering is actually consistent and you cannot have a + * sequence which cyclically is monotone without end. + */ + val diff = ((System.identityHashCode(a) & 0xffffffffL) - (System.identityHashCode(b) & 0xffffffffL)) + if (diff > 0) 1 else if (diff < 0) -1 else 0 + } + }) protected val messageQueue: MessageQueue = mailboxType match { case u: UnboundedMailbox ⇒ new QueueBasedMessageQueue with UnboundedMessageQueueSemantics {