Merge pull request #211 from jboner/wip-1406-migration-patriknw
Migration kit and start of migration documentation. See #1406.
This commit is contained in:
commit
43059d6584
11 changed files with 1015 additions and 1 deletions
|
|
@ -0,0 +1,64 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.actor
|
||||
|
||||
import java.io.File
|
||||
|
||||
import com.typesafe.config.Config
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import com.typesafe.config.ConfigParseOptions
|
||||
import com.typesafe.config.ConfigResolveOptions
|
||||
|
||||
@deprecated("use ActorSystem instead", "2.0")
|
||||
object GlobalActorSystem extends ActorSystemImpl("GlobalSystem", OldConfigurationLoader.defaultConfig) {
|
||||
start()
|
||||
}
|
||||
|
||||
/**
|
||||
* Loads configuration (akka.conf) from same location as Akka 1.x
|
||||
*/
|
||||
@deprecated("use default config location or write your own configuration loader", "2.0")
|
||||
object OldConfigurationLoader {
|
||||
|
||||
val defaultConfig: Config = {
|
||||
val cfg = fromProperties orElse fromClasspath orElse fromHome getOrElse emptyConfig
|
||||
val config = cfg.withFallback(ConfigFactory.defaultReference)
|
||||
config.checkValid(ConfigFactory.defaultReference, "akka")
|
||||
config
|
||||
}
|
||||
|
||||
// file extensions (.conf, .json, .properties), are handled by parseFileAnySyntax
|
||||
val defaultLocation: String = (systemMode orElse envMode).map("akka." + _).getOrElse("akka")
|
||||
|
||||
private def envMode = System.getenv("AKKA_MODE") match {
|
||||
case null | "" ⇒ None
|
||||
case value ⇒ Some(value)
|
||||
}
|
||||
|
||||
private def systemMode = System.getProperty("akka.mode") match {
|
||||
case null | "" ⇒ None
|
||||
case value ⇒ Some(value)
|
||||
}
|
||||
|
||||
private def configParseOptions = ConfigParseOptions.defaults.setAllowMissing(false)
|
||||
|
||||
private def fromProperties = try {
|
||||
val property = Option(System.getProperty("akka.config"))
|
||||
property.map(p ⇒
|
||||
ConfigFactory.systemProperties.withFallback(
|
||||
ConfigFactory.parseFileAnySyntax(new File(p), configParseOptions)))
|
||||
} catch { case _ ⇒ None }
|
||||
|
||||
private def fromClasspath = try {
|
||||
Option(ConfigFactory.systemProperties.withFallback(
|
||||
ConfigFactory.parseResourcesAnySyntax(ActorSystem.getClass, "/" + defaultLocation, configParseOptions)))
|
||||
} catch { case _ ⇒ None }
|
||||
|
||||
private def fromHome = try {
|
||||
Option(ConfigFactory.systemProperties.withFallback(
|
||||
ConfigFactory.parseFileAnySyntax(new File(ActorSystem.GlobalHome.get + "/config/" + defaultLocation), configParseOptions)))
|
||||
} catch { case _ ⇒ None }
|
||||
|
||||
private def emptyConfig = ConfigFactory.systemProperties
|
||||
}
|
||||
171
akka-actor-migration/src/main/scala/akka/actor/OldActor.scala
Normal file
171
akka-actor-migration/src/main/scala/akka/actor/OldActor.scala
Normal file
|
|
@ -0,0 +1,171 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.actor
|
||||
|
||||
import akka.japi.Creator
|
||||
import akka.util.Timeout
|
||||
import akka.dispatch.Future
|
||||
import akka.dispatch.OldFuture
|
||||
import akka.util.Duration
|
||||
import java.util.concurrent.TimeUnit
|
||||
import java.net.InetSocketAddress
|
||||
|
||||
/**
|
||||
* Migration replacement for `object akka.actor.Actor`.
|
||||
*/
|
||||
@deprecated("use ActorRefFactory (ActorSystem or ActorContext) to create actors", "2.0")
|
||||
object OldActor {
|
||||
|
||||
/**
|
||||
* Creates an ActorRef out of the Actor with type T.
|
||||
* It will be automatically started, i.e. remove old call to `start()`.
|
||||
*
|
||||
*/
|
||||
@deprecated("use ActorRefFactory (ActorSystem or ActorContext) to create actors", "2.0")
|
||||
def actorOf[T <: Actor: Manifest]: ActorRef = actorOf(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]])
|
||||
|
||||
/**
|
||||
* Creates an ActorRef out of the Actor of the specified Class.
|
||||
* It will be automatically started, i.e. remove old call to `start()`.
|
||||
*/
|
||||
@deprecated("use ActorRefFactory (ActorSystem or ActorContext) to create actors", "2.0")
|
||||
def actorOf(clazz: Class[_ <: Actor]): ActorRef = GlobalActorSystem.actorOf(Props(clazz))
|
||||
|
||||
/**
|
||||
* Creates an ActorRef out of the Actor. Allows you to pass in a factory function
|
||||
* that creates the Actor. Please note that this function can be invoked multiple
|
||||
* times if for example the Actor is supervised and needs to be restarted.
|
||||
*
|
||||
* It will be automatically started, i.e. remove old call to `start()`.
|
||||
*/
|
||||
@deprecated("use ActorRefFactory (ActorSystem or ActorContext) to create actors", "2.0")
|
||||
def actorOf(factory: ⇒ Actor): ActorRef = GlobalActorSystem.actorOf(Props(factory))
|
||||
|
||||
/**
|
||||
* Creates an ActorRef out of the Actor. Allows you to pass in a factory (Creator<Actor>)
|
||||
* that creates the Actor. Please note that this function can be invoked multiple
|
||||
* times if for example the Actor is supervised and needs to be restarted.
|
||||
* <p/>
|
||||
* JAVA API
|
||||
*/
|
||||
@deprecated("use ActorRefFactory (ActorSystem or ActorContext) to create actors", "2.0")
|
||||
def actorOf(creator: Creator[Actor]): ActorRef = GlobalActorSystem.actorOf(Props(creator))
|
||||
|
||||
@deprecated("OldActor.remote should not be used", "2.0")
|
||||
lazy val remote: OldRemoteSupport = new OldRemoteSupport
|
||||
|
||||
}
|
||||
|
||||
@deprecated("use Actor", "2.0")
|
||||
abstract class OldActor extends Actor {
|
||||
|
||||
implicit def askTimeout: Timeout = akka.migration.askTimeout
|
||||
|
||||
implicit def future2OldFuture[T](future: Future[T]): OldFuture[T] = akka.migration.future2OldFuture(future)
|
||||
|
||||
implicit def actorRef2OldActorRef(actorRef: ActorRef) = new OldActorRef(actorRef)
|
||||
|
||||
@deprecated("Use context.become instead", "2.0")
|
||||
def become(behavior: Receive, discardOld: Boolean = true) = context.become(behavior, discardOld)
|
||||
|
||||
@deprecated("Use context.unbecome instead", "2.0")
|
||||
def unbecome() = context.unbecome()
|
||||
|
||||
class OldActorRef(actorRef: ActorRef) {
|
||||
@deprecated("Actors are automatically started when creatd, i.e. remove old call to start()", "2.0")
|
||||
def start(): ActorRef = actorRef
|
||||
|
||||
@deprecated("Stop with ActorSystem or ActorContext instead", "2.0")
|
||||
def exit() = stop()
|
||||
|
||||
@deprecated("Stop with ActorSystem or ActorContext instead", "2.0")
|
||||
def stop(): Unit = context.stop(actorRef)
|
||||
|
||||
@deprecated("Use context.getReceiveTimeout instead", "2.0")
|
||||
def getReceiveTimeout(): Option[Long] = context.receiveTimeout.map(_.toMillis)
|
||||
|
||||
@deprecated("Use context.setReceiveTimeout instead", "2.0")
|
||||
def setReceiveTimeout(timeout: Long) = context.setReceiveTimeout(Duration(timeout, TimeUnit.MILLISECONDS))
|
||||
|
||||
@deprecated("Use context.getReceiveTimeout instead", "2.0")
|
||||
def receiveTimeout: Option[Long] = getReceiveTimeout()
|
||||
|
||||
@deprecated("Use context.setReceiveTimeout instead", "2.0")
|
||||
def receiveTimeout_=(timeout: Option[Long]) = setReceiveTimeout(timeout.getOrElse(0L))
|
||||
|
||||
@deprecated("Use self.isTerminated instead", "2.0")
|
||||
def isShutdown: Boolean = self.isTerminated
|
||||
|
||||
@deprecated("Use sender instead", "2.0")
|
||||
def channel() = context.sender
|
||||
|
||||
@deprecated("Use sender instead", "2.0")
|
||||
def sender() = Some(context.sender)
|
||||
|
||||
@deprecated("Use sender ! instead", "2.0")
|
||||
def reply(message: Any) = context.sender.!(message, context.self)
|
||||
|
||||
@deprecated("Use sender ! instead", "2.0")
|
||||
def tryReply(message: Any): Boolean = {
|
||||
reply(message)
|
||||
true
|
||||
}
|
||||
|
||||
@deprecated("Use sender ! instead", "2.0")
|
||||
def tryTell(message: Any)(implicit sender: ActorRef = context.self): Boolean = {
|
||||
actorRef.!(message)(sender)
|
||||
true
|
||||
}
|
||||
|
||||
@deprecated("Use sender ! akka.actor.Status.Failure(e) instead", "2.0")
|
||||
def sendException(ex: Throwable): Boolean = {
|
||||
context.sender.!(akka.actor.Status.Failure(ex), context.self)
|
||||
true
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class OldRemoteSupport {
|
||||
|
||||
@deprecated("remote.start is not needed", "2.0")
|
||||
def start() {}
|
||||
|
||||
@deprecated("remote.start is not needed, use configuration to specify RemoteActorRefProvider, host and port", "2.0")
|
||||
def start(host: String, port: Int) {}
|
||||
|
||||
@deprecated("remote.start is not needed, use configuration to specify RemoteActorRefProvider, host and port", "2.0")
|
||||
def start(host: String, port: Int, loader: ClassLoader) {}
|
||||
|
||||
@deprecated("remote.shutdown is not needed", "2.0")
|
||||
def shutdown() {}
|
||||
|
||||
@deprecated("use actorFor in ActorRefProvider (ActorSystem or ActorContext) instead", "2.0")
|
||||
def actorFor(classNameOrServiceId: String, hostname: String, port: Int): ActorRef =
|
||||
GlobalActorSystem.actorFor("akka://%s@%s:%s/user/%s".format(GlobalActorSystem.name, hostname, port, classNameOrServiceId))
|
||||
|
||||
@deprecated("use actorFor in ActorRefProvider (ActorSystem or ActorContext) instead", "2.0")
|
||||
def actorFor(classNameOrServiceId: String, hostname: String, port: Int, loader: ClassLoader): ActorRef =
|
||||
actorFor(classNameOrServiceId, hostname, port)
|
||||
|
||||
@deprecated("use actorFor in ActorRefProvider (ActorSystem or ActorContext) instead", "2.0")
|
||||
def actorFor(serviceId: String, className: String, hostname: String, port: Int): ActorRef =
|
||||
actorFor(serviceId, hostname, port)
|
||||
|
||||
@deprecated("use actorFor in ActorRefProvider (ActorSystem or ActorContext) instead", "2.0")
|
||||
def actorFor(serviceId: String, className: String, hostname: String, port: Int, loader: ClassLoader): ActorRef =
|
||||
actorFor(serviceId, hostname, port)
|
||||
|
||||
@deprecated("use actorFor in ActorRefProvider (ActorSystem or ActorContext) instead", "2.0")
|
||||
def actorFor(classNameOrServiceId: String, timeout: Long, hostname: String, port: Int): ActorRef =
|
||||
actorFor(classNameOrServiceId, hostname, port)
|
||||
|
||||
@deprecated("use actorFor in ActorRefProvider (ActorSystem or ActorContext) instead", "2.0")
|
||||
def actorFor(classNameOrServiceId: String, timeout: Long, hostname: String, port: Int, loader: ClassLoader): ActorRef =
|
||||
actorFor(classNameOrServiceId, hostname, port)
|
||||
|
||||
@deprecated("use actorFor in ActorRefProvider (ActorSystem or ActorContext) instead", "2.0")
|
||||
def actorFor(serviceId: String, className: String, timeout: Long, hostname: String, port: Int): ActorRef =
|
||||
actorFor(serviceId, hostname, port)
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,75 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.actor
|
||||
|
||||
import java.util.concurrent.TimeUnit
|
||||
import akka.util.Duration
|
||||
|
||||
/**
|
||||
* Migration replacement for `object akka.actor.Scheduler`.
|
||||
*/
|
||||
@deprecated("use ActorSystem.scheduler instead", "2.0")
|
||||
object OldScheduler {
|
||||
|
||||
/**
|
||||
* Schedules to send the specified message to the receiver after initialDelay and then repeated after delay
|
||||
*/
|
||||
@deprecated("use ActorSystem.scheduler instead", "2.0")
|
||||
def schedule(receiver: ActorRef, message: Any, initialDelay: Long, delay: Long, timeUnit: TimeUnit): Cancellable =
|
||||
GlobalActorSystem.scheduler.schedule(
|
||||
Duration(initialDelay, timeUnit),
|
||||
Duration(delay, timeUnit),
|
||||
receiver,
|
||||
message)
|
||||
|
||||
/**
|
||||
* Schedules to run specified function to the receiver after initialDelay and then repeated after delay
|
||||
*/
|
||||
@deprecated("use ActorSystem.scheduler instead", "2.0")
|
||||
def schedule(f: () ⇒ Unit, initialDelay: Long, delay: Long, timeUnit: TimeUnit): Cancellable =
|
||||
GlobalActorSystem.scheduler.schedule(
|
||||
Duration(initialDelay, timeUnit),
|
||||
Duration(delay, timeUnit),
|
||||
new Runnable { def run = f() })
|
||||
|
||||
/**
|
||||
* Schedules to run specified runnable to the receiver after initialDelay and then repeated after delay.
|
||||
*/
|
||||
@deprecated("use ActorSystem.scheduler instead", "2.0")
|
||||
def schedule(runnable: Runnable, initialDelay: Long, delay: Long, timeUnit: TimeUnit): Cancellable =
|
||||
GlobalActorSystem.scheduler.schedule(
|
||||
Duration(initialDelay, timeUnit),
|
||||
Duration(delay, timeUnit),
|
||||
runnable)
|
||||
|
||||
/**
|
||||
* Schedules to send the specified message to the receiver after delay
|
||||
*/
|
||||
@deprecated("use ActorSystem.scheduler instead", "2.0")
|
||||
def scheduleOnce(receiver: ActorRef, message: Any, delay: Long, timeUnit: TimeUnit): Cancellable =
|
||||
GlobalActorSystem.scheduler.scheduleOnce(
|
||||
Duration(delay, timeUnit),
|
||||
receiver,
|
||||
message)
|
||||
|
||||
/**
|
||||
* Schedules a function to be run after delay.
|
||||
*/
|
||||
@deprecated("use ActorSystem.scheduler instead", "2.0")
|
||||
def scheduleOnce(f: () ⇒ Unit, delay: Long, timeUnit: TimeUnit): Cancellable =
|
||||
GlobalActorSystem.scheduler.scheduleOnce(
|
||||
Duration(delay, timeUnit),
|
||||
new Runnable { def run = f() })
|
||||
|
||||
/**
|
||||
* Schedules a runnable to be run after delay,
|
||||
*/
|
||||
@deprecated("use ActorSystem.scheduler instead", "2.0")
|
||||
def scheduleOnce(runnable: Runnable, delay: Long, timeUnit: TimeUnit): Cancellable =
|
||||
GlobalActorSystem.scheduler.scheduleOnce(
|
||||
Duration(delay, timeUnit),
|
||||
runnable)
|
||||
|
||||
}
|
||||
|
||||
162
akka-actor-migration/src/main/scala/akka/config/OldConfig.scala
Normal file
162
akka-actor-migration/src/main/scala/akka/config/OldConfig.scala
Normal file
|
|
@ -0,0 +1,162 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.config
|
||||
import akka.actor.GlobalActorSystem
|
||||
import com.typesafe.config.Config
|
||||
|
||||
/**
|
||||
* Migration replacement for `object akka.config.Config`.
|
||||
*/
|
||||
@deprecated("use ActorSystem.settings.config instead", "2.0")
|
||||
object OldConfig {
|
||||
|
||||
val config = new OldConfiguration(GlobalActorSystem.settings.config)
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Migration adapter for `akka.config.Configuration`
|
||||
*/
|
||||
@deprecated("use ActorSystem.settings.config (com.typesafe.config.Config) instead", "2.0")
|
||||
class OldConfiguration(config: Config) {
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def contains(key: String): Boolean = config.hasPath(key)
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def keys: Iterable[String] = config.root.keySet.asScala
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getAny(key: String): Option[Any] = {
|
||||
try {
|
||||
Option(config.getAnyRef(key))
|
||||
} catch {
|
||||
case _ ⇒ None
|
||||
}
|
||||
}
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getAny(key: String, defaultValue: Any): Any = getAny(key).getOrElse(defaultValue)
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getSeqAny(key: String): Seq[Any] = {
|
||||
try {
|
||||
config.getAnyRefList(key).asScala
|
||||
} catch {
|
||||
case _ ⇒ Seq.empty[Any]
|
||||
}
|
||||
}
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getString(key: String): Option[String] =
|
||||
try {
|
||||
Option(config.getString(key))
|
||||
} catch {
|
||||
case _ ⇒ None
|
||||
}
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getString(key: String, defaultValue: String): String = getString(key).getOrElse(defaultValue)
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getList(key: String): Seq[String] = {
|
||||
try {
|
||||
config.getStringList(key).asScala
|
||||
} catch {
|
||||
case _ ⇒ Seq.empty[String]
|
||||
}
|
||||
}
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getInt(key: String): Option[Int] = {
|
||||
try {
|
||||
Option(config.getInt(key))
|
||||
} catch {
|
||||
case _ ⇒ None
|
||||
}
|
||||
}
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getInt(key: String, defaultValue: Int): Int = getInt(key).getOrElse(defaultValue)
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getLong(key: String): Option[Long] = {
|
||||
try {
|
||||
Option(config.getLong(key))
|
||||
} catch {
|
||||
case _ ⇒ None
|
||||
}
|
||||
}
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getLong(key: String, defaultValue: Long): Long = getLong(key).getOrElse(defaultValue)
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getFloat(key: String): Option[Float] = {
|
||||
try {
|
||||
Option(config.getDouble(key).toFloat)
|
||||
} catch {
|
||||
case _ ⇒ None
|
||||
}
|
||||
}
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getFloat(key: String, defaultValue: Float): Float = getFloat(key).getOrElse(defaultValue)
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getDouble(key: String): Option[Double] = {
|
||||
try {
|
||||
Option(config.getDouble(key))
|
||||
} catch {
|
||||
case _ ⇒ None
|
||||
}
|
||||
}
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getDouble(key: String, defaultValue: Double): Double = getDouble(key).getOrElse(defaultValue)
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getBoolean(key: String): Option[Boolean] = {
|
||||
try {
|
||||
Option(config.getBoolean(key))
|
||||
} catch {
|
||||
case _ ⇒ None
|
||||
}
|
||||
}
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getBoolean(key: String, defaultValue: Boolean): Boolean = getBoolean(key).getOrElse(defaultValue)
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getBool(key: String): Option[Boolean] = getBoolean(key)
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getBool(key: String, defaultValue: Boolean): Boolean = getBoolean(key, defaultValue)
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def apply(key: String): String = getString(key) match {
|
||||
case None ⇒ throw new ConfigurationException("undefined config: " + key)
|
||||
case Some(v) ⇒ v
|
||||
}
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def apply(key: String, defaultValue: String) = getString(key, defaultValue)
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def apply(key: String, defaultValue: Int) = getInt(key, defaultValue)
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def apply(key: String, defaultValue: Long) = getLong(key, defaultValue)
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def apply(key: String, defaultValue: Boolean) = getBool(key, defaultValue)
|
||||
|
||||
@deprecated("use new com.typesafe.config.Config API instead", "2.0")
|
||||
def getSection(name: String): Option[OldConfiguration] = {
|
||||
try {
|
||||
Option(new OldConfiguration(config.getConfig(name)))
|
||||
} catch {
|
||||
case _ ⇒ None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,65 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.dispatch
|
||||
|
||||
import java.util.concurrent.TimeoutException
|
||||
import akka.util.duration._
|
||||
import akka.AkkaException
|
||||
import akka.util.BoxedType
|
||||
import akka.util.Duration
|
||||
import akka.actor.GlobalActorSystem
|
||||
|
||||
/**
|
||||
* Some old methods made available through implicit conversion in
|
||||
* [[akka.migration]].
|
||||
*/
|
||||
@deprecated("use new Future api instead", "2.0")
|
||||
class OldFuture[T](future: Future[T]) {
|
||||
|
||||
@deprecated("use akka.dispatch.Await.result instead", "2.0")
|
||||
def get: T = try {
|
||||
Await.result(future, GlobalActorSystem.settings.ActorTimeout.duration)
|
||||
} catch {
|
||||
case e: TimeoutException ⇒ throw new FutureTimeoutException(e.getMessage, e)
|
||||
}
|
||||
|
||||
@deprecated("use akka.dispatch.Await.ready instead", "2.0")
|
||||
def await: Future[T] = await(GlobalActorSystem.settings.ActorTimeout.duration)
|
||||
|
||||
@deprecated("use akka.dispatch.Await.ready instead", "2.0")
|
||||
def await(atMost: Duration) = try {
|
||||
Await.ready(future, atMost)
|
||||
future
|
||||
} catch {
|
||||
case e: TimeoutException ⇒ throw new FutureTimeoutException(e.getMessage, e)
|
||||
}
|
||||
|
||||
@deprecated("use new Future api instead", "2.0")
|
||||
def as[A](implicit m: Manifest[A]): Option[A] = {
|
||||
try await catch { case _: FutureTimeoutException ⇒ }
|
||||
future.value match {
|
||||
case None ⇒ None
|
||||
case Some(Left(ex)) ⇒ throw ex
|
||||
case Some(Right(v)) ⇒ Some(BoxedType(m.erasure).cast(v).asInstanceOf[A])
|
||||
}
|
||||
}
|
||||
|
||||
@deprecated("use new Future api instead", "2.0")
|
||||
def asSilently[A](implicit m: Manifest[A]): Option[A] = {
|
||||
try await catch { case _: FutureTimeoutException ⇒ }
|
||||
future.value match {
|
||||
case None ⇒ None
|
||||
case Some(Left(ex)) ⇒ throw ex
|
||||
case Some(Right(v)) ⇒
|
||||
try Some(BoxedType(m.erasure).cast(v).asInstanceOf[A])
|
||||
catch { case _: ClassCastException ⇒ None }
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@deprecated("Await throws java.util.concurrent.TimeoutException", "2.0")
|
||||
class FutureTimeoutException(message: String, cause: Throwable = null) extends AkkaException(message, cause) {
|
||||
def this(message: String) = this(message, null)
|
||||
}
|
||||
|
|
@ -0,0 +1,81 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.event
|
||||
|
||||
import akka.actor.GlobalActorSystem
|
||||
|
||||
/**
|
||||
* Migration replacement for `akka.event.EventHandler`
|
||||
*/
|
||||
@deprecated("use Logging instead", "2.0")
|
||||
object OldEventHandler {
|
||||
|
||||
@deprecated("use Logging instead", "2.0")
|
||||
def error(cause: Throwable, instance: AnyRef, message: ⇒ String) {
|
||||
val log = Logging.getLogger(GlobalActorSystem, instance)
|
||||
if (log.isErrorEnabled) log.error(cause, message)
|
||||
}
|
||||
|
||||
@deprecated("use Logging instead", "2.0")
|
||||
def error(cause: Throwable, instance: AnyRef, message: Any) {
|
||||
val log = Logging.getLogger(GlobalActorSystem, instance)
|
||||
if (log.isErrorEnabled) log.error(cause, message.toString)
|
||||
}
|
||||
|
||||
@deprecated("use Logging instead", "2.0")
|
||||
def error(instance: AnyRef, message: ⇒ String) {
|
||||
val log = Logging.getLogger(GlobalActorSystem, instance)
|
||||
if (log.isErrorEnabled) log.error(message.toString)
|
||||
}
|
||||
|
||||
@deprecated("use Logging instead", "2.0")
|
||||
def error(instance: AnyRef, message: Any) {
|
||||
val log = Logging.getLogger(GlobalActorSystem, instance)
|
||||
if (log.isErrorEnabled) log.error(message.toString)
|
||||
}
|
||||
|
||||
@deprecated("use Logging instead", "2.0")
|
||||
def warning(instance: AnyRef, message: ⇒ String) {
|
||||
val log = Logging.getLogger(GlobalActorSystem, instance)
|
||||
if (log.isWarningEnabled) log.warning(message)
|
||||
}
|
||||
|
||||
@deprecated("use Logging instead", "2.0")
|
||||
def warning(instance: AnyRef, message: Any) {
|
||||
val log = Logging.getLogger(GlobalActorSystem, instance)
|
||||
if (log.isWarningEnabled) log.warning(message.toString)
|
||||
}
|
||||
|
||||
@deprecated("use Logging instead", "2.0")
|
||||
def info(instance: AnyRef, message: ⇒ String) {
|
||||
val log = Logging.getLogger(GlobalActorSystem, instance)
|
||||
if (log.isInfoEnabled) log.info(message)
|
||||
}
|
||||
|
||||
@deprecated("use Logging instead", "2.0")
|
||||
def info(instance: AnyRef, message: Any) {
|
||||
val log = Logging.getLogger(GlobalActorSystem, instance)
|
||||
if (log.isInfoEnabled) log.info(message.toString)
|
||||
}
|
||||
|
||||
@deprecated("use Logging instead", "2.0")
|
||||
def debug(instance: AnyRef, message: ⇒ String) {
|
||||
val log = Logging.getLogger(GlobalActorSystem, instance)
|
||||
if (log.isDebugEnabled) log.debug(message)
|
||||
}
|
||||
|
||||
@deprecated("use Logging instead", "2.0")
|
||||
def debug(instance: AnyRef, message: Any) {
|
||||
val log = Logging.getLogger(GlobalActorSystem, instance)
|
||||
if (log.isDebugEnabled) log.debug(message.toString)
|
||||
}
|
||||
|
||||
@deprecated("use Logging instead", "2.0")
|
||||
def isInfoEnabled = Logging.getLogger(GlobalActorSystem, this).isInfoEnabled
|
||||
|
||||
@deprecated("use Logging instead", "2.0")
|
||||
def isDebugEnabled = Logging.getLogger(GlobalActorSystem, this).isDebugEnabled
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,34 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka
|
||||
|
||||
import akka.dispatch.Future
|
||||
import akka.dispatch.OldFuture
|
||||
import akka.util.Timeout
|
||||
import akka.actor.GlobalActorSystem
|
||||
import akka.dispatch.MessageDispatcher
|
||||
import akka.actor.ActorRef
|
||||
|
||||
package object migration {
|
||||
|
||||
implicit def future2OldFuture[T](future: Future[T]): OldFuture[T] = new OldFuture[T](future)
|
||||
|
||||
implicit def askTimeout: Timeout = GlobalActorSystem.settings.ActorTimeout
|
||||
|
||||
implicit def defaultDispatcher: MessageDispatcher = GlobalActorSystem.dispatcher
|
||||
|
||||
implicit def actorRef2OldActorRef(actorRef: ActorRef) = new OldActorRef(actorRef)
|
||||
|
||||
class OldActorRef(actorRef: ActorRef) {
|
||||
@deprecated("Actors are automatically started when creatd, i.e. remove old call to start()", "2.0")
|
||||
def start(): ActorRef = actorRef
|
||||
|
||||
@deprecated("Stop with ActorSystem or ActorContext instead", "2.0")
|
||||
def exit() = stop()
|
||||
|
||||
@deprecated("Stop with ActorSystem or ActorContext instead", "2.0")
|
||||
def stop(): Unit = GlobalActorSystem.stop(actorRef)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,3 +1,5 @@
|
|||
.. _jmm:
|
||||
|
||||
Akka and the Java Memory Model
|
||||
================================
|
||||
|
||||
|
|
|
|||
|
|
@ -4,6 +4,10 @@
|
|||
Migration Guide 1.3.x to 2.0.x
|
||||
################################
|
||||
|
||||
.. sidebar:: Contents
|
||||
|
||||
.. contents:: :local:
|
||||
|
||||
Actors
|
||||
======
|
||||
|
||||
|
|
@ -13,9 +17,177 @@ significant amount of time.
|
|||
|
||||
Detailed migration guide will be written.
|
||||
|
||||
Migration Kit
|
||||
=============
|
||||
|
||||
Nobody likes a big refactoring that takes several days to complete until
|
||||
anything is able to run again. Therefore we provide a migration kit that
|
||||
makes it possible to do the migration changes in smaller steps.
|
||||
|
||||
The migration kit only covers the most common usage of Akka. It is not intended
|
||||
as a final solution. The whole migration kit is deprecated and will be removed in
|
||||
Akka 2.1.
|
||||
|
||||
The migration kit is provided in separate jar files. Add the following dependency::
|
||||
|
||||
"com.typesafe.akka" % "akka-actor-migration" % "2.0-SNAPSHOT"
|
||||
|
||||
The first step of the migration is to do some trivial replacements.
|
||||
Search and replace the following (be careful with the non qualified names):
|
||||
|
||||
==================================== ====================================
|
||||
Search Replace with
|
||||
==================================== ====================================
|
||||
``akka.actor.Actor`` ``akka.actor.OldActor``
|
||||
``extends Actor`` ``extends OldActor``
|
||||
``akka.actor.Scheduler`` ``akka.actor.OldScheduler``
|
||||
``Scheduler`` ``OldScheduler``
|
||||
``akka.event.EventHandler`` ``akka.event.OldEventHandler``
|
||||
``EventHandler`` ``OldEventHandler``
|
||||
``akka.config.Config`` ``akka.config.OldConfig``
|
||||
``Config`` ``OldConfig``
|
||||
==================================== ====================================
|
||||
|
||||
For Scala users the migration kit also contains some implicit conversions to be
|
||||
able to use some old methods. These conversions are useful from tests or other
|
||||
code used outside actors.
|
||||
|
||||
::
|
||||
|
||||
import akka.migration._
|
||||
|
||||
Thereafter you need to fix compilation errors that are not handled by the migration
|
||||
kit, such as:
|
||||
|
||||
* Definition of supervisors
|
||||
* Definition of dispatchers
|
||||
* ActorRegistry
|
||||
|
||||
When everything compiles you continue by replacing/removing the ``OldXxx`` classes
|
||||
one-by-one from the migration kit with appropriate migration.
|
||||
|
||||
When using the migration kit there will be one global actor system, which loads
|
||||
the configuration ``akka.conf`` from the same locations as in Akka 1.x.
|
||||
This means that while you are using the migration kit you should not create your
|
||||
own ``ActorSystem``, but instead use the ``akka.actor.GlobalActorSystem``.
|
||||
In order to voluntarily exit the JVM you must ``shutdown`` the ``GlobalActorSystem``
|
||||
Last task of the migration would be to create your own ``ActorSystem``.
|
||||
|
||||
|
||||
Unordered Collection of Migration Items
|
||||
=======================================
|
||||
|
||||
Creating and starting actors
|
||||
----------------------------
|
||||
|
||||
Actors are created by passing in a ``Props`` instance into the actorOf factory method in
|
||||
a ``ActorRefProvider``, which is the ``ActorSystem`` or ``ActorContext``.
|
||||
Use the system to create top level actors. Use the context to
|
||||
create actors from other actors. The difference is how the supervisor hierarchy is arranged.
|
||||
When using the context the current actor will be supervisor of the created child actor.
|
||||
When using the system it will be a top level actor, that is supervised by the system
|
||||
(internal guardian actor).
|
||||
|
||||
``ActorRef.start()`` has been removed. Actors are now started automatically when created.
|
||||
Remove all invocations of ``ActorRef.start()``.
|
||||
|
||||
v1.3::
|
||||
|
||||
val myActor = Actor.actorOf[MyActor]
|
||||
myActor.start()
|
||||
|
||||
v2.0::
|
||||
|
||||
// top level actor
|
||||
val firstActor = system.actorOf(Props[FirstActor], name = "first")
|
||||
|
||||
// child actor
|
||||
class FirstActor extends Actor {
|
||||
val myActor = context.actorOf(Props[MyActor], name = "myactor")
|
||||
|
||||
Documentation:
|
||||
|
||||
* :ref:`actors-scala`
|
||||
* :ref:`untyped-actors-java`
|
||||
|
||||
Stopping actors
|
||||
---------------
|
||||
|
||||
``ActorRef.stop()`` has been moved. Use ``ActorSystem`` or ``ActorContext`` to stop actors.
|
||||
|
||||
v1.3::
|
||||
|
||||
actorRef.stop()
|
||||
self.stop()
|
||||
actorRef ! PoisonPill
|
||||
|
||||
v2.0::
|
||||
|
||||
context.stop(someChild)
|
||||
context.stop(self)
|
||||
system.stop(actorRef)
|
||||
actorRef ! PoisonPill
|
||||
|
||||
*Stop all actors*
|
||||
|
||||
v1.3::
|
||||
|
||||
ActorRegistry.shutdownAll()
|
||||
|
||||
v2.0::
|
||||
|
||||
system.shutdown()
|
||||
|
||||
Documentation:
|
||||
|
||||
* :ref:`actors-scala`
|
||||
* :ref:`untyped-actors-java`
|
||||
|
||||
Identifying Actors
|
||||
------------------
|
||||
|
||||
In v1.3 actors have ``uuid`` and ``id`` field. In v2.0 each actor has a unique logical ``path``.
|
||||
|
||||
The ``ActorRegistry`` has been replaced by actor paths and lookup with
|
||||
``actorFor`` in ``ActorRefProvider`` (``ActorSystem`` or ``ActorContext``).
|
||||
|
||||
v1.3::
|
||||
|
||||
val actor = Actor.registry.actorFor(uuid)
|
||||
val actors = Actor.registry.actorsFor(id)
|
||||
|
||||
v2.0::
|
||||
|
||||
val actor = context.actorFor("/user/serviceA/aggregator")
|
||||
|
||||
Documentation:
|
||||
|
||||
* :ref:`addressing`
|
||||
* :ref:`actors-scala`
|
||||
* :ref:`untyped-actors-java`
|
||||
|
||||
Reply to messages
|
||||
-----------------
|
||||
|
||||
``self.channel`` has been replaced with unified reply mechanism using ``sender`` (Scala)
|
||||
or ``getSender()`` (Java). This works for both tell (!) and ask (?).
|
||||
|
||||
v1.3::
|
||||
|
||||
self.channel ! result
|
||||
self.channel tryTell result
|
||||
self.reply(result)
|
||||
self.tryReply(result)
|
||||
|
||||
v2.0::
|
||||
|
||||
sender ! result
|
||||
|
||||
Documentation:
|
||||
|
||||
* :ref:`actors-scala`
|
||||
* :ref:`untyped-actors-java`
|
||||
|
||||
``ActorRef.ask()``
|
||||
------------------
|
||||
|
||||
|
|
@ -28,7 +200,185 @@ reply to be received; it is independent of the timeout applied when awaiting
|
|||
completion of the :class:`Future`, however, the actor will complete the
|
||||
:class:`Future` with an :class:`AskTimeoutException` when it stops itself.
|
||||
|
||||
Documentation:
|
||||
|
||||
* :ref:`actors-scala`
|
||||
* :ref:`untyped-actors-java`
|
||||
|
||||
ActorPool
|
||||
---------
|
||||
|
||||
The ActorPool has been replaced by dynamically resizable routers.
|
||||
|
||||
Documentation:
|
||||
|
||||
* :ref:`routing-scala`
|
||||
* :ref:`routing-java`
|
||||
|
||||
``UntypedActor.getContext()`` (Java API only)
|
||||
---------------------------------------------
|
||||
|
||||
``getContext()`` in the Java API for UntypedActor is renamed to
|
||||
``getSelf()``.
|
||||
|
||||
v1.3::
|
||||
|
||||
actorRef.tell("Hello", getContext());
|
||||
|
||||
v2.0::
|
||||
|
||||
actorRef.tell("Hello", getSelf());
|
||||
|
||||
Documentation:
|
||||
|
||||
* :ref:`untyped-actors-java`
|
||||
|
||||
Logging
|
||||
-------
|
||||
|
||||
EventHandler API has been replaced by LoggingAdapter, which publish log messages
|
||||
to the event bus. You can still plugin your own actor as event listener with the
|
||||
``akka.event-handlers`` configuration property.
|
||||
|
||||
v1.3::
|
||||
|
||||
EventHandler.error(exception, this, message)
|
||||
EventHandler.warning(this, message)
|
||||
EventHandler.info(this, message)
|
||||
EventHandler.debug(this, message)
|
||||
EventHandler.debug(this, "Processing took %s ms".format(duration))
|
||||
|
||||
v2.0::
|
||||
|
||||
import akka.event.Logging
|
||||
|
||||
val log = Logging(context.system, this)
|
||||
log.error(exception, this, message)
|
||||
log.warning(this, message)
|
||||
log.info(this, message)
|
||||
log.debug(this, message)
|
||||
log.debug(this, "Processing took {} ms", duration)
|
||||
|
||||
Documentation:
|
||||
|
||||
* :ref:`logging-scala`
|
||||
* :ref:`logging-java`
|
||||
* :ref:`event-bus-scala`
|
||||
* :ref:`event-bus-java`
|
||||
|
||||
Supervision
|
||||
-----------
|
||||
|
||||
Akka v2.0 implements parental supervision. Actors can only be created by other actors — where the top-level
|
||||
actor is provided by the library — and each created actor is supervised by its parent.
|
||||
In contrast to the special supervision relationship between parent and child, each actor may monitor any
|
||||
other actor for termination.
|
||||
|
||||
v1.3::
|
||||
|
||||
self.link(actorRef)
|
||||
self.unlink(actorRef)
|
||||
|
||||
v2.0::
|
||||
|
||||
class WatchActor extends Actor {
|
||||
val actorRef = ...
|
||||
// Terminated message will be delivered when the actorRef actor
|
||||
// is stopped
|
||||
context.watch(actorRef)
|
||||
|
||||
val supervisedChild = context.actorOf(Props[ChildActor])
|
||||
|
||||
def receive = {
|
||||
case Terminated(`actorRef`) ⇒ ...
|
||||
}
|
||||
}
|
||||
|
||||
Note that ``link`` in v1.3 established a supervision relation, which ``watch`` doesn't.
|
||||
``watch`` is only a way to get notification, ``Terminated`` message, when the monitored
|
||||
actor has been stopped.
|
||||
|
||||
*Refererence to the supervisor*
|
||||
|
||||
v1.3::
|
||||
|
||||
self.supervisor
|
||||
|
||||
v2.0::
|
||||
|
||||
context.parent
|
||||
|
||||
*Fault handling strategy*
|
||||
|
||||
v1.3::
|
||||
|
||||
val supervisor = Supervisor(
|
||||
SupervisorConfig(
|
||||
AllForOneStrategy(List(classOf[Exception]), 3, 1000),
|
||||
Supervise(
|
||||
actorOf[MyActor1],
|
||||
Permanent) ::
|
||||
Supervise(
|
||||
actorOf[MyActor2],
|
||||
Permanent) ::
|
||||
Nil))
|
||||
|
||||
v2.0::
|
||||
|
||||
val strategy = OneForOneStrategy({
|
||||
case _: ArithmeticException ⇒ Resume
|
||||
case _: NullPointerException ⇒ Restart
|
||||
case _: IllegalArgumentException ⇒ Stop
|
||||
case _: Exception ⇒ Escalate
|
||||
}: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000))
|
||||
|
||||
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(strategy), "supervisor")
|
||||
|
||||
Documentation:
|
||||
|
||||
* :ref:`supervision`
|
||||
* :ref:`fault-tolerance-java`
|
||||
* :ref:`fault-tolerance-scala`
|
||||
* :ref:`actors-scala`
|
||||
* :ref:`untyped-actors-java`
|
||||
|
||||
Spawn
|
||||
-----
|
||||
|
||||
``spawn`` has been removed and can be implemented like this, if needed. Be careful to not
|
||||
access any shared mutable state closed over by the body.
|
||||
|
||||
::
|
||||
|
||||
def spawn(body: ⇒ Unit) {
|
||||
system.actorOf(Props(ctx ⇒ { case "go" ⇒ try body finally ctx.stop(ctx.self) })) ! "go"
|
||||
}
|
||||
|
||||
Documentation:
|
||||
|
||||
* :ref:`jmm`
|
||||
|
||||
HotSwap
|
||||
-------
|
||||
|
||||
In v2.0 ``become`` and ``unbecome`` metods are located in ``ActorContext``, i.e. ``context.become`` and ``context.unbecome``.
|
||||
|
||||
The special ``HotSwap`` and ``RevertHotswap`` messages in v1.3 has been removed. Similar can be
|
||||
implemented with your own message and using ``context.become`` and ``context.unbecome``
|
||||
in the actor receiving the message.
|
||||
|
||||
* :ref:`actors-scala`
|
||||
* :ref:`untyped-actors-java`
|
||||
|
||||
More to be written
|
||||
------------------
|
||||
|
||||
* Futures
|
||||
* Dispatchers
|
||||
* STM
|
||||
* TypedActors
|
||||
* Routing
|
||||
* Remoting
|
||||
* Scheduler
|
||||
* Configuration
|
||||
* ...?
|
||||
|
|
@ -7,6 +7,9 @@ package akka.docs.actor
|
|||
import akka.actor.Actor
|
||||
import akka.actor.Props
|
||||
import akka.event.Logging
|
||||
|
||||
//#imports1
|
||||
|
||||
import akka.dispatch.Future
|
||||
import akka.actor.ActorSystem
|
||||
import org.scalatest.{ BeforeAndAfterAll, WordSpec }
|
||||
|
|
|
|||
|
|
@ -31,7 +31,7 @@ object AkkaBuild extends Build {
|
|||
Unidoc.unidocExclude := Seq(samples.id, tutorials.id),
|
||||
Dist.distExclude := Seq(actorTests.id, akkaSbtPlugin.id, docs.id)
|
||||
),
|
||||
aggregate = Seq(actor, testkit, actorTests, remote, slf4j, agent, transactor, mailboxes, kernel, akkaSbtPlugin, samples, tutorials, docs)
|
||||
aggregate = Seq(actor, testkit, actorTests, remote, slf4j, agent, transactor, mailboxes, kernel, akkaSbtPlugin, actorMigration, samples, tutorials, docs)
|
||||
)
|
||||
|
||||
lazy val actor = Project(
|
||||
|
|
@ -213,6 +213,13 @@ object AkkaBuild extends Build {
|
|||
)
|
||||
)
|
||||
|
||||
lazy val actorMigration = Project(
|
||||
id = "akka-actor-migration",
|
||||
base = file("akka-actor-migration"),
|
||||
dependencies = Seq(actor, testkit % "test->test"),
|
||||
settings = defaultSettings
|
||||
)
|
||||
|
||||
lazy val akkaSbtPlugin = Project(
|
||||
id = "akka-sbt-plugin",
|
||||
base = file("akka-sbt-plugin"),
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue