incorporate Patrik’s feedback
- rename to DynamicAccess - rename to createInstanceFor / createClassFor - fix a few little things
This commit is contained in:
parent
b193bcee04
commit
dca309c535
19 changed files with 56 additions and 58 deletions
|
|
@ -42,8 +42,8 @@ public class JavaExtension {
|
|||
|
||||
public final ExtendedActorSystem system;
|
||||
|
||||
public OtherExtension(ExtendedActorSystem i) {
|
||||
system = i;
|
||||
public OtherExtension(ExtendedActorSystem system) {
|
||||
this.system = system;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import akka.dispatch._
|
|||
import akka.util._
|
||||
import scala.collection.immutable.Stack
|
||||
import java.lang.{ UnsupportedOperationException, IllegalStateException }
|
||||
import akka.serialization.Serialization
|
||||
import akka.serialization.{ Serialization, JavaSerializer }
|
||||
import akka.event.EventStream
|
||||
import scala.annotation.tailrec
|
||||
import java.util.concurrent.{ ConcurrentHashMap }
|
||||
|
|
@ -399,7 +399,7 @@ case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef)
|
|||
private[akka] object DeadLetterActorRef {
|
||||
class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance?
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
private def readResolve(): AnyRef = akka.serialization.JavaSerializer.currentSystem.value.deadLetters
|
||||
private def readResolve(): AnyRef = JavaSerializer.currentSystem.value.deadLetters
|
||||
}
|
||||
|
||||
val serialized = new SerializedDeadLetterActorRef
|
||||
|
|
|
|||
|
|
@ -318,12 +318,12 @@ class LocalActorRefProvider(
|
|||
settings: ActorSystem.Settings,
|
||||
eventStream: EventStream,
|
||||
scheduler: Scheduler,
|
||||
propertyMaster: PropertyMaster) =
|
||||
dynamicAccess: DynamicAccess) =
|
||||
this(_systemName,
|
||||
settings,
|
||||
eventStream,
|
||||
scheduler,
|
||||
new Deployer(settings, propertyMaster))
|
||||
new Deployer(settings, dynamicAccess))
|
||||
|
||||
val rootPath: ActorPath = RootActorPath(Address("akka", _systemName))
|
||||
|
||||
|
|
|
|||
|
|
@ -330,7 +330,7 @@ abstract class ExtendedActorSystem extends ActorSystem {
|
|||
* set on all threads created by the ActorSystem, if one was set during
|
||||
* creation.
|
||||
*/
|
||||
def propertyMaster: PropertyMaster
|
||||
def dynamicAccess: DynamicAccess
|
||||
}
|
||||
|
||||
class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Config) extends ExtendedActorSystem {
|
||||
|
|
@ -360,7 +360,7 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf
|
|||
* This is an extension point: by overriding this method, subclasses can
|
||||
* control all reflection activities of an actor system.
|
||||
*/
|
||||
protected def createPropertyMaster(): PropertyMaster = new ReflectivePropertyMaster(findClassLoader)
|
||||
protected def createDynamicAccess(): DynamicAccess = new ReflectiveDynamicAccess(findClassLoader)
|
||||
|
||||
protected def findClassLoader: ClassLoader = {
|
||||
def findCaller(get: Int ⇒ Class[_]): ClassLoader = {
|
||||
|
|
@ -382,8 +382,8 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf
|
|||
getClass.getClassLoader
|
||||
}
|
||||
|
||||
private val _pm: PropertyMaster = createPropertyMaster()
|
||||
def propertyMaster: PropertyMaster = _pm
|
||||
private val _pm: DynamicAccess = createDynamicAccess()
|
||||
def dynamicAccess: DynamicAccess = _pm
|
||||
|
||||
def logConfiguration(): Unit = log.info(settings.toString)
|
||||
|
||||
|
|
@ -441,9 +441,9 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf
|
|||
classOf[Settings] -> settings,
|
||||
classOf[EventStream] -> eventStream,
|
||||
classOf[Scheduler] -> scheduler,
|
||||
classOf[PropertyMaster] -> propertyMaster)
|
||||
classOf[DynamicAccess] -> dynamicAccess)
|
||||
|
||||
propertyMaster.getInstanceFor[ActorRefProvider](ProviderClass, arguments) match {
|
||||
dynamicAccess.createInstanceFor[ActorRefProvider](ProviderClass, arguments) match {
|
||||
case Left(e) ⇒ throw e
|
||||
case Right(p) ⇒ p
|
||||
}
|
||||
|
|
@ -465,7 +465,7 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf
|
|||
def locker: Locker = provider.locker
|
||||
|
||||
val dispatchers: Dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(
|
||||
threadFactory, eventStream, deadLetterMailbox, scheduler, propertyMaster))
|
||||
threadFactory, eventStream, deadLetterMailbox, scheduler, dynamicAccess))
|
||||
|
||||
val dispatcher: MessageDispatcher = dispatchers.defaultGlobalDispatcher
|
||||
|
||||
|
|
@ -584,7 +584,7 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf
|
|||
private def loadExtensions() {
|
||||
import scala.collection.JavaConversions._
|
||||
settings.config.getStringList("akka.extensions") foreach { fqcn ⇒
|
||||
propertyMaster.getObjectFor[AnyRef](fqcn).fold(_ ⇒ propertyMaster.getInstanceFor[AnyRef](fqcn, Seq()), Right(_)) match {
|
||||
dynamicAccess.getObjectFor[AnyRef](fqcn).fold(_ ⇒ dynamicAccess.createInstanceFor[AnyRef](fqcn, Seq()), Right(_)) match {
|
||||
case Right(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup());
|
||||
case Right(p: ExtensionId[_]) ⇒ registerExtension(p);
|
||||
case Right(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn)
|
||||
|
|
|
|||
|
|
@ -82,7 +82,7 @@ case object NoScopeGiven extends Scope {
|
|||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class Deployer(val settings: ActorSystem.Settings, val propertyMaster: PropertyMaster) {
|
||||
class Deployer(val settings: ActorSystem.Settings, val dynamicAccess: DynamicAccess) {
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
|
|
@ -124,7 +124,7 @@ class Deployer(val settings: ActorSystem.Settings, val propertyMaster: PropertyM
|
|||
case "broadcast" ⇒ BroadcastRouter(nrOfInstances, routees, resizer)
|
||||
case fqn ⇒
|
||||
val args = Seq(classOf[Config] -> deployment)
|
||||
propertyMaster.getInstanceFor[RouterConfig](fqn, args) match {
|
||||
dynamicAccess.createInstanceFor[RouterConfig](fqn, args) match {
|
||||
case Right(router) ⇒ router
|
||||
case Left(exception) ⇒
|
||||
throw new IllegalArgumentException(
|
||||
|
|
|
|||
|
|
@ -7,21 +7,20 @@ import akka.util.NonFatal
|
|||
import java.lang.reflect.InvocationTargetException
|
||||
|
||||
/**
|
||||
* The property master is responsible for acquiring all props needed for a
|
||||
* performance; in Akka this is the class which is used for
|
||||
* The DynamicAccess implementation is the class which is used for
|
||||
* loading all configurable parts of an actor system (the
|
||||
* [[akka.actor.ReflectivePropertyMaster]] is the default implementation).
|
||||
* [[akka.actor.ReflectiveDynamicAccess]] is the default implementation).
|
||||
*
|
||||
* This is an internal facility and users are not expected to encounter it
|
||||
* unless they are extending Akka in ways which go beyond simple Extensions.
|
||||
*/
|
||||
private[akka] trait PropertyMaster {
|
||||
trait DynamicAccess {
|
||||
|
||||
/**
|
||||
* Obtain a `Class[_]` object loaded with the right class loader (i.e. the one
|
||||
* returned by `classLoader`).
|
||||
*/
|
||||
def getClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]]
|
||||
def createClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]]
|
||||
|
||||
/**
|
||||
* Obtain an object conforming to the type T, which is expected to be
|
||||
|
|
@ -30,7 +29,7 @@ private[akka] trait PropertyMaster {
|
|||
* `args` argument. The exact usage of args depends on which type is requested,
|
||||
* see the relevant requesting code for details.
|
||||
*/
|
||||
def getInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T]
|
||||
def createInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T]
|
||||
|
||||
/**
|
||||
* Obtain the Scala “object” instance for the given fully-qualified class name, if there is one.
|
||||
|
|
@ -45,17 +44,17 @@ private[akka] trait PropertyMaster {
|
|||
|
||||
}
|
||||
|
||||
object PropertyMaster {
|
||||
object DynamicAccess {
|
||||
|
||||
/**
|
||||
* Convenience method which given a `Class[_]` object and a constructor description
|
||||
* will create a new instance of that class.
|
||||
*
|
||||
* {{{
|
||||
* val obj = PropertyMaster.getInstanceFor(clazz, Seq(classOf[Config] -> config, classOf[String] -> name))
|
||||
* val obj = DynamicAccess.createInstanceFor(clazz, Seq(classOf[Config] -> config, classOf[String] -> name))
|
||||
* }}}
|
||||
*/
|
||||
def getInstanceFor[T: ClassManifest](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] = {
|
||||
def createInstanceFor[T: ClassManifest](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] = {
|
||||
val types = args.map(_._1).toArray
|
||||
val values = args.map(_._2).toArray
|
||||
withErrorHandling {
|
||||
|
|
@ -86,17 +85,17 @@ object PropertyMaster {
|
|||
}
|
||||
|
||||
/**
|
||||
* This is the default [[akka.actor.PropertyMaster]] implementation used by [[akka.actor.ActorSystemImpl]]
|
||||
* This is the default [[akka.actor.DynamicAccess]] implementation used by [[akka.actor.ActorSystemImpl]]
|
||||
* unless overridden. It uses reflection to turn fully-qualified class names into `Class[_]` objects
|
||||
* and creates instances from there using `getDeclaredConstructor()` and invoking that. The class loader
|
||||
* to be used for all this is determined by the [[akka.actor.ActorSystemImpl]]’s `findClassLoader` method
|
||||
* by default.
|
||||
*/
|
||||
class ReflectivePropertyMaster(val classLoader: ClassLoader) extends PropertyMaster {
|
||||
class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAccess {
|
||||
|
||||
import PropertyMaster.withErrorHandling
|
||||
import DynamicAccess.withErrorHandling
|
||||
|
||||
override def getClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] =
|
||||
override def createClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] =
|
||||
try {
|
||||
val c = classLoader.loadClass(fqcn).asInstanceOf[Class[_ <: T]]
|
||||
val t = classManifest[T].erasure
|
||||
|
|
@ -105,8 +104,8 @@ class ReflectivePropertyMaster(val classLoader: ClassLoader) extends PropertyMas
|
|||
case NonFatal(e) ⇒ Left(e)
|
||||
}
|
||||
|
||||
override def getInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] =
|
||||
getClassFor(fqcn).fold(Left(_), { c ⇒
|
||||
override def createInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] =
|
||||
createClassFor(fqcn).fold(Left(_), { c ⇒
|
||||
val types = args.map(_._1).toArray
|
||||
val values = args.map(_._2).toArray
|
||||
withErrorHandling {
|
||||
|
|
@ -119,7 +118,7 @@ class ReflectivePropertyMaster(val classLoader: ClassLoader) extends PropertyMas
|
|||
})
|
||||
|
||||
override def getObjectFor[T: ClassManifest](fqcn: String): Either[Throwable, T] = {
|
||||
getClassFor(fqcn).fold(Left(_), { c ⇒
|
||||
createClassFor(fqcn).fold(Left(_), { c ⇒
|
||||
withErrorHandling {
|
||||
val module = c.getDeclaredField("MODULE$")
|
||||
module.setAccessible(true)
|
||||
|
|
@ -97,7 +97,7 @@ abstract class ExtensionKey[T <: Extension](implicit m: ClassManifest[T]) extend
|
|||
|
||||
override def lookup(): ExtensionId[T] = this
|
||||
def createExtension(system: ExtendedActorSystem): T =
|
||||
PropertyMaster.getInstanceFor[T](m.erasure, Seq(classOf[ExtendedActorSystem] -> system)) match {
|
||||
DynamicAccess.createInstanceFor[T](m.erasure, Seq(classOf[ExtendedActorSystem] -> system)) match {
|
||||
case Left(ex) ⇒ throw ex
|
||||
case Right(r) ⇒ r
|
||||
}
|
||||
|
|
|
|||
|
|
@ -368,7 +368,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
|
|||
}
|
||||
case fqcn ⇒
|
||||
val args = Seq(classOf[Config] -> config)
|
||||
prerequisites.propertyMaster.getInstanceFor[MailboxType](fqcn, args) match {
|
||||
prerequisites.dynamicAccess.createInstanceFor[MailboxType](fqcn, args) match {
|
||||
case Right(instance) ⇒ instance
|
||||
case Left(exception) ⇒
|
||||
throw new IllegalArgumentException(
|
||||
|
|
@ -387,7 +387,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
|
|||
val args = Seq(
|
||||
classOf[Config] -> config,
|
||||
classOf[DispatcherPrerequisites] -> prerequisites)
|
||||
prerequisites.propertyMaster.getInstanceFor[ExecutorServiceConfigurator](fqcn, args) match {
|
||||
prerequisites.dynamicAccess.createInstanceFor[ExecutorServiceConfigurator](fqcn, args) match {
|
||||
case Right(instance) ⇒ instance
|
||||
case Left(exception) ⇒ throw new IllegalArgumentException(
|
||||
("""Cannot instantiate ExecutorServiceConfigurator ("executor = [%s]"), defined in [%s],
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import scala.collection.JavaConverters.mapAsJavaMapConverter
|
|||
import com.typesafe.config.{ ConfigFactory, Config }
|
||||
|
||||
import Dispatchers.DefaultDispatcherId
|
||||
import akka.actor.{ Scheduler, PropertyMaster, ActorSystem }
|
||||
import akka.actor.{ Scheduler, DynamicAccess, ActorSystem }
|
||||
import akka.event.Logging.Warning
|
||||
import akka.event.EventStream
|
||||
import akka.util.Duration
|
||||
|
|
@ -21,7 +21,7 @@ trait DispatcherPrerequisites {
|
|||
def eventStream: EventStream
|
||||
def deadLetterMailbox: Mailbox
|
||||
def scheduler: Scheduler
|
||||
def propertyMaster: PropertyMaster
|
||||
def dynamicAccess: DynamicAccess
|
||||
}
|
||||
|
||||
case class DefaultDispatcherPrerequisites(
|
||||
|
|
@ -29,7 +29,7 @@ case class DefaultDispatcherPrerequisites(
|
|||
val eventStream: EventStream,
|
||||
val deadLetterMailbox: Mailbox,
|
||||
val scheduler: Scheduler,
|
||||
val propertyMaster: PropertyMaster) extends DispatcherPrerequisites
|
||||
val dynamicAccess: DynamicAccess) extends DispatcherPrerequisites
|
||||
|
||||
object Dispatchers {
|
||||
/**
|
||||
|
|
@ -139,7 +139,7 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
|
|||
case "PinnedDispatcher" ⇒ new PinnedDispatcherConfigurator(cfg, prerequisites)
|
||||
case fqn ⇒
|
||||
val args = Seq(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites)
|
||||
prerequisites.propertyMaster.getInstanceFor[MessageDispatcherConfigurator](fqn, args) match {
|
||||
prerequisites.dynamicAccess.createInstanceFor[MessageDispatcherConfigurator](fqn, args) match {
|
||||
case Right(configurator) ⇒ configurator
|
||||
case Left(exception) ⇒
|
||||
throw new IllegalArgumentException(
|
||||
|
|
|
|||
|
|
@ -100,7 +100,7 @@ trait LoggingBus extends ActorEventBus {
|
|||
if loggerName != StandardOutLoggerName
|
||||
} yield {
|
||||
try {
|
||||
system.propertyMaster.getClassFor[Actor](loggerName) match {
|
||||
system.dynamicAccess.createClassFor[Actor](loggerName) match {
|
||||
case Right(actorClass) ⇒ addLogger(system, actorClass, level, logName)
|
||||
case Left(exception) ⇒ throw exception
|
||||
}
|
||||
|
|
|
|||
|
|
@ -128,12 +128,12 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
|||
|
||||
/**
|
||||
* Tries to load the specified Serializer by the fully-qualified name; the actual
|
||||
* loading is performed by the system’s [[akka.actor.PropertyMaster]].
|
||||
* loading is performed by the system’s [[akka.actor.DynamicAccess]].
|
||||
*/
|
||||
def serializerOf(serializerFQN: String): Either[Throwable, Serializer] = {
|
||||
val pm = system.propertyMaster
|
||||
pm.getInstanceFor[Serializer](serializerFQN, Seq(classOf[ExtendedActorSystem] -> system))
|
||||
.fold(_ ⇒ pm.getInstanceFor[Serializer](serializerFQN, Seq()), Right(_))
|
||||
val dynamicAccess = system.dynamicAccess
|
||||
dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq(classOf[ExtendedActorSystem] -> system))
|
||||
.fold(_ ⇒ dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq()), Right(_))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.serialization
|
|||
|
||||
import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream }
|
||||
import akka.util.ClassLoaderObjectInputStream
|
||||
import akka.actor.PropertyMaster
|
||||
import akka.actor.DynamicAccess
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import scala.util.DynamicVariable
|
||||
|
||||
|
|
@ -19,8 +19,8 @@ import scala.util.DynamicVariable
|
|||
* <ul>
|
||||
* <li>taking exactly one argument of type [[akka.actor.ExtendedActorSystem]];
|
||||
* this should be the preferred one because all reflective loading of classes
|
||||
* during deserialization should use ExtendedActorSystem.propertyMaster (see
|
||||
* [[akka.actor.PropertyMaster]]), and</li>
|
||||
* during deserialization should use ExtendedActorSystem.dynamicAccess (see
|
||||
* [[akka.actor.DynamicAccess]]), and</li>
|
||||
* <li>without arguments, which is only an option if the serializer does not
|
||||
* load classes using reflection.</li>
|
||||
* </ul>
|
||||
|
|
@ -49,7 +49,7 @@ trait Serializer extends scala.Serializable {
|
|||
|
||||
/**
|
||||
* Produces an object from an array of bytes, with an optional type-hint;
|
||||
* the class should be loaded using ActorSystem.propertyMaster.
|
||||
* the class should be loaded using ActorSystem.dynamicAccess.
|
||||
*/
|
||||
def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef
|
||||
}
|
||||
|
|
@ -108,7 +108,7 @@ class JavaSerializer(val system: ExtendedActorSystem) extends Serializer {
|
|||
}
|
||||
|
||||
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = {
|
||||
val in = new ClassLoaderObjectInputStream(system.propertyMaster.classLoader, new ByteArrayInputStream(bytes))
|
||||
val in = new ClassLoaderObjectInputStream(system.dynamicAccess.classLoader, new ByteArrayInputStream(bytes))
|
||||
val obj = JavaSerializer.currentSystem.withValue(system) {
|
||||
in.readObject
|
||||
}
|
||||
|
|
|
|||
|
|
@ -14,7 +14,6 @@ import static org.junit.Assert.*;
|
|||
|
||||
import akka.serialization.*;
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.actor.PropertyMaster;
|
||||
import com.typesafe.config.*;
|
||||
|
||||
//#imports
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.docs.serialization
|
|||
import org.scalatest.matchers.MustMatchers
|
||||
import akka.testkit._
|
||||
//#imports
|
||||
import akka.actor.{ ActorSystem, PropertyMaster }
|
||||
import akka.actor.ActorSystem
|
||||
import akka.serialization._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
|
|
|
|||
|
|
@ -14,7 +14,7 @@ object MessageSerializer {
|
|||
def deserialize(system: ExtendedActorSystem, messageProtocol: MessageProtocol): AnyRef = {
|
||||
val clazz =
|
||||
if (messageProtocol.hasMessageManifest) {
|
||||
system.propertyMaster.getClassFor[AnyRef](messageProtocol.getMessageManifest.toStringUtf8)
|
||||
system.dynamicAccess.createClassFor[AnyRef](messageProtocol.getMessageManifest.toStringUtf8)
|
||||
.fold(throw _, Some(_))
|
||||
} else None
|
||||
SerializationExtension(system)
|
||||
|
|
|
|||
|
|
@ -27,11 +27,11 @@ class RemoteActorRefProvider(
|
|||
val settings: ActorSystem.Settings,
|
||||
val eventStream: EventStream,
|
||||
val scheduler: Scheduler,
|
||||
val propertyMaster: PropertyMaster) extends ActorRefProvider {
|
||||
val dynamicAccess: DynamicAccess) extends ActorRefProvider {
|
||||
|
||||
val remoteSettings = new RemoteSettings(settings.config, systemName)
|
||||
|
||||
val deployer = new RemoteDeployer(settings, propertyMaster)
|
||||
val deployer = new RemoteDeployer(settings, dynamicAccess)
|
||||
|
||||
private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, deployer)
|
||||
|
||||
|
|
@ -83,7 +83,7 @@ class RemoteActorRefProvider(
|
|||
classOf[ActorSystemImpl] -> system,
|
||||
classOf[RemoteActorRefProvider] -> this)
|
||||
|
||||
system.propertyMaster.getInstanceFor[RemoteTransport](fqn, args) match {
|
||||
system.dynamicAccess.createInstanceFor[RemoteTransport](fqn, args) match {
|
||||
case Left(problem) ⇒ throw new RemoteTransportException("Could not load remote transport layer " + fqn, problem)
|
||||
case Right(remote) ⇒ remote
|
||||
}
|
||||
|
|
|
|||
|
|
@ -12,7 +12,7 @@ case class RemoteScope(node: Address) extends Scope {
|
|||
def withFallback(other: Scope): Scope = this
|
||||
}
|
||||
|
||||
class RemoteDeployer(_settings: ActorSystem.Settings, _pm: PropertyMaster) extends Deployer(_settings, _pm) {
|
||||
class RemoteDeployer(_settings: ActorSystem.Settings, _pm: DynamicAccess) extends Deployer(_settings, _pm) {
|
||||
|
||||
override protected def parseConfig(path: String, config: Config): Option[Deploy] = {
|
||||
import scala.collection.JavaConverters._
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@
|
|||
package akka.serialization
|
||||
|
||||
import com.google.protobuf.Message
|
||||
import akka.actor.PropertyMaster
|
||||
import akka.actor.DynamicAccess
|
||||
|
||||
/**
|
||||
* This Serializer serializes `com.google.protobuf.Message`s
|
||||
|
|
|
|||
|
|
@ -121,7 +121,7 @@ object TestActorRef {
|
|||
def apply[T <: Actor](implicit m: Manifest[T], system: ActorSystem): TestActorRef[T] = apply[T](randomName)
|
||||
|
||||
def apply[T <: Actor](name: String)(implicit m: Manifest[T], system: ActorSystem): TestActorRef[T] = apply[T](Props({
|
||||
PropertyMaster.getInstanceFor[T](m.erasure, Seq()) match {
|
||||
DynamicAccess.createInstanceFor[T](m.erasure, Seq()) match {
|
||||
case Right(value) ⇒ value
|
||||
case Left(exception) ⇒ throw new ActorInitializationException(null,
|
||||
"Could not instantiate Actor" +
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue