Rename akka package to org.apache.pekko

This commit is contained in:
Matthew de Detrich 2022-11-12 10:21:24 +01:00 committed by Matthew de Detrich
parent 494d62515e
commit f84e8db3cb
3474 changed files with 75898 additions and 72025 deletions

View file

@ -14,21 +14,21 @@ ignored-files = [
// 1. adated source code
// 2. protobuf generated messages
ignored-packages = [
"akka.dispatch",
"akka.protobuf",
"org.apache.pekko.dispatch",
"org.apache.pekko.protobuf",
"sun.reflect",
"akka.cluster.protobuf.msg",
"akka.cluster.metrics.protobuf.msg",
"akka.cluster.sharding.protobuf.msg",
"akka.cluster.sharding.typed.internal.protobuf",
"akka.cluster.client.protobuf.msg",
"akka.cluster.pubsub.protobuf.msg",
"akka.cluster.typed.internal.protobuf",
"akka.cluster.ddata.protobuf.msg",
"akka.remote.testconductor",
"akka.persistence.serialization",
"org.apache.pekko.cluster.protobuf.msg",
"org.apache.pekko.cluster.metrics.protobuf.msg",
"org.apache.pekko.cluster.sharding.protobuf.msg",
"org.apache.pekko.cluster.sharding.typed.internal.protobuf",
"org.apache.pekko.cluster.client.protobuf.msg",
"org.apache.pekko.cluster.pubsub.protobuf.msg",
"org.apache.pekko.cluster.typed.internal.protobuf",
"org.apache.pekko.cluster.ddata.protobuf.msg",
"org.apache.pekko.remote.testconductor",
"org.apache.pekko.persistence.serialization",
//in tests
"akka.remote.artery.protobuf",
"org.apache.pekko.remote.artery.protobuf",
//in docs
"docs.ddata.protobuf.msg",
"docs.persistence.proto"

View file

@ -46,5 +46,5 @@ SortImports.blocks = [
"scala.",
"*",
"com.sun."
"akka."
"org.apache.pekko."
]

View file

@ -263,7 +263,7 @@ in the file that describes briefly why the incompatibility can be ignored.
Situations when it may be acceptable to ignore a MiMa issued warning include:
- if it is touching any class marked as `private[akka]`, `/** INTERNAL API*/` or similar markers
- if it is touching any class marked as `private[pekko]`, `/** INTERNAL API*/` or similar markers
- if it is concerning internal classes (often recognisable by package names like `dungeon`, `impl`, `internal` etc.)
- if it is adding API to classes / traits which are only meant for extension by Akka itself, i.e. should not be extended by end-users
- other tricky situations
@ -542,7 +542,7 @@ Scala has proven the most viable way to do it, as long as you keep the following
1. Provide `getX` style accessors for values in the Java APIs
1. Place classes not part of the public APIs in a shared `internal` package. This package can contain implementations of
both Java and Scala APIs. Make such classes `private[akka]` and also, since that becomes `public` from Java's point of
both Java and Scala APIs. Make such classes `private[pekko]` and also, since that becomes `public` from Java's point of
view, annotate with `@InternalApi` and add a scaladoc saying `INTERNAL API`
1. Traits that are part of the Java API should only be used to define pure interfaces, as soon as there are implementations of methods, prefer

View file

@ -2,7 +2,7 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed
package org.apache.pekko.actor.testkit.typed
import java.util.Optional
@ -11,13 +11,14 @@ import scala.compat.java8.OptionConverters._
import org.slf4j.Marker
import org.slf4j.event.Level
import akka.annotation.InternalApi
import akka.util.OptionVal
import org.apache.pekko
import pekko.annotation.InternalApi
import pekko.util.OptionVal
/**
* Representation of a Log Event issued by a [[akka.actor.typed.Behavior]]
* when testing with [[akka.actor.testkit.typed.scaladsl.BehaviorTestKit]]
* or [[akka.actor.testkit.typed.javadsl.BehaviorTestKit]].
* Representation of a Log Event issued by a [[pekko.actor.typed.Behavior]]
* when testing with [[pekko.actor.testkit.typed.scaladsl.BehaviorTestKit]]
* or [[pekko.actor.testkit.typed.javadsl.BehaviorTestKit]].
*/
final case class CapturedLogEvent(level: Level, message: String, cause: Option[Throwable], marker: Option[Marker]) {
@ -80,7 +81,7 @@ object CapturedLogEvent {
* INTERNAL API
*/
@InternalApi
private[akka] def apply(
private[pekko] def apply(
level: Level,
message: String,
errorCause: OptionVal[Throwable],

View file

@ -2,27 +2,28 @@
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed
package org.apache.pekko.actor.testkit.typed
import scala.compat.java8.FunctionConverters._
import scala.concurrent.duration.FiniteDuration
import akka.actor.typed.{ ActorRef, Behavior, Props }
import akka.annotation.{ DoNotInherit, InternalApi }
import akka.util.JavaDurationConverters._
import akka.util.unused
import org.apache.pekko
import pekko.actor.typed.{ ActorRef, Behavior, Props }
import pekko.annotation.{ DoNotInherit, InternalApi }
import pekko.util.JavaDurationConverters._
import pekko.util.unused
/**
* All tracked effects for the [[akka.actor.testkit.typed.scaladsl.BehaviorTestKit]] and
* [[akka.actor.testkit.typed.javadsl.BehaviorTestKit]] must extend this type.
* All tracked effects for the [[pekko.actor.testkit.typed.scaladsl.BehaviorTestKit]] and
* [[pekko.actor.testkit.typed.javadsl.BehaviorTestKit]] must extend this type.
*
* Factories/types for effects are available through [[akka.actor.testkit.typed.javadsl.Effects]]
* and [[akka.actor.testkit.typed.javadsl.Effects]]
* Factories/types for effects are available through [[pekko.actor.testkit.typed.javadsl.Effects]]
* and [[pekko.actor.testkit.typed.javadsl.Effects]]
*
* Not for user extension
*/
@DoNotInherit
abstract class Effect private[akka] ()
abstract class Effect private[pekko] ()
object Effect {
@ -86,10 +87,10 @@ object Effect {
/**
* INTERNAL API
* Spawning adapters is private[akka]
* Spawning adapters is private[pekko]
*/
@InternalApi
private[akka] final class SpawnedAdapter[T](val name: String, val ref: ActorRef[T])
private[pekko] final class SpawnedAdapter[T](val name: String, val ref: ActorRef[T])
extends Effect
with Product1[String]
with Serializable {
@ -108,10 +109,10 @@ object Effect {
/**
* INTERNAL API
* Spawning adapters is private[akka]
* Spawning adapters is private[pekko]
*/
@InternalApi
private[akka] object SpawnedAdapter {
private[pekko] object SpawnedAdapter {
def apply[T](name: String): SpawnedAdapter[T] = new SpawnedAdapter(name, null)
def unapply[T](s: SpawnedAdapter[T]): Option[Tuple1[String]] = Some(Tuple1(s.name))
}
@ -121,7 +122,7 @@ object Effect {
* The behavior spawned an anonymous adapter, through `context.spawnMessageAdapter`
*/
@InternalApi
private[akka] final class SpawnedAnonymousAdapter[T](val ref: ActorRef[T])
private[pekko] final class SpawnedAnonymousAdapter[T](val ref: ActorRef[T])
extends Effect
with Product
with Serializable {
@ -144,7 +145,7 @@ object Effect {
* INTERNAL API
*/
@InternalApi
private[akka] object SpawnedAnonymousAdapter {
private[pekko] object SpawnedAnonymousAdapter {
def apply[T]() = new SpawnedAnonymousAdapter[T](null)
def unapply[T](@unused s: SpawnedAnonymousAdapter[T]): Boolean = true
}
@ -214,7 +215,7 @@ object Effect {
}
object TimerScheduled {
import akka.util.JavaDurationConverters._
import pekko.util.JavaDurationConverters._
sealed trait TimerMode
case object FixedRateMode extends TimerMode

View file

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

View file

@ -2,7 +2,7 @@
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed
package org.apache.pekko.actor.testkit.typed
import java.util.Optional
@ -11,7 +11,8 @@ import scala.compat.java8.OptionConverters._
import org.slf4j.Marker
import org.slf4j.event.Level
import akka.util.ccompat.JavaConverters._
import org.apache.pekko
import pekko.util.ccompat.JavaConverters._
object LoggingEvent {
@ -43,8 +44,8 @@ object LoggingEvent {
}
/**
* Representation of logging event when testing with [[akka.actor.testkit.typed.scaladsl.LoggingTestKit]]
* or [[akka.actor.testkit.typed.javadsl.LoggingTestKit]].
* Representation of logging event when testing with [[pekko.actor.testkit.typed.scaladsl.LoggingTestKit]]
* or [[pekko.actor.testkit.typed.javadsl.LoggingTestKit]].
*/
final case class LoggingEvent(
level: Level,
@ -72,7 +73,7 @@ final case class LoggingEvent(
* Java API
*/
def getMdc: java.util.Map[String, String] = {
import akka.util.ccompat.JavaConverters._
import pekko.util.ccompat.JavaConverters._
mdc.asJava
}

View file

@ -2,7 +2,7 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed
package org.apache.pekko.actor.testkit.typed
import scala.util.control.NoStackTrace

View file

@ -2,17 +2,18 @@
* Copyright (C) 2017-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed
package org.apache.pekko.actor.testkit.typed
import scala.concurrent.duration.{ Duration, FiniteDuration }
import com.typesafe.config.Config
import akka.actor.typed.ActorSystem
import akka.actor.typed.Extension
import akka.actor.typed.ExtensionId
import akka.util.JavaDurationConverters._
import akka.util.Timeout
import org.apache.pekko
import pekko.actor.typed.ActorSystem
import pekko.actor.typed.Extension
import pekko.actor.typed.ExtensionId
import pekko.util.JavaDurationConverters._
import pekko.util.Timeout
object TestKitSettings {
@ -54,7 +55,7 @@ object TestKitSettings {
final class TestKitSettings(val config: Config) {
import akka.util.Helpers._
import pekko.util.Helpers._
val TestTimeFactor: Double = config
.getDouble("timefactor")

View file

@ -2,7 +2,7 @@
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
package org.apache.pekko.actor.testkit.typed.internal
import java.util.concurrent.{ CompletionStage, ThreadFactory }
import scala.compat.java8.FutureConverters
@ -11,29 +11,30 @@ import scala.annotation.nowarn
import com.typesafe.config.{ Config, ConfigFactory }
import org.slf4j.Logger
import org.slf4j.LoggerFactory
import akka.{ actor => classic }
import akka.Done
import akka.actor.{ ActorPath, ActorRefProvider, Address, ReflectiveDynamicAccess }
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.DispatcherSelector
import akka.actor.typed.Dispatchers
import akka.actor.typed.Extension
import akka.actor.typed.ExtensionId
import akka.actor.typed.Props
import akka.actor.typed.Scheduler
import akka.actor.typed.Settings
import akka.actor.typed.internal.ActorRefImpl
import akka.actor.typed.internal.InternalRecipientRef
import akka.actor.typed.receptionist.Receptionist
import akka.annotation.InternalApi
import org.apache.pekko
import pekko.{ actor => classic }
import pekko.Done
import pekko.actor.{ ActorPath, ActorRefProvider, Address, ReflectiveDynamicAccess }
import pekko.actor.typed.ActorRef
import pekko.actor.typed.ActorSystem
import pekko.actor.typed.Behavior
import pekko.actor.typed.DispatcherSelector
import pekko.actor.typed.Dispatchers
import pekko.actor.typed.Extension
import pekko.actor.typed.ExtensionId
import pekko.actor.typed.Props
import pekko.actor.typed.Scheduler
import pekko.actor.typed.Settings
import pekko.actor.typed.internal.ActorRefImpl
import pekko.actor.typed.internal.InternalRecipientRef
import pekko.actor.typed.receptionist.Receptionist
import pekko.annotation.InternalApi
/**
* INTERNAL API
*/
@nowarn
@InternalApi private[akka] final class ActorSystemStub(
@InternalApi private[pekko] final class ActorSystemStub(
val name: String,
config: Config = ActorSystemStub.config.defaultReference)
extends ActorSystem[Nothing]
@ -60,7 +61,7 @@ import akka.annotation.InternalApi
// impl ActorRefImpl
override def isLocal: Boolean = true
// impl ActorRefImpl
override def sendSystem(signal: akka.actor.typed.internal.SystemMessage): Unit =
override def sendSystem(signal: pekko.actor.typed.internal.SystemMessage): Unit =
throw new UnsupportedOperationException("must not send SYSTEM message to ActorSystemStub")
// impl InternalRecipientRef, ask not supported
@ -84,7 +85,7 @@ import akka.annotation.InternalApi
val controlledExecutor = new ControlledExecutor
implicit override def executionContext: scala.concurrent.ExecutionContextExecutor = controlledExecutor
override def dispatchers: akka.actor.typed.Dispatchers = new Dispatchers {
override def dispatchers: pekko.actor.typed.Dispatchers = new Dispatchers {
def lookup(selector: DispatcherSelector): ExecutionContextExecutor = controlledExecutor
def shutdown(): Unit = ()
}
@ -125,7 +126,7 @@ import akka.annotation.InternalApi
def address: Address = rootPath.address
}
@InternalApi private[akka] object ActorSystemStub {
@InternalApi private[pekko] object ActorSystemStub {
object config {
// this is backward compatible with the old behaviour, hence it uses the loader used to load the test-kit
// which is not necessarily the one used to load the tests...

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
package org.apache.pekko.actor.testkit.typed.internal
import java.util
@ -11,32 +11,33 @@ import scala.collection.immutable
import scala.reflect.ClassTag
import scala.util.control.Exception.Catcher
import scala.util.control.NonFatal
import akka.actor.ActorPath
import akka.actor.testkit.typed.{ CapturedLogEvent, Effect }
import akka.actor.testkit.typed.Effect._
import akka.actor.typed.internal.AdaptWithRegisteredMessageAdapter
import akka.actor.typed.{ ActorRef, Behavior, BehaviorInterceptor, PostStop, Signal, TypedActorContext }
import akka.actor.typed.receptionist.Receptionist
import akka.actor.typed.scaladsl.Behaviors
import akka.annotation.InternalApi
import akka.util.ccompat.JavaConverters._
import org.apache.pekko
import pekko.actor.ActorPath
import pekko.actor.testkit.typed.{ CapturedLogEvent, Effect }
import pekko.actor.testkit.typed.Effect._
import pekko.actor.typed.internal.AdaptWithRegisteredMessageAdapter
import pekko.actor.typed.{ ActorRef, Behavior, BehaviorInterceptor, PostStop, Signal, TypedActorContext }
import pekko.actor.typed.receptionist.Receptionist
import pekko.actor.typed.scaladsl.Behaviors
import pekko.annotation.InternalApi
import pekko.util.ccompat.JavaConverters._
/**
* INTERNAL API
*/
@InternalApi
private[akka] final class BehaviorTestKitImpl[T](
private[pekko] final class BehaviorTestKitImpl[T](
system: ActorSystemStub,
_path: ActorPath,
_initialBehavior: Behavior[T])
extends akka.actor.testkit.typed.javadsl.BehaviorTestKit[T]
with akka.actor.testkit.typed.scaladsl.BehaviorTestKit[T] {
extends pekko.actor.testkit.typed.javadsl.BehaviorTestKit[T]
with pekko.actor.testkit.typed.scaladsl.BehaviorTestKit[T] {
// really this should be private, make so when we port out tests that need it
private[akka] val context: EffectfulActorContext[T] =
private[pekko] val context: EffectfulActorContext[T] =
new EffectfulActorContext[T](system, _path, () => currentBehavior)
private[akka] def as[U]: BehaviorTestKitImpl[U] = this.asInstanceOf[BehaviorTestKitImpl[U]]
private[pekko] def as[U]: BehaviorTestKitImpl[U] = this.asInstanceOf[BehaviorTestKitImpl[U]]
private var currentUncanonical = _initialBehavior
private var current = {
@ -174,7 +175,7 @@ private[akka] final class BehaviorTestKitImpl[T](
override def receptionistInbox(): TestInboxImpl[Receptionist.Command] = context.system.receptionistInbox
}
private[akka] object BehaviorTestKitImpl {
private[pekko] object BehaviorTestKitImpl {
object Interceptor extends BehaviorInterceptor[Any, Any]() {
// Intercept a internal message adaptors related messages, forward the rest

View file

@ -2,17 +2,18 @@
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
package org.apache.pekko.actor.testkit.typed.internal
import ch.qos.logback.classic.spi.ILoggingEvent
import ch.qos.logback.core.AppenderBase
import akka.annotation.InternalApi
import org.apache.pekko
import pekko.annotation.InternalApi
/**
* INTERNAL API
*/
@InternalApi private[akka] object CapturingAppender {
@InternalApi private[pekko] object CapturingAppender {
import LogbackUtil._
private val CapturingAppenderName = "CapturingAppender"
@ -36,17 +37,17 @@ import akka.annotation.InternalApi
*
* Logging from tests can be silenced by this appender. When there is a test failure
* the captured logging events are flushed to the appenders defined for the
* akka.actor.testkit.typed.internal.CapturingAppenderDelegate logger.
* org.apache.pekko.actor.testkit.typed.internal.CapturingAppenderDelegate logger.
*
* The flushing on test failure is handled by [[akka.actor.testkit.typed.scaladsl.LogCapturing]]
* for ScalaTest and [[akka.actor.testkit.typed.javadsl.LogCapturing]] for JUnit.
* The flushing on test failure is handled by [[pekko.actor.testkit.typed.scaladsl.LogCapturing]]
* for ScalaTest and [[pekko.actor.testkit.typed.javadsl.LogCapturing]] for JUnit.
*
* Use configuration like the following the logback-test.xml:
*
* {{{
* <appender name="CapturingAppender" class="akka.actor.testkit.typed.internal.CapturingAppender" />
* <appender name="CapturingAppender" class="org.apache.pekko.actor.testkit.typed.internal.CapturingAppender" />
*
* <logger name="akka.actor.testkit.typed.internal.CapturingAppenderDelegate" >
* <logger name="org.apache.pekko.actor.testkit.typed.internal.CapturingAppenderDelegate" >
* <appender-ref ref="STDOUT"/>
* </logger>
*
@ -55,7 +56,7 @@ import akka.annotation.InternalApi
* </root>
* }}}
*/
@InternalApi private[akka] class CapturingAppender extends AppenderBase[ILoggingEvent] {
@InternalApi private[pekko] class CapturingAppender extends AppenderBase[ILoggingEvent] {
import LogbackUtil._
private var buffer: Vector[ILoggingEvent] = Vector.empty
@ -71,7 +72,7 @@ import akka.annotation.InternalApi
* Also clears the buffer..
*/
def flush(): Unit = synchronized {
import akka.util.ccompat.JavaConverters._
import pekko.util.ccompat.JavaConverters._
val logbackLogger = getLogbackLogger(classOf[CapturingAppender].getName + "Delegate")
val appenders = logbackLogger.iteratorForAppenders().asScala.filterNot(_ == this).toList
for (event <- buffer; appender <- appenders) {

View file

@ -2,19 +2,19 @@
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
package org.apache.pekko.actor.testkit.typed.internal
import java.util.LinkedList
import scala.concurrent.ExecutionContextExecutor
import akka.annotation.InternalApi
import org.apache.pekko.annotation.InternalApi
/**
* INTERNAL API
*/
@InternalApi
private[akka] final class ControlledExecutor extends ExecutionContextExecutor {
private[pekko] final class ControlledExecutor extends ExecutionContextExecutor {
private val tasks = new LinkedList[Runnable]
def queueSize: Int = tasks.size()

View file

@ -2,23 +2,24 @@
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
package org.apache.pekko.actor.testkit.typed.internal
import java.util.concurrent.ConcurrentLinkedQueue
import scala.annotation.tailrec
import akka.{ actor => classic }
import akka.actor.ActorRefProvider
import akka.actor.typed.ActorRef
import akka.actor.typed.internal.{ ActorRefImpl, SystemMessage }
import akka.actor.typed.internal.InternalRecipientRef
import akka.annotation.InternalApi
import org.apache.pekko
import pekko.{ actor => classic }
import pekko.actor.ActorRefProvider
import pekko.actor.typed.ActorRef
import pekko.actor.typed.internal.{ ActorRefImpl, SystemMessage }
import pekko.actor.typed.internal.InternalRecipientRef
import pekko.annotation.InternalApi
/**
* INTERNAL API
*/
@InternalApi private[akka] final class DebugRef[T](override val path: classic.ActorPath, override val isLocal: Boolean)
@InternalApi private[pekko] final class DebugRef[T](override val path: classic.ActorPath, override val isLocal: Boolean)
extends ActorRef[T]
with ActorRefImpl[T]
with InternalRecipientRef[T] {

View file

@ -2,16 +2,17 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
package org.apache.pekko.actor.testkit.typed.internal
import java.util.concurrent.ConcurrentLinkedQueue
import akka.actor.testkit.typed.Effect
import akka.actor.testkit.typed.Effect._
import akka.actor.typed.internal.TimerSchedulerCrossDslSupport
import akka.actor.typed.{ ActorRef, Behavior, Props }
import akka.actor.{ ActorPath, Cancellable }
import akka.annotation.InternalApi
import org.apache.pekko
import pekko.actor.testkit.typed.Effect
import pekko.actor.testkit.typed.Effect._
import pekko.actor.typed.internal.TimerSchedulerCrossDslSupport
import pekko.actor.typed.{ ActorRef, Behavior, Props }
import pekko.actor.{ ActorPath, Cancellable }
import pekko.annotation.InternalApi
import scala.concurrent.duration.FiniteDuration
import scala.reflect.ClassTag
@ -19,13 +20,13 @@ import scala.reflect.ClassTag
/**
* INTERNAL API
*/
@InternalApi private[akka] final class EffectfulActorContext[T](
@InternalApi private[pekko] final class EffectfulActorContext[T](
system: ActorSystemStub,
path: ActorPath,
currentBehaviorProvider: () => Behavior[T])
extends StubbedActorContext[T](system, path, currentBehaviorProvider) {
private[akka] val effectQueue = new ConcurrentLinkedQueue[Effect]
private[pekko] val effectQueue = new ConcurrentLinkedQueue[Effect]
override def spawnAnonymous[U](behavior: Behavior[U], props: Props = Props.empty): ActorRef[U] = {
val ref = super.spawnAnonymous(behavior, props)
@ -47,7 +48,7 @@ import scala.reflect.ClassTag
effectQueue.offer(MessageAdapter(implicitly[ClassTag[U]].runtimeClass.asInstanceOf[Class[U]], f))
ref
}
override def messageAdapter[U](messageClass: Class[U], f: akka.japi.function.Function[U, T]): ActorRef[U] = {
override def messageAdapter[U](messageClass: Class[U], f: pekko.japi.function.Function[U, T]): ActorRef[U] = {
val ref = super.messageAdapter(messageClass, f)
effectQueue.offer(MessageAdapter[U, T](messageClass, f.apply))
ref

View file

@ -2,18 +2,18 @@
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
package org.apache.pekko.actor.testkit.typed.internal
import org.slf4j.LoggerFactory
import org.slf4j.event.Level
import akka.annotation.InternalApi
import org.apache.pekko.annotation.InternalApi
import scala.annotation.tailrec
/**
* INTERNAL API
*/
@InternalApi private[akka] object LogbackUtil {
@InternalApi private[pekko] object LogbackUtil {
def loggerNameOrRoot(loggerName: String): String =
if (loggerName == "") org.slf4j.Logger.ROOT_LOGGER_NAME else loggerName

View file

@ -2,7 +2,7 @@
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
package org.apache.pekko.actor.testkit.typed.internal
import java.util.function.Supplier
@ -12,18 +12,19 @@ import scala.util.matching.Regex
import org.slf4j.event.Level
import akka.actor.testkit.typed.LoggingEvent
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.javadsl
import akka.actor.testkit.typed.scaladsl
import akka.actor.typed.ActorSystem
import akka.annotation.InternalApi
import akka.testkit.TestKit
import org.apache.pekko
import pekko.actor.testkit.typed.LoggingEvent
import pekko.actor.testkit.typed.TestKitSettings
import pekko.actor.testkit.typed.javadsl
import pekko.actor.testkit.typed.scaladsl
import pekko.actor.typed.ActorSystem
import pekko.annotation.InternalApi
import pekko.testkit.TestKit
/**
* INTERNAL API
*/
@InternalApi private[akka] object LoggingTestKitImpl {
@InternalApi private[pekko] object LoggingTestKitImpl {
def empty: LoggingTestKitImpl =
new LoggingTestKitImpl(1, None, None, None, None, None, None, Map.empty, checkExcess = true, None)
}
@ -31,7 +32,7 @@ import akka.testkit.TestKit
/**
* INTERNAL API
*/
@InternalApi private[akka] final case class LoggingTestKitImpl(
@InternalApi private[pekko] final case class LoggingTestKitImpl(
occurrences: Int,
logLevel: Option[Level],
loggerName: Option[String],
@ -150,7 +151,7 @@ import akka.testkit.TestKit
copy(mdc = newMdc)
override def withMdc(newMdc: java.util.Map[String, String]): javadsl.LoggingTestKit = {
import akka.util.ccompat.JavaConverters._
import pekko.util.ccompat.JavaConverters._
withMdc(newMdc.asScala.toMap)
}

View file

@ -2,15 +2,16 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
package org.apache.pekko.actor.testkit.typed.internal
import akka.actor.testkit.typed.CapturedLogEvent
import akka.actor.typed._
import akka.actor.typed.internal._
import akka.actor.{ ActorPath, ActorRefProvider, InvalidMessageException }
import akka.annotation.InternalApi
import akka.util.Helpers
import akka.{ actor => classic }
import org.apache.pekko
import pekko.actor.testkit.typed.CapturedLogEvent
import pekko.actor.typed._
import pekko.actor.typed.internal._
import pekko.actor.{ ActorPath, ActorRefProvider, InvalidMessageException }
import pekko.annotation.InternalApi
import pekko.util.Helpers
import pekko.{ actor => classic }
import org.slf4j.Logger
import org.slf4j.helpers.{ MessageFormatter, SubstituteLoggerFactory }
@ -26,7 +27,7 @@ import scala.concurrent.duration.FiniteDuration
* This reference cannot watch other references.
*/
@InternalApi
private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T, FunctionRef[T]) => Unit)
private[pekko] final class FunctionRef[-T](override val path: ActorPath, send: (T, FunctionRef[T]) => Unit)
extends ActorRef[T]
with ActorRefImpl[T]
with InternalRecipientRef[T] {
@ -58,7 +59,7 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
* provides only stubs for the effects an Actor can perform and replaces
* created child Actors by a synchronous Inbox (see `Inbox.sync`).
*/
@InternalApi private[akka] class StubbedActorContext[T](
@InternalApi private[pekko] class StubbedActorContext[T](
val system: ActorSystemStub,
val path: ActorPath,
currentBehaviorProvider: () => Behavior[T])
@ -75,7 +76,7 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
/**
* INTERNAL API
*/
@InternalApi private[akka] val selfInbox = new TestInboxImpl[T](path)
@InternalApi private[pekko] val selfInbox = new TestInboxImpl[T](path)
override val self = selfInbox.ref
private var _children = TreeMap.empty[String, BehaviorTestKitImpl[_]]
@ -84,7 +85,7 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
private val logger: Logger = substituteLoggerFactory.getLogger("StubbedLogger")
private var unhandled: List[T] = Nil
private[akka] def classicActorContext =
private[pekko] def classicActorContext =
throw new UnsupportedOperationException(
"No classic ActorContext available with the stubbed actor context, to spawn materializers and run streams you will need a real actor")
@ -159,7 +160,7 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
/**
* INTERNAL API
*/
@InternalApi private[akka] def internalSpawnMessageAdapter[U](f: U => T, name: String): ActorRef[U] = {
@InternalApi private[pekko] def internalSpawnMessageAdapter[U](f: U => T, name: String): ActorRef[U] = {
val n = if (name != "") s"${childName.next()}-$name" else childName.next()
val p = (path / n).withUid(rnd().nextInt())
@ -228,7 +229,7 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
* this method.
*/
def logEntries: List[CapturedLogEvent] = {
import akka.util.ccompat.JavaConverters._
import pekko.util.ccompat.JavaConverters._
substituteLoggerFactory.getEventQueue
.iterator()
.asScala
@ -248,7 +249,7 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
def clearLog(): Unit =
substituteLoggerFactory.getEventQueue.clear()
override private[akka] def onUnhandled(msg: T): Unit =
override private[pekko] def onUnhandled(msg: T): Unit =
unhandled = msg :: unhandled
/**
@ -261,6 +262,6 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
*/
def clearUnhandled(): Unit = unhandled = Nil
override private[akka] def currentBehavior: Behavior[T] = currentBehaviorProvider()
override private[pekko] def currentBehavior: Behavior[T] = currentBehaviorProvider()
}

View file

@ -2,14 +2,15 @@
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
package org.apache.pekko.actor.testkit.typed.internal
import ch.qos.logback.classic.spi.ILoggingEvent
import ch.qos.logback.classic.spi.ThrowableProxy
import ch.qos.logback.core.AppenderBase
import akka.actor.testkit.typed.LoggingEvent
import akka.annotation.InternalApi
import org.apache.pekko
import pekko.actor.testkit.typed.LoggingEvent
import pekko.annotation.InternalApi
/**
* INTERNAL API
@ -20,7 +21,7 @@ import akka.annotation.InternalApi
* `TestAppender` is currently requiring Logback as SLF4J implementation.
* Similar can probably be implemented with other backends, such as Log4j2.
*/
@InternalApi private[akka] object TestAppender {
@InternalApi private[pekko] object TestAppender {
import LogbackUtil._
private val TestAppenderName = "AkkaTestAppender"
@ -63,14 +64,14 @@ import akka.annotation.InternalApi
/**
* INTERNAL API
*/
@InternalApi private[akka] class TestAppender extends AppenderBase[ILoggingEvent] {
@InternalApi private[pekko] class TestAppender extends AppenderBase[ILoggingEvent] {
import LogbackUtil._
private var filters: List[LoggingTestKitImpl] = Nil
// invocations are synchronized via doAppend in AppenderBase
override def append(event: ILoggingEvent): Unit = {
import akka.util.ccompat.JavaConverters._
import pekko.util.ccompat.JavaConverters._
val throwable = event.getThrowableProxy match {
case p: ThrowableProxy =>

View file

@ -2,24 +2,25 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
package org.apache.pekko.actor.testkit.typed.internal
import java.util.concurrent.ConcurrentLinkedQueue
import scala.annotation.tailrec
import scala.collection.immutable
import akka.actor.ActorPath
import akka.actor.typed.ActorRef
import akka.annotation.InternalApi
import org.apache.pekko
import pekko.actor.ActorPath
import pekko.actor.typed.ActorRef
import pekko.annotation.InternalApi
/**
* INTERNAL API
*/
@InternalApi
private[akka] final class TestInboxImpl[T](path: ActorPath)
extends akka.actor.testkit.typed.javadsl.TestInbox[T]
with akka.actor.testkit.typed.scaladsl.TestInbox[T] {
private[pekko] final class TestInboxImpl[T](path: ActorPath)
extends pekko.actor.testkit.typed.javadsl.TestInbox[T]
with pekko.actor.testkit.typed.scaladsl.TestInbox[T] {
private val q = new ConcurrentLinkedQueue[T]
@ -50,6 +51,6 @@ private[akka] final class TestInboxImpl[T](path: ActorPath)
def hasMessages: Boolean = q.peek() != null
@InternalApi private[akka] def as[U]: TestInboxImpl[U] = this.asInstanceOf[TestInboxImpl[U]]
@InternalApi private[pekko] def as[U]: TestInboxImpl[U] = this.asInstanceOf[TestInboxImpl[U]]
}

View file

@ -2,23 +2,24 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
package org.apache.pekko.actor.testkit.typed.internal
import scala.concurrent.{ Await, TimeoutException }
import scala.concurrent.duration.Duration
import scala.util.control.Exception.Catcher
import scala.util.control.NonFatal
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Props }
import akka.actor.typed.scaladsl.ActorContext
import akka.actor.typed.scaladsl.Behaviors
import akka.annotation.InternalApi
import org.apache.pekko
import pekko.actor.typed.{ ActorRef, ActorSystem, Behavior, Props }
import pekko.actor.typed.scaladsl.ActorContext
import pekko.actor.typed.scaladsl.Behaviors
import pekko.annotation.InternalApi
/**
* INTERNAL API
*/
@InternalApi
private[akka] object ActorTestKitGuardian {
private[pekko] object ActorTestKitGuardian {
sealed trait TestKitCommand
final case class SpawnActor[T](name: String, behavior: Behavior[T], replyTo: ActorRef[ActorRef[T]], props: Props)
extends TestKitCommand
@ -64,13 +65,13 @@ private[akka] object ActorTestKitGuardian {
* INTERNAL API
*/
@InternalApi
private[akka] object TestKitUtils {
private[pekko] object TestKitUtils {
// common internal utility impls for Java and Scala
private val TestKitRegex = """akka\.testkit\.typed\.(?:javadsl|scaladsl)\.ActorTestKit(?:\$.*)?""".r
private val TestKitRegex = """org\.apache\.pekko\.testkit\.typed\.(?:javadsl|scaladsl)\.ActorTestKit(?:\$.*)?""".r
def testNameFromCallStack(classToStartFrom: Class[_]): String =
akka.testkit.TestKitUtils.testNameFromCallStack(classToStartFrom, TestKitRegex)
pekko.testkit.TestKitUtils.testNameFromCallStack(classToStartFrom, TestKitRegex)
/**
* Sanitize the `name` to be used as valid actor system name by
@ -78,7 +79,7 @@ private[akka] object TestKitUtils {
* class name and then the short class name will be used.
*/
def scrubActorSystemName(name: String): String =
akka.testkit.TestKitUtils.scrubActorSystemName(name)
pekko.testkit.TestKitUtils.scrubActorSystemName(name)
def shutdown(system: ActorSystem[_], timeout: Duration, throwIfShutdownTimesOut: Boolean): Unit = {
system.terminate()

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
package org.apache.pekko.actor.testkit.typed.internal
import java.time.{ Duration => JDuration }
import java.util.{ List => JList }
@ -16,29 +16,30 @@ import scala.concurrent.duration._
import scala.reflect.ClassTag
import scala.util.control.NonFatal
import akka.actor.ActorRefProvider
import akka.actor.ExtendedActorSystem
import akka.actor.testkit.typed.FishingOutcome
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.javadsl.{ TestProbe => JavaTestProbe }
import akka.actor.testkit.typed.scaladsl.{ TestProbe => ScalaTestProbe }
import akka.actor.testkit.typed.scaladsl.TestDuration
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.Signal
import akka.actor.typed.Terminated
import akka.actor.typed.internal.InternalRecipientRef
import akka.actor.typed.scaladsl.Behaviors
import akka.annotation.InternalApi
import akka.japi.function.Creator
import akka.util.BoxedType
import akka.util.JavaDurationConverters._
import akka.util.PrettyDuration._
import akka.util.ccompat.JavaConverters._
import org.apache.pekko
import pekko.actor.ActorRefProvider
import pekko.actor.ExtendedActorSystem
import pekko.actor.testkit.typed.FishingOutcome
import pekko.actor.testkit.typed.TestKitSettings
import pekko.actor.testkit.typed.javadsl.{ TestProbe => JavaTestProbe }
import pekko.actor.testkit.typed.scaladsl.{ TestProbe => ScalaTestProbe }
import pekko.actor.testkit.typed.scaladsl.TestDuration
import pekko.actor.typed.ActorRef
import pekko.actor.typed.ActorSystem
import pekko.actor.typed.Behavior
import pekko.actor.typed.Signal
import pekko.actor.typed.Terminated
import pekko.actor.typed.internal.InternalRecipientRef
import pekko.actor.typed.scaladsl.Behaviors
import pekko.annotation.InternalApi
import pekko.japi.function.Creator
import pekko.util.BoxedType
import pekko.util.JavaDurationConverters._
import pekko.util.PrettyDuration._
import pekko.util.ccompat.JavaConverters._
@InternalApi
private[akka] object TestProbeImpl {
private[pekko] object TestProbeImpl {
private final case class WatchActor[U](actor: ActorRef[U])
private case object Stop
@ -64,7 +65,7 @@ private[akka] object TestProbeImpl {
}
@InternalApi
private[akka] final class TestProbeImpl[M](name: String, system: ActorSystem[_])
private[pekko] final class TestProbeImpl[M](name: String, system: ActorSystem[_])
extends JavaTestProbe[M]
with ScalaTestProbe[M]
with InternalRecipientRef[M] {
@ -72,7 +73,7 @@ private[akka] final class TestProbeImpl[M](name: String, system: ActorSystem[_])
import TestProbeImpl._
// have to use same global counter as Classic TestKit to ensure unique names
private def testActorId = akka.testkit.TestKit.testActorId
private def testActorId = pekko.testkit.TestKit.testActorId
protected implicit val settings: TestKitSettings = TestKitSettings(system)
private val queue = new LinkedBlockingDeque[M]
private val terminations = new LinkedBlockingDeque[Terminated]
@ -412,5 +413,5 @@ private[akka] final class TestProbeImpl[M](name: String, system: ActorSystem[_])
// impl InternalRecipientRef
def isTerminated: Boolean = false
override private[akka] def asJava: JavaTestProbe[M] = this
override private[pekko] def asJava: JavaTestProbe[M] = this
}

View file

@ -2,25 +2,26 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl
package org.apache.pekko.actor.testkit.typed.javadsl
import java.time.Duration
import com.typesafe.config.Config
import akka.actor.DeadLetter
import akka.actor.Dropped
import akka.actor.UnhandledMessage
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.internal.TestKitUtils
import akka.actor.testkit.typed.scaladsl
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.Props
import akka.actor.typed.Scheduler
import akka.util.JavaDurationConverters._
import akka.util.Timeout
import org.apache.pekko
import pekko.actor.DeadLetter
import pekko.actor.Dropped
import pekko.actor.UnhandledMessage
import pekko.actor.testkit.typed.TestKitSettings
import pekko.actor.testkit.typed.internal.TestKitUtils
import pekko.actor.testkit.typed.scaladsl
import pekko.actor.typed.ActorRef
import pekko.actor.typed.ActorSystem
import pekko.actor.typed.Behavior
import pekko.actor.typed.Props
import pekko.actor.typed.Scheduler
import pekko.util.JavaDurationConverters._
import pekko.util.Timeout
object ActorTestKit {
@ -52,7 +53,7 @@ object ActorTestKit {
/**
* Create a testkit using the provided name.
*
* It will create an [[akka.actor.typed.ActorSystem]] with this name,
* It will create an [[pekko.actor.typed.ActorSystem]] with this name,
* e.g. threads will include the name.
* When the test has completed you should terminate the `ActorSystem` and
* the testkit with [[ActorTestKit#shutdownTestKit]].
@ -80,7 +81,7 @@ object ActorTestKit {
* Create a test kit named based on the provided name,
* and uses the provided custom config for the actor system.
*
* It will create an [[akka.actor.typed.ActorSystem]] with this name,
* It will create an [[pekko.actor.typed.ActorSystem]] with this name,
* e.g. threads will include the name.
*
* It will also used the provided customConfig provided to create the `ActorSystem`
@ -92,10 +93,10 @@ object ActorTestKit {
new ActorTestKit(scaladsl.ActorTestKit(name, customConfig))
/**
* Create an [[akka.actor.typed.ActorSystem]] named based on the provided name,
* Create an [[pekko.actor.typed.ActorSystem]] named based on the provided name,
* use the provided custom config for the actor system, and the testkit will use the provided setting.
*
* It will create an [[akka.actor.typed.ActorSystem]] with this name,
* It will create an [[pekko.actor.typed.ActorSystem]] with this name,
* e.g. threads will include the name.
*
* It will also used the provided customConfig provided to create the `ActorSystem`, and provided setting.
@ -117,7 +118,7 @@ object ActorTestKit {
}
/**
* Shutdown the given [[akka.actor.typed.ActorSystem]] and block until it shuts down,
* Shutdown the given [[pekko.actor.typed.ActorSystem]] and block until it shuts down,
* if more time than `system-shutdown-default` passes an exception is thrown
* (can be configured with `throw-on-shutdown-timeout`).
*/
@ -127,7 +128,7 @@ object ActorTestKit {
}
/**
* Shutdown the given [[akka.actor.typed.ActorSystem]] and block until it shuts down,
* Shutdown the given [[pekko.actor.typed.ActorSystem]] and block until it shuts down,
* if more time than `system-shutdown-default` passes an exception is thrown
* (can be configured with `throw-on-shutdown-timeout`).
*/
@ -157,10 +158,10 @@ object ActorTestKit {
*
* For synchronous testing of a `Behavior` see [[BehaviorTestKit]]
*/
final class ActorTestKit private[akka] (delegate: akka.actor.testkit.typed.scaladsl.ActorTestKit) {
final class ActorTestKit private[pekko] (delegate: pekko.actor.testkit.typed.scaladsl.ActorTestKit) {
/**
* The default timeout as specified with the config/[[akka.actor.testkit.typed.TestKitSettings]]
* The default timeout as specified with the config/[[pekko.actor.testkit.typed.TestKitSettings]]
*/
def timeout: Timeout = delegate.timeout

View file

@ -2,13 +2,14 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl
package org.apache.pekko.actor.testkit.typed.javadsl
import akka.actor.testkit.typed.internal.{ ActorSystemStub, BehaviorTestKitImpl }
import akka.actor.testkit.typed.{ CapturedLogEvent, Effect }
import akka.actor.typed.receptionist.Receptionist
import akka.actor.typed.{ ActorRef, Behavior, Signal }
import akka.annotation.{ ApiMayChange, DoNotInherit }
import org.apache.pekko
import pekko.actor.testkit.typed.internal.{ ActorSystemStub, BehaviorTestKitImpl }
import pekko.actor.testkit.typed.{ CapturedLogEvent, Effect }
import pekko.actor.typed.receptionist.Receptionist
import pekko.actor.typed.{ ActorRef, Behavior, Signal }
import pekko.annotation.{ ApiMayChange, DoNotInherit }
import com.typesafe.config.Config
import java.util.concurrent.ThreadLocalRandom
@ -19,7 +20,7 @@ object BehaviorTestKit {
* JAVA API
*/
@ApiMayChange
def applicationTestConfig: Config = akka.actor.testkit.typed.scaladsl.BehaviorTestKit.ApplicationTestConfig
def applicationTestConfig: Config = pekko.actor.testkit.typed.scaladsl.BehaviorTestKit.ApplicationTestConfig
/**
* JAVA API
@ -49,7 +50,7 @@ object BehaviorTestKit {
}
/**
* Used for synchronous testing [[akka.actor.typed.Behavior]]s. Stores all effects e.g. Spawning of children,
* Used for synchronous testing [[pekko.actor.typed.Behavior]]s. Stores all effects e.g. Spawning of children,
* watching and offers access to what effects have taken place.
*
* Not for user extension or instantiation. See `BehaviorTestKit.create` factory methods
@ -61,7 +62,7 @@ object BehaviorTestKit {
abstract class BehaviorTestKit[T] {
/**
* Requests the oldest [[Effect]] or [[akka.actor.testkit.typed.javadsl.Effects.noEffects]] if no effects
* Requests the oldest [[Effect]] or [[pekko.actor.testkit.typed.javadsl.Effects.noEffects]] if no effects
* have taken place. The effect is consumed, subsequent calls won't
* will not include this effect.
*/
@ -80,7 +81,7 @@ abstract class BehaviorTestKit[T] {
def childInbox[U](child: ActorRef[U]): TestInbox[U]
/**
* Get the [[akka.actor.typed.Behavior]] testkit for the given child [[akka.actor.typed.ActorRef]].
* Get the [[pekko.actor.typed.Behavior]] testkit for the given child [[pekko.actor.typed.ActorRef]].
*/
def childTestKit[U](child: ActorRef[U]): BehaviorTestKit[U]

View file

@ -2,19 +2,20 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl
package org.apache.pekko.actor.testkit.typed.javadsl
import java.time.Duration
import akka.actor.typed.{ ActorRef, Behavior, Props }
import akka.util.JavaDurationConverters._
import org.apache.pekko
import pekko.actor.typed.{ ActorRef, Behavior, Props }
import pekko.util.JavaDurationConverters._
/**
* Factories for behavior effects for [[BehaviorTestKit]], each effect has a suitable equals and can be used to compare
* actual effects to expected ones.
*/
object Effects {
import akka.actor.testkit.typed.Effect._
import org.apache.pekko.actor.testkit.typed.Effect._
/**
* The behavior spawned a named child with the given behavior with no specific props
@ -99,7 +100,7 @@ object Effects {
delay: Duration,
mode: TimerScheduled.TimerMode,
overriding: Boolean,
send: akka.japi.function.Effect): TimerScheduled[U] =
send: pekko.japi.function.Effect): TimerScheduled[U] =
TimerScheduled(key, msg, delay.asScala, mode, overriding)(send.apply _)
/**

View file

@ -2,7 +2,7 @@
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl
package org.apache.pekko.actor.testkit.typed.javadsl
import scala.util.control.NonFatal
@ -11,7 +11,7 @@ import org.junit.runner.Description
import org.junit.runners.model.Statement
import org.slf4j.LoggerFactory
import akka.actor.testkit.typed.internal.CapturingAppender
import org.apache.pekko.actor.testkit.typed.internal.CapturingAppender
/**
* JUnit `TestRule` to make log lines appear only when the test failed.
@ -24,9 +24,9 @@ import akka.actor.testkit.typed.internal.CapturingAppender
* Requires Logback and configuration like the following the logback-test.xml:
*
* {{{
* <appender name="CapturingAppender" class="akka.actor.testkit.typed.internal.CapturingAppender" />
* <appender name="CapturingAppender" class="org.apache.pekko.actor.testkit.typed.internal.CapturingAppender" />
*
* <logger name="akka.actor.testkit.typed.internal.CapturingAppenderDelegate" >
* <logger name="org.apache.pekko.actor.testkit.typed.internal.CapturingAppenderDelegate" >
* <appender-ref ref="STDOUT"/>
* </logger>
*

View file

@ -2,16 +2,17 @@
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl
package org.apache.pekko.actor.testkit.typed.javadsl
import java.util.function.Supplier
import org.slf4j.event.Level
import akka.actor.testkit.typed.LoggingEvent
import akka.actor.testkit.typed.internal.LoggingTestKitImpl
import akka.actor.typed.ActorSystem
import akka.annotation.DoNotInherit
import org.apache.pekko
import pekko.actor.testkit.typed.LoggingEvent
import pekko.actor.testkit.typed.internal.LoggingTestKitImpl
import pekko.actor.typed.ActorSystem
import pekko.annotation.DoNotInherit
/**
* Facilities for verifying logs.

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl
package org.apache.pekko.actor.testkit.typed.javadsl
import java.time.Duration
@ -10,9 +10,10 @@ import scala.annotation.varargs
import com.typesafe.config.Config
import akka.actor.typed.ActorSystem
import akka.actor.typed.internal.adapter.SchedulerAdapter
import akka.util.JavaDurationConverters._
import org.apache.pekko
import pekko.actor.typed.ActorSystem
import pekko.actor.typed.internal.adapter.SchedulerAdapter
import pekko.util.JavaDurationConverters._
/**
* Manual time allows you to do async tests while controlling the scheduler of the system.
@ -25,7 +26,7 @@ object ManualTime {
/**
* Config that needs to be in place for the actor system to use the manual
*/
def config(): Config = akka.actor.testkit.typed.scaladsl.ManualTime.config
def config(): Config = pekko.actor.testkit.typed.scaladsl.ManualTime.config
/**
* Access the manual scheduler, note that you need to setup the actor system/testkit with [[ManualTime.config]]
@ -35,11 +36,11 @@ object ManualTime {
system.scheduler match {
case adapter: SchedulerAdapter =>
adapter.classicScheduler match {
case sc: akka.testkit.ExplicitlyTriggeredScheduler => new ManualTime(sc)
case sc: pekko.testkit.ExplicitlyTriggeredScheduler => new ManualTime(sc)
case _ =>
throw new IllegalArgumentException(
"ActorSystem not configured with explicitly triggered scheduler, " +
"make sure to include akka.actor.testkit.typed.scaladsl.ManualTime.config() when setting up the test")
"make sure to include org.apache.pekko.actor.testkit.typed.scaladsl.ManualTime.config() when setting up the test")
}
case s =>
throw new IllegalArgumentException(
@ -51,7 +52,7 @@ object ManualTime {
/**
* Not for user instantiation, see [[ManualTime#get]]
*/
final class ManualTime(delegate: akka.testkit.ExplicitlyTriggeredScheduler) {
final class ManualTime(delegate: pekko.testkit.ExplicitlyTriggeredScheduler) {
/**
* Advance the clock by the specified duration, executing all outstanding jobs on the calling thread before returning.

View file

@ -2,10 +2,11 @@
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl
package org.apache.pekko.actor.testkit.typed.javadsl
import akka.actor.testkit.typed.scaladsl
import akka.actor.typed.ActorSystem
import org.apache.pekko
import pekko.actor.testkit.typed.scaladsl
import pekko.actor.typed.ActorSystem
/**
* Utilities to test serialization.

View file

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

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl
package org.apache.pekko.actor.testkit.typed.javadsl
import java.time.Duration
@ -11,17 +11,18 @@ import com.typesafe.config.ConfigFactory
import org.junit.Rule
import org.junit.rules.ExternalResource
import akka.actor.DeadLetter
import akka.actor.Dropped
import akka.actor.UnhandledMessage
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.internal.TestKitUtils
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.Props
import akka.actor.typed.Scheduler
import akka.util.Timeout
import org.apache.pekko
import pekko.actor.DeadLetter
import pekko.actor.Dropped
import pekko.actor.UnhandledMessage
import pekko.actor.testkit.typed.TestKitSettings
import pekko.actor.testkit.typed.internal.TestKitUtils
import pekko.actor.typed.ActorRef
import pekko.actor.typed.ActorSystem
import pekko.actor.typed.Behavior
import pekko.actor.typed.Props
import pekko.actor.typed.Scheduler
import pekko.util.Timeout
/**
* A Junit external resource for the [[ActorTestKit]], making it possible to have Junit manage the lifecycle of the testkit.
@ -59,7 +60,7 @@ final class TestKitJunitResource(_kit: ActorTestKit) extends ExternalResource {
def this() = this(ActorTestKit.create(TestKitUtils.testNameFromCallStack(classOf[TestKitJunitResource])))
/**
* Use a custom [[akka.actor.typed.ActorSystem]] for the actor system.
* Use a custom [[pekko.actor.typed.ActorSystem]] for the actor system.
*/
def this(system: ActorSystem[_]) = this(ActorTestKit.create(system))
@ -79,7 +80,7 @@ final class TestKitJunitResource(_kit: ActorTestKit) extends ExternalResource {
this(ActorTestKit.create(TestKitUtils.testNameFromCallStack(classOf[TestKitJunitResource]), customConfig))
/**
* Use a custom config for the actor system, and a custom [[akka.actor.testkit.typed.TestKitSettings]].
* Use a custom config for the actor system, and a custom [[pekko.actor.testkit.typed.TestKitSettings]].
*/
def this(customConfig: Config, settings: TestKitSettings) =
this(ActorTestKit.create(TestKitUtils.testNameFromCallStack(classOf[TestKitJunitResource]), customConfig, settings))

View file

@ -2,22 +2,23 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl
package org.apache.pekko.actor.testkit.typed.javadsl
import java.time.Duration
import java.util.{ List => JList }
import java.util.function.Supplier
import akka.japi.function.Creator
import akka.actor.testkit.typed.FishingOutcome
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.internal.TestProbeImpl
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.RecipientRef
import akka.actor.typed.internal.InternalRecipientRef
import akka.annotation.DoNotInherit
import akka.util.unused
import org.apache.pekko
import pekko.japi.function.Creator
import pekko.actor.testkit.typed.FishingOutcome
import pekko.actor.testkit.typed.TestKitSettings
import pekko.actor.testkit.typed.internal.TestProbeImpl
import pekko.actor.typed.ActorRef
import pekko.actor.typed.ActorSystem
import pekko.actor.typed.RecipientRef
import pekko.actor.typed.internal.InternalRecipientRef
import pekko.annotation.DoNotInherit
import pekko.util.unused
object FishingOutcomes {
@ -29,17 +30,17 @@ object FishingOutcomes {
/**
* Consume this message and continue with the next
*/
def continueAndIgnore(): FishingOutcome = akka.actor.testkit.typed.FishingOutcome.ContinueAndIgnore
def continueAndIgnore(): FishingOutcome = pekko.actor.testkit.typed.FishingOutcome.ContinueAndIgnore
/**
* Complete fishing and return this message
*/
def complete(): FishingOutcome = akka.actor.testkit.typed.FishingOutcome.Complete
def complete(): FishingOutcome = pekko.actor.testkit.typed.FishingOutcome.Complete
/**
* Fail fishing with a custom error message
*/
def fail(error: String): FishingOutcome = akka.actor.testkit.typed.FishingOutcome.Fail(error)
def fail(error: String): FishingOutcome = pekko.actor.testkit.typed.FishingOutcome.Fail(error)
}
object TestProbe {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import java.util.concurrent.TimeoutException
import java.util.concurrent.atomic.AtomicInteger
@ -15,23 +15,24 @@ import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import org.slf4j.LoggerFactory
import akka.actor.DeadLetter
import akka.actor.DeadLetterSuppression
import akka.actor.Dropped
import akka.actor.UnhandledMessage
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.internal.ActorTestKitGuardian
import akka.actor.testkit.typed.internal.TestKitUtils
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.Props
import akka.actor.typed.Scheduler
import akka.actor.typed.eventstream.EventStream
import akka.actor.typed.scaladsl.AskPattern._
import akka.actor.typed.scaladsl.adapter._
import akka.annotation.InternalApi
import akka.util.Timeout
import org.apache.pekko
import pekko.actor.DeadLetter
import pekko.actor.DeadLetterSuppression
import pekko.actor.Dropped
import pekko.actor.UnhandledMessage
import pekko.actor.testkit.typed.TestKitSettings
import pekko.actor.testkit.typed.internal.ActorTestKitGuardian
import pekko.actor.testkit.typed.internal.TestKitUtils
import pekko.actor.typed.ActorRef
import pekko.actor.typed.ActorSystem
import pekko.actor.typed.Behavior
import pekko.actor.typed.Props
import pekko.actor.typed.Scheduler
import pekko.actor.typed.eventstream.EventStream
import pekko.actor.typed.scaladsl.AskPattern._
import pekko.actor.typed.scaladsl.adapter._
import pekko.annotation.InternalApi
import pekko.util.Timeout
object ActorTestKit {
@ -77,7 +78,7 @@ object ActorTestKit {
/**
* Create a testkit using the provided name.
*
* It will create an [[akka.actor.typed.ActorSystem]] with this name,
* It will create an [[pekko.actor.typed.ActorSystem]] with this name,
* e.g. threads will include the name.
* When the test has completed you should terminate the `ActorSystem` and
* the testkit with [[ActorTestKit#shutdownTestKit]].
@ -113,7 +114,7 @@ object ActorTestKit {
* Create a test kit named based on the provided name,
* and uses the provided custom config for the actor system.
*
* It will create an [[akka.actor.typed.ActorSystem]] with this name,
* It will create an [[pekko.actor.typed.ActorSystem]] with this name,
* e.g. threads will include the name.
*
* It will also used the provided customConfig provided to create the `ActorSystem`
@ -128,10 +129,10 @@ object ActorTestKit {
}
/**
* Create an [[akka.actor.typed.ActorSystem]] named based on the provided name,
* Create an [[pekko.actor.typed.ActorSystem]] named based on the provided name,
* use the provided custom config for the actor system, and the testkit will use the provided setting.
*
* It will create an [[akka.actor.typed.ActorSystem]] with this name,
* It will create an [[pekko.actor.typed.ActorSystem]] with this name,
* e.g. threads will include the name.
*
* It will also used the provided customConfig provided to create the `ActorSystem`, and provided setting.
@ -146,7 +147,7 @@ object ActorTestKit {
}
/**
* Shutdown the given [[akka.actor.typed.ActorSystem]] and block until it shuts down,
* Shutdown the given [[pekko.actor.typed.ActorSystem]] and block until it shuts down,
* if more time than `TestKitSettings.DefaultActorSystemShutdownTimeout` passes an exception is thrown
*/
def shutdown(system: ActorSystem[_]): Unit = {
@ -155,7 +156,7 @@ object ActorTestKit {
}
/**
* Shutdown the given [[akka.actor.typed.ActorSystem]] and block until it shuts down
* Shutdown the given [[pekko.actor.typed.ActorSystem]] and block until it shuts down
* or the `duration` hits. If the timeout hits `verifySystemShutdown` decides
*/
def shutdown(system: ActorSystem[_], timeout: Duration, throwIfShutdownFails: Boolean = false): Unit =
@ -181,7 +182,7 @@ object ActorTestKit {
*
* For synchronous testing of a `Behavior` see [[BehaviorTestKit]]
*/
final class ActorTestKit private[akka] (
final class ActorTestKit private[pekko] (
val internalSystem: ActorSystem[_],
internalTestKitGuardian: ActorRef[ActorTestKitGuardian.TestKitCommand],
settings: Option[TestKitSettings]) {
@ -318,10 +319,10 @@ final class ActorTestKit private[akka] (
// FIXME needed for Akka internal tests but, users shouldn't spawn system actors?
@InternalApi
private[akka] def systemActor[T](behavior: Behavior[T], name: String): ActorRef[T] =
private[pekko] def systemActor[T](behavior: Behavior[T], name: String): ActorRef[T] =
system.systemActorOf(behavior, name)
@InternalApi
private[akka] def systemActor[T](behavior: Behavior[T]): ActorRef[T] =
private[pekko] def systemActor[T](behavior: Behavior[T]): ActorRef[T] =
system.systemActorOf(behavior, childName.next())
}

View file

@ -2,21 +2,22 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import akka.actor.DeadLetter
import akka.actor.Dropped
import akka.actor.UnhandledMessage
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.internal.TestKitUtils
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.Props
import akka.util.Timeout
import org.apache.pekko
import pekko.actor.DeadLetter
import pekko.actor.Dropped
import pekko.actor.UnhandledMessage
import pekko.actor.testkit.typed.TestKitSettings
import pekko.actor.testkit.typed.internal.TestKitUtils
import pekko.actor.typed.ActorRef
import pekko.actor.typed.ActorSystem
import pekko.actor.typed.Behavior
import pekko.actor.typed.Props
import pekko.util.Timeout
object ActorTestKitBase {
def testNameFromCallStack(): String = TestKitUtils.testNameFromCallStack(classOf[ActorTestKitBase])
@ -47,7 +48,7 @@ abstract class ActorTestKitBase(val testKit: ActorTestKit) {
def this(config: Config) = this(ActorTestKit(ActorTestKitBase.testNameFromCallStack(), config))
/**
* Use a custom config for the actor system, and a custom [[akka.actor.testkit.typed.TestKitSettings]].
* Use a custom config for the actor system, and a custom [[pekko.actor.testkit.typed.TestKitSettings]].
*/
def this(config: Config, settings: TestKitSettings) =
this(ActorTestKit(ActorTestKitBase.testNameFromCallStack(), config, settings))

View file

@ -2,13 +2,14 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import akka.actor.testkit.typed.internal.{ ActorSystemStub, BehaviorTestKitImpl }
import akka.actor.testkit.typed.{ CapturedLogEvent, Effect }
import akka.actor.typed.receptionist.Receptionist
import akka.actor.typed.{ ActorRef, Behavior, Signal, TypedActorContext }
import akka.annotation.{ ApiMayChange, DoNotInherit }
import org.apache.pekko
import pekko.actor.testkit.typed.internal.{ ActorSystemStub, BehaviorTestKitImpl }
import pekko.actor.testkit.typed.{ CapturedLogEvent, Effect }
import pekko.actor.typed.receptionist.Receptionist
import pekko.actor.typed.{ ActorRef, Behavior, Signal, TypedActorContext }
import pekko.annotation.{ ApiMayChange, DoNotInherit }
import com.typesafe.config.Config
import java.util.concurrent.ThreadLocalRandom
@ -34,7 +35,7 @@ object BehaviorTestKit {
}
/**
* Used for synchronous testing [[akka.actor.typed.Behavior]]s. Stores all effects e.g. Spawning of children,
* Used for synchronous testing [[pekko.actor.typed.Behavior]]s. Stores all effects e.g. Spawning of children,
* watching and offers access to what effects have taken place.
*
* For asynchronous testing of `Behavior`s running see [[ActorTestKit]]
@ -46,10 +47,10 @@ object BehaviorTestKit {
trait BehaviorTestKit[T] {
// FIXME it is weird that this is public but it is used in BehaviorSpec, could we avoid that?
private[akka] def context: TypedActorContext[T]
private[pekko] def context: TypedActorContext[T]
/**
* Requests the oldest [[Effect]] or [[akka.actor.testkit.typed.Effect.NoEffects]] if no effects
* Requests the oldest [[Effect]] or [[pekko.actor.testkit.typed.Effect.NoEffects]] if no effects
* have taken place. The effect is consumed, subsequent calls won't
* will not include this effect.
*/
@ -67,7 +68,7 @@ trait BehaviorTestKit[T] {
def childInbox[U](child: ActorRef[U]): TestInbox[U]
/**
* Get the [[akka.actor.typed.Behavior]] testkit for the given child [[akka.actor.typed.ActorRef]].
* Get the [[pekko.actor.typed.Behavior]] testkit for the given child [[pekko.actor.typed.ActorRef]].
*/
def childTestKit[U](child: ActorRef[U]): BehaviorTestKit[U]

View file

@ -2,18 +2,19 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import scala.concurrent.duration.FiniteDuration
import akka.actor.typed.{ ActorRef, Behavior, Props }
import org.apache.pekko
import pekko.actor.typed.{ ActorRef, Behavior, Props }
/**
* Factories for behavior effects for [[BehaviorTestKit]], each effect has a suitable equals and can be used to compare
* actual effects to expected ones.
*/
object Effects {
import akka.actor.testkit.typed.Effect._
import pekko.actor.testkit.typed.Effect._
/**
* The behavior spawned a named child with the given behavior with no specific props

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import scala.util.control.NonFatal
@ -11,7 +11,7 @@ import org.scalatest.Outcome
import org.scalatest.TestSuite
import org.slf4j.LoggerFactory
import akka.actor.testkit.typed.internal.CapturingAppender
import org.apache.pekko.actor.testkit.typed.internal.CapturingAppender
/**
* Mixin this trait to a ScalaTest test to make log lines appear only when the test failed.
@ -19,9 +19,9 @@ import akka.actor.testkit.typed.internal.CapturingAppender
* Requires Logback and configuration like the following the logback-test.xml:
*
* {{{
* <appender name="CapturingAppender" class="akka.actor.testkit.typed.internal.CapturingAppender" />
* <appender name="CapturingAppender" class="org.apache.pekko.actor.testkit.typed.internal.CapturingAppender" />
*
* <logger name="akka.actor.testkit.typed.internal.CapturingAppenderDelegate" >
* <logger name="org.apache.pekko.actor.testkit.typed.internal.CapturingAppenderDelegate" >
* <appender-ref ref="STDOUT"/>
* </logger>
*

View file

@ -2,16 +2,17 @@
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import scala.reflect.ClassTag
import org.slf4j.event.Level
import akka.actor.testkit.typed.LoggingEvent
import akka.actor.testkit.typed.internal.LoggingTestKitImpl
import akka.actor.typed.ActorSystem
import akka.annotation.DoNotInherit
import org.apache.pekko
import pekko.actor.testkit.typed.LoggingEvent
import pekko.actor.testkit.typed.internal.LoggingTestKitImpl
import pekko.actor.typed.ActorSystem
import pekko.annotation.DoNotInherit
/**
* Facilities for verifying logs.

View file

@ -2,15 +2,16 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import scala.annotation.varargs
import scala.concurrent.duration.{ Duration, FiniteDuration }
import com.typesafe.config.{ Config, ConfigFactory }
import akka.actor.typed.ActorSystem
import akka.actor.typed.internal.adapter.SchedulerAdapter
import org.apache.pekko
import pekko.actor.typed.ActorSystem
import pekko.actor.typed.internal.adapter.SchedulerAdapter
/**
* Manual time allows you to do async tests while controlling the scheduler of the system.
@ -24,7 +25,8 @@ object ManualTime {
* Config needed to use the `ExplicitlyTriggeredScheduler`
*/
val config: Config =
ConfigFactory.parseString("""akka.scheduler.implementation = "akka.testkit.ExplicitlyTriggeredScheduler"""")
ConfigFactory.parseString(
"""akka.scheduler.implementation = "org.apache.pekko.testkit.ExplicitlyTriggeredScheduler"""")
/**
* Access the manual scheduler, note that you need to setup the actor system/testkit with [[ManualTime.config]]
@ -34,11 +36,11 @@ object ManualTime {
system.scheduler match {
case adapter: SchedulerAdapter =>
adapter.classicScheduler match {
case sc: akka.testkit.ExplicitlyTriggeredScheduler => new ManualTime(sc)
case sc: pekko.testkit.ExplicitlyTriggeredScheduler => new ManualTime(sc)
case _ =>
throw new IllegalArgumentException(
"ActorSystem not configured with explicitly triggered scheduler, " +
"make sure to include akka.actor.testkit.typed.scaladsl.ManualTime.config() when setting up the test")
"make sure to include org.apache.pekko.actor.testkit.typed.scaladsl.ManualTime.config() when setting up the test")
}
case s =>
throw new IllegalArgumentException(
@ -50,7 +52,7 @@ object ManualTime {
/**
* Not for user instantiation, see [[ManualTime#apply]]
*/
final class ManualTime(delegate: akka.testkit.ExplicitlyTriggeredScheduler) {
final class ManualTime(delegate: pekko.testkit.ExplicitlyTriggeredScheduler) {
/**
* Advance the clock by the specified duration, executing all outstanding jobs on the calling thread before returning.

View file

@ -2,7 +2,7 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
@ -12,8 +12,9 @@ import org.scalatest.concurrent.ScalaFutures
import org.scalatest.matchers.should.Matchers
import org.scalatest.time.Span
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.typed.ActorSystem
import org.apache.pekko
import pekko.actor.testkit.typed.TestKitSettings
import pekko.actor.typed.ActorSystem
/**
* A ScalaTest base class for the [[ActorTestKit]], making it possible to have ScalaTest manage the lifecycle of the testkit.
@ -44,7 +45,7 @@ abstract class ScalaTestWithActorTestKit(testKit: ActorTestKit)
def this() = this(ActorTestKit(ActorTestKitBase.testNameFromCallStack()))
/**
* Use a custom [[akka.actor.typed.ActorSystem]] for the actor system.
* Use a custom [[pekko.actor.typed.ActorSystem]] for the actor system.
*/
def this(system: ActorSystem[_]) = this(ActorTestKit(system))
@ -60,14 +61,14 @@ abstract class ScalaTestWithActorTestKit(testKit: ActorTestKit)
def this(config: Config) = this(ActorTestKit(ActorTestKitBase.testNameFromCallStack(), config))
/**
* Use a custom config for the actor system, and a custom [[akka.actor.testkit.typed.TestKitSettings]].
* Use a custom config for the actor system, and a custom [[pekko.actor.testkit.typed.TestKitSettings]].
*/
def this(config: Config, settings: TestKitSettings) =
this(ActorTestKit(ActorTestKitBase.testNameFromCallStack(), config, settings))
/**
* `PatienceConfig` from [[akka.actor.testkit.typed.TestKitSettings#DefaultTimeout]].
* `DefaultTimeout` is dilated with [[akka.actor.testkit.typed.TestKitSettings#TestTimeFactor]],
* `PatienceConfig` from [[pekko.actor.testkit.typed.TestKitSettings#DefaultTimeout]].
* `DefaultTimeout` is dilated with [[pekko.actor.testkit.typed.TestKitSettings#TestTimeFactor]],
* which means that the patience is also dilated.
*/
implicit val patience: PatienceConfig =

View file

@ -2,12 +2,13 @@
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import akka.actor.typed.ActorSystem
import akka.actor.typed.scaladsl.adapter._
import akka.serialization.SerializationExtension
import akka.serialization.Serializers
import org.apache.pekko
import pekko.actor.typed.ActorSystem
import pekko.actor.typed.scaladsl.adapter._
import pekko.serialization.SerializationExtension
import pekko.serialization.Serializers
/**
* Utilities to test serialization.

View file

@ -2,16 +2,17 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import java.util.concurrent.ThreadLocalRandom
import scala.collection.immutable
import akka.actor.{ Address, RootActorPath }
import akka.actor.testkit.typed.internal.TestInboxImpl
import akka.actor.typed.ActorRef
import akka.annotation.{ ApiMayChange, DoNotInherit }
import org.apache.pekko
import pekko.actor.{ Address, RootActorPath }
import pekko.actor.testkit.typed.internal.TestInboxImpl
import pekko.actor.typed.ActorRef
import pekko.annotation.{ ApiMayChange, DoNotInherit }
@ApiMayChange
object TestInbox {
@ -20,14 +21,14 @@ object TestInbox {
new TestInboxImpl((address / name).withUid(uid))
}
private[akka] val address = RootActorPath(Address("akka.actor.typed.inbox", "anonymous"))
private[pekko] val address = RootActorPath(Address("akka.actor.typed.inbox", "anonymous"))
}
/**
* Utility for use as an [[ActorRef]] when *synchronously* testing [[akka.actor.typed.Behavior]]
* with [[akka.actor.testkit.typed.javadsl.BehaviorTestKit]].
* Utility for use as an [[ActorRef]] when *synchronously* testing [[pekko.actor.typed.Behavior]]
* with [[pekko.actor.testkit.typed.javadsl.BehaviorTestKit]].
*
* If you plan to use a real [[akka.actor.typed.ActorSystem]] then use [[akka.actor.testkit.typed.javadsl.TestProbe]]
* If you plan to use a real [[pekko.actor.typed.ActorSystem]] then use [[pekko.actor.testkit.typed.javadsl.TestProbe]]
* for asynchronous testing.
*
* Use factory `apply` in companion to create instances

View file

@ -2,21 +2,22 @@
* Copyright (C) 2017-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import scala.collection.immutable
import scala.concurrent.duration._
import scala.reflect.ClassTag
import akka.actor.testkit.typed.FishingOutcome
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.internal.TestProbeImpl
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.RecipientRef
import akka.actor.typed.internal.InternalRecipientRef
import akka.annotation.DoNotInherit
import akka.annotation.InternalApi
import org.apache.pekko
import pekko.actor.testkit.typed.FishingOutcome
import pekko.actor.testkit.typed.TestKitSettings
import pekko.actor.testkit.typed.internal.TestProbeImpl
import pekko.actor.typed.ActorRef
import pekko.actor.typed.ActorSystem
import pekko.actor.typed.RecipientRef
import pekko.actor.typed.internal.InternalRecipientRef
import pekko.annotation.DoNotInherit
import pekko.annotation.InternalApi
object FishingOutcomes {
@ -259,5 +260,5 @@ object TestProbe {
* INTERNAL API
*/
@InternalApi
private[akka] def asJava: akka.actor.testkit.typed.javadsl.TestProbe[M]
private[pekko] def asJava: pekko.actor.testkit.typed.javadsl.TestProbe[M]
}

View file

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

View file

@ -2,18 +2,18 @@
* Copyright (C) 2017-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package jdocs.akka.actor.testkit.typed.javadsl;
package jdocs.org.apache.pekko.actor.testkit.typed.javadsl;
import akka.actor.typed.ActorRef;
import akka.actor.typed.Behavior;
import akka.actor.typed.Scheduler;
import akka.actor.typed.javadsl.AskPattern;
import akka.actor.typed.javadsl.Behaviors;
import org.apache.pekko.actor.typed.ActorRef;
import org.apache.pekko.actor.typed.Behavior;
import org.apache.pekko.actor.typed.Scheduler;
import org.apache.pekko.actor.typed.javadsl.AskPattern;
import org.apache.pekko.actor.typed.javadsl.Behaviors;
// #test-header
import akka.actor.testkit.typed.javadsl.ActorTestKit;
import org.apache.pekko.actor.testkit.typed.javadsl.ActorTestKit;
// #test-header
import akka.actor.testkit.typed.javadsl.TestProbe;
import org.apache.pekko.actor.testkit.typed.javadsl.TestProbe;
import org.junit.AfterClass;
import org.junit.Test;
import org.scalatestplus.junit.JUnitSuite;

View file

@ -2,17 +2,17 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package jdocs.akka.actor.testkit.typed.javadsl;
package jdocs.org.apache.pekko.actor.testkit.typed.javadsl;
import static jdocs.akka.actor.testkit.typed.javadsl.AsyncTestingExampleTest.Echo;
import static jdocs.org.apache.pekko.actor.testkit.typed.javadsl.AsyncTestingExampleTest.Echo;
import akka.actor.testkit.typed.javadsl.LogCapturing;
import org.apache.pekko.actor.testkit.typed.javadsl.LogCapturing;
import org.junit.Rule;
// #junit-integration
import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.testkit.typed.javadsl.TestProbe;
import akka.actor.typed.ActorRef;
import org.apache.pekko.actor.testkit.typed.javadsl.TestKitJunitResource;
import org.apache.pekko.actor.testkit.typed.javadsl.TestProbe;
import org.apache.pekko.actor.typed.ActorRef;
import org.junit.ClassRule;
import org.junit.Test;

View file

@ -2,15 +2,15 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package jdocs.akka.actor.testkit.typed.javadsl;
package jdocs.org.apache.pekko.actor.testkit.typed.javadsl;
import static jdocs.akka.actor.testkit.typed.javadsl.AsyncTestingExampleTest.*;
import static jdocs.org.apache.pekko.actor.testkit.typed.javadsl.AsyncTestingExampleTest.*;
// #log-capturing
import akka.actor.testkit.typed.javadsl.LogCapturing;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.testkit.typed.javadsl.TestProbe;
import akka.actor.typed.ActorRef;
import org.apache.pekko.actor.testkit.typed.javadsl.LogCapturing;
import org.apache.pekko.actor.testkit.typed.javadsl.TestKitJunitResource;
import org.apache.pekko.actor.testkit.typed.javadsl.TestProbe;
import org.apache.pekko.actor.typed.ActorRef;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;

View file

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

View file

@ -2,15 +2,15 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package jdocs.akka.actor.testkit.typed.javadsl;
package jdocs.org.apache.pekko.actor.testkit.typed.javadsl;
// #imports
import akka.actor.testkit.typed.CapturedLogEvent;
import akka.actor.testkit.typed.Effect;
import akka.actor.testkit.typed.javadsl.BehaviorTestKit;
import akka.actor.testkit.typed.javadsl.TestInbox;
import akka.actor.typed.*;
import akka.actor.typed.javadsl.*;
import org.apache.pekko.actor.testkit.typed.CapturedLogEvent;
import org.apache.pekko.actor.testkit.typed.Effect;
import org.apache.pekko.actor.testkit.typed.javadsl.BehaviorTestKit;
import org.apache.pekko.actor.testkit.typed.javadsl.TestInbox;
import org.apache.pekko.actor.typed.*;
import org.apache.pekko.actor.typed.javadsl.*;
import java.util.HashMap;
import java.util.List;
import java.util.Optional;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package jdocs.akka.actor.testkit.typed.javadsl;
package jdocs.org.apache.pekko.actor.testkit.typed.javadsl;
// #default-application-conf
import com.typesafe.config.ConfigFactory;

View file

@ -2,12 +2,12 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl;
package org.apache.pekko.actor.testkit.typed.javadsl;
import akka.actor.typed.ActorRef;
import akka.actor.typed.Props;
import akka.actor.typed.Scheduler;
import akka.actor.typed.javadsl.Behaviors;
import org.apache.pekko.actor.typed.ActorRef;
import org.apache.pekko.actor.typed.Props;
import org.apache.pekko.actor.typed.Scheduler;
import org.apache.pekko.actor.typed.javadsl.Behaviors;
import java.time.Duration;
import java.util.List;

View file

@ -2,10 +2,10 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl;
package org.apache.pekko.actor.testkit.typed.javadsl;
import akka.Done;
import akka.actor.typed.javadsl.Behaviors;
import org.apache.pekko.Done;
import org.apache.pekko.actor.typed.javadsl.Behaviors;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
@ -15,7 +15,7 @@ import java.util.HashMap;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import static akka.Done.done;
import static org.apache.pekko.Done.done;
import static org.junit.Assert.assertEquals;
public class ActorTestKitTest extends JUnitSuite {

View file

@ -2,15 +2,15 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl;
package org.apache.pekko.actor.testkit.typed.javadsl;
import akka.Done;
import akka.actor.testkit.typed.CapturedLogEvent;
import akka.actor.testkit.typed.Effect;
import akka.actor.typed.ActorRef;
import akka.actor.typed.Behavior;
import akka.actor.typed.Props;
import akka.actor.typed.javadsl.Behaviors;
import org.apache.pekko.Done;
import org.apache.pekko.actor.testkit.typed.CapturedLogEvent;
import org.apache.pekko.actor.testkit.typed.Effect;
import org.apache.pekko.actor.typed.ActorRef;
import org.apache.pekko.actor.typed.Behavior;
import org.apache.pekko.actor.typed.Props;
import org.apache.pekko.actor.typed.javadsl.Behaviors;
import org.junit.Ignore;
import org.junit.Test;
import org.scalatestplus.junit.JUnitSuite;
@ -66,10 +66,11 @@ public class BehaviorTestKitTest extends JUnitSuite {
public static class CreateMessageAdapter implements Command {
private final Class<Object> clazz;
private final akka.japi.function.Function<Object, Command> f;
private final org.apache.pekko.japi.function.Function<Object, Command> f;
@SuppressWarnings("unchecked")
public CreateMessageAdapter(Class clazz, akka.japi.function.Function<Object, Command> f) {
public CreateMessageAdapter(
Class clazz, org.apache.pekko.japi.function.Function<Object, Command> f) {
this.clazz = clazz;
this.f = f;
}

View file

@ -2,10 +2,10 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl;
package org.apache.pekko.actor.testkit.typed.javadsl;
import akka.actor.testkit.typed.LoggingEvent;
import akka.actor.testkit.typed.TestException;
import org.apache.pekko.actor.testkit.typed.LoggingEvent;
import org.apache.pekko.actor.testkit.typed.TestException;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;

View file

@ -2,14 +2,14 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl;
package org.apache.pekko.actor.testkit.typed.javadsl;
import java.time.Duration;
import java.util.Arrays;
import java.util.List;
import akka.actor.testkit.typed.scaladsl.TestProbeSpec;
import akka.actor.testkit.typed.scaladsl.TestProbeSpec.*;
import org.apache.pekko.actor.testkit.typed.scaladsl.TestProbeSpec;
import org.apache.pekko.actor.testkit.typed.scaladsl.TestProbeSpec.*;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
@ -27,7 +27,7 @@ public class TestProbeTest extends JUnitSuite {
public void testReceiveMessage() {
TestProbe<EventT> probe = TestProbe.create(testKit.system());
List<EventT> eventsT = akka.japi.Util.javaArrayList(TestProbeSpec.eventsT(10));
List<EventT> eventsT = org.apache.pekko.japi.Util.javaArrayList(TestProbeSpec.eventsT(10));
eventsT.forEach(
e -> {
@ -42,7 +42,7 @@ public class TestProbeTest extends JUnitSuite {
public void testReceiveMessageMaxDuration() {
TestProbe<EventT> probe = TestProbe.create(testKit.system());
List<EventT> eventsT = akka.japi.Util.javaArrayList(TestProbeSpec.eventsT(2));
List<EventT> eventsT = org.apache.pekko.japi.Util.javaArrayList(TestProbeSpec.eventsT(2));
eventsT.forEach(
e -> {

View file

@ -15,16 +15,16 @@
<!--
Logging from tests are silenced by this appender. When there is a test failure
the captured logging events are flushed to the appenders defined for the
akka.actor.testkit.typed.internal.CapturingAppenderDelegate logger.
org.apache.pekko.actor.testkit.typed.internal.CapturingAppenderDelegate logger.
-->
<appender name="CapturingAppender" class="akka.actor.testkit.typed.internal.CapturingAppender" />
<appender name="CapturingAppender" class="org.apache.pekko.actor.testkit.typed.internal.CapturingAppender" />
<!--
The appenders defined for this CapturingAppenderDelegate logger are used
when there is a test failure and all logging events from the test are
flushed to these appenders.
-->
<logger name="akka.actor.testkit.typed.internal.CapturingAppenderDelegate" >
<logger name="org.apache.pekko.actor.testkit.typed.internal.CapturingAppenderDelegate" >
<appender-ref ref="STDOUT"/>
</logger>

View file

@ -2,18 +2,19 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package docs.akka.actor.testkit.typed.scaladsl
package docs.org.apache.pekko.actor.testkit.typed.scaladsl
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.Scheduler
import org.apache.pekko
import pekko.actor.testkit.typed.scaladsl.LogCapturing
import pekko.actor.typed.Scheduler
//#test-header
import akka.actor.testkit.typed.scaladsl.ActorTestKit
import org.apache.pekko.actor.testkit.typed.scaladsl.ActorTestKit
//#test-header
import akka.actor.typed._
import akka.actor.typed.scaladsl.AskPattern._
import akka.actor.typed.scaladsl.Behaviors
import akka.util.Timeout
import pekko.actor.typed._
import pekko.actor.typed.scaladsl.AskPattern._
import pekko.actor.typed.scaladsl.Behaviors
import pekko.util.Timeout
//#test-header
import org.scalatest.BeforeAndAfterAll
import org.scalatest.matchers.should.Matchers

View file

@ -2,15 +2,16 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package docs.akka.actor.testkit.typed.scaladsl
package docs.org.apache.pekko.actor.testkit.typed.scaladsl
//#manual-scheduling-simple
import scala.concurrent.duration._
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.ManualTime
import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.scaladsl.Behaviors
import org.apache.pekko
import pekko.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import pekko.actor.testkit.typed.scaladsl.ManualTime
import pekko.actor.testkit.typed.scaladsl.TestProbe
import pekko.actor.testkit.typed.scaladsl.LogCapturing
import pekko.actor.typed.scaladsl.Behaviors
import org.scalatest.wordspec.AnyWordSpecLike
class ManualTimerExampleSpec

View file

@ -2,15 +2,16 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package docs.akka.actor.testkit.typed.scaladsl
package docs.org.apache.pekko.actor.testkit.typed.scaladsl
import scala.annotation.nowarn
import docs.akka.actor.testkit.typed.scaladsl.AsyncTestingExampleSpec.Echo
import docs.org.apache.pekko.actor.testkit.typed.scaladsl.AsyncTestingExampleSpec.Echo
//#log-capturing
import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.apache.pekko
import pekko.actor.testkit.typed.scaladsl.LogCapturing
//#scalatest-integration
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import pekko.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import org.scalatest.wordspec.AnyWordSpecLike
//#scalatest-integration

View file

@ -2,15 +2,16 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package docs.akka.actor.testkit.typed.scaladsl
package docs.org.apache.pekko.actor.testkit.typed.scaladsl
//#imports
import akka.actor.testkit.typed.CapturedLogEvent
import akka.actor.testkit.typed.Effect._
import akka.actor.testkit.typed.scaladsl.BehaviorTestKit
import akka.actor.testkit.typed.scaladsl.TestInbox
import akka.actor.typed._
import akka.actor.typed.scaladsl._
import org.apache.pekko
import pekko.actor.testkit.typed.CapturedLogEvent
import pekko.actor.testkit.typed.Effect._
import pekko.actor.testkit.typed.scaladsl.BehaviorTestKit
import pekko.actor.testkit.typed.scaladsl.TestInbox
import pekko.actor.typed._
import pekko.actor.typed.scaladsl._
import com.typesafe.config.ConfigFactory
import org.slf4j.event.Level

View file

@ -2,7 +2,7 @@
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package docs.akka.actor.testkit.typed.scaladsl
package docs.org.apache.pekko.actor.testkit.typed.scaladsl
object TestConfigExample {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import scala.concurrent.Promise
@ -11,13 +11,14 @@ import org.scalatest.BeforeAndAfterAll
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.{ AnyWordSpec, AnyWordSpecLike }
import akka.Done
import akka.actor.Dropped
import akka.actor.UnhandledMessage
import akka.actor.testkit.typed.internal.ActorTestKitGuardian
import akka.actor.typed.ActorSystem
import akka.actor.typed.eventstream.EventStream
import akka.actor.typed.scaladsl.Behaviors
import org.apache.pekko
import pekko.Done
import pekko.actor.Dropped
import pekko.actor.UnhandledMessage
import pekko.actor.testkit.typed.internal.ActorTestKitGuardian
import pekko.actor.typed.ActorSystem
import pekko.actor.typed.eventstream.EventStream
import pekko.actor.typed.scaladsl.Behaviors
class ActorTestKitSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing {
@ -114,14 +115,14 @@ class ActorTestKitSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike wi
}
"have unique names for probes across untyped testkit" in {
import akka.actor.typed.scaladsl.adapter._
import pekko.actor.typed.scaladsl.adapter._
createTestProbe()
akka.testkit.TestProbe()(system.toClassic)
pekko.testkit.TestProbe()(system.toClassic)
// not throw
}
"allow subscriptions for unhandled" in {
import akka.actor.typed.scaladsl.adapter._
import pekko.actor.typed.scaladsl.adapter._
val probe = testKit.createUnhandledMessageProbe()
system.eventStream ! EventStream.Publish(UnhandledMessage("message", probe.ref.toClassic, probe.ref.toClassic))
probe.receiveMessage().message should ===("message")
@ -135,7 +136,7 @@ class ActorTestKitSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike wi
"allow subscriptions for dropped messages" in {
val probe = testKit.createDroppedMessageProbe()
system.eventStream ! EventStream.Publish(Dropped("message", "it had gone bad", akka.actor.ActorRef.noSender))
system.eventStream ! EventStream.Publish(Dropped("message", "it had gone bad", pekko.actor.ActorRef.noSender))
probe.receiveMessage().message should ===("message")
}

View file

@ -2,17 +2,18 @@
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import akka.Done
import akka.actor.Address
import akka.actor.testkit.typed.Effect._
import akka.actor.testkit.typed.scaladsl.BehaviorTestKitSpec.Parent._
import akka.actor.testkit.typed.scaladsl.BehaviorTestKitSpec.{ Child, Parent }
import akka.actor.testkit.typed.{ CapturedLogEvent, Effect }
import akka.actor.typed.receptionist.{ Receptionist, ServiceKey }
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorRef, Behavior, Props, Terminated }
import org.apache.pekko
import pekko.Done
import pekko.actor.Address
import pekko.actor.testkit.typed.Effect._
import pekko.actor.testkit.typed.scaladsl.BehaviorTestKitSpec.Parent._
import pekko.actor.testkit.typed.scaladsl.BehaviorTestKitSpec.{ Child, Parent }
import pekko.actor.testkit.typed.{ CapturedLogEvent, Effect }
import pekko.actor.typed.receptionist.{ Receptionist, ServiceKey }
import pekko.actor.typed.scaladsl.Behaviors
import pekko.actor.typed.{ ActorRef, Behavior, Props, Terminated }
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
import org.slf4j.event.Level

View file

@ -2,12 +2,12 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import org.scalatest.wordspec.AnyWordSpecLike
import org.slf4j.event.Level
import akka.actor.testkit.typed.LoggingEvent
import org.apache.pekko.actor.testkit.typed.LoggingEvent
class LoggingTestKitSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import java.util.concurrent.atomic.AtomicInteger
@ -11,7 +11,7 @@ import scala.concurrent.Future
import org.scalatest.wordspec.AnyWordSpecLike
import org.slf4j.LoggerFactory
import akka.actor.testkit.typed.TestException
import org.apache.pekko.actor.testkit.typed.TestException
class TestAppenderSpec
extends ScalaTestWithActorTestKit(

View file

@ -2,14 +2,14 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
package org.apache.pekko.actor.testkit.typed.scaladsl
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import org.scalatest.wordspec.AnyWordSpecLike
import akka.actor.typed.scaladsl.Behaviors
import org.apache.pekko.actor.typed.scaladsl.Behaviors
class TestProbeSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing {

View file

@ -2,11 +2,11 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor;
package org.apache.pekko.actor;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.AkkaSpec;
import akka.testkit.TestProbe;
import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
import org.apache.pekko.testkit.AkkaSpec;
import org.apache.pekko.testkit.TestProbe;
import org.junit.ClassRule;
import org.junit.Test;
import org.scalatestplus.junit.JUnitSuite;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor;
package org.apache.pekko.actor;
import static org.junit.Assert.*;
import static java.util.stream.Collectors.toCollection;
@ -13,7 +13,7 @@ import java.util.stream.IntStream;
import org.junit.Assert;
import org.junit.Test;
import akka.japi.Creator;
import org.apache.pekko.japi.Creator;
import org.scalatestplus.junit.JUnitSuite;

View file

@ -2,14 +2,14 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor;
package org.apache.pekko.actor;
import java.time.Duration;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.AkkaSpec;
import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
import org.apache.pekko.testkit.AkkaSpec;
import org.junit.ClassRule;
import org.junit.Test;

View file

@ -2,9 +2,9 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor;
package org.apache.pekko.actor;
import akka.testkit.AkkaJUnitActorSystemResource;
import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor;
package org.apache.pekko.actor;
import org.junit.Test;
import org.scalatestplus.junit.JUnitSuite;

View file

@ -2,21 +2,21 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor;
package org.apache.pekko.actor;
import akka.event.Logging;
import akka.event.Logging.LoggerInitialized;
import akka.japi.Creator;
import akka.japi.Pair;
import akka.japi.Util;
import akka.japi.tuple.Tuple22;
import akka.japi.tuple.Tuple4;
import akka.routing.GetRoutees;
import akka.routing.FromConfig;
import akka.routing.NoRouter;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.AkkaSpec;
import akka.testkit.TestProbe;
import org.apache.pekko.event.Logging;
import org.apache.pekko.event.Logging.LoggerInitialized;
import org.apache.pekko.japi.Creator;
import org.apache.pekko.japi.Pair;
import org.apache.pekko.japi.Util;
import org.apache.pekko.japi.tuple.Tuple22;
import org.apache.pekko.japi.tuple.Tuple4;
import org.apache.pekko.routing.GetRoutees;
import org.apache.pekko.routing.FromConfig;
import org.apache.pekko.routing.NoRouter;
import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
import org.apache.pekko.testkit.AkkaSpec;
import org.apache.pekko.testkit.TestProbe;
import org.junit.ClassRule;
import org.junit.Test;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor;
package org.apache.pekko.actor;
public class JavaAPITestActor extends UntypedAbstractActor {
public static String ANSWER = "got it!";

View file

@ -2,11 +2,11 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor;
package org.apache.pekko.actor;
import akka.testkit.AkkaJUnitActorSystemResource;
import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
import org.junit.*;
import akka.testkit.AkkaSpec;
import org.apache.pekko.testkit.AkkaSpec;
import com.typesafe.config.ConfigFactory;
import org.scalatestplus.junit.JUnitSuite;
@ -66,7 +66,7 @@ public class JavaExtension extends JUnitSuite {
new AkkaJUnitActorSystemResource(
"JavaExtension",
ConfigFactory.parseString(
"akka.extensions = [ \"akka.actor.JavaExtension$TestExtensionId\" ]")
"akka.extensions = [ \"org.apache.pekko.actor.JavaExtension$TestExtensionId\" ]")
.withFallback(AkkaSpec.testConf()));
private final ActorSystem system = actorSystemResource.getSystem();

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor;
package org.apache.pekko.actor;
public class NonPublicClass {
public static Props createProps() {

View file

@ -2,9 +2,9 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor;
package org.apache.pekko.actor;
import akka.japi.Creator;
import org.apache.pekko.japi.Creator;
public class NonStaticCreator implements Creator<UntypedAbstractActor> {
@Override

View file

@ -2,10 +2,10 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor;
package org.apache.pekko.actor;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.TestProbe;
import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
import org.apache.pekko.testkit.TestProbe;
import org.junit.ClassRule;
import org.junit.Test;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor;
package org.apache.pekko.actor;
import static org.junit.Assert.*;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor;
package org.apache.pekko.actor;
import java.lang.annotation.*;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.setup;
package org.apache.pekko.actor.setup;
import org.junit.Test;
import org.scalatestplus.junit.JUnitSuite;

View file

@ -2,12 +2,12 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.dispatch;
package org.apache.pekko.dispatch;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.actor.ActorSystem;
import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
import org.apache.pekko.actor.ActorSystem;
import akka.japi.*;
import org.apache.pekko.japi.*;
import org.junit.ClassRule;
import org.scalatestplus.junit.JUnitSuite;
import scala.concurrent.Await;
@ -23,9 +23,9 @@ import java.util.LinkedList;
import java.lang.Iterable;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import static akka.japi.Util.classTag;
import static org.apache.pekko.japi.Util.classTag;
import akka.testkit.AkkaSpec;
import org.apache.pekko.testkit.AkkaSpec;
public class JavaFutureTests extends JUnitSuite {

View file

@ -2,9 +2,9 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.event;
package org.apache.pekko.event;
import akka.actor.AbstractActor;
import org.apache.pekko.actor.AbstractActor;
import java.util.Collections;
import java.util.LinkedHashMap;

View file

@ -2,17 +2,17 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.event;
package org.apache.pekko.event;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.Props;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.event.Logging.Error;
import akka.event.ActorWithMDC.Log;
import static akka.event.Logging.*;
import org.apache.pekko.actor.ActorRef;
import org.apache.pekko.actor.ActorSystem;
import org.apache.pekko.actor.Props;
import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
import org.apache.pekko.event.Logging.Error;
import org.apache.pekko.event.ActorWithMDC.Log;
import static org.apache.pekko.event.Logging.*;
import akka.testkit.javadsl.TestKit;
import org.apache.pekko.testkit.javadsl.TestKit;
import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory;
import org.junit.Before;

View file

@ -2,12 +2,12 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.japi;
package org.apache.pekko.japi;
import akka.actor.ExtendedActorSystem;
import akka.event.LoggingAdapter;
import akka.event.NoLogging;
import akka.serialization.JavaSerializer;
import org.apache.pekko.actor.ExtendedActorSystem;
import org.apache.pekko.event.LoggingAdapter;
import org.apache.pekko.event.NoLogging;
import org.apache.pekko.serialization.JavaSerializer;
import org.junit.Test;
import org.scalatestplus.junit.JUnitSuite;

View file

@ -2,10 +2,10 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.japi;
package org.apache.pekko.japi;
import akka.japi.pf.FI;
import akka.japi.pf.Match;
import org.apache.pekko.japi.pf.FI;
import org.apache.pekko.japi.pf.Match;
import org.junit.Assert;
import org.junit.Test;
import org.scalatestplus.junit.JUnitSuite;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.japi;
package org.apache.pekko.japi;
import org.junit.Assert;
import org.junit.Test;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.japi.pf;
package org.apache.pekko.japi.pf;
import org.junit.Test;
import org.scalatestplus.junit.JUnitSuite;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2017-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.japi.pf;
package org.apache.pekko.japi.pf;
import java.util.Arrays;
import java.util.List;
@ -10,7 +10,7 @@ import org.junit.Test;
import org.junit.Before;
import org.scalatestplus.junit.JUnitSuite;
import akka.actor.AbstractActor.Receive;
import org.apache.pekko.actor.AbstractActor.Receive;
import static org.junit.Assert.*;

View file

@ -2,12 +2,12 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.pattern;
package org.apache.pekko.pattern;
import akka.actor.*;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.AkkaSpec;
import akka.util.JavaDurationConverters;
import org.apache.pekko.actor.*;
import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
import org.apache.pekko.testkit.AkkaSpec;
import org.apache.pekko.util.JavaDurationConverters;
import org.junit.ClassRule;
import org.junit.Test;
import org.scalatestplus.junit.JUnitSuite;

View file

@ -2,14 +2,14 @@
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.pattern;
package org.apache.pekko.pattern;
import akka.actor.*;
import akka.dispatch.Futures;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.AkkaSpec;
import akka.testkit.TestProbe;
import akka.util.Timeout;
import org.apache.pekko.actor.*;
import org.apache.pekko.dispatch.Futures;
import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
import org.apache.pekko.testkit.AkkaSpec;
import org.apache.pekko.testkit.TestProbe;
import org.apache.pekko.util.Timeout;
import org.junit.ClassRule;
import org.junit.Test;
import org.scalatestplus.junit.JUnitSuite;
@ -23,8 +23,8 @@ import java.util.concurrent.*;
import java.time.Duration;
import java.util.concurrent.atomic.AtomicInteger;
import static akka.pattern.Patterns.ask;
import static akka.pattern.Patterns.pipe;
import static org.apache.pekko.pattern.Patterns.ask;
import static org.apache.pekko.pattern.Patterns.pipe;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.junit.Assert.assertEquals;

View file

@ -2,13 +2,13 @@
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.pattern;
package org.apache.pekko.pattern;
import akka.actor.ActorRef;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.AkkaSpec;
import akka.testkit.TestException;
import akka.testkit.TestProbe;
import org.apache.pekko.actor.ActorRef;
import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
import org.apache.pekko.testkit.AkkaSpec;
import org.apache.pekko.testkit.TestException;
import org.apache.pekko.testkit.TestProbe;
import org.junit.Assert;
import org.junit.ClassRule;
import org.junit.Test;
@ -19,7 +19,7 @@ import java.util.concurrent.CompletionStage;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import static akka.pattern.Patterns.askWithStatus;
import static org.apache.pekko.pattern.Patterns.askWithStatus;
import static org.junit.Assert.*;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util;
package org.apache.pekko.util;
import org.junit.Test;
import org.scalatestplus.junit.JUnitSuite;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util;
package org.apache.pekko.util;
import org.junit.Test;
import org.scalatestplus.junit.JUnitSuite;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util;
package org.apache.pekko.util;
import java.io.Serializable;

View file

@ -1,6 +1,6 @@
akka {
# for the akka.actor.ExtensionSpec
library-extensions += "akka.actor.InstanceCountingExtension"
# for the org.apache.pekko.actor.ExtensionSpec
library-extensions += "org.apache.pekko.actor.InstanceCountingExtension"
}
# FIXME Some test depend on this setting when running on windows.

View file

@ -2,10 +2,11 @@
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util
package org.apache.pekko.util
import akka.testkit.AkkaSpec
import akka.util.LineNumbers._
import org.apache.pekko
import pekko.testkit.AkkaSpec
import pekko.util.LineNumbers._
class LineNumberSpec extends AkkaSpec {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util
package org.apache.pekko.util
import org.scalactic.TypeCheckedTripleEquals
import org.scalatest.matchers.should.Matchers

View file

@ -2,10 +2,11 @@
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util
package org.apache.pekko.util
import akka.testkit.AkkaSpec
import akka.util.LineNumbers._
import org.apache.pekko
import pekko.testkit.AkkaSpec
import pekko.util.LineNumbers._
class LineNumberSpec extends AkkaSpec {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util
package org.apache.pekko.util
import org.scalactic.TypeCheckedTripleEquals
import org.scalatest.matchers.should.Matchers

View file

@ -2,10 +2,11 @@
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util
package org.apache.pekko.util
import akka.testkit.AkkaSpec
import akka.util.LineNumbers._
import org.apache.pekko
import pekko.testkit.AkkaSpec
import pekko.util.LineNumbers._
class LineNumberSpec extends AkkaSpec {

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