Scala 3 cross compatible akka-actor / akka-testkit / akka-actor-tests (#29956)

* Remove @switch when it doesn't take effect
* Use ActorRef.noSender
* Minor tweaks to SchedulerSpec
* Disambiguate TypedActor for Scala 3
* Bump ScalaTest to a version compatible with Scala 3
* Bump ScalaCheck
* Disambiguate Event in SupervisorHierarchySpec
* Scala 3 compatible EventBusSpec
* Prevent private unused variables to be erased by Scala 3
* Bump mockito
* Explicit actorRef2Scala import
* restore original .scalafix.conf
* Scala 3 compatible tailrec
* Reminder to re add switch annotation in case
* Move to nowarn instead of silencer
* Bump to Scala 2.12.13
* Cross compatible annotations
* fix docs generation
* adapt the build for Scala 3
* fix errors but bus
* remove more SerialVersion from trait
* scalacheck only from scalatest
* cross-compile akka-actor-tests
* restore cross-compilation
* early initializers workaround
* scalacheck switch
* cross compatible FSM.State class
* cross compatible LARS spec
* Change results to pass LineNumberSpec
* fix stackoverflow in AsyncDnsResolverIntegrationSpec
* FSM.State unapply
* fix Scala 2.13 mima
* SerialVersionRemover compiler plugin
* removed unused nowarns
This commit is contained in:
Andrea Peruffo 2021-02-01 15:38:29 +00:00 committed by GitHub
parent 0f27ed6189
commit da70e0ccd4
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
370 changed files with 4058 additions and 1373 deletions

1
.gitignore vendored
View file

@ -17,6 +17,7 @@
.*.swp .*.swp
.DS_Store .DS_Store
.bloop/ .bloop/
.bsp/
.cache .cache
.classpath .classpath
.codefellow .codefellow

View file

@ -9,7 +9,7 @@ import java.util.concurrent.{ CompletionStage, ThreadFactory }
import scala.compat.java8.FutureConverters import scala.compat.java8.FutureConverters
import scala.concurrent._ import scala.concurrent._
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.slf4j.Logger import org.slf4j.Logger
import org.slf4j.LoggerFactory import org.slf4j.LoggerFactory
@ -35,7 +35,7 @@ import akka.annotation.InternalApi
/** /**
* INTERNAL API * INTERNAL API
*/ */
@silent @nowarn
@InternalApi private[akka] final class ActorSystemStub(val name: String) @InternalApi private[akka] final class ActorSystemStub(val name: String)
extends ActorSystem[Nothing] extends ActorSystem[Nothing]
with ActorRef[Nothing] with ActorRef[Nothing]

View file

@ -4,7 +4,7 @@
package docs.akka.actor.testkit.typed.scaladsl package docs.akka.actor.testkit.typed.scaladsl
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import docs.akka.actor.testkit.typed.scaladsl.AsyncTestingExampleSpec.Echo import docs.akka.actor.testkit.typed.scaladsl.AsyncTestingExampleSpec.Echo
//#log-capturing //#log-capturing
@ -16,7 +16,7 @@ import org.scalatest.wordspec.AnyWordSpecLike
//#scalatest-integration //#scalatest-integration
//#log-capturing //#log-capturing
@silent @nowarn
//#scalatest-integration //#scalatest-integration
class ScalaTestIntegrationExampleSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike { class ScalaTestIntegrationExampleSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike {

View file

@ -4,7 +4,7 @@
package akka.pattern; package akka.pattern;
import akka.actor.Actor; import akka.actor.ActorRef;
import akka.testkit.AkkaJUnitActorSystemResource; import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
import akka.testkit.TestException; import akka.testkit.TestException;
@ -81,7 +81,7 @@ public class StatusReplyTest extends JUnitSuite {
CompletionStage<Object> response = askWithStatus(probe.ref(), "request", Duration.ofSeconds(3)); CompletionStage<Object> response = askWithStatus(probe.ref(), "request", Duration.ofSeconds(3));
probe.expectMsg("request"); probe.expectMsg("request");
probe.lastSender().tell(StatusReply.success("woho"), Actor.noSender()); probe.lastSender().tell(StatusReply.success("woho"), ActorRef.noSender());
Object result = response.toCompletableFuture().get(3, TimeUnit.SECONDS); Object result = response.toCompletableFuture().get(3, TimeUnit.SECONDS);
assertEquals("woho", result); assertEquals("woho", result);
@ -93,7 +93,7 @@ public class StatusReplyTest extends JUnitSuite {
CompletionStage<Object> response = askWithStatus(probe.ref(), "request", Duration.ofSeconds(3)); CompletionStage<Object> response = askWithStatus(probe.ref(), "request", Duration.ofSeconds(3));
probe.expectMsg("request"); probe.expectMsg("request");
probe.lastSender().tell(StatusReply.error("boho"), Actor.noSender()); probe.lastSender().tell(StatusReply.error("boho"), ActorRef.noSender());
try { try {
Object result = response.toCompletableFuture().get(3, TimeUnit.SECONDS); Object result = response.toCompletableFuture().get(3, TimeUnit.SECONDS);
@ -110,7 +110,7 @@ public class StatusReplyTest extends JUnitSuite {
CompletionStage<Object> response = askWithStatus(probe.ref(), "request", Duration.ofSeconds(3)); CompletionStage<Object> response = askWithStatus(probe.ref(), "request", Duration.ofSeconds(3));
probe.expectMsg("request"); probe.expectMsg("request");
probe.lastSender().tell(StatusReply.error(new TestException("boho")), Actor.noSender()); probe.lastSender().tell(StatusReply.error(new TestException("boho")), ActorRef.noSender());
try { try {
Object result = response.toCompletableFuture().get(3, TimeUnit.SECONDS); Object result = response.toCompletableFuture().get(3, TimeUnit.SECONDS);

View file

@ -0,0 +1,58 @@
/*
* Copyright (C) 2014-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util
import akka.testkit.AkkaSpec
import akka.util.LineNumbers._
class LineNumberSpec extends AkkaSpec {
"LineNumbers" when {
"writing Scala" must {
import LineNumberSpecCodeForScala._
"work for small functions" in {
LineNumbers(oneline) should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 13, 13))
}
"work for larger functions" in {
val result = LineNumbers(twoline)
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 15, 15))
}
"work for partial functions" in {
LineNumbers(partial) should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 20, 22))
}
"work for `def`" in {
val result = LineNumbers(method("foo"))
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 26, 27))
}
}
"writing Java" must {
val l = new LineNumberSpecCodeForJava
"work for small functions" in {
// because how java Lambdas are implemented/designed
LineNumbers(l.f1()) should ===(SourceFileLines("LineNumberSpecCodeForJava.java", 20, 20))
}
"work for larger functions" in {
// because how java Lambdas are implemented/designed
LineNumbers(l.f2()) should ===(SourceFileLines("LineNumberSpecCodeForJava.java", 25, 26))
}
"work for anonymous classes" in {
LineNumbers(l.f3()) should ===(SourceFileLines("LineNumberSpecCodeForJava.java", 31, 36))
}
}
}
}

View file

@ -0,0 +1,71 @@
/*
* Copyright (C) 2015-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util
import org.scalactic.TypeCheckedTripleEquals
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
object TypedMultiMapSpec {
trait AbstractKey { type Type }
final case class Key[T](t: T) extends AbstractKey { final override type Type = T }
final case class MyValue[T](t: T)
type KV[K <: AbstractKey] = MyValue[K#Type]
}
class TypedMultiMapSpec extends AnyWordSpec with Matchers with TypeCheckedTripleEquals {
import TypedMultiMapSpec._
"A TypedMultiMap" must {
"retain and remove values for the same key" in {
val m1 = TypedMultiMap.empty[AbstractKey, KV]
val m2 = m1.inserted(Key(1))(MyValue(42))
m2.get(Key(1)) should ===(Set(MyValue(42)))
m2.removed(Key(1))(MyValue(42)).get(Key(1)) should ===(Set.empty[MyValue[Int]])
val m3 = m2.inserted(Key(1))(MyValue(43))
m3.get(Key(1)) should ===(Set(MyValue(42), MyValue(43)))
m3.removed(Key(1))(MyValue(42)).get(Key(1)) should ===(Set(MyValue(43)))
}
"retain and remove values for multiple keys" in {
val m1 = TypedMultiMap.empty[AbstractKey, KV]
val m2 = m1.inserted(Key(1))(MyValue(42)).inserted(Key(2))(MyValue(43))
m2.get(Key(1)) should ===(Set(MyValue(42)))
m2.removed(Key(1))(MyValue(42)).get(Key(1)) should ===(Set.empty[MyValue[Int]])
m2.get(Key(2)) should ===(Set(MyValue(43)))
m2.removed(Key(1))(MyValue(42)).get(Key(2)) should ===(Set(MyValue(43)))
}
"remove a value from all keys" in {
val m1 = TypedMultiMap.empty[AbstractKey, KV]
val m2 = m1.inserted(Key(1))(MyValue(42)).inserted(Key(2))(MyValue(43)).inserted(Key(2))(MyValue(42))
val m3 = m2.valueRemoved(MyValue(42))
m3.get(Key(1)) should ===(Set.empty[MyValue[Int]])
m3.get(Key(2)) should ===(Set(MyValue(43)))
m3.keySet should ===(Set[AbstractKey](Key(2)))
}
"remove all values from a key" in {
val m1 = TypedMultiMap.empty[AbstractKey, KV]
val m2 = m1.inserted(Key(1))(MyValue(42)).inserted(Key(2))(MyValue(43)).inserted(Key(2))(MyValue(42))
val m3 = m2.keyRemoved(Key(1))
m3.get(Key(1)) should ===(Set.empty[MyValue[Int]])
m3.get(Key(2)) should ===(Set(MyValue(42), MyValue(43)))
m3.keySet should ===(Set[AbstractKey](Key(2)))
}
"reject invalid insertions" in {
"TypedMultiMap.empty[AbstractKey, KV].inserted(Key(1))(MyValue(42L))" shouldNot compile
}
"reject invalid removals" in {
"TypedMultiMap.empty[AbstractKey, KV].removed(Key(1))(MyValue(42L))" shouldNot compile
}
}
}

View file

@ -0,0 +1,58 @@
/*
* Copyright (C) 2014-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util
import akka.testkit.AkkaSpec
import akka.util.LineNumbers._
class LineNumberSpec extends AkkaSpec {
"LineNumbers" when {
"writing Scala" must {
import LineNumberSpecCodeForScala._
"work for small functions" in {
LineNumbers(oneline) should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 13, 13))
}
"work for larger functions" in {
val result = LineNumbers(twoline)
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 15, 17))
}
"work for partial functions" in {
LineNumbers(partial) should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 21, 22))
}
"work for `def`" in {
val result = LineNumbers(method("foo"))
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 25, 27))
}
}
"writing Java" must {
val l = new LineNumberSpecCodeForJava
"work for small functions" in {
// because how java Lambdas are implemented/designed
LineNumbers(l.f1()) should ===(SourceFileLines("LineNumberSpecCodeForJava.java", 20, 20))
}
"work for larger functions" in {
// because how java Lambdas are implemented/designed
LineNumbers(l.f2()) should ===(SourceFileLines("LineNumberSpecCodeForJava.java", 25, 26))
}
"work for anonymous classes" in {
LineNumbers(l.f3()) should ===(SourceFileLines("LineNumberSpecCodeForJava.java", 31, 36))
}
}
}
}

View file

@ -9,7 +9,9 @@ import java.util.concurrent.CountDownLatch
import java.util.concurrent.atomic._ import java.util.concurrent.atomic._
import scala.concurrent.{ Await, Future } import scala.concurrent.{ Await, Future }
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
import akka.actor.Actor._ import akka.actor.Actor._
import akka.pattern.ask import akka.pattern.ask
import akka.testkit._ import akka.testkit._
@ -153,6 +155,7 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS
"have a non null context after termination" in { "have a non null context after termination" in {
class StopBeforeFutureFinishes(val latch: CountDownLatch) extends Actor { class StopBeforeFutureFinishes(val latch: CountDownLatch) extends Actor {
import context.dispatcher import context.dispatcher
import akka.pattern._ import akka.pattern._
override def receive: Receive = { override def receive: Receive = {

View file

@ -92,11 +92,11 @@ class ActorPathSpec extends AnyWordSpec with Matchers {
"detect valid and invalid chars in host names when not using AddressFromURIString, e.g. docker host given name" in { "detect valid and invalid chars in host names when not using AddressFromURIString, e.g. docker host given name" in {
Seq( Seq(
Address("akka", "sys", Some("valid"), Some(0)), Address("akka", "sys", "valid", 0),
Address("akka", "sys", Some("is_valid.org"), Some(0)), Address("akka", "sys", "is_valid.org", 0),
Address("akka", "sys", Some("fu.is_valid.org"), Some(0))).forall(_.hasInvalidHostCharacters) shouldBe false Address("akka", "sys", "fu.is_valid.org", 0)).forall(_.hasInvalidHostCharacters) shouldBe false
Seq(Address("akka", "sys", Some("in_valid"), Some(0)), Address("akka", "sys", Some("invalid._org"), Some(0))) Seq(Address("akka", "sys", "in_valid", 0), Address("akka", "sys", "invalid._org", 0))
.forall(_.hasInvalidHostCharacters) shouldBe true .forall(_.hasInvalidHostCharacters) shouldBe true
intercept[MalformedURLException](AddressFromURIString("akka://sys@in_valid:5001")) intercept[MalformedURLException](AddressFromURIString("akka://sys@in_valid:5001"))
@ -104,19 +104,19 @@ class ActorPathSpec extends AnyWordSpec with Matchers {
"not fail fast if the check is called on valid chars in host names" in { "not fail fast if the check is called on valid chars in host names" in {
Seq( Seq(
Address("akka", "sys", Some("localhost"), Some(0)), Address("akka", "sys", "localhost", 0),
Address("akka", "sys", Some("is_valid.org"), Some(0)), Address("akka", "sys", "is_valid.org", 0),
Address("akka", "sys", Some("fu.is_valid.org"), Some(0))).foreach(_.checkHostCharacters()) Address("akka", "sys", "fu.is_valid.org", 0)).foreach(_.checkHostCharacters())
} }
"fail fast if the check is called when invalid chars are in host names" in { "fail fast if the check is called when invalid chars are in host names" in {
Seq( Seq(
Address("akka", "sys", Some("localhost"), Some(0)), Address("akka", "sys", "localhost", 0),
Address("akka", "sys", Some("is_valid.org"), Some(0)), Address("akka", "sys", "is_valid.org", 0),
Address("akka", "sys", Some("fu.is_valid.org"), Some(0))).foreach(_.checkHostCharacters()) Address("akka", "sys", "fu.is_valid.org", 0)).foreach(_.checkHostCharacters())
intercept[IllegalArgumentException](Address("akka", "sys", Some("in_valid"), Some(0)).checkHostCharacters()) intercept[IllegalArgumentException](Address("akka", "sys", "in_valid", 0).checkHostCharacters())
intercept[IllegalArgumentException](Address("akka", "sys", Some("invalid._org"), Some(0)).checkHostCharacters()) intercept[IllegalArgumentException](Address("akka", "sys", "invalid._org", 0).checkHostCharacters())
} }
} }
} }

View file

@ -11,7 +11,7 @@ import scala.concurrent.{ Await, Future }
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.language.postfixOps import scala.language.postfixOps
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import akka.actor.setup.ActorSystemSetup import akka.actor.setup.ActorSystemSetup
@ -64,7 +64,7 @@ object ActorSystemSpec {
} }
} }
@silent @nowarn
final case class FastActor(latch: TestLatch, testActor: ActorRef) extends Actor { final case class FastActor(latch: TestLatch, testActor: ActorRef) extends Actor {
val ref1 = context.actorOf(Props.empty) val ref1 = context.actorOf(Props.empty)
context.actorSelection(ref1.path.toString).tell(Identify(ref1), testActor) context.actorSelection(ref1.path.toString).tell(Identify(ref1), testActor)
@ -79,11 +79,11 @@ object ActorSystemSpec {
extends MessageDispatcherConfigurator(_config, _prerequisites) { extends MessageDispatcherConfigurator(_config, _prerequisites) {
private val instance = new Dispatcher( private val instance = new Dispatcher(
this, this,
config.getString("id"), this.config.getString("id"),
config.getInt("throughput"), this.config.getInt("throughput"),
config.getNanosDuration("throughput-deadline-time"), this.config.getNanosDuration("throughput-deadline-time"),
configureExecutor(), configureExecutor(),
config.getMillisDuration("shutdown-timeout")) { this.config.getMillisDuration("shutdown-timeout")) {
val doneIt = new Switch val doneIt = new Switch
override protected[akka] def registerForExecution( override protected[akka] def registerForExecution(
mbox: Mailbox, mbox: Mailbox,
@ -113,7 +113,7 @@ object ActorSystemSpec {
} }
@silent @nowarn
class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSender { class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSender {
import ActorSystemSpec.FastActor import ActorSystemSpec.FastActor

View file

@ -7,7 +7,7 @@ package akka.actor
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import language.postfixOps import language.postfixOps
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
@ -97,7 +97,7 @@ object ActorWithStashSpec {
} }
@silent @nowarn
class ActorWithStashSpec extends AkkaSpec with DefaultTimeout with BeforeAndAfterEach { class ActorWithStashSpec extends AkkaSpec with DefaultTimeout with BeforeAndAfterEach {
import ActorWithStashSpec._ import ActorWithStashSpec._

View file

@ -7,7 +7,7 @@ package akka.actor
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import language.postfixOps import language.postfixOps
import akka.actor.Props.EmptyActor import akka.actor.Props.EmptyActor
@ -107,7 +107,7 @@ object DeathWatchSpec {
} }
} }
@silent @nowarn
trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout => trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout =>
import DeathWatchSpec._ import DeathWatchSpec._

View file

@ -8,12 +8,13 @@ import java.util.concurrent.atomic.AtomicInteger
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.testkit.EventFilter
import akka.testkit.TestKit._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.scalatest.matchers.should.Matchers import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec import org.scalatest.wordspec.AnyWordSpec
import akka.testkit.EventFilter
import akka.testkit.TestKit._
object TestExtension extends ExtensionId[TestExtension] with ExtensionIdProvider { object TestExtension extends ExtensionId[TestExtension] with ExtensionIdProvider {
def lookup = this def lookup = this
def createExtension(s: ExtendedActorSystem) = new TestExtension(s) def createExtension(s: ExtendedActorSystem) = new TestExtension(s)

View file

@ -10,7 +10,7 @@ import scala.concurrent.duration._
import scala.util.Failure import scala.util.Failure
import scala.util.Success import scala.util.Success
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import language.postfixOps import language.postfixOps
import akka.testkit._ import akka.testkit._
@ -33,7 +33,7 @@ object LocalActorRefProviderSpec {
""" """
} }
@silent @nowarn
class LocalActorRefProviderSpec extends AkkaSpec(LocalActorRefProviderSpec.config) { class LocalActorRefProviderSpec extends AkkaSpec(LocalActorRefProviderSpec.config) {
"An LocalActorRefProvider" must { "An LocalActorRefProvider" must {
@ -134,7 +134,7 @@ class LocalActorRefProviderSpec extends AkkaSpec(LocalActorRefProviderSpec.confi
val actors = val actors =
for (_ <- 1 to 4) for (_ <- 1 to 4)
yield Future(system.actorOf(Props(new Actor { def receive = { case _ => } }), address)) yield Future(system.actorOf(Props(new Actor { def receive = { case _ => } }), address))
val set = Set() ++ actors.map(a => val set: Set[Any] = Set() ++ actors.map(a =>
Await.ready(a, timeout.duration).value match { Await.ready(a, timeout.duration).value match {
case Some(Success(_: ActorRef)) => 1 case Some(Success(_: ActorRef)) => 1
case Some(Failure(_: InvalidActorNameException)) => 2 case Some(Failure(_: InvalidActorNameException)) => 2

View file

@ -4,7 +4,7 @@
package akka.actor package akka.actor
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.util.unused import akka.util.unused
@ -60,7 +60,7 @@ class PropsCreationSpec extends AkkaSpec("""
"Props Java API" must { "Props Java API" must {
"work with create(creator)" in { "work with create(creator)" in {
@silent @nowarn
val p = Props.create(OneParamActorCreator) val p = Props.create(OneParamActorCreator)
system.actorOf(p) system.actorOf(p)
} }

View file

@ -9,7 +9,7 @@ import java.lang.Thread.sleep
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import language.postfixOps import language.postfixOps
import akka.pattern.ask import akka.pattern.ask
@ -19,7 +19,7 @@ import akka.testkit.EventFilter
import akka.testkit.TestEvent._ import akka.testkit.TestEvent._
import akka.testkit.TestLatch import akka.testkit.TestLatch
@silent @nowarn
class RestartStrategySpec extends AkkaSpec with DefaultTimeout { class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
override def atStartup(): Unit = { override def atStartup(): Unit = {

View file

@ -15,7 +15,7 @@ import scala.util.control.NoStackTrace
import scala.util.control.NonFatal import scala.util.control.NonFatal
import atomic.{ AtomicInteger, AtomicReference } import atomic.{ AtomicInteger, AtomicReference }
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import language.postfixOps import language.postfixOps
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
@ -684,47 +684,48 @@ class LightArrayRevolverSchedulerSpec extends AkkaSpec(SchedulerSpec.testConfRev
def reportFailure(t: Throwable): Unit = { t.printStackTrace() } def reportFailure(t: Throwable): Unit = { t.printStackTrace() }
} }
@silent @nowarn
def withScheduler(start: Long = 0L, _startTick: Int = 0, config: Config = ConfigFactory.empty)( def withScheduler(start: Long = 0L, _startTick: Int = 0, config: Config = ConfigFactory.empty)(
thunk: (Scheduler with Closeable, Driver) => Unit): Unit = { thunk: (Scheduler with Closeable, Driver) => Unit): Unit = {
import akka.actor.{ LightArrayRevolverScheduler => LARS }
val lbq = new AtomicReference[LinkedBlockingQueue[Long]](new LinkedBlockingQueue[Long]) val lbq = new AtomicReference[LinkedBlockingQueue[Long]](new LinkedBlockingQueue[Long])
val prb = TestProbe() val prb = TestProbe()
val tf = system.asInstanceOf[ActorSystemImpl].threadFactory val tf = system.asInstanceOf[ActorSystemImpl].threadFactory
val sched =
new { @volatile var time = start } with LARS(config.withFallback(system.settings.config), log, tf) {
override protected def clock(): Long = {
// println(s"clock=$time")
time
}
override protected def getShutdownTimeout: FiniteDuration = (10 seconds).dilated @volatile var time: Long = start
val sched = new LightArrayRevolverScheduler(config.withFallback(system.settings.config), log, tf) {
override protected def waitNanos(ns: Long): Unit = { override protected def clock(): Long = {
// println(s"waiting $ns") // println(s"clock=$time")
prb.ref ! ns time
try time += (lbq.get match {
case q: LinkedBlockingQueue[Long] => q.take()
case _ => 0L
})
catch {
case _: InterruptedException => Thread.currentThread.interrupt()
}
}
override protected def startTick: Int = _startTick
} }
override protected def getShutdownTimeout: FiniteDuration = (10 seconds).dilated
override protected def waitNanos(ns: Long): Unit = {
// println(s"waiting $ns")
prb.ref ! ns
try time += (lbq.get match {
case q: LinkedBlockingQueue[Long] => q.take()
case null => 0L
})
catch {
case _: InterruptedException => Thread.currentThread.interrupt()
}
}
override protected def startTick: Int = _startTick
}
val driver = new Driver { val driver = new Driver {
def wakeUp(d: FiniteDuration) = lbq.get match { def wakeUp(d: FiniteDuration) = lbq.get match {
case q: LinkedBlockingQueue[Long] => q.offer(d.toNanos) case q: LinkedBlockingQueue[Long] => q.offer(d.toNanos)
case _ => case null =>
} }
def expectWait(): FiniteDuration = probe.expectMsgType[Long].nanos def expectWait(): FiniteDuration = probe.expectMsgType[Long].nanos
def probe = prb def probe = prb
def step = sched.TickDuration def step = sched.TickDuration
def close() = lbq.getAndSet(null) match { def close() = lbq.getAndSet(null) match {
case q: LinkedBlockingQueue[Long] => q.offer(0L) case q: LinkedBlockingQueue[Long] => q.offer(0L)
case _ => case null =>
} }
} }
driver.expectWait() driver.expectWait()

View file

@ -16,7 +16,7 @@ import scala.util.Random
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import SupervisorStrategy.{ Directive, Restart, Resume, Stop } import SupervisorStrategy.{ Directive, Restart, Resume, Stop }
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import language.postfixOps import language.postfixOps
@ -464,7 +464,7 @@ object SupervisorHierarchySpec {
startWith(Idle, size * 1000) startWith(Idle, size * 1000)
when(Idle) { when(Idle) {
case Event(Init, _) => case this.Event(Init, _) =>
hierarchy = context.watch( hierarchy = context.watch(
context.actorOf(Props(new Hierarchy(size, breadth, self, 0, random)).withDispatcher("hierarchy"), "head")) context.actorOf(Props(new Hierarchy(size, breadth, self, 0, random)).withDispatcher("hierarchy"), "head"))
startSingleTimer("phase", StateTimeout, 5 seconds) startSingleTimer("phase", StateTimeout, 5 seconds)
@ -472,7 +472,7 @@ object SupervisorHierarchySpec {
} }
when(Init) { when(Init) {
case Event(Ready(ref), _) => case this.Event(Ready(ref), _) =>
if (children contains ref) { if (children contains ref) {
testActor ! "children not unique" testActor ! "children not unique"
stop() stop()
@ -481,7 +481,7 @@ object SupervisorHierarchySpec {
if (children.size == size) goto(Stress) if (children.size == size) goto(Stress)
else stay() else stay()
} }
case Event(StateTimeout, _) => case this.Event(StateTimeout, _) =>
testActor ! "did not get children list" testActor ! "did not get children list"
stop() stop()
} }
@ -514,10 +514,10 @@ object SupervisorHierarchySpec {
var ignoreNotResumedLogs = true var ignoreNotResumedLogs = true
when(Stress) { when(Stress) {
case Event(Work, _) if idleChildren.isEmpty => case this.Event(Work, _) if idleChildren.isEmpty =>
context.system.scheduler.scheduleOnce(workSchedule, self, Work)(context.dispatcher) context.system.scheduler.scheduleOnce(workSchedule, self, Work)(context.dispatcher)
stay() stay()
case Event(Work, x) if x > 0 => case this.Event(Work, x) if x > 0 =>
nextJob.next() match { nextJob.next() match {
case Ping(ref) => ref ! "ping" case Ping(ref) => ref ! "ping"
case Fail(ref, dir) => case Fail(ref, dir) =>
@ -538,15 +538,15 @@ object SupervisorHierarchySpec {
if (idleChildren.nonEmpty) self ! Work if (idleChildren.nonEmpty) self ! Work
else context.system.scheduler.scheduleOnce(workSchedule, self, Work)(context.dispatcher) else context.system.scheduler.scheduleOnce(workSchedule, self, Work)(context.dispatcher)
stay().using(x - 1) stay().using(x - 1)
case Event(Work, _) => if (pingChildren.isEmpty) goto(LastPing) else goto(Finishing) case this.Event(Work, _) => if (pingChildren.isEmpty) goto(LastPing) else goto(Finishing)
case Event(Died(path), _) => case this.Event(Died(path), _) =>
bury(path) bury(path)
stay() stay()
case Event("pong", _) => case this.Event("pong", _) =>
pingChildren -= sender() pingChildren -= sender()
idleChildren :+= sender() idleChildren :+= sender()
stay() stay()
case Event(StateTimeout, todo) => case this.Event(StateTimeout, todo) =>
log.info("dumping state due to StateTimeout") log.info("dumping state due to StateTimeout")
log.info( log.info(
"children: " + children.size + " pinged: " + pingChildren.size + " idle: " + idleChildren.size + " work: " + todo) "children: " + children.size + " pinged: " + pingChildren.size + " idle: " + idleChildren.size + " work: " + todo)
@ -563,11 +563,11 @@ object SupervisorHierarchySpec {
} }
when(Finishing) { when(Finishing) {
case Event("pong", _) => case this.Event("pong", _) =>
pingChildren -= sender() pingChildren -= sender()
idleChildren :+= sender() idleChildren :+= sender()
if (pingChildren.isEmpty) goto(LastPing) else stay() if (pingChildren.isEmpty) goto(LastPing) else stay()
case Event(Died(ref), _) => case this.Event(Died(ref), _) =>
bury(ref) bury(ref)
if (pingChildren.isEmpty) goto(LastPing) else stay() if (pingChildren.isEmpty) goto(LastPing) else stay()
} }
@ -580,11 +580,11 @@ object SupervisorHierarchySpec {
} }
when(LastPing) { when(LastPing) {
case Event("pong", _) => case this.Event("pong", _) =>
pingChildren -= sender() pingChildren -= sender()
idleChildren :+= sender() idleChildren :+= sender()
if (pingChildren.isEmpty) goto(Stopping) else stay() if (pingChildren.isEmpty) goto(Stopping) else stay()
case Event(Died(ref), _) => case this.Event(Died(ref), _) =>
bury(ref) bury(ref)
if (pingChildren.isEmpty) goto(Stopping) else stay() if (pingChildren.isEmpty) goto(Stopping) else stay()
} }
@ -596,9 +596,9 @@ object SupervisorHierarchySpec {
} }
when(Stopping, stateTimeout = 5.seconds.dilated) { when(Stopping, stateTimeout = 5.seconds.dilated) {
case Event(PongOfDeath, _) => stay() case this.Event(PongOfDeath, _) => stay()
case Event(Terminated(r), _) if r == hierarchy => case this.Event(Terminated(r), _) if r == hierarchy =>
@silent @nowarn
val undead = children.filterNot(_.isTerminated) val undead = children.filterNot(_.isTerminated)
if (undead.nonEmpty) { if (undead.nonEmpty) {
log.info("undead:\n" + undead.mkString("\n")) log.info("undead:\n" + undead.mkString("\n"))
@ -623,7 +623,7 @@ object SupervisorHierarchySpec {
testActor ! "stressTestSuccessful" testActor ! "stressTestSuccessful"
stop() stop()
} }
case Event(StateTimeout, _) => case this.Event(StateTimeout, _) =>
errors :+= self -> ErrorLog("timeout while Stopping", Vector.empty) errors :+= self -> ErrorLog("timeout while Stopping", Vector.empty)
println(system.asInstanceOf[ActorSystemImpl].printTree) println(system.asInstanceOf[ActorSystemImpl].printTree)
getErrors(hierarchy, 10) getErrors(hierarchy, 10)
@ -631,13 +631,13 @@ object SupervisorHierarchySpec {
idleChildren.foreach(println) idleChildren.foreach(println)
testActor ! "timeout in Stopping" testActor ! "timeout in Stopping"
stop() stop()
case Event(e: ErrorLog, _) => case this.Event(e: ErrorLog, _) =>
errors :+= sender() -> e errors :+= sender() -> e
goto(Failed) goto(Failed)
} }
when(GC, stateTimeout = 10 seconds) { when(GC, stateTimeout = 10 seconds) {
case Event(GCcheck(weak), _) => case this.Event(GCcheck(weak), _) =>
val next = weak.filter(_.get ne null) val next = weak.filter(_.get ne null)
if (next.nonEmpty) { if (next.nonEmpty) {
context.system.scheduler.scheduleOnce(workSchedule, self, GCcheck(next))(context.dispatcher) context.system.scheduler.scheduleOnce(workSchedule, self, GCcheck(next))(context.dispatcher)
@ -647,7 +647,7 @@ object SupervisorHierarchySpec {
testActor ! "stressTestSuccessful" testActor ! "stressTestSuccessful"
stop() stop()
} }
case Event(StateTimeout, _) => case this.Event(StateTimeout, _) =>
testActor ! "timeout in GC" testActor ! "timeout in GC"
stop() stop()
} }
@ -655,22 +655,22 @@ object SupervisorHierarchySpec {
var errors = Vector.empty[(ActorRef, ErrorLog)] var errors = Vector.empty[(ActorRef, ErrorLog)]
when(Failed, stateTimeout = 5.seconds.dilated) { when(Failed, stateTimeout = 5.seconds.dilated) {
case Event(e: ErrorLog, _) => case this.Event(e: ErrorLog, _) =>
if (!e.msg.startsWith("not resumed") || !ignoreNotResumedLogs) if (!e.msg.startsWith("not resumed") || !ignoreNotResumedLogs)
errors :+= sender() -> e errors :+= sender() -> e
stay() stay()
case Event(Terminated(r), _) if r == hierarchy => case this.Event(Terminated(r), _) if r == hierarchy =>
printErrors() printErrors()
testActor ! "stressTestFailed" testActor ! "stressTestFailed"
stop() stop()
case Event(StateTimeout, _) => case this.Event(StateTimeout, _) =>
getErrors(hierarchy, 10) getErrors(hierarchy, 10)
printErrors() printErrors()
testActor ! "timeout in Failed" testActor ! "timeout in Failed"
stop() stop()
case Event("pong", _) => stay() // dont care? case this.Event("pong", _) => stay() // dont care?
case Event(Work, _) => stay() case this.Event(Work, _) => stay()
case Event(Died(_), _) => stay() case this.Event(Died(_), _) => stay()
} }
def getErrors(target: ActorRef, depth: Int): Unit = { def getErrors(target: ActorRef, depth: Int): Unit = {
@ -712,19 +712,19 @@ object SupervisorHierarchySpec {
} }
whenUnhandled { whenUnhandled {
case Event(Ready(ref), _) => case this.Event(Ready(ref), _) =>
activeChildren :+= ref activeChildren :+= ref
children :+= ref children :+= ref
idleChildren :+= ref idleChildren :+= ref
stay() stay()
case Event(e: ErrorLog, _) => case this.Event(e: ErrorLog, _) =>
if (e.msg.startsWith("not resumed")) stay() if (e.msg.startsWith("not resumed")) stay()
else { else {
errors :+= sender() -> e errors :+= sender() -> e
// dont stop the hierarchy, that is going to happen all by itself and in the right order // dont stop the hierarchy, that is going to happen all by itself and in the right order
goto(Failed) goto(Failed)
} }
case Event(StateTimeout, _) => case this.Event(StateTimeout, _) =>
println("pingChildren:\n" + pingChildren.view.map(_.path.toString).toSeq.sorted.mkString("\n")) println("pingChildren:\n" + pingChildren.view.map(_.path.toString).toSeq.sorted.mkString("\n"))
ignoreNotResumedLogs = false ignoreNotResumedLogs = false
// make sure that we get the logs of the remaining pingChildren // make sure that we get the logs of the remaining pingChildren
@ -732,10 +732,10 @@ object SupervisorHierarchySpec {
// this will ensure that the error logs get printed and we stop the test // this will ensure that the error logs get printed and we stop the test
context.stop(hierarchy) context.stop(hierarchy)
goto(Failed) goto(Failed)
case Event(Abort, _) => case this.Event(Abort, _) =>
log.info("received Abort") log.info("received Abort")
goto(Failed) goto(Failed)
case Event(msg, _) => case this.Event(msg, _) =>
testActor ! ("received unexpected msg: " + msg) testActor ! ("received unexpected msg: " + msg)
stop() stop()
} }

View file

@ -10,7 +10,7 @@ import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.control.NonFatal import scala.util.control.NonFatal
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import language.postfixOps import language.postfixOps
import akka.pattern.ask import akka.pattern.ask
@ -29,7 +29,7 @@ object SupervisorMiscSpec {
""" """
} }
@silent @nowarn
class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with DefaultTimeout { class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with DefaultTimeout {
"A Supervisor" must { "A Supervisor" must {

View file

@ -13,7 +13,7 @@ import scala.concurrent.{ Await, Future }
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.language.postfixOps import scala.language.postfixOps
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
import akka.actor.TypedActor._ import akka.actor.TypedActor._
@ -26,7 +26,7 @@ import akka.util.Timeout
object TypedActorSpec { object TypedActorSpec {
@silent @nowarn
val config = """ val config = """
pooled-dispatcher { pooled-dispatcher {
type = "akka.dispatch.BalancingDispatcherConfigurator" type = "akka.dispatch.BalancingDispatcherConfigurator"
@ -70,9 +70,9 @@ object TypedActorSpec {
trait Foo { trait Foo {
def pigdog(): String def pigdog(): String
@silent @nowarn
@throws(classOf[TimeoutException]) @throws(classOf[TimeoutException])
def self = TypedActor.self[Foo] def self = akka.actor.TypedActor.self[Foo]
def futurePigdog(): Future[String] def futurePigdog(): Future[String]
@ -136,7 +136,7 @@ object TypedActorSpec {
Future.successful(pigdog() + numbered) Future.successful(pigdog() + numbered)
} }
@silent @nowarn
def futureComposePigdogFrom(foo: Foo): Future[String] = { def futureComposePigdogFrom(foo: Foo): Future[String] = {
foo.futurePigdog(500 millis).map(_.toUpperCase) foo.futurePigdog(500 millis).map(_.toUpperCase)
} }
@ -192,8 +192,8 @@ object TypedActorSpec {
with LifeCycles with LifeCycles
with Receiver { with Receiver {
@silent @nowarn
private def ensureContextAvailable[T](f: => T): T = TypedActor.context match { private def ensureContextAvailable[T](f: => T): T = akka.actor.TypedActor.context match {
case null => throw new IllegalStateException("TypedActor.context is null!") case null => throw new IllegalStateException("TypedActor.context is null!")
case _ => f case _ => f
} }
@ -247,7 +247,7 @@ object TypedActorSpec {
} }
@silent @nowarn
class TypedActorSpec class TypedActorSpec
extends AkkaSpec(TypedActorSpec.config) extends AkkaSpec(TypedActorSpec.config)
with BeforeAndAfterEach with BeforeAndAfterEach
@ -259,23 +259,25 @@ class TypedActorSpec
def newFooBar: Foo = newFooBar(timeout.duration) def newFooBar: Foo = newFooBar(timeout.duration)
def newFooBar(d: FiniteDuration): Foo = def newFooBar(d: FiniteDuration): Foo =
TypedActor(system).typedActorOf(TypedProps[Bar](classOf[Foo], classOf[Bar]).withTimeout(Timeout(d))) akka.actor.TypedActor(system).typedActorOf(TypedProps[Bar](classOf[Foo], classOf[Bar]).withTimeout(Timeout(d)))
def newFooBar(dispatcher: String, d: FiniteDuration): Foo = def newFooBar(dispatcher: String, d: FiniteDuration): Foo =
TypedActor(system).typedActorOf( akka.actor
TypedProps[Bar](classOf[Foo], classOf[Bar]).withTimeout(Timeout(d)).withDispatcher(dispatcher)) .TypedActor(system)
.typedActorOf(TypedProps[Bar](classOf[Foo], classOf[Bar]).withTimeout(Timeout(d)).withDispatcher(dispatcher))
def newStacked(): Stacked = def newStacked(): Stacked =
TypedActor(system).typedActorOf( akka.actor
TypedProps[StackedImpl](classOf[Stacked], classOf[StackedImpl]).withTimeout(timeout)) .TypedActor(system)
.typedActorOf(TypedProps[StackedImpl](classOf[Stacked], classOf[StackedImpl]).withTimeout(timeout))
def mustStop(typedActor: AnyRef) = TypedActor(system).stop(typedActor) should ===(true) def mustStop(typedActor: AnyRef) = akka.actor.TypedActor(system).stop(typedActor) should ===(true)
"TypedActors" must { "TypedActors" must {
"be able to instantiate" in { "be able to instantiate" in {
val t = newFooBar val t = newFooBar
TypedActor(system).isTypedActor(t) should ===(true) akka.actor.TypedActor(system).isTypedActor(t) should ===(true)
mustStop(t) mustStop(t)
} }
@ -285,13 +287,13 @@ class TypedActorSpec
} }
"not stop non-started ones" in { "not stop non-started ones" in {
TypedActor(system).stop(null) should ===(false) akka.actor.TypedActor(system).stop(null) should ===(false)
} }
"throw an IllegalStateException when TypedActor.self is called in the wrong scope" in { "throw an IllegalStateException when TypedActor.self is called in the wrong scope" in {
filterEvents(EventFilter[IllegalStateException]("Calling")) { filterEvents(EventFilter[IllegalStateException]("Calling")) {
intercept[IllegalStateException] { intercept[IllegalStateException] {
TypedActor.self[Foo] akka.actor.TypedActor.self[Foo]
}.getMessage should ===("Calling TypedActor.self outside of a TypedActor implementation method!") }.getMessage should ===("Calling TypedActor.self outside of a TypedActor implementation method!")
} }
} }
@ -304,7 +306,7 @@ class TypedActorSpec
"be able to call toString" in { "be able to call toString" in {
val t = newFooBar val t = newFooBar
t.toString should ===(TypedActor(system).getActorRefFor(t).toString) t.toString should ===(akka.actor.TypedActor(system).getActorRefFor(t).toString)
mustStop(t) mustStop(t)
} }
@ -317,7 +319,7 @@ class TypedActorSpec
"be able to call hashCode" in { "be able to call hashCode" in {
val t = newFooBar val t = newFooBar
t.hashCode should ===(TypedActor(system).getActorRefFor(t).hashCode) t.hashCode should ===(akka.actor.TypedActor(system).getActorRefFor(t).hashCode)
mustStop(t) mustStop(t)
} }
@ -398,7 +400,7 @@ class TypedActorSpec
case e: IllegalStateException if e.getMessage == "expected" => SupervisorStrategy.Resume case e: IllegalStateException if e.getMessage == "expected" => SupervisorStrategy.Resume
} }
def receive = { def receive = {
case p: TypedProps[_] => context.sender() ! TypedActor(context).typedActorOf(p) case p: TypedProps[_] => context.sender() ! akka.actor.TypedActor(context).typedActorOf(p)
} }
})) }))
val t = Await.result( val t = Await.result(
@ -431,7 +433,7 @@ class TypedActorSpec
t.optionPigdog() should ===(Some("Pigdog")) t.optionPigdog() should ===(Some("Pigdog"))
mustStop(t) mustStop(t)
val ta: F = TypedActor(system).typedActorOf(TypedProps[FI]()) val ta: F = akka.actor.TypedActor(system).typedActorOf(TypedProps[FI]())
intercept[IllegalStateException] { ta.f(true) }.getMessage should ===("expected") intercept[IllegalStateException] { ta.f(true) }.getMessage should ===("expected")
ta.f(false) should ===(1) ta.f(false) should ===(1)
@ -447,7 +449,7 @@ class TypedActorSpec
} }
"be able to support implementation only typed actors" in within(timeout.duration) { "be able to support implementation only typed actors" in within(timeout.duration) {
val t: Foo = TypedActor(system).typedActorOf(TypedProps[Bar]()) val t: Foo = akka.actor.TypedActor(system).typedActorOf(TypedProps[Bar]())
val f = t.futurePigdog(200 millis) val f = t.futurePigdog(200 millis)
val f2 = t.futurePigdog(Duration.Zero) val f2 = t.futurePigdog(Duration.Zero)
f2.isCompleted should ===(false) f2.isCompleted should ===(false)
@ -457,7 +459,7 @@ class TypedActorSpec
} }
"be able to support implementation only typed actors with complex interfaces" in { "be able to support implementation only typed actors with complex interfaces" in {
val t: Stackable1 with Stackable2 = TypedActor(system).typedActorOf(TypedProps[StackedImpl]()) val t: Stackable1 with Stackable2 = akka.actor.TypedActor(system).typedActorOf(TypedProps[StackedImpl]())
t.stackable1 should ===("foo") t.stackable1 should ===("foo")
t.stackable2 should ===("bar") t.stackable2 should ===("bar")
mustStop(t) mustStop(t)
@ -477,7 +479,7 @@ class TypedActorSpec
"be able to serialize and deserialize invocations" in { "be able to serialize and deserialize invocations" in {
import java.io._ import java.io._
JavaSerializer.currentSystem.withValue(system.asInstanceOf[ExtendedActorSystem]) { JavaSerializer.currentSystem.withValue(system.asInstanceOf[ExtendedActorSystem]) {
val m = TypedActor.MethodCall(classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]()) val m = akka.actor.TypedActor.MethodCall(classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]())
val baos = new ByteArrayOutputStream(8192 * 4) val baos = new ByteArrayOutputStream(8192 * 4)
val out = new ObjectOutputStream(baos) val out = new ObjectOutputStream(baos)
@ -486,7 +488,7 @@ class TypedActorSpec
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall] val mNew = in.readObject().asInstanceOf[akka.actor.TypedActor.MethodCall]
mNew.method should ===(m.method) mNew.method should ===(m.method)
} }
@ -496,7 +498,7 @@ class TypedActorSpec
import java.io._ import java.io._
val someFoo: Foo = new Bar val someFoo: Foo = new Bar
JavaSerializer.currentSystem.withValue(system.asInstanceOf[ExtendedActorSystem]) { JavaSerializer.currentSystem.withValue(system.asInstanceOf[ExtendedActorSystem]) {
val m = TypedActor.MethodCall( val m = akka.actor.TypedActor.MethodCall(
classOf[Foo].getDeclaredMethod( classOf[Foo].getDeclaredMethod(
"testMethodCallSerialization", "testMethodCallSerialization",
Array[Class[_]](classOf[Foo], classOf[String], classOf[Int], classOf[WithStringSerializedClass]): _*), Array[Class[_]](classOf[Foo], classOf[String], classOf[Int], classOf[WithStringSerializedClass]): _*),
@ -509,7 +511,7 @@ class TypedActorSpec
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall] val mNew = in.readObject().asInstanceOf[akka.actor.TypedActor.MethodCall]
mNew.method should ===(m.method) mNew.method should ===(m.method)
mNew.parameters should have size 4 mNew.parameters should have size 4
@ -550,7 +552,7 @@ class TypedActorSpec
"be able to override lifecycle callbacks" in { "be able to override lifecycle callbacks" in {
val latch = new CountDownLatch(16) val latch = new CountDownLatch(16)
val ta = TypedActor(system) val ta = akka.actor.TypedActor(system)
val t: LifeCycles = ta.typedActorOf(TypedProps[LifeCyclesImpl](classOf[LifeCycles], new LifeCyclesImpl(latch))) val t: LifeCycles = ta.typedActorOf(TypedProps[LifeCyclesImpl](classOf[LifeCycles], new LifeCyclesImpl(latch)))
EventFilter[IllegalStateException]("Crash!", occurrences = 1).intercept { EventFilter[IllegalStateException]("Crash!", occurrences = 1).intercept {
t.crash() t.crash()
@ -571,7 +573,7 @@ class TypedActorSpec
} }
} }
@silent @nowarn
class TypedActorRouterSpec class TypedActorRouterSpec
extends AkkaSpec(TypedActorSpec.config) extends AkkaSpec(TypedActorSpec.config)
with BeforeAndAfterEach with BeforeAndAfterEach
@ -583,9 +585,9 @@ class TypedActorRouterSpec
def newFooBar: Foo = newFooBar(timeout.duration) def newFooBar: Foo = newFooBar(timeout.duration)
def newFooBar(d: FiniteDuration): Foo = def newFooBar(d: FiniteDuration): Foo =
TypedActor(system).typedActorOf(TypedProps[Bar](classOf[Foo], classOf[Bar]).withTimeout(Timeout(d))) akka.actor.TypedActor(system).typedActorOf(TypedProps[Bar](classOf[Foo], classOf[Bar]).withTimeout(Timeout(d)))
def mustStop(typedActor: AnyRef) = TypedActor(system).stop(typedActor) should ===(true) def mustStop(typedActor: AnyRef) = akka.actor.TypedActor(system).stop(typedActor) should ===(true)
"TypedActor Router" must { "TypedActor Router" must {
@ -595,15 +597,15 @@ class TypedActorRouterSpec
val t3 = newFooBar val t3 = newFooBar
val t4 = newFooBar val t4 = newFooBar
val routees = List(t1, t2, t3, t4).map { t => val routees = List(t1, t2, t3, t4).map { t =>
TypedActor(system).getActorRefFor(t).path.toStringWithoutAddress akka.actor.TypedActor(system).getActorRefFor(t).path.toStringWithoutAddress
} }
TypedActor(system).isTypedActor(t1) should ===(true) akka.actor.TypedActor(system).isTypedActor(t1) should ===(true)
TypedActor(system).isTypedActor(t2) should ===(true) akka.actor.TypedActor(system).isTypedActor(t2) should ===(true)
val router = system.actorOf(RoundRobinGroup(routees).props(), "router") val router = system.actorOf(RoundRobinGroup(routees).props(), "router")
val typedRouter = TypedActor(system).typedActorOf[Foo, Foo](TypedProps[Foo](), router) val typedRouter = akka.actor.TypedActor(system).typedActorOf[Foo, Foo](TypedProps[Foo](), router)
info("got = " + typedRouter.optionPigdog()) info("got = " + typedRouter.optionPigdog())
info("got = " + typedRouter.optionPigdog()) info("got = " + typedRouter.optionPigdog())

View file

@ -12,7 +12,7 @@ import scala.annotation.tailrec
import scala.concurrent.{ Await, Future } import scala.concurrent.{ Await, Future }
import scala.concurrent.duration._ import scala.concurrent.duration._
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import com.typesafe.config.Config import com.typesafe.config.Config
import language.postfixOps import language.postfixOps
import org.scalatest.Assertions._ import org.scalatest.Assertions._
@ -602,7 +602,7 @@ class DispatcherModelSpec extends ActorModelSpec(DispatcherModelSpec.config) {
} }
} }
@silent @nowarn
object BalancingDispatcherModelSpec { object BalancingDispatcherModelSpec {
import ActorModelSpec._ import ActorModelSpec._
@ -641,7 +641,7 @@ object BalancingDispatcherModelSpec {
} }
} }
@silent @nowarn
class BalancingDispatcherModelSpec extends ActorModelSpec(BalancingDispatcherModelSpec.config) { class BalancingDispatcherModelSpec extends ActorModelSpec(BalancingDispatcherModelSpec.config) {
import ActorModelSpec._ import ActorModelSpec._

View file

@ -12,7 +12,7 @@ import scala.concurrent.duration._
import language.postfixOps import language.postfixOps
import akka.actor.{ Actor, Props } import akka.actor.{ actorRef2Scala, Actor, Props }
import akka.pattern.ask import akka.pattern.ask
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout

View file

@ -10,7 +10,7 @@ import scala.concurrent.Await
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
import akka.actor.{ Actor, Props } import akka.actor.{ actorRef2Scala, Actor, Props }
import akka.pattern.ask import akka.pattern.ask
import akka.testkit._ import akka.testkit._
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec

View file

@ -6,6 +6,7 @@ package akka.actor.dungeon
import akka.actor.Actor import akka.actor.Actor
import akka.actor.Props import akka.actor.Props
import akka.actor.actorRef2Scala
import akka.testkit._ import akka.testkit._
object DispatchSpec { object DispatchSpec {

View file

@ -22,8 +22,8 @@ class ActorSystemSetupSpec extends AnyWordSpec with Matchers {
val setup = DummySetup("Al Dente") val setup = DummySetup("Al Dente")
val setups = ActorSystemSetup().withSetup(setup) val setups = ActorSystemSetup().withSetup(setup)
setups.get[DummySetup] should ===(Some(setup)) (setups.get[DummySetup]: Option[Setup]) should ===(Some(setup))
setups.get[DummySetup2] should ===(None) (setups.get[DummySetup2]: Option[Setup]) should ===(None)
} }
"replace setup if already defined" in { "replace setup if already defined" in {
@ -31,7 +31,7 @@ class ActorSystemSetupSpec extends AnyWordSpec with Matchers {
val setup2 = DummySetup("Earl E. Bird") val setup2 = DummySetup("Earl E. Bird")
val setups = ActorSystemSetup().withSetup(setup1).withSetup(setup2) val setups = ActorSystemSetup().withSetup(setup1).withSetup(setup2)
setups.get[DummySetup] should ===(Some(setup2)) (setups.get[DummySetup]: Option[Setup]) should ===(Some(setup2))
} }
"provide a fluent creation alternative" in { "provide a fluent creation alternative" in {
@ -40,8 +40,8 @@ class ActorSystemSetupSpec extends AnyWordSpec with Matchers {
val c = DummySetup2("Amanda Reckonwith") val c = DummySetup2("Amanda Reckonwith")
val setups = a and b and c val setups = a and b and c
setups.get[DummySetup] should ===(Some(b)) (setups.get[DummySetup]: Option[Setup]) should ===(Some(b))
setups.get[DummySetup2] should ===(Some(c)) (setups.get[DummySetup2]: Option[Setup]) should ===(Some(c))
} }
"be created with a set of setups" in { "be created with a set of setups" in {
@ -60,7 +60,7 @@ class ActorSystemSetupSpec extends AnyWordSpec with Matchers {
val setup = DummySetup("Tad Moore") val setup = DummySetup("Tad Moore")
system = ActorSystem("name", ActorSystemSetup(setup)) system = ActorSystem("name", ActorSystemSetup(setup))
system.settings.setup.get[DummySetup] should ===(Some(setup)) (system.settings.setup.get[DummySetup]: Option[Setup]) should ===(Some(setup))
} finally { } finally {
TestKit.shutdownActorSystem(system) TestKit.shutdownActorSystem(system)

View file

@ -4,7 +4,7 @@
package akka.dispatch package akka.dispatch
import akka.actor.{ Actor, Props } import akka.actor.{ actorRef2Scala, Actor, Props }
import akka.testkit.{ AkkaSpec, DefaultTimeout } import akka.testkit.{ AkkaSpec, DefaultTimeout }
object ControlAwareDispatcherSpec { object ControlAwareDispatcherSpec {

View file

@ -13,6 +13,7 @@ import scala.concurrent.duration._
import akka.actor.Actor import akka.actor.Actor
import akka.actor.Props import akka.actor.Props
import akka.actor.actorRef2Scala
import akka.testkit.{ AkkaSpec, DefaultTimeout, TestLatch } import akka.testkit.{ AkkaSpec, DefaultTimeout, TestLatch }
import akka.testkit.CallingThreadDispatcher import akka.testkit.CallingThreadDispatcher
import akka.testkit.TestActorRef import akka.testkit.TestActorRef

View file

@ -6,7 +6,7 @@ package akka.dispatch
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.actor.{ Actor, Props } import akka.actor.{ actorRef2Scala, Actor, Props }
import akka.testkit.{ AkkaSpec, ImplicitSender } import akka.testkit.{ AkkaSpec, ImplicitSender }
object ForkJoinPoolStarvationSpec { object ForkJoinPoolStarvationSpec {

View file

@ -9,7 +9,7 @@ import scala.concurrent.duration._
import com.typesafe.config.Config import com.typesafe.config.Config
import language.postfixOps import language.postfixOps
import akka.actor.{ Actor, ActorSystem, Props } import akka.actor.{ actorRef2Scala, Actor, ActorSystem, Props }
import akka.testkit.{ AkkaSpec, DefaultTimeout } import akka.testkit.{ AkkaSpec, DefaultTimeout }
import akka.util.unused import akka.util.unused
@ -83,7 +83,7 @@ class PriorityDispatcherSpec extends AkkaSpec(PriorityDispatcherSpec.config) wit
})) }))
expectMsgType[List[Int]] should ===(msgs) (expectMsgType[List[Int]]: List[Int]) should ===(msgs)
} }
} }

View file

@ -9,7 +9,7 @@ import scala.concurrent.duration._
import com.typesafe.config.Config import com.typesafe.config.Config
import language.postfixOps import language.postfixOps
import akka.actor.{ Actor, ActorSystem, Props } import akka.actor.{ actorRef2Scala, Actor, ActorSystem, Props }
import akka.testkit.{ AkkaSpec, DefaultTimeout } import akka.testkit.{ AkkaSpec, DefaultTimeout }
import akka.util.unused import akka.util.unused
@ -90,7 +90,7 @@ class StablePriorityDispatcherSpec extends AkkaSpec(StablePriorityDispatcherSpec
// should come out in the same order in which they were sent. // should come out in the same order in which they were sent.
val lo = (1 to 100) toList val lo = (1 to 100) toList
val hi = shuffled.filter { _ > 100 } val hi = shuffled.filter { _ > 100 }
expectMsgType[List[Int]] should ===(lo ++ hi) (expectMsgType[List[Int]]: List[Int]) should ===(lo ++ hi)
} }
} }
} }

View file

@ -10,6 +10,7 @@ import akka.actor.Actor
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.actor.Props import akka.actor.Props
import akka.actor.actorRef2Scala
import akka.testkit._ import akka.testkit._
object AddressTerminatedTopicBenchSpec { object AddressTerminatedTopicBenchSpec {

View file

@ -10,7 +10,7 @@ import com.typesafe.config.{ Config, ConfigFactory }
import language.postfixOps import language.postfixOps
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
import akka.actor.{ Actor, ActorRef, ActorSystem, PoisonPill, Props } import akka.actor.{ actorRef2Scala, Actor, ActorRef, ActorSystem, PoisonPill, Props }
import akka.japi.Procedure import akka.japi.Procedure
import akka.testkit._ import akka.testkit._
@ -29,19 +29,19 @@ abstract class EventBusSpec(busName: String, conf: Config = ConfigFactory.empty(
def createNewEventBus(): BusType def createNewEventBus(): BusType
def createEvents(numberOfEvents: Int): Iterable[BusType#Event] final val bus = createNewEventBus()
def createSubscriber(pipeTo: ActorRef): BusType#Subscriber def createEvents(numberOfEvents: Int): Iterable[bus.Event]
def classifierFor(event: BusType#Event): BusType#Classifier def createSubscriber(pipeTo: ActorRef): bus.Subscriber
def disposeSubscriber(system: ActorSystem, subscriber: BusType#Subscriber): Unit def classifierFor(event: bus.Event): bus.Classifier
lazy val bus = createNewEventBus() def disposeSubscriber(system: ActorSystem, subscriber: bus.Subscriber): Unit
busName must { busName must {
def createNewSubscriber() = createSubscriber(testActor).asInstanceOf[bus.Subscriber] def createNewSubscriber() = createSubscriber(testActor).asInstanceOf[bus.Subscriber]
def getClassifierFor(event: BusType#Event) = classifierFor(event).asInstanceOf[bus.Classifier] def getClassifierFor(event: bus.Event) = classifierFor(event).asInstanceOf[bus.Classifier]
def createNewEvents(numberOfEvents: Int): Iterable[bus.Event] = def createNewEvents(numberOfEvents: Int): Iterable[bus.Event] =
createEvents(numberOfEvents).asInstanceOf[Iterable[bus.Event]] createEvents(numberOfEvents).asInstanceOf[Iterable[bus.Event]]

View file

@ -176,11 +176,11 @@ class EventStreamSpec extends AkkaSpec(EventStreamSpec.config) {
es.subscribe(a4.ref, classOf[CCATBT]) should ===(true) es.subscribe(a4.ref, classOf[CCATBT]) should ===(true)
es.publish(tm1) es.publish(tm1)
es.publish(tm2) es.publish(tm2)
a1.expectMsgType[AT] should ===(tm2) (a1.expectMsgType[AT]: AT) should ===(tm2)
a2.expectMsgType[BT] should ===(tm2) (a2.expectMsgType[BT]: BT) should ===(tm2)
a3.expectMsgType[CC] should ===(tm1) (a3.expectMsgType[CC]: CC) should ===(tm1)
a3.expectMsgType[CC] should ===(tm2) (a3.expectMsgType[CC]: CC) should ===(tm2)
a4.expectMsgType[CCATBT] should ===(tm2) (a4.expectMsgType[CCATBT]: CCATBT) should ===(tm2)
es.unsubscribe(a1.ref, classOf[AT]) should ===(true) es.unsubscribe(a1.ref, classOf[AT]) should ===(true)
es.unsubscribe(a2.ref, classOf[BT]) should ===(true) es.unsubscribe(a2.ref, classOf[BT]) should ===(true)
es.unsubscribe(a3.ref, classOf[CC]) should ===(true) es.unsubscribe(a3.ref, classOf[CC]) should ===(true)
@ -200,10 +200,10 @@ class EventStreamSpec extends AkkaSpec(EventStreamSpec.config) {
es.unsubscribe(a3.ref, classOf[CC]) should ===(true) es.unsubscribe(a3.ref, classOf[CC]) should ===(true)
es.publish(tm1) es.publish(tm1)
es.publish(tm2) es.publish(tm2)
a1.expectMsgType[AT] should ===(tm2) (a1.expectMsgType[AT]: AT) should ===(tm2)
a2.expectMsgType[BT] should ===(tm2) (a2.expectMsgType[BT]: BT) should ===(tm2)
a3.expectNoMessage(1 second) a3.expectNoMessage(1 second)
a4.expectMsgType[CCATBT] should ===(tm2) (a4.expectMsgType[CCATBT]: CCATBT) should ===(tm2)
es.unsubscribe(a1.ref, classOf[AT]) should ===(true) es.unsubscribe(a1.ref, classOf[AT]) should ===(true)
es.unsubscribe(a2.ref, classOf[BT]) should ===(true) es.unsubscribe(a2.ref, classOf[BT]) should ===(true)
es.unsubscribe(a4.ref, classOf[CCATBT]) should ===(true) es.unsubscribe(a4.ref, classOf[CCATBT]) should ===(true)
@ -222,10 +222,10 @@ class EventStreamSpec extends AkkaSpec(EventStreamSpec.config) {
es.unsubscribe(a3.ref) es.unsubscribe(a3.ref)
es.publish(tm1) es.publish(tm1)
es.publish(tm2) es.publish(tm2)
a1.expectMsgType[AT] should ===(tm2) (a1.expectMsgType[AT]: AT) should ===(tm2)
a2.expectMsgType[BT] should ===(tm2) (a2.expectMsgType[BT]: BT) should ===(tm2)
a3.expectNoMessage(1 second) a3.expectNoMessage(1 second)
a4.expectMsgType[CCATBT] should ===(tm2) (a4.expectMsgType[CCATBT]: CCATBT) should ===(tm2)
es.unsubscribe(a1.ref, classOf[AT]) should ===(true) es.unsubscribe(a1.ref, classOf[AT]) should ===(true)
es.unsubscribe(a2.ref, classOf[BT]) should ===(true) es.unsubscribe(a2.ref, classOf[BT]) should ===(true)
es.unsubscribe(a4.ref, classOf[CCATBT]) should ===(true) es.unsubscribe(a4.ref, classOf[CCATBT]) should ===(true)
@ -241,8 +241,8 @@ class EventStreamSpec extends AkkaSpec(EventStreamSpec.config) {
es.subscribe(a2.ref, classOf[BT]) should ===(true) es.subscribe(a2.ref, classOf[BT]) should ===(true)
es.publish(tm1) es.publish(tm1)
es.publish(tm2) es.publish(tm2)
a1.expectMsgType[AT] should ===(tm2) (a1.expectMsgType[AT]: AT) should ===(tm2)
a2.expectMsgType[BT] should ===(tm2) (a2.expectMsgType[BT]: BT) should ===(tm2)
es.unsubscribe(a1.ref, classOf[AT]) should ===(true) es.unsubscribe(a1.ref, classOf[AT]) should ===(true)
es.unsubscribe(a2.ref, classOf[BT]) should ===(true) es.unsubscribe(a2.ref, classOf[BT]) should ===(true)
} }
@ -261,9 +261,9 @@ class EventStreamSpec extends AkkaSpec(EventStreamSpec.config) {
es.unsubscribe(a3.ref, classOf[CCATBT]) should ===(true) es.unsubscribe(a3.ref, classOf[CCATBT]) should ===(true)
es.publish(tm1) es.publish(tm1)
es.publish(tm2) es.publish(tm2)
a1.expectMsgType[AT] should ===(tm2) (a1.expectMsgType[AT]: AT) should ===(tm2)
a2.expectMsgType[BT] should ===(tm2) (a2.expectMsgType[BT]: BT) should ===(tm2)
a3.expectMsgType[CC] should ===(tm1) (a3.expectMsgType[CC]: CC) should ===(tm1)
es.unsubscribe(a1.ref, classOf[AT]) should ===(true) es.unsubscribe(a1.ref, classOf[AT]) should ===(true)
es.unsubscribe(a2.ref, classOf[BT]) should ===(true) es.unsubscribe(a2.ref, classOf[BT]) should ===(true)
es.unsubscribe(a3.ref, classOf[CC]) should ===(true) es.unsubscribe(a3.ref, classOf[CC]) should ===(true)
@ -276,12 +276,12 @@ class EventStreamSpec extends AkkaSpec(EventStreamSpec.config) {
es.subscribe(a1.ref, classOf[AT]) should ===(true) es.subscribe(a1.ref, classOf[AT]) should ===(true)
es.publish(tm1) es.publish(tm1)
a1.expectMsgType[AT] should ===(tm1) (a1.expectMsgType[AT]: AT) should ===(tm1)
a2.expectNoMessage(1 second) a2.expectNoMessage(1 second)
es.subscribe(a2.ref, classOf[BTT]) should ===(true) es.subscribe(a2.ref, classOf[BTT]) should ===(true)
es.publish(tm1) es.publish(tm1)
a1.expectMsgType[AT] should ===(tm1) (a1.expectMsgType[AT]: AT) should ===(tm1)
a2.expectMsgType[BTT] should ===(tm1) (a2.expectMsgType[BTT]: BTT) should ===(tm1)
es.unsubscribe(a1.ref, classOf[AT]) should ===(true) es.unsubscribe(a1.ref, classOf[AT]) should ===(true)
es.unsubscribe(a2.ref, classOf[BTT]) should ===(true) es.unsubscribe(a2.ref, classOf[BTT]) should ===(true)
} }

View file

@ -10,7 +10,7 @@ import scala.util.control.NoStackTrace
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.actor.{ Actor, ActorLogging, Props } import akka.actor.{ actorRef2Scala, Actor, ActorLogging, Props }
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
@deprecated("Use SLF4J instead.", "2.6.0") @deprecated("Use SLF4J instead.", "2.6.0")

View file

@ -7,12 +7,12 @@ package akka.io
import java.security.Security import java.security.Security
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import akka.actor.Props import akka.actor.Props
import akka.testkit.{ AkkaSpec, TestActorRef } import akka.testkit.{ AkkaSpec, TestActorRef }
@silent @nowarn
class InetAddressDnsResolverSpec extends AkkaSpec(""" class InetAddressDnsResolverSpec extends AkkaSpec("""
akka.io.dns.inet-address.positive-ttl = default akka.io.dns.inet-address.positive-ttl = default
akka.io.dns.inet-address.negative-ttl = default akka.io.dns.inet-address.negative-ttl = default
@ -120,7 +120,7 @@ class InetAddressDnsResolverSpec extends AkkaSpec("""
} }
@silent @nowarn
class InetAddressDnsResolverConfigSpec extends AkkaSpec(""" class InetAddressDnsResolverConfigSpec extends AkkaSpec("""
akka.io.dns.inet-address.positive-ttl = forever akka.io.dns.inet-address.positive-ttl = forever
akka.io.dns.inet-address.negative-ttl = never akka.io.dns.inet-address.negative-ttl = never

View file

@ -11,6 +11,7 @@ import Tcp._
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.actor.actorRef2Scala
import akka.dispatch.ExecutionContexts import akka.dispatch.ExecutionContexts
import akka.io.Inet.SocketOption import akka.io.Inet.SocketOption
import akka.testkit.{ AkkaSpec, TestProbe } import akka.testkit.{ AkkaSpec, TestProbe }

View file

@ -9,6 +9,7 @@ import java.net.InetSocketAddress
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.actor.actorRef2Scala
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender import akka.testkit.ImplicitSender
import akka.testkit.SocketUtil.temporaryServerAddresses import akka.testkit.SocketUtil.temporaryServerAddresses

View file

@ -8,6 +8,7 @@ import java.net.DatagramSocket
import java.net.InetSocketAddress import java.net.InetSocketAddress
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.actor.actorRef2Scala
import akka.io.Inet._ import akka.io.Inet._
import akka.io.Udp._ import akka.io.Udp._
import akka.testkit.{ AkkaSpec, ImplicitSender, TestProbe } import akka.testkit.{ AkkaSpec, ImplicitSender, TestProbe }

View file

@ -8,12 +8,13 @@ import java.net.InetAddress
import scala.concurrent.duration._ import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import CachePolicy.Ttl import CachePolicy.Ttl
import akka.io.{ Dns, IO } import akka.io.{ Dns, IO }
import akka.io.dns.DnsProtocol.{ Ip, RequestType, Srv } import akka.io.dns.DnsProtocol.{ Ip, RequestType, Srv }
import akka.pattern.ask import akka.pattern.ask
import akka.testkit.{ AkkaSpec, SocketUtil } import akka.testkit.SocketUtil
import akka.testkit.SocketUtil.Both import akka.testkit.SocketUtil.Both
import akka.testkit.WithLogCapturing import akka.testkit.WithLogCapturing
import akka.util.Timeout import akka.util.Timeout
@ -25,14 +26,14 @@ The configuration to start a bind DNS server in Docker with this configuration
is included, and the test will automatically start this container when the is included, and the test will automatically start this container when the
test starts and tear it down when it finishes. test starts and tear it down when it finishes.
*/ */
class AsyncDnsResolverIntegrationSpec extends AkkaSpec(s""" class AsyncDnsResolverIntegrationSpec extends DockerBindDnsService(ConfigFactory.parseString(s"""
akka.loglevel = DEBUG akka.loglevel = DEBUG
akka.loggers = ["akka.testkit.SilenceAllTestEventListener"] akka.loggers = ["akka.testkit.SilenceAllTestEventListener"]
akka.io.dns.resolver = async-dns akka.io.dns.resolver = async-dns
akka.io.dns.async-dns.nameservers = ["localhost:${AsyncDnsResolverIntegrationSpec.dockerDnsServerPort}"] akka.io.dns.async-dns.nameservers = ["localhost:${AsyncDnsResolverIntegrationSpec.dockerDnsServerPort}"]
akka.io.dns.async-dns.search-domains = ["foo.test", "test"] akka.io.dns.async-dns.search-domains = ["foo.test", "test"]
akka.io.dns.async-dns.ndots = 2 akka.io.dns.async-dns.ndots = 2
""") with DockerBindDnsService with WithLogCapturing { """)) with WithLogCapturing {
val duration = 10.seconds val duration = 10.seconds
implicit val timeout: Timeout = Timeout(duration) implicit val timeout: Timeout = Timeout(duration)

View file

@ -8,6 +8,7 @@ import scala.concurrent.duration._
import scala.util.Try import scala.util.Try
import scala.util.control.NonFatal import scala.util.control.NonFatal
import com.typesafe.config.Config
import com.spotify.docker.client.DefaultDockerClient import com.spotify.docker.client.DefaultDockerClient
import com.spotify.docker.client.DockerClient.{ ListContainersParam, LogsParam } import com.spotify.docker.client.DockerClient.{ ListContainersParam, LogsParam }
import com.spotify.docker.client.messages.{ ContainerConfig, HostConfig, PortBinding } import com.spotify.docker.client.messages.{ ContainerConfig, HostConfig, PortBinding }
@ -16,7 +17,7 @@ import org.scalatest.concurrent.Eventually
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.util.ccompat.JavaConverters._ import akka.util.ccompat.JavaConverters._
trait DockerBindDnsService extends Eventually { self: AkkaSpec => abstract class DockerBindDnsService(config: Config) extends AkkaSpec(config) with Eventually {
val client = DefaultDockerClient.fromEnv().build() val client = DefaultDockerClient.fromEnv().build()
val hostPort: Int val hostPort: Int
@ -27,7 +28,7 @@ trait DockerBindDnsService extends Eventually { self: AkkaSpec =>
override def atStartup(): Unit = { override def atStartup(): Unit = {
log.info("Running on port port {}", hostPort) log.info("Running on port port {}", hostPort)
self.atStartup() super.atStartup()
// https://github.com/sameersbn/docker-bind/pull/61 // https://github.com/sameersbn/docker-bind/pull/61
val image = "raboof/bind:9.11.3-20180713-nochown" val image = "raboof/bind:9.11.3-20180713-nochown"
@ -87,7 +88,7 @@ trait DockerBindDnsService extends Eventually { self: AkkaSpec =>
} }
override def afterTermination(): Unit = { override def afterTermination(): Unit = {
self.afterTermination() super.afterTermination()
id.foreach(client.killContainer) id.foreach(client.killContainer)
id.foreach(client.removeContainer) id.foreach(client.removeContainer)
} }

View file

@ -8,8 +8,9 @@ import java.net.InetAddress
import scala.collection.immutable.Seq import scala.collection.immutable.Seq
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import akka.actor.actorRef2Scala
import akka.io.Dns import akka.io.Dns
import akka.io.dns.AAAARecord import akka.io.dns.AAAARecord
import akka.io.dns.CachePolicy.Ttl import akka.io.dns.CachePolicy.Ttl
@ -18,7 +19,7 @@ import akka.testkit.{ AkkaSpec, ImplicitSender }
import akka.testkit.WithLogCapturing import akka.testkit.WithLogCapturing
// tests deprecated DNS API // tests deprecated DNS API
@silent("deprecated") @nowarn("msg=deprecated")
class AsyncDnsManagerSpec extends AkkaSpec(""" class AsyncDnsManagerSpec extends AkkaSpec("""
akka.loglevel = DEBUG akka.loglevel = DEBUG
akka.loggers = ["akka.testkit.SilenceAllTestEventListener"] akka.loggers = ["akka.testkit.SilenceAllTestEventListener"]
@ -49,7 +50,7 @@ class AsyncDnsManagerSpec extends AkkaSpec("""
"provide access to cache" in { "provide access to cache" in {
dns ! AsyncDnsManager.GetCache dns ! AsyncDnsManager.GetCache
(expectMsgType[akka.io.SimpleDnsCache] should be).theSameInstanceAs(Dns(system).cache) ((expectMsgType[akka.io.SimpleDnsCache]: akka.io.SimpleDnsCache) should be).theSameInstanceAs(Dns(system).cache)
} }
} }

View file

@ -11,7 +11,7 @@ import scala.concurrent.duration._
import com.typesafe.config.{ Config, ConfigFactory, ConfigValueFactory } import com.typesafe.config.{ Config, ConfigFactory, ConfigValueFactory }
import akka.actor.{ ActorRef, ExtendedActorSystem, Props } import akka.actor.{ actorRef2Scala, ActorRef, ExtendedActorSystem, Props }
import akka.actor.Status.Failure import akka.actor.Status.Failure
import akka.io.SimpleDnsCache import akka.io.SimpleDnsCache
import akka.io.dns.{ AAAARecord, ARecord, DnsSettings, SRVRecord } import akka.io.dns.{ AAAARecord, ARecord, DnsSettings, SRVRecord }

View file

@ -10,6 +10,7 @@ import java.util.concurrent.atomic.AtomicBoolean
import scala.collection.immutable.Seq import scala.collection.immutable.Seq
import akka.actor.Props import akka.actor.Props
import akka.actor.actorRef2Scala
import akka.io.Udp import akka.io.Udp
import akka.io.dns.{ RecordClass, RecordType } import akka.io.dns.{ RecordClass, RecordType }
import akka.io.dns.internal.DnsClient.{ Answer, Question4 } import akka.io.dns.internal.DnsClient.{ Answer, Question4 }

View file

@ -9,6 +9,7 @@ import java.net.InetSocketAddress
import scala.collection.immutable.Seq import scala.collection.immutable.Seq
import akka.actor.Props import akka.actor.Props
import akka.actor.actorRef2Scala
import akka.io.Tcp import akka.io.Tcp
import akka.io.Tcp.{ Connected, PeerClosed, Register } import akka.io.Tcp.{ Connected, PeerClosed, Register }
import akka.io.dns.{ RecordClass, RecordType } import akka.io.dns.{ RecordClass, RecordType }

View file

@ -7,15 +7,16 @@ package akka.pattern
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.Failure import scala.util.Failure
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import language.postfixOps import language.postfixOps
import akka.actor._ import akka.actor._
import akka.testkit.WithLogCapturing
import akka.testkit.{ AkkaSpec, TestProbe } import akka.testkit.{ AkkaSpec, TestProbe }
import akka.testkit.WithLogCapturing
import akka.util.Timeout import akka.util.Timeout
@silent @nowarn
class AskSpec extends AkkaSpec(""" class AskSpec extends AkkaSpec("""
akka.loglevel = DEBUG akka.loglevel = DEBUG
akka.loggers = ["akka.testkit.SilenceAllTestEventListener"] akka.loggers = ["akka.testkit.SilenceAllTestEventListener"]
@ -26,7 +27,7 @@ class AskSpec extends AkkaSpec("""
implicit val timeout: Timeout = Timeout(5.seconds) implicit val timeout: Timeout = Timeout(5.seconds)
val echo = system.actorOf(Props(new Actor { def receive = { case x => sender() ! x } })) val echo = system.actorOf(Props(new Actor { def receive = { case x => sender() ! x } }))
val f = echo ? "ping" val f = echo ? "ping"
f.futureValue should ===("ping") Await.result(f, timeout.duration) should ===("ping")
} }
"return broken promises on DeadLetters" in { "return broken promises on DeadLetters" in {

View file

@ -9,7 +9,7 @@ import java.util.concurrent.{ CountDownLatch, TimeUnit }
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.language.postfixOps import scala.language.postfixOps
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import akka.actor._ import akka.actor._
import akka.pattern.TestActor.NormalException import akka.pattern.TestActor.NormalException
@ -154,7 +154,7 @@ class BackoffOnRestartSupervisorSpec extends AkkaSpec("""
"accept commands while child is terminating" in { "accept commands while child is terminating" in {
val postStopLatch = new CountDownLatch(1) val postStopLatch = new CountDownLatch(1)
@silent @nowarn
val options = BackoffOpts val options = BackoffOpts
.onFailure(Props(new SlowlyFailingActor(postStopLatch)), "someChildName", 1 nanos, 1 nanos, 0.0) .onFailure(Props(new SlowlyFailingActor(postStopLatch)), "someChildName", 1 nanos, 1 nanos, 0.0)
.withMaxNrOfRetries(-1) .withMaxNrOfRetries(-1)

View file

@ -16,6 +16,7 @@ import scala.util.Success
import scala.util.Try import scala.util.Try
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.actor.actorRef2Scala
import akka.testkit._ import akka.testkit._
object CircuitBreakerSpec { object CircuitBreakerSpec {

View file

@ -14,6 +14,7 @@ import akka.actor.Actor
import akka.actor.ActorLogging import akka.actor.ActorLogging
import akka.actor.Props import akka.actor.Props
import akka.actor.Status.Failure import akka.actor.Status.Failure
import akka.actor.actorRef2Scala
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender import akka.testkit.ImplicitSender

View file

@ -10,7 +10,7 @@ import scala.concurrent.duration._
import language.postfixOps import language.postfixOps
import akka.actor.{ Actor, Props } import akka.actor.{ actorRef2Scala, Actor, Props }
import akka.testkit.{ AkkaSpec, TestLatch } import akka.testkit.{ AkkaSpec, TestLatch }
object PatternSpec { object PatternSpec {

View file

@ -4,7 +4,7 @@
package akka.pattern package akka.pattern
import scala.concurrent.Promise import scala.concurrent.{ Await, Promise }
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor._ import akka.actor._
@ -32,7 +32,7 @@ class PromiseRefSpec extends AkkaSpec with ImplicitSender {
})) }))
target ! Request(promiseRef.ref) target ! Request(promiseRef.ref)
promiseRef.future.futureValue should ===(Response) Await.result(promiseRef.future, 5.seconds) should ===(Response)
} }
"throw IllegalArgumentException on negative timeout" in { "throw IllegalArgumentException on negative timeout" in {
@ -48,7 +48,7 @@ class PromiseRefSpec extends AkkaSpec with ImplicitSender {
val promiseRef = PromiseRef(5.seconds) val promiseRef = PromiseRef(5.seconds)
promiseRef.ref ! FirstMessage promiseRef.ref ! FirstMessage
promiseRef.future.futureValue should ===(FirstMessage) Await.result(promiseRef.future, 5.seconds) should ===(FirstMessage)
promiseRef.ref ! SecondMessage promiseRef.ref ! SecondMessage
deadListener.expectMsgType[DeadLetter].message should ===(SecondMessage) deadListener.expectMsgType[DeadLetter].message should ===(SecondMessage)

View file

@ -4,15 +4,17 @@
package akka.pattern package akka.pattern
import scala.concurrent.{ Await, Future }
import scala.concurrent.duration._
import org.scalatest.concurrent.ScalaFutures
import akka.Done import akka.Done
import akka.actor.actorRef2Scala
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.TestException import akka.testkit.TestException
import akka.testkit.TestProbe import akka.testkit.TestProbe
import akka.util.Timeout import akka.util.Timeout
import org.scalatest.concurrent.ScalaFutures
import scala.concurrent.Future
import scala.concurrent.duration._
class StatusReplySpec extends AkkaSpec with ScalaFutures { class StatusReplySpec extends AkkaSpec with ScalaFutures {
@ -66,7 +68,7 @@ class StatusReplySpec extends AkkaSpec with ScalaFutures {
val result = probe.ref.askWithStatus("request") val result = probe.ref.askWithStatus("request")
probe.expectMsg("request") probe.expectMsg("request")
probe.lastSender ! StatusReply.Success("woho") probe.lastSender ! StatusReply.Success("woho")
result.futureValue should ===("woho") Await.result(result, timeout.duration) should ===("woho")
} }
"unwrap Error with message" in { "unwrap Error with message" in {

View file

@ -4,6 +4,7 @@
package akka.pattern.extended package akka.pattern.extended
import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor._ import akka.actor._
@ -29,7 +30,7 @@ class ExplicitAskSpec extends AkkaSpec {
})) }))
val f = target ? (respondTo => Request(respondTo)) val f = target ? (respondTo => Request(respondTo))
f.futureValue should ===(Response(target)) Await.result(f, timeout.duration) should ===(Response(target))
} }
"work for ActorSelection" in { "work for ActorSelection" in {
@ -43,7 +44,7 @@ class ExplicitAskSpec extends AkkaSpec {
val selection = system.actorSelection("/user/select-echo") val selection = system.actorSelection("/user/select-echo")
val f = selection ? (respondTo => Request(respondTo)) val f = selection ? (respondTo => Request(respondTo))
f.futureValue should ===(Response(target)) Await.result(f, timeout.duration) should ===(Response(target))
} }
} }

View file

@ -12,7 +12,7 @@ import scala.concurrent.duration._
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
import akka.actor.{ Actor, Props } import akka.actor.{ actorRef2Scala, Actor, Props }
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.testkit.{ AkkaSpec, ImplicitSender, TestLatch } import akka.testkit.{ AkkaSpec, ImplicitSender, TestLatch }

View file

@ -8,7 +8,7 @@ import java.util.concurrent.atomic.AtomicInteger
import scala.concurrent.Await import scala.concurrent.Await
import akka.actor.{ Actor, Props } import akka.actor.{ actorRef2Scala, Actor, Props }
import akka.pattern.ask import akka.pattern.ask
import akka.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender, TestLatch } import akka.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender, TestLatch }

View file

@ -8,12 +8,12 @@ import scala.collection.immutable
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import com.typesafe.config.Config import com.typesafe.config.Config
import language.postfixOps import language.postfixOps
import akka.ConfigurationException import akka.ConfigurationException
import akka.actor.{ Actor, ActorRef, Deploy, Props } import akka.actor.{ actorRef2Scala, Actor, ActorRef, Deploy, Props }
import akka.actor.ActorPath import akka.actor.ActorPath
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
@ -170,7 +170,7 @@ class ConfiguredLocalRoutingSpec
"not get confused when trying to wildcard-configure children" in { "not get confused when trying to wildcard-configure children" in {
system.actorOf(FromConfig.props(routeeProps = Props(classOf[SendRefAtStartup], testActor)), "weird") system.actorOf(FromConfig.props(routeeProps = Props(classOf[SendRefAtStartup], testActor)), "weird")
val recv = (for (_ <- 1 to 3) yield expectMsgType[ActorRef].path.elements.mkString("/", "/", "")).toSet val recv = (for (_ <- 1 to 3) yield expectMsgType[ActorRef].path.elements.mkString("/", "/", "")).toSet
@silent @nowarn
val expc = Set('a', 'b', 'c').map(i => "/user/weird/$" + i) val expc = Set('a', 'b', 'c').map(i => "/user/weird/$" + i)
recv should ===(expc) recv should ===(expc)
expectNoMessage(1 second) expectNoMessage(1 second)

View file

@ -10,6 +10,7 @@ import scala.concurrent.ExecutionContextExecutor
import akka.actor.Actor import akka.actor.Actor
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.actor.Props import akka.actor.Props
import akka.actor.actorRef2Scala
import akka.pattern.ask import akka.pattern.ask
import akka.routing.ConsistentHashingRouter.ConsistentHashMapping import akka.routing.ConsistentHashingRouter.ConsistentHashMapping
import akka.routing.ConsistentHashingRouter.ConsistentHashable import akka.routing.ConsistentHashingRouter.ConsistentHashable

View file

@ -11,7 +11,7 @@ import scala.concurrent.duration._
import language.postfixOps import language.postfixOps
import akka.actor.{ Actor, Props } import akka.actor.{ actorRef2Scala, Actor, Props }
import akka.pattern.ask import akka.pattern.ask
import akka.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender, TestLatch } import akka.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender, TestLatch }

View file

@ -10,7 +10,7 @@ import scala.concurrent.duration._
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import language.postfixOps import language.postfixOps
import akka.actor.{ Actor, ActorRef, ActorSystem, Props } import akka.actor.{ actorRef2Scala, Actor, ActorRef, ActorSystem, Props }
import akka.pattern.ask import akka.pattern.ask
import akka.testkit._ import akka.testkit._
import akka.testkit.TestEvent._ import akka.testkit.TestEvent._

View file

@ -15,6 +15,7 @@ import akka.actor.Actor
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.actor.Props import akka.actor.Props
import akka.actor.Terminated import akka.actor.Terminated
import akka.actor.actorRef2Scala
import akka.pattern.ask import akka.pattern.ask
import akka.testkit._ import akka.testkit._

View file

@ -10,6 +10,7 @@ import akka.actor.Actor
import akka.actor.ActorIdentity import akka.actor.ActorIdentity
import akka.actor.Identify import akka.actor.Identify
import akka.actor.Props import akka.actor.Props
import akka.actor.actorRef2Scala
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
class RouteeCreationSpec extends AkkaSpec { class RouteeCreationSpec extends AkkaSpec {

View file

@ -10,7 +10,7 @@ import java.util.concurrent.atomic.AtomicInteger
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.{ Actor, Props } import akka.actor.{ actorRef2Scala, Actor, Props }
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.actor.Status import akka.actor.Status
import akka.pattern.ask import akka.pattern.ask

View file

@ -8,7 +8,7 @@ import java.util.concurrent.ConcurrentHashMap
import scala.concurrent.Await import scala.concurrent.Await
import akka.actor.{ Actor, Props } import akka.actor.{ actorRef2Scala, Actor, Props }
import akka.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender, TestLatch } import akka.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender, TestLatch }
class SmallestMailboxSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { class SmallestMailboxSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {

View file

@ -9,7 +9,7 @@ import java.util.concurrent.atomic.AtomicInteger
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.{ Actor, ActorRef, ActorSystem, Props } import akka.actor.{ actorRef2Scala, Actor, ActorRef, ActorSystem, Props }
import akka.actor.Status.Failure import akka.actor.Status.Failure
import akka.pattern.{ ask, AskTimeoutException } import akka.pattern.{ ask, AskTimeoutException }
import akka.testkit._ import akka.testkit._

View file

@ -9,6 +9,7 @@ import java.nio.{ ByteBuffer, ByteOrder }
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.actor.actorRef2Scala
import akka.testkit._ import akka.testkit._
object DisabledJavaSerializerWarningSpec { object DisabledJavaSerializerWarningSpec {

View file

@ -12,7 +12,7 @@ import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import SerializationTests._ import SerializationTests._
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import com.typesafe.config._ import com.typesafe.config._
import language.postfixOps import language.postfixOps
import test.akka.serialization.NoVerification import test.akka.serialization.NoVerification
@ -58,10 +58,10 @@ object SerializationTests {
protected[akka] trait Marker protected[akka] trait Marker
protected[akka] trait Marker2 protected[akka] trait Marker2
@silent // can't use unused otherwise case class below gets a deprecated @nowarn // can't use unused otherwise case class below gets a deprecated
class SimpleMessage(s: String) extends Marker class SimpleMessage(s: String) extends Marker
@silent @nowarn
class ExtendedSimpleMessage(s: String, i: Int) extends SimpleMessage(s) class ExtendedSimpleMessage(s: String, i: Int) extends SimpleMessage(s)
trait AnotherInterface extends Marker trait AnotherInterface extends Marker

View file

@ -101,7 +101,7 @@ class BoundedBlockingQueueSpec
val TestContext(queue, events, _, _, _, _) = newBoundedBlockingQueue(1) val TestContext(queue, events, _, _, _, _) = newBoundedBlockingQueue(1)
queue.offer("1") queue.offer("1")
mustBlockFor(100 milliseconds) { mustBlockFor(100.milliseconds) {
queue.put("2") queue.put("2")
} }
(events should contain).inOrder(offer("1"), awaitNotFull) (events should contain).inOrder(offer("1"), awaitNotFull)
@ -114,12 +114,12 @@ class BoundedBlockingQueueSpec
val f = Future(queue.put("b")) val f = Future(queue.put("b"))
after(10 milliseconds) { after(10.milliseconds) {
f.isCompleted should be(false) f.isCompleted should be(false)
queue.take() queue.take()
} }
Await.result(f, 3 seconds) Await.result(f, 3.seconds)
(events should contain).inOrder(offer("a"), poll, offer("b")) (events should contain).inOrder(offer("a"), poll, offer("b"))
} }
@ -130,14 +130,14 @@ class BoundedBlockingQueueSpec
// Blocks until another thread signals `notFull` // Blocks until another thread signals `notFull`
val f = Future(queue.put("b")) val f = Future(queue.put("b"))
after(10 milliseconds) { after(10.milliseconds) {
f.isCompleted should be(false) f.isCompleted should be(false)
lock.lockInterruptibly() lock.lockInterruptibly()
notFull.signal() notFull.signal()
lock.unlock() lock.unlock()
} }
mustBlockFor(100 milliseconds, f) mustBlockFor(100.milliseconds, f)
events.toList should containInSequence(offer("a"), awaitNotFull, signalNotFull, getSize, awaitNotFull) events.toList should containInSequence(offer("a"), awaitNotFull, signalNotFull, getSize, awaitNotFull)
events shouldNot contain(offer("b")) events shouldNot contain(offer("b"))
} }
@ -162,7 +162,7 @@ class BoundedBlockingQueueSpec
"block when the queue is empty" in { "block when the queue is empty" in {
val TestContext(queue, events, _, _, _, _) = newBoundedBlockingQueue(1) val TestContext(queue, events, _, _, _, _) = newBoundedBlockingQueue(1)
mustBlockFor(100 milliseconds) { mustBlockFor(100.milliseconds) {
queue.take() queue.take()
} }
events should contain(awaitNotEmpty) events should contain(awaitNotEmpty)
@ -173,12 +173,12 @@ class BoundedBlockingQueueSpec
val TestContext(queue, events, _, _, _, _) = newBoundedBlockingQueue(1) val TestContext(queue, events, _, _, _, _) = newBoundedBlockingQueue(1)
val f = Future(queue.take()) val f = Future(queue.take())
after(10 milliseconds) { after(10.milliseconds) {
f.isCompleted should be(false) f.isCompleted should be(false)
queue.put("a") queue.put("a")
} }
Await.ready(f, 3 seconds) Await.ready(f, 3.seconds)
(events should contain).inOrder(awaitNotEmpty, offer("a"), poll) (events should contain).inOrder(awaitNotEmpty, offer("a"), poll)
} }
@ -189,7 +189,7 @@ class BoundedBlockingQueueSpec
val f = Future(queue.take()) val f = Future(queue.take())
// Cause `notFull` signal, but don't fill the queue // Cause `notFull` signal, but don't fill the queue
after(10 milliseconds) { after(10.milliseconds) {
f.isCompleted should be(false) f.isCompleted should be(false)
lock.lockInterruptibly() lock.lockInterruptibly()
notEmpty.signal() notEmpty.signal()
@ -197,7 +197,7 @@ class BoundedBlockingQueueSpec
} }
// `f` should still block since the queue is still empty // `f` should still block since the queue is still empty
mustBlockFor(100 milliseconds, f) mustBlockFor(100.milliseconds, f)
events.toList should containInSequence(getSize, awaitNotEmpty, signalNotEmpty, getSize, awaitNotEmpty) events.toList should containInSequence(getSize, awaitNotEmpty, signalNotEmpty, getSize, awaitNotEmpty)
events shouldNot contain(poll) events shouldNot contain(poll)
} }
@ -252,11 +252,11 @@ class BoundedBlockingQueueSpec
notFull.manualTimeControl(true) notFull.manualTimeControl(true)
val f = Future(queue.offer("World", 100, TimeUnit.MILLISECONDS)) val f = Future(queue.offer("World", 100, TimeUnit.MILLISECONDS))
after(10 milliseconds) { after(10.milliseconds) {
f.isCompleted should be(false) f.isCompleted should be(false)
notFull.advanceTime(99 milliseconds) notFull.advanceTime(99.milliseconds)
} }
mustBlockFor(100 milliseconds, f) mustBlockFor(100.milliseconds, f)
events shouldNot contain(offer("World")) events shouldNot contain(offer("World"))
} }
@ -274,12 +274,12 @@ class BoundedBlockingQueueSpec
notFull.manualTimeControl(true) notFull.manualTimeControl(true)
val f = Future(queue.offer("World", 100, TimeUnit.MILLISECONDS)) val f = Future(queue.offer("World", 100, TimeUnit.MILLISECONDS))
notFull.advanceTime(99 milliseconds) notFull.advanceTime(99.milliseconds)
after(50 milliseconds) { after(50.milliseconds) {
f.isCompleted should be(false) f.isCompleted should be(false)
queue.take() queue.take()
} }
Await.result(f, 3 seconds) should equal(true) Await.result(f, 3.seconds) should equal(true)
(events should contain).inOrder(awaitNotFull, signalNotFull, offer("World")) (events should contain).inOrder(awaitNotFull, signalNotFull, offer("World"))
} }
@ -290,7 +290,7 @@ class BoundedBlockingQueueSpec
val f = Future(queue.offer("World", 1000, TimeUnit.DAYS)) val f = Future(queue.offer("World", 1000, TimeUnit.DAYS))
// Cause `notFull` signal, but don't fill the queue // Cause `notFull` signal, but don't fill the queue
after(10 milliseconds) { after(10.milliseconds) {
f.isCompleted should be(false) f.isCompleted should be(false)
lock.lockInterruptibly() lock.lockInterruptibly()
notFull.signal() notFull.signal()
@ -298,7 +298,7 @@ class BoundedBlockingQueueSpec
} }
// `f` should still block since the queue is still empty // `f` should still block since the queue is still empty
mustBlockFor(100 milliseconds, f) mustBlockFor(100.milliseconds, f)
events.toList should containInSequence(getSize, awaitNotFull, signalNotFull, getSize, awaitNotFull) events.toList should containInSequence(getSize, awaitNotFull, signalNotFull, getSize, awaitNotFull)
events shouldNot contain(offer("World")) events shouldNot contain(offer("World"))
} }

View file

@ -6,12 +6,12 @@ package akka.util
import java.io.InputStream import java.io.InputStream
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
import scala.collection.mutable import scala.collection.mutable
import scala.collection.mutable.ArrayBuilder import scala.collection.mutable.ArrayBuilder
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
class ByteStringInitializationSpec extends AnyWordSpec with Matchers { class ByteStringInitializationSpec extends AnyWordSpec with Matchers {
"ByteString intialization" should { "ByteString intialization" should {
"not get confused by initializing CompactByteString before ByteString" in { "not get confused by initializing CompactByteString before ByteString" in {

View file

@ -12,7 +12,7 @@ import java.nio.ByteOrder.{ BIG_ENDIAN, LITTLE_ENDIAN }
import scala.collection.mutable.Builder import scala.collection.mutable.Builder
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import org.apache.commons.codec.binary.Hex.encodeHex import org.apache.commons.codec.binary.Hex.encodeHex
import org.scalacheck.{ Arbitrary, Gen } import org.scalacheck.{ Arbitrary, Gen }
import org.scalacheck.Arbitrary.arbitrary import org.scalacheck.Arbitrary.arbitrary
@ -167,7 +167,7 @@ class ByteStringSpec extends AnyWordSpec with Matchers with Checkers {
body(bsA, bsB) == body(vecA, vecB) body(bsA, bsB) == body(vecA, vecB)
} }
@silent @nowarn
def likeVecIt(bs: ByteString)(body: BufferedIterator[Byte] => Any, strict: Boolean = true): Boolean = { def likeVecIt(bs: ByteString)(body: BufferedIterator[Byte] => Any, strict: Boolean = true): Boolean = {
val bsIterator = bs.iterator val bsIterator = bs.iterator
val vecIterator = Vector(bs: _*).iterator.buffered val vecIterator = Vector(bs: _*).iterator.buffered
@ -175,7 +175,7 @@ class ByteStringSpec extends AnyWordSpec with Matchers with Checkers {
(!strict || (bsIterator.toSeq == vecIterator.toSeq)) (!strict || (bsIterator.toSeq == vecIterator.toSeq))
} }
@silent @nowarn
def likeVecIts(a: ByteString, b: ByteString)( def likeVecIts(a: ByteString, b: ByteString)(
body: (BufferedIterator[Byte], BufferedIterator[Byte]) => Any, body: (BufferedIterator[Byte], BufferedIterator[Byte]) => Any,
strict: Boolean = true): Boolean = { strict: Boolean = true): Boolean = {

View file

@ -6,7 +6,7 @@ package akka.actor.typed
import java.util.function.{ Function => F1 } import java.util.function.{ Function => F1 }
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import org.scalactic.TypeCheckedTripleEquals import org.scalactic.TypeCheckedTripleEquals
import org.scalatest.matchers.should.Matchers import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike import org.scalatest.wordspec.AnyWordSpecLike
@ -22,7 +22,7 @@ import akka.japi.pf.{ FI, PFBuilder }
object BehaviorSpec { object BehaviorSpec {
sealed trait Command { sealed trait Command {
@silent @nowarn
def expectedResponse(context: TypedActorContext[Command]): Seq[Event] = Nil def expectedResponse(context: TypedActorContext[Command]): Seq[Event] = Nil
} }
case object GetSelf extends Command { case object GetSelf extends Command {
@ -73,9 +73,9 @@ object BehaviorSpec {
trait Common extends AnyWordSpecLike with Matchers with TypeCheckedTripleEquals with LogCapturing { trait Common extends AnyWordSpecLike with Matchers with TypeCheckedTripleEquals with LogCapturing {
type Aux >: Null <: AnyRef type Aux >: Null <: AnyRef
def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux)
@silent("never used") @nowarn("msg=never used")
def checkAux(signal: Signal, aux: Aux): Unit = () def checkAux(signal: Signal, aux: Aux): Unit = ()
@silent("never used") @nowarn("msg=never used")
def checkAux(command: Command, aux: Aux): Unit = () def checkAux(command: Command, aux: Aux): Unit = ()
case class Init(behv: Behavior[Command], inbox: TestInbox[Event], aux: Aux) { case class Init(behv: Behavior[Command], inbox: TestInbox[Event], aux: Aux) {

View file

@ -6,7 +6,7 @@ package docs.akka.typed
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import org.scalatest.wordspec.AnyWordSpecLike import org.scalatest.wordspec.AnyWordSpecLike
object FaultToleranceDocSpec { object FaultToleranceDocSpec {
@ -77,7 +77,7 @@ object FaultToleranceDocSpec {
// #bubbling-example // #bubbling-example
} }
@silent("never used") @nowarn("msg=never used")
class FaultToleranceDocSpec extends ScalaTestWithActorTestKit(""" class FaultToleranceDocSpec extends ScalaTestWithActorTestKit("""
# silenced to not put noise in test logs # silenced to not put noise in test logs
akka.loglevel = off akka.loglevel = off

View file

@ -12,7 +12,7 @@ import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing import akka.actor.testkit.typed.scaladsl.LogCapturing
import docs.akka.typed.IntroSpec.HelloWorld import docs.akka.typed.IntroSpec.HelloWorld
import org.scalatest.wordspec.AnyWordSpecLike import org.scalatest.wordspec.AnyWordSpecLike
import com.github.ghik.silencer.silent import scala.annotation.nowarn
//#imports1 //#imports1
import akka.actor.typed.Behavior import akka.actor.typed.Behavior
@ -33,7 +33,7 @@ import akka.util.Timeout
object SpawnProtocolDocSpec { object SpawnProtocolDocSpec {
// Silent because we want to name the unused 'context' parameter // Silent because we want to name the unused 'context' parameter
@silent("never used") @nowarn("msg=never used")
//#main //#main
object HelloWorldMain { object HelloWorldMain {
def apply(): Behavior[SpawnProtocol.Command] = def apply(): Behavior[SpawnProtocol.Command] =

View file

@ -14,7 +14,7 @@ import akka.actor.typed.SupervisorStrategy
import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.FiniteDuration
import akka.Done import akka.Done
import com.github.ghik.silencer.silent import scala.annotation.nowarn
//#oo-style //#oo-style
//#fun-style //#fun-style
@ -481,7 +481,7 @@ object StyleGuideDocExamples {
//#pattern-match-guard //#pattern-match-guard
} }
@silent @nowarn
private def counter(remaining: Int): Behavior[Command] = { private def counter(remaining: Int): Behavior[Command] = {
//#pattern-match-without-guard //#pattern-match-without-guard
Behaviors.receiveMessage { Behaviors.receiveMessage {
@ -507,7 +507,7 @@ object StyleGuideDocExamples {
} }
//#pattern-match-unhandled //#pattern-match-unhandled
@silent @nowarn
object partial { object partial {
//#pattern-match-partial //#pattern-match-partial
private val zero: Behavior[Command] = { private val zero: Behavior[Command] = {

View file

@ -9,7 +9,7 @@ import akka.actor.typed.Behavior
import akka.actor.typed.Extension import akka.actor.typed.Extension
import akka.actor.typed.ExtensionId import akka.actor.typed.ExtensionId
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.concurrent.Future import scala.concurrent.Future
@ -30,7 +30,7 @@ object DatabasePool extends ExtensionId[DatabasePool] {
} }
//#extension-id //#extension-id
@silent @nowarn
//#extension //#extension
class DatabasePool(system: ActorSystem[_]) extends Extension { class DatabasePool(system: ActorSystem[_]) extends Extension {
// database configuration can be loaded from config // database configuration can be loaded from config
@ -41,7 +41,7 @@ class DatabasePool(system: ActorSystem[_]) extends Extension {
} }
//#extension //#extension
@silent @nowarn
object ExtensionDocSpec { object ExtensionDocSpec {
val config = ConfigFactory.parseString(""" val config = ConfigFactory.parseString("""
#config #config

View file

@ -9,7 +9,7 @@ import akka.actor.typed.PostStop
import akka.actor.typed.PreRestart import akka.actor.typed.PreRestart
import akka.actor.typed.{ Behavior, SupervisorStrategy } import akka.actor.typed.{ Behavior, SupervisorStrategy }
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import scala.concurrent.duration._ import scala.concurrent.duration._
@ -112,7 +112,7 @@ object SupervisionCompileOnly {
} }
def claimResource(): Resource = ??? def claimResource(): Resource = ???
@silent("never used") @nowarn("msg=never used")
//#restart-PreRestart-signal //#restart-PreRestart-signal
def withPreRestart: Behavior[String] = { def withPreRestart: Behavior[String] = {
Behaviors Behaviors

View file

@ -13,7 +13,7 @@ import java.util.concurrent.CompletionStage
import scala.concurrent.{ ExecutionContextExecutor, Future } import scala.concurrent.{ ExecutionContextExecutor, Future }
import scala.reflect.ClassTag import scala.reflect.ClassTag
import scala.util.Try import scala.util.Try
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import org.slf4j.Logger import org.slf4j.Logger
import org.slf4j.LoggerFactory import org.slf4j.LoggerFactory
import akka.actor.Address import akka.actor.Address
@ -219,7 +219,7 @@ import scala.util.Success
} }
// Java API impl // Java API impl
@silent("never used") // resClass is just a pretend param @nowarn("msg=never used") // resClass is just a pretend param
override def ask[Req, Res]( override def ask[Req, Res](
resClass: Class[Res], resClass: Class[Res],
target: RecipientRef[Req], target: RecipientRef[Req],

View file

@ -7,7 +7,7 @@ package akka.actor.typed.scaladsl
import java.util.concurrent.TimeoutException import java.util.concurrent.TimeoutException
import scala.concurrent.Future import scala.concurrent.Future
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import akka.actor.typed.ActorRef import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem import akka.actor.typed.ActorSystem
import akka.actor.typed.RecipientRef import akka.actor.typed.RecipientRef
@ -103,7 +103,7 @@ object AskPattern {
* *
* @tparam Res The response protocol, what the other actor sends back * @tparam Res The response protocol, what the other actor sends back
*/ */
@silent("never used") @nowarn("msg=never used")
def ask[Res](replyTo: ActorRef[Res] => Req)(implicit timeout: Timeout, scheduler: Scheduler): Future[Res] = { def ask[Res](replyTo: ActorRef[Res] => Req)(implicit timeout: Timeout, scheduler: Scheduler): Future[Res] = {
// We do not currently use the implicit sched, but want to require it // We do not currently use the implicit sched, but want to require it
// because it might be needed when we move to a 'native' typed runtime, see #24219 // because it might be needed when we move to a 'native' typed runtime, see #24219

View file

@ -4,13 +4,13 @@
package akka.japi.function package akka.japi.function
import com.github.ghik.silencer.silent import scala.annotation.nowarn
[3..22#/** [3..22#/**
* A Function interface. Used to create 1-arg first-class-functions is Java. * A Function interface. Used to create 1-arg first-class-functions is Java.
* `Serializable` is needed to be able to grab line number for Java 8 lambdas. * `Serializable` is needed to be able to grab line number for Java 8 lambdas.
*/ */
@silent @nowarn
@SerialVersionUID(##1L) @SerialVersionUID(##1L)
trait Function1[[#-T1#], +R] extends java.io.Serializable { trait Function1[[#-T1#], +R] extends java.io.Serializable {
@throws(classOf[Exception]) @throws(classOf[Exception])
@ -24,7 +24,7 @@ trait Function1[[#-T1#], +R] extends java.io.Serializable {
* A Procedure is like a Function, but it doesn't produce a return value. * A Procedure is like a Function, but it doesn't produce a return value.
* `Serializable` is needed to be able to grab line number for Java 8 lambdas. * `Serializable` is needed to be able to grab line number for Java 8 lambdas.
*/ */
@silent("@SerialVersionUID has no effect") @nowarn("msg=@SerialVersionUID has no effect")
@SerialVersionUID(##1L) @SerialVersionUID(##1L)
trait Procedure1[[#-T1#]] extends java.io.Serializable { trait Procedure1[[#-T1#]] extends java.io.Serializable {
@throws(classOf[Exception]) @throws(classOf[Exception])

View file

@ -0,0 +1,5 @@
# Internal ActorClassificationUnsubscriber
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.event.ActorClassificationUnsubscriber.start")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.event.ActorClassificationUnsubscriber.start$default$3")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.event.ActorClassificationUnsubscriber.this")
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.event.ActorClassificationUnsubscriber")

View file

@ -7,7 +7,7 @@ package akka.compat
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.{ ExecutionContext, Future => SFuture } import scala.concurrent.{ ExecutionContext, Future => SFuture }
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.util.ccompat._ import akka.util.ccompat._
@ -20,7 +20,7 @@ import akka.util.ccompat._
* *
* Remove these classes as soon as support for Scala 2.12 is dropped! * Remove these classes as soon as support for Scala 2.12 is dropped!
*/ */
@silent @InternalApi private[akka] object Future { @nowarn @InternalApi private[akka] object Future {
def fold[T, R](futures: IterableOnce[SFuture[T]])(zero: R)(op: (R, T) => R)( def fold[T, R](futures: IterableOnce[SFuture[T]])(zero: R)(op: (R, T) => R)(
implicit executor: ExecutionContext): SFuture[R] = implicit executor: ExecutionContext): SFuture[R] =
SFuture.fold[T, R](futures)(zero)(op)(executor) SFuture.fold[T, R](futures)(zero)(op)(executor)

View file

@ -0,0 +1,26 @@
/*
* Copyright (C) 2009-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.compat
import scala.annotation.nowarn
import akka.annotation.InternalApi
/**
* INTERNAL API
*
* Compatibility wrapper for `scala.PartialFunction` to be able to compile the same code
* against Scala 2.12, 2.13, 3.0
*
* Remove these classes as soon as support for Scala 2.12 is dropped!
*/
@InternalApi private[akka] object PartialFunction {
def fromFunction[A, B](f: (A) => B): scala.PartialFunction[A, B] = {
@nowarn val pf = scala.PartialFunction(f)
pf
}
}

View file

@ -0,0 +1,23 @@
/*
* Copyright (C) 2009-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.compat
import akka.annotation.InternalApi
/**
* INTERNAL API
*
* Compatibility wrapper for `scala.PartialFunction` to be able to compile the same code
* against Scala 2.12, 2.13, 3.0
*
* Remove these classes as soon as support for Scala 2.12 is dropped!
*/
@InternalApi private[akka] object PartialFunction {
def fromFunction[A, B](f: (A) => B): scala.PartialFunction[A, B] = {
scala.PartialFunction.fromFunction(f)
}
}

View file

@ -16,7 +16,7 @@ import scala.collection.immutable.{ IndexedSeq, IndexedSeqOps, StrictOptimizedSe
import scala.collection.mutable.{ Builder, WrappedArray } import scala.collection.mutable.{ Builder, WrappedArray }
import scala.reflect.ClassTag import scala.reflect.ClassTag
import com.github.ghik.silencer.silent import scala.annotation.nowarn
object ByteString { object ByteString {
@ -897,7 +897,7 @@ sealed abstract class ByteString
* Java API: Returns an Iterable of read-only ByteBuffers that directly wraps this ByteStrings * Java API: Returns an Iterable of read-only ByteBuffers that directly wraps this ByteStrings
* all fragments. Will always have at least one entry. * all fragments. Will always have at least one entry.
*/ */
@silent @nowarn
def getByteBuffers(): JIterable[ByteBuffer] = { def getByteBuffers(): JIterable[ByteBuffer] = {
import scala.collection.JavaConverters.asJavaIterableConverter import scala.collection.JavaConverters.asJavaIterableConverter
asByteBuffers.asJava asByteBuffers.asJava

View file

@ -0,0 +1,50 @@
/*
* Copyright (C) 2009-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.compat
import scala.collection.immutable
import scala.concurrent.{ ExecutionContext, Future => SFuture }
import akka.annotation.InternalApi
/**
* INTERNAL API
*
* Compatibility wrapper for `scala.concurrent.Future` to be able to compile the same code
* against Scala 2.12, 2.13
*
* Remove these classes as soon as support for Scala 2.12 is dropped!
*/
@InternalApi private[akka] object Future {
def fold[T, R](futures: IterableOnce[SFuture[T]])(zero: R)(op: (R, T) => R)(
implicit executor: ExecutionContext): SFuture[R] = {
// This will have performance implications since the elements are copied to a Vector
SFuture.foldLeft[T, R](futures.iterator.to(immutable.Iterable))(zero)(op)(executor)
}
def fold[T, R](futures: immutable.Iterable[SFuture[T]])(zero: R)(op: (R, T) => R)(
implicit executor: ExecutionContext): SFuture[R] =
SFuture.foldLeft[T, R](futures)(zero)(op)(executor)
def reduce[T, R >: T](futures: IterableOnce[SFuture[T]])(op: (R, T) => R)(
implicit executor: ExecutionContext): SFuture[R] = {
// This will have performance implications since the elements are copied to a Vector
SFuture.reduceLeft[T, R](futures.iterator.to(immutable.Iterable))(op)(executor)
}
def reduce[T, R >: T](futures: immutable.Iterable[SFuture[T]])(op: (R, T) => R)(
implicit executor: ExecutionContext): SFuture[R] =
SFuture.reduceLeft[T, R](futures)(op)(executor)
def find[T](futures: IterableOnce[SFuture[T]])(p: T => Boolean)(
implicit executor: ExecutionContext): SFuture[Option[T]] = {
// This will have performance implications since the elements are copied to a Vector
SFuture.find[T](futures.iterator.to(immutable.Iterable))(p)(executor)
}
def find[T](futures: immutable.Iterable[SFuture[T]])(p: T => Boolean)(
implicit executor: ExecutionContext): SFuture[Option[T]] =
SFuture.find[T](futures)(p)(executor)
}

View file

@ -0,0 +1,23 @@
/*
* Copyright (C) 2009-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.compat
import akka.annotation.InternalApi
/**
* INTERNAL API
*
* Compatibility wrapper for `scala.PartialFunction` to be able to compile the same code
* against Scala 2.12, 2.13, 3.0
*
* Remove these classes as soon as support for Scala 2.12 is dropped!
*/
@InternalApi private[akka] object PartialFunction {
def fromFunction[A, B](f: (A) => B): scala.PartialFunction[A, B] = {
scala.PartialFunction.fromFunction(f)
}
}

View file

@ -0,0 +1,19 @@
/*
* Copyright (C) 2009-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.dispatch.internal
import scala.concurrent.ExecutionContext
import akka.annotation.InternalApi
/**
* Factory to create same thread ec. Not intended to be called from any other site than to create [[akka.dispatch.ExecutionContexts#parasitic]]
*
* INTERNAL API
*/
@InternalApi
private[dispatch] object SameThreadExecutionContext {
def apply(): ExecutionContext = ExecutionContext.parasitic
}

View file

@ -0,0 +1,699 @@
/*
* Copyright (C) 2009-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util
import java.nio.{ ByteBuffer, ByteOrder }
import scala.annotation.tailrec
import scala.collection.BufferedIterator
import scala.collection.LinearSeq
import scala.collection.mutable.ListBuffer
import scala.reflect.ClassTag
import akka.util.Collections.EmptyImmutableSeq
object ByteIterator {
object ByteArrayIterator {
protected[akka] def apply(array: Array[Byte]): ByteArrayIterator =
new ByteArrayIterator(array, 0, array.length)
protected[akka] def apply(array: Array[Byte], from: Int, until: Int): ByteArrayIterator =
new ByteArrayIterator(array, from, until)
val empty: ByteArrayIterator = apply(Array.emptyByteArray)
}
class ByteArrayIterator private (private var array: Array[Byte], private var from: Int, private var until: Int)
extends ByteIterator {
iterator =>
@inline final def len: Int = until - from
@inline final def hasNext: Boolean = from < until
@inline final def head: Byte = array(from)
final def next(): Byte = {
if (!hasNext) EmptyImmutableSeq.iterator.next()
else {
val i = from; from = from + 1; array(i)
}
}
def clear(): Unit = { this.array = Array.emptyByteArray; from = 0; until = from }
final override def size: Int = { val l = len; clear(); l }
final override def ++(that: IterableOnce[Byte]): ByteIterator = that match {
case that: ByteIterator =>
if (that.isEmpty) this
else if (this.isEmpty) that
else
that match {
case that: ByteArrayIterator =>
if ((this.array eq that.array) && (this.until == that.from)) {
this.until = that.until
that.clear()
this
} else {
val result = MultiByteArrayIterator(List(this, that))
this.clear()
result
}
case that: MultiByteArrayIterator => this ++: that
}
case _ => super.++(that)
}
final override def clone: ByteArrayIterator = new ByteArrayIterator(array, from, until)
final override def take(n: Int): this.type = {
if (n < len) until = { if (n > 0) (from + n) else from }
this
}
final override def drop(n: Int): this.type = {
if (n > 0) from = { if (n < len) (from + n) else until }
this
}
final override def takeWhile(p: Byte => Boolean): this.type = {
val prev = from
dropWhile(p)
until = from; from = prev
this
}
final override def dropWhile(p: Byte => Boolean): this.type = {
var stop = false
while (!stop && hasNext) {
if (p(array(from))) {
from = from + 1
} else {
stop = true
}
}
this
}
override def copyToArray[B >: Byte](xs: Array[B], start: Int): Int =
this.copyToArray(xs, start, xs.length)
override def copyToArray[B >: Byte](xs: Array[B]): Int =
this.copyToArray(xs, 0, xs.length)
final override def copyToArray[B >: Byte](xs: Array[B], start: Int, len: Int): Int = {
val n = 0 max ((xs.length - start) min this.len min len)
Array.copy(this.array, from, xs, start, n)
this.drop(n)
n
}
final override def toByteString: ByteString = {
val result =
if ((from == 0) && (until == array.length)) ByteString.ByteString1C(array)
else ByteString.ByteString1(array, from, len)
clear()
result
}
@throws[NoSuchElementException]
def getBytes(xs: Array[Byte], offset: Int, n: Int): this.type = {
if (n <= this.len) {
Array.copy(this.array, this.from, xs, offset, n)
this.drop(n)
} else throw new NoSuchElementException("next on empty iterator")
}
private def wrappedByteBuffer: ByteBuffer = ByteBuffer.wrap(array, from, len).asReadOnlyBuffer
def getShorts(xs: Array[Short], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = {
wrappedByteBuffer.order(byteOrder).asShortBuffer.get(xs, offset, n); drop(2 * n)
}
def getInts(xs: Array[Int], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = {
wrappedByteBuffer.order(byteOrder).asIntBuffer.get(xs, offset, n); drop(4 * n)
}
def getLongs(xs: Array[Long], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = {
wrappedByteBuffer.order(byteOrder).asLongBuffer.get(xs, offset, n); drop(8 * n)
}
def getFloats(xs: Array[Float], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = {
wrappedByteBuffer.order(byteOrder).asFloatBuffer.get(xs, offset, n); drop(4 * n)
}
def getDoubles(xs: Array[Double], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = {
wrappedByteBuffer.order(byteOrder).asDoubleBuffer.get(xs, offset, n); drop(8 * n)
}
def copyToBuffer(buffer: ByteBuffer): Int = {
val copyLength = math.min(buffer.remaining, len)
if (copyLength > 0) {
buffer.put(array, from, copyLength)
drop(copyLength)
}
copyLength
}
def asInputStream: java.io.InputStream = new java.io.InputStream {
override def available: Int = iterator.len
def read: Int = if (hasNext) (next().toInt & 0xff) else -1
override def read(b: Array[Byte], off: Int, len: Int): Int = {
if ((off < 0) || (len < 0) || (off + len > b.length)) throw new IndexOutOfBoundsException
if (len == 0) 0
else if (!isEmpty) {
val nRead = math.min(available(), len)
copyToArray(b, off, nRead)
nRead
} else -1
}
override def skip(n: Long): Long = {
val nSkip = math.min(iterator.len, n.toInt)
iterator.drop(nSkip)
nSkip
}
}
}
object MultiByteArrayIterator {
protected val clearedList: List[ByteArrayIterator] = List(ByteArrayIterator.empty)
val empty: MultiByteArrayIterator = new MultiByteArrayIterator(Nil)
protected[akka] def apply(iterators: LinearSeq[ByteArrayIterator]): MultiByteArrayIterator =
new MultiByteArrayIterator(iterators)
}
class MultiByteArrayIterator private (private var iterators: LinearSeq[ByteArrayIterator]) extends ByteIterator {
// After normalization:
// * iterators.isEmpty == false
// * (!iterator.head.isEmpty || iterators.tail.isEmpty) == true
private def normalize(): this.type = {
@tailrec def norm(xs: LinearSeq[ByteArrayIterator]): LinearSeq[ByteArrayIterator] = {
if (xs.isEmpty) MultiByteArrayIterator.clearedList
else if (xs.head.isEmpty) norm(xs.tail)
else xs
}
iterators = norm(iterators)
this
}
normalize()
@inline private def current: ByteArrayIterator = iterators.head
@inline private def dropCurrent(): Unit = { iterators = iterators.tail }
@inline def clear(): Unit = { iterators = MultiByteArrayIterator.empty.iterators }
@inline final def hasNext: Boolean = current.hasNext
@inline final def head: Byte = current.head
final def next(): Byte = {
val result = current.next()
normalize()
result
}
final override def len: Int = iterators.foldLeft(0) { _ + _.len }
final override def size: Int = {
val result = len
clear()
result
}
private[akka] def ++:(that: ByteArrayIterator): this.type = {
iterators = that +: iterators
this
}
final override def ++(that: IterableOnce[Byte]): ByteIterator = that match {
case that: ByteIterator =>
if (that.isEmpty) this
else if (this.isEmpty) that
else {
that match {
case that: ByteArrayIterator =>
iterators = this.iterators :+ that
that.clear()
this
case that: MultiByteArrayIterator =>
iterators = this.iterators ++ that.iterators
that.clear()
this
}
}
case _ => super.++(that)
}
final override def clone: MultiByteArrayIterator = {
val clonedIterators: List[ByteArrayIterator] = iterators.iterator.map(_.clone).to(List)
new MultiByteArrayIterator(clonedIterators)
}
/** For performance sensitive code, call take() directly on ByteString (it's optimised there) */
final override def take(n: Int): this.type = {
var rest = n
val builder = new ListBuffer[ByteArrayIterator]
while ((rest > 0) && !iterators.isEmpty) {
current.take(rest)
if (current.hasNext) {
rest -= current.len
builder += current
}
iterators = iterators.tail
}
iterators = builder.result()
normalize()
}
/** For performance sensitive code, call drop() directly on ByteString (it's optimised there) */
final override def drop(n: Int): this.type =
if ((n > 0) && !isEmpty) {
val nCurrent = math.min(n, current.len)
current.drop(n)
val rest = n - nCurrent
assert(current.isEmpty || (rest == 0))
normalize()
drop(rest)
} else this
final override def takeWhile(p: Byte => Boolean): this.type = {
var stop = false
val builder = new ListBuffer[ByteArrayIterator]
while (!stop && !iterators.isEmpty) {
val lastLen = current.len
current.takeWhile(p)
if (current.hasNext) builder += current
if (current.len < lastLen) stop = true
dropCurrent()
}
iterators = builder.result()
normalize()
}
@tailrec final override def dropWhile(p: Byte => Boolean): this.type =
if (!isEmpty) {
current.dropWhile(p)
val dropMore = current.isEmpty
normalize()
if (dropMore) dropWhile(p) else this
} else this
final override def copyToArray[B >: Byte](xs: Array[B], start: Int, len: Int): Int = {
var pos = start
var rest = len
while ((rest > 0) && !iterators.isEmpty && pos < xs.length) {
val n = 0 max ((xs.length - pos) min current.len min rest)
current.copyToArray(xs, pos, n)
pos += n
rest -= n
if (current.isEmpty) {
dropCurrent()
}
}
normalize()
len - rest
}
override def foreach[@specialized U](f: Byte => U): Unit = {
iterators.foreach { _.foreach(f) }
clear()
}
final override def toByteString: ByteString = {
if (iterators.tail.isEmpty) iterators.head.toByteString
else {
val result = iterators.foldLeft(ByteString.empty) { _ ++ _.toByteString }
clear()
result
}
}
@tailrec protected final def getToArray[A](xs: Array[A], offset: Int, n: Int, elemSize: Int)(getSingle: => A)(
getMult: (Array[A], Int, Int) => Unit): this.type =
if (n <= 0) this
else {
if (isEmpty) EmptyImmutableSeq.iterator.next()
val nDone = if (current.len >= elemSize) {
val nCurrent = math.min(n, current.len / elemSize)
getMult(xs, offset, nCurrent)
nCurrent
} else {
xs(offset) = getSingle
1
}
normalize()
getToArray(xs, offset + nDone, n - nDone, elemSize)(getSingle)(getMult)
}
def getBytes(xs: Array[Byte], offset: Int, n: Int): this.type =
getToArray(xs, offset, n, 1) { getByte } { current.getBytes(_, _, _) }
def getShorts(xs: Array[Short], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type =
getToArray(xs, offset, n, 2) { getShort(byteOrder) } { current.getShorts(_, _, _)(byteOrder) }
def getInts(xs: Array[Int], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type =
getToArray(xs, offset, n, 4) { getInt(byteOrder) } { current.getInts(_, _, _)(byteOrder) }
def getLongs(xs: Array[Long], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type =
getToArray(xs, offset, n, 8) { getLong(byteOrder) } { current.getLongs(_, _, _)(byteOrder) }
def getFloats(xs: Array[Float], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type =
getToArray(xs, offset, n, 8) { getFloat(byteOrder) } { current.getFloats(_, _, _)(byteOrder) }
def getDoubles(xs: Array[Double], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type =
getToArray(xs, offset, n, 8) { getDouble(byteOrder) } { current.getDoubles(_, _, _)(byteOrder) }
/** For performance sensitive code, call copyToBuffer() directly on ByteString (it's optimised there) */
override def copyToBuffer(buffer: ByteBuffer): Int = {
// the fold here is better than indexing into the LinearSeq
val n = iterators.foldLeft(0) { _ + _.copyToBuffer(buffer) }
normalize()
n
}
def asInputStream: java.io.InputStream = new java.io.InputStream {
override def available: Int = current.len
def read: Int = if (hasNext) (next().toInt & 0xff) else -1
override def read(b: Array[Byte], off: Int, len: Int): Int = {
val nRead = current.asInputStream.read(b, off, len)
normalize()
nRead
}
override def skip(n: Long): Long = {
@tailrec def skipImpl(n: Long, skipped: Long): Long =
if (n > 0) {
if (!isEmpty) {
val m = current.asInputStream.skip(n)
normalize()
val newN = n - m
val newSkipped = skipped + m
if (newN > 0) skipImpl(newN, newSkipped)
else newSkipped
} else 0
} else 0
skipImpl(n, 0)
}
}
}
}
/**
* An iterator over a ByteString.
*/
abstract class ByteIterator extends BufferedIterator[Byte] {
def len: Int
def head: Byte
def next(): Byte
protected def clear(): Unit
def ++(that: IterableOnce[Byte]): ByteIterator = {
val it = that.iterator
if (it.isEmpty) this
else ByteIterator.ByteArrayIterator(it.toArray)
}
// *must* be overridden by derived classes. This construction is necessary
// to specialize the return type, as the method is already implemented in
// the parent class.
override def clone: ByteIterator =
throw new UnsupportedOperationException("Method clone is not implemented in ByteIterator")
override def duplicate: (ByteIterator, ByteIterator) = (this, clone)
// *must* be overridden by derived classes. This construction is necessary
// to specialize the return type, as the method is already implemented in
// the parent class.
override def take(n: Int): this.type =
throw new UnsupportedOperationException("Method take is not implemented in ByteIterator")
// *must* be overridden by derived classes. This construction is necessary
// to specialize the return type, as the method is already implemented in
// the parent class.
override def drop(n: Int): this.type =
throw new UnsupportedOperationException("Method drop is not implemented in ByteIterator")
override def slice(from: Int, until: Int): this.type = {
if (from > 0) drop(from).take(until - from)
else take(until)
}
// *must* be overridden by derived classes. This construction is necessary
// to specialize the return type, as the method is already implemented in
// the parent class.
override def takeWhile(p: Byte => Boolean): this.type =
throw new UnsupportedOperationException("Method takeWhile is not implemented in ByteIterator")
// *must* be overridden by derived classes. This construction is necessary
// to specialize the return type, as the method is already implemented in
// the parent class.
override def dropWhile(p: Byte => Boolean): this.type =
throw new UnsupportedOperationException("Method dropWhile is not implemented in ByteIterator")
override def span(p: Byte => Boolean): (ByteIterator, ByteIterator) = {
val that = clone
this.takeWhile(p)
that.drop(this.len)
(this, that)
}
override def indexWhere(p: Byte => Boolean, from: Int = 0): Int = {
var index = 0
while (index < from) {
next()
index += 1
}
var found = false
while (!found && hasNext) if (p(next())) {
found = true
} else {
index += 1
}
if (found) index else -1
}
def indexOf(elem: Byte): Int = indexOf(elem, 0)
def indexOf(elem: Byte, from: Int): Int = indexWhere(_ == elem, from)
override def indexOf[B >: Byte](elem: B): Int = indexOf(elem, 0)
override def indexOf[B >: Byte](elem: B, from: Int): Int = indexWhere(_ == elem, from)
def toByteString: ByteString
override def toSeq: ByteString = toByteString
override def foreach[@specialized U](f: Byte => U): Unit =
while (hasNext) f(next())
override def foldLeft[@specialized B](z: B)(op: (B, Byte) => B): B = {
var acc = z
foreach { byte =>
acc = op(acc, byte)
}
acc
}
override def toArray[B >: Byte](implicit arg0: ClassTag[B]): Array[B] = {
val target = new Array[B](len)
copyToArray(target)
target
}
/**
* Get a single Byte from this iterator. Identical to next().
*/
def getByte: Byte = next()
/**
* Get a single Short from this iterator.
*/
def getShort(implicit byteOrder: ByteOrder): Short = {
if (byteOrder == ByteOrder.BIG_ENDIAN)
((next() & 0xff) << 8 | (next() & 0xff) << 0).toShort
else if (byteOrder == ByteOrder.LITTLE_ENDIAN)
((next() & 0xff) << 0 | (next() & 0xff) << 8).toShort
else throw new IllegalArgumentException("Unknown byte order " + byteOrder)
}
/**
* Get a single Int from this iterator.
*/
def getInt(implicit byteOrder: ByteOrder): Int = {
if (byteOrder == ByteOrder.BIG_ENDIAN)
((next() & 0xff) << 24
| (next() & 0xff) << 16
| (next() & 0xff) << 8
| (next() & 0xff) << 0)
else if (byteOrder == ByteOrder.LITTLE_ENDIAN)
((next() & 0xff) << 0
| (next() & 0xff) << 8
| (next() & 0xff) << 16
| (next() & 0xff) << 24)
else throw new IllegalArgumentException("Unknown byte order " + byteOrder)
}
/**
* Get a single Long from this iterator.
*/
def getLong(implicit byteOrder: ByteOrder): Long = {
if (byteOrder == ByteOrder.BIG_ENDIAN)
((next().toLong & 0xff) << 56
| (next().toLong & 0xff) << 48
| (next().toLong & 0xff) << 40
| (next().toLong & 0xff) << 32
| (next().toLong & 0xff) << 24
| (next().toLong & 0xff) << 16
| (next().toLong & 0xff) << 8
| (next().toLong & 0xff) << 0)
else if (byteOrder == ByteOrder.LITTLE_ENDIAN)
((next().toLong & 0xff) << 0
| (next().toLong & 0xff) << 8
| (next().toLong & 0xff) << 16
| (next().toLong & 0xff) << 24
| (next().toLong & 0xff) << 32
| (next().toLong & 0xff) << 40
| (next().toLong & 0xff) << 48
| (next().toLong & 0xff) << 56)
else throw new IllegalArgumentException("Unknown byte order " + byteOrder)
}
/**
* Get a Long from this iterator where only the least significant `n`
* bytes were encoded.
*/
def getLongPart(n: Int)(implicit byteOrder: ByteOrder): Long = {
if (byteOrder == ByteOrder.BIG_ENDIAN) {
var x = 0L
(1 to n).foreach(_ => x = (x << 8) | (next() & 0xff))
x
} else if (byteOrder == ByteOrder.LITTLE_ENDIAN) {
var x = 0L
(0 until n).foreach(i => x |= (next() & 0xff) << 8 * i)
x
} else throw new IllegalArgumentException("Unknown byte order " + byteOrder)
}
def getFloat(implicit byteOrder: ByteOrder): Float =
java.lang.Float.intBitsToFloat(getInt(byteOrder))
def getDouble(implicit byteOrder: ByteOrder): Double =
java.lang.Double.longBitsToDouble(getLong(byteOrder))
/**
* Get a specific number of Bytes from this iterator. In contrast to
* copyToArray, this method will fail if this.len &lt; xs.length.
*/
def getBytes(xs: Array[Byte]): this.type = getBytes(xs, 0, xs.length)
/**
* Get a specific number of Bytes from this iterator. In contrast to
* copyToArray, this method will fail if length &lt; n or if (xs.length - offset) &lt; n.
*/
def getBytes(xs: Array[Byte], offset: Int, n: Int): this.type
/**
* Get a specific number of Bytes from this iterator. In contrast to
* copyToArray, this method will fail if this.len &lt; n.
*/
def getBytes(n: Int): Array[Byte] = {
val bytes = new Array[Byte](n)
getBytes(bytes, 0, n)
bytes
}
/**
* Get a ByteString with specific number of Bytes from this iterator. In contrast to
* copyToArray, this method will fail if this.len &lt; n.
*/
def getByteString(n: Int): ByteString = {
val bs = clone.take(n).toByteString
drop(n)
bs
}
/**
* Get a number of Shorts from this iterator.
*/
def getShorts(xs: Array[Short])(implicit byteOrder: ByteOrder): this.type =
getShorts(xs, 0, xs.length)(byteOrder)
/**
* Get a number of Shorts from this iterator.
*/
def getShorts(xs: Array[Short], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type
/**
* Get a number of Ints from this iterator.
*/
def getInts(xs: Array[Int])(implicit byteOrder: ByteOrder): this.type =
getInts(xs, 0, xs.length)(byteOrder)
/**
* Get a number of Ints from this iterator.
*/
def getInts(xs: Array[Int], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type
/**
* Get a number of Longs from this iterator.
*/
def getLongs(xs: Array[Long])(implicit byteOrder: ByteOrder): this.type =
getLongs(xs, 0, xs.length)(byteOrder)
/**
* Get a number of Longs from this iterator.
*/
def getLongs(xs: Array[Long], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type
/**
* Get a number of Floats from this iterator.
*/
def getFloats(xs: Array[Float])(implicit byteOrder: ByteOrder): this.type =
getFloats(xs, 0, xs.length)(byteOrder)
/**
* Get a number of Floats from this iterator.
*/
def getFloats(xs: Array[Float], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type
/**
* Get a number of Doubles from this iterator.
*/
def getDoubles(xs: Array[Double])(implicit byteOrder: ByteOrder): this.type =
getDoubles(xs, 0, xs.length)(byteOrder)
/**
* Get a number of Doubles from this iterator.
*/
def getDoubles(xs: Array[Double], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type
/**
* Copy as many bytes as possible to a ByteBuffer, starting from it's
* current position. This method will not overflow the buffer.
*
* @param buffer a ByteBuffer to copy bytes to
* @return the number of bytes actually copied
*/
/** For performance sensitive code, call take() directly on ByteString (it's optimised there) */
def copyToBuffer(buffer: ByteBuffer): Int
/**
* Directly wraps this ByteIterator in an InputStream without copying.
* Read and skip operations on the stream will advance the iterator
* accordingly.
*/
def asInputStream: java.io.InputStream
}

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,18 @@
/*
* Copyright (C) 2019-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util.ccompat
import scala.annotation.Annotation
import akka.annotation.InternalApi
/**
* INTERNAL API
*
* Annotation to mark files that need ccompat to be imported for Scala 2.11 and/or 2.12,
* but not 2.13. Gets rid of the 'unused import' warning on 2.13.
*/
@InternalApi
private[akka] class ccompatUsedUntil213 extends Annotation

View file

@ -0,0 +1,23 @@
/*
* Copyright (C) 2018-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util
/**
* INTERNAL API
*
* Based on https://github.com/scala/scala-collection-compat/blob/master/compat/src/main/scala-2.13/scala/collection/compat/package.scala
* but reproduced here so we don't need to add a dependency on this library. It contains much more than we need right now, and is
* not promising binary compatibility yet at the time of writing.
*/
package object ccompat {
private[akka] type Factory[-A, +C] = scala.collection.Factory[A, C]
private[akka] val Factory = scala.collection.Factory
// When we drop support for 2.12 we can delete this concept
// and import scala.jdk.CollectionConverters.Ops._ instead
object JavaConverters
extends scala.collection.convert.AsJavaExtensions
with scala.collection.convert.AsScalaExtensions
}

View file

@ -10,7 +10,7 @@ import scala.concurrent.ExecutionContextExecutor
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import scala.runtime.BoxedUnit import scala.runtime.BoxedUnit
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import akka.annotation.DoNotInherit import akka.annotation.DoNotInherit
import akka.japi.pf.ReceiveBuilder import akka.japi.pf.ReceiveBuilder
@ -273,7 +273,7 @@ abstract class AbstractActor extends Actor {
// TODO In 2.6.0 we can remove deprecation and make the method final // TODO In 2.6.0 we can remove deprecation and make the method final
@deprecated("Override preRestart with message parameter with Optional type instead", "2.5.0") @deprecated("Override preRestart with message parameter with Optional type instead", "2.5.0")
@throws(classOf[Exception]) @throws(classOf[Exception])
@silent("deprecated") @nowarn("msg=deprecated")
override def preRestart(reason: Throwable, message: Option[Any]): Unit = { override def preRestart(reason: Throwable, message: Option[Any]): Unit = {
import scala.compat.java8.OptionConverters._ import scala.compat.java8.OptionConverters._
preRestart(reason, message.asJava) preRestart(reason, message.asJava)

View file

@ -171,8 +171,10 @@ abstract class AbstractFSM[S, D] extends FSM[S, D] {
* <b>Multiple handlers may be installed, and every one of them will be * <b>Multiple handlers may be installed, and every one of them will be
* called, not only the first one matching.</b> * called, not only the first one matching.</b>
*/ */
final def onTransition(transitionHandler: UnitApply2[S, S]): Unit = final def onTransition(transitionHandler: UnitApply2[S, S]): Unit = {
super.onTransition(transitionHandler(_: S, _: S)) val pf: PartialFunction[(S, S), Unit] = akka.compat.PartialFunction.fromFunction(transitionHandler(_: S, _: S))
super.onTransition(pf)
}
/** /**
* Set handler which is called upon reception of unhandled messages. Calling * Set handler which is called upon reception of unhandled messages. Calling

View file

@ -13,7 +13,7 @@ import scala.concurrent.ExecutionContextExecutor
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import scala.util.control.NonFatal import scala.util.control.NonFatal
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import akka.actor.dungeon.ChildrenContainer import akka.actor.dungeon.ChildrenContainer
import akka.annotation.{ InternalApi, InternalStableApi } import akka.annotation.{ InternalApi, InternalStableApi }
@ -406,7 +406,7 @@ private[akka] object ActorCell {
* supported APIs in this place. This is not the API you were looking * supported APIs in this place. This is not the API you were looking
* for! (waves hand) * for! (waves hand)
*/ */
@silent("deprecated") @nowarn("msg=deprecated")
private[akka] class ActorCell( private[akka] class ActorCell(
val system: ActorSystemImpl, val system: ActorSystemImpl,
val self: InternalActorRef, val self: InternalActorRef,
@ -687,7 +687,7 @@ private[akka] class ActorCell(
} }
@InternalStableApi @InternalStableApi
@silent("never used") @nowarn("msg=never used")
final protected def clearActorFields(actorInstance: Actor, recreate: Boolean): Unit = { final protected def clearActorFields(actorInstance: Actor, recreate: Boolean): Unit = {
currentMessage = null currentMessage = null
behaviorStack = emptyBehaviorStack behaviorStack = emptyBehaviorStack

View file

@ -6,10 +6,10 @@ package akka.actor
import java.lang.{ StringBuilder => JStringBuilder } import java.lang.{ StringBuilder => JStringBuilder }
import java.net.MalformedURLException import java.net.MalformedURLException
import scala.annotation.{ switch, tailrec } import scala.annotation.tailrec
import scala.collection.immutable import scala.collection.immutable
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import akka.japi.Util.immutableSeq import akka.japi.Util.immutableSeq
@ -87,7 +87,9 @@ object ActorPath {
final def validatePathElement(element: String, fullPath: String): Unit = { final def validatePathElement(element: String, fullPath: String): Unit = {
def fullPathMsg = if (fullPath ne null) s""" (in path [$fullPath])""" else "" def fullPathMsg = if (fullPath ne null) s""" (in path [$fullPath])""" else ""
(findInvalidPathElementCharPosition(element): @switch) match { // If the number of cases increase remember to add a `@switch` annotation e.g.:
// (findInvalidPathElementCharPosition(element): @switch) match {
(findInvalidPathElementCharPosition(element)) match {
case ValidPathCode => case ValidPathCode =>
// valid // valid
case EmptyPathCode => case EmptyPathCode =>
@ -153,8 +155,7 @@ object ActorPath {
* references are compared the unique id of the actor is not taken into account * references are compared the unique id of the actor is not taken into account
* when comparing actor paths. * when comparing actor paths.
*/ */
@silent("@SerialVersionUID has no effect on traits") @nowarn("msg=@SerialVersionUID has no effect on traits")
@silent("deprecated")
@SerialVersionUID(1L) @SerialVersionUID(1L)
sealed trait ActorPath extends Comparable[ActorPath] with Serializable { sealed trait ActorPath extends Comparable[ActorPath] with Serializable {
@ -203,6 +204,7 @@ sealed trait ActorPath extends Comparable[ActorPath] with Serializable {
/** /**
* Java API: Sequence of names for this path from root to this. Performance implication: has to allocate a list. * Java API: Sequence of names for this path from root to this. Performance implication: has to allocate a list.
*/ */
@nowarn("msg=deprecated")
def getElements: java.lang.Iterable[String] = def getElements: java.lang.Iterable[String] =
scala.collection.JavaConverters.asJavaIterableConverter(elements).asJava scala.collection.JavaConverters.asJavaIterableConverter(elements).asJava

View file

@ -9,6 +9,7 @@ import java.util.concurrent.atomic.AtomicLong
import scala.annotation.implicitNotFound import scala.annotation.implicitNotFound
import scala.concurrent.{ ExecutionContextExecutor, Future, Promise } import scala.concurrent.{ ExecutionContextExecutor, Future, Promise }
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.ConfigurationException import akka.ConfigurationException
import akka.annotation.DoNotInherit import akka.annotation.DoNotInherit
import akka.annotation.InternalApi import akka.annotation.InternalApi
@ -320,7 +321,7 @@ private[akka] object LocalActorRefProvider {
private class SystemGuardian(override val supervisorStrategy: SupervisorStrategy, val guardian: ActorRef) private class SystemGuardian(override val supervisorStrategy: SupervisorStrategy, val guardian: ActorRef)
extends Actor extends Actor
with RequiresMessageQueue[UnboundedMessageQueueSemantics] { with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import SystemGuardian._ import akka.actor.SystemGuardian._
var terminationHooks = Set.empty[ActorRef] var terminationHooks = Set.empty[ActorRef]
@ -391,7 +392,7 @@ private[akka] class LocalActorRefProvider private[akka] (
override val rootPath: ActorPath = RootActorPath(Address("akka", _systemName)) override val rootPath: ActorPath = RootActorPath(Address("akka", _systemName))
private[akka] val log: MarkerLoggingAdapter = private[akka] val log: MarkerLoggingAdapter =
Logging.withMarker(eventStream, getClass) Logging.withMarker(eventStream, classOf[LocalActorRefProvider])
/* /*
* This dedicated logger is used whenever a deserialization failure occurs * This dedicated logger is used whenever a deserialization failure occurs
@ -584,7 +585,7 @@ private[akka] class LocalActorRefProvider private[akka] (
ref ref
} }
lazy val tempContainer = new VirtualPathContainer(system.provider, tempNode, rootGuardian, log) lazy val tempContainer: VirtualPathContainer = new VirtualPathContainer(system.provider, tempNode, rootGuardian, log)
def registerTempActor(actorRef: InternalActorRef, path: ActorPath): Unit = { def registerTempActor(actorRef: InternalActorRef, path: ActorPath): Unit = {
assert(path.parent eq tempNode, "cannot registerTempActor() with anything not obtained from tempPath()") assert(path.parent eq tempNode, "cannot registerTempActor() with anything not obtained from tempPath()")

View file

@ -16,7 +16,7 @@ import scala.concurrent.duration._
import scala.language.implicitConversions import scala.language.implicitConversions
import scala.util.Success import scala.util.Success
import com.github.ghik.silencer.silent import scala.annotation.nowarn
import akka.dispatch.ExecutionContexts import akka.dispatch.ExecutionContexts
import akka.pattern.ask import akka.pattern.ask
@ -328,7 +328,7 @@ private[akka] final case class ActorSelectionMessage(
/** /**
* INTERNAL API * INTERNAL API
*/ */
@silent("@SerialVersionUID has no effect on traits") @nowarn("msg=@SerialVersionUID has no effect on traits")
@SerialVersionUID(1L) @SerialVersionUID(1L)
private[akka] sealed trait SelectionPathElement private[akka] sealed trait SelectionPathElement

View file

@ -1043,7 +1043,7 @@ private[akka] class ActorSystemImpl(
} }
def start(): this.type = _start def start(): this.type = _start
def registerOnTermination[T](code: => T): Unit = { registerOnTermination(new Runnable { def run = code }) } def registerOnTermination[T](code: => T): Unit = { registerOnTermination(new Runnable { def run() = code }) }
def registerOnTermination(code: Runnable): Unit = { terminationCallbacks.add(code) } def registerOnTermination(code: Runnable): Unit = { terminationCallbacks.add(code) }
@volatile private var terminating = false @volatile private var terminating = false

View file

@ -33,6 +33,14 @@ final case class Address private (protocol: String, system: String, host: Option
def this(protocol: String, system: String) = this(protocol, system, None, None) def this(protocol: String, system: String) = this(protocol, system, None, None)
def this(protocol: String, system: String, host: String, port: Int) = this(protocol, system, Option(host), Some(port)) def this(protocol: String, system: String, host: String, port: Int) = this(protocol, system, Option(host), Some(port))
def copy(
protocol: String = protocol,
system: String = system,
host: Option[String] = host,
port: Option[Int] = port) = {
Address(protocol, system, host, port)
}
/** /**
* Java API: The hostname if specified or empty optional if not * Java API: The hostname if specified or empty optional if not
*/ */

View file

@ -18,8 +18,10 @@ import scala.concurrent.duration._
import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.FiniteDuration
import scala.util.Try import scala.util.Try
import scala.util.control.NonFatal import scala.util.control.NonFatal
import com.typesafe.config.Config import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.Done import akka.Done
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.dispatch.ExecutionContexts import akka.dispatch.ExecutionContexts
@ -353,7 +355,7 @@ final class CoordinatedShutdown private[akka] (
import CoordinatedShutdown.{ Reason, UnknownReason } import CoordinatedShutdown.{ Reason, UnknownReason }
/** INTERNAL API */ /** INTERNAL API */
private[akka] val log = Logging(system, getClass) private[akka] val log = Logging(system, classOf[CoordinatedShutdown])
private val knownPhases = phases.keySet ++ phases.values.flatMap(_.dependsOn) private val knownPhases = phases.keySet ++ phases.values.flatMap(_.dependsOn)
/** INTERNAL API */ /** INTERNAL API */

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