Skip to content

Commit

Permalink
Merge pull request #251 from jboner/wip-1719-ExtendedActorSystem-∂π
Browse files Browse the repository at this point in the history
add ExtendedActorSystem to shield ActorSystemImpl
  • Loading branch information
rkuhn committed Jan 24, 2012
2 parents 996b2dc + d5c23bc commit 1399b04
Show file tree
Hide file tree
Showing 21 changed files with 85 additions and 56 deletions.
6 changes: 3 additions & 3 deletions akka-actor-tests/src/test/java/akka/actor/JavaExtension.java
Expand Up @@ -23,15 +23,15 @@ public ExtensionId<TestExtension> lookup() {
return TestExtensionId.TestExtensionProvider;
}

public TestExtension createExtension(ActorSystemImpl i) {
public TestExtension createExtension(ExtendedActorSystem i) {
return new TestExtension(i);
}
}

static class TestExtension implements Extension {
public final ActorSystemImpl system;
public final ExtendedActorSystem system;

public TestExtension(ActorSystemImpl i) {
public TestExtension(ExtendedActorSystem i) {
system = i;
}
}
Expand Down
Expand Up @@ -15,11 +15,11 @@ class JavaExtensionSpec extends JavaExtension with JUnitSuite

object TestExtension extends ExtensionId[TestExtension] with ExtensionIdProvider {
def lookup = this
def createExtension(s: ActorSystemImpl) = new TestExtension(s)
def createExtension(s: ExtendedActorSystem) = new TestExtension(s)
}

// Dont't place inside ActorSystemSpec object, since it will not be garbage collected and reference to system remains
class TestExtension(val system: ActorSystemImpl) extends Extension
class TestExtension(val system: ExtendedActorSystem) extends Extension

@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExtension$"]""") {
Expand Down
@@ -1,11 +1,8 @@
package akka.performance.trading.domain

import java.util.concurrent.atomic.AtomicInteger
import akka.actor.Extension
import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
import akka.actor.ActorSystemImpl
import akka.actor.ActorSystem

import akka.actor.{ ExtensionIdProvider, ExtensionId, Extension, ExtendedActorSystem, ActorSystem }

abstract trait TradeObserver {
def trade(bid: Bid, ask: Ask)
Expand Down Expand Up @@ -38,5 +35,5 @@ object TotalTradeCounterExtension
extends ExtensionId[TotalTradeCounter]
with ExtensionIdProvider {
override def lookup = TotalTradeCounterExtension
override def createExtension(system: ActorSystemImpl) = new TotalTradeCounter
override def createExtension(system: ExtendedActorSystem) = new TotalTradeCounter
}
42 changes: 41 additions & 1 deletion akka-actor/src/main/scala/akka/actor/ActorSystem.scala
Expand Up @@ -130,6 +130,13 @@ object ActorSystem {
* }}}
*
* Where no name is given explicitly, one will be automatically generated.
*
* <b><i>Important Notice:</i></o>
*
* This class is not meant to be extended by user code. If you want to
* actually roll your own Akka, it will probably be better to look into
* extending [[akka.actor.ExtendedActorSystem]] instead, but beware that you
* are completely on your own in that case!
*/
abstract class ActorSystem extends ActorRefFactory {
import ActorSystem._
Expand Down Expand Up @@ -286,7 +293,40 @@ abstract class ActorSystem extends ActorRefFactory {
def hasExtension(ext: ExtensionId[_ <: Extension]): Boolean
}

class ActorSystemImpl(val name: String, applicationConfig: Config) extends ActorSystem {
/**
* More powerful interface to the actor system’s implementation which is presented to extensions (see [[akka.actor.Extension]]).
*
* <b><i>Important Notice:</i></o>
*
* This class is not meant to be extended by user code. If you want to
* actually roll your own Akka, beware that you are completely on your own in
* that case!
*/
abstract class ExtendedActorSystem extends ActorSystem {

/**
* The ActorRefProvider is the only entity which creates all actor references within this actor system.
*/
def provider: ActorRefProvider

/**
* The top-level supervisor of all actors created using system.actorOf(...).
*/
def guardian: InternalActorRef

/**
* The top-level supervisor of all system-internal services like logging.
*/
def systemGuardian: InternalActorRef

/**
* Implementation of the mechanism which is used for watch()/unwatch().
*/
def deathWatch: DeathWatch

}

class ActorSystemImpl(val name: String, applicationConfig: Config) extends ExtendedActorSystem {

if (!name.matches("""^\w+$"""))
throw new IllegalArgumentException("invalid ActorSystem name [" + name + "], must contain only word characters (i.e. [a-zA-Z_0-9])")
Expand Down
10 changes: 4 additions & 6 deletions akka-actor/src/main/scala/akka/actor/Extension.scala
Expand Up @@ -18,10 +18,8 @@ import akka.util.ReflectiveAccess
* The extension itself can be created in any way desired and has full access
* to the ActorSystem implementation.
*
*/

/**
* Marker interface to signify an Akka Extension
* This trait is only a marker interface to signify an Akka Extension, see
* [[akka.actor.ExtensionKey]] for a concise way of formulating extensions.
*/
trait Extension

Expand All @@ -47,7 +45,7 @@ trait ExtensionId[T <: Extension] {
* Is used by Akka to instantiate the Extension identified by this ExtensionId,
* internal use only.
*/
def createExtension(system: ActorSystemImpl): T
def createExtension(system: ExtendedActorSystem): T
}

/**
Expand Down Expand Up @@ -94,7 +92,7 @@ abstract class ExtensionKey[T <: Extension](implicit m: ClassManifest[T]) extend
def this(clazz: Class[T]) = this()(ClassManifest.fromClass(clazz))

override def lookup(): ExtensionId[T] = this
def createExtension(system: ActorSystemImpl): T =
def createExtension(system: ExtendedActorSystem): T =
ReflectiveAccess.createInstance[T](m.erasure, Array[Class[_]](classOf[ActorSystemImpl]), Array[AnyRef](system)) match {
case Left(ex) throw ex
case Right(r) r
Expand Down
2 changes: 1 addition & 1 deletion akka-actor/src/main/scala/akka/actor/IO.scala
Expand Up @@ -744,7 +744,7 @@ final class IOManager private (system: ActorSystem) extends Extension {

object IOManager extends ExtensionId[IOManager] with ExtensionIdProvider {
override def lookup = this
override def createExtension(system: ActorSystemImpl) = new IOManager(system)
override def createExtension(system: ExtendedActorSystem) = new IOManager(system)
}

/**
Expand Down
4 changes: 2 additions & 2 deletions akka-actor/src/main/scala/akka/actor/TypedActor.scala
Expand Up @@ -80,7 +80,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
override def get(system: ActorSystem): TypedActorExtension = super.get(system)

def lookup() = this
def createExtension(system: ActorSystemImpl): TypedActorExtension = new TypedActorExtension(system)
def createExtension(system: ExtendedActorSystem): TypedActorExtension = new TypedActorExtension(system)

/**
* Returns a contextual TypedActorFactory of this extension, this means that any TypedActors created by this TypedActorExtension
Expand Down Expand Up @@ -531,7 +531,7 @@ case class ContextualTypedActorFactory(typedActor: TypedActorExtension, actorFac
override def isTypedActor(proxyOrNot: AnyRef): Boolean = typedActor.isTypedActor(proxyOrNot)
}

class TypedActorExtension(system: ActorSystemImpl) extends TypedActorFactory with Extension {
class TypedActorExtension(system: ExtendedActorSystem) extends TypedActorFactory with Extension {
import TypedActor._ //Import the goodies from the companion object
protected def actorFactory: ActorRefFactory = system
protected def typedActor = this
Expand Down
Expand Up @@ -9,7 +9,7 @@ import akka.util.ReflectiveAccess
import scala.util.DynamicVariable
import com.typesafe.config.Config
import akka.config.ConfigurationException
import akka.actor.{ Extension, ActorSystem, ActorSystemImpl }
import akka.actor.{ Extension, ActorSystem, ExtendedActorSystem }

case class NoSerializerFoundException(m: String) extends AkkaException(m)

Expand Down Expand Up @@ -55,7 +55,7 @@ object Serialization {
* Serialization module. Contains methods for serialization and deserialization as well as
* locating a Serializer for a particular class as defined in the mapping in the 'akka.conf' file.
*/
class Serialization(val system: ActorSystemImpl) extends Extension {
class Serialization(val system: ExtendedActorSystem) extends Extension {
import Serialization._

val settings = new Settings(system.settings.config)
Expand Down
Expand Up @@ -3,7 +3,7 @@
*/
package akka.serialization

import akka.actor.{ ActorSystem, ExtensionId, ExtensionIdProvider, ActorSystemImpl }
import akka.actor.{ ActorSystem, ExtensionId, ExtensionIdProvider, ExtendedActorSystem }

/**
* SerializationExtension is an Akka Extension to interact with the Serialization
Expand All @@ -12,5 +12,5 @@ import akka.actor.{ ActorSystem, ExtensionId, ExtensionIdProvider, ActorSystemIm
object SerializationExtension extends ExtensionId[Serialization] with ExtensionIdProvider {
override def get(system: ActorSystem): Serialization = super.get(system)
override def lookup = SerializationExtension
override def createExtension(system: ActorSystemImpl): Serialization = new Serialization(system)
override def createExtension(system: ExtendedActorSystem): Serialization = new Serialization(system)
}
Expand Up @@ -42,7 +42,7 @@ public CountExtension lookup() {

//This method will be called by Akka
// to instantiate our Extension
public CountExtensionImpl createExtension(ActorSystemImpl system) {
public CountExtensionImpl createExtension(ExtendedActorSystem system) {
return new CountExtensionImpl();
}
}
Expand Down
Expand Up @@ -8,7 +8,7 @@
import akka.actor.AbstractExtensionId;
import akka.actor.ExtensionIdProvider;
import akka.actor.ActorSystem;
import akka.actor.ActorSystemImpl;
import akka.actor.ExtendedActorSystem;
import akka.util.Duration;
import com.typesafe.config.Config;
import java.util.concurrent.TimeUnit;
Expand Down Expand Up @@ -44,7 +44,7 @@ public Settings lookup() {
return Settings.SettingsProvider;
}

public SettingsImpl createExtension(ActorSystemImpl system) {
public SettingsImpl createExtension(ExtendedActorSystem system) {
return new SettingsImpl(system.settings().config());
}
}
Expand Down
Expand Up @@ -23,7 +23,7 @@ class CountExtensionImpl extends Extension {
//#extensionid
import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
import akka.actor.ActorSystemImpl
import akka.actor.ExtendedActorSystem

object CountExtension
extends ExtensionId[CountExtensionImpl]
Expand All @@ -36,7 +36,7 @@ object CountExtension

//This method will be called by Akka
// to instantiate our Extension
override def createExtension(system: ActorSystemImpl) = new CountExtensionImpl
override def createExtension(system: ExtendedActorSystem) = new CountExtensionImpl
}
//#extensionid

Expand Down
Expand Up @@ -7,7 +7,7 @@ package akka.docs.extension
import akka.actor.Extension
import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
import akka.actor.ActorSystemImpl
import akka.actor.ExtendedActorSystem
import akka.util.Duration
import com.typesafe.config.Config
import java.util.concurrent.TimeUnit
Expand All @@ -29,7 +29,7 @@ object Settings extends ExtensionId[SettingsImpl] with ExtensionIdProvider {

override def lookup = Settings

override def createExtension(system: ActorSystemImpl) = new SettingsImpl(system.settings.config)
override def createExtension(system: ExtendedActorSystem) = new SettingsImpl(system.settings.config)
}
//#extensionid

Expand Down
Expand Up @@ -11,7 +11,7 @@ import akka.actor._
object BeanstalkBasedMailboxExtension extends ExtensionId[BeanstalkMailboxSettings] with ExtensionIdProvider {
override def get(system: ActorSystem): BeanstalkMailboxSettings = super.get(system)
def lookup() = this
def createExtension(system: ActorSystemImpl) = new BeanstalkMailboxSettings(system.settings.config)
def createExtension(system: ExtendedActorSystem) = new BeanstalkMailboxSettings(system.settings.config)
}

class BeanstalkMailboxSettings(val config: Config) extends Extension {
Expand Down
Expand Up @@ -11,7 +11,7 @@ import akka.actor._
object FileBasedMailboxExtension extends ExtensionId[FileBasedMailboxSettings] with ExtensionIdProvider {
override def get(system: ActorSystem): FileBasedMailboxSettings = super.get(system)
def lookup() = this
def createExtension(system: ActorSystemImpl) = new FileBasedMailboxSettings(system.settings.config)
def createExtension(system: ExtendedActorSystem) = new FileBasedMailboxSettings(system.settings.config)
}

class FileBasedMailboxSettings(val config: Config) extends Extension {
Expand Down
Expand Up @@ -11,7 +11,7 @@ import akka.actor._
object MongoBasedMailboxExtension extends ExtensionId[MongoBasedMailboxSettings] with ExtensionIdProvider {
override def get(system: ActorSystem): MongoBasedMailboxSettings = super.get(system)
def lookup() = this
def createExtension(system: ActorSystemImpl) = new MongoBasedMailboxSettings(system.settings.config)
def createExtension(system: ExtendedActorSystem) = new MongoBasedMailboxSettings(system.settings.config)
}

class MongoBasedMailboxSettings(val config: Config) extends Extension {
Expand Down
Expand Up @@ -9,7 +9,7 @@ import akka.actor._
object RedisBasedMailboxExtension extends ExtensionId[RedisBasedMailboxSettings] with ExtensionIdProvider {
override def get(system: ActorSystem): RedisBasedMailboxSettings = super.get(system)
def lookup() = this
def createExtension(system: ActorSystemImpl) = new RedisBasedMailboxSettings(system.settings.config)
def createExtension(system: ExtendedActorSystem) = new RedisBasedMailboxSettings(system.settings.config)
}

class RedisBasedMailboxSettings(val config: Config) extends Extension {
Expand Down
Expand Up @@ -11,7 +11,7 @@ import akka.actor._
object ZooKeeperBasedMailboxExtension extends ExtensionId[ZooKeeperBasedMailboxSettings] with ExtensionIdProvider {
override def get(system: ActorSystem): ZooKeeperBasedMailboxSettings = super.get(system)
def lookup() = this
def createExtension(system: ActorSystemImpl) = new ZooKeeperBasedMailboxSettings(system.settings.config)
def createExtension(system: ExtendedActorSystem) = new ZooKeeperBasedMailboxSettings(system.settings.config)
}
class ZooKeeperBasedMailboxSettings(val config: Config) extends Extension {

Expand Down
Expand Up @@ -3,26 +3,20 @@
*/
package akka.testkit

import akka.event.Logging.{ Warning, Error }
import java.lang.ref.WeakReference
import java.util.concurrent.locks.ReentrantLock
import java.util.LinkedList
import java.util.concurrent.RejectedExecutionException
import akka.util.Switch
import java.lang.ref.WeakReference

import scala.annotation.tailrec
import akka.actor.{ ActorCell, ActorRef, ActorSystem }
import akka.dispatch._
import akka.actor.Scheduler
import akka.event.EventStream
import akka.util.Duration
import akka.util.duration._
import java.util.concurrent.TimeUnit
import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
import akka.actor.ActorSystemImpl
import akka.actor.Extension

import com.typesafe.config.Config

import CallingThreadDispatcher.Id
import akka.actor.{ ExtensionIdProvider, ExtensionId, Extension, ExtendedActorSystem, ActorRef, ActorCell }
import akka.dispatch.{ TaskInvocation, SystemMessage, Suspend, Resume, MessageDispatcherConfigurator, MessageDispatcher, Mailbox, Envelope, DispatcherPrerequisites, DefaultSystemMessageQueue }
import akka.util.duration.intToDurationInt
import akka.util.{ Switch, Duration }

/*
* Locking rules:
*
Expand All @@ -42,7 +36,7 @@ import com.typesafe.config.Config

private[testkit] object CallingThreadDispatcherQueues extends ExtensionId[CallingThreadDispatcherQueues] with ExtensionIdProvider {
override def lookup = CallingThreadDispatcherQueues
override def createExtension(system: ActorSystemImpl): CallingThreadDispatcherQueues = new CallingThreadDispatcherQueues
override def createExtension(system: ExtendedActorSystem): CallingThreadDispatcherQueues = new CallingThreadDispatcherQueues
}

private[testkit] class CallingThreadDispatcherQueues extends Extension {
Expand Down
Expand Up @@ -6,11 +6,11 @@ package akka.testkit
import com.typesafe.config.Config
import akka.util.Duration
import java.util.concurrent.TimeUnit.MILLISECONDS
import akka.actor.{ ExtensionId, ActorSystem, Extension, ActorSystemImpl }
import akka.actor.{ ExtensionId, ActorSystem, Extension, ExtendedActorSystem }

object TestKitExtension extends ExtensionId[TestKitSettings] {
override def get(system: ActorSystem): TestKitSettings = super.get(system)
def createExtension(system: ActorSystemImpl): TestKitSettings = new TestKitSettings(system.settings.config)
def createExtension(system: ExtendedActorSystem): TestKitSettings = new TestKitSettings(system.settings.config)
}

class TestKitSettings(val config: Config) extends Extension {
Expand Down
Expand Up @@ -24,7 +24,7 @@ case class ZeroMQVersion(major: Int, minor: Int, patch: Int) {
*/
object ZeroMQExtension extends ExtensionId[ZeroMQExtension] with ExtensionIdProvider {
def lookup() = this
def createExtension(system: ActorSystemImpl) = new ZeroMQExtension(system)
def createExtension(system: ExtendedActorSystem) = new ZeroMQExtension(system)

private val minVersionString = "2.1.0"
private val minVersion = JZMQ.makeVersion(2, 1, 0)
Expand Down

0 comments on commit 1399b04

Please sign in to comment.