move everything into package akka.pattern

This commit is contained in:
Roland 2012-01-18 10:18:51 +01:00
parent 9c762dec20
commit 2bed2cb954
59 changed files with 73 additions and 73 deletions

View file

@ -8,7 +8,7 @@ import akka.testkit._
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
import akka.util.duration._ import akka.util.duration._
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
object ActorFireForgetRequestReplySpec { object ActorFireForgetRequestReplySpec {

View file

@ -12,7 +12,7 @@ import akka.testkit._
import akka.util.duration._ import akka.util.duration._
import java.util.concurrent.atomic._ import java.util.concurrent.atomic._
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
object ActorLifeCycleSpec { object ActorLifeCycleSpec {

View file

@ -6,7 +6,7 @@ package akka.actor
import akka.testkit._ import akka.testkit._
import akka.util.duration._ import akka.util.duration._
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
object ActorLookupSpec { object ActorLookupSpec {

View file

@ -15,7 +15,7 @@ import akka.util.ReflectiveAccess
import akka.serialization.Serialization import akka.serialization.Serialization
import java.util.concurrent.{ CountDownLatch, TimeUnit } import java.util.concurrent.{ CountDownLatch, TimeUnit }
import akka.dispatch.{ Await, DefaultPromise, Promise, Future } import akka.dispatch.{ Await, DefaultPromise, Promise, Future }
import akka.patterns.ask import akka.pattern.ask
object ActorRefSpec { object ActorRefSpec {

View file

@ -10,7 +10,7 @@ import akka.testkit.DefaultTimeout
import java.util.concurrent.TimeoutException import java.util.concurrent.TimeoutException
import akka.dispatch.Await import akka.dispatch.Await
import akka.util.Timeout import akka.util.Timeout
import akka.patterns.ask import akka.pattern.ask
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll with DefaultTimeout { class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll with DefaultTimeout {

View file

@ -8,7 +8,7 @@ import akka.testkit._
import akka.util.duration._ import akka.util.duration._
import java.util.concurrent.atomic._ import java.util.concurrent.atomic._
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class LocalDeathWatchSpec extends AkkaSpec with ImplicitSender with DefaultTimeout with DeathWatchSpec class LocalDeathWatchSpec extends AkkaSpec with ImplicitSender with DefaultTimeout with DeathWatchSpec

View file

@ -9,7 +9,7 @@ import akka.util.duration._
import Actor._ import Actor._
import akka.util.Duration import akka.util.Duration
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
object ForwardActorSpec { object ForwardActorSpec {
val ExpectedMessage = "FOO" val ExpectedMessage = "FOO"

View file

@ -11,7 +11,7 @@ import akka.util.cps._
import scala.util.continuations._ import scala.util.continuations._
import akka.testkit._ import akka.testkit._
import akka.dispatch.{ Await, Future } import akka.dispatch.{ Await, Future }
import akka.patterns.ask import akka.pattern.ask
object IOActorSpec { object IOActorSpec {
import IO._ import IO._

View file

@ -14,7 +14,7 @@ import akka.testkit.AkkaSpec
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.testkit.TestLatch import akka.testkit.TestLatch
import akka.util.duration._ import akka.util.duration._
import akka.patterns.ask import akka.pattern.ask
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class RestartStrategySpec extends AkkaSpec with DefaultTimeout { class RestartStrategySpec extends AkkaSpec with DefaultTimeout {

View file

@ -8,7 +8,7 @@ import java.util.concurrent.{ CountDownLatch, ConcurrentLinkedQueue, TimeUnit }
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.testkit.TestLatch import akka.testkit.TestLatch
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout { class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout {

View file

@ -8,7 +8,7 @@ import akka.testkit._
import java.util.concurrent.{ TimeUnit, CountDownLatch } import java.util.concurrent.{ TimeUnit, CountDownLatch }
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
object SupervisorHierarchySpec { object SupervisorHierarchySpec {
class FireWorkerException(msg: String) extends Exception(msg) class FireWorkerException(msg: String) extends Exception(msg)

View file

@ -8,7 +8,7 @@ import akka.dispatch.{ PinnedDispatcher, Dispatchers, Await }
import java.util.concurrent.{ TimeUnit, CountDownLatch } import java.util.concurrent.{ TimeUnit, CountDownLatch }
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.patterns.ask import akka.pattern.ask
object SupervisorMiscSpec { object SupervisorMiscSpec {
val config = """ val config = """

View file

@ -11,7 +11,7 @@ import akka.testkit.TestEvent._
import akka.testkit._ import akka.testkit._
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
object SupervisorSpec { object SupervisorSpec {
val Timeout = 5 seconds val Timeout = 5 seconds

View file

@ -12,7 +12,7 @@ import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender import akka.testkit.ImplicitSender
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.dispatch.{ Await, Dispatchers } import akka.dispatch.{ Await, Dispatchers }
import akka.patterns.ask import akka.pattern.ask
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class SupervisorTreeSpec extends AkkaSpec with ImplicitSender with DefaultTimeout { class SupervisorTreeSpec extends AkkaSpec with ImplicitSender with DefaultTimeout {

View file

@ -11,7 +11,7 @@ import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender import akka.testkit.ImplicitSender
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender with DefaultTimeout { class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender with DefaultTimeout {

View file

@ -18,7 +18,7 @@ import java.util.concurrent.{ TimeUnit, CountDownLatch }
import akka.japi.{ Creator, Option JOption } import akka.japi.{ Creator, Option JOption }
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.dispatch.{ Await, Dispatchers, Future, Promise } import akka.dispatch.{ Await, Dispatchers, Future, Promise }
import akka.patterns.ask import akka.pattern.ask
object TypedActorSpec { object TypedActorSpec {

View file

@ -20,7 +20,7 @@ import akka.util.duration._
import akka.event.Logging.Error import akka.event.Logging.Error
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.util.Duration import akka.util.Duration
import akka.patterns.ask import akka.pattern.ask
object ActorModelSpec { object ActorModelSpec {

View file

@ -8,7 +8,7 @@ import akka.util.Duration
import akka.util.duration._ import akka.util.duration._
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.dispatch.{ Await, PinnedDispatcher, Dispatchers, Dispatcher } import akka.dispatch.{ Await, PinnedDispatcher, Dispatchers, Dispatcher }
import akka.patterns.ask import akka.pattern.ask
object DispatcherActorSpec { object DispatcherActorSpec {
val config = """ val config = """

View file

@ -7,7 +7,7 @@ import akka.actor.{ Props, Actor }
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
import akka.dispatch.{ Await, PinnedDispatcher, Dispatchers } import akka.dispatch.{ Await, PinnedDispatcher, Dispatchers }
import akka.patterns.ask import akka.pattern.ask
object PinnedActorSpec { object PinnedActorSpec {
val config = """ val config = """

View file

@ -9,7 +9,7 @@ import akka.actor.future2actor
import akka.util.duration._ import akka.util.duration._
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.patterns.ask import akka.pattern.ask
class Future2ActorSpec extends AkkaSpec with DefaultTimeout { class Future2ActorSpec extends AkkaSpec with DefaultTimeout {

View file

@ -16,7 +16,7 @@ import akka.testkit.DefaultTimeout
import akka.testkit.TestLatch import akka.testkit.TestLatch
import java.util.concurrent.{ TimeoutException, TimeUnit, CountDownLatch } import java.util.concurrent.{ TimeoutException, TimeUnit, CountDownLatch }
import scala.runtime.NonLocalReturnControl import scala.runtime.NonLocalReturnControl
import akka.patterns.ask import akka.pattern.ask
object FutureSpec { object FutureSpec {
class TestActor extends Actor { class TestActor extends Actor {

View file

@ -2,7 +2,7 @@ package akka.dispatch
import akka.actor.{ Props, LocalActorRef, Actor } import akka.actor.{ Props, LocalActorRef, Actor }
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.patterns.ask import akka.pattern.ask
import akka.util.duration._ import akka.util.duration._
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import com.typesafe.config.Config import com.typesafe.config.Config

View file

@ -6,7 +6,7 @@ import akka.util.duration._
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger } import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger }
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.dispatch.{ Await, Promise, Future } import akka.dispatch.{ Await, Promise, Future }
import akka.patterns.ask import akka.pattern.ask
object ActorPoolSpec { object ActorPoolSpec {

View file

@ -6,7 +6,7 @@ import java.util.concurrent.atomic.AtomicInteger
import akka.testkit._ import akka.testkit._
import akka.util.duration._ import akka.util.duration._
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ConfiguredLocalRoutingSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { class ConfiguredLocalRoutingSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {

View file

@ -12,7 +12,7 @@ import akka.dispatch.Await
import akka.util.Duration import akka.util.Duration
import akka.config.ConfigurationException import akka.config.ConfigurationException
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.patterns.ask import akka.pattern.ask
object RoutingSpec { object RoutingSpec {

View file

@ -13,7 +13,7 @@ import akka.util.Timeout
import akka.util.duration._ import akka.util.duration._
import scala.reflect.BeanInfo import scala.reflect.BeanInfo
import com.google.protobuf.Message import com.google.protobuf.Message
import akka.patterns.ask import akka.pattern.ask
class ProtobufSerializer extends Serializer { class ProtobufSerializer extends Serializer {
val ARRAY_OF_BYTE_ARRAY = Array[Class[_]](classOf[Array[Byte]]) val ARRAY_OF_BYTE_ARRAY = Array[Class[_]](classOf[Array[Byte]])

View file

@ -5,7 +5,7 @@ import akka.routing._
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.dispatch.Await import akka.dispatch.Await
import akka.util.duration._ import akka.util.duration._
import akka.patterns.ask import akka.pattern.ask
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class Ticket703Spec extends AkkaSpec { class Ticket703Spec extends AkkaSpec {

View file

@ -168,7 +168,7 @@ trait LocalRef extends ActorRefScope {
} }
/** /**
* Trait for matching on ActorRefs which have access to a provider; this is used in akka.patterns.ask. * Trait for matching on ActorRefs which have access to a provider; this is used in akka.pattern.ask.
*/ */
trait ActorRefWithProvider { this: InternalActorRef trait ActorRefWithProvider { this: InternalActorRef
def provider: ActorRefProvider def provider: ActorRefProvider
@ -472,7 +472,7 @@ class AskTimeoutException(message: String, cause: Throwable) extends TimeoutExce
} }
/** /**
* Akka private optimized representation of the temporary actor spawned to * Akka private optimized representation of the temporary actor spawned to
* receive the reply to an "ask" operation. * receive the reply to an "ask" operation.
*/ */
private[akka] final class PromiseActorRef( private[akka] final class PromiseActorRef(

View file

@ -7,7 +7,7 @@ import akka.config.ConfigurationException
import akka.actor._ import akka.actor._
import akka.event._ import akka.event._
import akka.dispatch._ import akka.dispatch._
import akka.patterns.ask import akka.pattern.ask
import akka.util.duration._ import akka.util.duration._
import akka.util.Timeout import akka.util.Timeout
import akka.util.Timeout._ import akka.util.Timeout._

View file

@ -396,7 +396,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
case "equals" (args.length == 1 && (proxy eq args(0)) || actor == extension.getActorRefFor(args(0))).asInstanceOf[AnyRef] //Force boxing of the boolean case "equals" (args.length == 1 && (proxy eq args(0)) || actor == extension.getActorRefFor(args(0))).asInstanceOf[AnyRef] //Force boxing of the boolean
case "hashCode" actor.hashCode.asInstanceOf[AnyRef] case "hashCode" actor.hashCode.asInstanceOf[AnyRef]
case _ case _
import akka.patterns.ask import akka.pattern.ask
MethodCall(method, args) match { MethodCall(method, args) match {
case m if m.isOneWay actor ! m; null //Null return value case m if m.isOneWay actor ! m; null //Null return value
case m if m.returnsFuture_? actor.?(m, timeout) case m if m.returnsFuture_? actor.?(m, timeout)

View file

@ -158,7 +158,7 @@ trait LoggingBus extends ActorEventBus {
val name = "log" + Extension(system).id() + "-" + simpleName(clazz) val name = "log" + Extension(system).id() + "-" + simpleName(clazz)
val actor = system.systemActorOf(Props(clazz), name) val actor = system.systemActorOf(Props(clazz), name)
implicit val timeout = Timeout(3 seconds) implicit val timeout = Timeout(3 seconds)
import akka.patterns.ask import akka.pattern.ask
val response = try Await.result(actor ? InitializeLogger(this), timeout.duration) catch { val response = try Await.result(actor ? InitializeLogger(this), timeout.duration) catch {
case _: TimeoutException case _: TimeoutException
publish(Warning(simpleName(this), "Logger " + name + " did not respond within " + timeout + " to InitializeLogger(bus)")) publish(Warning(simpleName(this), "Logger " + name + " did not respond within " + timeout + " to InitializeLogger(bus)"))

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.patterns package akka.pattern
import akka.actor.{ ActorRef, InternalActorRef, AskTimeoutException } import akka.actor.{ ActorRef, InternalActorRef, AskTimeoutException }
import akka.dispatch.{ Future, Promise } import akka.dispatch.{ Future, Promise }
@ -37,7 +37,7 @@ final class AskableActorRef(val actorRef: ActorRef) {
* *
* [see the [[akka.dispatch.Future]] companion object for a description of `flow`] * [see the [[akka.dispatch.Future]] companion object for a description of `flow`]
*/ */
def ask(message: AnyRef)(implicit timeout: Timeout): Future[Any] = akka.patterns.ask(actorRef, message)(timeout) def ask(message: AnyRef)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message)(timeout)
/** /**
* Sends a message asynchronously and returns a [[akka.dispatch.Future]] * Sends a message asynchronously and returns a [[akka.dispatch.Future]]
@ -67,7 +67,7 @@ final class AskableActorRef(val actorRef: ActorRef) {
* *
* [see the [[akka.dispatch.Future]] companion object for a description of `flow`] * [see the [[akka.dispatch.Future]] companion object for a description of `flow`]
*/ */
def ?(message: Any)(implicit timeout: Timeout): Future[Any] = akka.patterns.ask(actorRef, message) def ?(message: Any)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message)
/* /*
* FIXME: I think this should be removed, since it introduces an ambiguity * FIXME: I think this should be removed, since it introduces an ambiguity

View file

@ -1,12 +1,12 @@
/** /**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka package akka.pattern
object Patterns { object Patterns {
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.dispatch.Future import akka.dispatch.Future
import akka.patterns.{ ask scalaAsk } import akka.pattern.{ ask scalaAsk }
import akka.util.Timeout import akka.util.Timeout
/** /**
@ -68,4 +68,4 @@ object Patterns {
* }}} * }}}
*/ */
def ask(actor: ActorRef, message: Any, timeoutMillis: Long): Future[AnyRef] = scalaAsk(actor, message)(new Timeout(timeoutMillis)).asInstanceOf[Future[AnyRef]] def ask(actor: ActorRef, message: Any, timeoutMillis: Long): Future[AnyRef] = scalaAsk(actor, message)(new Timeout(timeoutMillis)).asInstanceOf[Future[AnyRef]]
} }

View file

@ -3,7 +3,7 @@
*/ */
package akka package akka
package object patterns { package object pattern {
import akka.actor.{ ActorRef, InternalActorRef, ActorRefWithProvider, AskTimeoutException } import akka.actor.{ ActorRef, InternalActorRef, ActorRefWithProvider, AskTimeoutException }
import akka.dispatch.{ Future, Promise } import akka.dispatch.{ Future, Promise }
@ -15,7 +15,7 @@ package object patterns {
* `ask(actorRef, message)(timeout)` method defined here. * `ask(actorRef, message)(timeout)` method defined here.
* *
* {{{ * {{{
* import akka.patterns.ask * import akka.pattern.ask
* *
* val future = actor ? message // => ask(actor, message) * val future = actor ? message // => ask(actor, message)
* val future = actor ask message // => ask(actor, message) * val future = actor ask message // => ask(actor, message)

View file

@ -7,7 +7,7 @@ package akka.agent
import akka.actor._ import akka.actor._
import akka.japi.{ Function JFunc, Procedure JProc } import akka.japi.{ Function JFunc, Procedure JProc }
import akka.dispatch._ import akka.dispatch._
import akka.patterns.ask import akka.pattern.ask
import akka.util.Timeout import akka.util.Timeout
import scala.concurrent.stm._ import scala.concurrent.stm._

View file

@ -13,7 +13,7 @@ import akka.actor.Props;
import akka.actor.Terminated; import akka.actor.Terminated;
import akka.actor.UntypedActor; import akka.actor.UntypedActor;
import akka.dispatch.Await; import akka.dispatch.Await;
import static akka.Patterns.ask; import static akka.pattern.Patterns.ask;
import akka.util.Duration; import akka.util.Duration;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
import akka.testkit.TestProbe; import akka.testkit.TestProbe;

View file

@ -38,7 +38,7 @@ import org.junit.Test;
import scala.Option; import scala.Option;
import java.lang.Object; import java.lang.Object;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import akka.Patterns; import akka.pattern.Patterns;
import static org.junit.Assert.*; import static org.junit.Assert.*;

View file

@ -55,7 +55,7 @@ import akka.actor.UntypedActor;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.Props; import akka.actor.Props;
import akka.dispatch.Futures; import akka.dispatch.Futures;
import akka.Patterns; import akka.pattern.Patterns;
import static org.junit.Assert.*; import static org.junit.Assert.*;

View file

@ -10,7 +10,7 @@ import org.junit.Test;
//#imports //#imports
import akka.actor.*; import akka.actor.*;
import akka.dispatch.Await; import akka.dispatch.Await;
import static akka.Patterns.ask; import static akka.pattern.Patterns.ask;
import akka.transactor.Coordinated; import akka.transactor.Coordinated;
import akka.util.Duration; import akka.util.Duration;
import akka.util.Timeout; import akka.util.Timeout;

View file

@ -220,7 +220,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
"using ask" in { "using ask" in {
//#using-ask //#using-ask
import akka.patterns.ask import akka.pattern.ask
class MyActor extends Actor { class MyActor extends Actor {
def receive = { def receive = {
@ -245,7 +245,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
//#using-implicit-timeout //#using-implicit-timeout
import akka.util.duration._ import akka.util.duration._
import akka.util.Timeout import akka.util.Timeout
import akka.patterns.ask import akka.pattern.ask
implicit val timeout = Timeout(500 millis) implicit val timeout = Timeout(500 millis)
val future = myActor ? "hello" val future = myActor ? "hello"
//#using-implicit-timeout //#using-implicit-timeout
@ -257,7 +257,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
val myActor = system.actorOf(Props(new FirstActor)) val myActor = system.actorOf(Props(new FirstActor))
//#using-explicit-timeout //#using-explicit-timeout
import akka.util.duration._ import akka.util.duration._
import akka.patterns.ask import akka.pattern.ask
val future = myActor ? ("hello", timeout = 500 millis) val future = myActor ? ("hello", timeout = 500 millis)
//#using-explicit-timeout //#using-explicit-timeout
Await.result(future, 500 millis) must be("hello") Await.result(future, 500 millis) must be("hello")

View file

@ -44,7 +44,7 @@ class FutureDocSpec extends AkkaSpec {
val msg = "hello" val msg = "hello"
//#ask-blocking //#ask-blocking
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
implicit val timeout = system.settings.ActorTimeout implicit val timeout = system.settings.ActorTimeout
val future = actor ? msg // enabled by the ask import val future = actor ? msg // enabled by the ask import
@ -59,7 +59,7 @@ class FutureDocSpec extends AkkaSpec {
implicit val timeout = system.settings.ActorTimeout implicit val timeout = system.settings.ActorTimeout
//#map-to //#map-to
import akka.dispatch.Future import akka.dispatch.Future
import akka.patterns.ask import akka.pattern.ask
val future: Future[String] = ask(actor, msg).mapTo[String] val future: Future[String] = ask(actor, msg).mapTo[String]
//#map-to //#map-to
@ -149,7 +149,7 @@ class FutureDocSpec extends AkkaSpec {
val msg2 = 2 val msg2 = 2
implicit val timeout = system.settings.ActorTimeout implicit val timeout = system.settings.ActorTimeout
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
//#composing-wrong //#composing-wrong
val f1 = ask(actor1, msg1) val f1 = ask(actor1, msg1)
@ -173,7 +173,7 @@ class FutureDocSpec extends AkkaSpec {
val msg2 = 2 val msg2 = 2
implicit val timeout = system.settings.ActorTimeout implicit val timeout = system.settings.ActorTimeout
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
//#composing //#composing
val f1 = ask(actor1, msg1) val f1 = ask(actor1, msg1)
@ -195,7 +195,7 @@ class FutureDocSpec extends AkkaSpec {
val oddActor = system.actorOf(Props[OddActor]) val oddActor = system.actorOf(Props[OddActor])
//#sequence-ask //#sequence-ask
// oddActor returns odd numbers sequentially from 1 as a List[Future[Int]] // oddActor returns odd numbers sequentially from 1 as a List[Future[Int]]
val listOfFutures = List.fill(100)(akka.patterns.ask(oddActor, GetNext).mapTo[Int]) val listOfFutures = List.fill(100)(akka.pattern.ask(oddActor, GetNext).mapTo[Int])
// now we have a Future[List[Int]] // now we have a Future[List[Int]]
val futureList = Future.sequence(listOfFutures) val futureList = Future.sequence(listOfFutures)
@ -243,7 +243,7 @@ class FutureDocSpec extends AkkaSpec {
val actor = system.actorOf(Props[MyActor]) val actor = system.actorOf(Props[MyActor])
val msg1 = -1 val msg1 = -1
//#recover //#recover
val future = akka.patterns.ask(actor, msg1) recover { val future = akka.pattern.ask(actor, msg1) recover {
case e: ArithmeticException 0 case e: ArithmeticException 0
} }
//#recover //#recover

View file

@ -8,7 +8,7 @@ import annotation.tailrec
import akka.actor.{ Props, Actor } import akka.actor.{ Props, Actor }
import akka.util.duration._ import akka.util.duration._
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
case class FibonacciNumber(nbr: Int) case class FibonacciNumber(nbr: Int)

View file

@ -120,7 +120,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
import akka.testkit.TestActorRef import akka.testkit.TestActorRef
import akka.util.duration._ import akka.util.duration._
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
val actorRef = TestActorRef(new MyActor) val actorRef = TestActorRef(new MyActor)
// hypothetical message stimulating a '42' answer // hypothetical message stimulating a '42' answer
@ -204,7 +204,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
"demonstrate probe reply" in { "demonstrate probe reply" in {
import akka.testkit.TestProbe import akka.testkit.TestProbe
import akka.util.duration._ import akka.util.duration._
import akka.patterns.ask import akka.pattern.ask
//#test-probe-reply //#test-probe-reply
val probe = TestProbe() val probe = TestProbe()
val future = probe.ref ? "hello" val future = probe.ref ? "hello"

View file

@ -141,7 +141,7 @@ class TransactorDocSpec extends AkkaSpec {
import akka.dispatch.Await import akka.dispatch.Await
import akka.util.duration._ import akka.util.duration._
import akka.util.Timeout import akka.util.Timeout
import akka.patterns.ask import akka.pattern.ask
val system = ActorSystem("app") val system = ActorSystem("app")

View file

@ -22,7 +22,7 @@ import scala.annotation.tailrec
import com.google.protobuf.ByteString import com.google.protobuf.ByteString
import java.util.concurrent.TimeoutException import java.util.concurrent.TimeoutException
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
/** /**
* Interface for node membership change listener. * Interface for node membership change listener.

View file

@ -5,7 +5,7 @@ import akka.routing._
import akka.actor.{ Actor, Props } import akka.actor.{ Actor, Props }
import akka.testkit._ import akka.testkit._
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
object DirectRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { object DirectRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec {
override def NrOfNodes = 2 override def NrOfNodes = 2

View file

@ -6,7 +6,7 @@ import akka.routing._
import akka.testkit._ import akka.testkit._
import akka.util.duration._ import akka.util.duration._
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
object NewRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { object NewRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec {
override def NrOfNodes = 2 override def NrOfNodes = 2

View file

@ -5,7 +5,7 @@ import akka.remote._
import akka.routing._ import akka.routing._
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
object RandomRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { object RandomRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec {
override def NrOfNodes = 4 override def NrOfNodes = 4

View file

@ -5,7 +5,7 @@ import akka.remote._
import akka.routing._ import akka.routing._
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
object RoundRobinRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { object RoundRobinRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec {
override def NrOfNodes = 4 override def NrOfNodes = 4

View file

@ -7,7 +7,7 @@ import akka.testkit._
import akka.actor._ import akka.actor._
import com.typesafe.config._ import com.typesafe.config._
import akka.dispatch.Await import akka.dispatch.Await
import akka.patterns.ask import akka.pattern.ask
object RemoteCommunicationSpec { object RemoteCommunicationSpec {
class Echo extends Actor { class Echo extends Actor {

View file

@ -13,7 +13,7 @@ import java.util.concurrent.atomic.AtomicLong
import akka.event.EventStream import akka.event.EventStream
import scala.collection.immutable.Stack import scala.collection.immutable.Stack
import akka.dispatch._ import akka.dispatch._
import akka.patterns.ask import akka.pattern.ask
/** /**
* This special ActorRef is exclusively for use during unit testing in a single-threaded environment. Therefore, it * This special ActorRef is exclusively for use during unit testing in a single-threaded environment. Therefore, it

View file

@ -17,7 +17,7 @@ import akka.actor.DeadLetter
import java.util.concurrent.TimeoutException import java.util.concurrent.TimeoutException
import akka.dispatch.{ Await, MessageDispatcher } import akka.dispatch.{ Await, MessageDispatcher }
import akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
import akka.patterns.ask import akka.pattern.ask
object TimingTest extends Tag("timing") object TimingTest extends Tag("timing")

View file

@ -10,7 +10,7 @@ import akka.event.Logging.Warning
import akka.dispatch.{ Future, Promise, Await } import akka.dispatch.{ Future, Promise, Await }
import akka.util.duration._ import akka.util.duration._
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.patterns.ask import akka.pattern.ask
/** /**
* Test whether TestActorRef behaves as an ActorRef should, besides its own spec. * Test whether TestActorRef behaves as an ActorRef should, besides its own spec.

View file

@ -6,7 +6,7 @@ import org.scalatest.{ BeforeAndAfterEach, WordSpec }
import akka.actor._ import akka.actor._
import akka.util.duration._ import akka.util.duration._
import akka.dispatch.{ Await, Future } import akka.dispatch.{ Await, Future }
import akka.patterns.ask import akka.pattern.ask
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class TestProbeSpec extends AkkaSpec with DefaultTimeout { class TestProbeSpec extends AkkaSpec with DefaultTimeout {

View file

@ -18,7 +18,7 @@ import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory; import akka.actor.UntypedActorFactory;
import akka.dispatch.Await; import akka.dispatch.Await;
import akka.dispatch.Future; import akka.dispatch.Future;
import static akka.Patterns.ask; import static akka.pattern.Patterns.ask;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
import akka.testkit.EventFilter; import akka.testkit.EventFilter;
import akka.testkit.ErrorFilter; import akka.testkit.ErrorFilter;

View file

@ -18,7 +18,7 @@ import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory; import akka.actor.UntypedActorFactory;
import akka.dispatch.Await; import akka.dispatch.Await;
import akka.dispatch.Future; import akka.dispatch.Future;
import static akka.Patterns.ask; import static akka.pattern.Patterns.ask;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
import akka.testkit.EventFilter; import akka.testkit.EventFilter;
import akka.testkit.ErrorFilter; import akka.testkit.ErrorFilter;

View file

@ -12,7 +12,7 @@ import akka.util.duration._
import akka.util.Timeout import akka.util.Timeout
import akka.testkit._ import akka.testkit._
import scala.concurrent.stm._ import scala.concurrent.stm._
import akka.patterns.ask import akka.pattern.ask
object CoordinatedIncrement { object CoordinatedIncrement {
case class Increment(friends: Seq[ActorRef]) case class Increment(friends: Seq[ActorRef])

View file

@ -15,7 +15,7 @@ import akka.testkit.TestEvent.Mute
import scala.concurrent.stm._ import scala.concurrent.stm._
import scala.util.Random.{ nextInt random } import scala.util.Random.{ nextInt random }
import java.util.concurrent.CountDownLatch import java.util.concurrent.CountDownLatch
import akka.patterns.ask import akka.pattern.ask
object FickleFriends { object FickleFriends {
case class FriendlyIncrement(friends: Seq[ActorRef], timeout: Timeout, latch: CountDownLatch) case class FriendlyIncrement(friends: Seq[ActorRef], timeout: Timeout, latch: CountDownLatch)

View file

@ -10,7 +10,7 @@ import akka.util.duration._
import akka.util.Timeout import akka.util.Timeout
import akka.testkit._ import akka.testkit._
import scala.concurrent.stm._ import scala.concurrent.stm._
import akka.patterns.ask import akka.pattern.ask
object TransactorIncrement { object TransactorIncrement {
case class Increment(friends: Seq[ActorRef], latch: TestLatch) case class Increment(friends: Seq[ActorRef], latch: TestLatch)