Merge remote-tracking branch 'origin/master' into wip-1581-patterns-ask

This commit is contained in:
Roland 2012-01-18 14:20:13 +01:00
commit 1daaee98aa
141 changed files with 4660 additions and 5579 deletions

View file

@ -7,14 +7,11 @@ package akka.docs.actor
import akka.actor.Actor
import akka.actor.Props
import akka.event.Logging
import akka.dispatch.Future
//#imports1
//#imports2
import akka.dispatch.Future
import akka.actor.ActorSystem
//#imports2
import org.scalatest.{ BeforeAndAfterAll, WordSpec }
import org.scalatest.matchers.MustMatchers
import akka.testkit._
@ -114,7 +111,6 @@ object SwapperApp extends App {
//#swapper
//#receive-orElse
import akka.actor.Actor.Receive
abstract class GenericActor extends Actor {
// to be defined in subclassing actor
@ -169,10 +165,10 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
system.eventStream.subscribe(testActor, classOf[Logging.Info])
myActor ! "test"
expectMsgPF(1 second) { case Logging.Info(_, "received test") true }
expectMsgPF(1 second) { case Logging.Info(_, _, "received test") true }
myActor ! "unknown"
expectMsgPF(1 second) { case Logging.Info(_, "received unknown message") true }
expectMsgPF(1 second) { case Logging.Info(_, _, "received unknown message") true }
system.eventStream.unsubscribe(testActor)
system.eventStream.publish(TestEvent.UnMute(filter))
@ -201,11 +197,9 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
val props3 = Props(new MyActor)
val props4 = Props(
creator = { () new MyActor },
dispatcher = "my-dispatcher",
timeout = Timeout(100))
dispatcher = "my-dispatcher")
val props5 = props1.withCreator(new MyActor)
val props6 = props5.withDispatcher("my-dispatcher")
val props7 = props6.withTimeout(Timeout(100))
//#creating-props-config
}
@ -321,4 +315,22 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
a ! "kill"
expectMsg("finished")
}
"using pattern gracefulStop" in {
val actorRef = system.actorOf(Props[MyActor])
//#gracefulStop
import akka.pattern.gracefulStop
import akka.dispatch.Await
import akka.actor.ActorTimeoutException
try {
val stopped: Future[Boolean] = gracefulStop(actorRef, 5 seconds)(system)
Await.result(stopped, 6 seconds)
// the actor has been stopped
} catch {
case e: ActorTimeoutException // the actor wasn't stopped within 5 seconds
}
//#gracefulStop
}
}

View file

@ -6,7 +6,7 @@ package akka.docs.actor
//#imports
import akka.dispatch.{ Promise, Future, Await }
import akka.util.duration._
import akka.actor.{ ActorContext, TypedActor, Props }
import akka.actor.{ ActorContext, TypedActor, TypedProps }
//#imports
@ -100,14 +100,11 @@ class TypedActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
"create a typed actor" in {
//#typed-actor-create1
val mySquarer: Squarer =
TypedActor(system).typedActorOf[Squarer, SquarerImpl]()
TypedActor(system).typedActorOf(TypedProps[SquarerImpl]())
//#typed-actor-create1
//#typed-actor-create2
val otherSquarer: Squarer =
TypedActor(system).typedActorOf(classOf[Squarer],
new SquarerImpl("foo"),
Props(),
"name")
TypedActor(system).typedActorOf(TypedProps(classOf[Squarer], new SquarerImpl("foo")), "name")
//#typed-actor-create2
//#typed-actor-calls
@ -145,7 +142,7 @@ class TypedActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
"supercharge" in {
//#typed-actor-supercharge-usage
val awesomeFooBar = TypedActor(system).typedActorOf[Foo with Bar, FooBar]()
val awesomeFooBar: Foo with Bar = TypedActor(system).typedActorOf(TypedProps[FooBar]())
awesomeFooBar.doFoo(10)
val f = awesomeFooBar.doBar("yes")

View file

@ -41,10 +41,8 @@ object DispatcherDocSpec {
type = Dispatcher
core-pool-size-factor = 8.0
max-pool-size-factor = 16.0
# Specifies the bounded capacity of the task queue
task-queue-size = 100
# Specifies which type of task queue will be used, can be "array" or "linked" (default)
task-queue-type = "array"
# Specifies the bounded capacity of the mailbox queue
mailbox-capacity = 100
throughput = 3
}
//#my-bounded-config

View file

@ -38,15 +38,33 @@ object LoggingDocSpec {
class MyEventListener extends Actor {
def receive = {
case InitializeLogger(_) sender ! LoggerInitialized
case Error(cause, logSource, message) // ...
case Warning(logSource, message) // ...
case Info(logSource, message) // ...
case Debug(logSource, message) // ...
case InitializeLogger(_) sender ! LoggerInitialized
case Error(cause, logSource, logClass, message) // ...
case Warning(logSource, logClass, message) // ...
case Info(logSource, logClass, message) // ...
case Debug(logSource, logClass, message) // ...
}
}
//#my-event-listener
//#my-source
import akka.event.LogSource
import akka.actor.ActorSystem
object MyType {
implicit val logSource: LogSource[AnyRef] = new LogSource[AnyRef] {
def genString(o: AnyRef): String = o.getClass.getName
override def getClazz(o: AnyRef): Class[_] = o.getClass
}
}
class MyType(system: ActorSystem) {
import MyType._
import akka.event.Logging
val log = Logging(system, this)
}
//#my-source
}
class LoggingDocSpec extends AkkaSpec {

View file

@ -1,24 +0,0 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.docs.routing
import akka.routing.{ BasicNoBackoffFilter, SmallestMailboxSelector, DefaultActorPool }
import akka.actor.{ ActorRef, Props, Actor }
//#testPool
class TestPool extends Actor with DefaultActorPool with SmallestMailboxSelector with BasicNoBackoffFilter {
def capacity(delegates: Seq[ActorRef]) = 5
protected def receive = _route
def rampupRate = 0.1
def selectionCount = 1
def partialFill = true
def instance(defaults: Props) = context.actorOf(defaults.withCreator(new Actor {
def receive = {
case _ // do something
}
}))
}
//#testPool

View file

@ -1,26 +0,0 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.docs.routing
import akka.actor.ActorRef
//#boundedCapacitor
trait BoundedCapacitor {
def lowerBound: Int
def upperBound: Int
def capacity(delegates: Seq[ActorRef]): Int = {
val current = delegates length
var delta = _eval(delegates)
val proposed = current + delta
if (proposed < lowerBound) delta += (lowerBound - proposed)
else if (proposed > upperBound) delta -= (proposed - upperBound)
delta
}
protected def _eval(delegates: Seq[ActorRef]): Int
}
//#boundedCapacitor

View file

@ -1,19 +0,0 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.docs.routing
import akka.routing.ActorPool
import akka.actor.ActorRef
//#capacityStrategy
trait CapacityStrategy {
import ActorPool._
def pressure(delegates: Seq[ActorRef]): Int
def filter(pressure: Int, capacity: Int): Int
protected def _eval(delegates: Seq[ActorRef]): Int =
filter(pressure(delegates), delegates.size)
}
//#capacityStrategy

View file

@ -9,6 +9,7 @@ import akka.actor.{ Props, Actor }
import akka.util.duration._
import akka.dispatch.Await
import akka.pattern.ask
import akka.routing.SmallestMailboxRouter
case class FibonacciNumber(nbr: Int)
@ -47,7 +48,7 @@ class ParentActor extends Actor {
case "rrr"
//#roundRobinRouter
val roundRobinRouter =
context.actorOf(Props[PrintlnActor].withRouter(RoundRobinRouter()), "router")
context.actorOf(Props[PrintlnActor].withRouter(RoundRobinRouter(5)), "router")
1 to 10 foreach {
i roundRobinRouter ! i
}
@ -55,22 +56,30 @@ class ParentActor extends Actor {
case "rr"
//#randomRouter
val randomRouter =
context.actorOf(Props[PrintlnActor].withRouter(RandomRouter()), "router")
context.actorOf(Props[PrintlnActor].withRouter(RandomRouter(5)), "router")
1 to 10 foreach {
i randomRouter ! i
}
//#randomRouter
case "smr"
//#smallestMailboxRouter
val smallestMailboxRouter =
context.actorOf(Props[PrintlnActor].withRouter(SmallestMailboxRouter(5)), "router")
1 to 10 foreach {
i smallestMailboxRouter ! i
}
//#smallestMailboxRouter
case "br"
//#broadcastRouter
val broadcastRouter =
context.actorOf(Props[PrintlnActor].withRouter(BroadcastRouter()), "router")
context.actorOf(Props[PrintlnActor].withRouter(BroadcastRouter(5)), "router")
broadcastRouter ! "this is a broadcast message"
//#broadcastRouter
case "sgfcr"
//#scatterGatherFirstCompletedRouter
val scatterGatherFirstCompletedRouter = context.actorOf(
Props[FibonacciActor].withRouter(ScatterGatherFirstCompletedRouter(within = 2 seconds)),
"router")
Props[FibonacciActor].withRouter(ScatterGatherFirstCompletedRouter(
nrOfInstances = 5, within = 2 seconds)), "router")
implicit val timeout = context.system.settings.ActorTimeout
val futureResult = scatterGatherFirstCompletedRouter ? FibonacciNumber(10)
val result = Await.result(futureResult, timeout.duration)

View file

@ -4,7 +4,8 @@
package akka.docs.routing
import akka.actor.{ Actor, Props, ActorSystem }
import akka.routing.RoundRobinRouter
import com.typesafe.config.ConfigFactory
import akka.routing.FromConfig
case class Message(nbr: Int)
@ -15,10 +16,37 @@ class ExampleActor extends Actor {
}
object RouterWithConfigExample extends App {
val system = ActorSystem("Example")
val config = ConfigFactory.parseString("""
//#config
akka.actor.deployment {
/router {
router = round-robin
nr-of-instances = 5
}
}
//#config
//#config-resize
akka.actor.deployment {
/router2 {
router = round-robin
resizer {
lower-bound = 2
upper-bound = 15
}
}
}
//#config-resize
""")
val system = ActorSystem("Example", config)
//#configurableRouting
val router = system.actorOf(Props[PrintlnActor].withRouter(RoundRobinRouter()),
"exampleActor")
val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()),
"router")
//#configurableRouting
1 to 10 foreach { i router ! Message(i) }
//#configurableRoutingWithResizer
val router2 = system.actorOf(Props[ExampleActor].withRouter(FromConfig()),
"router2")
//#configurableRoutingWithResizer
1 to 10 foreach { i router2 ! Message(i) }
}

View file

@ -5,6 +5,7 @@ package akka.docs.routing
import akka.routing.RoundRobinRouter
import akka.actor.{ ActorRef, Props, Actor, ActorSystem }
import akka.routing.DefaultResizer
case class Message1(nbr: Int)
@ -31,4 +32,12 @@ object RoutingProgrammaticallyExample extends App {
RoundRobinRouter(routees = routees)))
//#programmaticRoutingRoutees
1 to 6 foreach { i router2 ! Message1(i) }
//#programmaticRoutingWithResizer
val resizer = DefaultResizer(lowerBound = 2, upperBound = 15)
val router3 = system.actorOf(Props[ExampleActor1].withRouter(
RoundRobinRouter(resizer = Some(resizer))))
//#programmaticRoutingWithResizer
1 to 6 foreach { i router3 ! Message1(i) }
}