Merge branch 'master' of github.com:jboner/akka
This commit is contained in:
commit
f59b4c642f
43 changed files with 985 additions and 1584 deletions
|
|
@ -205,7 +205,7 @@ object ActorModelSpec {
|
|||
await(deadline)(stats.restarts.get() == restarts)
|
||||
} catch {
|
||||
case e ⇒
|
||||
system.eventStream.publish(Error(e, dispatcher.toString, "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions +
|
||||
system.eventStream.publish(Error(e, Option(dispatcher).toString, "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions +
|
||||
",res=" + resumes + ",reg=" + registers + ",unreg=" + unregisters +
|
||||
",recv=" + msgsReceived + ",proc=" + msgsProcessed + ",restart=" + restarts))
|
||||
throw e
|
||||
|
|
|
|||
|
|
@ -20,7 +20,6 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) {
|
|||
val config = settings.config
|
||||
import config._
|
||||
|
||||
getList("akka.boot").asScala.toSeq must equal(Nil)
|
||||
getString("akka.version") must equal("2.0-SNAPSHOT")
|
||||
settings.ConfigVersion must equal("2.0-SNAPSHOT")
|
||||
|
||||
|
|
|
|||
|
|
@ -12,9 +12,6 @@ akka {
|
|||
# Home directory of Akka, modules in the deploy directory will be loaded
|
||||
home = ""
|
||||
|
||||
# Comma separated list of the enabled modules. Options: ["cluster", "camel", "http"]
|
||||
enabled-modules = []
|
||||
|
||||
# Event handlers to register at boot time (Logging$DefaultLogger logs to STDOUT)
|
||||
event-handlers = ["akka.event.Logging$DefaultLogger"]
|
||||
|
||||
|
|
@ -35,15 +32,6 @@ akka {
|
|||
# FIXME: clarify "extensions" here, "Akka Extensions (<link to docs>)"
|
||||
extensions = []
|
||||
|
||||
# These boot classes are loaded (and created) automatically when the Akka Microkernel boots up
|
||||
# Can be used to bootstrap your application(s)
|
||||
# Should be the FQN (Fully Qualified Name) of the boot class which needs to have a default constructor
|
||||
# boot = ["sample.camel.Boot",
|
||||
# "sample.rest.java.Boot",
|
||||
# "sample.rest.scala.Boot",
|
||||
# "sample.security.Boot"]
|
||||
boot = []
|
||||
|
||||
actor {
|
||||
|
||||
provider = "akka.actor.LocalActorRefProvider"
|
||||
|
|
|
|||
|
|
@ -281,7 +281,8 @@ trait Actor {
|
|||
// =========================================
|
||||
|
||||
private[akka] final def apply(msg: Any) = {
|
||||
val behaviorStack = context.hotswap
|
||||
// FIXME this should all go into ActorCell
|
||||
val behaviorStack = context.asInstanceOf[ActorCell].hotswap
|
||||
msg match {
|
||||
case msg if behaviorStack.nonEmpty && behaviorStack.head.isDefinedAt(msg) ⇒ behaviorStack.head.apply(msg)
|
||||
case msg if behaviorStack.isEmpty && processingBehavior.isDefinedAt(msg) ⇒ processingBehavior.apply(msg)
|
||||
|
|
|
|||
|
|
@ -67,7 +67,7 @@ trait ActorContext extends ActorRefFactory {
|
|||
def setReceiveTimeout(timeout: Duration): Unit
|
||||
|
||||
/**
|
||||
* Resets the current receive timeout.
|
||||
* Clears the receive timeout, i.e. deactivates this feature.
|
||||
*/
|
||||
def resetReceiveTimeout(): Unit
|
||||
|
||||
|
|
@ -83,16 +83,6 @@ trait ActorContext extends ActorRefFactory {
|
|||
*/
|
||||
def unbecome(): Unit
|
||||
|
||||
/**
|
||||
* Returns the current message envelope.
|
||||
*/
|
||||
def currentMessage: Envelope
|
||||
|
||||
/**
|
||||
* Returns a stack with the hotswapped behaviors (as Scala PartialFunction).
|
||||
*/
|
||||
def hotswap: Stack[PartialFunction[Any, Unit]]
|
||||
|
||||
/**
|
||||
* Returns the sender 'ActorRef' of the current message.
|
||||
*/
|
||||
|
|
@ -109,10 +99,6 @@ trait ActorContext extends ActorRefFactory {
|
|||
*/
|
||||
implicit def dispatcher: MessageDispatcher
|
||||
|
||||
def handleFailure(child: ActorRef, cause: Throwable): Unit
|
||||
|
||||
def handleChildTerminated(child: ActorRef): Unit
|
||||
|
||||
/**
|
||||
* The system that the actor belongs to.
|
||||
* Importing this member will place a implicit MessageDispatcher in scope.
|
||||
|
|
|
|||
|
|
@ -18,34 +18,57 @@ import akka.event.LoggingAdapter
|
|||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
|
||||
/**
|
||||
* ActorRef is an immutable and serializable handle to an Actor.
|
||||
* <p/>
|
||||
* Create an ActorRef for an Actor by using the factory method on the Actor object.
|
||||
* <p/>
|
||||
* Here is an example on how to create an actor with a default constructor.
|
||||
* <pre>
|
||||
* import Actor._
|
||||
* Immutable and serializable handle to an actor, which may or may not reside
|
||||
* on the local host or inside the same [[akka.actor.ActorSystem]]. An ActorRef
|
||||
* can be obtained from an [[akka.actor.ActorRefFactory]], an interface which
|
||||
* is implemented by ActorSystem and [[akka.actor.ActorContext]]. This means
|
||||
* actors can be created top-level in the ActorSystem or as children of an
|
||||
* existing actor, but only from within that actor.
|
||||
*
|
||||
* val actor = actorOf(Props[MyActor]
|
||||
* actor ! message
|
||||
* actor.stop()
|
||||
* </pre>
|
||||
* ActorRefs can be freely shared among actors by message passing. Message
|
||||
* passing conversely is their only purpose, as demonstrated in the following
|
||||
* examples:
|
||||
*
|
||||
* You can also create and start actors like this:
|
||||
* <pre>
|
||||
* val actor = actorOf(Props[MyActor]
|
||||
* </pre>
|
||||
* Scala:
|
||||
* {{{
|
||||
* class ExampleActor extends Actor {
|
||||
* val other = context.actorOf(Props[OtherActor], "childName") // will be destroyed and re-created upon restart by default
|
||||
*
|
||||
* Here is an example on how to create an actor with a non-default constructor.
|
||||
* <pre>
|
||||
* import Actor._
|
||||
* def receive {
|
||||
* case Request1(msg) => other ! refine(msg) // uses this actor as sender reference, reply goes to us
|
||||
* case Request2(msg) => other.tell(msg, sender) // forward sender reference, enabling direct reply
|
||||
* case Request3(msg) => sender ! (other ? msg) // will reply with a Future for holding other’s reply (implicit timeout from "akka.actor.timeout")
|
||||
* }
|
||||
* }
|
||||
* }}}
|
||||
*
|
||||
* val actor = actorOf(Props(new MyActor(...))
|
||||
* actor ! message
|
||||
* actor.stop()
|
||||
* </pre>
|
||||
* Java:
|
||||
* {{{
|
||||
* public class ExampleActor Extends UntypedActor {
|
||||
* // this child will be destroyed and re-created upon restart by default
|
||||
* final ActorRef other = getContext().actorOf(new Props(OtherActor.class), "childName");
|
||||
*
|
||||
* The natural ordering of ActorRef is defined in terms of its [[akka.actor.ActorPath]].
|
||||
* @Override
|
||||
* public void onReceive(Object o) {
|
||||
* if (o instanceof Request1) {
|
||||
* val msg = ((Request1) o).getMsg();
|
||||
* other.tell(msg); // uses this actor as sender reference, reply goes to us
|
||||
*
|
||||
* } else if (o instanceof Request2) {
|
||||
* val msg = ((Request2) o).getMsg();
|
||||
* other.tell(msg, getSender()); // forward sender reference, enabling direct reply
|
||||
*
|
||||
* } else if (o instanceof Request3) {
|
||||
* val msg = ((Request3) o).getMsg();
|
||||
* getSender().tell(other.ask(msg, 5000)); // reply with Future for holding the other’s reply (timeout 5 seconds)
|
||||
*
|
||||
* }
|
||||
* }
|
||||
* }
|
||||
* }}}
|
||||
*
|
||||
* ActorRef does not have a method for terminating the actor it points to, use
|
||||
* [[akka.actor.ActorRefFactory]]`.stop(child)` for this purpose.
|
||||
*/
|
||||
abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable {
|
||||
scalaRef: InternalActorRef ⇒
|
||||
|
|
|
|||
|
|
@ -356,6 +356,9 @@ class LocalActorRefProvider(
|
|||
case StopChild(child) ⇒ context.stop(child); sender ! "ok"
|
||||
case m ⇒ deadLetters ! DeadLetter(m, sender, self)
|
||||
}
|
||||
|
||||
// guardian MUST NOT lose its children during restart
|
||||
override def preRestart(cause: Throwable, msg: Option[Any]) {}
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
@ -373,6 +376,9 @@ class LocalActorRefProvider(
|
|||
case StopChild(child) ⇒ context.stop(child); sender ! "ok"
|
||||
case m ⇒ deadLetters ! DeadLetter(m, sender, self)
|
||||
}
|
||||
|
||||
// guardian MUST NOT lose its children during restart
|
||||
override def preRestart(cause: Throwable, msg: Option[Any]) {}
|
||||
}
|
||||
|
||||
private val guardianFaultHandlingStrategy = {
|
||||
|
|
|
|||
|
|
@ -99,9 +99,6 @@ object ActorSystem {
|
|||
case "" ⇒ None
|
||||
case x ⇒ Some(x)
|
||||
}
|
||||
val BootClasses: Seq[String] = getStringList("akka.boot").asScala
|
||||
|
||||
val EnabledModules: Seq[String] = getStringList("akka.enabled-modules").asScala
|
||||
|
||||
val SchedulerTickDuration = Duration(getMilliseconds("akka.scheduler.tickDuration"), MILLISECONDS)
|
||||
val SchedulerTicksPerWheel = getInt("akka.scheduler.ticksPerWheel")
|
||||
|
|
|
|||
|
|
@ -1,66 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.actor
|
||||
|
||||
import java.io.File
|
||||
import java.net.{ URL, URLClassLoader }
|
||||
import java.util.jar.JarFile
|
||||
import akka.util.Bootable
|
||||
|
||||
/**
|
||||
* Handles all modules in the deploy directory (load and unload)
|
||||
*/
|
||||
trait BootableActorLoaderService extends Bootable {
|
||||
|
||||
def system: ActorSystem
|
||||
|
||||
val BOOT_CLASSES = system.settings.BootClasses
|
||||
lazy val applicationLoader = createApplicationClassLoader()
|
||||
|
||||
protected def createApplicationClassLoader(): Option[ClassLoader] = Some({
|
||||
if (system.settings.Home.isDefined) {
|
||||
val DEPLOY = system.settings.Home.get + "/deploy"
|
||||
val DEPLOY_DIR = new File(DEPLOY)
|
||||
if (!DEPLOY_DIR.exists) {
|
||||
System.exit(-1)
|
||||
}
|
||||
val filesToDeploy = DEPLOY_DIR.listFiles.toArray.toList
|
||||
.asInstanceOf[List[File]].filter(_.getName.endsWith(".jar"))
|
||||
var dependencyJars: List[URL] = Nil
|
||||
filesToDeploy.map { file ⇒
|
||||
val jarFile = new JarFile(file)
|
||||
val en = jarFile.entries
|
||||
while (en.hasMoreElements) {
|
||||
val name = en.nextElement.getName
|
||||
if (name.endsWith(".jar")) dependencyJars ::= new File(
|
||||
String.format("jar:file:%s!/%s", jarFile.getName, name)).toURI.toURL
|
||||
}
|
||||
}
|
||||
val toDeploy = filesToDeploy.map(_.toURI.toURL)
|
||||
val allJars = toDeploy ::: dependencyJars
|
||||
|
||||
new URLClassLoader(allJars.toArray, Thread.currentThread.getContextClassLoader)
|
||||
} else Thread.currentThread.getContextClassLoader
|
||||
})
|
||||
|
||||
abstract override def onLoad() = {
|
||||
super.onLoad()
|
||||
|
||||
applicationLoader foreach Thread.currentThread.setContextClassLoader
|
||||
|
||||
for (loader ← applicationLoader; clazz ← BOOT_CLASSES) {
|
||||
loader.loadClass(clazz).newInstance
|
||||
}
|
||||
}
|
||||
|
||||
abstract override def onUnload() = {
|
||||
super.onUnload()
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API for the default JAX-RS/Mist Initializer
|
||||
*/
|
||||
class DefaultBootableActorLoaderService(val system: ActorSystem) extends BootableActorLoaderService
|
||||
|
|
@ -46,15 +46,15 @@ object IO {
|
|||
override def asReadable = this
|
||||
|
||||
def read(len: Int)(implicit actor: Actor with IO): ByteString @cps[IOSuspendable[Any]] = shift { cont: (ByteString ⇒ IOSuspendable[Any]) ⇒
|
||||
ByteStringLength(cont, this, actor.context.currentMessage, len)
|
||||
ByteStringLength(cont, this, actor.context.asInstanceOf[ActorCell].currentMessage, len)
|
||||
}
|
||||
|
||||
def read()(implicit actor: Actor with IO): ByteString @cps[IOSuspendable[Any]] = shift { cont: (ByteString ⇒ IOSuspendable[Any]) ⇒
|
||||
ByteStringAny(cont, this, actor.context.currentMessage)
|
||||
ByteStringAny(cont, this, actor.context.asInstanceOf[ActorCell].currentMessage)
|
||||
}
|
||||
|
||||
def read(delimiter: ByteString, inclusive: Boolean = false)(implicit actor: Actor with IO): ByteString @cps[IOSuspendable[Any]] = shift { cont: (ByteString ⇒ IOSuspendable[Any]) ⇒
|
||||
ByteStringDelimited(cont, this, actor.context.currentMessage, delimiter, inclusive, 0)
|
||||
ByteStringDelimited(cont, this, actor.context.asInstanceOf[ActorCell].currentMessage, delimiter, inclusive, 0)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -158,7 +158,7 @@ trait IO {
|
|||
}
|
||||
run()
|
||||
case msg if _next ne Idle ⇒
|
||||
_messages enqueue context.currentMessage
|
||||
_messages enqueue context.asInstanceOf[ActorCell].currentMessage
|
||||
case msg if _receiveIO.isDefinedAt(msg) ⇒
|
||||
_next = reset { _receiveIO(msg); Idle }
|
||||
run()
|
||||
|
|
|
|||
|
|
@ -1,91 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.util
|
||||
import akka.actor.ActorSystem
|
||||
|
||||
/*
|
||||
* This class is responsible for booting up a stack of bundles and then shutting them down
|
||||
*/
|
||||
class AkkaLoader(system: ActorSystem) {
|
||||
private val hasBooted = new Switch(false)
|
||||
|
||||
@volatile
|
||||
private var _bundles: Option[Bootable] = None
|
||||
|
||||
def bundles = _bundles;
|
||||
|
||||
/*
|
||||
* Boot initializes the specified bundles
|
||||
*/
|
||||
def boot(withBanner: Boolean, b: Bootable): Unit = hasBooted switchOn {
|
||||
if (withBanner) printBanner()
|
||||
println("Starting Akka...")
|
||||
b.onLoad()
|
||||
Thread.currentThread.setContextClassLoader(getClass.getClassLoader)
|
||||
_bundles = Some(b)
|
||||
println("Akka started successfully")
|
||||
}
|
||||
|
||||
/*
|
||||
* Shutdown, well, shuts down the bundles used in boot
|
||||
*/
|
||||
def shutdown() {
|
||||
hasBooted switchOff {
|
||||
println("Shutting down Akka...")
|
||||
_bundles.foreach(_.onUnload())
|
||||
_bundles = None
|
||||
println("Akka succesfully shut down")
|
||||
}
|
||||
}
|
||||
|
||||
private def printBanner() {
|
||||
println("""
|
||||
==============================================================================
|
||||
|
||||
ZZ:
|
||||
ZZZZ
|
||||
ZZZZZZ
|
||||
ZZZ' ZZZ
|
||||
~7 7ZZ' ZZZ
|
||||
:ZZZ: IZZ' ZZZ
|
||||
,OZZZZ.~ZZ? ZZZ
|
||||
ZZZZ' 'ZZZ$ ZZZ
|
||||
. $ZZZ ~ZZ$ ZZZ
|
||||
.=Z?. .ZZZO ~ZZ7 OZZ
|
||||
.ZZZZ7..:ZZZ~ 7ZZZ ZZZ~
|
||||
.$ZZZ$Z+.ZZZZ ZZZ: ZZZ$
|
||||
.,ZZZZ?' =ZZO= .OZZ 'ZZZ
|
||||
.$ZZZZ+ .ZZZZ IZZZ ZZZ$
|
||||
.ZZZZZ' .ZZZZ' .ZZZ$ ?ZZZ
|
||||
.ZZZZZZ' .OZZZ? ?ZZZ 'ZZZ$
|
||||
.?ZZZZZZ' .ZZZZ? .ZZZ? 'ZZZO
|
||||
.+ZZZZZZ?' .7ZZZZ' .ZZZZ :ZZZZ
|
||||
.ZZZZZZ$' .?ZZZZZ' .~ZZZZ 'ZZZZ.
|
||||
|
||||
|
||||
NNNNN $NNNN+
|
||||
NNNNN $NNNN+
|
||||
NNNNN $NNNN+
|
||||
NNNNN $NNNN+
|
||||
NNNNN $NNNN+
|
||||
=NNNNNNNNND$ NNNNN DDDDDD: $NNNN+ DDDDDN NDDNNNNNNNN,
|
||||
NNNNNNNNNNNNND NNNNN DNNNNN $NNNN+ 8NNNNN= :NNNNNNNNNNNNNN
|
||||
NNNNN$ DNNNNN NNNNN $NNNNN~ $NNNN+ NNNNNN NNNNN, :NNNNN+
|
||||
?DN~ NNNNN NNNNN MNNNNN $NNNN+:NNNNN7 $ND =NNNNN
|
||||
DNNNNN NNNNNDNNNN$ $NNNNDNNNNN :DNNNNN
|
||||
ZNDNNNNNNNNND NNNNNNNNNND, $NNNNNNNNNNN DNDNNNNNNNNNN
|
||||
NNNNNNNDDINNNNN NNNNNNNNNNND $NNNNNNNNNNND ONNNNNNND8+NNNNN
|
||||
:NNNND NNNNN NNNNNN DNNNN, $NNNNNO 7NNNND NNNNNO :NNNNN
|
||||
DNNNN NNNNN NNNNN DNNNN $NNNN+ 8NNNNN NNNNN $NNNNN
|
||||
DNNNNO NNNNNN NNNNN NNNNN $NNNN+ NNNNN$ NNNND, ,NNNNND
|
||||
NNNNNNDDNNNNNNNN NNNNN =NNNNN $NNNN+ DNNNN? DNNNNNNDNNNNNNNND
|
||||
NNNNNNNNN NNNN$ NNNNN 8NNNND $NNNN+ NNNNN= ,DNNNNNNND NNNNN$
|
||||
|
||||
==============================================================================
|
||||
Running version %s
|
||||
==============================================================================
|
||||
""".format(ActorSystem.Version))
|
||||
}
|
||||
}
|
||||
|
|
@ -1,11 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.util
|
||||
import akka.actor.ActorSystem
|
||||
|
||||
trait Bootable {
|
||||
def onLoad() {}
|
||||
def onUnload() {}
|
||||
}
|
||||
|
|
@ -84,14 +84,6 @@ to use from an sbt project) use the ``publish-local`` command::
|
|||
sbt publish-local
|
||||
|
||||
|
||||
Publish to local Maven repository
|
||||
---------------------------------
|
||||
|
||||
If you want to deploy the artifacts to your local Maven repository use::
|
||||
|
||||
sbt publish
|
||||
|
||||
|
||||
sbt interactive mode
|
||||
--------------------
|
||||
|
||||
|
|
@ -129,6 +121,6 @@ Dependencies
|
|||
You can look at the Ivy dependency resolution information that is created on
|
||||
``sbt update`` and found in ``~/.ivy2/cache``. For example, the
|
||||
``~/.ivy2/cache/com.typesafe.akka-akka-remote-compile.xml`` file contains
|
||||
the resolution information for the akka-cluster module compile dependencies. If
|
||||
the resolution information for the akka-remote module compile dependencies. If
|
||||
you open this file in a web browser you will get an easy to navigate view of
|
||||
dependencies.
|
||||
|
|
|
|||
147
akka-docs/disabled/agents.rst
Normal file
147
akka-docs/disabled/agents.rst
Normal file
|
|
@ -0,0 +1,147 @@
|
|||
Agents (Scala)
|
||||
==============
|
||||
|
||||
.. sidebar:: Contents
|
||||
|
||||
.. contents:: :local:
|
||||
|
||||
Agents in Akka were inspired by `agents in Clojure <http://clojure.org/agents>`_.
|
||||
|
||||
Agents provide asynchronous change of individual locations. Agents are bound to a single storage location for their lifetime, and only allow mutation of that location (to a new state) to occur as a result of an action. Update actions are functions that are asynchronously applied to the Agent's state and whose return value becomes the Agent's new state. The state of an Agent should be immutable.
|
||||
|
||||
While updates to Agents are asynchronous, the state of an Agent is always immediately available for reading by any thread (using ``get`` or ``apply``) without any messages.
|
||||
|
||||
Agents are reactive. The update actions of all Agents get interleaved amongst threads in a thread pool. At any point in time, at most one ``send`` action for each Agent is being executed. Actions dispatched to an agent from another thread will occur in the order they were sent, potentially interleaved with actions dispatched to the same agent from other sources.
|
||||
|
||||
If an Agent is used within an enclosing transaction, then it will participate in that transaction. 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.
|
||||
|
||||
Creating and stopping Agents
|
||||
----------------------------
|
||||
|
||||
Agents are created by invoking ``Agent(value)`` passing in the Agent's initial value.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
val agent = Agent(5)
|
||||
|
||||
An Agent will be running until you invoke ``close`` on it. Then it will be eligible for garbage collection (unless you hold on to it in some way).
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
agent.close()
|
||||
|
||||
Updating Agents
|
||||
---------------
|
||||
|
||||
You update an Agent by sending a function that transforms the current value or by sending just a new value. The Agent will apply the new value or function atomically and asynchronously. The update is done in a fire-forget manner and you are only guaranteed that it will be applied. There is no guarantee of when the update will be applied but dispatches to an Agent from a single thread will occur in order. You apply a value or a function by invoking the ``send`` function.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
// send a value
|
||||
agent send 7
|
||||
|
||||
// send a function
|
||||
agent send (_ + 1)
|
||||
agent send (_ * 2)
|
||||
|
||||
You can also dispatch a function to update the internal state but on its own thread. This does not use the reactive thread pool and can be used for long-running or blocking operations. You do this with the ``sendOff`` method. Dispatches using either ``sendOff`` or ``send`` will still be executed in order.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
// sendOff a function
|
||||
agent sendOff (longRunningOrBlockingFunction)
|
||||
|
||||
Reading an Agent's value
|
||||
------------------------
|
||||
|
||||
Agents can be dereferenced, e.g. you can get an Agent's value, by invoking the Agent with parenthesis like this:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
val result = agent()
|
||||
|
||||
Or by using the get method.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
val result = agent.get
|
||||
|
||||
Reading an Agent's current value does not involve any message passing and happens immediately. So while updates to an Agent are asynchronous, reading the state of an Agent is synchronous.
|
||||
|
||||
Awaiting an Agent's value
|
||||
-------------------------
|
||||
|
||||
It is also possible to read the value after all currently queued ``send``\s have completed. You can do this with ``await``:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
val result = agent.await
|
||||
|
||||
You can also get a ``Future`` to this value, that will be completed after the currently queued updates have completed:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
val future = agent.future
|
||||
// ...
|
||||
val result = future.await.result.get
|
||||
|
||||
Transactional Agents
|
||||
--------------------
|
||||
|
||||
If an Agent is used within an enclosing transaction, then it will participate in that transaction. If you send to an Agent within a transaction then the dispatch to the Agent will be held until that transaction commits, and discarded if the transaction is aborted.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.agent.Agent
|
||||
import akka.stm._
|
||||
|
||||
def transfer(from: Agent[Int], to: Agent[Int], amount: Int): Boolean = {
|
||||
atomic {
|
||||
if (from.get < amount) false
|
||||
else {
|
||||
from send (_ - amount)
|
||||
to send (_ + amount)
|
||||
true
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
val from = Agent(100)
|
||||
val to = Agent(20)
|
||||
val ok = transfer(from, to, 50)
|
||||
|
||||
from() // -> 50
|
||||
to() // -> 70
|
||||
|
||||
Monadic usage
|
||||
-------------
|
||||
|
||||
Agents are also monadic, allowing you to compose operations using for-comprehensions. In a monadic usage, new Agents are created leaving the original Agents untouched. So the old values (Agents) are still available as-is. They are so-called 'persistent'.
|
||||
|
||||
Example of a monadic usage:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
val agent1 = Agent(3)
|
||||
val agent2 = Agent(5)
|
||||
|
||||
// uses foreach
|
||||
var result = 0
|
||||
for (value <- agent1) {
|
||||
result = value + 1
|
||||
}
|
||||
|
||||
// uses map
|
||||
val agent3 =
|
||||
for (value <- agent1) yield value + 1
|
||||
|
||||
// uses flatMap
|
||||
val agent4 = for {
|
||||
value1 <- agent1
|
||||
value2 <- agent2
|
||||
} yield value1 + value2
|
||||
|
||||
agent1.close()
|
||||
agent2.close()
|
||||
agent3.close()
|
||||
agent4.close()
|
||||
|
|
@ -8,13 +8,13 @@
|
|||
How can I use and deploy Akka?
|
||||
==============================
|
||||
|
||||
Akka can be used in two different ways:
|
||||
Akka can be used in different ways:
|
||||
|
||||
- As a library: used as a regular JAR on the classpath and/or in a web app, to
|
||||
be put into ``WEB-INF/lib``
|
||||
|
||||
- As a microkernel: stand-alone microkernel, embedding a servlet container along
|
||||
with many other services
|
||||
- As a stand alone application by instantiating ActorSystem in a main class or
|
||||
using the :ref:`microkernel`
|
||||
|
||||
|
||||
Using Akka as library
|
||||
|
|
@ -38,48 +38,7 @@ on other hosts. Please note that remoting service does not speak HTTP over port
|
|||
|
||||
|
||||
Using Akka as a stand alone microkernel
|
||||
---------------------------------------
|
||||
----------------------------------------
|
||||
|
||||
Akka can also be run as a stand-alone microkernel. It implements a full
|
||||
enterprise stack. See the :ref:`microkernel` for more information.
|
||||
|
||||
Using the Akka sbt plugin to package your application
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
The Akka sbt plugin can create a full Akka microkernel deployment for your sbt
|
||||
project.
|
||||
|
||||
To use the plugin, first add a plugin definition to your sbt project by creating
|
||||
``project/plugins.sbt`` with::
|
||||
|
||||
resolvers += Classpaths.typesafeResolver
|
||||
|
||||
addSbtPlugin("com.typesafe.akka" % "akka-sbt-plugin" % "2.0-SNAPSHOT")
|
||||
|
||||
Then use the AkkaKernelPlugin settings. In a 'light' configuration (build.sbt)::
|
||||
|
||||
seq(akka.sbt.AkkaKernelPlugin.distSettings: _*)
|
||||
|
||||
Or in a 'full' configuration (Build.scala). For example::
|
||||
|
||||
import sbt._
|
||||
import sbt.Keys._
|
||||
import akka.sbt.AkkaKernelPlugin
|
||||
|
||||
object SomeBuild extends Build {
|
||||
lazy val someProject = Project(
|
||||
id = "some-project",
|
||||
base = file("."),
|
||||
settings = Defaults.defaultSettings ++ AkkaKernelPlugin.distSettings ++ Seq(
|
||||
organization := "org.some",
|
||||
version := "0.1",
|
||||
scalaVersion := "2.9.1"
|
||||
resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/",
|
||||
libraryDependencies += "com.typesafe.akka" % "akka-kernel" % "2.0-SNAPSHOT"
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
To build a microkernel deployment use the ``dist`` task::
|
||||
|
||||
sbt dist
|
||||
Akka can also be run as a stand-alone microkernel. See :ref:`microkernel` for
|
||||
more information.
|
||||
|
|
|
|||
|
|
@ -63,10 +63,10 @@ Downloading and installing Akka
|
|||
-------------------------------
|
||||
|
||||
To build and run the tutorial sample from the command line, you have to download
|
||||
Akka. If you prefer to use SBT to build and run the sample then you can skip
|
||||
this section and jump to the next one.
|
||||
Akka. If you prefer to use SBT to build and run the sample then you can skip this
|
||||
section and jump to the next one.
|
||||
|
||||
Let's get the ``akka-actors-2.0-SNAPSHOT.zip`` distribution of Akka from
|
||||
Let's get the ``akka-2.0-SNAPSHOT.zip`` distribution of Akka from
|
||||
http://akka.io/downloads/ which includes everything we need for this
|
||||
tutorial. Once you have downloaded the distribution unzip it in the folder you
|
||||
would like to have Akka installed in. In my case I choose to install it in
|
||||
|
|
@ -77,46 +77,55 @@ You need to do one more thing in order to install Akka properly: set the
|
|||
I'm opening up a shell, navigating down to the distribution, and setting the
|
||||
``AKKA_HOME`` variable::
|
||||
|
||||
$ cd /Users/jboner/tools/akka-actors-2.0-SNAPSHOT
|
||||
$ cd /Users/jboner/tools/akka-2.0-SNAPSHOT
|
||||
$ export AKKA_HOME=`pwd`
|
||||
$ echo $AKKA_HOME
|
||||
/Users/jboner/tools/akka-actors-2.0-SNAPSHOT
|
||||
/Users/jboner/tools/akka-2.0-SNAPSHOT
|
||||
|
||||
The distribution looks like this::
|
||||
|
||||
$ ls -1
|
||||
bin
|
||||
config
|
||||
deploy
|
||||
doc
|
||||
lib
|
||||
src
|
||||
|
||||
- In the ``bin`` directory we have scripts for starting the Akka Microkernel.
|
||||
- In the ``config`` directory we have the Akka conf files.
|
||||
- In the ``doc`` directory we have the documentation, API, doc JARs, and also
|
||||
the source files for the tutorials.
|
||||
- In the ``deploy`` directory we can place applications to be run with the microkernel.
|
||||
- In the ``doc`` directory we have the documentation, API, and doc JARs.
|
||||
- In the ``lib`` directory we have the Scala and Akka JARs.
|
||||
- In the ``src`` directory we have the source JARs for Akka.
|
||||
|
||||
|
||||
The only JAR we will need for this tutorial (apart from the
|
||||
``scala-library.jar`` JAR) is the ``akka-actor-2.0-SNAPSHOT.jar`` JAR in the ``lib/akka``
|
||||
directory. This is a self-contained JAR with zero dependencies and contains
|
||||
everything we need to write a system using Actors.
|
||||
|
||||
Akka is very modular and has many JARs for containing different features. The core distribution has seven modules:
|
||||
Akka is very modular and has many JARs for containing different features. The
|
||||
modules are:
|
||||
|
||||
- ``akka-actor-2.0-SNAPSHOT.jar`` -- Standard Actors
|
||||
- ``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-slf4j-2.0-SNAPSHOT.jar`` -- SLF4J Event Handler Listener for logging with SLF4J
|
||||
- ``akka-testkit-2.0-SNAPSHOT.jar`` -- Toolkit for testing Actors
|
||||
- ``akka-actor`` -- Actors
|
||||
|
||||
The Akka Microkernel distribution also includes these jars:
|
||||
- ``akka-remote`` -- Remote Actors
|
||||
|
||||
- ``akka-slf4j`` -- SLF4J Event Handler Listener for logging with SLF4J
|
||||
|
||||
- ``akka-testkit`` -- Toolkit for testing Actors
|
||||
|
||||
- ``akka-kernel`` -- Akka microkernel for running a bare-bones mini application server
|
||||
|
||||
- ``akka-durable-mailboxes`` -- Durable mailboxes: file-based, MongoDB, Redis, Zookeeper
|
||||
|
||||
- ``akka-amqp`` -- AMQP integration
|
||||
|
||||
.. - ``akka-stm-2.0-SNAPSHOT.jar`` -- STM (Software Transactional Memory), transactors and transactional datastructures
|
||||
.. - ``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)
|
||||
.. - ``akka-camel-typed-2.0-SNAPSHOT.jar`` -- Apache Camel Typed Actors integration
|
||||
.. - ``akka-spring-2.0-SNAPSHOT.jar`` -- Spring framework integration
|
||||
|
||||
- ``akka-kernel-2.0-SNAPSHOT.jar`` -- Akka microkernel for running a bare-bones mini application server (embeds Jetty etc.)
|
||||
- ``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)
|
||||
- ``akka-camel-typed-2.0-SNAPSHOT.jar`` -- Apache Camel Typed Actors integration
|
||||
- ``akka-spring-2.0-SNAPSHOT.jar`` -- Spring framework integration
|
||||
|
||||
|
||||
Downloading and installing Maven
|
||||
|
|
@ -158,16 +167,19 @@ Here is the layout that Maven created::
|
|||
|
||||
As you can see we already have a Java source file called ``App.java``, let's now rename it to ``Pi.java``.
|
||||
|
||||
We also need to edit the ``pom.xml`` build file. Let's add the dependency we need as well as the Maven repository it should download it from. The Akka Maven repository can be found at `<http://akka.io/repository>`_
|
||||
and Typesafe provides `<http://repo.typesafe.com/typesafe/releases/>`_ that proxies several other repositories, including akka.io.
|
||||
It should now look something like this:
|
||||
We also need to edit the ``pom.xml`` build file. Let's add the dependency we
|
||||
need as well as the Maven repository it should download it from. The Akka Maven
|
||||
repository can be found at http://akka.io/releases/ and Typesafe provides
|
||||
http://repo.typesafe.com/typesafe/releases/ that proxies several other
|
||||
repositories, including akka.io. It should now look something like this:
|
||||
|
||||
.. code-block:: xml
|
||||
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
|
||||
http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<name>akka-tutorial-first-java</name>
|
||||
|
|
@ -213,28 +225,15 @@ Start writing the code
|
|||
|
||||
Now it's about time to start hacking.
|
||||
|
||||
We start by creating a ``Pi.java`` file and adding these import statements at the top of the file::
|
||||
We start by creating a ``Pi.java`` file and adding these import statements at the top of the file:
|
||||
|
||||
package akka.tutorial.first.java;
|
||||
|
||||
import static akka.actor.Actors.actorOf;
|
||||
import static akka.actor.Actors.poisonPill;
|
||||
import static java.util.Arrays.asList;
|
||||
|
||||
import akka.actor.Props;
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.UntypedActor;
|
||||
import akka.actor.UntypedActorFactory;
|
||||
import akka.routing.CyclicIterator;
|
||||
import akka.routing.InfiniteIterator;
|
||||
import akka.routing.Routing.Broadcast;
|
||||
import akka.routing.UntypedLoadBalancer;
|
||||
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java#imports
|
||||
|
||||
If you are using Maven in this tutorial then create the file in the ``src/main/java/akka/tutorial/first/java`` directory.
|
||||
|
||||
If you are using the command line tools then create the file wherever you want. I will create it in a directory called ``tutorial`` at the root of the Akka distribution, e.g. in ``$AKKA_HOME/tutorial/akka/tutorial/first/java/Pi.java``.
|
||||
If you are using the command line tools then create the file wherever you want.
|
||||
We will create it in a directory called ``tutorial`` at the root of the Akka distribution,
|
||||
e.g. in ``$AKKA_HOME/tutorial/akka/tutorial/first/java/Pi.java``.
|
||||
|
||||
Creating the messages
|
||||
---------------------
|
||||
|
|
@ -247,466 +246,101 @@ With this in mind, let's now create the messages that we want to have flowing in
|
|||
- ``Work`` -- sent from the ``Master`` actor to the ``Worker`` actors containing the work assignment
|
||||
- ``Result`` -- sent from the ``Worker`` actors to the ``Master`` actor containing the result from the worker's calculation
|
||||
|
||||
Messages sent to actors should always be immutable to avoid sharing mutable state. So let's start by creating three messages as immutable POJOs. We also create a wrapper ``Pi`` class to hold our implementation::
|
||||
Messages sent to actors should always be immutable to avoid sharing mutable state. So let's start by creating three messages as immutable POJOs. We also create a wrapper ``Pi`` class to hold our implementation:
|
||||
|
||||
public class Pi {
|
||||
|
||||
static class Calculate {}
|
||||
|
||||
static class Work {
|
||||
private final int start;
|
||||
private final int nrOfElements;
|
||||
|
||||
public Work(int start, int nrOfElements) {
|
||||
this.start = start;
|
||||
this.nrOfElements = nrOfElements;
|
||||
}
|
||||
|
||||
public int getStart() { return start; }
|
||||
public int getNrOfElements() { return nrOfElements; }
|
||||
}
|
||||
|
||||
static class Result {
|
||||
private final double value;
|
||||
|
||||
public Result(double value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
public double getValue() { return value; }
|
||||
}
|
||||
}
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java#messages
|
||||
|
||||
Creating the worker
|
||||
-------------------
|
||||
|
||||
Now we can create the worker actor. This is done by extending in the ``UntypedActor`` base class and defining the ``onReceive`` method. The ``onReceive`` method defines our message handler. We expect it to be able to handle the ``Work`` message so we need to add a handler for this message::
|
||||
Now we can create the worker actor. This is done by extending in the ``UntypedActor`` base class and defining the ``onReceive`` method. The ``onReceive`` method defines our message handler. We expect it to be able to handle the ``Work`` message so we need to add a handler for this message:
|
||||
|
||||
static class Worker extends UntypedActor {
|
||||
|
||||
// message handler
|
||||
public void onReceive(Object message) {
|
||||
if (message instanceof Work) {
|
||||
Work work = (Work) message;
|
||||
|
||||
// perform the work
|
||||
double result = calculatePiFor(work.getStart(), work.getNrOfElements());
|
||||
|
||||
// reply with the result
|
||||
getContext().reply(new Result(result));
|
||||
|
||||
} else throw new IllegalArgumentException("Unknown message [" + message + "]");
|
||||
}
|
||||
}
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java#worker
|
||||
:exclude: calculatePiFor
|
||||
|
||||
As you can see we have now created an ``UntypedActor`` with a ``onReceive`` method as a handler for the ``Work`` message. In this handler we invoke the ``calculatePiFor(..)`` method, wrap the result in a ``Result`` message and send it back to the original sender using ``getContext().reply(..)``. In Akka the sender reference is implicitly passed along with the message so that the receiver can always reply or store away the sender reference for future use.
|
||||
|
||||
The only thing missing in our ``Worker`` actor is the implementation on the ``calculatePiFor(..)`` method::
|
||||
The only thing missing in our ``Worker`` actor is the implementation on the ``calculatePiFor(..)`` method:
|
||||
|
||||
// define the work
|
||||
private double calculatePiFor(int start, int nrOfElements) {
|
||||
double acc = 0.0;
|
||||
for (int i = start * nrOfElements; i <= ((start + 1) * nrOfElements - 1); i++) {
|
||||
acc += 4.0 * (1 - (i % 2) * 2) / (2 * i + 1);
|
||||
}
|
||||
return acc;
|
||||
}
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java#calculatePiFor
|
||||
|
||||
Creating the master
|
||||
-------------------
|
||||
|
||||
The master actor is a little bit more involved. In its constructor we need to create the workers (the ``Worker`` actors) and start them. We will also wrap them in a load-balancing router to make it easier to spread out the work evenly between the workers. Let's do that first::
|
||||
The master actor is a little bit more involved. In its constructor we create a round-robin router
|
||||
to make it easier to spread out the work evenly between the workers. Let's do that first:
|
||||
|
||||
static class Master extends UntypedActor {
|
||||
...
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java#create-router
|
||||
|
||||
static class PiRouter extends UntypedLoadBalancer {
|
||||
private final InfiniteIterator<ActorRef> workers;
|
||||
|
||||
public PiRouter(ActorRef[] workers) {
|
||||
this.workers = new CyclicIterator<ActorRef>(asList(workers));
|
||||
}
|
||||
|
||||
public InfiniteIterator<ActorRef> seq() {
|
||||
return workers;
|
||||
}
|
||||
}
|
||||
|
||||
public Master(...) {
|
||||
...
|
||||
|
||||
// create the workers
|
||||
final ActorRef[] workers = new ActorRef[nrOfWorkers];
|
||||
for (int i = 0; i < nrOfWorkers; i++) {
|
||||
workers[i] = actorOf(new Props(Worker.class));
|
||||
}
|
||||
|
||||
// wrap them with a load-balancing router
|
||||
ActorRef router = actorOf(new Props(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new PiRouter(workers);
|
||||
}
|
||||
}));
|
||||
}
|
||||
}
|
||||
|
||||
As you can see we are using the ``actorOf`` factory method to create actors, this method returns as an ``ActorRef`` which is a reference to our newly created actor. This method is available in the ``Actors`` object but is usually imported::
|
||||
|
||||
import static akka.actor.Actors.actorOf;
|
||||
|
||||
One thing to note is that we used two different versions of the ``actorOf`` method. For creating the ``Worker`` actor we just pass in the class but to create the ``PiRouter`` actor we can't do that since the constructor in the ``PiRouter`` class takes arguments, instead we need to use the ``UntypedActorFactory`` which unfortunately is a bit more verbose.
|
||||
|
||||
``actorOf`` is the only way to create an instance of an Actor, this is enforced by Akka runtime. The ``actorOf`` method instantiates the actor and returns, not an instance to the actor, but an instance to an ``ActorRef``. This reference is the handle through which you communicate with the actor. It is immutable, serializable and location-aware meaning that it "remembers" its original actor even if it is sent to other nodes across the network and can be seen as the equivalent to the Erlang actor's PID.
|
||||
|
||||
The actor's life-cycle is:
|
||||
|
||||
- Created & Started -- ``Actor.actorOf(Props[MyActor])`` -- can receive messages
|
||||
- Stopped -- ``actorRef.stop()`` -- can **not** receive messages
|
||||
|
||||
Once the actor has been stopped it is dead and can not be started again.
|
||||
|
||||
Now we have a router that is representing all our workers in a single abstraction. If you paid attention to the code above, you saw that we were using the ``nrOfWorkers`` variable. This variable and others we have to pass to the ``Master`` actor in its constructor. So now let's create the master actor. We have to pass in three integer variables:
|
||||
Now we have a router that is representing all our workers in a single
|
||||
abstraction. So now let's create the master actor. We pass it three integer variables:
|
||||
|
||||
- ``nrOfWorkers`` -- defining how many workers we should start up
|
||||
- ``nrOfMessages`` -- defining how many number chunks to send out to the workers
|
||||
- ``nrOfElements`` -- defining how big the number chunks sent to each worker should be
|
||||
|
||||
Here is the master actor::
|
||||
Here is the master actor:
|
||||
|
||||
static class Master extends UntypedActor {
|
||||
private final int nrOfMessages;
|
||||
private final int nrOfElements;
|
||||
private final CountDownLatch latch;
|
||||
|
||||
private double pi;
|
||||
private int nrOfResults;
|
||||
private long start;
|
||||
|
||||
private ActorRef router;
|
||||
|
||||
static class PiRouter extends UntypedLoadBalancer {
|
||||
private final InfiniteIterator<ActorRef> workers;
|
||||
|
||||
public PiRouter(ActorRef[] workers) {
|
||||
this.workers = new CyclicIterator<ActorRef>(asList(workers));
|
||||
}
|
||||
|
||||
public InfiniteIterator<ActorRef> seq() {
|
||||
return workers;
|
||||
}
|
||||
}
|
||||
|
||||
public Master(
|
||||
int nrOfWorkers, int nrOfMessages, int nrOfElements, CountDownLatch latch) {
|
||||
this.nrOfMessages = nrOfMessages;
|
||||
this.nrOfElements = nrOfElements;
|
||||
this.latch = latch;
|
||||
|
||||
// create the workers
|
||||
final ActorRef[] workers = new ActorRef[nrOfWorkers];
|
||||
for (int i = 0; i < nrOfWorkers; i++) {
|
||||
workers[i] = actorOf(new Props(Worker.class));
|
||||
}
|
||||
|
||||
// wrap them with a load-balancing router
|
||||
router = actorOf(new Props(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new PiRouter(workers);
|
||||
}
|
||||
}));
|
||||
}
|
||||
|
||||
// message handler
|
||||
public void onReceive(Object message) { ... }
|
||||
|
||||
@Override
|
||||
public void preStart() {
|
||||
start = System.currentTimeMillis();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postStop() {
|
||||
// tell the world that the calculation is complete
|
||||
System.out.println(String.format(
|
||||
"\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis",
|
||||
pi, (System.currentTimeMillis() - start)));
|
||||
latch.countDown();
|
||||
}
|
||||
}
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java#master
|
||||
:exclude: handle-messages
|
||||
|
||||
A couple of things are worth explaining further.
|
||||
|
||||
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the ``Master`` actor. This latch is only used for plumbing (in this specific tutorial), to have a simple way of letting the outside world knowing when the master can deliver the result and shut down. In more idiomatic Akka code, as we will see in part two of this tutorial series, we would not use a latch but other abstractions and functions like ``Channel``, ``Future`` and ``ask()`` to achieve the same thing in a non-blocking way. But for simplicity let's stick to a ``CountDownLatch`` for now.
|
||||
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the
|
||||
``Master`` actor. This latch is only used for plumbing (in this specific
|
||||
tutorial), to have a simple way of letting the outside world knowing when the
|
||||
master can deliver the result and shut down. In more idiomatic Akka code
|
||||
we would not use a latch but other abstractions and functions like ``Future``
|
||||
and ``ask()`` to achieve the same thing in a non-blocking way.
|
||||
But for simplicity let's stick to a ``CountDownLatch`` for now.
|
||||
|
||||
Second, we are adding a couple of life-cycle callback methods; ``preStart`` and ``postStop``. In the ``preStart`` callback we are recording the time when the actor is started and in the ``postStop`` callback we are printing out the result (the approximation of Pi) and the time it took to calculate it. In this call we also invoke ``latch.countDown()`` to tell the outside world that we are done.
|
||||
Second, we are adding a couple of life-cycle callback methods; ``preStart`` and
|
||||
``postStop``. In the ``preStart`` callback we are recording the time when the
|
||||
actor is started and in the ``postStop`` callback we are printing out the result
|
||||
(the approximation of Pi) and the time it took to calculate it. In this call we
|
||||
also invoke ``latch.countDown()`` to tell the outside world that we are done.
|
||||
|
||||
But we are not done yet. We are missing the message handler for the ``Master`` actor. This message handler needs to be able to react to two different messages:
|
||||
But we are not done yet. We are missing the message handler for the ``Master`` actor.
|
||||
This message handler needs to be able to react to two different messages:
|
||||
|
||||
- ``Calculate`` -- which should start the calculation
|
||||
- ``Result`` -- which should aggregate the different results
|
||||
|
||||
The ``Calculate`` handler is sending out work to all the ``Worker`` actors and after doing that it also sends a ``new Broadcast(poisonPill())`` message to the router, which will send out the ``PoisonPill`` message to all the actors it is representing (in our case all the ``Worker`` actors). ``PoisonPill`` is a special kind of message that tells the receiver to shut itself down using the normal shutdown method; ``getContext().stop()``, and is created through the ``poisonPill()`` method. We also send a ``PoisonPill`` to the router itself (since it's also an actor that we want to shut down).
|
||||
The ``Calculate`` handler is sending out work to all the ``Worker`` via its router.
|
||||
|
||||
The ``Result`` handler is simpler, here we get the value from the ``Result`` message and aggregate it to our ``pi`` member variable. We also keep track of how many results we have received back, and if that matches the number of tasks sent out, the ``Master`` actor considers itself done and shuts down.
|
||||
The ``Result`` handler gets the value from the ``Result`` message and aggregates it to
|
||||
our ``pi`` member variable. We also keep track of how many results we have received back,
|
||||
and if that matches the number of tasks sent out, the ``Master`` actor considers itself done and
|
||||
invokes the ``self.stop()`` method to stop itself *and* all its supervised actors.
|
||||
In this case it has one supervised actor, the router, and this in turn has ``nrOfWorkers`` supervised actors.
|
||||
All of them will be stopped automatically as the invocation of any supervisor's ``stop`` method
|
||||
will propagate down to all its supervised 'children'.
|
||||
|
||||
Let's capture this in code::
|
||||
Let's capture this in code:
|
||||
|
||||
// message handler
|
||||
public void onReceive(Object message) {
|
||||
|
||||
if (message instanceof Calculate) {
|
||||
// schedule work
|
||||
for (int start = 0; start < nrOfMessages; start++) {
|
||||
router.tell(new Work(start, nrOfElements), getContext());
|
||||
}
|
||||
|
||||
// send a PoisonPill to all workers telling them to shut down themselves
|
||||
router.tell(new Broadcast(poisonPill()));
|
||||
|
||||
// send a PoisonPill to the router, telling him to shut himself down
|
||||
router.tell(poisonPill());
|
||||
|
||||
} else if (message instanceof Result) {
|
||||
|
||||
// handle result from the worker
|
||||
Result result = (Result) message;
|
||||
pi += result.getValue();
|
||||
nrOfResults += 1;
|
||||
if (nrOfResults == nrOfMessages) getContext().stop();
|
||||
|
||||
} else throw new IllegalArgumentException("Unknown message [" + message + "]");
|
||||
}
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java#master-receive
|
||||
|
||||
Bootstrap the calculation
|
||||
-------------------------
|
||||
|
||||
Now the only thing that is left to implement is the runner that should bootstrap and run the calculation for us. We do that by adding a ``main`` method to the enclosing ``Pi`` class in which we create a new instance of ``Pi`` and invoke method ``calculate`` in which we start up the ``Master`` actor and wait for it to finish::
|
||||
Now the only thing that is left to implement is the runner that should bootstrap and run the calculation for us.
|
||||
We do that by adding a ``main`` method to the enclosing ``Pi`` class in which we create a new instance of ``Pi`` and
|
||||
invoke method ``calculate`` in which we start up the ``Master`` actor and wait for it to finish:
|
||||
|
||||
public class Pi {
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java#app
|
||||
:exclude: actors-and-messages
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
Pi pi = new Pi();
|
||||
pi.calculate(4, 10000, 10000);
|
||||
}
|
||||
|
||||
public void calculate(final int nrOfWorkers, final int nrOfElements, final int nrOfMessages)
|
||||
throws Exception {
|
||||
|
||||
// this latch is only plumbing to know when the calculation is completed
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
|
||||
// create the master
|
||||
ActorRef master = actorOf(new Props(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch);
|
||||
}
|
||||
}));
|
||||
|
||||
// start the calculation
|
||||
master.tell(new Calculate());
|
||||
|
||||
// wait for master to shut down
|
||||
latch.await();
|
||||
}
|
||||
}
|
||||
As you can see the *calculate* method above it creates an ActorSystem and this is the Akka container which
|
||||
will contain all actors created in that "context". An example of how to create actors in the container
|
||||
is the *'system.actorOf(...)'* line in the calculate method. In this case we create a top level actor.
|
||||
If you instead where in an actor context, i.e. inside an actor creating other actors, you should use
|
||||
*this.getContext.actorOf(...)*. This is illustrated in the Master code above.
|
||||
|
||||
That's it. Now we are done.
|
||||
|
||||
Before we package it up and run it, let's take a look at the full code now, with package declaration, imports and all::
|
||||
|
||||
package akka.tutorial.first.java;
|
||||
|
||||
import static akka.actor.Actors.actorOf;
|
||||
import static akka.actor.Actors.poisonPill;
|
||||
import static java.util.Arrays.asList;
|
||||
|
||||
import akka.actor.Props;
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.UntypedActor;
|
||||
import akka.actor.UntypedActorFactory;
|
||||
import akka.routing.CyclicIterator;
|
||||
import akka.routing.InfiniteIterator;
|
||||
import akka.routing.Routing.Broadcast;
|
||||
import akka.routing.UntypedLoadBalancer;
|
||||
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
public class Pi {
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
Pi pi = new Pi();
|
||||
pi.calculate(4, 10000, 10000);
|
||||
}
|
||||
|
||||
// ====================
|
||||
// ===== Messages =====
|
||||
// ====================
|
||||
static class Calculate {}
|
||||
|
||||
static class Work {
|
||||
private final int start;
|
||||
private final int nrOfElements;
|
||||
|
||||
public Work(int start, int nrOfElements) {
|
||||
this.start = start;
|
||||
this.nrOfElements = nrOfElements;
|
||||
}
|
||||
|
||||
public int getStart() { return start; }
|
||||
public int getNrOfElements() { return nrOfElements; }
|
||||
}
|
||||
|
||||
static class Result {
|
||||
private final double value;
|
||||
|
||||
public Result(double value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
public double getValue() { return value; }
|
||||
}
|
||||
|
||||
// ==================
|
||||
// ===== Worker =====
|
||||
// ==================
|
||||
static class Worker extends UntypedActor {
|
||||
|
||||
// define the work
|
||||
private double calculatePiFor(int start, int nrOfElements) {
|
||||
double acc = 0.0;
|
||||
for (int i = start * nrOfElements; i <= ((start + 1) * nrOfElements - 1); i++) {
|
||||
acc += 4.0 * (1 - (i % 2) * 2) / (2 * i + 1);
|
||||
}
|
||||
return acc;
|
||||
}
|
||||
|
||||
// message handler
|
||||
public void onReceive(Object message) {
|
||||
if (message instanceof Work) {
|
||||
Work work = (Work) message;
|
||||
|
||||
// perform the work
|
||||
double result = calculatePiFor(work.getStart(), work.getNrOfElements())
|
||||
|
||||
// reply with the result
|
||||
getContext().reply(new Result(result));
|
||||
|
||||
} else throw new IllegalArgumentException("Unknown message [" + message + "]");
|
||||
}
|
||||
}
|
||||
|
||||
// ==================
|
||||
// ===== Master =====
|
||||
// ==================
|
||||
static class Master extends UntypedActor {
|
||||
private final int nrOfMessages;
|
||||
private final int nrOfElements;
|
||||
private final CountDownLatch latch;
|
||||
|
||||
private double pi;
|
||||
private int nrOfResults;
|
||||
private long start;
|
||||
|
||||
private ActorRef router;
|
||||
|
||||
static class PiRouter extends UntypedLoadBalancer {
|
||||
private final InfiniteIterator<ActorRef> workers;
|
||||
|
||||
public PiRouter(ActorRef[] workers) {
|
||||
this.workers = new CyclicIterator<ActorRef>(asList(workers));
|
||||
}
|
||||
|
||||
public InfiniteIterator<ActorRef> seq() {
|
||||
return workers;
|
||||
}
|
||||
}
|
||||
|
||||
public Master(
|
||||
int nrOfWorkers, int nrOfMessages, int nrOfElements, CountDownLatch latch) {
|
||||
|
||||
this.nrOfMessages = nrOfMessages;
|
||||
this.nrOfElements = nrOfElements;
|
||||
this.latch = latch;
|
||||
|
||||
// create the workers
|
||||
final ActorRef[] workers = new ActorRef[nrOfWorkers];
|
||||
for (int i = 0; i < nrOfWorkers; i++) {
|
||||
workers[i] = actorOf(new Props(Worker.class));
|
||||
}
|
||||
|
||||
// wrap them with a load-balancing router
|
||||
router = actorOf(new Props(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new PiRouter(workers);
|
||||
}
|
||||
}));
|
||||
}
|
||||
|
||||
// message handler
|
||||
public void onReceive(Object message) {
|
||||
|
||||
if (message instanceof Calculate) {
|
||||
// schedule work
|
||||
for (int start = 0; start < nrOfMessages; start++) {
|
||||
router.tell(new Work(start, nrOfElements), getContext());
|
||||
}
|
||||
|
||||
// send a PoisonPill to all workers telling them to shut down themselves
|
||||
router.tell(new Broadcast(poisonPill()));
|
||||
|
||||
// send a PoisonPill to the router, telling him to shut himself down
|
||||
router.tell(poisonPill());
|
||||
|
||||
} else if (message instanceof Result) {
|
||||
|
||||
// handle result from the worker
|
||||
Result result = (Result) message;
|
||||
pi += result.getValue();
|
||||
nrOfResults += 1;
|
||||
if (nrOfResults == nrOfMessages) getContext().stop();
|
||||
|
||||
} else throw new IllegalArgumentException("Unknown message [" + message + "]");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preStart() {
|
||||
start = System.currentTimeMillis();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postStop() {
|
||||
// tell the world that the calculation is complete
|
||||
System.out.println(String.format(
|
||||
"\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis",
|
||||
pi, (System.currentTimeMillis() - start)));
|
||||
latch.countDown();
|
||||
}
|
||||
}
|
||||
|
||||
// ==================
|
||||
// ===== Run it =====
|
||||
// ==================
|
||||
public void calculate(final int nrOfWorkers, final int nrOfElements, final int nrOfMessages)
|
||||
throws Exception {
|
||||
|
||||
// this latch is only plumbing to know when the calculation is completed
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
|
||||
// create the master
|
||||
ActorRef master = actorOf(new Props(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch);
|
||||
}
|
||||
}));
|
||||
|
||||
// start the calculation
|
||||
master.tell(new Calculate());
|
||||
|
||||
// wait for master to shut down
|
||||
latch.await();
|
||||
}
|
||||
}
|
||||
Before we package it up and run it, let's take a look at the full code now, with package declaration, imports and all:
|
||||
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java
|
||||
|
||||
Run it as a command line application
|
||||
------------------------------------
|
||||
|
|
@ -729,11 +363,11 @@ and the ``scala-library.jar`` JAR files to the classpath as well as the classes
|
|||
we compiled ourselves::
|
||||
|
||||
$ java \
|
||||
-cp lib/scala-library.jar:lib/akka/akka-actor-2.0-SNAPSHOT.jar:tutorial \
|
||||
-cp lib/scala-library.jar:lib/akka/akka-actor-2.0-SNAPSHOT.jar:. \
|
||||
akka.tutorial.java.first.Pi
|
||||
|
||||
Pi estimate: 3.1435501812459323
|
||||
Calculation time: 822 millis
|
||||
Calculation time: 609 millis
|
||||
|
||||
Yippee! It is working.
|
||||
|
||||
|
|
@ -750,7 +384,7 @@ When this in done we can run our application directly inside Maven::
|
|||
$ mvn exec:java -Dexec.mainClass="akka.tutorial.first.java.Pi"
|
||||
...
|
||||
Pi estimate: 3.1435501812459323
|
||||
Calculation time: 939 millis
|
||||
Calculation time: 597 millis
|
||||
|
||||
Yippee! It is working.
|
||||
|
||||
|
|
|
|||
|
|
@ -6,32 +6,69 @@ Getting Started Tutorial (Scala with Eclipse): First Chapter
|
|||
Introduction
|
||||
------------
|
||||
|
||||
Welcome to the first tutorial on how to get started with `Akka <http://akka.io>`_ and `Scala <http://scala-lang.org>`_. We assume that you already know what Akka and Scala are and will now focus on the steps necessary to start your first project. We will be using `Eclipse <http://www.eclipse.org/downloads/>`_, and the `Scala plugin for Eclipse <http://www.scala-ide.org/>`_.
|
||||
Welcome to the first tutorial on how to get started with Akka and Scala. We
|
||||
assume that you already know what Akka and Scala are and will now focus on the
|
||||
steps necessary to start your first project.
|
||||
|
||||
The sample application that we will create is using actors to calculate the value of Pi. Calculating Pi is a CPU intensive operation and we will utilize Akka Actors to write a concurrent solution that scales out to multi-core processors. This sample will be extended in future tutorials to use Akka Remote Actors to scale out on multiple machines in a cluster.
|
||||
There are two variations of this first tutorial:
|
||||
|
||||
We will be using an algorithm that is called "embarrassingly parallel" which just means that each job is completely isolated and not coupled with any other job. Since this algorithm is so parallelizable it suits the actor model very well.
|
||||
- creating a standalone project and run it from the command line
|
||||
- creating a SBT (Simple Build Tool) project and running it from within SBT
|
||||
|
||||
Since they are so similar we will present them both.
|
||||
|
||||
The sample application that we will create is using actors to calculate the
|
||||
value of Pi. Calculating Pi is a CPU intensive operation and we will utilize
|
||||
Akka Actors to write a concurrent solution that scales out to multi-core
|
||||
processors. This sample will be extended in future tutorials to use Akka Remote
|
||||
Actors to scale out on multiple machines in a cluster.
|
||||
|
||||
We will be using an algorithm that is called "embarrassingly parallel" which
|
||||
just means that each job is completely isolated and not coupled with any other
|
||||
job. Since this algorithm is so parallelizable it suits the actor model very
|
||||
well.
|
||||
|
||||
Here is the formula for the algorithm we will use:
|
||||
|
||||
.. image:: ../images/pi-formula.png
|
||||
|
||||
In this particular algorithm the master splits the series into chunks which are sent out to each worker actor to be processed. When each worker has processed its chunk it sends a result back to the master which aggregates the total result.
|
||||
In this particular algorithm the master splits the series into chunks which are
|
||||
sent out to each worker actor to be processed. When each worker has processed
|
||||
its chunk it sends a result back to the master which aggregates the total
|
||||
result.
|
||||
|
||||
|
||||
Tutorial source code
|
||||
--------------------
|
||||
|
||||
If you want don't want to type in the code and/or set up an SBT project then you can check out the full tutorial from the Akka GitHub repository. It is in the ``akka-tutorials/akka-tutorial-first`` module. You can also browse it online `here`__, with the actual source code `here`__.
|
||||
If you want don't want to type in the code and/or set up an SBT project then you can
|
||||
check out the full tutorial from the Akka GitHub repository. It is in the
|
||||
``akka-tutorials/akka-tutorial-first`` module. You can also browse it online
|
||||
`here`__, with the actual source code `here`__.
|
||||
|
||||
__ https://github.com/jboner/akka/tree/master/akka-tutorials/akka-tutorial-first
|
||||
__ https://github.com/jboner/akka/blob/master/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala
|
||||
|
||||
To check out the code using Git invoke the following::
|
||||
|
||||
$ git clone git://github.com/jboner/akka.git
|
||||
|
||||
Then you can navigate down to the tutorial::
|
||||
|
||||
$ cd akka/akka-tutorials/akka-tutorial-first
|
||||
|
||||
|
||||
Prerequisites
|
||||
-------------
|
||||
|
||||
This tutorial assumes that you have Java 1.6 or later installed on you machine and ``java`` on your ``PATH``. You also need to know how to run commands in a shell (ZSH, Bash, DOS etc.) and a recent version of Eclipse (at least `3.6 - Helios <http://www.eclipse.org/downloads/>`_).
|
||||
This tutorial assumes that you have Java 1.6 or later installed on you machine
|
||||
and ``java`` on your ``PATH``. You also need to know how to run commands in a
|
||||
shell (ZSH, Bash, DOS etc.) and a decent text editor or IDE to type in the Scala
|
||||
code.
|
||||
|
||||
If you want to run the example from the command line as well, you need to make sure that ``$JAVA_HOME`` environment variable is set to the root of the Java distribution. You also need to make sure that the ``$JAVA_HOME/bin`` is on your ``PATH``::
|
||||
You need to make sure that ``$JAVA_HOME`` environment variable is set to the
|
||||
root of the Java distribution. You also need to make sure that the
|
||||
``$JAVA_HOME/bin`` is on your ``PATH``::
|
||||
|
||||
$ export JAVA_HOME=..root of java distribution..
|
||||
$ export PATH=$PATH:$JAVA_HOME/bin
|
||||
|
|
@ -48,10 +85,10 @@ Downloading and installing Akka
|
|||
-------------------------------
|
||||
|
||||
To build and run the tutorial sample from the command line, you have to download
|
||||
Akka. If you prefer to use SBT to build and run the sample then you can skip
|
||||
this section and jump to the next one.
|
||||
Akka. If you prefer to use SBT to build and run the sample then you can skip this
|
||||
section and jump to the next one.
|
||||
|
||||
Let's get the ``akka-actors-2.0-SNAPSHOT.zip`` distribution of Akka from
|
||||
Let's get the ``akka-2.0-SNAPSHOT.zip`` distribution of Akka from
|
||||
http://akka.io/downloads/ which includes everything we need for this
|
||||
tutorial. Once you have downloaded the distribution unzip it in the folder you
|
||||
would like to have Akka installed in. In my case I choose to install it in
|
||||
|
|
@ -62,103 +99,138 @@ You need to do one more thing in order to install Akka properly: set the
|
|||
I'm opening up a shell, navigating down to the distribution, and setting the
|
||||
``AKKA_HOME`` variable::
|
||||
|
||||
$ cd /Users/jboner/tools/akka-actors-2.0-SNAPSHOT
|
||||
$ cd /Users/jboner/tools/akka-2.0-SNAPSHOT
|
||||
$ export AKKA_HOME=`pwd`
|
||||
$ echo $AKKA_HOME
|
||||
/Users/jboner/tools/akka-actors-2.0-SNAPSHOT
|
||||
/Users/jboner/tools/akka-2.0-SNAPSHOT
|
||||
|
||||
The distribution looks like this::
|
||||
|
||||
$ ls -1
|
||||
bin
|
||||
config
|
||||
deploy
|
||||
doc
|
||||
lib
|
||||
src
|
||||
|
||||
- In the ``bin`` directory we have scripts for starting the Akka Microkernel.
|
||||
- In the ``config`` directory we have the Akka conf files.
|
||||
- In the ``doc`` directory we have the documentation, API, doc JARs, and also
|
||||
the source files for the tutorials.
|
||||
- In the ``deploy`` directory we can place applications to be run with the microkernel.
|
||||
- In the ``doc`` directory we have the documentation, API, and doc JARs.
|
||||
- In the ``lib`` directory we have the Scala and Akka JARs.
|
||||
- In the ``src`` directory we have the source JARs for Akka.
|
||||
|
||||
|
||||
The only JAR we will need for this tutorial (apart from the
|
||||
``scala-library.jar`` JAR) is the ``akka-actor-2.0-SNAPSHOT.jar`` JAR in the ``lib/akka``
|
||||
directory. This is a self-contained JAR with zero dependencies and contains
|
||||
everything we need to write a system using Actors.
|
||||
|
||||
Akka is very modular and has many JARs for containing different features. The core distribution has seven modules:
|
||||
Akka is very modular and has many JARs for containing different features. The
|
||||
modules are:
|
||||
|
||||
- ``akka-actor-2.0-SNAPSHOT.jar`` -- Standard Actors
|
||||
- ``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-slf4j-2.0-SNAPSHOT.jar`` -- SLF4J Event Handler Listener for logging with SLF4J
|
||||
- ``akka-testkit-2.0-SNAPSHOT.jar`` -- Toolkit for testing Actors
|
||||
- ``akka-actor`` -- Actors
|
||||
|
||||
The Akka Microkernel distribution also includes these jars:
|
||||
- ``akka-remote`` -- Remote Actors
|
||||
|
||||
- ``akka-kernel-2.0-SNAPSHOT.jar`` -- Akka microkernel for running a bare-bones mini application server (embeds Jetty etc.)
|
||||
- ``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)
|
||||
- ``akka-camel-typed-2.0-SNAPSHOT.jar`` -- Apache Camel Typed Actors integration
|
||||
- ``akka-spring-2.0-SNAPSHOT.jar`` -- Spring framework integration
|
||||
- ``akka-slf4j`` -- SLF4J Event Handler Listener for logging with SLF4J
|
||||
|
||||
- ``akka-testkit`` -- Toolkit for testing Actors
|
||||
|
||||
- ``akka-kernel`` -- Akka microkernel for running a bare-bones mini application server
|
||||
|
||||
- ``akka-durable-mailboxes`` -- Durable mailboxes: file-based, MongoDB, Redis, Zookeeper
|
||||
|
||||
- ``akka-amqp`` -- AMQP integration
|
||||
|
||||
.. - ``akka-stm-2.0-SNAPSHOT.jar`` -- STM (Software Transactional Memory), transactors and transactional datastructures
|
||||
.. - ``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)
|
||||
.. - ``akka-camel-typed-2.0-SNAPSHOT.jar`` -- Apache Camel Typed Actors integration
|
||||
.. - ``akka-spring-2.0-SNAPSHOT.jar`` -- Spring framework integration
|
||||
|
||||
|
||||
Downloading and installing the Scala IDE for Eclipse
|
||||
----------------------------------------------------
|
||||
|
||||
If you want to use Eclipse for coding your Akka tutorial, you need to install the Scala plugin for Eclipse. This plugin comes with its own version of Scala, so if you don't plan to run the example from the command line, you don't need to download the Scala distribution (and you can skip the next section).
|
||||
If you want to use Eclipse for coding your Akka tutorial, you need to install the Scala plugin for Eclipse.
|
||||
This plugin comes with its own version of Scala, so if you don't plan to run the example from the command line,
|
||||
you don't need to download the Scala distribution (and you can skip the next section).
|
||||
|
||||
You can install this plugin using the regular update mechanism. First choose a version of the IDE from `http://download.scala-ide.org <http://download.scala-ide.org>`_. We recommend you choose 2.0.x, which comes with Scala 2.9. Copy the corresponding URL and then choose ``Help/Install New Software`` and paste the URL you just copied. You should see something similar to the following image.
|
||||
You can install this plugin using the regular update mechanism. First choose a version of the IDE from
|
||||
`http://download.scala-ide.org <http://download.scala-ide.org>`_. We recommend you choose 2.0.x, which
|
||||
comes with Scala 2.9. Copy the corresponding URL and then choose ``Help/Install New Software`` and paste
|
||||
the URL you just copied. You should see something similar to the following image.
|
||||
|
||||
.. image:: ../images/install-beta2-updatesite.png
|
||||
|
||||
Make sure you select both the ``JDT Weaving for Scala`` and the ``Scala IDE for Eclipse`` plugins. The other plugin is optional, and contains the source code of the plugin itself.
|
||||
Make sure you select both the ``JDT Weaving for Scala`` and the ``Scala IDE for Eclipse`` plugins.
|
||||
The other plugin is optional, and contains the source code of the plugin itself.
|
||||
|
||||
Once the installation is finished, you need to restart Eclipse. The first time the plugin starts it will open a diagnostics window and offer to fix several settings, such as the delay for content assist (code-completion) or the shown completion proposal types.
|
||||
Once the installation is finished, you need to restart Eclipse. The first time the plugin starts it will
|
||||
open a diagnostics window and offer to fix several settings, such as the delay for content assist (code-completion)
|
||||
or the shown completion proposal types.
|
||||
|
||||
.. image:: ../images/diagnostics-window.png
|
||||
|
||||
Accept the recommended settings, and follow the instructions if you need to increase the heap size of Eclipse.
|
||||
|
||||
Check that the installation succeeded by creating a new Scala project (``File/New>Scala Project``), and typing some code. You should have content-assist, hyperlinking to definitions, instant error reporting, and so on.
|
||||
Check that the installation succeeded by creating a new Scala project (``File/New>Scala Project``), and typing some code.
|
||||
You should have content-assist, hyperlinking to definitions, instant error reporting, and so on.
|
||||
|
||||
.. image:: ../images/example-code.png
|
||||
|
||||
You are ready to code now!
|
||||
|
||||
|
||||
Downloading and installing Scala
|
||||
--------------------------------
|
||||
|
||||
To build and run the tutorial sample from the command line, you have to install the Scala distribution. If you prefer to use Eclipse to build and run the sample then you can skip this section and jump to the next one.
|
||||
To build and run the tutorial sample from the command line, you have to install
|
||||
the Scala distribution. If you prefer to use SBT to build and run the sample
|
||||
then you can skip this section and jump to the next one.
|
||||
|
||||
Scala can be downloaded from `http://www.scala-lang.org/downloads <http://www.scala-lang.org/downloads>`_. Browse there and download the Scala 2.9.0 release. If you pick the ``tgz`` or ``zip`` distribution then just unzip it where you want it installed. If you pick the IzPack Installer then double click on it and follow the instructions.
|
||||
Scala can be downloaded from http://www.scala-lang.org/downloads. Browse there
|
||||
and download the Scala 2.9.1 release. If you pick the ``tgz`` or ``zip``
|
||||
distribution then just unzip it where you want it installed. If you pick the
|
||||
IzPack Installer then double click on it and follow the instructions.
|
||||
|
||||
You also need to make sure that the ``scala-2.9.0/bin`` (if that is the directory where you installed Scala) is on your ``PATH``::
|
||||
You also need to make sure that the ``scala-2.9.1/bin`` (if that is the
|
||||
directory where you installed Scala) is on your ``PATH``::
|
||||
|
||||
$ export PATH=$PATH:scala-2.9.0/bin
|
||||
$ export PATH=$PATH:scala-2.9.1/bin
|
||||
|
||||
You can test your installation by invoking scala::
|
||||
|
||||
$ scala -version
|
||||
Scala code runner version 2.9.0.final -- Copyright 2002-2011, LAMP/EPFL
|
||||
Scala code runner version 2.9.1.final -- Copyright 2002-2011, LAMP/EPFL
|
||||
|
||||
Looks like we are all good. Finally let's create a source file ``Pi.scala`` for the tutorial and put it in the root of the Akka distribution in the ``tutorial`` directory (you have to create it first).
|
||||
Looks like we are all good. Finally let's create a source file ``Pi.scala`` for
|
||||
the tutorial and put it in the root of the Akka distribution in the ``tutorial``
|
||||
directory (you have to create it first).
|
||||
|
||||
Some tools require you to set the ``SCALA_HOME`` environment variable to the
|
||||
root of the Scala distribution, however Akka does not require that.
|
||||
|
||||
Some tools require you to set the ``SCALA_HOME`` environment variable to the root of the Scala distribution, however Akka does not require that.
|
||||
|
||||
Creating an Akka project in Eclipse
|
||||
---------------------------------------
|
||||
-----------------------------------
|
||||
|
||||
If you have not already done so, now is the time to create an Eclipse project for our tutorial. Use the ``New Scala Project`` wizard and accept the default settings. Once the project is open, we need to add the akka libraries to the *build path*. Right click on the project and choose ``Properties``, then click on ``Java Build Path``. Go to ``Libraries`` and click on ``Add External Jars..``, then navigate to the location where you installed akka and choose ``akka-actor.jar``. You should see something similar to this:
|
||||
If you have not already done so, now is the time to create an Eclipse project for our tutorial.
|
||||
Use the ``New Scala Project`` wizard and accept the default settings. Once the project is open,
|
||||
we need to add the akka libraries to the *build path*. Right click on the project and choose ``Properties``,
|
||||
then click on ``Java Build Path``. Go to ``Libraries`` and click on ``Add External Jars..``, then navigate
|
||||
to the location where you installed akka and choose ``akka-actor.jar``. You should see something similar to this:
|
||||
|
||||
.. image:: ../images/build-path.png
|
||||
|
||||
|
||||
Using SBT in Eclipse
|
||||
^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
If you are an `SBT <https://github.com/harrah/xsbt/wiki>`_ user, you can follow the :ref:`getting-started-first-scala-download-sbt` instruction and additionally install the ``sbteclipse`` plugin. This adds support for generating Eclipse project files from your SBT project.
|
||||
You need to install the plugin as described in the `README of sbteclipse <https://github.com/typesafehub/sbteclipse>`_
|
||||
If you are an `SBT <https://github.com/harrah/xsbt/wiki>`_ user, you can follow the :ref:`getting-started-first-scala-download-sbt`
|
||||
instruction and additionally install the ``sbteclipse`` plugin. This adds support for generating Eclipse project files
|
||||
from your SBT project. You need to install the plugin as described in the `README of sbteclipse
|
||||
<https://github.com/typesafehub/sbteclipse>`_
|
||||
|
||||
Then run the ``eclipse`` target to generate the Eclipse project::
|
||||
|
||||
|
|
@ -173,12 +245,14 @@ The options `create-src` and `with-sources` are useful::
|
|||
* create-src to create the common source directories, e.g. src/main/scala, src/main/test
|
||||
* with-sources to create source attachments for the library dependencies
|
||||
|
||||
Next you need to import this project in Eclipse, by choosing ``Eclipse/Import.. Existing Projects into Workspace``. Navigate to the directory where you defined your SBT project and choose import:
|
||||
Next you need to import this project in Eclipse, by choosing ``Eclipse/Import.. Existing Projects into Workspace``.
|
||||
Navigate to the directory where you defined your SBT project and choose import:
|
||||
|
||||
.. image:: ../images/import-project.png
|
||||
|
||||
Now we have the basis for an Akka Eclipse application, so we can..
|
||||
|
||||
|
||||
Start writing the code
|
||||
----------------------
|
||||
|
||||
|
|
@ -186,10 +260,14 @@ The design we are aiming for is to have one ``Master`` actor initiating the comp
|
|||
|
||||
With this in mind, let's now create the messages that we want to have flowing in the system.
|
||||
|
||||
|
||||
Creating the messages
|
||||
---------------------
|
||||
|
||||
We start by creating a package for our application, let's call it ``akka.tutorial.first.scala``. We start by creating case classes for each type of message in our application, so we can place them in a hierarchy, call it ``PiMessage``. Right click on the package and choose ``New Scala Class``, and enter ``PiMessage`` for the name of the class.
|
||||
We start by creating a package for our application, let's call it ``akka.tutorial.first.scala``.
|
||||
We start by creating case classes for each type of message in our application, so we can place them in a hierarchy,
|
||||
call it ``PiMessage``. Right click on the package and choose ``New Scala Class``, and enter ``PiMessage`` as
|
||||
the name of the class.
|
||||
|
||||
We need three different messages:
|
||||
|
||||
|
|
@ -197,204 +275,156 @@ We need three different messages:
|
|||
- ``Work`` -- sent from the ``Master`` actor to the ``Worker`` actors containing the work assignment
|
||||
- ``Result`` -- sent from the ``Worker`` actors to the ``Master`` actor containing the result from the worker's calculation
|
||||
|
||||
Messages sent to actors should always be immutable to avoid sharing mutable state. In Scala we have 'case classes' which make excellent messages. So let's start by creating three messages as case classes. We also create a common base trait for our messages (that we define as being ``sealed`` in order to prevent creating messages outside our control)::
|
||||
Messages sent to actors should always be immutable to avoid sharing mutable state.
|
||||
In Scala we have 'case classes' which make excellent messages. So let's start by creating three messages as case classes.
|
||||
We also create a common base trait for our messages (that we define as being ``sealed`` in order to prevent creating messages
|
||||
outside our control):
|
||||
|
||||
package akka.tutorial.first.scala
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#messages
|
||||
|
||||
sealed trait PiMessage
|
||||
|
||||
case object Calculate extends PiMessage
|
||||
|
||||
case class Work(start: Int, nrOfElements: Int) extends PiMessage
|
||||
|
||||
case class Result(value: Double) extends PiMessage
|
||||
|
||||
Creating the worker
|
||||
-------------------
|
||||
|
||||
Now we can create the worker actor. Create a new class called ``Worker`` as before. We need to mix in the ``Actor`` trait and defining the ``receive`` method. The ``receive`` method defines our message handler. We expect it to be able to handle the ``Work`` message so we need to add a handler for this message::
|
||||
Now we can create the worker actor. This is done by mixing in the ``Actor``
|
||||
trait and defining the ``receive`` method. The ``receive`` method defines our
|
||||
message handler. We expect it to be able to handle the ``Work`` message so we
|
||||
need to add a handler for this message:
|
||||
|
||||
class Worker extends Actor {
|
||||
def receive = {
|
||||
case Work(start, nrOfElements) =>
|
||||
self reply Result(calculatePiFor(start, nrOfElements)) // perform the work
|
||||
}
|
||||
}
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#worker
|
||||
:exclude: calculatePiFor
|
||||
|
||||
The ``Actor`` trait is defined in ``akka.actor`` and you can either import it explicitly, or let Eclipse do it for you when it cannot resolve the ``Actor`` trait. The quick fix option (``Ctrl-F1``) will offer two options:
|
||||
The ``Actor`` trait is defined in ``akka.actor`` and you can either import it explicitly,
|
||||
or let Eclipse do it for you when it cannot resolve the ``Actor`` trait.
|
||||
The quick fix option (``Ctrl-F1``) will offer two options:
|
||||
|
||||
.. image:: ../images/quickfix.png
|
||||
|
||||
Choose the Akka Actor and move on.
|
||||
|
||||
As you can see we have now created an ``Actor`` with a ``receive`` method as a handler for the ``Work`` message. In this handler we invoke the ``calculatePiFor(..)`` method, wrap the result in a ``Result`` message and send it back to the original sender using ``self.reply``. In Akka the sender reference is implicitly passed along with the message so that the receiver can always reply or store away the sender reference for future use.
|
||||
As you can see we have now created an ``Actor`` with a ``receive`` method as a
|
||||
handler for the ``Work`` message. In this handler we invoke the
|
||||
``calculatePiFor(..)`` method, wrap the result in a ``Result`` message and send
|
||||
it back asynchronously to the original sender using the ``sender`` reference.
|
||||
In Akka the sender reference is implicitly passed along with the message so that
|
||||
the receiver can always reply or store away the sender reference for future use.
|
||||
|
||||
The only thing missing in our ``Worker`` actor is the implementation on the ``calculatePiFor(..)`` method. While there are many ways we can implement this algorithm in Scala, in this introductory tutorial we have chosen an imperative style using a for comprehension and an accumulator::
|
||||
The only thing missing in our ``Worker`` actor is the implementation on the
|
||||
``calculatePiFor(..)`` method. While there are many ways we can implement this
|
||||
algorithm in Scala, in this introductory tutorial we have chosen an imperative
|
||||
style using a for comprehension and an accumulator:
|
||||
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#calculatePiFor
|
||||
|
||||
def calculatePiFor(start: Int, nrOfElements: Int): Double = {
|
||||
var acc = 0.0
|
||||
for (i <- start until (start + nrOfElements))
|
||||
acc += 4.0 * (1 - (i % 2) * 2) / (2 * i + 1)
|
||||
acc
|
||||
}
|
||||
|
||||
Creating the master
|
||||
-------------------
|
||||
|
||||
Now create a new class for the master actor. The master actor is a little bit more involved. In its constructor we need to create the workers (the ``Worker`` actors) and start them. We will also wrap them in a load-balancing router to make it easier to spread out the work evenly between the workers. First we need to add some imports::
|
||||
Now create a new class for the master actor. The master actor is a little bit
|
||||
more involved. In its constructor we create a round-robin router to make it easier
|
||||
to spread out the work evenly between the workers. First we need to add some imports:
|
||||
|
||||
import akka.actor.{Actor, PoisonPill}
|
||||
import akka.routing.{Routing, CyclicIterator}
|
||||
import Routing._
|
||||
import akka.dispatch.Dispatchers
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#imports
|
||||
|
||||
import java.util.concurrent.CountDownLatch
|
||||
and then we can create the router:
|
||||
|
||||
and then we can create the workers::
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#create-router
|
||||
|
||||
// create the workers
|
||||
val workers = Vector.fill(nrOfWorkers)(actorOf(Props[Worker])
|
||||
|
||||
// wrap them with a load-balancing router
|
||||
val router = Routing.loadBalancerActor(CyclicIterator(workers))
|
||||
|
||||
As you can see we are using the ``actorOf`` factory method to create actors, this method returns as an ``ActorRef`` which is a reference to our newly created actor. This method is available in the ``Actor`` object but is usually imported::
|
||||
|
||||
import akka.actor.Actor.actorOf
|
||||
|
||||
There are two versions of ``actorOf``; one of them taking a actor type and the other one an instance of an actor. The former one (``actorOf(Props[MyActor]``) is used when the actor class has a no-argument constructor while the second one (``actorOf(Props(new MyActor(..))``) is used when the actor class has a constructor that takes arguments. This is the only way to create an instance of an Actor and the ``actorOf`` method ensures this. The latter version is using call-by-name and lazily creates the actor within the scope of the ``actorOf`` method. The ``actorOf`` method instantiates the actor and returns, not an instance to the actor, but an instance to an ``ActorRef``. This reference is the handle through which you communicate with the actor. It is immutable, serializable and location-aware meaning that it "remembers" its original actor even if it is sent to other nodes across the network and can be seen as the equivalent to the Erlang actor's PID.
|
||||
|
||||
The actor's life-cycle is:
|
||||
|
||||
- Created -- ``Actor.actorOf(Props[MyActor]`` -- can **not** receive messages
|
||||
- Started -- ``actorRef`` -- can receive messages
|
||||
- Stopped -- ``actorRef.stop()`` -- can **not** receive messages
|
||||
|
||||
Once the actor has been stopped it is dead and can not be started again.
|
||||
|
||||
Now we have a router that is representing all our workers in a single abstraction. If you paid attention to the code above, you saw that we were using the ``nrOfWorkers`` variable. This variable and others we have to pass to the ``Master`` actor in its constructor. So now let's create the master actor. We have to pass in three integer variables:
|
||||
Now we have a router that is representing all our workers in a single
|
||||
abstraction. So now let's create the master actor. We pass it three integer variables:
|
||||
|
||||
- ``nrOfWorkers`` -- defining how many workers we should start up
|
||||
- ``nrOfMessages`` -- defining how many number chunks to send out to the workers
|
||||
- ``nrOfElements`` -- defining how big the number chunks sent to each worker should be
|
||||
|
||||
Here is the master actor::
|
||||
Here is the master actor:
|
||||
|
||||
class Master(
|
||||
nrOfWorkers: Int, nrOfMessages: Int, nrOfElements: Int, latch: CountDownLatch)
|
||||
extends Actor {
|
||||
|
||||
var pi: Double = _
|
||||
var nrOfResults: Int = _
|
||||
var start: Long = _
|
||||
|
||||
// create the workers
|
||||
val workers = Vector.fill(nrOfWorkers)(actorOf(Props[Worker])
|
||||
|
||||
// wrap them with a load-balancing router
|
||||
val router = Routing.loadBalancerActor(CyclicIterator(workers))
|
||||
|
||||
def receive = { ... }
|
||||
|
||||
override def preStart() {
|
||||
start = System.currentTimeMillis
|
||||
}
|
||||
|
||||
override def postStop() {
|
||||
// tell the world that the calculation is complete
|
||||
println(
|
||||
"\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis"
|
||||
.format(pi, (System.currentTimeMillis - start)))
|
||||
latch.countDown()
|
||||
}
|
||||
}
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#master
|
||||
:exclude: handle-messages
|
||||
|
||||
A couple of things are worth explaining further.
|
||||
|
||||
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the ``Master`` actor. This latch is only used for plumbing (in this specific tutorial), to have a simple way of letting the outside world knowing when the master can deliver the result and shut down. In more idiomatic Akka code, as we will see in part two of this tutorial series, we would not use a latch but other abstractions and functions like ``Channel``, ``Future`` and ``?`` to achieve the same thing in a non-blocking way. But for simplicity let's stick to a ``CountDownLatch`` for now.
|
||||
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the
|
||||
``Master`` actor. This latch is only used for plumbing (in this specific
|
||||
tutorial), to have a simple way of letting the outside world knowing when the
|
||||
master can deliver the result and shut down. In more idiomatic Akka code
|
||||
we would not use a latch but other abstractions and functions like ``Future``
|
||||
and ``?`` to achieve the same thing in a non-blocking way.
|
||||
But for simplicity let's stick to a ``CountDownLatch`` for now.
|
||||
|
||||
Second, we are adding a couple of life-cycle callback methods; ``preStart`` and ``postStop``. In the ``preStart`` callback we are recording the time when the actor is started and in the ``postStop`` callback we are printing out the result (the approximation of Pi) and the time it took to calculate it. In this call we also invoke ``latch.countDown`` to tell the outside world that we are done.
|
||||
Second, we are adding a couple of life-cycle callback methods; ``preStart`` and
|
||||
``postStop``. In the ``preStart`` callback we are recording the time when the
|
||||
actor is started and in the ``postStop`` callback we are printing out the result
|
||||
(the approximation of Pi) and the time it took to calculate it. In this call we
|
||||
also invoke ``latch.countDown()`` to tell the outside world that we are done.
|
||||
|
||||
But we are not done yet. We are missing the message handler for the ``Master`` actor. This message handler needs to be able to react to two different messages:
|
||||
But we are not done yet. We are missing the message handler for the ``Master``
|
||||
actor. This message handler needs to be able to react to two different messages:
|
||||
|
||||
- ``Calculate`` -- which should start the calculation
|
||||
- ``Result`` -- which should aggregate the different results
|
||||
|
||||
The ``Calculate`` handler is sending out work to all the ``Worker`` actors and after doing that it also sends a ``Broadcast(PoisonPill)`` message to the router, which will send out the ``PoisonPill`` message to all the actors it is representing (in our case all the ``Worker`` actors). ``PoisonPill`` is a special kind of message that tells the receiver to shut itself down using the normal shutdown method; ``self.stop``. We also send a ``PoisonPill`` to the router itself (since it's also an actor that we want to shut down).
|
||||
The ``Calculate`` handler is sending out work to all the ``Worker`` via its router.
|
||||
|
||||
The ``Result`` handler is simpler, here we get the value from the ``Result`` message and aggregate it to our ``pi`` member variable. We also keep track of how many results we have received back, and if that matches the number of tasks sent out, the ``Master`` actor considers itself done and shuts down.
|
||||
The ``Result`` handler gets the value from the ``Result`` message and aggregates it to
|
||||
our ``pi`` member variable. We also keep track of how many results we have received back,
|
||||
and if that matches the number of tasks sent out, the ``Master`` actor considers itself done and
|
||||
invokes the ``self.stop()`` method to stop itself *and* all its supervised actors.
|
||||
In this case it has one supervised actor, the router, and this in turn has ``nrOfWorkers`` supervised actors.
|
||||
All of them will be stopped automatically as the invocation of any supervisor's ``stop`` method
|
||||
will propagate down to all its supervised 'children'.
|
||||
|
||||
Let's capture this in code::
|
||||
Let's capture this in code:
|
||||
|
||||
// message handler
|
||||
def receive = {
|
||||
case Calculate =>
|
||||
// schedule work
|
||||
for (i <- 0 until nrOfMessages) router ! Work(i * nrOfElements, nrOfElements)
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#master-receive
|
||||
|
||||
// send a PoisonPill to all workers telling them to shut down themselves
|
||||
router ! Broadcast(PoisonPill)
|
||||
|
||||
// send a PoisonPill to the router, telling him to shut himself down
|
||||
router ! PoisonPill
|
||||
|
||||
case Result(value) =>
|
||||
// handle result from the worker
|
||||
pi += value
|
||||
nrOfResults += 1
|
||||
if (nrOfResults == nrOfMessages) self.stop()
|
||||
}
|
||||
|
||||
Bootstrap the calculation
|
||||
-------------------------
|
||||
|
||||
Now the only thing that is left to implement is the runner that should bootstrap and run the calculation for us. We do that by creating an object that we call ``Pi``, here we can extend the ``App`` trait in Scala, which means that we will be able to run this as an application directly from the command line or using the Eclipse Runner.
|
||||
Now the only thing that is left to implement is the runner that should bootstrap and run the calculation for us.
|
||||
We do that by creating an object that we call ``Pi``, here we can extend the ``App`` trait in Scala,
|
||||
which means that we will be able to run this as an application directly from the command line or using the Eclipse Runner.
|
||||
|
||||
The ``Pi`` object is a perfect container module for our actors and messages, so let's put them all there. We also create a method ``calculate`` in which we start up the ``Master`` actor and wait for it to finish::
|
||||
The ``Pi`` object is a perfect container module for our actors and messages, so let's put them all there.
|
||||
We also create a method ``calculate`` in which we start up the ``Master`` actor and wait for it to finish:
|
||||
|
||||
object Pi extends App {
|
||||
|
||||
calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000)
|
||||
|
||||
... // actors and messages
|
||||
|
||||
def calculate(nrOfWorkers: Int, nrOfElements: Int, nrOfMessages: Int) {
|
||||
|
||||
// this latch is only plumbing to know when the calculation is completed
|
||||
val latch = new CountDownLatch(1)
|
||||
|
||||
// create the master
|
||||
val master = actorOf(Props(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)))
|
||||
|
||||
// start the calculation
|
||||
master ! Calculate
|
||||
|
||||
// wait for master to shut down
|
||||
latch.await()
|
||||
}
|
||||
}
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#app
|
||||
:exclude: actors-and-messages
|
||||
|
||||
That's it. Now we are done.
|
||||
|
||||
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::
|
||||
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::
|
||||
|
||||
Pi estimate: 3.1435501812459323
|
||||
Calculation time: 858 millis
|
||||
Calculation time: 632 millis
|
||||
|
||||
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:
|
||||
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
|
||||
|
||||
Once you finished your run configuration, click ``Run``. You should see the same output in the ``Console`` window. You can use the same configuration for debugging the application, by choosing ``Run/Debug History`` or just ``Debug As``.
|
||||
Once you finished your run configuration, click ``Run``. You should see the same output in the ``Console`` window.
|
||||
You can use the same configuration for debugging the application, by choosing ``Run/Debug History`` or just ``Debug As``.
|
||||
|
||||
Conclusion
|
||||
----------
|
||||
|
||||
We have learned how to create our first Akka project using Akka's actors to speed up a computation-intensive problem by scaling out on multi-core processors (also known as scaling up). We have also learned to compile and run an Akka project using Eclipse.
|
||||
We have learned how to create our first Akka project using Akka's actors to
|
||||
speed up a computation-intensive problem by scaling out on multi-core processors
|
||||
(also known as scaling up). We have also learned to compile and run an Akka
|
||||
project using either the tools on the command line or the SBT build system.
|
||||
|
||||
If you have a multi-core machine then I encourage you to try out different number of workers (number of working actors) by tweaking the ``nrOfWorkers`` variable to for example; 2, 4, 6, 8 etc. to see performance improvement by scaling up.
|
||||
If you have a multi-core machine then I encourage you to try out different
|
||||
number of workers (number of working actors) by tweaking the ``nrOfWorkers``
|
||||
variable to for example; 2, 4, 6, 8 etc. to see performance improvement by
|
||||
scaling up.
|
||||
|
||||
Happy hakking.
|
||||
|
|
|
|||
|
|
@ -40,6 +40,24 @@ sent out to each worker actor to be processed. When each worker has processed
|
|||
its chunk it sends a result back to the master which aggregates the total
|
||||
result.
|
||||
|
||||
Tutorial source code
|
||||
--------------------
|
||||
|
||||
If you want don't want to type in the code and/or set up an SBT project then you can
|
||||
check out the full tutorial from the Akka GitHub repository. It is in the
|
||||
``akka-tutorials/akka-tutorial-first`` module. You can also browse it online
|
||||
`here`__, with the actual source code `here`__.
|
||||
|
||||
__ https://github.com/jboner/akka/tree/master/akka-tutorials/akka-tutorial-first
|
||||
__ https://github.com/jboner/akka/blob/master/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala
|
||||
|
||||
To check out the code using Git invoke the following::
|
||||
|
||||
$ git clone git://github.com/jboner/akka.git
|
||||
|
||||
Then you can navigate down to the tutorial::
|
||||
|
||||
$ cd akka/akka-tutorials/akka-tutorial-first
|
||||
|
||||
Prerequisites
|
||||
=============
|
||||
|
|
@ -68,10 +86,10 @@ Downloading and installing Akka
|
|||
===============================
|
||||
|
||||
To build and run the tutorial sample from the command line, you have to download
|
||||
Akka. If you prefer to use SBT to build and run the sample then you can skipthis
|
||||
Akka. If you prefer to use SBT to build and run the sample then you can skip this
|
||||
section and jump to the next one.
|
||||
|
||||
Let's get the ``akka-actors-2.0-SNAPSHOT.zip`` distribution of Akka from
|
||||
Let's get the ``akka-2.0-SNAPSHOT.zip`` distribution of Akka from
|
||||
http://akka.io/downloads/ which includes everything we need for this
|
||||
tutorial. Once you have downloaded the distribution unzip it in the folder you
|
||||
would like to have Akka installed in. In my case I choose to install it in
|
||||
|
|
@ -82,47 +100,54 @@ You need to do one more thing in order to install Akka properly: set the
|
|||
I'm opening up a shell, navigating down to the distribution, and setting the
|
||||
``AKKA_HOME`` variable::
|
||||
|
||||
$ cd /Users/jboner/tools/akka-actors-2.0-SNAPSHOT
|
||||
$ cd /Users/jboner/tools/akka-2.0-SNAPSHOT
|
||||
$ export AKKA_HOME=`pwd`
|
||||
$ echo $AKKA_HOME
|
||||
/Users/jboner/tools/akka-actors-2.0-SNAPSHOT
|
||||
/Users/jboner/tools/akka-2.0-SNAPSHOT
|
||||
|
||||
The distribution looks like this::
|
||||
|
||||
$ ls -1
|
||||
bin
|
||||
config
|
||||
deploy
|
||||
doc
|
||||
lib
|
||||
src
|
||||
|
||||
- In the ``bin`` directory we have scripts for starting the Akka Microkernel.
|
||||
- In the ``config`` directory we have the Akka conf files.
|
||||
- In the ``doc`` directory we have the documentation, API, doc JARs, and also
|
||||
the source files for the tutorials.
|
||||
- In the ``deploy`` directory we can place applications to be run with the microkernel.
|
||||
- In the ``doc`` directory we have the documentation, API, and doc JARs.
|
||||
- In the ``lib`` directory we have the Scala and Akka JARs.
|
||||
- In the ``src`` directory we have the source JARs for Akka.
|
||||
|
||||
|
||||
The only JAR we will need for this tutorial (apart from the
|
||||
``scala-library.jar`` JAR) is the ``akka-actor-2.0-SNAPSHOT.jar`` JAR in the ``lib/akka``
|
||||
directory. This is a self-contained JAR with zero dependencies and contains
|
||||
everything we need to write a system using Actors.
|
||||
|
||||
Akka is very modular and has many JARs for containing different features. The
|
||||
core distribution has seven modules:
|
||||
modules are:
|
||||
|
||||
- ``akka-actor-2.0-SNAPSHOT.jar`` -- Standard Actors
|
||||
- ``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-slf4j-2.0-SNAPSHOT.jar`` -- SLF4J Event Handler Listener for logging with SLF4J
|
||||
- ``akka-testkit-2.0-SNAPSHOT.jar`` -- Toolkit for testing Actors
|
||||
- ``akka-actor`` -- Actors
|
||||
|
||||
The Akka Microkernel distribution also includes these jars:
|
||||
- ``akka-remote`` -- Remote Actors
|
||||
|
||||
- ``akka-kernel-2.0-SNAPSHOT.jar`` -- Akka microkernel for running a bare-bones mini application server (embeds Jetty etc.)
|
||||
- ``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)
|
||||
- ``akka-camel-typed-2.0-SNAPSHOT.jar`` -- Apache Camel Typed Actors integration
|
||||
- ``akka-spring-2.0-SNAPSHOT.jar`` -- Spring framework integration
|
||||
- ``akka-slf4j`` -- SLF4J Event Handler Listener for logging with SLF4J
|
||||
|
||||
- ``akka-testkit`` -- Toolkit for testing Actors
|
||||
|
||||
- ``akka-kernel`` -- Akka microkernel for running a bare-bones mini application server
|
||||
|
||||
- ``akka-durable-mailboxes`` -- Durable mailboxes: file-based, MongoDB, Redis, Zookeeper
|
||||
|
||||
- ``akka-amqp`` -- AMQP integration
|
||||
|
||||
.. - ``akka-stm-2.0-SNAPSHOT.jar`` -- STM (Software Transactional Memory), transactors and transactional datastructures
|
||||
.. - ``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)
|
||||
.. - ``akka-camel-typed-2.0-SNAPSHOT.jar`` -- Apache Camel Typed Actors integration
|
||||
.. - ``akka-spring-2.0-SNAPSHOT.jar`` -- Spring framework integration
|
||||
|
||||
|
||||
Downloading and installing Scala
|
||||
|
|
@ -133,19 +158,19 @@ the Scala distribution. If you prefer to use SBT to build and run the sample
|
|||
then you can skip this section and jump to the next one.
|
||||
|
||||
Scala can be downloaded from http://www.scala-lang.org/downloads. Browse there
|
||||
and download the Scala 2.9.0 release. If you pick the ``tgz`` or ``zip``
|
||||
and download the Scala 2.9.1 release. If you pick the ``tgz`` or ``zip``
|
||||
distribution then just unzip it where you want it installed. If you pick the
|
||||
IzPack Installer then double click on it and follow the instructions.
|
||||
|
||||
You also need to make sure that the ``scala-2.9.0/bin`` (if that is the
|
||||
You also need to make sure that the ``scala-2.9.1/bin`` (if that is the
|
||||
directory where you installed Scala) is on your ``PATH``::
|
||||
|
||||
$ export PATH=$PATH:scala-2.9.0/bin
|
||||
$ export PATH=$PATH:scala-2.9.1/bin
|
||||
|
||||
You can test your installation by invoking scala::
|
||||
|
||||
$ scala -version
|
||||
Scala code runner version 2.9.0.final -- Copyright 2002-2011, LAMP/EPFL
|
||||
Scala code runner version 2.9.1.final -- Copyright 2002-2011, LAMP/EPFL
|
||||
|
||||
Looks like we are all good. Finally let's create a source file ``Pi.scala`` for
|
||||
the tutorial and put it in the root of the Akka distribution in the ``tutorial``
|
||||
|
|
@ -199,14 +224,12 @@ files.
|
|||
Not needed in this tutorial, but if you would like to use additional Akka
|
||||
modules beyond ``akka-actor``, you can add these as ``libraryDependencies`` in
|
||||
``build.sbt``. Note that there must be a blank line between each. Here is an
|
||||
example adding ``akka-remote`` and ``akka-stm``::
|
||||
example adding ``akka-remote``::
|
||||
|
||||
libraryDependencies += "com.typesafe.akka" % "akka-actor" % "2.0-SNAPSHOT"
|
||||
|
||||
libraryDependencies += "com.typesafe.akka" % "akka-remote" % "2.0-SNAPSHOT"
|
||||
|
||||
libraryDependencies += "com.typesafe.akka" % "akka-stm" % "2.0-SNAPSHOT"
|
||||
|
||||
So, now we are all set.
|
||||
|
||||
SBT itself needs a whole bunch of dependencies but our project will only need
|
||||
|
|
@ -221,7 +244,7 @@ Now it's about time to start hacking.
|
|||
We start by creating a ``Pi.scala`` file and adding these import statements at
|
||||
the top of the file:
|
||||
|
||||
.. includecode:: code/tutorials/first/Pi.scala#imports
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#imports
|
||||
|
||||
If you are using SBT in this tutorial then create the file in the
|
||||
``src/main/scala`` directory.
|
||||
|
|
@ -256,7 +279,7 @@ start by creating three messages as case classes. We also create a common base
|
|||
trait for our messages (that we define as being ``sealed`` in order to prevent
|
||||
creating messages outside our control):
|
||||
|
||||
.. includecode:: code/tutorials/first/Pi.scala#messages
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#messages
|
||||
|
||||
|
||||
Creating the worker
|
||||
|
|
@ -267,67 +290,34 @@ trait and defining the ``receive`` method. The ``receive`` method defines our
|
|||
message handler. We expect it to be able to handle the ``Work`` message so we
|
||||
need to add a handler for this message:
|
||||
|
||||
.. includecode:: code/tutorials/first/Pi.scala#worker
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#worker
|
||||
:exclude: calculatePiFor
|
||||
|
||||
As you can see we have now created an ``Actor`` with a ``receive`` method as a
|
||||
handler for the ``Work`` message. In this handler we invoke the
|
||||
``calculatePiFor(..)`` method, wrap the result in a ``Result`` message and send
|
||||
it back to the original sender using ``self.reply``. In Akka the sender
|
||||
reference is implicitly passed along with the message so that the receiver can
|
||||
always reply or store away the sender reference for future use.
|
||||
it back asynchronously to the original sender using the ``sender`` reference.
|
||||
In Akka the sender reference is implicitly passed along with the message so that
|
||||
the receiver can always reply or store away the sender reference for future use.
|
||||
|
||||
The only thing missing in our ``Worker`` actor is the implementation on the
|
||||
``calculatePiFor(..)`` method. While there are many ways we can implement this
|
||||
algorithm in Scala, in this introductory tutorial we have chosen an imperative
|
||||
style using a for comprehension and an accumulator:
|
||||
|
||||
.. includecode:: code/tutorials/first/Pi.scala#calculatePiFor
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#calculatePiFor
|
||||
|
||||
|
||||
Creating the master
|
||||
===================
|
||||
|
||||
The master actor is a little bit more involved. In its constructor we need to
|
||||
create the workers (the ``Worker`` actors) and start them. We will also wrap
|
||||
them in a load-balancing router to make it easier to spread out the work evenly
|
||||
between the workers. Let's do that first:
|
||||
The master actor is a little bit more involved. In its constructor we create a round-robin router
|
||||
to make it easier to spread out the work evenly between the workers. Let's do that first:
|
||||
|
||||
.. includecode:: code/tutorials/first/Pi.scala#create-workers
|
||||
|
||||
As you can see we are using the ``actorOf`` factory method to create actors,
|
||||
this method returns as an ``ActorRef`` which is a reference to our newly created
|
||||
actor. This method is available in the ``Actor`` object but is usually
|
||||
imported::
|
||||
|
||||
import akka.actor.Actor.actorOf
|
||||
|
||||
There are two versions of ``actorOf``; one of them taking a actor type and the
|
||||
other one an instance of an actor. The former one (``actorOf(Props[MyActor]``) is used
|
||||
when the actor class has a no-argument constructor while the second one
|
||||
(``actorOf(Props(new MyActor(..))``) is used when the actor class has a constructor
|
||||
that takes arguments. This is the only way to create an instance of an Actor and
|
||||
the ``actorOf`` method ensures this. The latter version is using call-by-name
|
||||
and lazily creates the actor within the scope of the ``actorOf`` method. The
|
||||
``actorOf`` method instantiates the actor and returns, not an instance to the
|
||||
actor, but an instance to an ``ActorRef``. This reference is the handle through
|
||||
which you communicate with the actor. It is immutable, serializable and
|
||||
location-aware meaning that it "remembers" its original actor even if it is sent
|
||||
to other nodes across the network and can be seen as the equivalent to the
|
||||
Erlang actor's PID.
|
||||
|
||||
The actor's life-cycle is:
|
||||
|
||||
- Created & Started -- ``Actor.actorOf(Props[MyActor])`` -- can receive messages
|
||||
- Stopped -- ``actorRef.stop()`` -- can **not** receive messages
|
||||
|
||||
Once the actor has been stopped it is dead and can not be started again.
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#create-router
|
||||
|
||||
Now we have a router that is representing all our workers in a single
|
||||
abstraction. If you paid attention to the code above, you saw that we were using
|
||||
the ``nrOfWorkers`` variable. This variable and others we have to pass to the
|
||||
``Master`` actor in its constructor. So now let's create the master actor. We
|
||||
have to pass in three integer variables:
|
||||
abstraction. So now let's create the master actor. We pass it three integer variables:
|
||||
|
||||
- ``nrOfWorkers`` -- defining how many workers we should start up
|
||||
- ``nrOfMessages`` -- defining how many number chunks to send out to the workers
|
||||
|
|
@ -335,7 +325,7 @@ have to pass in three integer variables:
|
|||
|
||||
Here is the master actor:
|
||||
|
||||
.. includecode:: code/tutorials/first/Pi.scala#master
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#master
|
||||
:exclude: handle-messages
|
||||
|
||||
A couple of things are worth explaining further.
|
||||
|
|
@ -343,17 +333,16 @@ A couple of things are worth explaining further.
|
|||
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the
|
||||
``Master`` actor. This latch is only used for plumbing (in this specific
|
||||
tutorial), to have a simple way of letting the outside world knowing when the
|
||||
master can deliver the result and shut down. In more idiomatic Akka code, as we
|
||||
will see in part two of this tutorial series, we would not use a latch but other
|
||||
abstractions and functions like ``Channel``, ``Future`` and ``?`` to achieve the
|
||||
same thing in a non-blocking way. But for simplicity let's stick to a
|
||||
``CountDownLatch`` for now.
|
||||
master can deliver the result and shut down. In more idiomatic Akka code
|
||||
we would not use a latch but other abstractions and functions like ``Future``
|
||||
and ``?`` to achieve the same thing in a non-blocking way.
|
||||
But for simplicity let's stick to a ``CountDownLatch`` for now.
|
||||
|
||||
Second, we are adding a couple of life-cycle callback methods; ``preStart`` and
|
||||
``postStop``. In the ``preStart`` callback we are recording the time when the
|
||||
actor is started and in the ``postStop`` callback we are printing out the result
|
||||
(the approximation of Pi) and the time it took to calculate it. In this call we
|
||||
also invoke ``latch.countDown`` to tell the outside world that we are done.
|
||||
also invoke ``latch.countDown()`` to tell the outside world that we are done.
|
||||
|
||||
But we are not done yet. We are missing the message handler for the ``Master``
|
||||
actor. This message handler needs to be able to react to two different messages:
|
||||
|
|
@ -361,22 +350,19 @@ actor. This message handler needs to be able to react to two different messages:
|
|||
- ``Calculate`` -- which should start the calculation
|
||||
- ``Result`` -- which should aggregate the different results
|
||||
|
||||
The ``Calculate`` handler is sending out work to all the ``Worker`` actors and
|
||||
after doing that it also sends a ``Broadcast(PoisonPill)`` message to the
|
||||
router, which will send out the ``PoisonPill`` message to all the actors it is
|
||||
representing (in our case all the ``Worker`` actors). ``PoisonPill`` is a
|
||||
special kind of message that tells the receiver to shut itself down using the
|
||||
normal shutdown method; ``self.stop``. We also send a ``PoisonPill`` to the
|
||||
router itself (since it's also an actor that we want to shut down).
|
||||
The ``Calculate`` handler is sending out work to all the ``Worker`` via its router.
|
||||
|
||||
The ``Result`` handler is simpler, here we get the value from the ``Result``
|
||||
message and aggregate it to our ``pi`` member variable. We also keep track of
|
||||
how many results we have received back, and if that matches the number of tasks
|
||||
sent out, the ``Master`` actor considers itself done and shuts down.
|
||||
The ``Result`` handler gets the value from the ``Result`` message and aggregates it to
|
||||
our ``pi`` member variable. We also keep track of how many results we have received back,
|
||||
and if that matches the number of tasks sent out, the ``Master`` actor considers itself done and
|
||||
invokes the ``self.stop()`` method to stop itself *and* all its supervised actors.
|
||||
In this case it has one supervised actor, the router, and this in turn has ``nrOfWorkers`` supervised actors.
|
||||
All of them will be stopped automatically as the invocation of any supervisor's ``stop`` method
|
||||
will propagate down to all its supervised 'children'.
|
||||
|
||||
Let's capture this in code:
|
||||
|
||||
.. includecode:: code/tutorials/first/Pi.scala#master-receive
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#master-receive
|
||||
|
||||
|
||||
Bootstrap the calculation
|
||||
|
|
@ -391,29 +377,35 @@ The ``Pi`` object is a perfect container module for our actors and messages, so
|
|||
let's put them all there. We also create a method ``calculate`` in which we
|
||||
start up the ``Master`` actor and wait for it to finish:
|
||||
|
||||
.. includecode:: code/tutorials/first/Pi.scala#app
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#app
|
||||
:exclude: actors-and-messages
|
||||
|
||||
As you can see the *calculate* method above it creates an ActorSystem and this is the Akka container which
|
||||
will contain all actors created in that "context". An example of how to create actors in the container
|
||||
is the *'system.actorOf(...)'* line in the calculate method. In this case we create a top level actor.
|
||||
If you instead where in an actor context, i.e. inside an actor creating other actors, you should use
|
||||
*context.actorOf(...)*. This is illustrated in the Master code above.
|
||||
|
||||
That's it. Now we are done.
|
||||
|
||||
But before we package it up and run it, let's take a look at the full code now,
|
||||
with package declaration, imports and all:
|
||||
|
||||
.. includecode:: code/tutorials/first/Pi.scala
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala
|
||||
|
||||
|
||||
Run it as a command line application
|
||||
====================================
|
||||
|
||||
If you have not typed in (or copied) the code for the tutorial as
|
||||
``$AKKA_HOME/tutorial/Pi.scala`` then now is the time. When that's done open up
|
||||
a shell and step in to the Akka distribution (``cd $AKKA_HOME``).
|
||||
If you have not typed in (or copied) the code for the tutorial as in
|
||||
``$AKKA_HOME/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala`` then now is the time.
|
||||
When that's done open up a shell and step in to the Akka distribution (``cd $AKKA_HOME``).
|
||||
|
||||
First we need to compile the source file. That is done with Scala's compiler
|
||||
``scalac``. Our application depends on the ``akka-actor-2.0-SNAPSHOT.jar`` JAR
|
||||
file, so let's add that to the compiler classpath when we compile the source::
|
||||
|
||||
$ scalac -cp lib/akka/akka-actor-2.0-SNAPSHOT.jar tutorial/Pi.scala
|
||||
$ scalac -cp lib/akka/akka-actor-2.0-SNAPSHOT.jar Pi.scala
|
||||
|
||||
When we have compiled the source file we are ready to run the application. This
|
||||
is done with ``java`` but yet again we need to add the
|
||||
|
|
@ -426,7 +418,7 @@ compiled ourselves::
|
|||
akka.tutorial.first.scala.Pi
|
||||
|
||||
Pi estimate: 3.1435501812459323
|
||||
Calculation time: 858 millis
|
||||
Calculation time: 553 millis
|
||||
|
||||
Yippee! It is working.
|
||||
|
||||
|
|
@ -445,7 +437,7 @@ When this in done we can run our application directly inside SBT::
|
|||
> run
|
||||
...
|
||||
Pi estimate: 3.1435501812459323
|
||||
Calculation time: 942 millis
|
||||
Calculation time: 531 millis
|
||||
|
||||
Yippee! It is working.
|
||||
|
||||
|
|
|
|||
|
|
@ -36,8 +36,7 @@ Download
|
|||
--------
|
||||
|
||||
There are several ways to download Akka. You can download the full distribution
|
||||
with microkernel, which includes all modules. You can download just the core
|
||||
distribution or just the actors distribution. Or you can use a build tool like
|
||||
with microkernel, which includes all modules. Or you can use a build tool like
|
||||
Maven or sbt to download dependencies from the Akka Maven repository.
|
||||
|
||||
Modules
|
||||
|
|
@ -54,7 +53,7 @@ Akka is very modular and has many JARs for containing different features.
|
|||
- ``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)
|
||||
- ``akka-camel-typed-2.0-SNAPSHOT.jar`` -- Apache Camel Typed Actors integration
|
||||
- ``akka-spring-2.0-SNAPSHOT.jar`` -- Spring framework integration
|
||||
- ``akka-kernel-2.0-SNAPSHOT.jar`` -- Akka microkernel for running a bare-bones mini application server (embeds Jetty etc.)
|
||||
- ``akka-kernel-2.0-SNAPSHOT.jar`` -- Akka microkernel for running a bare-bones mini application server
|
||||
|
||||
How to see the JARs dependencies of each Akka module is described in the
|
||||
:ref:`dependencies` section. Worth noting is that ``akka-actor`` has zero
|
||||
|
|
@ -69,18 +68,18 @@ http://akka.io/downloads and unzip it.
|
|||
Using a snapshot version
|
||||
------------------------
|
||||
|
||||
The Akka nightly snapshots are published to
|
||||
http://repo.typesafe.com/typesafe/maven-timestamps/ and are versioned with a
|
||||
timestamp. You need to choose a timestamped version to work with and can decide
|
||||
when to update to a newer version.
|
||||
The Akka nightly snapshots are published to http://akka.io/snapshots/ and are
|
||||
versioned with both ``SNAPSHOT`` and timestamps. You can choose a timestamped
|
||||
version to work with and can decide when to update to a newer version. The Akka
|
||||
snapshots repository is also proxied through http://repo.typesafe.com/typesafe/snapshots/
|
||||
which includes proxies for several other repositories that Akka modules depend on.
|
||||
|
||||
Microkernel
|
||||
^^^^^^^^^^^
|
||||
-----------
|
||||
|
||||
The Akka Modules distribution includes the microkernel. To run the microkernel:
|
||||
|
||||
* Set the AKKA_HOME environment variable to the root of the Akka distribution.
|
||||
* To start the kernel use the scripts in the ``bin`` directory and deploy all samples applications from ``./deploy`` dir.
|
||||
The Akka distribution includes the microkernel. To run the microkernel put your
|
||||
application jar in the ``deploy`` directory and use the scripts in the ``bin``
|
||||
directory.
|
||||
|
||||
More information is available in the documentation of the :ref:`microkernel`.
|
||||
|
||||
|
|
@ -88,7 +87,7 @@ Using a build tool
|
|||
------------------
|
||||
|
||||
Akka can be used with build tools that support Maven repositories. The Akka
|
||||
Maven repository can be found at http://akka.io/repository/ and Typesafe provides
|
||||
Maven repository can be found at http://akka.io/realeses/ and Typesafe provides
|
||||
http://repo.typesafe.com/typesafe/releases/ that proxies several other
|
||||
repositories, including akka.io.
|
||||
|
||||
|
|
@ -121,7 +120,7 @@ Summary of the essential parts for using Akka with Maven:
|
|||
<version>2.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
|
||||
**Note**: for snapshot versions akka uses specific timestamped versions.
|
||||
**Note**: for snapshot versions both ``SNAPSHOT`` and timestamped versions are published.
|
||||
|
||||
|
||||
Using Akka with SBT
|
||||
|
|
@ -153,7 +152,7 @@ Using Akka with Eclipse
|
|||
Information about how to use Akka with Eclipse, including how to create an Akka Eclipse project from scratch,
|
||||
can be found in the :ref:`getting-started-first-scala-eclipse`.
|
||||
|
||||
Setup SBT project and then use `sbteclipse <https://github.com/typesafehub/sbteclipse>`_ to generate Eclipse project.
|
||||
Setup SBT project and then use `sbteclipse <https://github.com/typesafehub/sbteclipse>`_ to generate Eclipse project.
|
||||
|
||||
Using Akka with IntelliJ IDEA
|
||||
-----------------------------
|
||||
|
|
@ -166,7 +165,6 @@ Build from sources
|
|||
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>`_
|
||||
|
||||
Continue reading the page on :ref:`building-akka`
|
||||
|
||||
|
|
|
|||
|
|
@ -1,7 +1,8 @@
|
|||
Examples of use-cases for Akka
|
||||
==============================
|
||||
|
||||
There is a great discussion on use-cases for Akka with some good write-ups by production users `here <http://stackoverflow.com/questions/4493001/good-use-case-for-akka/4494512#4494512>`_
|
||||
There is a great discussion on use-cases for Akka with some good write-ups by production
|
||||
users `here <http://stackoverflow.com/questions/4493001/good-use-case-for-akka/4494512#4494512>`_
|
||||
|
||||
Here are some of the areas where Akka is being deployed into production
|
||||
-----------------------------------------------------------------------
|
||||
|
|
|
|||
|
|
@ -34,7 +34,7 @@ Actors give you:
|
|||
|
||||
- Simple and high-level abstractions for concurrency and parallelism.
|
||||
- Asynchronous, non-blocking and highly performant event-driven programming model.
|
||||
- Very lightweight event-driven processes (create ~6.5 million actors on 4GB RAM).
|
||||
- Very lightweight event-driven processes (approximately 2.7 million actors per GB RAM).
|
||||
|
||||
See :ref:`actors-scala` and :ref:`untyped-actors-java`
|
||||
|
||||
|
|
@ -47,16 +47,11 @@ systems that self-heal.
|
|||
|
||||
See :ref:`fault-tolerance-scala` and :ref:`fault-tolerance-java`
|
||||
|
||||
Software Transactional Memory (STM)
|
||||
-----------------------------------
|
||||
|
||||
See :ref:`stm-scala` and :ref:`stm-java`
|
||||
|
||||
Transactors
|
||||
-----------
|
||||
|
||||
Transactors combine actors and STM into transactional actors. Allows you to
|
||||
compose atomic message flows with automatic retry and rollback.
|
||||
Transactors combine actors and STM (Software Transactional Memory) into transactional actors.
|
||||
It allows you to compose atomic message flows with automatic retry and rollback.
|
||||
|
||||
See :ref:`transactors-scala` and :ref:`transactors-java`
|
||||
|
||||
|
|
@ -81,7 +76,6 @@ Akka can be used in two different ways
|
|||
- As a library: used by a web app, to be put into ‘WEB-INF/lib’ or as a regular
|
||||
JAR on your classpath.
|
||||
|
||||
- As a microkernel: stand-alone kernel, embedding a servlet container and all
|
||||
the other modules.
|
||||
- As a microkernel: stand-alone kernel to drop your application into.
|
||||
|
||||
See the :ref:`deployment-scenarios` for details.
|
||||
|
|
|
|||
|
|
@ -26,10 +26,8 @@ of concurrency-paradigms, allowing for users to choose the right tool for the
|
|||
job.
|
||||
|
||||
The integration possibilities for Akka Actors are immense through the Apache
|
||||
Camel integration. We provide Software Transactional Memory concurrency control
|
||||
through the excellent Multiverse project, and have integrated that with Actors,
|
||||
creating Transactors for coordinated concurrent transactions. We have Agents and
|
||||
Dataflow concurrency as well.
|
||||
Camel integration. We have Transactors for coordinated concurrent transactions,
|
||||
as well as Agents and Dataflow concurrency.
|
||||
|
||||
|
||||
What's a good use-case for Akka?
|
||||
|
|
@ -54,14 +52,12 @@ have a commercial product called Akka Atmos which provides the following
|
|||
features:
|
||||
|
||||
#. Management through Dashboard, JMX and REST
|
||||
#. Monitoring through Dashboard, JMX and SNMP
|
||||
#. Dapper-style tracing of messages across components and remote nodes
|
||||
#. A configurable alert system
|
||||
#. Real-time statistics
|
||||
#. Very low overhead monitoring agents (should always be on in production)
|
||||
#. Consolidation of statistics and logging information to a single node
|
||||
#. Data analysis through Hadoop
|
||||
#. Storage of statistics data for later processing
|
||||
#. Provisioning and rolling upgrades through a dashboard
|
||||
#. Provisioning and rolling upgrades
|
||||
|
||||
Read more `here <http://typesafe.com/products/typesafe-subscription>`_.
|
||||
|
|
|
|||
|
|
@ -5,4 +5,32 @@
|
|||
Microkernel
|
||||
#############
|
||||
|
||||
The Akka Spring module has not been migrated to Akka 2.0-SNAPSHOT yet.
|
||||
The Akka Microkernel is included in the Akka download found at `downloads`_.
|
||||
|
||||
.. _downloads: http://akka.io/downloads
|
||||
|
||||
To run an application with the microkernel you need to create a Bootable class
|
||||
that handles the startup and shutdown the application. An example is included below.
|
||||
|
||||
Put your application jar in the ``deploy`` directory to have it automatically
|
||||
loaded.
|
||||
|
||||
To start the kernel use the scripts in the ``bin`` directory, passing the boot
|
||||
classes for your application.
|
||||
|
||||
There is a simple example of an application setup for running with the
|
||||
microkernel included in the akka download. This can be run with the following
|
||||
command (on a unix-based system):
|
||||
|
||||
.. code-block:: none
|
||||
|
||||
bin/akka sample.kernel.hello.HelloKernel
|
||||
|
||||
Use Ctrl-C to interrupt and exit the microkernel.
|
||||
|
||||
On a Windows machine you can also use the bin/akka.bat script.
|
||||
|
||||
The code for the Hello Kernel example (see the HelloKernel class for an example
|
||||
of creating a Bootable):
|
||||
|
||||
.. includecode:: ../../akka-samples/akka-sample-hello-kernel/src/main/scala/sample/kernel/hello/HelloKernel.scala
|
||||
|
|
|
|||
|
|
@ -27,41 +27,44 @@ Akka uses Git and is hosted at `Github <http://github.com>`_.
|
|||
* Akka: clone the Akka repository from `<http://github.com/jboner/akka>`_
|
||||
|
||||
|
||||
`Maven Repository <http://akka.io/repository/>`_
|
||||
================================================
|
||||
`Releases Repository <http://akka.io/releases/>`_
|
||||
=================================================
|
||||
|
||||
The Akka Maven repository can be found at `<http://akka.io/repository>`_.
|
||||
The Akka Maven repository can be found at http://akka.io/releases/.
|
||||
|
||||
Typesafe provides `<http://repo.typesafe.com/typesafe/releases/>`_ that proxies several other repositories, including akka.io.
|
||||
It is convenient to use the Typesafe repository, since it includes all external dependencies of Akka.
|
||||
It is a "best-effort" service, and if it is unavailable you may need to use the underlying repositories
|
||||
directly.
|
||||
Typesafe provides http://repo.typesafe.com/typesafe/releases/ that proxies
|
||||
several other repositories, including akka.io. It is convenient to use the
|
||||
Typesafe repository, since it includes all external dependencies of Akka. It is
|
||||
a "best-effort" service, and if it is unavailable you may need to use the
|
||||
underlying repositories directly.
|
||||
|
||||
* http://akka.io/repository
|
||||
* http://repository.codehaus.org
|
||||
* http://akka.io/releases/
|
||||
* http://repository.codehaus.org/
|
||||
* http://guiceyfruit.googlecode.com/svn/repo/releases/
|
||||
* http://repository.jboss.org/nexus/content/groups/public/
|
||||
* http://download.java.net/maven/2
|
||||
* http://oss.sonatype.org/content/repositories/releases
|
||||
* http://download.java.net/maven/glassfish
|
||||
* http://databinder.net/repo
|
||||
* http://download.java.net/maven/2/
|
||||
* http://oss.sonatype.org/content/repositories/releases/
|
||||
* http://download.java.net/maven/glassfish/
|
||||
* http://databinder.net/repo/
|
||||
|
||||
SNAPSHOT Versions
|
||||
=================
|
||||
|
||||
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/akka-snapshots/com/typesafe/akka/akka-actor/>`_.
|
||||
`Snapshots Repository <http://akka.io/snapshots/>`_
|
||||
===================================================
|
||||
|
||||
Nightly builds are available in http://akka.io/snapshots/ and proxied through
|
||||
http://repo.typesafe.com/typesafe/snapshots/ as both ``SNAPSHOT`` and
|
||||
timestamped versions.
|
||||
|
||||
For timestamped versions, pick a timestamp from
|
||||
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/akka-snapshots/"
|
||||
|
||||
Define the library dependencies with the timestamp as version::
|
||||
|
||||
libraryDependencies += "com.typesafe.akka" % "akka-actor" % "2.0-20111118-000627"
|
||||
resolvers += "Typesafe Snapshots" at "http://repo.typesafe.com/typesafe/snapshots/"
|
||||
|
||||
libraryDependencies += "com.typesafe.akka" % "akka-remote" % "2.0-20111118-000627"
|
||||
Define the library dependencies with the timestamp as version. For example::
|
||||
|
||||
libraryDependencies += "com.typesafe.akka" % "akka-actor" % "2.0-20111215-000549"
|
||||
|
||||
libraryDependencies += "com.typesafe.akka" % "akka-remote" % "2.0-20111215-000549"
|
||||
|
|
|
|||
|
|
@ -1,147 +1,4 @@
|
|||
Agents (Scala)
|
||||
==============
|
||||
|
||||
.. sidebar:: Contents
|
||||
|
||||
.. contents:: :local:
|
||||
|
||||
Agents in Akka were inspired by `agents in Clojure <http://clojure.org/agents>`_.
|
||||
|
||||
Agents provide asynchronous change of individual locations. Agents are bound to a single storage location for their lifetime, and only allow mutation of that location (to a new state) to occur as a result of an action. Update actions are functions that are asynchronously applied to the Agent's state and whose return value becomes the Agent's new state. The state of an Agent should be immutable.
|
||||
|
||||
While updates to Agents are asynchronous, the state of an Agent is always immediately available for reading by any thread (using ``get`` or ``apply``) without any messages.
|
||||
|
||||
Agents are reactive. The update actions of all Agents get interleaved amongst threads in a thread pool. At any point in time, at most one ``send`` action for each Agent is being executed. Actions dispatched to an agent from another thread will occur in the order they were sent, potentially interleaved with actions dispatched to the same agent from other sources.
|
||||
|
||||
If an Agent is used within an enclosing transaction, then it will participate in that transaction. 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.
|
||||
|
||||
Creating and stopping Agents
|
||||
----------------------------
|
||||
|
||||
Agents are created by invoking ``Agent(value)`` passing in the Agent's initial value.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
val agent = Agent(5)
|
||||
|
||||
An Agent will be running until you invoke ``close`` on it. Then it will be eligible for garbage collection (unless you hold on to it in some way).
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
agent.close()
|
||||
|
||||
Updating Agents
|
||||
---------------
|
||||
|
||||
You update an Agent by sending a function that transforms the current value or by sending just a new value. The Agent will apply the new value or function atomically and asynchronously. The update is done in a fire-forget manner and you are only guaranteed that it will be applied. There is no guarantee of when the update will be applied but dispatches to an Agent from a single thread will occur in order. You apply a value or a function by invoking the ``send`` function.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
// send a value
|
||||
agent send 7
|
||||
|
||||
// send a function
|
||||
agent send (_ + 1)
|
||||
agent send (_ * 2)
|
||||
|
||||
You can also dispatch a function to update the internal state but on its own thread. This does not use the reactive thread pool and can be used for long-running or blocking operations. You do this with the ``sendOff`` method. Dispatches using either ``sendOff`` or ``send`` will still be executed in order.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
// sendOff a function
|
||||
agent sendOff (longRunningOrBlockingFunction)
|
||||
|
||||
Reading an Agent's value
|
||||
------------------------
|
||||
|
||||
Agents can be dereferenced, e.g. you can get an Agent's value, by invoking the Agent with parenthesis like this:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
val result = agent()
|
||||
|
||||
Or by using the get method.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
val result = agent.get
|
||||
|
||||
Reading an Agent's current value does not involve any message passing and happens immediately. So while updates to an Agent are asynchronous, reading the state of an Agent is synchronous.
|
||||
|
||||
Awaiting an Agent's value
|
||||
-------------------------
|
||||
|
||||
It is also possible to read the value after all currently queued ``send``\s have completed. You can do this with ``await``:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
val result = agent.await
|
||||
|
||||
You can also get a ``Future`` to this value, that will be completed after the currently queued updates have completed:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
val future = agent.future
|
||||
// ...
|
||||
val result = future.await.result.get
|
||||
|
||||
Transactional Agents
|
||||
--------------------
|
||||
|
||||
If an Agent is used within an enclosing transaction, then it will participate in that transaction. If you send to an Agent within a transaction then the dispatch to the Agent will be held until that transaction commits, and discarded if the transaction is aborted.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.agent.Agent
|
||||
import akka.stm._
|
||||
|
||||
def transfer(from: Agent[Int], to: Agent[Int], amount: Int): Boolean = {
|
||||
atomic {
|
||||
if (from.get < amount) false
|
||||
else {
|
||||
from send (_ - amount)
|
||||
to send (_ + amount)
|
||||
true
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
val from = Agent(100)
|
||||
val to = Agent(20)
|
||||
val ok = transfer(from, to, 50)
|
||||
|
||||
from() // -> 50
|
||||
to() // -> 70
|
||||
|
||||
Monadic usage
|
||||
-------------
|
||||
|
||||
Agents are also monadic, allowing you to compose operations using for-comprehensions. In a monadic usage, new Agents are created leaving the original Agents untouched. So the old values (Agents) are still available as-is. They are so-called 'persistent'.
|
||||
|
||||
Example of a monadic usage:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
val agent1 = Agent(3)
|
||||
val agent2 = Agent(5)
|
||||
|
||||
// uses foreach
|
||||
var result = 0
|
||||
for (value <- agent1) {
|
||||
result = value + 1
|
||||
}
|
||||
|
||||
// uses map
|
||||
val agent3 =
|
||||
for (value <- agent1) yield value + 1
|
||||
|
||||
// uses flatMap
|
||||
val agent4 = for {
|
||||
value1 <- agent1
|
||||
value2 <- agent2
|
||||
} yield value1 + value2
|
||||
|
||||
agent1.close()
|
||||
agent2.close()
|
||||
agent3.close()
|
||||
agent4.close()
|
||||
The Akka Agents module has not been migrated to Akka 2.0-SNAPSHOT yet.
|
||||
|
|
@ -1,23 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.http
|
||||
|
||||
import akka.config.Config
|
||||
import akka.util.{ Bootable, AkkaLoader }
|
||||
import akka.cluster.BootableRemoteActorService
|
||||
import akka.actor.BootableActorLoaderService
|
||||
|
||||
class DefaultAkkaLoader extends AkkaLoader {
|
||||
def boot(): Unit = boot(true, new EmbeddedAppServer with BootableActorLoaderService with BootableRemoteActorService)
|
||||
}
|
||||
|
||||
/**
|
||||
* Can be used to boot Akka
|
||||
*
|
||||
* java -cp ... akka.http.Main
|
||||
*/
|
||||
object Main extends DefaultAkkaLoader {
|
||||
def main(args: Array[String]) = boot
|
||||
}
|
||||
|
|
@ -1,73 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.http
|
||||
|
||||
import javax.ws.rs.core.UriBuilder
|
||||
import javax.servlet.ServletConfig
|
||||
import java.io.File
|
||||
|
||||
import akka.actor.BootableActorLoaderService
|
||||
import akka.util.Bootable
|
||||
|
||||
import org.eclipse.jetty.xml.XmlConfiguration
|
||||
import org.eclipse.jetty.server.{ Handler, Server }
|
||||
import org.eclipse.jetty.server.handler.{ HandlerList, HandlerCollection, ContextHandler }
|
||||
import java.net.URL
|
||||
import akka.AkkaException
|
||||
|
||||
/**
|
||||
* Handles the Akka Comet Support (load/unload)
|
||||
*/
|
||||
trait EmbeddedAppServer extends Bootable {
|
||||
self: BootableActorLoaderService ⇒
|
||||
|
||||
import akka.config.Config._
|
||||
|
||||
val REST_HOSTNAME = config.getString("akka.http.hostname", "localhost")
|
||||
val REST_PORT = config.getInt("akka.http.port", 9998)
|
||||
|
||||
val isRestEnabled = config.getList("akka.enabled-modules").exists(_ == "http")
|
||||
|
||||
protected var server: Option[Server] = None
|
||||
|
||||
protected def findJettyConfigXML: Option[URL] =
|
||||
Option(applicationLoader.getOrElse(this.getClass.getClassLoader).getResource("microkernel-server.xml")) orElse
|
||||
HOME.map(home ⇒ new File(home + "/config/microkernel-server.xml").toURI.toURL)
|
||||
|
||||
abstract override def onLoad = {
|
||||
super.onLoad
|
||||
if (isRestEnabled) {
|
||||
|
||||
val configuration = new XmlConfiguration(findJettyConfigXML.getOrElse(sys.error("microkernel-server.xml not found!")))
|
||||
|
||||
System.setProperty("jetty.port", REST_PORT.toString)
|
||||
System.setProperty("jetty.host", REST_HOSTNAME)
|
||||
|
||||
HOME.foreach(home ⇒ System.setProperty("jetty.home", home + "/deploy/root"))
|
||||
|
||||
server = Option(configuration.configure.asInstanceOf[Server]) map { s ⇒ //Set the correct classloader to our contexts
|
||||
applicationLoader foreach { loader ⇒
|
||||
//We need to provide the correct classloader to the servlets
|
||||
def setClassLoader(handlers: Seq[Handler]) {
|
||||
handlers foreach {
|
||||
case c: ContextHandler ⇒ c.setClassLoader(loader)
|
||||
case c: HandlerCollection ⇒ setClassLoader(c.getHandlers)
|
||||
case _ ⇒
|
||||
}
|
||||
}
|
||||
setClassLoader(s.getHandlers)
|
||||
}
|
||||
//Start the server
|
||||
s.start()
|
||||
s
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
abstract override def onUnload = {
|
||||
super.onUnload
|
||||
server foreach { _.stop() }
|
||||
}
|
||||
}
|
||||
|
|
@ -1,36 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2010 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.kernel
|
||||
|
||||
import akka.http.EmbeddedAppServer
|
||||
import akka.util.AkkaLoader
|
||||
import akka.cluster.BootableRemoteActorService
|
||||
import akka.actor.BootableActorLoaderService
|
||||
import akka.camel.CamelService
|
||||
|
||||
import java.util.concurrent.CountDownLatch
|
||||
|
||||
object Main {
|
||||
val keepAlive = new CountDownLatch(2)
|
||||
|
||||
def main(args: Array[String]) = {
|
||||
Kernel.boot
|
||||
keepAlive.await
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The Akka Kernel, is used to start And postStop Akka in standalone/kernel mode.
|
||||
*/
|
||||
object Kernel extends AkkaLoader {
|
||||
|
||||
def boot(): Unit = boot(true, new EmbeddedAppServer with BootableActorLoaderService with BootableRemoteActorService with CamelService)
|
||||
|
||||
// For testing purposes only
|
||||
def startRemoteService(): Unit = bundles.foreach(_ match {
|
||||
case x: BootableRemoteActorService ⇒ x.startRemoteService()
|
||||
case _ ⇒
|
||||
})
|
||||
}
|
||||
185
akka-kernel/src/main/scala/akka/kernel/Main.scala
Normal file
185
akka-kernel/src/main/scala/akka/kernel/Main.scala
Normal file
|
|
@ -0,0 +1,185 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2010 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.kernel
|
||||
|
||||
import akka.actor.ActorSystem
|
||||
import java.io.File
|
||||
import java.lang.Boolean.getBoolean
|
||||
import java.net.{ URL, URLClassLoader }
|
||||
import java.util.jar.JarFile
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
/**
|
||||
* To use the microkernel at least one 'boot class' needs to be specified.
|
||||
* A boot class implements this interface ([[akka.kernel.Bootable]]) and
|
||||
* must have an empty default constructor.
|
||||
*
|
||||
* ActorSystems can be created within the boot class.
|
||||
*
|
||||
* An example of a simple boot class:
|
||||
* {{{
|
||||
* class BootApp extends Bootable {
|
||||
* val system = ActorSystem("app")
|
||||
*
|
||||
* def startup = {
|
||||
* system.actorOf(Props[FirstActor]) ! FirstMessage
|
||||
* }
|
||||
*
|
||||
* def shutdown = {
|
||||
* system.shutdown()
|
||||
* }
|
||||
* }
|
||||
* }}}
|
||||
*
|
||||
* Boot classes are specified as main arguments to the microkernel.
|
||||
*
|
||||
* For example, using the akka script an application can be started with
|
||||
* the following at the command line:
|
||||
* {{{
|
||||
* bin/akka org.app.BootApp
|
||||
* }}}
|
||||
*/
|
||||
trait Bootable {
|
||||
/**
|
||||
* Callback run on microkernel startup.
|
||||
* Create initial actors and messages here.
|
||||
*/
|
||||
def startup(): Unit
|
||||
|
||||
/**
|
||||
* Callback run on microkernel shutdown.
|
||||
* Shutdown actor systems here.
|
||||
*/
|
||||
def shutdown(): Unit
|
||||
}
|
||||
|
||||
/**
|
||||
* Main class for running the microkernel.
|
||||
*/
|
||||
object Main {
|
||||
val quiet = getBoolean("akka.kernel.quiet")
|
||||
|
||||
def log(s: String) = if (!quiet) println(s)
|
||||
|
||||
def main(args: Array[String]) = {
|
||||
if (args.isEmpty) {
|
||||
log("[error] No boot classes specified")
|
||||
System.exit(1)
|
||||
}
|
||||
|
||||
log(banner)
|
||||
log("Starting Akka...")
|
||||
log("Running Akka " + ActorSystem.Version)
|
||||
|
||||
val classLoader = createClassLoader()
|
||||
|
||||
Thread.currentThread.setContextClassLoader(classLoader)
|
||||
|
||||
val bootClasses: Seq[String] = args.toSeq
|
||||
val bootables: Seq[Bootable] = bootClasses map { c ⇒ classLoader.loadClass(c).newInstance.asInstanceOf[Bootable] }
|
||||
|
||||
for (bootable ← bootables) {
|
||||
log("Starting up " + bootable.getClass.getName)
|
||||
bootable.startup()
|
||||
}
|
||||
|
||||
addShutdownHook(bootables)
|
||||
|
||||
log("Successfully started Akka")
|
||||
}
|
||||
|
||||
def createClassLoader(): ClassLoader = {
|
||||
if (ActorSystem.GlobalHome.isDefined) {
|
||||
val home = ActorSystem.GlobalHome.get
|
||||
val deploy = new File(home, "deploy")
|
||||
if (deploy.exists) {
|
||||
loadDeployJars(deploy)
|
||||
} else {
|
||||
log("[warning] No deploy dir found at " + deploy)
|
||||
Thread.currentThread.getContextClassLoader
|
||||
}
|
||||
} else {
|
||||
log("[warning] Akka home is not defined")
|
||||
Thread.currentThread.getContextClassLoader
|
||||
}
|
||||
}
|
||||
|
||||
def loadDeployJars(deploy: File): ClassLoader = {
|
||||
val jars = deploy.listFiles.filter(_.getName.endsWith(".jar"))
|
||||
|
||||
val nestedJars = jars flatMap { jar ⇒
|
||||
val jarFile = new JarFile(jar)
|
||||
val jarEntries = jarFile.entries.asScala.toArray.filter(_.getName.endsWith(".jar"))
|
||||
jarEntries map { entry ⇒ new File("jar:file:%s!/%s" format (jarFile.getName, entry.getName)) }
|
||||
}
|
||||
|
||||
val urls = (jars ++ nestedJars) map { _.toURI.toURL }
|
||||
|
||||
urls foreach { url ⇒ log("Deploying " + url) }
|
||||
|
||||
new URLClassLoader(urls, Thread.currentThread.getContextClassLoader)
|
||||
}
|
||||
|
||||
def addShutdownHook(bootables: Seq[Bootable]): Unit = {
|
||||
Runtime.getRuntime.addShutdownHook(new Thread(new Runnable {
|
||||
def run = {
|
||||
log("")
|
||||
log("Shutting down Akka...")
|
||||
|
||||
for (bootable ← bootables) {
|
||||
log("Shutting down " + bootable.getClass.getName)
|
||||
bootable.shutdown()
|
||||
}
|
||||
|
||||
log("Successfully shut down Akka")
|
||||
}
|
||||
}))
|
||||
}
|
||||
|
||||
def banner = """
|
||||
==============================================================================
|
||||
|
||||
ZZ:
|
||||
ZZZZ
|
||||
ZZZZZZ
|
||||
ZZZ' ZZZ
|
||||
~7 7ZZ' ZZZ
|
||||
:ZZZ: IZZ' ZZZ
|
||||
,OZZZZ.~ZZ? ZZZ
|
||||
ZZZZ' 'ZZZ$ ZZZ
|
||||
. $ZZZ ~ZZ$ ZZZ
|
||||
.=Z?. .ZZZO ~ZZ7 OZZ
|
||||
.ZZZZ7..:ZZZ~ 7ZZZ ZZZ~
|
||||
.$ZZZ$Z+.ZZZZ ZZZ: ZZZ$
|
||||
.,ZZZZ?' =ZZO= .OZZ 'ZZZ
|
||||
.$ZZZZ+ .ZZZZ IZZZ ZZZ$
|
||||
.ZZZZZ' .ZZZZ' .ZZZ$ ?ZZZ
|
||||
.ZZZZZZ' .OZZZ? ?ZZZ 'ZZZ$
|
||||
.?ZZZZZZ' .ZZZZ? .ZZZ? 'ZZZO
|
||||
.+ZZZZZZ?' .7ZZZZ' .ZZZZ :ZZZZ
|
||||
.ZZZZZZ$' .?ZZZZZ' .~ZZZZ 'ZZZZ.
|
||||
|
||||
|
||||
NNNNN $NNNN+
|
||||
NNNNN $NNNN+
|
||||
NNNNN $NNNN+
|
||||
NNNNN $NNNN+
|
||||
NNNNN $NNNN+
|
||||
=NNNNNNNNND$ NNNNN DDDDDD: $NNNN+ DDDDDN NDDNNNNNNNN,
|
||||
NNNNNNNNNNNNND NNNNN DNNNNN $NNNN+ 8NNNNN= :NNNNNNNNNNNNNN
|
||||
NNNNN$ DNNNNN NNNNN $NNNNN~ $NNNN+ NNNNNN NNNNN, :NNNNN+
|
||||
?DN~ NNNNN NNNNN MNNNNN $NNNN+:NNNNN7 $ND =NNNNN
|
||||
DNNNNN NNNNNDNNNN$ $NNNNDNNNNN :DNNNNN
|
||||
ZNDNNNNNNNNND NNNNNNNNNND, $NNNNNNNNNNN DNDNNNNNNNNNN
|
||||
NNNNNNNDDINNNNN NNNNNNNNNNND $NNNNNNNNNNND ONNNNNNND8+NNNNN
|
||||
:NNNND NNNNN NNNNNN DNNNN, $NNNNNO 7NNNND NNNNNO :NNNNN
|
||||
DNNNN NNNNN NNNNN DNNNN $NNNN+ 8NNNNN NNNNN $NNNNN
|
||||
DNNNNO NNNNNN NNNNN NNNNN $NNNN+ NNNNN$ NNNND, ,NNNNND
|
||||
NNNNNNDDNNNNNNNN NNNNN =NNNNN $NNNN+ DNNNN? DNNNNNNDNNNNNNNND
|
||||
NNNNNNNNN NNNN$ NNNNN 8NNNND $NNNN+ NNNNN= ,DNNNNNNND NNNNN$
|
||||
|
||||
==============================================================================
|
||||
"""
|
||||
}
|
||||
|
|
@ -1,33 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.servlet
|
||||
|
||||
import akka.cluster.BootableRemoteActorService
|
||||
import akka.actor.BootableActorLoaderService
|
||||
import akka.config.Config
|
||||
import akka.util.{ Bootable, AkkaLoader }
|
||||
|
||||
import javax.servlet.{ ServletContextListener, ServletContextEvent }
|
||||
|
||||
/**
|
||||
* This class can be added to web.xml mappings as a listener to start and postStop Akka.
|
||||
*
|
||||
* <web-system>
|
||||
* ...
|
||||
* <listener>
|
||||
* <listener-class>akka.servlet.Initializer</listener-class>
|
||||
* </listener>
|
||||
* ...
|
||||
* </web-system>
|
||||
*/
|
||||
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)
|
||||
}
|
||||
24
akka-kernel/src/main/scripts/akka
Executable file
24
akka-kernel/src/main/scripts/akka
Executable file
|
|
@ -0,0 +1,24 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
declare quiet="false"
|
||||
|
||||
while true; do
|
||||
case "$1" in
|
||||
-q | --quiet ) quiet="true"; shift ;;
|
||||
* ) break ;;
|
||||
esac
|
||||
done
|
||||
|
||||
[[ "$@" ]] || {
|
||||
echo "No boot classes specified"
|
||||
echo "Usage: bin/akka org.somewhere.BootClass"
|
||||
exit 1
|
||||
}
|
||||
|
||||
declare AKKA_HOME="$(cd "$(cd "$(dirname "$0")"; pwd -P)"/..; pwd)"
|
||||
|
||||
[ -n "$JAVA_OPTS" ] || JAVA_OPTS="-Xmx1024M -Xms1024M -Xss1M -XX:MaxPermSize=256M -XX:+UseParallelGC -XX:OnOutOfMemoryError=\"kill -9 %p\""
|
||||
|
||||
[ -n "$AKKA_CLASSPATH" ] || AKKA_CLASSPATH="$AKKA_HOME/lib/scala-library.jar:$AKKA_HOME/lib/akka/*:$AKKA_HOME/config"
|
||||
|
||||
java "$JAVA_OPTS" -cp "$AKKA_CLASSPATH" -Dakka.home="$AKKA_HOME" -Dakka.kernel.quiet=$quiet akka.kernel.Main "$@"
|
||||
|
|
@ -1,6 +1,7 @@
|
|||
@echo off
|
||||
|
||||
set AKKA_HOME=%~dp0..
|
||||
set JAVA_OPTS=-Xms1024M -Xmx1024M -Xss1M -XX:MaxPermSize=256M -XX:+UseParallelGC
|
||||
set JAVA_OPTS=-Xmx1024M -Xms1024M -Xss1M -XX:MaxPermSize=256M -XX:+UseParallelGC
|
||||
set AKKA_CLASSPATH=%AKKA_HOME%\lib\scala-library.jar;%AKKA_HOME%\config;%AKKA_HOME%\lib\akka\*
|
||||
|
||||
java %JAVA_OPTS% -cp "%AKKA_CLASSPATH%" -Dakka.home="%AKKA_HOME%" akka.kernel.Main
|
||||
java %JAVA_OPTS% -cp "%AKKA_CLASSPATH%" -Dakka.home="%AKKA_HOME%" akka.kernel.Main %*
|
||||
|
|
@ -1,47 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.remote
|
||||
|
||||
import akka.actor.{ Actor, BootableActorLoaderService }
|
||||
import akka.util.{ ReflectiveAccess, Bootable }
|
||||
|
||||
// TODO: remove me - remoting is enabled through the RemoteActorRefProvider
|
||||
|
||||
/**
|
||||
* This bundle/service is responsible for booting up and shutting down the remote actors facility.
|
||||
* <p/>
|
||||
* It is used in Kernel.
|
||||
*/
|
||||
/*
|
||||
trait BootableRemoteActorService extends Bootable {
|
||||
self: BootableActorLoaderService ⇒
|
||||
|
||||
def settings: RemoteServerSettings
|
||||
|
||||
protected lazy val remoteServerThread = new Thread(new Runnable() {
|
||||
def run = system.remote.start(self.applicationLoader.getOrElse(null)) //Use config host/port
|
||||
}, "Akka RemoteModule Service")
|
||||
|
||||
def startRemoteService() { remoteServerThread.start() }
|
||||
|
||||
abstract override def onLoad() {
|
||||
if (system.reflective.ClusterModule.isEnabled && settings.isRemotingEnabled) {
|
||||
system.eventHandler.info(this, "Initializing Remote Actors Service...")
|
||||
startRemoteService()
|
||||
system.eventHandler.info(this, "Remote Actors Service initialized")
|
||||
}
|
||||
super.onLoad()
|
||||
}
|
||||
|
||||
abstract override def onUnload() {
|
||||
system.eventHandler.info(this, "Shutting down Remote Actors Service")
|
||||
|
||||
system.remote.shutdown()
|
||||
if (remoteServerThread.isAlive) remoteServerThread.join(1000)
|
||||
system.eventHandler.info(this, "Remote Actors Service has been shut down")
|
||||
super.onUnload()
|
||||
}
|
||||
}
|
||||
*/
|
||||
|
|
@ -0,0 +1,37 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package sample.kernel.hello
|
||||
|
||||
import akka.actor.{ Actor, ActorSystem, Props }
|
||||
import akka.kernel.Bootable
|
||||
|
||||
case object Start
|
||||
|
||||
class HelloActor extends Actor {
|
||||
val worldActor = context.actorOf(Props[WorldActor])
|
||||
|
||||
def receive = {
|
||||
case Start ⇒ worldActor ! "Hello"
|
||||
case message: String ⇒
|
||||
println("Received message '%s'" format message)
|
||||
}
|
||||
}
|
||||
|
||||
class WorldActor extends Actor {
|
||||
def receive = {
|
||||
case message: String ⇒ sender ! (message.toUpperCase + " world!")
|
||||
}
|
||||
}
|
||||
|
||||
class HelloKernel extends Bootable {
|
||||
val system = ActorSystem("hellokernel")
|
||||
|
||||
def startup = {
|
||||
system.actorOf(Props[HelloActor]) ! Start
|
||||
}
|
||||
|
||||
def shutdown = {
|
||||
system.shutdown()
|
||||
}
|
||||
}
|
||||
|
|
@ -95,7 +95,9 @@ object AkkaKernelPlugin extends Plugin {
|
|||
}
|
||||
|
||||
def isKernelProject(dependencies: Seq[ModuleID]): Boolean = {
|
||||
dependencies.exists(moduleId ⇒ moduleId.organization == "se.scalablesolutions.akka" && moduleId.name == "akka-kernel")
|
||||
dependencies.exists { d ⇒
|
||||
(d.organization == "com.typesafe.akka" || d.organization == "se.scalablesolutions.akka") && d.name == "akka-kernel"
|
||||
}
|
||||
}
|
||||
|
||||
private def defaultConfigSourceDirs = (sourceDirectory, unmanagedResourceDirectories) map { (src, resources) ⇒
|
||||
|
|
@ -128,7 +130,7 @@ object AkkaKernelPlugin extends Plugin {
|
|||
|AKKA_CLASSPATH="$AKKA_HOME/lib/*:$AKKA_HOME/config"
|
||||
|JAVA_OPTS="%s"
|
||||
|
|
||||
|java $JAVA_OPTS -cp "$AKKA_CLASSPATH" -Dakka.home="$AKKA_HOME" %s
|
||||
|java $JAVA_OPTS -cp "$AKKA_CLASSPATH" -Dakka.home="$AKKA_HOME" %s "$@"
|
||||
|""".stripMargin.format(jvmOptions, mainClass)
|
||||
|
||||
private def distBatScript =
|
||||
|
|
@ -137,7 +139,7 @@ object AkkaKernelPlugin extends Plugin {
|
|||
|set AKKA_CLASSPATH=%%AKKA_HOME%%\lib\*;%%AKKA_HOME%%\config
|
||||
|set JAVA_OPTS=%s
|
||||
|
|
||||
|java %%JAVA_OPTS%% -cp "%%AKKA_CLASSPATH%%" -Dakka.home="%%AKKA_HOME%%" %s
|
||||
|java %%JAVA_OPTS%% -cp "%%AKKA_CLASSPATH%%" -Dakka.home="%%AKKA_HOME%%" %s %%*
|
||||
|""".stripMargin.format(jvmOptions, mainClass)
|
||||
|
||||
private def setExecutable(target: File, executable: Boolean): Option[String] = {
|
||||
|
|
@ -194,9 +196,9 @@ object AkkaKernelPlugin extends Plugin {
|
|||
private def projectInfo(projectRef: ProjectRef, project: ResolvedProject, buildStruct: BuildStructure, state: State,
|
||||
allProjects: Map[ProjectRef, ResolvedProject]): SubProjectInfo = {
|
||||
|
||||
def optionalSetting[A](key: ScopedSetting[A]) = key in projectRef get buildStruct.data
|
||||
def optionalSetting[A](key: SettingKey[A]) = key in projectRef get buildStruct.data
|
||||
|
||||
def setting[A](key: ScopedSetting[A], errorMessage: ⇒ String) = {
|
||||
def setting[A](key: SettingKey[A], errorMessage: ⇒ String) = {
|
||||
optionalSetting(key) getOrElse {
|
||||
logger(state).error(errorMessage);
|
||||
throw new IllegalArgumentException()
|
||||
|
|
@ -204,7 +206,7 @@ object AkkaKernelPlugin extends Plugin {
|
|||
}
|
||||
|
||||
def evaluateTask[T](taskKey: sbt.Project.ScopedKey[sbt.Task[T]]) = {
|
||||
EvaluateTask.evaluateTask(buildStruct, taskKey, state, projectRef, false, EvaluateTask.SystemProcessors)
|
||||
EvaluateTask(buildStruct, taskKey, state, projectRef).map(_._2)
|
||||
}
|
||||
|
||||
val projDeps: Seq[ModuleID] = evaluateTask(Keys.projectDependencies) match {
|
||||
|
|
|
|||
|
|
@ -101,13 +101,16 @@ public class Pi {
|
|||
|
||||
private ActorRef router;
|
||||
|
||||
public Master(final int nrOfWorkers, int nrOfMessages, int nrOfElements, CountDownLatch latch) {
|
||||
public Master(final int nrOfWorkers, int nrOfMessages,
|
||||
int nrOfElements, CountDownLatch latch) {
|
||||
this.nrOfMessages = nrOfMessages;
|
||||
this.nrOfElements = nrOfElements;
|
||||
this.latch = latch;
|
||||
|
||||
//#create-router
|
||||
router = this.getContext().actorOf(new Props().withCreator(Worker.class).withRouter(new RoundRobinRouter(nrOfWorkers)), "pi");
|
||||
router = this.getContext().actorOf(new Props().withCreator(
|
||||
Worker.class).withRouter(new RoundRobinRouter(nrOfWorkers)),
|
||||
"pi");
|
||||
//#create-router
|
||||
}
|
||||
|
||||
|
|
@ -144,8 +147,10 @@ public class Pi {
|
|||
//#master
|
||||
//#actors-and-messages
|
||||
|
||||
public void calculate(final int nrOfWorkers, final int nrOfElements, final int nrOfMessages)
|
||||
throws Exception {
|
||||
public void calculate(final int nrOfWorkers,
|
||||
final int nrOfElements,
|
||||
final int nrOfMessages)
|
||||
throws Exception {
|
||||
// Create an Akka system
|
||||
final ActorSystem system = ActorSystem.create();
|
||||
|
||||
|
|
|
|||
|
|
@ -50,7 +50,8 @@ object Pi extends App {
|
|||
var start: Long = _
|
||||
|
||||
//#create-router
|
||||
val router = context.actorOf(Props(new Worker).withRouter(RoundRobinRouter(nrOfInstances = nrOfWorkers)), "pi")
|
||||
val router = context.actorOf(Props(new Worker).withRouter(
|
||||
RoundRobinRouter(nrOfInstances = nrOfWorkers)), "pi")
|
||||
//#create-router
|
||||
|
||||
//#master-receive
|
||||
|
|
@ -89,7 +90,9 @@ object Pi extends App {
|
|||
val latch = new CountDownLatch(1)
|
||||
|
||||
// create the master
|
||||
val master = system.actorOf(Props(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)), "master")
|
||||
val master = system.actorOf(Props(new Master(
|
||||
nrOfWorkers, nrOfMessages, nrOfElements, latch)),
|
||||
"master")
|
||||
|
||||
// start the calculation
|
||||
master ! Calculate
|
||||
|
|
|
|||
|
|
@ -1,2 +0,0 @@
|
|||
# In this file you can override any option defined in the 'akka-reference.conf' file.
|
||||
# Copy in all or parts of the 'akka-reference.conf' file and modify as you please.
|
||||
2
config/application.conf
Normal file
2
config/application.conf
Normal file
|
|
@ -0,0 +1,2 @@
|
|||
# In this file you can override any option defined in the 'reference.conf' files.
|
||||
# Copy in all or parts of the 'reference.conf' files and modify as you please.
|
||||
|
|
@ -1,106 +0,0 @@
|
|||
<?xml version="1.0"?>
|
||||
<!DOCTYPE Configure PUBLIC "-//Jetty//Configure//EN" "http://www.eclipse.org/jetty/configure.dtd">
|
||||
|
||||
<!-- =============================================================== -->
|
||||
<!-- Configure the Jetty Server -->
|
||||
<!-- -->
|
||||
<!-- Documentation of this file format can be found at: -->
|
||||
<!-- http://wiki.eclipse.org/Jetty/Reference/jetty.xml_syntax -->
|
||||
<!-- -->
|
||||
<!-- Additional configuration files are available in $JETTY_HOME/etc -->
|
||||
<!-- and can be mixed in. For example: -->
|
||||
<!-- java -jar start.jar etc/jetty.xml etc/jetty-ssl.xml -->
|
||||
<!-- -->
|
||||
<!-- See start.ini file for the default configuration files -->
|
||||
<!-- =============================================================== -->
|
||||
|
||||
|
||||
<Configure id="Server" class="org.eclipse.jetty.server.Server">
|
||||
|
||||
<!-- =========================================================== -->
|
||||
<!-- Server Thread Pool -->
|
||||
<!-- =========================================================== -->
|
||||
<Set name="ThreadPool">
|
||||
<New class="org.eclipse.jetty.util.thread.ExecutorThreadPool">
|
||||
</New>
|
||||
</Set>
|
||||
|
||||
<!-- =========================================================== -->
|
||||
<!-- Set connectors -->
|
||||
<!-- =========================================================== -->
|
||||
|
||||
<Call name="addConnector">
|
||||
<Arg>
|
||||
<New class="org.eclipse.jetty.server.nio.SelectChannelConnector">
|
||||
<Set name="host"><SystemProperty name="jetty.host" /></Set>
|
||||
<Set name="port"><SystemProperty name="jetty.port" default="8080"/></Set>
|
||||
<Set name="maxIdleTime">300000</Set>
|
||||
<Set name="Acceptors">2</Set>
|
||||
<Set name="statsOn">false</Set>
|
||||
<Set name="confidentialPort">8443</Set>
|
||||
<Set name="lowResourcesConnections">20000</Set>
|
||||
<Set name="lowResourcesMaxIdleTime">5000</Set>
|
||||
</New>
|
||||
</Arg>
|
||||
</Call>
|
||||
|
||||
<!-- Uncomment this and enter your SSL config/credentials to enable https
|
||||
<Call name="addConnector">
|
||||
<Arg>
|
||||
<New class="org.eclipse.jetty.server.ssl.SslSelectChannelConnector">
|
||||
<Set name="Port">8443</Set>
|
||||
<Set name="maxIdleTime">30000</Set>
|
||||
<Set name="Acceptors">2</Set>
|
||||
<Set name="AcceptQueueSize">100</Set>
|
||||
<Set name="Keystore"><SystemProperty name="jetty.home" default="." />/etc/keystore</Set>
|
||||
<Set name="Password">PASSWORD</Set>
|
||||
<Set name="KeyPassword">KEYPASSWORD</Set>
|
||||
<Set name="truststore"><SystemProperty name="jetty.home" default="." />/etc/keystore</Set>
|
||||
<Set name="trustPassword">TRUSTPASSWORD</Set>
|
||||
</New>
|
||||
</Arg>
|
||||
</Call>
|
||||
-->
|
||||
|
||||
<!-- =========================================================== -->
|
||||
<!-- Set handler Collection Structure -->
|
||||
<!-- =========================================================== -->
|
||||
<Set name="handler">
|
||||
<New id="Handlers" class="org.eclipse.jetty.server.handler.HandlerCollection">
|
||||
<Set name="handlers">
|
||||
<Array type="org.eclipse.jetty.server.Handler">
|
||||
<!--Item>
|
||||
<New id="AkkaRestHandler" class="org.eclipse.jetty.servlet.ServletContextHandler">
|
||||
<Set name="contextPath">/</Set>
|
||||
<Call name="addServlet">
|
||||
<Arg>akka.http.AkkaRestServlet</Arg>
|
||||
<Arg>/*</Arg>
|
||||
</Call>
|
||||
</New>
|
||||
</Item-->
|
||||
<Item>
|
||||
<New id="AkkaMistHandler" class="org.eclipse.jetty.servlet.ServletContextHandler">
|
||||
<Set name="contextPath">/</Set>
|
||||
<Call name="addServlet">
|
||||
<Arg>akka.http.AkkaMistServlet</Arg>
|
||||
<Arg>/*</Arg>
|
||||
</Call>
|
||||
</New>
|
||||
</Item>
|
||||
<Item>
|
||||
<New id="DefaultHandler" class="org.eclipse.jetty.server.handler.DefaultHandler"/>
|
||||
</Item>
|
||||
</Array>
|
||||
</Set>
|
||||
</New>
|
||||
</Set>
|
||||
|
||||
<!-- =========================================================== -->
|
||||
<!-- extra options -->
|
||||
<!-- =========================================================== -->
|
||||
<Set name="stopAtShutdown">true</Set>
|
||||
<Set name="sendServerVersion">true</Set>
|
||||
<Set name="sendDateHeader">true</Set>
|
||||
<Set name="gracefulShutdown">1000</Set>
|
||||
|
||||
</Configure>
|
||||
|
|
@ -30,7 +30,7 @@ object AkkaBuild extends Build {
|
|||
Unidoc.unidocExclude := Seq(samples.id, tutorials.id),
|
||||
Dist.distExclude := Seq(actorTests.id, akkaSbtPlugin.id, docs.id)
|
||||
),
|
||||
aggregate = Seq(actor, testkit, actorTests, remote, slf4j, amqp, mailboxes, akkaSbtPlugin, samples, tutorials, docs)
|
||||
aggregate = Seq(actor, testkit, actorTests, remote, slf4j, amqp, mailboxes, kernel, akkaSbtPlugin, samples, tutorials, docs)
|
||||
)
|
||||
|
||||
lazy val actor = Project(
|
||||
|
|
@ -183,14 +183,14 @@ object AkkaBuild extends Build {
|
|||
// )
|
||||
// )
|
||||
|
||||
// lazy val kernel = Project(
|
||||
// id = "akka-kernel",
|
||||
// base = file("akka-kernel"),
|
||||
// dependencies = Seq(cluster, slf4j, spring),
|
||||
// settings = defaultSettings ++ Seq(
|
||||
// libraryDependencies ++= Dependencies.kernel
|
||||
// )
|
||||
// )
|
||||
lazy val kernel = Project(
|
||||
id = "akka-kernel",
|
||||
base = file("akka-kernel"),
|
||||
dependencies = Seq(actor, testkit % "test->test"),
|
||||
settings = defaultSettings ++ Seq(
|
||||
libraryDependencies ++= Dependencies.kernel
|
||||
)
|
||||
)
|
||||
|
||||
lazy val akkaSbtPlugin = Project(
|
||||
id = "akka-sbt-plugin",
|
||||
|
|
@ -204,7 +204,7 @@ object AkkaBuild extends Build {
|
|||
id = "akka-samples",
|
||||
base = file("akka-samples"),
|
||||
settings = parentSettings,
|
||||
aggregate = Seq(fsmSample, helloSample)
|
||||
aggregate = Seq(fsmSample, helloSample, helloKernelSample)
|
||||
)
|
||||
|
||||
lazy val fsmSample = Project(
|
||||
|
|
@ -221,6 +221,13 @@ object AkkaBuild extends Build {
|
|||
settings = defaultSettings
|
||||
)
|
||||
|
||||
lazy val helloKernelSample = Project(
|
||||
id = "akka-sample-hello-kernel",
|
||||
base = file("akka-samples/akka-sample-hello-kernel"),
|
||||
dependencies = Seq(kernel),
|
||||
settings = defaultSettings
|
||||
)
|
||||
|
||||
lazy val tutorials = Project(
|
||||
id = "akka-tutorials",
|
||||
base = file("akka-tutorials"),
|
||||
|
|
@ -377,9 +384,7 @@ object Dependencies {
|
|||
|
||||
val spring = Seq(springBeans, springContext, Test.junit, Test.scalatest)
|
||||
|
||||
val kernel = Seq(
|
||||
jettyUtil, jettyXml, jettyServlet, jacksonCore, staxApi
|
||||
)
|
||||
val kernel = Seq(Test.scalatest, Test.junit)
|
||||
|
||||
// TODO: resolve Jetty version conflict
|
||||
// val sampleCamel = Seq(camelCore, camelSpring, commonsCodec, Runtime.camelJms, Runtime.activemq, Runtime.springJms,
|
||||
|
|
|
|||
|
|
@ -52,10 +52,12 @@ object Dist {
|
|||
(baseDirectory, distSources, distUnzipped, version, distFile, streams) map {
|
||||
(projectBase, allSources, unzipped, version, zipFile, s) => {
|
||||
val base = unzipped / ("akka-" + version)
|
||||
val scripts = (projectBase / "scripts" / "microkernel" * "*").get
|
||||
val scripts = (projectBase / "akka-kernel" / "src" / "main" / "scripts" * "*").get
|
||||
val bin = base / "bin"
|
||||
val configSources = projectBase / "config"
|
||||
val config = base / "config"
|
||||
val deploy = base / "deploy"
|
||||
val deployReadme = deploy / "readme"
|
||||
val doc = base / "doc" / "akka"
|
||||
val api = doc / "api"
|
||||
val docs = doc / "docs"
|
||||
|
|
@ -66,9 +68,10 @@ object Dist {
|
|||
val libAkka = lib / "akka"
|
||||
val src = base / "src" / "akka"
|
||||
IO.delete(unzipped)
|
||||
// TODO: re-enable bin and config dirs, and add deploy dir, when akka-kernel is enabled
|
||||
//copyFilesTo(scripts, bin, setExecutable = true)
|
||||
//IO.copyDirectory(configSources, config)
|
||||
copyFilesTo(scripts, bin, setExecutable = true)
|
||||
IO.copyDirectory(configSources, config)
|
||||
IO.createDirectory(deploy)
|
||||
IO.write(deployReadme, "Place application jars in this directory")
|
||||
IO.copyDirectory(allSources.api, api)
|
||||
IO.copyDirectory(allSources.docs, docs)
|
||||
copyFilesTo(allSources.docJars, docJars)
|
||||
|
|
|
|||
|
|
@ -1,9 +0,0 @@
|
|||
#!/bin/bash
|
||||
|
||||
AKKA_HOME="$(cd "$(cd "$(dirname "$0")"; pwd -P)"/..; pwd)"
|
||||
|
||||
[ -n "$JAVA_OPTS" ] || JAVA_OPTS="-Xms1536M -Xmx1536M -Xss1M -XX:MaxPermSize=256M -XX:+UseParallelGC"
|
||||
|
||||
[ -n "$AKKA_CLASSPATH" ] || AKKA_CLASSPATH="$AKKA_HOME/lib/scala-library.jar:$AKKA_HOME/lib/akka/*:$AKKA_HOME/config"
|
||||
|
||||
java $JAVA_OPTS -cp "$AKKA_CLASSPATH" -Dakka.home="$AKKA_HOME" akka.kernel.Main
|
||||
Loading…
Add table
Add a link
Reference in a new issue