Rename Typed TestKit, #25084

Artifact:
akka-testkit-typed -> akka-actor-testkit-typed

Package:
akka.testkit.typed -> akka.actor.testkit.typed

Config:
akka.typed -> akka.actor.typed
akka.testkit.typed -> akka.actor.testkit.typed
This commit is contained in:
Patrik Nordwall 2018-05-15 19:38:43 +02:00
parent 34bb7fee3c
commit b08d3acb4b
112 changed files with 285 additions and 276 deletions

View file

@ -1,11 +1,11 @@
############################################ ##################################################
# Akka Typed Testkit Reference Config File # # Akka Actor Typed Testkit Reference Config File #
############################################ ##################################################
# This is the reference config file that contains all the default settings. # This is the reference config file that contains all the default settings.
# Make your edits/overrides in your application.conf. # Make your edits/overrides in your application.conf.
akka.actor.typed.test { akka.actor.testkit.typed {
# factor by which to scale timeouts during tests, e.g. to account for shared # factor by which to scale timeouts during tests, e.g. to account for shared
# build system load # build system load
timefactor = 1.0 timefactor = 1.0

View file

@ -0,0 +1,20 @@
/**
* Copyright (C) 2014-2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed
import akka.annotation.DoNotInherit
/**
* All tracked effects for the [[akka.actor.testkit.typed.scaladsl.BehaviorTestKit]] and
* [[akka.actor.testkit.typed.javadsl.BehaviorTestKit]] must extend this type.
*
* Factories/types for effects are available through [[akka.actor.testkit.typed.scaladsl.Effects]]
* and [[akka.actor.testkit.typed.javadsl.Effects]]
*
* Not for user extension
*/
@DoNotInherit
abstract class Effect private[akka] ()

View file

@ -2,15 +2,15 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed package akka.actor.testkit.typed
import akka.annotation.DoNotInherit import akka.annotation.DoNotInherit
/** /**
* Not for user extension. * Not for user extension.
* *
* Instances are available from `FishingOutcomes` in the respective dsls: [[akka.testkit.typed.scaladsl.FishingOutcomes]] * Instances are available from `FishingOutcomes` in the respective dsls: [[akka.actor.testkit.typed.scaladsl.FishingOutcomes]]
* and [[akka.testkit.typed.javadsl.FishingOutcomes]] * and [[akka.actor.testkit.typed.javadsl.FishingOutcomes]]
*/ */
@DoNotInherit sealed trait FishingOutcome @DoNotInherit sealed trait FishingOutcome

View file

@ -2,7 +2,7 @@
* Copyright (C) 2017-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2017-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed package akka.actor.testkit.typed
import com.typesafe.config.Config import com.typesafe.config.Config
@ -19,27 +19,27 @@ final case class TE(message: String) extends RuntimeException(message) with NoSt
object TestKitSettings { object TestKitSettings {
/** /**
* Reads configuration settings from `akka.actor.typed.test` section. * Reads configuration settings from `akka.actor.testkit.typed` section.
*/ */
def apply(system: ActorSystem[_]): TestKitSettings = def apply(system: ActorSystem[_]): TestKitSettings =
apply(system.settings.config.getConfig("akka.actor.typed.test")) apply(system.settings.config.getConfig("akka.actor.testkit.typed"))
/** /**
* Reads configuration settings from given `Config` that * Reads configuration settings from given `Config` that
* must have the same layout as the `akka.actor.typed.test` section. * must have the same layout as the `akka.actor.testkit.typed` section.
*/ */
def apply(config: Config): TestKitSettings = def apply(config: Config): TestKitSettings =
new TestKitSettings(config) new TestKitSettings(config)
/** /**
* Java API: Reads configuration settings from `akka.actor.typed.test` section. * Java API: Reads configuration settings from `akka.actor.testkit.typed` section.
*/ */
def create(system: ActorSystem[_]): TestKitSettings = def create(system: ActorSystem[_]): TestKitSettings =
apply(system) apply(system)
/** /**
* Reads configuration settings from given `Config` that * Reads configuration settings from given `Config` that
* must have the same layout as the `akka.actor.typed.test` section. * must have the same layout as the `akka.actor.testkit.typed` section.
*/ */
def create(config: Config): TestKitSettings = def create(config: Config): TestKitSettings =
new TestKitSettings(config) new TestKitSettings(config)

View file

@ -2,7 +2,7 @@
* Copyright (C) 2016-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2016-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.internal package akka.actor.testkit.typed.internal
import java.util.concurrent.{ CompletionStage, ThreadFactory } import java.util.concurrent.{ CompletionStage, ThreadFactory }

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.internal package akka.actor.testkit.typed.internal
import java.util import java.util
@ -10,8 +10,8 @@ import akka.actor.ActorPath
import akka.actor.typed.{ Behavior, PostStop, Signal, ActorRef } import akka.actor.typed.{ Behavior, PostStop, Signal, ActorRef }
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.testkit.typed.Effect import akka.actor.testkit.typed.Effect
import akka.testkit.typed.scaladsl.Effects._ import akka.actor.testkit.typed.scaladsl.Effects._
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
@ -25,8 +25,8 @@ import scala.util.control.NonFatal
*/ */
@InternalApi @InternalApi
private[akka] final class BehaviorTestKitImpl[T](_path: ActorPath, _initialBehavior: Behavior[T]) private[akka] final class BehaviorTestKitImpl[T](_path: ActorPath, _initialBehavior: Behavior[T])
extends akka.testkit.typed.javadsl.BehaviorTestKit[T] extends akka.actor.testkit.typed.javadsl.BehaviorTestKit[T]
with akka.testkit.typed.scaladsl.BehaviorTestKit[T] { with akka.actor.testkit.typed.scaladsl.BehaviorTestKit[T] {
// really this should be private, make so when we port out tests that need it // really this should be private, make so when we port out tests that need it
private[akka] val ctx = new EffectfulActorContext[T](_path) private[akka] val ctx = new EffectfulActorContext[T](_path)

View file

@ -2,7 +2,7 @@
* Copyright (C) 2016-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2016-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.internal package akka.actor.testkit.typed.internal
import java.util.LinkedList import java.util.LinkedList

View file

@ -2,7 +2,7 @@
* Copyright (C) 2016-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2016-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.internal package akka.actor.testkit.typed.internal
import java.util.concurrent.ConcurrentLinkedQueue import java.util.concurrent.ConcurrentLinkedQueue

View file

@ -2,15 +2,15 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.internal package akka.actor.testkit.typed.internal
import java.util.concurrent.ConcurrentLinkedQueue import java.util.concurrent.ConcurrentLinkedQueue
import akka.actor.{ Cancellable, ActorPath } import akka.actor.{ Cancellable, ActorPath }
import akka.actor.typed.{ ActorRef, Behavior, Props } import akka.actor.typed.{ ActorRef, Behavior, Props }
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.testkit.typed.Effect import akka.actor.testkit.typed.Effect
import akka.testkit.typed.scaladsl.Effects._ import akka.actor.testkit.typed.scaladsl.Effects._
import scala.concurrent.duration.{ Duration, FiniteDuration } import scala.concurrent.duration.{ Duration, FiniteDuration }

View file

@ -2,12 +2,12 @@
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.internal package akka.actor.testkit.typed.internal
import akka.actor.typed._ import akka.actor.typed._
import akka.actor.typed.internal._ import akka.actor.typed.internal._
import akka.actor.typed.internal.adapter.AbstractLogger import akka.actor.typed.internal.adapter.AbstractLogger
import akka.testkit.typed.scaladsl.TestInbox import akka.actor.testkit.typed.scaladsl.TestInbox
import akka.actor.{ ActorPath, InvalidMessageException } import akka.actor.{ ActorPath, InvalidMessageException }
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.event.Logging import akka.event.Logging

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.internal package akka.actor.testkit.typed.internal
import java.util.concurrent.ConcurrentLinkedQueue import java.util.concurrent.ConcurrentLinkedQueue
@ -18,8 +18,8 @@ import scala.collection.immutable
*/ */
@InternalApi @InternalApi
private[akka] final class TestInboxImpl[T](path: ActorPath) private[akka] final class TestInboxImpl[T](path: ActorPath)
extends akka.testkit.typed.javadsl.TestInbox[T] extends akka.actor.testkit.typed.javadsl.TestInbox[T]
with akka.testkit.typed.scaladsl.TestInbox[T] { with akka.actor.testkit.typed.scaladsl.TestInbox[T] {
private val q = new ConcurrentLinkedQueue[T] private val q = new ConcurrentLinkedQueue[T]

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.internal package akka.actor.testkit.typed.internal
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Props } import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Props }

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.internal package akka.actor.testkit.typed.internal
import java.time import java.time
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
@ -12,9 +12,9 @@ import java.util.function.Supplier
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Terminated } import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Terminated }
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.testkit.typed.javadsl.{ TestProbe JavaTestProbe } import akka.actor.testkit.typed.javadsl.{ TestProbe JavaTestProbe }
import akka.testkit.typed.scaladsl.{ TestDuration, TestProbe ScalaTestProbe } import akka.actor.testkit.typed.scaladsl.{ TestDuration, TestProbe ScalaTestProbe }
import akka.testkit.typed.{ FishingOutcome, TestKitSettings } import akka.actor.testkit.typed.{ FishingOutcome, TestKitSettings }
import akka.util.PrettyDuration._ import akka.util.PrettyDuration._
import akka.util.{ BoxedType, Timeout } import akka.util.{ BoxedType, Timeout }
import akka.util.JavaDurationConverters._ import akka.util.JavaDurationConverters._

View file

@ -2,15 +2,15 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.javadsl package akka.actor.testkit.typed.javadsl
import java.time.Duration import java.time.Duration
import akka.actor.Scheduler import akka.actor.Scheduler
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Props } import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Props }
import akka.testkit.typed.TestKitSettings import akka.actor.testkit.typed.TestKitSettings
import akka.testkit.typed.internal.TestKitUtils import akka.actor.testkit.typed.internal.TestKitUtils
import akka.testkit.typed.scaladsl.{ ActorTestKit ScalaTestKit } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit ScalaTestKit }
import akka.util.Timeout import akka.util.Timeout
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.util.JavaDurationConverters._ import akka.util.JavaDurationConverters._
@ -39,7 +39,7 @@ object ActorTestKit {
/** /**
* Create a testkit named with this test class, and use a custom config for the actor system, * Create a testkit named with this test class, and use a custom config for the actor system,
* and a custom [[akka.testkit.typed.TestKitSettings]] * and a custom [[akka.actor.testkit.typed.TestKitSettings]]
*/ */
def create(testClass: Class[_], customConfig: Config, settings: TestKitSettings) = new ActorTestKit(new ScalaTestKit { def create(testClass: Class[_], customConfig: Config, settings: TestKitSettings) = new ActorTestKit(new ScalaTestKit {
override def name = TestKitUtils.testNameFromCallStack(testClass) override def name = TestKitUtils.testNameFromCallStack(testClass)
@ -97,10 +97,10 @@ object ActorTestKit {
* *
* For synchronous testing of a `Behavior` see [[BehaviorTestKit]] * For synchronous testing of a `Behavior` see [[BehaviorTestKit]]
*/ */
final class ActorTestKit protected (delegate: akka.testkit.typed.scaladsl.ActorTestKit) { final class ActorTestKit protected (delegate: akka.actor.testkit.typed.scaladsl.ActorTestKit) {
/** /**
* The default timeout as specified with the config/[[akka.testkit.typed.TestKitSettings]] * The default timeout as specified with the config/[[akka.actor.testkit.typed.TestKitSettings]]
*/ */
def timeout: Timeout = delegate.timeout def timeout: Timeout = delegate.timeout

View file

@ -2,17 +2,17 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.javadsl package akka.actor.testkit.typed.javadsl
import akka.actor.typed.{ Behavior, Signal, ActorRef } import akka.actor.typed.{ Behavior, Signal, ActorRef }
import akka.annotation.DoNotInherit import akka.annotation.DoNotInherit
import akka.testkit.typed.Effect import akka.actor.testkit.typed.Effect
import akka.testkit.typed.internal.BehaviorTestKitImpl import akka.actor.testkit.typed.internal.BehaviorTestKitImpl
import java.util.concurrent.ThreadLocalRandom import java.util.concurrent.ThreadLocalRandom
object BehaviorTestKit { object BehaviorTestKit {
import akka.testkit.typed.scaladsl.TestInbox.address import akka.actor.testkit.typed.scaladsl.TestInbox.address
/** /**
* JAVA API * JAVA API
@ -40,7 +40,7 @@ object BehaviorTestKit {
@DoNotInherit @DoNotInherit
abstract class BehaviorTestKit[T] { abstract class BehaviorTestKit[T] {
/** /**
* Requests the oldest [[Effect]] or [[akka.testkit.typed.javadsl.Effects.noEffects]] if no effects * Requests the oldest [[Effect]] or [[akka.actor.testkit.typed.javadsl.Effects.noEffects]] if no effects
* have taken place. The effect is consumed, subsequent calls won't * have taken place. The effect is consumed, subsequent calls won't
* will not include this effect. * will not include this effect.
*/ */

View file

@ -2,12 +2,12 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.javadsl package akka.actor.testkit.typed.javadsl
import java.time.Duration import java.time.Duration
import akka.actor.typed.{ ActorRef, Behavior, Props } import akka.actor.typed.{ ActorRef, Behavior, Props }
import akka.testkit.typed.Effect import akka.actor.testkit.typed.Effect
import akka.util.JavaDurationConverters._ import akka.util.JavaDurationConverters._
/** /**
@ -15,7 +15,7 @@ import akka.util.JavaDurationConverters._
* actual effects to expected ones. * actual effects to expected ones.
*/ */
object Effects { object Effects {
import akka.testkit.typed.scaladsl.Effects._ import akka.actor.testkit.typed.scaladsl.Effects._
/** /**
* The behavior spawned a named child with the given behavior with no specific props * The behavior spawned a named child with the given behavior with no specific props

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.javadsl package akka.actor.testkit.typed.javadsl
import java.time.Duration import java.time.Duration
import akka.actor.typed.ActorSystem import akka.actor.typed.ActorSystem
@ -21,7 +21,7 @@ object ManualTime {
/** /**
* Config that needs to be in place for the actor system to use the manual * Config that needs to be in place for the actor system to use the manual
*/ */
def config(): Config = akka.testkit.typed.scaladsl.ManualTime.config def config(): Config = akka.actor.testkit.typed.scaladsl.ManualTime.config
/** /**
* Access the manual scheduler, note that you need to setup the actor system/testkit with [[config()]] for this to * Access the manual scheduler, note that you need to setup the actor system/testkit with [[config()]] for this to
@ -31,7 +31,7 @@ object ManualTime {
system.scheduler match { system.scheduler match {
case sc: akka.testkit.ExplicitlyTriggeredScheduler new ManualTime(sc) case sc: akka.testkit.ExplicitlyTriggeredScheduler new ManualTime(sc)
case _ throw new IllegalArgumentException("ActorSystem not configured with explicitly triggered scheduler, " + case _ throw new IllegalArgumentException("ActorSystem not configured with explicitly triggered scheduler, " +
"make sure to include akka.testkit.typed.javadsl.ManualTime.config() when setting up the test") "make sure to include akka.actor.testkit.typed.javadsl.ManualTime.config() when setting up the test")
} }
} }

View file

@ -2,11 +2,11 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.javadsl package akka.actor.testkit.typed.javadsl
import akka.actor.typed.ActorRef import akka.actor.typed.ActorRef
import akka.annotation.DoNotInherit import akka.annotation.DoNotInherit
import akka.testkit.typed.internal.TestInboxImpl import akka.actor.testkit.typed.internal.TestInboxImpl
import java.util.concurrent.ThreadLocalRandom import java.util.concurrent.ThreadLocalRandom
@ -14,7 +14,7 @@ import scala.collection.JavaConverters._
import scala.collection.immutable import scala.collection.immutable
object TestInbox { object TestInbox {
import akka.testkit.typed.scaladsl.TestInbox.address import akka.actor.testkit.typed.scaladsl.TestInbox.address
def create[T](name: String): TestInbox[T] = { def create[T](name: String): TestInbox[T] = {
val uid = ThreadLocalRandom.current().nextInt() val uid = ThreadLocalRandom.current().nextInt()
@ -28,9 +28,9 @@ object TestInbox {
/** /**
* Utility for use as an [[ActorRef]] when *synchronously* testing [[akka.actor.typed.Behavior]] * Utility for use as an [[ActorRef]] when *synchronously* testing [[akka.actor.typed.Behavior]]
* with [[akka.testkit.typed.javadsl.BehaviorTestKit]]. * with [[akka.actor.testkit.typed.javadsl.BehaviorTestKit]].
* *
* If you plan to use a real [[akka.actor.typed.ActorSystem]] then use [[akka.testkit.typed.javadsl.TestProbe]] * If you plan to use a real [[akka.actor.typed.ActorSystem]] then use [[akka.actor.testkit.typed.javadsl.TestProbe]]
* for asynchronous testing. * for asynchronous testing.
* *
* Use `TestInbox.create` factory methods to create instances * Use `TestInbox.create` factory methods to create instances

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.javadsl package akka.actor.testkit.typed.javadsl
import akka.actor.Scheduler import akka.actor.Scheduler
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Props } import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Props }

View file

@ -2,16 +2,16 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.javadsl package akka.actor.testkit.typed.javadsl
import java.time.Duration import java.time.Duration
import java.util.function.Supplier import java.util.function.Supplier
import akka.actor.typed.{ ActorRef, ActorSystem } import akka.actor.typed.{ ActorRef, ActorSystem }
import akka.annotation.DoNotInherit import akka.annotation.DoNotInherit
import akka.testkit.typed.internal.TestProbeImpl import akka.actor.testkit.typed.internal.TestProbeImpl
import akka.testkit.typed.{ FishingOutcome, TestKitSettings } import akka.actor.testkit.typed.{ FishingOutcome, TestKitSettings }
import akka.testkit.typed.scaladsl.TestDuration import akka.actor.testkit.typed.scaladsl.TestDuration
import akka.util.JavaDurationConverters._ import akka.util.JavaDurationConverters._
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.FiniteDuration
@ -27,17 +27,17 @@ object FishingOutcomes {
/** /**
* Consume this message and continue with the next * Consume this message and continue with the next
*/ */
def continueAndIgnore(): FishingOutcome = akka.testkit.typed.FishingOutcome.ContinueAndIgnore def continueAndIgnore(): FishingOutcome = akka.actor.testkit.typed.FishingOutcome.ContinueAndIgnore
/** /**
* Complete fishing and return this message * Complete fishing and return this message
*/ */
def complete(): FishingOutcome = akka.testkit.typed.FishingOutcome.Complete def complete(): FishingOutcome = akka.actor.testkit.typed.FishingOutcome.Complete
/** /**
* Fail fishing with a custom error message * Fail fishing with a custom error message
*/ */
def fail(error: String): FishingOutcome = akka.testkit.typed.FishingOutcome.Fail(error) def fail(error: String): FishingOutcome = akka.actor.testkit.typed.FishingOutcome.Fail(error)
} }
object TestProbe { object TestProbe {
@ -81,7 +81,7 @@ abstract class TestProbe[M] {
/** /**
* Obtain time remaining for execution of the innermost enclosing `within` * Obtain time remaining for execution of the innermost enclosing `within`
* block or missing that it returns the properly dilated default for this * block or missing that it returns the properly dilated default for this
* case from settings (key "akka.actor.typed.test.single-expect-default"). * case from settings (key "akka.actor.testkit.typed.single-expect-default").
*/ */
def getRemainingOrDefault: Duration def getRemainingOrDefault: Duration
@ -105,7 +105,7 @@ abstract class TestProbe[M] {
* the remaining time governed by the innermost enclosing `within` block. * the remaining time governed by the innermost enclosing `within` block.
* *
* Note that the timeout is scaled using Duration.dilated, which uses the * Note that the timeout is scaled using Duration.dilated, which uses the
* configuration entry "akka.actor.typed.test.timefactor", while the min Duration is not. * configuration entry "akka.actor.testkit.typed.timefactor", while the min Duration is not.
* *
* {{{ * {{{
* val ret = within(50 millis) { * val ret = within(50 millis) {
@ -158,7 +158,7 @@ abstract class TestProbe[M] {
def expectNoMessage(max: Duration): Unit def expectNoMessage(max: Duration): Unit
/** /**
* Assert that no message is received. Waits for the default period configured as `akka.actor.typed.test.expect-no-message-default` * Assert that no message is received. Waits for the default period configured as `akka.actor.testkit.typed.expect-no-message-default`
* That value is dilated. * That value is dilated.
*/ */
def expectNoMessage(): Unit def expectNoMessage(): Unit

View file

@ -2,13 +2,13 @@
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.scaladsl package akka.actor.testkit.typed.scaladsl
import akka.actor.typed.scaladsl.AskPattern._ import akka.actor.typed.scaladsl.AskPattern._
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Props } import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Props }
import akka.annotation.{ ApiMayChange, InternalApi } import akka.annotation.{ ApiMayChange, InternalApi }
import akka.testkit.typed.TestKitSettings import akka.actor.testkit.typed.TestKitSettings
import akka.testkit.typed.internal.{ ActorTestKitGuardian, TestKitUtils } import akka.actor.testkit.typed.internal.{ ActorTestKitGuardian, TestKitUtils }
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import scala.concurrent.Await import scala.concurrent.Await

View file

@ -2,12 +2,12 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.scaladsl package akka.actor.testkit.typed.scaladsl
import akka.actor.typed.{ Behavior, Signal, ActorRef } import akka.actor.typed.{ Behavior, Signal, ActorRef }
import akka.annotation.DoNotInherit import akka.annotation.DoNotInherit
import akka.testkit.typed.Effect import akka.actor.testkit.typed.Effect
import akka.testkit.typed.internal.BehaviorTestKitImpl import akka.actor.testkit.typed.internal.BehaviorTestKitImpl
import java.util.concurrent.ThreadLocalRandom import java.util.concurrent.ThreadLocalRandom
@ -15,7 +15,7 @@ import scala.collection.immutable
import scala.reflect.ClassTag import scala.reflect.ClassTag
object BehaviorTestKit { object BehaviorTestKit {
import akka.testkit.typed.scaladsl.TestInbox.address import akka.actor.testkit.typed.scaladsl.TestInbox.address
def apply[T](initialBehavior: Behavior[T], name: String): BehaviorTestKit[T] = { def apply[T](initialBehavior: Behavior[T], name: String): BehaviorTestKit[T] = {
val uid = ThreadLocalRandom.current().nextInt() val uid = ThreadLocalRandom.current().nextInt()
@ -41,7 +41,7 @@ trait BehaviorTestKit[T] {
private[akka] def ctx: akka.actor.typed.ActorContext[T] private[akka] def ctx: akka.actor.typed.ActorContext[T]
/** /**
* Requests the oldest [[Effect]] or [[akka.testkit.typed.scaladsl.Effects.NoEffects]] if no effects * Requests the oldest [[Effect]] or [[akka.actor.testkit.typed.scaladsl.Effects.NoEffects]] if no effects
* have taken place. The effect is consumed, subsequent calls won't * have taken place. The effect is consumed, subsequent calls won't
* will not include this effect. * will not include this effect.
*/ */

View file

@ -2,10 +2,10 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.scaladsl package akka.actor.testkit.typed.scaladsl
import akka.actor.typed.{ ActorRef, Behavior, Props } import akka.actor.typed.{ ActorRef, Behavior, Props }
import akka.testkit.typed.Effect import akka.actor.testkit.typed.Effect
import scala.concurrent.duration.{ Duration, FiniteDuration } import scala.concurrent.duration.{ Duration, FiniteDuration }

View file

@ -2,7 +2,7 @@
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.scaladsl package akka.actor.testkit.typed.scaladsl
import akka.actor.typed.ActorSystem import akka.actor.typed.ActorSystem
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
@ -30,7 +30,7 @@ object ManualTime {
system.scheduler match { system.scheduler match {
case sc: akka.testkit.ExplicitlyTriggeredScheduler new ManualTime(sc) case sc: akka.testkit.ExplicitlyTriggeredScheduler new ManualTime(sc)
case _ throw new IllegalArgumentException("ActorSystem not configured with explicitly triggered scheduler, " + case _ throw new IllegalArgumentException("ActorSystem not configured with explicitly triggered scheduler, " +
"make sure to include akka.testkit.typed.scaladsl.ManualTime.config() when setting up the test") "make sure to include akka.actor.testkit.typed.scaladsl.ManualTime.config() when setting up the test")
} }
} }

View file

@ -2,12 +2,12 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.scaladsl package akka.actor.testkit.typed.scaladsl
import akka.actor.{ Address, RootActorPath } import akka.actor.{ Address, RootActorPath }
import akka.actor.typed.ActorRef import akka.actor.typed.ActorRef
import akka.annotation.{ ApiMayChange, DoNotInherit } import akka.annotation.{ ApiMayChange, DoNotInherit }
import akka.testkit.typed.internal.TestInboxImpl import akka.actor.testkit.typed.internal.TestInboxImpl
import java.util.concurrent.ThreadLocalRandom import java.util.concurrent.ThreadLocalRandom
@ -25,9 +25,9 @@ object TestInbox {
/** /**
* Utility for use as an [[ActorRef]] when *synchronously* testing [[akka.actor.typed.Behavior]] * Utility for use as an [[ActorRef]] when *synchronously* testing [[akka.actor.typed.Behavior]]
* with [[akka.testkit.typed.javadsl.BehaviorTestKit]]. * with [[akka.actor.testkit.typed.javadsl.BehaviorTestKit]].
* *
* If you plan to use a real [[akka.actor.typed.ActorSystem]] then use [[akka.testkit.typed.javadsl.TestProbe]] * If you plan to use a real [[akka.actor.typed.ActorSystem]] then use [[akka.actor.testkit.typed.javadsl.TestProbe]]
* for asynchronous testing. * for asynchronous testing.
* *
* Use factory `apply` in companion to create instances * Use factory `apply` in companion to create instances

View file

@ -2,12 +2,12 @@
* Copyright (C) 2017-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2017-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.scaladsl package akka.actor.testkit.typed.scaladsl
import akka.actor.typed.{ ActorRef, ActorSystem } import akka.actor.typed.{ ActorRef, ActorSystem }
import akka.annotation.DoNotInherit import akka.annotation.DoNotInherit
import akka.testkit.typed.internal.TestProbeImpl import akka.actor.testkit.typed.internal.TestProbeImpl
import akka.testkit.typed.{ FishingOutcome, TestKitSettings } import akka.actor.testkit.typed.{ FishingOutcome, TestKitSettings }
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.reflect.ClassTag import scala.reflect.ClassTag
@ -61,7 +61,7 @@ object TestProbe {
/** /**
* Obtain time remaining for execution of the innermost enclosing `within` * Obtain time remaining for execution of the innermost enclosing `within`
* block or missing that it returns the properly dilated default for this * block or missing that it returns the properly dilated default for this
* case from settings (key "akka.actor.typed.test.single-expect-default"). * case from settings (key "akka.actor.testkit.typed.single-expect-default").
*/ */
def remainingOrDefault: FiniteDuration def remainingOrDefault: FiniteDuration
@ -85,7 +85,7 @@ object TestProbe {
* the remaining time governed by the innermost enclosing `within` block. * the remaining time governed by the innermost enclosing `within` block.
* *
* Note that the timeout is scaled using Duration.dilated, which uses the * Note that the timeout is scaled using Duration.dilated, which uses the
* configuration entry "akka.actor.typed.test.timefactor", while the min Duration is not. * configuration entry "akka.actor.testkit.typed.timefactor", while the min Duration is not.
* *
* {{{ * {{{
* val ret = within(50 millis) { * val ret = within(50 millis) {
@ -134,7 +134,7 @@ object TestProbe {
def expectNoMessage(max: FiniteDuration): Unit def expectNoMessage(max: FiniteDuration): Unit
/** /**
* Assert that no message is received. Waits for the default period configured as `akka.actor.typed.test.expect-no-message-default` * Assert that no message is received. Waits for the default period configured as `akka.actor.testkit.typed.expect-no-message-default`
* That value is dilated. * That value is dilated.
*/ */
def expectNoMessage(): Unit def expectNoMessage(): Unit

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed package akka.actor.testkit.typed
import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.FiniteDuration
@ -15,7 +15,7 @@ package object scaladsl {
* *
* {{{ * {{{
* import scala.concurrent.duration._ * import scala.concurrent.duration._
* import akka.testkit.typed.scaladsl._ * import akka.actor.testkit.typed.scaladsl._
* 10.milliseconds.dilated * 10.milliseconds.dilated
* }}} * }}}
* *

View file

@ -2,7 +2,7 @@
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.javadsl; package akka.actor.testkit.typed.javadsl;
import akka.Done; import akka.Done;
import akka.actor.typed.javadsl.Behaviors; import akka.actor.typed.javadsl.Behaviors;

View file

@ -2,13 +2,13 @@
* Copyright (C) 2017-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2017-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.javadsl; package akka.actor.testkit.typed.javadsl;
import akka.actor.typed.ActorRef; import akka.actor.typed.ActorRef;
import akka.actor.typed.Behavior; import akka.actor.typed.Behavior;
import akka.actor.typed.javadsl.Behaviors; import akka.actor.typed.javadsl.Behaviors;
import akka.testkit.typed.javadsl.ActorTestKit; import akka.actor.testkit.typed.javadsl.ActorTestKit;
import akka.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Test; import org.junit.Test;

View file

@ -2,10 +2,10 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.javadsl; package akka.actor.testkit.typed.javadsl;
// #junit-integration // #junit-integration
import akka.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
@ -21,4 +21,4 @@ public class JunitIntegrationExampleTest {
} }
} }
// #junit-integration // #junit-integration

View file

@ -2,13 +2,13 @@
* Copyright (C) 2017-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2017-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.javadsl; package akka.actor.testkit.typed.javadsl;
//#manual-scheduling-simple //#manual-scheduling-simple
import akka.actor.typed.Behavior; import akka.actor.typed.Behavior;
import akka.testkit.typed.javadsl.ManualTime; import akka.actor.testkit.typed.javadsl.ManualTime;
import akka.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
import java.time.Duration; import java.time.Duration;
@ -17,7 +17,7 @@ import akka.actor.typed.javadsl.Behaviors;
import org.junit.Test; import org.junit.Test;
import akka.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
public class ManualTimerExampleTest extends JUnitSuite { public class ManualTimerExampleTest extends JUnitSuite {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.javadsl; package akka.actor.testkit.typed.javadsl;
//#imports //#imports
import akka.actor.typed.*; import akka.actor.typed.*;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.javadsl; package akka.actor.testkit.typed.javadsl;
import akka.actor.typed.ActorRef; import akka.actor.typed.ActorRef;
import akka.actor.typed.ActorSystem; import akka.actor.typed.ActorSystem;

View file

@ -2,11 +2,11 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.scaladsl package akka.actor.testkit.typed.scaladsl
// NOTE: do not optimize import, unused import is here on purpose for docs // NOTE: do not optimize import, unused import is here on purpose for docs
//#scalatest-glue //#scalatest-glue
import akka.testkit.typed.scaladsl.ActorTestKit import akka.actor.testkit.typed.scaladsl.ActorTestKit
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec } import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
abstract class AbstractActorSpec extends WordSpec with ActorTestKit with Matchers with BeforeAndAfterAll { abstract class AbstractActorSpec extends WordSpec with ActorTestKit with Matchers with BeforeAndAfterAll {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.scaladsl package akka.actor.testkit.typed.scaladsl
import akka.actor.typed.Terminated import akka.actor.typed.Terminated
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors

View file

@ -2,7 +2,7 @@
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.scaladsl package akka.actor.testkit.typed.scaladsl
import akka.actor.typed._ import akka.actor.typed._
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors

View file

@ -2,14 +2,14 @@
* Copyright (C) 2014-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2014-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.scaladsl package akka.actor.testkit.typed.scaladsl
import akka.Done import akka.Done
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorRef, Behavior, Props } import akka.actor.typed.{ ActorRef, Behavior, Props }
import akka.testkit.typed.scaladsl.Effects.{ NoEffects, Spawned, SpawnedAdapter, SpawnedAnonymous, SpawnedAnonymousAdapter, Unwatched, Watched } import akka.actor.testkit.typed.scaladsl.Effects.{ NoEffects, Spawned, SpawnedAdapter, SpawnedAnonymous, SpawnedAnonymousAdapter, Watched, Unwatched }
import akka.testkit.typed.scaladsl.BehaviorTestKitSpec.{ Child, Father } import akka.actor.testkit.typed.scaladsl.BehaviorTestKitSpec.{ Child, Father }
import akka.testkit.typed.scaladsl.BehaviorTestKitSpec.Father._ import akka.actor.testkit.typed.scaladsl.BehaviorTestKitSpec.Father._
import org.scalatest.{ Matchers, WordSpec } import org.scalatest.{ Matchers, WordSpec }
object BehaviorTestKitSpec { object BehaviorTestKitSpec {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.scaladsl package akka.actor.testkit.typed.scaladsl
//#manual-scheduling-simple //#manual-scheduling-simple
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors

View file

@ -2,12 +2,12 @@
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.scaladsl package akka.actor.testkit.typed.scaladsl
//#imports //#imports
import akka.actor.typed._ import akka.actor.typed._
import akka.actor.typed.scaladsl._ import akka.actor.typed.scaladsl._
import akka.testkit.typed.scaladsl.Effects._ import akka.actor.testkit.typed.scaladsl.Effects._
//#imports //#imports
import org.scalatest.{ Matchers, WordSpec } import org.scalatest.{ Matchers, WordSpec }

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.testkit.typed.scaladsl package akka.actor.testkit.typed.scaladsl
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors

View file

@ -56,7 +56,7 @@ public class ExtensionsTest extends JUnitSuite {
@Test @Test
public void loadJavaExtensionsFromConfig() { public void loadJavaExtensionsFromConfig() {
Config cfg = ConfigFactory.parseString( Config cfg = ConfigFactory.parseString(
"akka.typed.extensions += \"akka.actor.typed.ExtensionsTest$MyExtension\"").resolve(); "akka.actor.typed.extensions += \"akka.actor.typed.ExtensionsTest$MyExtension\"").resolve();
final ActorSystem<Object> system = ActorSystem.create( final ActorSystem<Object> system = ActorSystem.create(
Behavior.empty(), Behavior.empty(),
"loadJavaExtensionsFromConfig", "loadJavaExtensionsFromConfig",

View file

@ -7,8 +7,8 @@ package akka.actor.typed.javadsl;
import akka.actor.typed.ActorRef; import akka.actor.typed.ActorRef;
import akka.actor.typed.Behavior; import akka.actor.typed.Behavior;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
import akka.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import akka.util.Timeout; import akka.util.Timeout;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;

View file

@ -8,7 +8,7 @@ import akka.actor.typed.Behavior;
import akka.event.Logging; import akka.event.Logging;
import akka.japi.pf.PFBuilder; import akka.japi.pf.PFBuilder;
import akka.testkit.CustomEventFilter; import akka.testkit.CustomEventFilter;
import akka.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import com.typesafe.config.ConfigFactory; import com.typesafe.config.ConfigFactory;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;

View file

@ -8,7 +8,7 @@ import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import akka.Done; import akka.Done;
import akka.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
import scala.concurrent.duration.Duration; import scala.concurrent.duration.Duration;

View file

@ -9,7 +9,7 @@ import akka.actor.typed.ActorSystem;
import akka.actor.typed.Behavior; import akka.actor.typed.Behavior;
import akka.actor.typed.Props; import akka.actor.typed.Props;
import akka.actor.typed.javadsl.*; import akka.actor.typed.javadsl.*;
import akka.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import akka.util.Timeout; import akka.util.Timeout;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;

View file

@ -12,8 +12,8 @@ import akka.Done;
import akka.actor.typed.ActorRef; import akka.actor.typed.ActorRef;
import akka.actor.typed.Behavior; import akka.actor.typed.Behavior;
import akka.actor.typed.javadsl.Behaviors; import akka.actor.typed.javadsl.Behaviors;
import akka.testkit.typed.javadsl.TestInbox; import akka.actor.testkit.typed.javadsl.TestInbox;
import akka.testkit.typed.javadsl.BehaviorTestKit; import akka.actor.testkit.typed.javadsl.BehaviorTestKit;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;

View file

@ -6,7 +6,7 @@ package akka.actor.typed
import akka.actor.InvalidMessageException import akka.actor.InvalidMessageException
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.reflect.ClassTag import scala.reflect.ClassTag

View file

@ -10,7 +10,7 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.Behaviors._ import akka.actor.typed.scaladsl.Behaviors._
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import akka.testkit.EventFilter import akka.testkit.EventFilter
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import akka.util.Timeout import akka.util.Timeout
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.scalatest.concurrent.ScalaFutures import org.scalatest.concurrent.ScalaFutures

View file

@ -12,7 +12,7 @@ import akka.japi.pf.{ FI, PFBuilder }
import java.util.function.{ Function F1 } import java.util.function.{ Function F1 }
import akka.Done import akka.Done
import akka.testkit.typed.scaladsl.{ BehaviorTestKit, TestInbox } import akka.actor.testkit.typed.scaladsl.{ BehaviorTestKit, TestInbox }
object BehaviorSpec { object BehaviorSpec {
sealed trait Command { sealed trait Command {

View file

@ -5,8 +5,8 @@
package akka.actor.typed package akka.actor.typed
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.typed.TestKitSettings import akka.actor.testkit.typed.TestKitSettings
import akka.testkit.typed.scaladsl._ import akka.actor.testkit.typed.scaladsl._
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace

View file

@ -53,7 +53,7 @@ object InstanceCountingExtension extends ExtensionId[DummyExtension1] {
object ExtensionsSpec { object ExtensionsSpec {
val config = ConfigFactory.parseString( val config = ConfigFactory.parseString(
""" """
akka.typed { akka.actor.typed {
library-extensions += "akka.actor.typed.InstanceCountingExtension" library-extensions += "akka.actor.typed.InstanceCountingExtension"
} }
""").resolve() """).resolve()
@ -97,7 +97,7 @@ class ExtensionsSpec extends TypedAkkaSpec {
"load extensions from the configuration" in "load extensions from the configuration" in
withEmptyActorSystem("ExtensionsSpec03", Some(ConfigFactory.parseString( withEmptyActorSystem("ExtensionsSpec03", Some(ConfigFactory.parseString(
""" """
akka.typed.extensions = ["akka.actor.typed.DummyExtension1$", "akka.actor.typed.SlowExtension$"] akka.actor.typed.extensions = ["akka.actor.typed.DummyExtension1$", "akka.actor.typed.SlowExtension$"]
""")) """))
) { system ) { system
system.hasExtension(DummyExtension1) should ===(true) system.hasExtension(DummyExtension1) should ===(true)
@ -111,7 +111,7 @@ class ExtensionsSpec extends TypedAkkaSpec {
def create(): Unit = { def create(): Unit = {
ActorSystem[Any](Behavior.EmptyBehavior, "ExtensionsSpec04", ConfigFactory.parseString( ActorSystem[Any](Behavior.EmptyBehavior, "ExtensionsSpec04", ConfigFactory.parseString(
""" """
akka.typed.extensions = ["akka.actor.typed.FailingToLoadExtension$"] akka.actor.typed.extensions = ["akka.actor.typed.FailingToLoadExtension$"]
""")) """))
} }
@ -136,7 +136,7 @@ class ExtensionsSpec extends TypedAkkaSpec {
"allow for auto-loading of library-extensions" in "allow for auto-loading of library-extensions" in
withEmptyActorSystem("ExtensionsSpec06") { system withEmptyActorSystem("ExtensionsSpec06") { system
val listedExtensions = system.settings.config.getStringList("akka.typed.library-extensions") val listedExtensions = system.settings.config.getStringList("akka.actor.typed.library-extensions")
listedExtensions.size should be > 0 listedExtensions.size should be > 0
// could be initalized by other tests, so at least once // could be initalized by other tests, so at least once
InstanceCountingExtension.createCount.get() should be > 0 InstanceCountingExtension.createCount.get() should be > 0
@ -146,7 +146,7 @@ class ExtensionsSpec extends TypedAkkaSpec {
intercept[RuntimeException] { intercept[RuntimeException] {
withEmptyActorSystem( withEmptyActorSystem(
"ExtensionsSpec07", "ExtensionsSpec07",
Some(ConfigFactory.parseString("""akka.typed.library-extensions += "akka.actor.typed.FailingToLoadExtension$"""")) Some(ConfigFactory.parseString("""akka.actor.typed.library-extensions += "akka.actor.typed.FailingToLoadExtension$""""))
) { _ () } ) { _ () }
} }
@ -154,7 +154,7 @@ class ExtensionsSpec extends TypedAkkaSpec {
intercept[RuntimeException] { intercept[RuntimeException] {
withEmptyActorSystem( withEmptyActorSystem(
"ExtensionsSpec08", "ExtensionsSpec08",
Some(ConfigFactory.parseString("""akka.typed.library-extensions += "akka.actor.typed.MissingExtension"""")) Some(ConfigFactory.parseString("""akka.actor.typed.library-extensions += "akka.actor.typed.MissingExtension""""))
) { _ () } ) { _ () }
} }
@ -202,7 +202,7 @@ class ExtensionsSpec extends TypedAkkaSpec {
"override extensions via ActorSystemSetup" in "override extensions via ActorSystemSetup" in
withEmptyActorSystem("ExtensionsSpec10", Some(ConfigFactory.parseString( withEmptyActorSystem("ExtensionsSpec10", Some(ConfigFactory.parseString(
""" """
akka.typed.extensions = ["akka.actor.typed.DummyExtension1$", "akka.actor.typed.SlowExtension$"] akka.actor.typed.extensions = ["akka.actor.typed.DummyExtension1$", "akka.actor.typed.SlowExtension$"]
""")), """)),
Some(ActorSystemSetup(new DummyExtension1Setup(sys new DummyExtension1ViaSetup))) Some(ActorSystemSetup(new DummyExtension1Setup(sys new DummyExtension1ViaSetup)))
) { system ) { system

View file

@ -7,8 +7,8 @@ package akka.actor.typed
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.typed.TestKitSettings import akka.actor.testkit.typed.TestKitSettings
import akka.testkit.typed.scaladsl._ import akka.actor.testkit.typed.scaladsl._
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
object OrElseSpec { object OrElseSpec {

View file

@ -11,8 +11,8 @@ import akka.actor.ActorInitializationException
import akka.actor.typed.scaladsl.{ Behaviors, MutableBehavior } import akka.actor.typed.scaladsl.{ Behaviors, MutableBehavior }
import akka.actor.typed.scaladsl.Behaviors._ import akka.actor.typed.scaladsl.Behaviors._
import akka.testkit.EventFilter import akka.testkit.EventFilter
import akka.testkit.typed.scaladsl._ import akka.actor.testkit.typed.scaladsl._
import akka.testkit.typed._ import akka.actor.testkit.typed._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.scalatest.{ Matchers, WordSpec } import org.scalatest.{ Matchers, WordSpec }

View file

@ -13,7 +13,7 @@ import scala.util.control.NoStackTrace
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.TimerScheduler import akka.actor.typed.scaladsl.TimerScheduler
import akka.testkit.TimingTest import akka.testkit.TimingTest
import akka.testkit.typed.scaladsl.{ ActorTestKit, _ } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, _ }
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
class TimerSpec extends ActorTestKit with WordSpecLike with TypedAkkaSpecWithShutdown { class TimerSpec extends ActorTestKit with WordSpecLike with TypedAkkaSpecWithShutdown {

View file

@ -4,7 +4,7 @@
package akka.actor.typed package akka.actor.typed
import akka.testkit.typed.scaladsl.{ TestInbox, ActorTestKit } import akka.actor.testkit.typed.scaladsl.{ TestInbox, ActorTestKit }
import org.scalactic.TypeCheckedTripleEquals import org.scalactic.TypeCheckedTripleEquals
import org.scalatest.concurrent.{ Eventually, ScalaFutures } import org.scalatest.concurrent.{ Eventually, ScalaFutures }
import org.scalatest.time.Span import org.scalatest.time.Span

View file

@ -9,7 +9,7 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.MutableBehavior import akka.actor.typed.scaladsl.MutableBehavior
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import akka.testkit.EventFilter import akka.testkit.EventFilter
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import scala.concurrent._ import scala.concurrent._
import scala.concurrent.duration._ import scala.concurrent.duration._

View file

@ -8,7 +8,7 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.{ ActorRef, TypedAkkaSpecWithShutdown } import akka.actor.typed.{ ActorRef, TypedAkkaSpecWithShutdown }
import akka.serialization.{ JavaSerializer, SerializationExtension } import akka.serialization.{ JavaSerializer, SerializationExtension }
import akka.testkit.typed.scaladsl.ActorTestKit import akka.actor.testkit.typed.scaladsl.ActorTestKit
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
object ActorRefSerializationSpec { object ActorRefSerializationSpec {

View file

@ -8,7 +8,7 @@ package internal
import akka.Done import akka.Done
import akka.actor.InvalidMessageException import akka.actor.InvalidMessageException
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.typed.scaladsl.TestInbox import akka.actor.testkit.typed.scaladsl.TestInbox
import org.scalatest._ import org.scalatest._
import org.scalatest.concurrent.{ Eventually, ScalaFutures } import org.scalatest.concurrent.{ Eventually, ScalaFutures }

View file

@ -9,8 +9,8 @@ import akka.actor.typed.receptionist.Receptionist._
import akka.actor.typed.receptionist.{ Receptionist, ServiceKey } import akka.actor.typed.receptionist.{ Receptionist, ServiceKey }
import akka.actor.typed.scaladsl.AskPattern._ import akka.actor.typed.scaladsl.AskPattern._
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.typed.TestKitSettings import akka.actor.testkit.typed.TestKitSettings
import akka.testkit.typed.scaladsl.{ ActorTestKit, BehaviorTestKit, TestInbox, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, BehaviorTestKit, TestInbox, TestProbe }
import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually
import scala.concurrent.Future import scala.concurrent.Future

View file

@ -9,7 +9,7 @@ import akka.actor.typed.internal.ActorRefSerializationSpec
import akka.actor.typed.receptionist.ServiceKey import akka.actor.typed.receptionist.ServiceKey
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import akka.serialization.SerializationExtension import akka.serialization.SerializationExtension
import akka.testkit.typed.scaladsl.ActorTestKit import akka.actor.testkit.typed.scaladsl.ActorTestKit
class ServiceKeySerializationSpec extends ActorTestKit with TypedAkkaSpecWithShutdown { class ServiceKeySerializationSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {

View file

@ -7,7 +7,7 @@ package akka.actor.typed.scaladsl
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.{ ActorRef, PostStop, Props, TypedAkkaSpecWithShutdown } import akka.actor.typed.{ ActorRef, PostStop, Props, TypedAkkaSpecWithShutdown }
import akka.testkit.EventFilter import akka.testkit.EventFilter
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.concurrent.TimeoutException import scala.concurrent.TimeoutException

View file

@ -11,7 +11,7 @@ import akka.actor.typed.{ Behavior, LogMarker, TestException, TypedAkkaSpec }
import akka.event.Logging import akka.event.Logging
import akka.event.Logging.{ LogEventWithCause, LogEventWithMarker } import akka.event.Logging.{ LogEventWithCause, LogEventWithMarker }
import akka.testkit.EventFilter import akka.testkit.EventFilter
import akka.testkit.typed.scaladsl.ActorTestKit import akka.actor.testkit.typed.scaladsl.ActorTestKit
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
class ActorLoggingSpec extends ActorTestKit with TypedAkkaSpec { class ActorLoggingSpec extends ActorTestKit with TypedAkkaSpec {

View file

@ -7,7 +7,7 @@ package scaladsl
import akka.Done import akka.Done
import akka.NotUsed import akka.NotUsed
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
final class GracefulStopSpec extends ActorTestKit with TypedAkkaSpecWithShutdown { final class GracefulStopSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {

View file

@ -11,7 +11,7 @@ import akka.actor.typed.Props
import akka.actor.typed.TestException import akka.actor.typed.TestException
import akka.actor.typed.TypedAkkaSpecWithShutdown import akka.actor.typed.TypedAkkaSpecWithShutdown
import akka.testkit.EventFilter import akka.testkit.EventFilter
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
object MessageAdapterSpec { object MessageAdapterSpec {

View file

@ -6,7 +6,7 @@ package akka.actor.typed
package scaladsl package scaladsl
import akka.Done import akka.Done
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
final class OnSignalSpec extends ActorTestKit with TypedAkkaSpecWithShutdown { final class OnSignalSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {

View file

@ -5,7 +5,7 @@
package akka.actor.typed package akka.actor.typed
package scaladsl package scaladsl
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
class ReceivePartialSpec extends ActorTestKit with TypedAkkaSpecWithShutdown { class ReceivePartialSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {

View file

@ -5,8 +5,8 @@
package akka.actor.typed.scaladsl package akka.actor.typed.scaladsl
import akka.actor.typed.Behavior import akka.actor.typed.Behavior
import akka.testkit.typed.internal.StubbedActorContext import akka.actor.testkit.typed.internal.StubbedActorContext
import akka.testkit.typed.scaladsl.TestInbox import akka.actor.testkit.typed.scaladsl.TestInbox
import org.scalatest.{ Matchers, WordSpec } import org.scalatest.{ Matchers, WordSpec }
class StashBufferSpec extends WordSpec with Matchers { class StashBufferSpec extends WordSpec with Matchers {

View file

@ -5,7 +5,7 @@
package akka.actor.typed package akka.actor.typed
package scaladsl package scaladsl
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
object StashSpec { object StashSpec {
sealed trait Command sealed trait Command

View file

@ -6,7 +6,7 @@ package akka.actor.typed.scaladsl
import akka.Done import akka.Done
import akka.actor.typed.{ PostStop, TypedAkkaSpecWithShutdown } import akka.actor.typed.{ PostStop, TypedAkkaSpecWithShutdown }
import akka.testkit.typed.scaladsl.ActorTestKit import akka.actor.testkit.typed.scaladsl.ActorTestKit
import scala.concurrent.Promise import scala.concurrent.Promise

View file

@ -6,7 +6,7 @@ package docs.akka.typed
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ DeathPactException, SupervisorStrategy, TypedAkkaSpecWithShutdown } import akka.actor.typed.{ DeathPactException, SupervisorStrategy, TypedAkkaSpecWithShutdown }
import akka.testkit.typed.scaladsl.ActorTestKit import akka.actor.testkit.typed.scaladsl.ActorTestKit
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
class FaultToleranceDocSpec extends ActorTestKit with TypedAkkaSpecWithShutdown { class FaultToleranceDocSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {

View file

@ -7,7 +7,7 @@ package docs.akka.typed
//#imports //#imports
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorSystem, Logger, PostStop } import akka.actor.typed.{ ActorSystem, Logger, PostStop }
import akka.testkit.typed.scaladsl.ActorTestKit import akka.actor.testkit.typed.scaladsl.ActorTestKit
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._

View file

@ -9,7 +9,7 @@ import java.net.URI
import akka.NotUsed import akka.NotUsed
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, TypedAkkaSpecWithShutdown } import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, TypedAkkaSpecWithShutdown }
import akka.actor.typed.scaladsl.{ Behaviors, TimerScheduler } import akka.actor.typed.scaladsl.{ Behaviors, TimerScheduler }
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import akka.util.Timeout import akka.util.Timeout
import scala.concurrent.Future import scala.concurrent.Future

View file

@ -10,7 +10,7 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Terminated } import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Terminated }
//#imports //#imports
import akka.testkit.typed.scaladsl.ActorTestKit import akka.actor.testkit.typed.scaladsl.ActorTestKit
import java.net.URLEncoder import java.net.URLEncoder
import java.nio.charset.StandardCharsets import java.nio.charset.StandardCharsets
import scala.concurrent.Await import scala.concurrent.Await

View file

@ -10,7 +10,7 @@ import java.nio.charset.StandardCharsets
import akka.actor.typed._ import akka.actor.typed._
import akka.actor.typed.scaladsl.{ ActorContext, Behaviors, MutableBehavior } import akka.actor.typed.scaladsl.{ ActorContext, Behaviors, MutableBehavior }
import akka.testkit.typed.scaladsl.ActorTestKit import akka.actor.testkit.typed.scaladsl.ActorTestKit
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.Await import scala.concurrent.Await

View file

@ -11,7 +11,7 @@ import akka.Done
import akka.actor.typed.ActorRef import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior import akka.actor.typed.Behavior
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.typed.scaladsl.{ BehaviorTestKit, TestInbox } import akka.actor.testkit.typed.scaladsl.{ BehaviorTestKit, TestInbox }
import org.scalatest.Matchers import org.scalatest.Matchers
import org.scalatest.WordSpec import org.scalatest.WordSpec

View file

@ -1,6 +1,6 @@
akka.typed { akka.actor.typed {
# List FQCN of `akka.typed.ExtensionId`s which shall be loaded at actor system startup. # List FQCN of `akka.actor.typed.ExtensionId`s which shall be loaded at actor system startup.
# Should be on the format: 'extensions = ["com.example.MyExtId1", "com.example.MyExtId2"]' etc. # Should be on the format: 'extensions = ["com.example.MyExtId1", "com.example.MyExtId2"]' etc.
# See the Akka Documentation for more info about Extensions # See the Akka Documentation for more info about Extensions
extensions = [] extensions = []
@ -13,7 +13,7 @@ akka.typed {
# #
# Should not be set by end user applications in 'application.conf', use the extensions property for that # Should not be set by end user applications in 'application.conf', use the extensions property for that
# #
library-extensions = ${?akka.typed.library-extensions} [] library-extensions = ${?akka.actor.typed.library-extensions} []
# Receptionist is started eagerly to allow clustered receptionist to gather remote registrations early on. # Receptionist is started eagerly to allow clustered receptionist to gather remote registrations early on.
library-extensions += "akka.actor.typed.receptionist.Receptionist" library-extensions += "akka.actor.typed.receptionist.Receptionist"

View file

@ -46,7 +46,7 @@ trait Extension
* // can be loaded eagerly on system startup through configuration * // can be loaded eagerly on system startup through configuration
* // note that the name is the JVM/Java class name, with a dollar sign in the end * // note that the name is the JVM/Java class name, with a dollar sign in the end
* // and not the Scala object name * // and not the Scala object name
* akka.typed.extensions = ["com.example.MyExt$"] * akka.actor.typed.extensions = ["com.example.MyExt$"]
* *
* // Allows access like this from Scala * // Allows access like this from Scala
* MyExt().someMethodOnTheExtension() * MyExt().someMethodOnTheExtension()
@ -57,7 +57,7 @@ trait Extension
* *Java API* * *Java API*
* *
* To implement an extension in Java you should first create an `ExtensionId` singleton by implementing a static method * To implement an extension in Java you should first create an `ExtensionId` singleton by implementing a static method
* called `getInstance`, this is needed to be able to list the extension among the `akka.typed.extensions` in the configuration * called `getInstance`, this is needed to be able to list the extension among the `akka.actor.typed.extensions` in the configuration
* and have it loaded when the actor system starts up. * and have it loaded when the actor system starts up.
* *
* {{{ * {{{
@ -90,7 +90,7 @@ trait Extension
* } * }
* *
* // can be loaded eagerly on system startup through configuration * // can be loaded eagerly on system startup through configuration
* akka.typed.extensions = ["com.example.MyExt"] * akka.actor.typed.extensions = ["com.example.MyExt"]
* *
* // Allows access like this from Scala * // Allows access like this from Scala
* MyExt.someMethodOnTheExtension() * MyExt.someMethodOnTheExtension()

View file

@ -62,8 +62,8 @@ trait ExtensionsImpl extends Extensions { self: ActorSystem[_] ⇒
} }
} }
loadExtensions("akka.typed.library-extensions", throwOnLoadFail = true) loadExtensions("akka.actor.typed.library-extensions", throwOnLoadFail = true)
loadExtensions("akka.typed.extensions", throwOnLoadFail = false) loadExtensions("akka.actor.typed.extensions", throwOnLoadFail = false)
} }
final override def hasExtension(ext: ExtensionId[_ <: Extension]): Boolean = findExtension(ext) != null final override def hasExtension(ext: ExtensionId[_ <: Extension]): Boolean = findExtension(ext) != null

View file

@ -9,7 +9,7 @@ import akka.cluster.sharding.typed.scaladsl.EntityTypeKey
import akka.cluster.sharding.typed.scaladsl.ClusterSharding import akka.cluster.sharding.typed.scaladsl.ClusterSharding
import akka.cluster.typed.{ MultiDcClusterActors, MultiNodeTypedClusterSpec } import akka.cluster.typed.{ MultiDcClusterActors, MultiNodeTypedClusterSpec }
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec } import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
import akka.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.util.Timeout import akka.util.Timeout
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.scalatest.concurrent.ScalaFutures import org.scalatest.concurrent.ScalaFutures

View file

@ -8,7 +8,7 @@ import akka.actor.typed.TypedAkkaSpecWithShutdown
import akka.actor.typed.internal.adapter.ActorSystemAdapter import akka.actor.typed.internal.adapter.ActorSystemAdapter
import akka.cluster.sharding.typed.internal.ShardingSerializer import akka.cluster.sharding.typed.internal.ShardingSerializer
import akka.serialization.SerializationExtension import akka.serialization.SerializationExtension
import akka.testkit.typed.scaladsl.ActorTestKit import akka.actor.testkit.typed.scaladsl.ActorTestKit
class ShardingSerializerSpec extends ActorTestKit with TypedAkkaSpecWithShutdown { class ShardingSerializerSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {

View file

@ -12,7 +12,7 @@ import akka.cluster.sharding.typed.ClusterShardingSettings
import akka.cluster.typed.Cluster import akka.cluster.typed.Cluster
import akka.cluster.typed.Join import akka.cluster.typed.Join
import akka.persistence.typed.scaladsl.{ Effect, PersistentBehaviors } import akka.persistence.typed.scaladsl.{ Effect, PersistentBehaviors }
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
object ClusterShardingPersistenceSpec { object ClusterShardingPersistenceSpec {

View file

@ -21,7 +21,7 @@ import akka.cluster.typed.Cluster
import akka.cluster.typed.Join import akka.cluster.typed.Join
import akka.cluster.typed.Leave import akka.cluster.typed.Leave
import akka.serialization.SerializerWithStringManifest import akka.serialization.SerializerWithStringManifest
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import akka.util.Timeout import akka.util.Timeout
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.scalatest.time.Span import org.scalatest.time.Span

View file

@ -9,7 +9,7 @@ import akka.actor.typed.scaladsl.adapter._
import akka.cluster.MemberStatus import akka.cluster.MemberStatus
import akka.remote.testconductor.RoleName import akka.remote.testconductor.RoleName
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec } import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
import akka.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._ import scala.concurrent.duration._

View file

@ -6,7 +6,7 @@ package akka.cluster.typed;
import akka.cluster.ClusterEvent; import akka.cluster.ClusterEvent;
import akka.actor.typed.ActorSystem; import akka.actor.typed.ActorSystem;
import akka.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import com.typesafe.config.Config; import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory; import com.typesafe.config.ConfigFactory;
import org.junit.Test; import org.junit.Test;
@ -62,4 +62,4 @@ public class ClusterApiTest extends JUnitSuite {
} }
} }

View file

@ -10,7 +10,7 @@ import akka.actor.typed.javadsl.*;
import akka.cluster.ClusterEvent; import akka.cluster.ClusterEvent;
import akka.cluster.typed.*; import akka.cluster.typed.*;
//#cluster-imports //#cluster-imports
import akka.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import docs.akka.cluster.typed.BasicClusterManualSpec; import docs.akka.cluster.typed.BasicClusterManualSpec;
// FIXME these tests are awaiting typed Java testkit to be able to await cluster forming like in BasicClusterExampleSpec // FIXME these tests are awaiting typed Java testkit to be able to await cluster forming like in BasicClusterExampleSpec

View file

@ -12,8 +12,8 @@ import akka.actor.typed.scaladsl.adapter._
import akka.cluster.Cluster import akka.cluster.Cluster
import akka.cluster.ddata.typed.scaladsl.Replicator._ import akka.cluster.ddata.typed.scaladsl.Replicator._
import akka.cluster.ddata.{ GCounter, GCounterKey, ReplicatedData } import akka.cluster.ddata.{ GCounter, GCounterKey, ReplicatedData }
import akka.testkit.typed.scaladsl._ import akka.actor.testkit.typed.scaladsl._
import akka.testkit.typed.TestKitSettings import akka.actor.testkit.typed.TestKitSettings
import akka.util.Timeout import akka.util.Timeout
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually

View file

@ -7,7 +7,7 @@ package akka.cluster.typed
import akka.actor.InvalidMessageException import akka.actor.InvalidMessageException
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, PostStop, Terminated } import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, PostStop, Terminated }
import akka.testkit.typed.scaladsl.TestInbox import akka.actor.testkit.typed.scaladsl.TestInbox
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.scalatest._ import org.scalatest._
import org.scalatest.concurrent.{ Eventually, ScalaFutures } import org.scalatest.concurrent.{ Eventually, ScalaFutures }

View file

@ -8,8 +8,8 @@ import akka.actor.typed.TypedAkkaSpecWithShutdown
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import akka.cluster.ClusterEvent._ import akka.cluster.ClusterEvent._
import akka.cluster.MemberStatus import akka.cluster.MemberStatus
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import akka.testkit.typed.TestKitSettings import akka.actor.testkit.typed.TestKitSettings
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.scalatest.concurrent.ScalaFutures import org.scalatest.concurrent.ScalaFutures
@ -31,7 +31,7 @@ object ClusterApiSpec {
allow-java-serialization = off allow-java-serialization = off
} }
# generous timeout for cluster forming probes # generous timeout for cluster forming probes
akka.actor.typed.test.default-timeout = 10s akka.actor.testkit.typed.default-timeout = 10s
""") """)
} }

View file

@ -9,8 +9,8 @@ import java.nio.charset.StandardCharsets
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import akka.testkit.typed.TestKitSettings import akka.actor.testkit.typed.TestKitSettings
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import akka.actor.typed.{ ActorRef, ActorRefResolver, Props, TypedAkkaSpecWithShutdown } import akka.actor.typed.{ ActorRef, ActorRefResolver, Props, TypedAkkaSpecWithShutdown }
import akka.serialization.SerializerWithStringManifest import akka.serialization.SerializerWithStringManifest
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory

View file

@ -6,7 +6,7 @@ package akka.cluster.typed
import akka.actor.typed.{ ActorRef, Behavior, Props, TypedAkkaSpecWithShutdown } import akka.actor.typed.{ ActorRef, Behavior, Props, TypedAkkaSpecWithShutdown }
import akka.persistence.typed.scaladsl.{ Effect, PersistentBehaviors } import akka.persistence.typed.scaladsl.{ Effect, PersistentBehaviors }
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
object ClusterSingletonPersistenceSpec { object ClusterSingletonPersistenceSpec {

View file

@ -12,7 +12,7 @@ import akka.actor.typed.receptionist.{ Receptionist, ServiceKey }
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorRef, ActorRefResolver, ActorSystem, TypedAkkaSpecWithShutdown } import akka.actor.typed.{ ActorRef, ActorRefResolver, ActorSystem, TypedAkkaSpecWithShutdown }
import akka.serialization.SerializerWithStringManifest import akka.serialization.SerializerWithStringManifest
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import akka.util.Timeout import akka.util.Timeout
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory

View file

@ -7,7 +7,7 @@ package akka.cluster.typed
import akka.actor.typed.ActorSystem import akka.actor.typed.ActorSystem
import akka.actor.typed.TypedAkkaSpecWithShutdown import akka.actor.typed.TypedAkkaSpecWithShutdown
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._ import scala.concurrent.duration._

View file

@ -9,7 +9,7 @@ import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.{ Behavior, TypedAkkaSpecWithShutdown } import akka.actor.typed.{ Behavior, TypedAkkaSpecWithShutdown }
import akka.cluster.typed.internal.receptionist.ClusterReceptionist import akka.cluster.typed.internal.receptionist.ClusterReceptionist
import akka.serialization.SerializationExtension import akka.serialization.SerializationExtension
import akka.testkit.typed.scaladsl.ActorTestKit import akka.actor.testkit.typed.scaladsl.ActorTestKit
class AkkaClusterTypedSerializerSpec extends ActorTestKit with TypedAkkaSpecWithShutdown { class AkkaClusterTypedSerializerSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {

View file

@ -14,8 +14,8 @@ import akka.actor.typed.{ ActorRef, ActorRefResolver }
import akka.cluster.MemberStatus import akka.cluster.MemberStatus
import akka.cluster.typed.{ Cluster, Join } import akka.cluster.typed.{ Cluster, Join }
import akka.serialization.SerializerWithStringManifest import akka.serialization.SerializerWithStringManifest
import akka.testkit.typed.FishingOutcome import akka.actor.testkit.typed.FishingOutcome
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import org.scalatest.{ Matchers, WordSpec } import org.scalatest.{ Matchers, WordSpec }
@ -37,8 +37,6 @@ object ClusterReceptionistSpec {
"akka.cluster.typed.internal.receptionist.ClusterReceptionistSpec$$Ping" = test "akka.cluster.typed.internal.receptionist.ClusterReceptionistSpec$$Ping" = test
"akka.cluster.typed.internal.receptionist.ClusterReceptionistSpec$$Pong$$" = test "akka.cluster.typed.internal.receptionist.ClusterReceptionistSpec$$Pong$$" = test
"akka.cluster.typed.internal.receptionist.ClusterReceptionistSpec$$Perish$$" = test "akka.cluster.typed.internal.receptionist.ClusterReceptionistSpec$$Perish$$" = test
# for now, using Java serializers is good enough (tm), see #23687
# "akka.typed.internal.receptionist.ReceptionistImpl$$DefaultServiceKey" = test
} }
} }
akka.remote.artery.enabled = true akka.remote.artery.enabled = true

View file

@ -14,7 +14,7 @@ import akka.cluster.ClusterEvent._
import akka.cluster.MemberStatus import akka.cluster.MemberStatus
import akka.cluster.typed._ import akka.cluster.typed._
//#cluster-imports //#cluster-imports
import akka.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import org.scalatest.concurrent.{ Eventually, ScalaFutures } import org.scalatest.concurrent.{ Eventually, ScalaFutures }
import org.scalatest.time.{ Millis, Seconds, Span } import org.scalatest.time.{ Millis, Seconds, Span }

View file

@ -84,7 +84,7 @@ Java
@scala[That adds some implicit extension methods that are added to untyped and typed `ActorSystem` and `ActorContext` in both directions.] @scala[That adds some implicit extension methods that are added to untyped and typed `ActorSystem` and `ActorContext` in both directions.]
@java[To convert between typed and untyped there are adapter methods in `akka.typed.javadsl.Adapter`.] Note the inline comments in the example above. @java[To convert between typed and untyped there are adapter methods in `akka.actor.typed.javadsl.Adapter`.] Note the inline comments in the example above.
## Typed to untyped ## Typed to untyped

View file

@ -6,7 +6,7 @@ To use Akka TestKit Typed, add the module to your project:
@@dependency[sbt,Maven,Gradle] { @@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka group=com.typesafe.akka
artifact=akka-testkit-typed_$scala.binary_version$ artifact=akka-actor-testkit-typed_$scala.binary_version$
version=$akka.version$ version=$akka.version$
scope=test scope=test
} }
@ -31,6 +31,17 @@ This module is currently marked as @ref:[may change](../common/may-change.md) in
@@@ @@@
## Dependency
To use Akka TestKit Typed, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
artifact=akka-actor-testkit-typed_$scala.binary_version$
version=$akka.version$
scope=test
}
## Synchronous behavior testing ## Synchronous behavior testing
The following demonstrates how to test: The following demonstrates how to test:
@ -43,27 +54,27 @@ The following demonstrates how to test:
The examples below require the following imports: The examples below require the following imports:
Scala Scala
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #imports } : @@snip [SyncTestingExampleSpec.scala]($akka$/akka-actor-testkit-typed/src/test/scala/akka/actor/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #imports }
Java Java
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #imports } : @@snip [SyncTestingExampleTest.java]($akka$/akka-actor-testkit-typed/src/test/java/akka/actor/testkit/typed/javadsl/SyncTestingExampleTest.java) { #imports }
Each of the tests are testing an actor that based on the message executes a different effect to be tested: Each of the tests are testing an actor that based on the message executes a different effect to be tested:
Scala Scala
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #under-test } : @@snip [SyncTestingExampleSpec.scala]($akka$/akka-actor-testkit-typed/src/test/scala/akka/actor/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #under-test }
Java Java
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #under-test } : @@snip [SyncTestingExampleTest.java]($akka$/akka-actor-testkit-typed/src/test/java/akka/actor/testkit/typed/javadsl/SyncTestingExampleTest.java) { #under-test }
For creating a child actor a noop actor is created: For creating a child actor a noop actor is created:
Scala Scala
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #child } : @@snip [SyncTestingExampleSpec.scala]($akka$/akka-actor-testkit-typed/src/test/scala/akka/actor/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #child }
Java Java
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #child } : @@snip [SyncTestingExampleTest.java]($akka$/akka-actor-testkit-typed/src/test/java/akka/actor/testkit/typed/javadsl/SyncTestingExampleTest.java) { #child }
All of the tests make use of the `BehaviorTestkit` to avoid the need for a real `ActorContext`. Some of the tests All of the tests make use of the `BehaviorTestkit` to avoid the need for a real `ActorContext`. Some of the tests
make use of the `TestInbox` which allows the creation of an `ActorRef` that can be used for synchronous testing, similar to the make use of the `TestInbox` which allows the creation of an `ActorRef` that can be used for synchronous testing, similar to the
@ -75,18 +86,18 @@ make use of the `TestInbox` which allows the creation of an `ActorRef` that can
With a name: With a name:
Scala Scala
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-child } : @@snip [SyncTestingExampleSpec.scala]($akka$/akka-actor-testkit-typed/src/test/scala/akka/actor/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-child }
Java Java
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-child } : @@snip [SyncTestingExampleTest.java]($akka$/akka-actor-testkit-typed/src/test/java/akka/actor/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-child }
Anonymously: Anonymously:
Scala Scala
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-anonymous-child } : @@snip [SyncTestingExampleSpec.scala]($akka$/akka-actor-testkit-typed/src/test/scala/akka/actor/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-anonymous-child }
Java Java
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-anonymous-child } : @@snip [SyncTestingExampleTest.java]($akka$/akka-actor-testkit-typed/src/test/java/akka/actor/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-anonymous-child }
### Sending messages ### Sending messages
@ -94,31 +105,31 @@ For testing sending a message a `TestInbox` is created that provides an `ActorRe
messages that have been sent to it. messages that have been sent to it.
Scala Scala
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-message } : @@snip [SyncTestingExampleSpec.scala]($akka$/akka-actor-testkit-typed/src/test/scala/akka/actor/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-message }
Java Java
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-message } : @@snip [SyncTestingExampleTest.java]($akka$/akka-actor-testkit-typed/src/test/java/akka/actor/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-message }
Another use case is sending a message to a child actor you can do this by looking up the 'TestInbox' for Another use case is sending a message to a child actor you can do this by looking up the 'TestInbox' for
a child actor from the 'BehaviorTestKit': a child actor from the 'BehaviorTestKit':
Scala Scala
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-child-message } : @@snip [SyncTestingExampleSpec.scala]($akka$/akka-actor-testkit-typed/src/test/scala/akka/actor/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-child-message }
Java Java
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-child-message } : @@snip [SyncTestingExampleTest.java]($akka$/akka-actor-testkit-typed/src/test/java/akka/actor/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-child-message }
For anonymous children the actor names are generated in a deterministic way: For anonymous children the actor names are generated in a deterministic way:
Scala Scala
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-child-message-anonymous } : @@snip [SyncTestingExampleSpec.scala]($akka$/akka-actor-testkit-typed/src/test/scala/akka/actor/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-child-message-anonymous }
Java Java
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-child-message-anonymous } : @@snip [SyncTestingExampleTest.java]($akka$/akka-actor-testkit-typed/src/test/java/akka/actor/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-child-message-anonymous }
### Testing other effects ### Testing other effects
The `BehaviorTestkit` keeps track other effects you can verify, look at the sub-classes of `akka.testkit.typed.Effect` The `BehaviorTestkit` keeps track other effects you can verify, look at the sub-classes of `akka.actor.testkit.typed.Effect`
* SpawnedAdapter * SpawnedAdapter
* Stopped * Stopped
@ -142,10 +153,10 @@ the same in that a single procedure drives the test.
Actor under test: Actor under test:
Scala Scala
: @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/AsyncTestingExampleSpec.scala) { #under-test } : @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-actor-testkit-typed/src/test/scala/akka/actor/testkit/typed/scaladsl/AsyncTestingExampleSpec.scala) { #under-test }
Java Java
: @@snip [AsyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/AsyncTestingExampleTest.java) { #under-test } : @@snip [AsyncTestingExampleTest.java]($akka$/akka-actor-testkit-typed/src/test/java/akka/actor/testkit/typed/javadsl/AsyncTestingExampleTest.java) { #under-test }
@scala[Tests extend `ActorTestKit`. This provides access to]@java[Tests create an instance of `ActorTestKit`. This provides access to] @scala[Tests extend `ActorTestKit`. This provides access to]@java[Tests create an instance of `ActorTestKit`. This provides access to]
@ -154,18 +165,18 @@ Java
* A hook to shut down the ActorSystem from the test suite * A hook to shut down the ActorSystem from the test suite
Scala Scala
: @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/AsyncTestingExampleSpec.scala) { #test-header } : @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-actor-testkit-typed/src/test/scala/akka/actor/testkit/typed/scaladsl/AsyncTestingExampleSpec.scala) { #test-header }
Java Java
: @@snip [AsyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/AsyncTestingExampleTest.java) { #test-header } : @@snip [AsyncTestingExampleTest.java]($akka$/akka-actor-testkit-typed/src/test/java/akka/actor/testkit/typed/javadsl/AsyncTestingExampleTest.java) { #test-header }
Your test is responsible for shutting down the `ActorSystem` e.g. using `BeforeAndAfterAll` when using ScalaTest Your test is responsible for shutting down the `ActorSystem` e.g. using `BeforeAndAfterAll` when using ScalaTest
Scala Scala
: @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/AsyncTestingExampleSpec.scala) { #test-shutdown } : @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-actor-testkit-typed/src/test/scala/akka/actor/testkit/typed/scaladsl/AsyncTestingExampleSpec.scala) { #test-shutdown }
Java Java
: @@snip [AsyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/AsyncTestingExampleTest.java) { #test-shutdown } : @@snip [AsyncTestingExampleTest.java]($akka$/akka-actor-testkit-typed/src/test/java/akka/actor/testkit/typed/javadsl/AsyncTestingExampleTest.java) { #test-shutdown }
The following demonstrates: The following demonstrates:
@ -174,24 +185,24 @@ The following demonstrates:
* Verifying that the actor under test responds via the `TestProbe` * Verifying that the actor under test responds via the `TestProbe`
Scala Scala
: @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/AsyncTestingExampleSpec.scala) { #test-spawn } : @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-actor-testkit-typed/src/test/scala/akka/actor/testkit/typed/scaladsl/AsyncTestingExampleSpec.scala) { #test-spawn }
Java Java
: @@snip [AsyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/AsyncTestingExampleTest.java) { #test-spawn } : @@snip [AsyncTestingExampleTest.java]($akka$/akka-actor-testkit-typed/src/test/java/akka/actor/testkit/typed/javadsl/AsyncTestingExampleTest.java) { #test-spawn }
Actors can also be spawned anonymously: Actors can also be spawned anonymously:
Scala Scala
: @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/AsyncTestingExampleSpec.scala) { #test-spawn-anonymous } : @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-actor-testkit-typed/src/test/scala/akka/actor/testkit/typed/scaladsl/AsyncTestingExampleSpec.scala) { #test-spawn-anonymous }
Java Java
: @@snip [AsyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/AsyncTestingExampleTest.java) { #test-spawn-anonymous } : @@snip [AsyncTestingExampleTest.java]($akka$/akka-actor-testkit-typed/src/test/java/akka/actor/testkit/typed/javadsl/AsyncTestingExampleTest.java) { #test-spawn-anonymous }
### Test framework integration ### Test framework integration
@@@ div { .group-java } @@@ div { .group-java }
If you are using JUnit you can use `akka.testkit.typed.javadsl.TestKitJunitResource` to have the async test kit automatically If you are using JUnit you can use `akka.actor.testkit.typed.javadsl.TestKitJunitResource` to have the async test kit automatically
shutdown when the test is complete. shutdown when the test is complete.
Note that the dependency on JUnit is marked as optional from the test kit module, so your project must explicitly include Note that the dependency on JUnit is marked as optional from the test kit module, so your project must explicitly include
@ -207,10 +218,10 @@ hook it into a ScalaTest test suite.
@@@ @@@
Scala Scala
: @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/AbstractActorSpec.scala) { #scalatest-glue } : @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-actor-testkit-typed/src/test/scala/akka/actor/testkit/typed/scaladsl/AbstractActorSpec.scala) { #scalatest-glue }
Java Java
: @@snip [AsyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/JunitIntegrationExampleTest.java) { #junit-integration } : @@snip [AsyncTestingExampleTest.java]($akka$/akka-actor-testkit-typed/src/test/java/akka/actor/testkit/typed/javadsl/JunitIntegrationExampleTest.java) { #junit-integration }
### Controlling the scheduler ### Controlling the scheduler
@ -221,7 +232,7 @@ Making such tests more reliable by using generous timeouts make the tests take a
For such situations, we provide a scheduler where you can manually, explicitly advance the clock. For such situations, we provide a scheduler where you can manually, explicitly advance the clock.
Scala Scala
: @@snip [ManualTimerExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/ManualTimerExampleSpec.scala) { #manual-scheduling-simple } : @@snip [ManualTimerExampleSpec.scala]($akka$/akka-actor-testkit-typed/src/test/scala/akka/actor/testkit/typed/scaladsl/ManualTimerExampleSpec.scala) { #manual-scheduling-simple }
Java Java
: @@snip [ManualTimerExampleTest.scala]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/ManualTimerExampleTest.java) { #manual-scheduling-simple } : @@snip [ManualTimerExampleTest.scala]($akka$/akka-actor-testkit-typed/src/test/java/akka/actor/testkit/typed/javadsl/ManualTimerExampleTest.java) { #manual-scheduling-simple }

View file

@ -6,7 +6,7 @@ package akka.persistence.typed.javadsl;
import akka.actor.Scheduler; import akka.actor.Scheduler;
import akka.actor.typed.ActorRef; import akka.actor.typed.ActorRef;
import akka.testkit.typed.javadsl.TestInbox; import akka.actor.testkit.typed.javadsl.TestInbox;
import akka.util.Timeout; import akka.util.Timeout;
import java.util.*; import java.util.*;

View file

@ -12,8 +12,8 @@ import akka.actor.typed.javadsl.Behaviors;
import akka.japi.Pair; import akka.japi.Pair;
import akka.japi.function.Function3; import akka.japi.function.Function3;
import akka.persistence.typed.scaladsl.PersistentBehaviorSpec; import akka.persistence.typed.scaladsl.PersistentBehaviorSpec;
import akka.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;

View file

@ -10,7 +10,7 @@ import akka.actor.typed.scaladsl.adapter.TypedActorSystemOps
import akka.persistence.typed.scaladsl.PersistentBehaviors.CommandHandler import akka.persistence.typed.scaladsl.PersistentBehaviors.CommandHandler
import akka.persistence.typed.scaladsl.{ Effect, PersistentBehavior, PersistentBehaviors } import akka.persistence.typed.scaladsl.{ Effect, PersistentBehavior, PersistentBehaviors }
import akka.testkit.TestLatch import akka.testkit.TestLatch
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually

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