Merged with current master

This commit is contained in:
Viktor Klang 2011-12-13 10:04:43 +01:00
commit bf01045779
144 changed files with 6879 additions and 8250 deletions

1
.gitignore vendored
View file

@ -16,6 +16,7 @@ reports
dist
target
deploy/*.jar
.history
data
out
logs

View file

@ -1,4 +0,0 @@
update
reload
projects
exit

View file

@ -6,8 +6,6 @@ import akka.testkit.AkkaSpec
/**
* Tests the behavior of the executor based event driven dispatcher when multiple actors are being dispatched on it.
*
* @author Jan Van Besien
*/
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class DispatcherActorsSpec extends AkkaSpec {

View file

@ -58,13 +58,13 @@ class DispatchersSpec extends AkkaSpec {
dispatcher.map(_.throughput) must be(Some(17))
}
"use defined properties when fromConfig" in {
val dispatcher = fromConfig("myapp.mydispatcher", cfg = dispatcherConf)
"use defined properties when newFromConfig" in {
val dispatcher = newFromConfig("myapp.mydispatcher", defaultGlobalDispatcher, dispatcherConf)
dispatcher.throughput must be(17)
}
"use specific name when fromConfig" in {
val dispatcher = fromConfig("myapp.mydispatcher", cfg = dispatcherConf)
"use specific name when newFromConfig" in {
val dispatcher = newFromConfig("myapp.mydispatcher", defaultGlobalDispatcher, dispatcherConf)
dispatcher.name must be("mydispatcher")
}

View file

@ -127,7 +127,7 @@ class TellThroughputSeparateDispatchersPerformanceSpec extends PerformanceSpec {
yield system.actorOf(Props(new Destination).withDispatcher(clientDispatcher))
val clients = for ((dest, j) destinations.zipWithIndex)
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(clientDispatcher))
*/
*/
val start = System.nanoTime
clients.foreach(_ ! Run)

View file

@ -40,6 +40,16 @@ class DurationSpec extends WordSpec with MustMatchers {
(minf + minf) must be(minf)
}
"support fromNow" in {
val dead = 2.seconds.fromNow
val dead2 = 2 seconds fromNow
dead.timeLeft must be > 1.second
dead2.timeLeft must be > 1.second
1.second.sleep
dead.timeLeft must be < 1.second
dead2.timeLeft must be < 1.second
}
}
}

View file

@ -482,10 +482,10 @@ public class HashedWheelTimer implements Timer {
buf.append("deadline: ");
if (remaining > 0) {
buf.append(remaining);
buf.append(" ms later, ");
buf.append(" ns later, ");
} else if (remaining < 0) {
buf.append(-remaining);
buf.append(" ms ago, ");
buf.append(" ns ago, ");
} else {
buf.append("now, ");
}

View file

@ -14,8 +14,6 @@ import java.net.{ InetAddress, UnknownHostException }
* <li>toString that includes exception name, message and uuid</li>
* <li>toLongString which also includes the stack trace</li>
* </ul>
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class AkkaException(message: String = "", cause: Throwable = null) extends RuntimeException(message, cause) with Serializable {
val uuid = "%s_%s".format(AkkaException.hostname, newUuid)

View file

@ -93,7 +93,7 @@ case class ActorInterruptedException private[akka] (cause: Throwable)
/**
* This message is thrown by default when an Actors behavior doesn't match a message
*/
case class UnhandledMessageException(msg: Any, ref: ActorRef = null) extends Exception {
case class UnhandledMessageException(msg: Any, ref: ActorRef = null) extends RuntimeException {
def this(msg: String) = this(msg, null)
@ -167,8 +167,6 @@ object Actor {
*
* <p/>
* The Actor's own ActorRef is available in the 'self' member variable.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait Actor {

View file

@ -46,8 +46,6 @@ import java.util.concurrent.atomic.AtomicBoolean
* </pre>
*
* The natural ordering of ActorRef is defined in terms of its [[akka.actor.ActorPath]].
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable {
scalaRef: InternalActorRef
@ -83,17 +81,29 @@ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable
final def tell(msg: Any, sender: ActorRef): Unit = this.!(msg)(sender)
/**
* Akka Java API. <p/>
* Akka Java API.
*
* Sends a message asynchronously returns a future holding the eventual reply message.
* <p/>
*
* <b>NOTE:</b>
* Use this method with care. In most cases it is better to use 'tell' together with the 'getContext().getSender()' to
* implement request/response message exchanges.
* <p/>
* If you are sending messages using <code>ask</code> then you <b>have to</b> use <code>getContext().sender().tell(...)</code>
* to send a reply message to the original sender. If not then the sender will block until the timeout expires.
* Use this method with care. In most cases it is better to use 'tell' together with the sender
* parameter to implement non-blocking request/response message exchanges.
*
* If you are sending messages using <code>ask</code> and using blocking operations on the Future, such as
* 'get', then you <b>have to</b> use <code>getContext().sender().tell(...)</code>
* in the target actor to send a reply message to the original sender, and thereby completing the Future,
* otherwise the sender will block until the timeout expires.
*
* When using future callbacks, inside actors you need to carefully avoid closing over
* the containing actors reference, i.e. do not call methods or access mutable state
* on the enclosing actor from within the callback. This would break the actor
* encapsulation and may introduce synchronization bugs and race conditions because
* the callback will be scheduled concurrently to the enclosing actor. Unfortunately
* there is not yet a way to detect these illegal accesses at compile time.
*/
def ask(message: AnyRef, timeout: Long): Future[AnyRef] = ?(message, Timeout(timeout)).asInstanceOf[Future[AnyRef]]
def ask(message: AnyRef, timeout: Timeout): Future[AnyRef] = ?(message, timeout).asInstanceOf[Future[AnyRef]]
def ask(message: AnyRef, timeoutMillis: Long): Future[AnyRef] = ask(message, new Timeout(timeoutMillis))
/**
* Forwards the message and passes the original sender actor as the sender.
@ -148,6 +158,21 @@ trait ScalaActorRef { ref: ActorRef ⇒
/**
* Sends a message asynchronously, returning a future which may eventually hold the reply.
* <b>NOTE:</b>
* Use this method with care. In most cases it is better to use '!' together with implicit or explicit
* sender parameter to implement non-blocking request/response message exchanges.
*
* If you are sending messages using <code>ask</code> and using blocking operations on the Future, such as
* 'get', then you <b>have to</b> use <code>getContext().sender().tell(...)</code>
* in the target actor to send a reply message to the original sender, and thereby completing the Future,
* otherwise the sender will block until the timeout expires.
*
* When using future callbacks, inside actors you need to carefully avoid closing over
* the containing actors reference, i.e. do not call methods or access mutable state
* on the enclosing actor from within the callback. This would break the actor
* encapsulation and may introduce synchronization bugs and race conditions because
* the callback will be scheduled concurrently to the enclosing actor. Unfortunately
* there is not yet a way to detect these illegal accesses at compile time.
*/
def ?(message: Any)(implicit timeout: Timeout): Future[Any]
@ -188,8 +213,6 @@ private[akka] case object Nobody extends MinimalActorRef {
/**
* Local (serializable) ActorRef that is used when referencing the Actor on its "home" node.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class LocalActorRef private[akka] (
system: ActorSystemImpl,

View file

@ -30,13 +30,11 @@ trait ActorDeployer {
/**
* Deployer maps actor paths to actor deployments.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream, val nodename: String) extends ActorDeployer {
val deploymentConfig = new DeploymentConfig(nodename)
val log = Logging(eventStream, "Deployer")
private val log = Logging(eventStream, "Deployer")
val instance: ActorDeployer = {
val deployer = new LocalDeployer()
@ -74,11 +72,14 @@ class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream,
private[akka] def lookupDeploymentFor(path: String): Option[Deploy] =
instance.lookupDeploymentFor(path)
private[akka] def deploymentsInConfig: List[Deploy] = {
for (path pathsInConfig) yield lookupInConfig(path)
private def deploymentsInConfig: List[Deploy] = {
val allDeployments = settings.config.getConfig("akka.actor.deployment")
val defaultDeployment = allDeployments.getConfig("default")
// foreach akka.actor.deployment.<path>
for (path pathsInConfig) yield parseDeploymentConfig(allDeployments.getConfig(path), defaultDeployment, path)
}
private[akka] def pathsInConfig: List[String] = {
private def pathsInConfig: List[String] = {
def pathSubstring(path: String) = {
val i = path.indexOf(".")
if (i == -1) path else path.substring(0, i)
@ -92,21 +93,16 @@ class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream,
}
/**
* Lookup deployment in 'akka.conf' configuration file.
* Parse deployment in supplied deployment Config, using the
* defaultDeployment Config as fallback.
* The path is the actor path and used for error reporting.
*
*/
private[akka] def lookupInConfig(path: String, configuration: Config = settings.config): Deploy = {
private def parseDeploymentConfig(deployment: Config, defaultDeployment: Config, path: String): Deploy = {
import scala.collection.JavaConverters._
import akka.util.ReflectiveAccess.getClassFor
val defaultDeploymentConfig = configuration.getConfig("akka.actor.deployment.default")
// --------------------------------
// akka.actor.deployment.<path>
// --------------------------------
val deploymentKey = "akka.actor.deployment." + path
val deployment = configuration.getConfig(deploymentKey)
val deploymentWithFallback = deployment.withFallback(defaultDeploymentConfig)
val deploymentWithFallback = deployment.withFallback(defaultDeployment)
// --------------------------------
// akka.actor.deployment.<path>.router
// --------------------------------
@ -128,7 +124,7 @@ class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream,
if (router == Direct) OneNrOfInstances
else {
def invalidNrOfInstances(wasValue: Any) = new ConfigurationException(
"Config option [" + deploymentKey +
"Deployment config option [" + path +
".nr-of-instances] needs to be either [\"auto\"] or [1-N] - was [" +
wasValue + "]")
@ -155,7 +151,7 @@ class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream,
case "" None
case impl
val implementationClass = getClassFor[Actor](impl).fold(e throw new ConfigurationException(
"Config option [" + deploymentKey + ".create-as.class] load failed", e), identity)
"Deployment config option [" + path + ".create-as.class] load failed", e), identity)
Some(ActorRecipe(implementationClass))
}
@ -167,7 +163,7 @@ class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream,
// --------------------------------
def parseRemote: Scope = {
def raiseRemoteNodeParsingError() = throw new ConfigurationException(
"Config option [" + deploymentKey +
"Deployment config option [" + path +
".remote.nodes] needs to be a list with elements on format \"<hostname>:<port>\", was [" + remoteNodes.mkString(", ") + "]")
val remoteAddresses = remoteNodes map { node
@ -190,7 +186,7 @@ class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream,
// --------------------------------
def parseCluster: Scope = {
def raiseHomeConfigError() = throw new ConfigurationException(
"Config option [" + deploymentKey +
"Deployment config option [" + path +
".cluster.preferred-nodes] needs to be a list with elements on format\n'host:<hostname>', 'ip:<ip address>' or 'node:<node name>', was [" +
clusterPreferredNodes + "]")
@ -222,7 +218,7 @@ class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream,
case "transaction-log" TransactionLog
case "data-grid" DataGrid
case unknown
throw new ConfigurationException("Config option [" + deploymentKey +
throw new ConfigurationException("Deployment config option [" + path +
".cluster.replication.storage] needs to be either [\"transaction-log\"] or [\"data-grid\"] - was [" +
unknown + "]")
}
@ -230,7 +226,7 @@ class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream,
case "write-through" WriteThrough
case "write-behind" WriteBehind
case unknown
throw new ConfigurationException("Config option [" + deploymentKey +
throw new ConfigurationException("Deployment config option [" + path +
".cluster.replication.strategy] needs to be either [\"write-through\"] or [\"write-behind\"] - was [" +
unknown + "]")
}
@ -269,8 +265,6 @@ class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream,
/**
* Simple local deployer, only for internal use.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class LocalDeployer extends ActorDeployer {
private val deployments = new ConcurrentHashMap[String, Deploy]

View file

@ -215,8 +215,6 @@ object DeploymentConfig {
* Module holding the programmatic deployment configuration classes.
* Defines the deployment specification.
* Most values have defaults and can be left out.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class DeploymentConfig(val nodename: String) {

View file

@ -564,7 +564,6 @@ trait FSM[S, D] extends ListenerManagement {
/**
* Stackable trait for FSM which adds a rolling event log.
*
* @author Roland Kuhn
* @since 1.2
*/
trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor

View file

@ -48,8 +48,6 @@ import akka.dispatch.{ MessageDispatcher, Promise }
* }
* }
* </pre>
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
abstract class UntypedActor extends Actor {
@ -75,30 +73,36 @@ abstract class UntypedActor extends Actor {
/**
* User overridable callback.
* <p/>
* Is called when an Actor is started, this only happens at most once in the life of an actor.
* Is called when an Actor is started.
* Actor are automatically started asynchronously when created.
* Empty default implementation.
*/
override def preStart() {}
/**
* User overridable callback.
* <p/>
* Is called when 'actor.stop()' is invoked.
* Is called asynchronously after 'actor.stop()' is invoked.
* Empty default implementation.
*/
override def postStop() {}
/**
* User overridable callback.
* <p/>
* Is called on a crashed Actor right BEFORE it is restarted to allow clean up of resources before Actor is terminated.
* Is called on a crashed Actor right BEFORE it is restarted to allow clean
* up of resources before Actor is terminated.
* By default it calls postStop()
*/
override def preRestart(reason: Throwable, lastMessage: Option[Any]) {}
override def preRestart(reason: Throwable, message: Option[Any]) { postStop() }
/**
* User overridable callback.
* <p/>
* Is called right AFTER restart on the newly created Actor to allow reinitialization after an Actor crash.
* By default it calls preStart()
*/
override def postRestart(reason: Throwable) {}
override def postRestart(reason: Throwable) { preStart() }
/**
* User overridable callback.
@ -117,7 +121,5 @@ abstract class UntypedActor extends Actor {
/**
* Factory closure for an UntypedActor, to be used with 'Actors.actorOf(factory)'.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait UntypedActorFactory extends Creator[Actor]

View file

@ -84,8 +84,6 @@ object ChangeListener {
/**
* Node address holds the node name and the cluster name and can be used as a hash lookup key for a Node instance.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class NodeAddress(val clusterName: String, val nodeName: String) {
if ((clusterName eq null) || clusterName == "") throw new NullPointerException("Cluster name must not be null or empty string")
@ -183,8 +181,6 @@ trait NodeMetricsManager {
/**
* Interface for cluster node.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait ClusterNode {
import ChangeListener._

View file

@ -17,9 +17,6 @@ import akka.event.EventStream
import akka.actor.ActorSystem.Settings
import com.typesafe.config.Config
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
final case class Envelope(val message: Any, val sender: ActorRef) {
if (message.isInstanceOf[AnyRef] && (message.asInstanceOf[AnyRef] eq null)) throw new InvalidMessageException("Message is null")
}
@ -86,9 +83,6 @@ object MessageDispatcher {
implicit def defaultDispatcher(implicit system: ActorSystem) = system.dispatcher
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) extends AbstractMessageDispatcher with Serializable {
import MessageDispatcher._
@ -137,7 +131,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
shutdownScheduleUpdater.get(this) match {
case UNSCHEDULED
if (shutdownScheduleUpdater.compareAndSet(this, UNSCHEDULED, SCHEDULED)) {
scheduler.scheduleOnce(shutdownTimeout, shutdownAction)
scheduleShutdownAction()
()
} else ifSensibleToDoSoThenScheduleShutdown()
case SCHEDULED
@ -148,6 +142,13 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
case _ ()
}
private def scheduleShutdownAction(): Unit = {
// IllegalStateException is thrown if scheduler has been shutdown
try scheduler.scheduleOnce(shutdownTimeout, shutdownAction) catch {
case _: IllegalStateException shutdown()
}
}
private final val taskCleanup: () Unit =
() if (inhabitantsUpdater.decrementAndGet(this) == 0) ifSensibleToDoSoThenScheduleShutdown()
@ -185,9 +186,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
}
case RESCHEDULED
if (shutdownScheduleUpdater.compareAndSet(MessageDispatcher.this, RESCHEDULED, SCHEDULED))
try scheduler.scheduleOnce(shutdownTimeout, this) catch {
case _: IllegalStateException shutdown()
}
scheduleShutdownAction()
else run()
}
}
@ -262,7 +261,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
}
/**
* Trait to be used for hooking in new dispatchers into Dispatchers.fromConfig
* Trait to be used for hooking in new dispatchers into Dispatchers.from(cfg: Config)
*/
abstract class MessageDispatcherConfigurator() {
/**
@ -279,9 +278,10 @@ abstract class MessageDispatcherConfigurator() {
}
}
def configureThreadPool(config: Config,
settings: Settings,
createDispatcher: (ThreadPoolConfig) MessageDispatcher): ThreadPoolConfigDispatcherBuilder = {
def configureThreadPool(
config: Config,
settings: Settings,
createDispatcher: (ThreadPoolConfig) MessageDispatcher): ThreadPoolConfigDispatcherBuilder = {
import ThreadPoolConfigDispatcherBuilder.conf_?
//Apply the following options to the config if they are present in the config

View file

@ -28,8 +28,6 @@ import akka.util.Duration
*
* @see akka.dispatch.BalancingDispatcher
* @see akka.dispatch.Dispatchers
*
* @author Viktor Klang
*/
class BalancingDispatcher(
_prerequisites: DispatcherPrerequisites,

View file

@ -55,7 +55,6 @@ import java.util.concurrent._
* But the preferred way of creating dispatchers is to use
* the {@link akka.dispatch.Dispatchers} factory object.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
* @param throughput positive integer indicates the dispatcher will only process so much messages at a time from the
* mailbox, without checking the mailboxes of other actors. Zero or negative means the dispatcher
* always continues until the mailbox is empty.
@ -153,4 +152,4 @@ abstract class PriorityGenerator extends java.util.Comparator[Envelope] {
final def compare(thisMessage: Envelope, thatMessage: Envelope): Int =
gen(thisMessage.message) - gen(thatMessage.message)
}
}

View file

@ -53,8 +53,6 @@ case class DefaultDispatcherPrerequisites(
* .build();
* </pre>
* <p/>
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: DispatcherPrerequisites) {
@ -173,12 +171,11 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
new BalancingDispatcher(prerequisites, name, throughput, throughputDeadline, mailboxType,
config, settings.DispatcherDefaultShutdown), ThreadPoolConfig())
/**
* Utility function that tries to load the specified dispatcher config from the akka.conf
* Creates a new dispatcher as specified in configuration
* or if not defined it uses the supplied dispatcher.
* Uses default values from default-dispatcher, i.e. all options doesn't need to be defined
* in config.
* Uses default values from default-dispatcher, i.e. all options doesn't need to be defined.
*/
def fromConfig(key: String, default: MessageDispatcher = defaultGlobalDispatcher, cfg: Config = settings.config): MessageDispatcher = {
def newFromConfig(key: String, default: MessageDispatcher, cfg: Config): MessageDispatcher = {
import scala.collection.JavaConverters._
def simpleName = key.substring(key.lastIndexOf('.') + 1)
cfg.hasPath(key) match {
@ -190,9 +187,17 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
}
}
/**
* Creates a new dispatcher as specified in configuration, or if not defined it uses
* the default dispatcher.
* Uses default configuration values from default-dispatcher, i.e. all options doesn't
* need to be defined.
*/
def newFromConfig(key: String): MessageDispatcher = newFromConfig(key, defaultGlobalDispatcher, settings.config)
/*
* Creates of obtains a dispatcher from a ConfigMap according to the format below.
* Uses default values from default-dispatcher.
* Uses default values from default-dispatcher.
*
* my-dispatcher {
* type = "Dispatcher" # Must be one of the following

View file

@ -33,9 +33,6 @@ object Mailbox {
final val debug = false
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
abstract class Mailbox(val actor: ActorCell) extends MessageQueue with SystemMessageQueue with Runnable {
import Mailbox._

View file

@ -14,8 +14,6 @@ import java.util.concurrent.TimeUnit
/**
* Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class PinnedDispatcher(
_prerequisites: DispatcherPrerequisites,

View file

@ -144,9 +144,6 @@ case class ThreadPoolConfigDispatcherBuilder(dispatcherFactory: (ThreadPoolConfi
def configure(fs: Option[Function[ThreadPoolConfigDispatcherBuilder, ThreadPoolConfigDispatcherBuilder]]*): ThreadPoolConfigDispatcherBuilder = fs.foldLeft(this)((c, f) f.map(_(c)).getOrElse(c))
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class MonitorableThreadFactory(val name: String, val daemonic: Boolean = false) extends ThreadFactory {
protected val counter = new AtomicLong
@ -157,9 +154,6 @@ class MonitorableThreadFactory(val name: String, val daemonic: Boolean = false)
}
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object MonitorableThread {
val DEFAULT_NAME = "MonitorableThread".intern
@ -168,9 +162,6 @@ object MonitorableThread {
val alive = new AtomicInteger
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class MonitorableThread(runnable: Runnable, name: String)
extends Thread(runnable, name + "-" + MonitorableThread.created.incrementAndGet) {

View file

@ -10,7 +10,6 @@ import annotation.target._
* This annotation marks a feature which is not yet considered stable and may
* change or be removed in a future release.
*
* @author Roland Kuhn
* @since 1.2
*/
@getter

View file

@ -26,8 +26,6 @@ trait VersionedIterable[A] {
/**
* Manages connections (ActorRefs) for a router.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait ConnectionManager {
/**

View file

@ -15,8 +15,6 @@ import scala.collection.mutable.{ Buffer, Map }
* Consistent Hashing node ring abstraction.
*
* Not thread-safe, to be used from within an Actor or protected some other way.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class ConsistentHash[T](nodes: Seq[T], replicas: Int) {
private val cluster = Buffer[T]()

View file

@ -19,8 +19,6 @@ sealed trait RouterType
/**
* Used for declarative configuration of Routing.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object RouterType {
@ -92,8 +90,6 @@ object RoutedProps {
/**
* The Router is responsible for sending a message to one (or more) of its connections. Connections are stored in the
* {@link FailureDetector} and each Router should be linked to only one {@link FailureDetector}.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait Router {
@ -296,8 +292,6 @@ class BroadcastRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeo
/**
* A DirectRouter a Router that only has a single connected actorRef and forwards all request to that actorRef.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class DirectRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter {
@ -338,8 +332,6 @@ class DirectRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout)
/**
* A Router that randomly selects one of the target connections to send a message to.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RandomRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter {
import java.security.SecureRandom
@ -380,8 +372,6 @@ class RandomRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout)
/**
* A Router that uses round-robin to select a connection. For concurrent calls, round robin is just a best effort.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RoundRobinRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter {

View file

@ -7,7 +7,6 @@ package akka.serialization
import akka.actor.Actor
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
* trait Serializer extends scala.Serializable {
* @volatile
* var classLoader: Option[ClassLoader] = None

View file

@ -4,9 +4,6 @@
package akka.util
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object Convert {
def intToBytes(value: Int): Array[Byte] = {

View file

@ -6,9 +6,6 @@ package akka.util
import java.security.{ MessageDigest, SecureRandom }
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object Crypt {
val hex = "0123456789ABCDEF"
val lineSeparator = System.getProperty("line.separator")

View file

@ -38,7 +38,19 @@ case class Timer(duration: Duration, throwExceptionOnTimeout: Boolean = false) {
}
}
case class Deadline(d: Duration) {
def +(other: Duration): Deadline = copy(d = d + other)
def -(other: Duration): Deadline = copy(d = d - other)
def -(other: Deadline): Duration = d - other.d
def timeLeft: Duration = this - Deadline.now
}
object Deadline {
def now: Deadline = Deadline(Duration(System.nanoTime, NANOSECONDS))
}
object Duration {
implicit def timeLeft(implicit d: Deadline): Duration = d.timeLeft
def apply(length: Long, unit: TimeUnit): Duration = new FiniteDuration(length, unit)
def apply(length: Double, unit: TimeUnit): Duration = fromNanos(unit.toNanos(1) * length)
def apply(length: Long, unit: String): Duration = new FiniteDuration(length, timeUnit(unit))
@ -129,10 +141,7 @@ object Duration {
override def *(factor: Double): Duration = throw new IllegalArgumentException("cannot multiply Undefined duration")
override def /(factor: Double): Duration = throw new IllegalArgumentException("cannot divide Undefined duration")
override def /(other: Duration): Double = throw new IllegalArgumentException("cannot divide Undefined duration")
def >(other: Duration) = throw new IllegalArgumentException("cannot compare Undefined duration")
def >=(other: Duration) = throw new IllegalArgumentException("cannot compare Undefined duration")
def <(other: Duration) = throw new IllegalArgumentException("cannot compare Undefined duration")
def <=(other: Duration) = throw new IllegalArgumentException("cannot compare Undefined duration")
def compare(other: Duration) = throw new IllegalArgumentException("cannot compare Undefined duration")
def unary_- : Duration = throw new IllegalArgumentException("cannot negate Undefined duration")
}
@ -183,10 +192,7 @@ object Duration {
*/
val Inf: Duration = new Duration with Infinite {
override def toString = "Duration.Inf"
def >(other: Duration) = true
def >=(other: Duration) = true
def <(other: Duration) = false
def <=(other: Duration) = false
def compare(other: Duration) = 1
def unary_- : Duration = MinusInf
}
@ -196,10 +202,7 @@ object Duration {
*/
val MinusInf: Duration = new Duration with Infinite {
override def toString = "Duration.MinusInf"
def >(other: Duration) = false
def >=(other: Duration) = false
def <(other: Duration) = true
def <=(other: Duration) = true
def compare(other: Duration) = -1
def unary_- : Duration = Inf
}
@ -255,7 +258,7 @@ object Duration {
* val d3 = d2 + 1.millisecond
* </pre>
*/
abstract class Duration extends Serializable {
abstract class Duration extends Serializable with Ordered[Duration] {
def length: Long
def unit: TimeUnit
def toNanos: Long
@ -267,10 +270,6 @@ abstract class Duration extends Serializable {
def toDays: Long
def toUnit(unit: TimeUnit): Double
def printHMS: String
def <(other: Duration): Boolean
def <=(other: Duration): Boolean
def >(other: Duration): Boolean
def >=(other: Duration): Boolean
def +(other: Duration): Duration
def -(other: Duration): Duration
def *(factor: Double): Duration
@ -281,6 +280,7 @@ abstract class Duration extends Serializable {
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)
def fromNow: Deadline = Deadline.now + this
// Java API
def lt(other: Duration) = this < other
@ -329,37 +329,12 @@ class FiniteDuration(val length: Long, val unit: TimeUnit) extends Duration {
def printHMS = "%02d:%02d:%06.3f".format(toHours, toMinutes % 60, toMillis / 1000. % 60)
def <(other: Duration) = {
def compare(other: Duration) =
if (other.finite_?) {
toNanos < other.asInstanceOf[FiniteDuration].toNanos
} else {
other > this
}
}
def <=(other: Duration) = {
if (other.finite_?) {
toNanos <= other.asInstanceOf[FiniteDuration].toNanos
} else {
other >= this
}
}
def >(other: Duration) = {
if (other.finite_?) {
toNanos > other.asInstanceOf[FiniteDuration].toNanos
} else {
other < this
}
}
def >=(other: Duration) = {
if (other.finite_?) {
toNanos >= other.asInstanceOf[FiniteDuration].toNanos
} else {
other <= this
}
}
val me = toNanos
val o = other.toNanos
if (me > o) 1 else if (me < o) -1 else 0
} else -other.compare(this)
def +(other: Duration) = {
if (!other.finite_?) {
@ -397,6 +372,8 @@ class FiniteDuration(val length: Long, val unit: TimeUnit) extends Duration {
}
class DurationInt(n: Int) {
import duration.Classifier
def nanoseconds = Duration(n, NANOSECONDS)
def nanos = Duration(n, NANOSECONDS)
def nanosecond = Duration(n, NANOSECONDS)
@ -423,9 +400,38 @@ class DurationInt(n: Int) {
def days = Duration(n, DAYS)
def day = Duration(n, DAYS)
def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS))
def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS))
def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES))
def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES))
def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS))
def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS))
def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS))
def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS))
}
class DurationLong(n: Long) {
import duration.Classifier
def nanoseconds = Duration(n, NANOSECONDS)
def nanos = Duration(n, NANOSECONDS)
def nanosecond = Duration(n, NANOSECONDS)
@ -452,9 +458,38 @@ class DurationLong(n: Long) {
def days = Duration(n, DAYS)
def day = Duration(n, DAYS)
def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS))
def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS))
def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES))
def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES))
def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS))
def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS))
def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS))
def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS))
}
class DurationDouble(d: Double) {
import duration.Classifier
def nanoseconds = Duration(d, NANOSECONDS)
def nanos = Duration(d, NANOSECONDS)
def nanosecond = Duration(d, NANOSECONDS)
@ -481,5 +516,32 @@ class DurationDouble(d: Double) {
def days = Duration(d, DAYS)
def day = Duration(d, DAYS)
def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS))
def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS))
def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS))
def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS))
def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS))
def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS))
def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS))
def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS))
def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS))
def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS))
def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS))
def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS))
def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, SECONDS))
def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, SECONDS))
def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MINUTES))
def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MINUTES))
def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, HOURS))
def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, HOURS))
def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, DAYS))
def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, DAYS))
}

View file

@ -21,8 +21,6 @@ import java.lang.{ Float ⇒ JFloat, Double ⇒ JDouble }
* result
* }
* </pre>
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object HashCode {
val SEED = 23

View file

@ -8,17 +8,14 @@ import java.util.Comparator
import scala.annotation.tailrec
import java.util.regex.Pattern
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object Helpers {
def makePattern(s: String): Pattern = Pattern.compile("^\\Q" + s.replace("?", "\\E.\\Q").replace("*", "\\E.*\\Q") + "\\E$")
def compareIdentityHash(a: AnyRef, b: AnyRef): Int = {
/*
* make sure that there is no overflow or underflow in comparisons, so
* that the ordering is actually consistent and you cannot have a
* 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))

View file

@ -13,8 +13,6 @@ import scala.collection.mutable
* An implementation of a ConcurrentMultiMap
* Adds/remove is serialized over the specified key
* Reads are fully concurrent <-- el-cheapo
*
* @author Viktor Klang
*/
class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) {
@ -192,7 +190,5 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) {
* An implementation of a ConcurrentMultiMap
* Adds/remove is serialized over the specified key
* Reads are fully concurrent <-- el-cheapo
*
* @author Viktor Klang
*/
class ConcurrentMultiMap[K, V](mapSize: Int, valueComparator: Comparator[V]) extends Index[K, V](mapSize, valueComparator)

View file

@ -9,9 +9,6 @@ import java.lang.management.ManagementFactory
import javax.management.{ ObjectInstance, ObjectName, InstanceAlreadyExistsException, InstanceNotFoundException }
import akka.actor.ActorSystem
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object JMX {
private val mbeanServer = ManagementFactory.getPlatformMBeanServer

View file

@ -9,8 +9,6 @@ import akka.actor.{ ActorInitializationException, ActorRef }
/**
* A manager for listener actors. Intended for mixin by observables.
*
* @author Martin Krasser
*/
trait ListenerManagement {

View file

@ -7,9 +7,6 @@ package akka.util
import java.util.concurrent.locks.{ ReentrantLock }
import java.util.concurrent.atomic.{ AtomicBoolean }
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
final class ReentrantGuard {
final val lock = new ReentrantLock

View file

@ -7,6 +7,23 @@ package akka.util
import java.util.concurrent.TimeUnit
package object duration {
trait Classifier[C] {
type R
def convert(d: Duration): R
}
object span
implicit object spanConvert extends Classifier[span.type] {
type R = Duration
def convert(d: Duration) = d
}
object fromNow
implicit object fromNowConvert extends Classifier[fromNow.type] {
type R = Deadline
def convert(d: Duration) = Deadline.now + d
}
implicit def intToDurationInt(n: Int) = new DurationInt(n)
implicit def longToDurationLong(n: Long) = new DurationLong(n)
implicit def doubleToDurationDouble(d: Double) = new DurationDouble(d)

View file

@ -21,9 +21,6 @@ entry number it will use MAX_INTEGER). Once all the entries have been processed,
new one for its use.
*/
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object BookKeeperServer {
val port = 3181
val zkServers = "localhost:2181"

View file

@ -57,8 +57,6 @@ import akka.dispatch.{Await, Dispatchers, Future, PinnedDispatcher}
/**
* JMX MBean for the cluster service.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait ClusterNodeMBean {
@ -140,8 +138,6 @@ trait ClusterNodeMBean {
/**
* Module for the Cluster. Also holds global state such as configuration data etc.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object Cluster {
val EMPTY_STRING = "".intern
@ -257,8 +253,6 @@ object Cluster {
*
* /clusterName/'actor-address-to-uuids'/actorAddress/actorUuid
* </pre>
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class DefaultClusterNode private[akka] (
val nodeAddress: NodeAddress,
@ -1596,9 +1590,6 @@ class DefaultClusterNode private[akka] (
}
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class MembershipChildListener(self: ClusterNode) extends IZkChildListener with ErrorHandler {
def handleChildChange(parentPath: String, currentChilds: JList[String]) {
withErrorHandler {
@ -1638,9 +1629,6 @@ class MembershipChildListener(self: ClusterNode) extends IZkChildListener with E
}
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class StateListener(self: ClusterNode) extends IZkStateListener {
def handleStateChanged(state: KeeperState) {
state match {
@ -1666,9 +1654,6 @@ class StateListener(self: ClusterNode) extends IZkStateListener {
}
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait ErrorHandler {
def withErrorHandler[T](body: T) = {
try {
@ -1681,9 +1666,6 @@ trait ErrorHandler {
}
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object RemoteClusterDaemon {
val Address = "akka-cluster-daemon".intern
@ -1695,8 +1677,6 @@ object RemoteClusterDaemon {
* Internal "daemon" actor for cluster internal communication.
*
* It acts as the brain of the cluster that responds to cluster events (messages) and undertakes action.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {

View file

@ -21,8 +21,6 @@ import annotation.tailrec
/**
* ClusterActorRef factory and locator.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object ClusterActorRef {
import FailureDetectorType._
@ -77,8 +75,6 @@ object ClusterActorRef {
/**
* ActorRef representing a one or many instances of a clustered, load-balanced and sometimes replicated actor
* where the instances can reside on other nodes in the cluster.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
private[akka] class ClusterActorRef(props: RoutedProps, val address: String) extends AbstractRoutedActorRef(props) {

View file

@ -24,10 +24,6 @@ import java.util.concurrent.{ CountDownLatch, TimeUnit }
/**
* A ClusterDeployer is responsible for deploying a Deploy.
*
* FIXME Document: what does Deploy mean?
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object ClusterDeployer extends ActorDeployer {
val clusterName = Cluster.name

View file

@ -26,19 +26,12 @@ import java.util.Enumeration
// FIXME allow user to choose dynamically between 'async' and 'sync' tx logging (asyncAddEntry(byte[] data, AddCallback cb, Object ctx))
// FIXME clean up old entries in log after doing a snapshot
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class ReplicationException(message: String, cause: Throwable = null) extends AkkaException(message) {
def this(msg: String) = this(msg, null)
}
/**
* TODO: Explain something about threadsafety.
*
* A TransactionLog makes chunks of data durable.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class TransactionLog private (
ledger: LedgerHandle,
@ -352,7 +345,7 @@ class TransactionLog private (
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
* TODO: Documentation.
*/
object TransactionLog {
@ -563,8 +556,6 @@ object TransactionLog {
/**
* TODO: Documentation.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object LocalBookKeeperEnsemble {
private val isRunning = new Switch(false)

View file

@ -7,9 +7,6 @@ package akka.cluster.sample
import akka.cluster._
import akka.dispatch.Futures
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object ComputeGridSample {
//sample.cluster.ComputeGridSample.fun2

View file

@ -6,6 +6,7 @@
{% extends "basic/layout.html" %}
{% set script_files = script_files + ['_static/theme_extras.js'] %}
{% set css_files = css_files + ['_static/print.css'] %}
{% set is_snapshot = version.endswith("-SNAPSHOT") %}
{# do not display relbars #}
{% block relbar1 %}{% endblock %}
@ -37,7 +38,11 @@
{%- endif -%}
<h1 class="heading"><a href="{{ pathto('index') }}"><span>{{ shorttitle|e }}</span></a></h1>
<h2 class="heading"><a href="{{ pathto('index') }}"><span>Version {{ version|e }}</span></a></h2>
{%- if is_snapshot -%}
<h2 class="rightheading"><span><a href="http://akka.io/docs/akka/snapshot/Akka.pdf">PDF</a></span></h2>
{%- else -%}
<h2 class="rightheading"><span><a href="http://akka.io/docs/akka/{{ version|e }}/Akka.pdf">PDF</a></span></h2>
{%- endif -%}
{%- endblock %}
</div>
<div class="topnav">

View file

@ -11,7 +11,6 @@ Simple Trading system.
Compares:
- Synchronous Scala solution
- Scala library Actors
- Fire-forget

View file

@ -56,22 +56,6 @@ Sample parallel computing with Akka and Scala API
`<https://github.com/yannart/ParallelPolynomialIntegral>`_
Akka, Facebook Graph API, WebGL sample
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Showcasing Akka Mist HTTP module
`<https://github.com/buka/fbgl1>`_
Akka Mist Sample
^^^^^^^^^^^^^^^^
`<https://github.com/buka/akka-mist-sample>`_
Another Akka Mist Sample
^^^^^^^^^^^^^^^^^^^^^^^^
`<https://github.com/nppssk/akka-http-sbt>`_
Bank application
^^^^^^^^^^^^^^^^

View file

@ -1,282 +0,0 @@
.. _durable-mailboxes:
###################
Durable Mailboxes
###################
Overview
========
Akka supports a set of durable mailboxes. A durable mailbox is a replacement for
the standard actor mailbox that is durable. What this means in practice is that
if there are pending messages in the actor's mailbox when the node of the actor
resides on crashes, then when you restart the node, the actor will be able to
continue processing as if nothing had happened; with all pending messages still
in its mailbox.
.. sidebar:: **IMPORTANT**
None of these mailboxes work with blocking message send, e.g. the message
send operations that are relying on futures; ``?`` or ``ask``. If the node
has crashed and then restarted, the thread that was blocked waiting for the
reply is gone and there is no way we can deliver the message.
The durable mailboxes currently supported are:
- ``FileDurableMailboxStorage`` -- backed by a journaling transaction log on the local file system
- ``RedisDurableMailboxStorage`` -- backed by Redis
- ``ZooKeeperDurableMailboxStorage`` -- backed by ZooKeeper
- ``BeanstalkDurableMailboxStorage`` -- backed by Beanstalkd
- ``MongoNaiveDurableMailboxStorage`` -- backed by MongoDB
We'll walk through each one of these in detail in the sections below.
Soon Akka will also have:
- ``AmqpDurableMailboxStorage`` -- AMQP based mailbox (default RabbitMQ)
- ``JmsDurableMailboxStorage`` -- JMS based mailbox (default ActiveMQ)
File-based durable mailbox
==========================
This mailbox is backed by a journaling transaction log on the local file
system. It is the simplest want to use since it does not require an extra
infrastructure piece to administer, but it is usually sufficient and just what
you need.
The durable dispatchers and their configuration options reside in the
``akka.actor.mailbox`` package.
You configure durable mailboxes through the "Akka"-only durable dispatchers, the
actor is oblivious to which type of mailbox it is using. Here is an example::
val dispatcher = DurableDispatcher(
"my:service",
FileDurableMailboxStorage)
// Then set the actors dispatcher to this dispatcher
or for a thread-based durable dispatcher::
self.dispatcher = DurablePinnedDispatcher(
self,
FileDurableMailboxStorage)
There are 2 different durable dispatchers, ``DurableDispatcher`` and
``DurablePinnedDispatcher``, which are durable versions of
``Dispatcher`` and ``PinnedDispatcher``.
This gives you an excellent way of creating bulkheads in your application, where
groups of actors sharing the same dispatcher also share the same backing
storage.
Read more about that in the :ref:`dispatchers-scala` documentation.
You can also configure and tune the file-based durable mailbox. This is done in
the ``akka.actor.mailbox.file-based`` section in the ``akka.conf`` configuration
file.
.. code-block:: none
akka {
actor {
mailbox {
file-based {
directory-path = "./_mb"
max-items = 2147483647
max-size = 2147483647
max-items = 2147483647
max-age = 0
max-journal-size = 16777216 # 16 * 1024 * 1024
max-memory-size = 134217728 # 128 * 1024 * 1024
max-journal-overflow = 10
max-journal-size-absolute = 9223372036854775807
discard-old-when-full = on
keep-journal = on
sync-journal = off
}
}
}
}
.. todo:: explain all the above options in detail
Redis-based durable mailbox
===========================
This mailbox is backed by a Redis queue. `Redis <http://redis.io>`_ Is a very
fast NOSQL database that has a wide range of data structure abstractions, one of
them is a queue which is what we are using in this implementation. This means
that you have to start up a Redis server that can host these durable
mailboxes. Read more in the Redis documentation on how to do that.
Here is an example of how you can configure your dispatcher to use this mailbox::
val dispatcher = DurableDispatcher(
"my:service",
RedisDurableMailboxStorage)
or for a thread-based durable dispatcher::
self.dispatcher = DurablePinnedDispatcher(
self,
RedisDurableMailboxStorage)
You also need to configure the IP and port for the Redis server. This is done in
the ``akka.actor.mailbox.redis`` section in the ``akka.conf`` configuration
file.
.. code-block:: none
akka {
actor {
mailbox {
redis {
hostname = "127.0.0.1"
port = 6379
}
}
}
}
ZooKeeper-based durable mailbox
===============================
This mailbox is backed by `ZooKeeper <http://zookeeper.apache.org/>`_. ZooKeeper
is a centralized service for maintaining configuration information, naming,
providing distributed synchronization, and providing group services This means
that you have to start up a ZooKeeper server (for production a ZooKeeper server
ensamble) that can host these durable mailboxes. Read more in the ZooKeeper
documentation on how to do that.
Akka is using ZooKeeper for many other things, for example the clustering
support so if you're using that you love to run a ZooKeeper server anyway and
there will not be that much more work to set up this durable mailbox.
Here is an example of how you can configure your dispatcher to use this mailbox::
val dispatcher = DurableDispatcher(
"my:service",
ZooKeeperDurableMailboxStorage)
or for a thread-based durable dispatcher::
self.dispatcher = DurablePinnedDispatcher(
self,
ZooKeeperDurableMailboxStorage)
You also need to configure ZooKeeper server addresses, timeouts, etc. This is
done in the ``akka.actor.mailbox.zookeeper`` section in the ``akka.conf``
configuration file.
.. code-block:: none
akka {
actor {
mailbox {
zookeeper {
server-addresses = "localhost:2181"
session-timeout = 60
connection-timeout = 30
blocking-queue = on
}
}
}
}
Beanstalk-based durable mailbox
===============================
This mailbox is backed by `Beanstalkd <http://kr.github.com/beanstalkd/>`_.
Beanstalk is a simple, fast work queue. This means that you have to start up a
Beanstalk server that can host these durable mailboxes. Read more in the
Beanstalk documentation on how to do that. ::
val dispatcher = DurableDispatcher(
"my:service",
BeanstalkDurableMailboxStorage)
or for a thread-based durable dispatcher. ::
self.dispatcher = DurablePinnedDispatcher(
self,
BeanstalkDurableMailboxStorage)
You also need to configure the IP, and port, and so on, for the Beanstalk
server. This is done in the ``akka.actor.mailbox.beanstalk`` section in the
``akka.conf`` configuration file.
.. code-block:: none
akka {
actor {
mailbox {
beanstalk {
hostname = "127.0.0.1"
port = 11300
reconnect-window = 5
message-submit-delay = 0
message-submit-timeout = 5
message-time-to-live = 120
}
}
}
}
MongoDB-based Durable Mailboxes
===============================
This mailbox is backed by `MongoDB <http://mongodb.org>`_.
MongoDB is a fast, lightweight and scalable document-oriented database. It contains a number of
features cohesive to a fast, reliable & durable queueing mechanism which the Akka Mailbox takes advantage of.
Akka's implementations of MongoDB mailboxes are built on top of the purely asynchronous MongoDB driver (often known as `Hammersmith <http://github.com/bwmcadams/hammersmith>`_ and ``com.mongodb.async``) and as such are purely callback based with a Netty network layer. This makes them extremely fast & lightweight versus building on other MongoDB implementations such as `mongo-java-driver <http://github.com/mongodb/mongo-java-driver>`_ and `Casbah <http://github.com/mongodb/casbah>`_.
You will need to configure the URI for the MongoDB server, using the URI Format specified in the `MongoDB Documentation <http://www.mongodb.org/display/DOCS/Connections>`_. This is done in
the ``akka.actor.mailbox.mongodb`` section in the ``akka.conf`` configuration
file.
.. code-block:: none
mongodb {
# Any specified collection name will be used as a prefix for collections that use durable mongo mailboxes
uri = "mongodb://localhost/akka.mailbox" # Follow Mongo URI Spec - http://www.mongodb.org/display/DOCS/Connections
# Configurable timeouts for certain ops
timeout {
read = 3000 # number of milliseconds to wait for a read to succeed before timing out the future
write = 3000 # number of milliseconds to wait for a write to succeed before timing out the future
}
}
You must specify a hostname (and optionally port) and at *least* a Database name. If you specify a collection name, it will be used as a 'prefix' for the collections Akka creates to store mailbox messages. Otherwise, collections will be prefixed with ``mailbox.``
It is also possible to configure the timeout threshholds for Read and Write operations in the ``timeout`` block.
Currently Akka offers only one "type" of MongoDB based Mailbox but there are plans to support at least
one other kind which uses a different queueing strategy.
'Naive' MongoDB-based Durable Mailbox
-------------------------------------
The currently supported mailbox is considered "Naive" as it removes messages (using the ``findAndRemove``
command) from the MongoDB datastore as soon as the actor consumes them. This could cause message loss
if an actor crashes before completely processing a message. It is not a problem per sé, but behavior
users should be aware of.
Here is an example of how you can configure your dispatcher to use this mailbox::
val dispatcher = DurableDispatcher(
"my:service",
MongoNaiveDurableMailboxStorage)
or for a thread-based durable dispatcher::
self.dispatcher = DurablePinnedDispatcher(
self,
MongoNaiveDurableMailboxStorage)

View file

@ -5,4 +5,3 @@ Cluster
:maxdepth: 2
cluster
durable-mailbox

View file

@ -48,4 +48,18 @@ method calls instead:
assert (diff.lt(fivesec));
assert (Duration.Zero().lt(Duration.Inf()));
Deadline
========
Durations have a brother name :class:`Deadline`, which is a class holding a representation
of an absolute point in time, and support deriving a duration from this by calculating the
difference between now and the deadline. This is useful when you want to keep one overall
deadline without having to take care of the book-keeping wrt. the passing of time yourself::
val deadline = 10 seconds fromNow
// do something which takes time
awaitCond(..., deadline.timeLeft)
In Java you create these from durations::
final Deadline d = Duration.create(5, "seconds").fromNow();

View file

@ -35,7 +35,7 @@ multi-JVM testing::
base = file("akka-cluster"),
settings = defaultSettings ++ MultiJvmPlugin.settings ++ Seq(
extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src =>
(name: String) => (src ** (name + ".conf")).get.headOption.map("-Dakka.config=" + _.absolutePath).toSeq
(name: String) => (src ** (name + ".conf")).get.headOption.map("-Dconfig.file=" + _.absolutePath).toSeq
},
test in Test <<= (test in Test) dependsOn (test in MultiJvm)
)
@ -176,10 +176,10 @@ and add the options to them.
-Dakka.cluster.nodename=node3 -Dakka.remote.port=9993
Overriding akka.conf options
----------------------------
Overriding configuration options
--------------------------------
You can also override the options in the ``akka.conf`` file with different options for each
You can also override the options in the :ref:`configuration` file with different options for each
spawned JVM. You do that by creating a file named after the node in the test with suffix
``.conf`` and put them in the same directory as the test .

View file

@ -26,4 +26,5 @@ Scott Clasen Committer
Roland Kuhn Committer
Patrik Nordwall Committer patrik DOT nordwall AT gmail DOT com
Derek Williams Committer derek AT nebvin DOT ca
Henrik Engström Committer
=================== ========================== ====================================

2903
akka-docs/disabled/camel.rst Normal file

File diff suppressed because it is too large Load diff

View file

@ -48,8 +48,8 @@ cluster node.
Cluster configuration
~~~~~~~~~~~~~~~~~~~~~
Cluster is configured in the ``akka.cloud.cluster`` section in the ``akka.conf``
configuration file. Here you specify the default addresses to the ZooKeeper
Cluster is configured in the ``akka.cloud.cluster`` section in the :ref:`configuration`.
Here you specify the default addresses to the ZooKeeper
servers, timeouts, if compression should be on or off, and so on.
.. code-block:: conf
@ -594,7 +594,7 @@ Consolidation and management of the Akka configuration file
Not implemented yet.
The actor configuration file ``akka.conf`` will also be stored into the cluster
The actor :ref:`configuration` file will also be stored into the cluster
and it will be possible to have one single configuration file, stored on the server, and pushed out to all
the nodes that joins the cluster. Each node only needs to be configured with the ZooKeeper
server address and the master configuration will only reside in one single place

107
akka-docs/disabled/http.rst Normal file
View file

@ -0,0 +1,107 @@
.. _http-module:
HTTP
====
.. sidebar:: Contents
.. contents:: :local:
Module stability: **SOLID**
When deploying in a servlet container:
--------------------------------------------
If you deploy Akka in a JEE container, don't forget to create an Akka initialization and cleanup hook:
.. code-block:: scala
package com.my //<--- your own package
import akka.util.AkkaLoader
import akka.cluster.BootableRemoteActorService
import akka.actor.BootableActorLoaderService
import javax.servlet.{ServletContextListener, ServletContextEvent}
/**
* This class can be added to web.xml mappings as a listener to start and postStop Akka.
*<web-app>
* ...
* <listener>
* <listener-class>com.my.Initializer</listener-class>
* </listener>
* ...
*</web-app>
*/
class Initializer extends ServletContextListener {
lazy val loader = new AkkaLoader
def contextDestroyed(e: ServletContextEvent): Unit = loader.shutdown
def contextInitialized(e: ServletContextEvent): Unit =
loader.boot(true, new BootableActorLoaderService with BootableRemoteActorService) //<--- Important
// loader.boot(true, new BootableActorLoaderService {}) // If you don't need akka-remote
}
For Java users, it's currently only possible to use BootableActorLoaderService, but you'll need to use: akka.actor.DefaultBootableActorLoaderService
Then you just declare it in your web.xml:
.. code-block:: xml
<web-app>
...
<listener>
<listener-class>your.package.Initializer</listener-class>
</listener>
...
</web-app>
Adapting your own Akka Initializer for the Servlet Container
------------------------------------------------------------
If you want to use akka-camel or any other modules that have their own "Bootable"'s you'll need to write your own Initializer, which is _ultra_ simple, see below for an example on how to include Akka-camel.
.. code-block:: scala
package com.my //<--- your own package
import akka.cluster.BootableRemoteActorService
import akka.actor.BootableActorLoaderService
import akka.camel.CamelService
import javax.servlet.{ServletContextListener, ServletContextEvent}
/**
* This class can be added to web.xml mappings as a listener to start and postStop Akka.
*<web-app>
* ...
* <listener>
* <listener-class>com.my.Initializer</listener-class>
* </listener>
* ...
*</web-app>
*/
class Initializer extends ServletContextListener {
lazy val loader = new AkkaLoader
def contextDestroyed(e: ServletContextEvent): Unit = loader.shutdown
def contextInitialized(e: ServletContextEvent): Unit =
loader.boot(true, new BootableActorLoaderService with BootableRemoteActorService with CamelService) //<--- Important
}
Using Akka with the Pinky REST/MVC framework
--------------------------------------------
Pinky has a slick Akka integration. Read more `here <http://wiki.github.com/pk11/pinky/release-13>`_
jetty-run in SBT
----------------
If you want to use jetty-run in SBT you need to exclude the version of Jetty that is bundled in akka-http:
.. code-block:: scala
override def ivyXML =
<dependencies>
<dependency org="com.typesafe.akka" name="akka-http" rev="AKKA_VERSION_GOES_HERE">
<exclude module="jetty"/>
</dependency>
</dependencies>

View file

@ -0,0 +1,515 @@
.. _stm-java:
Software Transactional Memory (Java)
====================================
.. sidebar:: Contents
.. contents:: :local:
Module stability: **SOLID**
Overview of STM
---------------
An `STM <http://en.wikipedia.org/wiki/Software_transactional_memory>`_ turns the Java heap into a transactional data set with begin/commit/rollback semantics. Very much like a regular database. It implements the first three letters in ACID; ACI:
* (failure) Atomicity: all changes during the execution of a transaction make it, or none make it. This only counts for transactional datastructures.
* Consistency: a transaction gets a consistent of reality (in Akka you get the Oracle version of the SERIALIZED isolation level).
* Isolated: changes made by concurrent execution transactions are not visible to each other.
Generally, the STM is not needed that often when working with Akka. Some use-cases (that we can think of) are:
- When you really need composable message flows across many actors updating their **internal local** state but need them to do that atomically in one big transaction. Might not often, but when you do need this then you are screwed without it.
- When you want to share a datastructure across actors.
- When you need to use the persistence modules.
Akkas STM implements the concept in `Clojures <http://clojure.org/>`_ STM view on state in general. Please take the time to read `this excellent document <http://clojure.org/state>`_ and view `this presentation <http://www.infoq.com/presentations/Value-Identity-State-Rich-Hickey>`_ by Rich Hickey (the genius behind Clojure), since it forms the basis of Akkas view on STM and state in general.
The STM is based on Transactional References (referred to as Refs). Refs are memory cells, holding an (arbitrary) immutable value, that implement CAS (Compare-And-Swap) semantics and are managed and enforced by the STM for coordinated changes across many Refs. They are implemented using the excellent `Multiverse STM <http://multiverse.codehaus.org/overview.html>`_.
Working with immutable collections can sometimes give bad performance due to extensive copying. Scala provides so-called persistent datastructures which makes working with immutable collections fast. They are immutable but with constant time access and modification. The use of structural sharing and an insert or update does not ruin the old structure, hence “persistent”. Makes working with immutable composite types fast. The persistent datastructures currently consist of a Map and Vector.
Simple example
--------------
Here is a simple example of an incremental counter using STM. This shows creating a ``Ref``, a transactional reference, and then modifying it within a transaction, which is delimited by an ``Atomic`` anonymous inner class.
.. code-block:: java
import akka.stm.*;
final Ref<Integer> ref = new Ref<Integer>(0);
public int counter() {
return new Atomic<Integer>() {
public Integer atomically() {
int inc = ref.get() + 1;
ref.set(inc);
return inc;
}
}.execute();
}
counter();
// -> 1
counter();
// -> 2
Ref
---
Refs (transactional references) are mutable references to values and through the STM allow the safe sharing of mutable data. To ensure safety the value stored in a Ref should be immutable. The value referenced by a Ref can only be accessed or swapped within a transaction. Refs separate identity from value.
Creating a Ref
^^^^^^^^^^^^^^
You can create a Ref with or without an initial value.
.. code-block:: java
import akka.stm.*;
// giving an initial value
final Ref<Integer> ref = new Ref<Integer>(0);
// specifying a type but no initial value
final Ref<Integer> ref = new Ref<Integer>();
Accessing the value of a Ref
^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Use ``get`` to access the value of a Ref. Note that if no initial value has been given then the value is initially ``null``.
.. code-block:: java
import akka.stm.*;
final Ref<Integer> ref = new Ref<Integer>(0);
Integer value = new Atomic<Integer>() {
public Integer atomically() {
return ref.get();
}
}.execute();
// -> value = 0
Changing the value of a Ref
^^^^^^^^^^^^^^^^^^^^^^^^^^^
To set a new value for a Ref you can use ``set`` (or equivalently ``swap``), which sets the new value and returns the old value.
.. code-block:: java
import akka.stm.*;
final Ref<Integer> ref = new Ref<Integer>(0);
new Atomic() {
public Object atomically() {
return ref.set(5);
}
}.execute();
Transactions
------------
A transaction is delimited using an ``Atomic`` anonymous inner class.
.. code-block:: java
new Atomic() {
public Object atomically() {
// ...
}
}.execute();
All changes made to transactional objects are isolated from other changes, all make it or non make it (so failure atomicity) and are consistent. With the AkkaSTM you automatically have the Oracle version of the SERIALIZED isolation level, lower isolation is not possible. To make it fully serialized, set the writeskew property that checks if a writeskew problem is allowed to happen.
Retries
^^^^^^^
A transaction is automatically retried when it runs into some read or write conflict, until the operation completes, an exception (throwable) is thrown or when there are too many retries. When a read or writeconflict is encountered, the transaction uses a bounded exponential backoff to prevent cause more contention and give other transactions some room to complete.
If you are using non transactional resources in an atomic block, there could be problems because a transaction can be retried. If you are using print statements or logging, it could be that they are called more than once. So you need to be prepared to deal with this. One of the possible solutions is to work with a deferred or compensating task that is executed after the transaction aborts or commits.
Unexpected retries
^^^^^^^^^^^^^^^^^^
It can happen for the first few executions that you get a few failures of execution that lead to unexpected retries, even though there is not any read or writeconflict. The cause of this is that speculative transaction configuration/selection is used. There are transactions optimized for a single transactional object, for 1..n and for n to unlimited. So based on the execution of the transaction, the system learns; it begins with a cheap one and upgrades to more expensive ones. Once it has learned, it will reuse this knowledge. It can be activated/deactivated using the speculative property on the TransactionFactoryBuilder. In most cases it is best use the default value (enabled) so you get more out of performance.
Coordinated transactions and Transactors
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
If you need coordinated transactions across actors or threads then see :ref:`transactors-java`.
Configuring transactions
^^^^^^^^^^^^^^^^^^^^^^^^
It's possible to configure transactions. The ``Atomic`` class can take a ``TransactionFactory``, which can determine properties of the transaction. A default transaction factory is used if none is specified. You can create a ``TransactionFactory`` with a ``TransactionFactoryBuilder``.
Configuring transactions with a ``TransactionFactory``:
.. code-block:: java
import akka.stm.*;
TransactionFactory txFactory = new TransactionFactoryBuilder()
.setReadonly(true)
.build();
new Atomic<Object>(txFactory) {
public Object atomically() {
// read only transaction
return ...;
}
}.execute();
The following settings are possible on a TransactionFactory:
- familyName - Family name for transactions. Useful for debugging because the familyName is shown in exceptions, logging and in the future also will be used for profiling.
- readonly - Sets transaction as readonly. Readonly transactions are cheaper and can be used to prevent modification to transactional objects.
- maxRetries - The maximum number of times a transaction will retry.
- timeout - The maximum time a transaction will block for.
- trackReads - Whether all reads should be tracked. Needed for blocking operations. Readtracking makes a transaction more expensive, but makes subsequent reads cheaper and also lowers the chance of a readconflict.
- writeSkew - Whether writeskew is allowed. Disable with care.
- blockingAllowed - Whether explicit retries are allowed.
- interruptible - Whether a blocking transaction can be interrupted if it is blocked.
- speculative - Whether speculative configuration should be enabled.
- quickRelease - Whether locks should be released as quickly as possible (before whole commit).
- propagation - For controlling how nested transactions behave.
- traceLevel - Transaction trace level.
You can also specify the default values for some of these options in :ref:`configuration`.
Transaction lifecycle listeners
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
It's possible to have code that will only run on the successful commit of a transaction, or when a transaction aborts. You can do this by adding ``deferred`` or ``compensating`` blocks to a transaction.
.. code-block:: java
import akka.stm.*;
import static akka.stm.StmUtils.deferred;
import static akka.stm.StmUtils.compensating;
new Atomic() {
public Object atomically() {
deferred(new Runnable() {
public void run() {
// executes when transaction commits
}
});
compensating(new Runnable() {
public void run() {
// executes when transaction aborts
}
});
// ...
return something;
}
}.execute();
Blocking transactions
^^^^^^^^^^^^^^^^^^^^^
You can block in a transaction until a condition is met by using an explicit ``retry``. To use ``retry`` you also need to configure the transaction to allow explicit retries.
Here is an example of using ``retry`` to block until an account has enough money for a withdrawal. This is also an example of using actors and STM together.
.. code-block:: java
import akka.stm.*;
public class Transfer {
private final Ref<Double> from;
private final Ref<Double> to;
private final double amount;
public Transfer(Ref<Double> from, Ref<Double> to, double amount) {
this.from = from;
this.to = to;
this.amount = amount;
}
public Ref<Double> getFrom() { return from; }
public Ref<Double> getTo() { return to; }
public double getAmount() { return amount; }
}
.. code-block:: java
import akka.stm.*;
import static akka.stm.StmUtils.retry;
import akka.actor.*;
import akka.util.FiniteDuration;
import java.util.concurrent.TimeUnit;
import akka.event.EventHandler;
public class Transferer extends UntypedActor {
TransactionFactory txFactory = new TransactionFactoryBuilder()
.setBlockingAllowed(true)
.setTrackReads(true)
.setTimeout(new FiniteDuration(60, TimeUnit.SECONDS))
.build();
public void onReceive(Object message) throws Exception {
if (message instanceof Transfer) {
Transfer transfer = (Transfer) message;
final Ref<Double> from = transfer.getFrom();
final Ref<Double> to = transfer.getTo();
final double amount = transfer.getAmount();
new Atomic(txFactory) {
public Object atomically() {
if (from.get() < amount) {
EventHandler.info(this, "not enough money - retrying");
retry();
}
EventHandler.info(this, "transferring");
from.set(from.get() - amount);
to.set(to.get() + amount);
return null;
}
}.execute();
}
}
}
.. code-block:: java
import akka.stm.*;
import akka.actor.*;
public class Main {
public static void main(String...args) throws Exception {
final Ref<Double> account1 = new Ref<Double>(100.0);
final Ref<Double> account2 = new Ref<Double>(100.0);
ActorRef transferer = Actors.actorOf(Transferer.class);
transferer.tell(new Transfer(account1, account2, 500.0));
// Transferer: not enough money - retrying
new Atomic() {
public Object atomically() {
return account1.set(account1.get() + 2000);
}
}.execute();
// Transferer: transferring
Thread.sleep(1000);
Double acc1 = new Atomic<Double>() {
public Double atomically() {
return account1.get();
}
}.execute();
Double acc2 = new Atomic<Double>() {
public Double atomically() {
return account2.get();
}
}.execute();
System.out.println("Account 1: " + acc1);
// Account 1: 1600.0
System.out.println("Account 2: " + acc2);
// Account 2: 600.0
transferer.stop();
}
}
Alternative blocking transactions
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
You can also have two alternative blocking transactions, one of which can succeed first, with ``EitherOrElse``.
.. code-block:: java
import akka.stm.*;
public class Branch {
private final Ref<Integer> left;
private final Ref<Integer> right;
private final double amount;
public Branch(Ref<Integer> left, Ref<Integer> right, int amount) {
this.left = left;
this.right = right;
this.amount = amount;
}
public Ref<Integer> getLeft() { return left; }
public Ref<Integer> getRight() { return right; }
public double getAmount() { return amount; }
}
.. code-block:: java
import akka.actor.*;
import akka.stm.*;
import static akka.stm.StmUtils.retry;
import akka.util.FiniteDuration;
import java.util.concurrent.TimeUnit;
import akka.event.EventHandler;
public class Brancher extends UntypedActor {
TransactionFactory txFactory = new TransactionFactoryBuilder()
.setBlockingAllowed(true)
.setTrackReads(true)
.setTimeout(new FiniteDuration(60, TimeUnit.SECONDS))
.build();
public void onReceive(Object message) throws Exception {
if (message instanceof Branch) {
Branch branch = (Branch) message;
final Ref<Integer> left = branch.getLeft();
final Ref<Integer> right = branch.getRight();
final double amount = branch.getAmount();
new Atomic<Integer>(txFactory) {
public Integer atomically() {
return new EitherOrElse<Integer>() {
public Integer either() {
if (left.get() < amount) {
EventHandler.info(this, "not enough on left - retrying");
retry();
}
EventHandler.info(this, "going left");
return left.get();
}
public Integer orElse() {
if (right.get() < amount) {
EventHandler.info(this, "not enough on right - retrying");
retry();
}
EventHandler.info(this, "going right");
return right.get();
}
}.execute();
}
}.execute();
}
}
}
.. code-block:: java
import akka.stm.*;
import akka.actor.*;
public class Main2 {
public static void main(String...args) throws Exception {
final Ref<Integer> left = new Ref<Integer>(100);
final Ref<Integer> right = new Ref<Integer>(100);
ActorRef brancher = Actors.actorOf(Brancher.class);
brancher.tell(new Branch(left, right, 500));
// not enough on left - retrying
// not enough on right - retrying
Thread.sleep(1000);
new Atomic() {
public Object atomically() {
return right.set(right.get() + 1000);
}
}.execute();
// going right
brancher.stop();
}
}
Transactional datastructures
----------------------------
Akka provides two datastructures that are managed by the STM.
- TransactionalMap
- TransactionalVector
TransactionalMap and TransactionalVector look like regular mutable datastructures, they even implement the standard Scala 'Map' and 'RandomAccessSeq' interfaces, but they are implemented using persistent datastructures and managed references under the hood. Therefore they are safe to use in a concurrent environment. Underlying TransactionalMap is HashMap, an immutable Map but with near constant time access and modification operations. Similarly TransactionalVector uses a persistent Vector. See the Persistent Datastructures section below for more details.
Like managed references, TransactionalMap and TransactionalVector can only be modified inside the scope of an STM transaction.
Here is an example of creating and accessing a TransactionalMap:
.. code-block:: java
import akka.stm.*;
// assuming a User class
final TransactionalMap<String, User> users = new TransactionalMap<String, User>();
// fill users map (in a transaction)
new Atomic() {
public Object atomically() {
users.put("bill", new User("bill"));
users.put("mary", new User("mary"));
users.put("john", new User("john"));
return null;
}
}.execute();
// access users map (in a transaction)
User user = new Atomic<User>() {
public User atomically() {
return users.get("bill").get();
}
}.execute();
Here is an example of creating and accessing a TransactionalVector:
.. code-block:: java
import akka.stm.*;
// assuming an Address class
final TransactionalVector<Address> addresses = new TransactionalVector<Address>();
// fill addresses vector (in a transaction)
new Atomic() {
public Object atomically() {
addresses.add(new Address("somewhere"));
addresses.add(new Address("somewhere else"));
return null;
}
}.execute();
// access addresses vector (in a transaction)
Address address = new Atomic<Address>() {
public Address atomically() {
return addresses.get(0);
}
}.execute();
Persistent datastructures
-------------------------
Akka's STM should only be used with immutable data. This can be costly if you have large datastructures and are using a naive copy-on-write. In order to make working with immutable datastructures fast enough Scala provides what are called Persistent Datastructures. There are currently two different ones:
- HashMap (`scaladoc <http://www.scala-lang.org/api/current/scala/collection/immutable/HashMap.html>`__)
- Vector (`scaladoc <http://www.scala-lang.org/api/current/scala/collection/immutable/Vector.html>`__)
They are immutable and each update creates a completely new version but they are using clever structural sharing in order to make them almost as fast, for both read and update, as regular mutable datastructures.
This illustration is taken from Rich Hickey's presentation. Copyright Rich Hickey 2009.
.. image:: ../images/clojure-trees.png

View file

@ -0,0 +1,271 @@
.. _transactors-java:
Transactors (Java)
==================
.. sidebar:: Contents
.. contents:: :local:
Module stability: **SOLID**
Why Transactors?
----------------
Actors are excellent for solving problems where you have many independent processes that can work in isolation and only interact with other Actors through message passing. This model fits many problems. But the actor model is unfortunately a terrible model for implementing truly shared state. E.g. when you need to have consensus and a stable view of state across many components. The classic example is the bank account where clients can deposit and withdraw, in which each operation needs to be atomic. For detailed discussion on the topic see `this JavaOne presentation <http://www.slideshare.net/jboner/state-youre-doing-it-wrong-javaone-2009>`_.
**STM** on the other hand is excellent for problems where you need consensus and a stable view of the state by providing compositional transactional shared state. Some of the really nice traits of STM are that transactions compose, and it raises the abstraction level from lock-based concurrency.
Akka's Transactors combine Actors and STM to provide the best of the Actor model (concurrency and asynchronous event-based programming) and STM (compositional transactional shared state) by providing transactional, compositional, asynchronous, event-based message flows.
If you need Durability then you should not use one of the in-memory data structures but one of the persistent ones.
Generally, the STM is not needed very often when working with Akka. Some use-cases (that we can think of) are:
- When you really need composable message flows across many actors updating their **internal local** state but need them to do that atomically in one big transaction. Might not often, but when you do need this then you are screwed without it.
- When you want to share a datastructure across actors.
- When you need to use the persistence modules.
Actors and STM
^^^^^^^^^^^^^^
You can combine Actors and STM in several ways. An Actor may use STM internally so that particular changes are guaranteed to be atomic. Actors may also share transactional datastructures as the STM provides safe shared state across threads.
It's also possible to coordinate transactions across Actors or threads so that either the transactions in a set all commit successfully or they all fail. This is the focus of Transactors and the explicit support for coordinated transactions in this section.
Coordinated transactions
------------------------
Akka provides an explicit mechanism for coordinating transactions across actors. Under the hood it uses a ``CountDownCommitBarrier``, similar to a CountDownLatch.
Here is an example of coordinating two simple counter UntypedActors so that they both increment together in coordinated transactions. If one of them was to fail to increment, the other would also fail.
.. code-block:: java
import akka.actor.ActorRef;
public class Increment {
private final ActorRef friend;
public Increment() {
this.friend = null;
}
public Increment(ActorRef friend) {
this.friend = friend;
}
public boolean hasFriend() {
return friend != null;
}
public ActorRef getFriend() {
return friend;
}
}
.. code-block:: java
import akka.actor.UntypedActor;
import akka.stm.Ref;
import akka.transactor.Atomically;
import akka.transactor.Coordinated;
public class Counter extends UntypedActor {
private Ref<Integer> count = new Ref(0);
private void increment() {
count.set(count.get() + 1);
}
public void onReceive(Object incoming) throws Exception {
if (incoming instanceof Coordinated) {
Coordinated coordinated = (Coordinated) incoming;
Object message = coordinated.getMessage();
if (message instanceof Increment) {
Increment increment = (Increment) message;
if (increment.hasFriend()) {
increment.getFriend().tell(coordinated.coordinate(new Increment()));
}
coordinated.atomic(new Atomically() {
public void atomically() {
increment();
}
});
}
} else if (incoming.equals("GetCount")) {
getContext().reply(count.get());
}
}
}
.. code-block:: java
ActorRef counter1 = actorOf(Counter.class);
ActorRef counter2 = actorOf(Counter.class);
counter1.tell(new Coordinated(new Increment(counter2)));
To start a new coordinated transaction that you will also participate in, just create a ``Coordinated`` object:
.. code-block:: java
Coordinated coordinated = new Coordinated();
To start a coordinated transaction that you won't participate in yourself you can create a ``Coordinated`` object with a message and send it directly to an actor. The recipient of the message will be the first member of the coordination set:
.. code-block:: java
actor.tell(new Coordinated(new Message()));
To include another actor in the same coordinated transaction that you've created or received, use the ``coordinate`` method on that object. This will increment the number of parties involved by one and create a new ``Coordinated`` object to be sent.
.. code-block:: java
actor.tell(coordinated.coordinate(new Message()));
To enter the coordinated transaction use the atomic method of the coordinated object. This accepts either an ``akka.transactor.Atomically`` object, or an ``Atomic`` object the same as used normally in the STM (just don't execute it - the coordination will do that).
.. code-block:: java
coordinated.atomic(new Atomically() {
public void atomically() {
// do something in a transaction
}
});
The coordinated transaction will wait for the other transactions before committing. If any of the coordinated transactions fail then they all fail.
UntypedTransactor
-----------------
UntypedTransactors are untyped actors that provide a general pattern for coordinating transactions, using the explicit coordination described above.
Here's an example of a simple untyped transactor that will join a coordinated transaction:
.. code-block:: java
import akka.transactor.UntypedTransactor;
import akka.stm.Ref;
public class Counter extends UntypedTransactor {
Ref<Integer> count = new Ref<Integer>(0);
@Override
public void atomically(Object message) {
if (message instanceof Increment) {
count.set(count.get() + 1);
}
}
}
You could send this Counter transactor a ``Coordinated(Increment)`` message. If you were to send it just an ``Increment`` message it will create its own ``Coordinated`` (but in this particular case wouldn't be coordinating transactions with any other transactors).
To coordinate with other transactors override the ``coordinate`` method. The ``coordinate`` method maps a message to a set of ``SendTo`` objects, pairs of ``ActorRef`` and a message. You can use the ``include`` and ``sendTo`` methods to easily coordinate with other transactors.
Example of coordinating an increment, similar to the explicitly coordinated example:
.. code-block:: java
import akka.transactor.UntypedTransactor;
import akka.transactor.SendTo;
import akka.stm.Ref;
import java.util.Set;
public class Counter extends UntypedTransactor {
Ref<Integer> count = new Ref<Integer>(0);
@Override
public Set<SendTo> coordinate(Object message) {
if (message instanceof Increment) {
Increment increment = (Increment) message;
if (increment.hasFriend())
return include(increment.getFriend(), new Increment());
}
return nobody();
}
@Override
public void atomically(Object message) {
if (message instanceof Increment) {
count.set(count.get() + 1);
}
}
}
To execute directly before or after the coordinated transaction, override the ``before`` and ``after`` methods. They do not execute within the transaction.
To completely bypass coordinated transactions override the ``normally`` method. Any message matched by ``normally`` will not be matched by the other methods, and will not be involved in coordinated transactions. In this method you can implement normal actor behavior, or use the normal STM atomic for local transactions.
Coordinating Typed Actors
-------------------------
It's also possible to use coordinated transactions with typed actors. You can explicitly pass around ``Coordinated`` objects, or use built-in support with the ``@Coordinated`` annotation and the ``Coordination.coordinate`` method.
To specify a method should use coordinated transactions add the ``@Coordinated`` annotation. **Note**: the ``@Coordinated`` annotation will only work with void (one-way) methods.
.. code-block:: java
public interface Counter {
@Coordinated public void increment();
public Integer get();
}
To coordinate transactions use a ``coordinate`` block. This accepts either an ``akka.transactor.Atomically`` object, or an ``Atomic`` object liked used in the STM (but don't execute it). The first boolean parameter specifies whether or not to wait for the transactions to complete.
.. code-block:: java
Coordination.coordinate(true, new Atomically() {
public void atomically() {
counter1.increment();
counter2.increment();
}
});
Here's an example of using ``@Coordinated`` with a TypedActor to coordinate increments:
.. code-block:: java
import akka.transactor.annotation.Coordinated;
public interface Counter {
@Coordinated public void increment();
public Integer get();
}
.. code-block:: java
import akka.actor.TypedActor;
import akka.stm.Ref;
public class CounterImpl extends TypedActor implements Counter {
private Ref<Integer> count = new Ref<Integer>(0);
public void increment() {
count.set(count.get() + 1);
}
public Integer get() {
return count.get();
}
}
.. code-block:: java
Counter counter1 = (Counter) TypedActor.newInstance(Counter.class, CounterImpl.class);
Counter counter2 = (Counter) TypedActor.newInstance(Counter.class, CounterImpl.class);
Coordination.coordinate(true, new Atomically() {
public void atomically() {
counter1.increment();
counter2.increment();
}
});
TypedActor.stop(counter1);
TypedActor.stop(counter2);

View file

@ -0,0 +1,40 @@
.. _microkernel:
#############
Microkernel
#############
Run the microkernel
===================
To start the kernel use the scripts in the ``bin`` directory.
All services are configured in the :ref:`configuration` file in the ``config`` directory.
Services you want to be started up automatically should be listed in the list of ``boot`` classes in
the :ref:`configuration`.
Put your application in the ``deploy`` directory.
Akka Home
---------
Note that the microkernel needs to know where the Akka home is (the base
directory of the microkernel). The above scripts do this for you. Otherwise, you
can set Akka home by:
* Specifying the ``AKKA_HOME`` environment variable
* Specifying the ``-Dakka.home`` java option
.. _hello-microkernel:
Hello Microkernel
=================
There is a very simple Akka Mist sample project included in the microkernel
``deploy`` directory. Start the microkernel with the start script and then go to
http://localhost:9998 to say Hello to the microkernel.

View file

@ -0,0 +1,537 @@
.. _stm-scala:
#######################################
Software Transactional Memory (Scala)
#######################################
.. sidebar:: Contents
.. contents:: :local:
Overview of STM
===============
An `STM <http://en.wikipedia.org/wiki/Software_transactional_memory>`_ turns the
Java heap into a transactional data set with begin/commit/rollback
semantics. Very much like a regular database. It implements the first three
letters in ACID; ACI:
* Atomic
* Consistent
* Isolated
Generally, the STM is not needed very often when working with Akka. Some
use-cases (that we can think of) are:
- When you really need composable message flows across many actors updating
their **internal local** state but need them to do that atomically in one big
transaction. Might not be often, but when you do need this then you are
screwed without it.
- When you want to share a datastructure across actors.
- When you need to use the persistence modules.
Akkas STM implements the concept in `Clojure's <clojure>`_ STM view on state in
general. Please take the time to read `this excellent document <clojure-state>`_
and view `this presentation <clojure-presentation>`_ by Rich Hickey (the genius
behind Clojure), since it forms the basis of Akkas view on STM and state in
general.
.. _clojure: http://clojure.org/
.. _clojure-state: http://clojure.org/state
.. _clojure-presentation: http://www.infoq.com/presentations/Value-Identity-State-Rich-Hickey
The STM is based on Transactional References (referred to as Refs). Refs are
memory cells, holding an (arbitrary) immutable value, that implement CAS
(Compare-And-Swap) semantics and are managed and enforced by the STM for
coordinated changes across many Refs. They are implemented using the excellent
`Multiverse STM <multiverse>`_.
.. _multiverse: http://multiverse.codehaus.org/overview.html
Working with immutable collections can sometimes give bad performance due to
extensive copying. Scala provides so-called persistent datastructures which
makes working with immutable collections fast. They are immutable but with
constant time access and modification. They use structural sharing and an insert
or update does not ruin the old structure, hence “persistent”. Makes working
with immutable composite types fast. The persistent datastructures currently
consist of a Map and Vector.
Simple example
==============
Here is a simple example of an incremental counter using STM. This shows
creating a ``Ref``, a transactional reference, and then modifying it within a
transaction, which is delimited by ``atomic``.
.. includecode:: code/StmDocSpec.scala#simple
Ref
---
Refs (transactional references) are mutable references to values and through the STM allow the safe sharing of mutable data. Refs separate identity from value. To ensure safety the value stored in a Ref should be immutable (they can of course contain refs themselves). The value referenced by a Ref can only be accessed or swapped within a transaction. If a transaction is not available, the call will be executed in its own transaction (the call will be atomic). This is a different approach than the Clojure Refs, where a missing transaction results in an error.
Creating a Ref
^^^^^^^^^^^^^^
You can create a Ref with or without an initial value.
.. code-block:: scala
import akka.stm._
// giving an initial value
val ref = Ref(0)
// specifying a type but no initial value
val ref = Ref[Int]
Accessing the value of a Ref
^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Use ``get`` to access the value of a Ref. Note that if no initial value has been given then the value is initially ``null``.
.. code-block:: scala
import akka.stm._
val ref = Ref(0)
atomic {
ref.get
}
// -> 0
If there is a chance that the value of a Ref is null then you can use ``opt``, which will create an Option, either Some(value) or None, or you can provide a default value with ``getOrElse``. You can also check for null using ``isNull``.
.. code-block:: scala
import akka.stm._
val ref = Ref[Int]
atomic {
ref.opt // -> None
ref.getOrElse(0) // -> 0
ref.isNull // -> true
}
Changing the value of a Ref
^^^^^^^^^^^^^^^^^^^^^^^^^^^
To set a new value for a Ref you can use ``set`` (or equivalently ``swap``), which sets the new value and returns the old value.
.. code-block:: scala
import akka.stm._
val ref = Ref(0)
atomic {
ref.set(5)
}
// -> 0
atomic {
ref.get
}
// -> 5
You can also use ``alter`` which accepts a function that takes the old value and creates a new value of the same type.
.. code-block:: scala
import akka.stm._
val ref = Ref(0)
atomic {
ref alter (_ + 5)
}
// -> 5
val inc = (i: Int) => i + 1
atomic {
ref alter inc
}
// -> 6
Refs in for-comprehensions
^^^^^^^^^^^^^^^^^^^^^^^^^^
Ref is monadic and can be used in for-comprehensions.
.. code-block:: scala
import akka.stm._
val ref = Ref(1)
atomic {
for (value <- ref) {
// do something with value
}
}
val anotherRef = Ref(3)
atomic {
for {
value1 <- ref
value2 <- anotherRef
} yield (value1 + value2)
}
// -> Ref(4)
val emptyRef = Ref[Int]
atomic {
for {
value1 <- ref
value2 <- emptyRef
} yield (value1 + value2)
}
// -> Ref[Int]
Transactions
------------
A transaction is delimited using ``atomic``.
.. code-block:: scala
atomic {
// ...
}
All changes made to transactional objects are isolated from other changes, all make it or non make it (so failure atomicity) and are consistent. With the AkkaSTM you automatically have the Oracle version of the SERIALIZED isolation level, lower isolation is not possible. To make it fully serialized, set the writeskew property that checks if a writeskew problem is allowed to happen.
Retries
^^^^^^^
A transaction is automatically retried when it runs into some read or write conflict, until the operation completes, an exception (throwable) is thrown or when there are too many retries. When a read or writeconflict is encountered, the transaction uses a bounded exponential backoff to prevent cause more contention and give other transactions some room to complete.
If you are using non transactional resources in an atomic block, there could be problems because a transaction can be retried. If you are using print statements or logging, it could be that they are called more than once. So you need to be prepared to deal with this. One of the possible solutions is to work with a deferred or compensating task that is executed after the transaction aborts or commits.
Unexpected retries
^^^^^^^^^^^^^^^^^^
It can happen for the first few executions that you get a few failures of execution that lead to unexpected retries, even though there is not any read or writeconflict. The cause of this is that speculative transaction configuration/selection is used. There are transactions optimized for a single transactional object, for 1..n and for n to unlimited. So based on the execution of the transaction, the system learns; it begins with a cheap one and upgrades to more expensive ones. Once it has learned, it will reuse this knowledge. It can be activated/deactivated using the speculative property on the TransactionFactory. In most cases it is best use the default value (enabled) so you get more out of performance.
Coordinated transactions and Transactors
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
If you need coordinated transactions across actors or threads then see :ref:`transactors-scala`.
Configuring transactions
^^^^^^^^^^^^^^^^^^^^^^^^
It's possible to configure transactions. The ``atomic`` method can take an implicit or explicit ``TransactionFactory``, which can determine properties of the transaction. A default transaction factory is used if none is specified explicitly or there is no implicit ``TransactionFactory`` in scope.
Configuring transactions with an **implicit** ``TransactionFactory``:
.. code-block:: scala
import akka.stm._
implicit val txFactory = TransactionFactory(readonly = true)
atomic {
// read only transaction
}
Configuring transactions with an **explicit** ``TransactionFactory``:
.. code-block:: scala
import akka.stm._
val txFactory = TransactionFactory(readonly = true)
atomic(txFactory) {
// read only transaction
}
The following settings are possible on a TransactionFactory:
- ``familyName`` - Family name for transactions. Useful for debugging.
- ``readonly`` - Sets transaction as readonly. Readonly transactions are cheaper.
- ``maxRetries`` - The maximum number of times a transaction will retry.
- ``timeout`` - The maximum time a transaction will block for.
- ``trackReads`` - Whether all reads should be tracked. Needed for blocking operations.
- ``writeSkew`` - Whether writeskew is allowed. Disable with care.
- ``blockingAllowed`` - Whether explicit retries are allowed.
- ``interruptible`` - Whether a blocking transaction can be interrupted.
- ``speculative`` - Whether speculative configuration should be enabled.
- ``quickRelease`` - Whether locks should be released as quickly as possible (before whole commit).
- ``propagation`` - For controlling how nested transactions behave.
- ``traceLevel`` - Transaction trace level.
You can also specify the default values for some of these options in the :ref:`configuration`.
You can also determine at which level a transaction factory is shared or not shared, which affects the way in which the STM can optimise transactions.
Here is a shared transaction factory for all instances of an actor.
.. code-block:: scala
import akka.actor._
import akka.stm._
object MyActor {
implicit val txFactory = TransactionFactory(readonly = true)
}
class MyActor extends Actor {
import MyActor.txFactory
def receive = {
case message: String =>
atomic {
// read only transaction
}
}
}
Here's a similar example with an individual transaction factory for each instance of an actor.
.. code-block:: scala
import akka.actor._
import akka.stm._
class MyActor extends Actor {
implicit val txFactory = TransactionFactory(readonly = true)
def receive = {
case message: String =>
atomic {
// read only transaction
}
}
}
Transaction lifecycle listeners
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
It's possible to have code that will only run on the successful commit of a transaction, or when a transaction aborts. You can do this by adding ``deferred`` or ``compensating`` blocks to a transaction.
.. code-block:: scala
import akka.stm._
atomic {
deferred {
// executes when transaction commits
}
compensating {
// executes when transaction aborts
}
}
Blocking transactions
^^^^^^^^^^^^^^^^^^^^^
You can block in a transaction until a condition is met by using an explicit ``retry``. To use ``retry`` you also need to configure the transaction to allow explicit retries.
Here is an example of using ``retry`` to block until an account has enough money for a withdrawal. This is also an example of using actors and STM together.
.. code-block:: scala
import akka.stm._
import akka.actor._
import akka.util.duration._
import akka.event.EventHandler
type Account = Ref[Double]
case class Transfer(from: Account, to: Account, amount: Double)
class Transferer extends Actor {
implicit val txFactory = TransactionFactory(blockingAllowed = true, trackReads = true, timeout = 60 seconds)
def receive = {
case Transfer(from, to, amount) =>
atomic {
if (from.get < amount) {
EventHandler.info(this, "not enough money - retrying")
retry
}
EventHandler.info(this, "transferring")
from alter (_ - amount)
to alter (_ + amount)
}
}
}
val account1 = Ref(100.0)
val account2 = Ref(100.0)
val transferer = Actor.actorOf(new Transferer)
transferer ! Transfer(account1, account2, 500.0)
// INFO Transferer: not enough money - retrying
atomic { account1 alter (_ + 2000) }
// INFO Transferer: transferring
atomic { account1.get }
// -> 1600.0
atomic { account2.get }
// -> 600.0
transferer.stop()
Alternative blocking transactions
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
You can also have two alternative blocking transactions, one of which can succeed first, with ``either-orElse``.
.. code-block:: scala
import akka.stm._
import akka.actor._
import akka.util.duration._
import akka.event.EventHandler
case class Branch(left: Ref[Int], right: Ref[Int], amount: Int)
class Brancher extends Actor {
implicit val txFactory = TransactionFactory(blockingAllowed = true, trackReads = true, timeout = 60 seconds)
def receive = {
case Branch(left, right, amount) =>
atomic {
either {
if (left.get < amount) {
EventHandler.info(this, "not enough on left - retrying")
retry
}
log.info("going left")
} orElse {
if (right.get < amount) {
EventHandler.info(this, "not enough on right - retrying")
retry
}
log.info("going right")
}
}
}
}
val ref1 = Ref(0)
val ref2 = Ref(0)
val brancher = Actor.actorOf(new Brancher)
brancher ! Branch(ref1, ref2, 1)
// INFO Brancher: not enough on left - retrying
// INFO Brancher: not enough on right - retrying
atomic { ref2 alter (_ + 1) }
// INFO Brancher: not enough on left - retrying
// INFO Brancher: going right
brancher.stop()
Transactional datastructures
----------------------------
Akka provides two datastructures that are managed by the STM.
- ``TransactionalMap``
- ``TransactionalVector``
``TransactionalMap`` and ``TransactionalVector`` look like regular mutable datastructures, they even implement the standard Scala 'Map' and 'RandomAccessSeq' interfaces, but they are implemented using persistent datastructures and managed references under the hood. Therefore they are safe to use in a concurrent environment. Underlying TransactionalMap is HashMap, an immutable Map but with near constant time access and modification operations. Similarly ``TransactionalVector`` uses a persistent Vector. See the Persistent Datastructures section below for more details.
Like managed references, ``TransactionalMap`` and ``TransactionalVector`` can only be modified inside the scope of an STM transaction.
*IMPORTANT*: There have been some problems reported when using transactional datastructures with 'lazy' initialization. Avoid that.
Here is how you create these transactional datastructures:
.. code-block:: scala
import akka.stm._
// assuming something like
case class User(name: String)
case class Address(location: String)
// using initial values
val map = TransactionalMap("bill" -> User("bill"))
val vector = TransactionalVector(Address("somewhere"))
// specifying types
val map = TransactionalMap[String, User]
val vector = TransactionalVector[Address]
``TransactionalMap`` and ``TransactionalVector`` wrap persistent datastructures with transactional references and provide a standard Scala interface. This makes them convenient to use.
Here is an example of using a ``Ref`` and a ``HashMap`` directly:
.. code-block:: scala
import akka.stm._
import scala.collection.immutable.HashMap
case class User(name: String)
val ref = Ref(HashMap[String, User]())
atomic {
val users = ref.get
val newUsers = users + ("bill" -> User("bill")) // creates a new HashMap
ref.swap(newUsers)
}
atomic {
ref.get.apply("bill")
}
// -> User("bill")
Here is the same example using ``TransactionalMap``:
.. code-block:: scala
import akka.stm._
case class User(name: String)
val users = TransactionalMap[String, User]
atomic {
users += "bill" -> User("bill")
}
atomic {
users("bill")
}
// -> User("bill")
Persistent datastructures
-------------------------
Akka's STM should only be used with immutable data. This can be costly if you have large datastructures and are using a naive copy-on-write. In order to make working with immutable datastructures fast enough Scala provides what are called Persistent Datastructures. There are currently two different ones:
* ``HashMap`` (`scaladoc <http://www.scala-lang.org/api/current/scala/collection/immutable/HashMap.html>`__)
* ``Vector`` (`scaladoc <http://www.scala-lang.org/api/current/scala/collection/immutable/Vector.html>`__)
They are immutable and each update creates a completely new version but they are using clever structural sharing in order to make them almost as fast, for both read and update, as regular mutable datastructures.
This illustration is taken from Rich Hickey's presentation. Copyright Rich Hickey 2009.
.. image:: ../images/clojure-trees.png
Ants simulation sample
----------------------
One fun and very enlightening visual demo of STM, actors and transactional references is the `Ant simulation sample <http://github.com/jboner/akka/tree/master/akka-samples/akka-sample-ants/>`_. I encourage you to run it and read through the code since it's a good example of using actors with STM.

View file

@ -0,0 +1,250 @@
.. _transactors-scala:
Transactors (Scala)
===================
.. sidebar:: Contents
.. contents:: :local:
Module stability: **SOLID**
Why Transactors?
----------------
Actors are excellent for solving problems where you have many independent processes that can work in isolation and only interact with other Actors through message passing. This model fits many problems. But the actor model is unfortunately a terrible model for implementing truly shared state. E.g. when you need to have consensus and a stable view of state across many components. The classic example is the bank account where clients can deposit and withdraw, in which each operation needs to be atomic. For detailed discussion on the topic see `this JavaOne presentation <http://www.slideshare.net/jboner/state-youre-doing-it-wrong-javaone-2009>`_.
**STM** on the other hand is excellent for problems where you need consensus and a stable view of the state by providing compositional transactional shared state. Some of the really nice traits of STM are that transactions compose, and it raises the abstraction level from lock-based concurrency.
Akka's Transactors combine Actors and STM to provide the best of the Actor model (concurrency and asynchronous event-based programming) and STM (compositional transactional shared state) by providing transactional, compositional, asynchronous, event-based message flows.
If you need Durability then you should not use one of the in-memory data structures but one of the persistent ones.
Generally, the STM is not needed very often when working with Akka. Some use-cases (that we can think of) are:
- When you really need composable message flows across many actors updating their **internal local** state but need them to do that atomically in one big transaction. Might not often, but when you do need this then you are screwed without it.
- When you want to share a datastructure across actors.
- When you need to use the persistence modules.
Actors and STM
^^^^^^^^^^^^^^
You can combine Actors and STM in several ways. An Actor may use STM internally so that particular changes are guaranteed to be atomic. Actors may also share transactional datastructures as the STM provides safe shared state across threads.
It's also possible to coordinate transactions across Actors or threads so that either the transactions in a set all commit successfully or they all fail. This is the focus of Transactors and the explicit support for coordinated transactions in this section.
Coordinated transactions
------------------------
Akka provides an explicit mechanism for coordinating transactions across Actors. Under the hood it uses a ``CountDownCommitBarrier``, similar to a CountDownLatch.
Here is an example of coordinating two simple counter Actors so that they both increment together in coordinated transactions. If one of them was to fail to increment, the other would also fail.
.. code-block:: scala
import akka.transactor.Coordinated
import akka.stm.Ref
import akka.actor.{Actor, ActorRef}
case class Increment(friend: Option[ActorRef] = None)
case object GetCount
class Counter extends Actor {
val count = Ref(0)
def receive = {
case coordinated @ Coordinated(Increment(friend)) => {
friend foreach (_ ! coordinated(Increment()))
coordinated atomic {
count alter (_ + 1)
}
}
case GetCount => self.reply(count.get)
}
}
val counter1 = Actor.actorOf[Counter]
val counter2 = Actor.actorOf[Counter]
counter1 ! Coordinated(Increment(Some(counter2)))
...
(counter1 ? GetCount).as[Int] // Some(1)
counter1.stop()
counter2.stop()
To start a new coordinated transaction that you will also participate in, just create a ``Coordinated`` object:
.. code-block:: scala
val coordinated = Coordinated()
To start a coordinated transaction that you won't participate in yourself you can create a ``Coordinated`` object with a message and send it directly to an actor. The recipient of the message will be the first member of the coordination set:
.. code-block:: scala
actor ! Coordinated(Message)
To receive a coordinated message in an actor simply match it in a case statement:
.. code-block:: scala
def receive = {
case coordinated @ Coordinated(Message) => ...
}
To include another actor in the same coordinated transaction that you've created or received, use the apply method on that object. This will increment the number of parties involved by one and create a new ``Coordinated`` object to be sent.
.. code-block:: scala
actor ! coordinated(Message)
To enter the coordinated transaction use the atomic method of the coordinated object:
.. code-block:: scala
coordinated atomic {
// do something in transaction ...
}
The coordinated transaction will wait for the other transactions before committing. If any of the coordinated transactions fail then they all fail.
Transactor
----------
Transactors are actors that provide a general pattern for coordinating transactions, using the explicit coordination described above.
Here's an example of a simple transactor that will join a coordinated transaction:
.. code-block:: scala
import akka.transactor.Transactor
import akka.stm.Ref
case object Increment
class Counter extends Transactor {
val count = Ref(0)
override def atomically = {
case Increment => count alter (_ + 1)
}
}
You could send this Counter transactor a ``Coordinated(Increment)`` message. If you were to send it just an ``Increment`` message it will create its own ``Coordinated`` (but in this particular case wouldn't be coordinating transactions with any other transactors).
To coordinate with other transactors override the ``coordinate`` method. The ``coordinate`` method maps a message to a set of ``SendTo`` objects, pairs of ``ActorRef`` and a message. You can use the ``include`` and ``sendTo`` methods to easily coordinate with other transactors. The ``include`` method will send on the same message that was received to other transactors. The ``sendTo`` method allows you to specify both the actor to send to, and the message to send.
Example of coordinating an increment:
.. code-block:: scala
import akka.transactor.Transactor
import akka.stm.Ref
import akka.actor.ActorRef
case object Increment
class FriendlyCounter(friend: ActorRef) extends Transactor {
val count = Ref(0)
override def coordinate = {
case Increment => include(friend)
}
override def atomically = {
case Increment => count alter (_ + 1)
}
}
Using ``include`` to include more than one transactor:
.. code-block:: scala
override def coordinate = {
case Message => include(actor1, actor2, actor3)
}
Using ``sendTo`` to coordinate transactions but pass-on a different message than the one that was received:
.. code-block:: scala
override def coordinate = {
case Message => sendTo(someActor -> SomeOtherMessage)
case SomeMessage => sendTo(actor1 -> Message1, actor2 -> Message2)
}
To execute directly before or after the coordinated transaction, override the ``before`` and ``after`` methods. These methods also expect partial functions like the receive method. They do not execute within the transaction.
To completely bypass coordinated transactions override the ``normally`` method. Any message matched by ``normally`` will not be matched by the other methods, and will not be involved in coordinated transactions. In this method you can implement normal actor behavior, or use the normal STM atomic for local transactions.
Coordinating Typed Actors
-------------------------
It's also possible to use coordinated transactions with typed actors. You can explicitly pass around ``Coordinated`` objects, or use built-in support with the ``@Coordinated`` annotation and the ``Coordination.coordinate`` method.
To specify a method should use coordinated transactions add the ``@Coordinated`` annotation. **Note**: the ``@Coordinated`` annotation only works with methods that return Unit (one-way methods).
.. code-block:: scala
trait Counter {
@Coordinated def increment()
def get: Int
}
To coordinate transactions use a ``coordinate`` block:
.. code-block:: scala
coordinate {
counter1.increment()
counter2.increment()
}
Here's an example of using ``@Coordinated`` with a TypedActor to coordinate increments.
.. code-block:: scala
import akka.actor.TypedActor
import akka.stm.Ref
import akka.transactor.annotation.Coordinated
import akka.transactor.Coordination._
trait Counter {
@Coordinated def increment()
def get: Int
}
class CounterImpl extends TypedActor with Counter {
val ref = Ref(0)
def increment() { ref alter (_ + 1) }
def get = ref.get
}
...
val counter1 = TypedActor.newInstance(classOf[Counter], classOf[CounterImpl])
val counter2 = TypedActor.newInstance(classOf[Counter], classOf[CounterImpl])
coordinate {
counter1.increment()
counter2.increment()
}
TypedActor.stop(counter1)
TypedActor.stop(counter2)
The ``coordinate`` block will wait for the transactions to complete. If you do not want to wait then you can specify this explicitly:
.. code-block:: scala
coordinate(wait = false) {
counter1.increment()
counter2.increment()
}

View file

@ -0,0 +1,335 @@
.. _spring-module:
####################
Spring Integration
####################
Module stability: **STABLE**
Akkas integration with the `Spring Framework <http://www.springsource.org>`_ supplies the Spring way of using the Typed Actor Java API and for CamelService configuration for :ref:`camel-spring-applications`. It uses Spring's custom namespaces to create Typed Actors, supervisor hierarchies and a CamelService in a Spring environment.
Contents:
.. contents:: :local:
To use the custom name space tags for Akka you have to add the XML schema definition to your spring configuration. It is available at `http://akka.io/akka-1.0.xsd <http://akka.io/akka.xsd>`_. The namespace for Akka is:
.. code-block:: xml
xmlns:akka="http://akka.io/schema/akka"
Example header for Akka Spring configuration:
.. code-block:: xml
<?xml version="1.0" encoding="UTF-8"?>
<beans xmlns="http://www.springframework.org/schema/beans"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns:akka="http://akka.io/schema/akka"
xsi:schemaLocation="
http://www.springframework.org/schema/beans
http://www.springframework.org/schema/beans/spring-beans-3.0.xsd
http://akka.io/schema/akka
http://akka.io/akka-1.0.xsd">
-
Actors
------
Actors in Java are created by extending the 'UntypedActor' class and implementing the 'onReceive' method.
Example how to create Actors with the Spring framework:
.. code-block:: xml
<akka:untyped-actor id="myActor"
implementation="com.biz.MyActor"
scope="singleton"
autostart="false"
depends-on="someBean"> <!-- or a comma-separated list of beans -->
<property name="aProperty" value="somePropertyValue"/>
<property name="aDependency" ref="someBeanOrActorDependency"/>
</akka:untyped-actor>
Supported scopes are singleton and prototype. Dependencies and properties are set with Springs ``<property/>`` element.
A dependency can be either a ``<akka:untyped-actor/>`` or a regular ``<bean/>``.
Get the Actor from the Spring context:
.. code-block:: java
ApplicationContext context = new ClassPathXmlApplicationContext("akka-spring-config.xml");
ActorRef actorRef = (ActorRef) context.getBean("myActor");
Typed Actors
------------
Here are some examples how to create Typed Actors with the Spring framework:
Creating a Typed Actor:
^^^^^^^^^^^^^^^^^^^^^^^
.. code-block:: xml
<beans>
<akka:typed-actor id="myActor"
interface="com.biz.MyPOJO"
implementation="com.biz.MyPOJOImpl"
transactional="true"
timeout="1000"
scope="singleton"
depends-on="someBean"> <!-- or a comma-separated list of beans -->
<property name="aProperty" value="somePropertyValue"/>
<property name="aDependency" ref="someBeanOrActorDependency"/>
</akka:typed-actor>
</beans>
Supported scopes are singleton and prototype. Dependencies and properties are set with Springs ``<property/>`` element.
A dependency can be either a ``<akka:typed-actor/>`` or a regular ``<bean/>``.
Get the Typed Actor from the Spring context:
.. code-block:: java
ApplicationContext context = new ClassPathXmlApplicationContext("akka-spring-config.xml");
MyPojo myPojo = (MyPojo) context.getBean("myActor");
Remote Actors
-------------
For details on server managed and client managed remote actors see Remote Actor documentation.
Configuration for a client managed remote Actor
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
::
<akka:untyped-actor id="remote-untyped-actor"
implementation="com.biz.MyActor"
timeout="2000">
<akka:remote host="localhost" port="9992" managed-by="client"/>
</akka:untyped-actor>
The default for 'managed-by' is "client", so in the above example it could be left out.
Configuration for a server managed remote Actor
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Server side
***********
::
<akka:untyped-actor id="server-managed-remote-untyped-actor"
implementation="com.biz.MyActor">
<akka:remote host="localhost" port="9990" managed-by="server"/>
</akka:untyped-actor>
<!-- register with custom service name -->
<akka:untyped-actor id="server-managed-remote-untyped-actor-custom-id"
implementation="com.biz.MyActor">
<akka:remote host="localhost" port="9990" service-name="my-service"/>
</akka:untyped-actor>
If the server specified by 'host' and 'port' does not exist it will not be registered.
Client side
***********
::
<!-- service-name could be custom name or class name -->
<akka:actor-for id="client-1" host="localhost" port="9990" service-name="my-service"/>
Configuration for a client managed remote Typed Actor
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
.. code-block:: xml
<akka:typed-actor id="remote-typed-actor"
interface="com.biz.MyPojo"
implementation="com.biz.MyPojoImpl"
timeout="2000">
<akka:remote host="localhost" port="9999" />
</akka:typed-actor>
Configuration for a server managed remote Typed Actor
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Sever side setup
****************
::
<akka:typed-actor id="server-managed-remote-typed-actor-custom-id"
interface="com.biz.IMyPojo"
implementation="com.biz.MyPojo"
timeout="2000">
<akka:remote host="localhost" port="9999" service-name="mypojo-service"/>
</akka:typed-actor>
Client side setup
*****************
::
<!-- always specify the interface for typed actor -->
<akka:actor-for id="typed-client"
interface="com.biz.MyPojo"
host="localhost"
port="9999"
service-name="mypojo-service"/>
Dispatchers
-----------
Configuration for a Typed Actor or Untyped Actor with a custom dispatcher
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
If you don't want to use the default dispatcher you can define your own dispatcher in the spring configuration. For more information on dispatchers have a look at Dispatchers documentation.
.. code-block:: xml
<akka:typed-actor id="remote-typed-actor"
interface="com.biz.MyPOJO"
implementation="com.biz.MyPOJOImpl"
timeout="2000">
<akka:dispatcher id="my-dispatcher" type="executor-based-event-driven" name="myDispatcher">
<akka:thread-pool queue="unbounded-linked-blocking-queue" capacity="100" />
</akka:dispatcher>
</akka:typed-actor>
<akka:untyped-actor id="untyped-actor-with-thread-based-dispatcher"
implementation="com.biz.MyActor">
<akka:dispatcher type="thread-based" name="threadBasedDispatcher"/>
</akka:untyped-actor>
If you want to or have to share the dispatcher between Actors you can define a dispatcher and reference it from the Typed Actor configuration:
.. code-block:: xml
<akka:dispatcher id="dispatcher-1"
type="executor-based-event-driven"
name="myDispatcher">
<akka:thread-pool queue="bounded-array-blocking-queue"
capacity="100"
fairness="true"
core-pool-size="1"
max-pool-size="20"
keep-alive="3000"
rejection-policy="caller-runs-policy"/>
</akka:dispatcher>
<akka:typed-actor id="typed-actor-with-dispatcher-ref"
interface="com.biz.MyPOJO"
implementation="com.biz.MyPOJOImpl"
timeout="1000">
<akka:dispatcher ref="dispatcher-1"/>
</akka:typed-actor>
The following dispatcher types are available in spring configuration:
* executor-based-event-driven
* executor-based-event-driven-work-stealing
* thread-based
The following queue types are configurable for dispatchers using thread pools:
* bounded-linked-blocking-queue
* unbounded-linked-blocking-queue
* synchronous-queue
* bounded-array-blocking-queue
If you have set up your IDE to be XSD-aware you can easily write your configuration through auto-completion.
Stopping Typed Actors and Untyped Actors
----------------------------------------
Actors with scope singleton are stopped when the application context is closed. Actors with scope prototype must be stopped by the application.
Supervisor Hierarchies
----------------------
The supervisor configuration in Spring follows the declarative configuration for the Java API. Have a look at Akka's approach to fault tolerance.
Example spring supervisor configuration
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
.. code-block:: xml
<beans>
<akka:supervision id="my-supervisor">
<akka:restart-strategy failover="AllForOne"
retries="3"
timerange="1000">
<akka:trap-exits>
<akka:trap-exit>java.io.IOException</akka:trap-exit>
</akka:trap-exits>
</akka:restart-strategy>
<akka:typed-actors>
<akka:typed-actor interface="com.biz.MyPOJO"
implementation="com.biz.MyPOJOImpl"
lifecycle="permanent"
timeout="1000"/>
<akka:typed-actor interface="com.biz.AnotherPOJO"
implementation="com.biz.AnotherPOJOImpl"
lifecycle="temporary"
timeout="1000"/>
<akka:typed-actor interface ="com.biz.FooBar"
implementation ="com.biz.FooBarImpl"
lifecycle="permanent"
transactional="true"
timeout="1000" />
</akka:typed-actors>
</akka:supervision>
<akka:supervision id="supervision-untyped-actors">
<akka:restart-strategy failover="AllForOne" retries="3" timerange="1000">
<akka:trap-exits>
<akka:trap-exit>java.io.IOException</akka:trap-exit>
<akka:trap-exit>java.lang.NullPointerException</akka:trap-exit>
</akka:trap-exits>
</akka:restart-strategy>
<akka:untyped-actors>
<akka:untyped-actor implementation="com.biz.PingActor"
lifecycle="permanent"/>
<akka:untyped-actor implementation="com.biz.PongActor"
lifecycle="permanent"/>
</akka:untyped-actors>
</akka:supervision>
</beans>
Get the TypedActorConfigurator from the Spring context
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
.. code-block:: java
TypedActorConfigurator myConfigurator = (TypedActorConfigurator) context.getBean("my-supervisor");
MyPojo myPojo = (MyPOJO) myConfigurator.getInstance(MyPojo.class);
Property Placeholders
---------------------
The Akka configuration can be made available as property placeholders by using a custom property placeholder configurer for Configgy:
::
<akka:property-placeholder location="akka.conf"/>
<akka:untyped-actor id="actor-1" implementation="com.biz.MyActor" timeout="${akka.actor.timeout}">
<akka:remote host="${akka.remote.server.hostname}" port="${akka.remote.server.port}"/>
</akka:untyped-actor>
Camel configuration
-------------------
For details refer to the :ref:`camel-module` documentation:
* CamelService configuration for :ref:`camel-spring-applications`
* Access to Typed Actors :ref:`camel-typed-actors-using-spring`

View file

@ -1,3 +1,5 @@
.. _configuration:
Configuration
=============

View file

@ -9,7 +9,8 @@ There is an Event Handler which takes the place of a logging system in Akka:
akka.event.EventHandler
You can configure which event handlers should be registered at boot time. That is done using the 'event-handlers' element in akka.conf. Here you can also define the log level.
You can configure which event handlers should be registered at boot time. That is done using the 'event-handlers' element in
the :ref:`configuration`. Here you can also define the log level.
.. code-block:: ruby

View file

@ -61,6 +61,8 @@ volatile variable rule. This means that you, the Akka user, do not need to worry
such a "happens before" relation, because it is the responsibility of Akka. So you have your hands free to deal with your
business logic, and the Akka framework makes sure that those rules are guaranteed on your behalf.
.. _jmm-shared-state:
Actors and shared mutable state
-------------------------------

View file

@ -14,7 +14,8 @@ also need a SLF4J backend, we recommend `Logback <http://logback.qos.ch/>`_:
Event Handler
-------------
This module includes a SLF4J Event Handler that works with Akka's standard Event Handler. You enabled it in the 'event-handlers' element in akka.conf. Here you can also define the log level.
This module includes a SLF4J Event Handler that works with Akka's standard Event Handler. You enabled it in the 'event-handlers' element in
the :ref:`configuration`. Here you can also define the log level.
.. code-block:: ruby

View file

@ -29,12 +29,12 @@ Actors as services
The simplest way you can use Akka is to use the actors as services in your Web
application. All thats needed to do that is to put the Akka charts as well as
its dependency jars into ``WEB-INF/lib``. You also need to put the ``akka.conf``
config file in the ``$AKKA_HOME/config`` directory. Now you can create your
its dependency jars into ``WEB-INF/lib``. You also need to put the :ref:`configuration`
file in the ``$AKKA_HOME/config`` directory. Now you can create your
Actors as regular services referenced from your Web application. You should also
be able to use the Remoting service, e.g. be able to make certain Actors remote
on other hosts. Please note that remoting service does not speak HTTP over port
80, but a custom protocol over the port is specified in ``akka.conf``.
80, but a custom protocol over the port is specified in :ref:`configuration`.
Using Akka as a stand alone microkernel

View file

@ -108,7 +108,6 @@ Akka is very modular and has many JARs for containing different features. The co
- ``akka-typed-actor-2.0-SNAPSHOT.jar`` -- Typed Actors
- ``akka-remote-2.0-SNAPSHOT.jar`` -- Remote Actors
- ``akka-stm-2.0-SNAPSHOT.jar`` -- STM (Software Transactional Memory), transactors and transactional datastructures
- ``akka-http-2.0-SNAPSHOT.jar`` -- Akka Mist for continuation-based asynchronous HTTP and also Jersey integration
- ``akka-slf4j-2.0-SNAPSHOT.jar`` -- SLF4J Event Handler Listener for logging with SLF4J
- ``akka-testkit-2.0-SNAPSHOT.jar`` -- Toolkit for testing Actors
@ -730,18 +729,12 @@ we compiled ourselves::
$ java \
-cp lib/scala-library.jar:lib/akka/akka-actor-2.0-SNAPSHOT.jar:tutorial \
akka.tutorial.java.first.Pi
AKKA_HOME is defined as [/Users/jboner/tools/akka-actors-2.0-SNAPSHOT]
loading config from [/Users/jboner/tools/akka-actors-2.0-SNAPSHOT/config/akka.conf].
Pi estimate: 3.1435501812459323
Calculation time: 822 millis
Yippee! It is working.
If you have not defined the ``AKKA_HOME`` environment variable then Akka can't
find the ``akka.conf`` configuration file and will print out a ``Cant load
akka.conf`` warning. This is ok since it will then just use the defaults.
Run it inside Maven
-------------------
@ -759,8 +752,6 @@ When this in done we can run our application directly inside Maven::
Yippee! It is working.
If you have not defined an the ``AKKA_HOME`` environment variable then Akka can't find the ``akka.conf`` configuration file and will print out a ``Cant load akka.conf`` warning. This is ok since it will then just use the defaults.
Conclusion
----------

View file

@ -93,7 +93,6 @@ Akka is very modular and has many JARs for containing different features. The co
- ``akka-typed-actor-2.0-SNAPSHOT.jar`` -- Typed Actors
- ``akka-remote-2.0-SNAPSHOT.jar`` -- Remote Actors
- ``akka-stm-2.0-SNAPSHOT.jar`` -- STM (Software Transactional Memory), transactors and transactional datastructures
- ``akka-http-2.0-SNAPSHOT.jar`` -- Akka Mist for continuation-based asynchronous HTTP and also Jersey integration
- ``akka-slf4j-2.0-SNAPSHOT.jar`` -- SLF4J Event Handler Listener for logging with SLF4J
- ``akka-testkit-2.0-SNAPSHOT.jar`` -- Toolkit for testing Actors
@ -383,15 +382,10 @@ Run it from Eclipse
Eclipse builds your project on every save when ``Project/Build Automatically`` is set. If not, bring you project up to date by clicking ``Project/Build Project``. If there are no compilation errors, you can right-click in the editor where ``Pi`` is defined, and choose ``Run as.. /Scala application``. If everything works fine, you should see::
AKKA_HOME is defined as [/Users/jboner/tools/akka-actors-2.0-SNAPSHOT]
loading config from [/Users/jboner/tools/akka-actors-2.0-SNAPSHOT/config/akka.conf].
Pi estimate: 3.1435501812459323
Calculation time: 858 millis
If you have not defined an the ``AKKA_HOME`` environment variable then Akka can't find the ``akka.conf`` configuration file and will print out a ``Cant load akka.conf`` warning. This is ok since it will then just use the defaults.
You can also define a new Run configuration, by going to ``Run/Run Configurations``. Create a new ``Scala application`` and choose the tutorial project and the main class to be ``akkatutorial.Pi``. You can pass additional command line arguments to the JVM on the ``Arguments`` page, for instance to define where ``akka.conf`` is:
You can also define a new Run configuration, by going to ``Run/Run Configurations``. Create a new ``Scala application`` and choose the tutorial project and the main class to be ``akkatutorial.Pi``. You can pass additional command line arguments to the JVM on the ``Arguments`` page, for instance to define where :ref:`configuration` is:
.. image:: ../images/run-config.png

View file

@ -114,7 +114,6 @@ core distribution has seven modules:
- ``akka-typed-actor-2.0-SNAPSHOT.jar`` -- Typed Actors
- ``akka-remote-2.0-SNAPSHOT.jar`` -- Remote Actors
- ``akka-stm-2.0-SNAPSHOT.jar`` -- STM (Software Transactional Memory), transactors and transactional datastructures
- ``akka-http-2.0-SNAPSHOT.jar`` -- Akka Mist for continuation-based asynchronous HTTP and also Jersey integration
- ``akka-slf4j-2.0-SNAPSHOT.jar`` -- SLF4J Event Handler Listener for logging with SLF4J
- ``akka-testkit-2.0-SNAPSHOT.jar`` -- Toolkit for testing Actors
@ -425,19 +424,12 @@ compiled ourselves::
$ java \
-cp lib/scala-library.jar:lib/akka/akka-actor-2.0-SNAPSHOT.jar:. \
akka.tutorial.first.scala.Pi
AKKA_HOME is defined as [/Users/jboner/tools/akka-actors-2.0-SNAPSHOT]
loading config from [/Users/jboner/tools/akka-actors-2.0-SNAPSHOT/config/akka.conf].
Pi estimate: 3.1435501812459323
Calculation time: 858 millis
Yippee! It is working.
If you have not defined the ``AKKA_HOME`` environment variable then Akka can't
find the ``akka.conf`` configuration file and will print out a ``Cant load
akka.conf`` warning. This is ok since it will then just use the defaults.
Run it inside SBT
=================
@ -457,11 +449,6 @@ When this in done we can run our application directly inside SBT::
Yippee! It is working.
If you have not defined an the ``AKKA_HOME`` environment variable then Akka
can't find the ``akka.conf`` configuration file and will print out a ``Cant
load akka.conf`` warning. This is ok since it will then just use the defaults.
Conclusion
==========

View file

@ -49,7 +49,6 @@ Akka is very modular and has many JARs for containing different features.
- ``akka-typed-actor-2.0-SNAPSHOT.jar`` -- Typed Actors
- ``akka-remote-2.0-SNAPSHOT.jar`` -- Remote Actors
- ``akka-stm-2.0-SNAPSHOT.jar`` -- STM (Software Transactional Memory), transactors and transactional datastructures
- ``akka-http-2.0-SNAPSHOT.jar`` -- Akka Mist for continuation-based asynchronous HTTP and also Jersey integration
- ``akka-slf4j-2.0-SNAPSHOT.jar`` -- SLF4J Event Handler Listener
- ``akka-testkit-2.0-SNAPSHOT.jar`` -- Toolkit for testing Actors
- ``akka-camel-2.0-SNAPSHOT.jar`` -- Apache Camel Actors integration (it's the best way to have your Akka application communicate with the rest of the world)

View file

@ -1,98 +0,0 @@
ActorRegistry (Java)
====================
Module stability: **SOLID**
ActorRegistry: Finding Actors
-----------------------------
Actors can be looked up using the 'akka.actor.Actors.registry()' object. Through this registry you can look up actors by:
* uuid com.eaio.uuid.UUID this uses the ``uuid`` field in the Actor class, returns the actor reference for the actor with specified uuid, if one exists, otherwise None
* id string this uses the ``id`` field in the Actor class, which can be set by the user (default is the class name), returns all actor references to actors with specified id
* parameterized type - returns a ``ActorRef[]`` with all actors that are a subtype of this specific type
* specific actor class - returns a ``ActorRef[]`` with all actors of this exact class
Actors are automatically registered in the ActorRegistry when they are started and removed when they are stopped. But you can explicitly register and unregister ActorRef's if you need to using the ``register`` and ``unregister`` methods.
Here is a summary of the API for finding actors:
.. code-block:: java
import static akka.actor.Actors.*;
Option<ActorRef> actor = registry().actorFor(uuid);
ActorRef[] actors = registry().actors();
ActorRef[] otherActors = registry().actorsFor(id);
ActorRef[] moreActors = registry().actorsFor(clazz);
You can shut down all Actors in the system by invoking:
.. code-block:: java
registry().shutdownAll();
If you want to know when a new Actor is added to or removed from the registry, you can use the subscription API on the registry. You can register an Actor that should be notified when an event happens in the ActorRegistry:
.. code-block:: java
void addListener(ActorRef listener);
void removeListener(ActorRef listener);
The messages sent to this Actor are:
.. code-block:: java
public class ActorRegistered {
ActorRef getActor();
String getAddress();
}
public class ActorUnregistered {
ActorRef actor();
String getAddress();
}
public class TypedActorRegistered {
ActorRef getActor();
String getAddress();
Object getProxy();
}
public class TypedActorUnregistered {
ActorRef actor();
String getAddress();
Object getProxy();
}
So your listener Actor needs to be able to handle these two messages. Example:
.. code-block:: java
import akka.actor.ActorRegistered;
import akka.actor.ActorUnregistered;
import akka.actor.TypedActorRegistered;
import akka.actor.TypedActorUnregistered;
import akka.actor.UntypedActor;
import akka.event.EventHandler;
public class RegistryListener extends UntypedActor {
public void onReceive(Object message) throws Exception {
if (message instanceof ActorRegistered) {
ActorRegistered event = (ActorRegistered) message;
EventHandler.info(this, String.format("Actor registered: %s - %s",
event.actor().actorClassName(), event.actor().getUuid()));
event.actor().actorClassName(), event.actor().getUuid()));
} else if (message instanceof ActorUnregistered) {
// ...
}
}
}
The above actor can be added as listener of registry events:
.. code-block:: java
import static akka.actor.Actors.*;
ActorRef listener = actorOf(RegistryListener.class);
registry().addListener(listener);

View file

@ -0,0 +1,17 @@
package akka.docs.actor;
import akka.actor.ActorRef;
import static akka.actor.Actors.*;
import akka.actor.UntypedActor;
//#context-actorOf
public class FirstUntypedActor extends UntypedActor {
ActorRef myActor = getContext().actorOf(MyActor.class);
//#context-actorOf
public void onReceive(Object message) {
myActor.forward(message, getContext());
myActor.tell(poisonPill());
}
}

View file

@ -0,0 +1,25 @@
package akka.docs.actor;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
//#immutable-message
public class ImmutableMessage {
private final int sequenceNumber;
private final List<String> values;
public ImmutableMessage(int sequenceNumber, List<String> values) {
this.sequenceNumber = sequenceNumber;
this.values = Collections.unmodifiableList(new ArrayList<String>(values));
}
public int getSequenceNumber() {
return sequenceNumber;
}
public List<String> getValues() {
return values;
}
}
//#immutable-message

View file

@ -0,0 +1,26 @@
package akka.docs.actor;
//#receive-timeout
import akka.actor.Actors;
import akka.actor.ReceiveTimeout;
import akka.actor.UnhandledMessageException;
import akka.actor.UntypedActor;
import akka.util.Duration;
public class MyReceivedTimeoutUntypedActor extends UntypedActor {
public MyReceivedTimeoutUntypedActor() {
getContext().setReceiveTimeout(Duration.parse("30 seconds"));
}
public void onReceive(Object message) {
if (message.equals("Hello")) {
getSender().tell("Hello world");
} else if (message == Actors.receiveTimeout()) {
throw new RuntimeException("received timeout");
} else {
throw new UnhandledMessageException(message, getSelf());
}
}
}
//#receive-timeout

View file

@ -0,0 +1,20 @@
package akka.docs.actor;
//#my-untyped-actor
import akka.actor.UntypedActor;
import akka.actor.UnhandledMessageException;
import akka.event.Logging;
import akka.event.LoggingAdapter;
public class MyUntypedActor extends UntypedActor {
LoggingAdapter log = Logging.getLogger(getContext().system(), this);
public void onReceive(Object message) throws Exception {
if (message instanceof String)
log.info("Received String message: {}", message);
else
throw new UnhandledMessageException(message, getSelf());
}
}
//#my-untyped-actor

View file

@ -0,0 +1,53 @@
package akka.docs.actor;
import static akka.docs.actor.UntypedActorSwapper.Swap.SWAP;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.UnhandledMessageException;
import akka.actor.UntypedActor;
import akka.event.Logging;
import akka.event.LoggingAdapter;
import akka.japi.Procedure;
//#swapper
public class UntypedActorSwapper {
public static class Swap {
public static Swap SWAP = new Swap();
private Swap() {
}
}
public static class Swapper extends UntypedActor {
LoggingAdapter log = Logging.getLogger(getContext().system(), this);
public void onReceive(Object message) {
if (message == SWAP) {
log.info("Hi");
getContext().become(new Procedure<Object>() {
@Override
public void apply(Object message) {
log.info("Ho");
getContext().unbecome(); // resets the latest 'become' (just for fun)
}
});
} else {
throw new UnhandledMessageException(message, getSelf());
}
}
}
public static void main(String... args) {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef swap = system.actorOf(Swapper.class);
swap.tell(SWAP); // logs Hi
swap.tell(SWAP); // logs Ho
swap.tell(SWAP); // logs Hi
swap.tell(SWAP); // logs Ho
swap.tell(SWAP); // logs Hi
swap.tell(SWAP); // logs Ho
}
}
//#swapper

View file

@ -0,0 +1,5 @@
package akka.docs.actor
import org.scalatest.junit.JUnitSuite
class UntypedActorTest extends UntypedActorTestBase with JUnitSuite

View file

@ -0,0 +1,233 @@
package akka.docs.actor;
//#imports
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
//#imports
//#import-future
import akka.dispatch.Future;
import akka.dispatch.Await;
import akka.util.Duration;
//#import-future
//#import-actors
import static akka.actor.Actors.*;
//#import-actors
//#import-procedure
import akka.japi.Procedure;
//#import-procedure
import akka.actor.Props;
import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory;
import akka.dispatch.MessageDispatcher;
import org.junit.Test;
import scala.Option;
import java.lang.Object;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.*;
public class UntypedActorTestBase {
@Test
public void systemActorOf() {
//#system-actorOf
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(MyUntypedActor.class);
//#system-actorOf
myActor.tell("test");
system.stop();
}
@Test
public void contextActorOf() {
//#context-actorOf
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(MyUntypedActor.class);
//#context-actorOf
myActor.tell("test");
system.stop();
}
@Test
public void constructorActorOf() {
ActorSystem system = ActorSystem.create("MySystem");
//#creating-constructor
// allows passing in arguments to the MyActor constructor
ActorRef myActor = system.actorOf(new UntypedActorFactory() {
public UntypedActor create() {
return new MyActor("...");
}
});
//#creating-constructor
myActor.tell("test");
system.stop();
}
@Test
public void propsActorOf() {
ActorSystem system = ActorSystem.create("MySystem");
//#creating-props
MessageDispatcher dispatcher = system.dispatcherFactory().newFromConfig("my-dispatcher");
ActorRef myActor = system.actorOf(new Props().withCreator(MyUntypedActor.class).withDispatcher(dispatcher),
"myactor");
//#creating-props
myActor.tell("test");
system.stop();
}
@Test
public void usingAsk() {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(new UntypedActorFactory() {
public UntypedActor create() {
return new MyAskActor();
}
});
//#using-ask
Future<Object> future = myActor.ask("Hello", 1000);
Object result = Await.result(future, Duration.create(1, TimeUnit.SECONDS));
//#using-ask
system.stop();
}
@Test
public void receiveTimeout() {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(MyReceivedTimeoutUntypedActor.class);
myActor.tell("Hello");
system.stop();
}
@Test
public void usePoisonPill() {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(MyUntypedActor.class);
//#poison-pill
myActor.tell(poisonPill());
//#poison-pill
system.stop();
}
@Test
public void useKill() {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef victim = system.actorOf(MyUntypedActor.class);
//#kill
victim.tell(kill());
//#kill
system.stop();
}
@Test
public void useBecome() {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(new UntypedActorFactory() {
public UntypedActor create() {
return new HotSwapActor();
}
});
myActor.tell("foo");
myActor.tell("bar");
myActor.tell("bar");
system.stop();
}
public static class MyActor extends UntypedActor {
public MyActor(String s) {
}
public void onReceive(Object message) throws Exception {
try {
operation();
} catch (Exception e) {
getSender().tell(new akka.actor.Status.Failure(e));
throw e;
}
}
private void operation() {
}
//#lifecycle-callbacks
public void preStart() {
}
public void preRestart(Throwable reason, Option<Object> message) {
postStop();
}
public void postRestart(Throwable reason) {
preStart();
}
public void postStop() {
}
//#lifecycle-callbacks
}
public static class MyAskActor extends UntypedActor {
public void onReceive(Object message) throws Exception {
//#reply-exception
try {
String result = operation();
getSender().tell(result);
} catch (Exception e) {
getSender().tell(new akka.actor.Status.Failure(e));
throw e;
}
//#reply-exception
}
private String operation() {
return "Hi";
}
}
//#hot-swap-actor
public static class HotSwapActor extends UntypedActor {
Procedure<Object> angry = new Procedure<Object>() {
@Override
public void apply(Object message) {
if (message.equals("foo")) {
getSender().tell("I am already angry?");
} else if (message.equals("foo")) {
getContext().become(happy);
}
}
};
Procedure<Object> happy = new Procedure<Object>() {
@Override
public void apply(Object message) {
if (message.equals("bar")) {
getSender().tell("I am already happy :-)");
} else if (message.equals("foo")) {
getContext().become(angry);
}
}
};
public void onReceive(Object message) {
if (message.equals("bar")) {
getContext().become(angry);
} else if (message.equals("foo")) {
getContext().become(happy);
}
}
}
//#hot-swap-actor
}

View file

@ -19,7 +19,7 @@ Default dispatcher
------------------
For most scenarios the default settings are the best. Here we have one single event-based dispatcher for all Actors created. The default dispatcher used is "GlobalDispatcher" which also is retrievable in ``akka.dispatch.Dispatchers.globalDispatcher``.
The Dispatcher specified in the akka.conf as "default-dispatcher" is as ``Dispatchers.defaultGlobalDispatcher``.
The Dispatcher specified in the :ref:`configuration` as "default-dispatcher" is as ``Dispatchers.defaultGlobalDispatcher``.
The "GlobalDispatcher" is not configurable but will use default parameters given by Akka itself.
@ -124,16 +124,13 @@ Here is an example:
...
}
This 'Dispatcher' allows you to define the 'throughput' it should have. This defines the number of messages for a specific Actor the dispatcher should process in one single sweep.
Setting this to a higher number will increase throughput but lower fairness, and vice versa. If you don't specify it explicitly then it uses the default value defined in the 'akka.conf' configuration file:
.. code-block:: xml
actor {
throughput = 5
}
If you don't define a the 'throughput' option in the configuration file then the default value of '5' will be used.
The standard :class:`Dispatcher` allows you to define the ``throughput`` it
should have, as shown above. This defines the number of messages for a specific
Actor the dispatcher should process in one single sweep; in other words, the
dispatcher will bunch up to ``throughput`` message invocations together when
having elected an actor to run. Setting this to a higher number will increase
throughput but lower fairness, and vice versa. If you don't specify it explicitly
then it uses the value (5) defined for ``default-dispatcher`` in the :ref:`configuration`.
Browse the :ref:`scaladoc` or look at the code for all the options available.

View file

@ -42,7 +42,7 @@ A common use case within Akka is to have some computation performed concurrently
return "Hello" + "World!";
}
});
String result = f.get(); //Blocks until timeout, default timeout is set in akka.conf, otherwise 5 seconds
String result = f.get(); //Blocks until timeout, default timeout is set in :ref:`configuration`, otherwise 5 seconds
In the above code the block passed to ``future`` will be executed by the default ``Dispatcher``, with the return value of the block used to complete the ``Future`` (in this case, the result would be the string: "HelloWorld"). Unlike a ``Future`` that is returned from an ``UntypedActor``, this ``Future`` is properly typed, and we also avoid the overhead of managing an ``UntypedActor``.

View file

@ -8,7 +8,6 @@ Java API
untyped-actors
typed-actors
actor-registry
futures
dataflow
stm

View file

@ -3,529 +3,4 @@
Software Transactional Memory (Java)
====================================
.. sidebar:: Contents
.. contents:: :local:
Module stability: **SOLID**
Overview of STM
---------------
An `STM <http://en.wikipedia.org/wiki/Software_transactional_memory>`_ turns the Java heap into a transactional data set with begin/commit/rollback semantics. Very much like a regular database. It implements the first three letters in ACID; ACI:
* (failure) Atomicity: all changes during the execution of a transaction make it, or none make it. This only counts for transactional datastructures.
* Consistency: a transaction gets a consistent of reality (in Akka you get the Oracle version of the SERIALIZED isolation level).
* Isolated: changes made by concurrent execution transactions are not visible to each other.
Generally, the STM is not needed that often when working with Akka. Some use-cases (that we can think of) are:
- When you really need composable message flows across many actors updating their **internal local** state but need them to do that atomically in one big transaction. Might not often, but when you do need this then you are screwed without it.
- When you want to share a datastructure across actors.
- When you need to use the persistence modules.
Akkas STM implements the concept in `Clojures <http://clojure.org/>`_ STM view on state in general. Please take the time to read `this excellent document <http://clojure.org/state>`_ and view `this presentation <http://www.infoq.com/presentations/Value-Identity-State-Rich-Hickey>`_ by Rich Hickey (the genius behind Clojure), since it forms the basis of Akkas view on STM and state in general.
The STM is based on Transactional References (referred to as Refs). Refs are memory cells, holding an (arbitrary) immutable value, that implement CAS (Compare-And-Swap) semantics and are managed and enforced by the STM for coordinated changes across many Refs. They are implemented using the excellent `Multiverse STM <http://multiverse.codehaus.org/overview.html>`_.
Working with immutable collections can sometimes give bad performance due to extensive copying. Scala provides so-called persistent datastructures which makes working with immutable collections fast. They are immutable but with constant time access and modification. The use of structural sharing and an insert or update does not ruin the old structure, hence “persistent”. Makes working with immutable composite types fast. The persistent datastructures currently consist of a Map and Vector.
Simple example
--------------
Here is a simple example of an incremental counter using STM. This shows creating a ``Ref``, a transactional reference, and then modifying it within a transaction, which is delimited by an ``Atomic`` anonymous inner class.
.. code-block:: java
import akka.stm.*;
final Ref<Integer> ref = new Ref<Integer>(0);
public int counter() {
return new Atomic<Integer>() {
public Integer atomically() {
int inc = ref.get() + 1;
ref.set(inc);
return inc;
}
}.execute();
}
counter();
// -> 1
counter();
// -> 2
Ref
---
Refs (transactional references) are mutable references to values and through the STM allow the safe sharing of mutable data. To ensure safety the value stored in a Ref should be immutable. The value referenced by a Ref can only be accessed or swapped within a transaction. Refs separate identity from value.
Creating a Ref
^^^^^^^^^^^^^^
You can create a Ref with or without an initial value.
.. code-block:: java
import akka.stm.*;
// giving an initial value
final Ref<Integer> ref = new Ref<Integer>(0);
// specifying a type but no initial value
final Ref<Integer> ref = new Ref<Integer>();
Accessing the value of a Ref
^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Use ``get`` to access the value of a Ref. Note that if no initial value has been given then the value is initially ``null``.
.. code-block:: java
import akka.stm.*;
final Ref<Integer> ref = new Ref<Integer>(0);
Integer value = new Atomic<Integer>() {
public Integer atomically() {
return ref.get();
}
}.execute();
// -> value = 0
Changing the value of a Ref
^^^^^^^^^^^^^^^^^^^^^^^^^^^
To set a new value for a Ref you can use ``set`` (or equivalently ``swap``), which sets the new value and returns the old value.
.. code-block:: java
import akka.stm.*;
final Ref<Integer> ref = new Ref<Integer>(0);
new Atomic() {
public Object atomically() {
return ref.set(5);
}
}.execute();
Transactions
------------
A transaction is delimited using an ``Atomic`` anonymous inner class.
.. code-block:: java
new Atomic() {
public Object atomically() {
// ...
}
}.execute();
All changes made to transactional objects are isolated from other changes, all make it or non make it (so failure atomicity) and are consistent. With the AkkaSTM you automatically have the Oracle version of the SERIALIZED isolation level, lower isolation is not possible. To make it fully serialized, set the writeskew property that checks if a writeskew problem is allowed to happen.
Retries
^^^^^^^
A transaction is automatically retried when it runs into some read or write conflict, until the operation completes, an exception (throwable) is thrown or when there are too many retries. When a read or writeconflict is encountered, the transaction uses a bounded exponential backoff to prevent cause more contention and give other transactions some room to complete.
If you are using non transactional resources in an atomic block, there could be problems because a transaction can be retried. If you are using print statements or logging, it could be that they are called more than once. So you need to be prepared to deal with this. One of the possible solutions is to work with a deferred or compensating task that is executed after the transaction aborts or commits.
Unexpected retries
^^^^^^^^^^^^^^^^^^
It can happen for the first few executions that you get a few failures of execution that lead to unexpected retries, even though there is not any read or writeconflict. The cause of this is that speculative transaction configuration/selection is used. There are transactions optimized for a single transactional object, for 1..n and for n to unlimited. So based on the execution of the transaction, the system learns; it begins with a cheap one and upgrades to more expensive ones. Once it has learned, it will reuse this knowledge. It can be activated/deactivated using the speculative property on the TransactionFactoryBuilder. In most cases it is best use the default value (enabled) so you get more out of performance.
Coordinated transactions and Transactors
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
If you need coordinated transactions across actors or threads then see :ref:`transactors-java`.
Configuring transactions
^^^^^^^^^^^^^^^^^^^^^^^^
It's possible to configure transactions. The ``Atomic`` class can take a ``TransactionFactory``, which can determine properties of the transaction. A default transaction factory is used if none is specified. You can create a ``TransactionFactory`` with a ``TransactionFactoryBuilder``.
Configuring transactions with a ``TransactionFactory``:
.. code-block:: java
import akka.stm.*;
TransactionFactory txFactory = new TransactionFactoryBuilder()
.setReadonly(true)
.build();
new Atomic<Object>(txFactory) {
public Object atomically() {
// read only transaction
return ...;
}
}.execute();
The following settings are possible on a TransactionFactory:
- familyName - Family name for transactions. Useful for debugging because the familyName is shown in exceptions, logging and in the future also will be used for profiling.
- readonly - Sets transaction as readonly. Readonly transactions are cheaper and can be used to prevent modification to transactional objects.
- maxRetries - The maximum number of times a transaction will retry.
- timeout - The maximum time a transaction will block for.
- trackReads - Whether all reads should be tracked. Needed for blocking operations. Readtracking makes a transaction more expensive, but makes subsequent reads cheaper and also lowers the chance of a readconflict.
- writeSkew - Whether writeskew is allowed. Disable with care.
- blockingAllowed - Whether explicit retries are allowed.
- interruptible - Whether a blocking transaction can be interrupted if it is blocked.
- speculative - Whether speculative configuration should be enabled.
- quickRelease - Whether locks should be released as quickly as possible (before whole commit).
- propagation - For controlling how nested transactions behave.
- traceLevel - Transaction trace level.
You can also specify the default values for some of these options in akka.conf. Here they are with their default values:
::
stm {
fair = on # Should global transactions be fair or non-fair (non fair yield better performance)
max-retries = 1000
timeout = 5 # Default timeout for blocking transactions and transaction set (in unit defined by
# the time-unit property)
write-skew = true
blocking-allowed = false
interruptible = false
speculative = true
quick-release = true
propagation = "requires"
trace-level = "none"
}
Transaction lifecycle listeners
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
It's possible to have code that will only run on the successful commit of a transaction, or when a transaction aborts. You can do this by adding ``deferred`` or ``compensating`` blocks to a transaction.
.. code-block:: java
import akka.stm.*;
import static akka.stm.StmUtils.deferred;
import static akka.stm.StmUtils.compensating;
new Atomic() {
public Object atomically() {
deferred(new Runnable() {
public void run() {
// executes when transaction commits
}
});
compensating(new Runnable() {
public void run() {
// executes when transaction aborts
}
});
// ...
return something;
}
}.execute();
Blocking transactions
^^^^^^^^^^^^^^^^^^^^^
You can block in a transaction until a condition is met by using an explicit ``retry``. To use ``retry`` you also need to configure the transaction to allow explicit retries.
Here is an example of using ``retry`` to block until an account has enough money for a withdrawal. This is also an example of using actors and STM together.
.. code-block:: java
import akka.stm.*;
public class Transfer {
private final Ref<Double> from;
private final Ref<Double> to;
private final double amount;
public Transfer(Ref<Double> from, Ref<Double> to, double amount) {
this.from = from;
this.to = to;
this.amount = amount;
}
public Ref<Double> getFrom() { return from; }
public Ref<Double> getTo() { return to; }
public double getAmount() { return amount; }
}
.. code-block:: java
import akka.stm.*;
import static akka.stm.StmUtils.retry;
import akka.actor.*;
import akka.util.FiniteDuration;
import java.util.concurrent.TimeUnit;
import akka.event.EventHandler;
public class Transferer extends UntypedActor {
TransactionFactory txFactory = new TransactionFactoryBuilder()
.setBlockingAllowed(true)
.setTrackReads(true)
.setTimeout(new FiniteDuration(60, TimeUnit.SECONDS))
.build();
public void onReceive(Object message) throws Exception {
if (message instanceof Transfer) {
Transfer transfer = (Transfer) message;
final Ref<Double> from = transfer.getFrom();
final Ref<Double> to = transfer.getTo();
final double amount = transfer.getAmount();
new Atomic(txFactory) {
public Object atomically() {
if (from.get() < amount) {
EventHandler.info(this, "not enough money - retrying");
retry();
}
EventHandler.info(this, "transferring");
from.set(from.get() - amount);
to.set(to.get() + amount);
return null;
}
}.execute();
}
}
}
.. code-block:: java
import akka.stm.*;
import akka.actor.*;
public class Main {
public static void main(String...args) throws Exception {
final Ref<Double> account1 = new Ref<Double>(100.0);
final Ref<Double> account2 = new Ref<Double>(100.0);
ActorRef transferer = Actors.actorOf(Transferer.class);
transferer.tell(new Transfer(account1, account2, 500.0));
// Transferer: not enough money - retrying
new Atomic() {
public Object atomically() {
return account1.set(account1.get() + 2000);
}
}.execute();
// Transferer: transferring
Thread.sleep(1000);
Double acc1 = new Atomic<Double>() {
public Double atomically() {
return account1.get();
}
}.execute();
Double acc2 = new Atomic<Double>() {
public Double atomically() {
return account2.get();
}
}.execute();
System.out.println("Account 1: " + acc1);
// Account 1: 1600.0
System.out.println("Account 2: " + acc2);
// Account 2: 600.0
transferer.stop();
}
}
Alternative blocking transactions
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
You can also have two alternative blocking transactions, one of which can succeed first, with ``EitherOrElse``.
.. code-block:: java
import akka.stm.*;
public class Branch {
private final Ref<Integer> left;
private final Ref<Integer> right;
private final double amount;
public Branch(Ref<Integer> left, Ref<Integer> right, int amount) {
this.left = left;
this.right = right;
this.amount = amount;
}
public Ref<Integer> getLeft() { return left; }
public Ref<Integer> getRight() { return right; }
public double getAmount() { return amount; }
}
.. code-block:: java
import akka.actor.*;
import akka.stm.*;
import static akka.stm.StmUtils.retry;
import akka.util.FiniteDuration;
import java.util.concurrent.TimeUnit;
import akka.event.EventHandler;
public class Brancher extends UntypedActor {
TransactionFactory txFactory = new TransactionFactoryBuilder()
.setBlockingAllowed(true)
.setTrackReads(true)
.setTimeout(new FiniteDuration(60, TimeUnit.SECONDS))
.build();
public void onReceive(Object message) throws Exception {
if (message instanceof Branch) {
Branch branch = (Branch) message;
final Ref<Integer> left = branch.getLeft();
final Ref<Integer> right = branch.getRight();
final double amount = branch.getAmount();
new Atomic<Integer>(txFactory) {
public Integer atomically() {
return new EitherOrElse<Integer>() {
public Integer either() {
if (left.get() < amount) {
EventHandler.info(this, "not enough on left - retrying");
retry();
}
EventHandler.info(this, "going left");
return left.get();
}
public Integer orElse() {
if (right.get() < amount) {
EventHandler.info(this, "not enough on right - retrying");
retry();
}
EventHandler.info(this, "going right");
return right.get();
}
}.execute();
}
}.execute();
}
}
}
.. code-block:: java
import akka.stm.*;
import akka.actor.*;
public class Main2 {
public static void main(String...args) throws Exception {
final Ref<Integer> left = new Ref<Integer>(100);
final Ref<Integer> right = new Ref<Integer>(100);
ActorRef brancher = Actors.actorOf(Brancher.class);
brancher.tell(new Branch(left, right, 500));
// not enough on left - retrying
// not enough on right - retrying
Thread.sleep(1000);
new Atomic() {
public Object atomically() {
return right.set(right.get() + 1000);
}
}.execute();
// going right
brancher.stop();
}
}
Transactional datastructures
----------------------------
Akka provides two datastructures that are managed by the STM.
- TransactionalMap
- TransactionalVector
TransactionalMap and TransactionalVector look like regular mutable datastructures, they even implement the standard Scala 'Map' and 'RandomAccessSeq' interfaces, but they are implemented using persistent datastructures and managed references under the hood. Therefore they are safe to use in a concurrent environment. Underlying TransactionalMap is HashMap, an immutable Map but with near constant time access and modification operations. Similarly TransactionalVector uses a persistent Vector. See the Persistent Datastructures section below for more details.
Like managed references, TransactionalMap and TransactionalVector can only be modified inside the scope of an STM transaction.
Here is an example of creating and accessing a TransactionalMap:
.. code-block:: java
import akka.stm.*;
// assuming a User class
final TransactionalMap<String, User> users = new TransactionalMap<String, User>();
// fill users map (in a transaction)
new Atomic() {
public Object atomically() {
users.put("bill", new User("bill"));
users.put("mary", new User("mary"));
users.put("john", new User("john"));
return null;
}
}.execute();
// access users map (in a transaction)
User user = new Atomic<User>() {
public User atomically() {
return users.get("bill").get();
}
}.execute();
Here is an example of creating and accessing a TransactionalVector:
.. code-block:: java
import akka.stm.*;
// assuming an Address class
final TransactionalVector<Address> addresses = new TransactionalVector<Address>();
// fill addresses vector (in a transaction)
new Atomic() {
public Object atomically() {
addresses.add(new Address("somewhere"));
addresses.add(new Address("somewhere else"));
return null;
}
}.execute();
// access addresses vector (in a transaction)
Address address = new Atomic<Address>() {
public Address atomically() {
return addresses.get(0);
}
}.execute();
Persistent datastructures
-------------------------
Akka's STM should only be used with immutable data. This can be costly if you have large datastructures and are using a naive copy-on-write. In order to make working with immutable datastructures fast enough Scala provides what are called Persistent Datastructures. There are currently two different ones:
- HashMap (`scaladoc <http://www.scala-lang.org/api/current/scala/collection/immutable/HashMap.html>`__)
- Vector (`scaladoc <http://www.scala-lang.org/api/current/scala/collection/immutable/Vector.html>`__)
They are immutable and each update creates a completely new version but they are using clever structural sharing in order to make them almost as fast, for both read and update, as regular mutable datastructures.
This illustration is taken from Rich Hickey's presentation. Copyright Rich Hickey 2009.
.. image:: ../images/clojure-trees.png
Documentation of Akka STM has not been migrated to Akka 2.0-SNAPSHOT yet.

View file

@ -3,269 +3,4 @@
Transactors (Java)
==================
.. sidebar:: Contents
.. contents:: :local:
Module stability: **SOLID**
Why Transactors?
----------------
Actors are excellent for solving problems where you have many independent processes that can work in isolation and only interact with other Actors through message passing. This model fits many problems. But the actor model is unfortunately a terrible model for implementing truly shared state. E.g. when you need to have consensus and a stable view of state across many components. The classic example is the bank account where clients can deposit and withdraw, in which each operation needs to be atomic. For detailed discussion on the topic see `this JavaOne presentation <http://www.slideshare.net/jboner/state-youre-doing-it-wrong-javaone-2009>`_.
**STM** on the other hand is excellent for problems where you need consensus and a stable view of the state by providing compositional transactional shared state. Some of the really nice traits of STM are that transactions compose, and it raises the abstraction level from lock-based concurrency.
Akka's Transactors combine Actors and STM to provide the best of the Actor model (concurrency and asynchronous event-based programming) and STM (compositional transactional shared state) by providing transactional, compositional, asynchronous, event-based message flows.
If you need Durability then you should not use one of the in-memory data structures but one of the persistent ones.
Generally, the STM is not needed very often when working with Akka. Some use-cases (that we can think of) are:
- When you really need composable message flows across many actors updating their **internal local** state but need them to do that atomically in one big transaction. Might not often, but when you do need this then you are screwed without it.
- When you want to share a datastructure across actors.
- When you need to use the persistence modules.
Actors and STM
^^^^^^^^^^^^^^
You can combine Actors and STM in several ways. An Actor may use STM internally so that particular changes are guaranteed to be atomic. Actors may also share transactional datastructures as the STM provides safe shared state across threads.
It's also possible to coordinate transactions across Actors or threads so that either the transactions in a set all commit successfully or they all fail. This is the focus of Transactors and the explicit support for coordinated transactions in this section.
Coordinated transactions
------------------------
Akka provides an explicit mechanism for coordinating transactions across actors. Under the hood it uses a ``CountDownCommitBarrier``, similar to a CountDownLatch.
Here is an example of coordinating two simple counter UntypedActors so that they both increment together in coordinated transactions. If one of them was to fail to increment, the other would also fail.
.. code-block:: java
import akka.actor.ActorRef;
public class Increment {
private final ActorRef friend;
public Increment() {
this.friend = null;
}
public Increment(ActorRef friend) {
this.friend = friend;
}
public boolean hasFriend() {
return friend != null;
}
public ActorRef getFriend() {
return friend;
}
}
.. code-block:: java
import akka.actor.UntypedActor;
import akka.stm.Ref;
import akka.transactor.Atomically;
import akka.transactor.Coordinated;
public class Counter extends UntypedActor {
private Ref<Integer> count = new Ref(0);
private void increment() {
count.set(count.get() + 1);
}
public void onReceive(Object incoming) throws Exception {
if (incoming instanceof Coordinated) {
Coordinated coordinated = (Coordinated) incoming;
Object message = coordinated.getMessage();
if (message instanceof Increment) {
Increment increment = (Increment) message;
if (increment.hasFriend()) {
increment.getFriend().tell(coordinated.coordinate(new Increment()));
}
coordinated.atomic(new Atomically() {
public void atomically() {
increment();
}
});
}
} else if (incoming.equals("GetCount")) {
getContext().reply(count.get());
}
}
}
.. code-block:: java
ActorRef counter1 = actorOf(Counter.class);
ActorRef counter2 = actorOf(Counter.class);
counter1.tell(new Coordinated(new Increment(counter2)));
To start a new coordinated transaction that you will also participate in, just create a ``Coordinated`` object:
.. code-block:: java
Coordinated coordinated = new Coordinated();
To start a coordinated transaction that you won't participate in yourself you can create a ``Coordinated`` object with a message and send it directly to an actor. The recipient of the message will be the first member of the coordination set:
.. code-block:: java
actor.tell(new Coordinated(new Message()));
To include another actor in the same coordinated transaction that you've created or received, use the ``coordinate`` method on that object. This will increment the number of parties involved by one and create a new ``Coordinated`` object to be sent.
.. code-block:: java
actor.tell(coordinated.coordinate(new Message()));
To enter the coordinated transaction use the atomic method of the coordinated object. This accepts either an ``akka.transactor.Atomically`` object, or an ``Atomic`` object the same as used normally in the STM (just don't execute it - the coordination will do that).
.. code-block:: java
coordinated.atomic(new Atomically() {
public void atomically() {
// do something in a transaction
}
});
The coordinated transaction will wait for the other transactions before committing. If any of the coordinated transactions fail then they all fail.
UntypedTransactor
-----------------
UntypedTransactors are untyped actors that provide a general pattern for coordinating transactions, using the explicit coordination described above.
Here's an example of a simple untyped transactor that will join a coordinated transaction:
.. code-block:: java
import akka.transactor.UntypedTransactor;
import akka.stm.Ref;
public class Counter extends UntypedTransactor {
Ref<Integer> count = new Ref<Integer>(0);
@Override
public void atomically(Object message) {
if (message instanceof Increment) {
count.set(count.get() + 1);
}
}
}
You could send this Counter transactor a ``Coordinated(Increment)`` message. If you were to send it just an ``Increment`` message it will create its own ``Coordinated`` (but in this particular case wouldn't be coordinating transactions with any other transactors).
To coordinate with other transactors override the ``coordinate`` method. The ``coordinate`` method maps a message to a set of ``SendTo`` objects, pairs of ``ActorRef`` and a message. You can use the ``include`` and ``sendTo`` methods to easily coordinate with other transactors.
Example of coordinating an increment, similar to the explicitly coordinated example:
.. code-block:: java
import akka.transactor.UntypedTransactor;
import akka.transactor.SendTo;
import akka.stm.Ref;
import java.util.Set;
public class Counter extends UntypedTransactor {
Ref<Integer> count = new Ref<Integer>(0);
@Override
public Set<SendTo> coordinate(Object message) {
if (message instanceof Increment) {
Increment increment = (Increment) message;
if (increment.hasFriend())
return include(increment.getFriend(), new Increment());
}
return nobody();
}
@Override
public void atomically(Object message) {
if (message instanceof Increment) {
count.set(count.get() + 1);
}
}
}
To execute directly before or after the coordinated transaction, override the ``before`` and ``after`` methods. They do not execute within the transaction.
To completely bypass coordinated transactions override the ``normally`` method. Any message matched by ``normally`` will not be matched by the other methods, and will not be involved in coordinated transactions. In this method you can implement normal actor behavior, or use the normal STM atomic for local transactions.
Coordinating Typed Actors
-------------------------
It's also possible to use coordinated transactions with typed actors. You can explicitly pass around ``Coordinated`` objects, or use built-in support with the ``@Coordinated`` annotation and the ``Coordination.coordinate`` method.
To specify a method should use coordinated transactions add the ``@Coordinated`` annotation. **Note**: the ``@Coordinated`` annotation will only work with void (one-way) methods.
.. code-block:: java
public interface Counter {
@Coordinated public void increment();
public Integer get();
}
To coordinate transactions use a ``coordinate`` block. This accepts either an ``akka.transactor.Atomically`` object, or an ``Atomic`` object liked used in the STM (but don't execute it). The first boolean parameter specifies whether or not to wait for the transactions to complete.
.. code-block:: java
Coordination.coordinate(true, new Atomically() {
public void atomically() {
counter1.increment();
counter2.increment();
}
});
Here's an example of using ``@Coordinated`` with a TypedActor to coordinate increments:
.. code-block:: java
import akka.transactor.annotation.Coordinated;
public interface Counter {
@Coordinated public void increment();
public Integer get();
}
.. code-block:: java
import akka.actor.TypedActor;
import akka.stm.Ref;
public class CounterImpl extends TypedActor implements Counter {
private Ref<Integer> count = new Ref<Integer>(0);
public void increment() {
count.set(count.get() + 1);
}
public Integer get() {
return count.get();
}
}
.. code-block:: java
Counter counter1 = (Counter) TypedActor.newInstance(Counter.class, CounterImpl.class);
Counter counter2 = (Counter) TypedActor.newInstance(Counter.class, CounterImpl.class);
Coordination.coordinate(true, new Atomically() {
public void atomically() {
counter1.increment();
counter2.increment();
}
});
TypedActor.stop(counter1);
TypedActor.stop(counter2);
Documentation of Akka Transactors has not been migrated to Akka 2.0-SNAPSHOT yet.

View file

@ -185,7 +185,7 @@ Messages and immutability
**IMPORTANT**: Messages can be any kind of object but have to be immutable (there is a workaround, see next section). Java or Scala cant enforce immutability (yet) so this has to be by convention. Primitives like String, int, Long are always immutable. Apart from these you have to create your own immutable objects to send as messages. If you pass on a reference to an instance that is mutable then this instance can be modified concurrently by two different Typed Actors and the Actor model is broken leaving you with NO guarantees and most likely corrupt data.
Akka can help you in this regard. It allows you to turn on an option for serializing all messages, e.g. all parameters to the Typed Actor effectively making a deep clone/copy of the parameters. This will make sending mutable messages completely safe. This option is turned on in the $AKKA_HOME/config/akka.conf config file like this:
Akka can help you in this regard. It allows you to turn on an option for serializing all messages, e.g. all parameters to the Typed Actor effectively making a deep clone/copy of the parameters. This will make sending mutable messages completely safe. This option is turned on in the :ref:`configuration` file like this:
.. code-block:: ruby

View file

@ -1,370 +1,470 @@
.. _untyped-actors-java:
Actors (Java)
=============
################
Actors (Java)
################
.. sidebar:: Contents
.. contents:: :local:
Module stability: **SOLID**
The `Actor Model <http://en.wikipedia.org/wiki/Actor_model>`_ provides a higher level of abstraction for writing concurrent and distributed systems. It alleviates the developer from having to deal with explicit locking and thread management, making it easier to write correct concurrent and parallel systems. Actors were defined in the 1973 paper by Carl Hewitt but have been popularized by the Erlang language, and used for example at Ericsson with great success to build highly concurrent and reliable telecom systems.
The `Actor Model`_ provides a higher level of abstraction for writing concurrent
and distributed systems. It alleviates the developer from having to deal with
explicit locking and thread management, making it easier to write correct
concurrent and parallel systems. Actors were defined in the 1973 paper by Carl
Hewitt but have been popularized by the Erlang language, and used for example at
Ericsson with great success to build highly concurrent and reliable telecom
systems.
The API of Akkas Actors is similar to Scala Actors which has borrowed some of
its syntax from Erlang.
.. _Actor Model: http://en.wikipedia.org/wiki/Actor_model
Creating Actors
===============
Defining an Actor class
-----------------------
Actors in Java are created either by extending the 'UntypedActor' class and implementing the 'onReceive' method. This method takes the message as a parameter.
Actor in Java are implemented by extending the ``UntypedActor`` class and implementing the
:meth:`onReceive` method. This method takes the message as a parameter.
Here is an example:
.. includecode:: code/akka/docs/actor/MyUntypedActor.java#my-untyped-actor
Creating Actors with default constructor
----------------------------------------
.. includecode:: code/akka/docs/actor/UntypedActorTestBase.java
:include: imports,system-actorOf
The call to :meth:`actorOf` returns an instance of ``ActorRef``. This is a handle to
the ``UntypedActor`` instance which you can use to interact with the ``UntypedActor``. The
``ActorRef`` is immutable and has a one to one relationship with the Actor it
represents. The ``ActorRef`` is also serializable and network-aware. This means
that you can serialize it, send it over the wire and use it on a remote host and
it will still be representing the same Actor on the original node, across the
network.
In the above example the actor was created from the system. It is also possible
to create actors from other actors with the actor ``context``. 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).
.. includecode:: code/akka/docs/actor/FirstUntypedActor.java#context-actorOf
Actors are automatically started asynchronously when created.
When you create the ``UntypedActor`` then it will automatically call the ``preStart``
callback method on the ``UntypedActor`` class. This is an excellent place to
add initialization code for the actor.
.. code-block:: java
import akka.actor.UntypedActor;
import akka.event.EventHandler;
public class SampleUntypedActor extends UntypedActor {
public void onReceive(Object message) throws Exception {
if (message instanceof String)
EventHandler.info(this, String.format("Received String message: %s",
message));
else
throw new IllegalArgumentException("Unknown message: " + message);
}
@Override
public void preStart() {
... // initialization code
}
Creating Actors
^^^^^^^^^^^^^^^
Creating an Actor is done using the 'akka.actor.Actors.actorOf' factory method. This method returns a reference to the UntypedActor's ActorRef. This 'ActorRef' is an immutable serializable reference that you should use to communicate with the actor, send messages, link to it etc. This reference also functions as the context for the actor and holds run-time type information such as sender of the last message,
.. code-block:: java
ActorRef myActor = Actors.actorOf(SampleUntypedActor.class);
Normally you would want to import the 'actorOf' method like this:
.. code-block:: java
import static akka.actor.Actors.*;
ActorRef myActor = actorOf(SampleUntypedActor.class);
To avoid prefix it with 'Actors' every time you use it.
You can also create & start the actor in one statement:
.. code-block:: java
ActorRef myActor = actorOf(SampleUntypedActor.class);
The call to 'actorOf' returns an instance of 'ActorRef'. This is a handle to the 'UntypedActor' instance which you can use to interact with the Actor, like send messages to it etc. more on this shortly. The 'ActorRef' is immutable and has a one to one relationship with the Actor it represents. The 'ActorRef' is also serializable and network-aware. This means that you can serialize it, send it over the wire and use it on a remote host and it will still be representing the same Actor on the original node, across the network.
Creating Actors with non-default constructor
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
--------------------------------------------
If your UntypedActor has a constructor that takes parameters then you can't create it using 'actorOf(clazz)'. Instead you can use a variant of 'actorOf' that takes an instance of an 'UntypedActorFactory' in which you can create the Actor in any way you like. If you use this method then you to make sure that no one can get a reference to the actor instance. If they can get a reference it then they can touch state directly in bypass the whole actor dispatching mechanism and create race conditions which can lead to corrupt data.
If your UntypedActor has a constructor that takes parameters then you can't create it using 'actorOf(clazz)'.
Instead you can use a variant of ``actorOf`` that takes an instance of an 'UntypedActorFactory'
in which you can create the Actor in any way you like. If you use this method then you to make sure that
no one can get a reference to the actor instance. If they can get a reference it then they can
touch state directly in bypass the whole actor dispatching mechanism and create race conditions
which can lead to corrupt data.
Here is an example:
.. code-block:: java
ActorRef actor = actorOf(new UntypedActorFactory() {
public UntypedActor create() {
return new MyUntypedActor("service:name", 5);
}
});
.. includecode:: code/akka/docs/actor/UntypedActorTestBase.java#creating-constructor
This way of creating the Actor is also great for integrating with Dependency Injection (DI) frameworks like Guice or Spring.
UntypedActor context
--------------------
Creating Actors with Props
--------------------------
The UntypedActor base class contains almost no member fields or methods to invoke. It only has the 'onReceive(Object message)' method, which is defining the Actor's message handler, and some life-cycle callbacks that you can choose to implement:
## preStart
## postStop
## preRestart
## postRestart
``Props`` is a configuration object to specify additional things for the actor to
be created, such as the ``MessageDispatcher``.
Most of the API is in the UnypedActorRef a reference for the actor. This reference is available in the 'getContext()' method in the UntypedActor (or you can use its alias, the 'context()' method, if you prefer. Here, for example, you find methods to reply to messages, send yourself messages, define timeouts, fault tolerance etc., start and stop etc.
.. includecode:: code/akka/docs/actor/UntypedActorTestBase.java#creating-props
Identifying Actors
------------------
Each ActorRef has two methods:
* getContext().getUuid();
* getContext().getId();
UntypedActor API
================
The difference is that the 'uuid' is generated by the runtime, guaranteed to be unique and can't be modified. While the 'id' can be set by the user (using 'getContext().setId(...)', and defaults to Actor class name. You can retrieve Actors by both UUID and ID using the 'ActorRegistry', see the section further down for details.
The :class:`UntypedActor` class defines only one abstract method, the above mentioned
:meth:`onReceive(Object message)`, which implements the behavior of the actor.
Messages and immutability
-------------------------
In addition, it offers:
**IMPORTANT**: Messages can be any kind of object but have to be immutable. Akka cant enforce immutability (yet) so this has to be by convention.
* :obj:`getSelf()` reference to the :class:`ActorRef` of the actor
* :obj:`getSender()` reference sender Actor of the last received message, typically used as described in :ref:`UntypedActor.Reply`
* :obj:`getContext()` exposes contextual information for the actor and the current message, such as:
Send messages
* factory methods to create child actors (:meth:`actorOf`)
* system that the actor belongs to
* parent supervisor
* supervised children
* hotswap behavior stack as described in :ref:`UntypedActor.HotSwap`
The remaining visible methods are user-overridable life-cycle hooks which are
described in the following:
.. includecode:: code/akka/docs/actor/UntypedActorTestBase.java#lifecycle-callbacks
The implementations shown above are the defaults provided by the :class:`UntypedActor`
class.
Start Hook
----------
Right after starting the actor, its :meth:`preStart` method is invoked.
::
@Override
public void preStart() {
// registering with other actors
someService.tell(Register(getSelf());
}
Restart Hooks
-------------
Messages are sent to an Actor through one of the 'send' methods.
* 'tell' means “fire-and-forget”, e.g. send a message asynchronously and return immediately.
* 'ask' sends a message asynchronously and returns a 'Future'.
All actors are supervised, i.e. linked to another actor with a fault
handling strategy. Actors will be restarted in case an exception is thrown while
processing a message. This restart involves the hooks mentioned above:
In all these methods you have the option of passing along your 'ActorRef' context variable. Make it a practice of doing so because it will allow the receiver actors to be able to respond to your message, since the sender reference is sent along with the message.
1. The old actor is informed by calling :meth:`preRestart` with the exception
which caused the restart and the message which triggered that exception; the
latter may be ``None`` if the restart was not caused by processing a
message, e.g. when a supervisor does not trap the exception and is restarted
in turn by its supervisor. This method is the best place for cleaning up,
preparing hand-over to the fresh actor instance, etc.
By default it calls :meth:`postStop`.
2. The initial factory from the ``actorOf`` call is used
to produce the fresh instance.
3. The new actors :meth:`postRestart` method is invoked with the exception
which caused the restart. By default the :meth:`preStart`
is called, just as in the normal start-up case.
Fire-forget
^^^^^^^^^^^
This is the preferred way of sending messages. No blocking waiting for a message. Give best concurrency and scalability characteristics.
An actor restart replaces only the actual actor object; the contents of the
mailbox and the hotswap stack are unaffected by the restart, so processing of
messages will resume after the :meth:`postRestart` hook returns. The message
that triggered the exception will not be received again. Any message
sent to an actor while it is being restarted will be queued to its mailbox as
usual.
Stop Hook
---------
After stopping an actor, its :meth:`postStop` hook is called, which may be used
e.g. for deregistering this actor from other services. This hook is guaranteed
to run after message queuing has been disabled for this actor, i.e. messages
sent to a stopped actor will be redirected to the :obj:`deadLetters` of the
:obj:`ActorSystem`.
Identifying Actors
==================
FIXME Actor Path documentation
Messages and immutability
=========================
**IMPORTANT**: Messages can be any kind of object but have to be
immutable. Akka cant enforce immutability (yet) so this has to be by
convention.
Here is an example of an immutable message:
.. includecode:: code/akka/docs/actor/ImmutableMessage.java#immutable-message
Send messages
=============
Messages are sent to an Actor through one of the following methods.
* ``tell`` means “fire-and-forget”, e.g. send a message asynchronously and return
immediately.
* ``ask`` sends a message asynchronously and returns a :class:`Future`
representing a possible reply.
Message ordering is guaranteed on a per-sender basis.
In all these methods you have the option of passing along your own ``ActorRef``.
Make it a practice of doing so because it will allow the receiver actors to be able to respond
to your message, since the sender reference is sent along with the message.
Tell: Fire-forget
-----------------
This is the preferred way of sending messages. No blocking waiting for a
message. This gives the best concurrency and scalability characteristics.
.. code-block:: java
actor.tell("Hello");
Or with the sender reference passed along:
Or with the sender reference passed along with the message and available to the receiving Actor
in its ``getSender: ActorRef`` member field. The target actor can use this
to reply to the original sender, by using ``getSender().tell(replyMsg)``.
.. code-block:: java
actor.tell("Hello", getContext());
actor.tell("Hello", getSelf());
If invoked from within an Actor, then the sending actor reference will be implicitly passed along with the message and available to the receiving Actor in its 'getContext().getSender();' method. He can use this to reply to the original sender or use the 'getContext().reply(message);' method.
If invoked without the sender parameter the sender will be
:obj:`deadLetters` actor reference in the target actor.
If invoked from an instance that is **not** an Actor there will be no implicit sender passed along the message and you will get an 'IllegalStateException' if you call 'getContext().reply(..)'.
Ask: Send-And-Receive-Future
----------------------------
Send-And-Receive-Future
^^^^^^^^^^^^^^^^^^^^^^^
Using 'ask' will send a message to the receiving Actor asynchronously and will immediately return a 'Future'.
Using ``ask`` will send a message to the receiving Actor asynchronously and
will immediately return a :class:`Future`:
.. code-block:: java
Future future = actorRef.ask("Hello", getContext(), 1000);
long timeoutMillis = 1000;
Future future = actorRef.ask("Hello", timeoutMillis);
The 'Future' interface looks like this:
The receiving actor should reply to this message, which will complete the
future with the reply message as value; ``getSender.tell(result)``.
.. code-block:: java
To complete the future with an exception you need send a Failure message to the sender.
This is not done automatically when an actor throws an exception while processing a
message.
interface Future<T> {
void await();
boolean isCompleted();
boolean isExpired();
long timeoutInNanos();
Option<T> result();
Option<Throwable> exception();
Future<T> onComplete(Procedure<Future<T>> procedure);
}
.. includecode:: code/akka/docs/actor/UntypedActorTestBase.java#reply-exception
So the normal way of working with futures is something like this:
If the actor does not complete the future, it will expire after the timeout period,
specified as parameter to the ``ask`` method.
.. code-block:: java
See :ref:`futures-java` for more information on how to await or query a
future.
Future future = actorRef.ask("Hello", getContext(), 1000);
future.await();
if (future.isCompleted()) {
Option resultOption = future.result();
if (resultOption.isDefined()) {
Object result = resultOption.get();
...
}
... // whatever
}
The ``onComplete``, ``onResult``, or ``onTimeout`` methods of the ``Future`` can be
used to register a callback to get a notification when the Future completes.
Gives you a way to avoid blocking.
The 'onComplete' callback can be used to register a callback to get a notification when the Future completes. Gives you a way to avoid blocking.
.. warning::
When using future callbacks, inside actors you need to carefully avoid closing over
the containing actors reference, i.e. do not call methods or access mutable state
on the enclosing actor from within the callback. This would break the actor
encapsulation and may introduce synchronization bugs and race conditions because
the callback will be scheduled concurrently to the enclosing actor. Unfortunately
there is not yet a way to detect these illegal accesses at compile time. See also:
:ref:`jmm-shared-state`
The future returned from the ``ask`` method can conveniently be passed around or
chained with further processing steps, but sometimes you just need the value,
even if that entails waiting for it (but keep in mind that waiting inside an
actor is prone to dead-locks, e.g. if obtaining the result depends on
processing another message on this actor).
.. includecode:: code/akka/docs/actor/UntypedActorTestBase.java
:include: import-future,using-ask
Forward message
^^^^^^^^^^^^^^^
---------------
You can forward a message from one actor to another. This means that the original sender address/reference is maintained even though the message is going through a 'mediator'. This can be useful when writing actors that work as routers, load-balancers, replicators etc. You need to pass along your ActorRef context variable as well.
You can forward a message from one actor to another. This means that the
original sender address/reference is maintained even though the message is going
through a 'mediator'. This can be useful when writing actors that work as
routers, load-balancers, replicators etc.
You need to pass along your context variable as well.
.. code-block:: java
getContext().forward(message, getContext());
myActor.forward(message, getContext());
Receive messages
----------------
================
When an actor receives a message it is passed into the 'onReceive' method, this is an abstract method on the 'UntypedActor' base class that needs to be defined.
When an actor receives a message it is passed into the ``onReceive`` method, this is
an abstract method on the ``UntypedActor`` base class that needs to be defined.
Here is an example:
.. code-block:: java
.. includecode:: code/akka/docs/actor/MyUntypedActor.java#my-untyped-actor
public class SampleUntypedActor extends UntypedActor {
An alternative to using if-instanceof checks is to use `Apache Commons MethodUtils
<http://commons.apache.org/beanutils/api/org/apache/commons/beanutils/MethodUtils.html#invokeMethod(java.lang.Object,%20java.lang.String,%20java.lang.Object)>`_
to invoke a named method whose parameter type matches the message type.
public void onReceive(Object message) throws Exception {
if (message instanceof String)
EventHandler.info(this, String.format("Received String message: %s", message));
else
throw new IllegalArgumentException("Unknown message: " + message);
}
}
.. _UntypedActor.Reply:
Reply to messages
-----------------
=================
Reply using the channel
^^^^^^^^^^^^^^^^^^^^^^^
If you want to have a handle to an object to whom you can reply to the message, you can use the Channel abstraction.
Simply call getContext().channel() and then you can forward that to others, store it away or otherwise until you want to reply,
which you do by Channel.tell(msg)
If you want to have a handle for replying to a message, you can use
``getSender()``, which gives you an ActorRef. You can reply by sending to
that ActorRef with ``getSender().tell(replyMsg)``. You can also store the ActorRef
for replying later, or passing on to other actors. If there is no sender (a
message was sent without an actor or future context) then the sender
defaults to a 'dead-letter' actor ref.
.. code-block:: java
public void onReceive(Object message) throws Exception {
if (message instanceof String) {
String msg = (String)message;
if (msg.equals("Hello")) {
// Reply to original sender of message using the channel
getContext().channel().tell(msg + " from " + getContext().getUuid());
}
}
public void onReceive(Object request) {
String result = process(request);
getSender().tell(result); // will have dead-letter actor as default
}
We recommend that you as first choice use the channel abstraction instead of the other ways described in the following sections.
Initial receive timeout
=======================
Reply using the 'tryReply' and 'reply' methods
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
A timeout mechanism can be used to receive a message when no initial message is
received within a certain time. To receive this timeout you have to set the
``receiveTimeout`` property and declare handing for the ReceiveTimeout
message.
If you want to send a message back to the original sender of the message you just received then you can use the 'getContext().reply(..)' method.
.. code-block:: java
public void onReceive(Object message) throws Exception {
if (message instanceof String) {
String msg = (String)message;
if (msg.equals("Hello")) {
// Reply to original sender of message using the 'reply' method
getContext().reply(msg + " from " + getContext().getUuid());
}
}
}
In this case we will a reply back to the Actor that sent the message.
The 'reply' method throws an 'IllegalStateException' if unable to determine what to reply to, e.g. the sender has not been passed along with the message when invoking one of 'send*' methods. You can also use the more forgiving 'tryReply' method which returns 'true' if reply was sent, and 'false' if unable to determine what to reply to.
.. code-block:: java
public void onReceive(Object message) throws Exception {
if (message instanceof String) {
String msg = (String)message;
if (msg.equals("Hello")) {
// Reply to original sender of message using the 'reply' method
if (getContext().tryReply(msg + " from " + getContext().getUuid())) ... // success
else ... // handle failure
}
}
}
Summary of reply semantics and options
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
* ``getContext().reply(...)`` can be used to reply to an ``Actor`` or a
``Future`` from within an actor; the current actor will be passed as reply
channel if the current channel supports this.
* ``getContext().channel`` is a reference providing an abstraction for the
reply channel; this reference may be passed to other actors or used by
non-actor code.
.. note::
There used to be two methods for determining the sending Actor or Future for the current invocation:
* ``getContext().getSender()`` yielded a :class:`Option[ActorRef]`
* ``getContext().getSenderFuture()`` yielded a :class:`Option[CompletableFuture[Any]]`
These two concepts have been unified into the ``channel``. If you need to
know the nature of the channel, you may do so using instance tests::
if (getContext().channel() instanceof ActorRef) {
...
} else if (getContext().channel() instanceof ActorPromise) {
...
}
Promise represents the write-side of a Future, enabled by the methods
* success(..)
* break(..)
Starting actors
---------------
Actors are started when they are created by invoking the actorOf method.
.. code-block:: java
ActorRef actor = actorOf(SampleUntypedActor.class);
When you create the actor then it will automatically call the 'preStart' callback method on the 'UntypedActor'. This is an excellent place to add initialization code for the actor.
.. code-block:: java
@Override
void preStart() {
... // initialization code
}
.. includecode:: code/akka/docs/actor/MyReceivedTimeoutUntypedActor.java#receive-timeout
Stopping actors
---------------
===============
Actors are stopped by invoking the stop method.
Actors are stopped by invoking the ``stop`` method of the ``ActorRef``.
The actual termination of the actor is performed asynchronously, i.e.
``stop`` may return before the actor is stopped.
.. code-block:: java
actor.stop();
When stop is called then a call to the postStop callback method will take place. The Actor can use this callback to implement shutdown behavior.
Processing of the current message, if any, will continue before the actor is stopped,
but additional messages in the mailbox will not be processed. By default these
messages are sent to the :obj:`deadLetters` of the :obj:`ActorSystem`, but that
depends on the mailbox implementation.
When stop is called then a call to the ``def postStop`` callback method will
take place. The ``Actor`` can use this callback to implement shutdown behavior.
.. code-block:: java
@Override
void postStop() {
public void postStop() {
... // clean up resources
}
You can shut down all Actors in the system by invoking:
.. code-block:: java
All Actors are stopped when the ``ActorSystem`` is stopped.
Supervised actors are stopped when the supervisor is stopped, i.e. children are stopped
when parent is stopped.
Actors.registry().shutdownAll();
PoisonPill
----------
You can also send an actor the akka.actor.PoisonPill message, which will stop the actor when the message is processed.
If the sender is a Future, the Future will be completed with an akka.actor.ActorKilledException("PoisonPill")
You can also send an actor the ``akka.actor.PoisonPill`` message, which will
stop the actor when the message is processed. ``PoisonPill`` is enqueued as
ordinary messages and will be handled after messages that were already queued
in the mailbox.
If the ``PoisonPill`` was sent with ``ask``, the ``Future`` will be completed with an
``akka.actor.ActorKilledException("PoisonPill")``.
Use it like this:
.. includecode:: code/akka/docs/actor/UntypedActorTestBase.java
:include: import-actors,poison-pill
.. _UntypedActor.HotSwap:
HotSwap
=======
Upgrade
-------
Akka supports hotswapping the Actors message loop (e.g. its implementation) at
runtime. Use the ``getContext().become`` method from within the Actor.
The hotswapped code is kept in a Stack which can be pushed and popped.
.. warning::
Please note that the actor will revert to its original behavior when restarted by its Supervisor.
To hotswap the Actor using ``getContext().become``:
.. includecode:: code/akka/docs/actor/UntypedActorTestBase.java
:include: import-procedure,hot-swap-actor
The ``become`` method is useful for many different things, such as to implement
a Finite State Machine (FSM).
Here is another little cute example of ``become`` and ``unbecome`` in action:
.. includecode:: code/akka/docs/actor/UntypedActorSwapper.java#swapper
Downgrade
---------
Since the hotswapped code is pushed to a Stack you can downgrade the code as
well. Use the ``getContext().unbecome`` method from within the Actor.
.. code-block:: java
import static akka.actor.Actors.*;
actor.tell(poisonPill());
public void onReceive(Object message) {
if (message.equals("revert")) getContext().unbecome();
}
Killing an Actor
----------------
================
You can kill an actor by sending a 'new Kill()' message. This will restart the actor through regular supervisor semantics.
You can kill an actor by sending a ``Kill`` message. This will restart the actor
through regular supervisor semantics.
Use it like this:
.. code-block:: java
.. includecode:: code/akka/docs/actor/UntypedActorTestBase.java
:include: import-actors,kill
import static akka.actor.Actors.*;
Actors and exceptions
=====================
// kill the actor called 'victim'
victim.tell(kill());
It can happen that while a message is being processed by an actor, that some
kind of exception is thrown, e.g. a database exception.
Actor life-cycle
----------------
What happens to the Message
---------------------------
The actor has a well-defined non-circular life-cycle.
If an exception is thrown while a message is being processed (so taken of his
mailbox and handed over the the receive), then this message will be lost. It is
important to understand that it is not put back on the mailbox. So if you want
to retry processing of a message, you need to deal with it yourself by catching
the exception and retry your flow. Make sure that you put a bound on the number
of retries since you don't want a system to livelock (so consuming a lot of cpu
cycles without making progress).
::
What happens to the mailbox
---------------------------
NEW (newly created actor) - can't receive messages (yet)
=> STARTED (when 'start' is invoked) - can receive messages
=> SHUT DOWN (when 'exit' or 'stop' is invoked) - can't do anything
If an exception is thrown while a message is being processed, nothing happens to
the mailbox. If the actor is restarted, the same mailbox will be there. So all
messages on that mailbox, will be there as well.
What happens to the actor
-------------------------
If an exception is thrown, the actor instance is discarded and a new instance is
created. This new instance will now be used in the actor references to this actor
(so this is done invisible to the developer). Note that this means that current
state of the failing actor instance is lost if you don't store and restore it in
``preRestart`` and ``postRestart`` callbacks.

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,31 @@
package akka.docs.actor.mailbox
//#imports
import akka.actor.Actor
import akka.actor.Props
import akka.actor.mailbox.FileDurableMailboxType
//#imports
import org.scalatest.{ BeforeAndAfterAll, WordSpec }
import org.scalatest.matchers.MustMatchers
import akka.testkit.AkkaSpec
class MyActor extends Actor {
def receive = {
case x
}
}
class DurableMailboxDocSpec extends AkkaSpec {
"define dispatcher with durable mailbox" in {
//#define-dispatcher
val dispatcher = system.dispatcherFactory.newDispatcher(
"my-dispatcher", throughput = 1, mailboxType = FileDurableMailboxType).build
val myActor = system.actorOf(Props[MyActor].withDispatcher(dispatcher), name = "myactor")
//#define-dispatcher
myActor ! "hello"
}
}

View file

@ -0,0 +1,5 @@
package akka.docs.actor.mailbox
import org.scalatest.junit.JUnitSuite
class DurableMailboxDocTest extends DurableMailboxDocTestBase with JUnitSuite

View file

@ -0,0 +1,41 @@
package akka.docs.actor.mailbox;
//#imports
import akka.actor.mailbox.DurableMailboxType;
import akka.dispatch.MessageDispatcher;
import akka.actor.UntypedActorFactory;
import akka.actor.UntypedActor;
import akka.actor.Props;
//#imports
import org.junit.Test;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import static org.junit.Assert.*;
public class DurableMailboxDocTestBase {
@Test
public void defineDispatcher() {
ActorSystem system = ActorSystem.create("MySystem");
//#define-dispatcher
MessageDispatcher dispatcher = system.dispatcherFactory()
.newDispatcher("my-dispatcher", 1, DurableMailboxType.fileDurableMailboxType()).build();
ActorRef myActor = system.actorOf(new Props().withDispatcher(dispatcher).withCreator(new UntypedActorFactory() {
public UntypedActor create() {
return new MyUntypedActor();
}
}));
//#define-dispatcher
myActor.tell("test");
system.stop();
}
public static class MyUntypedActor extends UntypedActor {
public void onReceive(Object message) {
}
}
}

View file

@ -0,0 +1,221 @@
.. _durable-mailboxes:
###################
Durable Mailboxes
###################
.. sidebar:: Contents
.. contents:: :local:
Overview
========
Akka supports a set of durable mailboxes. A durable mailbox is a replacement for
the standard actor mailbox that is durable. What this means in practice is that
if there are pending messages in the actor's mailbox when the node of the actor
resides on crashes, then when you restart the node, the actor will be able to
continue processing as if nothing had happened; with all pending messages still
in its mailbox.
None of these mailboxes implements transactions for current message. It's possible
if the actor crashes after receiving a message, but before completing processing of
it, that the message could be lost.
.. warning:: **IMPORTANT**
None of these mailboxes work with blocking message send, e.g. the message
send operations that are relying on futures; ``?`` or ``ask``. If the node
has crashed and then restarted, the thread that was blocked waiting for the
reply is gone and there is no way we can deliver the message.
The durable mailboxes currently supported are:
- ``FileBasedMailbox`` -- backed by a journaling transaction log on the local file system
- ``RedisBasedMailbox`` -- backed by Redis
- ``ZooKeeperBasedMailbox`` -- backed by ZooKeeper
- ``BeanstalkBasedMailbox`` -- backed by Beanstalkd
- ``MongoBasedMailbox`` -- backed by MongoDB
We'll walk through each one of these in detail in the sections below.
You can easily implement your own mailbox. Look at the existing implementations for inspiration.
Soon Akka will also have:
- ``AmqpBasedMailbox`` -- AMQP based mailbox (default RabbitMQ)
- ``JmsBasedMailbox`` -- JMS based mailbox (default ActiveMQ)
.. _DurableMailbox.General:
General Usage
-------------
The durable mailboxes and their configuration options reside in the
``akka.actor.mailbox`` package.
You configure durable mailboxes through the dispatcher. The
actor is oblivious to which type of mailbox it is using.
Here is an example in Scala:
.. includecode:: code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala
:include: imports,define-dispatcher
Corresponding example in Java:
.. includecode:: code/akka/docs/actor/mailbox/DurableMailboxDocTestBase.java
:include: imports,define-dispatcher
The actor is oblivious to which type of mailbox it is using.
This gives you an excellent way of creating bulkheads in your application, where
groups of actors sharing the same dispatcher also share the same backing
storage. Read more about that in the :ref:`dispatchers-scala` documentation.
File-based durable mailbox
==========================
This mailbox is backed by a journaling transaction log on the local file
system. It is the simplest to use since it does not require an extra
infrastructure piece to administer, but it is usually sufficient and just what
you need.
You configure durable mailboxes through the dispatcher, as described in
:ref:`DurableMailbox.General` with the following mailbox type.
Scala::
mailbox = akka.actor.mailbox.FileDurableMailboxType
Java::
akka.actor.mailbox.DurableMailboxType.fileDurableMailboxType()
You can also configure and tune the file-based durable mailbox. This is done in
the ``akka.actor.mailbox.file-based`` section in the :ref:`configuration`.
.. literalinclude:: ../../akka-durable-mailboxes/akka-file-mailbox/src/main/resources/reference.conf
:language: none
Redis-based durable mailbox
===========================
This mailbox is backed by a Redis queue. `Redis <http://redis.io>`_ Is a very
fast NOSQL database that has a wide range of data structure abstractions, one of
them is a queue which is what we are using in this implementation. This means
that you have to start up a Redis server that can host these durable
mailboxes. Read more in the Redis documentation on how to do that.
You configure durable mailboxes through the dispatcher, as described in
:ref:`DurableMailbox.General` with the following mailbox type.
Scala::
mailbox = akka.actor.mailbox.RedisDurableMailboxType
Java::
akka.actor.mailbox.DurableMailboxType.redisDurableMailboxType()
You also need to configure the IP and port for the Redis server. This is done in
the ``akka.actor.mailbox.redis`` section in the :ref:`configuration`.
.. literalinclude:: ../../akka-durable-mailboxes/akka-redis-mailbox/src/main/resources/reference.conf
:language: none
ZooKeeper-based durable mailbox
===============================
This mailbox is backed by `ZooKeeper <http://zookeeper.apache.org/>`_. ZooKeeper
is a centralized service for maintaining configuration information, naming,
providing distributed synchronization, and providing group services This means
that you have to start up a ZooKeeper server (for production a ZooKeeper server
ensamble) that can host these durable mailboxes. Read more in the ZooKeeper
documentation on how to do that.
You configure durable mailboxes through the dispatcher, as described in
:ref:`DurableMailbox.General` with the following mailbox type.
Scala::
mailbox = akka.actor.mailbox.ZooKeeperDurableMailboxType
Java::
akka.actor.mailbox.DurableMailboxType.zooKeeperDurableMailboxType()
You also need to configure ZooKeeper server addresses, timeouts, etc. This is
done in the ``akka.actor.mailbox.zookeeper`` section in the :ref:`configuration`.
.. literalinclude:: ../../akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/resources/reference.conf
:language: none
Beanstalk-based durable mailbox
===============================
This mailbox is backed by `Beanstalkd <http://kr.github.com/beanstalkd/>`_.
Beanstalk is a simple, fast work queue. This means that you have to start up a
Beanstalk server that can host these durable mailboxes. Read more in the
Beanstalk documentation on how to do that.
You configure durable mailboxes through the dispatcher, as described in
:ref:`DurableMailbox.General` with the following mailbox type.
Scala::
mailbox = akka.actor.mailbox.BeanstalkDurableMailboxType
Java::
akka.actor.mailbox.DurableMailboxType.beanstalkDurableMailboxType()
You also need to configure the IP, and port, and so on, for the Beanstalk
server. This is done in the ``akka.actor.mailbox.beanstalk`` section in the
:ref:`configuration`.
.. literalinclude:: ../../akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/resources/reference.conf
:language: none
MongoDB-based Durable Mailboxes
===============================
This mailbox is backed by `MongoDB <http://mongodb.org>`_.
MongoDB is a fast, lightweight and scalable document-oriented database. It contains a number of
features cohesive to a fast, reliable & durable queueing mechanism which the Akka Mailbox takes advantage of.
Akka's implementations of MongoDB mailboxes are built on top of the purely asynchronous MongoDB driver
(often known as `Hammersmith <http://github.com/bwmcadams/hammersmith>`_ and ``com.mongodb.async``)
and as such are purely callback based with a Netty network layer. This makes them extremely fast &
lightweight versus building on other MongoDB implementations such as
`mongo-java-driver <http://github.com/mongodb/mongo-java-driver>`_ and `Casbah <http://github.com/mongodb/casbah>`_.
You configure durable mailboxes through the dispatcher, as described in
:ref:`DurableMailbox.General` with the following mailbox type.
Scala::
mailbox = akka.actor.mailbox.MongoDurableMailboxType
Java::
akka.actor.mailbox.DurableMailboxType.mongoDurableMailboxType()
You will need to configure the URI for the MongoDB server, using the URI Format specified in the
`MongoDB Documentation <http://www.mongodb.org/display/DOCS/Connections>`_. This is done in
the ``akka.actor.mailbox.mongodb`` section in the :ref:`configuration`.
.. literalinclude:: ../../akka-durable-mailboxes/akka-mongo-mailbox/src/main/resources/reference.conf
:language: none
You must specify a hostname (and optionally port) and at *least* a Database name. If you specify a
collection name, it will be used as a 'prefix' for the collections Akka creates to store mailbox messages.
Otherwise, collections will be prefixed with ``mailbox.``
It is also possible to configure the timeout thresholds for Read and Write operations in the ``timeout`` block.

View file

@ -0,0 +1,55 @@
.. _http-module:
HTTP
====
.. sidebar:: Contents
.. contents:: :local:
When deploying in a servlet container:
--------------------------------------------
If you deploy Akka in a JEE container, don't forget to create an Akka initialization and cleanup hook:
.. code-block:: scala
package com.my //<--- your own package
import akka.util.AkkaLoader
import akka.cluster.BootableRemoteActorService
import akka.actor.BootableActorLoaderService
import javax.servlet.{ServletContextListener, ServletContextEvent}
/**
* This class can be added to web.xml mappings as a listener to start and postStop Akka.
*<web-app>
* ...
* <listener>
* <listener-class>com.my.Initializer</listener-class>
* </listener>
* ...
*</web-app>
*/
class Initializer extends ServletContextListener {
lazy val loader = new AkkaLoader
def contextDestroyed(e: ServletContextEvent): Unit = loader.shutdown
def contextInitialized(e: ServletContextEvent): Unit =
loader.boot(true, new BootableActorLoaderService with BootableRemoteActorService) //<--- Important
// loader.boot(true, new BootableActorLoaderService {}) // If you don't need akka-remote
}
For Java users, it's currently only possible to use BootableActorLoaderService, but you'll need to use: akka.actor.DefaultBootableActorLoaderService
Then you just declare it in your web.xml:
.. code-block:: xml
<web-app>
...
<listener>
<listener-class>your.package.Initializer</listener-class>
</listener>
...
</web-app>

View file

@ -4,6 +4,8 @@ Modules
.. toctree::
:maxdepth: 2
durable-mailbox
http
microkernel
camel
spring

View file

@ -5,37 +5,4 @@
Microkernel
#############
Run the microkernel
===================
To start the kernel use the scripts in the ``bin`` directory.
All services are configured in the ``config/akka.conf`` configuration file. See
the Akka documentation on Configuration for more details. Services you want to
be started up automatically should be listed in the list of ``boot`` classes in
the configuration.
Put your application in the ``deploy`` directory.
Akka Home
---------
Note that the microkernel needs to know where the Akka home is (the base
directory of the microkernel). The above scripts do this for you. Otherwise, you
can set Akka home by:
* Specifying the ``AKKA_HOME`` environment variable
* Specifying the ``-Dakka.home`` java option
.. _hello-microkernel:
Hello Microkernel
=================
There is a very simple Akka Mist sample project included in the microkernel
``deploy`` directory. Start the microkernel with the start script and then go to
http://localhost:9998 to say Hello to the microkernel.
The Akka Spring module has not been migrated to Akka 2.0-SNAPSHOT yet.

View file

@ -5,331 +5,4 @@
Spring Integration
####################
Module stability: **STABLE**
Akkas integration with the `Spring Framework <http://www.springsource.org>`_ supplies the Spring way of using the Typed Actor Java API and for CamelService configuration for :ref:`camel-spring-applications`. It uses Spring's custom namespaces to create Typed Actors, supervisor hierarchies and a CamelService in a Spring environment.
Contents:
.. contents:: :local:
To use the custom name space tags for Akka you have to add the XML schema definition to your spring configuration. It is available at `http://akka.io/akka-1.0.xsd <http://akka.io/akka.xsd>`_. The namespace for Akka is:
.. code-block:: xml
xmlns:akka="http://akka.io/schema/akka"
Example header for Akka Spring configuration:
.. code-block:: xml
<?xml version="1.0" encoding="UTF-8"?>
<beans xmlns="http://www.springframework.org/schema/beans"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns:akka="http://akka.io/schema/akka"
xsi:schemaLocation="
http://www.springframework.org/schema/beans
http://www.springframework.org/schema/beans/spring-beans-3.0.xsd
http://akka.io/schema/akka
http://akka.io/akka-1.0.xsd">
-
Actors
------
Actors in Java are created by extending the 'UntypedActor' class and implementing the 'onReceive' method.
Example how to create Actors with the Spring framework:
.. code-block:: xml
<akka:untyped-actor id="myActor"
implementation="com.biz.MyActor"
scope="singleton"
autostart="false"
depends-on="someBean"> <!-- or a comma-separated list of beans -->
<property name="aProperty" value="somePropertyValue"/>
<property name="aDependency" ref="someBeanOrActorDependency"/>
</akka:untyped-actor>
Supported scopes are singleton and prototype. Dependencies and properties are set with Springs ``<property/>`` element.
A dependency can be either a ``<akka:untyped-actor/>`` or a regular ``<bean/>``.
Get the Actor from the Spring context:
.. code-block:: java
ApplicationContext context = new ClassPathXmlApplicationContext("akka-spring-config.xml");
ActorRef actorRef = (ActorRef) context.getBean("myActor");
Typed Actors
------------
Here are some examples how to create Typed Actors with the Spring framework:
Creating a Typed Actor:
^^^^^^^^^^^^^^^^^^^^^^^
.. code-block:: xml
<beans>
<akka:typed-actor id="myActor"
interface="com.biz.MyPOJO"
implementation="com.biz.MyPOJOImpl"
transactional="true"
timeout="1000"
scope="singleton"
depends-on="someBean"> <!-- or a comma-separated list of beans -->
<property name="aProperty" value="somePropertyValue"/>
<property name="aDependency" ref="someBeanOrActorDependency"/>
</akka:typed-actor>
</beans>
Supported scopes are singleton and prototype. Dependencies and properties are set with Springs ``<property/>`` element.
A dependency can be either a ``<akka:typed-actor/>`` or a regular ``<bean/>``.
Get the Typed Actor from the Spring context:
.. code-block:: java
ApplicationContext context = new ClassPathXmlApplicationContext("akka-spring-config.xml");
MyPojo myPojo = (MyPojo) context.getBean("myActor");
Remote Actors
-------------
For details on server managed and client managed remote actors see Remote Actor documentation.
Configuration for a client managed remote Actor
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
::
<akka:untyped-actor id="remote-untyped-actor"
implementation="com.biz.MyActor"
timeout="2000">
<akka:remote host="localhost" port="9992" managed-by="client"/>
</akka:untyped-actor>
The default for 'managed-by' is "client", so in the above example it could be left out.
Configuration for a server managed remote Actor
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Server side
***********
::
<akka:untyped-actor id="server-managed-remote-untyped-actor"
implementation="com.biz.MyActor">
<akka:remote host="localhost" port="9990" managed-by="server"/>
</akka:untyped-actor>
<!-- register with custom service name -->
<akka:untyped-actor id="server-managed-remote-untyped-actor-custom-id"
implementation="com.biz.MyActor">
<akka:remote host="localhost" port="9990" service-name="my-service"/>
</akka:untyped-actor>
If the server specified by 'host' and 'port' does not exist it will not be registered.
Client side
***********
::
<!-- service-name could be custom name or class name -->
<akka:actor-for id="client-1" host="localhost" port="9990" service-name="my-service"/>
Configuration for a client managed remote Typed Actor
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
.. code-block:: xml
<akka:typed-actor id="remote-typed-actor"
interface="com.biz.MyPojo"
implementation="com.biz.MyPojoImpl"
timeout="2000">
<akka:remote host="localhost" port="9999" />
</akka:typed-actor>
Configuration for a server managed remote Typed Actor
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Sever side setup
****************
::
<akka:typed-actor id="server-managed-remote-typed-actor-custom-id"
interface="com.biz.IMyPojo"
implementation="com.biz.MyPojo"
timeout="2000">
<akka:remote host="localhost" port="9999" service-name="mypojo-service"/>
</akka:typed-actor>
Client side setup
*****************
::
<!-- always specify the interface for typed actor -->
<akka:actor-for id="typed-client"
interface="com.biz.MyPojo"
host="localhost"
port="9999"
service-name="mypojo-service"/>
Dispatchers
-----------
Configuration for a Typed Actor or Untyped Actor with a custom dispatcher
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
If you don't want to use the default dispatcher you can define your own dispatcher in the spring configuration. For more information on dispatchers have a look at Dispatchers documentation.
.. code-block:: xml
<akka:typed-actor id="remote-typed-actor"
interface="com.biz.MyPOJO"
implementation="com.biz.MyPOJOImpl"
timeout="2000">
<akka:dispatcher id="my-dispatcher" type="executor-based-event-driven" name="myDispatcher">
<akka:thread-pool queue="unbounded-linked-blocking-queue" capacity="100" />
</akka:dispatcher>
</akka:typed-actor>
<akka:untyped-actor id="untyped-actor-with-thread-based-dispatcher"
implementation="com.biz.MyActor">
<akka:dispatcher type="thread-based" name="threadBasedDispatcher"/>
</akka:untyped-actor>
If you want to or have to share the dispatcher between Actors you can define a dispatcher and reference it from the Typed Actor configuration:
.. code-block:: xml
<akka:dispatcher id="dispatcher-1"
type="executor-based-event-driven"
name="myDispatcher">
<akka:thread-pool queue="bounded-array-blocking-queue"
capacity="100"
fairness="true"
core-pool-size="1"
max-pool-size="20"
keep-alive="3000"
rejection-policy="caller-runs-policy"/>
</akka:dispatcher>
<akka:typed-actor id="typed-actor-with-dispatcher-ref"
interface="com.biz.MyPOJO"
implementation="com.biz.MyPOJOImpl"
timeout="1000">
<akka:dispatcher ref="dispatcher-1"/>
</akka:typed-actor>
The following dispatcher types are available in spring configuration:
* executor-based-event-driven
* executor-based-event-driven-work-stealing
* thread-based
The following queue types are configurable for dispatchers using thread pools:
* bounded-linked-blocking-queue
* unbounded-linked-blocking-queue
* synchronous-queue
* bounded-array-blocking-queue
If you have set up your IDE to be XSD-aware you can easily write your configuration through auto-completion.
Stopping Typed Actors and Untyped Actors
----------------------------------------
Actors with scope singleton are stopped when the application context is closed. Actors with scope prototype must be stopped by the application.
Supervisor Hierarchies
----------------------
The supervisor configuration in Spring follows the declarative configuration for the Java API. Have a look at Akka's approach to fault tolerance.
Example spring supervisor configuration
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
.. code-block:: xml
<beans>
<akka:supervision id="my-supervisor">
<akka:restart-strategy failover="AllForOne"
retries="3"
timerange="1000">
<akka:trap-exits>
<akka:trap-exit>java.io.IOException</akka:trap-exit>
</akka:trap-exits>
</akka:restart-strategy>
<akka:typed-actors>
<akka:typed-actor interface="com.biz.MyPOJO"
implementation="com.biz.MyPOJOImpl"
lifecycle="permanent"
timeout="1000"/>
<akka:typed-actor interface="com.biz.AnotherPOJO"
implementation="com.biz.AnotherPOJOImpl"
lifecycle="temporary"
timeout="1000"/>
<akka:typed-actor interface ="com.biz.FooBar"
implementation ="com.biz.FooBarImpl"
lifecycle="permanent"
transactional="true"
timeout="1000" />
</akka:typed-actors>
</akka:supervision>
<akka:supervision id="supervision-untyped-actors">
<akka:restart-strategy failover="AllForOne" retries="3" timerange="1000">
<akka:trap-exits>
<akka:trap-exit>java.io.IOException</akka:trap-exit>
<akka:trap-exit>java.lang.NullPointerException</akka:trap-exit>
</akka:trap-exits>
</akka:restart-strategy>
<akka:untyped-actors>
<akka:untyped-actor implementation="com.biz.PingActor"
lifecycle="permanent"/>
<akka:untyped-actor implementation="com.biz.PongActor"
lifecycle="permanent"/>
</akka:untyped-actors>
</akka:supervision>
</beans>
Get the TypedActorConfigurator from the Spring context
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
.. code-block:: java
TypedActorConfigurator myConfigurator = (TypedActorConfigurator) context.getBean("my-supervisor");
MyPojo myPojo = (MyPOJO) myConfigurator.getInstance(MyPojo.class);
Property Placeholders
---------------------
The Akka configuration can be made available as property placeholders by using a custom property placeholder configurer for Configgy:
::
<akka:property-placeholder location="akka.conf"/>
<akka:untyped-actor id="actor-1" implementation="com.biz.MyActor" timeout="${akka.actor.timeout}">
<akka:remote host="${akka.remote.server.hostname}" port="${akka.remote.server.port}"/>
</akka:untyped-actor>
Camel configuration
-------------------
For details refer to the :ref:`camel-module` documentation:
* CamelService configuration for :ref:`camel-spring-applications`
* Access to Typed Actors :ref:`camel-typed-actors-using-spring`
The Akka Spring module has not been migrated to Akka 2.0-SNAPSHOT yet.

View file

@ -25,7 +25,6 @@
Akka uses Git and is hosted at `Github <http://github.com>`_.
* Akka: clone the Akka repository from `<http://github.com/jboner/akka>`_
* Akka Modules: clone the Akka Modules repository from `<http://github.com/jboner/akka-modules>`_
`Maven Repository <http://akka.io/repository/>`_
@ -50,14 +49,14 @@ directly.
SNAPSHOT Versions
=================
Nightly builds are available in `<http://repo.typesafe.com/typesafe/maven-timestamps/>`_ repository as
Nightly builds are available in `<http://repo.typesafe.com/typesafe/akka-snapshots/>`_ repository as
timestamped snapshot versions. Pick a timestamp from
`<http://repo.typesafe.com/typesafe/maven-timestamps/com/typesafe/akka/akka-actor/>`_.
`<http://repo.typesafe.com/typesafe/akka-snapshots/com/typesafe/akka/akka-actor/>`_.
All Akka modules that belong to the same build have the same timestamp.
Make sure that you add the repository to the sbt resolvers or maven repositories::
resolvers += "Typesafe Timestamp Repo" at "http://repo.typesafe.com/typesafe/maven-timestamps/"
resolvers += "Typesafe Timestamp Repo" at "http://repo.typesafe.com/typesafe/akka-snapshots/"
Define the library dependencies with the timestamp as version::

View file

@ -1,447 +0,0 @@
Migration Guide 0.10.x to 1.0.x
====================================
Akka & Akka Modules separated into two different repositories and distributions
-------------------------------------------------------------------------------
Akka is split up into two different parts:
* Akka - Reflects all the sections under 'Scala API' and 'Java API' in the navigation bar.
* Akka Modules - Reflects all the sections under 'Add-on modules' in the navigation bar.
Download the release you need (Akka core or Akka Modules) from `<http://akka.io/downloads>`_ and unzip it.
----
Changed Akka URI
----------------
http://akkasource.org changed to http://akka.io
Reflects XSDs, Maven repositories, ScalaDoc etc.
----
Removed 'se.scalablesolutions' prefix
-------------------------------------
We have removed some boilerplate by shortening the Akka package from
**se.scalablesolutions.akka** to just **akka** so just do a search-replace in your project,
we apologize for the inconvenience, but we did it for our users.
----
Akka-core is no more
--------------------
Akka-core has been split into akka-actor, akka-stm, akka-typed-actor & akka-remote this means that you need to update any deps you have on akka-core.
----
Config
------
Turning on/off modules
^^^^^^^^^^^^^^^^^^^^^^
All the 'service = on' elements for turning modules on and off have been replaced by a top-level list of the enabled services.
Services available for turning on/off are:
* "remote"
* "http"
* "camel"
**All** services are **OFF** by default. Enable the ones you are using.
.. code-block:: ruby
akka {
enabled-modules = [] # Comma separated list of the enabled modules. Options: ["remote", "camel", "http"]
}
Renames
^^^^^^^
* 'rest' section - has been renamed to 'http' to align with the module name 'akka-http'.
* 'storage' section - has been renamed to 'persistence' to align with the module name 'akka-persistence'.
.. code-block:: ruby
akka {
http {
..
}
persistence {
..
}
}
----
Important changes from RC2-RC3
------------------------------
**akka.config.SupervisionSupervise**
**Scala**
.. code-block:: scala
def apply(actorRef: ActorRef, lifeCycle: LifeCycle, registerAsRemoteService: Boolean = false)
- boolean instead of remoteAddress, registers that actor with it's id as service name on the local server
**akka.actor.Actors now is the API for Java to interact with Actors, Remoting and ActorRegistry:**
**Java**
.. code-block:: java
import static akka.actor.Actors.*; // <-- The important part
actorOf();
remote().actorOf();
registry().actorsFor("foo");
***akka.actor.Actor now is the API for Scala to interact with Actors, Remoting and ActorRegistry:***
**Scala**
.. code-block:: scala
import akka.actor.Actor._ // <-- The important part
actorOf().method
remote.actorOf()
registry.actorsFor("foo")
**object UntypedActor has been deleted and replaced with akka.actor.Actors/akka.actor.Actor (Java/Scala)**
- UntypedActor.actorOf -> Actors.actorOf (Java) or Actor.actorOf (Scala)
**object ActorRegistry has been deleted and replaced with akka.actor.Actors.registry()/akka.actor.Actor.registry (Java/Scala)**
- ActorRegistry. -> Actors.registry(). (Java) or Actor.registry. (Scala)
**object RemoteClient has been deleted and replaced with akka.actor.Actors.remote()/akka.actor.Actor.remote (Java/Scala)**
- RemoteClient -> Actors.remote() (Java) or Actor.remote (Scala)
**object RemoteServer has been deleted and replaced with akka.actor.Actors.remote()/akka.actor.Actor.remote (Java/Scala)**
- RemoteServer - deleted -> Actors.remote() (Java) or Actor.remote (Scala)
**classes RemoteActor, RemoteUntypedActor and RemoteUntypedConsumerActors has been deleted and replaced with akka.actor.Actors.remote().actorOf(x, host port)/akka.actor.Actor.remote.actorOf(x, host, port)**
- RemoteActor, RemoteUntypedActor - deleted, use: remote().actorOf(YourActor.class, host, port) (Java) or remote.actorOf[YourActor](host, port)
**Remoted spring-actors now default to spring id as service-name, use "service-name" attribute on "remote"-tag to override**
**Listeners for RemoteServer and RemoteClient** are now registered on Actors.remote().addListener (Java) or Actor.remote.addListener (Scala), this means that all listeners get all remote events, both remote server evens and remote client events, **so adjust your code accordingly.**
**ActorRef.startLinkRemote has been removed since one specified on creation wether the actor is client-managed or not.**
Important change from RC3 to RC4
--------------------------------
The Akka-Spring namespace has changed from akkasource.org and scalablesolutions.se to http://akka.io/schema and http://akka.io/akka-<version>.xsd
Module akka-actor
-----------------
The Actor.init callback has been renamed to "preStart" to align with the general callback naming and is more clear about when it's called.
The Actor.shutdown callback has been renamed to "postStop" to align with the general callback naming and is more clear about when it's called.
The Actor.initTransactionalState callback has been removed, logic should be moved to preStart and be wrapped in an atomic block
**se.scalablesolutions.akka.config.ScalaConfig** and **se.scalablesolutions.akka.config.JavaConfig** have been merged into **akka.config.Supervision**
**RemoteAddress** has moved from **se.scalablesolutions.akka.config.ScalaConfig** to **akka.config**
The ActorRef.lifeCycle has changed signature from Option[LifeCycle] to LifeCycle, this means you need to change code that looks like this:
**self.lifeCycle = Some(LifeCycle(Permanent))** to **self.lifeCycle = Permanent**
The equivalent to **self.lifeCycle = None** is **self.lifeCycle = UndefinedLifeCycle**
**LifeCycle(Permanent)** becomes **Permanent**
**new LifeCycle(permanent())** becomes **permanent()** (need to do: import static se.scalablesolutions.akka.config.Supervision.*; first)
**JavaConfig.Component** and **ScalaConfig.Component** have been consolidated and renamed as **Supervision.SuperviseTypedActor**
**self.trapExit** has been moved into the FaultHandlingStrategy, and **ActorRef.faultHandler** has switched type from Option[FaultHandlingStrategy]
to FaultHandlingStrategy:
**Scala**
.. code-block:: scala
import akka.config.Supervision._
self.faultHandler = OneForOneStrategy(List(classOf[Exception]), 3, 5000)
**Java**
.. code-block:: java
import static akka.Supervision.*;
getContext().setFaultHandler(new OneForOneStrategy(new Class[] { Exception.class },50,1000))
**RestartStrategy, AllForOne, OneForOne** have been replaced with **AllForOneStrategy** and **OneForOneStrategy** in **se.scalablesolutions.akka.config.Supervision**
**Scala**
.. code-block:: scala
import akka.config.Supervision._
SupervisorConfig(
OneForOneStrategy(List(classOf[Exception]), 3, 5000),
Supervise(pingpong1,Permanent) :: Nil
)
**Java**
.. code-block:: java
import static akka.Supervision.*;
new SupervisorConfig(
new OneForOneStrategy(new Class[] { Exception.class },50,1000),
new Server[] { new Supervise(pingpong1, permanent()) }
)
***We have removed the following factory methods:***
**Actor.actor { case foo => bar }**
**Actor.transactor { case foo => bar }**
**Actor.temporaryActor { case foo => bar }**
**Actor.init {} receive { case foo => bar }**
They started the actor and no config was possible, it was inconsistent and irreparable.
replace with your own factories, or:
**Scala**
.. code-block:: scala
actorOf( new Actor { def receive = { case foo => bar } } ).start
actorOf( new Actor { self.lifeCycle = Temporary; def receive = { case foo => bar } } ).start
ReceiveTimeout is now rescheduled after every message, before there was only an initial timeout.
To stop rescheduling of ReceiveTimeout, set **receiveTimeout = None**
HotSwap
-------
HotSwap does no longer use behavior stacking by default, but that is an option to both "become" and HotSwap.
HotSwap now takes for Scala a Function from ActorRef to a Receive, the ActorRef passed in is the reference to self, so you can do self.reply() etc.
----
Module akka-stm
---------------
The STM stuff is now in its own module. This means that there is no support for transactions or transactors in akka-actor.
Local and global
^^^^^^^^^^^^^^^^
The **local/global** distinction has been dropped. This means that if the following general import was being used:
**Scala**
.. code-block:: scala
import akka.stm.local._
this is now just:
**Scala**
.. code-block:: scala
import akka.stm._
Coordinated is the new global
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
There is a new explicit mechanism for coordinated transactions. See the `Scala Transactors <transactors-scala>`_ and `Java Transactors <transactors-java>`_ documentation for more information. Coordinated transactions and transactors are found in the ``akka.transactor`` package now. The usage of transactors has changed.
Agents
^^^^^^
Agent is now in the akka-stm module and has moved to the ``akka.agent`` package. The implementation has been reworked and is now closer to Clojure agents. There is not much difference in general usage, the main changes involve interaction with the STM.
While updates to Agents are asynchronous, the state of an Agent is always immediately available for reading by any thread. Agents are integrated with the STM - any dispatches made in a transaction are held until that transaction commits, and are discarded if it is retried or aborted. There is a new ``sendOff`` method for long-running or blocking update functions.
----
Module akka-camel
-----------------
Access to the CamelService managed by CamelServiceManager has changed:
* Method service renamed to mandatoryService (Scala)
* Method service now returns Option[CamelService] (Scala)
* Introduced method getMandatoryService() (Java)
* Introduced method getService() (Java)
**Scala**
.. code-block:: scala
import se.scalablesolutions.akka.camel.CamelServiceManager._
import se.scalablesolutions.akka.camel.CamelService
val o: Option[CamelService] = service
val s: CamelService = mandatoryService
**Java**
.. code-block:: java
import se.scalablesolutions.akka.camel.CamelService;
import se.scalablesolutions.akka.japi.Option;
import static se.scalablesolutions.akka.camel.CamelServiceManager.*;
Option<CamelService> o = getService();
CamelService s = getMandatoryService();
Access to the CamelContext and ProducerTemplate managed by CamelContextManager has changed:
* Method context renamed to mandatoryContext (Scala)
* Method template renamed to mandatoryTemplate (Scala)
* Method service now returns Option[CamelContext] (Scala)
* Method template now returns Option[ProducerTemplate] (Scala)
* Introduced method getMandatoryContext() (Java)
* Introduced method getContext() (Java)
* Introduced method getMandatoryTemplate() (Java)
* Introduced method getTemplate() (Java)
**Scala**
.. code-block:: scala
import org.apache.camel.CamelContext
import org.apache.camel.ProducerTemplate
import se.scalablesolutions.akka.camel.CamelContextManager._
val co: Option[CamelContext] = context
val to: Option[ProducerTemplate] = template
val c: CamelContext = mandatoryContext
val t: ProducerTemplate = mandatoryTemplate
**Java**
.. code-block:: java
import org.apache.camel.CamelContext;
import org.apache.camel.ProducerTemplate;
import se.scalablesolutions.akka.japi.Option;
import static se.scalablesolutions.akka.camel.CamelContextManager.*;
Option<CamelContext> co = getContext();
Option<ProducerTemplate> to = getTemplate();
CamelContext c = getMandatoryContext();
ProducerTemplate t = getMandatoryTemplate();
The following methods have been renamed on class se.scalablesolutions.akka.camel.Message:
* bodyAs(Class) has been renamed to getBodyAs(Class)
* headerAs(String, Class) has been renamed to getHeaderAs(String, Class)
The API for waiting for consumer endpoint activation and de-activation has been changed
* CamelService.expectEndpointActivationCount has been removed and replaced by CamelService.awaitEndpointActivation
* CamelService.expectEndpointDeactivationCount has been removed and replaced by CamelService.awaitEndpointDeactivation
**Scala**
.. code-block:: scala
import se.scalablesolutions.akka.actor.Actor
import se.scalablesolutions.akka.camel.CamelServiceManager._
val s = startCamelService
val actor = Actor.actorOf[SampleConsumer]
// wait for 1 consumer being activated
s.awaitEndpointActivation(1) {
actor.start
}
// wait for 1 consumer being de-activated
s.awaitEndpointDeactivation(1) {
actor.stop
}
s.stop
**Java**
.. code-block:: java
import java.util.concurrent.TimeUnit;
import se.scalablesolutions.akka.actor.ActorRef;
import se.scalablesolutions.akka.actor.Actors;
import se.scalablesolutions.akka.camel.CamelService;
import se.scalablesolutions.akka.japi.SideEffect;
import static se.scalablesolutions.akka.camel.CamelServiceManager.*;
CamelService s = startCamelService();
final ActorRef actor = Actors.actorOf(SampleUntypedConsumer.class);
// wait for 1 consumer being activated
s.awaitEndpointActivation(1, new SideEffect() {
public void apply() {
actor.start();
}
});
// wait for 1 consumer being de-activated
s.awaitEndpointDeactivation(1, new SideEffect() {
public void apply() {
actor.stop();
}
});
s.stop();
Module Akka-Http
----------------
Atmosphere support has been removed. If you were using akka.comet.AkkaServlet for Jersey support only,
you can switch that to: akka.http.AkkaRestServlet and it should work just like before.
Atmosphere has been removed because we have a new async http support in the form of Akka Mist, a very thin bridge
between Servlet3.0/JettyContinuations and Actors, enabling Http-as-messages, read more about it here:
http://doc.akka.io/http#Mist%20-%20Lightweight%20Asynchronous%20HTTP
If you really need Atmosphere support, you can add it yourself by following the steps listed at the start of:
http://doc.akka.io/comet
Module akka-spring
------------------
The Akka XML schema URI has changed to http://akka.io/schema/akka
.. code-block:: xml
<beans xmlns="http://www.springframework.org/schema/beans"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns:akka="http://akka.io/schema/akka"
xsi:schemaLocation="
http://www.springframework.org/schema/beans
http://www.springframework.org/schema/beans/spring-beans-3.0.xsd
http://akka.io/schema/akka
http://akka.io/akka-1.0.xsd">
<!-- ... -->
</beans>

View file

@ -1,94 +0,0 @@
Migration Guide 0.7.x to 0.8.x
==============================
This is a case-by-case migration guide from Akka 0.7.x (on Scala 2.7.7) to Akka 0.8.x (on Scala 2.8.x)
------------------------------------------------------------------------------------------------------
Cases:
------
Actor.send is removed and replaced in full with Actor.!
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
.. code-block:: scala
myActor send "test"
becomes
.. code-block:: scala
myActor ! "test"
Actor.! now has it's implicit sender defaulted to None
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
.. code-block:: scala
def !(message: Any)(implicit sender: Option[Actor] = None)
"import Actor.Sender.Self" has been removed because it's not needed anymore
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Remove
.. code-block:: scala
import Actor.Sender.Self
Actor.spawn now uses manifests instead of concrete class types
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
.. code-block:: scala
val someActor = spawn(classOf[MyActor])
becomes
.. code-block:: scala
val someActor = spawn[MyActor]
Actor.spawnRemote now uses manifests instead of concrete class types
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
.. code-block:: scala
val someActor = spawnRemote(classOf[MyActor],"somehost",1337)
becomes
.. code-block:: scala
val someActor = spawnRemote[MyActor]("somehost",1337)
Actor.spawnLink now uses manifests instead of concrete class types
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
.. code-block:: scala
val someActor = spawnLink(classOf[MyActor])
becomes
.. code-block:: scala
val someActor = spawnLink[MyActor]
Actor.spawnLinkRemote now uses manifests instead of concrete class types
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
.. code-block:: scala
val someActor = spawnLinkRemote(classOf[MyActor],"somehost",1337)
becomes
.. code-block:: scala
val someActor = spawnLinkRemote[MyActor]("somehost",1337)
**Transaction.atomic and friends are moved into Transaction.Local._ and Transaction.Global._**
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
We now make a difference between transaction management that are local within a thread and global across many threads (and actors).

View file

@ -1,172 +0,0 @@
Migration Guide 0.8.x to 0.9.x
==============================
**This document describes between the 0.8.x and the 0.9 release.**
Background for the new ActorRef
-------------------------------
In the work towards 0.9 release we have now done a major change to how Actors are created. In short we have separated identity and value, created an 'ActorRef' that holds the actual Actor instance. This allows us to do many great things such as for example:
* Create serializable, immutable, network-aware Actor references that can be freely shared across the network. They "remember" their origin and will always work as expected.
* Not only kill and restart the same supervised Actor instance when it has crashed (as we do now), but dereference it, throw it away and make it eligible for garbage collection.
* etc. much more
These work very much like the 'PID' (process id) in Erlang.
These changes means that there is no difference in defining Actors. You still use the old Actor trait, all methods are there etc. But you can't just new this Actor up and send messages to it since all its public API methods are gone. They now reside in a new class; 'ActorRef' and use need to use instances of this class to interact with the Actor (sending messages etc.).
Here is a short migration guide with the things that you have to change. It is a big conceptual change but in practice you don't have to change much.
Creating Actors with default constructor
----------------------------------------
From:
.. code-block:: scala
val a = new MyActor
a ! msg
To:
.. code-block:: scala
import Actor._
val a = actorOf[MyActor]
a ! msg
You can also start it in the same statement:
.. code-block:: scala
val a = actorOf[MyActor]
Creating Actors with non-default constructor
--------------------------------------------
From:
.. code-block:: scala
val a = new MyActor(..)
a ! msg
To:
.. code-block:: scala
import Actor._
val a = actorOf(new MyActor(..))
a ! msg
Use of 'self' ActorRef API
--------------------------
Where you have used 'this' to refer to the Actor from within itself now use 'self':
.. code-block:: scala
self ! MessageToMe
Now the Actor trait only has the callbacks you can implement:
* receive
* postRestart/preRestart
* init/shutdown
It has no state at all.
All API has been moved to ActorRef. The Actor is given its ActorRef through the 'self' member variable.
Here you find functions like:
* !, !!, !!! and forward
* link, unlink, startLink, spawnLink etc
* makeTransactional, makeRemote etc.
* start, stop
* etc.
Here you also find fields like
* dispatcher = ...
* id = ...
* lifeCycle = ...
* faultHandler = ...
* trapExit = ...
* etc.
This means that to use them you have to prefix them with 'self', like this:
.. code-block:: scala
self ! Message
However, for convenience you can import these functions and fields like below, which will allow you do drop the 'self' prefix:
.. code-block:: scala
class MyActor extends Actor {
import self._
id = ...
dispatcher = ...
spawnLink[OtherActor]
...
}
Serialization
-------------
If you want to serialize it yourself, here is how to do it:
.. code-block:: scala
val actorRef1 = actorOf[MyActor]
val bytes = actorRef1.toBinary
val actorRef2 = ActorRef.fromBinary(bytes)
If you are also using Protobuf then you can use the methods that work with Protobuf's Messages directly.
.. code-block:: scala
val actorRef1 = actorOf[MyActor]
val protobufMessage = actorRef1.toProtocol
val actorRef2 = ActorRef.fromProtocol(protobufMessage)
Camel
-----
Some methods of the se.scalablesolutions.akka.camel.Message class have been deprecated in 0.9. These are
.. code-block:: scala
package se.scalablesolutions.akka.camel
case class Message(...) {
// ...
@deprecated def bodyAs[T](clazz: Class[T]): T
@deprecated def setBodyAs[T](clazz: Class[T]): Message
// ...
}
They will be removed in 1.0. Instead use
.. code-block:: scala
package se.scalablesolutions.akka.camel
case class Message(...) {
// ...
def bodyAs[T](implicit m: Manifest[T]): T =
def setBodyAs[T](implicit m: Manifest[T]): Message
// ...
}
Usage example:
.. code-block:: scala
val m = Message(1.4)
val b = m.bodyAs[String]

View file

@ -1,47 +0,0 @@
Migration Guide 0.9.x to 0.10.x
===============================
Module akka-camel
-----------------
The following list summarizes the breaking changes since Akka 0.9.1.
* CamelService moved from package se.scalablesolutions.akka.camel.service one level up to se.scalablesolutions.akka.camel.
* CamelService.newInstance removed. For starting and stopping a CamelService, applications should use
* CamelServiceManager.startCamelService and
* CamelServiceManager.stopCamelService.
* Existing def receive = produce method definitions from Producer implementations must be removed (resolves compile error: method receive needs override modifier).
* The Producer.async method and the related Sync trait have been removed. This is now fully covered by Camel's `asynchronous routing engine <http://camel.apache.org/asynchronous-processing.html>`_.
* @consume annotation can not placed any longer on actors (i.e. on type-level), only on typed actor methods. Consumer actors must mixin the Consumer trait.
* @consume annotation moved to package se.scalablesolutions.akka.camel.
Logging
-------
We've switched to Logback (SLF4J compatible) for the logging, if you're having trouble seeing your log output you'll need to make sure that there's a logback.xml available on the classpath or you'll need to specify the location of the logback.xml file via the system property, ex: -Dlogback.configurationFile=/path/to/logback.xml
Configuration
-------------
* The configuration is now JSON-style (see below).
* Now you can define the time-unit to be used throughout the config file:
.. code-block:: ruby
akka {
version = "0.10"
time-unit = "seconds" # default timeout time unit for all timeout properties throughout the config
actor {
timeout = 5 # default timeout for future based invocations
throughput = 5 # default throughput for Dispatcher
}
...
}
RemoteClient events
-------------------
All events now has a reference to the RemoteClient instance instead of 'hostname' and 'port'. This is more flexible. Enables simpler reconnecting etc.

View file

@ -1,78 +0,0 @@
.. _migration-1.1:
################################
Migration Guide 1.0.x to 1.1.x
################################
**Akka has now moved to Scala 2.9.x**
Akka Actor
==========
- is now dependency free, with the exception of the dependency on the
``scala-library.jar``
- does not bundle any logging anymore, but you can subscribe to events within
Akka by registering an event handler on akka.event.EventHandler or by specifying
the ``FQN`` of an Actor in the akka.conf under akka.event-handlers; there is an
``akka-slf4j`` module which still provides the Logging trait and a default
``SLF4J`` logger adapter.
Don't forget to add a SLF4J backend though, we recommend:
.. code-block:: scala
lazy val logback = "ch.qos.logback" % "logback-classic" % "0.9.28" % "runtime"
- If you used HawtDispatcher and want to continue using it, you need to include
akka-dispatcher-extras.jar from Akka Modules, in your akka.conf you need to
specify: ``akka.dispatch.HawtDispatcherConfigurator`` instead of
``HawtDispatcher``
- FSM: the onTransition method changed from Function1 to PartialFunction; there
is an implicit conversion for the precise types in place, but it may be
necessary to add an underscore if you are passing an eta-expansion (using a
method as function value).
Akka Typed Actor
================
- All methods starting with ``get*`` are deprecated and will be removed in post
1.1 release.
Akka Remote
===========
- ``UnparsebleException`` has been renamed to
``CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorException(exception,
classname, message)``
Akka HTTP
=========
- akka.servlet.Initializer has been moved to ``akka-kernel`` to be able to have
``akka-http`` not depend on ``akka-remote``. If you don't want to use the class
for kernel, just create your own version of ``akka.servlet.Initializer``, it's
just a couple of lines of code and there are instructions in
the :ref:`http-module` docs.
- akka.http.ListWriter has been removed in full, if you use it and want to keep
using it, here's the code: `ListWriter`_.
- Jersey-server is now a "provided" dependency for ``akka-http``, so you'll need
to add the dependency to your project, it's built against Jersey 1.3
.. _ListWriter: https://github.com/jboner/akka/blob/v1.0/akka-http/src/main/scala/akka/http/ListWriter.scala
Akka Testkit
============
- The TestKit moved into the akka-testkit subproject and correspondingly into the
``akka.testkit`` package.

View file

@ -1,6 +0,0 @@
.. _migration-1.2:
################################
Migration Guide 1.1.x to 1.2.x
################################

Some files were not shown because too many files have changed in this diff Show more