+ */
+
+package akka.actor.typed.scaladsl
+
+import java.util.concurrent.CountDownLatch
+import java.util.concurrent.TimeUnit
+
+import scala.concurrent.Future
+import scala.util.Failure
+import scala.util.Success
+
+import akka.actor.testkit.typed.scaladsl.LogCapturing
+import akka.actor.testkit.typed.scaladsl.LoggingTestKit
+import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
+import akka.actor.typed.ActorRef
+import akka.actor.typed.Behavior
+import akka.actor.typed.scaladsl.ActorThreadSpec.Echo
+import org.scalatest.wordspec.AnyWordSpecLike
+
+object ActorThreadSpec {
+ object Echo {
+ final case class Msg(i: Int, replyTo: ActorRef[Int])
+
+ def apply(): Behavior[Msg] =
+ Behaviors.receiveMessage {
+ case Msg(i, replyTo) =>
+ replyTo ! i
+ Behaviors.same
+ }
+ }
+
+}
+
+class ActorThreadSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing {
+
+ "Actor thread-safety checks" must {
+
+ "detect illegal access to ActorContext from outside" in {
+ @volatile var context: ActorContext[String] = null
+ val probe = createTestProbe[String]()
+
+ spawn(Behaviors.setup[String] { ctx =>
+ // here it's ok
+ ctx.children
+ context = ctx
+ probe.ref ! "initialized"
+ Behaviors.empty
+ })
+
+ probe.expectMessage("initialized")
+ intercept[UnsupportedOperationException] {
+ context.children
+ }.getMessage should include("Unsupported access to ActorContext")
+
+ }
+
+ "detect illegal access to ActorContext from other thread when processing message" in {
+ val probe = createTestProbe[UnsupportedOperationException]()
+
+ val ref = spawn(Behaviors.receive[CountDownLatch] {
+ case (context, latch) =>
+ Future {
+ try {
+ context.children
+ } catch {
+ case e: UnsupportedOperationException =>
+ probe.ref ! e
+ }
+ }(context.executionContext)
+ latch.await(5, TimeUnit.SECONDS)
+ Behaviors.same
+ })
+
+ val l = new CountDownLatch(1)
+ try {
+ ref ! l
+ probe.receiveMessage().getMessage should include("Unsupported access to ActorContext")
+ } finally {
+ l.countDown()
+ }
+ }
+
+ "detect illegal access to ActorContext from other thread after processing message" in {
+ val probe = createTestProbe[UnsupportedOperationException]()
+
+ val ref = spawn(Behaviors.receive[CountDownLatch] {
+ case (context, latch) =>
+ Future {
+ try {
+ latch.await(5, TimeUnit.SECONDS)
+ context.children
+ } catch {
+ case e: UnsupportedOperationException =>
+ probe.ref ! e
+ }
+ }(context.executionContext)
+
+ Behaviors.stopped
+ })
+
+ val l = new CountDownLatch(1)
+ try {
+ ref ! l
+ probe.expectTerminated(ref)
+ } finally {
+ l.countDown()
+ }
+ probe.receiveMessage().getMessage should include("Unsupported access to ActorContext")
+ }
+
+ "detect illegal access from child" in {
+ val probe = createTestProbe[UnsupportedOperationException]()
+
+ val ref = spawn(Behaviors.receive[String] {
+ case (context, _) =>
+ // really bad idea to define a child actor like this
+ context.spawnAnonymous(Behaviors.setup[String] { _ =>
+ try {
+ context.children
+ } catch {
+ case e: UnsupportedOperationException =>
+ probe.ref ! e
+ }
+ Behaviors.empty
+ })
+ Behaviors.same
+ })
+
+ ref ! "hello"
+ probe.receiveMessage().getMessage should include("Unsupported access to ActorContext")
+ }
+
+ "allow access from message adapter" in {
+ val probe = createTestProbe[String]()
+ val echo = spawn(Echo())
+
+ spawn(Behaviors.setup[String] { context =>
+ val replyAdapter = context.messageAdapter[Int] { i =>
+ // this is allowed because the mapping function is running in the target actor
+ context.children
+ i.toString
+ }
+ echo ! Echo.Msg(17, replyAdapter)
+
+ Behaviors.receiveMessage { msg =>
+ probe.ref ! msg
+ Behaviors.same
+ }
+ })
+
+ probe.expectMessage("17")
+ }
+
+ "allow access from ask response mapper" in {
+ val probe = createTestProbe[String]()
+ val echo = spawn(Echo())
+
+ spawn(Behaviors.setup[String] { context =>
+ context.ask[Echo.Msg, Int](echo, Echo.Msg(18, _)) {
+ case Success(i) =>
+ // this is allowed because the mapping function is running in the target actor
+ context.children
+ i.toString
+ case Failure(e) => throw e
+ }
+
+ Behaviors.receiveMessage { msg =>
+ probe.ref ! msg
+ Behaviors.same
+ }
+ })
+
+ probe.expectMessage("18")
+ }
+
+ "detect wrong context in construction of AbstractBehavior" in {
+ val probe = createTestProbe[String]()
+ val ref = spawn(Behaviors.setup[String] { context =>
+ // missing setup new AbstractBehavior and passing in parent's context
+ val child = context.spawnAnonymous(new AbstractBehavior[String](context) {
+ override def onMessage(msg: String): Behavior[String] = {
+ probe.ref ! msg
+ Behaviors.same
+ }
+ })
+
+ Behaviors.receiveMessage { msg =>
+ child ! msg
+ Behaviors.same
+ }
+ })
+
+ // 2 occurrences because one from PostStop also
+ LoggingTestKit
+ .error[IllegalStateException]
+ .withMessageContains("was created with wrong ActorContext")
+ .withOccurrences(2)
+ .expect {
+ // it's not detected when spawned, but when processing message
+ ref ! "hello"
+ probe.expectNoMessage()
+ }
+ }
+
+ "detect illegal access from AbstractBehavior constructor" in {
+ val probe = createTestProbe[UnsupportedOperationException]()
+
+ spawn(Behaviors.setup[String] { context =>
+ context.spawnAnonymous(
+ Behaviors.setup[String](_ =>
+ // wrongly using parent's context
+ new AbstractBehavior[String](context) {
+ try {
+ this.context.children
+ } catch {
+ case e: UnsupportedOperationException =>
+ probe.ref ! e
+ }
+
+ override def onMessage(msg: String): Behavior[String] = {
+ Behaviors.same
+ }
+ }))
+
+ Behaviors.empty
+ })
+
+ probe.receiveMessage().getMessage should include("Unsupported access to ActorContext")
+ }
+
+ "detect sharing of same AbstractBehavior instance" in {
+ // extremely contrived example, but the creativity among users can be great
+ @volatile var behv: Behavior[CountDownLatch] = null
+
+ val ref1 = spawn(Behaviors.setup[CountDownLatch] { context =>
+ behv = new AbstractBehavior[CountDownLatch](context) {
+ override def onMessage(latch: CountDownLatch): Behavior[CountDownLatch] = {
+ latch.await(5, TimeUnit.SECONDS)
+ Behaviors.same
+ }
+ }
+ behv
+ })
+
+ eventually(behv shouldNot equal(null))
+
+ // spawning same instance again
+ val ref2 = spawn(behv)
+
+ val latch1 = new CountDownLatch(1)
+ try {
+ ref1 ! latch1
+
+ // 2 occurrences because one from PostStop also
+ LoggingTestKit
+ .error[IllegalStateException]
+ .withMessageContains("was created with wrong ActorContext")
+ .withOccurrences(2)
+ .expect {
+ ref2 ! new CountDownLatch(0)
+ }
+ } finally {
+ latch1.countDown()
+ }
+ }
+
+ }
+
+}
diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/AdaptationFailureSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/AdaptationFailureSpec.scala
index 36d3098ac0..653b0556f4 100644
--- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/AdaptationFailureSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/AdaptationFailureSpec.scala
@@ -4,6 +4,10 @@
package akka.actor.typed.scaladsl
+import scala.concurrent.Promise
+
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.Done
import akka.actor.testkit.typed.TestException
import akka.actor.testkit.typed.scaladsl.LogCapturing
@@ -13,9 +17,6 @@ import akka.actor.typed.MessageAdaptionFailure
import akka.actor.typed.PreRestart
import akka.actor.typed.Signal
import akka.actor.typed.Terminated
-import org.scalatest.wordspec.AnyWordSpecLike
-
-import scala.concurrent.Promise
object AdaptationFailureSpec {
diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/DispatcherSelectorSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/DispatcherSelectorSpec.scala
index 20407251fa..f22c27f81c 100644
--- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/DispatcherSelectorSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/DispatcherSelectorSpec.scala
@@ -4,19 +4,22 @@
package akka.actor.typed.scaladsl
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.BootstrapSetup
import akka.actor.setup.ActorSystemSetup
import akka.actor.testkit.typed.scaladsl.ActorTestKit
+import akka.actor.testkit.typed.scaladsl.LogCapturing
+import akka.actor.testkit.typed.scaladsl.LoggingTestKit
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
-import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.Props
import akka.actor.typed.SpawnProtocol
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object DispatcherSelectorSpec {
val config = ConfigFactory.parseString("""
@@ -40,13 +43,15 @@ object DispatcherSelectorSpec {
}
-class DispatcherSelectorSpec
- extends ScalaTestWithActorTestKit(DispatcherSelectorSpec.config)
+class DispatcherSelectorSpec(config: Config)
+ extends ScalaTestWithActorTestKit(config)
with AnyWordSpecLike
with LogCapturing {
import DispatcherSelectorSpec.PingPong
import DispatcherSelectorSpec.PingPong._
+ def this() = this(DispatcherSelectorSpec.config)
+
"DispatcherSelector" must {
"select dispatcher from config" in {
@@ -58,6 +63,14 @@ class DispatcherSelectorSpec
response.threadName should startWith("DispatcherSelectorSpec-ping-pong-dispatcher")
}
+ "detect unknown dispatcher from config" in {
+ val probe = createTestProbe[Pong]()
+ LoggingTestKit.error("Spawn failed").expect {
+ val ref = spawn(PingPong(), Props.empty.withDispatcherFromConfig("unknown"))
+ probe.expectTerminated(ref)
+ }
+ }
+
"select same dispatcher as parent" in {
val parent = spawn(SpawnProtocol(), Props.empty.withDispatcherFromConfig("ping-pong-dispatcher"))
val childProbe = createTestProbe[ActorRef[Ping]]()
diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/GracefulStopSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/GracefulStopSpec.scala
index 60a88e8763..d63dd6271a 100644
--- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/GracefulStopSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/GracefulStopSpec.scala
@@ -5,12 +5,13 @@
package akka.actor.typed
package scaladsl
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.Done
import akka.NotUsed
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
-import akka.actor.testkit.typed.scaladsl.LogCapturing
-import org.scalatest.wordspec.AnyWordSpecLike
final class GracefulStopSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing {
diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/LoggerOpsSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/LoggerOpsSpec.scala
index 21a46adb97..b664fcc120 100644
--- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/LoggerOpsSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/LoggerOpsSpec.scala
@@ -4,11 +4,12 @@
package akka.actor.typed.scaladsl
+import org.scalatest.wordspec.AnyWordSpecLike
+import org.slf4j.LoggerFactory
+
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.LoggingTestKit
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
-import akka.actor.testkit.typed.scaladsl.LogCapturing
-import org.slf4j.LoggerFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object LoggerOpsSpec {
case class Value1(i: Int)
diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/MessageAdapterSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/MessageAdapterSpec.scala
index 2abd1e3a94..d6829e2a2e 100644
--- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/MessageAdapterSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/MessageAdapterSpec.scala
@@ -4,20 +4,19 @@
package akka.actor.typed.scaladsl
-import akka.actor.UnhandledMessage
-import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.TestException
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.LoggingTestKit
+import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
+import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
import akka.actor.typed.PostStop
import akka.actor.typed.Props
-import akka.actor.testkit.typed.scaladsl.TestProbe
-import akka.actor.testkit.typed.scaladsl.LogCapturing
-import akka.actor.typed.eventstream.EventStream
-import com.typesafe.config.ConfigFactory
-import org.slf4j.event.Level
-import org.scalatest.wordspec.AnyWordSpecLike
+import akka.actor.typed.internal.AdaptMessage
object MessageAdapterSpec {
val config = ConfigFactory.parseString("""
@@ -151,8 +150,7 @@ class MessageAdapterSpec
Behaviors.same
})
- val unhandledProbe = createTestProbe[UnhandledMessage]()
- system.eventStream ! EventStream.Subscribe(unhandledProbe.ref)
+ val unhandledProbe = createUnhandledMessageProbe()
val probe = TestProbe[Wrapped]()
val snitch = Behaviors.setup[Wrapped] { context =>
@@ -273,13 +271,15 @@ class MessageAdapterSpec
}
- "log wrapped message of DeadLetter" in {
+ "redirect to DeadLetter after termination" in {
case class Ping(sender: ActorRef[Pong])
case class Pong(greeting: String)
case class PingReply(response: Pong)
val pingProbe = createTestProbe[Ping]()
+ val deadLetterProbe = testKit.createDeadLetterProbe()
+
val snitch = Behaviors.setup[PingReply] { context =>
val replyTo = context.messageAdapter[Pong](PingReply)
pingProbe.ref ! Ping(replyTo)
@@ -289,13 +289,13 @@ class MessageAdapterSpec
createTestProbe().expectTerminated(ref)
- LoggingTestKit.empty
- .withLogLevel(Level.INFO)
- .withMessageRegex("Pong.*wrapped in.*AdaptMessage.*dead letters encountered")
- .expect {
- pingProbe.receiveMessage().sender ! Pong("hi")
- }
-
+ pingProbe.receiveMessage().sender ! Pong("hi")
+ val deadLetter = deadLetterProbe.receiveMessage()
+ deadLetter.message match {
+ case AdaptMessage(Pong("hi"), _) => // passed through the FunctionRef
+ case Pong("hi") => // FunctionRef stopped
+ case unexpected => fail(s"Unexpected message [$unexpected], expected Pong or AdaptMessage(Pong)")
+ }
}
}
diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/OnSignalSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/OnSignalSpec.scala
index 58f1e8bbf4..f17b26993f 100644
--- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/OnSignalSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/OnSignalSpec.scala
@@ -5,12 +5,13 @@
package akka.actor.typed
package scaladsl
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.Done
+import akka.actor.testkit.typed.scaladsl.ActorTestKit
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
-import akka.actor.testkit.typed.scaladsl.LogCapturing
-import org.scalatest.wordspec.AnyWordSpecLike
-import akka.actor.testkit.typed.scaladsl.ActorTestKit
final class OnSignalSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing {
diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/ReceivePartialSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/ReceivePartialSpec.scala
index cb4fe4b9ec..5ac7e970b8 100644
--- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/ReceivePartialSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/ReceivePartialSpec.scala
@@ -5,14 +5,17 @@
package akka.actor.typed
package scaladsl
+import scala.concurrent.ExecutionContextExecutor
+
+import org.scalatest.wordspec.AnyWordSpecLike
+
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
-import akka.actor.testkit.typed.scaladsl.LogCapturing
-import org.scalatest.wordspec.AnyWordSpecLike
class ReceivePartialSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing {
- implicit val ec = system.executionContext
+ implicit val ec: ExecutionContextExecutor = system.executionContext
"An immutable partial" must {
"correctly install the receiveMessage handler" in {
diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/RoutersSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/RoutersSpec.scala
index ca8c657f74..6ce527e426 100644
--- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/RoutersSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/RoutersSpec.scala
@@ -5,16 +5,21 @@
package akka.actor.typed.scaladsl
import java.util.concurrent.atomic.AtomicInteger
-import akka.actor.testkit.typed.scaladsl.{ LogCapturing, LoggingTestKit, ScalaTestWithActorTestKit, TestProbe }
-import akka.actor.typed.eventstream.EventStream
-import akka.actor.typed.internal.routing.{ GroupRouterImpl, RoutingLogics }
-import akka.actor.typed.receptionist.{ Receptionist, ServiceKey }
-import akka.actor.typed.scaladsl.adapter._
-import akka.actor.typed.{ ActorRef, Behavior }
-import akka.actor.{ ActorSystem, Dropped }
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
+import akka.actor.ActorSystem
+import akka.actor.testkit.typed.scaladsl.LogCapturing
+import akka.actor.testkit.typed.scaladsl.LoggingTestKit
+import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
+import akka.actor.typed.ActorRef
+import akka.actor.typed.Behavior
+import akka.actor.typed.internal.routing.GroupRouterImpl
+import akka.actor.typed.internal.routing.RoutingLogics
+import akka.actor.typed.receptionist.Receptionist
+import akka.actor.typed.receptionist.ServiceKey
+import akka.actor.typed.scaladsl.adapter._
+
class RoutersSpec extends ScalaTestWithActorTestKit("""
akka.loglevel=debug
""") with AnyWordSpecLike with Matchers with LogCapturing {
@@ -144,13 +149,12 @@ class RoutersSpec extends ScalaTestWithActorTestKit("""
"publish Dropped messages when there are no routees available" in {
val serviceKey = ServiceKey[String]("group-routing-2")
val group = spawn(Routers.group(serviceKey), "group-router-2")
- val probe = TestProbe[Dropped]()
- system.eventStream ! EventStream.Subscribe(probe.ref)
+ val probe = createDroppedMessageProbe()
(0 to 3).foreach { n =>
val msg = s"message-$n"
group ! msg
- probe.expectMessageType[Dropped]
+ probe.receiveMessage()
}
testKit.stop(group)
@@ -207,8 +211,8 @@ class RoutersSpec extends ScalaTestWithActorTestKit("""
val router = spawn(Behaviors.setup[String](context =>
new GroupRouterImpl(context, serviceKey, false, new RoutingLogics.RoundRobinLogic[String], true)))
- val reachableProbe = createTestProbe[String]
- val unreachableProbe = createTestProbe[String]
+ val reachableProbe = createTestProbe[String]()
+ val unreachableProbe = createTestProbe[String]()
router
.unsafeUpcast[Any] ! Receptionist.Listing(serviceKey, Set(reachableProbe.ref), Set(unreachableProbe.ref), false)
router ! "one"
@@ -222,7 +226,7 @@ class RoutersSpec extends ScalaTestWithActorTestKit("""
val router = spawn(Behaviors.setup[String](context =>
new GroupRouterImpl(context, serviceKey, false, new RoutingLogics.RoundRobinLogic[String], true)))
- val unreachableProbe = createTestProbe[String]
+ val unreachableProbe = createTestProbe[String]()
router.unsafeUpcast[Any] ! Receptionist.Listing(
serviceKey,
Set.empty[ActorRef[String]],
diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/StashBufferSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/StashBufferSpec.scala
index 484b1ae7e2..bc56511023 100644
--- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/StashBufferSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/StashBufferSpec.scala
@@ -4,13 +4,14 @@
package akka.actor.typed.scaladsl
-import akka.actor.typed.Behavior
-import akka.actor.testkit.typed.internal.StubbedActorContext
-import akka.actor.testkit.typed.scaladsl.TestInbox
-import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.testkit.typed.internal.StubbedActorContext
+import akka.actor.testkit.typed.scaladsl.LogCapturing
+import akka.actor.testkit.typed.scaladsl.TestInbox
+import akka.actor.typed.Behavior
+
class StashBufferSpec extends AnyWordSpec with Matchers with LogCapturing {
val context = new StubbedActorContext[String](
@@ -195,11 +196,57 @@ class StashBufferSpec extends AnyWordSpec with Matchers with LogCapturing {
buffer.size should ===(0)
}
+ "clear" in {
+ val buffer = StashBuffer[String](context, 10)
+ buffer.stash("m1")
+ buffer.stash("m2")
+ buffer.clear()
+ buffer.size should ===(0)
+ buffer.stash("m3")
+ buffer.size should ===(1)
+ }
+
+ "be able to clear while unstashing" in {
+ val buffer = StashBuffer[String](context, 10)
+ buffer.stash("m1")
+ buffer.stash("m2")
+ buffer.stash("clear")
+ buffer.stash("m3")
+
+ val valueInbox = TestInbox[String]()
+ def behavior(state: String): Behavior[String] =
+ Behaviors.receive[String] { (_, message) =>
+ if (message == "get") {
+ valueInbox.ref ! state
+ Behaviors.same
+ } else if (message == "clear") {
+ buffer.clear()
+ Behaviors.same
+ } else {
+ behavior(state + message)
+ }
+ }
+
+ val b2 = buffer.unstashAll(behavior(""))
+ buffer.size should ===(0)
+
+ buffer.stash("get")
+ buffer.unstashAll(b2)
+ // clear called before processing m3 so not included
+ valueInbox.expectMessage("m1m2")
+ }
+
"fail quick on invalid start behavior" in {
val stash = StashBuffer[String](context, 10)
stash.stash("one")
intercept[IllegalArgumentException](stash.unstashAll(Behaviors.unhandled))
}
+ "answer thruthfully about its capacity" in {
+ val capacity = 42
+ val stash = StashBuffer[String](context, capacity)
+
+ stash.capacity should ===(capacity)
+ }
}
}
diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/StashSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/StashSpec.scala
index 90edf7cbf6..9b2b7ad359 100644
--- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/StashSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/StashSpec.scala
@@ -8,17 +8,15 @@ package scaladsl
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
-import akka.actor.DeadLetter
import scala.concurrent.duration._
-import akka.actor.UnhandledMessage
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.TestException
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.LoggingTestKit
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
-import akka.actor.testkit.typed.scaladsl.LogCapturing
-import akka.actor.typed.eventstream.EventStream
-import org.scalatest.wordspec.AnyWordSpecLike
object AbstractStashSpec {
sealed trait Command
@@ -321,18 +319,15 @@ class UnstashingSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with
"Unstashing" must {
"work with initial Behaviors.same" in {
- // FIXME #26148 unstashAll doesn't support Behavior.same
- pending
-
val probe = TestProbe[String]()
// unstashing is inside setup
- val ref = spawn(Behaviors.receive[String] {
- case (_, "unstash") =>
+ val ref = spawn(Behaviors.receiveMessage[String] {
+ case "unstash" =>
Behaviors.withStash(10) { stash =>
stash.stash("one")
stash.unstashAll(Behaviors.same)
}
- case (_, msg) =>
+ case msg =>
probe.ref ! msg
Behaviors.same
})
@@ -344,8 +339,8 @@ class UnstashingSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with
"work with intermediate Behaviors.same" in {
val probe = TestProbe[String]()
// unstashing is inside setup
- val ref = spawn(Behaviors.receivePartial[String] {
- case (_, "unstash") =>
+ val ref = spawn(Behaviors.receiveMessagePartial[String] {
+ case "unstash" =>
Behaviors.withStash(10) { stash =>
stash.stash("one")
stash.stash("two")
@@ -364,20 +359,17 @@ class UnstashingSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with
}
"work with supervised initial Behaviors.same" in {
- // FIXME #26148 unstashAll doesn't support Behavior.same
- pending
-
val probe = TestProbe[String]()
// unstashing is inside setup
val ref = spawn(
Behaviors
- .supervise(Behaviors.receivePartial[String] {
- case (_, "unstash") =>
+ .supervise(Behaviors.receiveMessagePartial[String] {
+ case "unstash" =>
Behaviors.withStash(10) { stash =>
stash.stash("one")
stash.unstashAll(Behaviors.same)
}
- case (_, msg) =>
+ case msg =>
probe.ref ! msg
Behaviors.same
})
@@ -394,8 +386,8 @@ class UnstashingSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with
// unstashing is inside setup
val ref = spawn(
Behaviors
- .supervise(Behaviors.receivePartial[String] {
- case (_, "unstash") =>
+ .supervise(Behaviors.receiveMessagePartial[String] {
+ case "unstash" =>
Behaviors.withStash(10) { stash =>
stash.stash("one")
stash.stash("two")
@@ -414,6 +406,47 @@ class UnstashingSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with
probe.expectMessage("three")
}
+ "work with Behaviors.same when switching Behavior while unstashing one" in {
+ val probe = TestProbe[String]()
+
+ val ref = spawn(Behaviors.receiveMessage[String] {
+ case "unstash" =>
+ Behaviors.withStash(10) {
+ stash =>
+ def expectingA: Behaviors.Receive[String] = Behaviors.receiveMessage {
+ case "a" =>
+ probe.ref ! "a"
+ stash.unstash(expectingB, 1, identity)
+ case other =>
+ probe.ref ! s"unexpected [$other] when expecting [a]"
+ Behaviors.stopped
+ }
+
+ def expectingB: Behaviors.Receive[String] = Behaviors.receiveMessage {
+ case b @ ("b1" | "b2") =>
+ probe.ref ! b
+ stash.unstash(Behaviors.same, 1, identity)
+ case other =>
+ probe.ref ! s"unexpected [$other] when expecting [b]"
+ Behaviors.stopped
+ }
+
+ stash.stash("a")
+ stash.stash("b1")
+ stash.stash("b2")
+ stash.unstash(expectingA, 1, identity)
+ }
+ case other =>
+ probe.ref ! s"unexpected [$other] in first Behavior"
+ Behaviors.stopped
+ })
+
+ ref ! "unstash"
+ probe.expectMessage("a")
+ probe.expectMessage("b1")
+ probe.expectMessage("b2")
+ }
+
def testPostStop(probe: TestProbe[String], ref: ActorRef[String]): Unit = {
ref ! "stash"
ref ! "stash"
@@ -592,8 +625,7 @@ class UnstashingSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with
"deal with unhandled the same way as normal unhandled" in {
val probe = TestProbe[String]()
- val unhandledProbe = createTestProbe[UnhandledMessage]()
- system.eventStream ! EventStream.Subscribe(unhandledProbe.ref)
+ val unhandledProbe = createUnhandledMessageProbe()
val ref = spawn(Behaviors.withStash[String](10) { stash =>
stash.stash("unhandled")
@@ -629,7 +661,7 @@ class UnstashingSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with
}
"deal with initial stop" in {
- val probe = TestProbe[Any]
+ val probe = TestProbe[Any]()
val ref = spawn(Behaviors.withStash[String](10) { stash =>
stash.stash("one")
@@ -644,8 +676,8 @@ class UnstashingSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with
}
"deal with stop" in {
- val probe = TestProbe[Any]
- system.eventStream ! EventStream.Subscribe(probe.ref.narrow[DeadLetter])
+ val probe = TestProbe[Any]()
+ val deadLetterProbe = createDeadLetterProbe()
val ref = spawn(Behaviors.withStash[String](10) { stash =>
stash.stash("one")
@@ -663,12 +695,12 @@ class UnstashingSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with
})
ref ! "unstash"
probe.expectMessage("one")
- probe.expectMessageType[DeadLetter].message should equal("two")
+ deadLetterProbe.receiveMessage().message should equal("two")
probe.expectTerminated(ref)
}
"work with initial same" in {
- val probe = TestProbe[Any]
+ val probe = TestProbe[Any]()
val ref = spawn(Behaviors.withStash[String](10) { stash =>
stash.stash("one")
stash.stash("two")
diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/StopSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/StopSpec.scala
index 26533a1e78..c5cb2df912 100644
--- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/StopSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/StopSpec.scala
@@ -4,15 +4,16 @@
package akka.actor.typed.scaladsl
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.Done
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
-import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed
import akka.actor.typed.Behavior
import akka.actor.typed.BehaviorInterceptor
import akka.actor.typed.PostStop
-import org.scalatest.wordspec.AnyWordSpecLike
class StopSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing {
import BehaviorInterceptor._
diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/adapter/AdapterSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/adapter/AdapterSpec.scala
index 0064d8760d..2f79e69b43 100644
--- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/adapter/AdapterSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/adapter/AdapterSpec.scala
@@ -5,25 +5,27 @@
package akka.actor.typed.scaladsl.adapter
import scala.util.control.NoStackTrace
+
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
+import akka.{ actor => classic }
+import akka.Done
+import akka.NotUsed
+import akka.actor.ActorInitializationException
import akka.actor.InvalidMessageException
import akka.actor.testkit.typed.TestException
-import akka.actor.typed.scaladsl.Behaviors
+import akka.actor.testkit.typed.scaladsl.LogCapturing
+import akka.actor.testkit.typed.scaladsl.LoggingTestKit
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.Terminated
-import akka.testkit._
-import akka.Done
-import akka.NotUsed
-import akka.actor.ActorInitializationException
-import akka.actor.testkit.typed.scaladsl.LogCapturing
-import akka.actor.testkit.typed.scaladsl.LoggingTestKit
import akka.actor.typed.internal.adapter.SchedulerAdapter
+import akka.actor.typed.scaladsl.Behaviors
import akka.serialization.SerializationExtension
-import akka.{ actor => classic }
-import org.scalatest.BeforeAndAfterAll
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
+import akka.testkit._
object AdapterSpec {
val classic1: classic.Props = classic.Props(new Classic1)
@@ -184,7 +186,7 @@ object AdapterSpec {
class AdapterSpec extends AnyWordSpec with Matchers with BeforeAndAfterAll with LogCapturing {
import AdapterSpec._
- implicit val system = akka.actor.ActorSystem("AdapterSpec")
+ implicit val system: classic.ActorSystem = akka.actor.ActorSystem("AdapterSpec")
def typedSystem: ActorSystem[Nothing] = system.toTyped
"ActorSystem adaption" must {
diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/adapter/GuardianStartupSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/adapter/GuardianStartupSpec.scala
index 31b442f6ad..d925085896 100644
--- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/adapter/GuardianStartupSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/adapter/GuardianStartupSpec.scala
@@ -7,14 +7,15 @@ package akka.actor.typed.scaladsl.adapter
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
+import org.scalatest.concurrent.ScalaFutures
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.actor.ActorSystemImpl
import akka.actor.testkit.typed.scaladsl.ActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.ActorSystem
import akka.actor.typed.scaladsl.Behaviors
-import org.scalatest.concurrent.ScalaFutures
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
class GuardianStartupSpec extends AnyWordSpec with Matchers with ScalaFutures with LogCapturing {
diff --git a/akka-actor-typed-tests/src/test/scala/docs/akka/typed/GracefulStopDocSpec.scala b/akka-actor-typed-tests/src/test/scala/docs/akka/typed/GracefulStopDocSpec.scala
index ea8f8c9f10..a18d6750d9 100644
--- a/akka-actor-typed-tests/src/test/scala/docs/akka/typed/GracefulStopDocSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/docs/akka/typed/GracefulStopDocSpec.scala
@@ -28,7 +28,7 @@ object GracefulStopDocSpec {
object MasterControlProgram {
sealed trait Command
final case class SpawnJob(name: String) extends Command
- final case object GracefulShutdown extends Command
+ case object GracefulShutdown extends Command
// Predefined cleanup operation
def cleanup(log: Logger): Unit = log.info("Cleaning up!")
diff --git a/akka-actor-typed-tests/src/test/scala/docs/akka/typed/InteractionPatternsSpec.scala b/akka-actor-typed-tests/src/test/scala/docs/akka/typed/InteractionPatternsSpec.scala
index 7cc87c4bcc..2472d7f85b 100644
--- a/akka-actor-typed-tests/src/test/scala/docs/akka/typed/InteractionPatternsSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/docs/akka/typed/InteractionPatternsSpec.scala
@@ -418,6 +418,8 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with AnyWordSpec
// keep this out of the sample as it uses the testkit spawn
val cookieFabric = spawn(CookieFabric())
+ val theSystem = testKit.system
+
// #standalone-ask
import akka.actor.typed.scaladsl.AskPattern._
@@ -426,6 +428,8 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with AnyWordSpec
// asking someone requires a timeout if the timeout hits without response
// the ask is failed with a TimeoutException
implicit val timeout: Timeout = 3.seconds
+ // implicit ActorSystem in scope
+ implicit val system: ActorSystem[_] = theSystem
val result: Future[CookieFabric.Reply] = cookieFabric.ask(ref => CookieFabric.GiveMeCookies(3, ref))
diff --git a/akka-actor-typed-tests/src/test/scala/docs/akka/typed/StashDocSpec.scala b/akka-actor-typed-tests/src/test/scala/docs/akka/typed/StashDocSpec.scala
index f64ce71f22..3c627ec6cc 100644
--- a/akka-actor-typed-tests/src/test/scala/docs/akka/typed/StashDocSpec.scala
+++ b/akka-actor-typed-tests/src/test/scala/docs/akka/typed/StashDocSpec.scala
@@ -31,7 +31,7 @@ object StashDocSpec {
final case class Save(value: String, replyTo: ActorRef[Done]) extends Command
final case class Get(replyTo: ActorRef[String]) extends Command
private final case class InitialState(value: String) extends Command
- private final case object SaveSuccess extends Command
+ private case object SaveSuccess extends Command
private final case class DBError(cause: Throwable) extends Command
def apply(id: String, db: DB): Behavior[Command] = {
diff --git a/akka-actor-typed-tests/src/test/scala/docs/akka/typed/StyleGuideDocExamples.scala b/akka-actor-typed-tests/src/test/scala/docs/akka/typed/StyleGuideDocExamples.scala
index 3ceb783724..591155a15f 100644
--- a/akka-actor-typed-tests/src/test/scala/docs/akka/typed/StyleGuideDocExamples.scala
+++ b/akka-actor-typed-tests/src/test/scala/docs/akka/typed/StyleGuideDocExamples.scala
@@ -428,7 +428,7 @@ object StyleGuideDocExamples {
import akka.actor.typed.scaladsl.AskPattern._
import akka.util.Timeout
- implicit val timeout = Timeout(3.seconds)
+ implicit val timeout: Timeout = Timeout(3.seconds)
val counter: ActorRef[Command] = ???
val result: Future[OperationResult] = counter.ask(replyTo => Increment(delta = 2, replyTo))
diff --git a/akka-actor-typed/src/main/mima-filters/2.6.4.backwards.excludes/issue-28747-ProducerController.excludes b/akka-actor-typed/src/main/mima-filters/2.6.4.backwards.excludes/issue-28747-ProducerController.excludes
new file mode 100644
index 0000000000..7483dbd36a
--- /dev/null
+++ b/akka-actor-typed/src/main/mima-filters/2.6.4.backwards.excludes/issue-28747-ProducerController.excludes
@@ -0,0 +1,2 @@
+# #28747 Change of private ProducerController#Settings constructor (also ApiMayChange)
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.typed.delivery.ProducerController#Settings.this")
diff --git a/akka-actor-typed/src/main/mima-filters/2.6.4.backwards.excludes/issue-28786-stash-clear.excludes b/akka-actor-typed/src/main/mima-filters/2.6.4.backwards.excludes/issue-28786-stash-clear.excludes
new file mode 100644
index 0000000000..136e6687e1
--- /dev/null
+++ b/akka-actor-typed/src/main/mima-filters/2.6.4.backwards.excludes/issue-28786-stash-clear.excludes
@@ -0,0 +1,3 @@
+# #28786 add clear to StashBuffer, ok because of DoNotInherit
+ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.typed.javadsl.StashBuffer.clear")
+ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.typed.scaladsl.StashBuffer.clear")
diff --git a/akka-actor-typed/src/main/mima-filters/2.6.4.backwards.excludes/pr-28739-expose-stashbuffer-capacity.excludes b/akka-actor-typed/src/main/mima-filters/2.6.4.backwards.excludes/pr-28739-expose-stashbuffer-capacity.excludes
new file mode 100644
index 0000000000..9fe23b3edc
--- /dev/null
+++ b/akka-actor-typed/src/main/mima-filters/2.6.4.backwards.excludes/pr-28739-expose-stashbuffer-capacity.excludes
@@ -0,0 +1,4 @@
+# https://github.com/akka/akka/pull/28739#issuecomment-602527515
+# StashBuffer trait is not meant to be extended by external users, and is marked as such with a @DoNotInherit
+ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.typed.javadsl.StashBuffer.capacity")
+ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.typed.scaladsl.StashBuffer.capacity")
\ No newline at end of file
diff --git a/akka-actor-typed/src/main/mima-filters/2.6.4.backwards.excludes/reliable-delivery.excludes b/akka-actor-typed/src/main/mima-filters/2.6.4.backwards.excludes/reliable-delivery.excludes
index e59db0fb17..07a92c4b2e 100644
--- a/akka-actor-typed/src/main/mima-filters/2.6.4.backwards.excludes/reliable-delivery.excludes
+++ b/akka-actor-typed/src/main/mima-filters/2.6.4.backwards.excludes/reliable-delivery.excludes
@@ -1,2 +1,9 @@
# Changes to internals of reliable delivery
ProblemFilters.exclude[Problem]("akka.actor.typed.delivery.internal.*")
+
+# #28720 Dynamically adjust the ConsumerController's resend interval
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.typed.delivery.ConsumerController#Settings.resendInterval")
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.typed.delivery.ConsumerController#Settings.withResendInterval")
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.typed.delivery.ConsumerController#Settings.withResendInterval")
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.typed.delivery.ConsumerController#Settings.getResendInterval")
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.typed.delivery.ConsumerController#Settings.this")
diff --git a/akka-actor-typed/src/main/mima-filters/2.6.5.backwards.excludes/current-actor-thread.excludes b/akka-actor-typed/src/main/mima-filters/2.6.5.backwards.excludes/current-actor-thread.excludes
new file mode 100644
index 0000000000..f4eae2851d
--- /dev/null
+++ b/akka-actor-typed/src/main/mima-filters/2.6.5.backwards.excludes/current-actor-thread.excludes
@@ -0,0 +1,7 @@
+# add internal currentActorThread to ActorContext
+ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.typed.scaladsl.ActorContext.setCurrentActorThread")
+ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.typed.scaladsl.ActorContext.clearCurrentActorThread")
+ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.typed.scaladsl.ActorContext.checkCurrentActorThread")
+ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.typed.internal.ActorContextImpl.akka$actor$typed$internal$ActorContextImpl$$_currentActorThread_=")
+ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.typed.internal.ActorContextImpl.akka$actor$typed$internal$ActorContextImpl$$_currentActorThread")
+
diff --git a/akka-actor-typed/src/main/mima-filters/2.6.x.backwards.excludes/jfr.excludes b/akka-actor-typed/src/main/mima-filters/2.6.x.backwards.excludes/jfr.excludes
new file mode 100644
index 0000000000..e210e56a38
--- /dev/null
+++ b/akka-actor-typed/src/main/mima-filters/2.6.x.backwards.excludes/jfr.excludes
@@ -0,0 +1,2 @@
+# JDK9+ classes for JFR
+ProblemFilters.exclude[Problem]("akka.actor.typed.internal.jfr.*")
diff --git a/akka-actor-typed/src/main/resources/reference.conf b/akka-actor-typed/src/main/resources/reference.conf
index a3476814a3..3cb166b939 100644
--- a/akka-actor-typed/src/main/resources/reference.conf
+++ b/akka-actor-typed/src/main/resources/reference.conf
@@ -16,7 +16,7 @@ akka.actor.typed {
library-extensions = ${?akka.actor.typed.library-extensions} []
# Receptionist is started eagerly to allow clustered receptionist to gather remote registrations early on.
- library-extensions += "akka.actor.typed.receptionist.Receptionist"
+ library-extensions += "akka.actor.typed.receptionist.Receptionist$"
# While an actor is restarted (waiting for backoff to expire and children to stop)
# incoming messages and signals are stashed, and delivered later to the newly restarted
@@ -76,6 +76,10 @@ akka.reliable-delivery {
# The ProducerController retries requests to the durable queue this
# number of times before failing.
retry-attempts = 10
+
+ # The ProducerController retries sending the first message with this interval
+ # until it has been confirmed.
+ resend-first-interval = 1s
}
}
@@ -86,8 +90,10 @@ akka.reliable-delivery {
flow-control-window = 50
# The ConsumerController resends flow control messages to the
- # ProducerController with this interval.
- resend-interval = 1s
+ # ProducerController with the resend-interval-min, and increasing
+ # it gradually to resend-interval-max when idle.
+ resend-interval-min = 2s
+ resend-interval-max = 30s
# If this is enabled lost messages will not be resent, but flow control is used.
# This can be more efficient since messages don't have to be
diff --git a/akka-actor-typed/src/main/scala-jdk-9/akka/actor/typed/internal/jfr/Events.scala b/akka-actor-typed/src/main/scala-jdk-9/akka/actor/typed/internal/jfr/Events.scala
index c6545507dd..625b3657f9 100644
--- a/akka-actor-typed/src/main/scala-jdk-9/akka/actor/typed/internal/jfr/Events.scala
+++ b/akka-actor-typed/src/main/scala-jdk-9/akka/actor/typed/internal/jfr/Events.scala
@@ -4,13 +4,14 @@
package akka.actor.typed.internal.jfr
-import akka.annotation.InternalApi
import jdk.jfr.Category
import jdk.jfr.Enabled
import jdk.jfr.Event
import jdk.jfr.Label
import jdk.jfr.StackTrace
+import akka.annotation.InternalApi
+
// requires jdk9+ to compile
// for editing these in IntelliJ, open module settings, change JDK dependency to 11 for only this module
@@ -85,7 +86,8 @@ final class DeliveryProducerReceived(val producerId: String, val currentSeqNr: L
@Enabled(true)
@StackTrace(false)
@Category(Array("Akka", "Delivery", "ProducerController")) @Label("Delivery ProducerController received demand request")
-final class DeliveryProducerReceivedRequest(val producerId: String, val requestedSeqNr: Long) extends Event
+final class DeliveryProducerReceivedRequest(val producerId: String, val requestedSeqNr: Long, confirmedSeqNr: Long)
+ extends Event
/** INTERNAL API */
@InternalApi
@@ -129,14 +131,14 @@ final class DeliveryConsumerReceivedPreviousInProgress(val producerId: String, v
@Enabled(true)
@StackTrace(false)
@Category(Array("Akka", "Delivery", "ConsumerController")) @Label("Delivery ConsumerController received duplicate")
-final class DeliveryConsumerDuplicate(val pid: String, val expectedSeqNr: Long, val seqNr: Long) extends Event
+final class DeliveryConsumerDuplicate(val producerId: String, val expectedSeqNr: Long, val seqNr: Long) extends Event
/** INTERNAL API */
@InternalApi
@Enabled(true)
@StackTrace(false)
@Category(Array("Akka", "Delivery", "ConsumerController")) @Label("Delivery ConsumerController received missing")
-final class DeliveryConsumerMissing(val pid: String, val expectedSeqNr: Long, val seqNr: Long) extends Event
+final class DeliveryConsumerMissing(val producerId: String, val expectedSeqNr: Long, val seqNr: Long) extends Event
/** INTERNAL API */
@InternalApi
@@ -159,3 +161,10 @@ final class DeliveryConsumerSentRequest(val producerId: String, val requestedSeq
@StackTrace(false)
@Category(Array("Akka", "Delivery", "ConsumerController")) @Label("Delivery ConsumerController producer changed")
final class DeliveryConsumerChangedProducer(val producerId: String) extends Event
+
+/** INTERNAL API */
+@InternalApi
+@Enabled(true)
+@StackTrace(false)
+@Category(Array("Akka", "Delivery", "ConsumerController")) @Label("Delivery ConsumerController stash is full")
+final class DeliveryConsumerStashFull(val producerId: String, val seqNr: Long) extends Event
diff --git a/akka-actor-typed/src/main/scala-jdk-9/akka/actor/typed/internal/jfr/JFRActorFlightRecorder.scala b/akka-actor-typed/src/main/scala-jdk-9/akka/actor/typed/internal/jfr/JFRActorFlightRecorder.scala
index deb9244ee3..8f90d0eea7 100644
--- a/akka-actor-typed/src/main/scala-jdk-9/akka/actor/typed/internal/jfr/JFRActorFlightRecorder.scala
+++ b/akka-actor-typed/src/main/scala-jdk-9/akka/actor/typed/internal/jfr/JFRActorFlightRecorder.scala
@@ -41,8 +41,8 @@ private[akka] final class JFRActorFlightRecorder(val system: ActorSystem[_]) ext
new DeliveryProducerResentFirstUnconfirmed(producerId, seqNr).commit()
override def producerReceived(producerId: String, currentSeqNr: Long): Unit =
new DeliveryProducerReceived(producerId, currentSeqNr).commit()
- override def producerReceivedRequest(producerId: String, requestedSeqNr: Long): Unit =
- new DeliveryProducerReceivedRequest(producerId, requestedSeqNr).commit()
+ override def producerReceivedRequest(producerId: String, requestedSeqNr: Long, confirmedSeqNr: Long): Unit =
+ new DeliveryProducerReceivedRequest(producerId, requestedSeqNr, confirmedSeqNr).commit()
override def producerReceivedResend(producerId: String, fromSeqNr: Long): Unit =
new DeliveryProducerReceivedResend(producerId, fromSeqNr).commit()
@@ -54,15 +54,17 @@ private[akka] final class JFRActorFlightRecorder(val system: ActorSystem[_]) ext
new DeliveryConsumerReceived(producerId, seqNr).commit()
override def consumerReceivedPreviousInProgress(producerId: String, seqNr: Long, stashed: Int): Unit =
new DeliveryConsumerReceivedPreviousInProgress(producerId, seqNr: Long, stashed).commit()
- override def consumerDuplicate(pid: String, expectedSeqNr: Long, seqNr: Long): Unit =
- new DeliveryConsumerDuplicate(pid, expectedSeqNr, seqNr).commit()
- override def consumerMissing(pid: String, expectedSeqNr: Long, seqNr: Long): Unit =
- new DeliveryConsumerMissing(pid, expectedSeqNr, seqNr).commit()
+ override def consumerDuplicate(producerId: String, expectedSeqNr: Long, seqNr: Long): Unit =
+ new DeliveryConsumerDuplicate(producerId, expectedSeqNr, seqNr).commit()
+ override def consumerMissing(producerId: String, expectedSeqNr: Long, seqNr: Long): Unit =
+ new DeliveryConsumerMissing(producerId, expectedSeqNr, seqNr).commit()
override def consumerReceivedResend(seqNr: Long): Unit =
new DeliveryConsumerReceivedResend(seqNr).commit()
override def consumerSentRequest(producerId: String, requestedSeqNr: Long): Unit =
new DeliveryConsumerSentRequest(producerId, requestedSeqNr).commit()
override def consumerChangedProducer(producerId: String): Unit =
new DeliveryConsumerChangedProducer(producerId).commit()
+ override def consumerStashFull(producerId: String, seqNr: Long): Unit =
+ new DeliveryConsumerStashFull(producerId, seqNr).commit()
}
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/ActorRef.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/ActorRef.scala
index cfefcea1eb..e5a10553cb 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/ActorRef.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/ActorRef.scala
@@ -4,10 +4,11 @@
package akka.actor.typed
-import akka.annotation.DoNotInherit
-import akka.{ actor => classic }
import scala.annotation.unchecked.uncheckedVariance
+
+import akka.{ actor => classic }
import akka.actor.typed.internal.InternalRecipientRef
+import akka.annotation.DoNotInherit
/**
* An ActorRef is the identity or address of an Actor instance. It is valid
@@ -76,8 +77,8 @@ private[akka] object SerializedActorRef {
}
def toAddress[T](actorRef: ActorRef[T]) = {
- import akka.serialization.JavaSerializer.currentSystem
import akka.actor.typed.scaladsl.adapter._
+ import akka.serialization.JavaSerializer.currentSystem
val resolver = ActorRefResolver(currentSystem.value.toTyped)
resolver.toSerializationFormat(actorRef)
}
@@ -89,8 +90,8 @@ private[akka] object SerializedActorRef {
*/
@SerialVersionUID(1L)
private[akka] final case class SerializedActorRef[T] private (address: String) {
- import akka.serialization.JavaSerializer.currentSystem
import akka.actor.typed.scaladsl.adapter._
+ import akka.serialization.JavaSerializer.currentSystem
def this(actorRef: ActorRef[T]) =
this(SerializedActorRef.toAddress(actorRef))
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/ActorSystem.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/ActorSystem.scala
index ad57858958..8e1570ae94 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/ActorSystem.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/ActorSystem.scala
@@ -6,19 +6,20 @@ package akka.actor.typed
import java.util.concurrent.{ CompletionStage, ThreadFactory }
-import akka.actor.setup.ActorSystemSetup
-import akka.actor.typed.eventstream.EventStream
-import akka.actor.typed.internal.adapter.{ ActorSystemAdapter, GuardianStartupBehavior, PropsAdapter }
-import akka.actor.typed.internal.{ EventStreamExtension, InternalRecipientRef }
-import akka.actor.typed.receptionist.Receptionist
-import akka.actor.{ Address, BootstrapSetup, ClassicActorSystemProvider }
-import akka.annotation.DoNotInherit
-import akka.util.Helpers.Requiring
-import akka.{ Done, actor => classic }
+import scala.concurrent.{ ExecutionContextExecutor, Future }
+
import com.typesafe.config.{ Config, ConfigFactory }
import org.slf4j.Logger
-import scala.concurrent.{ ExecutionContextExecutor, Future }
+import akka.{ Done, actor => classic }
+import akka.actor.{ Address, BootstrapSetup, ClassicActorSystemProvider }
+import akka.actor.setup.ActorSystemSetup
+import akka.actor.typed.eventstream.EventStream
+import akka.actor.typed.internal.{ EventStreamExtension, InternalRecipientRef }
+import akka.actor.typed.internal.adapter.{ ActorSystemAdapter, GuardianStartupBehavior, PropsAdapter }
+import akka.actor.typed.receptionist.Receptionist
+import akka.annotation.DoNotInherit
+import akka.util.Helpers.Requiring
/**
* An ActorSystem is home to a hierarchy of Actors. It is created using
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/Behavior.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/Behavior.scala
index 0754296668..30f8da4937 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/Behavior.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/Behavior.scala
@@ -4,6 +4,10 @@
package akka.actor.typed
+import scala.annotation.switch
+import scala.annotation.tailrec
+import scala.reflect.ClassTag
+
import akka.actor.InvalidMessageException
import akka.actor.typed.internal.BehaviorImpl
import akka.actor.typed.internal.BehaviorImpl.DeferredBehavior
@@ -13,10 +17,6 @@ import akka.actor.typed.internal.InterceptorImpl
import akka.annotation.DoNotInherit
import akka.annotation.InternalApi
-import scala.annotation.switch
-import scala.annotation.tailrec
-import scala.reflect.ClassTag
-
/**
* The behavior of an actor defines how it reacts to the messages that it
* receives. The message may either be of the type that the Actor declares
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/Dispatchers.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/Dispatchers.scala
index c5d4f99b06..c0c4c55cc2 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/Dispatchers.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/Dispatchers.scala
@@ -4,10 +4,10 @@
package akka.actor.typed
-import akka.annotation.InternalApi
-
import scala.concurrent.ExecutionContextExecutor
+import akka.annotation.InternalApi
+
object Dispatchers {
/**
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/Extensions.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/Extensions.scala
index a07a054b73..624a362679 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/Extensions.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/Extensions.scala
@@ -4,8 +4,8 @@
package akka.actor.typed
-import akka.annotation.DoNotInherit
import akka.actor.setup.Setup
+import akka.annotation.DoNotInherit
/**
* Marker trait/interface for extensions. An extension can be registered in the ActorSystem and is guaranteed to only
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/LogOptions.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/LogOptions.scala
index 02ab47bfb9..eb70c31d01 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/LogOptions.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/LogOptions.scala
@@ -6,10 +6,11 @@ package akka.actor.typed
import java.util.Optional
-import akka.annotation.{ DoNotInherit, InternalApi }
import org.slf4j.Logger
import org.slf4j.event.Level
+import akka.annotation.{ DoNotInherit, InternalApi }
+
/**
* Logging options when using `Behaviors.logMessages`.
*/
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/Props.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/Props.scala
index dcac621bd0..3172348d15 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/Props.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/Props.scala
@@ -4,16 +4,15 @@
package akka.actor.typed
+import scala.annotation.tailrec
+import scala.annotation.varargs
+import scala.reflect.ClassTag
+
+import akka.actor.typed.internal.PropsImpl._
import akka.annotation.DoNotInherit
import akka.annotation.InternalApi
-
-import scala.annotation.tailrec
-import scala.reflect.ClassTag
-import akka.actor.typed.internal.PropsImpl._
import akka.util.ccompat.JavaConverters._
-import scala.annotation.varargs
-
object Props {
/**
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/Scheduler.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/Scheduler.scala
index 0f235be803..1b2a9a746e 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/Scheduler.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/Scheduler.scala
@@ -4,12 +4,12 @@
package akka.actor.typed
-import akka.actor.Cancellable
-import akka.annotation.DoNotInherit
-
import scala.concurrent.ExecutionContext
import scala.concurrent.duration.FiniteDuration
+import akka.actor.Cancellable
+import akka.annotation.DoNotInherit
+
/**
* The ActorSystem facility for scheduling tasks.
*
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/SpawnProtocol.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/SpawnProtocol.scala
index c25c8eee5c..7feffeeaac 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/SpawnProtocol.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/SpawnProtocol.scala
@@ -5,6 +5,7 @@
package akka.actor.typed
import scala.annotation.tailrec
+
import akka.actor.typed.scaladsl.Behaviors
import akka.annotation.DoNotInherit
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/SupervisorStrategy.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/SupervisorStrategy.scala
index b84c6a033e..ed9c46faad 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/SupervisorStrategy.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/SupervisorStrategy.scala
@@ -4,13 +4,14 @@
package akka.actor.typed
-import akka.annotation.InternalApi
-
-import scala.concurrent.duration.FiniteDuration
import scala.concurrent.duration.Duration
-import akka.util.JavaDurationConverters._
+import scala.concurrent.duration.FiniteDuration
+
import org.slf4j.event.Level
+import akka.annotation.InternalApi
+import akka.util.JavaDurationConverters._
+
object SupervisorStrategy {
/**
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/ConsumerController.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/ConsumerController.scala
index 58a0230b66..7df92fe216 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/ConsumerController.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/ConsumerController.scala
@@ -8,6 +8,8 @@ import java.time.{ Duration => JavaDuration }
import scala.concurrent.duration._
+import com.typesafe.config.Config
+
import akka.actor.DeadLetterSuppression
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
@@ -21,7 +23,6 @@ import akka.annotation.ApiMayChange
import akka.annotation.DoNotInherit
import akka.annotation.InternalApi
import akka.util.JavaDurationConverters._
-import com.typesafe.config.Config
/**
* `ConsumerController` and [[ProducerController]] or [[WorkPullingProducerController]] are used
@@ -161,7 +162,8 @@ object ConsumerController {
def apply(config: Config): Settings = {
new Settings(
flowControlWindow = config.getInt("flow-control-window"),
- resendInterval = config.getDuration("resend-interval").asScala,
+ resendIntervalMin = config.getDuration("resend-interval-min").asScala,
+ resendIntervalMax = config.getDuration("resend-interval-max").asScala,
onlyFlowControl = config.getBoolean("only-flow-control"))
}
@@ -182,7 +184,8 @@ object ConsumerController {
final class Settings private (
val flowControlWindow: Int,
- val resendInterval: FiniteDuration,
+ val resendIntervalMin: FiniteDuration,
+ val resendIntervalMax: FiniteDuration,
val onlyFlowControl: Boolean) {
def withFlowControlWindow(newFlowControlWindow: Int): Settings =
@@ -191,20 +194,32 @@ object ConsumerController {
/**
* Scala API
*/
- def withResendInterval(newResendInterval: FiniteDuration): Settings =
- copy(resendInterval = newResendInterval)
+ def withResendIntervalMin(newResendIntervalMin: FiniteDuration): Settings =
+ copy(resendIntervalMin = newResendIntervalMin)
+
+ /**
+ * Scala API
+ */
+ def withResendIntervalMax(newResendIntervalMax: FiniteDuration): Settings =
+ copy(resendIntervalMax = newResendIntervalMax)
/**
* Java API
*/
- def withResendInterval(newResendInterval: JavaDuration): Settings =
- copy(resendInterval = newResendInterval.asScala)
+ def withResendIntervalMin(newResendIntervalMin: JavaDuration): Settings =
+ copy(resendIntervalMin = newResendIntervalMin.asScala)
/**
* Java API
*/
- def getResendInterval(): JavaDuration =
- resendInterval.asJava
+ def withResendIntervalMax(newResendIntervalMax: JavaDuration): Settings =
+ copy(resendIntervalMax = newResendIntervalMax.asScala)
+
+ /**
+ * Java API
+ */
+ def getResendIntervalMax(): JavaDuration =
+ resendIntervalMax.asJava
def withOnlyFlowControl(newOnlyFlowControl: Boolean): Settings =
copy(onlyFlowControl = newOnlyFlowControl)
@@ -214,12 +229,13 @@ object ConsumerController {
*/
private def copy(
flowControlWindow: Int = flowControlWindow,
- resendInterval: FiniteDuration = resendInterval,
+ resendIntervalMin: FiniteDuration = resendIntervalMin,
+ resendIntervalMax: FiniteDuration = resendIntervalMax,
onlyFlowControl: Boolean = onlyFlowControl) =
- new Settings(flowControlWindow, resendInterval, onlyFlowControl)
+ new Settings(flowControlWindow, resendIntervalMin, resendIntervalMax, onlyFlowControl)
override def toString: String =
- s"Settings($flowControlWindow, $resendInterval, $onlyFlowControl)"
+ s"Settings($flowControlWindow, $resendIntervalMin, $onlyFlowControl)"
}
def apply[A](): Behavior[Command[A]] =
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/DurableProducerQueue.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/DurableProducerQueue.scala
index 4f8ce85a4a..38d577ab49 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/DurableProducerQueue.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/DurableProducerQueue.scala
@@ -7,11 +7,10 @@ package akka.actor.typed.delivery
import scala.collection.immutable
import akka.actor.typed.ActorRef
+import akka.actor.typed.delivery.internal.DeliverySerializable
import akka.annotation.ApiMayChange
import akka.annotation.InternalApi
-import akka.actor.typed.delivery.internal.DeliverySerializable
-
/**
* Actor message protocol for storing and confirming reliable delivery of messages. A [[akka.actor.typed.Behavior]]
* implementation of this protocol can optionally be used with [[ProducerController]] when messages shall survive
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/ProducerController.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/ProducerController.scala
index e1562ffabc..c9eca025dd 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/ProducerController.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/ProducerController.scala
@@ -11,6 +11,8 @@ import scala.compat.java8.OptionConverters._
import scala.concurrent.duration._
import scala.reflect.ClassTag
+import com.typesafe.config.Config
+
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
@@ -20,7 +22,6 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.annotation.ApiMayChange
import akka.annotation.InternalApi
import akka.util.JavaDurationConverters._
-import com.typesafe.config.Config
/**
* Point-to-point reliable delivery between a single producer actor sending messages and a single consumer
@@ -76,6 +77,9 @@ import com.typesafe.config.Config
* The `producerId` is used in logging and included as MDC entry with key `"producerId"`. It's propagated
* to the `ConsumerController` and is useful for correlating log messages. It can be any `String` but it's
* recommended to use a unique identifier of representing the producer.
+ *
+ * If the `DurableProducerQueue` is defined it is created as a child actor of the `ProducerController` actor.
+ * It will use the same dispatcher as the parent `ProducerController`.
*/
@ApiMayChange // TODO #28719 when removing ApiMayChange consider removing `case class` for some of the messages
object ProducerController {
@@ -151,7 +155,8 @@ object ProducerController {
def apply(config: Config): Settings = {
new Settings(
durableQueueRequestTimeout = config.getDuration("durable-queue.request-timeout").asScala,
- durableQueueRetryAttempts = config.getInt("durable-queue.retry-attempts"))
+ durableQueueRetryAttempts = config.getInt("durable-queue.retry-attempts"),
+ durableQueueResendFirstInterval = config.getDuration("durable-queue.resend-first-interval").asScala)
}
/**
@@ -169,7 +174,10 @@ object ProducerController {
apply(config)
}
- final class Settings private (val durableQueueRequestTimeout: FiniteDuration, val durableQueueRetryAttempts: Int) {
+ final class Settings private (
+ val durableQueueRequestTimeout: FiniteDuration,
+ val durableQueueRetryAttempts: Int,
+ val durableQueueResendFirstInterval: FiniteDuration) {
def withDurableQueueRetryAttempts(newDurableQueueRetryAttempts: Int): Settings =
copy(durableQueueRetryAttempts = newDurableQueueRetryAttempts)
@@ -180,12 +188,24 @@ object ProducerController {
def withDurableQueueRequestTimeout(newDurableQueueRequestTimeout: FiniteDuration): Settings =
copy(durableQueueRequestTimeout = newDurableQueueRequestTimeout)
+ /**
+ * Scala API
+ */
+ def withDurableQueueResendFirstInterval(newDurableQueueResendFirstInterval: FiniteDuration): Settings =
+ copy(durableQueueResendFirstInterval = newDurableQueueResendFirstInterval)
+
/**
* Java API
*/
def withDurableQueueRequestTimeout(newDurableQueueRequestTimeout: JavaDuration): Settings =
copy(durableQueueRequestTimeout = newDurableQueueRequestTimeout.asScala)
+ /**
+ * Java API
+ */
+ def withDurableQueueResendFirstInterval(newDurableQueueResendFirstInterval: JavaDuration): Settings =
+ copy(durableQueueResendFirstInterval = newDurableQueueResendFirstInterval.asScala)
+
/**
* Java API
*/
@@ -197,11 +217,12 @@ object ProducerController {
*/
private def copy(
durableQueueRequestTimeout: FiniteDuration = durableQueueRequestTimeout,
- durableQueueRetryAttempts: Int = durableQueueRetryAttempts) =
- new Settings(durableQueueRequestTimeout, durableQueueRetryAttempts)
+ durableQueueRetryAttempts: Int = durableQueueRetryAttempts,
+ durableQueueResendFirstInterval: FiniteDuration = durableQueueResendFirstInterval) =
+ new Settings(durableQueueRequestTimeout, durableQueueRetryAttempts, durableQueueResendFirstInterval)
override def toString: String =
- s"Settings($durableQueueRequestTimeout, $durableQueueRetryAttempts)"
+ s"Settings($durableQueueRequestTimeout, $durableQueueRetryAttempts, $durableQueueResendFirstInterval)"
}
def apply[A: ClassTag](
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/WorkPullingProducerController.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/WorkPullingProducerController.scala
index ffdde54b0a..9135adf33c 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/WorkPullingProducerController.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/WorkPullingProducerController.scala
@@ -6,9 +6,11 @@ package akka.actor.typed.delivery
import java.util.Optional
-import scala.reflect.ClassTag
import scala.compat.java8.OptionConverters._
import scala.concurrent.duration.FiniteDuration
+import scala.reflect.ClassTag
+
+import com.typesafe.config.Config
import akka.Done
import akka.actor.typed.ActorRef
@@ -19,7 +21,6 @@ import akka.actor.typed.receptionist.ServiceKey
import akka.actor.typed.scaladsl.Behaviors
import akka.annotation.ApiMayChange
import akka.util.JavaDurationConverters._
-import com.typesafe.config.Config
/**
* Work pulling is a pattern where several worker actors pull tasks in their own pace from
@@ -90,6 +91,10 @@ import com.typesafe.config.Config
* The `producerId` is used in logging and included as MDC entry with key `"producerId"`. It's propagated
* to the `ConsumerController` and is useful for correlating log messages. It can be any `String` but it's
* recommended to use a unique identifier of representing the producer.
+ *
+ * If the `DurableProducerQueue` is defined it is created as a child actor of the `WorkPullingProducerController` actor.
+ * `ProducerController` actors are created for each registered worker. Those child actors use the same dispatcher
+ * as the parent `WorkPullingProducerController`.
*/
@ApiMayChange // TODO #28719 when removing ApiMayChange consider removing `case class` for some of the messages
object WorkPullingProducerController {
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/internal/ConsumerControllerImpl.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/internal/ConsumerControllerImpl.scala
index 9765872df0..362b6efc87 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/internal/ConsumerControllerImpl.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/internal/ConsumerControllerImpl.scala
@@ -4,6 +4,8 @@
package akka.actor.typed.delivery.internal
+import scala.concurrent.duration.FiniteDuration
+
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
import akka.actor.typed.PostStop
@@ -19,6 +21,7 @@ import akka.actor.typed.scaladsl.LoggerOps
import akka.actor.typed.scaladsl.StashBuffer
import akka.actor.typed.scaladsl.TimerScheduler
import akka.annotation.InternalApi
+import akka.util.ConstantFun.scalaIdentityFunction
/**
* INTERNAL API
@@ -72,7 +75,7 @@ import akka.annotation.InternalApi
/** For commands defined in public ConsumerController */
trait UnsealedInternalCommand extends InternalCommand
- private final case object Retry extends InternalCommand
+ private case object Retry extends InternalCommand
private final case class ConsumerTerminated(consumer: ActorRef[_]) extends InternalCommand
@@ -115,34 +118,45 @@ import akka.annotation.InternalApi
}
Behaviors.withTimers { timers =>
// wait for the `Start` message from the consumer, SequencedMessage will be stashed
- def waitForStart(
- registering: Option[ActorRef[ProducerController.Command[A]]]): Behavior[InternalCommand] = {
+ def waitForStart(registering: Option[ActorRef[ProducerController.Command[A]]], stopping: Boolean)
+ : Behavior[InternalCommand] = {
Behaviors.receiveMessagePartial {
case reg: RegisterToProducerController[A] @unchecked =>
reg.producerController ! ProducerController.RegisterConsumer(context.self)
- waitForStart(Some(reg.producerController))
+ waitForStart(Some(reg.producerController), stopping)
case s: Start[A] @unchecked =>
ConsumerControllerImpl.enforceLocalConsumer(s.deliverTo)
context.watchWith(s.deliverTo, ConsumerTerminated(s.deliverTo))
flightRecorder.consumerStarted(context.self.path)
+ val retryTimer = new RetryTimer(timers, settings.resendIntervalMin, settings.resendIntervalMax)
val activeBehavior =
- new ConsumerControllerImpl[A](context, timers, stashBuffer, settings)
- .active(initialState(context, s, registering))
+ new ConsumerControllerImpl[A](context, retryTimer, stashBuffer, settings)
+ .active(initialState(context, s, registering, stopping))
context.log.debug("Received Start, unstash [{}] messages.", stashBuffer.size)
- stashBuffer.unstashAll(activeBehavior)
+ stashBuffer.unstash(activeBehavior, 1, scalaIdentityFunction)
case seqMsg: SequencedMessage[A] @unchecked =>
- stashBuffer.stash(seqMsg)
+ if (stashBuffer.isFull) {
+ flightRecorder.consumerStashFull(seqMsg.producerId, seqMsg.seqNr)
+ context.log.debug(
+ "Received SequencedMessage seqNr [{}], stashing before Start, discarding message because stash is full.",
+ seqMsg.seqNr)
+ } else {
+ context.log.trace(
+ "Received SequencedMessage seqNr [{}], stashing before Start, stashed size [{}].",
+ seqMsg.seqNr,
+ stashBuffer.size + 1)
+ stashBuffer.stash(seqMsg)
+ }
Behaviors.same
- case d: DeliverThenStop[_] =>
+ case _: DeliverThenStop[_] =>
if (stashBuffer.isEmpty) {
Behaviors.stopped
} else {
- stashBuffer.stash(d)
- Behaviors.same
+ waitForStart(registering, stopping = true)
}
case Retry =>
@@ -160,8 +174,8 @@ import akka.annotation.InternalApi
}
- timers.startTimerWithFixedDelay(Retry, Retry, settings.resendInterval)
- waitForStart(None)
+ timers.startTimerWithFixedDelay(Retry, Retry, settings.resendIntervalMin)
+ waitForStart(None, stopping = false)
}
}
}
@@ -179,7 +193,8 @@ import akka.annotation.InternalApi
private def initialState[A](
context: ActorContext[InternalCommand],
start: Start[A],
- registering: Option[ActorRef[ProducerController.Command[A]]]): State[A] = {
+ registering: Option[ActorRef[ProducerController.Command[A]]],
+ stopping: Boolean): State[A] = {
State(
producerController = context.system.deadLetters,
"n/a",
@@ -188,18 +203,54 @@ import akka.annotation.InternalApi
confirmedSeqNr = 0,
requestedSeqNr = 0,
registering,
- stopping = false)
+ stopping)
}
def enforceLocalConsumer(ref: ActorRef[_]): Unit = {
if (ref.path.address.hasGlobalScope)
throw new IllegalArgumentException(s"Consumer [$ref] should be local.")
}
+
+ private class RetryTimer(
+ timers: TimerScheduler[ConsumerControllerImpl.InternalCommand],
+ val minBackoff: FiniteDuration,
+ maxBackoff: FiniteDuration) {
+ private var _interval = minBackoff
+
+ def interval(): FiniteDuration =
+ _interval
+
+ def start(): Unit = {
+ _interval = minBackoff
+ timers.startTimerWithFixedDelay(Retry, _interval)
+ }
+
+ def scheduleNext(): Unit = {
+ val newInterval =
+ if (_interval eq maxBackoff)
+ maxBackoff
+ else
+ maxBackoff.min(_interval * 1.5) match {
+ case f: FiniteDuration => f
+ case _ => maxBackoff
+ }
+ if (newInterval != _interval) {
+ _interval = newInterval
+ timers.startTimerWithFixedDelay(Retry, _interval)
+ }
+ }
+
+ def reset(): Unit = {
+ if (_interval ne minBackoff)
+ start()
+ }
+
+ }
}
private class ConsumerControllerImpl[A](
context: ActorContext[ConsumerControllerImpl.InternalCommand],
- timers: TimerScheduler[ConsumerControllerImpl.InternalCommand],
+ retryTimer: ConsumerControllerImpl.RetryTimer,
stashBuffer: StashBuffer[ConsumerControllerImpl.InternalCommand],
settings: ConsumerController.Settings) {
@@ -216,7 +267,9 @@ private class ConsumerControllerImpl[A](
private val flightRecorder = ActorFlightRecorder(context.system).delivery
- startRetryTimer()
+ private val traceEnabled = context.log.isTraceEnabled
+
+ retryTimer.start()
private def resendLost = !settings.onlyFlowControl
@@ -231,18 +284,20 @@ private class ConsumerControllerImpl[A](
val expectedSeqNr = s.receivedSeqNr + 1
flightRecorder.consumerReceived(pid, seqNr)
+ retryTimer.reset()
if (s.isProducerChanged(seqMsg)) {
- if (seqMsg.first)
+ if (seqMsg.first && traceEnabled)
context.log.trace("Received first SequencedMessage seqNr [{}], delivering to consumer.", seqNr)
receiveChangedProducer(s, seqMsg)
} else if (s.registering.isDefined) {
context.log.debug(
"Received SequencedMessage seqNr [{}], discarding message because registering to new ProducerController.",
seqNr)
- Behaviors.same
+ stashBuffer.unstash(Behaviors.same, 1, scalaIdentityFunction)
} else if (s.isNextExpected(seqMsg)) {
- context.log.trace("Received SequencedMessage seqNr [{}], delivering to consumer.", seqNr)
+ if (traceEnabled)
+ context.log.trace("Received SequencedMessage seqNr [{}], delivering to consumer.", seqNr)
deliver(s.copy(receivedSeqNr = seqNr), seqMsg)
} else if (seqNr > expectedSeqNr) {
flightRecorder.consumerMissing(pid, expectedSeqNr, seqNr)
@@ -253,18 +308,19 @@ private class ConsumerControllerImpl[A](
if (resendLost) "requesting resend from expected seqNr" else "delivering to consumer anyway")
if (resendLost) {
seqMsg.producerController ! Resend(fromSeqNr = expectedSeqNr)
+ stashBuffer.clear()
+ retryTimer.start()
resending(s)
} else {
- s.consumer ! Delivery(seqMsg.message, context.self, pid, seqNr)
- waitingForConfirmation(s.copy(receivedSeqNr = seqNr), seqMsg)
+ deliver(s.copy(receivedSeqNr = seqNr), seqMsg)
}
} else { // seqNr < expectedSeqNr
flightRecorder.consumerDuplicate(pid, expectedSeqNr, seqNr)
context.log.debug2("Received duplicate SequencedMessage seqNr [{}], expected [{}].", seqNr, expectedSeqNr)
if (seqMsg.first)
- active(retryRequest(s))
+ stashBuffer.unstash(active(retryRequest(s)), 1, scalaIdentityFunction)
else
- Behaviors.same
+ stashBuffer.unstash(Behaviors.same, 1, scalaIdentityFunction)
}
case Retry =>
@@ -320,6 +376,8 @@ private class ConsumerControllerImpl[A](
seqMsg.producerController)
// request resend of all unconfirmed, and mark first
seqMsg.producerController ! Resend(0)
+ stashBuffer.clear()
+ retryTimer.start()
resending(s)
} else {
context.log.warnN(
@@ -328,7 +386,7 @@ private class ConsumerControllerImpl[A](
seqNr,
seqMsg.producerController,
s.producerController)
- Behaviors.same
+ stashBuffer.unstash(Behaviors.same, 1, scalaIdentityFunction)
}
}
@@ -353,13 +411,15 @@ private class ConsumerControllerImpl[A](
// ProducerController with the missing seqNr. Other SequencedMessage with different seqNr will be
// discarded since they were in flight before the Resend request and will anyway be sent again.
private def resending(s: State[A]): Behavior[InternalCommand] = {
+ if (stashBuffer.nonEmpty)
+ throw new IllegalStateException("StashBuffer should be cleared before resending.")
Behaviors
.receiveMessage[InternalCommand] {
case seqMsg: SequencedMessage[A] =>
val seqNr = seqMsg.seqNr
if (s.isProducerChanged(seqMsg)) {
- if (seqMsg.first)
+ if (seqMsg.first && traceEnabled)
context.log.trace("Received first SequencedMessage seqNr [{}], delivering to consumer.", seqNr)
receiveChangedProducer(s, seqMsg)
} else if (s.registering.isDefined) {
@@ -426,7 +486,11 @@ private class ConsumerControllerImpl[A](
.receiveMessage[InternalCommand] {
case Confirmed =>
val seqNr = seqMsg.seqNr
- context.log.trace("Received Confirmed seqNr [{}] from consumer, stashed size [{}].", seqNr, stashBuffer.size)
+ if (traceEnabled)
+ context.log.trace(
+ "Received Confirmed seqNr [{}] from consumer, stashed size [{}].",
+ seqNr,
+ stashBuffer.size)
val newRequestedSeqNr =
if (seqMsg.first) {
@@ -447,11 +511,12 @@ private class ConsumerControllerImpl[A](
seqNr,
newRequestedSeqNr)
s.producerController ! Request(confirmedSeqNr = seqNr, newRequestedSeqNr, resendLost, viaTimeout = false)
- startRetryTimer() // reset interval since Request was just sent
+ retryTimer.start() // reset interval since Request was just sent
newRequestedSeqNr
} else {
if (seqMsg.ack) {
- context.log.trace("Sending Ack seqNr [{}].", seqNr)
+ if (traceEnabled)
+ context.log.trace("Sending Ack seqNr [{}].", seqNr)
s.producerController ! Ack(confirmedSeqNr = seqNr)
}
s.requestedSeqNr
@@ -464,32 +529,39 @@ private class ConsumerControllerImpl[A](
s.producerController ! Ack(seqNr)
}
} else {
- // FIXME #28718 can we use unstashOne instead of all?
- stashBuffer.unstashAll(active(s.copy(confirmedSeqNr = seqNr, requestedSeqNr = newRequestedSeqNr)))
+ stashBuffer.unstash(
+ active(s.copy(confirmedSeqNr = seqNr, requestedSeqNr = newRequestedSeqNr)),
+ 1,
+ scalaIdentityFunction)
}
case msg: SequencedMessage[A] =>
- flightRecorder.consumerReceivedPreviousInProgress(seqMsg.producerId, seqMsg.seqNr, stashBuffer.size + 1)
- if (msg.seqNr == seqMsg.seqNr && msg.producerController == seqMsg.producerController) {
- flightRecorder.consumerDuplicate(msg.producerId, seqMsg.seqNr + 1, msg.seqNr)
+ flightRecorder.consumerReceivedPreviousInProgress(msg.producerId, msg.seqNr, stashBuffer.size + 1)
+ val expectedSeqNr = seqMsg.seqNr + stashBuffer.size + 1
+ if (msg.seqNr < expectedSeqNr && msg.producerController == seqMsg.producerController) {
+ flightRecorder.consumerDuplicate(msg.producerId, expectedSeqNr, msg.seqNr)
context.log.debug("Received duplicate SequencedMessage seqNr [{}].", msg.seqNr)
} else if (stashBuffer.isFull) {
// possible that the stash is full if ProducerController resends unconfirmed (duplicates)
// dropping them since they can be resent
+ flightRecorder.consumerStashFull(msg.producerId, msg.seqNr)
context.log.debug(
"Received SequencedMessage seqNr [{}], discarding message because stash is full.",
msg.seqNr)
} else {
- context.log.trace(
- "Received SequencedMessage seqNr [{}], stashing while waiting for consumer to confirm [{}].",
- msg.seqNr,
- seqMsg.seqNr)
+ if (traceEnabled)
+ context.log.traceN(
+ "Received SequencedMessage seqNr [{}], stashing while waiting for consumer to confirm [{}], stashed size [{}].",
+ msg.seqNr,
+ seqMsg.seqNr,
+ stashBuffer.size + 1)
stashBuffer.stash(msg)
}
Behaviors.same
case Retry =>
- receiveRetry(s, () => waitingForConfirmation(retryRequest(s), seqMsg))
+ // no retries when waitingForConfirmation, will be performed from (idle) active
+ Behaviors.same
case start: Start[A] =>
start.deliverTo ! Delivery(seqMsg.message, context.self, seqMsg.producerId, seqMsg.seqNr)
@@ -513,6 +585,9 @@ private class ConsumerControllerImpl[A](
}
private def receiveRetry(s: State[A], nextBehavior: () => Behavior[InternalCommand]): Behavior[InternalCommand] = {
+ retryTimer.scheduleNext()
+ if (retryTimer.interval() != retryTimer.minBackoff)
+ context.log.debug("Schedule next retry in [{} ms]", retryTimer.interval().toMillis)
s.registering match {
case None => nextBehavior()
case Some(reg) =>
@@ -545,6 +620,7 @@ private class ConsumerControllerImpl[A](
"Register to new ProducerController [{}], previous was [{}].",
reg.producerController,
s.producerController)
+ retryTimer.start()
reg.producerController ! ProducerController.RegisterConsumer(context.self)
nextBehavior(s.copy(registering = Some(reg.producerController)))
} else {
@@ -573,25 +649,17 @@ private class ConsumerControllerImpl[A](
Behaviors.unhandled
}
- private def startRetryTimer(): Unit = {
- timers.startTimerWithFixedDelay(Retry, Retry, settings.resendInterval)
- }
-
// in case the Request or the SequencedMessage triggering the Request is lost
private def retryRequest(s: State[A]): State[A] = {
if (s.producerController == context.system.deadLetters) {
s
} else {
- // TODO #28720 Maybe try to adjust the retry frequency. Maybe some exponential backoff and less need for it when
- // SequenceMessage are arriving. On the other hand it might be too much overhead to reschedule of each
- // incoming SequenceMessage.
val newRequestedSeqNr = if (resendLost) s.requestedSeqNr else s.receivedSeqNr + flowControlWindow / 2
flightRecorder.consumerSentRequest(s.producerId, newRequestedSeqNr)
context.log.debug(
"Retry sending Request with confirmedSeqNr [{}], requestUpToSeqNr [{}].",
s.confirmedSeqNr,
newRequestedSeqNr)
- // TODO #28720 maybe watch the producer to avoid sending retry Request to dead producer
s.producerController ! Request(s.confirmedSeqNr, newRequestedSeqNr, resendLost, viaTimeout = true)
s.copy(requestedSeqNr = newRequestedSeqNr)
}
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/internal/ProducerControllerImpl.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/internal/ProducerControllerImpl.scala
index 00e9ff6794..c08aaa3908 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/internal/ProducerControllerImpl.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/internal/ProducerControllerImpl.scala
@@ -6,7 +6,6 @@ package akka.actor.typed.delivery.internal
import java.util.concurrent.TimeoutException
-import scala.concurrent.duration._
import scala.reflect.ClassTag
import scala.util.Failure
import scala.util.Success
@@ -14,6 +13,7 @@ import scala.util.Success
import akka.actor.DeadLetterSuppression
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
+import akka.actor.typed.DispatcherSelector
import akka.actor.typed.delivery.ConsumerController
import akka.actor.typed.delivery.ConsumerController.SequencedMessage
import akka.actor.typed.delivery.DurableProducerQueue
@@ -192,7 +192,7 @@ object ProducerControllerImpl {
settings: ProducerController.Settings): Option[ActorRef[DurableProducerQueue.Command[A]]] = {
durableQueueBehavior.map { b =>
- val ref = context.spawn(b, "durable")
+ val ref = context.spawn(b, "durable", DispatcherSelector.sameAsParent())
context.watchWith(ref, DurableQueueTerminated)
askLoadState(context, Some(ref), settings, attempt = 1)
ref
@@ -352,6 +352,7 @@ private class ProducerControllerImpl[A: ClassTag](
import ProducerControllerImpl._
private val flightRecorder = ActorFlightRecorder(context.system).delivery
+ private val traceEnabled = context.log.isTraceEnabled
// for the durableQueue StoreMessageSent ask
private implicit val askTimeout: Timeout = settings.durableQueueRequestTimeout
@@ -359,7 +360,7 @@ private class ProducerControllerImpl[A: ClassTag](
def onMsg(m: A, newReplyAfterStore: Map[SeqNr, ActorRef[SeqNr]], ack: Boolean): Behavior[InternalCommand] = {
checkOnMsgRequestedState()
- if (context.log.isTraceEnabled)
+ if (traceEnabled)
context.log.trace("Sending [{}] with seqNr [{}].", m.getClass.getName, s.currentSeqNr)
val seqMsg = SequencedMessage(producerId, s.currentSeqNr, m, s.currentSeqNr == s.firstSeqNr, ack)(context.self)
val newUnconfirmed =
@@ -367,7 +368,7 @@ private class ProducerControllerImpl[A: ClassTag](
else Vector.empty // no resending, no need to keep unconfirmed
if (s.currentSeqNr == s.firstSeqNr)
- timers.startTimerWithFixedDelay(ResendFirst, ResendFirst, 1.second)
+ timers.startTimerWithFixedDelay(ResendFirst, delay = settings.durableQueueResendFirstInterval)
flightRecorder.producerSent(producerId, seqMsg.seqNr)
s.send(seqMsg)
@@ -401,7 +402,7 @@ private class ProducerControllerImpl[A: ClassTag](
newRequestedSeqNr: SeqNr,
supportResend: Boolean,
viaTimeout: Boolean): Behavior[InternalCommand] = {
- flightRecorder.producerReceivedRequest(producerId, newRequestedSeqNr)
+ flightRecorder.producerReceivedRequest(producerId, newRequestedSeqNr, newConfirmedSeqNr)
context.log.debugN(
"Received Request, confirmed [{}], requested [{}], current [{}]",
newConfirmedSeqNr,
@@ -449,7 +450,8 @@ private class ProducerControllerImpl[A: ClassTag](
}
def receiveAck(newConfirmedSeqNr: SeqNr): Behavior[InternalCommand] = {
- context.log.trace2("Received Ack, confirmed [{}], current [{}].", newConfirmedSeqNr, s.currentSeqNr)
+ if (traceEnabled)
+ context.log.trace2("Received Ack, confirmed [{}], current [{}].", newConfirmedSeqNr, s.currentSeqNr)
val stateAfterAck = onAck(newConfirmedSeqNr)
if (newConfirmedSeqNr == s.firstSeqNr && stateAfterAck.unconfirmed.nonEmpty) {
resendUnconfirmed(stateAfterAck.unconfirmed)
@@ -459,7 +461,7 @@ private class ProducerControllerImpl[A: ClassTag](
def onAck(newConfirmedSeqNr: SeqNr): State[A] = {
val (replies, newReplyAfterStore) = s.replyAfterStore.partition { case (seqNr, _) => seqNr <= newConfirmedSeqNr }
- if (replies.nonEmpty)
+ if (replies.nonEmpty && traceEnabled)
context.log.trace("Sending confirmation replies from [{}] to [{}].", replies.head._1, replies.last._1)
replies.foreach {
case (seqNr, replyTo) => replyTo ! seqNr
@@ -489,7 +491,8 @@ private class ProducerControllerImpl[A: ClassTag](
throw new IllegalStateException(s"currentSeqNr [${s.currentSeqNr}] not matching stored seqNr [$seqNr]")
s.replyAfterStore.get(seqNr).foreach { replyTo =>
- context.log.trace("Sending confirmation reply to [{}] after storage.", seqNr)
+ if (traceEnabled)
+ context.log.trace("Sending confirmation reply to [{}] after storage.", seqNr)
replyTo ! seqNr
}
val newReplyAfterStore = s.replyAfterStore - seqNr
@@ -559,7 +562,7 @@ private class ProducerControllerImpl[A: ClassTag](
consumerController,
newFirstSeqNr)
if (s.unconfirmed.nonEmpty) {
- timers.startTimerWithFixedDelay(ResendFirst, ResendFirst, 1.second)
+ timers.startTimerWithFixedDelay(ResendFirst, delay = settings.durableQueueResendFirstInterval)
context.self ! ResendFirst
}
// update the send function
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/internal/WorkPullingProducerControllerImpl.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/internal/WorkPullingProducerControllerImpl.scala
index 70444d90bf..d6c846b83c 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/internal/WorkPullingProducerControllerImpl.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/delivery/internal/WorkPullingProducerControllerImpl.scala
@@ -15,6 +15,7 @@ import scala.util.Success
import akka.Done
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
+import akka.actor.typed.DispatcherSelector
import akka.actor.typed.delivery.ConsumerController
import akka.actor.typed.delivery.DurableProducerQueue
import akka.actor.typed.delivery.DurableProducerQueue.ConfirmationQualifier
@@ -230,7 +231,7 @@ import akka.util.Timeout
settings: WorkPullingProducerController.Settings): Option[ActorRef[DurableProducerQueue.Command[A]]] = {
durableQueueBehavior.map { b =>
- val ref = context.spawn(b, "durable")
+ val ref = context.spawn(b, "durable", DispatcherSelector.sameAsParent())
context.watchWith(ref, DurableQueueTerminated)
askLoadState(context, Some(ref), settings, attempt = 1)
ref
@@ -277,6 +278,7 @@ private class WorkPullingProducerControllerImpl[A: ClassTag](
import WorkPullingProducerController.WorkerStats
import WorkPullingProducerControllerImpl._
+ private val traceEnabled = context.log.isTraceEnabled
private val durableQueueAskTimeout: Timeout = settings.producerControllerSettings.durableQueueRequestTimeout
private val workerAskTimeout: Timeout = settings.internalAskTimeout
@@ -287,7 +289,7 @@ private class WorkPullingProducerControllerImpl[A: ClassTag](
def onMessage(msg: A, wasStashed: Boolean, replyTo: Option[ActorRef[Done]], totalSeqNr: TotalSeqNr): State[A] = {
val consumersWithDemand = s.out.iterator.filter { case (_, out) => out.askNextTo.isDefined }.toVector
- if (context.log.isTraceEnabled)
+ if (traceEnabled)
context.log.traceN(
"Received message seqNr [{}], wasStashed [{}], consumersWithDemand [{}], hasRequested [{}].",
totalSeqNr,
@@ -336,7 +338,8 @@ private class WorkPullingProducerControllerImpl[A: ClassTag](
}
def tellRequestNext(): Unit = {
- context.log.trace("Sending RequestNext to producer, seqNr [{}].", totalSeqNr)
+ if (traceEnabled)
+ context.log.trace("Sending RequestNext to producer, seqNr [{}].", totalSeqNr)
s.producer ! requestNext
}
@@ -451,7 +454,8 @@ private class WorkPullingProducerControllerImpl[A: ClassTag](
def receiveStoreMessageSentCompleted(seqNr: SeqNr, m: A) = {
s.replyAfterStore.get(seqNr).foreach { replyTo =>
- context.log.trace("Sending reply for seqNr [{}] after storage.", seqNr)
+ if (traceEnabled)
+ context.log.trace("Sending reply for seqNr [{}] after storage.", seqNr)
replyTo ! Done
}
@@ -483,7 +487,8 @@ private class WorkPullingProducerControllerImpl[A: ClassTag](
}
if (confirmed.nonEmpty) {
- context.log.trace("Received Ack seqNr [{}] from worker [{}].", confirmedSeqNr, outState.confirmationQualifier)
+ if (traceEnabled)
+ context.log.trace("Received Ack seqNr [{}] from worker [{}].", confirmedSeqNr, outState.confirmationQualifier)
confirmed.foreach {
case Unconfirmed(_, _, _, None) => // no reply
case Unconfirmed(_, _, _, Some(replyTo)) =>
@@ -508,10 +513,11 @@ private class WorkPullingProducerControllerImpl[A: ClassTag](
s.out.get(outKey) match {
case Some(outState) =>
val confirmedSeqNr = w.next.confirmedSeqNr
- context.log.trace2(
- "Received RequestNext from worker [{}], confirmedSeqNr [{}].",
- w.next.producerId,
- confirmedSeqNr)
+ if (traceEnabled)
+ context.log.trace2(
+ "Received RequestNext from worker [{}], confirmedSeqNr [{}].",
+ w.next.producerId,
+ confirmedSeqNr)
val newUnconfirmed = onAck(outState, confirmedSeqNr)
@@ -527,7 +533,9 @@ private class WorkPullingProducerControllerImpl[A: ClassTag](
} else if (s.requested) {
active(s.copy(out = newOut))
} else {
- context.log.trace("Sending RequestNext to producer after RequestNext from worker [{}].", w.next.producerId)
+ if (traceEnabled)
+ context.log
+ .trace("Sending RequestNext to producer after RequestNext from worker [{}].", w.next.producerId)
s.producer ! requestNext
active(s.copy(out = newOut, requested = true))
}
@@ -549,7 +557,8 @@ private class WorkPullingProducerControllerImpl[A: ClassTag](
context.log.debug2("Registered worker [{}], with producerId [{}].", c, outKey)
val p = context.spawn(
ProducerController[A](outKey, durableQueueBehavior = None, settings.producerControllerSettings),
- uuid)
+ uuid,
+ DispatcherSelector.sameAsParent())
p ! ProducerController.Start(workerRequestNextAdapter)
p ! ProducerController.RegisterConsumer(c)
acc.copy(out = acc.out.updated(outKey, OutState(p, c, 0L, Vector.empty, None)))
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/eventstream/EventStream.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/eventstream/EventStream.scala
index 97ea12189d..727a927003 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/eventstream/EventStream.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/eventstream/EventStream.scala
@@ -4,9 +4,10 @@
package akka.actor.typed.eventstream
+import scala.reflect.ClassTag
+
import akka.actor.typed.ActorRef
import akka.annotation.{ DoNotInherit, InternalApi }
-import scala.reflect.ClassTag
object EventStream {
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ActorContextImpl.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ActorContextImpl.scala
index ce71d32ab8..a1ba9b6e41 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ActorContextImpl.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ActorContextImpl.scala
@@ -10,22 +10,23 @@ import java.util.ArrayList
import java.util.Optional
import java.util.concurrent.CompletionStage
-import akka.actor.Address
-import akka.actor.typed.internal.adapter.ActorSystemAdapter
-
import scala.concurrent.{ ExecutionContextExecutor, Future }
import scala.reflect.ClassTag
import scala.util.Try
-import akka.annotation.InternalApi
-import akka.dispatch.ExecutionContexts
-import akka.util.{ BoxedType, Timeout }
-import akka.util.Timeout
-import akka.util.JavaDurationConverters._
-import akka.util.OptionVal
+
import com.github.ghik.silencer.silent
import org.slf4j.Logger
import org.slf4j.LoggerFactory
+import akka.actor.Address
+import akka.actor.typed.internal.adapter.ActorSystemAdapter
+import akka.annotation.InternalApi
+import akka.dispatch.ExecutionContexts
+import akka.util.{ BoxedType, Timeout }
+import akka.util.JavaDurationConverters._
+import akka.util.OptionVal
+import akka.util.Timeout
+
/**
* INTERNAL API
*/
@@ -94,10 +95,16 @@ import org.slf4j.LoggerFactory
private var _messageAdapters: List[(Class[_], Any => T)] = Nil
private var _timer: OptionVal[TimerSchedulerImpl[T]] = OptionVal.None
+ // _currentActorThread is on purpose not volatile. Used from `checkCurrentActorThread`.
+ // It will always see the right value when accessed from the right thread.
+ // Possible that it would NOT detect illegal access sometimes but that's ok.
+ private var _currentActorThread: OptionVal[Thread] = OptionVal.None
+
// context-shared timer needed to allow for nested timer usage
def timer: TimerSchedulerImpl[T] = _timer match {
case OptionVal.Some(timer) => timer
case OptionVal.None =>
+ checkCurrentActorThread()
val timer = new TimerSchedulerImpl[T](this)
_timer = OptionVal.Some(timer)
timer
@@ -151,6 +158,7 @@ import org.slf4j.LoggerFactory
}
override def log: Logger = {
+ checkCurrentActorThread()
val logging = loggingContext()
ActorMdc.setMdc(logging)
logging.logger
@@ -159,6 +167,7 @@ import org.slf4j.LoggerFactory
override def getLog: Logger = log
override def setLoggerName(name: String): Unit = {
+ checkCurrentActorThread()
_logging = OptionVal.Some(loggingContext().withLogger(LoggerFactory.getLogger(name)))
}
@@ -246,6 +255,7 @@ import org.slf4j.LoggerFactory
internalMessageAdapter(messageClass, f.apply)
private def internalMessageAdapter[U](messageClass: Class[U], f: U => T): ActorRef[U] = {
+ checkCurrentActorThread()
// replace existing adapter for same class, only one per class is supported to avoid unbounded growth
// in case "same" adapter is added repeatedly
val boxedMessageClass = BoxedType(messageClass).asInstanceOf[Class[U]]
@@ -267,4 +277,44 @@ import org.slf4j.LoggerFactory
* INTERNAL API
*/
@InternalApi private[akka] def messageAdapters: List[(Class[_], Any => T)] = _messageAdapters
+
+ /**
+ * INTERNAL API
+ */
+ @InternalApi private[akka] def setCurrentActorThread(): Unit = {
+ _currentActorThread match {
+ case OptionVal.None =>
+ _currentActorThread = OptionVal.Some(Thread.currentThread())
+ case OptionVal.Some(t) =>
+ throw new IllegalStateException(
+ s"Invalid access by thread from the outside of $self. " +
+ s"Current message is processed by $t, but also accessed from from ${Thread.currentThread()}.")
+ }
+ }
+
+ /**
+ * INTERNAL API
+ */
+ @InternalApi private[akka] def clearCurrentActorThread(): Unit = {
+ _currentActorThread = OptionVal.None
+ }
+
+ /**
+ * INTERNAL API
+ */
+ @InternalApi private[akka] def checkCurrentActorThread(): Unit = {
+ val callerThread = Thread.currentThread()
+ _currentActorThread match {
+ case OptionVal.Some(t) =>
+ if (callerThread ne t) {
+ throw new UnsupportedOperationException(
+ s"Unsupported access to ActorContext operation from the outside of $self. " +
+ s"Current message is processed by $t, but ActorContext was called from $callerThread.")
+ }
+ case OptionVal.None =>
+ throw new UnsupportedOperationException(
+ s"Unsupported access to ActorContext from the outside of $self. " +
+ s"No message is currently processed by the actor, but ActorContext was called from $callerThread.")
+ }
+ }
}
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ActorFlightRecorder.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ActorFlightRecorder.scala
index f841f1bfb2..faf8956b81 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ActorFlightRecorder.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ActorFlightRecorder.scala
@@ -60,18 +60,19 @@ private[akka] trait ActorFlightRecorder extends Extension {
def producerResentFirst(producerId: String, firstSeqNr: Long): Unit
def producerResentFirstUnconfirmed(producerId: String, seqNr: Long): Unit
def producerReceived(producerId: String, currentSeqNr: Long): Unit
- def producerReceivedRequest(producerId: String, requestedSeqNr: Long): Unit
+ def producerReceivedRequest(producerId: String, requestedSeqNr: Long, confirmedSeqNr: Long): Unit
def producerReceivedResend(producerId: String, fromSeqNr: Long): Unit
def consumerCreated(path: ActorPath): Unit
def consumerStarted(path: ActorPath): Unit
def consumerReceived(producerId: String, seqNr: Long): Unit
def consumerReceivedPreviousInProgress(producerId: String, seqNr: Long, stashed: Int): Unit
- def consumerDuplicate(pid: String, expectedSeqNr: Long, seqNr: Long): Unit
- def consumerMissing(pid: String, expectedSeqNr: Long, seqNr: Long): Unit
+ def consumerDuplicate(producerId: String, expectedSeqNr: Long, seqNr: Long): Unit
+ def consumerMissing(producerId: String, expectedSeqNr: Long, seqNr: Long): Unit
def consumerReceivedResend(seqNr: Long): Unit
def consumerSentRequest(producerId: String, requestedSeqNr: Long): Unit
def consumerChangedProducer(producerId: String): Unit
+ def consumerStashFull(producerId: String, seqNr: Long): Unit
}
/**
@@ -99,17 +100,18 @@ private[akka] case object NoOpActorFlightRecorder extends ActorFlightRecorder {
override def producerResentFirst(producerId: String, firstSeqNr: Long): Unit = ()
override def producerResentFirstUnconfirmed(producerId: String, seqNr: Long): Unit = ()
override def producerReceived(producerId: String, currentSeqNr: Long): Unit = ()
- override def producerReceivedRequest(producerId: String, requestedSeqNr: Long): Unit = ()
+ override def producerReceivedRequest(producerId: String, requestedSeqNr: Long, confirmedSeqNr: Long): Unit = ()
override def producerReceivedResend(producerId: String, fromSeqNr: Long): Unit = ()
override def consumerCreated(path: ActorPath): Unit = ()
override def consumerStarted(path: ActorPath): Unit = ()
override def consumerReceived(producerId: String, seqNr: Long): Unit = ()
override def consumerReceivedPreviousInProgress(producerId: String, seqNr: Long, stashed: Int): Unit = ()
- override def consumerDuplicate(pid: String, expectedSeqNr: Long, seqNr: Long): Unit = ()
- override def consumerMissing(pid: String, expectedSeqNr: Long, seqNr: Long): Unit = ()
+ override def consumerDuplicate(producerId: String, expectedSeqNr: Long, seqNr: Long): Unit = ()
+ override def consumerMissing(producerId: String, expectedSeqNr: Long, seqNr: Long): Unit = ()
override def consumerReceivedResend(seqNr: Long): Unit = ()
override def consumerSentRequest(producerId: String, requestedSeqNr: Long): Unit = ()
override def consumerChangedProducer(producerId: String): Unit = ()
+ override def consumerStashFull(producerId: String, seqNr: Long): Unit = ()
}
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ActorMdc.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ActorMdc.scala
index f21fa01e78..3e7cf02224 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ActorMdc.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ActorMdc.scala
@@ -4,9 +4,10 @@
package akka.actor.typed.internal
-import akka.annotation.InternalApi
import org.slf4j.MDC
+import akka.annotation.InternalApi
+
/**
* INTERNAL API
*/
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/BehaviorImpl.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/BehaviorImpl.scala
index d1f3788305..fb7646d11c 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/BehaviorImpl.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/BehaviorImpl.scala
@@ -7,10 +7,10 @@ package internal
import scala.reflect.ClassTag
-import akka.util.LineNumbers
-import akka.annotation.InternalApi
import akka.actor.typed.{ TypedActorContext => AC }
import akka.actor.typed.scaladsl.{ ActorContext => SAC }
+import akka.annotation.InternalApi
+import akka.util.LineNumbers
import akka.util.OptionVal
/**
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/EventStreamExtension.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/EventStreamExtension.scala
index 469ef1eaec..0ff219371d 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/EventStreamExtension.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/EventStreamExtension.scala
@@ -4,9 +4,9 @@
package akka.actor.typed.internal
-import akka.actor.typed.internal.adapter.EventStreamAdapter
import akka.actor.typed._
import akka.actor.typed.eventstream.EventStream
+import akka.actor.typed.internal.adapter.EventStreamAdapter
import akka.actor.typed.scaladsl.adapter._
import akka.annotation.InternalApi
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ExtensionsImpl.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ExtensionsImpl.scala
index 66fe0d477e..11c4dda25f 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ExtensionsImpl.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/ExtensionsImpl.scala
@@ -6,13 +6,13 @@ package akka.actor.typed.internal
import java.util.concurrent.{ ConcurrentHashMap, CountDownLatch }
-import akka.annotation.InternalApi
-import akka.actor.typed.{ ActorSystem, Extension, ExtensionId, Extensions }
import scala.annotation.tailrec
import scala.util.{ Failure, Success, Try }
-import akka.util.ccompat.JavaConverters._
+import akka.actor.typed.{ ActorSystem, Extension, ExtensionId, Extensions }
import akka.actor.typed.ExtensionSetup
+import akka.annotation.InternalApi
+import akka.util.ccompat.JavaConverters._
/**
* INTERNAL API
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/InterceptorImpl.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/InterceptorImpl.scala
index bd3998250c..7f2d5447ad 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/InterceptorImpl.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/InterceptorImpl.scala
@@ -5,15 +5,17 @@
package akka.actor.typed.internal
import scala.reflect.ClassTag
-import akka.actor.typed
-import akka.actor.typed.scaladsl.Behaviors
-import akka.actor.typed.LogOptions
-import akka.actor.typed._
-import akka.annotation.InternalApi
-import akka.util.LineNumbers
+
import org.slf4j.LoggerFactory
import org.slf4j.event.Level
+import akka.actor.typed
+import akka.actor.typed._
+import akka.actor.typed.LogOptions
+import akka.actor.typed.scaladsl.Behaviors
+import akka.annotation.InternalApi
+import akka.util.LineNumbers
+
/**
* Provides the impl of any behavior that could nest another behavior
*
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/LoggerClass.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/LoggerClass.scala
index 0145ec1147..710498a790 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/LoggerClass.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/LoggerClass.scala
@@ -4,11 +4,11 @@
package akka.actor.typed.internal
+import scala.util.control.NonFatal
+
import akka.annotation.InternalApi
import akka.util.OptionVal
-import scala.util.control.NonFatal
-
/**
* INTERNAL API
*/
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/PoisonPill.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/PoisonPill.scala
index e22726b902..a3aa58111d 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/PoisonPill.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/PoisonPill.scala
@@ -4,11 +4,11 @@
package akka.actor.typed.internal
-import akka.actor.typed.TypedActorContext
import akka.actor.typed.Behavior
import akka.actor.typed.BehaviorInterceptor
import akka.actor.typed.BehaviorSignalInterceptor
import akka.actor.typed.Signal
+import akka.actor.typed.TypedActorContext
import akka.annotation.InternalApi
/**
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/PropsImpl.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/PropsImpl.scala
index 5be43b12a6..de98b551d2 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/PropsImpl.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/PropsImpl.scala
@@ -4,8 +4,8 @@
package akka.actor.typed.internal
-import akka.actor.typed.ActorTags
import akka.actor.typed.{ DispatcherSelector, MailboxSelector, Props }
+import akka.actor.typed.ActorTags
import akka.annotation.InternalApi
/**
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/StashBufferImpl.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/StashBufferImpl.scala
index 0e5d16dcbe..47dab79f59 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/StashBufferImpl.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/StashBufferImpl.scala
@@ -6,10 +6,10 @@ package akka.actor.typed.internal
import java.util.function.{ Function => JFunction }
-import akka.actor.DeadLetter
-
import scala.annotation.tailrec
import scala.util.control.NonFatal
+
+import akka.actor.DeadLetter
import akka.actor.typed.Behavior
import akka.actor.typed.Signal
import akka.actor.typed.TypedActorContext
@@ -19,6 +19,7 @@ import akka.actor.typed.scaladsl.ActorContext
import akka.annotation.{ InternalApi, InternalStableApi }
import akka.japi.function.Procedure
import akka.util.{ unused, ConstantFun }
+import akka.util.OptionVal
/**
* INTERNAL API
@@ -47,6 +48,8 @@ import akka.util.{ unused, ConstantFun }
private var _size: Int = if (_first eq null) 0 else 1
+ private var currentBehaviorWhenUnstashInProgress: OptionVal[Behavior[T]] = OptionVal.None
+
override def isEmpty: Boolean = _first eq null
override def nonEmpty: Boolean = !isEmpty
@@ -75,6 +78,13 @@ import akka.util.{ unused, ConstantFun }
this
}
+ override def clear(): Unit = {
+ _first = null
+ _last = null
+ _size = 0
+ stashCleared(ctx)
+ }
+
@InternalStableApi
private def createNode(message: T, @unused ctx: scaladsl.ActorContext[T]): Node[T] = {
new Node(null, message)
@@ -128,15 +138,24 @@ import akka.util.{ unused, ConstantFun }
if (isEmpty)
behavior // optimization
else {
- val iter = new Iterator[Node[T]] {
- override def hasNext: Boolean = StashBufferImpl.this.nonEmpty
- override def next(): Node[T] = {
- val next = StashBufferImpl.this.dropHeadForUnstash()
- unstashed(ctx, next)
- next
- }
- }.take(math.min(numberOfMessages, size))
- interpretUnstashedMessages(behavior, ctx, iter, wrap)
+ // currentBehaviorWhenUnstashInProgress is needed to keep track of current Behavior for Behaviors.same
+ // when unstash is called when a previous unstash is already in progress (in same call stack)
+ val unstashAlreadyInProgress = currentBehaviorWhenUnstashInProgress.isDefined
+ try {
+ val iter = new Iterator[Node[T]] {
+ override def hasNext: Boolean = StashBufferImpl.this.nonEmpty
+
+ override def next(): Node[T] = {
+ val next = StashBufferImpl.this.dropHeadForUnstash()
+ unstashed(ctx, next)
+ next
+ }
+ }.take(math.min(numberOfMessages, size))
+ interpretUnstashedMessages(behavior, ctx, iter, wrap)
+ } finally {
+ if (!unstashAlreadyInProgress)
+ currentBehaviorWhenUnstashInProgress = OptionVal.None
+ }
}
}
@@ -147,6 +166,7 @@ import akka.util.{ unused, ConstantFun }
wrap: T => T): Behavior[T] = {
@tailrec def interpretOne(b: Behavior[T]): Behavior[T] = {
val b2 = Behavior.start(b, ctx)
+ currentBehaviorWhenUnstashInProgress = OptionVal.Some(b2)
if (!Behavior.isAlive(b2) || !messages.hasNext) b2
else {
val node = messages.next()
@@ -183,7 +203,10 @@ import akka.util.{ unused, ConstantFun }
if (Behavior.isUnhandled(started))
throw new IllegalArgumentException("Cannot unstash with unhandled as starting behavior")
else if (started == BehaviorImpl.same) {
- ctx.asScala.currentBehavior
+ currentBehaviorWhenUnstashInProgress match {
+ case OptionVal.None => ctx.asScala.currentBehavior
+ case OptionVal.Some(c) => c
+ }
} else started
if (Behavior.isAlive(actualInitialBehavior)) {
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/Supervision.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/Supervision.scala
index 699e509667..506c8b1966 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/Supervision.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/Supervision.scala
@@ -13,6 +13,8 @@ import scala.reflect.ClassTag
import scala.util.control.Exception.Catcher
import scala.util.control.NonFatal
+import org.slf4j.event.Level
+
import akka.actor.DeadLetterSuppression
import akka.actor.Dropped
import akka.actor.typed.BehaviorInterceptor.PreStartTarget
@@ -25,7 +27,6 @@ import akka.annotation.InternalApi
import akka.event.Logging
import akka.util.OptionVal
import akka.util.unused
-import org.slf4j.event.Level
/**
* INTERNAL API
@@ -195,7 +196,7 @@ private class RestartSupervisor[T, Thr <: Throwable: ClassTag](initial: Behavior
private def deadlineHasTimeLeft: Boolean = deadline match {
case OptionVal.None => true
- case OptionVal.Some(d) => d.hasTimeLeft
+ case OptionVal.Some(d) => d.hasTimeLeft()
}
override def aroundSignal(ctx: TypedActorContext[Any], signal: Signal, target: SignalTarget[T]): Behavior[T] = {
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/TimerSchedulerImpl.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/TimerSchedulerImpl.scala
index 7b8639d359..affbf8fa1d 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/TimerSchedulerImpl.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/TimerSchedulerImpl.scala
@@ -8,6 +8,9 @@ package internal
import java.time.Duration
import scala.concurrent.duration.FiniteDuration
+
+import org.slf4j.Logger
+
import akka.actor.Cancellable
import akka.actor.NotInfluenceReceiveTimeout
import akka.actor.typed.scaladsl.ActorContext
@@ -16,7 +19,6 @@ import akka.annotation.InternalApi
import akka.dispatch.ExecutionContexts
import akka.util.JavaDurationConverters._
import akka.util.OptionVal
-import org.slf4j.Logger
/**
* INTERNAL API
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/WithMdcBehaviorInterceptor.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/WithMdcBehaviorInterceptor.scala
index 0d0b426893..76effd7d69 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/WithMdcBehaviorInterceptor.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/WithMdcBehaviorInterceptor.scala
@@ -4,11 +4,12 @@
package akka.actor.typed.internal
-import akka.actor.typed.{ Behavior, BehaviorInterceptor, Signal, TypedActorContext }
-import akka.annotation.InternalApi
+import scala.reflect.ClassTag
+
import org.slf4j.MDC
-import scala.reflect.ClassTag
+import akka.actor.typed.{ Behavior, BehaviorInterceptor, Signal, TypedActorContext }
+import akka.annotation.InternalApi
/**
* INTERNAL API
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorAdapter.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorAdapter.scala
index ba7285606a..c57d90b3c2 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorAdapter.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorAdapter.scala
@@ -8,21 +8,21 @@ package adapter
import java.lang.reflect.InvocationTargetException
-import akka.actor.typed.internal.BehaviorImpl.DeferredBehavior
-import akka.actor.typed.internal.BehaviorImpl.StoppedBehavior
-import akka.actor.typed.internal.TimerSchedulerImpl.TimerMsg
-import akka.actor.typed.internal.adapter.ActorAdapter.TypedActorFailedException
-import akka.actor.ActorInitializationException
-import akka.actor.ActorRefWithCell
-import akka.annotation.InternalApi
-import akka.util.OptionVal
-import akka.{ actor => classic }
-
import scala.annotation.switch
import scala.annotation.tailrec
import scala.util.control.Exception.Catcher
import scala.util.control.NonFatal
+import akka.{ actor => classic }
+import akka.actor.ActorInitializationException
+import akka.actor.ActorRefWithCell
+import akka.actor.typed.internal.BehaviorImpl.DeferredBehavior
+import akka.actor.typed.internal.BehaviorImpl.StoppedBehavior
+import akka.actor.typed.internal.TimerSchedulerImpl.TimerMsg
+import akka.actor.typed.internal.adapter.ActorAdapter.TypedActorFailedException
+import akka.annotation.InternalApi
+import akka.util.OptionVal
+
/**
* INTERNAL API
*/
@@ -75,6 +75,7 @@ import scala.util.control.NonFatal
def receive: Receive = ActorAdapter.DummyReceive
override protected[akka] def aroundReceive(receive: Receive, msg: Any): Unit = {
+ ctx.setCurrentActorThread()
try {
// as we know we never become in "normal" typed actors, it is just the current behavior that
// changes, we can avoid some overhead with the partial function/behavior stack of untyped entirely
@@ -104,7 +105,10 @@ import scala.util.control.NonFatal
case msg: T @unchecked =>
handleMessage(msg)
}
- } finally ctx.clearMdc()
+ } finally {
+ ctx.clearCurrentActorThread()
+ ctx.clearMdc()
+ }
}
private def handleMessage(msg: T): Unit = {
@@ -206,32 +210,38 @@ import scala.util.control.NonFatal
}
override val supervisorStrategy = classic.OneForOneStrategy(loggingEnabled = false) {
- case TypedActorFailedException(cause) =>
- // These have already been optionally logged by typed supervision
- recordChildFailure(cause)
- classic.SupervisorStrategy.Stop
case ex =>
- val isTypedActor = sender() match {
- case afwc: ActorRefWithCell =>
- afwc.underlying.props.producer.actorClass == classOf[ActorAdapter[_]]
+ ctx.setCurrentActorThread()
+ try ex match {
+ case TypedActorFailedException(cause) =>
+ // These have already been optionally logged by typed supervision
+ recordChildFailure(cause)
+ classic.SupervisorStrategy.Stop
case _ =>
- false
- }
- recordChildFailure(ex)
- val logMessage = ex match {
- case e: ActorInitializationException if e.getCause ne null =>
- e.getCause match {
- case ex: InvocationTargetException if ex.getCause ne null => ex.getCause.getMessage
- case ex => ex.getMessage
+ val isTypedActor = sender() match {
+ case afwc: ActorRefWithCell =>
+ afwc.underlying.props.producer.actorClass == classOf[ActorAdapter[_]]
+ case _ =>
+ false
}
- case e => e.getMessage
+ recordChildFailure(ex)
+ val logMessage = ex match {
+ case e: ActorInitializationException if e.getCause ne null =>
+ e.getCause match {
+ case ex: InvocationTargetException if ex.getCause ne null => ex.getCause.getMessage
+ case ex => ex.getMessage
+ }
+ case e => e.getMessage
+ }
+ // log at Error as that is what the supervision strategy would have done.
+ ctx.log.error(logMessage, ex)
+ if (isTypedActor)
+ classic.SupervisorStrategy.Stop
+ else
+ ActorAdapter.classicSupervisorDecider(ex)
+ } finally {
+ ctx.clearCurrentActorThread()
}
- // log at Error as that is what the supervision strategy would have done.
- ctx.log.error(logMessage, ex)
- if (isTypedActor)
- classic.SupervisorStrategy.Stop
- else
- ActorAdapter.classicSupervisorDecider(ex)
}
private def recordChildFailure(ex: Throwable): Unit = {
@@ -241,6 +251,30 @@ import scala.util.control.NonFatal
}
}
+ override protected[akka] def aroundPreStart(): Unit = {
+ ctx.setCurrentActorThread()
+ try super.aroundPreStart()
+ finally ctx.clearCurrentActorThread()
+ }
+
+ override protected[akka] def aroundPreRestart(reason: Throwable, message: Option[Any]): Unit = {
+ ctx.setCurrentActorThread()
+ try super.aroundPreRestart(reason, message)
+ finally ctx.clearCurrentActorThread()
+ }
+
+ override protected[akka] def aroundPostRestart(reason: Throwable): Unit = {
+ ctx.setCurrentActorThread()
+ try super.aroundPostRestart(reason)
+ finally ctx.clearCurrentActorThread()
+ }
+
+ override protected[akka] def aroundPostStop(): Unit = {
+ ctx.setCurrentActorThread()
+ try super.aroundPostStop()
+ finally ctx.clearCurrentActorThread()
+ }
+
override def preStart(): Unit = {
try {
if (Behavior.isAlive(behavior)) {
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorContextAdapter.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorContextAdapter.scala
index 38aca106e5..b29a44111d 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorContextAdapter.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorContextAdapter.scala
@@ -6,12 +6,12 @@ package akka.actor.typed
package internal
package adapter
-import akka.annotation.InternalApi
-
-import akka.{ actor => classic }
import scala.concurrent.ExecutionContextExecutor
import scala.concurrent.duration._
+import akka.{ actor => classic }
+import akka.annotation.InternalApi
+
@InternalApi
private[akka] object ActorContextAdapter {
@@ -58,13 +58,26 @@ private[akka] object ActorContextAdapter {
final override val self = ActorRefAdapter(classicContext.self)
final override val system = ActorSystemAdapter(classicContext.system)
private[akka] def classicActorContext = classicContext
- override def children: Iterable[ActorRef[Nothing]] = classicContext.children.map(ActorRefAdapter(_))
- override def child(name: String): Option[ActorRef[Nothing]] = classicContext.child(name).map(ActorRefAdapter(_))
- override def spawnAnonymous[U](behavior: Behavior[U], props: Props = Props.empty): ActorRef[U] =
+ override def children: Iterable[ActorRef[Nothing]] = {
+ checkCurrentActorThread()
+ classicContext.children.map(ActorRefAdapter(_))
+ }
+ override def child(name: String): Option[ActorRef[Nothing]] = {
+ checkCurrentActorThread()
+ classicContext.child(name).map(ActorRefAdapter(_))
+ }
+ override def spawnAnonymous[U](behavior: Behavior[U], props: Props = Props.empty): ActorRef[U] = {
+ checkCurrentActorThread()
ActorRefFactoryAdapter.spawnAnonymous(classicContext, behavior, props, rethrowTypedFailure = true)
- override def spawn[U](behavior: Behavior[U], name: String, props: Props = Props.empty): ActorRef[U] =
+ }
+
+ override def spawn[U](behavior: Behavior[U], name: String, props: Props = Props.empty): ActorRef[U] = {
+ checkCurrentActorThread()
ActorRefFactoryAdapter.spawn(classicContext, behavior, name, props, rethrowTypedFailure = true)
- override def stop[U](child: ActorRef[U]): Unit =
+ }
+
+ override def stop[U](child: ActorRef[U]): Unit = {
+ checkCurrentActorThread()
if (child.path.parent == self.path) { // only if a direct child
toClassic(child) match {
case f: akka.actor.FunctionRef =>
@@ -90,16 +103,29 @@ private[akka] object ActorContextAdapter {
s"but [$child] is not a child of [$self]. Stopping other actors has to be expressed as " +
"an explicit stop message that the actor accepts.")
}
+ }
- override def watch[U](other: ActorRef[U]): Unit = { classicContext.watch(toClassic(other)) }
- override def watchWith[U](other: ActorRef[U], msg: T): Unit = { classicContext.watchWith(toClassic(other), msg) }
- override def unwatch[U](other: ActorRef[U]): Unit = { classicContext.unwatch(toClassic(other)) }
+ override def watch[U](other: ActorRef[U]): Unit = {
+ checkCurrentActorThread()
+ classicContext.watch(toClassic(other))
+ }
+ override def watchWith[U](other: ActorRef[U], msg: T): Unit = {
+ checkCurrentActorThread()
+ classicContext.watchWith(toClassic(other), msg)
+ }
+ override def unwatch[U](other: ActorRef[U]): Unit = {
+ checkCurrentActorThread()
+ classicContext.unwatch(toClassic(other))
+ }
var receiveTimeoutMsg: T = null.asInstanceOf[T]
override def setReceiveTimeout(d: FiniteDuration, msg: T): Unit = {
+ checkCurrentActorThread()
receiveTimeoutMsg = msg
classicContext.setReceiveTimeout(d)
}
override def cancelReceiveTimeout(): Unit = {
+ checkCurrentActorThread()
+
receiveTimeoutMsg = null.asInstanceOf[T]
classicContext.setReceiveTimeout(Duration.Undefined)
}
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorRefAdapter.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorRefAdapter.scala
index 33553620a1..5bd523bec7 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorRefAdapter.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorRefAdapter.scala
@@ -6,9 +6,9 @@ package akka.actor.typed
package internal
package adapter
+import akka.{ actor => classic }
import akka.actor.ActorRefProvider
import akka.actor.InvalidMessageException
-import akka.{ actor => classic }
import akka.annotation.InternalApi
import akka.dispatch.sysmsg
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorSystemAdapter.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorSystemAdapter.scala
index 478a8409a1..d5318e3c45 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorSystemAdapter.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorSystemAdapter.scala
@@ -8,6 +8,10 @@ import java.util.concurrent.CompletionStage
import scala.compat.java8.FutureConverters
import scala.concurrent.ExecutionContextExecutor
+
+import org.slf4j.{ Logger, LoggerFactory }
+
+import akka.{ actor => classic }
import akka.Done
import akka.actor
import akka.actor.{ ActorRefProvider, Address, ExtendedActorSystem, InvalidMessageException }
@@ -29,8 +33,6 @@ import akka.actor.typed.internal.PropsImpl.DispatcherSameAsParent
import akka.actor.typed.internal.SystemMessage
import akka.actor.typed.scaladsl.Behaviors
import akka.annotation.InternalApi
-import akka.{ actor => classic }
-import org.slf4j.{ Logger, LoggerFactory }
/**
* INTERNAL API. Lightweight wrapper for presenting a classic ActorSystem to a Behavior (via the context).
@@ -51,7 +53,7 @@ import org.slf4j.{ Logger, LoggerFactory }
import ActorRefAdapter.sendSystemMessage
- override private[akka] def classicSystem: classic.ActorSystem = system
+ override def classicSystem: classic.ActorSystem = system
// Members declared in akka.actor.typed.ActorRef
override def tell(msg: T): Unit = {
@@ -155,12 +157,5 @@ private[akka] object ActorSystemAdapter {
new LoadTypedExtensions(system)
}
- def toClassic[U](sys: ActorSystem[_]): classic.ActorSystem =
- sys match {
- case adapter: ActorSystemAdapter[_] => adapter.classicSystem
- case _ =>
- throw new UnsupportedOperationException(
- "Only adapted classic ActorSystem permissible " +
- s"($sys of class ${sys.getClass.getName})")
- }
+ def toClassic[U](sys: ActorSystem[_]): classic.ActorSystem = sys.classicSystem
}
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/SchedulerAdapter.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/SchedulerAdapter.scala
index 68cb4cbdd4..5f90975fae 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/SchedulerAdapter.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/SchedulerAdapter.scala
@@ -6,13 +6,13 @@ package akka.actor.typed.internal.adapter
import java.time.Duration
+import scala.concurrent.ExecutionContext
+import scala.concurrent.duration.FiniteDuration
+
import akka.actor.Cancellable
import akka.actor.typed.Scheduler
import akka.annotation.InternalApi
-import scala.concurrent.ExecutionContext
-import scala.concurrent.duration.FiniteDuration
-
/**
* INTERNAL API
*/
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/pubsub/TopicImpl.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/pubsub/TopicImpl.scala
index d3e276287e..dfa1bb00cd 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/pubsub/TopicImpl.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/pubsub/TopicImpl.scala
@@ -4,6 +4,8 @@
package akka.actor.typed.internal.pubsub
+import scala.reflect.ClassTag
+
import akka.actor.Dropped
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
@@ -16,8 +18,6 @@ import akka.actor.typed.scaladsl.LoggerOps
import akka.actor.typed.scaladsl.adapter._
import akka.annotation.InternalApi
-import scala.reflect.ClassTag
-
/**
* INTERNAL API
*/
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/receptionist/ReceptionistMessages.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/receptionist/ReceptionistMessages.scala
index 76d9613963..92c507da6a 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/receptionist/ReceptionistMessages.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/receptionist/ReceptionistMessages.scala
@@ -5,8 +5,8 @@
package akka.actor.typed.internal.receptionist
import akka.actor.typed.ActorRef
-import akka.actor.typed.receptionist.Receptionist.Command
import akka.actor.typed.receptionist.{ Receptionist, ServiceKey }
+import akka.actor.typed.receptionist.Receptionist.Command
import akka.annotation.InternalApi
import akka.util.ccompat.JavaConverters._
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/AbstractBehavior.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/AbstractBehavior.scala
index 7ccc276c0e..23a7dfc2a3 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/AbstractBehavior.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/AbstractBehavior.scala
@@ -50,12 +50,13 @@ abstract class AbstractBehavior[T](context: ActorContext[T]) extends ExtensibleB
protected def getContext: ActorContext[T] = context
- private def checkRightContext(ctx: TypedActorContext[T]): Unit =
+ private def checkRightContext(ctx: TypedActorContext[T]): Unit = {
if (ctx.asJava ne context)
throw new IllegalStateException(
s"Actor [${ctx.asJava.getSelf}] of AbstractBehavior class " +
s"[${getClass.getName}] was created with wrong ActorContext [${context.asJava.getSelf}]. " +
"Wrap in Behaviors.setup and pass the context to the constructor of AbstractBehavior.")
+ }
@throws(classOf[Exception])
override final def receive(ctx: TypedActorContext[T], msg: T): Behavior[T] = {
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/ActorContext.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/ActorContext.scala
index 30b36af6f3..87eab9f4dd 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/ActorContext.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/ActorContext.scala
@@ -5,16 +5,16 @@
package akka.actor.typed.javadsl
import java.time.Duration
-
-import akka.annotation.DoNotInherit
-import akka.actor.ClassicActorContextProvider
-import akka.actor.typed._
import java.util.Optional
import java.util.concurrent.CompletionStage
+import scala.concurrent.ExecutionContextExecutor
+
import org.slf4j.Logger
-import scala.concurrent.ExecutionContextExecutor
+import akka.actor.ClassicActorContextProvider
+import akka.actor.typed._
+import akka.annotation.DoNotInherit
/**
* An Actor is given by the combination of a [[Behavior]] and a context in
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/Adapter.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/Adapter.scala
index 530ddbd335..47e0e8732a 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/Adapter.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/Adapter.scala
@@ -5,14 +5,14 @@
package akka.actor.typed.javadsl
import akka.actor
+import akka.actor.typed.ActorRef
+import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.Props
-import akka.actor.typed.ActorRef
-import akka.actor.typed.scaladsl.adapter._
-import akka.actor.typed.ActorSystem
import akka.actor.typed.Scheduler
import akka.actor.typed.SupervisorStrategy
import akka.actor.typed.internal.adapter.ActorContextAdapter
+import akka.actor.typed.scaladsl.adapter._
import akka.japi.Creator
/**
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/AskPattern.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/AskPattern.scala
index fb9fae6c90..22547cd992 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/AskPattern.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/AskPattern.scala
@@ -8,13 +8,13 @@ package javadsl
import java.time.Duration
import java.util.concurrent.CompletionStage
+import scala.compat.java8.FutureConverters._
+
import akka.actor.typed.Scheduler
import akka.actor.typed.scaladsl.AskPattern._
import akka.japi.function.{ Function => JFunction }
import akka.util.JavaDurationConverters._
-import scala.compat.java8.FutureConverters._
-
/**
* The ask-pattern implements the initiator side of a request–reply protocol.
*
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/BehaviorBuilder.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/BehaviorBuilder.scala
index cd3d182487..470e371c3e 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/BehaviorBuilder.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/BehaviorBuilder.scala
@@ -6,15 +6,16 @@ package akka.actor.typed.javadsl
import scala.annotation.tailrec
-import akka.japi.function.{ Function => JFunction }
-import akka.japi.function.Creator
-import akka.japi.function.{ Predicate => JPredicate }
-import akka.annotation.InternalApi
+import BehaviorBuilder._
+
import akka.actor.typed.Behavior
import akka.actor.typed.ExtensibleBehavior
import akka.actor.typed.Signal
import akka.actor.typed.TypedActorContext
-import BehaviorBuilder._
+import akka.annotation.InternalApi
+import akka.japi.function.{ Function => JFunction }
+import akka.japi.function.{ Predicate => JPredicate }
+import akka.japi.function.Creator
import akka.util.OptionVal
/**
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/Behaviors.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/Behaviors.scala
index 935621039d..301b492ef1 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/Behaviors.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/Behaviors.scala
@@ -7,6 +7,8 @@ package akka.actor.typed.javadsl
import java.util.Collections
import java.util.function.{ Supplier, Function => JFunction }
+import scala.reflect.ClassTag
+
import akka.actor.typed._
import akka.actor.typed.internal.{
BehaviorImpl,
@@ -17,10 +19,8 @@ import akka.actor.typed.internal.{
}
import akka.japi.function.{ Effect, Function2 => JapiFunction2 }
import akka.japi.pf.PFBuilder
-import akka.util.unused
import akka.util.ccompat.JavaConverters._
-
-import scala.reflect.ClassTag
+import akka.util.unused
/**
* Factories for [[akka.actor.typed.Behavior]].
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/ReceiveBuilder.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/ReceiveBuilder.scala
index 2c811a86cb..1ef3509e80 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/ReceiveBuilder.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/ReceiveBuilder.scala
@@ -4,14 +4,14 @@
package akka.actor.typed.javadsl
-import akka.actor.typed.MessageAdaptionFailure
-
import scala.annotation.tailrec
-import akka.japi.function.Creator
+
+import akka.actor.typed.{ Behavior, Signal }
+import akka.actor.typed.MessageAdaptionFailure
+import akka.annotation.InternalApi
import akka.japi.function.{ Function => JFunction }
import akka.japi.function.{ Predicate => JPredicate }
-import akka.actor.typed.{ Behavior, Signal }
-import akka.annotation.InternalApi
+import akka.japi.function.Creator
import akka.util.OptionVal
/**
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/StashBuffer.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/StashBuffer.scala
index 5c9e97f975..fac124ea41 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/StashBuffer.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/StashBuffer.scala
@@ -42,6 +42,13 @@ import akka.japi.function.Procedure
*/
def size: Int
+ /**
+ * What is the capacity of this buffer.
+ *
+ * @return the capacity of this buffer
+ */
+ def capacity: Int
+
/**
* @return `true` if no more messages can be added, i.e. size equals the capacity of the stash buffer
*/
@@ -74,6 +81,11 @@ import akka.japi.function.Procedure
*/
def forEach(f: Procedure[T]): Unit
+ /**
+ * Removes all messages from the buffer.
+ */
+ def clear(): Unit
+
/**
* Process all stashed messages with the `behavior` and the returned
* [[Behavior]] from each processed message. The `StashBuffer` will be
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/pubsub/Topic.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/pubsub/Topic.scala
index e88dc70cff..043ab7f295 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/pubsub/Topic.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/pubsub/Topic.scala
@@ -4,14 +4,14 @@
package akka.actor.typed.pubsub
+import scala.reflect.ClassTag
+
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
import akka.actor.typed.internal.pubsub.TopicImpl
import akka.actor.typed.scaladsl.Behaviors
import akka.annotation.DoNotInherit
-import scala.reflect.ClassTag
-
/**
* A pub sub topic is an actor that handles subscribing to a topic and publishing messages to all subscribed actors.
*
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/receptionist/Receptionist.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/receptionist/Receptionist.scala
index 34794597fc..f4ab958ced 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/receptionist/Receptionist.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/receptionist/Receptionist.scala
@@ -4,12 +4,12 @@
package akka.actor.typed.receptionist
+import scala.reflect.ClassTag
+
import akka.actor.typed.{ ActorRef, ActorSystem, Extension, ExtensionId, ExtensionSetup }
import akka.actor.typed.internal.receptionist._
import akka.annotation.DoNotInherit
-
import akka.util.ccompat.JavaConverters._
-import scala.reflect.ClassTag
/**
* Register and discover actors that implement a service with a protocol defined by a [[ServiceKey]].
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/AbstractBehavior.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/AbstractBehavior.scala
index c0a2c32d05..d830cd4eb1 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/AbstractBehavior.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/AbstractBehavior.scala
@@ -4,8 +4,8 @@
package akka.actor.typed.scaladsl
-import akka.actor.typed.MessageAdaptionFailure
import akka.actor.typed.{ Behavior, ExtensibleBehavior, Signal, TypedActorContext }
+import akka.actor.typed.MessageAdaptionFailure
/**
* An actor `Behavior` can be implemented by extending this class and implement the
@@ -70,12 +70,13 @@ abstract class AbstractBehavior[T](protected val context: ActorContext[T]) exten
@throws(classOf[Exception])
def onSignal: PartialFunction[Signal, Behavior[T]] = PartialFunction.empty
- private def checkRightContext(ctx: TypedActorContext[T]): Unit =
+ private def checkRightContext(ctx: TypedActorContext[T]): Unit = {
if (ctx.asJava ne context)
throw new IllegalStateException(
s"Actor [${ctx.asJava.getSelf}] of AbstractBehavior class " +
s"[${getClass.getName}] was created with wrong ActorContext [${context.asJava.getSelf}]. " +
"Wrap in Behaviors.setup and pass the context to the constructor of AbstractBehavior.")
+ }
@throws(classOf[Exception])
override final def receive(ctx: TypedActorContext[T], msg: T): Behavior[T] = {
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/ActorContext.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/ActorContext.scala
index 587ce1c437..86dfb5f1d2 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/ActorContext.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/ActorContext.scala
@@ -4,18 +4,19 @@
package akka.actor.typed.scaladsl
-import akka.actor.ClassicActorContextProvider
-import akka.actor.typed._
-import akka.annotation.DoNotInherit
-import akka.util.Timeout
-
import scala.concurrent.{ ExecutionContextExecutor, Future }
import scala.concurrent.duration.FiniteDuration
import scala.reflect.ClassTag
import scala.util.Try
-import akka.annotation.InternalApi
+
import org.slf4j.Logger
+import akka.actor.ClassicActorContextProvider
+import akka.actor.typed._
+import akka.annotation.DoNotInherit
+import akka.annotation.InternalApi
+import akka.util.Timeout
+
/**
* An Actor is given by the combination of a [[Behavior]] and a context in
* which this behavior is executed. As per the Actor Model an Actor can perform
@@ -336,4 +337,19 @@ trait ActorContext[T] extends TypedActorContext[T] with ClassicActorContextProvi
@InternalApi
private[akka] def clearMdc(): Unit
+ /**
+ * INTERNAL API
+ */
+ @InternalApi private[akka] def setCurrentActorThread(): Unit
+
+ /**
+ * INTERNAL API
+ */
+ @InternalApi private[akka] def clearCurrentActorThread(): Unit
+
+ /**
+ * INTERNAL API
+ */
+ @InternalApi private[akka] def checkCurrentActorThread(): Unit
+
}
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/AskPattern.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/AskPattern.scala
index 9e29e8ac78..5894b194e1 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/AskPattern.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/AskPattern.scala
@@ -7,17 +7,19 @@ package akka.actor.typed.scaladsl
import java.util.concurrent.TimeoutException
import scala.concurrent.Future
+
+import com.github.ghik.silencer.silent
+
import akka.actor.{ Address, RootActorPath }
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
+import akka.actor.typed.RecipientRef
import akka.actor.typed.Scheduler
import akka.actor.typed.internal.{ adapter => adapt }
-import akka.annotation.InternalApi
-import akka.pattern.PromiseActorRef
-import akka.util.Timeout
-import akka.actor.typed.RecipientRef
import akka.actor.typed.internal.InternalRecipientRef
-import com.github.ghik.silencer.silent
+import akka.annotation.{ InternalApi, InternalStableApi }
+import akka.pattern.PromiseActorRef
+import akka.util.{ unused, Timeout }
/**
* The ask-pattern implements the initiator side of a request–reply protocol.
@@ -144,14 +146,19 @@ object AskPattern {
val ref: ActorRef[U] = _ref
val future: Future[U] = _future
val promiseRef: PromiseActorRef = _promiseRef
+
+ @InternalStableApi
+ private[akka] def ask[T](target: InternalRecipientRef[T], message: T, @unused timeout: Timeout): Future[U] = {
+ target ! message
+ future
+ }
}
private def askClassic[T, U](target: InternalRecipientRef[T], timeout: Timeout, f: ActorRef[U] => T): Future[U] = {
val p = new PromiseRef[U](target, timeout)
val m = f(p.ref)
if (p.promiseRef ne null) p.promiseRef.messageClassName = m.getClass.getName
- target ! m
- p.future
+ p.ask(target, m, timeout)
}
/**
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/Behaviors.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/Behaviors.scala
index 80eea36d4b..df7ffa8b0b 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/Behaviors.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/Behaviors.scala
@@ -5,11 +5,11 @@
package akka.actor.typed
package scaladsl
-import akka.annotation.{ DoNotInherit, InternalApi }
-import akka.actor.typed.internal._
-
import scala.reflect.{ classTag, ClassTag }
+import akka.actor.typed.internal._
+import akka.annotation.{ DoNotInherit, InternalApi }
+
/**
* Factories for [[akka.actor.typed.Behavior]].
*/
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/StashBuffer.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/StashBuffer.scala
index 33ae4a2f09..add12b5dac 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/StashBuffer.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/StashBuffer.scala
@@ -55,6 +55,13 @@ import akka.annotation.{ DoNotInherit, InternalApi }
*/
def size: Int
+ /**
+ * What is the capacity of this buffer.
+ *
+ * @return the capacity of this buffer
+ */
+ def capacity: Int
+
/**
* @return `true` if no more messages can be added, i.e. size equals the capacity of the stash buffer
*/
@@ -85,6 +92,11 @@ import akka.annotation.{ DoNotInherit, InternalApi }
*/
def foreach(f: T => Unit): Unit
+ /**
+ * Removes all messages from the buffer.
+ */
+ def clear(): Unit
+
/**
* Process all stashed messages with the `behavior` and the returned
* [[Behavior]] from each processed message. The `StashBuffer` will be
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/adapter/package.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/adapter/package.scala
index fc259448b6..2cfe0f0987 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/adapter/package.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/adapter/package.scala
@@ -74,7 +74,7 @@ package object adapter {
* Extension methods added to [[akka.actor.typed.ActorSystem]].
*/
implicit class TypedActorSystemOps(val sys: ActorSystem[_]) extends AnyVal {
- def toClassic: akka.actor.ActorSystem = ActorSystemAdapter.toClassic(sys)
+ def toClassic: akka.actor.ActorSystem = sys.classicSystem
/**
* INTERNAL API
diff --git a/akka-actor/src/main/java/akka/annotation/InternalStableApi.java b/akka-actor/src/main/java/akka/annotation/InternalStableApi.java
index b3e938874f..5078ae7763 100644
--- a/akka-actor/src/main/java/akka/annotation/InternalStableApi.java
+++ b/akka-actor/src/main/java/akka/annotation/InternalStableApi.java
@@ -11,8 +11,15 @@ import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
/**
- * Marks internal members that shouldn't be changed without considering possible usage outside of
- * the Akka core modules.
+ * Marks APIs that are considered internal to Akka and should not be accessed by user code but that
+ * are used across Akka project boundaries and therefore shouldn't be changed without considering
+ * possible usage outside of the Akka core modules.
+ *
+ * If a method/class annotated with this annotation is part of a public API and has the Scala
+ * {@code private[akka]} access restriction, which leads to a public method from Java, there should
+ * be a javadoc/scaladoc comment where the first line MUST include {@code INTERNAL API} in order to
+ * be easily identifiable from generated documentation. Additional information may be put on the
+ * same line as the INTERNAL API comment in order to clarify further.
*/
@Documented
@Retention(RetentionPolicy.CLASS) // to be accessible by MiMa
diff --git a/akka-actor/src/main/java/akka/dispatch/affinity/OnSpinWait.java b/akka-actor/src/main/java/akka/dispatch/affinity/OnSpinWait.java
new file mode 100644
index 0000000000..dd6d44b050
--- /dev/null
+++ b/akka-actor/src/main/java/akka/dispatch/affinity/OnSpinWait.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright (C) 2009-2020 Lightbend Inc.
+ */
+
+package akka.dispatch.affinity;
+
+import akka.util.Unsafe;
+import java.lang.invoke.MethodHandle;
+import java.lang.invoke.MethodHandles;
+import akka.annotation.InternalApi;
+import static java.lang.invoke.MethodType.methodType;
+
+/**
+ * INTERNAL API
+ */
+@InternalApi
+final class OnSpinWait {
+ private final static MethodHandle handle;
+
+ public final static void spinWait() throws Throwable {
+ handle.invoke(); // Will be inlined as an invokeExact since the callsite matches the MH definition of () -> void
+ }
+
+ static {
+ final MethodHandle noop = MethodHandles.constant(Object.class, null).asType(methodType(Void.TYPE));
+ MethodHandle impl;
+ try {
+ impl = MethodHandles.lookup().findStatic(Thread.class, "onSpinWait", methodType(Void.TYPE));
+ } catch (NoSuchMethodException nsme) {
+ impl = noop;
+ } catch (IllegalAccessException iae) {
+ impl = noop;
+ }
+ handle = impl;
+ };
+}
\ No newline at end of file
diff --git a/akka-actor/src/main/mima-filters/2.6.4.backwards.excludes/kernel-legacy.excludes b/akka-actor/src/main/mima-filters/2.6.4.backwards.excludes/kernel-legacy.excludes
new file mode 100644
index 0000000000..cfa9e9317c
--- /dev/null
+++ b/akka-actor/src/main/mima-filters/2.6.4.backwards.excludes/kernel-legacy.excludes
@@ -0,0 +1,5 @@
+# leftover from microkernel, removed in Akka 2.4
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorSystem.GlobalHome")
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorSystem.SystemHome")
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorSystem.EnvHome")
+
diff --git a/akka-actor/src/main/mima-filters/2.6.5.backwards.excludes/27614-no-reflection-in-actorcell.excludes b/akka-actor/src/main/mima-filters/2.6.5.backwards.excludes/27614-no-reflection-in-actorcell.excludes
new file mode 100644
index 0000000000..e131db5de0
--- /dev/null
+++ b/akka-actor/src/main/mima-filters/2.6.5.backwards.excludes/27614-no-reflection-in-actorcell.excludes
@@ -0,0 +1,7 @@
+# #25040 changes to ActorCell internals
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorCell.setActorFields")
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorCell.clearActorCellFields")
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorCell.actor_=")
+ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.dungeon.FaultHandling.akka$actor$dungeon$FaultHandling$$_failed")
+ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.dungeon.FaultHandling.akka$actor$dungeon$FaultHandling$$_failed_=")
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.util.Reflect.lookupAndSetField")
diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf
index 9d3ba13270..76f4eceef8 100644
--- a/akka-actor/src/main/resources/reference.conf
+++ b/akka-actor/src/main/resources/reference.conf
@@ -73,7 +73,7 @@ akka {
#
# Should not be set by end user applications in 'application.conf', use the extensions property for that
#
- library-extensions = ${?akka.library-extensions} ["akka.serialization.SerializationExtension"]
+ library-extensions = ${?akka.library-extensions} ["akka.serialization.SerializationExtension$"]
# List FQCN of extensions which shall be loaded at actor system startup.
# Should be on the format: 'extensions = ["foo", "bar"]' etc.
diff --git a/akka-actor/src/main/scala-2.12/akka/compat/Future.scala b/akka-actor/src/main/scala-2.12/akka/compat/Future.scala
index 3549b360e3..5d733b8acb 100644
--- a/akka-actor/src/main/scala-2.12/akka/compat/Future.scala
+++ b/akka-actor/src/main/scala-2.12/akka/compat/Future.scala
@@ -4,13 +4,14 @@
package akka.compat
-import akka.annotation.InternalApi
-
-import scala.concurrent.{ ExecutionContext, Future => SFuture }
import scala.collection.immutable
-import akka.util.ccompat._
+import scala.concurrent.{ ExecutionContext, Future => SFuture }
+
import com.github.ghik.silencer.silent
+import akka.annotation.InternalApi
+import akka.util.ccompat._
+
/**
* INTERNAL API
*
diff --git a/akka-actor/src/main/scala-2.13-/akka/dispatch/internal/SameThreadExecutionContext.scala b/akka-actor/src/main/scala-2.12/akka/dispatch/internal/SameThreadExecutionContext.scala
similarity index 100%
rename from akka-actor/src/main/scala-2.13-/akka/dispatch/internal/SameThreadExecutionContext.scala
rename to akka-actor/src/main/scala-2.12/akka/dispatch/internal/SameThreadExecutionContext.scala
index 2f27fb99ad..38aff64431 100644
--- a/akka-actor/src/main/scala-2.13-/akka/dispatch/internal/SameThreadExecutionContext.scala
+++ b/akka-actor/src/main/scala-2.12/akka/dispatch/internal/SameThreadExecutionContext.scala
@@ -4,11 +4,11 @@
package akka.dispatch.internal
+import scala.concurrent.ExecutionContext
+
import akka.annotation.InternalApi
import akka.dispatch.BatchingExecutor
-import scala.concurrent.ExecutionContext
-
/**
* Factory to create same thread ec. Not intended to be called from any other site than to create [[akka.dispatch.ExecutionContexts#parasitic]]
*
diff --git a/akka-actor/src/main/scala-2.12/akka/util/ByteIterator.scala b/akka-actor/src/main/scala-2.12/akka/util/ByteIterator.scala
index aaac46b5d5..da0c4eb918 100644
--- a/akka-actor/src/main/scala-2.12/akka/util/ByteIterator.scala
+++ b/akka-actor/src/main/scala-2.12/akka/util/ByteIterator.scala
@@ -4,8 +4,6 @@
package akka.util
-import akka.util.Collections.EmptyImmutableSeq
-
import java.nio.{ ByteBuffer, ByteOrder }
import scala.annotation.tailrec
@@ -13,6 +11,8 @@ import scala.collection.LinearSeq
import scala.collection.mutable.ListBuffer
import scala.reflect.ClassTag
+import akka.util.Collections.EmptyImmutableSeq
+
object ByteIterator {
object ByteArrayIterator {
diff --git a/akka-actor/src/main/scala-2.13-/akka/util/ByteString.scala b/akka-actor/src/main/scala-2.12/akka/util/ByteString.scala
similarity index 100%
rename from akka-actor/src/main/scala-2.13-/akka/util/ByteString.scala
rename to akka-actor/src/main/scala-2.12/akka/util/ByteString.scala
index 82a3885592..9a6333b39c 100644
--- a/akka-actor/src/main/scala-2.13-/akka/util/ByteString.scala
+++ b/akka-actor/src/main/scala-2.12/akka/util/ByteString.scala
@@ -5,18 +5,18 @@
package akka.util
import java.io.{ ObjectInputStream, ObjectOutputStream }
-import java.nio.{ ByteBuffer, ByteOrder }
import java.lang.{ Iterable => JIterable }
+import java.nio.{ ByteBuffer, ByteOrder }
+import java.nio.charset.{ Charset, StandardCharsets }
import java.util.Base64
import scala.annotation.{ tailrec, varargs }
import scala.collection.IndexedSeqOptimized
-import scala.collection.mutable.{ Builder, WrappedArray }
+import scala.collection.generic.CanBuildFrom
import scala.collection.immutable
import scala.collection.immutable.{ IndexedSeq, VectorBuilder }
-import scala.collection.generic.CanBuildFrom
+import scala.collection.mutable.{ Builder, WrappedArray }
import scala.reflect.ClassTag
-import java.nio.charset.{ Charset, StandardCharsets }
object ByteString {
diff --git a/akka-actor/src/main/scala-2.13-/akka/util/ccompat/CompatImpl.scala b/akka-actor/src/main/scala-2.12/akka/util/ccompat/CompatImpl.scala
similarity index 100%
rename from akka-actor/src/main/scala-2.13-/akka/util/ccompat/CompatImpl.scala
rename to akka-actor/src/main/scala-2.12/akka/util/ccompat/CompatImpl.scala
diff --git a/akka-actor/src/main/scala-2.13+/akka/util/ccompat/ccompatUsedUntil213.scala b/akka-actor/src/main/scala-2.12/akka/util/ccompat/ccompatUsedUntil213.scala
similarity index 100%
rename from akka-actor/src/main/scala-2.13+/akka/util/ccompat/ccompatUsedUntil213.scala
rename to akka-actor/src/main/scala-2.12/akka/util/ccompat/ccompatUsedUntil213.scala
diff --git a/akka-actor/src/main/scala-2.13-/akka/util/ccompat/package.scala b/akka-actor/src/main/scala-2.12/akka/util/ccompat/package.scala
similarity index 100%
rename from akka-actor/src/main/scala-2.13-/akka/util/ccompat/package.scala
rename to akka-actor/src/main/scala-2.12/akka/util/ccompat/package.scala
index 68b492ea11..d60e398756 100644
--- a/akka-actor/src/main/scala-2.13-/akka/util/ccompat/package.scala
+++ b/akka-actor/src/main/scala-2.12/akka/util/ccompat/package.scala
@@ -4,11 +4,11 @@
package akka.util
-import scala.language.implicitConversions
-import scala.language.higherKinds
-import scala.collection.{ GenTraversable, immutable => i, mutable => m }
import scala.{ collection => c }
+import scala.collection.{ GenTraversable, immutable => i, mutable => m }
import scala.collection.generic.{ CanBuildFrom, GenericCompanion, Sorted, SortedSetFactory }
+import scala.language.higherKinds
+import scala.language.implicitConversions
/**
* INTERNAL API
diff --git a/akka-actor/src/main/scala-2.13/akka/compat/Future.scala b/akka-actor/src/main/scala-2.13/akka/compat/Future.scala
index baf9ddd557..6cae239b68 100644
--- a/akka-actor/src/main/scala-2.13/akka/compat/Future.scala
+++ b/akka-actor/src/main/scala-2.13/akka/compat/Future.scala
@@ -4,9 +4,10 @@
package akka.compat
-import akka.annotation.InternalApi
-import scala.concurrent.{ ExecutionContext, Future => SFuture }
import scala.collection.immutable
+import scala.concurrent.{ ExecutionContext, Future => SFuture }
+
+import akka.annotation.InternalApi
/**
* INTERNAL API
diff --git a/akka-actor/src/main/scala-2.13+/akka/dispatch/internal/SameThreadExecutionContext.scala b/akka-actor/src/main/scala-2.13/akka/dispatch/internal/SameThreadExecutionContext.scala
similarity index 100%
rename from akka-actor/src/main/scala-2.13+/akka/dispatch/internal/SameThreadExecutionContext.scala
rename to akka-actor/src/main/scala-2.13/akka/dispatch/internal/SameThreadExecutionContext.scala
index 005b0ee59d..5599c7fdd4 100644
--- a/akka-actor/src/main/scala-2.13+/akka/dispatch/internal/SameThreadExecutionContext.scala
+++ b/akka-actor/src/main/scala-2.13/akka/dispatch/internal/SameThreadExecutionContext.scala
@@ -4,10 +4,10 @@
package akka.dispatch.internal
-import akka.annotation.InternalApi
-
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]]
*
diff --git a/akka-actor/src/main/scala-2.13+/akka/util/ByteIterator.scala b/akka-actor/src/main/scala-2.13/akka/util/ByteIterator.scala
similarity index 99%
rename from akka-actor/src/main/scala-2.13+/akka/util/ByteIterator.scala
rename to akka-actor/src/main/scala-2.13/akka/util/ByteIterator.scala
index 4f15f7e147..b5252803bc 100644
--- a/akka-actor/src/main/scala-2.13+/akka/util/ByteIterator.scala
+++ b/akka-actor/src/main/scala-2.13/akka/util/ByteIterator.scala
@@ -4,8 +4,6 @@
package akka.util
-import akka.util.Collections.EmptyImmutableSeq
-
import java.nio.{ ByteBuffer, ByteOrder }
import scala.annotation.tailrec
@@ -14,6 +12,8 @@ import scala.collection.LinearSeq
import scala.collection.mutable.ListBuffer
import scala.reflect.ClassTag
+import akka.util.Collections.EmptyImmutableSeq
+
object ByteIterator {
object ByteArrayIterator {
@@ -168,7 +168,7 @@ object ByteIterator {
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)
+ val nRead = math.min(available(), len)
copyToArray(b, off, nRead)
nRead
} else -1
@@ -269,7 +269,7 @@ object ByteIterator {
}
iterators = iterators.tail
}
- iterators = builder.result
+ iterators = builder.result()
normalize()
}
@@ -294,7 +294,7 @@ object ByteIterator {
if (current.len < lastLen) stop = true
dropCurrent()
}
- iterators = builder.result
+ iterators = builder.result()
normalize()
}
diff --git a/akka-actor/src/main/scala-2.13+/akka/util/ByteString.scala b/akka-actor/src/main/scala-2.13/akka/util/ByteString.scala
similarity index 99%
rename from akka-actor/src/main/scala-2.13+/akka/util/ByteString.scala
rename to akka-actor/src/main/scala-2.13/akka/util/ByteString.scala
index 450b57544c..e6009e1f0f 100644
--- a/akka-actor/src/main/scala-2.13+/akka/util/ByteString.scala
+++ b/akka-actor/src/main/scala-2.13/akka/util/ByteString.scala
@@ -5,16 +5,17 @@
package akka.util
import java.io.{ ObjectInputStream, ObjectOutputStream }
-import java.nio.{ ByteBuffer, ByteOrder }
import java.lang.{ Iterable => JIterable }
+import java.nio.{ ByteBuffer, ByteOrder }
import java.nio.charset.{ Charset, StandardCharsets }
import java.util.Base64
import scala.annotation.{ tailrec, varargs }
-import scala.collection.mutable.{ Builder, WrappedArray }
import scala.collection.{ immutable, mutable }
import scala.collection.immutable.{ IndexedSeq, IndexedSeqOps, StrictOptimizedSeqOps, VectorBuilder }
+import scala.collection.mutable.{ Builder, WrappedArray }
import scala.reflect.ClassTag
+
import com.github.ghik.silencer.silent
object ByteString {
@@ -1334,7 +1335,7 @@ final class ByteStringBuilder extends Builder[Byte, ByteString] {
if (_length == 0) ByteString.empty
else {
clearTemp()
- val bytestrings = _builder.result
+ val bytestrings = _builder.result()
if (bytestrings.size == 1)
bytestrings.head
else
diff --git a/akka-actor/src/main/scala-2.13-/akka/util/ccompat/ccompatUsedUntil213.scala b/akka-actor/src/main/scala-2.13/akka/util/ccompat/ccompatUsedUntil213.scala
similarity index 100%
rename from akka-actor/src/main/scala-2.13-/akka/util/ccompat/ccompatUsedUntil213.scala
rename to akka-actor/src/main/scala-2.13/akka/util/ccompat/ccompatUsedUntil213.scala
diff --git a/akka-actor/src/main/scala-2.13+/akka/util/ccompat/package.scala b/akka-actor/src/main/scala-2.13/akka/util/ccompat/package.scala
similarity index 100%
rename from akka-actor/src/main/scala-2.13+/akka/util/ccompat/package.scala
rename to akka-actor/src/main/scala-2.13/akka/util/ccompat/package.scala
diff --git a/akka-actor/src/main/scala/akka/Done.scala b/akka-actor/src/main/scala/akka/Done.scala
index 4e823ac768..1f236e1c6f 100644
--- a/akka-actor/src/main/scala/akka/Done.scala
+++ b/akka-actor/src/main/scala/akka/Done.scala
@@ -5,6 +5,7 @@
package akka
import java.io.Serializable
+
import akka.annotation.DoNotInherit
/**
diff --git a/akka-actor/src/main/scala/akka/Main.scala b/akka-actor/src/main/scala/akka/Main.scala
index 53cc36c90c..707aecfa45 100644
--- a/akka-actor/src/main/scala/akka/Main.scala
+++ b/akka-actor/src/main/scala/akka/Main.scala
@@ -4,14 +4,15 @@
package akka
+import scala.util.control.NonFatal
+
+import akka.actor.Actor
+import akka.actor.ActorLogging
+import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.ExtendedActorSystem
-import akka.actor.Actor
-import akka.actor.Terminated
-import akka.actor.ActorLogging
import akka.actor.Props
-import akka.actor.ActorRef
-import scala.util.control.NonFatal
+import akka.actor.Terminated
/**
* Main class to start an [[akka.actor.ActorSystem]] with one
diff --git a/akka-actor/src/main/scala/akka/actor/AbstractActor.scala b/akka-actor/src/main/scala/akka/actor/AbstractActor.scala
index 3397c34c59..4c7714bb86 100644
--- a/akka-actor/src/main/scala/akka/actor/AbstractActor.scala
+++ b/akka-actor/src/main/scala/akka/actor/AbstractActor.scala
@@ -4,17 +4,17 @@
package akka.actor
-import akka.annotation.DoNotInherit
-import akka.japi.pf.ReceiveBuilder
-
-import scala.runtime.BoxedUnit
import java.util.Optional
-import akka.util.JavaDurationConverters
-import com.github.ghik.silencer.silent
-
import scala.concurrent.ExecutionContextExecutor
import scala.concurrent.duration.Duration
+import scala.runtime.BoxedUnit
+
+import com.github.ghik.silencer.silent
+
+import akka.annotation.DoNotInherit
+import akka.japi.pf.ReceiveBuilder
+import akka.util.JavaDurationConverters
/**
* Java API: compatible with lambda expressions
diff --git a/akka-actor/src/main/scala/akka/actor/AbstractFSM.scala b/akka-actor/src/main/scala/akka/actor/AbstractFSM.scala
index 9aa670ace0..4a44395854 100644
--- a/akka-actor/src/main/scala/akka/actor/AbstractFSM.scala
+++ b/akka-actor/src/main/scala/akka/actor/AbstractFSM.scala
@@ -4,9 +4,10 @@
package akka.actor
-import akka.util.JavaDurationConverters._
import scala.concurrent.duration.FiniteDuration
+import akka.util.JavaDurationConverters._
+
/**
* Java API: compatible with lambda expressions
*
@@ -34,8 +35,9 @@ abstract class AbstractFSM[S, D] extends FSM[S, D] {
import java.util.{ List => JList }
import FSM._
- import akka.japi.pf.FI._
+
import akka.japi.pf._
+ import akka.japi.pf.FI._
/**
* Returns this AbstractActor's ActorContext
diff --git a/akka-actor/src/main/scala/akka/actor/AbstractProps.scala b/akka-actor/src/main/scala/akka/actor/AbstractProps.scala
index e7506cb13a..db3a1d2b16 100644
--- a/akka-actor/src/main/scala/akka/actor/AbstractProps.scala
+++ b/akka-actor/src/main/scala/akka/actor/AbstractProps.scala
@@ -5,11 +5,13 @@
package akka.actor
import java.lang.reflect.{ Modifier, ParameterizedType, TypeVariable }
+import java.lang.reflect.Constructor
+
+import scala.annotation.tailrec
+import scala.annotation.varargs
+
import akka.japi.Creator
import akka.util.Reflect
-import scala.annotation.varargs
-import scala.annotation.tailrec
-import java.lang.reflect.Constructor
/**
*
@@ -20,9 +22,15 @@ private[akka] trait AbstractProps {
/**
* INTERNAL API
*/
- private[akka] def validate(clazz: Class[_]) =
- if (Modifier.isAbstract(clazz.getModifiers))
+ private[akka] def validate(clazz: Class[_]): Unit = {
+ if (Modifier.isAbstract(clazz.getModifiers)) {
throw new IllegalArgumentException(s"Actor class [${clazz.getName}] must not be abstract")
+ } else if (!classOf[Actor].isAssignableFrom(clazz) &&
+ !classOf[IndirectActorProducer].isAssignableFrom(clazz)) {
+ throw new IllegalArgumentException(
+ s"Actor class [${clazz.getName}] must be subClass of akka.actor.Actor or akka.actor.IndirectActorProducer.")
+ }
+ }
/**
* Java API: create a Props given a class and its constructor arguments.
diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala
index 18c6001a78..1f6c0cd137 100644
--- a/akka-actor/src/main/scala/akka/actor/Actor.scala
+++ b/akka-actor/src/main/scala/akka/actor/Actor.scala
@@ -4,14 +4,15 @@
package akka.actor
-import akka.AkkaException
-import akka.event.LoggingAdapter
import java.util.Optional
import scala.annotation.tailrec
import scala.beans.BeanProperty
import scala.util.control.NoStackTrace
+
+import akka.AkkaException
import akka.annotation.InternalApi
+import akka.event.LoggingAdapter
import akka.util.unused
/**
@@ -500,7 +501,7 @@ trait Actor {
* self ! message
*
*/
- implicit final val self = context.self //MUST BE A VAL, TRUST ME
+ implicit final val self: ActorRef = context.self //MUST BE A VAL, TRUST ME
/**
* The reference sender Actor of the last received message.
diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala
index 55d885f52c..ec3c26a980 100644
--- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala
+++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala
@@ -12,14 +12,16 @@ import scala.collection.immutable
import scala.concurrent.ExecutionContextExecutor
import scala.concurrent.duration.Duration
import scala.util.control.NonFatal
+
+import com.github.ghik.silencer.silent
+
import akka.actor.dungeon.ChildrenContainer
+import akka.annotation.{ InternalApi, InternalStableApi }
import akka.dispatch.{ Envelope, MessageDispatcher }
import akka.dispatch.sysmsg._
import akka.event.Logging.{ Debug, Error, LogEvent }
import akka.japi.Procedure
-import akka.util.{ unused, Reflect }
-import akka.annotation.{ InternalApi, InternalStableApi }
-import com.github.ghik.silencer.silent
+import akka.util.unused
/**
* The actor context - the view of the actor cell from the actor.
@@ -408,7 +410,7 @@ private[akka] object ActorCell {
private[akka] class ActorCell(
val system: ActorSystemImpl,
val self: InternalActorRef,
- final val props: Props, // Must be final so that it can be properly cleared in clearActorCellFields
+ _initialProps: Props,
val dispatcher: MessageDispatcher,
val parent: InternalActorRef)
extends AbstractActor.ActorContext
@@ -419,6 +421,9 @@ private[akka] class ActorCell(
with dungeon.DeathWatch
with dungeon.FaultHandling {
+ private[this] var _props = _initialProps
+ def props: Props = _props
+
import ActorCell._
final def isLocal = true
@@ -433,7 +438,6 @@ private[akka] class ActorCell(
protected def uid: Int = self.path.uid
private[this] var _actor: Actor = _
def actor: Actor = _actor
- protected def actor_=(a: Actor): Unit = _actor = a
var currentMessage: Envelope = _
private var behaviorStack: List[Actor.Receive] = emptyBehaviorStack
private[this] var sysmsgStash: LatestFirstSystemMessageList = SystemMessageList.LNil
@@ -613,6 +617,7 @@ private[akka] class ActorCell(
// If no becomes were issued, the actors behavior is its receive method
behaviorStack = if (behaviorStack.isEmpty) instance.receive :: behaviorStack else behaviorStack
+ _actor = instance
instance
} finally {
val stackAfter = contextStack.get
@@ -622,29 +627,28 @@ private[akka] class ActorCell(
}
protected def create(failure: Option[ActorInitializationException]): Unit = {
- def clearOutActorIfNonNull(): Unit = {
- if (actor != null) {
+ def failActor(): Unit =
+ if (_actor != null) {
clearActorFields(actor, recreate = false)
- actor = null // ensure that we know that we failed during creation
+ setFailedFatally()
+ _actor = null // ensure that we know that we failed during creation
}
- }
failure.foreach { throw _ }
try {
val created = newActor()
- actor = created
created.aroundPreStart()
checkReceiveTimeout(reschedule = true)
if (system.settings.DebugLifecycle)
publish(Debug(self.path.toString, clazz(created), "started (" + created + ")"))
} catch {
case e: InterruptedException =>
- clearOutActorIfNonNull()
+ failActor()
Thread.currentThread().interrupt()
throw ActorInitializationException(self, "interruption during creation", e)
case NonFatal(e) =>
- clearOutActorIfNonNull()
+ failActor()
e match {
case i: InstantiationException =>
throw ActorInitializationException(
@@ -682,25 +686,17 @@ private[akka] class ActorCell(
case _ =>
}
- final protected def clearActorCellFields(cell: ActorCell): Unit = {
- cell.unstashAll()
- if (!Reflect.lookupAndSetField(classOf[ActorCell], cell, "props", ActorCell.terminatedProps))
- throw new IllegalArgumentException("ActorCell has no props field")
- }
-
+ @InternalStableApi
+ @silent("never used")
final protected def clearActorFields(actorInstance: Actor, recreate: Boolean): Unit = {
- setActorFields(actorInstance, context = null, self = if (recreate) self else system.deadLetters)
currentMessage = null
behaviorStack = emptyBehaviorStack
}
-
- final protected def setActorFields(actorInstance: Actor, context: ActorContext, self: ActorRef): Unit =
- if (actorInstance ne null) {
- if (!Reflect.lookupAndSetField(actorInstance.getClass, actorInstance, "context", context)
- || !Reflect.lookupAndSetField(actorInstance.getClass, actorInstance, "self", self))
- throw IllegalActorStateException(
- s"${actorInstance.getClass} is not an Actor class. It doesn't extend the 'Actor' trait")
- }
+ final protected def clearFieldsForTermination(): Unit = {
+ unstashAll()
+ _props = ActorCell.terminatedProps
+ _actor = null
+ }
// logging is not the main purpose, and if it fails there’s nothing we can do
protected final def publish(e: LogEvent): Unit =
diff --git a/akka-actor/src/main/scala/akka/actor/ActorPath.scala b/akka-actor/src/main/scala/akka/actor/ActorPath.scala
index 242778785f..69e7e21dcf 100644
--- a/akka-actor/src/main/scala/akka/actor/ActorPath.scala
+++ b/akka-actor/src/main/scala/akka/actor/ActorPath.scala
@@ -3,14 +3,16 @@
*/
package akka.actor
+import java.lang.{ StringBuilder => JStringBuilder }
+import java.net.MalformedURLException
+
import scala.annotation.{ switch, tailrec }
import scala.collection.immutable
-import akka.japi.Util.immutableSeq
-import java.net.MalformedURLException
-import java.lang.{ StringBuilder => JStringBuilder }
import com.github.ghik.silencer.silent
+import akka.japi.Util.immutableSeq
+
/**
* Java API
*/
diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala
index ab00d74c26..268bb923f0 100644
--- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala
+++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala
@@ -6,12 +6,12 @@ package akka.actor
import java.util.concurrent.ConcurrentHashMap
-import akka.annotation.InternalApi
-
import scala.annotation.tailrec
import scala.collection.immutable
import scala.util.control.NonFatal
+
import akka.annotation.DoNotInherit
+import akka.annotation.InternalApi
import akka.dispatch._
import akka.dispatch.sysmsg._
import akka.event.AddressTerminatedTopic
@@ -809,6 +809,15 @@ private[akka] class VirtualPathContainer(
}
}
+/**
+ * INTERNAL API
+ */
+@InternalApi private[akka] object FunctionRef {
+ def deadLetterMessageHandler(system: ActorSystem): (ActorRef, Any) => Unit = { (sender, msg) =>
+ system.deadLetters.tell(msg, sender)
+ }
+}
+
/**
* INTERNAL API
*
@@ -826,17 +835,20 @@ private[akka] class VirtualPathContainer(
* [[FunctionRef#unwatch]] must be called to avoid a resource leak, which is different
* from an ordinary actor.
*/
-private[akka] final class FunctionRef(
+@InternalApi private[akka] final class FunctionRef(
override val path: ActorPath,
override val provider: ActorRefProvider,
system: ActorSystem,
f: (ActorRef, Any) => Unit)
extends MinimalActorRef {
+ // var because it's replaced in `stop`
+ private var messageHandler: (ActorRef, Any) => Unit = f
+
override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit = {
message match {
case AddressTerminated(address) => addressTerminated(address)
- case _ => f(sender, message)
+ case _ => messageHandler(sender, message)
}
}
@@ -922,7 +934,13 @@ private[akka] final class FunctionRef(
}
}
- override def stop(): Unit = sendTerminated()
+ override def stop(): Unit = {
+ sendTerminated()
+ // The messageHandler function may close over a large object graph (such as an Akka Stream)
+ // so we replace the messageHandler function to make that available for garbage collection.
+ // Doesn't matter if the change isn't visible immediately, volatile not needed.
+ messageHandler = FunctionRef.deadLetterMessageHandler(system)
+ }
private def addWatcher(watchee: ActorRef, watcher: ActorRef): Unit = {
val selfTerminated = this.synchronized {
diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala
index 852c88eff2..fcc0c3a850 100644
--- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala
+++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala
@@ -4,23 +4,23 @@
package akka.actor
-import akka.dispatch.sysmsg._
-import akka.dispatch.{ Mailboxes, RequiresMessageQueue, UnboundedMessageQueueSemantics }
-import akka.routing._
-import akka.event._
-import akka.util.Helpers
-import akka.util.Collections.EmptyImmutableSeq
-
-import scala.util.control.NonFatal
import java.util.concurrent.atomic.AtomicLong
-import scala.concurrent.{ ExecutionContextExecutor, Future, Promise }
import scala.annotation.implicitNotFound
+import scala.concurrent.{ ExecutionContextExecutor, Future, Promise }
+import scala.util.control.NonFatal
+
import akka.ConfigurationException
import akka.annotation.DoNotInherit
import akka.annotation.InternalApi
+import akka.dispatch.{ Mailboxes, RequiresMessageQueue, UnboundedMessageQueueSemantics }
import akka.dispatch.Dispatchers
+import akka.dispatch.sysmsg._
+import akka.event._
+import akka.routing._
import akka.serialization.Serialization
+import akka.util.Collections.EmptyImmutableSeq
+import akka.util.Helpers
import akka.util.OptionVal
/**
diff --git a/akka-actor/src/main/scala/akka/actor/ActorSelection.scala b/akka-actor/src/main/scala/akka/actor/ActorSelection.scala
index 63c371ac3c..52b318531f 100644
--- a/akka-actor/src/main/scala/akka/actor/ActorSelection.scala
+++ b/akka-actor/src/main/scala/akka/actor/ActorSelection.scala
@@ -4,26 +4,26 @@
package akka.actor
-import scala.language.implicitConversions
import java.util.concurrent.CompletionStage
+import java.util.regex.Pattern
-import scala.language.implicitConversions
import scala.annotation.tailrec
import scala.collection.immutable
+import scala.compat.java8.FutureConverters
import scala.concurrent.Future
import scala.concurrent.Promise
import scala.concurrent.duration._
+import scala.language.implicitConversions
+import scala.language.implicitConversions
import scala.util.Success
-import java.util.regex.Pattern
+import com.github.ghik.silencer.silent
+
+import akka.dispatch.ExecutionContexts
import akka.pattern.ask
import akka.routing.MurmurHash
import akka.util.{ Helpers, JavaDurationConverters, Timeout }
-import akka.dispatch.ExecutionContexts
-
-import scala.compat.java8.FutureConverters
import akka.util.ccompat._
-import com.github.ghik.silencer.silent
/**
* An ActorSelection is a logical view of a section of an ActorSystem's tree of Actors,
diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala
index a8e1988299..ea63ad7591 100644
--- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala
+++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala
@@ -9,29 +9,30 @@ import java.util.Optional
import java.util.concurrent._
import java.util.concurrent.atomic.AtomicReference
-import akka.actor.dungeon.ChildrenContainer
-import akka.actor.setup.{ ActorSystemSetup, Setup }
-import akka.annotation.InternalApi
-import akka.ConfigurationException
-import akka.annotation.DoNotInherit
-import akka.dispatch._
-import akka.event._
-import akka.japi.Util.immutableSeq
-import akka.util.Helpers.toRootLowerCase
-import akka.util._
-import com.typesafe.config.{ Config, ConfigFactory }
import scala.annotation.tailrec
import scala.collection.immutable
import scala.compat.java8.FutureConverters
import scala.compat.java8.OptionConverters._
-import scala.concurrent.duration.Duration
-import scala.concurrent.blocking
import scala.concurrent.{ ExecutionContext, ExecutionContextExecutor, Future, Promise }
-import scala.util.control.{ ControlThrowable, NonFatal }
+import scala.concurrent.blocking
+import scala.concurrent.duration.Duration
import scala.util.{ Failure, Success, Try }
+import scala.util.control.{ ControlThrowable, NonFatal }
+import com.typesafe.config.{ Config, ConfigFactory }
+
+import akka.ConfigurationException
+import akka.actor.dungeon.ChildrenContainer
+import akka.actor.setup.{ ActorSystemSetup, Setup }
+import akka.annotation.DoNotInherit
+import akka.annotation.InternalApi
+import akka.dispatch._
+import akka.event._
import akka.event.Logging.DefaultLogger
+import akka.japi.Util.immutableSeq
import akka.serialization.SerializationExtension
+import akka.util._
+import akka.util.Helpers.toRootLowerCase
object BootstrapSetup {
@@ -170,18 +171,6 @@ object ActorSystem {
val Version: String = akka.Version.current // generated file
- val EnvHome: Option[String] = System.getenv("AKKA_HOME") match {
- case null | "" | "." => None
- case value => Some(value)
- }
-
- val SystemHome: Option[String] = System.getProperty("akka.home") match {
- case null | "" => None
- case value => Some(value)
- }
-
- val GlobalHome: Option[String] = SystemHome.orElse(EnvHome)
-
/**
* Creates a new ActorSystem with the name "default",
* obtains the current ClassLoader by first inspecting the current threads' getContextClassLoader,
@@ -400,9 +389,10 @@ object ActorSystem {
cfg
}
- import akka.util.Helpers.ConfigOps
import config._
+ import akka.util.Helpers.ConfigOps
+
final val ConfigVersion: String = getString("akka.version")
private final val providerSelectionSetup = setup
@@ -976,7 +966,7 @@ private[akka] class ActorSystemImpl(
def /(actorName: String): ActorPath = guardian.path / actorName
def /(path: Iterable[String]): ActorPath = guardian.path / path
- override private[akka] def classicSystem: ActorSystem = this
+ override def classicSystem: ActorSystem = this
// Used for ManifestInfo.checkSameVersion
private def allModules: List[String] =
@@ -1029,7 +1019,7 @@ private[akka] class ActorSystemImpl(
_initialized = true
if (settings.LogDeadLetters > 0)
- logDeadLetterListener = Some(systemActorOf(Props[DeadLetterListener], "deadLetterListener"))
+ logDeadLetterListener = Some(systemActorOf(Props[DeadLetterListener](), "deadLetterListener"))
eventStream.startUnsubscriber()
ManifestInfo(this).checkSameVersion("Akka", allModules, logWarning = true)
if (!terminating)
@@ -1188,9 +1178,11 @@ private[akka] class ActorSystemImpl(
* when the extension cannot be found at all we throw regardless of this setting)
*/
def loadExtensions(key: String, throwOnLoadFail: Boolean): Unit = {
+
immutableSeq(settings.config.getStringList(key)).foreach { fqcn =>
dynamicAccess.getObjectFor[AnyRef](fqcn).recoverWith {
- case _ => dynamicAccess.createInstanceFor[AnyRef](fqcn, Nil)
+ case firstProblem =>
+ dynamicAccess.createInstanceFor[AnyRef](fqcn, Nil).recoverWith { case _ => Failure(firstProblem) }
} match {
case Success(p: ExtensionIdProvider) =>
registerExtension(p.lookup())
diff --git a/akka-actor/src/main/scala/akka/actor/Address.scala b/akka-actor/src/main/scala/akka/actor/Address.scala
index 6b57f7c4be..ef5c1f11e5 100644
--- a/akka-actor/src/main/scala/akka/actor/Address.scala
+++ b/akka-actor/src/main/scala/akka/actor/Address.scala
@@ -3,16 +3,17 @@
*/
package akka.actor
+import java.net.MalformedURLException
import java.net.URI
import java.net.URISyntaxException
-import java.net.MalformedURLException
import java.util.Optional
import scala.annotation.tailrec
import scala.collection.immutable
-import akka.annotation.InternalApi
import scala.compat.java8.OptionConverters._
+import akka.annotation.InternalApi
+
/**
* The address specifies the physical location under which an Actor can be
* reached. Examples are local addresses, identified by the ActorSystem’s
diff --git a/akka-actor/src/main/scala/akka/actor/ClassicActorSystemProvider.scala b/akka-actor/src/main/scala/akka/actor/ClassicActorSystemProvider.scala
index 66a9cb7330..d7473e04cc 100644
--- a/akka-actor/src/main/scala/akka/actor/ClassicActorSystemProvider.scala
+++ b/akka-actor/src/main/scala/akka/actor/ClassicActorSystemProvider.scala
@@ -15,9 +15,10 @@ import akka.annotation.InternalApi
@DoNotInherit
trait ClassicActorSystemProvider {
- /** INTERNAL API */
- @InternalApi
- private[akka] def classicSystem: ActorSystem
+ /**
+ * Allows access to the classic `akka.actor.ActorSystem` even for `akka.actor.typed.ActorSystem[_]`s.
+ */
+ def classicSystem: ActorSystem
}
/**
diff --git a/akka-actor/src/main/scala/akka/actor/CoordinatedShutdown.scala b/akka-actor/src/main/scala/akka/actor/CoordinatedShutdown.scala
index e7f9731a58..8985cef7da 100644
--- a/akka-actor/src/main/scala/akka/actor/CoordinatedShutdown.scala
+++ b/akka-actor/src/main/scala/akka/actor/CoordinatedShutdown.scala
@@ -4,30 +4,29 @@
package akka.actor
-import scala.concurrent.duration._
-import scala.compat.java8.FutureConverters._
-import scala.compat.java8.OptionConverters._
+import java.util.Optional
import java.util.concurrent._
import java.util.concurrent.TimeUnit.MILLISECONDS
-import scala.concurrent.{ Await, ExecutionContext, Future, Promise }
-
-import akka.Done
-import com.typesafe.config.Config
-import scala.concurrent.duration.FiniteDuration
-import scala.annotation.tailrec
-
-import com.typesafe.config.ConfigFactory
-import akka.pattern.after
-import scala.util.control.NonFatal
-
-import akka.event.Logging
-import akka.dispatch.ExecutionContexts
-import scala.util.Try
import java.util.concurrent.atomic.AtomicReference
import java.util.function.Supplier
-import java.util.Optional
+import scala.annotation.tailrec
+import scala.compat.java8.FutureConverters._
+import scala.compat.java8.OptionConverters._
+import scala.concurrent.{ Await, ExecutionContext, Future, Promise }
+import scala.concurrent.duration._
+import scala.concurrent.duration.FiniteDuration
+import scala.util.Try
+import scala.util.control.NonFatal
+
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+
+import akka.Done
import akka.annotation.InternalApi
+import akka.dispatch.ExecutionContexts
+import akka.event.Logging
+import akka.pattern.after
import akka.util.{ OptionVal, Timeout }
object CoordinatedShutdown extends ExtensionId[CoordinatedShutdown] with ExtensionIdProvider {
@@ -710,7 +709,7 @@ final class CoordinatedShutdown private[akka] (
val deadline = Deadline.now + timeout
val timeoutFut = try {
after(timeout, system.scheduler) {
- if (phaseName == CoordinatedShutdown.PhaseActorSystemTerminate && deadline.hasTimeLeft) {
+ if (phaseName == CoordinatedShutdown.PhaseActorSystemTerminate && deadline.hasTimeLeft()) {
// too early, i.e. triggered by system termination
result
} else if (result.isCompleted)
diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala
index 12e294275d..0ea6c929f9 100644
--- a/akka-actor/src/main/scala/akka/actor/Deployer.scala
+++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala
@@ -6,13 +6,14 @@ package akka.actor
import java.util.concurrent.atomic.AtomicReference
-import akka.routing._
-import akka.util.WildcardIndex
-import com.github.ghik.silencer.silent
-import com.typesafe.config._
import scala.annotation.tailrec
+import com.github.ghik.silencer.silent
+import com.typesafe.config._
+
import akka.annotation.InternalApi
+import akka.routing._
+import akka.util.WildcardIndex
object Deploy {
final val NoDispatcherGiven = ""
@@ -119,13 +120,13 @@ final class Deploy(
new Deploy(path, config, routerConfig, scope, dispatcher, mailbox, tags)
override def productElement(n: Int): Any = n match {
- case 1 => path
- case 2 => config
- case 3 => routerConfig
- case 4 => scope
- case 5 => dispatcher
- case 6 => mailbox
- case 7 => tags
+ case 0 => path
+ case 1 => config
+ case 2 => routerConfig
+ case 3 => scope
+ case 4 => dispatcher
+ case 5 => mailbox
+ case 6 => tags
}
override def productArity: Int = 7
diff --git a/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala b/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala
index 8a5c8b8190..b86ef3170e 100644
--- a/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala
+++ b/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala
@@ -4,12 +4,12 @@
package akka.actor
-import akka.annotation.DoNotInherit
-
import scala.collection.immutable
import scala.reflect.ClassTag
import scala.util.Try
+import akka.annotation.DoNotInherit
+
/**
* The DynamicAccess implementation is the class which is used for
* loading all configurable parts of an actor system (the
diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala
index c21daab440..91a2eb0ac8 100644
--- a/akka-actor/src/main/scala/akka/actor/FSM.scala
+++ b/akka-actor/src/main/scala/akka/actor/FSM.scala
@@ -4,15 +4,18 @@
package akka.actor
-import language.implicitConversions
-import scala.concurrent.duration.Duration
import scala.collection.mutable
-import scala.concurrent.duration.FiniteDuration
+import scala.concurrent.ExecutionContextExecutor
import scala.concurrent.duration._
-import akka.routing.{ Deafen, Listen, Listeners }
-import akka.annotation.InternalApi
-import akka.util.{ unused, JavaDurationConverters }
+import scala.concurrent.duration.Duration
+import scala.concurrent.duration.FiniteDuration
+
import com.github.ghik.silencer.silent
+import language.implicitConversions
+
+import akka.annotation.InternalApi
+import akka.routing.{ Deafen, Listen, Listeners }
+import akka.util.{ unused, JavaDurationConverters }
object FSM {
@@ -121,7 +124,7 @@ object FSM {
extends NoSerializationVerificationNeeded {
private var ref: Option[Cancellable] = _
private val scheduler = context.system.scheduler
- private implicit val executionContext = context.dispatcher
+ private implicit val executionContext: ExecutionContextExecutor = context.dispatcher
def schedule(actor: ActorRef, timeout: FiniteDuration): Unit = {
val timerMsg = msg match {
@@ -465,7 +468,7 @@ trait FSM[S, D] extends Actor with Listeners with ActorLogging {
/**
* Produce change descriptor to stop this FSM actor including specified reason.
*/
- final def stop(reason: Reason, stateData: D): State = stay.using(stateData).withStopReason(reason)
+ final def stop(reason: Reason, stateData: D): State = stay().using(stateData).withStopReason(reason)
final class TransformHelper(func: StateFunction) {
def using(andThen: PartialFunction[State, State]): StateFunction =
@@ -558,7 +561,7 @@ trait FSM[S, D] extends Actor with Listeners with ActorLogging {
if (timers contains name) {
timers(name).cancel()
}
- val timer = Timer(name, msg, mode, timerGen.next, this)(context)
+ val timer = Timer(name, msg, mode, timerGen.next(), this)(context)
timer.schedule(self, timeout)
timers(name) = timer
}
@@ -727,7 +730,7 @@ trait FSM[S, D] extends Actor with Listeners with ActorLogging {
private val handleEventDefault: StateFunction = {
case Event(value, _) =>
log.warning("unhandled event " + value + " in state " + stateName)
- stay
+ stay()
}
private var handleEvent: StateFunction = handleEventDefault
@@ -820,7 +823,7 @@ trait FSM[S, D] extends Actor with Listeners with ActorLogging {
private[akka] def makeTransition(nextState: State): Unit = {
if (!stateFunctions.contains(nextState.stateName)) {
- terminate(stay.withStopReason(Failure("Next state %s does not exist".format(nextState.stateName))))
+ terminate(stay().withStopReason(Failure("Next state %s does not exist".format(nextState.stateName))))
} else {
nextState.replies.reverse.foreach { r =>
sender() ! r
@@ -861,7 +864,7 @@ trait FSM[S, D] extends Actor with Listeners with ActorLogging {
* setting this instance’s state to terminated does no harm during restart
* since the new instance will initialize fresh using startWith()
*/
- terminate(stay.withStopReason(Shutdown))
+ terminate(stay().withStopReason(Shutdown))
super.postStop()
}
diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala
index be0d456d4f..3b1d3a5fba 100644
--- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala
+++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala
@@ -4,19 +4,22 @@
package akka.actor
-import java.lang.reflect.InvocationTargetException
-import language.implicitConversions
import java.lang.{ Iterable => JIterable }
+import java.lang.reflect.InvocationTargetException
import java.util.concurrent.TimeUnit
+
+import scala.collection.immutable
+import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.duration.Duration
+import scala.util.control.NonFatal
+
+import language.implicitConversions
+
+import akka.event.Logging.Error
+import akka.event.Logging.LogEvent
+import akka.event.Logging.Warning
import akka.japi.Util.immutableSeq
import akka.util.JavaDurationConverters._
-import scala.collection.mutable.ArrayBuffer
-import scala.collection.immutable
-import scala.concurrent.duration.Duration
-import akka.event.Logging.LogEvent
-import akka.event.Logging.Error
-import akka.event.Logging.Warning
-import scala.util.control.NonFatal
import akka.util.ccompat._
/**
diff --git a/akka-actor/src/main/scala/akka/actor/IndirectActorProducer.scala b/akka-actor/src/main/scala/akka/actor/IndirectActorProducer.scala
index 6e618a9b4f..9ee941f831 100644
--- a/akka-actor/src/main/scala/akka/actor/IndirectActorProducer.scala
+++ b/akka-actor/src/main/scala/akka/actor/IndirectActorProducer.scala
@@ -4,11 +4,12 @@
package akka.actor
-import akka.japi.Creator
-import akka.util.Reflect
+import scala.collection.immutable
+
import com.github.ghik.silencer.silent
-import scala.collection.immutable
+import akka.japi.Creator
+import akka.util.Reflect
/**
* This interface defines a class of actor creation strategies deviating from
diff --git a/akka-actor/src/main/scala/akka/actor/LightArrayRevolverScheduler.scala b/akka-actor/src/main/scala/akka/actor/LightArrayRevolverScheduler.scala
index 61c3401b53..5bd8462d26 100644
--- a/akka-actor/src/main/scala/akka/actor/LightArrayRevolverScheduler.scala
+++ b/akka-actor/src/main/scala/akka/actor/LightArrayRevolverScheduler.scala
@@ -7,16 +7,19 @@ package akka.actor
import java.io.Closeable
import java.util.concurrent.ThreadFactory
import java.util.concurrent.atomic.{ AtomicLong, AtomicReference }
+
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.{ Await, ExecutionContext, Future, Promise }
import scala.concurrent.duration._
import scala.util.control.NonFatal
+
import com.typesafe.config.Config
+
+import akka.dispatch.AbstractNodeQueue
import akka.event.LoggingAdapter
import akka.util.Helpers
import akka.util.Unsafe.{ instance => unsafe }
-import akka.dispatch.AbstractNodeQueue
/**
* This scheduler implementation is based on a revolving wheel of buckets,
@@ -38,8 +41,8 @@ class LightArrayRevolverScheduler(config: Config, log: LoggingAdapter, threadFac
extends Scheduler
with Closeable {
- import Helpers.Requiring
import Helpers.ConfigOps
+ import Helpers.Requiring
val WheelSize =
config
@@ -183,7 +186,7 @@ class LightArrayRevolverScheduler(config: Config, log: LoggingAdapter, threadFac
private val queue = new TaskQueue
private def schedule(ec: ExecutionContext, r: Runnable, delay: FiniteDuration): TimerTask =
- if (delay <= Duration.Zero) {
+ if (delay.length <= 0L) { // use simple comparision instead of Ordering for performance
if (stopped.get != null) throw SchedulerException("cannot enqueue after timer shutdown")
ec.execute(r)
NotCancellable
diff --git a/akka-actor/src/main/scala/akka/actor/Props.scala b/akka-actor/src/main/scala/akka/actor/Props.scala
index a170bbcd33..5dfd752362 100644
--- a/akka-actor/src/main/scala/akka/actor/Props.scala
+++ b/akka-actor/src/main/scala/akka/actor/Props.scala
@@ -4,14 +4,14 @@
package akka.actor
-import akka.actor.Deploy.{ NoDispatcherGiven, NoMailboxGiven }
-import akka.dispatch._
-import akka.routing._
-
import scala.annotation.varargs
import scala.collection.immutable
import scala.reflect.ClassTag
+import akka.actor.Deploy.{ NoDispatcherGiven, NoMailboxGiven }
+import akka.dispatch._
+import akka.routing._
+
/**
* Factory for Props instances.
*
@@ -39,7 +39,7 @@ object Props extends AbstractProps {
/**
* A Props instance whose creator will create an actor that doesn't respond to any message
*/
- final val empty = Props[EmptyActor]
+ final val empty = Props[EmptyActor]()
/**
* The default Props instance, uses the settings from the Props object starting with default*.
diff --git a/akka-actor/src/main/scala/akka/actor/ReflectiveDynamicAccess.scala b/akka-actor/src/main/scala/akka/actor/ReflectiveDynamicAccess.scala
index 049c2fe639..ef16eaff97 100644
--- a/akka-actor/src/main/scala/akka/actor/ReflectiveDynamicAccess.scala
+++ b/akka-actor/src/main/scala/akka/actor/ReflectiveDynamicAccess.scala
@@ -4,15 +4,15 @@
package akka.actor
-import scala.collection.immutable
import java.lang.reflect.InvocationTargetException
-import akka.annotation.DoNotInherit
-
+import scala.collection.immutable
import scala.reflect.ClassTag
import scala.util.Failure
import scala.util.Try
+import akka.annotation.DoNotInherit
+
/**
* This is the default [[akka.actor.DynamicAccess]] implementation used by [[akka.actor.ExtendedActorSystem]]
* unless overridden. It uses reflection to turn fully-qualified class names into `Class[_]` objects
diff --git a/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala b/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala
index a4bdb6564a..15999fcaec 100644
--- a/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala
+++ b/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala
@@ -9,14 +9,15 @@ import java.util.concurrent.locks.ReentrantLock
import scala.annotation.tailrec
import scala.collection.immutable
-import akka.actor.dungeon.ChildrenContainer
-import akka.event.Logging.Warning
-import akka.util.{ unused, Unsafe }
-import akka.dispatch._
-import akka.dispatch.sysmsg._
+import scala.util.control.NonFatal
+
import com.github.ghik.silencer.silent
-import scala.util.control.NonFatal
+import akka.actor.dungeon.ChildrenContainer
+import akka.dispatch._
+import akka.dispatch.sysmsg._
+import akka.event.Logging.Warning
+import akka.util.{ unused, Unsafe }
/**
* This actor ref starts out with some dummy cell (by default just enqueuing
@@ -149,7 +150,7 @@ private[akka] class RepointableActorRef(
def getChild(name: Iterator[String]): InternalActorRef =
if (name.hasNext) {
- name.next match {
+ name.next() match {
case ".." => getParent.getChild(name)
case "" => getChild(name)
case other =>
diff --git a/akka-actor/src/main/scala/akka/actor/Scheduler.scala b/akka-actor/src/main/scala/akka/actor/Scheduler.scala
index 1403ed264c..4d170dd462 100644
--- a/akka-actor/src/main/scala/akka/actor/Scheduler.scala
+++ b/akka-actor/src/main/scala/akka/actor/Scheduler.scala
@@ -7,15 +7,15 @@ package akka.actor
import java.util.concurrent.atomic.AtomicReference
import scala.annotation.tailrec
-
-import akka.util.JavaDurationConverters
-import com.github.ghik.silencer.silent
import scala.concurrent.ExecutionContext
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
-import akka.annotation.InternalApi
import com.github.ghik.silencer.silent
+import com.github.ghik.silencer.silent
+
+import akka.annotation.InternalApi
+import akka.util.JavaDurationConverters
/**
* This exception is thrown by Scheduler.schedule* when scheduling is not
diff --git a/akka-actor/src/main/scala/akka/actor/Stash.scala b/akka-actor/src/main/scala/akka/actor/Stash.scala
index 4ac6a5e2c7..14b29d5e19 100644
--- a/akka-actor/src/main/scala/akka/actor/Stash.scala
+++ b/akka-actor/src/main/scala/akka/actor/Stash.scala
@@ -5,6 +5,8 @@
package akka.actor
import scala.collection.immutable
+import scala.util.control.NoStackTrace
+
import akka.AkkaException
import akka.annotation.InternalStableApi
import akka.dispatch.{
@@ -14,8 +16,6 @@ import akka.dispatch.{
UnboundedDequeBasedMessageQueueSemantics
}
-import scala.util.control.NoStackTrace
-
/**
* The `Stash` trait enables an actor to temporarily stash away messages that can not or
* should not be handled using the actor's current behavior.
diff --git a/akka-actor/src/main/scala/akka/actor/Timers.scala b/akka-actor/src/main/scala/akka/actor/Timers.scala
index 5f25741e89..d3b021f1f2 100644
--- a/akka-actor/src/main/scala/akka/actor/Timers.scala
+++ b/akka-actor/src/main/scala/akka/actor/Timers.scala
@@ -4,10 +4,11 @@
package akka.actor
-import akka.util.JavaDurationConverters._
import scala.concurrent.duration.FiniteDuration
+
import akka.annotation.DoNotInherit
import akka.dispatch.Envelope
+import akka.util.JavaDurationConverters._
import akka.util.OptionVal
/**
diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala
index b39689c9f6..02bee1c57b 100644
--- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala
+++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala
@@ -4,26 +4,28 @@
package akka.actor
-import scala.util.control.NonFatal
-import scala.util.{ Failure, Success, Try }
+import java.io.ObjectStreamException
+import java.lang.reflect.{ InvocationHandler, InvocationTargetException, Method, Proxy }
+import java.util.concurrent.TimeoutException
+import java.util.concurrent.atomic.{ AtomicReference => AtomVar }
+
import scala.collection.immutable
+import scala.concurrent.{ Await, Future }
+import scala.concurrent.ExecutionContextExecutor
import scala.concurrent.duration.FiniteDuration
import scala.reflect.ClassTag
-import scala.concurrent.{ Await, Future }
+import scala.util.{ Failure, Success, Try }
+import scala.util.control.NonFatal
+import com.github.ghik.silencer.silent
+
+import akka.dispatch._
import akka.japi.{ Creator, Option => JOption }
import akka.japi.Util.{ immutableSeq, immutableSingletonSeq }
import akka.pattern.AskTimeoutException
-import akka.util.Timeout
-import akka.util.Reflect.instantiator
import akka.serialization.{ JavaSerializer, SerializationExtension, Serializers }
-import akka.dispatch._
-import java.util.concurrent.atomic.{ AtomicReference => AtomVar }
-import java.util.concurrent.TimeoutException
-import java.io.ObjectStreamException
-import java.lang.reflect.{ InvocationHandler, InvocationTargetException, Method, Proxy }
-
-import com.github.ghik.silencer.silent
+import akka.util.Reflect.instantiator
+import akka.util.Timeout
/**
* A TypedActorFactory is something that can created TypedActor instances.
@@ -47,7 +49,7 @@ trait TypedActorFactory {
*/
def stop(proxy: AnyRef): Boolean = getActorRefFor(proxy) match {
case null => false
- case ref => ref.asInstanceOf[InternalActorRef].stop; true
+ case ref => ref.asInstanceOf[InternalActorRef].stop(); true
}
/**
@@ -76,7 +78,7 @@ trait TypedActorFactory {
val proxyVar = new AtomVar[R] //Chicken'n'egg-resolver
val c = props.creator //Cache this to avoid closing over the Props
val i = props.interfaces //Cache this to avoid closing over the Props
- val ap = Props(new TypedActor.TypedActor[R, T](proxyVar, c(), i)).withDeploy(props.actorProps.deploy)
+ val ap = Props(new TypedActor.TypedActor[R, T](proxyVar, c(), i)).withDeploy(props.actorProps().deploy)
typedActor.createActorRefProxy(props, proxyVar, actorFactory.actorOf(ap))
}
@@ -87,7 +89,7 @@ trait TypedActorFactory {
val proxyVar = new AtomVar[R] //Chicken'n'egg-resolver
val c = props.creator //Cache this to avoid closing over the Props
val i = props.interfaces //Cache this to avoid closing over the Props
- val ap = Props(new akka.actor.TypedActor.TypedActor[R, T](proxyVar, c(), i)).withDeploy(props.actorProps.deploy)
+ val ap = Props(new akka.actor.TypedActor.TypedActor[R, T](proxyVar, c(), i)).withDeploy(props.actorProps().deploy)
typedActor.createActorRefProxy(props, proxyVar, actorFactory.actorOf(ap, name))
}
@@ -252,7 +254,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
/**
* Returns the default dispatcher (for a TypedActor) when inside a method call in a TypedActor.
*/
- implicit def dispatcher = context.dispatcher
+ implicit def dispatcher: ExecutionContextExecutor = context.dispatcher
/**
* INTERNAL API
@@ -271,7 +273,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
private val me = withContext[T](createInstance)
override def supervisorStrategy: SupervisorStrategy = me match {
- case l: Supervisor => l.supervisorStrategy
+ case l: Supervisor => l.supervisorStrategy()
case _ => super.supervisorStrategy
}
@@ -687,6 +689,7 @@ class TypedActorExtension(val system: ExtendedActorSystem) extends TypedActorFac
protected def typedActor = this
import system.settings
+
import akka.util.Helpers.ConfigOps
/**
diff --git a/akka-actor/src/main/scala/akka/actor/dungeon/Children.scala b/akka-actor/src/main/scala/akka/actor/dungeon/Children.scala
index c2bf27ce16..d493e42dc4 100644
--- a/akka-actor/src/main/scala/akka/actor/dungeon/Children.scala
+++ b/akka-actor/src/main/scala/akka/actor/dungeon/Children.scala
@@ -7,12 +7,13 @@ package akka.actor.dungeon
import java.util.Optional
import scala.annotation.tailrec
-import scala.util.control.NonFatal
import scala.collection.immutable
+import scala.util.control.NonFatal
+import com.github.ghik.silencer.silent
import akka.actor._
+import akka.annotation.InternalStableApi
import akka.serialization.{ Serialization, SerializationExtension, Serializers }
import akka.util.{ Helpers, Unsafe }
-import com.github.ghik.silencer.silent
private[akka] object Children {
val GetNobody = () => Nobody
@@ -180,6 +181,7 @@ private[akka] trait Children { this: ActorCell =>
case _ => null
}
+ @InternalStableApi
protected def suspendChildren(exceptFor: Set[ActorRef] = Set.empty): Unit =
childrenRefs.stats.foreach {
case ChildRestartStats(child, _, _) if !(exceptFor contains child) =>
diff --git a/akka-actor/src/main/scala/akka/actor/dungeon/DeathWatch.scala b/akka-actor/src/main/scala/akka/actor/dungeon/DeathWatch.scala
index 55d0fad98c..cd15ce390f 100644
--- a/akka-actor/src/main/scala/akka/actor/dungeon/DeathWatch.scala
+++ b/akka-actor/src/main/scala/akka/actor/dungeon/DeathWatch.scala
@@ -4,10 +4,10 @@
package akka.actor.dungeon
-import akka.dispatch.sysmsg.{ DeathWatchNotification, Unwatch, Watch }
-import akka.event.Logging.{ Debug, Warning }
import akka.actor.{ Actor, ActorCell, ActorRef, ActorRefScope, Address, InternalActorRef, Terminated }
+import akka.dispatch.sysmsg.{ DeathWatchNotification, Unwatch, Watch }
import akka.event.AddressTerminatedTopic
+import akka.event.Logging.{ Debug, Warning }
import akka.util.unused
private[akka] trait DeathWatch { this: ActorCell =>
diff --git a/akka-actor/src/main/scala/akka/actor/dungeon/Dispatch.scala b/akka-actor/src/main/scala/akka/actor/dungeon/Dispatch.scala
index f6520576ab..cbfa1509a9 100644
--- a/akka-actor/src/main/scala/akka/actor/dungeon/Dispatch.scala
+++ b/akka-actor/src/main/scala/akka/actor/dungeon/Dispatch.scala
@@ -5,22 +5,23 @@
package akka.actor.dungeon
import scala.annotation.tailrec
-import akka.AkkaException
-import akka.dispatch.{ Envelope, Mailbox }
-import akka.dispatch.sysmsg._
-import akka.event.Logging.Error
-import akka.util.Unsafe
-import akka.actor._
-import akka.annotation.InternalApi
-import akka.serialization.{ DisabledJavaSerializer, SerializationExtension, Serializers }
-
import scala.util.control.{ NoStackTrace, NonFatal }
import scala.util.control.Exception.Catcher
+
+import com.github.ghik.silencer.silent
+
+import akka.AkkaException
+import akka.actor._
+import akka.annotation.InternalApi
+import akka.dispatch.{ Envelope, Mailbox }
import akka.dispatch.MailboxType
import akka.dispatch.ProducesMessageQueue
import akka.dispatch.UnboundedMailbox
+import akka.dispatch.sysmsg._
+import akka.event.Logging.Error
+import akka.serialization.{ DisabledJavaSerializer, SerializationExtension, Serializers }
import akka.serialization.Serialization
-import com.github.ghik.silencer.silent
+import akka.util.Unsafe
@SerialVersionUID(1L)
final case class SerializationCheckFailedException private (msg: Object, cause: Throwable)
@@ -73,7 +74,7 @@ private[akka] trait Dispatch { this: ActorCell =>
*/
// we need to delay the failure to the point of actor creation so we can handle
// it properly in the normal way
- val actorClass = props.actorClass
+ val actorClass = props.actorClass()
val createMessage = mailboxType match {
case _: ProducesMessageQueue[_] if system.mailboxes.hasRequiredType(actorClass) =>
val req = system.mailboxes.getRequiredType(actorClass)
diff --git a/akka-actor/src/main/scala/akka/actor/dungeon/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/dungeon/FaultHandling.scala
index 0e3bc37ed9..8c498c788f 100644
--- a/akka-actor/src/main/scala/akka/actor/dungeon/FaultHandling.scala
+++ b/akka-actor/src/main/scala/akka/actor/dungeon/FaultHandling.scala
@@ -4,9 +4,12 @@
package akka.actor.dungeon
+import akka.actor.{ ActorCell, ActorInterruptedException, ActorRef, InternalActorRef }
+import akka.actor.ActorRefScope
import akka.actor.PostRestartException
import akka.actor.PreRestartException
-import akka.actor.{ Actor, ActorCell, ActorInterruptedException, ActorRef, InternalActorRef }
+import akka.annotation.InternalApi
+import akka.annotation.InternalStableApi
import akka.dispatch._
import akka.dispatch.sysmsg._
import akka.event.Logging
@@ -17,9 +20,19 @@ import scala.collection.immutable
import scala.concurrent.duration.Duration
import scala.util.control.Exception._
import scala.util.control.NonFatal
-import akka.actor.ActorRefScope
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[akka] object FaultHandling {
+ sealed trait FailedInfo
+ private case object NoFailedInfo extends FailedInfo
+ private final case class FailedRef(ref: ActorRef) extends FailedInfo
+ private case object FailedFatally extends FailedInfo
+}
private[akka] trait FaultHandling { this: ActorCell =>
+ import FaultHandling._
/* =================
* T H E R U L E S
@@ -44,11 +57,22 @@ private[akka] trait FaultHandling { this: ActorCell =>
* a restart with dying children)
* might well be replaced by ref to a Cancellable in the future (see #2299)
*/
- private var _failed: ActorRef = null
- private def isFailed: Boolean = _failed != null
- private def setFailed(perpetrator: ActorRef): Unit = _failed = perpetrator
- private def clearFailed(): Unit = _failed = null
- private def perpetrator: ActorRef = _failed
+ private var _failed: FailedInfo = NoFailedInfo
+ private def isFailed: Boolean = _failed.isInstanceOf[FailedRef]
+ private def isFailedFatally: Boolean = _failed eq FailedFatally
+ private def perpetrator: ActorRef = _failed match {
+ case FailedRef(ref) => ref
+ case _ => null
+ }
+ private def setFailed(perpetrator: ActorRef): Unit = _failed = _failed match {
+ case FailedFatally => FailedFatally
+ case _ => FailedRef(perpetrator)
+ }
+ private def clearFailed(): Unit = _failed = _failed match {
+ case FailedRef(_) => NoFailedInfo
+ case other => other
+ }
+ protected def setFailedFatally(): Unit = _failed = FailedFatally
/**
* Do re-create the actor in response to a failure.
@@ -65,7 +89,7 @@ private[akka] trait FaultHandling { this: ActorCell =>
val optionalMessage = if (currentMessage ne null) Some(currentMessage.message) else None
try {
// if the actor fails in preRestart, we can do nothing but log it: it’s best-effort
- if (failedActor.context ne null) failedActor.aroundPreRestart(cause, optionalMessage)
+ if (!isFailedFatally) failedActor.aroundPreRestart(cause, optionalMessage)
} catch handleNonFatalOrInterruptedException { e =>
val ex = PreRestartException(self, e, cause, optionalMessage)
publish(Error(ex, self.path.toString, clazz(failedActor), e.getMessage))
@@ -74,7 +98,7 @@ private[akka] trait FaultHandling { this: ActorCell =>
}
}
assert(mailbox.isSuspended, "mailbox must be suspended during restart, status=" + mailbox.currentStatus)
- if (!setChildrenTerminationReason(ChildrenContainer.Recreation(cause))) finishRecreate(cause, failedActor)
+ if (!setChildrenTerminationReason(ChildrenContainer.Recreation(cause))) finishRecreate(cause)
} else {
// need to keep that suspend counter balanced
faultResume(causedByFailure = null)
@@ -101,7 +125,7 @@ private[akka] trait FaultHandling { this: ActorCell =>
system.eventStream.publish(
Error(self.path.toString, clazz(actor), "changing Resume into Create after " + causedByFailure))
faultCreate()
- } else if (actor.context == null && causedByFailure != null) {
+ } else if (isFailedFatally && causedByFailure != null) {
system.eventStream.publish(
Error(self.path.toString, clazz(actor), "changing Resume into Restart after " + causedByFailure))
faultRecreate(causedByFailure)
@@ -174,6 +198,7 @@ private[akka] trait FaultHandling { this: ActorCell =>
}
}
+ @InternalStableApi
final def handleInvokeFailure(childrenNotToSuspend: immutable.Iterable[ActorRef], t: Throwable): Unit = {
// prevent any further messages to be processed until the actor has been restarted
if (!isFailed) try {
@@ -226,12 +251,11 @@ private[akka] trait FaultHandling { this: ActorCell =>
publish(Debug(self.path.toString, clazz(a), "stopped"))
clearActorFields(a, recreate = false)
- clearActorCellFields(this)
- actor = null
+ clearFieldsForTermination()
}
}
- private def finishRecreate(cause: Throwable, failedActor: Actor): Unit = {
+ private def finishRecreate(cause: Throwable): Unit = {
// need to keep a snapshot of the surviving children before the new actor instance creates new ones
val survivors = children
@@ -240,8 +264,6 @@ private[akka] trait FaultHandling { this: ActorCell =>
finally clearFailed() // must happen in any case, so that failure is propagated
val freshActor = newActor()
- actor = freshActor // this must happen before postRestart has a chance to fail
- if (freshActor eq failedActor) setActorFields(freshActor, this, self) // If the creator returns the same instance, we need to restore our nulled out fields.
freshActor.aroundPostRestart(cause)
checkReceiveTimeout(reschedule = true) // user may have set a receive timeout in preStart which is called from postRestart
@@ -255,6 +277,7 @@ private[akka] trait FaultHandling { this: ActorCell =>
publish(Error(e, self.path.toString, clazz(freshActor), "restarting " + child))
})
} catch handleNonFatalOrInterruptedException { e =>
+ setFailedFatally()
clearActorFields(actor, recreate = false) // in order to prevent preRestart() from happening again
handleInvokeFailure(survivors, PostRestartException(self, e, cause))
}
@@ -301,7 +324,7 @@ private[akka] trait FaultHandling { this: ActorCell =>
* then we are continuing the previously suspended recreate/create/terminate action
*/
status match {
- case Some(ChildrenContainer.Recreation(cause)) => finishRecreate(cause, actor)
+ case Some(ChildrenContainer.Recreation(cause)) => finishRecreate(cause)
case Some(ChildrenContainer.Creation()) => finishCreate()
case Some(ChildrenContainer.Termination) => finishTerminate()
case _ =>
diff --git a/akka-actor/src/main/scala/akka/actor/dungeon/ReceiveTimeout.scala b/akka-actor/src/main/scala/akka/actor/dungeon/ReceiveTimeout.scala
index f033423ea0..c09be5aac3 100644
--- a/akka-actor/src/main/scala/akka/actor/dungeon/ReceiveTimeout.scala
+++ b/akka-actor/src/main/scala/akka/actor/dungeon/ReceiveTimeout.scala
@@ -17,8 +17,8 @@ private[akka] object ReceiveTimeout {
private[akka] trait ReceiveTimeout { this: ActorCell =>
- import ReceiveTimeout._
import ActorCell._
+ import ReceiveTimeout._
private var receiveTimeoutData: (Duration, Cancellable) = emptyReceiveTimeoutData
diff --git a/akka-actor/src/main/scala/akka/actor/package.scala b/akka-actor/src/main/scala/akka/actor/package.scala
index 8d8912be35..7c31d57eaf 100644
--- a/akka-actor/src/main/scala/akka/actor/package.scala
+++ b/akka-actor/src/main/scala/akka/actor/package.scala
@@ -7,6 +7,6 @@ package akka
import language.implicitConversions
package object actor {
- implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef = ref.asInstanceOf[ScalaActorRef]
- implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef = ref.asInstanceOf[ActorRef]
+ @inline implicit final def actorRef2Scala(ref: ActorRef): ScalaActorRef = ref.asInstanceOf[ScalaActorRef]
+ @inline implicit final def scala2ActorRef(ref: ScalaActorRef): ActorRef = ref.asInstanceOf[ActorRef]
}
diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala
index 2a33402175..1c0ed50347 100644
--- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala
@@ -4,8 +4,16 @@
package akka.dispatch
-import java.util.concurrent._
import java.{ util => ju }
+import java.util.concurrent._
+
+import scala.annotation.tailrec
+import scala.concurrent.{ ExecutionContext, ExecutionContextExecutor }
+import scala.concurrent.duration.{ Duration, FiniteDuration }
+import scala.util.control.NonFatal
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
import akka.actor._
import akka.annotation.InternalStableApi
@@ -14,13 +22,6 @@ import akka.dispatch.sysmsg._
import akka.event.EventStream
import akka.event.Logging.{ Debug, Error, LogEventException }
import akka.util.{ unused, Index, Unsafe }
-import com.github.ghik.silencer.silent
-import com.typesafe.config.Config
-
-import scala.annotation.tailrec
-import scala.concurrent.{ ExecutionContext, ExecutionContextExecutor }
-import scala.concurrent.duration.{ Duration, FiniteDuration }
-import scala.util.control.NonFatal
final case class Envelope private (val message: Any, val sender: ActorRef)
@@ -75,7 +76,7 @@ private[akka] object MessageDispatcher {
if (debug) {
for {
d <- actors.keys
- a <- { println(d + " inhabitants: " + d.inhabitants); actors.valueIterator(d) }
+ a <- { println("" + d + " inhabitants: " + d.inhabitants); actors.valueIterator(d) }
} {
val status = if (a.isTerminated) " (terminated)" else " (alive)"
val messages = a match {
diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala
index 2fb512d805..a738defbd2 100644
--- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala
@@ -4,16 +4,18 @@
package akka.dispatch
-import akka.actor.ActorCell
-import akka.dispatch.sysmsg._
-import scala.annotation.tailrec
-import scala.concurrent.duration.Duration
-import akka.util.Helpers
import java.util.{ Comparator, Iterator }
import java.util.concurrent.ConcurrentSkipListSet
-import akka.actor.ActorSystemImpl
+
+import scala.annotation.tailrec
+import scala.concurrent.duration.Duration
import scala.concurrent.duration.FiniteDuration
+import akka.actor.ActorCell
+import akka.actor.ActorSystemImpl
+import akka.dispatch.sysmsg._
+import akka.util.Helpers
+
/**
* INTERNAL API: Use `BalancingPool` instead of this dispatcher directly.
*
@@ -103,7 +105,7 @@ private[akka] class BalancingDispatcher(
if (messageQueue.hasMessages
&& i.hasNext
&& (executorService.executor match {
- case lm: LoadMetrics => !lm.atFullThrottle
+ case lm: LoadMetrics => !lm.atFullThrottle()
case _ => true
})
&& !registerForExecution(i.next.mailbox, false, false))
diff --git a/akka-actor/src/main/scala/akka/dispatch/BatchingExecutor.scala b/akka-actor/src/main/scala/akka/dispatch/BatchingExecutor.scala
index 2dabe1b95b..cd96e3ea11 100644
--- a/akka-actor/src/main/scala/akka/dispatch/BatchingExecutor.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/BatchingExecutor.scala
@@ -4,10 +4,11 @@
package akka.dispatch
-import java.util.concurrent.{ Executor }
import java.util.ArrayDeque
-import scala.concurrent._
+import java.util.concurrent.Executor
+
import scala.annotation.tailrec
+import scala.concurrent._
/**
* All Batchables are automatically batched when submitted to a BatchingExecutor
diff --git a/akka-actor/src/main/scala/akka/dispatch/CachingConfig.scala b/akka-actor/src/main/scala/akka/dispatch/CachingConfig.scala
index e742bd5cef..2210b41d55 100644
--- a/akka-actor/src/main/scala/akka/dispatch/CachingConfig.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/CachingConfig.scala
@@ -7,10 +7,10 @@ package akka.dispatch
import java.util
import java.util.concurrent.{ ConcurrentHashMap, TimeUnit }
-import com.typesafe.config._
-
import scala.util.{ Failure, Success, Try }
+import com.typesafe.config._
+
/**
* INTERNAL API
*/
diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala
index 7eb7e631d5..7aa810ada0 100644
--- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala
@@ -4,18 +4,19 @@
package akka.dispatch
-import akka.event.Logging.Error
-import akka.actor.ActorCell
-import akka.event.Logging
-import akka.dispatch.sysmsg.SystemMessage
import java.util.concurrent.{ ExecutorService, RejectedExecutionException }
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater
import scala.concurrent.duration.Duration
import scala.concurrent.duration.FiniteDuration
-import java.util.concurrent.atomic.AtomicReferenceFieldUpdater
import com.github.ghik.silencer.silent
+import akka.actor.ActorCell
+import akka.dispatch.sysmsg.SystemMessage
+import akka.event.Logging
+import akka.event.Logging.Error
+
/**
* The event-based ``Dispatcher`` binds a set of Actors to a thread pool backed up by a
* `BlockingQueue`.
diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala
index f5d963ed42..85b5f4e83b 100644
--- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala
@@ -6,17 +6,18 @@ package akka.dispatch
import java.util.concurrent.{ ConcurrentHashMap, ThreadFactory }
-import com.typesafe.config.{ Config, ConfigFactory, ConfigValueType }
-import akka.actor.{ ActorSystem, DynamicAccess, Scheduler }
-import akka.event.Logging.Warning
-import akka.event.{ EventStream, LoggingAdapter }
-import akka.ConfigurationException
-import akka.annotation.{ DoNotInherit, InternalApi }
-import akka.util.Helpers.ConfigOps
-import com.github.ghik.silencer.silent
-
import scala.concurrent.ExecutionContext
+import com.github.ghik.silencer.silent
+import com.typesafe.config.{ Config, ConfigFactory, ConfigValueType }
+
+import akka.ConfigurationException
+import akka.actor.{ ActorSystem, DynamicAccess, Scheduler }
+import akka.annotation.{ DoNotInherit, InternalApi }
+import akka.event.{ EventStream, LoggingAdapter }
+import akka.event.Logging.Warning
+import akka.util.Helpers.ConfigOps
+
/**
* DispatcherPrerequisites represents useful contextual pieces when constructing a MessageDispatcher
*/
diff --git a/akka-actor/src/main/scala/akka/dispatch/ForkJoinExecutorConfigurator.scala b/akka-actor/src/main/scala/akka/dispatch/ForkJoinExecutorConfigurator.scala
index 61d198376e..2920bf8b70 100644
--- a/akka-actor/src/main/scala/akka/dispatch/ForkJoinExecutorConfigurator.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/ForkJoinExecutorConfigurator.scala
@@ -5,6 +5,7 @@
package akka.dispatch
import java.util.concurrent.{ ExecutorService, ForkJoinPool, ForkJoinTask, ThreadFactory }
+
import com.typesafe.config.Config
object ForkJoinExecutorConfigurator {
diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala
index 712ccf61f0..344c6d4437 100644
--- a/akka-actor/src/main/scala/akka/dispatch/Future.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala
@@ -4,23 +4,24 @@
package akka.dispatch
-import scala.runtime.{ AbstractPartialFunction, BoxedUnit }
-import akka.japi.{ Procedure, Function => JFunc, Option => JOption }
-
-import scala.concurrent.{ ExecutionContext, ExecutionContextExecutor, ExecutionContextExecutorService, Future, Promise }
import java.lang.{ Iterable => JIterable }
import java.util.{ LinkedList => JLinkedList }
import java.util.concurrent.{ Callable, Executor, ExecutorService }
-
-import scala.util.{ Failure, Success, Try }
-import java.util.concurrent.CompletionStage
import java.util.concurrent.CompletableFuture
+import java.util.concurrent.CompletionStage
+
+import scala.concurrent.{ ExecutionContext, ExecutionContextExecutor, ExecutionContextExecutorService, Future, Promise }
+import scala.runtime.{ AbstractPartialFunction, BoxedUnit }
+import scala.util.{ Failure, Success, Try }
+
+import com.github.ghik.silencer.silent
import akka.annotation.InternalApi
+import akka.annotation.InternalStableApi
import akka.compat
import akka.dispatch.internal.SameThreadExecutionContext
+import akka.japi.{ Procedure, Function => JFunc, Option => JOption }
import akka.util.unused
-import com.github.ghik.silencer.silent
/**
* ExecutionContexts is the Java API for ExecutionContexts
@@ -77,16 +78,17 @@ object ExecutionContexts {
def global(): ExecutionContextExecutor = ExecutionContext.global
/**
+ * INTERNAL API
+ *
* WARNING: Not A General Purpose ExecutionContext!
*
* This is an execution context which runs everything on the calling thread.
* It is very useful for actions which are known to be non-blocking and
* non-throwing in order to save a round-trip to the thread pool.
*
- * INTERNAL API
+ * Once Scala 2.12 is no longer supported this can be dropped in favour of directly using `ExecutionContext.parasitic`
*/
- // Once Scala 2.12 is no longer supported this can be dropped in favour of directly using [[ExecutionContext.parasitic]]
- @InternalApi
+ @InternalStableApi
private[akka] val parasitic: ExecutionContext = SameThreadExecutionContext()
/**
diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala
index 26f4e058b5..f75930f46f 100644
--- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala
@@ -4,22 +4,23 @@
package akka.dispatch
+import java.util.{ Comparator, Deque, PriorityQueue, Queue }
import java.util.concurrent._
import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.locks.ReentrantLock
-import java.util.{ Comparator, Deque, PriorityQueue, Queue }
+
+import scala.annotation.tailrec
+import scala.concurrent.duration.{ Duration, FiniteDuration }
+import scala.util.control.NonFatal
+
+import com.typesafe.config.Config
import akka.actor.{ ActorCell, ActorRef, ActorSystem, DeadLetter, InternalActorRef }
import akka.annotation.InternalStableApi
import akka.dispatch.sysmsg._
import akka.event.Logging.Error
-import akka.util.Helpers.ConfigOps
import akka.util.{ BoundedBlockingQueue, StablePriorityBlockingQueue, StablePriorityQueue, Unsafe }
-import com.typesafe.config.Config
-
-import scala.annotation.tailrec
-import scala.concurrent.duration.{ Duration, FiniteDuration }
-import scala.util.control.NonFatal
+import akka.util.Helpers.ConfigOps
/**
* INTERNAL API
@@ -265,7 +266,7 @@ private[akka] abstract class Mailbox(val messageQueue: MessageQueue)
if (shouldProcessMessage) {
val next = dequeue()
if (next ne null) {
- if (Mailbox.debug) println(actor.self + " processing message " + next)
+ if (Mailbox.debug) println("" + actor.self + " processing message " + next)
actor.invoke(next)
if (Thread.interrupted())
throw new InterruptedException("Interrupted while processing actor messages")
@@ -289,7 +290,7 @@ private[akka] abstract class Mailbox(val messageQueue: MessageQueue)
val msg = messageList.head
messageList = messageList.tail
msg.unlink()
- if (debug) println(actor.self + " processing system message " + msg + " with " + actor.childrenRefs)
+ if (debug) println("" + actor.self + " processing system message " + msg + " with " + actor.childrenRefs)
// we know here that systemInvoke ensures that only "fatal" exceptions get rethrown
actor.systemInvoke(msg)
if (Thread.interrupted())
@@ -468,7 +469,7 @@ private[akka] trait DefaultSystemMessageQueue { self: Mailbox =>
@tailrec
final def systemEnqueue(receiver: ActorRef, message: SystemMessage): Unit = {
assert(message.unlinked)
- if (Mailbox.debug) println(receiver + " having enqueued " + message)
+ if (Mailbox.debug) println("" + receiver + " having enqueued " + message)
val currentList = systemQueueGet
if (currentList.head == NoMessage) {
if (actor ne null) actor.dispatcher.mailboxes.deadLetterMailbox.systemEnqueue(receiver, message)
@@ -510,10 +511,10 @@ trait QueueBasedMessageQueue extends MessageQueue with MultipleConsumerSemantics
def hasMessages = !queue.isEmpty
def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit = {
if (hasMessages) {
- var envelope = dequeue
+ var envelope = dequeue()
while (envelope ne null) {
deadLetters.enqueue(owner, envelope)
- envelope = dequeue
+ envelope = dequeue()
}
}
}
diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailboxes.scala b/akka-actor/src/main/scala/akka/dispatch/Mailboxes.scala
index 67cc2812a0..a1dc167e21 100644
--- a/akka-actor/src/main/scala/akka/dispatch/Mailboxes.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/Mailboxes.scala
@@ -8,6 +8,12 @@ import java.lang.reflect.ParameterizedType
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.atomic.AtomicReference
+import scala.annotation.tailrec
+import scala.concurrent.duration.Duration
+import scala.util.control.NonFatal
+
+import com.typesafe.config.{ Config, ConfigFactory }
+
import akka.ConfigurationException
import akka.actor.{ Actor, ActorRef, ActorSystem, DeadLetter, Deploy, DynamicAccess, Props }
import akka.dispatch.sysmsg.{
@@ -19,11 +25,6 @@ import akka.dispatch.sysmsg.{
import akka.event.EventStream
import akka.event.Logging.Warning
import akka.util.Reflect
-import com.typesafe.config.{ Config, ConfigFactory }
-
-import scala.util.control.NonFatal
-import scala.annotation.tailrec
-import scala.concurrent.duration.Duration
object Mailboxes {
final val DefaultMailboxId = "akka.actor.default-mailbox"
@@ -139,7 +140,7 @@ private[akka] class Mailboxes(
protected[akka] def getMailboxType(props: Props, dispatcherConfig: Config): MailboxType = {
val id = dispatcherConfig.getString("id")
val deploy = props.deploy
- val actorClass = props.actorClass
+ val actorClass = props.actorClass()
lazy val actorRequirement = getRequiredType(actorClass)
val mailboxRequirement: Class[_] = getMailboxRequirement(dispatcherConfig)
diff --git a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala
index bd1844ec48..3f0510a4e9 100644
--- a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala
@@ -4,10 +4,11 @@
package akka.dispatch
-import akka.actor.ActorCell
import scala.concurrent.duration.Duration
import scala.concurrent.duration.FiniteDuration
+import akka.actor.ActorCell
+
/**
* Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue.
*
diff --git a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala
index fa95e326d1..c7436f3d84 100644
--- a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala
@@ -5,8 +5,6 @@
package akka.dispatch
import java.util.Collection
-import scala.concurrent.{ BlockContext, CanAwait }
-import scala.concurrent.duration.Duration
import java.util.concurrent.{
ArrayBlockingQueue,
BlockingQueue,
@@ -24,6 +22,9 @@ import java.util.concurrent.{
}
import java.util.concurrent.atomic.{ AtomicLong, AtomicReference }
+import scala.concurrent.{ BlockContext, CanAwait }
+import scala.concurrent.duration.Duration
+
object ThreadPoolConfig {
type QueueFactory = () => BlockingQueue[Runnable]
diff --git a/akka-actor/src/main/scala/akka/dispatch/affinity/AffinityPool.scala b/akka-actor/src/main/scala/akka/dispatch/affinity/AffinityPool.scala
index e252976a1f..6fb5cddeac 100644
--- a/akka-actor/src/main/scala/akka/dispatch/affinity/AffinityPool.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/affinity/AffinityPool.scala
@@ -4,26 +4,23 @@
package akka.dispatch.affinity
-import java.lang.invoke.MethodHandles
-import java.lang.invoke.MethodType.methodType
+import java.lang.Integer.reverseBytes
import java.util.Collections
-import java.util.concurrent.TimeUnit.MICROSECONDS
import java.util.concurrent._
+import java.util.concurrent.TimeUnit.MICROSECONDS
import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.locks.LockSupport
-import java.lang.Integer.reverseBytes
-
-import akka.dispatch._
-import akka.util.Helpers.Requiring
-import com.typesafe.config.Config
-
-import akka.annotation.{ ApiMayChange, InternalApi }
-import akka.event.Logging
-import akka.util.{ ImmutableIntMap, OptionVal, ReentrantGuard }
import scala.annotation.{ switch, tailrec }
import scala.collection.{ immutable, mutable }
-import scala.util.control.NonFatal
+
+import com.typesafe.config.Config
+
+import akka.annotation.{ ApiMayChange, InternalApi }
+import akka.dispatch._
+import akka.event.Logging
+import akka.util.{ ImmutableIntMap, ReentrantGuard }
+import akka.util.Helpers.Requiring
@InternalApi
@ApiMayChange
@@ -42,13 +39,6 @@ private[affinity] object AffinityPool {
// PoolState: all threads have been stopped, does not process tasks and does not accept new ones
final val Terminated = 5
- // Method handle to JDK9+ onSpinWait method
- private val onSpinWaitMethodHandle =
- try OptionVal.Some(MethodHandles.lookup.findStatic(classOf[Thread], "onSpinWait", methodType(classOf[Void])))
- catch {
- case NonFatal(_) => OptionVal.None
- }
-
type IdleState = Int
// IdleState: Initial state
final val Initial = 0
@@ -85,10 +75,7 @@ private[affinity] object AffinityPool {
idling = true
transitionTo(Spinning)
case Spinning =>
- onSpinWaitMethodHandle match {
- case OptionVal.Some(m) => m.invokeExact()
- case OptionVal.None =>
- }
+ OnSpinWait.spinWait()
turns += 1
if (turns > maxSpins)
transitionTo(Yielding)
diff --git a/akka-actor/src/main/scala/akka/dispatch/sysmsg/SystemMessage.scala b/akka-actor/src/main/scala/akka/dispatch/sysmsg/SystemMessage.scala
index a3ccdb1d81..3d691699b3 100644
--- a/akka-actor/src/main/scala/akka/dispatch/sysmsg/SystemMessage.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/sysmsg/SystemMessage.scala
@@ -5,6 +5,7 @@
package akka.dispatch.sysmsg
import scala.annotation.tailrec
+
import akka.actor.{ ActorInitializationException, ActorRef, InternalActorRef, PossiblyHarmful }
import akka.actor.DeadLetterSuppression
import akka.annotation.InternalStableApi
diff --git a/akka-actor/src/main/scala/akka/event/ActorClassificationUnsubscriber.scala b/akka-actor/src/main/scala/akka/event/ActorClassificationUnsubscriber.scala
index 6d0a76abd8..927147b082 100644
--- a/akka-actor/src/main/scala/akka/event/ActorClassificationUnsubscriber.scala
+++ b/akka-actor/src/main/scala/akka/event/ActorClassificationUnsubscriber.scala
@@ -4,10 +4,10 @@
package akka.event
-import akka.actor._
-import akka.event.Logging.simpleName
import java.util.concurrent.atomic.AtomicInteger
+import akka.actor._
+import akka.event.Logging.simpleName
import akka.util.unused
/**
diff --git a/akka-actor/src/main/scala/akka/event/AddressTerminatedTopic.scala b/akka-actor/src/main/scala/akka/event/AddressTerminatedTopic.scala
index 7dd3560a80..316f3ad805 100644
--- a/akka-actor/src/main/scala/akka/event/AddressTerminatedTopic.scala
+++ b/akka-actor/src/main/scala/akka/event/AddressTerminatedTopic.scala
@@ -7,6 +7,7 @@ package akka.event
import java.util.concurrent.atomic.AtomicReference
import scala.annotation.tailrec
+
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.AddressTerminated
diff --git a/akka-actor/src/main/scala/akka/event/EventBus.scala b/akka-actor/src/main/scala/akka/event/EventBus.scala
index f2f7fd7d80..8b1331efe3 100644
--- a/akka-actor/src/main/scala/akka/event/EventBus.scala
+++ b/akka-actor/src/main/scala/akka/event/EventBus.scala
@@ -4,13 +4,15 @@
package akka.event
-import akka.actor.{ ActorRef, ActorSystem }
-import akka.util.Index
-import java.util.concurrent.ConcurrentSkipListSet
import java.util.Comparator
-import akka.util.{ Subclassification, SubclassifiedIndex }
+import java.util.concurrent.ConcurrentSkipListSet
+import java.util.concurrent.atomic.AtomicReference
+
import scala.collection.immutable
-import java.util.concurrent.atomic.{ AtomicReference }
+
+import akka.actor.{ ActorRef, ActorSystem }
+import akka.util.{ Subclassification, SubclassifiedIndex }
+import akka.util.Index
/**
* Represents the base type for EventBuses
diff --git a/akka-actor/src/main/scala/akka/event/EventStream.scala b/akka-actor/src/main/scala/akka/event/EventStream.scala
index f053563353..82771cad9c 100644
--- a/akka-actor/src/main/scala/akka/event/EventStream.scala
+++ b/akka-actor/src/main/scala/akka/event/EventStream.scala
@@ -4,14 +4,15 @@
package akka.event
-import akka.actor.{ ActorRef, ActorSystem }
-import akka.event.Logging.simpleName
-import akka.util.Subclassification
import java.util.concurrent.atomic.AtomicReference
+import scala.annotation.tailrec
+
import com.github.ghik.silencer.silent
-import scala.annotation.tailrec
+import akka.actor.{ ActorRef, ActorSystem }
+import akka.event.Logging.simpleName
+import akka.util.Subclassification
/**
* An Akka EventStream is a pub-sub stream of events both system and user generated,
@@ -32,7 +33,7 @@ class EventStream(sys: ActorSystem, private val debug: Boolean) extends LoggingB
/** Either the list of subscribed actors, or a ref to an [[akka.event.EventStreamUnsubscriber]] */
private val initiallySubscribedOrUnsubscriber = new AtomicReference[Either[Set[ActorRef], ActorRef]](Left(Set.empty))
- protected implicit val subclassification = new Subclassification[Class[_]] {
+ protected implicit val subclassification: Subclassification[Classifier] = new Subclassification[Class[_]] {
def isEqual(x: Class[_], y: Class[_]) = x == y
def isSubclass(x: Class[_], y: Class[_]) = y.isAssignableFrom(x)
}
diff --git a/akka-actor/src/main/scala/akka/event/EventStreamUnsubscriber.scala b/akka-actor/src/main/scala/akka/event/EventStreamUnsubscriber.scala
index 7b240c661f..52598fc6a1 100644
--- a/akka-actor/src/main/scala/akka/event/EventStreamUnsubscriber.scala
+++ b/akka-actor/src/main/scala/akka/event/EventStreamUnsubscriber.scala
@@ -4,11 +4,11 @@
package akka.event
-import akka.actor._
-import akka.event.Logging.simpleName
import java.util.concurrent.atomic.AtomicInteger
+import akka.actor._
import akka.dispatch.Dispatchers
+import akka.event.Logging.simpleName
/**
* INTERNAL API
diff --git a/akka-actor/src/main/scala/akka/event/LoggerMailbox.scala b/akka-actor/src/main/scala/akka/event/LoggerMailbox.scala
index ce14253e3b..926b79a5ab 100644
--- a/akka-actor/src/main/scala/akka/event/LoggerMailbox.scala
+++ b/akka-actor/src/main/scala/akka/event/LoggerMailbox.scala
@@ -4,13 +4,14 @@
package akka.event
-import akka.dispatch.MessageQueue
-import akka.dispatch.MailboxType
-import akka.dispatch.UnboundedMailbox
import com.typesafe.config.Config
-import akka.actor.ActorSystem
+
import akka.actor.ActorRef
+import akka.actor.ActorSystem
+import akka.dispatch.MailboxType
+import akka.dispatch.MessageQueue
import akka.dispatch.ProducesMessageQueue
+import akka.dispatch.UnboundedMailbox
import akka.event.Logging.LogEvent
import akka.util.unused
@@ -39,7 +40,7 @@ private[akka] class LoggerMailbox(@unused owner: ActorRef, system: ActorSystem)
override def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit = {
if (hasMessages) {
val logLevel = system.eventStream.logLevel
- var envelope = dequeue
+ var envelope = dequeue()
// Drain all remaining messages to the StandardOutLogger.
// cleanUp is called after switching out the mailbox, which is why
// this kind of look works without a limit.
@@ -54,7 +55,7 @@ private[akka] class LoggerMailbox(@unused owner: ActorRef, system: ActorSystem)
case _ => // skip
}
- envelope = dequeue
+ envelope = dequeue()
}
}
super.cleanUp(owner, deadLetters)
diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala
index 5cd62aab21..26621bca24 100644
--- a/akka-actor/src/main/scala/akka/event/Logging.scala
+++ b/akka-actor/src/main/scala/akka/event/Logging.scala
@@ -7,22 +7,24 @@ package akka.event
import java.util.concurrent.TimeoutException
import java.util.concurrent.atomic.AtomicInteger
-import akka.actor.ActorSystem.Settings
-import akka.actor._
-import akka.annotation.{ DoNotInherit, InternalApi }
-import akka.dispatch.RequiresMessageQueue
-import akka.event.Logging._
-import akka.util.unused
-import akka.util.{ Helpers, ReentrantGuard }
-import akka.{ AkkaException, ConfigurationException }
-import com.github.ghik.silencer.silent
-
import scala.annotation.implicitNotFound
import scala.collection.immutable
import scala.concurrent.Await
import scala.language.existentials
import scala.util.control.{ NoStackTrace, NonFatal }
+import com.github.ghik.silencer.silent
+
+import akka.{ AkkaException, ConfigurationException }
+import akka.actor._
+import akka.actor.ActorSystem.Settings
+import akka.annotation.{ DoNotInherit, InternalApi }
+import akka.dispatch.RequiresMessageQueue
+import akka.event.Logging._
+import akka.util.{ Helpers, ReentrantGuard }
+import akka.util.Timeout
+import akka.util.unused
+
/**
* This trait brings log level handling to the EventStream: it reads the log
* levels for the initial logging (StandardOutLogger) and the loggers & level
@@ -198,7 +200,7 @@ trait LoggingBus extends ActorEventBus {
logName: String): ActorRef = {
val name = "log" + LogExt(system).id() + "-" + simpleName(clazz)
val actor = system.systemActorOf(Props(clazz).withDispatcher(system.settings.LoggersDispatcher), name)
- implicit def timeout = system.settings.LoggerStartTimeout
+ implicit def timeout: Timeout = system.settings.LoggerStartTimeout
import akka.pattern.ask
val response = try Await.result(actor ? InitializeLogger(this), timeout.duration)
catch {
diff --git a/akka-actor/src/main/scala/akka/event/LoggingReceive.scala b/akka-actor/src/main/scala/akka/event/LoggingReceive.scala
index 15b2a43f4a..9cb0d64e3e 100644
--- a/akka-actor/src/main/scala/akka/event/LoggingReceive.scala
+++ b/akka-actor/src/main/scala/akka/event/LoggingReceive.scala
@@ -4,14 +4,16 @@
package akka.event
+import scala.runtime.BoxedUnit
+
import language.existentials
+
+import akka.actor.AbstractActor
import akka.actor.Actor.Receive
-import akka.actor.ActorContext
import akka.actor.ActorCell
+import akka.actor.ActorContext
import akka.actor.DiagnosticActorLogging
import akka.event.Logging.{ LogEvent, LogLevel }
-import akka.actor.AbstractActor
-import scala.runtime.BoxedUnit
object LoggingReceive {
diff --git a/akka-actor/src/main/scala/akka/event/japi/EventBusJavaAPI.scala b/akka-actor/src/main/scala/akka/event/japi/EventBusJavaAPI.scala
index 1eed68f01b..0de71a0f5c 100644
--- a/akka-actor/src/main/scala/akka/event/japi/EventBusJavaAPI.scala
+++ b/akka-actor/src/main/scala/akka/event/japi/EventBusJavaAPI.scala
@@ -4,8 +4,8 @@
package akka.event.japi
-import akka.util.Subclassification
import akka.actor.{ ActorRef, ActorSystem }
+import akka.util.Subclassification
/**
* Java API: See documentation for [[akka.event.EventBus]]
@@ -50,7 +50,7 @@ abstract class LookupEventBus[E, S, C] extends EventBus[E, S, C] {
type Subscriber = S
type Classifier = C
- override protected def mapSize: Int = LookupEventBus.this.mapSize
+ override protected def mapSize: Int = LookupEventBus.this.mapSize()
override protected def compareSubscribers(a: S, b: S): Int =
LookupEventBus.this.compareSubscribers(a, b)
@@ -197,7 +197,7 @@ abstract class ManagedActorEventBus[E](system: ActorSystem) extends EventBus[E,
override val system = ManagedActorEventBus.this.system
- override protected def mapSize: Int = ManagedActorEventBus.this.mapSize
+ override protected def mapSize: Int = ManagedActorEventBus.this.mapSize()
override protected def classify(event: E): ActorRef =
ManagedActorEventBus.this.classify(event)
diff --git a/akka-actor/src/main/scala/akka/io/DirectByteBufferPool.scala b/akka-actor/src/main/scala/akka/io/DirectByteBufferPool.scala
index 075002ff83..287295af30 100644
--- a/akka-actor/src/main/scala/akka/io/DirectByteBufferPool.scala
+++ b/akka-actor/src/main/scala/akka/io/DirectByteBufferPool.scala
@@ -5,6 +5,7 @@
package akka.io
import java.nio.ByteBuffer
+
import scala.util.control.NonFatal
trait BufferPool {
diff --git a/akka-actor/src/main/scala/akka/io/Dns.scala b/akka-actor/src/main/scala/akka/io/Dns.scala
index 3d93cde0be..769b61ebea 100644
--- a/akka-actor/src/main/scala/akka/io/Dns.scala
+++ b/akka-actor/src/main/scala/akka/io/Dns.scala
@@ -6,23 +6,23 @@ package akka.io
import java.net.{ Inet4Address, Inet6Address, InetAddress, UnknownHostException }
import java.util.concurrent.ConcurrentHashMap
-
-import akka.actor._
-import akka.annotation.InternalApi
-import akka.routing.ConsistentHashingRouter.ConsistentHashable
-import com.typesafe.config.Config
import java.util.function.{ Function => JFunction }
+import scala.collection.immutable
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+
+import akka.actor._
import akka.annotation.DoNotInherit
+import akka.annotation.InternalApi
+import akka.event.Logging
import akka.io.dns.AAAARecord
import akka.io.dns.ARecord
import akka.io.dns.DnsProtocol
-import akka.util.unused
-import scala.collection.immutable
-
-import akka.event.Logging
+import akka.routing.ConsistentHashingRouter.ConsistentHashable
import akka.util.ccompat._
-import com.github.ghik.silencer.silent
+import akka.util.unused
/**
* Not for user extension.
diff --git a/akka-actor/src/main/scala/akka/io/Inet.scala b/akka-actor/src/main/scala/akka/io/Inet.scala
index cf6e6e551c..0597ab96fa 100644
--- a/akka-actor/src/main/scala/akka/io/Inet.scala
+++ b/akka-actor/src/main/scala/akka/io/Inet.scala
@@ -4,10 +4,10 @@
package akka.io
-import java.nio.channels.DatagramChannel
import java.net.DatagramSocket
import java.net.ServerSocket
import java.net.Socket
+import java.nio.channels.DatagramChannel
import akka.util.unused
diff --git a/akka-actor/src/main/scala/akka/io/InetAddressDnsProvider.scala b/akka-actor/src/main/scala/akka/io/InetAddressDnsProvider.scala
index e42adaef07..d30ef9f92b 100644
--- a/akka-actor/src/main/scala/akka/io/InetAddressDnsProvider.scala
+++ b/akka-actor/src/main/scala/akka/io/InetAddressDnsProvider.scala
@@ -4,9 +4,10 @@
package akka.io
-import akka.annotation.InternalApi
import com.github.ghik.silencer.silent
+import akka.annotation.InternalApi
+
/**
* INTERNAL API
*/
diff --git a/akka-actor/src/main/scala/akka/io/InetAddressDnsResolver.scala b/akka-actor/src/main/scala/akka/io/InetAddressDnsResolver.scala
index 601f974c0b..ca8c6b0b8a 100644
--- a/akka-actor/src/main/scala/akka/io/InetAddressDnsResolver.scala
+++ b/akka-actor/src/main/scala/akka/io/InetAddressDnsResolver.scala
@@ -4,29 +4,30 @@
package akka.io
+import java.net.{ InetAddress, UnknownHostException }
import java.net.Inet4Address
import java.net.Inet6Address
-import java.net.{ InetAddress, UnknownHostException }
import java.security.Security
import java.util.concurrent.TimeUnit
-import akka.actor.Status
-import akka.io.dns.CachePolicy._
+import scala.collection.immutable
+import scala.concurrent.duration._
+import scala.util.{ Failure, Success, Try }
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+
import akka.actor.{ Actor, ActorLogging }
+import akka.actor.Status
import akka.annotation.InternalApi
import akka.io.dns.AAAARecord
import akka.io.dns.ARecord
+import akka.io.dns.CachePolicy._
import akka.io.dns.DnsProtocol
import akka.io.dns.DnsProtocol.Ip
import akka.io.dns.DnsProtocol.Srv
import akka.io.dns.ResourceRecord
import akka.util.Helpers.Requiring
-import com.github.ghik.silencer.silent
-import com.typesafe.config.Config
-
-import scala.collection.immutable
-import scala.concurrent.duration._
-import scala.util.{ Failure, Success, Try }
/**
* INTERNAL API
diff --git a/akka-actor/src/main/scala/akka/io/SelectionHandler.scala b/akka-actor/src/main/scala/akka/io/SelectionHandler.scala
index 3b9f5df775..08aebfed7e 100644
--- a/akka-actor/src/main/scala/akka/io/SelectionHandler.scala
+++ b/akka-actor/src/main/scala/akka/io/SelectionHandler.scala
@@ -4,27 +4,27 @@
package akka.io
-import java.util.{ Iterator => JIterator }
-import java.util.concurrent.atomic.AtomicBoolean
import java.nio.channels.{ CancelledKeyException, SelectableChannel, SelectionKey }
+import java.nio.channels.ClosedChannelException
import java.nio.channels.SelectionKey._
import java.nio.channels.spi.SelectorProvider
+import java.util.{ Iterator => JIterator }
+import java.util.concurrent.atomic.AtomicBoolean
+
+import scala.annotation.tailrec
+import scala.concurrent.ExecutionContext
+import scala.util.Try
+import scala.util.control.NonFatal
import com.typesafe.config.Config
-import scala.annotation.tailrec
-import scala.util.control.NonFatal
-import scala.concurrent.ExecutionContext
-import akka.event.LoggingAdapter
+import akka.actor._
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
+import akka.event.Logging
+import akka.event.LoggingAdapter
+import akka.routing.RandomPool
import akka.util.Helpers.Requiring
import akka.util.SerializedSuspendableExecutionContext
-import akka.actor._
-import akka.routing.RandomPool
-import akka.event.Logging
-import java.nio.channels.ClosedChannelException
-
-import scala.util.Try
abstract class SelectionHandlerSettings(config: Config) {
import config._
diff --git a/akka-actor/src/main/scala/akka/io/SimpleDnsCache.scala b/akka-actor/src/main/scala/akka/io/SimpleDnsCache.scala
index d474aab4b0..ae30331233 100644
--- a/akka-actor/src/main/scala/akka/io/SimpleDnsCache.scala
+++ b/akka-actor/src/main/scala/akka/io/SimpleDnsCache.scala
@@ -6,19 +6,20 @@ package akka.io
import java.util.concurrent.atomic.AtomicReference
+import scala.annotation.tailrec
+import scala.collection.immutable
+
+import com.github.ghik.silencer.silent
+
import akka.actor.NoSerializationVerificationNeeded
import akka.annotation.InternalApi
+import akka.io.dns.{ AAAARecord, ARecord }
import akka.io.dns.CachePolicy.CachePolicy
import akka.io.dns.CachePolicy.Forever
import akka.io.dns.CachePolicy.Never
import akka.io.dns.CachePolicy.Ttl
import akka.io.dns.DnsProtocol
import akka.io.dns.DnsProtocol.{ Ip, RequestType, Resolved }
-import akka.io.dns.{ AAAARecord, ARecord }
-import com.github.ghik.silencer.silent
-
-import scala.annotation.tailrec
-import scala.collection.immutable
private[io] trait PeriodicCacheCleanup {
def cleanup(): Unit
@@ -30,7 +31,7 @@ class SimpleDnsCache extends Dns with PeriodicCacheCleanup with NoSerializationV
new Cache[(String, RequestType), Resolved](
immutable.SortedSet()(expiryEntryOrdering()),
immutable.Map(),
- () => clock))
+ () => clock()))
private val nanoBase = System.nanoTime()
diff --git a/akka-actor/src/main/scala/akka/io/SimpleDnsManager.scala b/akka-actor/src/main/scala/akka/io/SimpleDnsManager.scala
index e2ea4c1192..aa055ff321 100644
--- a/akka-actor/src/main/scala/akka/io/SimpleDnsManager.scala
+++ b/akka-actor/src/main/scala/akka/io/SimpleDnsManager.scala
@@ -6,12 +6,12 @@ package akka.io
import java.util.concurrent.TimeUnit
+import scala.concurrent.duration.Duration
+
import akka.actor.{ Actor, ActorLogging, Deploy, Props }
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
import akka.routing.FromConfig
-import scala.concurrent.duration.Duration
-
final class SimpleDnsManager(val ext: DnsExt)
extends Actor
with RequiresMessageQueue[UnboundedMessageQueueSemantics]
diff --git a/akka-actor/src/main/scala/akka/io/Tcp.scala b/akka-actor/src/main/scala/akka/io/Tcp.scala
index 3fd57eed9f..953851ae6e 100644
--- a/akka-actor/src/main/scala/akka/io/Tcp.scala
+++ b/akka-actor/src/main/scala/akka/io/Tcp.scala
@@ -4,24 +4,24 @@
package akka.io
+import java.lang.{ Iterable => JIterable }
import java.net.InetSocketAddress
import java.net.Socket
+import java.nio.file.{ Path, Paths }
-import akka.io.Inet._
+import scala.collection.immutable
+import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
import com.typesafe.config.Config
-import scala.concurrent.duration._
-import scala.collection.immutable
-import akka.util.ccompat.JavaConverters._
+import akka.actor._
+import akka.annotation.InternalApi
+import akka.io.Inet._
import akka.util.{ ByteString, Helpers }
import akka.util.Helpers.Requiring
import akka.util.JavaDurationConverters._
-import akka.actor._
-import java.lang.{ Iterable => JIterable }
-import java.nio.file.{ Path, Paths }
-
-import akka.annotation.InternalApi
-import com.github.ghik.silencer.silent
+import akka.util.ccompat.JavaConverters._
/**
* TCP Extension for Akka’s IO layer.
@@ -597,9 +597,10 @@ class TcpExt(system: ExtendedActorSystem) extends IO.Extension {
val Settings = new Settings(system.settings.config.getConfig("akka.io.tcp"))
class Settings private[TcpExt] (_config: Config) extends SelectionHandlerSettings(_config) {
- import akka.util.Helpers.ConfigOps
import _config._
+ import akka.util.Helpers.ConfigOps
+
val NrOfSelectors: Int = getInt("nr-of-selectors").requiring(_ > 0, "nr-of-selectors must be > 0")
val BatchAcceptLimit: Int = getInt("batch-accept-limit").requiring(_ > 0, "batch-accept-limit must be > 0")
@@ -690,8 +691,8 @@ object TcpSO extends SoJavaFactories {
}
object TcpMessage {
- import language.implicitConversions
import Tcp._
+ import language.implicitConversions
/**
* The Connect message is sent to the TCP manager actor, which is obtained via
diff --git a/akka-actor/src/main/scala/akka/io/TcpConnection.scala b/akka-actor/src/main/scala/akka/io/TcpConnection.scala
index 4384444b0e..d1b8ac5a7f 100644
--- a/akka-actor/src/main/scala/akka/io/TcpConnection.scala
+++ b/akka-actor/src/main/scala/akka/io/TcpConnection.scala
@@ -7,22 +7,23 @@ package akka.io
import java.io.IOException
import java.net.{ InetSocketAddress, SocketException }
import java.nio.ByteBuffer
-import java.nio.channels.SelectionKey._
import java.nio.channels.{ FileChannel, SocketChannel }
+import java.nio.channels.SelectionKey._
import java.nio.file.{ Path, Paths }
+import scala.annotation.tailrec
+import scala.collection.immutable
+import scala.concurrent.duration._
+import scala.util.control.{ NoStackTrace, NonFatal }
+
+import com.github.ghik.silencer.silent
+
import akka.actor._
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
import akka.io.Inet.SocketOption
import akka.io.SelectionHandler._
import akka.io.Tcp._
import akka.util.ByteString
-import com.github.ghik.silencer.silent
-
-import scala.annotation.tailrec
-import scala.collection.immutable
-import scala.concurrent.duration._
-import scala.util.control.{ NoStackTrace, NonFatal }
/**
* Base class for TcpIncomingConnection and TcpOutgoingConnection.
diff --git a/akka-actor/src/main/scala/akka/io/TcpIncomingConnection.scala b/akka-actor/src/main/scala/akka/io/TcpIncomingConnection.scala
index 76d5532583..ba20bca2ef 100644
--- a/akka-actor/src/main/scala/akka/io/TcpIncomingConnection.scala
+++ b/akka-actor/src/main/scala/akka/io/TcpIncomingConnection.scala
@@ -7,9 +7,11 @@ package akka.io
import java.nio.channels.SocketChannel
import scala.collection.immutable
+
+import com.github.ghik.silencer.silent
+
import akka.actor.ActorRef
import akka.io.Inet.SocketOption
-import com.github.ghik.silencer.silent
/**
* An actor handling the connection state machine for an incoming, already connected
diff --git a/akka-actor/src/main/scala/akka/io/TcpListener.scala b/akka-actor/src/main/scala/akka/io/TcpListener.scala
index 98d93c694a..3a31444b54 100644
--- a/akka-actor/src/main/scala/akka/io/TcpListener.scala
+++ b/akka-actor/src/main/scala/akka/io/TcpListener.scala
@@ -4,15 +4,16 @@
package akka.io
-import java.nio.channels.{ SelectionKey, ServerSocketChannel, SocketChannel }
import java.net.InetSocketAddress
+import java.nio.channels.{ SelectionKey, ServerSocketChannel, SocketChannel }
import scala.annotation.tailrec
import scala.util.control.NonFatal
+
import akka.actor._
+import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
import akka.io.SelectionHandler._
import akka.io.Tcp._
-import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
/**
* INTERNAL API
diff --git a/akka-actor/src/main/scala/akka/io/TcpManager.scala b/akka-actor/src/main/scala/akka/io/TcpManager.scala
index b9a9a44dce..65d4e2ea39 100644
--- a/akka-actor/src/main/scala/akka/io/TcpManager.scala
+++ b/akka-actor/src/main/scala/akka/io/TcpManager.scala
@@ -5,6 +5,7 @@
package akka.io
import Tcp._
+
import akka.actor.{ ActorLogging, Props }
/**
diff --git a/akka-actor/src/main/scala/akka/io/TcpOutgoingConnection.scala b/akka-actor/src/main/scala/akka/io/TcpOutgoingConnection.scala
index c3d00383d3..960e894dd6 100644
--- a/akka-actor/src/main/scala/akka/io/TcpOutgoingConnection.scala
+++ b/akka-actor/src/main/scala/akka/io/TcpOutgoingConnection.scala
@@ -7,15 +7,15 @@ package akka.io
import java.net.{ ConnectException, InetSocketAddress }
import java.nio.channels.{ SelectionKey, SocketChannel }
-import akka.actor.Status.Failure
-
-import scala.util.control.{ NoStackTrace, NonFatal }
import scala.concurrent.duration._
+import scala.util.control.{ NoStackTrace, NonFatal }
+
import akka.actor.{ ActorRef, ReceiveTimeout }
+import akka.actor.Status.Failure
import akka.annotation.InternalApi
-import akka.io.TcpConnection.CloseInformation
import akka.io.SelectionHandler._
import akka.io.Tcp._
+import akka.io.TcpConnection.CloseInformation
import akka.io.dns.DnsProtocol
/**
@@ -35,8 +35,8 @@ private[io] class TcpOutgoingConnection(
connect.pullMode) {
import TcpOutgoingConnection._
- import context._
import connect._
+ import context._
signDeathPact(commander)
diff --git a/akka-actor/src/main/scala/akka/io/Udp.scala b/akka-actor/src/main/scala/akka/io/Udp.scala
index f8fb701e78..e73915ae03 100644
--- a/akka-actor/src/main/scala/akka/io/Udp.scala
+++ b/akka-actor/src/main/scala/akka/io/Udp.scala
@@ -7,15 +7,16 @@ package akka.io
import java.net.DatagramSocket
import java.net.InetSocketAddress
+import scala.collection.immutable
+
+import com.github.ghik.silencer.silent
import com.typesafe.config.Config
-import scala.collection.immutable
-import akka.io.Inet.{ SoJavaFactories, SocketOption }
-import akka.util.Helpers.Requiring
-import akka.util.ByteString
import akka.actor._
+import akka.io.Inet.{ SoJavaFactories, SocketOption }
+import akka.util.ByteString
+import akka.util.Helpers.Requiring
import akka.util.ccompat._
-import com.github.ghik.silencer.silent
/**
* UDP Extension for Akka’s IO layer.
@@ -244,8 +245,10 @@ class UdpExt(system: ExtendedActorSystem) extends IO.Extension {
* Java API: factory methods for the message types used when communicating with the Udp service.
*/
object UdpMessage {
- import Udp._
import java.lang.{ Iterable => JIterable }
+
+ import Udp._
+
import akka.util.ccompat.JavaConverters._
/**
diff --git a/akka-actor/src/main/scala/akka/io/UdpConnected.scala b/akka-actor/src/main/scala/akka/io/UdpConnected.scala
index 8632e316cf..30d076cb31 100644
--- a/akka-actor/src/main/scala/akka/io/UdpConnected.scala
+++ b/akka-actor/src/main/scala/akka/io/UdpConnected.scala
@@ -8,12 +8,14 @@ import java.lang.{ Iterable => JIterable }
import java.net.InetSocketAddress
import scala.collection.immutable
+
+import com.github.ghik.silencer.silent
+
+import akka.actor._
import akka.io.Inet.SocketOption
import akka.io.Udp.UdpSettings
import akka.util.ByteString
-import akka.actor._
import akka.util.ccompat._
-import com.github.ghik.silencer.silent
/**
* UDP Extension for Akka’s IO layer.
@@ -180,8 +182,8 @@ class UdpConnectedExt(system: ExtendedActorSystem) extends IO.Extension {
* Java API: factory methods for the message types used when communicating with the UdpConnected service.
*/
object UdpConnectedMessage {
- import language.implicitConversions
import UdpConnected._
+ import language.implicitConversions
/**
* Send this message to the [[UdpExt#manager]] in order to bind to a local
diff --git a/akka-actor/src/main/scala/akka/io/UdpConnection.scala b/akka-actor/src/main/scala/akka/io/UdpConnection.scala
index f953a58bee..ced1774b0b 100644
--- a/akka-actor/src/main/scala/akka/io/UdpConnection.scala
+++ b/akka-actor/src/main/scala/akka/io/UdpConnection.scala
@@ -9,16 +9,16 @@ import java.nio.ByteBuffer
import java.nio.channels.DatagramChannel
import java.nio.channels.SelectionKey._
-import akka.actor.Status.Failure
-
import scala.annotation.tailrec
import scala.util.control.NonFatal
+
import akka.actor.{ Actor, ActorLogging, ActorRef }
+import akka.actor.Status.Failure
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
-import akka.util.{ unused, ByteString }
import akka.io.SelectionHandler._
import akka.io.UdpConnected._
import akka.io.dns.DnsProtocol
+import akka.util.{ unused, ByteString }
/**
* INTERNAL API
diff --git a/akka-actor/src/main/scala/akka/io/UdpListener.scala b/akka-actor/src/main/scala/akka/io/UdpListener.scala
index aaaaad92b5..262dc4df3d 100644
--- a/akka-actor/src/main/scala/akka/io/UdpListener.scala
+++ b/akka-actor/src/main/scala/akka/io/UdpListener.scala
@@ -10,12 +10,13 @@ import java.nio.channels.SelectionKey._
import scala.annotation.tailrec
import scala.util.control.NonFatal
+
import akka.actor.{ Actor, ActorLogging, ActorRef }
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
-import akka.util.ByteString
import akka.io.Inet.DatagramChannelCreator
import akka.io.SelectionHandler._
import akka.io.Udp._
+import akka.util.ByteString
/**
* INTERNAL API
diff --git a/akka-actor/src/main/scala/akka/io/UdpSender.scala b/akka-actor/src/main/scala/akka/io/UdpSender.scala
index 586b59af0c..e045461058 100644
--- a/akka-actor/src/main/scala/akka/io/UdpSender.scala
+++ b/akka-actor/src/main/scala/akka/io/UdpSender.scala
@@ -8,11 +8,13 @@ import java.nio.channels.DatagramChannel
import scala.collection.immutable
import scala.util.control.NonFatal
+
+import com.github.ghik.silencer.silent
+
+import akka.actor._
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
import akka.io.Inet.SocketOption
import akka.io.Udp._
-import akka.actor._
-import com.github.ghik.silencer.silent
/**
* INTERNAL API
diff --git a/akka-actor/src/main/scala/akka/io/WithUdpSend.scala b/akka-actor/src/main/scala/akka/io/WithUdpSend.scala
index 2e77ac6794..e3f8ca88bf 100644
--- a/akka-actor/src/main/scala/akka/io/WithUdpSend.scala
+++ b/akka-actor/src/main/scala/akka/io/WithUdpSend.scala
@@ -7,13 +7,13 @@ package akka.io
import java.net.InetSocketAddress
import java.nio.channels.{ DatagramChannel, SelectionKey }
-import akka.actor.{ Actor, ActorLogging, ActorRef }
-import akka.io.Udp.{ CommandFailed, Send }
-import akka.io.SelectionHandler._
-import akka.io.dns.DnsProtocol
-
import scala.util.control.NonFatal
+import akka.actor.{ Actor, ActorLogging, ActorRef }
+import akka.io.SelectionHandler._
+import akka.io.Udp.{ CommandFailed, Send }
+import akka.io.dns.DnsProtocol
+
/**
* INTERNAL API
*/
diff --git a/akka-actor/src/main/scala/akka/io/dns/CachePolicy.scala b/akka-actor/src/main/scala/akka/io/dns/CachePolicy.scala
index 8c818c64cc..9622b3e149 100644
--- a/akka-actor/src/main/scala/akka/io/dns/CachePolicy.scala
+++ b/akka-actor/src/main/scala/akka/io/dns/CachePolicy.scala
@@ -4,11 +4,11 @@
package akka.io.dns
+import scala.concurrent.duration.{ Duration, FiniteDuration, _ }
+
import akka.annotation.InternalApi
import akka.util.JavaDurationConverters._
-import scala.concurrent.duration.{ Duration, FiniteDuration, _ }
-
object CachePolicy {
sealed trait CachePolicy
diff --git a/akka-actor/src/main/scala/akka/io/dns/DnsProtocol.scala b/akka-actor/src/main/scala/akka/io/dns/DnsProtocol.scala
index 85ed71b567..39d2be13cb 100644
--- a/akka-actor/src/main/scala/akka/io/dns/DnsProtocol.scala
+++ b/akka-actor/src/main/scala/akka/io/dns/DnsProtocol.scala
@@ -10,11 +10,11 @@ import java.net.InetAddress
import java.net.UnknownHostException
import java.util
+import scala.collection.{ immutable => im }
+
import akka.actor.NoSerializationVerificationNeeded
import akka.io.IpVersionSelector
import akka.routing.ConsistentHashingRouter.ConsistentHashable
-
-import scala.collection.{ immutable => im }
import akka.util.ccompat.JavaConverters._
/**
@@ -30,7 +30,7 @@ object DnsProtocol {
sealed trait RequestType
final case class Ip(ipv4: Boolean = true, ipv6: Boolean = true) extends RequestType
- final case object Srv extends RequestType
+ case object Srv extends RequestType
/**
* Java API
diff --git a/akka-actor/src/main/scala/akka/io/dns/DnsResourceRecords.scala b/akka-actor/src/main/scala/akka/io/dns/DnsResourceRecords.scala
index f85a03e7a1..d9fa735ce0 100644
--- a/akka-actor/src/main/scala/akka/io/dns/DnsResourceRecords.scala
+++ b/akka-actor/src/main/scala/akka/io/dns/DnsResourceRecords.scala
@@ -6,16 +6,17 @@ package akka.io.dns
import java.net.{ Inet4Address, Inet6Address, InetAddress }
-import akka.actor.NoSerializationVerificationNeeded
-import akka.annotation.InternalApi
-import CachePolicy._
-import akka.annotation.DoNotInherit
-import akka.io.dns.internal.{ DomainName, _ }
-import akka.util.{ unused, ByteIterator, ByteString }
-
import scala.annotation.switch
import scala.concurrent.duration._
+import CachePolicy._
+
+import akka.actor.NoSerializationVerificationNeeded
+import akka.annotation.DoNotInherit
+import akka.annotation.InternalApi
+import akka.io.dns.internal.{ DomainName, _ }
+import akka.util.{ unused, ByteIterator, ByteString }
+
/**
* Not for user extension
*/
diff --git a/akka-actor/src/main/scala/akka/io/dns/DnsSettings.scala b/akka-actor/src/main/scala/akka/io/dns/DnsSettings.scala
index 3fbdb8d16b..71cb80a64b 100644
--- a/akka-actor/src/main/scala/akka/io/dns/DnsSettings.scala
+++ b/akka-actor/src/main/scala/akka/io/dns/DnsSettings.scala
@@ -8,21 +8,22 @@ import java.io.File
import java.net.{ InetSocketAddress, URI }
import java.util
+import scala.collection.immutable
+import scala.concurrent.duration.FiniteDuration
+import scala.util.{ Failure, Success, Try }
+
+import com.typesafe.config.{ Config, ConfigValueType }
+
import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi
+import akka.event.Logging
import akka.io.dns.CachePolicy.{ CachePolicy, Forever, Never, Ttl }
import akka.io.dns.internal.{ ResolvConf, ResolvConfParser }
import akka.util.Helpers
import akka.util.Helpers.Requiring
import akka.util.JavaDurationConverters._
-import akka.util.ccompat.JavaConverters._
import akka.util.ccompat._
-import com.typesafe.config.{ Config, ConfigValueType }
-import scala.collection.immutable
-import scala.concurrent.duration.FiniteDuration
-import scala.util.{ Failure, Success, Try }
-
-import akka.event.Logging
+import akka.util.ccompat.JavaConverters._
/** INTERNAL API */
@InternalApi
@@ -164,7 +165,6 @@ object DnsSettings {
def getNameserversUsingJNDI: Try[List[InetSocketAddress]] = {
import java.util
-
import javax.naming.Context
import javax.naming.directory.InitialDirContext
// Using jndi-dns to obtain the default name servers.
diff --git a/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsManager.scala b/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsManager.scala
index 026ab274da..c76c290a04 100644
--- a/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsManager.scala
+++ b/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsManager.scala
@@ -7,19 +7,21 @@ package akka.io.dns.internal
import java.net.InetSocketAddress
import java.util.concurrent.TimeUnit
-import akka.actor.{ Actor, ActorLogging, ActorRefFactory, Deploy, ExtendedActorSystem, Props, Timers }
-import akka.annotation.InternalApi
-import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
-import akka.io.PeriodicCacheCleanup
-import akka.io.dns.{ AAAARecord, ARecord, DnsProtocol, DnsSettings }
-import akka.io.dns.internal.AsyncDnsManager.CacheCleanup
-import akka.io.{ Dns, DnsExt, DnsProvider }
-import akka.routing.FromConfig
-import akka.util.Timeout
+import scala.concurrent.ExecutionContextExecutor
+import scala.concurrent.duration.Duration
+
import com.github.ghik.silencer.silent
import com.typesafe.config.Config
-import scala.concurrent.duration.Duration
+import akka.actor.{ Actor, ActorLogging, ActorRefFactory, Deploy, ExtendedActorSystem, Props, Timers }
+import akka.annotation.InternalApi
+import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
+import akka.io.{ Dns, DnsExt, DnsProvider }
+import akka.io.PeriodicCacheCleanup
+import akka.io.dns.{ AAAARecord, ARecord, DnsProtocol, DnsSettings }
+import akka.io.dns.internal.AsyncDnsManager.CacheCleanup
+import akka.routing.FromConfig
+import akka.util.Timeout
/**
* INTERNAL API
@@ -62,10 +64,10 @@ private[io] final class AsyncDnsManager(
ext.Settings.Dispatcher,
ext.provider)
- implicit val ec = context.dispatcher
+ implicit val ec: ExecutionContextExecutor = context.dispatcher
val settings = new DnsSettings(system, resolverConfig)
- implicit val timeout = Timeout(settings.ResolveTimeout)
+ implicit val timeout: Timeout = Timeout(settings.ResolveTimeout)
private val resolver = {
val props: Props = FromConfig.props(
diff --git a/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsProvider.scala b/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsProvider.scala
index f2d67c357c..c1f2d28193 100644
--- a/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsProvider.scala
+++ b/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsProvider.scala
@@ -4,9 +4,10 @@
package akka.io.dns.internal
+import com.github.ghik.silencer.silent
+
import akka.annotation.InternalApi
import akka.io._
-import com.github.ghik.silencer.silent
/**
* INTERNAL API
diff --git a/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsResolver.scala b/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsResolver.scala
index af63617b8d..2e4854447e 100644
--- a/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsResolver.scala
+++ b/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsResolver.scala
@@ -6,23 +6,24 @@ package akka.io.dns.internal
import java.net.{ Inet4Address, Inet6Address, InetAddress, InetSocketAddress }
-import akka.actor.{ Actor, ActorLogging, ActorRef, ActorRefFactory }
-import akka.annotation.InternalApi
-import akka.io.SimpleDnsCache
-import akka.io.dns.CachePolicy.{ Never, Ttl }
-import akka.io.dns.DnsProtocol.{ Ip, RequestType, Srv }
-import akka.io.dns.internal.DnsClient._
-import akka.io.dns._
-import akka.pattern.AskTimeoutException
-import akka.pattern.{ ask, pipe }
-import akka.util.{ Helpers, Timeout }
-import akka.util.PrettyDuration._
-
import scala.collection.immutable
+import scala.concurrent.ExecutionContextExecutor
import scala.concurrent.Future
import scala.util.Try
import scala.util.control.NonFatal
+import akka.actor.{ Actor, ActorLogging, ActorRef, ActorRefFactory }
+import akka.annotation.InternalApi
+import akka.io.SimpleDnsCache
+import akka.io.dns._
+import akka.io.dns.CachePolicy.{ Never, Ttl }
+import akka.io.dns.DnsProtocol.{ Ip, RequestType, Srv }
+import akka.io.dns.internal.DnsClient._
+import akka.pattern.{ ask, pipe }
+import akka.pattern.AskTimeoutException
+import akka.util.{ Helpers, Timeout }
+import akka.util.PrettyDuration._
+
/**
* INTERNAL API
*/
@@ -36,10 +37,10 @@ private[io] final class AsyncDnsResolver(
import AsyncDnsResolver._
- implicit val ec = context.dispatcher
+ implicit val ec: ExecutionContextExecutor = context.dispatcher
// For ask to DNS Client
- implicit val timeout = Timeout(settings.ResolveTimeout)
+ implicit val timeout: Timeout = Timeout(settings.ResolveTimeout)
val nameServers = settings.NameServers
diff --git a/akka-actor/src/main/scala/akka/io/dns/internal/DnsClient.scala b/akka-actor/src/main/scala/akka/io/dns/internal/DnsClient.scala
index f82ffa1266..a7a1c0b0b4 100644
--- a/akka-actor/src/main/scala/akka/io/dns/internal/DnsClient.scala
+++ b/akka-actor/src/main/scala/akka/io/dns/internal/DnsClient.scala
@@ -6,17 +6,18 @@ package akka.io.dns.internal
import java.net.{ InetAddress, InetSocketAddress }
-import akka.actor.Status.Failure
-import akka.actor.{ Actor, ActorLogging, ActorRef, NoSerializationVerificationNeeded, Props, Stash }
-import akka.annotation.InternalApi
-import akka.io.dns.{ RecordClass, RecordType, ResourceRecord }
-import akka.io.{ IO, Tcp, Udp }
-import akka.pattern.{ BackoffOpts, BackoffSupervisor }
+import scala.collection.{ immutable => im }
+import scala.concurrent.duration._
+import scala.util.Try
+
import com.github.ghik.silencer.silent
-import scala.collection.{ immutable => im }
-import scala.util.Try
-import scala.concurrent.duration._
+import akka.actor.{ Actor, ActorLogging, ActorRef, NoSerializationVerificationNeeded, Props, Stash }
+import akka.actor.Status.Failure
+import akka.annotation.InternalApi
+import akka.io.{ IO, Tcp, Udp }
+import akka.io.dns.{ RecordClass, RecordType, ResourceRecord }
+import akka.pattern.{ BackoffOpts, BackoffSupervisor }
/**
* INTERNAL API
@@ -39,7 +40,6 @@ import scala.concurrent.duration._
@InternalApi private[akka] class DnsClient(ns: InetSocketAddress) extends Actor with ActorLogging with Stash {
import DnsClient._
-
import context.system
val udp = IO(Udp)
diff --git a/akka-actor/src/main/scala/akka/io/dns/internal/DnsMessage.scala b/akka-actor/src/main/scala/akka/io/dns/internal/DnsMessage.scala
index 0748980a18..970a2b399e 100644
--- a/akka-actor/src/main/scala/akka/io/dns/internal/DnsMessage.scala
+++ b/akka-actor/src/main/scala/akka/io/dns/internal/DnsMessage.scala
@@ -4,15 +4,16 @@
package akka.io.dns.internal
-import akka.annotation.InternalApi
-import akka.io.dns.ResourceRecord
-import akka.util.{ ByteString, ByteStringBuilder }
-import com.github.ghik.silencer.silent
-
import scala.collection.GenTraversableOnce
import scala.collection.immutable.Seq
import scala.util.{ Failure, Success, Try }
+import com.github.ghik.silencer.silent
+
+import akka.annotation.InternalApi
+import akka.io.dns.ResourceRecord
+import akka.util.{ ByteString, ByteStringBuilder }
+
/**
* INTERNAL API
*/
diff --git a/akka-actor/src/main/scala/akka/io/dns/internal/ResolvConfParser.scala b/akka-actor/src/main/scala/akka/io/dns/internal/ResolvConfParser.scala
index 1144886052..424300f5f6 100644
--- a/akka-actor/src/main/scala/akka/io/dns/internal/ResolvConfParser.scala
+++ b/akka-actor/src/main/scala/akka/io/dns/internal/ResolvConfParser.scala
@@ -7,9 +7,10 @@ package akka.io.dns.internal
import java.io.File
import java.nio.file.Files
-import akka.util.ccompat.JavaConverters._
import scala.util.Try
+import akka.util.ccompat.JavaConverters._
+
private[dns] case class ResolvConf(search: List[String], ndots: Int)
private[dns] object ResolvConfParser {
diff --git a/akka-actor/src/main/scala/akka/io/dns/internal/TcpDnsClient.scala b/akka-actor/src/main/scala/akka/io/dns/internal/TcpDnsClient.scala
index 64975c5497..1a3f99773b 100644
--- a/akka-actor/src/main/scala/akka/io/dns/internal/TcpDnsClient.scala
+++ b/akka-actor/src/main/scala/akka/io/dns/internal/TcpDnsClient.scala
@@ -9,8 +9,8 @@ import java.net.InetSocketAddress
import akka.AkkaException
import akka.actor.{ Actor, ActorLogging, ActorRef, Stash }
import akka.annotation.InternalApi
-import akka.io.dns.internal.DnsClient.Answer
import akka.io.Tcp
+import akka.io.dns.internal.DnsClient.Answer
import akka.util.ByteString
/**
diff --git a/akka-actor/src/main/scala/akka/japi/JavaAPI.scala b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala
index 4a69227431..9531a25f29 100644
--- a/akka-actor/src/main/scala/akka/japi/JavaAPI.scala
+++ b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala
@@ -6,15 +6,16 @@ package akka.japi
import java.util.Collections.{ emptyList, singletonList }
-import akka.util.Collections.EmptyImmutableSeq
-import com.github.ghik.silencer.silent
-
import scala.collection.immutable
import scala.language.implicitConversions
import scala.reflect.ClassTag
import scala.runtime.AbstractPartialFunction
import scala.util.control.NoStackTrace
+import com.github.ghik.silencer.silent
+
+import akka.util.Collections.EmptyImmutableSeq
+
/**
* A Function interface. Used to create first-class-functions is Java.
*
diff --git a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala
index f546ab12c0..c59427c22f 100644
--- a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala
+++ b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala
@@ -6,18 +6,20 @@ package akka.pattern
import java.util.concurrent.TimeoutException
-import akka.actor._
-import akka.annotation.InternalApi
-import akka.dispatch.ExecutionContexts
-import akka.dispatch.sysmsg._
-import akka.util.{ Timeout, Unsafe }
-import com.github.ghik.silencer.silent
-
import scala.annotation.tailrec
import scala.concurrent.{ Future, Promise }
import scala.language.implicitConversions
import scala.util.{ Failure, Success }
+import com.github.ghik.silencer.silent
+
+import akka.actor._
+import akka.annotation.{ InternalApi, InternalStableApi }
+import akka.dispatch.ExecutionContexts
+import akka.dispatch.sysmsg._
+import akka.util.{ Timeout, Unsafe }
+import akka.util.unused
+
/**
* This is what is used to complete a Future that is returned from an ask/? call,
* when it times out. A typical reason for `AskTimeoutException` is that the recipient
@@ -338,9 +340,8 @@ final class AskableActorRef(val actorRef: ActorRef) extends AnyVal {
if (timeout.duration.length <= 0)
Future.failed[Any](AskableActorRef.negativeTimeoutException(actorRef, message, sender))
else {
- val a = PromiseActorRef(ref.provider, timeout, targetName = actorRef, message.getClass.getName, sender)
- actorRef.tell(message, a)
- a.result.future
+ PromiseActorRef(ref.provider, timeout, targetName = actorRef, message.getClass.getName, sender)
+ .ask(actorRef, message, timeout)
}
case _ => Future.failed[Any](AskableActorRef.unsupportedRecipientType(actorRef, message, sender))
}
@@ -375,8 +376,7 @@ final class ExplicitlyAskableActorRef(val actorRef: ActorRef) extends AnyVal {
val a = PromiseActorRef(ref.provider, timeout, targetName = actorRef, "unknown", sender)
val message = messageFactory(a)
a.messageClassName = message.getClass.getName
- actorRef.tell(message, a)
- a.result.future
+ a.ask(actorRef, message, timeout)
}
case _ if sender eq null =>
Future.failed[Any](
@@ -422,9 +422,8 @@ final class AskableActorSelection(val actorSel: ActorSelection) extends AnyVal {
if (timeout.duration.length <= 0)
Future.failed[Any](AskableActorRef.negativeTimeoutException(actorSel, message, sender))
else {
- val a = PromiseActorRef(ref.provider, timeout, targetName = actorSel, message.getClass.getName, sender)
- actorSel.tell(message, a)
- a.result.future
+ PromiseActorRef(ref.provider, timeout, targetName = actorSel, message.getClass.getName, sender)
+ .ask(actorSel, message, timeout)
}
case _ => Future.failed[Any](AskableActorRef.unsupportedRecipientType(actorSel, message, sender))
}
@@ -454,8 +453,7 @@ final class ExplicitlyAskableActorSelection(val actorSel: ActorSelection) extend
val a = PromiseActorRef(ref.provider, timeout, targetName = actorSel, "unknown", sender)
val message = messageFactory(a)
a.messageClassName = message.getClass.getName
- actorSel.tell(message, a)
- a.result.future
+ a.ask(actorSel, message, timeout)
}
case _ if sender eq null =>
Future.failed[Any](
@@ -572,7 +570,9 @@ private[akka] final class PromiseActorRef private (
}
override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit = state match {
- case Stopped | _: StoppedWithPath => provider.deadLetters ! message
+ case Stopped | _: StoppedWithPath =>
+ provider.deadLetters ! message
+ onComplete(message, alreadyCompleted = true)
case _ =>
if (message == null) throw InvalidMessageException("Message is null")
val promiseResult = message match {
@@ -580,8 +580,10 @@ private[akka] final class PromiseActorRef private (
case Status.Failure(f) => Failure(f)
case other => Success(other)
}
- if (!result.tryComplete(promiseResult))
+ val alreadyCompleted = !result.tryComplete(promiseResult)
+ if (alreadyCompleted)
provider.deadLetters ! message
+ onComplete(message, alreadyCompleted)
}
override def sendSystemMessage(message: SystemMessage): Unit = message match {
@@ -631,6 +633,24 @@ private[akka] final class PromiseActorRef private (
case Registering => stop() // spin until registration is completed before stopping
}
}
+
+ @InternalStableApi
+ private[akka] def ask(actorSel: ActorSelection, message: Any, @unused timeout: Timeout): Future[Any] = {
+ actorSel.tell(message, this)
+ result.future
+ }
+
+ @InternalStableApi
+ private[akka] def ask(actorRef: ActorRef, message: Any, @unused timeout: Timeout): Future[Any] = {
+ actorRef.tell(message, this)
+ result.future
+ }
+
+ @InternalStableApi
+ private[akka] def onComplete(@unused message: Any, @unused alreadyCompleted: Boolean): Unit = {}
+
+ @InternalStableApi
+ private[akka] def onTimeout(@unused timeout: Timeout): Unit = {}
}
/**
@@ -657,7 +677,7 @@ private[akka] object PromiseActorRef {
val a = new PromiseActorRef(provider, result, messageClassName)
implicit val ec = ExecutionContexts.parasitic
val f = scheduler.scheduleOnce(timeout.duration) {
- result.tryComplete {
+ val timedOut = result.tryComplete {
val wasSentBy = if (sender == ActorRef.noSender) "" else s" was sent by [$sender]"
val messagePart = s"Message of type [${a.messageClassName}]$wasSentBy."
Failure(
@@ -666,6 +686,9 @@ private[akka] object PromiseActorRef {
messagePart +
" A typical reason for `AskTimeoutException` is that the recipient actor didn't send a reply."))
}
+ if (timedOut) {
+ a.onTimeout(timeout)
+ }
}
result.future.onComplete { _ =>
try a.stop()
diff --git a/akka-actor/src/main/scala/akka/pattern/Backoff.scala b/akka-actor/src/main/scala/akka/pattern/Backoff.scala
index 5dadaa5674..45f628959c 100644
--- a/akka-actor/src/main/scala/akka/pattern/Backoff.scala
+++ b/akka-actor/src/main/scala/akka/pattern/Backoff.scala
@@ -4,13 +4,14 @@
package akka.pattern
+import scala.concurrent.duration.{ Duration, FiniteDuration }
+
+import com.github.ghik.silencer.silent
+
import akka.actor.{ OneForOneStrategy, Props, SupervisorStrategy }
import akka.annotation.DoNotInherit
import akka.pattern.internal.{ BackoffOnRestartSupervisor, BackoffOnStopSupervisor }
import akka.util.JavaDurationConverters._
-import com.github.ghik.silencer.silent
-
-import scala.concurrent.duration.{ Duration, FiniteDuration }
/**
* @deprecated This API is superseded by the [[BackoffOpts]] object.
@@ -635,5 +636,5 @@ private final case class BackoffOptionsImpl(
}
private sealed trait BackoffType
-private final case object StopImpliesFailure extends BackoffType
-private final case object RestartImpliesFailure extends BackoffType
+private case object StopImpliesFailure extends BackoffType
+private case object RestartImpliesFailure extends BackoffType
diff --git a/akka-actor/src/main/scala/akka/pattern/BackoffOptions.scala b/akka-actor/src/main/scala/akka/pattern/BackoffOptions.scala
index 9d36ed4f7a..6e893978ac 100644
--- a/akka-actor/src/main/scala/akka/pattern/BackoffOptions.scala
+++ b/akka-actor/src/main/scala/akka/pattern/BackoffOptions.scala
@@ -4,13 +4,13 @@
package akka.pattern
+import scala.concurrent.duration.{ Duration, FiniteDuration }
+
import akka.actor.{ OneForOneStrategy, Props, SupervisorStrategy }
import akka.annotation.DoNotInherit
import akka.pattern.internal.{ BackoffOnRestartSupervisor, BackoffOnStopSupervisor }
import akka.util.JavaDurationConverters._
-import scala.concurrent.duration.{ Duration, FiniteDuration }
-
/**
* Backoff options allow to specify a number of properties for backoff supervisors.
*/
@@ -424,5 +424,5 @@ private final case class BackoffOnFailureOptionsImpl[T](
}
private[akka] sealed trait BackoffReset
-private[akka] final case object ManualReset extends BackoffReset
+private[akka] case object ManualReset extends BackoffReset
private[akka] final case class AutoReset(resetBackoff: FiniteDuration) extends BackoffReset
diff --git a/akka-actor/src/main/scala/akka/pattern/BackoffSupervisor.scala b/akka-actor/src/main/scala/akka/pattern/BackoffSupervisor.scala
index 7ec00f1696..e85fe90dbe 100644
--- a/akka-actor/src/main/scala/akka/pattern/BackoffSupervisor.scala
+++ b/akka-actor/src/main/scala/akka/pattern/BackoffSupervisor.scala
@@ -7,14 +7,14 @@ package akka.pattern
import java.util.Optional
import java.util.concurrent.ThreadLocalRandom
+import scala.concurrent.duration.{ Duration, FiniteDuration }
+import scala.util.Try
+
import akka.actor.{ ActorRef, DeadLetterSuppression, OneForOneStrategy, Props, SupervisorStrategy }
import akka.annotation.InternalApi
import akka.pattern.internal.BackoffOnStopSupervisor
import akka.util.JavaDurationConverters._
-import scala.concurrent.duration.{ Duration, FiniteDuration }
-import scala.util.Try
-
object BackoffSupervisor {
/**
@@ -246,7 +246,7 @@ object BackoffSupervisor {
* Send this message to the `BackoffSupervisor` and it will reply with
* [[BackoffSupervisor.CurrentChild]] containing the `ActorRef` of the current child, if any.
*/
- final case object GetCurrentChild
+ case object GetCurrentChild
/**
* Java API: Send this message to the `BackoffSupervisor` and it will reply with
@@ -270,7 +270,7 @@ object BackoffSupervisor {
* Send this message to the `BackoffSupervisor` and it will reset the back-off.
* This should be used in conjunction with `withManualReset` in [[BackoffOptions]].
*/
- final case object Reset
+ case object Reset
/**
* Java API: Send this message to the `BackoffSupervisor` and it will reset the back-off.
@@ -282,7 +282,7 @@ object BackoffSupervisor {
* Send this message to the `BackoffSupervisor` and it will reply with
* [[BackoffSupervisor.RestartCount]] containing the current restart count.
*/
- final case object GetRestartCount
+ case object GetRestartCount
/**
* Java API: Send this message to the `BackoffSupervisor` and it will reply with
diff --git a/akka-actor/src/main/scala/akka/pattern/CircuitBreaker.scala b/akka-actor/src/main/scala/akka/pattern/CircuitBreaker.scala
index e19e5372a0..ca4b06c693 100644
--- a/akka-actor/src/main/scala/akka/pattern/CircuitBreaker.scala
+++ b/akka-actor/src/main/scala/akka/pattern/CircuitBreaker.scala
@@ -5,28 +5,27 @@
package akka.pattern
import java.util.Optional
+import java.util.concurrent.{ Callable, CompletionStage, CopyOnWriteArrayList }
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger, AtomicLong }
+import java.util.function.BiFunction
import java.util.function.Consumer
+import scala.compat.java8.FutureConverters
+import scala.concurrent.{ Await, ExecutionContext, Future, Promise }
+import scala.concurrent.TimeoutException
+import scala.concurrent.duration._
+import scala.util.{ Failure, Success, Try }
+import scala.util.control.NoStackTrace
+import scala.util.control.NonFatal
+
+import com.github.ghik.silencer.silent
+
import akka.AkkaException
import akka.actor.Scheduler
+import akka.dispatch.ExecutionContexts.parasitic
import akka.util.JavaDurationConverters._
import akka.util.Unsafe
-import scala.util.control.NoStackTrace
-import java.util.concurrent.{ Callable, CompletionStage, CopyOnWriteArrayList }
-import java.util.function.BiFunction
-
-import scala.concurrent.{ Await, ExecutionContext, Future, Promise }
-import scala.concurrent.duration._
-import scala.concurrent.TimeoutException
-import scala.util.control.NonFatal
-import scala.util.{ Failure, Success, Try }
-import akka.dispatch.ExecutionContexts.parasitic
-import com.github.ghik.silencer.silent
-
-import scala.compat.java8.FutureConverters
-
/**
* Companion object providing factory methods for Circuit Breaker which runs callbacks in caller's thread
*/
@@ -769,12 +768,12 @@ class CircuitBreaker(
materialize(body).onComplete {
case Success(result) =>
p.trySuccess(result)
- timeout.cancel
+ timeout.cancel()
case Failure(ex) =>
if (p.tryFailure(ex)) {
notifyCallFailureListeners(start)
}
- timeout.cancel
+ timeout.cancel()
}(parasitic)
p.future
}
diff --git a/akka-actor/src/main/scala/akka/pattern/FutureTimeoutSupport.scala b/akka-actor/src/main/scala/akka/pattern/FutureTimeoutSupport.scala
index 9f8580c3fa..dd9a3f08bf 100644
--- a/akka-actor/src/main/scala/akka/pattern/FutureTimeoutSupport.scala
+++ b/akka-actor/src/main/scala/akka/pattern/FutureTimeoutSupport.scala
@@ -4,15 +4,17 @@
package akka.pattern
-import scala.concurrent.{ ExecutionContext, Future, Promise }
-import akka.actor._
-import scala.util.control.NonFatal
-import scala.concurrent.duration.FiniteDuration
-import java.util.concurrent.CompletionStage
import java.util.concurrent.CompletableFuture
-import akka.dispatch.Futures
+import java.util.concurrent.CompletionStage
import java.util.function.BiConsumer
+import scala.concurrent.{ ExecutionContext, Future, Promise }
+import scala.concurrent.duration.FiniteDuration
+import scala.util.control.NonFatal
+
+import akka.actor._
+import akka.dispatch.Futures
+
trait FutureTimeoutSupport {
/**
diff --git a/akka-actor/src/main/scala/akka/pattern/GracefulStopSupport.scala b/akka-actor/src/main/scala/akka/pattern/GracefulStopSupport.scala
index 7d5ddf2402..c1e50fce25 100644
--- a/akka-actor/src/main/scala/akka/pattern/GracefulStopSupport.scala
+++ b/akka-actor/src/main/scala/akka/pattern/GracefulStopSupport.scala
@@ -4,14 +4,14 @@
package akka.pattern
-import akka.actor._
-import akka.dispatch.ExecutionContexts
-import akka.util.Timeout
-import akka.dispatch.sysmsg.{ Unwatch, Watch }
-
import scala.concurrent.Future
import scala.concurrent.duration.FiniteDuration
+import akka.actor._
+import akka.dispatch.ExecutionContexts
+import akka.dispatch.sysmsg.{ Unwatch, Watch }
+import akka.util.Timeout
+
trait GracefulStopSupport {
/**
diff --git a/akka-actor/src/main/scala/akka/pattern/Patterns.scala b/akka-actor/src/main/scala/akka/pattern/Patterns.scala
index d0c9ddf4c2..8197a42ca1 100644
--- a/akka-actor/src/main/scala/akka/pattern/Patterns.scala
+++ b/akka-actor/src/main/scala/akka/pattern/Patterns.scala
@@ -7,16 +7,19 @@ package akka.pattern
import java.util.Optional
import java.util.concurrent.{ Callable, CompletionStage, TimeUnit }
-import akka.actor.{ ActorSelection, Scheduler }
-import akka.util.JavaDurationConverters._
-
import scala.compat.java8.FutureConverters._
import scala.concurrent.ExecutionContext
+import akka.actor.{ ActorSelection, Scheduler }
+import akka.util.JavaDurationConverters._
+
/**
* Java API: for Akka patterns such as `ask`, `pipe` and others which work with [[java.util.concurrent.CompletionStage]].
*/
object Patterns {
+ import scala.concurrent.Future
+ import scala.concurrent.duration._
+
import akka.actor.ActorRef
import akka.japi
import akka.pattern.{
@@ -28,9 +31,6 @@ object Patterns {
}
import akka.util.Timeout
- import scala.concurrent.Future
- import scala.concurrent.duration._
-
/**
* Java API for `akka.pattern.ask`:
* Sends a message asynchronously and returns a [[scala.concurrent.Future]]
@@ -574,13 +574,13 @@ object Patterns {
*/
@deprecated("Use Patterns instead.", since = "2.5.19")
object PatternsCS {
+ import scala.concurrent.duration._
+
import akka.actor.ActorRef
import akka.japi
import akka.pattern.{ ask => scalaAsk, gracefulStop => scalaGracefulStop, retry => scalaRetry }
import akka.util.Timeout
- import scala.concurrent.duration._
-
/**
* Java API for `akka.pattern.ask`:
* Sends a message asynchronously and returns a [[java.util.concurrent.CompletionStage]]
diff --git a/akka-actor/src/main/scala/akka/pattern/PipeToSupport.scala b/akka-actor/src/main/scala/akka/pattern/PipeToSupport.scala
index 9d11920eff..b652d7afc0 100644
--- a/akka-actor/src/main/scala/akka/pattern/PipeToSupport.scala
+++ b/akka-actor/src/main/scala/akka/pattern/PipeToSupport.scala
@@ -4,14 +4,16 @@
package akka.pattern
-import language.implicitConversions
-import scala.concurrent.{ ExecutionContext, Future }
-import scala.util.{ Failure, Success }
-import akka.actor.{ Actor, ActorRef, Status }
-import akka.actor.ActorSelection
import java.util.concurrent.CompletionStage
import java.util.function.BiConsumer
+import scala.concurrent.{ ExecutionContext, Future }
+import scala.util.{ Failure, Success }
+
+import language.implicitConversions
+
+import akka.actor.{ Actor, ActorRef, Status }
+import akka.actor.ActorSelection
import akka.util.unused
trait PipeToSupport {
diff --git a/akka-actor/src/main/scala/akka/pattern/PromiseRef.scala b/akka-actor/src/main/scala/akka/pattern/PromiseRef.scala
index 98bc24a32e..65d6b75465 100644
--- a/akka-actor/src/main/scala/akka/pattern/PromiseRef.scala
+++ b/akka-actor/src/main/scala/akka/pattern/PromiseRef.scala
@@ -4,11 +4,11 @@
package akka.pattern
+import scala.concurrent.{ Future, Promise }
+
import akka.actor._
import akka.util.Timeout
-import scala.concurrent.{ Future, Promise }
-
/**
* A combination of a Future and an ActorRef associated with it, which points
* to an actor performing a task which will eventually resolve the Future.
diff --git a/akka-actor/src/main/scala/akka/pattern/RetrySupport.scala b/akka-actor/src/main/scala/akka/pattern/RetrySupport.scala
index dbdb051bf9..0380f2ba0b 100644
--- a/akka-actor/src/main/scala/akka/pattern/RetrySupport.scala
+++ b/akka-actor/src/main/scala/akka/pattern/RetrySupport.scala
@@ -4,13 +4,13 @@
package akka.pattern
+import scala.concurrent.{ ExecutionContext, Future }
+import scala.concurrent.duration.{ Duration, FiniteDuration }
+import scala.util.control.NonFatal
+
import akka.actor.Scheduler
import akka.util.ConstantFun
-import scala.concurrent.duration.{ Duration, FiniteDuration }
-import scala.concurrent.{ ExecutionContext, Future }
-import scala.util.control.NonFatal
-
/**
* This trait provides the retry utility function
*/
diff --git a/akka-actor/src/main/scala/akka/pattern/internal/BackoffOnRestartSupervisor.scala b/akka-actor/src/main/scala/akka/pattern/internal/BackoffOnRestartSupervisor.scala
index 22fa86f817..3f799dc361 100644
--- a/akka-actor/src/main/scala/akka/pattern/internal/BackoffOnRestartSupervisor.scala
+++ b/akka-actor/src/main/scala/akka/pattern/internal/BackoffOnRestartSupervisor.scala
@@ -4,13 +4,13 @@
package akka.pattern.internal
-import akka.actor.SupervisorStrategy._
+import scala.concurrent.duration._
+
import akka.actor.{ OneForOneStrategy, _ }
+import akka.actor.SupervisorStrategy._
import akka.annotation.InternalApi
import akka.pattern.{ BackoffReset, BackoffSupervisor, HandleBackoff }
-import scala.concurrent.duration._
-
/**
* INTERNAL API
*
diff --git a/akka-actor/src/main/scala/akka/pattern/internal/BackoffOnStopSupervisor.scala b/akka-actor/src/main/scala/akka/pattern/internal/BackoffOnStopSupervisor.scala
index a739098802..af94d4fa57 100644
--- a/akka-actor/src/main/scala/akka/pattern/internal/BackoffOnStopSupervisor.scala
+++ b/akka-actor/src/main/scala/akka/pattern/internal/BackoffOnStopSupervisor.scala
@@ -4,13 +4,12 @@
package akka.pattern.internal
-import akka.actor.SupervisorStrategy.{ Directive, Escalate }
+import scala.concurrent.duration.FiniteDuration
+
import akka.actor.{ Actor, ActorLogging, OneForOneStrategy, Props, SupervisorStrategy, Terminated }
+import akka.actor.SupervisorStrategy.{ Directive, Escalate }
import akka.annotation.InternalApi
import akka.pattern.{ BackoffReset, BackoffSupervisor, HandleBackoff }
-import akka.util.PrettyDuration
-
-import scala.concurrent.duration.FiniteDuration
/**
* INTERNAL API
@@ -52,7 +51,6 @@ import scala.concurrent.duration.FiniteDuration
case Terminated(ref) if child.contains(ref) =>
child = None
if (finalStopMessageReceived) {
- log.debug("Child terminated after final stop message, stopping supervisor")
context.stop(self)
} else {
val maxNrOfRetries = strategy match {
@@ -63,14 +61,13 @@ import scala.concurrent.duration.FiniteDuration
if (maxNrOfRetries == -1 || nextRestartCount <= maxNrOfRetries) {
val restartDelay = calculateDelay(restartCount, minBackoff, maxBackoff, randomFactor)
- log.debug("Supervised child terminated, restarting after [{}] back off", PrettyDuration.format(restartDelay))
context.system.scheduler.scheduleOnce(restartDelay, self, StartChild)
restartCount = nextRestartCount
} else {
- log.warning(
- "Supervised child exceeded max allowed number of restarts [{}] (restarded [{}] times), stopping supervisor",
- maxNrOfRetries,
- nextRestartCount)
+ log.debug(
+ s"Terminating on restart #{} which exceeds max allowed restarts ({})",
+ nextRestartCount,
+ maxNrOfRetries)
context.stop(self)
}
}
@@ -89,13 +86,11 @@ import scala.concurrent.duration.FiniteDuration
case None =>
replyWhileStopped match {
case Some(r) => sender() ! r
- case _ =>
+ case None => context.system.deadLetters.forward(msg)
}
finalStopMessage match {
case Some(fsm) if fsm(msg) => context.stop(self)
case _ =>
- // only send to dead letters if not replied nor final-stopped
- if (replyWhileStopped.isEmpty) context.system.deadLetters.forward(msg)
}
}
}
diff --git a/akka-actor/src/main/scala/akka/routing/Balancing.scala b/akka-actor/src/main/scala/akka/routing/Balancing.scala
index 16dfab74e6..0d5a8e0b9a 100644
--- a/akka-actor/src/main/scala/akka/routing/Balancing.scala
+++ b/akka-actor/src/main/scala/akka/routing/Balancing.scala
@@ -5,15 +5,17 @@
package akka.routing
import scala.collection.immutable
+
+import com.github.ghik.silencer.silent
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
+
import akka.actor.ActorContext
import akka.actor.ActorSystem
import akka.actor.Props
import akka.actor.SupervisorStrategy
import akka.dispatch.BalancingDispatcherConfigurator
import akka.dispatch.Dispatchers
-import com.github.ghik.silencer.silent
/**
* INTERNAL API
diff --git a/akka-actor/src/main/scala/akka/routing/Broadcast.scala b/akka-actor/src/main/scala/akka/routing/Broadcast.scala
index 89ac7da1a8..abe9a51880 100644
--- a/akka-actor/src/main/scala/akka/routing/Broadcast.scala
+++ b/akka-actor/src/main/scala/akka/routing/Broadcast.scala
@@ -5,12 +5,14 @@
package akka.routing
import scala.collection.immutable
-import akka.dispatch.Dispatchers
-import com.typesafe.config.Config
-import akka.actor.SupervisorStrategy
-import akka.japi.Util.immutableSeq
-import akka.actor.ActorSystem
+
import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+
+import akka.actor.ActorSystem
+import akka.actor.SupervisorStrategy
+import akka.dispatch.Dispatchers
+import akka.japi.Util.immutableSeq
object BroadcastRoutingLogic {
def apply(): BroadcastRoutingLogic = new BroadcastRoutingLogic
diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala
index 7324e9851d..a506b0f4dd 100644
--- a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala
+++ b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala
@@ -4,9 +4,10 @@
package akka.routing
+import java.util.Arrays
+
import scala.collection.immutable
import scala.reflect.ClassTag
-import java.util.Arrays
/**
* Consistent Hashing node ring implementation.
diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHashing.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHashing.scala
index 3a62f470d4..148bd43901 100644
--- a/akka-actor/src/main/scala/akka/routing/ConsistentHashing.scala
+++ b/akka-actor/src/main/scala/akka/routing/ConsistentHashing.scala
@@ -4,23 +4,23 @@
package akka.routing
-import scala.collection.immutable
-
-import akka.dispatch.Dispatchers
-import com.typesafe.config.Config
-import akka.actor.SupervisorStrategy
-import akka.japi.Util.immutableSeq
-import akka.actor.Address
-import akka.actor.ExtendedActorSystem
-import akka.actor.ActorSystem
import java.util.concurrent.atomic.AtomicReference
-import akka.serialization.SerializationExtension
+import scala.collection.immutable
import scala.util.control.NonFatal
-import akka.event.Logging
+import com.typesafe.config.Config
+
import akka.actor.ActorPath
+import akka.actor.ActorSystem
+import akka.actor.Address
+import akka.actor.ExtendedActorSystem
+import akka.actor.SupervisorStrategy
import akka.actor.WrappedMessage
+import akka.dispatch.Dispatchers
+import akka.event.Logging
+import akka.japi.Util.immutableSeq
+import akka.serialization.SerializationExtension
object ConsistentHashingRouter {
diff --git a/akka-actor/src/main/scala/akka/routing/Listeners.scala b/akka-actor/src/main/scala/akka/routing/Listeners.scala
index a66cfd6be6..3a1c58da27 100644
--- a/akka-actor/src/main/scala/akka/routing/Listeners.scala
+++ b/akka-actor/src/main/scala/akka/routing/Listeners.scala
@@ -4,9 +4,10 @@
package akka.routing
-import akka.actor.{ Actor, ActorRef }
import java.util.{ Set, TreeSet }
+import akka.actor.{ Actor, ActorRef }
+
sealed trait ListenerMessage
final case class Listen(listener: ActorRef) extends ListenerMessage
final case class Deafen(listener: ActorRef) extends ListenerMessage
diff --git a/akka-actor/src/main/scala/akka/routing/MurmurHash.scala b/akka-actor/src/main/scala/akka/routing/MurmurHash.scala
index a40e9cc88e..5b150c9668 100644
--- a/akka-actor/src/main/scala/akka/routing/MurmurHash.scala
+++ b/akka-actor/src/main/scala/akka/routing/MurmurHash.scala
@@ -21,9 +21,10 @@ package akka.routing
import java.lang.Integer.{ rotateLeft => rotl }
-import akka.util.ccompat._
import com.github.ghik.silencer.silent
+import akka.util.ccompat._
+
/**
* An object designed to generate well-distributed non-cryptographic
* hashes. It is designed to hash a collection of integers; along with
diff --git a/akka-actor/src/main/scala/akka/routing/OptimalSizeExploringResizer.scala b/akka-actor/src/main/scala/akka/routing/OptimalSizeExploringResizer.scala
index dfcc70b8e6..e08a502716 100644
--- a/akka-actor/src/main/scala/akka/routing/OptimalSizeExploringResizer.scala
+++ b/akka-actor/src/main/scala/akka/routing/OptimalSizeExploringResizer.scala
@@ -5,16 +5,17 @@
package akka.routing
import java.time.LocalDateTime
-
-import scala.collection.immutable
import java.util.concurrent.ThreadLocalRandom
+import scala.collection.immutable
import scala.concurrent.duration._
-import com.typesafe.config.Config
-import akka.actor._
-import akka.util.JavaDurationConverters._
+
import OptimalSizeExploringResizer._
+import com.typesafe.config.Config
+
+import akka.actor._
import akka.annotation.InternalApi
+import akka.util.JavaDurationConverters._
trait OptimalSizeExploringResizer extends Resizer {
diff --git a/akka-actor/src/main/scala/akka/routing/Random.scala b/akka-actor/src/main/scala/akka/routing/Random.scala
index ce72cc9ed5..53dbbc5ddb 100644
--- a/akka-actor/src/main/scala/akka/routing/Random.scala
+++ b/akka-actor/src/main/scala/akka/routing/Random.scala
@@ -4,15 +4,17 @@
package akka.routing
-import scala.collection.immutable
import java.util.concurrent.ThreadLocalRandom
-import akka.dispatch.Dispatchers
-import com.typesafe.config.Config
-import akka.actor.SupervisorStrategy
-import akka.japi.Util.immutableSeq
-import akka.actor.ActorSystem
+import scala.collection.immutable
+
import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+
+import akka.actor.ActorSystem
+import akka.actor.SupervisorStrategy
+import akka.dispatch.Dispatchers
+import akka.japi.Util.immutableSeq
object RandomRoutingLogic {
def apply(): RandomRoutingLogic = new RandomRoutingLogic
diff --git a/akka-actor/src/main/scala/akka/routing/Resizer.scala b/akka-actor/src/main/scala/akka/routing/Resizer.scala
index 26f94ee7f1..48612194d8 100644
--- a/akka-actor/src/main/scala/akka/routing/Resizer.scala
+++ b/akka-actor/src/main/scala/akka/routing/Resizer.scala
@@ -7,12 +7,11 @@ package akka.routing
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.atomic.AtomicLong
-import akka.AkkaException
-
import scala.collection.immutable
import com.typesafe.config.Config
+import akka.AkkaException
import akka.actor.Actor
import akka.actor.ActorCell
import akka.actor.ActorInitializationException
diff --git a/akka-actor/src/main/scala/akka/routing/RoundRobin.scala b/akka-actor/src/main/scala/akka/routing/RoundRobin.scala
index 7b4f64f4b8..c9e7fe379c 100644
--- a/akka-actor/src/main/scala/akka/routing/RoundRobin.scala
+++ b/akka-actor/src/main/scala/akka/routing/RoundRobin.scala
@@ -7,12 +7,14 @@ package akka.routing
import java.util.concurrent.atomic.AtomicLong
import scala.collection.immutable
-import akka.dispatch.Dispatchers
-import com.typesafe.config.Config
-import akka.actor.SupervisorStrategy
-import akka.japi.Util.immutableSeq
-import akka.actor.ActorSystem
+
import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+
+import akka.actor.ActorSystem
+import akka.actor.SupervisorStrategy
+import akka.dispatch.Dispatchers
+import akka.japi.Util.immutableSeq
object RoundRobinRoutingLogic {
def apply(): RoundRobinRoutingLogic = new RoundRobinRoutingLogic
diff --git a/akka-actor/src/main/scala/akka/routing/RoutedActorCell.scala b/akka-actor/src/main/scala/akka/routing/RoutedActorCell.scala
index 539050676c..b02f3bf8d7 100644
--- a/akka-actor/src/main/scala/akka/routing/RoutedActorCell.scala
+++ b/akka-actor/src/main/scala/akka/routing/RoutedActorCell.scala
@@ -4,6 +4,9 @@
package akka.routing
+import scala.collection.immutable
+import scala.concurrent.duration._
+
import akka.actor.Actor
import akka.actor.ActorCell
import akka.actor.ActorInitializationException
@@ -17,9 +20,6 @@ import akka.actor.SupervisorStrategy
import akka.actor.Terminated
import akka.dispatch.Envelope
import akka.dispatch.MessageDispatcher
-
-import scala.collection.immutable
-import scala.concurrent.duration._
import akka.util.ccompat._
/**
diff --git a/akka-actor/src/main/scala/akka/routing/RoutedActorRef.scala b/akka-actor/src/main/scala/akka/routing/RoutedActorRef.scala
index ac93668603..5936a4d94d 100644
--- a/akka-actor/src/main/scala/akka/routing/RoutedActorRef.scala
+++ b/akka-actor/src/main/scala/akka/routing/RoutedActorRef.scala
@@ -4,6 +4,8 @@
package akka.routing
+import com.github.ghik.silencer.silent
+
import akka.ConfigurationException
import akka.actor.ActorPath
import akka.actor.ActorSystemImpl
@@ -15,7 +17,6 @@ import akka.actor.UnstartedCell
import akka.dispatch.BalancingDispatcher
import akka.dispatch.MailboxType
import akka.dispatch.MessageDispatcher
-import com.github.ghik.silencer.silent
/**
* INTERNAL API
diff --git a/akka-actor/src/main/scala/akka/routing/Router.scala b/akka-actor/src/main/scala/akka/routing/Router.scala
index d2b62dc6ec..1702cd5fcd 100644
--- a/akka-actor/src/main/scala/akka/routing/Router.scala
+++ b/akka-actor/src/main/scala/akka/routing/Router.scala
@@ -5,11 +5,12 @@
package akka.routing
import scala.collection.immutable
+
import akka.actor.ActorRef
import akka.actor.ActorSelection
import akka.actor.InternalActorRef
-import akka.japi.Util.immutableSeq
import akka.actor.NoSerializationVerificationNeeded
+import akka.japi.Util.immutableSeq
/**
* The interface of the routing logic that is used in a [[Router]] to select
diff --git a/akka-actor/src/main/scala/akka/routing/RouterConfig.scala b/akka-actor/src/main/scala/akka/routing/RouterConfig.scala
index 980c661b6d..33f5c84627 100644
--- a/akka-actor/src/main/scala/akka/routing/RouterConfig.scala
+++ b/akka-actor/src/main/scala/akka/routing/RouterConfig.scala
@@ -5,19 +5,21 @@
package akka.routing
import scala.collection.immutable
+
+import com.github.ghik.silencer.silent
+
import akka.ConfigurationException
import akka.actor.ActorContext
import akka.actor.ActorPath
+import akka.actor.ActorSystem
import akka.actor.AutoReceivedMessage
import akka.actor.OneForOneStrategy
import akka.actor.Props
import akka.actor.SupervisorStrategy
import akka.actor.Terminated
import akka.dispatch.Dispatchers
-import akka.actor.ActorSystem
import akka.japi.Util.immutableSeq
import akka.util.unused
-import com.github.ghik.silencer.silent
/**
* This trait represents a router factory: it produces the actual router actor
diff --git a/akka-actor/src/main/scala/akka/routing/ScatterGatherFirstCompleted.scala b/akka-actor/src/main/scala/akka/routing/ScatterGatherFirstCompleted.scala
index ec27af01bd..cd369c5708 100644
--- a/akka-actor/src/main/scala/akka/routing/ScatterGatherFirstCompleted.scala
+++ b/akka-actor/src/main/scala/akka/routing/ScatterGatherFirstCompleted.scala
@@ -4,23 +4,26 @@
package akka.routing
+import java.util.concurrent.TimeoutException
+
import scala.collection.immutable
-import akka.dispatch.Dispatchers
-import com.typesafe.config.Config
-import akka.actor.SupervisorStrategy
-import akka.japi.Util.immutableSeq
-import akka.actor.ActorRef
+import scala.concurrent.Future
import scala.concurrent.Promise
+import scala.concurrent.duration.FiniteDuration
+
+import com.typesafe.config.Config
+
+import akka.actor.ActorRef
+import akka.actor.ActorSystem
+import akka.actor.SupervisorStrategy
+import akka.dispatch.Dispatchers
+import akka.dispatch.ExecutionContexts
+import akka.japi.Util.immutableSeq
import akka.pattern.ask
import akka.pattern.pipe
-import akka.dispatch.ExecutionContexts
-import scala.concurrent.duration.FiniteDuration
-import akka.util.Timeout
import akka.util.Helpers.ConfigOps
import akka.util.JavaDurationConverters._
-import akka.actor.ActorSystem
-import scala.concurrent.Future
-import java.util.concurrent.TimeoutException
+import akka.util.Timeout
/**
* Broadcasts the message to all routees, and replies with the first response.
diff --git a/akka-actor/src/main/scala/akka/routing/SmallestMailbox.scala b/akka-actor/src/main/scala/akka/routing/SmallestMailbox.scala
index b28d3e9829..3bb4308e30 100644
--- a/akka-actor/src/main/scala/akka/routing/SmallestMailbox.scala
+++ b/akka-actor/src/main/scala/akka/routing/SmallestMailbox.scala
@@ -4,17 +4,19 @@
package akka.routing
-import scala.annotation.tailrec
-import scala.collection.immutable
import java.util.concurrent.ThreadLocalRandom
+import scala.annotation.tailrec
+import scala.collection.immutable
+
+import com.github.ghik.silencer.silent
import com.typesafe.config.Config
+
import akka.actor.ActorCell
import akka.actor.ActorRefWithCell
+import akka.actor.ActorSystem
import akka.actor.SupervisorStrategy
import akka.dispatch.Dispatchers
-import akka.actor.ActorSystem
-import com.github.ghik.silencer.silent
object SmallestMailboxRoutingLogic {
def apply(): SmallestMailboxRoutingLogic = new SmallestMailboxRoutingLogic
diff --git a/akka-actor/src/main/scala/akka/routing/TailChopping.scala b/akka-actor/src/main/scala/akka/routing/TailChopping.scala
index bb692f383b..027dbe127f 100644
--- a/akka-actor/src/main/scala/akka/routing/TailChopping.scala
+++ b/akka-actor/src/main/scala/akka/routing/TailChopping.scala
@@ -7,18 +7,19 @@ package akka.routing
import java.util.concurrent.atomic.AtomicInteger
import scala.collection.immutable
+import scala.concurrent.{ ExecutionContext, Promise }
+import scala.concurrent.duration._
+import scala.util.Random
+
+import com.typesafe.config.Config
+
import akka.actor._
import akka.dispatch.Dispatchers
-import com.typesafe.config.Config
import akka.japi.Util.immutableSeq
-import scala.concurrent.{ ExecutionContext, Promise }
import akka.pattern.{ ask, pipe, AskTimeoutException }
-import scala.concurrent.duration._
+import akka.util.Helpers.ConfigOps
import akka.util.JavaDurationConverters._
import akka.util.Timeout
-import akka.util.Helpers.ConfigOps
-
-import scala.util.Random
/**
* As each message is sent to the router, the routees are randomly ordered. The message is sent to the
diff --git a/akka-actor/src/main/scala/akka/serialization/AsyncSerializer.scala b/akka-actor/src/main/scala/akka/serialization/AsyncSerializer.scala
index 8f5efbd567..fc691b6c52 100644
--- a/akka-actor/src/main/scala/akka/serialization/AsyncSerializer.scala
+++ b/akka-actor/src/main/scala/akka/serialization/AsyncSerializer.scala
@@ -6,10 +6,10 @@ package akka.serialization
import java.util.concurrent.CompletionStage
-import akka.actor.ExtendedActorSystem
-import scala.concurrent.duration.Duration
import scala.concurrent.{ Await, Future }
+import scala.concurrent.duration.Duration
+import akka.actor.ExtendedActorSystem
import akka.event.Logging
/**
diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala
index 70ccceb9a6..0fc71e1fac 100644
--- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala
+++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala
@@ -4,24 +4,24 @@
package akka.serialization
-import com.typesafe.config.Config
-import akka.actor._
-import akka.event.{ LogMarker, Logging, LoggingAdapter }
-import java.util.concurrent.ConcurrentHashMap
-
-import scala.collection.mutable.ArrayBuffer
import java.io.NotSerializableException
-
-import scala.util.{ DynamicVariable, Failure, Try }
-import scala.collection.immutable
-import scala.util.control.NonFatal
-import scala.util.Success
import java.nio.ByteBuffer
+import java.util.NoSuchElementException
+import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.atomic.AtomicReference
import scala.annotation.tailrec
-import java.util.NoSuchElementException
+import scala.collection.immutable
+import scala.collection.mutable.ArrayBuffer
+import scala.util.{ DynamicVariable, Failure, Try }
+import scala.util.Success
+import scala.util.control.NonFatal
+
+import com.typesafe.config.Config
+
+import akka.actor._
import akka.annotation.InternalApi
+import akka.event.{ LogMarker, Logging, LoggingAdapter }
import akka.util.ccompat._
@ccompatUsedUntil213
@@ -503,8 +503,21 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
/**
* Maps from a Serializer Identity (Int) to a Serializer instance (optimization)
*/
- val serializerByIdentity: Map[Int, Serializer] =
- Map(NullSerializer.identifier -> NullSerializer) ++ serializers.map { case (_, v) => (v.identifier, v) }
+ val serializerByIdentity: Map[Int, Serializer] = {
+ val zero: Map[Int, Serializer] = Map(NullSerializer.identifier -> NullSerializer)
+ serializers.foldLeft(zero) {
+ case (acc, (_, ser)) =>
+ val id = ser.identifier
+ acc.get(id) match {
+ case Some(existing) if existing != ser =>
+ throw new IllegalArgumentException(
+ s"Serializer identifier [$id] of [${ser.getClass.getName}] " +
+ s"is not unique. It is also used by [${acc(id).getClass.getName}].")
+ case _ =>
+ acc.updated(id, ser)
+ }
+ }
+ }
/**
* Serializers with id 0 - 1023 are stored in an array for quick allocation free access
diff --git a/akka-actor/src/main/scala/akka/serialization/SerializationExtension.scala b/akka-actor/src/main/scala/akka/serialization/SerializationExtension.scala
index 7a3164bfbc..41e4ea18f0 100644
--- a/akka-actor/src/main/scala/akka/serialization/SerializationExtension.scala
+++ b/akka-actor/src/main/scala/akka/serialization/SerializationExtension.scala
@@ -4,8 +4,8 @@
package akka.serialization
-import akka.actor.ClassicActorSystemProvider
import akka.actor.{ ActorSystem, ExtendedActorSystem, ExtensionId, ExtensionIdProvider }
+import akka.actor.ClassicActorSystemProvider
/**
* SerializationExtension is an Akka Extension to interact with the Serialization
diff --git a/akka-actor/src/main/scala/akka/serialization/SerializationSetup.scala b/akka-actor/src/main/scala/akka/serialization/SerializationSetup.scala
index 1f2f6d5ad4..ca2a25a38f 100644
--- a/akka-actor/src/main/scala/akka/serialization/SerializationSetup.scala
+++ b/akka-actor/src/main/scala/akka/serialization/SerializationSetup.scala
@@ -4,10 +4,10 @@
package akka.serialization
+import scala.collection.immutable
+
import akka.actor.ExtendedActorSystem
import akka.actor.setup.Setup
-
-import scala.collection.immutable
import akka.util.ccompat.JavaConverters._
object SerializationSetup {
diff --git a/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala b/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala
index d12fea7c4c..cde8b51251 100644
--- a/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala
+++ b/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala
@@ -4,9 +4,9 @@
package akka.util
-import java.util.concurrent.locks.{ Condition, ReentrantLock }
-import java.util.concurrent.{ BlockingQueue, TimeUnit }
import java.util.{ AbstractQueue, Collection, Iterator, Queue }
+import java.util.concurrent.{ BlockingQueue, TimeUnit }
+import java.util.concurrent.locks.{ Condition, ReentrantLock }
import annotation.tailrec
diff --git a/akka-actor/src/main/scala/akka/util/Collections.scala b/akka-actor/src/main/scala/akka/util/Collections.scala
index d07711f30b..a685302512 100644
--- a/akka-actor/src/main/scala/akka/util/Collections.scala
+++ b/akka-actor/src/main/scala/akka/util/Collections.scala
@@ -4,8 +4,8 @@
package akka.util
-import scala.collection.immutable
import scala.annotation.tailrec
+import scala.collection.immutable
/**
* INTERNAL API
diff --git a/akka-actor/src/main/scala/akka/util/ConstantFun.scala b/akka-actor/src/main/scala/akka/util/ConstantFun.scala
index a23e3f308f..62d6d16aac 100644
--- a/akka-actor/src/main/scala/akka/util/ConstantFun.scala
+++ b/akka-actor/src/main/scala/akka/util/ConstantFun.scala
@@ -5,8 +5,8 @@
package akka.util
import akka.annotation.InternalApi
-import akka.japi.function.{ Function => JFun, Function2 => JFun2 }
import akka.japi.{ Pair => JPair }
+import akka.japi.function.{ Function => JFun, Function2 => JFun2 }
/**
* INTERNAL API
diff --git a/akka-actor/src/main/scala/akka/util/HashCode.scala b/akka-actor/src/main/scala/akka/util/HashCode.scala
index a4009f22a7..f601445da8 100644
--- a/akka-actor/src/main/scala/akka/util/HashCode.scala
+++ b/akka-actor/src/main/scala/akka/util/HashCode.scala
@@ -4,8 +4,8 @@
package akka.util
-import java.lang.reflect.{ Array => JArray }
import java.lang.{ Float => JFloat, Double => JDouble }
+import java.lang.reflect.{ Array => JArray }
/**
* Set of methods which allow easy implementation of hashCode.
diff --git a/akka-actor/src/main/scala/akka/util/Helpers.scala b/akka-actor/src/main/scala/akka/util/Helpers.scala
index 21925d9769..d86cb4a092 100644
--- a/akka-actor/src/main/scala/akka/util/Helpers.scala
+++ b/akka-actor/src/main/scala/akka/util/Helpers.scala
@@ -4,16 +4,18 @@
package akka.util
-import java.util.Comparator
-import scala.annotation.tailrec
-import java.util.regex.Pattern
-import com.typesafe.config.Config
-import scala.concurrent.duration.FiniteDuration
-import scala.concurrent.duration.Duration
-import java.util.concurrent.TimeUnit
-import java.util.Locale
import java.time.{ Instant, LocalDateTime, ZoneId }
import java.time.format.DateTimeFormatter
+import java.util.Comparator
+import java.util.Locale
+import java.util.concurrent.TimeUnit
+import java.util.regex.Pattern
+
+import scala.annotation.tailrec
+import scala.concurrent.duration.Duration
+import scala.concurrent.duration.FiniteDuration
+
+import com.typesafe.config.Config
object Helpers {
diff --git a/akka-actor/src/main/scala/akka/util/ImmutableIntMap.scala b/akka-actor/src/main/scala/akka/util/ImmutableIntMap.scala
index 48c2abea9e..8a733e558e 100644
--- a/akka-actor/src/main/scala/akka/util/ImmutableIntMap.scala
+++ b/akka-actor/src/main/scala/akka/util/ImmutableIntMap.scala
@@ -4,9 +4,11 @@
package akka.util
import java.util.Arrays
-import akka.annotation.InternalApi
+
import scala.annotation.tailrec
+import akka.annotation.InternalApi
+
/**
* INTERNAL API
*/
diff --git a/akka-actor/src/main/scala/akka/util/Index.scala b/akka-actor/src/main/scala/akka/util/Index.scala
index b7e13c67fd..4911aedf5b 100644
--- a/akka-actor/src/main/scala/akka/util/Index.scala
+++ b/akka-actor/src/main/scala/akka/util/Index.scala
@@ -4,16 +4,16 @@
package akka.util
-import annotation.tailrec
-
-import java.util.concurrent.{ ConcurrentHashMap, ConcurrentSkipListSet }
import java.util.Comparator
+import java.util.concurrent.{ ConcurrentHashMap, ConcurrentSkipListSet }
import scala.collection.JavaConverters.collectionAsScalaIterableConverter
-import akka.util.ccompat.JavaConverters._
+import annotation.tailrec
import com.github.ghik.silencer.silent
+import akka.util.ccompat.JavaConverters._
+
/**
* An implementation of a ConcurrentMultiMap
* Adds/remove is serialized over the specified key
diff --git a/akka-actor/src/main/scala/akka/util/JavaDurationConverters.scala b/akka-actor/src/main/scala/akka/util/JavaDurationConverters.scala
index c344f72058..cefe1a2e28 100644
--- a/akka-actor/src/main/scala/akka/util/JavaDurationConverters.scala
+++ b/akka-actor/src/main/scala/akka/util/JavaDurationConverters.scala
@@ -7,9 +7,12 @@ import java.time.{ Duration => JDuration }
import scala.concurrent.duration.{ Duration, FiniteDuration }
+import akka.annotation.InternalStableApi
+
/**
* INTERNAL API
*/
+@InternalStableApi
private[akka] object JavaDurationConverters {
def asFiniteDuration(duration: JDuration): FiniteDuration = duration.asScala
diff --git a/akka-actor/src/main/scala/akka/util/LockUtil.scala b/akka-actor/src/main/scala/akka/util/LockUtil.scala
index af1d3cd8e8..5d1fd4fef5 100644
--- a/akka-actor/src/main/scala/akka/util/LockUtil.scala
+++ b/akka-actor/src/main/scala/akka/util/LockUtil.scala
@@ -4,8 +4,8 @@
package akka.util
-import java.util.concurrent.locks.ReentrantLock
import java.util.concurrent.atomic.AtomicBoolean
+import java.util.concurrent.locks.ReentrantLock
final class ReentrantGuard extends ReentrantLock {
diff --git a/akka-actor/src/main/scala/akka/util/ManifestInfo.scala b/akka-actor/src/main/scala/akka/util/ManifestInfo.scala
index 1df48570b2..e3c86c7f5c 100644
--- a/akka-actor/src/main/scala/akka/util/ManifestInfo.scala
+++ b/akka-actor/src/main/scala/akka/util/ManifestInfo.scala
@@ -4,12 +4,15 @@
package akka.util
-import scala.collection.immutable
import java.io.IOException
import java.util.Arrays
import java.util.jar.Attributes
import java.util.jar.Manifest
+import scala.collection.immutable
+
+import com.github.ghik.silencer.silent
+
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
import akka.actor.ExtendedActorSystem
@@ -17,7 +20,6 @@ import akka.actor.Extension
import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
import akka.event.Logging
-import com.github.ghik.silencer.silent
/**
* Akka extension that extracts [[ManifestInfo.Version]] information from META-INF/MANIFEST.MF in jar files
@@ -107,6 +109,26 @@ object ManifestInfo extends ExtensionId[ManifestInfo] with ExtensionIdProvider {
override def toString: String = version
}
+
+ /** INTERNAL API */
+ private[util] def checkSameVersion(
+ productName: String,
+ dependencies: immutable.Seq[String],
+ versions: Map[String, Version]): Option[String] = {
+ @silent("deprecated")
+ val filteredVersions = versions.filterKeys(dependencies.toSet)
+ val values = filteredVersions.values.toSet
+ if (values.size > 1) {
+ val highestVersion = values.max
+ val toBeUpdated = filteredVersions.collect { case (k, v) if v != highestVersion => s"$k" }.mkString(", ")
+ Some(
+ s"You are using version $highestVersion of $productName, but it appears " +
+ s"you (perhaps indirectly) also depend on older versions of related artifacts. " +
+ s"You can solve this by adding an explicit dependency on version $highestVersion " +
+ s"of the [$toBeUpdated] artifacts to your project. " +
+ "See also: https://doc.akka.io/docs/akka/current/common/binary-compatibility-rules.html#mixed-versioning-is-not-allowed")
+ } else None
+ }
}
/**
@@ -186,28 +208,16 @@ final class ManifestInfo(val system: ExtendedActorSystem) extends Extension {
dependencies: immutable.Seq[String],
logWarning: Boolean,
throwException: Boolean): Boolean = {
- @silent("deprecated")
- val filteredVersions = versions.filterKeys(dependencies.toSet)
- val values = filteredVersions.values.toSet
- if (values.size > 1) {
- val conflictingVersions = values.mkString(", ")
- val fullInfo = filteredVersions.map { case (k, v) => s"$k:$v" }.mkString(", ")
- val highestVersion = values.max
- val message = "Detected possible incompatible versions on the classpath. " +
- s"Please note that a given $productName version MUST be the same across all modules of $productName " +
- s"that you are using, e.g. if you use [$highestVersion] all other modules that are released together MUST be of the " +
- "same version. Make sure you're using a compatible set of libraries. " +
- s"Possibly conflicting versions [$conflictingVersions] in libraries [$fullInfo]"
+ ManifestInfo.checkSameVersion(productName, dependencies, versions) match {
+ case Some(message) =>
+ if (logWarning)
+ Logging(system, getClass).warning(message)
- if (logWarning)
- Logging(system, getClass).warning(message)
-
- if (throwException)
- throw new IllegalStateException(message)
- else
- false
- } else
- true
+ if (throwException)
+ throw new IllegalStateException(message)
+ else
+ false
+ case None => true
+ }
}
-
}
diff --git a/akka-actor/src/main/scala/akka/util/Reflect.scala b/akka-actor/src/main/scala/akka/util/Reflect.scala
index 7e9e14c054..c0245da59b 100644
--- a/akka-actor/src/main/scala/akka/util/Reflect.scala
+++ b/akka-actor/src/main/scala/akka/util/Reflect.scala
@@ -3,13 +3,16 @@
*/
package akka.util
-import scala.util.control.NonFatal
import java.lang.reflect.Constructor
-import scala.collection.immutable
-import java.lang.reflect.Type
-import scala.annotation.tailrec
import java.lang.reflect.ParameterizedType
+import java.lang.reflect.Type
+
+import akka.annotation.InternalApi
+
+import scala.annotation.tailrec
+import scala.collection.immutable
import scala.util.Try
+import scala.util.control.NonFatal
/**
* Collection of internal reflection utilities which may or may not be
@@ -17,6 +20,7 @@ import scala.util.Try
*
* INTERNAL API
*/
+@InternalApi
private[akka] object Reflect {
/**
@@ -137,33 +141,6 @@ private[akka] object Reflect {
rec(root)
}
- /**
- * INTERNAL API
- * Set a val inside a class.
- */
- @tailrec protected[akka] final def lookupAndSetField(
- clazz: Class[_],
- instance: AnyRef,
- name: String,
- value: Any): Boolean = {
- @tailrec def clearFirst(fields: Array[java.lang.reflect.Field], idx: Int): Boolean =
- if (idx < fields.length) {
- val field = fields(idx)
- if (field.getName == name) {
- field.setAccessible(true)
- field.set(instance, value)
- true
- } else clearFirst(fields, idx + 1)
- } else false
-
- clearFirst(clazz.getDeclaredFields, 0) || {
- clazz.getSuperclass match {
- case null => false // clazz == classOf[AnyRef]
- case sc => lookupAndSetField(sc, instance, name, value)
- }
- }
- }
-
/**
* INTERNAL API
*/
diff --git a/akka-actor/src/main/scala/akka/util/SerializedSuspendableExecutionContext.scala b/akka-actor/src/main/scala/akka/util/SerializedSuspendableExecutionContext.scala
index fd973d73b3..7e53f755ff 100644
--- a/akka-actor/src/main/scala/akka/util/SerializedSuspendableExecutionContext.scala
+++ b/akka-actor/src/main/scala/akka/util/SerializedSuspendableExecutionContext.scala
@@ -5,9 +5,11 @@
package akka.util
import java.util.concurrent.atomic.AtomicInteger
+
+import scala.annotation.{ switch, tailrec }
import scala.concurrent.ExecutionContext
import scala.util.control.NonFatal
-import scala.annotation.{ switch, tailrec }
+
import akka.dispatch.AbstractNodeQueue
private[akka] object SerializedSuspendableExecutionContext {
diff --git a/akka-actor/src/main/scala/akka/util/StablePriorityQueue.scala b/akka-actor/src/main/scala/akka/util/StablePriorityQueue.scala
index 3518e32ae4..23703f3438 100644
--- a/akka-actor/src/main/scala/akka/util/StablePriorityQueue.scala
+++ b/akka-actor/src/main/scala/akka/util/StablePriorityQueue.scala
@@ -4,9 +4,9 @@
package akka.util
+import java.util.{ AbstractQueue, Comparator, Iterator, PriorityQueue }
import java.util.concurrent.PriorityBlockingQueue
import java.util.concurrent.atomic.AtomicLong
-import java.util.{ AbstractQueue, Comparator, Iterator, PriorityQueue }
/**
* PriorityQueueStabilizer wraps a priority queue so that it respects FIFO for elements of equal priority.
diff --git a/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala b/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala
index 5311bbf3fb..a026709a3e 100644
--- a/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala
+++ b/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala
@@ -5,6 +5,7 @@
package akka.util
import scala.collection.immutable
+
import akka.util.ccompat._
/**
diff --git a/akka-actor/src/main/scala/akka/util/Timeout.scala b/akka-actor/src/main/scala/akka/util/Timeout.scala
index 4ed6110cb1..47a28fc65c 100644
--- a/akka-actor/src/main/scala/akka/util/Timeout.scala
+++ b/akka-actor/src/main/scala/akka/util/Timeout.scala
@@ -4,11 +4,12 @@
package akka.util
-import language.implicitConversions
-
import java.util.concurrent.TimeUnit
+
import scala.concurrent.duration.{ Duration, FiniteDuration }
+import language.implicitConversions
+
@SerialVersionUID(1L)
case class Timeout(duration: FiniteDuration) {
diff --git a/akka-bench-jmh/src/main/resources/logback.xml b/akka-bench-jmh/src/main/resources/logback.xml
index 8beec7b3f1..57e6df13ef 100644
--- a/akka-bench-jmh/src/main/resources/logback.xml
+++ b/akka-bench-jmh/src/main/resources/logback.xml
@@ -5,7 +5,7 @@
- %date{ISO8601} %-5level %logger %marker - %msg {%mdc}%n
+ %date{ISO8601} %-5level %logger %marker - %msg MDC: {%mdc}%n
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/ActorBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/ActorBenchmark.scala
index d791d712c2..e1ab29d962 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/ActorBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/ActorBenchmark.scala
@@ -4,12 +4,14 @@
package akka.actor
+import java.util.concurrent.TimeUnit
+
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import BenchmarkActors._
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
-import java.util.concurrent.TimeUnit
-import scala.concurrent.Await
-import BenchmarkActors._
-import scala.concurrent.duration._
object ActorBenchmark {
// Constants because they are used in annotations
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/ActorCreationBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/ActorCreationBenchmark.scala
index b9f898e2ba..9c88f1c810 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/ActorCreationBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/ActorCreationBenchmark.scala
@@ -4,10 +4,12 @@
package akka.actor
-import scala.concurrent.duration._
import java.util.concurrent.TimeUnit
-import org.openjdk.jmh.annotations._
+
import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import org.openjdk.jmh.annotations._
/*
regex checking:
@@ -26,7 +28,7 @@ hand checking:
class ActorCreationBenchmark {
implicit val system: ActorSystem = ActorSystem()
- final val props = Props[MyActor]
+ final val props = Props[MyActor]()
var i = 1
def name = {
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/ActorPathValidationBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/ActorPathValidationBenchmark.scala
index 91ba473399..403778be81 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/ActorPathValidationBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/ActorPathValidationBenchmark.scala
@@ -6,13 +6,13 @@ package akka.actor
import java.util.concurrent.TimeUnit
+import org.openjdk.jmh.annotations.{ Scope => JmhScope }
import org.openjdk.jmh.annotations.Benchmark
import org.openjdk.jmh.annotations.BenchmarkMode
import org.openjdk.jmh.annotations.Fork
import org.openjdk.jmh.annotations.Measurement
import org.openjdk.jmh.annotations.Mode
import org.openjdk.jmh.annotations.OutputTimeUnit
-import org.openjdk.jmh.annotations.{ Scope => JmhScope }
import org.openjdk.jmh.annotations.State
import org.openjdk.jmh.annotations.Warmup
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/AffinityPoolComparativeBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/AffinityPoolComparativeBenchmark.scala
index f8f4d15205..6320278203 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/AffinityPoolComparativeBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/AffinityPoolComparativeBenchmark.scala
@@ -6,11 +6,12 @@ package akka.actor
import java.util.concurrent.TimeUnit
-import akka.actor.BenchmarkActors._
-import akka.actor.ForkJoinActorBenchmark.cores
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
+import akka.actor.BenchmarkActors._
+import akka.actor.ForkJoinActorBenchmark.cores
+
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(1)
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/AffinityPoolIdleCPULevelBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/AffinityPoolIdleCPULevelBenchmark.scala
index 29479f57be..8802d64cee 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/AffinityPoolIdleCPULevelBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/AffinityPoolIdleCPULevelBenchmark.scala
@@ -6,10 +6,11 @@ package akka.actor
import java.util.concurrent.TimeUnit
-import akka.actor.BenchmarkActors._
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
+import akka.actor.BenchmarkActors._
+
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(1)
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/AffinityPoolRequestResponseBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/AffinityPoolRequestResponseBenchmark.scala
index e0b48448a0..3ad0f5ce36 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/AffinityPoolRequestResponseBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/AffinityPoolRequestResponseBenchmark.scala
@@ -6,11 +6,12 @@ package akka.actor
import java.util.concurrent.{ CountDownLatch, TimeUnit }
-import akka.actor.BenchmarkActors._
-import akka.actor.ForkJoinActorBenchmark.cores
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
+import akka.actor.BenchmarkActors._
+import akka.actor.ForkJoinActorBenchmark.cores
+
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(1)
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/BenchmarkActors.scala b/akka-bench-jmh/src/main/scala/akka/actor/BenchmarkActors.scala
index 2003ebc804..15f8e1ee04 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/BenchmarkActors.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/BenchmarkActors.scala
@@ -7,8 +7,8 @@ package akka.actor
import java.util.concurrent.{ CountDownLatch, TimeUnit }
import scala.concurrent.Await
-import scala.concurrent.duration.Duration
import scala.concurrent.duration._
+import scala.concurrent.duration.Duration
object BenchmarkActors {
@@ -48,7 +48,7 @@ object BenchmarkActors {
}
class EchoSender(messagesPerPair: Int, latch: CountDownLatch, batchSize: Int) extends Actor {
- private val echo = context.actorOf(Props[Echo].withDispatcher(context.props.dispatcher), "echo")
+ private val echo = context.actorOf(Props[Echo]().withDispatcher(context.props.dispatcher), "echo")
private var left = messagesPerPair / 2
private var batch = 0
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/DirectByteBufferPoolBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/DirectByteBufferPoolBenchmark.scala
index fcb845e846..e96d7f15ac 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/DirectByteBufferPoolBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/DirectByteBufferPoolBenchmark.scala
@@ -8,9 +8,10 @@ import java.nio.ByteBuffer
import java.util.Random
import java.util.concurrent.TimeUnit
-import akka.io.DirectByteBufferPool
import org.openjdk.jmh.annotations._
+import akka.io.DirectByteBufferPool
+
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.AverageTime))
@OutputTimeUnit(TimeUnit.NANOSECONDS)
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/ForkJoinActorBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/ForkJoinActorBenchmark.scala
index f1c26ac91f..49229e6875 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/ForkJoinActorBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/ForkJoinActorBenchmark.scala
@@ -4,14 +4,17 @@
package akka.actor
-import akka.testkit.TestProbe
+import java.util.concurrent.TimeUnit
+
+import scala.annotation.tailrec
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import BenchmarkActors._
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
-import java.util.concurrent.TimeUnit
-import scala.concurrent.Await
-import scala.annotation.tailrec
-import BenchmarkActors._
-import scala.concurrent.duration._
+
+import akka.testkit.TestProbe
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/JCToolsMailbox.scala b/akka-bench-jmh/src/main/scala/akka/actor/JCToolsMailbox.scala
index 1f3738dd48..2156c1618b 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/JCToolsMailbox.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/JCToolsMailbox.scala
@@ -4,16 +4,18 @@
package akka.actor
-import akka.dispatch.MailboxType
-import akka.dispatch.ProducesMessageQueue
-import akka.dispatch.BoundedNodeMessageQueue
-import com.typesafe.config.Config
-import akka.dispatch.MessageQueue
-import akka.dispatch.BoundedMessageQueueSemantics
-import scala.concurrent.duration.Duration
-import akka.dispatch.Envelope
-import org.jctools.queues.MpscGrowableArrayQueue
import scala.annotation.tailrec
+import scala.concurrent.duration.Duration
+
+import com.typesafe.config.Config
+import org.jctools.queues.MpscGrowableArrayQueue
+
+import akka.dispatch.BoundedMessageQueueSemantics
+import akka.dispatch.BoundedNodeMessageQueue
+import akka.dispatch.Envelope
+import akka.dispatch.MailboxType
+import akka.dispatch.MessageQueue
+import akka.dispatch.ProducesMessageQueue
case class JCToolsMailbox(val capacity: Int) extends MailboxType with ProducesMessageQueue[BoundedNodeMessageQueue] {
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/ManyToOneArrayMailbox.scala b/akka-bench-jmh/src/main/scala/akka/actor/ManyToOneArrayMailbox.scala
index 251ca5a623..928e1e9297 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/ManyToOneArrayMailbox.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/ManyToOneArrayMailbox.scala
@@ -4,16 +4,18 @@
package akka.actor
-import akka.dispatch.MailboxType
-import akka.dispatch.ProducesMessageQueue
-import akka.dispatch.BoundedNodeMessageQueue
-import com.typesafe.config.Config
-import akka.dispatch.MessageQueue
-import akka.dispatch.BoundedMessageQueueSemantics
-import scala.concurrent.duration.Duration
-import akka.dispatch.Envelope
-import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
import scala.annotation.tailrec
+import scala.concurrent.duration.Duration
+
+import com.typesafe.config.Config
+import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
+
+import akka.dispatch.BoundedMessageQueueSemantics
+import akka.dispatch.BoundedNodeMessageQueue
+import akka.dispatch.Envelope
+import akka.dispatch.MailboxType
+import akka.dispatch.MessageQueue
+import akka.dispatch.ProducesMessageQueue
/**
* ManyToOneArrayMailbox is a high-performance, multiple-producer single-consumer, bounded MailboxType,
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/RouterPoolCreationBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/RouterPoolCreationBenchmark.scala
index daf3cf9e9e..822d6cfe51 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/RouterPoolCreationBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/RouterPoolCreationBenchmark.scala
@@ -4,13 +4,16 @@
package akka.actor
+import java.util.concurrent.TimeUnit
+
import scala.concurrent.Await
import scala.concurrent.duration._
+
+import org.openjdk.jmh.annotations._
+
import akka.routing.RoundRobinPool
import akka.testkit.TestActors
import akka.testkit.TestProbe
-import org.openjdk.jmh.annotations._
-import java.util.concurrent.TimeUnit
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.SingleShotTime))
@@ -21,7 +24,7 @@ class RouterPoolCreationBenchmark {
implicit val system: ActorSystem = ActorSystem()
val probe = TestProbe()
- Props[TestActors.EchoActor]
+ Props[TestActors.EchoActor]()
@Param(Array("1000", "2000", "3000", "4000"))
var size = 0
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/ScheduleBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/ScheduleBenchmark.scala
index 0dc7cb6f4a..97b081039e 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/ScheduleBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/ScheduleBenchmark.scala
@@ -7,12 +7,13 @@ package akka.actor
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicInteger
-import akka.util.Timeout
-import org.openjdk.jmh.annotations._
-
+import scala.concurrent.{ Await, Promise }
import scala.concurrent.ExecutionContext.Implicits.global
import scala.concurrent.duration._
-import scala.concurrent.{ Await, Promise }
+
+import org.openjdk.jmh.annotations._
+
+import akka.util.Timeout
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/StashCreationBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/StashCreationBenchmark.scala
index bc4a8cbc79..6293dfbe6f 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/StashCreationBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/StashCreationBenchmark.scala
@@ -4,12 +4,15 @@
package akka.actor
+import java.util.concurrent.TimeUnit
+
import scala.concurrent.Await
import scala.concurrent.duration._
-import akka.testkit.TestProbe
+
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
-import java.util.concurrent.TimeUnit
+
+import akka.testkit.TestProbe
object StashCreationBenchmark {
class StashingActor extends Actor with Stash {
@@ -18,7 +21,7 @@ object StashCreationBenchmark {
}
}
- val props = Props[StashingActor]
+ val props = Props[StashingActor]()
}
@State(Scope.Benchmark)
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/TellOnlyBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/TellOnlyBenchmark.scala
index 0511ab9568..7ba3618a8f 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/TellOnlyBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/TellOnlyBenchmark.scala
@@ -4,14 +4,17 @@
package akka.actor
+import java.util.concurrent.TimeUnit
+
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import com.typesafe.config.{ Config, ConfigFactory }
+import org.openjdk.jmh.annotations._
+
import akka.dispatch._
import akka.testkit.TestProbe
import akka.util.Helpers.ConfigOps
-import com.typesafe.config.{ Config, ConfigFactory }
-import java.util.concurrent.TimeUnit
-import org.openjdk.jmh.annotations._
-import scala.concurrent.duration._
-import scala.concurrent.Await
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.SingleShotTime))
@@ -61,7 +64,7 @@ class TellOnlyBenchmark {
@Setup(Level.Iteration)
def setupIteration(): Unit = {
- actor = system.actorOf(Props[TellOnlyBenchmark.Echo].withDispatcher("dropping-dispatcher"))
+ actor = system.actorOf(Props[TellOnlyBenchmark.Echo]().withDispatcher("dropping-dispatcher"))
probe = TestProbe()
probe.watch(actor)
probe.send(actor, message)
@@ -97,7 +100,7 @@ object TellOnlyBenchmark {
class Echo extends Actor {
def receive = {
- case s @ `stop` =>
+ case `stop` =>
context.stop(self)
case m => sender ! m
}
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/typed/TypedActorBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/typed/TypedActorBenchmark.scala
index 809bba8b83..6bb211ad89 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/typed/TypedActorBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/typed/TypedActorBenchmark.scala
@@ -6,11 +6,12 @@ package akka.actor.typed
import java.util.concurrent.TimeUnit
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
-import scala.concurrent.Await
-import scala.concurrent.duration._
import akka.actor.typed.scaladsl.AskPattern._
object TypedActorBenchmark {
@@ -47,7 +48,7 @@ class TypedActorBenchmark {
implicit var system: ActorSystem[Start] = _
- implicit val askTimeout = akka.util.Timeout(timeout)
+ implicit val askTimeout: akka.util.Timeout = akka.util.Timeout(timeout)
@Setup(Level.Trial)
def setup(): Unit = {
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/typed/TypedBenchmarkActors.scala b/akka-bench-jmh/src/main/scala/akka/actor/typed/TypedBenchmarkActors.scala
index c00c5594c2..5233891baf 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/typed/TypedBenchmarkActors.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/typed/TypedBenchmarkActors.scala
@@ -16,7 +16,7 @@ object TypedBenchmarkActors {
// to avoid benchmark to be dominated by allocations of message
// we pass the respondTo actor ref into the behavior
- final case object Message
+ case object Message
private def echoBehavior(respondTo: ActorRef[Message.type]): Behavior[Message.type] = Behaviors.receive { (_, _) =>
respondTo ! Message
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/typed/TypedForkJoinActorBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/typed/TypedForkJoinActorBenchmark.scala
index 3c323a17ec..e82719a98e 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/typed/TypedForkJoinActorBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/typed/TypedForkJoinActorBenchmark.scala
@@ -6,26 +6,27 @@ package akka.actor.typed
import java.util.concurrent.TimeUnit
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations.Benchmark
-import org.openjdk.jmh.annotations.Level
-import org.openjdk.jmh.annotations.OperationsPerInvocation
-import org.openjdk.jmh.annotations.Param
-import org.openjdk.jmh.annotations.Setup
-import akka.actor.typed.scaladsl.AskPattern._
import org.openjdk.jmh.annotations.BenchmarkMode
import org.openjdk.jmh.annotations.Fork
+import org.openjdk.jmh.annotations.Level
import org.openjdk.jmh.annotations.Measurement
import org.openjdk.jmh.annotations.Mode
+import org.openjdk.jmh.annotations.OperationsPerInvocation
+import org.openjdk.jmh.annotations.Param
import org.openjdk.jmh.annotations.Scope
+import org.openjdk.jmh.annotations.Setup
import org.openjdk.jmh.annotations.State
import org.openjdk.jmh.annotations.TearDown
import org.openjdk.jmh.annotations.Threads
import org.openjdk.jmh.annotations.Warmup
-import scala.concurrent.duration._
-import scala.concurrent.Await
-import scala.concurrent.Future
+import akka.actor.typed.scaladsl.AskPattern._
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@@ -34,8 +35,8 @@ import scala.concurrent.Future
@Warmup(iterations = 10, time = 5, timeUnit = TimeUnit.SECONDS, batchSize = 1)
@Measurement(iterations = 10, time = 15, timeUnit = TimeUnit.SECONDS, batchSize = 1)
class TypedForkJoinActorBenchmark {
- import TypedForkJoinActorBenchmark._
import TypedBenchmarkActors._
+ import TypedForkJoinActorBenchmark._
@Param(Array("50"))
var tpt = 0
diff --git a/akka-bench-jmh/src/main/scala/akka/actor/typed/delivery/ReliableDeliveryBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/typed/delivery/ReliableDeliveryBenchmark.scala
index 575fbd5e4e..12e398138c 100644
--- a/akka-bench-jmh/src/main/scala/akka/actor/typed/delivery/ReliableDeliveryBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/actor/typed/delivery/ReliableDeliveryBenchmark.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2014-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
package akka.actor.typed.delivery
@@ -9,110 +9,190 @@ import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
+import scala.util.Failure
+import scala.util.Success
+
+import com.typesafe.config.ConfigFactory
+import org.openjdk.jmh.annotations._
import akka.Done
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
+import akka.actor.typed.delivery.ProducerController.MessageWithConfirmation
+import akka.actor.typed.receptionist.ServiceKey
import akka.actor.typed.scaladsl.AskPattern._
import akka.actor.typed.scaladsl.Behaviors
-import com.typesafe.config.ConfigFactory
-import org.openjdk.jmh.annotations._
-object ReliableDeliveryBenchmark {
+object Producer {
+ trait Command
- final val messagesPerOperation = 100000
- final val timeout = 30.seconds
+ case object Run extends Command
+ private case class WrappedRequestNext(r: ProducerController.RequestNext[Consumer.Command]) extends Command
+ private case object AskReply extends Command
- object Producer {
- trait Command
+ private implicit val askTimeout: akka.util.Timeout = 5.seconds
- case object Run extends Command
- private case class WrappedRequestNext(r: ProducerController.RequestNext[Consumer.Command]) extends Command
+ def apply(
+ numberOfMessages: Int,
+ useAsk: Boolean,
+ producerController: ActorRef[ProducerController.Command[Consumer.Command]]): Behavior[Command] = {
+ Behaviors.setup { context =>
+ val requestNextAdapter =
+ context.messageAdapter[ProducerController.RequestNext[Consumer.Command]](WrappedRequestNext(_))
- def apply(
- numberOfMessages: Int,
- producerController: ActorRef[ProducerController.Command[Consumer.Command]]): Behavior[Command] = {
- Behaviors.setup { context =>
- val requestNextAdapter =
- context.messageAdapter[ProducerController.RequestNext[Consumer.Command]](WrappedRequestNext(_))
-
- Behaviors.receiveMessage {
- case WrappedRequestNext(next) =>
- if (next.confirmedSeqNr >= numberOfMessages) {
- context.log.info("Completed {} messages", numberOfMessages)
- Behaviors.stopped
- } else {
- next.sendNextTo ! Consumer.TheMessage
- Behaviors.same
+ Behaviors.receiveMessage {
+ case WrappedRequestNext(next) =>
+ if (next.confirmedSeqNr >= numberOfMessages) {
+ context.log.info("Completed {} messages", numberOfMessages)
+ Behaviors.stopped
+ } else if (useAsk) {
+ context.ask[MessageWithConfirmation[Consumer.Command], ProducerController.SeqNr](
+ next.askNextTo,
+ askReplyTo => MessageWithConfirmation(Consumer.TheMessage, askReplyTo)) {
+ case Success(_) => AskReply
+ case Failure(e) => throw e
}
-
- case Run =>
- context.log.info("Starting {} messages", numberOfMessages)
- producerController ! ProducerController.Start(requestNextAdapter)
Behaviors.same
- }
- }
- }
- }
-
- object Consumer {
- trait Command
-
- case object TheMessage extends Command
-
- private case class WrappedDelivery(d: ConsumerController.Delivery[Command]) extends Command
-
- def apply(consumerController: ActorRef[ConsumerController.Command[Command]]): Behavior[Command] = {
- Behaviors.setup { context =>
- val deliveryAdapter =
- context.messageAdapter[ConsumerController.Delivery[Command]](WrappedDelivery(_))
- consumerController ! ConsumerController.Start(deliveryAdapter)
-
- Behaviors.receiveMessagePartial {
- case WrappedDelivery(d @ ConsumerController.Delivery(_, confirmTo)) =>
- context.log.trace("Processed {}", d.seqNr)
- confirmTo ! ConsumerController.Confirmed
+ } else {
+ next.sendNextTo ! Consumer.TheMessage
Behaviors.same
- }
- }
- }
- }
+ }
- object Guardian {
+ case Run =>
+ context.log.info("Starting {} messages", numberOfMessages)
+ producerController ! ProducerController.Start(requestNextAdapter)
+ Behaviors.same
- trait Command
- final case class Run(id: String, numberOfMessages: Int, replyTo: ActorRef[Done]) extends Command
- final case class ProducerTerminated(consumer: ActorRef[Consumer.Command], replyTo: ActorRef[Done]) extends Command
-
- def apply(): Behavior[Command] = {
- Behaviors.setup { context =>
- Behaviors.receiveMessage {
- case Run(id, numberOfMessages, replyTo) =>
- val consumerController = context.spawn(ConsumerController[Consumer.Command](), s"consumerController-$id")
- val consumer = context.spawn(Consumer(consumerController), s"consumer-$id")
-
- val producerController = context.spawn(
- ProducerController[Consumer.Command](id, durableQueueBehavior = None),
- s"producerController-$id")
- val producer = context.spawn(Producer(numberOfMessages, producerController), s"producer-$id")
- context.watchWith(producer, ProducerTerminated(consumer, replyTo))
-
- consumerController ! ConsumerController.RegisterToProducerController(producerController)
-
- producer ! Producer.Run
-
- Behaviors.same
-
- case ProducerTerminated(consumer, replyTo) =>
- context.stop(consumer)
- replyTo ! Done
- Behaviors.same
- }
+ case AskReply =>
+ Behaviors.same
}
}
}
}
+
+object Consumer {
+ trait Command
+
+ case object TheMessage extends Command
+
+ private case class WrappedDelivery(d: ConsumerController.Delivery[Command]) extends Command
+
+ def serviceKey(testName: String): ServiceKey[ConsumerController.Command[Command]] =
+ ServiceKey[ConsumerController.Command[Consumer.Command]](testName)
+
+ def apply(consumerController: ActorRef[ConsumerController.Command[Command]]): Behavior[Command] = {
+ Behaviors.setup { context =>
+ val traceEnabled = context.log.isTraceEnabled
+ val deliveryAdapter =
+ context.messageAdapter[ConsumerController.Delivery[Command]](WrappedDelivery(_))
+ consumerController ! ConsumerController.Start(deliveryAdapter)
+
+ Behaviors.receiveMessagePartial {
+ case WrappedDelivery(d @ ConsumerController.Delivery(_, confirmTo)) =>
+ if (traceEnabled)
+ context.log.trace("Processed {}", d.seqNr)
+ confirmTo ! ConsumerController.Confirmed
+ Behaviors.same
+ }
+ }
+ }
+}
+
+object WorkPullingProducer {
+ trait Command
+
+ case object Run extends Command
+ private case class WrappedRequestNext(r: WorkPullingProducerController.RequestNext[Consumer.Command]) extends Command
+
+ def apply(
+ numberOfMessages: Int,
+ producerController: ActorRef[WorkPullingProducerController.Command[Consumer.Command]]): Behavior[Command] = {
+ Behaviors.setup { context =>
+ val requestNextAdapter =
+ context.messageAdapter[WorkPullingProducerController.RequestNext[Consumer.Command]](WrappedRequestNext(_))
+ var remaining = numberOfMessages + context.system.settings.config
+ .getInt("akka.reliable-delivery.consumer-controller.flow-control-window")
+
+ Behaviors.receiveMessage {
+ case WrappedRequestNext(next) =>
+ remaining -= 1
+ if (remaining == 0) {
+ context.log.info("Completed {} messages", numberOfMessages)
+ Behaviors.stopped
+ } else {
+ next.sendNextTo ! Consumer.TheMessage
+ Behaviors.same
+ }
+
+ case Run =>
+ context.log.info("Starting {} messages", numberOfMessages)
+ producerController ! WorkPullingProducerController.Start(requestNextAdapter)
+ Behaviors.same
+ }
+ }
+ }
+}
+
+object Guardian {
+
+ trait Command
+ final case class RunPointToPoint(id: String, numberOfMessages: Int, useAsk: Boolean, replyTo: ActorRef[Done])
+ extends Command
+ final case class RunWorkPulling(id: String, numberOfMessages: Int, workers: Int, replyTo: ActorRef[Done])
+ extends Command
+ final case class ProducerTerminated(consumers: List[ActorRef[Consumer.Command]], replyTo: ActorRef[Done])
+ extends Command
+
+ def apply(): Behavior[Command] = {
+ Behaviors.setup { context =>
+ Behaviors.receiveMessage {
+ case RunPointToPoint(id, numberOfMessages, useAsk, replyTo) =>
+ // point-to-point
+ val consumerController =
+ context.spawn(ConsumerController[Consumer.Command](), s"consumerController-$id")
+ val consumers = List(context.spawn(Consumer(consumerController), s"consumer-$id"))
+
+ val producerController = context.spawn(
+ ProducerController[Consumer.Command](id, durableQueueBehavior = None),
+ s"producerController-$id")
+ val producer = context.spawn(Producer(numberOfMessages, useAsk, producerController), s"producer-$id")
+ consumerController ! ConsumerController.RegisterToProducerController(producerController)
+ context.watchWith(producer, ProducerTerminated(consumers, replyTo))
+ producer ! Producer.Run
+ Behaviors.same
+
+ case RunWorkPulling(id, numberOfMessages, workers, replyTo) =>
+ // workPulling
+ val sKey = Consumer.serviceKey(id)
+ val consumerController =
+ context.spawn(ConsumerController[Consumer.Command](sKey), s"consumerController-$id")
+ val consumers = (1 to workers).map { n =>
+ context.spawn(Consumer(consumerController), s"consumer-$n-$id")
+ }.toList
+
+ val producerController = context.spawn(
+ WorkPullingProducerController[Consumer.Command](id, sKey, durableQueueBehavior = None),
+ s"producerController-$id")
+ val producer = context.spawn(WorkPullingProducer(numberOfMessages, producerController), s"producer-$id")
+ context.watchWith(producer, ProducerTerminated(consumers, replyTo))
+ producer ! WorkPullingProducer.Run
+ Behaviors.same
+
+ case ProducerTerminated(consumers, replyTo) =>
+ consumers.foreach(context.stop)
+ replyTo ! Done
+ Behaviors.same
+ }
+ }
+ }
+}
+
+object ReliableDeliveryBenchmark {
+ final val messagesPerOperation = 100000
+ final val timeout = 30.seconds
+}
+
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(1)
@@ -122,12 +202,12 @@ object ReliableDeliveryBenchmark {
class ReliableDeliveryBenchmark {
import ReliableDeliveryBenchmark._
- @Param(Array("10", "50"))
+ @Param(Array("50"))
var window = 0
implicit var system: ActorSystem[Guardian.Command] = _
- implicit val askTimeout = akka.util.Timeout(timeout)
+ implicit val askTimeout: akka.util.Timeout = akka.util.Timeout(timeout)
@Setup(Level.Trial)
def setup(): Unit = {
@@ -150,8 +230,36 @@ class ReliableDeliveryBenchmark {
@Benchmark
@OperationsPerInvocation(messagesPerOperation)
- def echo(): Unit = {
- Await.result(system.ask(Guardian.Run(UUID.randomUUID().toString, messagesPerOperation, _)), timeout)
+ def pointToPoint(): Unit = {
+ Await.result(
+ system.ask(
+ Guardian.RunPointToPoint(s"point-to-point-${UUID.randomUUID()}", messagesPerOperation, useAsk = false, _)),
+ timeout)
+ }
+
+ @Benchmark
+ @OperationsPerInvocation(messagesPerOperation)
+ def pointToPointAsk(): Unit = {
+ Await.result(
+ system.ask(
+ Guardian.RunPointToPoint(s"point-to-point-${UUID.randomUUID()}", messagesPerOperation, useAsk = true, _)),
+ timeout)
+ }
+
+ @Benchmark
+ @OperationsPerInvocation(messagesPerOperation)
+ def workPulling1(): Unit = {
+ Await.result(
+ system.ask(Guardian.RunWorkPulling(s"work-pulling-${UUID.randomUUID()}", messagesPerOperation, workers = 1, _)),
+ timeout)
+ }
+
+ @Benchmark
+ @OperationsPerInvocation(messagesPerOperation)
+ def workPulling2(): Unit = {
+ Await.result(
+ system.ask(Guardian.RunWorkPulling(s"work-pulling-${UUID.randomUUID()}", messagesPerOperation, workers = 2, _)),
+ timeout)
}
}
diff --git a/akka-bench-jmh/src/main/scala/akka/cluster/ddata/ORSetMergeBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/cluster/ddata/ORSetMergeBenchmark.scala
index e2ad7b7ae4..e8abdac47c 100644
--- a/akka-bench-jmh/src/main/scala/akka/cluster/ddata/ORSetMergeBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/cluster/ddata/ORSetMergeBenchmark.scala
@@ -5,20 +5,22 @@
package akka.cluster.ddata
import java.util.concurrent.TimeUnit
+
+import org.openjdk.jmh.annotations.{ Scope => JmhScope }
import org.openjdk.jmh.annotations.Benchmark
import org.openjdk.jmh.annotations.BenchmarkMode
import org.openjdk.jmh.annotations.Fork
+import org.openjdk.jmh.annotations.Level
import org.openjdk.jmh.annotations.Measurement
import org.openjdk.jmh.annotations.Mode
import org.openjdk.jmh.annotations.OutputTimeUnit
-import org.openjdk.jmh.annotations.{ Scope => JmhScope }
-import org.openjdk.jmh.annotations.State
-import org.openjdk.jmh.annotations.Warmup
-import akka.cluster.UniqueAddress
-import akka.actor.Address
import org.openjdk.jmh.annotations.Param
import org.openjdk.jmh.annotations.Setup
-import org.openjdk.jmh.annotations.Level
+import org.openjdk.jmh.annotations.State
+import org.openjdk.jmh.annotations.Warmup
+
+import akka.actor.Address
+import akka.cluster.UniqueAddress
@Fork(2)
@State(JmhScope.Benchmark)
diff --git a/akka-bench-jmh/src/main/scala/akka/cluster/ddata/ORSetSerializationBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/cluster/ddata/ORSetSerializationBenchmark.scala
index efda7a1801..4ccbdf394a 100644
--- a/akka-bench-jmh/src/main/scala/akka/cluster/ddata/ORSetSerializationBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/cluster/ddata/ORSetSerializationBenchmark.scala
@@ -9,13 +9,8 @@ import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
-import akka.actor.ActorRef
-import akka.actor.ActorSystem
-import akka.actor.Props
-import akka.cluster.Cluster
-import akka.serialization.SerializationExtension
-import akka.serialization.Serializers
import com.typesafe.config.ConfigFactory
+import org.openjdk.jmh.annotations.{ Scope => JmhScope }
import org.openjdk.jmh.annotations.Benchmark
import org.openjdk.jmh.annotations.BenchmarkMode
import org.openjdk.jmh.annotations.Fork
@@ -25,7 +20,13 @@ import org.openjdk.jmh.annotations.OutputTimeUnit
import org.openjdk.jmh.annotations.State
import org.openjdk.jmh.annotations.TearDown
import org.openjdk.jmh.annotations.Warmup
-import org.openjdk.jmh.annotations.{ Scope => JmhScope }
+
+import akka.actor.ActorRef
+import akka.actor.ActorSystem
+import akka.actor.Props
+import akka.cluster.Cluster
+import akka.serialization.SerializationExtension
+import akka.serialization.Serializers
@Fork(2)
@State(JmhScope.Benchmark)
diff --git a/akka-bench-jmh/src/main/scala/akka/cluster/ddata/VersionVectorBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/cluster/ddata/VersionVectorBenchmark.scala
index 10d55b6389..c1f4153412 100644
--- a/akka-bench-jmh/src/main/scala/akka/cluster/ddata/VersionVectorBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/cluster/ddata/VersionVectorBenchmark.scala
@@ -5,20 +5,22 @@
package akka.cluster.ddata
import java.util.concurrent.TimeUnit
+
+import org.openjdk.jmh.annotations.{ Scope => JmhScope }
import org.openjdk.jmh.annotations.Benchmark
import org.openjdk.jmh.annotations.BenchmarkMode
import org.openjdk.jmh.annotations.Fork
+import org.openjdk.jmh.annotations.Level
import org.openjdk.jmh.annotations.Measurement
import org.openjdk.jmh.annotations.Mode
import org.openjdk.jmh.annotations.OutputTimeUnit
-import org.openjdk.jmh.annotations.{ Scope => JmhScope }
-import org.openjdk.jmh.annotations.State
-import org.openjdk.jmh.annotations.Warmup
-import akka.cluster.UniqueAddress
-import akka.actor.Address
import org.openjdk.jmh.annotations.Param
import org.openjdk.jmh.annotations.Setup
-import org.openjdk.jmh.annotations.Level
+import org.openjdk.jmh.annotations.State
+import org.openjdk.jmh.annotations.Warmup
+
+import akka.actor.Address
+import akka.cluster.UniqueAddress
@Fork(2)
@State(JmhScope.Benchmark)
diff --git a/akka-bench-jmh/src/main/scala/akka/dispatch/NodeQueueBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/dispatch/NodeQueueBenchmark.scala
index 4ed8064fde..9312619fd1 100644
--- a/akka-bench-jmh/src/main/scala/akka/dispatch/NodeQueueBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/dispatch/NodeQueueBenchmark.scala
@@ -4,12 +4,15 @@
package akka.dispatch
-import akka.actor._
-import org.openjdk.jmh.annotations._
-import com.typesafe.config.ConfigFactory
import java.util.concurrent.TimeUnit
+
import scala.concurrent.Await
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+import org.openjdk.jmh.annotations._
+
+import akka.actor._
import akka.testkit.TestProbe
object NodeQueueBenchmark {
@@ -39,7 +42,7 @@ mailbox {
mailbox-capacity = 1000000
}
""").withFallback(ConfigFactory.load())
- implicit val sys = ActorSystem("ANQ", config)
+ implicit val sys: ActorSystem = ActorSystem("ANQ", config)
val ref = sys.actorOf(Props(new Actor {
def receive = {
case Stop => sender() ! Stop
diff --git a/akka-bench-jmh/src/main/scala/akka/event/LogLevelAccessBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/event/LogLevelAccessBenchmark.scala
index af71c613ad..0f51c143e1 100644
--- a/akka-bench-jmh/src/main/scala/akka/event/LogLevelAccessBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/event/LogLevelAccessBenchmark.scala
@@ -6,9 +6,10 @@ package akka.event
import java.util.concurrent.TimeUnit
-import akka.event.Logging.LogLevel
import org.openjdk.jmh.annotations._
+import akka.event.Logging.LogLevel
+
@Fork(3)
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
diff --git a/akka-bench-jmh/src/main/scala/akka/persistence/LevelDbBatchingBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/persistence/LevelDbBatchingBenchmark.scala
index 598eb7c8c8..cf3848f01d 100644
--- a/akka-bench-jmh/src/main/scala/akka/persistence/LevelDbBatchingBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/persistence/LevelDbBatchingBenchmark.scala
@@ -6,14 +6,17 @@ package akka.persistence
import java.io.File
import java.util.concurrent.TimeUnit
+
import scala.concurrent.Await
import scala.concurrent.duration._
+
+import org.apache.commons.io.FileUtils
+import org.openjdk.jmh.annotations._
+
import akka.actor._
import akka.persistence.journal.AsyncWriteTarget._
import akka.persistence.journal.leveldb.{ SharedLeveldbJournal, SharedLeveldbStore }
import akka.testkit.TestProbe
-import org.apache.commons.io.FileUtils
-import org.openjdk.jmh.annotations._
/*
# OS: OSX 10.9.3
@@ -52,7 +55,7 @@ class LevelDbBatchingBenchmark {
SharedLeveldbJournal.setStore(store, sys)
probe = TestProbe()(sys)
- store = sys.actorOf(Props[SharedLeveldbStore], "store")
+ store = sys.actorOf(Props[SharedLeveldbStore](), "store")
}
@TearDown(Level.Trial)
diff --git a/akka-bench-jmh/src/main/scala/akka/persistence/PersistenceActorDeferBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/persistence/PersistenceActorDeferBenchmark.scala
index fa5512ff20..3429755cba 100644
--- a/akka-bench-jmh/src/main/scala/akka/persistence/PersistenceActorDeferBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/persistence/PersistenceActorDeferBenchmark.scala
@@ -4,14 +4,17 @@
package akka.persistence
+import java.io.File
+
+import scala.concurrent.Await
import scala.concurrent.duration._
+
+import org.apache.commons.io.FileUtils
import org.openjdk.jmh.annotations._
+import org.openjdk.jmh.annotations.Scope
+
import akka.actor._
import akka.testkit.TestProbe
-import java.io.File
-import org.apache.commons.io.FileUtils
-import org.openjdk.jmh.annotations.Scope
-import scala.concurrent.Await
/*
# OS: OSX 10.9.3
diff --git a/akka-bench-jmh/src/main/scala/akka/persistence/PersistentActorBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/persistence/PersistentActorBenchmark.scala
index 0676f06eac..e5f1872520 100644
--- a/akka-bench-jmh/src/main/scala/akka/persistence/PersistentActorBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/persistence/PersistentActorBenchmark.scala
@@ -4,14 +4,17 @@
package akka.persistence
+import java.io.File
+
+import scala.concurrent.Await
import scala.concurrent.duration._
+
+import org.apache.commons.io.FileUtils
import org.openjdk.jmh.annotations._
+import org.openjdk.jmh.annotations.Scope
+
import akka.actor._
import akka.testkit.TestProbe
-import java.io.File
-import org.apache.commons.io.FileUtils
-import org.openjdk.jmh.annotations.Scope
-import scala.concurrent.Await
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
diff --git a/akka-bench-jmh/src/main/scala/akka/persistence/PersistentActorWithAtLeastOnceDeliveryBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/persistence/PersistentActorWithAtLeastOnceDeliveryBenchmark.scala
index c621464f6c..03ab940536 100644
--- a/akka-bench-jmh/src/main/scala/akka/persistence/PersistentActorWithAtLeastOnceDeliveryBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/persistence/PersistentActorWithAtLeastOnceDeliveryBenchmark.scala
@@ -4,15 +4,18 @@
package akka.persistence
+import java.io.File
+
+import scala.concurrent.Await
import scala.concurrent.duration._
+import scala.concurrent.duration._
+
+import org.apache.commons.io.FileUtils
import org.openjdk.jmh.annotations._
+import org.openjdk.jmh.annotations.Scope
+
import akka.actor._
import akka.testkit.TestProbe
-import java.io.File
-import org.apache.commons.io.FileUtils
-import org.openjdk.jmh.annotations.Scope
-import scala.concurrent.duration._
-import scala.concurrent.Await
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@@ -45,7 +48,7 @@ class PersistentActorWithAtLeastOnceDeliveryBenchmark {
storageLocations.foreach(FileUtils.deleteDirectory)
- destinationActor = system.actorOf(Props[DestinationActor], "destination")
+ destinationActor = system.actorOf(Props[DestinationActor](), "destination")
noPersistPersistentActorWithAtLeastOnceDelivery = system.actorOf(
Props(classOf[NoPersistPersistentActorWithAtLeastOnceDelivery], dataCount, probe.ref, destinationActor.path),
diff --git a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala
index 32a9042a78..544e8f3bd6 100644
--- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala
@@ -10,6 +10,15 @@ import java.nio.ByteOrder
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
+import scala.annotation.tailrec
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+import org.openjdk.jmh.annotations._
+
import akka.Done
import akka.NotUsed
import akka.actor._
@@ -22,14 +31,6 @@ import akka.serialization.SerializationExtension
import akka.stream.SystemMaterializer
import akka.stream.scaladsl._
import akka.util.OptionVal
-import com.github.ghik.silencer.silent
-import com.typesafe.config.ConfigFactory
-import org.openjdk.jmh.annotations._
-
-import scala.annotation.tailrec
-import scala.concurrent.Await
-import scala.concurrent.Future
-import scala.concurrent.duration._
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
diff --git a/akka-bench-jmh/src/main/scala/akka/remote/artery/LiteralEncodingBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/LiteralEncodingBenchmark.scala
index 5b3ac99f31..aab8a1c884 100644
--- a/akka-bench-jmh/src/main/scala/akka/remote/artery/LiteralEncodingBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/LiteralEncodingBenchmark.scala
@@ -8,9 +8,11 @@ import java.nio.ByteBuffer
import java.nio.ByteOrder
import java.nio.charset.Charset
import java.util.concurrent.TimeUnit
-import akka.util.Unsafe
+
import org.openjdk.jmh.annotations._
+import akka.util.Unsafe
+
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
@BenchmarkMode(Array(Mode.Throughput))
diff --git a/akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala
index 4e4134d8e7..1bcc1057e5 100644
--- a/akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala
@@ -8,17 +8,18 @@ import java.util.concurrent.CountDownLatch
import java.util.concurrent.CyclicBarrier
import java.util.concurrent.TimeUnit
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
+import org.openjdk.jmh.annotations._
+
import akka.actor.ActorSystem
import akka.stream.KillSwitches
import akka.stream.OverflowStrategy
import akka.stream.SystemMaterializer
import akka.stream.scaladsl._
-import com.typesafe.config.ConfigFactory
-import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
-import org.openjdk.jmh.annotations._
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
@@ -31,7 +32,7 @@ class SendQueueBenchmark {
val config = ConfigFactory.parseString("""
""")
- implicit val system = ActorSystem("SendQueueBenchmark", config)
+ implicit val system: ActorSystem = ActorSystem("SendQueueBenchmark", config)
@Setup
def setup(): Unit = {
diff --git a/akka-bench-jmh/src/main/scala/akka/remote/compress/HeavyHittersBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/compress/HeavyHittersBenchmark.scala
index 27b6c251bc..1d71b9ad20 100644
--- a/akka-bench-jmh/src/main/scala/akka/remote/compress/HeavyHittersBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/remote/compress/HeavyHittersBenchmark.scala
@@ -6,10 +6,11 @@ package akka.remote.compress
import java.util.Random
-import akka.remote.artery.compress.TopHeavyHitters
import org.openjdk.jmh.annotations._
import org.openjdk.jmh.infra.Blackhole
+import akka.remote.artery.compress.TopHeavyHitters
+
/**
* On Macbook pro:
* [info] Benchmark (n) Mode Cnt Score Error Units
diff --git a/akka-bench-jmh/src/main/scala/akka/serialization/jackson/JacksonSerializationBench.scala b/akka-bench-jmh/src/main/scala/akka/serialization/jackson/JacksonSerializationBench.scala
index 612acc82f7..e11418911c 100644
--- a/akka-bench-jmh/src/main/scala/akka/serialization/jackson/JacksonSerializationBench.scala
+++ b/akka-bench-jmh/src/main/scala/akka/serialization/jackson/JacksonSerializationBench.scala
@@ -4,22 +4,23 @@
package akka.serialization.jackson
+import java.time.{ Duration => JDuration }
import java.time.Instant
import java.time.LocalDateTime
-import java.time.{ Duration => JDuration }
import java.util
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+import org.openjdk.jmh.annotations._
+
import akka.actor._
import akka.serialization.Serialization
import akka.serialization.SerializationExtension
import akka.serialization.SerializerWithStringManifest
-import com.github.ghik.silencer.silent
-import com.typesafe.config.ConfigFactory
-import org.openjdk.jmh.annotations._
object JacksonSerializationBench {
trait TestMessage
@@ -189,6 +190,10 @@ class JacksonSerializationBench {
@Param(Array("jackson-json", "jackson-cbor")) // "java"
private var serializerName: String = _
+ @silent("immutable val")
+ @Param(Array("off", "gzip", "lz4"))
+ private var compression: String = _
+
@Setup(Level.Trial)
def setupTrial(): Unit = {
val config = ConfigFactory.parseString(s"""
@@ -207,7 +212,7 @@ class JacksonSerializationBench {
}
}
akka.serialization.jackson.jackson-json.compression {
- algorithm = off
+ algorithm = $compression
compress-larger-than = 100 b
}
""")
@@ -221,10 +226,18 @@ class JacksonSerializationBench {
Await.result(system.terminate(), 5.seconds)
}
+ private var size = 0L
+
private def serializeDeserialize[T <: AnyRef](msg: T): T = {
serialization.findSerializerFor(msg) match {
case serializer: SerializerWithStringManifest =>
val blob = serializer.toBinary(msg)
+ if (size != blob.length) {
+ size = blob.length
+ println(
+ s"# Size is $size of ${msg.getClass.getName} with " +
+ s"${system.settings.config.getString("akka.serialization.jackson.jackson-json.compression.algorithm")}")
+ }
serializer.fromBinary(blob, serializer.manifest(msg)).asInstanceOf[T]
case serializer =>
val blob = serializer.toBinary(msg)
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/AskBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/AskBenchmark.scala
index df4227b454..a052368f66 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/AskBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/AskBenchmark.scala
@@ -7,6 +7,12 @@ package akka.stream
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+import org.openjdk.jmh.annotations._
+
import akka.NotUsed
import akka.actor.Actor
import akka.actor.ActorRef
@@ -17,11 +23,6 @@ import akka.remote.artery.LatchSink
import akka.stream.scaladsl._
import akka.stream.testkit.scaladsl.StreamTestKit
import akka.util.Timeout
-import com.typesafe.config.ConfigFactory
-import org.openjdk.jmh.annotations._
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
object AskBenchmark {
final val OperationsPerInvocation = 100000
@@ -42,14 +43,14 @@ class AskBenchmark {
}
""")
- implicit val system = ActorSystem("MapAsyncBenchmark", config)
+ implicit val system: ActorSystem = ActorSystem("MapAsyncBenchmark", config)
import system.dispatcher
var testSource: Source[java.lang.Integer, NotUsed] = _
var actor: ActorRef = _
- implicit val timeout = Timeout(10.seconds)
+ implicit val timeout: Timeout = Timeout(10.seconds)
@Param(Array("1", "4"))
var parallelism = 0
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/EmptySourceBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/EmptySourceBenchmark.scala
index 4dd4cf79e4..63003c5c50 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/EmptySourceBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/EmptySourceBenchmark.scala
@@ -6,18 +6,19 @@ package akka.stream
import java.util.concurrent.TimeUnit
-import akka.actor.ActorSystem
-import akka.stream.scaladsl._
-import org.openjdk.jmh.annotations._
-
import scala.concurrent._
import scala.concurrent.duration._
+import org.openjdk.jmh.annotations._
+
+import akka.actor.ActorSystem
+import akka.stream.scaladsl._
+
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class EmptySourceBenchmark {
- implicit val system = ActorSystem("EmptySourceBenchmark")
+ implicit val system: ActorSystem = ActorSystem("EmptySourceBenchmark")
@TearDown
def shutdown(): Unit = {
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/FlatMapConcatBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/FlatMapConcatBenchmark.scala
index da9c4bb94a..d27d90220f 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/FlatMapConcatBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/FlatMapConcatBenchmark.scala
@@ -7,6 +7,12 @@ package akka.stream
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+import org.openjdk.jmh.annotations._
+
import akka.NotUsed
import akka.actor.ActorSystem
import akka.remote.artery.BenchTestSource
@@ -14,11 +20,6 @@ import akka.remote.artery.LatchSink
import akka.stream.impl.fusing.GraphStages
import akka.stream.scaladsl._
import akka.stream.testkit.scaladsl.StreamTestKit
-import com.typesafe.config.ConfigFactory
-import org.openjdk.jmh.annotations._
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
object FlatMapConcatBenchmark {
final val OperationsPerInvocation = 100000
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/FlatMapMergeBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/FlatMapMergeBenchmark.scala
index abb757d0ed..f12613f84d 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/FlatMapMergeBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/FlatMapMergeBenchmark.scala
@@ -4,20 +4,23 @@
package akka.stream
-import akka.{ Done, NotUsed }
-import akka.actor.ActorSystem
-import akka.stream.scaladsl._
import java.util.concurrent.TimeUnit
-import akka.remote.artery.BenchTestSourceSameElement
-import org.openjdk.jmh.annotations._
+
import scala.concurrent._
import scala.concurrent.duration._
+import org.openjdk.jmh.annotations._
+
+import akka.{ Done, NotUsed }
+import akka.actor.ActorSystem
+import akka.remote.artery.BenchTestSourceSameElement
+import akka.stream.scaladsl._
+
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class FlatMapMergeBenchmark {
- implicit val system = ActorSystem("FlatMapMergeBenchmark")
+ implicit val system: ActorSystem = ActorSystem("FlatMapMergeBenchmark")
val NumberOfElements = 100000
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/FlowMapBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/FlowMapBenchmark.scala
index e624331f61..6638fa5527 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/FlowMapBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/FlowMapBenchmark.scala
@@ -4,18 +4,21 @@
package akka.stream
+import java.util.concurrent.Semaphore
import java.util.concurrent.TimeUnit
+
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.util.Success
+
+import com.typesafe.config.ConfigFactory
+import org.openjdk.jmh.annotations._
+
import akka.NotUsed
import akka.actor.ActorSystem
import akka.remote.artery.BenchTestSource
-import akka.stream.scaladsl._
-import com.typesafe.config.ConfigFactory
-import org.openjdk.jmh.annotations._
-import java.util.concurrent.Semaphore
-import scala.util.Success
import akka.stream.impl.fusing.GraphStages
-import scala.concurrent.Await
-import scala.concurrent.duration._
+import akka.stream.scaladsl._
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
@@ -48,7 +51,7 @@ class FlowMapBenchmark {
}
}""".stripMargin).withFallback(ConfigFactory.load())
- implicit val system = ActorSystem("test", config)
+ implicit val system: ActorSystem = ActorSystem("test", config)
@Param(Array("true", "false"))
var UseGraphStageIdentity = false
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/FramingBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/FramingBenchmark.scala
index 3e30c9a5f3..9deb5a8d2d 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/FramingBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/FramingBenchmark.scala
@@ -7,6 +7,14 @@ package akka.stream
import java.util.concurrent.Semaphore
import java.util.concurrent.TimeUnit
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.util.Random
+
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.openjdk.jmh.annotations._
+
import akka.NotUsed
import akka.actor.ActorSystem
import akka.remote.artery.BenchTestSourceSameElement
@@ -14,13 +22,6 @@ import akka.stream.scaladsl.Framing
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
import akka.util.ByteString
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.openjdk.jmh.annotations._
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
-import scala.util.Random
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/FusedGraphsBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/FusedGraphsBenchmark.scala
index a610645d9a..7b6bf29bd2 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/FusedGraphsBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/FusedGraphsBenchmark.scala
@@ -7,16 +7,17 @@ package akka.stream
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+import org.openjdk.jmh.annotations._
+import org.openjdk.jmh.annotations.OperationsPerInvocation
+
import akka.NotUsed
import akka.actor.ActorSystem
import akka.stream.scaladsl._
import akka.stream.stage._
-import com.typesafe.config.ConfigFactory
-import org.openjdk.jmh.annotations.OperationsPerInvocation
-import org.openjdk.jmh.annotations._
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
object FusedGraphsBenchmark {
val ElementCount = 100 * 1000
@@ -96,7 +97,7 @@ class IdentityStage extends GraphStage[FlowShape[MutableElement, MutableElement]
class FusedGraphsBenchmark {
import FusedGraphsBenchmark._
- implicit val system = ActorSystem(
+ implicit val system: ActorSystem = ActorSystem(
"test",
ConfigFactory.parseString(s"""
akka.stream.materializer.sync-processing-limit = ${Int.MaxValue}
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/GraphBuilderBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/GraphBuilderBenchmark.scala
index 8bd377a2a2..b32f3f0893 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/GraphBuilderBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/GraphBuilderBenchmark.scala
@@ -6,9 +6,10 @@ package akka.stream
import java.util.concurrent.TimeUnit
+import org.openjdk.jmh.annotations._
+
import akka.NotUsed
import akka.stream.scaladsl.RunnableGraph
-import org.openjdk.jmh.annotations._
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/InterpreterBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/InterpreterBenchmark.scala
index 6d0032fd18..00f4b08d7f 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/InterpreterBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/InterpreterBenchmark.scala
@@ -4,14 +4,15 @@
package akka.stream
-import akka.event._
-import akka.stream.impl.fusing.{ GraphInterpreterSpecKit, GraphStages }
-import akka.stream.impl.fusing.GraphStages
-import akka.stream.impl.fusing.GraphInterpreter.{ DownstreamBoundaryStageLogic, UpstreamBoundaryStageLogic }
-import akka.stream.stage._
+import java.util.concurrent.TimeUnit
+
import org.openjdk.jmh.annotations._
-import java.util.concurrent.TimeUnit
+import akka.event._
+import akka.stream.impl.fusing.{ GraphInterpreterSpecKit, GraphStages }
+import akka.stream.impl.fusing.GraphInterpreter.{ DownstreamBoundaryStageLogic, UpstreamBoundaryStageLogic }
+import akka.stream.impl.fusing.GraphStages
+import akka.stream.stage._
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/InvokeWithFeedbackBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/InvokeWithFeedbackBenchmark.scala
index 1cccae1476..2c40797683 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/InvokeWithFeedbackBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/InvokeWithFeedbackBenchmark.scala
@@ -6,18 +6,19 @@ package akka.stream
import java.util.concurrent.TimeUnit
-import akka.actor.ActorSystem
-import akka.stream.scaladsl._
-import org.openjdk.jmh.annotations._
-
import scala.concurrent._
import scala.concurrent.duration._
+import org.openjdk.jmh.annotations._
+
+import akka.actor.ActorSystem
+import akka.stream.scaladsl._
+
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class InvokeWithFeedbackBenchmark {
- implicit val system = ActorSystem("InvokeWithFeedbackBenchmark")
+ implicit val system: ActorSystem = ActorSystem("InvokeWithFeedbackBenchmark")
var sourceQueue: SourceQueueWithComplete[Int] = _
var sinkQueue: SinkQueueWithCancel[Int] = _
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/JsonFramingBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/JsonFramingBenchmark.scala
index f5b74125e5..6471172e99 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/JsonFramingBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/JsonFramingBenchmark.scala
@@ -6,9 +6,10 @@ package akka.stream
import java.util.concurrent.TimeUnit
+import org.openjdk.jmh.annotations._
+
import akka.stream.impl.JsonObjectParser
import akka.util.ByteString
-import org.openjdk.jmh.annotations._
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/MapAsyncBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/MapAsyncBenchmark.scala
index a77c8d25e5..2e7f97c96b 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/MapAsyncBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/MapAsyncBenchmark.scala
@@ -7,18 +7,19 @@ package akka.stream
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+import org.openjdk.jmh.annotations._
+
import akka.NotUsed
import akka.actor.ActorSystem
import akka.remote.artery.BenchTestSource
import akka.remote.artery.LatchSink
import akka.stream.scaladsl._
import akka.stream.testkit.scaladsl.StreamTestKit
-import com.typesafe.config.ConfigFactory
-import org.openjdk.jmh.annotations._
-
-import scala.concurrent.duration._
-import scala.concurrent.Await
-import scala.concurrent.Future
object MapAsyncBenchmark {
final val OperationsPerInvocation = 100000
@@ -39,7 +40,7 @@ class MapAsyncBenchmark {
}
""")
- implicit val system = ActorSystem("MapAsyncBenchmark", config)
+ implicit val system: ActorSystem = ActorSystem("MapAsyncBenchmark", config)
import system.dispatcher
var testSource: Source[java.lang.Integer, NotUsed] = _
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/MaterializationBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/MaterializationBenchmark.scala
index bdbd7bcadc..b758822f2b 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/MaterializationBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/MaterializationBenchmark.scala
@@ -6,15 +6,16 @@ package akka.stream
import java.util.concurrent.TimeUnit
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
+import org.openjdk.jmh.annotations._
+
import akka.Done
import akka.NotUsed
import akka.actor.ActorSystem
import akka.stream.scaladsl._
-import org.openjdk.jmh.annotations._
-
-import scala.concurrent.Await
-import scala.concurrent.Future
-import scala.concurrent.duration._
object MaterializationBenchmark {
@@ -96,7 +97,7 @@ class MaterializationBenchmark {
import MaterializationBenchmark._
- implicit val system = ActorSystem("MaterializationBenchmark")
+ implicit val system: ActorSystem = ActorSystem("MaterializationBenchmark")
var flowWithMap: RunnableGraph[NotUsed] = _
var graphWithJunctionsGradual: RunnableGraph[NotUsed] = _
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/PartitionHubBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/PartitionHubBenchmark.scala
index 0086667d7e..d5c2ba9cf1 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/PartitionHubBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/PartitionHubBenchmark.scala
@@ -7,19 +7,20 @@ package akka.stream
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+import org.openjdk.jmh.annotations._
+
import akka.NotUsed
import akka.actor.ActorSystem
import akka.remote.artery.BenchTestSource
import akka.remote.artery.FixedSizePartitionHub
import akka.remote.artery.LatchSink
-import akka.stream.scaladsl.PartitionHub
import akka.stream.scaladsl._
+import akka.stream.scaladsl.PartitionHub
import akka.stream.testkit.scaladsl.StreamTestKit
-import com.typesafe.config.ConfigFactory
-import org.openjdk.jmh.annotations._
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
object PartitionHubBenchmark {
final val OperationsPerInvocation = 100000
@@ -40,7 +41,7 @@ class PartitionHubBenchmark {
}
""")
- implicit val system = ActorSystem("PartitionHubBenchmark", config)
+ implicit val system: ActorSystem = ActorSystem("PartitionHubBenchmark", config)
@Param(Array("2", "5", "10", "20", "30"))
var NumberOfStreams = 0
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/SourceRefBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/SourceRefBenchmark.scala
index 043b9bffcb..498bb95ba3 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/SourceRefBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/SourceRefBenchmark.scala
@@ -7,16 +7,17 @@ package akka.stream
import java.util.concurrent.Semaphore
import java.util.concurrent.TimeUnit
-import akka.actor.ActorSystem
-import akka.remote.artery.BenchTestSource
-import akka.stream.scaladsl._
-import com.typesafe.config.ConfigFactory
-import org.openjdk.jmh.annotations._
-
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.Success
+import com.typesafe.config.ConfigFactory
+import org.openjdk.jmh.annotations._
+
+import akka.actor.ActorSystem
+import akka.remote.artery.BenchTestSource
+import akka.stream.scaladsl._
+
/*
Just a brief reference run (3.1 GHz Intel Core i7, MacBook Pro late 2017):
[info] SourceRefBenchmark.source_ref_100k_elements thrpt 10 724650.336 ± 233643.256 ops/s
@@ -33,7 +34,7 @@ class SourceRefBenchmark {
loglevel = "WARNING"
}""".stripMargin).withFallback(ConfigFactory.load())
- implicit val system = ActorSystem("test", config)
+ implicit val system: ActorSystem = ActorSystem("test", config)
final val successMarker = Success(1)
final val successFailure = Success(new Exception)
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/impl/OutputStreamSourceStageBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/impl/OutputStreamSourceStageBenchmark.scala
index 7f7bf45bdc..62648430be 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/impl/OutputStreamSourceStageBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/impl/OutputStreamSourceStageBenchmark.scala
@@ -6,15 +6,16 @@ package akka.stream.impl
import java.util.concurrent.TimeUnit
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import org.openjdk.jmh.annotations._
+import org.openjdk.jmh.annotations.TearDown
+
import akka.actor.ActorSystem
import akka.stream.scaladsl.Keep
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.StreamConverters
-import org.openjdk.jmh.annotations.TearDown
-import org.openjdk.jmh.annotations._
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
object OutputStreamSourceStageBenchmark {
final val WritesPerBench = 10000
@@ -24,7 +25,7 @@ object OutputStreamSourceStageBenchmark {
@BenchmarkMode(Array(Mode.Throughput))
class OutputStreamSourceStageBenchmark {
import OutputStreamSourceStageBenchmark.WritesPerBench
- implicit val system = ActorSystem("OutputStreamSourceStageBenchmark")
+ implicit val system: ActorSystem = ActorSystem("OutputStreamSourceStageBenchmark")
private val bytes: Array[Byte] = Array.emptyByteArray
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesBenchmark.scala
index 6355866a9b..623401ab70 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesBenchmark.scala
@@ -8,19 +8,20 @@ import java.nio.file.Files
import java.nio.file.Path
import java.util.concurrent.TimeUnit
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.Promise
+import scala.concurrent.duration._
+
+import org.openjdk.jmh.annotations._
+
+import akka.Done
+import akka.NotUsed
import akka.actor.ActorSystem
import akka.stream.Attributes
import akka.stream.IOResult
import akka.stream.scaladsl._
import akka.util.ByteString
-import akka.Done
-import akka.NotUsed
-import org.openjdk.jmh.annotations._
-
-import scala.concurrent.duration._
-import scala.concurrent.Await
-import scala.concurrent.Future
-import scala.concurrent.Promise
/**
* Benchmark (bufSize) Mode Cnt Score Error Units
@@ -31,7 +32,7 @@ import scala.concurrent.Promise
@BenchmarkMode(Array(Mode.AverageTime))
class FileSourcesBenchmark {
- implicit val system = ActorSystem("file-sources-benchmark")
+ implicit val system: ActorSystem = ActorSystem("file-sources-benchmark")
val file: Path = {
val line = ByteString("x" * 2048 + "\n")
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesScaleBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesScaleBenchmark.scala
index 4959f22081..6fbd0229af 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesScaleBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesScaleBenchmark.scala
@@ -8,18 +8,19 @@ import java.nio.file.Files
import java.nio.file.Path
import java.util.concurrent.TimeUnit
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
+import org.openjdk.jmh.annotations._
+import org.openjdk.jmh.annotations.BenchmarkMode
+import org.openjdk.jmh.annotations.Scope
+import org.openjdk.jmh.annotations.State
+
import akka.actor.ActorSystem
import akka.stream.IOResult
import akka.stream.scaladsl._
import akka.util.ByteString
-import org.openjdk.jmh.annotations.BenchmarkMode
-import org.openjdk.jmh.annotations.Scope
-import org.openjdk.jmh.annotations.State
-import org.openjdk.jmh.annotations._
-
-import scala.concurrent.Await
-import scala.concurrent.Future
-import scala.concurrent.duration._
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.AverageTime))
@@ -34,7 +35,7 @@ class FileSourcesScaleBenchmark {
* FileSourcesScaleBenchmark.flatMapMerge 2048 avgt 10 1.587 ± 0.118 s/op
* FileSourcesScaleBenchmark.mapAsync 2048 avgt 10 0.899 ± 0.103 s/op
*/
- implicit val system = ActorSystem("file-sources-benchmark")
+ implicit val system: ActorSystem = ActorSystem("file-sources-benchmark")
val FILES_NUMBER = 40
val files: Seq[Path] = {
diff --git a/akka-bench-jmh/src/main/scala/akka/util/ByteString_copyToBuffer_Benchmark.scala b/akka-bench-jmh/src/main/scala/akka/util/ByteString_copyToBuffer_Benchmark.scala
index dcc2b3c8d4..2858de857b 100644
--- a/akka-bench-jmh/src/main/scala/akka/util/ByteString_copyToBuffer_Benchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/util/ByteString_copyToBuffer_Benchmark.scala
@@ -7,9 +7,10 @@ package akka.util
import java.nio.ByteBuffer
import java.util.concurrent.TimeUnit
-import akka.util.ByteString.{ ByteString1C, ByteStrings }
import org.openjdk.jmh.annotations._
+import akka.util.ByteString.{ ByteString1C, ByteStrings }
+
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MILLISECONDS)
class ByteString_copyToBuffer_Benchmark {
diff --git a/akka-bench-jmh/src/main/scala/akka/util/ByteString_decode_Benchmark.scala b/akka-bench-jmh/src/main/scala/akka/util/ByteString_decode_Benchmark.scala
index 56fa5d40ca..dde5ac4b86 100644
--- a/akka-bench-jmh/src/main/scala/akka/util/ByteString_decode_Benchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/util/ByteString_decode_Benchmark.scala
@@ -7,9 +7,10 @@ package akka.util
import java.nio.charset.Charset
import java.util.concurrent.TimeUnit
-import akka.util.ByteString.{ ByteString1C, ByteStrings }
import org.openjdk.jmh.annotations._
+import akka.util.ByteString.{ ByteString1C, ByteStrings }
+
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MILLISECONDS)
class ByteString_decode_Benchmark {
diff --git a/akka-bench-jmh/src/main/scala/akka/util/ByteString_dropRight_Benchmark.scala b/akka-bench-jmh/src/main/scala/akka/util/ByteString_dropRight_Benchmark.scala
index 1c779e3433..10773da8e9 100644
--- a/akka-bench-jmh/src/main/scala/akka/util/ByteString_dropRight_Benchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/util/ByteString_dropRight_Benchmark.scala
@@ -6,10 +6,11 @@ package akka.util
import java.util.concurrent.TimeUnit
-import akka.util.ByteString.{ ByteString1, ByteStrings }
+import scala.util.Random
+
import org.openjdk.jmh.annotations._
-import scala.util.Random
+import akka.util.ByteString.{ ByteString1, ByteStrings }
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MILLISECONDS)
diff --git a/akka-bench-jmh/src/main/scala/akka/util/ByteString_dropSliceTake_Benchmark.scala b/akka-bench-jmh/src/main/scala/akka/util/ByteString_dropSliceTake_Benchmark.scala
index 4275bee57b..ad5f9a472b 100644
--- a/akka-bench-jmh/src/main/scala/akka/util/ByteString_dropSliceTake_Benchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/util/ByteString_dropSliceTake_Benchmark.scala
@@ -6,9 +6,10 @@ package akka.util
import java.util.concurrent.TimeUnit
-import akka.util.ByteString.{ ByteString1C, ByteStrings }
import org.openjdk.jmh.annotations._
+import akka.util.ByteString.{ ByteString1C, ByteStrings }
+
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MILLISECONDS)
class ByteString_dropSliceTake_Benchmark {
diff --git a/akka-bench-jmh/src/main/scala/akka/util/ByteString_drop_Benchmark.scala b/akka-bench-jmh/src/main/scala/akka/util/ByteString_drop_Benchmark.scala
index 235777186d..61e996094e 100644
--- a/akka-bench-jmh/src/main/scala/akka/util/ByteString_drop_Benchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/util/ByteString_drop_Benchmark.scala
@@ -6,10 +6,11 @@ package akka.util
import java.util.concurrent.TimeUnit
-import akka.util.ByteString.{ ByteString1, ByteStrings }
+import scala.util.Random
+
import org.openjdk.jmh.annotations._
-import scala.util.Random
+import akka.util.ByteString.{ ByteString1, ByteStrings }
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MILLISECONDS)
diff --git a/akka-bench-jmh/src/main/scala/akka/util/ByteString_take_Benchmark.scala b/akka-bench-jmh/src/main/scala/akka/util/ByteString_take_Benchmark.scala
index 8a307104fa..38d59d7d19 100644
--- a/akka-bench-jmh/src/main/scala/akka/util/ByteString_take_Benchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/util/ByteString_take_Benchmark.scala
@@ -6,10 +6,11 @@ package akka.util
import java.util.concurrent.TimeUnit
-import akka.util.ByteString.{ ByteString1, ByteStrings }
+import scala.util.Random
+
import org.openjdk.jmh.annotations.{ Benchmark, Measurement, Scope, State }
-import scala.util.Random
+import akka.util.ByteString.{ ByteString1, ByteStrings }
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MILLISECONDS)
diff --git a/akka-bench-jmh/src/main/scala/akka/util/ImmutableIntMapBench.scala b/akka-bench-jmh/src/main/scala/akka/util/ImmutableIntMapBench.scala
index 2d14bc4757..5c16c4c30f 100644
--- a/akka-bench-jmh/src/main/scala/akka/util/ImmutableIntMapBench.scala
+++ b/akka-bench-jmh/src/main/scala/akka/util/ImmutableIntMapBench.scala
@@ -4,10 +4,12 @@
package akka.util
-import org.openjdk.jmh.annotations._
import java.util.concurrent.TimeUnit
+
import scala.annotation.tailrec
+import org.openjdk.jmh.annotations._
+
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(1)
diff --git a/akka-bench-jmh/src/main/scala/akka/util/LruBoundedCacheBench.scala b/akka-bench-jmh/src/main/scala/akka/util/LruBoundedCacheBench.scala
index c9529fe9ed..e7c238baa3 100644
--- a/akka-bench-jmh/src/main/scala/akka/util/LruBoundedCacheBench.scala
+++ b/akka-bench-jmh/src/main/scala/akka/util/LruBoundedCacheBench.scala
@@ -7,10 +7,11 @@ package akka.util
import java.util
import java.util.concurrent.TimeUnit
-import akka.remote.artery.LruBoundedCache
+import scala.util.Random
+
import org.openjdk.jmh.annotations.{ Param, _ }
-import scala.util.Random
+import akka.remote.artery.LruBoundedCache
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MICROSECONDS)
diff --git a/akka-cluster-metrics/src/main/java/akka/cluster/metrics/protobuf/msg/ClusterMetricsMessages.java b/akka-cluster-metrics/src/main/java/akka/cluster/metrics/protobuf/msg/ClusterMetricsMessages.java
index badaa04cb6..0dd86a8e15 100644
--- a/akka-cluster-metrics/src/main/java/akka/cluster/metrics/protobuf/msg/ClusterMetricsMessages.java
+++ b/akka-cluster-metrics/src/main/java/akka/cluster/metrics/protobuf/msg/ClusterMetricsMessages.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2019-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
diff --git a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsCollector.scala b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsCollector.scala
index feab28a21a..ed7a705edb 100644
--- a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsCollector.scala
+++ b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsCollector.scala
@@ -4,20 +4,22 @@
package akka.cluster.metrics
-import akka.actor.Actor
-import akka.actor.ActorLogging
-import akka.actor.Props
-import akka.actor.Address
-import akka.cluster.ClusterEvent
-import akka.cluster.Member
-import akka.cluster.Cluster
-import scala.collection.immutable
-import akka.cluster.MemberStatus
import java.util.concurrent.ThreadLocalRandom
-import akka.actor.DeadLetterSuppression
+
+import scala.collection.immutable
import com.github.ghik.silencer.silent
+import akka.actor.Actor
+import akka.actor.ActorLogging
+import akka.actor.Address
+import akka.actor.DeadLetterSuppression
+import akka.actor.Props
+import akka.cluster.Cluster
+import akka.cluster.ClusterEvent
+import akka.cluster.Member
+import akka.cluster.MemberStatus
+
/**
* Runtime collection management commands.
*/
@@ -50,8 +52,8 @@ case object CollectionStopMessage extends CollectionControlMessage {
*/
private[metrics] class ClusterMetricsSupervisor extends Actor with ActorLogging {
val metrics = ClusterMetricsExtension(context.system)
- import metrics.settings._
import context._
+ import metrics.settings._
override val supervisorStrategy = metrics.strategy
@@ -132,8 +134,8 @@ private[metrics] object ClusterMetricsCollector {
* Actor responsible for periodic data sampling in the node and publication to the cluster.
*/
private[metrics] class ClusterMetricsCollector extends Actor with ActorLogging {
- import ClusterMetricsCollector._
import ClusterEvent._
+ import ClusterMetricsCollector._
import Member.addressOrdering
import context.dispatcher
val cluster = Cluster(context.system)
diff --git a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsExtension.scala b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsExtension.scala
index 5776f63adb..62f96a8235 100644
--- a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsExtension.scala
+++ b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsExtension.scala
@@ -4,21 +4,22 @@
package akka.cluster.metrics
-import akka.actor.ExtendedActorSystem
-import akka.actor.Extension
-import akka.actor.SupervisorStrategy
-import akka.event.LoggingAdapter
-import akka.event.Logging
+import scala.collection.immutable
+
import com.typesafe.config.Config
-import scala.collection.immutable
-import akka.actor.Props
+import akka.actor.ActorRef
+import akka.actor.ActorSystem
+import akka.actor.ClassicActorSystemProvider
import akka.actor.Deploy
+import akka.actor.ExtendedActorSystem
+import akka.actor.Extension
import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
-import akka.actor.ActorSystem
-import akka.actor.ActorRef
-import akka.actor.ClassicActorSystemProvider
+import akka.actor.Props
+import akka.actor.SupervisorStrategy
+import akka.event.Logging
+import akka.event.LoggingAdapter
/**
* Cluster metrics extension.
diff --git a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsRouting.scala b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsRouting.scala
index ed4b9b5b8c..e539609b2c 100644
--- a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsRouting.scala
+++ b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsRouting.scala
@@ -5,13 +5,15 @@
package akka.cluster.metrics
import java.util.Arrays
+import java.util.concurrent.ThreadLocalRandom
import java.util.concurrent.atomic.AtomicReference
import scala.annotation.tailrec
import scala.collection.immutable
-import java.util.concurrent.ThreadLocalRandom
+import com.github.ghik.silencer.silent
import com.typesafe.config.Config
+
import akka.actor.Actor
import akka.actor.ActorSystem
import akka.actor.Address
@@ -20,11 +22,10 @@ import akka.actor.NoSerializationVerificationNeeded
import akka.actor.Props
import akka.actor.SupervisorStrategy
import akka.cluster.Cluster
+import akka.cluster.routing.ClusterRouterSettingsBase
import akka.dispatch.Dispatchers
import akka.japi.Util.immutableSeq
import akka.routing._
-import akka.cluster.routing.ClusterRouterSettingsBase
-import com.github.ghik.silencer.silent
/**
* Load balancing of messages to cluster nodes based on cluster metric data.
diff --git a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsSettings.scala b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsSettings.scala
index cb701183ee..5fac7d0969 100644
--- a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsSettings.scala
+++ b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsSettings.scala
@@ -4,11 +4,13 @@
package akka.cluster.metrics
-import com.typesafe.config.Config
-import scala.concurrent.duration.FiniteDuration
-import akka.util.Helpers.Requiring
-import akka.util.Helpers.ConfigOps
import scala.concurrent.duration.Duration
+import scala.concurrent.duration.FiniteDuration
+
+import com.typesafe.config.Config
+
+import akka.util.Helpers.ConfigOps
+import akka.util.Helpers.Requiring
/**
* Metrics extension settings. Documented in: `src/main/resources/reference.conf`.
diff --git a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsStrategy.scala b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsStrategy.scala
index f29859517b..cd5a108a8f 100644
--- a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsStrategy.scala
+++ b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsStrategy.scala
@@ -5,6 +5,7 @@
package akka.cluster.metrics
import com.typesafe.config.Config
+
import akka.actor.OneForOneStrategy
import akka.util.Helpers.ConfigOps
diff --git a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/Metric.scala b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/Metric.scala
index 75d0c24dec..0e49ef9861 100644
--- a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/Metric.scala
+++ b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/Metric.scala
@@ -4,13 +4,14 @@
package akka.cluster.metrics
-import akka.actor.Address
-import scala.util.Success
import scala.util.Failure
+import scala.util.Success
import scala.util.Try
import com.github.ghik.silencer.silent
+import akka.actor.Address
+
/**
* Metrics key/value.
*
diff --git a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/MetricsCollector.scala b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/MetricsCollector.scala
index 06aad30061..387116dc9c 100644
--- a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/MetricsCollector.scala
+++ b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/MetricsCollector.scala
@@ -4,20 +4,22 @@
package akka.cluster.metrics
-import akka.actor.ActorSystem
-import akka.actor.ExtendedActorSystem
-import akka.event.Logging
-import akka.ConfigurationException
-import akka.actor.Address
-import java.lang.management.MemoryMXBean
-import java.lang.management.ManagementFactory
-import java.lang.management.OperatingSystemMXBean
-import java.lang.management.MemoryUsage
-import java.lang.System.{ currentTimeMillis => newTimestamp }
-import akka.cluster.Cluster
import java.io.Closeable
+import java.lang.System.{ currentTimeMillis => newTimestamp }
+import java.lang.management.ManagementFactory
+import java.lang.management.MemoryMXBean
+import java.lang.management.MemoryUsage
+import java.lang.management.OperatingSystemMXBean
+
import org.hyperic.sigar.SigarProxy
+import akka.ConfigurationException
+import akka.actor.ActorSystem
+import akka.actor.Address
+import akka.actor.ExtendedActorSystem
+import akka.cluster.Cluster
+import akka.event.Logging
+
/**
* Metrics sampler.
*
@@ -108,7 +110,7 @@ class JmxMetricsCollector(address: Address, decayFactor: Double) extends Metrics
* Samples and collects new data points.
* Creates a new instance each time.
*/
- def sample(): NodeMetrics = NodeMetrics(address, newTimestamp, metrics)
+ def sample(): NodeMetrics = NodeMetrics(address, newTimestamp, metrics())
/**
* Generate metrics set.
@@ -209,7 +211,7 @@ class SigarMetricsCollector(address: Address, decayFactor: Double, sigar: SigarP
override def metrics(): Set[Metric] = {
// Must obtain cpuPerc in one shot. See https://github.com/akka/akka/issues/16121
val cpuPerc = sigar.getCpuPerc
- super.metrics.union(Set(cpuCombined(cpuPerc), cpuStolen(cpuPerc)).flatten)
+ super.metrics().union(Set(cpuCombined(cpuPerc), cpuStolen(cpuPerc)).flatten)
}
/**
diff --git a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/Provision.scala b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/Provision.scala
index 7492b7995c..ef12ed69ea 100644
--- a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/Provision.scala
+++ b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/Provision.scala
@@ -5,13 +5,15 @@
package akka.cluster.metrics
import java.io.File
+
+import scala.language.postfixOps
+import scala.util.Failure
+import scala.util.Success
+import scala.util.Try
+
import kamon.sigar.SigarProvisioner
import org.hyperic.sigar.Sigar
import org.hyperic.sigar.SigarProxy
-import scala.language.postfixOps
-import scala.util.Success
-import scala.util.Failure
-import scala.util.Try
/**
* Provide sigar instance as `SigarProxy`.
diff --git a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/protobuf/MessageSerializer.scala b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/protobuf/MessageSerializer.scala
index 698475b638..00859d50b9 100644
--- a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/protobuf/MessageSerializer.scala
+++ b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/protobuf/MessageSerializer.scala
@@ -4,24 +4,23 @@
package akka.cluster.metrics.protobuf
-import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
-import java.util.zip.{ GZIPInputStream, GZIPOutputStream }
import java.{ lang => jl }
-
-import akka.actor.{ Address, ExtendedActorSystem }
-import akka.cluster.metrics.protobuf.msg.{ ClusterMetricsMessages => cm }
-import akka.cluster.metrics._
-import akka.serialization.{ BaseSerializer, SerializationExtension, SerializerWithStringManifest, Serializers }
-import akka.util.ClassLoaderObjectInputStream
-import akka.protobufv3.internal.{ ByteString, MessageLite }
-import akka.util.ccompat._
+import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
+import java.io.NotSerializableException
+import java.util.zip.{ GZIPInputStream, GZIPOutputStream }
import scala.annotation.tailrec
import scala.collection.immutable
-import akka.util.ccompat.JavaConverters._
-import java.io.NotSerializableException
+import akka.actor.{ Address, ExtendedActorSystem }
+import akka.cluster.metrics._
+import akka.cluster.metrics.protobuf.msg.{ ClusterMetricsMessages => cm }
import akka.dispatch.Dispatchers
+import akka.protobufv3.internal.{ ByteString, MessageLite }
+import akka.serialization.{ BaseSerializer, SerializationExtension, SerializerWithStringManifest, Serializers }
+import akka.util.ClassLoaderObjectInputStream
+import akka.util.ccompat._
+import akka.util.ccompat.JavaConverters._
/**
* Protobuf serializer for [[akka.cluster.metrics.ClusterMetricsMessage]] types.
diff --git a/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/ClusterMetricsExtensionSpec.scala b/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/ClusterMetricsExtensionSpec.scala
index 869cec5193..8533ed2f25 100644
--- a/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/ClusterMetricsExtensionSpec.scala
+++ b/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/ClusterMetricsExtensionSpec.scala
@@ -4,13 +4,15 @@
package akka.cluster.metrics
-import scala.language.postfixOps
import scala.concurrent.duration._
+import scala.language.postfixOps
+
import com.typesafe.config.ConfigFactory
+
+import akka.cluster.MemberStatus
+import akka.cluster.MultiNodeClusterSpec
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
-import akka.cluster.MultiNodeClusterSpec
-import akka.cluster.MemberStatus
trait ClusterMetricsCommonConfig extends MultiNodeConfig {
import ConfigFactory._
@@ -111,7 +113,7 @@ abstract class ClusterMetricsEnabledSpec
//awaitAssert(clusterView.clusterMetrics.size should ===(roles.size))
awaitAssert(metricsView.clusterMetrics.size should ===(roles.size))
val collector = MetricsCollector(cluster.system)
- collector.sample.metrics.size should be > (3)
+ collector.sample().metrics.size should be > (3)
enterBarrier("after")
}
"reflect the correct number of node metrics in cluster view" in within(30 seconds) {
@@ -132,7 +134,7 @@ abstract class ClusterMetricsEnabledSpec
class ClusterMetricsDisabledMultiJvmNode1 extends ClusterMetricsDisabledSpec
class ClusterMetricsDisabledMultiJvmNode2 extends ClusterMetricsDisabledSpec
-class ClusterMetricsDisabledMultiJvmNodv3 extends ClusterMetricsDisabledSpec
+class ClusterMetricsDisabledMultiJvmNode3 extends ClusterMetricsDisabledSpec
class ClusterMetricsDisabledMultiJvmNode4 extends ClusterMetricsDisabledSpec
class ClusterMetricsDisabledMultiJvmNode5 extends ClusterMetricsDisabledSpec
@@ -150,7 +152,7 @@ abstract class ClusterMetricsDisabledSpec
//clusterView.clusterMetrics.size should ===(0)
metricsView.clusterMetrics.size should ===(0)
ClusterMetricsExtension(system).subscribe(testActor)
- expectNoMessage
+ expectNoMessage()
// TODO ensure same contract
//clusterView.clusterMetrics.size should ===(0)
metricsView.clusterMetrics.size should ===(0)
diff --git a/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/ClusterMetricsRoutingSpec.scala b/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/ClusterMetricsRoutingSpec.scala
index ee8615312e..48dcfd13ed 100644
--- a/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/ClusterMetricsRoutingSpec.scala
+++ b/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/ClusterMetricsRoutingSpec.scala
@@ -4,7 +4,6 @@
package akka.cluster.metrics
-import language.postfixOps
import java.lang.management.ManagementFactory
import scala.concurrent.Await
@@ -12,19 +11,21 @@ import scala.concurrent.duration._
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
import akka.actor._
import akka.cluster.Cluster
import akka.cluster.MultiNodeClusterSpec
-import akka.pattern.ask
-import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
-import akka.routing.GetRoutees
-import akka.routing.FromConfig
-import akka.testkit.{ DefaultTimeout, ImplicitSender, LongRunningTest }
-import akka.routing.ActorRefRoutee
-import akka.routing.Routees
import akka.cluster.routing.ClusterRouterPool
import akka.cluster.routing.ClusterRouterPoolSettings
+import akka.pattern.ask
+import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
+import akka.routing.ActorRefRoutee
+import akka.routing.FromConfig
+import akka.routing.GetRoutees
+import akka.routing.Routees
import akka.serialization.jackson.CborSerializable
+import akka.testkit.{ DefaultTimeout, ImplicitSender, LongRunningTest }
import akka.util.unused
object AdaptiveLoadBalancingRouterConfig extends MultiNodeConfig {
@@ -150,7 +151,7 @@ abstract class AdaptiveLoadBalancingRouterSpec
ClusterRouterPool(
local = AdaptiveLoadBalancingPool(HeapMetricsSelector),
settings = ClusterRouterPoolSettings(totalInstances = 10, maxInstancesPerNode = 1, allowLocalRoutees = true))
- .props(Props[Echo]),
+ .props(Props[Echo]()),
name)
// it may take some time until router receives cluster member events
awaitAssert { currentRoutees(router).size should ===(roles.size) }
@@ -201,7 +202,7 @@ abstract class AdaptiveLoadBalancingRouterSpec
runOn(node2) {
within(20.seconds) {
- system.actorOf(Props[Memory], "memory") ! AllocateMemory
+ system.actorOf(Props[Memory](), "memory") ! AllocateMemory
expectMsg("done")
}
}
@@ -230,7 +231,7 @@ abstract class AdaptiveLoadBalancingRouterSpec
"create routees from configuration" taggedAs LongRunningTest in {
runOn(node1) {
- val router3 = system.actorOf(FromConfig.props(Props[Memory]), "router3")
+ val router3 = system.actorOf(FromConfig.props(Props[Memory]()), "router3")
// it may take some time until router receives cluster member events
awaitAssert { currentRoutees(router3).size should ===(9) }
val routees = currentRoutees(router3)
@@ -241,7 +242,7 @@ abstract class AdaptiveLoadBalancingRouterSpec
"create routees from cluster.enabled configuration" taggedAs LongRunningTest in {
runOn(node1) {
- val router4 = system.actorOf(FromConfig.props(Props[Memory]), "router4")
+ val router4 = system.actorOf(FromConfig.props(Props[Memory]()), "router4")
// it may take some time until router receives cluster member events
awaitAssert { currentRoutees(router4).size should ===(6) }
val routees = currentRoutees(router4)
diff --git a/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/sample/StatsSampleSpec.scala b/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/sample/StatsSampleSpec.scala
index cb8dda8266..6cb8ce15d7 100644
--- a/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/sample/StatsSampleSpec.scala
+++ b/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/sample/StatsSampleSpec.scala
@@ -107,8 +107,8 @@ abstract class StatsSampleSpec
Cluster(system).join(firstAddress)
//#join
- system.actorOf(Props[StatsWorker], "statsWorker")
- system.actorOf(Props[StatsService], "statsService")
+ system.actorOf(Props[StatsWorker](), "statsWorker")
+ system.actorOf(Props[StatsService](), "statsService")
receiveN(3).collect { case MemberUp(m) => m.address }.toSet should be(
Set(firstAddress, secondAddress, thirdAddress))
diff --git a/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/sample/StatsService.scala b/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/sample/StatsService.scala
index f950194c02..34cdd0d47b 100644
--- a/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/sample/StatsService.scala
+++ b/akka-cluster-metrics/src/multi-jvm/scala/akka/cluster/metrics/sample/StatsService.scala
@@ -4,18 +4,18 @@
package akka.cluster.metrics.sample
+import scala.concurrent.duration._
+
import akka.actor.{ Actor, ActorRef, Props, ReceiveTimeout }
import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope
import akka.routing.FromConfig
-import scala.concurrent.duration._
-
//#service
class StatsService extends Actor {
// This router is used both with lookup and deploy of routees. If you
// have a router with only lookup of routees you can use Props.empty
// instead of Props[StatsWorker.class].
- val workerRouter = context.actorOf(FromConfig.props(Props[StatsWorker]), name = "workerRouter")
+ val workerRouter = context.actorOf(FromConfig.props(Props[StatsWorker]()), name = "workerRouter")
def receive = {
case StatsJob(text) if text != "" =>
@@ -76,7 +76,7 @@ abstract class StatsService3 extends Actor {
ClusterRouterPool(
ConsistentHashingPool(0),
ClusterRouterPoolSettings(totalInstances = 100, maxInstancesPerNode = 3, allowLocalRoutees = false))
- .props(Props[StatsWorker]),
+ .props(Props[StatsWorker]()),
name = "workerRouter3")
//#router-deploy-in-code
}
diff --git a/akka-cluster-metrics/src/test/resources/logback-test.xml b/akka-cluster-metrics/src/test/resources/logback-test.xml
index 1e665afb96..201c1c058d 100644
--- a/akka-cluster-metrics/src/test/resources/logback-test.xml
+++ b/akka-cluster-metrics/src/test/resources/logback-test.xml
@@ -3,7 +3,7 @@
-
+
diff --git a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/ClusterMetricsExtensionSpec.scala b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/ClusterMetricsExtensionSpec.scala
index 46b0061880..7a4266919d 100644
--- a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/ClusterMetricsExtensionSpec.scala
+++ b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/ClusterMetricsExtensionSpec.scala
@@ -4,12 +4,13 @@
package akka.cluster.metrics
-import scala.language.postfixOps
import scala.concurrent.duration._
+import scala.language.postfixOps
+
import akka.actor._
-import akka.testkit._
-import akka.cluster.metrics.StandardMetrics._
import akka.cluster.Cluster
+import akka.cluster.metrics.StandardMetrics._
+import akka.testkit._
class ClusterMetricsExtensionSpec
extends AkkaSpec(MetricsConfig.clusterSigarMock)
diff --git a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/ClusterMetricsRoutingSpec.scala b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/ClusterMetricsRoutingSpec.scala
index fa990f9a5a..6213af6bf1 100644
--- a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/ClusterMetricsRoutingSpec.scala
+++ b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/ClusterMetricsRoutingSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster.metrics
-import akka.actor.Address
-import akka.cluster.metrics.StandardMetrics._
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.Address
+import akka.cluster.metrics.StandardMetrics._
+
class MetricsSelectorSpec extends AnyWordSpec with Matchers {
val factor = 0.3 // TODO read from reference.conf
diff --git a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/ClusterMetricsSettingsSpec.scala b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/ClusterMetricsSettingsSpec.scala
index 438c9f4de4..c94dafd6a1 100644
--- a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/ClusterMetricsSettingsSpec.scala
+++ b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/ClusterMetricsSettingsSpec.scala
@@ -4,11 +4,13 @@
package akka.cluster.metrics
-import language.postfixOps
-import akka.testkit.AkkaSpec
-import akka.dispatch.Dispatchers
import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
+import akka.dispatch.Dispatchers
+import akka.testkit.AkkaSpec
class ClusterMetricsSettingsSpec extends AkkaSpec {
diff --git a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/EWMASpec.scala b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/EWMASpec.scala
index 8f6c743e04..fc06d6c00d 100644
--- a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/EWMASpec.scala
+++ b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/EWMASpec.scala
@@ -4,12 +4,14 @@
package akka.cluster.metrics
-import scala.concurrent.duration._
-import akka.testkit.{ AkkaSpec, LongRunningTest }
import java.util.concurrent.ThreadLocalRandom
+import scala.concurrent.duration._
+
import com.github.ghik.silencer.silent
+import akka.testkit.{ AkkaSpec, LongRunningTest }
+
@silent
class EWMASpec extends AkkaSpec(MetricsConfig.defaultEnabled) with MetricsCollectorFactory {
@@ -82,7 +84,7 @@ class EWMASpec extends AkkaSpec(MetricsConfig.defaultEnabled) with MetricsCollec
// wait a while between each message to give the metrics a chance to change
Thread.sleep(100)
usedMemory = usedMemory ++ Array.fill(1024)(ThreadLocalRandom.current.nextInt(127).toByte)
- val changes = collector.sample.metrics.flatMap { latest =>
+ val changes = collector.sample().metrics.flatMap { latest =>
streamingDataSet.get(latest.name) match {
case None => Some(latest)
case Some(previous) =>
diff --git a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/MetricSpec.scala b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/MetricSpec.scala
index a11883a092..78144cf7c1 100644
--- a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/MetricSpec.scala
+++ b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/MetricSpec.scala
@@ -4,18 +4,19 @@
package akka.cluster.metrics
-import akka.cluster.metrics.StandardMetrics._
+import java.lang.System.{ currentTimeMillis => newTimestamp }
import scala.util.Failure
-import akka.actor.Address
-import akka.testkit.AkkaSpec
-import akka.testkit.ImplicitSender
-import java.lang.System.{ currentTimeMillis => newTimestamp }
import com.github.ghik.silencer.silent
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.Address
+import akka.cluster.metrics.StandardMetrics._
+import akka.testkit.AkkaSpec
+import akka.testkit.ImplicitSender
+
@silent
class MetricNumericConverterSpec extends AnyWordSpec with Matchers with MetricNumericConverter {
@@ -146,13 +147,13 @@ class MetricsGossipSpec
*/
def newSample(previousSample: Set[Metric]): Set[Metric] = {
// Metric.equals is based on name equality
- collector.sample.metrics.filter(previousSample.contains) ++ previousSample
+ collector.sample().metrics.filter(previousSample.contains) ++ previousSample
}
"A MetricsGossip" must {
"add new NodeMetrics" in {
- val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
- val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
+ val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample().metrics)
+ val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample().metrics)
m1.metrics.size should be > 3
m2.metrics.size should be > 3
@@ -168,8 +169,8 @@ class MetricsGossipSpec
}
"merge peer metrics" in {
- val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
- val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
+ val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample().metrics)
+ val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample().metrics)
val g1 = MetricsGossip.empty :+ m1 :+ m2
g1.nodes.size should ===(2)
@@ -183,9 +184,9 @@ class MetricsGossipSpec
}
"merge an existing metric set for a node and update node ring" in {
- val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
- val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
- val m3 = NodeMetrics(Address("akka", "sys", "a", 2556), newTimestamp, collector.sample.metrics)
+ val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample().metrics)
+ val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample().metrics)
+ val m3 = NodeMetrics(Address("akka", "sys", "a", 2556), newTimestamp, collector.sample().metrics)
val m2Updated = m2.copy(metrics = newSample(m2.metrics), timestamp = m2.timestamp + 1000)
val g1 = MetricsGossip.empty :+ m1 :+ m2
@@ -204,14 +205,14 @@ class MetricsGossipSpec
}
"get the current NodeMetrics if it exists in the local nodes" in {
- val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
+ val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample().metrics)
val g1 = MetricsGossip.empty :+ m1
g1.nodeMetricsFor(m1.address).map(_.metrics) should ===(Some(m1.metrics))
}
"remove a node if it is no longer Up" in {
- val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
- val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
+ val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample().metrics)
+ val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample().metrics)
val g1 = MetricsGossip.empty :+ m1 :+ m2
g1.nodes.size should ===(2)
@@ -223,8 +224,8 @@ class MetricsGossipSpec
}
"filter nodes" in {
- val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
- val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
+ val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample().metrics)
+ val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample().metrics)
val g1 = MetricsGossip.empty :+ m1 :+ m2
g1.nodes.size should ===(2)
@@ -243,16 +244,20 @@ class MetricValuesSpec extends AkkaSpec(MetricsConfig.defaultEnabled) with Metri
val collector = createMetricsCollector
- val node1 = NodeMetrics(Address("akka", "sys", "a", 2554), 1, collector.sample.metrics)
- val node2 = NodeMetrics(Address("akka", "sys", "a", 2555), 1, collector.sample.metrics)
+ val node1 = NodeMetrics(Address("akka", "sys", "a", 2554), 1, collector.sample().metrics)
+ val node2 = NodeMetrics(Address("akka", "sys", "a", 2555), 1, collector.sample().metrics)
val nodes: Seq[NodeMetrics] = {
(1 to 100).foldLeft(List(node1, node2)) { (nodes, _) =>
nodes.map { n =>
- n.copy(metrics = collector.sample.metrics.flatMap(latest =>
- n.metrics.collect {
- case streaming if latest.sameAs(streaming) => streaming :+ latest
- }))
+ n.copy(
+ metrics = collector
+ .sample()
+ .metrics
+ .flatMap(latest =>
+ n.metrics.collect {
+ case streaming if latest.sameAs(streaming) => streaming :+ latest
+ }))
}
}
}
diff --git a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/MetricsCollectorSpec.scala b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/MetricsCollectorSpec.scala
index 161a103188..710482a4cd 100644
--- a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/MetricsCollectorSpec.scala
+++ b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/MetricsCollectorSpec.scala
@@ -4,13 +4,15 @@
package akka.cluster.metrics
-import scala.language.postfixOps
import scala.concurrent.duration._
+import scala.language.postfixOps
import scala.util.Try
-import akka.testkit._
-import akka.cluster.metrics.StandardMetrics._
+
import com.github.ghik.silencer.silent
+import akka.cluster.metrics.StandardMetrics._
+import akka.testkit._
+
@silent
class MetricsCollectorSpec
extends AkkaSpec(MetricsConfig.defaultEnabled)
@@ -23,8 +25,8 @@ class MetricsCollectorSpec
"merge 2 metrics that are tracking the same metric" in {
for (_ <- 1 to 20) {
- val sample1 = collector.sample.metrics
- val sample2 = collector.sample.metrics
+ val sample1 = collector.sample().metrics
+ val sample2 = collector.sample().metrics
sample2.flatMap(latest =>
sample1.collect {
case peer if latest.sameAs(peer) =>
@@ -34,8 +36,8 @@ class MetricsCollectorSpec
m
})
- val sample3 = collector.sample.metrics
- val sample4 = collector.sample.metrics
+ val sample3 = collector.sample().metrics
+ val sample4 = collector.sample().metrics
sample4.flatMap(latest =>
sample3.collect {
case peer if latest.sameAs(peer) =>
@@ -55,7 +57,7 @@ class MetricsCollectorSpec
}
"collect accurate metrics for a node" in {
- val sample = collector.sample
+ val sample = collector.sample()
val metrics = sample.metrics.collect { case m => (m.name, m.value) }
val used = metrics.collectFirst { case (HeapMemoryUsed, b) => b }
val committed = metrics.collectFirst { case (HeapMemoryCommitted, b) => b }
@@ -93,7 +95,7 @@ class MetricsCollectorSpec
"collect 50 node metrics samples in an acceptable duration" taggedAs LongRunningTest in within(10 seconds) {
(1 to 50).foreach { _ =>
- val sample = collector.sample
+ val sample = collector.sample()
sample.metrics.size should be >= 3
Thread.sleep(100)
}
diff --git a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/TestUtil.scala b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/TestUtil.scala
index 2beea3e387..0421273ea9 100644
--- a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/TestUtil.scala
+++ b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/TestUtil.scala
@@ -4,27 +4,28 @@
package akka.cluster.metrics
-import scala.language.postfixOps
+import java.io.Closeable
import java.util.logging.LogManager
-import org.slf4j.bridge.SLF4JBridgeHandler
-import akka.testkit.AkkaSpec
-import akka.actor.ExtendedActorSystem
-import akka.actor.Address
-import java.io.Closeable
+import scala.language.postfixOps
-import akka.actor.ActorRef
-import akka.actor.Props
-import akka.actor.Actor
-import akka.dispatch.RequiresMessageQueue
-import akka.actor.Deploy
-import akka.dispatch.UnboundedMessageQueueSemantics
-import akka.actor.PoisonPill
-import akka.actor.ActorLogging
import org.scalatestplus.mockito.MockitoSugar
+import org.slf4j.bridge.SLF4JBridgeHandler
+
+import akka.actor.Actor
+import akka.actor.ActorLogging
+import akka.actor.ActorRef
import akka.actor.ActorSystem
+import akka.actor.Address
+import akka.actor.Deploy
+import akka.actor.ExtendedActorSystem
+import akka.actor.PoisonPill
+import akka.actor.Props
import akka.dispatch.Dispatchers
+import akka.dispatch.RequiresMessageQueue
+import akka.dispatch.UnboundedMessageQueueSemantics
import akka.remote.RARP
+import akka.testkit.AkkaSpec
/**
* Redirect different logging sources to SLF4J.
diff --git a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/WeightedRouteesSpec.scala b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/WeightedRouteesSpec.scala
index 3148a26091..656a063aa9 100644
--- a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/WeightedRouteesSpec.scala
+++ b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/WeightedRouteesSpec.scala
@@ -4,14 +4,16 @@
package akka.cluster.metrics
+import scala.Vector
+
import com.typesafe.config.ConfigFactory
+
import akka.actor.Address
import akka.actor.RootActorPath
import akka.remote.RARP
-import akka.testkit.AkkaSpec
-import akka.routing.ActorSelectionRoutee
import akka.routing.ActorRefRoutee
-import scala.Vector
+import akka.routing.ActorSelectionRoutee
+import akka.testkit.AkkaSpec
class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
akka.actor.provider = "cluster"
diff --git a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/protobuf/MessageSerializerSpec.scala b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/protobuf/MessageSerializerSpec.scala
index b46f279001..c682ab3bb8 100644
--- a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/protobuf/MessageSerializerSpec.scala
+++ b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/protobuf/MessageSerializerSpec.scala
@@ -5,10 +5,10 @@
package akka.cluster.metrics.protobuf
import akka.actor.{ Address, ExtendedActorSystem }
-import akka.testkit.AkkaSpec
import akka.cluster.MemberStatus
-import akka.cluster.metrics._
import akka.cluster.TestMember
+import akka.cluster.metrics._
+import akka.testkit.AkkaSpec
class MessageSerializerSpec extends AkkaSpec("""
akka.actor.provider = cluster
diff --git a/akka-cluster-sharding-typed/src/main/java/akka/cluster/sharding/typed/internal/protobuf/ShardingMessages.java b/akka-cluster-sharding-typed/src/main/java/akka/cluster/sharding/typed/internal/protobuf/ShardingMessages.java
index 48833056e2..85a2305110 100644
--- a/akka-cluster-sharding-typed/src/main/java/akka/cluster/sharding/typed/internal/protobuf/ShardingMessages.java
+++ b/akka-cluster-sharding-typed/src/main/java/akka/cluster/sharding/typed/internal/protobuf/ShardingMessages.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2019-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ClusterShardingQuery.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ClusterShardingQuery.scala
index 29a016eaff..2b5037d868 100644
--- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ClusterShardingQuery.scala
+++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ClusterShardingQuery.scala
@@ -4,9 +4,13 @@
package akka.cluster.sharding.typed
+import scala.concurrent.duration.FiniteDuration
+
import akka.actor.typed.ActorRef
+import akka.cluster.sharding.ShardRegion.ClusterShardingStats
import akka.cluster.sharding.ShardRegion.CurrentShardRegionState
import akka.cluster.sharding.typed.scaladsl.EntityTypeKey
+import akka.util.JavaDurationConverters
/**
* Protocol for querying sharding state e.g. A ShardRegion's state
@@ -16,6 +20,11 @@ sealed trait ClusterShardingQuery
/**
* Query the ShardRegion state for the given entity type key. This will get the state of the
* local ShardRegion's state.
+ *
+ * Intended for testing purpose to see when cluster sharding is "ready" or to monitor
+ * the state of the shard regions.
+ *
+ * For the statistics for the entire cluster, see [[GetClusterShardingStats]].
*/
final case class GetShardRegionState(entityTypeKey: EntityTypeKey[_], replyTo: ActorRef[CurrentShardRegionState])
extends ClusterShardingQuery {
@@ -30,4 +39,33 @@ final case class GetShardRegionState(entityTypeKey: EntityTypeKey[_], replyTo: A
this(entityTypeKey.asScala, replyTo)
}
-// TODO - GetClusterShardingStats
+/**
+ * Query the statistics about the currently running sharded entities in the
+ * entire cluster. If the given `timeout` is reached without answers from all
+ * shard regions the reply will contain an empty map of regions.
+ *
+ * Intended for testing purpose to see when cluster sharding is "ready" or to monitor
+ * the state of the shard regions.
+ *
+ * @param timeout the timeout applied to querying all alive regions
+ * @param replyTo the actor to send the result to
+ */
+final case class GetClusterShardingStats(
+ entityTypeKey: EntityTypeKey[_],
+ timeout: FiniteDuration,
+ replyTo: ActorRef[ClusterShardingStats])
+ extends ClusterShardingQuery {
+
+ /**
+ * Java API
+ *
+ * Query the statistics about the currently running sharded entities in the
+ * entire cluster. If the given `timeout` is reached without answers from all
+ * shard regions the reply will contain an empty map of regions.
+ */
+ def this(
+ entityTypeKey: javadsl.EntityTypeKey[_],
+ timeout: java.time.Duration,
+ replyTo: ActorRef[ClusterShardingStats]) =
+ this(entityTypeKey.asScala, JavaDurationConverters.asFiniteDuration(timeout), replyTo)
+}
diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ClusterShardingSettings.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ClusterShardingSettings.scala
index 26b1b40afd..a5df66b723 100644
--- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ClusterShardingSettings.scala
+++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ClusterShardingSettings.scala
@@ -5,6 +5,9 @@
package akka.cluster.sharding.typed
import scala.concurrent.duration.FiniteDuration
+
+import com.typesafe.config.Config
+
import akka.actor.typed.ActorSystem
import akka.annotation.InternalApi
import akka.cluster.ClusterSettings.DataCenter
@@ -13,7 +16,6 @@ import akka.cluster.sharding.{ ClusterShardingSettings => ClassicShardingSetting
import akka.cluster.singleton.{ ClusterSingletonManagerSettings => ClassicClusterSingletonManagerSettings }
import akka.cluster.typed.Cluster
import akka.cluster.typed.ClusterSingletonManagerSettings
-import com.typesafe.config.Config
import akka.util.JavaDurationConverters._
object ClusterShardingSettings {
@@ -123,9 +125,9 @@ object ClusterShardingSettings {
s"Not recognized StateStoreMode, only '${StateStoreModePersistence.name}' and '${StateStoreModeDData.name}' are supported.")
}
- final case object StateStoreModePersistence extends StateStoreMode { override def name = "persistence" }
+ case object StateStoreModePersistence extends StateStoreMode { override def name = "persistence" }
- final case object StateStoreModeDData extends StateStoreMode { override def name = "ddata" }
+ case object StateStoreModeDData extends StateStoreMode { override def name = "ddata" }
/**
* Java API
@@ -175,7 +177,7 @@ object ClusterShardingSettings {
val coordinatorStateWriteMajorityPlus: Int,
val coordinatorStateReadMajorityPlus: Int) {
- def this(classic: ClassicShardingSettings.TuningParameters) {
+ def this(classic: ClassicShardingSettings.TuningParameters) =
this(
bufferSize = classic.bufferSize,
coordinatorFailureBackoff = classic.coordinatorFailureBackoff,
@@ -197,8 +199,6 @@ object ClusterShardingSettings {
coordinatorStateWriteMajorityPlus = classic.coordinatorStateWriteMajorityPlus,
coordinatorStateReadMajorityPlus = classic.coordinatorStateReadMajorityPlus)
- }
-
require(
entityRecoveryStrategy == "all" || entityRecoveryStrategy == "constant",
s"Unknown 'entity-recovery-strategy' [$entityRecoveryStrategy], valid values are 'all' or 'constant'")
diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ShardedDaemonProcessSettings.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ShardedDaemonProcessSettings.scala
index aa98928c5f..cb69c9189b 100644
--- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ShardedDaemonProcessSettings.scala
+++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ShardedDaemonProcessSettings.scala
@@ -6,13 +6,14 @@ package akka.cluster.sharding.typed
import java.time.Duration
+import scala.concurrent.duration.FiniteDuration
+
+import com.typesafe.config.Config
+
import akka.actor.typed.ActorSystem
import akka.annotation.ApiMayChange
import akka.annotation.InternalApi
import akka.util.JavaDurationConverters._
-import com.typesafe.config.Config
-
-import scala.concurrent.duration.FiniteDuration
@ApiMayChange
object ShardedDaemonProcessSettings {
diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ShardingMessageExtractor.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ShardingMessageExtractor.scala
index 43d2c356ac..9df5aa8f92 100644
--- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ShardingMessageExtractor.scala
+++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ShardingMessageExtractor.scala
@@ -4,7 +4,7 @@
package akka.cluster.sharding.typed
-import akka.actor.WrappedMessage
+import akka.actor.{ InvalidMessageException, WrappedMessage }
import akka.util.unused
object ShardingMessageExtractor {
@@ -105,5 +105,11 @@ abstract class HashCodeNoEnvelopeMessageExtractor[M](val numberOfShards: Int) ex
*
* The alternative way of routing messages through sharding is to not use envelopes,
* and have the message types themselves carry identifiers.
+ *
+ * @param entityId The business domain identifier of the entity.
+ * @param message The message to be send to the entity.
+ * @throws `InvalidMessageException` if message is null.
*/
-final case class ShardingEnvelope[M](entityId: String, message: M) extends WrappedMessage
+final case class ShardingEnvelope[M](entityId: String, message: M) extends WrappedMessage {
+ if (message == null) throw InvalidMessageException("[null] is not an allowed message")
+}
diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/ShardingConsumerController.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/ShardingConsumerController.scala
index 4b91274c6b..1864c0ea0e 100644
--- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/ShardingConsumerController.scala
+++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/ShardingConsumerController.scala
@@ -6,6 +6,8 @@ package akka.cluster.sharding.typed.delivery
import java.util.function.{ Function => JFunction }
+import com.typesafe.config.Config
+
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
@@ -13,7 +15,6 @@ import akka.actor.typed.delivery.ConsumerController
import akka.actor.typed.scaladsl.Behaviors
import akka.annotation.ApiMayChange
import akka.cluster.sharding.typed.delivery.internal.ShardingConsumerControllerImpl
-import com.typesafe.config.Config
/**
* `ShardingConsumerController` is used together with [[ShardingProducerController]]. See the description
diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/ShardingProducerController.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/ShardingProducerController.scala
index ca84d81eb6..a00e16fa58 100644
--- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/ShardingProducerController.scala
+++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/ShardingProducerController.scala
@@ -10,6 +10,8 @@ import scala.compat.java8.OptionConverters._
import scala.concurrent.duration.FiniteDuration
import scala.reflect.ClassTag
+import com.typesafe.config.Config
+
import akka.Done
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
@@ -22,7 +24,6 @@ import akka.annotation.ApiMayChange
import akka.cluster.sharding.typed.ShardingEnvelope
import akka.cluster.sharding.typed.delivery.internal.ShardingProducerControllerImpl
import akka.util.JavaDurationConverters._
-import com.typesafe.config.Config
/**
* Reliable delivery between a producer actor sending messages to sharded consumer
@@ -84,6 +85,10 @@ import com.typesafe.config.Config
* The `producerId` is used in logging and included as MDC entry with key `"producerId"`. It's propagated
* to the `ConsumerController` and is useful for correlating log messages. It can be any `String` but it's
* recommended to use a unique identifier of representing the producer.
+ *
+ * If the `DurableProducerQueue` is defined it is created as a child actor of the `ShardingProducerController` actor.
+ * `ProducerController` actors are created for each destination entity. Those child actors use the same dispatcher
+ * as the parent `ShardingProducerController`.
*/
@ApiMayChange // TODO #28719 when removing ApiMayChange consider removing `case class` for some of the messages
object ShardingProducerController {
diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/internal/ShardingConsumerControllerImpl.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/internal/ShardingConsumerControllerImpl.scala
index cda803f069..a95a7e3627 100644
--- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/internal/ShardingConsumerControllerImpl.scala
+++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/internal/ShardingConsumerControllerImpl.scala
@@ -6,6 +6,7 @@ package akka.cluster.sharding.typed.delivery.internal
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
+import akka.actor.typed.DispatcherSelector
import akka.actor.typed.Terminated
import akka.actor.typed.delivery.ConsumerController
import akka.actor.typed.delivery.internal.ConsumerControllerImpl
@@ -92,7 +93,8 @@ private class ShardingConsumerControllerImpl[A](
context.log.debug("Starting ConsumerController for producerId [{}].", seqMsg.producerId)
val cc = context.spawn(
ConsumerController[A](settings.consumerControllerSettings),
- s"consumerController-${seqMsg.producerId}")
+ s"consumerController-${seqMsg.producerId}",
+ DispatcherSelector.sameAsParent())
context.watch(cc)
cc ! ConsumerController.Start(deliverTo)
cc ! seqMsg
diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/internal/ShardingProducerControllerImpl.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/internal/ShardingProducerControllerImpl.scala
index 878ea438c4..395a5c47ed 100644
--- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/internal/ShardingProducerControllerImpl.scala
+++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/delivery/internal/ShardingProducerControllerImpl.scala
@@ -13,6 +13,7 @@ import scala.util.Success
import akka.Done
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
+import akka.actor.typed.DispatcherSelector
import akka.actor.typed.delivery.ConsumerController
import akka.actor.typed.delivery.DurableProducerQueue
import akka.actor.typed.delivery.DurableProducerQueue.ConfirmationQualifier
@@ -236,7 +237,7 @@ import akka.util.Timeout
settings: ShardingProducerController.Settings): Option[ActorRef[DurableProducerQueue.Command[A]]] = {
durableQueueBehavior.map { b =>
- val ref = context.spawn(b, "durable")
+ val ref = context.spawn(b, "durable", DispatcherSelector.sameAsParent())
context.watchWith(ref, DurableQueueTerminated)
askLoadState(context, Some(ref), settings, attempt = 1)
ref
@@ -280,6 +281,7 @@ private class ShardingProducerControllerImpl[A: ClassTag](
private val durableQueueAskTimeout: Timeout = settings.producerControllerSettings.durableQueueRequestTimeout
private val entityAskTimeout: Timeout = settings.internalAskTimeout
+ private val traceEnabled = context.log.isTraceEnabled
private val requestNextAdapter: ActorRef[ProducerController.RequestNext[A]] =
context.messageAdapter(WrappedRequestNext.apply)
@@ -332,7 +334,8 @@ private class ShardingProducerControllerImpl[A: ClassTag](
}
val p = context.spawn(
ProducerController[A](outKey, durableQueueBehavior = None, settings.producerControllerSettings, send),
- entityId)
+ entityId,
+ DispatcherSelector.sameAsParent())
p ! ProducerController.Start(requestNextAdapter)
s.copy(
out = s.out.updated(
@@ -402,7 +405,8 @@ private class ShardingProducerControllerImpl[A: ClassTag](
def receiveAck(ack: Ack): Behavior[InternalCommand] = {
s.out.get(ack.outKey) match {
case Some(outState) =>
- context.log.trace2("Received Ack, confirmed [{}], current [{}].", ack.confirmedSeqNr, s.currentSeqNr)
+ if (traceEnabled)
+ context.log.trace2("Received Ack, confirmed [{}], current [{}].", ack.confirmedSeqNr, s.currentSeqNr)
val newUnconfirmed = onAck(outState, ack.confirmedSeqNr)
val newUsedNanoTime =
if (newUnconfirmed.size != outState.unconfirmed.size) System.nanoTime() else outState.usedNanoTime
@@ -424,7 +428,8 @@ private class ShardingProducerControllerImpl[A: ClassTag](
throw new IllegalStateException(s"Received RequestNext but already has demand for [$outKey]")
val confirmedSeqNr = w.next.confirmedSeqNr
- context.log.trace("Received RequestNext from [{}], confirmed seqNr [{}]", out.entityId, confirmedSeqNr)
+ if (traceEnabled)
+ context.log.trace("Received RequestNext from [{}], confirmed seqNr [{}]", out.entityId, confirmedSeqNr)
val newUnconfirmed = onAck(out, confirmedSeqNr)
if (out.buffered.nonEmpty) {
@@ -571,7 +576,7 @@ private class ShardingProducerControllerImpl[A: ClassTag](
}
private def send(msg: A, outKey: OutKey, outSeqNr: OutSeqNr, nextTo: ProducerController.RequestNext[A]): Unit = {
- if (context.log.isTraceEnabled)
+ if (traceEnabled)
context.log.traceN("Sending [{}] to [{}] with outSeqNr [{}].", msg.getClass.getName, outKey, outSeqNr)
implicit val askTimeout: Timeout = entityAskTimeout
context.ask[ProducerController.MessageWithConfirmation[A], OutSeqNr](
diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ClusterShardingImpl.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ClusterShardingImpl.scala
index b9104f31ec..dbf2aa39cd 100644
--- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ClusterShardingImpl.scala
+++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ClusterShardingImpl.scala
@@ -11,25 +11,23 @@ import java.util.concurrent.CompletionStage
import java.util.concurrent.ConcurrentHashMap
import scala.compat.java8.FutureConverters._
-
-import akka.util.JavaDurationConverters._
import scala.concurrent.Future
import akka.actor.ActorRefProvider
import akka.actor.ExtendedActorSystem
import akka.actor.InternalActorRef
-import akka.actor.typed.TypedActorContext
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.Props
+import akka.actor.typed.TypedActorContext
import akka.actor.typed.internal.InternalRecipientRef
import akka.actor.typed.internal.PoisonPill
import akka.actor.typed.internal.PoisonPillInterceptor
import akka.actor.typed.internal.adapter.ActorRefAdapter
import akka.actor.typed.internal.adapter.ActorSystemAdapter
import akka.actor.typed.scaladsl.Behaviors
-import akka.annotation.InternalApi
+import akka.annotation.{ InternalApi, InternalStableApi }
import akka.cluster.ClusterSettings.DataCenter
import akka.cluster.sharding.ShardCoordinator.LeastShardAllocationStrategy
import akka.cluster.sharding.ShardCoordinator.ShardAllocationStrategy
@@ -42,8 +40,8 @@ import akka.event.LoggingAdapter
import akka.japi.function.{ Function => JFunction }
import akka.pattern.AskTimeoutException
import akka.pattern.PromiseActorRef
-import akka.util.ByteString
-import akka.util.Timeout
+import akka.util.{ unused, ByteString, Timeout }
+import akka.util.JavaDurationConverters._
/**
* INTERNAL API
@@ -312,8 +310,7 @@ import akka.util.Timeout
val replyTo = new EntityPromiseRef[U](shardRegion.asInstanceOf[InternalActorRef], timeout)
val m = message(replyTo.ref)
if (replyTo.promiseRef ne null) replyTo.promiseRef.messageClassName = m.getClass.getName
- shardRegion ! ShardingEnvelope(entityId, m)
- replyTo.future
+ replyTo.ask(shardRegion, entityId, m, timeout)
}
def ask[U](message: JFunction[ActorRef[U], M], timeout: Duration): CompletionStage[U] =
@@ -350,6 +347,16 @@ import akka.util.Timeout
val ref: ActorRef[U] = _ref
val future: Future[U] = _future
val promiseRef: PromiseActorRef = _promiseRef
+
+ @InternalStableApi
+ private[akka] def ask[T](
+ shardRegion: akka.actor.ActorRef,
+ entityId: String,
+ message: T,
+ @unused timeout: Timeout): Future[U] = {
+ shardRegion ! ShardingEnvelope(entityId, message)
+ future
+ }
}
// impl InternalRecipientRef
diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/JoinConfigCompatCheckerClusterSharding.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/JoinConfigCompatCheckerClusterSharding.scala
index e66b8361de..ffa4efc090 100644
--- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/JoinConfigCompatCheckerClusterSharding.scala
+++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/JoinConfigCompatCheckerClusterSharding.scala
@@ -4,11 +4,12 @@
package akka.cluster.sharding.typed.internal
-import akka.annotation.InternalApi
-import akka.cluster.{ ConfigValidation, JoinConfigCompatChecker, Valid }
+import scala.collection.{ immutable => im }
+
import com.typesafe.config.Config
-import scala.collection.{ immutable => im }
+import akka.annotation.InternalApi
+import akka.cluster.{ ConfigValidation, JoinConfigCompatChecker, Valid }
/**
* INTERNAL API
diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ShardedDaemonProcessImpl.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ShardedDaemonProcessImpl.scala
index a809d597b5..5946f3d3b4 100644
--- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ShardedDaemonProcessImpl.scala
+++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ShardedDaemonProcessImpl.scala
@@ -6,6 +6,9 @@ package akka.cluster.sharding.typed.internal
import java.util.Optional
+import scala.compat.java8.OptionConverters._
+import scala.concurrent.duration.Duration
+import scala.reflect.ClassTag
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
@@ -13,13 +16,15 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.LoggerOps
import akka.annotation.InternalApi
import akka.cluster.sharding.ShardRegion.EntityId
-import akka.cluster.sharding.typed.ClusterShardingSettings
+import akka.cluster.sharding.typed.{
+ javadsl,
+ scaladsl,
+ ClusterShardingSettings,
+ ShardedDaemonProcessSettings,
+ ShardingEnvelope,
+ ShardingMessageExtractor
+}
import akka.cluster.sharding.typed.ClusterShardingSettings.{ RememberEntitiesStoreModeDData, StateStoreModeDData }
-import akka.cluster.sharding.typed.ShardingEnvelope
-import akka.cluster.sharding.typed.ShardingMessageExtractor
-import akka.cluster.sharding.typed.scaladsl
-import akka.cluster.sharding.typed.javadsl
-import akka.cluster.sharding.typed.ShardedDaemonProcessSettings
import akka.cluster.sharding.typed.scaladsl.ClusterSharding
import akka.cluster.sharding.typed.scaladsl.Entity
import akka.cluster.sharding.typed.scaladsl.EntityTypeKey
@@ -28,10 +33,6 @@ import akka.cluster.typed.Cluster
import akka.japi.function
import akka.util.PrettyDuration
-import scala.compat.java8.OptionConverters._
-import scala.concurrent.duration.Duration
-import scala.reflect.ClassTag
-
/**
* INTERNAL API
*/
diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ShardingState.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ShardingState.scala
index 4cf8c2147d..ec74d2b7c9 100644
--- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ShardingState.scala
+++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ShardingState.scala
@@ -3,27 +3,64 @@
*/
package akka.cluster.sharding.typed.internal
+
import akka.actor.typed.Behavior
+import akka.actor.typed.SupervisorStrategy
import akka.actor.typed.scaladsl.Behaviors
-import akka.cluster.sharding.{ ClusterSharding, ShardRegion }
-import akka.cluster.sharding.ShardRegion.CurrentShardRegionState
import akka.actor.typed.scaladsl.adapter._
import akka.annotation.InternalApi
-import akka.cluster.sharding.typed.{ ClusterShardingQuery, GetShardRegionState }
+import akka.cluster.sharding.ClusterSharding
+import akka.cluster.sharding.ShardRegion
+import akka.cluster.sharding.ShardRegion.CurrentShardRegionState
+import akka.cluster.sharding.typed.ClusterShardingQuery
+import akka.cluster.sharding.typed.GetClusterShardingStats
+import akka.cluster.sharding.typed.GetShardRegionState
/**
* INTERNAL API
*/
@InternalApi private[akka] object ShardingState {
- def behavior(classicSharding: ClusterSharding): Behavior[ClusterShardingQuery] = Behaviors.receiveMessage {
- case GetShardRegionState(key, replyTo) =>
- if (classicSharding.getShardTypeNames.contains(key.name)) {
- classicSharding.shardRegion(key.name).tell(ShardRegion.GetShardRegionState, replyTo.toClassic)
- } else {
- replyTo ! CurrentShardRegionState(Set.empty)
+ def behavior(classicSharding: ClusterSharding): Behavior[ClusterShardingQuery] = {
+ Behaviors
+ .supervise[ClusterShardingQuery] {
+ Behaviors.setup { context =>
+ Behaviors.receiveMessage {
+ case GetShardRegionState(key, replyTo) =>
+ if (classicSharding.getShardTypeNames.contains(key.name)) {
+ try {
+ classicSharding.shardRegion(key.name).tell(ShardRegion.GetShardRegionState, replyTo.toClassic)
+ } catch {
+ case e: IllegalStateException =>
+ // classicSharding.shardRegion may throw if not initialized
+ context.log.warn(e.getMessage)
+ replyTo ! CurrentShardRegionState(Set.empty)
+ }
+ } else {
+ replyTo ! CurrentShardRegionState(Set.empty)
+ }
+ Behaviors.same
+
+ case GetClusterShardingStats(key, timeout, replyTo) =>
+ if (classicSharding.getShardTypeNames.contains(key.name)) {
+ try {
+ classicSharding
+ .shardRegion(key.name)
+ .tell(ShardRegion.GetClusterShardingStats(timeout), replyTo.toClassic)
+ } catch {
+ case e: IllegalStateException =>
+ // classicSharding.shardRegion may throw if not initialized
+ context.log.warn(e.getMessage)
+ replyTo ! ShardRegion.ClusterShardingStats(Map.empty)
+ }
+ } else {
+ replyTo ! ShardRegion.ClusterShardingStats(Map.empty)
+ }
+ Behaviors.same
+ }
+ }
}
- Behaviors.same
+ .onFailure(SupervisorStrategy.restart)
}
}
diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/javadsl/ClusterSharding.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/javadsl/ClusterSharding.scala
index 96a4f54236..6ca84200e5 100644
--- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/javadsl/ClusterSharding.scala
+++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/javadsl/ClusterSharding.scala
@@ -9,18 +9,19 @@ import java.time.Duration
import java.util.Optional
import java.util.concurrent.CompletionStage
+import com.github.ghik.silencer.silent
+
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
-import akka.actor.typed.RecipientRef
import akka.actor.typed.Props
+import akka.actor.typed.RecipientRef
import akka.actor.typed.internal.InternalRecipientRef
import akka.annotation.DoNotInherit
import akka.annotation.InternalApi
import akka.cluster.sharding.ShardCoordinator.ShardAllocationStrategy
import akka.cluster.sharding.typed.internal.EntityTypeKeyImpl
import akka.japi.function.{ Function => JFunction }
-import com.github.ghik.silencer.silent
@FunctionalInterface
trait EntityFactory[M] {
diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/scaladsl/ClusterSharding.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/scaladsl/ClusterSharding.scala
index a40014dc34..02bf7778a4 100644
--- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/scaladsl/ClusterSharding.scala
+++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/scaladsl/ClusterSharding.scala
@@ -8,23 +8,23 @@ package scaladsl
import scala.concurrent.Future
import scala.reflect.ClassTag
-import akka.util.Timeout
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.Extension
import akka.actor.typed.ExtensionId
import akka.actor.typed.ExtensionSetup
-import akka.actor.typed.RecipientRef
import akka.actor.typed.Props
+import akka.actor.typed.RecipientRef
import akka.actor.typed.internal.InternalRecipientRef
import akka.annotation.DoNotInherit
import akka.annotation.InternalApi
import akka.cluster.ClusterSettings.DataCenter
import akka.cluster.sharding.ShardCoordinator.ShardAllocationStrategy
+import akka.cluster.sharding.ShardRegion.{ StartEntity => ClassicStartEntity }
import akka.cluster.sharding.typed.internal.ClusterShardingImpl
import akka.cluster.sharding.typed.internal.EntityTypeKeyImpl
-import akka.cluster.sharding.ShardRegion.{ StartEntity => ClassicStartEntity }
+import akka.util.Timeout
object ClusterSharding extends ExtensionId[ClusterSharding] {
diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/scaladsl/ShardedDaemonProcess.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/scaladsl/ShardedDaemonProcess.scala
index 276f7baf38..9e655f7c1b 100644
--- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/scaladsl/ShardedDaemonProcess.scala
+++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/scaladsl/ShardedDaemonProcess.scala
@@ -4,6 +4,8 @@
package akka.cluster.sharding.typed.scaladsl
+import scala.reflect.ClassTag
+
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.Extension
@@ -15,8 +17,6 @@ import akka.cluster.sharding.typed.ShardedDaemonProcessSettings
import akka.cluster.sharding.typed.internal.ShardedDaemonProcessImpl
import akka.cluster.sharding.typed.javadsl
-import scala.reflect.ClassTag
-
object ShardedDaemonProcess extends ExtensionId[ShardedDaemonProcess] {
override def createExtension(system: ActorSystem[_]): ShardedDaemonProcess = new ShardedDaemonProcessImpl(system)
}
diff --git a/akka-cluster-sharding-typed/src/multi-jvm/resources/logback-test.xml b/akka-cluster-sharding-typed/src/multi-jvm/resources/logback-test.xml
index e2d3457e51..e36652a969 100644
--- a/akka-cluster-sharding-typed/src/multi-jvm/resources/logback-test.xml
+++ b/akka-cluster-sharding-typed/src/multi-jvm/resources/logback-test.xml
@@ -5,7 +5,7 @@
- %date{ISO8601} %-5level %logger %marker - %msg {%mdc}%n
+ %date{ISO8601} %-5level %logger %marker - %msg MDC: {%mdc}%n
diff --git a/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/ClusterShardingStatsSpec.scala b/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/ClusterShardingStatsSpec.scala
new file mode 100644
index 0000000000..d441b29a08
--- /dev/null
+++ b/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/ClusterShardingStatsSpec.scala
@@ -0,0 +1,108 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.cluster.sharding.typed
+
+import com.typesafe.config.ConfigFactory
+import org.scalatest.concurrent.ScalaFutures
+
+import akka.actor.testkit.typed.scaladsl.TestProbe
+import akka.actor.typed.ActorRef
+import akka.actor.typed.Behavior
+import akka.actor.typed.scaladsl.Behaviors
+import akka.cluster.MultiNodeClusterSpec
+import akka.cluster.sharding.ShardRegion.ClusterShardingStats
+import akka.cluster.sharding.typed.scaladsl.ClusterSharding
+import akka.cluster.sharding.typed.scaladsl.Entity
+import akka.cluster.sharding.typed.scaladsl.EntityTypeKey
+import akka.cluster.typed.MultiNodeTypedClusterSpec
+import akka.remote.testkit.MultiNodeConfig
+import akka.remote.testkit.MultiNodeSpec
+import akka.serialization.jackson.CborSerializable
+
+object ClusterShardingStatsSpecConfig extends MultiNodeConfig {
+
+ val first = role("first")
+ val second = role("second")
+ val third = role("third")
+
+ commonConfig(ConfigFactory.parseString("""
+ akka.log-dead-letters-during-shutdown = off
+ akka.cluster.sharding.updating-state-timeout = 2s
+ akka.cluster.sharding.waiting-for-state-timeout = 2s
+ """).withFallback(MultiNodeClusterSpec.clusterConfig))
+
+}
+
+class ClusterShardingStatsSpecMultiJvmNode1 extends ClusterShardingStatsSpec
+class ClusterShardingStatsSpecMultiJvmNode2 extends ClusterShardingStatsSpec
+class ClusterShardingStatsSpecMultiJvmNode3 extends ClusterShardingStatsSpec
+
+object Pinger {
+ sealed trait Command extends CborSerializable
+ case class Ping(id: Int, ref: ActorRef[Pong]) extends Command
+ case class Pong(id: Int) extends CborSerializable
+
+ def apply(): Behavior[Command] = {
+ Behaviors.receiveMessage[Command] {
+ case Ping(id: Int, ref) =>
+ ref ! Pong(id)
+ Behaviors.same
+ }
+ }
+
+}
+
+abstract class ClusterShardingStatsSpec
+ extends MultiNodeSpec(ClusterShardingStatsSpecConfig)
+ with MultiNodeTypedClusterSpec
+ with ScalaFutures {
+
+ import ClusterShardingStatsSpecConfig._
+ import Pinger._
+
+ private val typeKey = EntityTypeKey[Command]("ping")
+
+ private val sharding = ClusterSharding(typedSystem)
+ private val settings = ClusterShardingSettings(typedSystem)
+ private val queryTimeout = settings.shardRegionQueryTimeout * roles.size.toLong //numeric widening y'all
+
+ "Cluster sharding stats" must {
+ "form cluster" in {
+ formCluster(first, second, third)
+ }
+
+ "get shard stats" in {
+ sharding.init(Entity(typeKey)(_ => Pinger()))
+ enterBarrier("sharding started")
+
+ runOn(first) {
+ val pongProbe = TestProbe[Pong]()
+
+ val entityRef1 = ClusterSharding(typedSystem).entityRefFor(typeKey, "ping-1")
+ entityRef1 ! Ping(1, pongProbe.ref)
+ pongProbe.receiveMessage()
+
+ val entityRef2 = ClusterSharding(typedSystem).entityRefFor(typeKey, "ping-2")
+ entityRef2 ! Ping(2, pongProbe.ref)
+ pongProbe.receiveMessage()
+ }
+ enterBarrier("sharding-initialized")
+
+ runOn(first, second, third) {
+ val replyToProbe = TestProbe[ClusterShardingStats]()
+ sharding.shardState ! GetClusterShardingStats(typeKey, queryTimeout, replyToProbe.ref)
+
+ val stats = replyToProbe.receiveMessage(queryTimeout)
+ stats.regions.size shouldEqual 3
+ stats.regions.values.flatMap(_.stats.values).sum shouldEqual 2
+ stats.regions.values.forall(_.failed.isEmpty) shouldBe true
+ }
+ enterBarrier("done")
+
+ }
+
+ }
+
+}
diff --git a/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/MultiDcClusterShardingSpec.scala b/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/MultiDcClusterShardingSpec.scala
index b36b0b4166..5e52393bd3 100644
--- a/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/MultiDcClusterShardingSpec.scala
+++ b/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/MultiDcClusterShardingSpec.scala
@@ -5,18 +5,20 @@
package akka.cluster.sharding.typed
import scala.concurrent.duration._
-import akka.actor.typed.ActorRef
-import akka.cluster.sharding.typed.scaladsl.EntityTypeKey
-import akka.cluster.sharding.typed.scaladsl.ClusterSharding
-import akka.cluster.sharding.typed.scaladsl.Entity
-import akka.cluster.typed.{ MultiDcPinger, MultiNodeTypedClusterSpec }
-import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
-import akka.actor.testkit.typed.scaladsl.TestProbe
-import akka.cluster.MultiNodeClusterSpec
-import akka.util.Timeout
+
import com.typesafe.config.ConfigFactory
import org.scalatest.concurrent.ScalaFutures
+import akka.actor.testkit.typed.scaladsl.TestProbe
+import akka.actor.typed.ActorRef
+import akka.cluster.MultiNodeClusterSpec
+import akka.cluster.sharding.typed.scaladsl.ClusterSharding
+import akka.cluster.sharding.typed.scaladsl.Entity
+import akka.cluster.sharding.typed.scaladsl.EntityTypeKey
+import akka.cluster.typed.{ MultiDcPinger, MultiNodeTypedClusterSpec }
+import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
+import akka.util.Timeout
+
object MultiDcClusterShardingSpecConfig extends MultiNodeConfig {
val first = role("first")
val second = role("second")
@@ -67,14 +69,14 @@ abstract class MultiDcClusterShardingSpec
"init sharding" in {
val sharding = ClusterSharding(typedSystem)
val shardRegion: ActorRef[ShardingEnvelope[Command]] = sharding.init(Entity(typeKey)(_ => MultiDcPinger()))
- val probe = TestProbe[Pong]
+ val probe = TestProbe[Pong]()
shardRegion ! ShardingEnvelope(entityId, Ping(probe.ref))
probe.expectMessage(max = 15.seconds, Pong(cluster.selfMember.dataCenter))
enterBarrier("sharding-initialized")
}
"be able to message via entity ref" in {
- val probe = TestProbe[Pong]
+ val probe = TestProbe[Pong]()
val entityRef = ClusterSharding(typedSystem).entityRefFor(typeKey, entityId)
entityRef ! Ping(probe.ref)
probe.expectMessage(Pong(cluster.selfMember.dataCenter))
@@ -94,7 +96,7 @@ abstract class MultiDcClusterShardingSpec
runOn(first, second) {
val proxy: ActorRef[ShardingEnvelope[Command]] = ClusterSharding(typedSystem).init(
Entity(typeKey)(_ => MultiDcPinger()).withSettings(ClusterShardingSettings(typedSystem).withDataCenter("dc2")))
- val probe = TestProbe[Pong]
+ val probe = TestProbe[Pong]()
proxy ! ShardingEnvelope(entityId, Ping(probe.ref))
probe.expectMessage(remainingOrDefault, Pong("dc2"))
}
@@ -108,7 +110,7 @@ abstract class MultiDcClusterShardingSpec
val proxy: ActorRef[ShardingEnvelope[Command]] =
ClusterSharding(system).init(Entity(typeKey)(_ => MultiDcPinger()).withDataCenter("dc2"))
//#proxy-dc
- val probe = TestProbe[Pong]
+ val probe = TestProbe[Pong]()
proxy ! ShardingEnvelope(entityId, Ping(probe.ref))
probe.expectMessage(remainingOrDefault, Pong("dc2"))
}
@@ -125,7 +127,7 @@ abstract class MultiDcClusterShardingSpec
val entityRef = ClusterSharding(system).entityRefFor(typeKey, entityId, "dc2")
//#proxy-dc-entityref
- val probe = TestProbe[Pong]
+ val probe = TestProbe[Pong]()
entityRef ! Ping(probe.ref)
probe.expectMessage(remainingOrDefault, Pong("dc2"))
}
diff --git a/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/ShardedDaemonProcessSpec.scala b/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/ShardedDaemonProcessSpec.scala
index c8e5ca0da3..5f1181b509 100644
--- a/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/ShardedDaemonProcessSpec.scala
+++ b/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/ShardedDaemonProcessSpec.scala
@@ -4,6 +4,11 @@
package akka.cluster.sharding.typed
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+import org.scalatest.concurrent.ScalaFutures
+
import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.typed.Behavior
import akka.actor.typed.receptionist.Receptionist
@@ -16,10 +21,6 @@ import akka.cluster.typed.MultiNodeTypedClusterSpec
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.serialization.jackson.CborSerializable
-import com.typesafe.config.ConfigFactory
-import org.scalatest.concurrent.ScalaFutures
-
-import scala.concurrent.duration._
object ShardedDaemonProcessSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/delivery/DeliveryThroughputSpec.scala b/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/delivery/DeliveryThroughputSpec.scala
new file mode 100644
index 0000000000..c0ff1853e6
--- /dev/null
+++ b/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/delivery/DeliveryThroughputSpec.scala
@@ -0,0 +1,511 @@
+/*
+ * Copyright (C) 2016-2020 Lightbend Inc.
+ */
+
+package akka.cluster.sharding.typed.delivery
+
+import java.util.concurrent.Executors
+import java.util.concurrent.TimeUnit.NANOSECONDS
+
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
+import akka.actor.ActorIdentity
+import akka.actor.Identify
+import akka.actor.testkit.typed.scaladsl.TestProbe
+import akka.actor.typed.ActorRef
+import akka.actor.typed.Behavior
+import akka.actor.typed.PostStop
+import akka.actor.typed.delivery.ConsumerController
+import akka.actor.typed.delivery.ProducerController
+import akka.actor.typed.delivery.WorkPullingProducerController
+import akka.actor.typed.receptionist.ServiceKey
+import akka.actor.typed.scaladsl.Behaviors
+import akka.actor.typed.scaladsl.adapter._
+import akka.cluster.MultiNodeClusterSpec
+import akka.cluster.sharding.typed.ShardingEnvelope
+import akka.cluster.sharding.typed.scaladsl.ClusterSharding
+import akka.cluster.sharding.typed.scaladsl.Entity
+import akka.cluster.sharding.typed.scaladsl.EntityTypeKey
+import akka.cluster.typed.MultiNodeTypedClusterSpec
+import akka.remote.artery.BenchmarkFileReporter
+import akka.remote.artery.PlotResult
+import akka.remote.artery.TestRateReporter
+import akka.remote.testconductor.RoleName
+import akka.remote.testkit.MultiNodeConfig
+import akka.remote.testkit.MultiNodeSpec
+import akka.remote.testkit.PerfFlamesSupport
+import akka.serialization.jackson.CborSerializable
+import akka.actor.typed.scaladsl.LoggerOps
+
+object DeliveryThroughputSpec extends MultiNodeConfig {
+ val first = role("first")
+ val second = role("second")
+ val third = role("third")
+
+ val barrierTimeout = 5.minutes
+
+ val cfg = ConfigFactory.parseString(s"""
+ # for serious measurements you should increase the totalMessagesFactor (30)
+ akka.test.DeliveryThroughputSpec.totalMessagesFactor = 10.0
+ akka.reliable-delivery {
+ consumer-controller.flow-control-window = 50
+ sharding.consumer-controller.flow-control-window = 50
+ sharding.producer-controller.cleanup-unused-after = 5s
+ }
+ akka {
+ loglevel = INFO
+ log-dead-letters = off
+ testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s
+ actor.allow-java-serialization = off
+ # quicker dissemination the service keys
+ cluster.typed.receptionist.distributed-data.write-consistency = all
+ cluster.sharding.passivate-idle-entity-after = 5s
+ }
+ """)
+
+ commonConfig(debugConfig(on = false).withFallback(cfg).withFallback(MultiNodeClusterSpec.clusterConfig))
+
+ nodeConfig(second, third) {
+ ConfigFactory.parseString("""
+ akka.cluster.roles = ["worker"]
+ """)
+ }
+
+ lazy val reporterExecutor = Executors.newFixedThreadPool(10)
+ def newRateReporter(name: String): TestRateReporter = {
+ val r = new TestRateReporter(name)
+ reporterExecutor.execute(r)
+ r
+ }
+
+ object Consumer {
+ trait Command
+
+ case object TheMessage extends Command with CborSerializable
+ case object Stop extends Command
+
+ private case class WrappedDelivery(d: ConsumerController.Delivery[Command]) extends Command
+
+ def apply(consumerController: ActorRef[ConsumerController.Start[Command]]): Behavior[Command] = {
+ Behaviors.setup { context =>
+ val traceEnabled = context.log.isTraceEnabled
+ val deliveryAdapter =
+ context.messageAdapter[ConsumerController.Delivery[Command]](WrappedDelivery(_))
+ consumerController ! ConsumerController.Start(deliveryAdapter)
+ val rateReporter = newRateReporter(context.self.path.elements.mkString("/", "/", ""))
+
+ var c = 0L
+
+ def report(): Unit = {
+ rateReporter.onMessage(1, 0) // not using the payload size
+ c += 1
+ }
+
+ Behaviors
+ .receiveMessagePartial[Command] {
+ case WrappedDelivery(d @ ConsumerController.Delivery(_, confirmTo)) =>
+ report()
+ if (traceEnabled)
+ context.log.trace("Processed {}", d.seqNr)
+ confirmTo ! ConsumerController.Confirmed
+ Behaviors.same
+ case Stop =>
+ Behaviors.stopped
+ }
+ .receiveSignal {
+ case (_, PostStop) =>
+ rateReporter.halt()
+ Behaviors.same
+ }
+
+ }
+
+ }
+ }
+
+ object Producer {
+ trait Command
+
+ case object Run extends Command
+ private case class WrappedRequestNext(r: ProducerController.RequestNext[Consumer.Command]) extends Command
+
+ def apply(
+ producerController: ActorRef[ProducerController.Command[Consumer.Command]],
+ testSettings: TestSettings,
+ plotRef: ActorRef[PlotResult],
+ reporter: BenchmarkFileReporter): Behavior[Command] = {
+ val numberOfMessages = testSettings.totalMessages
+
+ Behaviors.setup { context =>
+ val requestNextAdapter =
+ context.messageAdapter[ProducerController.RequestNext[Consumer.Command]](WrappedRequestNext(_))
+ var startTime = System.nanoTime()
+
+ Behaviors.receiveMessage {
+ case WrappedRequestNext(next) =>
+ if (next.confirmedSeqNr >= numberOfMessages) {
+ context.log.info("Completed {} messages", numberOfMessages)
+ reportEnd(startTime, testSettings, plotRef, reporter)
+ Behaviors.stopped
+ } else {
+ next.sendNextTo ! Consumer.TheMessage
+ Behaviors.same
+ }
+
+ case Run =>
+ context.log.info("Starting {} messages", numberOfMessages)
+ startTime = System.nanoTime()
+ producerController ! ProducerController.Start(requestNextAdapter)
+ Behaviors.same
+ }
+ }
+ }
+
+ def reportEnd(
+ startTime: Long,
+ testSettings: TestSettings,
+ plotRef: ActorRef[PlotResult],
+ resultReporter: BenchmarkFileReporter): Unit = {
+ val numberOfMessages = testSettings.totalMessages
+ val took = NANOSECONDS.toMillis(System.nanoTime - startTime)
+ val throughput = (numberOfMessages * 1000.0 / took)
+
+ resultReporter.reportResults(
+ s"=== ${resultReporter.testName} ${testSettings.testName}: " +
+ f"throughput ${throughput}%,.0f msg/s, " +
+ s"$took ms to deliver $numberOfMessages.")
+
+ plotRef ! PlotResult().add(testSettings.testName, throughput)
+ }
+ }
+
+ def serviceKey(testName: String) = ServiceKey[ConsumerController.Command[Consumer.Command]](testName)
+
+ object WorkPullingProducer {
+ trait Command
+
+ case object Run extends Command
+ private case class WrappedRequestNext(r: WorkPullingProducerController.RequestNext[Consumer.Command])
+ extends Command
+
+ def apply(
+ producerController: ActorRef[WorkPullingProducerController.Command[Consumer.Command]],
+ testSettings: TestSettings,
+ plotRef: ActorRef[PlotResult],
+ resultReporter: BenchmarkFileReporter): Behavior[Command] = {
+ val numberOfMessages = testSettings.totalMessages
+
+ Behaviors.setup { context =>
+ val requestNextAdapter =
+ context.messageAdapter[WorkPullingProducerController.RequestNext[Consumer.Command]](WrappedRequestNext(_))
+ var startTime = System.nanoTime()
+ var remaining = numberOfMessages + context.system.settings.config
+ .getInt("akka.reliable-delivery.consumer-controller.flow-control-window")
+
+ Behaviors.receiveMessage {
+ case WrappedRequestNext(next) =>
+ remaining -= 1
+ if (remaining == 0) {
+ context.log.info("Completed {} messages", numberOfMessages)
+ Producer.reportEnd(startTime, testSettings, plotRef, resultReporter)
+ Behaviors.stopped
+ } else {
+ next.sendNextTo ! Consumer.TheMessage
+ Behaviors.same
+ }
+
+ case Run =>
+ context.log.info("Starting {} messages", numberOfMessages)
+ startTime = System.nanoTime()
+ producerController ! WorkPullingProducerController.Start(requestNextAdapter)
+ Behaviors.same
+ }
+ }
+ }
+ }
+
+ def typeKey(testName: String) = EntityTypeKey[ConsumerController.SequencedMessage[Consumer.Command]](testName)
+
+ object ShardingProducer {
+ trait Command
+
+ case object Run extends Command
+ private case class WrappedRequestNext(r: ShardingProducerController.RequestNext[Consumer.Command]) extends Command
+ private case object PrintStatus extends Command
+
+ def apply(
+ producerController: ActorRef[ShardingProducerController.Command[Consumer.Command]],
+ testSettings: TestSettings,
+ plotRef: ActorRef[PlotResult],
+ resultReporter: BenchmarkFileReporter): Behavior[Command] = {
+ val numberOfMessages = testSettings.totalMessages
+
+ Behaviors.withTimers { timers =>
+ Behaviors.setup { context =>
+ timers.startTimerWithFixedDelay(PrintStatus, 1.second)
+ val requestNextAdapter =
+ context.messageAdapter[ShardingProducerController.RequestNext[Consumer.Command]](WrappedRequestNext(_))
+ var startTime = System.nanoTime()
+ var remaining = numberOfMessages + context.system.settings.config
+ .getInt("akka.reliable-delivery.sharding.consumer-controller.flow-control-window")
+ var latestDemand: ShardingProducerController.RequestNext[Consumer.Command] = null
+ var messagesSentToEachEntity: Map[String, Long] = Map.empty[String, Long].withDefaultValue(0L)
+
+ Behaviors.receiveMessage {
+ case WrappedRequestNext(next) =>
+ latestDemand = next
+ remaining -= 1
+ if (remaining == 0) {
+ context.log.info("Completed {} messages", numberOfMessages)
+ Producer.reportEnd(startTime, testSettings, plotRef, resultReporter)
+ Behaviors.stopped
+ } else {
+ val entityId = (remaining % testSettings.numberOfConsumers).toString
+ if (next.entitiesWithDemand(entityId) || !next.bufferedForEntitiesWithoutDemand.contains(entityId)) {
+ messagesSentToEachEntity =
+ messagesSentToEachEntity.updated(entityId, messagesSentToEachEntity(entityId) + 1L)
+
+ next.sendNextTo ! ShardingEnvelope(entityId, Consumer.TheMessage)
+ }
+ Behaviors.same
+ }
+ case Run =>
+ context.log.info("Starting {} messages", numberOfMessages)
+ startTime = System.nanoTime()
+ producerController ! ShardingProducerController.Start(requestNextAdapter)
+ Behaviors.same
+
+ case PrintStatus =>
+ context.log.infoN(
+ "Remaining {}. Latest demand {}. Messages sent {}. Expecting demand from {}",
+ remaining,
+ latestDemand,
+ messagesSentToEachEntity,
+ (remaining % testSettings.numberOfConsumers))
+ Behaviors.same
+ }
+ }
+ }
+ }
+
+ }
+
+ final case class TestSettings(testName: String, totalMessages: Long, numberOfConsumers: Int)
+
+}
+
+class DeliveryThroughputSpecMultiJvmNode1 extends DeliveryThroughputSpec
+class DeliveryThroughputSpecMultiJvmNode2 extends DeliveryThroughputSpec
+class DeliveryThroughputSpecMultiJvmNode3 extends DeliveryThroughputSpec
+
+abstract class DeliveryThroughputSpec
+ extends MultiNodeSpec(DeliveryThroughputSpec)
+ with MultiNodeTypedClusterSpec
+ with PerfFlamesSupport {
+
+ import DeliveryThroughputSpec._
+
+ private val totalMessagesFactor =
+ system.settings.config.getDouble("akka.test.DeliveryThroughputSpec.totalMessagesFactor")
+
+ private var plot = PlotResult()
+
+ private def adjustedTotalMessages(n: Long): Long = (n * totalMessagesFactor).toLong
+
+ override def initialParticipants = roles.size
+
+ override def afterAll(): Unit = {
+ reporterExecutor.shutdown()
+ runOn(first) {
+ println(plot.csv(system.name))
+ }
+ super.afterAll()
+ }
+
+ def identify(name: String, r: RoleName): akka.actor.ActorRef = {
+ val sel = system.actorSelection(node(r) / "user" / name)
+ sel.tell(Identify(None), testActor)
+ expectMsgType[ActorIdentity](10.seconds).ref.get
+ }
+
+ private val settingsToReport = List(
+ "akka.test.DeliveryThroughputSpec.totalMessagesFactor",
+ "akka.reliable-delivery.consumer-controller.flow-control-window")
+ private val resultReporter = BenchmarkFileReporter("DeliveryThroughputSpec", system, settingsToReport)
+
+ def testPointToPoint(testSettings: TestSettings): Unit = {
+ import testSettings._
+
+ runPerfFlames(first, second)(delay = 5.seconds)
+
+ runOn(second) {
+ val consumerController = system.spawn(ConsumerController[Consumer.Command](), s"consumerController-$testName")
+ val consumer = system.spawn(Consumer(consumerController), s"consumer-$testName")
+ enterBarrier(testName + "-consumer-started")
+ enterBarrier(testName + "-done")
+ consumer ! Consumer.Stop
+ }
+
+ runOn(first) {
+ enterBarrier(testName + "-consumer-started")
+ val consumerController =
+ identify(s"consumerController-$testName", second).toTyped[ConsumerController.Command[Consumer.Command]]
+ val plotProbe = TestProbe[PlotResult]()
+ val producerController = system.spawn(
+ ProducerController[Consumer.Command](testName, durableQueueBehavior = None),
+ s"producerController-$testName")
+ val producer =
+ system.spawn(Producer(producerController, testSettings, plotProbe.ref, resultReporter), s"producer-$testName")
+ producerController ! ProducerController.RegisterConsumer(consumerController)
+ producer ! Producer.Run
+ val terminationProbe = TestProbe()
+ terminationProbe.expectTerminated(producer, 1.minute)
+ val plotResult = plotProbe.receiveMessage()
+ plot = plot.addAll(plotResult)
+ enterBarrier(testName + "-done")
+ }
+
+ runOn(third) {
+ enterBarrier(testName + "-consumer-started")
+ enterBarrier(testName + "-done")
+ }
+
+ enterBarrier("after-" + testName)
+ }
+
+ def testWorkPulling(testSettings: TestSettings): Unit = {
+ import testSettings._
+
+ runPerfFlames(first, second, third)(delay = 5.seconds)
+
+ runOn(second, third) {
+ val range = if (myself == second) (1 to numberOfConsumers by 2) else (2 to numberOfConsumers by 2)
+ val consumers = range.map { n =>
+ val consumerController =
+ system.spawn(ConsumerController[Consumer.Command](serviceKey(testName)), s"consumerController-$n-$testName")
+ system.spawn(Consumer(consumerController), s"consumer-$n-$testName")
+ }
+ enterBarrier(testName + "-consumer-started")
+ enterBarrier(testName + "-done")
+ consumers.foreach(_ ! Consumer.Stop)
+ }
+
+ runOn(first) {
+ enterBarrier(testName + "-consumer-started")
+ val plotProbe = TestProbe[PlotResult]()
+ val producerController = system.spawn(
+ WorkPullingProducerController[Consumer.Command](testName, serviceKey(testName), durableQueueBehavior = None),
+ s"producerController-$testName")
+ val producer =
+ system.spawn(
+ WorkPullingProducer(producerController, testSettings, plotProbe.ref, resultReporter),
+ s"producer-$testName")
+ producer ! WorkPullingProducer.Run
+ val terminationProbe = TestProbe()
+ terminationProbe.expectTerminated(producer, 1.minute)
+ val plotResult = plotProbe.receiveMessage()
+ plot = plot.addAll(plotResult)
+ enterBarrier(testName + "-done")
+ }
+
+ enterBarrier("after-" + testName)
+ }
+
+ def testSharding(testSettings: TestSettings): Unit = {
+ import testSettings._
+
+ runPerfFlames(first, second, third)(delay = 5.seconds)
+
+ val region = ClusterSharding(typedSystem).init(Entity(typeKey(testName))(_ =>
+ ShardingConsumerController(consumerController => Consumer(consumerController))).withRole("worker"))
+ enterBarrier(testName + "-sharding-init")
+
+ runOn(first) {
+ val plotProbe = TestProbe[PlotResult]()
+ val producerController = system.spawn(
+ ShardingProducerController[Consumer.Command](testName, region, durableQueueBehavior = None),
+ s"producerController-$testName")
+ val producer =
+ system.spawn(
+ ShardingProducer(producerController, testSettings, plotProbe.ref, resultReporter),
+ s"producer-$testName")
+ producer ! ShardingProducer.Run
+ val terminationProbe = TestProbe()
+ terminationProbe.expectTerminated(producer, 1.minute)
+ val plotResult = plotProbe.receiveMessage()
+ plot = plot.addAll(plotResult)
+ }
+
+ enterBarrier("after-" + testName)
+ }
+
+ "Reliable delivery throughput" must {
+
+ "form cluster" in {
+ formCluster(first, second, third)
+ }
+
+ "warmup point-to-point" in {
+ val testSettings = TestSettings("warmup-point-to-point", adjustedTotalMessages(20000), 1)
+ testPointToPoint(testSettings)
+ }
+
+ "be measured for point-to-point" in {
+ val testSettings = TestSettings("1-to-1", adjustedTotalMessages(50000), 1)
+ testPointToPoint(testSettings)
+ }
+
+ "warmup work-pulling" in {
+ val testSettings = TestSettings("warmup-work-pulling", adjustedTotalMessages(10000), 2)
+ testWorkPulling(testSettings)
+ }
+
+ "be measured for work-pulling with 1 worker" in {
+ val testSettings = TestSettings("work-pulling-1", adjustedTotalMessages(20000), 1)
+ testWorkPulling(testSettings)
+ }
+
+ "be measured for work-pulling with 2 workers" in {
+ val testSettings = TestSettings("work-pulling-2", adjustedTotalMessages(30000), 2)
+ testWorkPulling(testSettings)
+ }
+
+ "be measured for work-pulling with 4 workers" in {
+ val testSettings = TestSettings("work-pulling-4", adjustedTotalMessages(40000), 4)
+ testWorkPulling(testSettings)
+ }
+
+ "be measured for work-pulling with 10 workers" in {
+ val testSettings = TestSettings("work-pulling-20", adjustedTotalMessages(40000), 10)
+ testWorkPulling(testSettings)
+ }
+
+ "warmup sharding" in {
+ val testSettings = TestSettings("warmup-sharding", adjustedTotalMessages(10000), 2)
+ testSharding(testSettings)
+ }
+
+ "be measured for sharding with 1 entity" in {
+ val testSettings = TestSettings("sharding-1", adjustedTotalMessages(20000), 1)
+ testSharding(testSettings)
+ }
+
+ "be measured for sharding with 2 entities" in {
+ val testSettings = TestSettings("sharding-2", adjustedTotalMessages(20000), 2)
+ testSharding(testSettings)
+ }
+
+ "be measured for sharding with 4 entities" in {
+ val testSettings = TestSettings("sharding-4", adjustedTotalMessages(20000), 4)
+ testSharding(testSettings)
+ }
+
+ "be measured for sharding with 10 entities" in {
+ val testSettings = TestSettings("sharding-10", adjustedTotalMessages(20000), 10)
+ testSharding(testSettings)
+ }
+
+ }
+}
diff --git a/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleDocTest.java b/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleDocTest.java
index 28710ac8a4..b0c8d057d8 100644
--- a/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleDocTest.java
+++ b/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleDocTest.java
@@ -4,33 +4,28 @@
package jdocs.akka.cluster.sharding.typed;
+import org.scalatestplus.junit.JUnitSuite;
+
+import static jdocs.akka.cluster.sharding.typed.AccountExampleWithEventHandlersInState.AccountEntity;
+
// #test
import java.math.BigDecimal;
-import java.util.UUID;
+import akka.actor.testkit.typed.javadsl.LogCapturing;
+import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
+import akka.actor.typed.ActorRef;
+import akka.persistence.testkit.javadsl.EventSourcedBehaviorTestKit;
+import akka.persistence.testkit.javadsl.EventSourcedBehaviorTestKit.CommandResultWithReply;
+import akka.persistence.typed.PersistenceId;
+import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
-
-import akka.actor.testkit.typed.javadsl.LogCapturing;
-import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
-import akka.actor.testkit.typed.javadsl.TestProbe;
-import akka.actor.typed.ActorRef;
-import akka.persistence.typed.PersistenceId;
+import static org.junit.Assert.assertTrue;
// #test
-// #test-events
-import akka.actor.typed.eventstream.EventStream;
-import akka.persistence.journal.inmem.InmemJournal;
-
-// #test-events
-
-import org.scalatestplus.junit.JUnitSuite;
-
-import static jdocs.akka.cluster.sharding.typed.AccountExampleWithEventHandlersInState.AccountEntity;
-
// #test
public class AccountExampleDocTest
// #test
@@ -38,100 +33,103 @@ public class AccountExampleDocTest
// #test
{
- // #inmem-config
- private static final String inmemConfig =
- "akka.persistence.journal.plugin = \"akka.persistence.journal.inmem\" \n"
- + "akka.persistence.journal.inmem.test-serialization = on \n";
+ // #testkit
+ @ClassRule
+ public static final TestKitJunitResource testKit =
+ new TestKitJunitResource(EventSourcedBehaviorTestKit.config());
- // #inmem-config
-
- // #snapshot-store-config
- private static final String snapshotConfig =
- "akka.persistence.snapshot-store.plugin = \"akka.persistence.snapshot-store.local\" \n"
- + "akka.persistence.snapshot-store.local.dir = \"target/snapshot-"
- + UUID.randomUUID().toString()
- + "\" \n";
- // #snapshot-store-config
-
- private static final String config = inmemConfig + snapshotConfig;
-
- @ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource(config);
+ private EventSourcedBehaviorTestKit<
+ AccountEntity.Command, AccountEntity.Event, AccountEntity.Account>
+ eventSourcedTestKit =
+ EventSourcedBehaviorTestKit.create(
+ testKit.system(), AccountEntity.create("1", PersistenceId.of("Account", "1")));
+ // #testkit
@Rule public final LogCapturing logCapturing = new LogCapturing();
+ @Before
+ public void beforeEach() {
+ eventSourcedTestKit.clear();
+ }
+
+ @Test
+ public void createWithEmptyBalance() {
+ CommandResultWithReply<
+ AccountEntity.Command,
+ AccountEntity.Event,
+ AccountEntity.Account,
+ AccountEntity.OperationResult>
+ result = eventSourcedTestKit.runCommand(AccountEntity.CreateAccount::new);
+ assertEquals(AccountEntity.Confirmed.INSTANCE, result.reply());
+ assertEquals(AccountEntity.AccountCreated.INSTANCE, result.event());
+ assertEquals(BigDecimal.ZERO, result.stateOfType(AccountEntity.OpenedAccount.class).balance);
+ }
+
@Test
public void handleWithdraw() {
- ActorRef ref =
- testKit.spawn(AccountEntity.create("1", PersistenceId.of("Account", "1")));
- TestProbe probe =
- testKit.createTestProbe(AccountEntity.OperationResult.class);
- ref.tell(new AccountEntity.CreateAccount(probe.getRef()));
- probe.expectMessage(AccountEntity.Confirmed.INSTANCE);
- ref.tell(new AccountEntity.Deposit(BigDecimal.valueOf(100), probe.getRef()));
- probe.expectMessage(AccountEntity.Confirmed.INSTANCE);
- ref.tell(new AccountEntity.Withdraw(BigDecimal.valueOf(10), probe.getRef()));
- probe.expectMessage(AccountEntity.Confirmed.INSTANCE);
+ eventSourcedTestKit.runCommand(AccountEntity.CreateAccount::new);
+
+ CommandResultWithReply<
+ AccountEntity.Command,
+ AccountEntity.Event,
+ AccountEntity.Account,
+ AccountEntity.OperationResult>
+ result1 =
+ eventSourcedTestKit.runCommand(
+ replyTo -> new AccountEntity.Deposit(BigDecimal.valueOf(100), replyTo));
+ assertEquals(AccountEntity.Confirmed.INSTANCE, result1.reply());
+ assertEquals(
+ BigDecimal.valueOf(100), result1.eventOfType(AccountEntity.Deposited.class).amount);
+ assertEquals(
+ BigDecimal.valueOf(100), result1.stateOfType(AccountEntity.OpenedAccount.class).balance);
+
+ CommandResultWithReply<
+ AccountEntity.Command,
+ AccountEntity.Event,
+ AccountEntity.Account,
+ AccountEntity.OperationResult>
+ result2 =
+ eventSourcedTestKit.runCommand(
+ replyTo -> new AccountEntity.Withdraw(BigDecimal.valueOf(10), replyTo));
+ assertEquals(AccountEntity.Confirmed.INSTANCE, result2.reply());
+ assertEquals(BigDecimal.valueOf(10), result2.eventOfType(AccountEntity.Withdrawn.class).amount);
+ assertEquals(
+ BigDecimal.valueOf(90), result2.stateOfType(AccountEntity.OpenedAccount.class).balance);
}
@Test
public void rejectWithdrawOverdraft() {
- ActorRef ref =
- testKit.spawn(AccountEntity.create("2", PersistenceId.of("Account", "2")));
- TestProbe probe =
- testKit.createTestProbe(AccountEntity.OperationResult.class);
- ref.tell(new AccountEntity.CreateAccount(probe.getRef()));
- probe.expectMessage(AccountEntity.Confirmed.INSTANCE);
- ref.tell(new AccountEntity.Deposit(BigDecimal.valueOf(100), probe.getRef()));
- probe.expectMessage(AccountEntity.Confirmed.INSTANCE);
- ref.tell(new AccountEntity.Withdraw(BigDecimal.valueOf(110), probe.getRef()));
- probe.expectMessageClass(AccountEntity.Rejected.class);
+ eventSourcedTestKit.runCommand(AccountEntity.CreateAccount::new);
+ eventSourcedTestKit.runCommand(
+ (ActorRef replyTo) ->
+ new AccountEntity.Deposit(BigDecimal.valueOf(100), replyTo));
+
+ CommandResultWithReply<
+ AccountEntity.Command,
+ AccountEntity.Event,
+ AccountEntity.Account,
+ AccountEntity.OperationResult>
+ result =
+ eventSourcedTestKit.runCommand(
+ replyTo -> new AccountEntity.Withdraw(BigDecimal.valueOf(110), replyTo));
+ result.replyOfType(AccountEntity.Rejected.class);
+ assertTrue(result.hasNoEvents());
}
@Test
public void handleGetBalance() {
- ActorRef ref =
- testKit.spawn(AccountEntity.create("3", PersistenceId.of("Account", "3")));
- TestProbe opProbe =
- testKit.createTestProbe(AccountEntity.OperationResult.class);
- ref.tell(new AccountEntity.CreateAccount(opProbe.getRef()));
- opProbe.expectMessage(AccountEntity.Confirmed.INSTANCE);
- ref.tell(new AccountEntity.Deposit(BigDecimal.valueOf(100), opProbe.getRef()));
- opProbe.expectMessage(AccountEntity.Confirmed.INSTANCE);
+ eventSourcedTestKit.runCommand(AccountEntity.CreateAccount::new);
+ eventSourcedTestKit.runCommand(
+ (ActorRef replyTo) ->
+ new AccountEntity.Deposit(BigDecimal.valueOf(100), replyTo));
- TestProbe getProbe =
- testKit.createTestProbe(AccountEntity.CurrentBalance.class);
- ref.tell(new AccountEntity.GetBalance(getProbe.getRef()));
- assertEquals(
- BigDecimal.valueOf(100),
- getProbe.expectMessageClass(AccountEntity.CurrentBalance.class).balance);
+ CommandResultWithReply<
+ AccountEntity.Command,
+ AccountEntity.Event,
+ AccountEntity.Account,
+ AccountEntity.CurrentBalance>
+ result = eventSourcedTestKit.runCommand(AccountEntity.GetBalance::new);
+ assertEquals(BigDecimal.valueOf(100), result.reply().balance);
}
-
- // #test
- // #test-events
- @Test
- public void storeEvents() {
- TestProbe eventProbe = testKit.createTestProbe();
- testKit
- .system()
- .eventStream()
- .tell(new EventStream.Subscribe<>(InmemJournal.Operation.class, eventProbe.getRef()));
-
- ActorRef ref =
- testKit.spawn(AccountEntity.create("4", PersistenceId.of("Account", "4")));
- TestProbe probe =
- testKit.createTestProbe(AccountEntity.OperationResult.class);
- ref.tell(new AccountEntity.CreateAccount(probe.getRef()));
- assertEquals(
- AccountEntity.AccountCreated.INSTANCE,
- eventProbe.expectMessageClass(InmemJournal.Write.class).event());
-
- ref.tell(new AccountEntity.Deposit(BigDecimal.valueOf(100), probe.getRef()));
- assertEquals(
- BigDecimal.valueOf(100),
- ((AccountEntity.Deposited) eventProbe.expectMessageClass(InmemJournal.Write.class).event())
- .amount);
- }
- // #test
- // #test-events
}
// #test
diff --git a/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleWithEventHandlersInState.java b/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleWithEventHandlersInState.java
index be6e5ba857..ef9f2c586d 100644
--- a/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleWithEventHandlersInState.java
+++ b/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleWithEventHandlersInState.java
@@ -38,10 +38,10 @@ public interface AccountExampleWithEventHandlersInState {
// Command
// #reply-command
- interface Command extends CborSerializable {}
+ interface Command extends CborSerializable {}
// #reply-command
- public static class CreateAccount implements Command {
+ public static class CreateAccount implements Command {
public final ActorRef replyTo;
@JsonCreator
@@ -50,7 +50,7 @@ public interface AccountExampleWithEventHandlersInState {
}
}
- public static class Deposit implements Command {
+ public static class Deposit implements Command {
public final BigDecimal amount;
public final ActorRef replyTo;
@@ -60,7 +60,7 @@ public interface AccountExampleWithEventHandlersInState {
}
}
- public static class Withdraw implements Command {
+ public static class Withdraw implements Command {
public final BigDecimal amount;
public final ActorRef replyTo;
@@ -70,7 +70,7 @@ public interface AccountExampleWithEventHandlersInState {
}
}
- public static class GetBalance implements Command {
+ public static class GetBalance implements Command {
public final ActorRef replyTo;
@JsonCreator
@@ -79,7 +79,7 @@ public interface AccountExampleWithEventHandlersInState {
}
}
- public static class CloseAccount implements Command {
+ public static class CloseAccount implements Command {
public final ActorRef replyTo;
@JsonCreator
diff --git a/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleWithMutableState.java b/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleWithMutableState.java
index 70da5c1bf3..2c9b9ac4fc 100644
--- a/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleWithMutableState.java
+++ b/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleWithMutableState.java
@@ -35,9 +35,9 @@ public interface AccountExampleWithMutableState {
EntityTypeKey.create(Command.class, "Account");
// Command
- interface Command extends CborSerializable {}
+ interface Command extends CborSerializable {}
- public static class CreateAccount implements Command {
+ public static class CreateAccount implements Command {
public final ActorRef replyTo;
@JsonCreator
@@ -46,7 +46,7 @@ public interface AccountExampleWithMutableState {
}
}
- public static class Deposit implements Command {
+ public static class Deposit implements Command {
public final BigDecimal amount;
public final ActorRef replyTo;
@@ -56,7 +56,7 @@ public interface AccountExampleWithMutableState {
}
}
- public static class Withdraw implements Command {
+ public static class Withdraw implements Command {
public final BigDecimal amount;
public final ActorRef replyTo;
@@ -66,7 +66,7 @@ public interface AccountExampleWithMutableState {
}
}
- public static class GetBalance implements Command {
+ public static class GetBalance implements Command {
public final ActorRef replyTo;
@JsonCreator
@@ -75,7 +75,7 @@ public interface AccountExampleWithMutableState {
}
}
- public static class CloseAccount implements Command {
+ public static class CloseAccount implements Command {
public final ActorRef replyTo;
@JsonCreator
diff --git a/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleWithNullState.java b/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleWithNullState.java
index 0402cce01a..0afce2091e 100644
--- a/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleWithNullState.java
+++ b/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleWithNullState.java
@@ -35,9 +35,9 @@ public interface AccountExampleWithNullState {
EntityTypeKey.create(Command.class, "Account");
// Command
- interface Command extends CborSerializable {}
+ interface Command extends CborSerializable {}
- public static class CreateAccount implements Command {
+ public static class CreateAccount implements Command {
public final ActorRef replyTo;
@JsonCreator
@@ -46,7 +46,7 @@ public interface AccountExampleWithNullState {
}
}
- public static class Deposit implements Command {
+ public static class Deposit implements Command {
public final BigDecimal amount;
public final ActorRef replyTo;
@@ -56,7 +56,7 @@ public interface AccountExampleWithNullState {
}
}
- public static class Withdraw implements Command {
+ public static class Withdraw implements Command {
public final BigDecimal amount;
public final ActorRef replyTo;
@@ -66,7 +66,7 @@ public interface AccountExampleWithNullState {
}
}
- public static class GetBalance implements Command {
+ public static class GetBalance implements Command {
public final ActorRef replyTo;
@JsonCreator
@@ -75,7 +75,7 @@ public interface AccountExampleWithNullState {
}
}
- public static class CloseAccount implements Command {
+ public static class CloseAccount implements Command {
public final ActorRef replyTo;
@JsonCreator
diff --git a/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ShardingCompileOnlyTest.java b/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ShardingCompileOnlyTest.java
index bd72ed5879..450e56db6b 100644
--- a/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ShardingCompileOnlyTest.java
+++ b/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ShardingCompileOnlyTest.java
@@ -24,6 +24,17 @@ import akka.persistence.typed.PersistenceId;
// #import
+// #get-shard-region-state
+import akka.cluster.sharding.typed.GetShardRegionState;
+import akka.cluster.sharding.ShardRegion.CurrentShardRegionState;
+
+// #get-shard-region-state
+// #get-cluster-sharding-stats
+import akka.cluster.sharding.typed.GetClusterShardingStats;
+import akka.cluster.sharding.ShardRegion.ClusterShardingStats;
+
+// #get-cluster-sharding-stats
+
import jdocs.akka.persistence.typed.BlogPostEntity;
interface ShardingCompileOnlyTest {
@@ -248,4 +259,31 @@ interface ShardingCompileOnlyTest {
ClusterSharding.get(system).entityRefFor(typeKey, entityId, "dc2");
// #proxy-dc-entityref
}
+
+ public static void shardRegionQqueryExample() {
+ ActorSystem system = ActorSystem.create(Behaviors.empty(), "ShardingExample");
+ ActorRef replyMessageAdapter = null;
+ EntityTypeKey typeKey = EntityTypeKey.create(Counter.Command.class, "Counter");
+
+ // #get-shard-region-state
+ ActorRef replyTo = replyMessageAdapter;
+
+ ClusterSharding.get(system).shardState().tell(new GetShardRegionState(typeKey, replyTo));
+ // #get-shard-region-state
+ }
+
+ public static void shardingStatsQqueryExample() {
+ ActorSystem system = ActorSystem.create(Behaviors.empty(), "ShardingExample");
+ ActorRef replyMessageAdapter = null;
+ EntityTypeKey typeKey = EntityTypeKey.create(Counter.Command.class, "Counter");
+
+ // #get-cluster-sharding-stats
+ ActorRef replyTo = replyMessageAdapter;
+ Duration timeout = Duration.ofSeconds(5);
+
+ ClusterSharding.get(system)
+ .shardState()
+ .tell(new GetClusterShardingStats(typeKey, timeout, replyTo));
+ // #get-cluster-sharding-stats
+ }
}
diff --git a/akka-cluster-sharding-typed/src/test/resources/logback-test.xml b/akka-cluster-sharding-typed/src/test/resources/logback-test.xml
index d3f08fca6e..3d08c776bd 100644
--- a/akka-cluster-sharding-typed/src/test/resources/logback-test.xml
+++ b/akka-cluster-sharding-typed/src/test/resources/logback-test.xml
@@ -5,7 +5,7 @@
- %date{ISO8601} %-5level %logger %marker - %msg {%mdc}%n
+ %date{ISO8601} %-5level %logger %marker - %msg MDC: {%mdc}%n
diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/JoinConfigCompatCheckerClusterShardingSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/JoinConfigCompatCheckerClusterShardingSpec.scala
index e530846796..4cde31a807 100644
--- a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/JoinConfigCompatCheckerClusterShardingSpec.scala
+++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/JoinConfigCompatCheckerClusterShardingSpec.scala
@@ -7,14 +7,15 @@ package akka.cluster.sharding.typed
import scala.collection.{ immutable => im }
import scala.concurrent.duration._
+import com.typesafe.config.{ Config, ConfigFactory }
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.CoordinatedShutdown
import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, LogCapturing, ScalaTestWithActorTestKit }
import akka.actor.typed.ActorSystem
import akka.cluster.{ Cluster => ClassicCluster }
import akka.testkit.LongRunningTest
-import com.typesafe.config.{ Config, ConfigFactory }
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpecLike
object JoinConfigCompatCheckerClusterShardingSpec {
@@ -45,8 +46,8 @@ class JoinConfigCompatCheckerClusterShardingSpec
with Matchers
with LogCapturing {
- import JoinConfigCompatCheckerClusterShardingSpec._
import CoordinatedShutdown.IncompatibleConfigurationDetectedReason
+ import JoinConfigCompatCheckerClusterShardingSpec._
private val clusterWaitDuration = 5.seconds
diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ShardingSerializerSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ShardingSerializerSpec.scala
index e1d3d59c6a..08dff6e091 100644
--- a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ShardingSerializerSpec.scala
+++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ShardingSerializerSpec.scala
@@ -4,12 +4,13 @@
package akka.cluster.sharding.typed
+import org.scalatest.wordspec.AnyWordSpecLike
+
+import akka.actor.testkit.typed.scaladsl.LogCapturing
+import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.typed.internal.adapter.ActorSystemAdapter
import akka.cluster.sharding.typed.internal.ShardingSerializer
import akka.serialization.SerializationExtension
-import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
-import akka.actor.testkit.typed.scaladsl.LogCapturing
-import org.scalatest.wordspec.AnyWordSpecLike
class ShardingSerializerSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing {
diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/delivery/DurableShardingSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/delivery/DurableShardingSpec.scala
index ef9497d483..73266a6104 100644
--- a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/delivery/DurableShardingSpec.scala
+++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/delivery/DurableShardingSpec.scala
@@ -6,6 +6,10 @@ package akka.cluster.sharding.typed.delivery
import java.util.UUID
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.Done
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
@@ -26,9 +30,6 @@ import akka.cluster.typed.Join
import akka.persistence.journal.inmem.InmemJournal
import akka.persistence.typed.PersistenceId
import akka.persistence.typed.delivery.EventSourcedProducerQueue
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object DurableShardingSpec {
def conf: Config =
diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/delivery/ReliableDeliveryShardingSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/delivery/ReliableDeliveryShardingSpec.scala
index 09374f0b39..175e2cae6b 100644
--- a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/delivery/ReliableDeliveryShardingSpec.scala
+++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/delivery/ReliableDeliveryShardingSpec.scala
@@ -8,6 +8,9 @@ import java.util.concurrent.atomic.AtomicInteger
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.Done
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.LoggingTestKit
@@ -20,14 +23,12 @@ import akka.actor.typed.delivery.TestConsumer
import akka.actor.typed.delivery.internal.ProducerControllerImpl
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.LoggerOps
-import akka.cluster.typed.Cluster
import akka.cluster.sharding.typed.ShardingEnvelope
import akka.cluster.sharding.typed.scaladsl.ClusterSharding
import akka.cluster.sharding.typed.scaladsl.Entity
import akka.cluster.sharding.typed.scaladsl.EntityTypeKey
+import akka.cluster.typed.Cluster
import akka.cluster.typed.Join
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object ReliableDeliveryShardingSpec {
val config = ConfigFactory.parseString("""
@@ -42,7 +43,7 @@ object ReliableDeliveryShardingSpec {
trait Command
final case class RequestNext(sendToRef: ActorRef[ShardingEnvelope[TestConsumer.Job]]) extends Command
- private final case object Tick extends Command
+ private case object Tick extends Command
def apply(producerController: ActorRef[ShardingProducerController.Start[TestConsumer.Job]]): Behavior[Command] = {
Behaviors.setup { context =>
diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ClusterShardingPersistenceSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ClusterShardingPersistenceSpec.scala
index 06ceb54bf2..b9daada054 100644
--- a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ClusterShardingPersistenceSpec.scala
+++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ClusterShardingPersistenceSpec.scala
@@ -13,28 +13,29 @@ import scala.concurrent.Future
import scala.concurrent.Promise
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.Done
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
-import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
import akka.actor.typed.PostStop
import akka.actor.typed.internal.PoisonPill
import akka.actor.typed.scaladsl.Behaviors
+import akka.cluster.sharding.{ ClusterSharding => ClassicClusterSharding }
import akka.cluster.sharding.ShardRegion.CurrentShardRegionState
import akka.cluster.sharding.ShardRegion.GetShardRegionState
import akka.cluster.sharding.typed.scaladsl.ClusterSharding.Passivate
import akka.cluster.sharding.typed.scaladsl.ClusterSharding.ShardCommand
-import akka.cluster.sharding.{ ClusterSharding => ClassicClusterSharding }
import akka.cluster.typed.Cluster
import akka.cluster.typed.Join
import akka.persistence.typed.PersistenceId
import akka.persistence.typed.RecoveryCompleted
import akka.persistence.typed.scaladsl.Effect
import akka.persistence.typed.scaladsl.EventSourcedBehavior
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object ClusterShardingPersistenceSpec {
val config = ConfigFactory.parseString("""
diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ClusterShardingSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ClusterShardingSpec.scala
index fa2d001833..9417b5b286 100644
--- a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ClusterShardingSpec.scala
+++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ClusterShardingSpec.scala
@@ -8,6 +8,9 @@ import scala.concurrent.duration._
import scala.util.Failure
import scala.util.Success
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl.ActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
@@ -27,8 +30,6 @@ import akka.pattern.AskTimeoutException
import akka.serialization.jackson.CborSerializable
import akka.util.Timeout
import akka.util.ccompat._
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
@ccompatUsedUntil213
object ClusterShardingSpec {
diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ClusterShardingStateSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ClusterShardingStateSpec.scala
index 48d575f7dc..f47ccdd52c 100644
--- a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ClusterShardingStateSpec.scala
+++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ClusterShardingStateSpec.scala
@@ -4,33 +4,34 @@
package akka.cluster.sharding.typed.scaladsl
-import akka.actor.testkit.typed.scaladsl.LogCapturing
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl.{ ScalaTestWithActorTestKit, TestProbe }
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.ActorRef
import akka.cluster.sharding.ShardRegion.{ CurrentShardRegionState, ShardState }
-import akka.cluster.sharding.typed.scaladsl.ClusterShardingSpec._
import akka.cluster.sharding.typed.{ GetShardRegionState, ShardingMessageExtractor }
+import akka.cluster.sharding.typed.scaladsl.ClusterShardingSpec._
import akka.cluster.typed.{ Cluster, Join }
-import org.scalatest.wordspec.AnyWordSpecLike
class ClusterShardingStateSpec
extends ScalaTestWithActorTestKit(ClusterShardingSpec.config)
with AnyWordSpecLike
with LogCapturing {
- val sharding = ClusterSharding(system)
+ private val sharding = ClusterSharding(system)
- val shardExtractor = ShardingMessageExtractor.noEnvelope[IdTestProtocol](10, IdStopPlz()) {
+ private val shardExtractor = ShardingMessageExtractor.noEnvelope[IdTestProtocol](10, IdStopPlz()) {
case IdReplyPlz(id, _) => id
case IdWhoAreYou(id, _) => id
case other => throw new IllegalArgumentException(s"Unexpected message $other")
}
- val cluster = Cluster(system)
+ private val cluster = Cluster(system)
val typeKey: EntityTypeKey[IdTestProtocol] = ClusterShardingSpec.typeKeyWithoutEnvelopes
- "Cluster Sharding" must {
+ "Cluster Sharding CurrentShardRegionState query" must {
"allow querying of the shard region state" in {
val probe = TestProbe[CurrentShardRegionState]()
cluster.manager ! Join(cluster.selfMember.address)
diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ClusterShardingStatsSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ClusterShardingStatsSpec.scala
new file mode 100644
index 0000000000..628d2f7a54
--- /dev/null
+++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ClusterShardingStatsSpec.scala
@@ -0,0 +1,74 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.cluster.sharding.typed.scaladsl
+
+import org.scalatest.wordspec.AnyWordSpecLike
+
+import akka.actor.testkit.typed.scaladsl.LogCapturing
+import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
+import akka.actor.typed.ActorRef
+import akka.cluster.sharding.ShardRegion.ClusterShardingStats
+import akka.cluster.sharding.ShardRegion.ShardRegionStats
+import akka.cluster.sharding.typed.ClusterShardingSettings
+import akka.cluster.sharding.typed.GetClusterShardingStats
+import akka.cluster.sharding.typed.scaladsl.ClusterShardingSpec._
+import akka.cluster.typed.Cluster
+import akka.cluster.typed.Join
+import akka.cluster.typed.SelfUp
+
+class ClusterShardingStatsSpec
+ extends ScalaTestWithActorTestKit(ClusterShardingSpec.config)
+ with AnyWordSpecLike
+ with LogCapturing {
+
+ private val sharding = ClusterSharding(system)
+
+ private val typeKey: EntityTypeKey[IdTestProtocol] = ClusterShardingSpec.typeKeyWithoutEnvelopes
+
+ private val shardExtractor = ClusterShardingSpec.idTestProtocolMessageExtractor
+
+ // no need to scale this up here for the cluster query versus one region
+ private val queryTimeout = ClusterShardingSettings(system).shardRegionQueryTimeout
+
+ "Cluster Sharding ClusterShardingStats query" must {
+ "return empty statistics if there are no running sharded entities" in {
+ val cluster = Cluster(system)
+ val upProbe = createTestProbe[SelfUp]()
+
+ cluster.subscriptions ! akka.cluster.typed.Subscribe(upProbe.ref, classOf[SelfUp])
+ cluster.manager ! Join(cluster.selfMember.address)
+ upProbe.expectMessageType[SelfUp]
+
+ val replyProbe = createTestProbe[ClusterShardingStats]()
+ sharding.shardState ! GetClusterShardingStats(typeKey, queryTimeout, replyProbe.ref)
+ replyProbe.expectMessage(ClusterShardingStats(Map.empty))
+ }
+
+ "allow querying of statistics of the currently running sharded entities in the entire cluster" in {
+ val shardingRef: ActorRef[IdTestProtocol] = sharding.init(
+ Entity(typeKey)(_ => ClusterShardingSpec.behaviorWithId())
+ .withStopMessage(IdStopPlz())
+ .withMessageExtractor(idTestProtocolMessageExtractor))
+
+ val replyProbe = createTestProbe[String]()
+ val id1 = "id1"
+ shardingRef ! IdReplyPlz(id1, replyProbe.ref)
+ replyProbe.expectMessage("Hello!")
+
+ val replyToProbe = createTestProbe[ClusterShardingStats]()
+ val replyTo = replyToProbe.ref
+
+ ClusterSharding(system).shardState ! GetClusterShardingStats(typeKey, queryTimeout, replyTo)
+ val stats = replyToProbe.receiveMessage()
+
+ val expect = ClusterShardingStats(
+ Map(Cluster(system).selfMember.address -> ShardRegionStats(Map(shardExtractor.shardId(id1) -> 1), Set.empty)))
+
+ stats shouldEqual expect
+ }
+
+ }
+
+}
diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ShardedDaemonProcessSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ShardedDaemonProcessSpec.scala
index bb1830521a..519c0a2953 100644
--- a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ShardedDaemonProcessSpec.scala
+++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/scaladsl/ShardedDaemonProcessSpec.scala
@@ -4,6 +4,11 @@
package akka.cluster.sharding.typed.scaladsl
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.typed.ActorRef
@@ -14,10 +19,6 @@ import akka.cluster.sharding.typed.ClusterShardingSettings
import akka.cluster.sharding.typed.ShardedDaemonProcessSettings
import akka.cluster.typed.Cluster
import akka.cluster.typed.Join
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
-
-import scala.concurrent.duration._
object ShardedDaemonProcessSpec {
// single node cluster config
diff --git a/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleDocSpec.scala b/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleDocSpec.scala
index e94ea0b306..fff3609ea3 100644
--- a/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleDocSpec.scala
+++ b/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleDocSpec.scala
@@ -5,102 +5,76 @@
package docs.akka.cluster.sharding.typed
//#test
-import java.util.UUID
-
+import akka.persistence.testkit.scaladsl.EventSourcedBehaviorTestKit
+import akka.persistence.typed.PersistenceId
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
-import akka.persistence.typed.PersistenceId
+import org.scalatest.BeforeAndAfterEach
import org.scalatest.wordspec.AnyWordSpecLike
//#test
-//#test-events
-import akka.persistence.journal.inmem.InmemJournal
-import akka.actor.typed.eventstream.EventStream
-
-//#test-events
-
import docs.akka.cluster.sharding.typed.AccountExampleWithEventHandlersInState.AccountEntity
-object AccountExampleDocSpec {
- val inmemConfig =
- //#inmem-config
- """
- akka.persistence.journal.plugin = "akka.persistence.journal.inmem"
- akka.persistence.journal.inmem.test-serialization = on
- """
- //#inmem-config
-
- val snapshotConfig =
- //#snapshot-store-config
- s"""
- akka.persistence.snapshot-store.plugin = "akka.persistence.snapshot-store.local"
- akka.persistence.snapshot-store.local.dir = "target/snapshot-${UUID.randomUUID().toString}"
- """
- //#snapshot-store-config
-}
-
//#test
-class AccountExampleDocSpec extends ScalaTestWithActorTestKit(s"""
- akka.persistence.journal.plugin = "akka.persistence.journal.inmem"
- akka.persistence.snapshot-store.plugin = "akka.persistence.snapshot-store.local"
- akka.persistence.snapshot-store.local.dir = "target/snapshot-${UUID.randomUUID().toString}"
- """) with AnyWordSpecLike with LogCapturing {
+//#testkit
+class AccountExampleDocSpec
+ extends ScalaTestWithActorTestKit(EventSourcedBehaviorTestKit.config)
+ //#testkit
+ with AnyWordSpecLike
+ with BeforeAndAfterEach
+ with LogCapturing {
+
+ private val eventSourcedTestKit =
+ EventSourcedBehaviorTestKit[AccountEntity.Command, AccountEntity.Event, AccountEntity.Account](
+ system,
+ AccountEntity("1", PersistenceId("Account", "1")))
+
+ override protected def beforeEach(): Unit = {
+ super.beforeEach()
+ eventSourcedTestKit.clear()
+ }
"Account" must {
+ "be created with zero balance" in {
+ val result = eventSourcedTestKit.runCommand[AccountEntity.OperationResult](AccountEntity.CreateAccount(_))
+ result.reply shouldBe AccountEntity.Confirmed
+ result.event shouldBe AccountEntity.AccountCreated
+ result.stateOfType[AccountEntity.OpenedAccount].balance shouldBe 0
+ }
+
"handle Withdraw" in {
- val probe = createTestProbe[AccountEntity.OperationResult]()
- val ref = spawn(AccountEntity("1", PersistenceId("Account", "1")))
- ref ! AccountEntity.CreateAccount(probe.ref)
- probe.expectMessage(AccountEntity.Confirmed)
- ref ! AccountEntity.Deposit(100, probe.ref)
- probe.expectMessage(AccountEntity.Confirmed)
- ref ! AccountEntity.Withdraw(10, probe.ref)
- probe.expectMessage(AccountEntity.Confirmed)
+ eventSourcedTestKit.runCommand[AccountEntity.OperationResult](AccountEntity.CreateAccount(_))
+
+ val result1 = eventSourcedTestKit.runCommand[AccountEntity.OperationResult](AccountEntity.Deposit(100, _))
+ result1.reply shouldBe AccountEntity.Confirmed
+ result1.event shouldBe AccountEntity.Deposited(100)
+ result1.stateOfType[AccountEntity.OpenedAccount].balance shouldBe 100
+
+ val result2 = eventSourcedTestKit.runCommand[AccountEntity.OperationResult](AccountEntity.Withdraw(10, _))
+ result2.reply shouldBe AccountEntity.Confirmed
+ result2.event shouldBe AccountEntity.Withdrawn(10)
+ result2.stateOfType[AccountEntity.OpenedAccount].balance shouldBe 90
}
"reject Withdraw overdraft" in {
- val probe = createTestProbe[AccountEntity.OperationResult]()
- val ref = spawn(AccountEntity("2", PersistenceId("Account", "2")))
- ref ! AccountEntity.CreateAccount(probe.ref)
- probe.expectMessage(AccountEntity.Confirmed)
- ref ! AccountEntity.Deposit(100, probe.ref)
- probe.expectMessage(AccountEntity.Confirmed)
- ref ! AccountEntity.Withdraw(110, probe.ref)
- probe.expectMessageType[AccountEntity.Rejected]
+ eventSourcedTestKit.runCommand[AccountEntity.OperationResult](AccountEntity.CreateAccount(_))
+ eventSourcedTestKit.runCommand[AccountEntity.OperationResult](AccountEntity.Deposit(100, _))
+
+ val result = eventSourcedTestKit.runCommand[AccountEntity.OperationResult](AccountEntity.Withdraw(110, _))
+ result.replyOfType[AccountEntity.Rejected]
+ result.hasNoEvents shouldBe true
}
"handle GetBalance" in {
- val opProbe = createTestProbe[AccountEntity.OperationResult]()
- val ref = spawn(AccountEntity("3", PersistenceId("Account", "3")))
- ref ! AccountEntity.CreateAccount(opProbe.ref)
- opProbe.expectMessage(AccountEntity.Confirmed)
- ref ! AccountEntity.Deposit(100, opProbe.ref)
- opProbe.expectMessage(AccountEntity.Confirmed)
+ eventSourcedTestKit.runCommand[AccountEntity.OperationResult](AccountEntity.CreateAccount(_))
+ eventSourcedTestKit.runCommand[AccountEntity.OperationResult](AccountEntity.Deposit(100, _))
- val getProbe = createTestProbe[AccountEntity.CurrentBalance]()
- ref ! AccountEntity.GetBalance(getProbe.ref)
- getProbe.expectMessage(AccountEntity.CurrentBalance(100))
+ val result = eventSourcedTestKit.runCommand[AccountEntity.CurrentBalance](AccountEntity.GetBalance(_))
+ result.reply.balance shouldBe 100
+ result.hasNoEvents shouldBe true
}
-
- //#test
- //#test-events
- "store events" in {
- val eventProbe = createTestProbe[InmemJournal.Operation]()
- system.eventStream ! EventStream.Subscribe(eventProbe.ref)
-
- val probe = createTestProbe[AccountEntity.OperationResult]()
- val ref = spawn(AccountEntity("4", PersistenceId("Account", "4")))
- ref ! AccountEntity.CreateAccount(probe.ref)
- eventProbe.expectMessageType[InmemJournal.Write].event should ===(AccountEntity.AccountCreated)
-
- ref ! AccountEntity.Deposit(100, probe.ref)
- probe.expectMessage(AccountEntity.Confirmed)
- eventProbe.expectMessageType[InmemJournal.Write].event should ===(AccountEntity.Deposited(100))
- }
- //#test-events
- //#test
}
}
//#test
diff --git a/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleSpec.scala b/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleSpec.scala
index 42a954b749..6574180b95 100644
--- a/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleSpec.scala
+++ b/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleSpec.scala
@@ -79,10 +79,8 @@ class AccountExampleSpec
}
"reject Withdraw overdraft" in {
- // AccountCommand[_] is the command type, but it should also be possible to narrow it to
- // AccountCommand[OperationResult]
val probe = createTestProbe[OperationResult]()
- val ref = ClusterSharding(system).entityRefFor[Command[OperationResult]](AccountEntity.TypeKey, "3")
+ val ref = ClusterSharding(system).entityRefFor[Command](AccountEntity.TypeKey, "3")
ref ! CreateAccount(probe.ref)
probe.expectMessage(Confirmed)
ref ! Deposit(100, probe.ref)
@@ -90,10 +88,12 @@ class AccountExampleSpec
ref ! Withdraw(110, probe.ref)
probe.expectMessageType[Rejected]
+ // Account.Command is the command type, but it should also be possible to narrow it
// ... thus restricting the entity ref from being sent other commands, e.g.:
+ // val ref2 = ClusterSharding(system).entityRefFor[Deposit](AccountEntity.TypeKey, "3")
// val probe2 = createTestProbe[CurrentBalance]()
// val msg = GetBalance(probe2.ref)
- // ref ! msg // type mismatch: GetBalance NOT =:= AccountCommand[OperationResult]
+ // ref2 ! msg // type mismatch: GetBalance NOT =:= Deposit
}
"handle GetBalance" in {
diff --git a/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleWithCommandHandlersInState.scala b/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleWithCommandHandlersInState.scala
index e41ff492de..c5f3772b16 100644
--- a/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleWithCommandHandlersInState.scala
+++ b/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleWithCommandHandlersInState.scala
@@ -24,14 +24,12 @@ object AccountExampleWithCommandHandlersInState {
//#account-entity
object AccountEntity {
// Command
- sealed trait Command[Reply <: CommandReply] extends CborSerializable {
- def replyTo: ActorRef[Reply]
- }
- final case class CreateAccount(replyTo: ActorRef[OperationResult]) extends Command[OperationResult]
- final case class Deposit(amount: BigDecimal, replyTo: ActorRef[OperationResult]) extends Command[OperationResult]
- final case class Withdraw(amount: BigDecimal, replyTo: ActorRef[OperationResult]) extends Command[OperationResult]
- final case class GetBalance(replyTo: ActorRef[CurrentBalance]) extends Command[CurrentBalance]
- final case class CloseAccount(replyTo: ActorRef[OperationResult]) extends Command[OperationResult]
+ sealed trait Command extends CborSerializable
+ final case class CreateAccount(replyTo: ActorRef[OperationResult]) extends Command
+ final case class Deposit(amount: BigDecimal, replyTo: ActorRef[OperationResult]) extends Command
+ final case class Withdraw(amount: BigDecimal, replyTo: ActorRef[OperationResult]) extends Command
+ final case class GetBalance(replyTo: ActorRef[CurrentBalance]) extends Command
+ final case class CloseAccount(replyTo: ActorRef[OperationResult]) extends Command
// Reply
sealed trait CommandReply extends CborSerializable
@@ -54,11 +52,11 @@ object AccountExampleWithCommandHandlersInState {
// State
sealed trait Account extends CborSerializable {
- def applyCommand(cmd: Command[_]): ReplyEffect
+ def applyCommand(cmd: Command): ReplyEffect
def applyEvent(event: Event): Account
}
case object EmptyAccount extends Account {
- override def applyCommand(cmd: Command[_]): ReplyEffect =
+ override def applyCommand(cmd: Command): ReplyEffect =
cmd match {
case CreateAccount(replyTo) =>
Effect.persist(AccountCreated).thenReply(replyTo)(_ => Confirmed)
@@ -76,7 +74,7 @@ object AccountExampleWithCommandHandlersInState {
case class OpenedAccount(balance: BigDecimal) extends Account {
require(balance >= Zero, "Account balance can't be negative")
- override def applyCommand(cmd: Command[_]): ReplyEffect =
+ override def applyCommand(cmd: Command): ReplyEffect =
cmd match {
case Deposit(amount, replyTo) =>
Effect.persist(Deposited(amount)).thenReply(replyTo)(_ => Confirmed)
@@ -115,28 +113,33 @@ object AccountExampleWithCommandHandlersInState {
}
case object ClosedAccount extends Account {
- override def applyCommand(cmd: Command[_]): ReplyEffect =
+ override def applyCommand(cmd: Command): ReplyEffect =
cmd match {
- case c @ (_: Deposit | _: Withdraw) =>
- Effect.reply(c.replyTo)(Rejected("Account is closed"))
+ case c: Deposit =>
+ replyClosed(c.replyTo)
+ case c: Withdraw =>
+ replyClosed(c.replyTo)
case GetBalance(replyTo) =>
Effect.reply(replyTo)(CurrentBalance(Zero))
case CloseAccount(replyTo) =>
- Effect.reply(replyTo)(Rejected("Account is already closed"))
+ replyClosed(replyTo)
case CreateAccount(replyTo) =>
- Effect.reply(replyTo)(Rejected("Account is already created"))
+ replyClosed(replyTo)
}
+ private def replyClosed(replyTo: ActorRef[AccountEntity.OperationResult]): ReplyEffect =
+ Effect.reply(replyTo)(Rejected(s"Account is closed"))
+
override def applyEvent(event: Event): Account =
throw new IllegalStateException(s"unexpected event [$event] in state [ClosedAccount]")
}
// when used with sharding, this TypeKey can be used in `sharding.init` and `sharding.entityRefFor`:
- val TypeKey: EntityTypeKey[Command[_]] =
- EntityTypeKey[Command[_]]("Account")
+ val TypeKey: EntityTypeKey[Command] =
+ EntityTypeKey[Command]("Account")
- def apply(persistenceId: PersistenceId): Behavior[Command[_]] = {
- EventSourcedBehavior.withEnforcedReplies[Command[_], Event, Account](
+ def apply(persistenceId: PersistenceId): Behavior[Command] = {
+ EventSourcedBehavior.withEnforcedReplies[Command, Event, Account](
persistenceId,
EmptyAccount,
(state, cmd) => state.applyCommand(cmd),
diff --git a/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleWithEventHandlersInState.scala b/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleWithEventHandlersInState.scala
index 7c4ca1cf46..dc4c888136 100644
--- a/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleWithEventHandlersInState.scala
+++ b/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleWithEventHandlersInState.scala
@@ -27,17 +27,15 @@ object AccountExampleWithEventHandlersInState {
object AccountEntity {
// Command
//#reply-command
- sealed trait Command[Reply <: CommandReply] extends CborSerializable {
- def replyTo: ActorRef[Reply]
- }
+ sealed trait Command extends CborSerializable
//#reply-command
- final case class CreateAccount(replyTo: ActorRef[OperationResult]) extends Command[OperationResult]
- final case class Deposit(amount: BigDecimal, replyTo: ActorRef[OperationResult]) extends Command[OperationResult]
+ final case class CreateAccount(replyTo: ActorRef[OperationResult]) extends Command
+ final case class Deposit(amount: BigDecimal, replyTo: ActorRef[OperationResult]) extends Command
//#reply-command
- final case class Withdraw(amount: BigDecimal, replyTo: ActorRef[OperationResult]) extends Command[OperationResult]
+ final case class Withdraw(amount: BigDecimal, replyTo: ActorRef[OperationResult]) extends Command
//#reply-command
- final case class GetBalance(replyTo: ActorRef[CurrentBalance]) extends Command[CurrentBalance]
- final case class CloseAccount(replyTo: ActorRef[OperationResult]) extends Command[OperationResult]
+ final case class GetBalance(replyTo: ActorRef[CurrentBalance]) extends Command
+ final case class CloseAccount(replyTo: ActorRef[OperationResult]) extends Command
// Reply
//#reply-command
@@ -89,20 +87,20 @@ object AccountExampleWithEventHandlersInState {
}
// when used with sharding, this TypeKey can be used in `sharding.init` and `sharding.entityRefFor`:
- val TypeKey: EntityTypeKey[Command[_]] =
- EntityTypeKey[Command[_]]("Account")
+ val TypeKey: EntityTypeKey[Command] =
+ EntityTypeKey[Command]("Account")
// Note that after defining command, event and state classes you would probably start here when writing this.
// When filling in the parameters of EventSourcedBehavior.apply you can use IntelliJ alt+Enter > createValue
// to generate the stub with types for the command and event handlers.
//#withEnforcedReplies
- def apply(accountNumber: String, persistenceId: PersistenceId): Behavior[Command[_]] = {
+ def apply(accountNumber: String, persistenceId: PersistenceId): Behavior[Command] = {
EventSourcedBehavior.withEnforcedReplies(persistenceId, EmptyAccount, commandHandler(accountNumber), eventHandler)
}
//#withEnforcedReplies
- private def commandHandler(accountNumber: String): (Account, Command[_]) => ReplyEffect[Event, Account] = {
+ private def commandHandler(accountNumber: String): (Account, Command) => ReplyEffect[Event, Account] = {
(state, cmd) =>
state match {
case EmptyAccount =>
@@ -122,18 +120,26 @@ object AccountExampleWithEventHandlersInState {
case ClosedAccount =>
cmd match {
- case c @ (_: Deposit | _: Withdraw) =>
- Effect.reply(c.replyTo)(Rejected(s"Account $accountNumber is closed"))
+ case c: Deposit =>
+ replyClosed(accountNumber, c.replyTo)
+ case c: Withdraw =>
+ replyClosed(accountNumber, c.replyTo)
case GetBalance(replyTo) =>
Effect.reply(replyTo)(CurrentBalance(Zero))
case CloseAccount(replyTo) =>
- Effect.reply(replyTo)(Rejected(s"Account $accountNumber is already closed"))
+ replyClosed(accountNumber, replyTo)
case CreateAccount(replyTo) =>
- Effect.reply(replyTo)(Rejected(s"Account $accountNumber is already closed"))
+ replyClosed(accountNumber, replyTo)
}
}
}
+ private def replyClosed(
+ accountNumber: String,
+ replyTo: ActorRef[AccountEntity.OperationResult]): ReplyEffect[Event, Account] = {
+ Effect.reply(replyTo)(Rejected(s"Account $accountNumber is closed"))
+ }
+
private val eventHandler: (Account, Event) => Account = { (state, event) =>
state.applyEvent(event)
}
diff --git a/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleWithOptionState.scala b/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleWithOptionState.scala
index 0061f125ff..16197c6023 100644
--- a/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleWithOptionState.scala
+++ b/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleWithOptionState.scala
@@ -24,14 +24,12 @@ object AccountExampleWithOptionState {
//#account-entity
object AccountEntity {
// Command
- sealed trait Command[Reply <: CommandReply] extends CborSerializable {
- def replyTo: ActorRef[Reply]
- }
- final case class CreateAccount(replyTo: ActorRef[OperationResult]) extends Command[OperationResult]
- final case class Deposit(amount: BigDecimal, replyTo: ActorRef[OperationResult]) extends Command[OperationResult]
- final case class Withdraw(amount: BigDecimal, replyTo: ActorRef[OperationResult]) extends Command[OperationResult]
- final case class GetBalance(replyTo: ActorRef[CurrentBalance]) extends Command[CurrentBalance]
- final case class CloseAccount(replyTo: ActorRef[OperationResult]) extends Command[OperationResult]
+ sealed trait Command extends CborSerializable
+ final case class CreateAccount(replyTo: ActorRef[OperationResult]) extends Command
+ final case class Deposit(amount: BigDecimal, replyTo: ActorRef[OperationResult]) extends Command
+ final case class Withdraw(amount: BigDecimal, replyTo: ActorRef[OperationResult]) extends Command
+ final case class GetBalance(replyTo: ActorRef[CurrentBalance]) extends Command
+ final case class CloseAccount(replyTo: ActorRef[OperationResult]) extends Command
// Reply
sealed trait CommandReply extends CborSerializable
@@ -54,13 +52,13 @@ object AccountExampleWithOptionState {
// State
sealed trait Account extends CborSerializable {
- def applyCommand(cmd: Command[_]): ReplyEffect
+ def applyCommand(cmd: Command): ReplyEffect
def applyEvent(event: Event): Account
}
case class OpenedAccount(balance: BigDecimal) extends Account {
require(balance >= Zero, "Account balance can't be negative")
- override def applyCommand(cmd: Command[_]): ReplyEffect =
+ override def applyCommand(cmd: Command): ReplyEffect =
cmd match {
case Deposit(amount, replyTo) =>
Effect.persist(Deposited(amount)).thenReply(replyTo)(_ => Confirmed)
@@ -99,28 +97,33 @@ object AccountExampleWithOptionState {
}
case object ClosedAccount extends Account {
- override def applyCommand(cmd: Command[_]): ReplyEffect =
+ override def applyCommand(cmd: Command): ReplyEffect =
cmd match {
- case c @ (_: Deposit | _: Withdraw) =>
- Effect.reply(c.replyTo)(Rejected("Account is closed"))
+ case c: Deposit =>
+ replyClosed(c.replyTo)
+ case c: Withdraw =>
+ replyClosed(c.replyTo)
case GetBalance(replyTo) =>
Effect.reply(replyTo)(CurrentBalance(Zero))
case CloseAccount(replyTo) =>
- Effect.reply(replyTo)(Rejected("Account is already closed"))
+ replyClosed(replyTo)
case CreateAccount(replyTo) =>
- Effect.reply(replyTo)(Rejected("Account is already created"))
+ replyClosed(replyTo)
}
+ private def replyClosed(replyTo: ActorRef[AccountEntity.OperationResult]): ReplyEffect =
+ Effect.reply(replyTo)(Rejected(s"Account is closed"))
+
override def applyEvent(event: Event): Account =
throw new IllegalStateException(s"unexpected event [$event] in state [ClosedAccount]")
}
// when used with sharding, this TypeKey can be used in `sharding.init` and `sharding.entityRefFor`:
- val TypeKey: EntityTypeKey[Command[_]] =
- EntityTypeKey[Command[_]]("Account")
+ val TypeKey: EntityTypeKey[Command] =
+ EntityTypeKey[Command]("Account")
- def apply(persistenceId: PersistenceId): Behavior[Command[_]] = {
- EventSourcedBehavior.withEnforcedReplies[Command[_], Event, Option[Account]](
+ def apply(persistenceId: PersistenceId): Behavior[Command] = {
+ EventSourcedBehavior.withEnforcedReplies[Command, Event, Option[Account]](
persistenceId,
None,
(state, cmd) =>
@@ -135,7 +138,7 @@ object AccountExampleWithOptionState {
})
}
- def onFirstCommand(cmd: Command[_]): ReplyEffect = {
+ def onFirstCommand(cmd: Command): ReplyEffect = {
cmd match {
case CreateAccount(replyTo) =>
Effect.persist(AccountCreated).thenReply(replyTo)(_ => Confirmed)
diff --git a/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/ShardingCompileOnlySpec.scala b/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/ShardingCompileOnlySpec.scala
index c6c8e8e52e..7af737140e 100644
--- a/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/ShardingCompileOnlySpec.scala
+++ b/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/ShardingCompileOnlySpec.scala
@@ -14,6 +14,9 @@ import com.github.ghik.silencer.silent
import docs.akka.persistence.typed.BlogPostEntity
import docs.akka.persistence.typed.BlogPostEntity.Command
+import akka.cluster.sharding.typed.scaladsl.ClusterSharding
+import akka.cluster.sharding.typed.scaladsl.EntityTypeKey
+
@silent
object ShardingCompileOnlySpec {
@@ -169,4 +172,42 @@ object ShardingCompileOnlySpec {
//#sharded-response
}
+ object ShardRegionStateQuery {
+
+ object Counter {
+ val TypeKey = EntityTypeKey[Basics.Counter.Command]("Counter")
+ }
+
+ val replyMessageAdapter: ActorRef[akka.cluster.sharding.ShardRegion.CurrentShardRegionState] = ???
+
+ //#get-shard-region-state
+ import akka.cluster.sharding.typed.GetShardRegionState
+ import akka.cluster.sharding.ShardRegion.CurrentShardRegionState
+
+ val replyTo: ActorRef[CurrentShardRegionState] = replyMessageAdapter
+
+ ClusterSharding(system).shardState ! GetShardRegionState(Counter.TypeKey, replyTo)
+ //#get-shard-region-state
+ }
+
+ object ClusterShardingStatsQuery {
+
+ object Counter {
+ val TypeKey = EntityTypeKey[Basics.Counter.Command]("Counter")
+ }
+
+ val replyMessageAdapter: ActorRef[akka.cluster.sharding.ShardRegion.ClusterShardingStats] = ???
+
+ //#get-cluster-sharding-stats
+ import akka.cluster.sharding.typed.GetClusterShardingStats
+ import akka.cluster.sharding.ShardRegion.ClusterShardingStats
+ import scala.concurrent.duration._
+
+ val replyTo: ActorRef[ClusterShardingStats] = replyMessageAdapter
+ val timeout: FiniteDuration = 5.seconds
+
+ ClusterSharding(system).shardState ! GetClusterShardingStats(Counter.TypeKey, timeout, replyTo)
+ //#get-cluster-sharding-stats
+ }
+
}
diff --git a/akka-cluster-sharding/src/main/mima-filters/2.6.5.backwards.excludes/shard-allocation-client.excludes b/akka-cluster-sharding/src/main/mima-filters/2.6.5.backwards.excludes/shard-allocation-client.excludes
new file mode 100644
index 0000000000..ca2bcab05a
--- /dev/null
+++ b/akka-cluster-sharding/src/main/mima-filters/2.6.5.backwards.excludes/shard-allocation-client.excludes
@@ -0,0 +1,4 @@
+# Add methods to trait not for user extension
+ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.cluster.sharding.external.javadsl.ExternalShardAllocationClient.setShardLocations")
+ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.cluster.sharding.external.scaladsl.ExternalShardAllocationClient.updateShardLocations")
+
diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ClusterSharding.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ClusterSharding.scala
index 9955cb7311..c9be7625d0 100755
--- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ClusterSharding.scala
+++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ClusterSharding.scala
@@ -11,7 +11,7 @@ import java.util.concurrent.ConcurrentHashMap
import scala.collection.immutable
import scala.concurrent.Await
import scala.util.control.NonFatal
-import akka.util.ccompat.JavaConverters._
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.ActorSystem
@@ -40,6 +40,7 @@ import akka.event.Logging
import akka.pattern.BackoffOpts
import akka.pattern.ask
import akka.util.ByteString
+import akka.util.ccompat.JavaConverters._
/**
* This extension provides sharding functionality of actors in a cluster.
@@ -185,7 +186,7 @@ class ClusterSharding(system: ExtendedActorSystem) extends Extension {
val guardianName: String =
system.settings.config.getString("akka.cluster.sharding.guardian-name")
val dispatcher = system.settings.config.getString("akka.cluster.sharding.use-dispatcher")
- system.systemActorOf(Props[ClusterShardingGuardian].withDispatcher(dispatcher), guardianName)
+ system.systemActorOf(Props[ClusterShardingGuardian]().withDispatcher(dispatcher), guardianName)
}
/**
diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ClusterShardingSettings.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ClusterShardingSettings.scala
index d19c7cc7f3..3921dc7a52 100644
--- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ClusterShardingSettings.scala
+++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ClusterShardingSettings.scala
@@ -4,8 +4,10 @@
package akka.cluster.sharding
-import scala.concurrent.duration.FiniteDuration
import scala.concurrent.duration._
+import scala.concurrent.duration.FiniteDuration
+
+import com.typesafe.config.Config
import akka.actor.ActorSystem
import akka.actor.NoSerializationVerificationNeeded
@@ -14,7 +16,6 @@ import akka.cluster.Cluster
import akka.cluster.singleton.ClusterSingletonManagerSettings
import akka.coordination.lease.LeaseUsageSettings
import akka.util.JavaDurationConverters._
-import com.typesafe.config.Config
object ClusterShardingSettings {
diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/JoinConfigCompatCheckSharding.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/JoinConfigCompatCheckSharding.scala
index fdffdbdc6d..1bc3e02901 100644
--- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/JoinConfigCompatCheckSharding.scala
+++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/JoinConfigCompatCheckSharding.scala
@@ -4,10 +4,12 @@
package akka.cluster.sharding
+import scala.collection.{ immutable => im }
+
+import com.typesafe.config.Config
+
import akka.annotation.InternalApi
import akka.cluster.{ ConfigValidation, JoinConfigCompatChecker }
-import com.typesafe.config.Config
-import scala.collection.{ immutable => im }
/**
* INTERNAL API
diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/RemoveInternalClusterShardingData.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/RemoveInternalClusterShardingData.scala
index c0946dee45..0525381989 100644
--- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/RemoveInternalClusterShardingData.scala
+++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/RemoveInternalClusterShardingData.scala
@@ -91,7 +91,7 @@ object RemoveInternalClusterShardingData {
if (journalPluginId == "") system.settings.config.getString("akka.persistence.journal.plugin")
else journalPluginId
if (resolvedJournalPluginId == "akka.persistence.journal.leveldb-shared") {
- val store = system.actorOf(Props[SharedLeveldbStore], "store")
+ val store = system.actorOf(Props[SharedLeveldbStore](), "store")
SharedLeveldbJournal.setStore(store, system)
}
diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/Shard.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/Shard.scala
index dd6f5bcbd9..a703952319 100644
--- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/Shard.scala
+++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/Shard.scala
@@ -78,7 +78,7 @@ private[akka] object Shard {
final case class LeaseAcquireResult(acquired: Boolean, reason: Option[Throwable]) extends DeadLetterSuppression
final case class LeaseLost(reason: Option[Throwable]) extends DeadLetterSuppression
- final case object LeaseRetry extends DeadLetterSuppression
+ case object LeaseRetry extends DeadLetterSuppression
private val LeaseRetryTimer = "lease-retry"
def props(
@@ -253,9 +253,10 @@ private[akka] class Shard(
import ShardRegion.Passivate
import ShardRegion.ShardInitialized
import ShardRegion.handOffStopperProps
+ import settings.tuningParameters._
+
import akka.cluster.sharding.ShardCoordinator.Internal.CoordinatorMessage
import akka.cluster.sharding.ShardRegion.ShardRegionCommand
- import settings.tuningParameters._
private val rememberEntitiesStore: Option[ActorRef] =
rememberEntitiesProvider.map { provider =>
diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardCoordinator.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardCoordinator.scala
index ac715ce135..a90d0cf96e 100644
--- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardCoordinator.scala
+++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardCoordinator.scala
@@ -8,19 +8,15 @@ import scala.collection.immutable
import scala.concurrent.Future
import scala.concurrent.duration._
import scala.util.Success
+import com.github.ghik.silencer.silent
import akka.actor._
import akka.actor.DeadLetterSuppression
import akka.annotation.InternalApi
import akka.cluster.Cluster
-import akka.cluster.ClusterEvent._
-import akka.cluster.ddata.LWWRegister
-import akka.cluster.ddata.LWWRegisterKey
-import akka.cluster.ddata.Replicator._
-import akka.dispatch.ExecutionContexts
-import akka.pattern.{ pipe, AskTimeoutException }
-import akka.persistence._
import akka.cluster.ClusterEvent
-import akka.cluster.ddata.SelfUniqueAddress
+import akka.cluster.ClusterEvent.{ ClusterShuttingDown, InitialStateAsEvents }
+import akka.cluster.ddata.Replicator._
+import akka.cluster.ddata.{ LWWRegister, LWWRegisterKey, SelfUniqueAddress }
import akka.cluster.sharding.DDataShardCoordinator.RememberEntitiesLoadTimeout
import akka.cluster.sharding.ShardRegion.ShardId
import akka.cluster.sharding.internal.EventSourcedRememberShards.MigrationMarker
@@ -29,11 +25,13 @@ import akka.cluster.sharding.internal.{
RememberEntitiesCoordinatorStore,
RememberEntitiesProvider
}
+import akka.dispatch.ExecutionContexts
import akka.event.BusLogging
import akka.event.Logging
+import akka.pattern.{ pipe, AskTimeoutException }
+import akka.persistence._
import akka.util.PrettyDuration._
import akka.util.Timeout
-import com.github.ghik.silencer.silent
/**
* @see [[ClusterSharding$ ClusterSharding extension]]
@@ -1139,6 +1137,7 @@ private[akka] class DDataShardCoordinator(
import DDataShardCoordinator._
import ShardCoordinator.Internal._
+
import akka.cluster.ddata.Replicator.Update
private val stateReadConsistency = settings.tuningParameters.coordinatorStateReadMajorityPlus match {
@@ -1150,8 +1149,8 @@ private[akka] class DDataShardCoordinator(
case additional => WriteMajorityPlus(settings.tuningParameters.waitingForStateTimeout, majorityMinCap, additional)
}
- implicit val node = Cluster(context.system)
- private implicit val selfUniqueAddress = SelfUniqueAddress(node.selfUniqueAddress)
+ implicit val node: Cluster = Cluster(context.system)
+ private implicit val selfUniqueAddress: SelfUniqueAddress = SelfUniqueAddress(node.selfUniqueAddress)
private val CoordinatorStateKey = LWWRegisterKey[State](s"${typeName}CoordinatorState")
private val initEmptyState = State.empty.withRememberEntities(settings.rememberEntities)
diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardRegion.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardRegion.scala
index 3f70540cd4..302c004206 100644
--- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardRegion.scala
+++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardRegion.scala
@@ -6,6 +6,14 @@ package akka.cluster.sharding
import java.net.URLEncoder
+import scala.annotation.tailrec
+import scala.collection.immutable
+import scala.concurrent.{ Future, Promise }
+import scala.concurrent.duration._
+import scala.reflect.ClassTag
+import scala.runtime.AbstractFunction1
+import scala.util.{ Failure, Success }
+
import akka.Done
import akka.actor._
import akka.annotation.InternalApi
@@ -207,7 +215,7 @@ object ShardRegion {
* the `ShardRegion` and then the `ShardRegion` actor will be stopped. You can `watch`
* it to know when it is completed.
*/
- @SerialVersionUID(1L) final case object GracefulShutdown extends ShardRegionCommand
+ @SerialVersionUID(1L) case object GracefulShutdown extends ShardRegionCommand
/**
* We must be sure that a shard is initialized before to start send messages to it.
@@ -227,10 +235,11 @@ object ShardRegion {
/**
* Send this message to the `ShardRegion` actor to request for [[CurrentRegions]],
* which contains the addresses of all registered regions.
+ *
* Intended for testing purpose to see when cluster sharding is "ready" or to monitor
* the state of the shard regions.
*/
- @SerialVersionUID(1L) final case object GetCurrentRegions extends ShardRegionQuery with ClusterShardingSerializable
+ @SerialVersionUID(1L) case object GetCurrentRegions extends ShardRegionQuery with ClusterShardingSerializable
/**
* Java API:
@@ -285,6 +294,7 @@ object ShardRegion {
* Send this message to the `ShardRegion` actor to request for [[ShardRegionStats]],
* which contains statistics about the currently running sharded entities in the
* entire region.
+ *
* Intended for testing purpose to see when cluster sharding is "ready" or to monitor
* the state of the shard regions.
*
diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/ExternalShardAllocation.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/ExternalShardAllocation.scala
index b4caafd471..9fb95e3096 100644
--- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/ExternalShardAllocation.scala
+++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/ExternalShardAllocation.scala
@@ -5,16 +5,15 @@
package akka.cluster.sharding.external
import java.util.concurrent.ConcurrentHashMap
+import java.util.function.{ Function => JFunction }
+import akka.actor.ClassicActorSystemProvider
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
-import akka.cluster.sharding.external.internal.ExternalShardAllocationClientImpl
-import java.util.function.{ Function => JFunction }
-
-import akka.actor.ClassicActorSystemProvider
import akka.annotation.ApiMayChange
+import akka.cluster.sharding.external.internal.ExternalShardAllocationClientImpl
/**
* API May Change
diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/ExternalShardAllocationStrategy.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/ExternalShardAllocationStrategy.scala
index 385a6631b1..7129de9a88 100644
--- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/ExternalShardAllocationStrategy.scala
+++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/ExternalShardAllocationStrategy.scala
@@ -4,6 +4,10 @@
package akka.cluster.sharding.external
+import scala.collection.immutable
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorRef
@@ -26,17 +30,13 @@ import akka.event.Logging
import akka.pattern.AskTimeoutException
import akka.util.Timeout
-import scala.collection.immutable
-import scala.concurrent.Future
-import scala.concurrent.duration._
-
object ExternalShardAllocationStrategy {
type ShardRegion = ActorRef
// local only messages
private[akka] final case class GetShardLocation(shard: ShardId)
- private[akka] final case object GetShardLocations
+ private[akka] case object GetShardLocations
private[akka] final case class GetShardLocationsResponse(desiredAllocations: Map[ShardId, Address])
private[akka] final case class GetShardLocationResponse(address: Option[Address])
@@ -91,9 +91,10 @@ class ExternalShardAllocationStrategy(systemProvider: ClassicActorSystemProvider
private val system = systemProvider.classicSystem
import ExternalShardAllocationStrategy._
- import akka.pattern.ask
import system.dispatcher
+ import akka.pattern.ask
+
private val log = Logging(system, classOf[ExternalShardAllocationStrategy])
private var shardState: ActorRef = _
diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/internal/ExternalShardAllocationClientImpl.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/internal/ExternalShardAllocationClientImpl.scala
index 08c6bbf569..b6646697b5 100644
--- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/internal/ExternalShardAllocationClientImpl.scala
+++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/internal/ExternalShardAllocationClientImpl.scala
@@ -6,6 +6,9 @@ package akka.cluster.sharding.external.internal
import java.util.concurrent.CompletionStage
+import scala.compat.java8.FutureConverters._
+import scala.concurrent.Future
+
import akka.Done
import akka.actor.ActorRef
import akka.actor.ActorSystem
@@ -29,14 +32,13 @@ import akka.cluster.sharding.external.ClientTimeoutException
import akka.cluster.sharding.external.ExternalShardAllocationStrategy
import akka.cluster.sharding.external.ExternalShardAllocationStrategy.ShardLocation
import akka.cluster.sharding.external.ShardLocations
+import akka.dispatch.MessageDispatcher
import akka.event.Logging
-import akka.util.Timeout
-import akka.util.PrettyDuration._
import akka.pattern.ask
-
-import scala.concurrent.Future
-import scala.compat.java8.FutureConverters._
import akka.util.JavaDurationConverters._
+import akka.util.PrettyDuration._
+import akka.util.Timeout
+import akka.util.ccompat.JavaConverters._
/**
* INTERNAL API
@@ -55,8 +57,8 @@ final private[external] class ExternalShardAllocationClientImpl(system: ActorSys
system.settings.config
.getDuration("akka.cluster.sharding.external-shard-allocation-strategy.client-timeout")
.asScala
- private implicit val askTimeout = Timeout(timeout * 2)
- private implicit val ec = system.dispatchers.internalDispatcher
+ private implicit val askTimeout: Timeout = Timeout(timeout * 2)
+ private implicit val ec: MessageDispatcher = system.dispatchers.internalDispatcher
private val Key = ExternalShardAllocationStrategy.ddataKey(typeName)
@@ -91,4 +93,21 @@ final private[external] class ExternalShardAllocationClientImpl(system: ActorSys
}
override def getShardLocations(): CompletionStage[ShardLocations] = shardLocations().toJava
+
+ override def updateShardLocations(locations: Map[ShardId, Address]): Future[Done] = {
+ log.debug("updateShardLocations {} for {}", locations, Key)
+ (replicator ? Update(Key, LWWMap.empty[ShardId, String], WriteLocal, None) { existing =>
+ locations.foldLeft(existing) {
+ case (acc, (shardId, address)) => acc.put(self, shardId, address.toString)
+ }
+ }).flatMap {
+ case UpdateSuccess(_, _) => Future.successful(Done)
+ case UpdateTimeout =>
+ Future.failed(new ClientTimeoutException(s"Unable to update shard location after ${timeout.duration.pretty}"))
+ }
+ }
+
+ override def setShardLocations(locations: java.util.Map[ShardId, Address]): CompletionStage[Done] = {
+ updateShardLocations(locations.asScala.toMap).toJava
+ }
}
diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/javadsl/ExternalShardAllocationClient.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/javadsl/ExternalShardAllocationClient.scala
index fd561757c3..ee8d73360f 100644
--- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/javadsl/ExternalShardAllocationClient.scala
+++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/javadsl/ExternalShardAllocationClient.scala
@@ -28,10 +28,21 @@ trait ExternalShardAllocationClient {
*
* @param shard The shard identifier
* @param location Location (akka node) to allocate the shard to
- * @return Confirmation that the update has been propagated to a majority of cluster nodes
+ * @return Conformation that the update has been written to the local node
*/
def setShardLocation(shard: ShardId, location: Address): CompletionStage[Done]
+ /**
+ * Update all of the provided ShardLocations.
+ * The [[Address]] should match one of the nodes in the cluster. If the node has not joined
+ * the cluster yet it will be moved to that node after the first cluster
+ * sharding rebalance it does.
+ *
+ * @param locations to update
+ * @return Confirmation that the update has been written to the local node
+ */
+ def setShardLocations(locations: java.util.Map[ShardId, Address]): CompletionStage[Done]
+
/**
* Get all the current shard locations that have been set via setShardLocation
*/
diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/scaladsl/ExternalShardAllocationClient.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/scaladsl/ExternalShardAllocationClient.scala
index 8f096f6a73..694e03c918 100644
--- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/scaladsl/ExternalShardAllocationClient.scala
+++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/external/scaladsl/ExternalShardAllocationClient.scala
@@ -4,14 +4,14 @@
package akka.cluster.sharding.external.scaladsl
+import scala.concurrent.Future
+
import akka.Done
import akka.actor.Address
import akka.annotation.ApiMayChange
import akka.cluster.sharding.ShardRegion.ShardId
import akka.cluster.sharding.external.ShardLocations
-import scala.concurrent.Future
-
/**
* API May Change
*
@@ -24,7 +24,7 @@ trait ExternalShardAllocationClient {
* Update the given shard's location. The [[Address]] should
* match one of the nodes in the cluster. If the node has not joined
* the cluster yet it will be moved to that node after the first cluster
- * sharding rebalance.
+ * sharding rebalance it does.
*
* @param shard The shard identifier
* @param location Location (akka node) to allocate the shard to
@@ -32,6 +32,17 @@ trait ExternalShardAllocationClient {
*/
def updateShardLocation(shard: ShardId, location: Address): Future[Done]
+ /**
+ * Update all of the provided ShardLocations.
+ * The [[Address]] should match one of the nodes in the cluster. If the node has not joined
+ * the cluster yet it will be moved to that node after the first cluster
+ * sharding rebalance it does.
+ *
+ * @param locations to update
+ * @return Confirmation that the update has been propagates to a majority of cluster nodes
+ */
+ def updateShardLocations(locations: Map[ShardId, Address]): Future[Done]
+
/**
* Get all the current shard locations that have been set via updateShardLocation
*/
diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/protobuf/ClusterShardingMessageSerializer.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/protobuf/ClusterShardingMessageSerializer.scala
index 0e60819549..a094cbdc7f 100644
--- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/protobuf/ClusterShardingMessageSerializer.scala
+++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/protobuf/ClusterShardingMessageSerializer.scala
@@ -5,24 +5,28 @@
package akka.cluster.sharding.protobuf
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream }
+import java.io.NotSerializableException
import java.util.zip.GZIPInputStream
import java.util.zip.GZIPOutputStream
import scala.annotation.tailrec
import scala.concurrent.duration._
import akka.util.ccompat.JavaConverters._
-
import scala.collection.immutable
+
import akka.actor.ActorRef
+import akka.actor.Address
import akka.actor.ExtendedActorSystem
import akka.cluster.sharding.Shard
import akka.cluster.sharding.ShardCoordinator
+import akka.cluster.sharding.ShardRegion._
import akka.cluster.sharding.protobuf.msg.{ ClusterShardingMessages => sm }
import akka.cluster.sharding.internal.EventSourcedRememberShards.{ MigrationMarker, State => RememberShardsState }
+import akka.cluster.sharding.protobuf.msg.ClusterShardingMessages
+import akka.protobufv3.internal.MessageLite
import akka.serialization.BaseSerializer
import akka.serialization.Serialization
import akka.serialization.SerializerWithStringManifest
-import akka.protobufv3.internal.MessageLite
import akka.util.ccompat._
import java.io.NotSerializableException
@@ -38,10 +42,10 @@ import akka.cluster.sharding.protobuf.msg.ClusterShardingMessages
private[akka] class ClusterShardingMessageSerializer(val system: ExtendedActorSystem)
extends SerializerWithStringManifest
with BaseSerializer {
- import ShardCoordinator.Internal._
import Shard.{ CurrentShardState, GetCurrentShardState }
import Shard.{ GetShardStats, ShardStats }
import akka.cluster.sharding.internal.EventSourcedRememberEntitiesStore.{ State => EntityState, EntityStopped }
+ import ShardCoordinator.Internal._
private final val BufferSize = 1024 * 4
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sbr/GlobalRegistry.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sbr/GlobalRegistry.scala
new file mode 100644
index 0000000000..5e834b71b5
--- /dev/null
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sbr/GlobalRegistry.scala
@@ -0,0 +1,97 @@
+/*
+ * Copyright (C) 2015-2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import scala.concurrent.duration._
+
+import akka.actor.Actor
+import akka.actor.ActorLogging
+import akka.actor.ActorRef
+import akka.actor.Address
+import akka.actor.Props
+import akka.cluster.Cluster
+import akka.cluster.sharding.ShardRegion
+import akka.serialization.jackson.CborSerializable
+
+object GlobalRegistry {
+ final case class Register(key: String, address: Address) extends CborSerializable
+ final case class Unregister(key: String, address: Address) extends CborSerializable
+ final case class DoubleRegister(key: String, msg: String) extends CborSerializable
+
+ def props(probe: ActorRef, onlyErrors: Boolean): Props =
+ Props(new GlobalRegistry(probe, onlyErrors))
+
+ object SingletonActor {
+ def props(registry: ActorRef): Props =
+ Props(new SingletonActor(registry))
+
+ val extractEntityId: ShardRegion.ExtractEntityId = {
+ case id: Int => (id.toString, id)
+ }
+
+ val extractShardId: ShardRegion.ExtractShardId = msg =>
+ msg match {
+ case id: Int => (id % 10).toString
+ }
+ }
+
+ class SingletonActor(registry: ActorRef) extends Actor with ActorLogging {
+ val key = self.path.toStringWithoutAddress + "-" + Cluster(context.system).selfDataCenter
+
+ override def preStart(): Unit = {
+ log.info("Starting")
+ registry ! Register(key, Cluster(context.system).selfAddress)
+ }
+
+ override def preRestart(reason: Throwable, message: Option[Any]): Unit = {
+ // don't call postStop
+ }
+
+ override def postStop(): Unit = {
+ log.info("Stopping")
+ registry ! Unregister(key, Cluster(context.system).selfAddress)
+ }
+
+ override def receive = {
+ case i: Int => sender() ! i
+ }
+ }
+}
+
+class GlobalRegistry(probe: ActorRef, onlyErrors: Boolean) extends Actor with ActorLogging {
+ import GlobalRegistry._
+
+ var registry = Map.empty[String, Address]
+ var unregisterTimestamp = Map.empty[String, Long]
+
+ override def receive = {
+ case r @ Register(key, address) =>
+ log.info("{}", r)
+ if (registry.contains(key)) {
+ val errMsg = s"trying to register $address, but ${registry(key)} was already registered for $key"
+ log.error(errMsg)
+ probe ! DoubleRegister(key, errMsg)
+ } else {
+ unregisterTimestamp.get(key).foreach { t =>
+ log.info("Unregister/register margin for [{}] was [{}] ms", key, (System.nanoTime() - t).nanos.toMillis)
+ }
+ registry += key -> address
+ if (!onlyErrors) probe ! r
+ }
+
+ case u @ Unregister(key, address) =>
+ log.info("{}", u)
+ if (!registry.contains(key))
+ probe ! s"$key was not registered"
+ else if (registry(key) != address)
+ probe ! s"${registry(key)} instead of $address was registered for $key"
+ else {
+ registry -= key
+ unregisterTimestamp += key -> System.nanoTime()
+ if (!onlyErrors) probe ! u
+ }
+ }
+
+}
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sbr/GremlinController.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sbr/GremlinController.scala
new file mode 100644
index 0000000000..c76e38359d
--- /dev/null
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sbr/GremlinController.scala
@@ -0,0 +1,60 @@
+/*
+ * Copyright (C) 2019-2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import akka.actor.Actor
+import akka.actor.ActorLogging
+import akka.actor.ActorRef
+import akka.actor.Address
+import akka.actor.ExtendedActorSystem
+import akka.actor.Props
+import akka.cluster.Cluster
+import akka.pattern.pipe
+import akka.remote.RemoteActorRefProvider
+import akka.remote.transport.ThrottlerTransportAdapter.Blackhole
+import akka.remote.transport.ThrottlerTransportAdapter.Direction
+import akka.remote.transport.ThrottlerTransportAdapter.SetThrottle
+import akka.remote.transport.ThrottlerTransportAdapter.Unthrottled
+import akka.serialization.jackson.CborSerializable
+
+object GremlinController {
+ final case class BlackholeNode(target: Address) extends CborSerializable
+ final case class PassThroughNode(target: Address) extends CborSerializable
+ case object GetAddress extends CborSerializable
+
+ def props: Props =
+ Props(new GremlinController)
+}
+
+class GremlinController extends Actor with ActorLogging {
+ import context.dispatcher
+
+ import GremlinController._
+ val transport =
+ context.system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].transport
+ val selfAddress = Cluster(context.system).selfAddress
+
+ override def receive = {
+ case GetAddress =>
+ sender() ! selfAddress
+ case BlackholeNode(target) =>
+ log.debug("Blackhole {} <-> {}", selfAddress, target)
+ transport.managementCommand(SetThrottle(target, Direction.Both, Blackhole)).pipeTo(sender())
+ case PassThroughNode(target) =>
+ log.debug("PassThrough {} <-> {}", selfAddress, target)
+ transport.managementCommand(SetThrottle(target, Direction.Both, Unthrottled)).pipeTo(sender())
+ }
+}
+
+object GremlinControllerProxy {
+ def props(target: ActorRef): Props =
+ Props(new GremlinControllerProxy(target))
+}
+
+class GremlinControllerProxy(target: ActorRef) extends Actor {
+ override def receive = {
+ case msg => target.forward(msg)
+ }
+}
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sbr/RandomizedBrainResolverIntegrationSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sbr/RandomizedBrainResolverIntegrationSpec.scala
new file mode 100644
index 0000000000..f03e3ec557
--- /dev/null
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sbr/RandomizedBrainResolverIntegrationSpec.scala
@@ -0,0 +1,409 @@
+/*
+ * Copyright (C) 2015-2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.util.Random
+
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.BeforeAndAfterEach
+
+import akka.actor._
+import akka.cluster.Cluster
+import akka.cluster.MemberStatus
+import akka.cluster.MultiNodeClusterSpec
+import akka.cluster.sharding.ClusterSharding
+import akka.cluster.sharding.ClusterShardingSettings
+import akka.cluster.singleton.ClusterSingletonManager
+import akka.cluster.singleton.ClusterSingletonManagerSettings
+import akka.pattern.ask
+import akka.remote.testconductor.RoleName
+import akka.remote.testkit.MultiNodeConfig
+import akka.remote.testkit.MultiNodeSpec
+import akka.testkit.ImplicitSender
+import akka.testkit.LongRunningTest
+import akka.testkit.TestKit
+import akka.testkit.TestProbe
+import akka.util.Timeout
+
+/*
+ * Depends on akka private classes so needs to be in this package
+ */
+object RandomizedSplitBrainResolverIntegrationSpec extends MultiNodeConfig {
+ val node1 = role("node1")
+ val node2 = role("node2")
+ val node3 = role("node3")
+ val node4 = role("node4")
+ val node5 = role("node5")
+ val node6 = role("node6")
+ val node7 = role("node7")
+ val node8 = role("node8")
+ val node9 = role("node9")
+
+ commonConfig(ConfigFactory.parseString(s"""
+ akka {
+ loglevel = INFO
+ cluster {
+ downing-provider-class = "akka.cluster.sbr.SplitBrainResolverProvider"
+ split-brain-resolver {
+ stable-after = 10s
+
+ active-strategy = lease-majority
+ lease-majority {
+ lease-implementation = test-lease
+ }
+ }
+
+ #failure-detector.acceptable-heartbeat-pause = 10s
+
+ # speedup timeout
+ sharding.handoff-timeout = 10 s
+
+ # this is starting singleton more aggressively than default (15)
+ singleton.min-number-of-hand-over-retries = 10
+ }
+ actor.provider = cluster
+ }
+
+ test-lease {
+ lease-class = akka.cluster.sbr.SbrTestLeaseActorClient
+ heartbeat-interval = 1s
+ heartbeat-timeout = 120s
+ lease-operation-timeout = 3s
+ }
+
+ test.random-seed = ${System.currentTimeMillis()}
+
+ akka.testconductor.barrier-timeout = 120 s
+ akka.cluster.run-coordinated-shutdown-when-down = off
+ """))
+
+ testTransport(on = true)
+
+}
+
+class RandomizedSplitBrainResolverIntegrationSpecMultiJvmNode1 extends RandomizedSplitBrainResolverIntegrationSpec
+class RandomizedSplitBrainResolverIntegrationSpecMultiJvmNode2 extends RandomizedSplitBrainResolverIntegrationSpec
+class RandomizedSplitBrainResolverIntegrationSpecMultiJvmNode3 extends RandomizedSplitBrainResolverIntegrationSpec
+class RandomizedSplitBrainResolverIntegrationSpecMultiJvmNode4 extends RandomizedSplitBrainResolverIntegrationSpec
+class RandomizedSplitBrainResolverIntegrationSpecMultiJvmNode5 extends RandomizedSplitBrainResolverIntegrationSpec
+class RandomizedSplitBrainResolverIntegrationSpecMultiJvmNode6 extends RandomizedSplitBrainResolverIntegrationSpec
+class RandomizedSplitBrainResolverIntegrationSpecMultiJvmNode7 extends RandomizedSplitBrainResolverIntegrationSpec
+class RandomizedSplitBrainResolverIntegrationSpecMultiJvmNode8 extends RandomizedSplitBrainResolverIntegrationSpec
+class RandomizedSplitBrainResolverIntegrationSpecMultiJvmNode9 extends RandomizedSplitBrainResolverIntegrationSpec
+
+class RandomizedSplitBrainResolverIntegrationSpec
+ extends MultiNodeSpec(RandomizedSplitBrainResolverIntegrationSpec)
+ with MultiNodeClusterSpec
+ with ImplicitSender
+ with BeforeAndAfterEach {
+ import GlobalRegistry._
+ import GremlinController._
+ import RandomizedSplitBrainResolverIntegrationSpec._
+
+ // counter for unique naming for each test
+ var c = 0
+ // to be shutdown in afterEach
+ var disposableSys: DisposableSys = _
+
+ override def expectedTestDuration = 3.minutes
+
+ object DisposableSys {
+ def apply(scenario: Scenario): DisposableSys = {
+ disposableSys = new DisposableSys(scenario)
+ disposableSys
+ }
+ }
+
+ override def afterEach(): Unit = {
+ if (disposableSys ne null)
+ disposableSys.shutdownSys()
+ }
+
+ class DisposableSys(scenario: Scenario) {
+
+ c += 1
+
+ val sys: ActorSystem = {
+
+ val sys = ActorSystem(system.name + "-" + c, system.settings.config)
+ val gremlinController = sys.actorOf(GremlinController.props, "gremlinController")
+ system.actorOf(GremlinControllerProxy.props(gremlinController), s"gremlinControllerProxy-$c")
+ sys
+ }
+
+ val singletonProbe = TestProbe()
+ val shardingProbe = TestProbe()
+ runOn(node1) {
+ system.actorOf(GlobalRegistry.props(singletonProbe.ref, true), s"singletonRegistry-$c")
+ system.actorOf(GlobalRegistry.props(shardingProbe.ref, true), s"shardingRegistry-$c")
+ if (scenario.usingLease)
+ system.actorOf(SbrTestLeaseActor.props, s"lease-${sys.name}")
+ }
+ enterBarrier("registry-started")
+
+ system.actorSelection(node(node1) / "user" / s"singletonRegistry-$c") ! Identify(None)
+ val singletonRegistry: ActorRef = expectMsgType[ActorIdentity].ref.get
+ system.actorSelection(node(node1) / "user" / s"shardingRegistry-$c") ! Identify(None)
+ val shardingRegistry: ActorRef = expectMsgType[ActorIdentity].ref.get
+
+ if (scenario.usingLease) {
+ system.actorSelection(node(node1) / "user" / s"lease-${sys.name}") ! Identify(None)
+ val leaseRef: ActorRef = expectMsgType[ActorIdentity].ref.get
+ SbrTestLeaseActorClientExt(sys).getActorLeaseClient().setActorLeaseRef(leaseRef)
+ }
+ enterBarrier("registry-located")
+
+ lazy val region = ClusterSharding(sys).shardRegion(s"Entity-$c")
+
+ def shutdownSys(): Unit = {
+ TestKit.shutdownActorSystem(sys, 10.seconds, verifySystemShutdown = true)
+ }
+
+ def gremlinControllerProxy(at: RoleName): ActorRef = {
+ system.actorSelection(node(at) / "user" / s"gremlinControllerProxy-$c") ! Identify(None)
+ expectMsgType[ActorIdentity].ref.get
+ }
+
+ def sysAddress(at: RoleName): Address = {
+ implicit val timeout = Timeout(3.seconds)
+ Await.result((gremlinControllerProxy(at) ? GetAddress).mapTo[Address], timeout.duration)
+ }
+
+ def blackhole(from: RoleName, to: RoleName): Unit = {
+ implicit val timeout = Timeout(3.seconds)
+ import system.dispatcher
+ val f = for {
+ target <- (gremlinControllerProxy(to) ? GetAddress).mapTo[Address]
+ done <- gremlinControllerProxy(from) ? BlackholeNode(target)
+ } yield done
+ Await.ready(f, timeout.duration * 2)
+ log.info("Blackhole {} <-> {}", from.name, to.name)
+ }
+
+ def passThrough(from: RoleName, to: RoleName): Unit = {
+ implicit val timeout = Timeout(3.seconds)
+ import system.dispatcher
+ val f = for {
+ target <- (gremlinControllerProxy(to) ? GetAddress).mapTo[Address]
+ done <- gremlinControllerProxy(from) ? PassThroughNode(target)
+ } yield done
+ Await.ready(f, timeout.duration * 2)
+ log.info("PassThrough {} <-> {}", from.name, to.name)
+ }
+
+ def join(from: RoleName, to: RoleName, awaitUp: Boolean): Unit = {
+ runOn(from) {
+ Cluster(sys).join(sysAddress(to))
+ createSingleton()
+ startSharding()
+ if (awaitUp)
+ awaitMemberUp()
+ }
+ enterBarrier(from.name + s"-joined-$c")
+ }
+
+ def awaitMemberUp(): Unit =
+ within(10.seconds) {
+ awaitAssert {
+ Cluster(sys).state.members.exists { m =>
+ m.address == Cluster(sys).selfAddress && m.status == MemberStatus.Up
+ } should be(true)
+ }
+ }
+
+ def createSingleton(): ActorRef = {
+ sys.actorOf(
+ ClusterSingletonManager.props(
+ singletonProps = SingletonActor.props(singletonRegistry),
+ terminationMessage = PoisonPill,
+ settings = ClusterSingletonManagerSettings(system)),
+ name = "singletonRegistry")
+ }
+
+ def startSharding(): Unit = {
+ ClusterSharding(sys).start(
+ typeName = s"Entity-$c",
+ entityProps = SingletonActor.props(shardingRegistry),
+ settings = ClusterShardingSettings(system),
+ extractEntityId = SingletonActor.extractEntityId,
+ extractShardId = SingletonActor.extractShardId)
+ }
+
+ def verify(): Unit = {
+ val nodes = roles.take(scenario.numberOfNodes)
+
+ def sendToSharding(expectReply: Boolean): Unit = {
+ runOn(nodes: _*) {
+ if (!Cluster(sys).isTerminated) {
+ val probe = TestProbe()(sys)
+ for (i <- 0 until 10) {
+ region.tell(i, probe.ref)
+ if (expectReply)
+ probe.expectMsg(3.seconds, i)
+ }
+ }
+ }
+ }
+
+ runOn(nodes: _*) {
+ log.info("Running {} {} in round {}", myself.name, Cluster(sys).selfUniqueAddress, c)
+ }
+ val randomSeed = sys.settings.config.getLong("test.random-seed")
+ val random = new Random(randomSeed)
+ enterBarrier(s"log-startup-$c")
+
+ within(3.minutes) {
+
+ join(nodes.head, nodes.head, awaitUp = true) // oldest
+ join(nodes.last, nodes.head, awaitUp = true) // next oldest
+ for (n <- nodes.tail.dropRight(1))
+ join(n, nodes.head, awaitUp = false)
+ runOn(nodes: _*) {
+ awaitMemberUp()
+ }
+ enterBarrier(s"all-up-$c")
+
+ singletonProbe.expectNoMessage(1.second)
+ shardingProbe.expectNoMessage(10.millis)
+
+ sendToSharding(expectReply = true)
+
+ enterBarrier(s"initialized-$c")
+ runOn(nodes: _*) {
+ log.info("Initialized {} {} in round {}", myself.name, Cluster(sys).selfUniqueAddress, c)
+ }
+
+ runOn(node1) {
+ val cleanSplit = random.nextBoolean()
+ val healCleanSplit = cleanSplit && random.nextBoolean()
+ val side1 = nodes.take(1 + random.nextInt(nodes.size - 1))
+ val side2 = nodes.drop(side1.size)
+
+ val numberOfFlaky = random.nextInt(5)
+ val healLastFlay = numberOfFlaky > 0 && random.nextBoolean()
+ val flaky: Map[Int, (RoleName, List[RoleName])] =
+ (0 until numberOfFlaky).map { i =>
+ val from = nodes(random.nextInt(nodes.size))
+ val targets = nodes.filterNot(_ == from)
+ val to = (0 to random.nextInt(math.min(5, targets.size))).map(j => targets(j)).toList
+ i -> (from -> to)
+ }.toMap
+
+ val delays = (0 until 10).map(_ => 2 + random.nextInt(13))
+
+ log.info(s"Generated $scenario with random seed [$randomSeed] in round [$c]: " +
+ s"cleanSplit [$cleanSplit], healCleanSplit [$healCleanSplit] " +
+ (if (cleanSplit) s"side1 [${side1.map(_.name).mkString(", ")}], side2 [${side2.map(_.name).mkString(", ")}] ") +
+ s"flaky [${flaky.map { case (_, (from, to)) => from.name -> to.map(_.name).mkString("(", ", ", ")") }.mkString("; ")}] " +
+ s"delays [${delays.mkString(", ")}]")
+
+ var delayIndex = 0
+ def nextDelay(): Unit = {
+ Thread.sleep(delays(delayIndex) * 1000)
+ delayIndex += 1
+ }
+
+ if (cleanSplit) {
+ for (n1 <- side1; n2 <- side2)
+ blackhole(n1, n2)
+
+ nextDelay()
+ }
+
+ flaky.foreach {
+ case (i, (from, to)) =>
+ if (i != 0) {
+ // heal previous flakiness
+ val (prevFrom, prevTo) = flaky(i - 1)
+ for (n <- prevTo)
+ passThrough(prevFrom, n)
+ }
+
+ for (n <- to)
+ blackhole(from, n)
+
+ nextDelay()
+ }
+
+ if (healLastFlay) {
+ val (prevFrom, prevTo) = flaky(flaky.size - 1)
+ for (n <- prevTo)
+ passThrough(prevFrom, n)
+
+ nextDelay()
+ }
+
+ if (healCleanSplit) {
+ for (n1 <- side1; n2 <- side2)
+ passThrough(n1, n2)
+ }
+ }
+ enterBarrier(s"scenario-done-$c")
+
+ runOn(nodes: _*) {
+ sendToSharding(expectReply = false)
+ singletonProbe.expectNoMessage(10.seconds)
+ shardingProbe.expectNoMessage(10.millis)
+
+ var loopLimit = 20
+ while (loopLimit != 0 && !Cluster(sys).isTerminated && Cluster(sys).state.unreachable.nonEmpty) {
+ sendToSharding(expectReply = false)
+ singletonProbe.expectNoMessage(5.seconds)
+ shardingProbe.expectNoMessage(10.millis)
+ loopLimit -= 1
+ }
+ }
+ enterBarrier(s"terminated-or-unreachable-removed-$c")
+
+ runOn(nodes: _*) {
+ (Cluster(sys).isTerminated || Cluster(sys).state.unreachable.isEmpty) should ===(true)
+ within(30.seconds) {
+ awaitAssert {
+ sendToSharding(expectReply = true)
+ }
+ }
+ singletonProbe.expectNoMessage(5.seconds)
+ shardingProbe.expectNoMessage(10.millis)
+ if (!Cluster(sys).isTerminated)
+ log.info(s"Survived ${Cluster(sys).state.members.size} members in round $c")
+ }
+
+ enterBarrier(s"verified-$c")
+ }
+ enterBarrier(s"after-$c")
+ }
+
+ }
+
+ private val leaseMajorityConfig = ConfigFactory.parseString("""akka.cluster.split-brain-resolver {
+ active-strategy = lease-majority
+ }""")
+
+ case class Scenario(cfg: Config, numberOfNodes: Int) {
+
+ val activeStrategy: String = cfg.getString("akka.cluster.split-brain-resolver.active-strategy")
+
+ override def toString: String =
+ s"Scenario($activeStrategy, $numberOfNodes)"
+
+ def usingLease: Boolean = activeStrategy.contains("lease")
+ }
+
+ val scenarios =
+ List(Scenario(leaseMajorityConfig, 3), Scenario(leaseMajorityConfig, 5), Scenario(leaseMajorityConfig, 9))
+
+ "SplitBrainResolver with lease" must {
+
+ for (scenario <- scenarios) {
+ scenario.toString taggedAs LongRunningTest in {
+ DisposableSys(scenario).verify()
+ }
+ }
+ }
+
+}
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sbr/SbrTestLeaseActor.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sbr/SbrTestLeaseActor.scala
new file mode 100644
index 0000000000..2622327267
--- /dev/null
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sbr/SbrTestLeaseActor.scala
@@ -0,0 +1,125 @@
+/*
+ * Copyright (C) 2019-2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import java.util.concurrent.atomic.AtomicReference
+
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
+import akka.actor.Actor
+import akka.actor.ActorLogging
+import akka.actor.ActorRef
+import akka.actor.ActorSystem
+import akka.actor.ExtendedActorSystem
+import akka.actor.Extension
+import akka.actor.ExtensionId
+import akka.actor.ExtensionIdProvider
+import akka.actor.Props
+import akka.coordination.lease.LeaseSettings
+import akka.coordination.lease.scaladsl.Lease
+import akka.pattern.ask
+import akka.serialization.jackson.CborSerializable
+import akka.util.Timeout
+
+object SbrTestLeaseActor {
+ def props: Props =
+ Props(new SbrTestLeaseActor)
+
+ final case class Acquire(owner: String) extends CborSerializable
+ final case class Release(owner: String) extends CborSerializable
+}
+
+class SbrTestLeaseActor extends Actor with ActorLogging {
+ import SbrTestLeaseActor._
+
+ var owner: Option[String] = None
+
+ override def receive = {
+ case Acquire(o) =>
+ owner match {
+ case None =>
+ log.info("ActorLease: acquired by [{}]", o)
+ owner = Some(o)
+ sender() ! true
+ case Some(`o`) =>
+ log.info("ActorLease: renewed by [{}]", o)
+ sender() ! true
+ case Some(existingOwner) =>
+ log.info("ActorLease: requested by [{}], but already held by [{}]", o, existingOwner)
+ sender() ! false
+ }
+
+ case Release(o) =>
+ owner match {
+ case None =>
+ log.info("ActorLease: released by [{}] but no owner", o)
+ owner = Some(o)
+ sender() ! true
+ case Some(`o`) =>
+ log.info("ActorLease: released by [{}]", o)
+ sender() ! true
+ case Some(existingOwner) =>
+ log.info("ActorLease: release attempt by [{}], but held by [{}]", o, existingOwner)
+ sender() ! false
+ }
+ }
+
+}
+
+object SbrTestLeaseActorClientExt extends ExtensionId[SbrTestLeaseActorClientExt] with ExtensionIdProvider {
+ override def get(system: ActorSystem): SbrTestLeaseActorClientExt = super.get(system)
+ override def lookup = SbrTestLeaseActorClientExt
+ override def createExtension(system: ExtendedActorSystem): SbrTestLeaseActorClientExt =
+ new SbrTestLeaseActorClientExt(system)
+}
+
+class SbrTestLeaseActorClientExt(val system: ExtendedActorSystem) extends Extension {
+
+ private val leaseClient = new AtomicReference[SbrTestLeaseActorClient]()
+
+ def getActorLeaseClient(): SbrTestLeaseActorClient = {
+ val lease = leaseClient.get
+ if (lease == null) throw new IllegalStateException("ActorLeaseClient must be set first")
+ lease
+ }
+
+ def setActorLeaseClient(client: SbrTestLeaseActorClient): Unit =
+ leaseClient.set(client)
+
+}
+
+class SbrTestLeaseActorClient(settings: LeaseSettings, system: ExtendedActorSystem) extends Lease(settings) {
+ import SbrTestLeaseActor.Acquire
+ import SbrTestLeaseActor.Release
+
+ SbrTestLeaseActorClientExt(system).setActorLeaseClient(this)
+
+ private implicit val timeout = Timeout(3.seconds)
+
+ private val _leaseRef = new AtomicReference[ActorRef]
+
+ private def leaseRef: ActorRef = {
+ val ref = _leaseRef.get
+ if (ref == null) throw new IllegalStateException("ActorLeaseRef must be set first")
+ ref
+ }
+
+ def setActorLeaseRef(ref: ActorRef): Unit =
+ _leaseRef.set(ref)
+
+ override def acquire(): Future[Boolean] = {
+ (leaseRef ? Acquire(settings.ownerName)).mapTo[Boolean]
+ }
+
+ override def acquire(leaseLostCallback: Option[Throwable] => Unit): Future[Boolean] =
+ acquire()
+
+ override def release(): Future[Boolean] = {
+ (leaseRef ? Release(settings.ownerName)).mapTo[Boolean]
+ }
+
+ override def checkLease(): Boolean = false
+}
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sbr/SplitBrainResolverIntegrationSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sbr/SplitBrainResolverIntegrationSpec.scala
new file mode 100644
index 0000000000..718edef025
--- /dev/null
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sbr/SplitBrainResolverIntegrationSpec.scala
@@ -0,0 +1,465 @@
+/*
+ * Copyright (C) 2015-2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import com.typesafe.config.ConfigValueFactory
+import org.scalatest.BeforeAndAfterEach
+
+import akka.actor._
+import akka.cluster.Cluster
+import akka.cluster.ClusterSettings.DataCenter
+import akka.cluster.ClusterSettings.DefaultDataCenter
+import akka.cluster.Member
+import akka.cluster.MemberStatus
+import akka.cluster.MultiNodeClusterSpec
+import akka.cluster.sharding.ClusterSharding
+import akka.cluster.sharding.ClusterShardingSettings
+import akka.cluster.singleton.ClusterSingletonManager
+import akka.cluster.singleton.ClusterSingletonManagerSettings
+import akka.pattern.ask
+import akka.remote.testconductor.RoleName
+import akka.remote.testkit.MultiNodeConfig
+import akka.remote.testkit.MultiNodeSpec
+import akka.testkit.ImplicitSender
+import akka.testkit.LongRunningTest
+import akka.testkit.TestKit
+import akka.testkit.TestProbe
+import akka.util.Timeout
+
+/*
+ * Depends on akka private classes so needs to be in this package
+ */
+object SplitBrainResolverIntegrationSpec extends MultiNodeConfig {
+ val node1 = role("node1")
+ val node2 = role("node2")
+ val node3 = role("node3")
+ val node4 = role("node4")
+ val node5 = role("node5")
+ val node6 = role("node6")
+ val node7 = role("node7")
+ val node8 = role("node8")
+ val node9 = role("node9")
+
+ commonConfig(ConfigFactory.parseString("""
+ akka {
+ loglevel = INFO
+ cluster {
+ downing-provider-class = "akka.cluster.sbr.SplitBrainResolverProvider"
+ split-brain-resolver.active-strategy = keep-majority
+ split-brain-resolver.stable-after = 10s
+
+ sharding.handoff-timeout = 5s
+ }
+
+ actor.provider = cluster
+ remote.log-remote-lifecycle-events = off
+ }
+
+ akka.coordinated-shutdown.run-by-jvm-shutdown-hook = off
+ akka.coordinated-shutdown.terminate-actor-system = off
+ akka.cluster.run-coordinated-shutdown-when-down = off
+ """))
+
+ testTransport(on = true)
+
+}
+
+class SplitBrainResolverIntegrationSpecMultiJvmNode1 extends SplitBrainResolverIntegrationSpec
+class SplitBrainResolverIntegrationSpecMultiJvmNode2 extends SplitBrainResolverIntegrationSpec
+class SplitBrainResolverIntegrationSpecMultiJvmNode3 extends SplitBrainResolverIntegrationSpec
+class SplitBrainResolverIntegrationSpecMultiJvmNode4 extends SplitBrainResolverIntegrationSpec
+class SplitBrainResolverIntegrationSpecMultiJvmNode5 extends SplitBrainResolverIntegrationSpec
+class SplitBrainResolverIntegrationSpecMultiJvmNode6 extends SplitBrainResolverIntegrationSpec
+class SplitBrainResolverIntegrationSpecMultiJvmNode7 extends SplitBrainResolverIntegrationSpec
+class SplitBrainResolverIntegrationSpecMultiJvmNode8 extends SplitBrainResolverIntegrationSpec
+class SplitBrainResolverIntegrationSpecMultiJvmNode9 extends SplitBrainResolverIntegrationSpec
+
+class SplitBrainResolverIntegrationSpec
+ extends MultiNodeSpec(SplitBrainResolverIntegrationSpec)
+ with MultiNodeClusterSpec
+ with ImplicitSender
+ with BeforeAndAfterEach {
+ import GlobalRegistry._
+ import GremlinController._
+ import SplitBrainResolverIntegrationSpec._
+
+ override def initialParticipants = roles.size
+
+ override def afterEach(): Unit = {
+ if (disposableSys ne null)
+ disposableSys.shutdownSys()
+ }
+
+ // counter for unique naming for each test
+ var c = 0
+ // to be shutdown in afterEach
+ var disposableSys: DisposableSys = _
+
+ override def expectedTestDuration = 10.minutes
+
+ object DisposableSys {
+ def apply(scenario: Scenario): DisposableSys = {
+ disposableSys = new DisposableSys(scenario)
+ disposableSys
+ }
+ }
+
+ class DisposableSys(scenario: Scenario) {
+
+ c += 1
+
+ val sys: ActorSystem = {
+ val dcName = scenario.dcDecider(myself)
+
+ val sys = ActorSystem(
+ system.name + "-" + c,
+ scenario.cfg
+ .withValue("akka.cluster.multi-data-center.self-data-center", ConfigValueFactory.fromAnyRef(dcName))
+ .withFallback(system.settings.config))
+ val gremlinController = sys.actorOf(GremlinController.props, "gremlinController")
+ system.actorOf(GremlinControllerProxy.props(gremlinController), s"gremlinControllerProxy-$c")
+ sys
+ }
+
+ val singletonProbe = TestProbe()
+ val shardingProbe = TestProbe()
+ runOn(node1) {
+ system.actorOf(GlobalRegistry.props(singletonProbe.ref, false), s"singletonRegistry-$c")
+ system.actorOf(GlobalRegistry.props(shardingProbe.ref, true), s"shardingRegistry-$c")
+ if (scenario.usingLease)
+ system.actorOf(SbrTestLeaseActor.props, s"lease-${sys.name}")
+ }
+ enterBarrier("registry-started")
+
+ system.actorSelection(node(node1) / "user" / s"singletonRegistry-$c") ! Identify(None)
+ val singletonRegistry: ActorRef = expectMsgType[ActorIdentity].ref.get
+ system.actorSelection(node(node1) / "user" / s"shardingRegistry-$c") ! Identify(None)
+ val shardingRegistry: ActorRef = expectMsgType[ActorIdentity].ref.get
+
+ if (scenario.usingLease) {
+ system.actorSelection(node(node1) / "user" / s"lease-${sys.name}") ! Identify(None)
+ val leaseRef: ActorRef = expectMsgType[ActorIdentity].ref.get
+ SbrTestLeaseActorClientExt(sys).getActorLeaseClient().setActorLeaseRef(leaseRef)
+ }
+
+ enterBarrier("registry-located")
+
+ lazy val region = ClusterSharding(sys).shardRegion(s"Entity-$c")
+
+ def shutdownSys(): Unit = {
+ TestKit.shutdownActorSystem(sys, 10.seconds, verifySystemShutdown = true)
+ }
+
+ def gremlinControllerProxy(at: RoleName): ActorRef = {
+ system.actorSelection(node(at) / "user" / s"gremlinControllerProxy-$c") ! Identify(None)
+ expectMsgType[ActorIdentity].ref.get
+ }
+
+ def sysAddress(at: RoleName): Address = {
+ implicit val timeout = Timeout(3.seconds)
+ Await.result((gremlinControllerProxy(at) ? GetAddress).mapTo[Address], timeout.duration)
+ }
+
+ def blackhole(from: RoleName, to: RoleName): Unit = {
+ implicit val timeout = Timeout(3.seconds)
+ import system.dispatcher
+ val f = for {
+ target <- (gremlinControllerProxy(to) ? GetAddress).mapTo[Address]
+ done <- gremlinControllerProxy(from) ? BlackholeNode(target)
+ } yield done
+ Await.ready(f, timeout.duration * 2)
+ log.info("Blackhole {} <-> {}", from.name, to.name)
+ }
+
+ def join(from: RoleName, to: RoleName, awaitUp: Boolean): Unit = {
+ runOn(from) {
+ Cluster(sys).join(sysAddress(to))
+ createSingleton()
+ startSharding()
+ if (awaitUp)
+ awaitMemberUp()
+ }
+ enterBarrier(from.name + s"-joined-$c")
+ }
+
+ def awaitMemberUp(): Unit =
+ within(10.seconds) {
+ awaitAssert {
+ Cluster(sys).state.members.exists { m =>
+ m.address == Cluster(sys).selfAddress && m.status == MemberStatus.Up
+ } should be(true)
+ }
+ }
+
+ def awaitAllMembersUp(nodes: RoleName*): Unit = {
+ val addresses = nodes.map(sysAddress).toSet
+ within(15.seconds) {
+ awaitAssert {
+ Cluster(sys).state.members.map(_.address) should ===(addresses)
+ Cluster(sys).state.members.foreach {
+ _.status should ===(MemberStatus.Up)
+ }
+ }
+ }
+ }
+
+ def createSingleton(): ActorRef = {
+ sys.actorOf(
+ ClusterSingletonManager.props(
+ singletonProps = SingletonActor.props(singletonRegistry),
+ terminationMessage = PoisonPill,
+ settings = ClusterSingletonManagerSettings(system)),
+ name = "singletonRegistry")
+ }
+
+ def startSharding(): Unit = {
+ ClusterSharding(sys).start(
+ typeName = s"Entity-$c",
+ entityProps = SingletonActor.props(shardingRegistry),
+ settings = ClusterShardingSettings(system),
+ extractEntityId = SingletonActor.extractEntityId,
+ extractShardId = SingletonActor.extractShardId)
+ }
+
+ def verify(): Unit = {
+ val side1 = roles.take(scenario.side1Size)
+ val side2 = roles.drop(scenario.side1Size).take(scenario.side2Size)
+
+ def singletonRegisterKey(node: RoleName): String =
+ "/user/singletonRegistry/singleton-" + scenario.dcDecider(node)
+
+ runOn(side1 ++ side2: _*) {
+ log.info("Running {} {} in round {}", myself.name, Cluster(sys).selfUniqueAddress, c)
+ }
+ enterBarrier(s"log-startup-$c")
+
+ within(90.seconds) {
+
+ join(side1.head, side1.head, awaitUp = true) // oldest
+ join(side2.head, side1.head, awaitUp = true) // next oldest
+ for (n <- side1.tail ++ side2.tail)
+ join(n, side1.head, awaitUp = false)
+ runOn(side1 ++ side2: _*) {
+ awaitAllMembersUp(side1 ++ side2: _*)
+ }
+ enterBarrier(s"all-up-$c")
+
+ runOn(node1) {
+ singletonProbe.within(25.seconds) {
+ singletonProbe.expectMsg(Register(singletonRegisterKey(node1), sysAddress(node1)))
+ }
+ shardingProbe.expectNoMessage(100.millis)
+ }
+
+ runOn(side1 ++ side2: _*) {
+ val probe = TestProbe()(sys)
+ for (i <- 0 until 10) {
+ region.tell(i, probe.ref)
+ probe.expectMsg(5.seconds, i)
+ }
+ }
+
+ enterBarrier(s"initialized-$c")
+ runOn(side1 ++ side2: _*) {
+ log.info("Initialized {} {} in round {}", myself.name, Cluster(sys).selfUniqueAddress, c)
+ }
+
+ runOn(node1) {
+ for (n1 <- side1; n2 <- side2)
+ blackhole(n1, n2)
+ }
+ enterBarrier(s"blackhole-$c")
+
+ val resolvedExpected = scenario.expected match {
+ case KeepLeader =>
+ import Member.addressOrdering
+ val address = (side1 ++ side2).map(sysAddress).min
+ if (side1.exists(sysAddress(_) == address)) KeepSide1
+ else if (side2.exists(sysAddress(_) == address)) KeepSide2
+ else ShutdownBoth
+ case other => other
+ }
+
+ resolvedExpected match {
+ case ShutdownBoth =>
+ runOn(side1 ++ side2: _*) {
+ awaitCond(Cluster(sys).isTerminated, max = 30.seconds)
+ }
+ enterBarrier(s"sys-terminated-$c")
+ runOn(node1) {
+ singletonProbe.within(20.seconds) {
+ singletonProbe.expectMsg(Unregister(singletonRegisterKey(side1.head), sysAddress(side1.head)))
+ }
+ shardingProbe.expectNoMessage(100.millis)
+ }
+
+ case KeepSide1 =>
+ runOn(side1: _*) {
+ val expectedAddresses = side1.map(sysAddress).toSet
+ within(remaining - 3.seconds) {
+ awaitAssert {
+ val probe = TestProbe()(sys)
+ for (i <- 0 until 10) {
+ region.tell(i, probe.ref)
+ probe.expectMsg(2.seconds, i)
+ }
+
+ Cluster(sys).state.members.map(_.address) should be(expectedAddresses)
+ }
+ }
+ }
+ runOn(side2: _*) {
+ awaitCond(Cluster(sys).isTerminated, max = 30.seconds)
+ }
+ enterBarrier(s"cluster-shutdown-verified-$c")
+ singletonProbe.expectNoMessage(1.second)
+ shardingProbe.expectNoMessage(100.millis)
+
+ case KeepSide2 =>
+ runOn(side1: _*) {
+ awaitCond(Cluster(sys).isTerminated, max = 30.seconds)
+ }
+ enterBarrier(s"sys-terminated-$c")
+ runOn(node1) {
+ singletonProbe.within(30.seconds) {
+ singletonProbe.expectMsg(Unregister(singletonRegisterKey(side1.head), sysAddress(side1.head)))
+ singletonProbe.expectMsg(Register(singletonRegisterKey(side2.head), sysAddress(side2.head)))
+ }
+ shardingProbe.expectNoMessage(100.millis)
+ }
+ runOn(side2: _*) {
+ val expectedAddresses = side2.map(sysAddress).toSet
+ within(remaining - 3.seconds) {
+ awaitAssert {
+ val probe = TestProbe()(sys)
+ for (i <- 0 until 10) {
+ region.tell(i, probe.ref)
+ probe.expectMsg(2.seconds, i)
+ }
+
+ Cluster(sys).state.members.map(_.address) should be(expectedAddresses)
+ }
+ }
+ }
+
+ case KeepAll =>
+ runOn((side1 ++ side2): _*) {
+ val expectedAddresses = (side1 ++ side2).map(sysAddress).toSet
+ within(remaining - 3.seconds) {
+ awaitAssert {
+ val probe = TestProbe()(sys)
+ for (i <- 0 until 10) {
+ region.tell(i, probe.ref)
+ probe.expectMsg(2.seconds, i)
+ }
+
+ Cluster(sys).state.members.map(_.address) should be(expectedAddresses)
+ }
+ }
+ Cluster(sys).isTerminated should be(false)
+ }
+ enterBarrier(s"cluster-intact-verified-$c")
+
+ case KeepLeader => throw new IllegalStateException // already resolved to other case
+ }
+
+ enterBarrier(s"verified-$c")
+ }
+ enterBarrier(s"after-$c")
+ }
+
+ }
+
+ private val staticQuorumConfig = ConfigFactory.parseString("""akka.cluster.split-brain-resolver {
+ active-strategy = static-quorum
+ static-quorum.quorum-size = 5
+ }""")
+
+ private val keepMajorityConfig = ConfigFactory.parseString("""akka.cluster.split-brain-resolver {
+ active-strategy = keep-majority
+ }""")
+ private val keepOldestConfig = ConfigFactory.parseString("""akka.cluster.split-brain-resolver {
+ active-strategy = keep-oldest
+ }""")
+ private val downAllConfig = ConfigFactory.parseString("""akka.cluster.split-brain-resolver {
+ active-strategy = down-all
+ }""")
+ private val leaseMajorityConfig = ConfigFactory.parseString("""akka.cluster.split-brain-resolver {
+ active-strategy = lease-majority
+ lease-majority {
+ lease-implementation = test-lease
+ acquire-lease-delay-for-minority = 3s
+ }
+ }
+ test-lease {
+ lease-class = akka.cluster.sbr.SbrTestLeaseActorClient
+ heartbeat-interval = 1s
+ heartbeat-timeout = 120s
+ lease-operation-timeout = 3s
+ }
+ """)
+
+ sealed trait Expected
+ case object KeepSide1 extends Expected
+ case object KeepSide2 extends Expected
+ case object ShutdownBoth extends Expected
+ case object KeepLeader extends Expected
+ case object KeepAll extends Expected
+
+ val defaultDcDecider: RoleName => DataCenter = _ => DefaultDataCenter
+
+ case class Scenario(
+ cfg: Config,
+ side1Size: Int,
+ side2Size: Int,
+ expected: Expected,
+ dcDecider: RoleName => DataCenter = defaultDcDecider // allows to set the dc per indexed node
+ ) {
+
+ val activeStrategy: String = cfg.getString("akka.cluster.split-brain-resolver.active-strategy")
+
+ override def toString: String = {
+ s"$expected when using $activeStrategy and side1=$side1Size and side2=$side2Size" +
+ (if (dcDecider ne defaultDcDecider) "with multi-DC" else "")
+ }
+
+ def usingLease: Boolean = activeStrategy.contains("lease")
+ }
+
+ val scenarios = List(
+ Scenario(staticQuorumConfig, 1, 2, ShutdownBoth),
+ Scenario(staticQuorumConfig, 4, 4, ShutdownBoth),
+ Scenario(staticQuorumConfig, 5, 4, KeepSide1),
+ Scenario(staticQuorumConfig, 1, 5, KeepSide2),
+ Scenario(staticQuorumConfig, 4, 5, KeepSide2),
+ Scenario(keepMajorityConfig, 2, 1, KeepSide1),
+ Scenario(keepMajorityConfig, 1, 2, KeepSide2),
+ Scenario(keepMajorityConfig, 4, 5, KeepSide2),
+ Scenario(keepMajorityConfig, 4, 4, KeepLeader),
+ Scenario(keepOldestConfig, 3, 3, KeepSide1),
+ Scenario(keepOldestConfig, 1, 1, KeepSide1),
+ Scenario(keepOldestConfig, 1, 2, KeepSide2), // because down-if-alone
+ Scenario(keepMajorityConfig, 3, 2, KeepAll, {
+ case `node1` | `node2` | `node3` => "dcA"
+ case _ => "dcB"
+ }),
+ Scenario(downAllConfig, 1, 2, ShutdownBoth),
+ Scenario(leaseMajorityConfig, 4, 5, KeepSide2))
+
+ "Cluster SplitBrainResolver" must {
+
+ for (scenario <- scenarios) {
+ scenario.toString taggedAs LongRunningTest in {
+ DisposableSys(scenario).verify()
+ }
+ }
+ }
+
+}
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingCustomShardAllocationSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingCustomShardAllocationSpec.scala
index d4780396bb..37468cca1a 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingCustomShardAllocationSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingCustomShardAllocationSpec.scala
@@ -4,6 +4,10 @@
package akka.cluster.sharding
+import scala.collection.immutable
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
import akka.actor._
import akka.cluster.sharding.ShardCoordinator.ShardAllocationStrategy
import akka.pattern.ask
@@ -11,10 +15,6 @@ import akka.remote.testconductor.RoleName
import akka.testkit._
import akka.util.Timeout
-import scala.collection.immutable
-import scala.concurrent.Future
-import scala.concurrent.duration._
-
object ClusterShardingCustomShardAllocationSpec {
case object AllocateReq
@@ -43,7 +43,7 @@ object ClusterShardingCustomShardAllocationSpec {
}
case class TestAllocationStrategy(ref: ActorRef) extends ShardAllocationStrategy {
- implicit val timeout = Timeout(3.seconds)
+ implicit val timeout: Timeout = Timeout(3.seconds)
override def allocateShard(
requester: ActorRef,
shardId: ShardRegion.ShardId,
@@ -113,7 +113,7 @@ abstract class ClusterShardingCustomShardAllocationSpec(multiNodeConfig: Cluster
lazy val region = ClusterSharding(system).shardRegion("Entity")
- lazy val allocator = system.actorOf(Props[Allocator], "allocator")
+ lazy val allocator = system.actorOf(Props[Allocator](), "allocator")
s"Cluster sharding ($mode) with custom allocation strategy" must {
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingFailureSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingFailureSpec.scala
index 91d4161dc0..4b34c615e3 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingFailureSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingFailureSpec.scala
@@ -4,6 +4,8 @@
package akka.cluster.sharding
+import scala.concurrent.duration._
+
import akka.actor._
import akka.cluster.sharding.ShardRegion.Passivate
import akka.cluster.sharding.ShardRegion.StartEntity
@@ -13,8 +15,6 @@ import akka.serialization.jackson.CborSerializable
import akka.testkit._
import akka.util.ccompat._
-import scala.concurrent.duration._
-
@ccompatUsedUntil213
object ClusterShardingFailureSpec {
case class Get(id: String) extends CborSerializable
@@ -107,7 +107,7 @@ abstract class ClusterShardingFailureSpec(multiNodeConfig: ClusterShardingFailur
startSharding(
system,
typeName = "Entity",
- entityProps = Props[Entity],
+ entityProps = Props[Entity](),
extractEntityId = extractEntityId,
extractShardId = extractShardId))
}
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStateSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStateSpec.scala
index 5701d8ee51..f0b6189c86 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStateSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStateSpec.scala
@@ -6,11 +6,12 @@ package akka.cluster.sharding
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.cluster.Cluster
import akka.cluster.ClusterEvent.CurrentClusterState
import akka.testkit.TestProbe
-import com.typesafe.config.ConfigFactory
object ClusterShardingGetStateSpec {
import MultiNodeClusterShardingSpec.PingPongActor
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStatsSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStatsSpec.scala
index 223453f3bd..25ccdbd0d5 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStatsSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStatsSpec.scala
@@ -7,8 +7,10 @@ package akka.cluster.sharding
import scala.concurrent.duration._
import akka.actor._
-import akka.cluster.{ Cluster, MemberStatus }
-import akka.testkit.{ TestDuration, TestProbe }
+import akka.cluster.Cluster
+import akka.cluster.MemberStatus
+import akka.testkit.TestDuration
+import akka.testkit.TestProbe
import com.typesafe.config.ConfigFactory
object ClusterShardingGetStatsSpec {
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGracefulShutdownSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGracefulShutdownSpec.scala
index 20cf8840a6..1018d2dc75 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGracefulShutdownSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGracefulShutdownSpec.scala
@@ -4,13 +4,13 @@
package akka.cluster.sharding
+import scala.concurrent.duration._
+
import akka.actor._
import akka.cluster.sharding.ShardRegion.GracefulShutdown
import akka.remote.testconductor.RoleName
import akka.testkit._
-import scala.concurrent.duration._
-
abstract class ClusterShardingGracefulShutdownSpecConfig(mode: String)
extends MultiNodeClusterShardingConfig(
mode,
@@ -56,7 +56,7 @@ abstract class ClusterShardingGracefulShutdownSpec(multiNodeConfig: ClusterShard
startSharding(
system,
typeName,
- entityProps = Props[ShardedEntity],
+ entityProps = Props[ShardedEntity](),
extractEntityId = MultiNodeClusterShardingSpec.intExtractEntityId,
extractShardId = MultiNodeClusterShardingSpec.intExtractShardId,
allocationStrategy =
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingLeavingSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingLeavingSpec.scala
index 8d6dee8ccd..6310944185 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingLeavingSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingLeavingSpec.scala
@@ -91,7 +91,7 @@ abstract class ClusterShardingLeavingSpec(multiNodeConfig: ClusterShardingLeavin
startSharding(
system,
typeName = "Entity",
- entityProps = Props[Entity],
+ entityProps = Props[Entity](),
extractEntityId = extractEntityId,
extractShardId = extractShardId)
}
@@ -120,7 +120,7 @@ abstract class ClusterShardingLeavingSpec(multiNodeConfig: ClusterShardingLeavin
"initialize shards" in {
runOn(first) {
- val shardLocations = system.actorOf(Props[ShardLocations], "shardLocations")
+ val shardLocations = system.actorOf(Props[ShardLocations](), "shardLocations")
val locations = (for (n <- 1 to 10) yield {
val id = n.toString
region ! Ping(id)
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingMinMembersSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingMinMembersSpec.scala
index f18991e222..339da2728a 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingMinMembersSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingMinMembersSpec.scala
@@ -4,14 +4,14 @@
package akka.cluster.sharding
+import scala.concurrent.duration._
+
import akka.actor._
import akka.cluster.MemberStatus
import akka.cluster.sharding.ShardRegion.{ ClusterShardingStats, GetClusterShardingStats }
import akka.testkit._
import akka.util.ccompat._
-import scala.concurrent.duration._
-
@ccompatUsedUntil213
abstract class ClusterShardingMinMembersSpecConfig(mode: String)
extends MultiNodeClusterShardingConfig(
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingQueriesSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingQueriesSpec.scala
index 762ee1f72b..457b9dc11f 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingQueriesSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingQueriesSpec.scala
@@ -6,11 +6,12 @@ package akka.cluster.sharding
import scala.concurrent.duration._
-import akka.actor.Props
-import akka.testkit.TestProbe
import com.typesafe.config.ConfigFactory
import org.scalatest.concurrent.ScalaFutures
+import akka.actor.Props
+import akka.testkit.TestProbe
+
object ClusterShardingQueriesSpec {
import MultiNodeClusterShardingSpec.PingPongActor
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRegistrationCoordinatedShutdownSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRegistrationCoordinatedShutdownSpec.scala
index 8326ed0326..f784a63f52 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRegistrationCoordinatedShutdownSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRegistrationCoordinatedShutdownSpec.scala
@@ -4,14 +4,14 @@
package akka.cluster.sharding
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
import akka.Done
import akka.actor._
import akka.cluster.MemberStatus
import akka.testkit.{ ImplicitSender, TestProbe }
-import scala.concurrent.Future
-import scala.concurrent.duration._
-
/**
* Test for issue #28416
*/
@@ -66,7 +66,7 @@ abstract class ClusterShardingRegistrationCoordinatedShutdownSpec
startSharding(
system,
typeName = "Entity",
- entityProps = Props[ShardedEntity],
+ entityProps = Props[ShardedEntity](),
extractEntityId = MultiNodeClusterShardingSpec.intExtractEntityId,
extractShardId = MultiNodeClusterShardingSpec.intExtractShardId)
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRememberEntitiesNewExtractorSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRememberEntitiesNewExtractorSpec.scala
index 9797b5c822..ebc7d3d03a 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRememberEntitiesNewExtractorSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRememberEntitiesNewExtractorSpec.scala
@@ -4,13 +4,14 @@
package akka.cluster.sharding
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.cluster.{ Cluster, MemberStatus }
import akka.persistence.journal.leveldb.SharedLeveldbJournal
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.duration._
object ClusterShardingRememberEntitiesNewExtractorSpec {
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRememberEntitiesPerfSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRememberEntitiesPerfSpec.scala
index cc7cfb8b46..240c1adac8 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRememberEntitiesPerfSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRememberEntitiesPerfSpec.scala
@@ -6,13 +6,14 @@ package akka.cluster.sharding
import java.util.concurrent.TimeUnit.NANOSECONDS
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.cluster.MemberStatus
import akka.testkit._
import akka.util.ccompat._
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.duration._
@ccompatUsedUntil213
object ClusterShardingRememberEntitiesPerfSpec {
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRememberEntitiesSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRememberEntitiesSpec.scala
index c5d9a49aa4..39835cd04f 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRememberEntitiesSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRememberEntitiesSpec.scala
@@ -6,11 +6,12 @@ package akka.cluster.sharding
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.cluster.{ Cluster, MemberStatus }
import akka.testkit._
import akka.util.ccompat._
-import com.typesafe.config.ConfigFactory
@ccompatUsedUntil213
object ClusterShardingRememberEntitiesSpec {
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRolePartitioningSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRolePartitioningSpec.scala
index 1b2ff0149a..3dff45b678 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRolePartitioningSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingRolePartitioningSpec.scala
@@ -4,12 +4,13 @@
package akka.cluster.sharding
+import scala.concurrent.duration._
+
+import com.typesafe.config.{ Config, ConfigFactory }
+
import akka.actor._
import akka.cluster.sharding.ShardRegion.{ ClusterShardingStats, GetClusterShardingStats }
import akka.testkit._
-import com.typesafe.config.{ Config, ConfigFactory }
-
-import scala.concurrent.duration._
// Tests the case where cluster roles are used with cluster.min-nr-of-members, no per role min set
// with 5 node cluster, 2 roles: 3 nodes role R1, 2 nodes role R2
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSingleShardPerEntitySpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSingleShardPerEntitySpec.scala
index 8ee217a0ab..b290095348 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSingleShardPerEntitySpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSingleShardPerEntitySpec.scala
@@ -4,13 +4,13 @@
package akka.cluster.sharding
+import scala.concurrent.duration._
+
import akka.actor._
import akka.remote.testconductor.RoleName
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._
-import scala.concurrent.duration._
-
/**
* one-to-one mapping between shards and entities is not efficient but some use that anyway
*/
@@ -45,7 +45,7 @@ abstract class ClusterShardingSingleShardPerEntitySpec
startSharding(
system,
typeName = "Entity",
- entityProps = Props[ShardedEntity],
+ entityProps = Props[ShardedEntity](),
extractEntityId = MultiNodeClusterShardingSpec.intExtractEntityId,
extractShardId = MultiNodeClusterShardingSpec.intExtractShardId))
}
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSpec.scala
index eb78b539ea..451ccb0ca5 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSpec.scala
@@ -4,6 +4,11 @@
package akka.cluster.sharding
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.cluster.Cluster
import akka.cluster.ddata.{ Replicator, ReplicatorSettings }
@@ -15,12 +20,8 @@ import akka.pattern.BackoffOpts
import akka.persistence.journal.leveldb.{ SharedLeveldbJournal, SharedLeveldbStore }
import akka.persistence.{ Persistence, PersistentActor }
import akka.remote.testconductor.RoleName
-import akka.testkit.TestEvent.Mute
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.duration._
-import scala.language.postfixOps
+import akka.testkit.TestEvent.Mute
object ClusterShardingSpec {
//#counter-actor
@@ -91,7 +92,7 @@ object ClusterShardingSpec {
//#supervisor
class CounterSupervisor extends Actor {
- val counter = context.actorOf(Props[Counter], "theCounter")
+ val counter = context.actorOf(Props[Counter](), "theCounter")
override val supervisorStrategy = OneForOneStrategy() {
case _: IllegalArgumentException => SupervisorStrategy.Resume
@@ -378,7 +379,7 @@ abstract class ClusterShardingSpec(multiNodeConfig: ClusterShardingSpecConfig)
// start the Persistence extension
Persistence(system)
runOn(controller) {
- system.actorOf(Props[SharedLeveldbStore], "store")
+ system.actorOf(Props[SharedLeveldbStore](), "store")
}
enterBarrier("peristence-started")
@@ -647,7 +648,7 @@ abstract class ClusterShardingSpec(multiNodeConfig: ClusterShardingSpecConfig)
//#counter-start
val counterRegion: ActorRef = ClusterSharding(system).start(
typeName = "Counter",
- entityProps = Props[Counter],
+ entityProps = Props[Counter](),
settings = ClusterShardingSettings(system),
extractEntityId = extractEntityId,
extractShardId = extractShardId)
@@ -656,7 +657,7 @@ abstract class ClusterShardingSpec(multiNodeConfig: ClusterShardingSpecConfig)
ClusterSharding(system).start(
typeName = "AnotherCounter",
- entityProps = Props[AnotherCounter],
+ entityProps = Props[AnotherCounter](),
settings = ClusterShardingSettings(system),
extractEntityId = extractEntityId,
extractShardId = extractShardId)
@@ -664,7 +665,7 @@ abstract class ClusterShardingSpec(multiNodeConfig: ClusterShardingSpecConfig)
//#counter-supervisor-start
ClusterSharding(system).start(
typeName = "SupervisedCounter",
- entityProps = Props[CounterSupervisor],
+ entityProps = Props[CounterSupervisor](),
settings = ClusterShardingSettings(system),
extractEntityId = extractEntityId,
extractShardId = extractShardId)
@@ -706,7 +707,7 @@ abstract class ClusterShardingSpec(multiNodeConfig: ClusterShardingSpecConfig)
runOn(first) {
val counterRegionViaStart: ActorRef = ClusterSharding(system).start(
typeName = "ApiTest",
- entityProps = Props[Counter],
+ entityProps = Props[Counter](),
settings = ClusterShardingSettings(system),
extractEntityId = extractEntityId,
extractShardId = extractShardId)
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ExternalShardAllocationSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ExternalShardAllocationSpec.scala
index e0164e50c2..89ee7a38ee 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ExternalShardAllocationSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ExternalShardAllocationSpec.scala
@@ -4,15 +4,16 @@
package akka.cluster.sharding
+import scala.concurrent.duration._
+
+import org.scalatest.concurrent.ScalaFutures
+
import akka.actor.{ Actor, ActorLogging, Address, Props }
import akka.cluster.Cluster
import akka.cluster.sharding.ExternalShardAllocationSpec.GiveMeYourHome.{ Get, Home }
import akka.cluster.sharding.external.{ ExternalShardAllocation, ExternalShardAllocationStrategy }
import akka.serialization.jackson.CborSerializable
import akka.testkit.{ ImplicitSender, TestProbe }
-import org.scalatest.concurrent.ScalaFutures
-
-import scala.concurrent.duration._
object ExternalShardAllocationSpecConfig
extends MultiNodeClusterShardingConfig(additionalConfig = """
@@ -68,8 +69,8 @@ abstract class ExternalShardAllocationSpec
with ImplicitSender
with ScalaFutures {
- import ExternalShardAllocationSpec.GiveMeYourHome._
import ExternalShardAllocationSpec._
+ import ExternalShardAllocationSpec.GiveMeYourHome._
import ExternalShardAllocationSpecConfig._
override implicit val patienceConfig: PatienceConfig = PatienceConfig(5.second)
@@ -86,7 +87,7 @@ abstract class ExternalShardAllocationSpec
lazy val shardRegion = startSharding(
system,
typeName = typeName,
- entityProps = Props[GiveMeYourHome],
+ entityProps = Props[GiveMeYourHome](),
extractEntityId = extractEntityId,
extractShardId = extractShardId,
allocationStrategy = new ExternalShardAllocationStrategy(system, typeName))
@@ -130,7 +131,7 @@ abstract class ExternalShardAllocationSpec
val forthAddress = address(forth)
runOn(second) {
system.log.info("Allocating {} on {}", onForthShardId, forthAddress)
- ExternalShardAllocation(system).clientFor(typeName).updateShardLocation(onForthShardId, forthAddress)
+ ExternalShardAllocation(system).clientFor(typeName).updateShardLocations(Map(onForthShardId -> forthAddress))
}
enterBarrier("allocated-to-new-node")
runOn(forth) {
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/MultiDcClusterShardingSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/MultiDcClusterShardingSpec.scala
index bcc1df84bb..82e78df0a6 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/MultiDcClusterShardingSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/MultiDcClusterShardingSpec.scala
@@ -4,16 +4,17 @@
package akka.cluster.sharding
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.{ Actor, ActorRef, Address, Props }
-import akka.cluster.sharding.ShardRegion.{ CurrentRegions, GetCurrentRegions }
import akka.cluster.{ Cluster, MemberStatus }
+import akka.cluster.sharding.ShardRegion.{ CurrentRegions, GetCurrentRegions }
import akka.remote.testconductor.RoleName
import akka.serialization.jackson.CborSerializable
import akka.testkit._
import akka.util.ccompat._
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.duration._
@ccompatUsedUntil213
object MultiDcClusterShardingSpec {
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/MultiNodeClusterShardingConfig.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/MultiNodeClusterShardingConfig.scala
index 87f27f5148..8c33ff22db 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/MultiNodeClusterShardingConfig.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/MultiNodeClusterShardingConfig.scala
@@ -6,10 +6,11 @@ package akka.cluster.sharding
import java.lang.reflect.Modifier
+import com.typesafe.config.{ Config, ConfigFactory }
+
import akka.cluster.MultiNodeClusterSpec
import akka.persistence.journal.leveldb.SharedLeveldbJournal
import akka.remote.testkit.MultiNodeConfig
-import com.typesafe.config.{ Config, ConfigFactory }
object MultiNodeClusterShardingConfig {
diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/MultiNodeClusterShardingSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/MultiNodeClusterShardingSpec.scala
index e2a2f71eaa..5a12553dff 100644
--- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/MultiNodeClusterShardingSpec.scala
+++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/MultiNodeClusterShardingSpec.scala
@@ -6,6 +6,10 @@ package akka.cluster.sharding
import java.io.File
+import scala.concurrent.duration._
+
+import org.apache.commons.io.FileUtils
+
import akka.actor.{ Actor, ActorIdentity, ActorLogging, ActorRef, ActorSystem, Identify, PoisonPill, Props }
import akka.cluster.MultiNodeClusterSpec
import akka.cluster.sharding.ShardCoordinator.ShardAllocationStrategy
@@ -16,9 +20,6 @@ import akka.remote.testkit.MultiNodeSpec
import akka.serialization.jackson.CborSerializable
import akka.testkit.{ TestActors, TestProbe }
import akka.util.ccompat._
-import org.apache.commons.io.FileUtils
-
-import scala.concurrent.duration._
@ccompatUsedUntil213
object MultiNodeClusterShardingSpec {
@@ -210,7 +211,7 @@ abstract class MultiNodeClusterShardingSpec(val config: MultiNodeClusterSharding
Persistence(system)
runOn(startOn) {
- system.actorOf(Props[SharedLeveldbStore], "store")
+ system.actorOf(Props[SharedLeveldbStore](), "store")
}
enterBarrier("persistence-started")
diff --git a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ClusterShardingInternalsSpec.scala b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ClusterShardingInternalsSpec.scala
index 992647ce92..31dd50da8b 100644
--- a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ClusterShardingInternalsSpec.scala
+++ b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ClusterShardingInternalsSpec.scala
@@ -4,6 +4,8 @@
package akka.cluster.sharding
+import scala.concurrent.duration._
+
import akka.actor.{ Actor, ActorRef, ExtendedActorSystem, NoSerializationVerificationNeeded, PoisonPill, Props }
import akka.cluster.ClusterSettings.DataCenter
import akka.cluster.sharding.ShardCoordinator.Internal.ShardStopped
@@ -12,8 +14,6 @@ import akka.cluster.sharding.ShardRegion.{ ExtractEntityId, ExtractShardId, Hand
import akka.testkit.WithLogCapturing
import akka.testkit.{ AkkaSpec, TestProbe }
-import scala.concurrent.duration._
-
object ClusterShardingInternalsSpec {
case class HandOffStopMessage() extends NoSerializationVerificationNeeded
class EmptyHandlerActor extends Actor {
diff --git a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ClusterShardingLeaseSpec.scala b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ClusterShardingLeaseSpec.scala
index 5212d386b0..5d9070356c 100644
--- a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ClusterShardingLeaseSpec.scala
+++ b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ClusterShardingLeaseSpec.scala
@@ -4,7 +4,7 @@
package akka.cluster.sharding
import akka.actor.Props
-import akka.cluster.{ Cluster, MemberStatus, TestLease, TestLeaseExt }
+import akka.cluster.{ Cluster, MemberStatus }
import akka.testkit.TestActors.EchoActor
import akka.testkit.WithLogCapturing
import akka.testkit.{ AkkaSpec, ImplicitSender }
@@ -15,6 +15,15 @@ import scala.concurrent.duration._
import scala.util.Success
import scala.util.control.NoStackTrace
+import com.typesafe.config.{ Config, ConfigFactory }
+
+import akka.actor.Props
+import akka.cluster.{ Cluster, MemberStatus }
+import akka.coordination.lease.TestLease
+import akka.coordination.lease.TestLeaseExt
+import akka.testkit.{ AkkaSpec, ImplicitSender }
+import akka.testkit.TestActors.EchoActor
+
object ClusterShardingLeaseSpec {
val config = ConfigFactory.parseString("""
akka.loglevel = DEBUG
@@ -78,7 +87,7 @@ class ClusterShardingLeaseSpec(config: Config, rememberEntities: Boolean)
}
ClusterSharding(system).start(
typeName = typeName,
- entityProps = Props[EchoActor],
+ entityProps = Props[EchoActor](),
settings = ClusterShardingSettings(system).withRememberEntities(rememberEntities),
extractEntityId = extractEntityId,
extractShardId = extractShardId)
diff --git a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/CoordinatedShutdownShardingSpec.scala b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/CoordinatedShutdownShardingSpec.scala
index 4a3cff57be..5b4257a699 100644
--- a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/CoordinatedShutdownShardingSpec.scala
+++ b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/CoordinatedShutdownShardingSpec.scala
@@ -6,6 +6,7 @@ package akka.cluster.sharding
import scala.concurrent.Future
import scala.concurrent.duration._
+
import akka.Done
import akka.actor.ActorSystem
import akka.actor.CoordinatedShutdown
diff --git a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/InactiveEntityPassivationSpec.scala b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/InactiveEntityPassivationSpec.scala
index 3260a1d6fb..3a7c7fb5b1 100644
--- a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/InactiveEntityPassivationSpec.scala
+++ b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/InactiveEntityPassivationSpec.scala
@@ -5,13 +5,15 @@
package akka.cluster.sharding
import scala.concurrent.duration._
+
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+
import akka.actor.{ Actor, ActorRef, Props }
import akka.cluster.Cluster
import akka.cluster.sharding.InactiveEntityPassivationSpec.Entity.GotIt
import akka.testkit.WithLogCapturing
import akka.testkit.{ AkkaSpec, TestProbe }
-import com.typesafe.config.ConfigFactory
-import com.typesafe.config.Config
object InactiveEntityPassivationSpec {
diff --git a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/JoinConfigCompatCheckShardingSpec.scala b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/JoinConfigCompatCheckShardingSpec.scala
index fb9eb40dd9..b71e5f64d7 100644
--- a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/JoinConfigCompatCheckShardingSpec.scala
+++ b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/JoinConfigCompatCheckShardingSpec.scala
@@ -4,6 +4,11 @@
package akka.cluster.sharding
+import scala.collection.{ immutable => im }
+import scala.concurrent.duration._
+
+import com.typesafe.config.{ Config, ConfigFactory }
+
import akka.actor.ActorSystem
import akka.cluster.{ Cluster, ClusterReadView }
import akka.testkit.WithLogCapturing
diff --git a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ProxyShardingSpec.scala b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ProxyShardingSpec.scala
index fe660ff1ca..0dd0c7357f 100644
--- a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ProxyShardingSpec.scala
+++ b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ProxyShardingSpec.scala
@@ -6,13 +6,15 @@ package akka.cluster.sharding
import scala.concurrent.Await
import scala.concurrent.duration._
+import scala.concurrent.duration.FiniteDuration
+
+import scala.concurrent.duration.FiniteDuration
+
import akka.actor.ActorRef
import akka.testkit.AkkaSpec
import akka.testkit.TestActors
import akka.testkit.WithLogCapturing
-import scala.concurrent.duration.FiniteDuration
-
object ProxyShardingSpec {
val config = """
akka.actor.provider = cluster
diff --git a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/RemoveInternalClusterShardingDataSpec.scala b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/RemoveInternalClusterShardingDataSpec.scala
index 0c4cdee779..348dfa3426 100644
--- a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/RemoveInternalClusterShardingDataSpec.scala
+++ b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/RemoveInternalClusterShardingDataSpec.scala
@@ -9,6 +9,9 @@ import java.io.File
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.Success
+
+import org.apache.commons.io.FileUtils
+
import akka.actor.ActorRef
import akka.actor.Props
import akka.cluster.Cluster
@@ -23,7 +26,6 @@ import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import akka.testkit.TestActors.EchoActor
import akka.testkit.WithLogCapturing
-import org.apache.commons.io.FileUtils
object RemoveInternalClusterShardingDataSpec {
val config = """
diff --git a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ShardRegionSpec.scala b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ShardRegionSpec.scala
index 62acdbdacb..8af5fcc363 100644
--- a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ShardRegionSpec.scala
+++ b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ShardRegionSpec.scala
@@ -6,14 +6,17 @@ package akka.cluster.sharding
import java.io.File
+import com.typesafe.config.ConfigFactory
+import org.apache.commons.io.FileUtils
+
import akka.actor.{ Actor, ActorLogging, ActorRef, ActorSystem, PoisonPill, Props }
+import akka.cluster.{ Cluster, MemberStatus }
import akka.cluster.ClusterEvent.CurrentClusterState
import akka.cluster.{ Cluster, MemberStatus }
import akka.testkit.TestEvent.Mute
import akka.testkit.WithLogCapturing
import akka.testkit.{ AkkaSpec, DeadLettersFilter, TestProbe }
-import com.typesafe.config.ConfigFactory
-import org.apache.commons.io.FileUtils
+import akka.testkit.TestEvent.Mute
object ShardRegionSpec {
val host = "127.0.0.1"
@@ -61,9 +64,10 @@ object ShardRegionSpec {
}
class ShardRegionSpec extends AkkaSpec(ShardRegionSpec.config) with WithLogCapturing {
- import ShardRegionSpec._
import scala.concurrent.duration._
+ import ShardRegionSpec._
+
val storageLocation = List(
new File(system.settings.config.getString("akka.cluster.sharding.distributed-data.durable.lmdb.dir")).getParentFile)
@@ -94,7 +98,7 @@ class ShardRegionSpec extends AkkaSpec(ShardRegionSpec.config) with WithLogCaptu
def startShard(sys: ActorSystem): ActorRef =
ClusterSharding(sys).start(
shardTypeName,
- Props[EntityActor],
+ Props[EntityActor](),
ClusterShardingSettings(system).withRememberEntities(true),
extractEntityId,
extractShardId)
diff --git a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ShardWithLeaseSpec.scala b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ShardWithLeaseSpec.scala
index 9796bf8fd4..e05d845d05 100644
--- a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ShardWithLeaseSpec.scala
+++ b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/ShardWithLeaseSpec.scala
@@ -9,9 +9,8 @@ import akka.actor.ActorLogging
import akka.actor.Props
import akka.cluster.Cluster
import akka.cluster.MemberStatus
-import akka.cluster.TestLeaseExt
import akka.cluster.sharding.ShardRegion.ShardId
-import akka.coordination.lease.LeaseUsageSettings
+import akka.coordination.lease.{ LeaseUsageSettings, TestLeaseExt }
import akka.testkit.AkkaSpec
import akka.testkit.EventFilter
import akka.testkit.TestProbe
@@ -32,7 +31,7 @@ object ShardWithLeaseSpec {
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
test-lease {
- lease-class = akka.cluster.TestLease
+ lease-class = akka.coordination.lease.TestLease
heartbeat-interval = 1s
heartbeat-timeout = 120s
lease-operation-timeout = 3s
diff --git a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/SupervisionSpec.scala b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/SupervisionSpec.scala
index 93c1d3df04..c9766379ed 100644
--- a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/SupervisionSpec.scala
+++ b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/SupervisionSpec.scala
@@ -4,15 +4,16 @@
package akka.cluster.sharding
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.{ Actor, ActorLogging, ActorRef, PoisonPill, Props }
import akka.cluster.Cluster
import akka.cluster.sharding.ShardRegion.Passivate
import akka.pattern.{ BackoffOpts, BackoffSupervisor }
import akka.testkit.WithLogCapturing
import akka.testkit.{ AkkaSpec, ImplicitSender }
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.duration._
object SupervisionSpec {
val config =
diff --git a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/external/ExternalShardAllocationStrategySpec.scala b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/external/ExternalShardAllocationStrategySpec.scala
index 7700e5700e..a41c3f51ff 100644
--- a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/external/ExternalShardAllocationStrategySpec.scala
+++ b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/external/ExternalShardAllocationStrategySpec.scala
@@ -4,6 +4,8 @@
package akka.cluster.sharding.external
+import scala.concurrent.duration._
+
import akka.cluster.sharding.external.ExternalShardAllocationStrategy.GetShardLocation
import akka.cluster.sharding.external.ExternalShardAllocationStrategy.GetShardLocationResponse
import akka.cluster.sharding.external.ExternalShardAllocationStrategy.GetShardLocations
@@ -11,8 +13,6 @@ import akka.testkit.AkkaSpec
import akka.testkit.TestProbe
import akka.util.Timeout
-import scala.concurrent.duration._
-
class ExternalShardAllocationStrategySpec extends AkkaSpec("""
akka.actor.provider = cluster
akka.loglevel = INFO
diff --git a/akka-cluster-tools/src/main/java/akka/cluster/client/protobuf/msg/ClusterClientMessages.java b/akka-cluster-tools/src/main/java/akka/cluster/client/protobuf/msg/ClusterClientMessages.java
index f363f6b23c..f88b1dce3e 100644
--- a/akka-cluster-tools/src/main/java/akka/cluster/client/protobuf/msg/ClusterClientMessages.java
+++ b/akka-cluster-tools/src/main/java/akka/cluster/client/protobuf/msg/ClusterClientMessages.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2019-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
diff --git a/akka-cluster-tools/src/main/java/akka/cluster/pubsub/protobuf/msg/DistributedPubSubMessages.java b/akka-cluster-tools/src/main/java/akka/cluster/pubsub/protobuf/msg/DistributedPubSubMessages.java
index 75b3a7a9ca..5475b9086d 100644
--- a/akka-cluster-tools/src/main/java/akka/cluster/pubsub/protobuf/msg/DistributedPubSubMessages.java
+++ b/akka-cluster-tools/src/main/java/akka/cluster/pubsub/protobuf/msg/DistributedPubSubMessages.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2019-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
diff --git a/akka-cluster-tools/src/main/resources/reference.conf b/akka-cluster-tools/src/main/resources/reference.conf
index 7e00206259..51c032e3f5 100644
--- a/akka-cluster-tools/src/main/resources/reference.conf
+++ b/akka-cluster-tools/src/main/resources/reference.conf
@@ -197,8 +197,10 @@ akka.cluster.singleton-proxy {
# The actor name of the singleton actor that is started by the ClusterSingletonManager
singleton-name = ${akka.cluster.singleton.singleton-name}
- # The role of the cluster nodes where the singleton can be deployed.
- # If the role is not specified then any node will do.
+ # The role of the cluster nodes where the singleton can be deployed.
+ # Corresponding to the role used by the `ClusterSingletonManager`. If the role is not
+ # specified it's a singleton among all nodes in the cluster, and the `ClusterSingletonManager`
+ # must then also be configured in same way.
role = ""
# Interval at which the proxy will try to resolve the singleton instance.
diff --git a/akka-cluster-tools/src/main/scala/akka/cluster/client/ClusterClient.scala b/akka-cluster-tools/src/main/scala/akka/cluster/client/ClusterClient.scala
index b03c699a32..8e0e62b34f 100644
--- a/akka-cluster-tools/src/main/scala/akka/cluster/client/ClusterClient.scala
+++ b/akka-cluster-tools/src/main/scala/akka/cluster/client/ClusterClient.scala
@@ -7,7 +7,11 @@ package akka.cluster.client
import java.net.URLEncoder
import scala.collection.immutable
+import scala.collection.immutable.{ HashMap, HashSet }
import scala.concurrent.duration._
+
+import com.typesafe.config.Config
+
import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.ActorLogging
@@ -34,15 +38,12 @@ import akka.cluster.Member
import akka.cluster.MemberStatus
import akka.cluster.pubsub._
import akka.japi.Util.immutableSeq
+import akka.remote.DeadlineFailureDetector
import akka.routing.ConsistentHash
import akka.routing.MurmurHash
-import com.typesafe.config.Config
-import akka.remote.DeadlineFailureDetector
import akka.util.MessageBuffer
import akka.util.ccompat._
-import scala.collection.immutable.{ HashMap, HashSet }
-
@ccompatUsedUntil213
@deprecated(
"Use Akka gRPC instead, see https://doc.akka.io/docs/akka/2.6/cluster-client.html#migration-to-akka-grpc",
@@ -948,9 +949,8 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
extends Actor
with ActorLogging {
- import DistributedPubSubMediator.{ Publish, Send, SendToAll }
-
import ClusterReceptionist.Internal._
+ import DistributedPubSubMediator.{ Publish, Send, SendToAll }
import settings._
val cluster = Cluster(context.system)
diff --git a/akka-cluster-tools/src/main/scala/akka/cluster/client/protobuf/ClusterClientMessageSerializer.scala b/akka-cluster-tools/src/main/scala/akka/cluster/client/protobuf/ClusterClientMessageSerializer.scala
index ce1cbcd1e1..f47f5a465d 100644
--- a/akka-cluster-tools/src/main/scala/akka/cluster/client/protobuf/ClusterClientMessageSerializer.scala
+++ b/akka-cluster-tools/src/main/scala/akka/cluster/client/protobuf/ClusterClientMessageSerializer.scala
@@ -4,16 +4,17 @@
package akka.cluster.client.protobuf
-import akka.util.ccompat.JavaConverters._
-import akka.actor.ExtendedActorSystem
-import akka.serialization.BaseSerializer
-import akka.serialization.SerializerWithStringManifest
-import akka.cluster.client.ClusterReceptionist
-import akka.cluster.client.protobuf.msg.{ ClusterClientMessages => cm }
import java.io.NotSerializableException
import com.github.ghik.silencer.silent
+import akka.actor.ExtendedActorSystem
+import akka.cluster.client.ClusterReceptionist
+import akka.cluster.client.protobuf.msg.{ ClusterClientMessages => cm }
+import akka.serialization.BaseSerializer
+import akka.serialization.SerializerWithStringManifest
+import akka.util.ccompat.JavaConverters._
+
/**
* INTERNAL API: Serializer of ClusterClient messages.
*/
diff --git a/akka-cluster-tools/src/main/scala/akka/cluster/pubsub/DistributedPubSubMediator.scala b/akka-cluster-tools/src/main/scala/akka/cluster/pubsub/DistributedPubSubMediator.scala
index 7e67fcb6ce..79a5692b3a 100644
--- a/akka-cluster-tools/src/main/scala/akka/cluster/pubsub/DistributedPubSubMediator.scala
+++ b/akka-cluster-tools/src/main/scala/akka/cluster/pubsub/DistributedPubSubMediator.scala
@@ -4,12 +4,16 @@
package akka.cluster.pubsub
+import java.net.URLDecoder
+import java.net.URLEncoder
+import java.util.concurrent.ThreadLocalRandom
+
import scala.collection.immutable
import scala.collection.immutable.Set
+import scala.collection.immutable.TreeMap
import scala.concurrent.duration._
-import java.util.concurrent.ThreadLocalRandom
-import java.net.URLEncoder
-import java.net.URLDecoder
+
+import com.typesafe.config.Config
import akka.actor._
import akka.annotation.DoNotInherit
@@ -17,18 +21,15 @@ import akka.cluster.Cluster
import akka.cluster.ClusterEvent._
import akka.cluster.Member
import akka.cluster.MemberStatus
-import akka.routing.RandomRoutingLogic
-import akka.routing.RoutingLogic
-import akka.routing.Routee
import akka.routing.ActorRefRoutee
+import akka.routing.BroadcastRoutingLogic
+import akka.routing.ConsistentHashingRoutingLogic
+import akka.routing.RandomRoutingLogic
+import akka.routing.RoundRobinRoutingLogic
+import akka.routing.Routee
import akka.routing.Router
import akka.routing.RouterEnvelope
-import akka.routing.RoundRobinRoutingLogic
-import akka.routing.ConsistentHashingRoutingLogic
-import akka.routing.BroadcastRoutingLogic
-
-import scala.collection.immutable.TreeMap
-import com.typesafe.config.Config
+import akka.routing.RoutingLogic
object DistributedPubSubSettings {
@@ -104,7 +105,7 @@ final class DistributedPubSubSettings(
routingLogic: RoutingLogic,
gossipInterval: FiniteDuration,
removedTimeToLive: FiniteDuration,
- maxDeltaElements: Int) {
+ maxDeltaElements: Int) =
this(
role,
routingLogic,
@@ -112,7 +113,6 @@ final class DistributedPubSubSettings(
removedTimeToLive,
maxDeltaElements,
sendToDeadLettersWhenNoSubscribers = true)
- }
require(
!routingLogic.isInstanceOf[ConsistentHashingRoutingLogic],
diff --git a/akka-cluster-tools/src/main/scala/akka/cluster/pubsub/protobuf/DistributedPubSubMessageSerializer.scala b/akka-cluster-tools/src/main/scala/akka/cluster/pubsub/protobuf/DistributedPubSubMessageSerializer.scala
index ea0af0ba29..c5254d750c 100644
--- a/akka-cluster-tools/src/main/scala/akka/cluster/pubsub/protobuf/DistributedPubSubMessageSerializer.scala
+++ b/akka-cluster-tools/src/main/scala/akka/cluster/pubsub/protobuf/DistributedPubSubMessageSerializer.scala
@@ -4,21 +4,23 @@
package akka.cluster.pubsub.protobuf
-import akka.serialization._
-import akka.actor.{ Address, ExtendedActorSystem }
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream }
-import akka.protobufv3.internal.{ ByteString, MessageLite }
-import java.util.zip.GZIPOutputStream
+import java.io.NotSerializableException
import java.util.zip.GZIPInputStream
+import java.util.zip.GZIPOutputStream
+
import scala.annotation.tailrec
-import akka.cluster.pubsub.protobuf.msg.{ DistributedPubSubMessages => dm }
-import akka.util.ccompat.JavaConverters._
+import scala.collection.immutable.TreeMap
+
+import akka.actor.{ Address, ExtendedActorSystem }
+import akka.actor.ActorRef
import akka.cluster.pubsub.DistributedPubSubMediator._
import akka.cluster.pubsub.DistributedPubSubMediator.Internal._
-import akka.actor.ActorRef
+import akka.cluster.pubsub.protobuf.msg.{ DistributedPubSubMessages => dm }
+import akka.protobufv3.internal.{ ByteString, MessageLite }
+import akka.serialization._
import akka.util.ccompat._
-import scala.collection.immutable.TreeMap
-import java.io.NotSerializableException
+import akka.util.ccompat.JavaConverters._
/**
* INTERNAL API: Protobuf serializer of DistributedPubSubMediator messages.
diff --git a/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonManager.scala b/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonManager.scala
index 2c484019ae..70dc6f22d1 100644
--- a/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonManager.scala
+++ b/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonManager.scala
@@ -10,6 +10,8 @@ import scala.concurrent.Promise
import scala.concurrent.duration._
import scala.util.control.NonFatal
+import com.typesafe.config.Config
+
import akka.AkkaException
import akka.Done
import akka.actor.Actor
@@ -26,8 +28,8 @@ import akka.actor.Props
import akka.actor.Terminated
import akka.annotation.DoNotInherit
import akka.annotation.InternalStableApi
-import akka.cluster.ClusterEvent._
import akka.cluster._
+import akka.cluster.ClusterEvent._
import akka.coordination.lease.LeaseUsageSettings
import akka.coordination.lease.scaladsl.Lease
import akka.coordination.lease.scaladsl.LeaseProvider
@@ -38,7 +40,6 @@ import akka.pattern.ask
import akka.pattern.pipe
import akka.util.JavaDurationConverters._
import akka.util.Timeout
-import com.typesafe.config.Config
object ClusterSingletonManagerSettings {
@@ -217,7 +218,7 @@ object ClusterSingletonManager {
final case class HandOverRetry(count: Int)
final case class TakeOverRetry(count: Int)
- final case object LeaseRetry
+ case object LeaseRetry
case object Cleanup
case object StartOldestChangedBuffer
@@ -482,8 +483,8 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
extends Actor
with FSM[ClusterSingletonManager.State, ClusterSingletonManager.Data] {
- import ClusterSingletonManager.Internal.OldestChangedBuffer._
import ClusterSingletonManager.Internal._
+ import ClusterSingletonManager.Internal.OldestChangedBuffer._
import settings._
val cluster = Cluster(context.system)
@@ -535,7 +536,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
removed += node -> (Deadline.now + 15.minutes)
def cleanupOverdueNotMemberAnyMore(): Unit = {
- removed = removed.filter { case (_, deadline) => deadline.hasTimeLeft }
+ removed = removed.filter { case (_, deadline) => deadline.hasTimeLeft() }
}
// for CoordinatedShutdown
@@ -613,7 +614,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
oldestChangedBuffer =
context.actorOf(Props(classOf[OldestChangedBuffer], role).withDispatcher(context.props.dispatcher))
getNextOldestChanged()
- stay
+ stay()
case Event(InitialOldestState(oldest, safeToBeOldest), _) =>
oldestChangedReceived = true
@@ -625,6 +626,10 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
goto(BecomingOldest).using(BecomingOldestData(oldest.filterNot(_ == cluster.selfUniqueAddress)))
else
goto(Younger).using(YoungerData(oldest.filterNot(_ == cluster.selfUniqueAddress)))
+
+ case Event(HandOverToMe, _) =>
+ // nothing to hand over in start
+ stay()
}
when(Younger) {
@@ -648,7 +653,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
case Some(oldest) if !previousOldest.contains(oldest) => oldest :: previousOldest
case _ => previousOldest
}
- stay.using(YoungerData(newPreviousOldest))
+ stay().using(YoungerData(newPreviousOldest))
}
case Event(MemberDowned(m), _) if m.uniqueAddress == cluster.selfUniqueAddress =>
@@ -661,14 +666,14 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
case Event(MemberRemoved(m, _), _) =>
scheduleDelayedMemberRemoved(m)
- stay
+ stay()
case Event(DelayedMemberRemoved(m), YoungerData(previousOldest)) =>
if (!selfExited)
logInfo("Member removed [{}]", m.address)
addRemoved(m.uniqueAddress)
// transition when OldestChanged
- stay.using(YoungerData(previousOldest.filterNot(_ == m.uniqueAddress)))
+ stay().using(YoungerData(previousOldest.filterNot(_ == m.uniqueAddress)))
case Event(HandOverToMe, _) =>
val selfStatus = cluster.selfMember.status
@@ -680,7 +685,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
sender() ! HandOverDone
}
- stay
+ stay()
}
when(BecomingOldest) {
@@ -689,7 +694,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
// confirmation that the hand-over process has started
logInfo("Hand-over in progress at [{}]", sender().path.address)
cancelTimer(HandOverRetryTimer)
- stay
+ stay()
case Event(HandOverDone, BecomingOldestData(previousOldest)) =>
previousOldest.headOption match {
@@ -701,11 +706,11 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
"Ignoring HandOverDone in BecomingOldest from [{}]. Expected previous oldest [{}]",
sender().path.address,
oldest.address)
- stay
+ stay()
}
case None =>
logInfo("Ignoring HandOverDone in BecomingOldest from [{}].", sender().path.address)
- stay
+ stay()
}
case Event(MemberDowned(m), _) if m.uniqueAddress == cluster.selfUniqueAddress =>
@@ -718,7 +723,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
case Event(MemberRemoved(m, _), _) =>
scheduleDelayedMemberRemoved(m)
- stay
+ stay()
case Event(DelayedMemberRemoved(m), BecomingOldestData(previousOldest)) =>
if (!selfExited)
@@ -727,11 +732,11 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
if (cluster.isTerminated) {
// don't act on DelayedMemberRemoved (starting singleton) if this node is shutting its self down,
// just wait for self MemberRemoved
- stay
+ stay()
} else if (previousOldest.contains(m.uniqueAddress) && previousOldest.forall(removed.contains))
tryGotoOldest()
else
- stay.using(BecomingOldestData(previousOldest.filterNot(_ == m.uniqueAddress)))
+ stay().using(BecomingOldestData(previousOldest.filterNot(_ == m.uniqueAddress)))
case Event(TakeOverFromMe, BecomingOldestData(previousOldest)) =>
val senderAddress = sender().path.address
@@ -741,7 +746,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
case None =>
// from unknown node, ignore
logInfo("Ignoring TakeOver request from unknown node in BecomingOldest from [{}].", senderAddress)
- stay
+ stay()
case Some(senderUniqueAddress) =>
previousOldest.headOption match {
case Some(oldest) =>
@@ -752,10 +757,10 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
"Ignoring TakeOver request in BecomingOldest from [{}]. Expected previous oldest [{}]",
sender().path.address,
oldest.address)
- stay
+ stay()
case None =>
sender() ! HandOverToMe
- stay.using(BecomingOldestData(senderUniqueAddress :: previousOldest))
+ stay().using(BecomingOldestData(senderUniqueAddress :: previousOldest))
}
}
@@ -812,7 +817,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
gotoOldest()
} else {
startSingleTimer(LeaseRetryTimer, LeaseRetry, leaseRetryInterval)
- stay.using(AcquiringLeaseData(leaseRequestInProgress = false, None))
+ stay().using(AcquiringLeaseData(leaseRequestInProgress = false, None))
}
case Event(Terminated(ref), AcquiringLeaseData(_, Some(singleton))) if ref == singleton =>
logInfo(
@@ -823,7 +828,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
case Event(AcquireLeaseFailure(t), _) =>
log.error(t, "failed to get lease (will be retried)")
startSingleTimer(LeaseRetryTimer, LeaseRetry, leaseRetryInterval)
- stay.using(AcquiringLeaseData(leaseRequestInProgress = false, None))
+ stay().using(AcquiringLeaseData(leaseRequestInProgress = false, None))
case Event(LeaseRetry, _) =>
// If lease was lost (so previous state was oldest) then we don't try and get the lease
// until the old singleton instance has been terminated so we know there isn't an
@@ -836,12 +841,12 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
case Event(TakeOverFromMe, _) =>
// already oldest, so confirm and continue like that
sender() ! HandOverToMe
- stay
+ stay()
case Event(SelfExiting, _) =>
selfMemberExited()
// complete memberExitingProgress when handOverDone
sender() ! Done // reply to ask
- stay
+ stay()
case Event(MemberDowned(m), _) if m.uniqueAddress == cluster.selfUniqueAddress =>
logInfo("Self downed, stopping ClusterSingletonManager")
stop()
@@ -863,7 +868,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
oldestOption match {
case Some(a) if a == cluster.selfUniqueAddress =>
// already oldest
- stay
+ stay()
case Some(a) if !selfExited && removed.contains(a) =>
// The member removal was not completed and the old removed node is considered
// oldest again. Safest is to terminate the singleton instance and goto Younger.
@@ -889,17 +894,17 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
case Event(TakeOverFromMe, _) =>
// already oldest, so confirm and continue like that
sender() ! HandOverToMe
- stay
+ stay()
case Event(Terminated(ref), d @ OldestData(Some(singleton))) if ref == singleton =>
logInfo(ClusterLogMarker.singletonTerminated, "Singleton actor [{}] was terminated", singleton.path)
- stay.using(d.copy(singleton = None))
+ stay().using(d.copy(singleton = None))
case Event(SelfExiting, _) =>
selfMemberExited()
// complete memberExitingProgress when handOverDone
sender() ! Done // reply to ask
- stay
+ stay()
case Event(MemberDowned(m), OldestData(singleton)) if m.uniqueAddress == cluster.selfUniqueAddress =>
singleton match {
@@ -936,7 +941,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
log.debug("Retry [{}], sending TakeOverFromMe to [{}]", count, newOldestOption.map(_.address))
newOldestOption.foreach(node => peer(node.address) ! TakeOverFromMe)
startSingleTimer(TakeOverRetryTimer, TakeOverRetry(count + 1), handOverRetryInterval)
- stay
+ stay()
} else
throw new ClusterSingletonManagerIsStuck(s"Expected hand-over to [$newOldestOption] never occurred")
@@ -953,13 +958,13 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
case Event(Terminated(ref), d @ WasOldestData(singleton, _)) if singleton.contains(ref) =>
logInfo(ClusterLogMarker.singletonTerminated, "Singleton actor [{}] was terminated", ref.path)
- stay.using(d.copy(singleton = None))
+ stay().using(d.copy(singleton = None))
case Event(SelfExiting, _) =>
selfMemberExited()
// complete memberExitingProgress when handOverDone
sender() ! Done // reply to ask
- stay
+ stay()
case Event(MemberDowned(m), WasOldestData(singleton, _)) if m.uniqueAddress == cluster.selfUniqueAddress =>
singleton match {
@@ -991,13 +996,13 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
case Event(HandOverToMe, HandingOverData(_, handOverTo)) if handOverTo.contains(sender()) =>
// retry
sender() ! HandOverInProgress
- stay
+ stay()
case Event(SelfExiting, _) =>
selfMemberExited()
// complete memberExitingProgress when handOverDone
sender() ! Done // reply to ask
- stay
+ stay()
}
def handOverDone(handOverTo: Option[ActorRef]): State = {
@@ -1049,33 +1054,33 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
selfMemberExited()
memberExitingProgress.trySuccess(Done)
sender() ! Done // reply to ask
- stay
+ stay()
case Event(MemberRemoved(m, _), _) if m.uniqueAddress == cluster.selfUniqueAddress && !selfExited =>
logInfo("Self removed, stopping ClusterSingletonManager")
stop()
case Event(MemberRemoved(m, _), _) =>
if (!selfExited) logInfo("Member removed [{}]", m.address)
addRemoved(m.uniqueAddress)
- stay
+ stay()
case Event(DelayedMemberRemoved(m), _) =>
if (!selfExited) logInfo("Member removed [{}]", m.address)
addRemoved(m.uniqueAddress)
- stay
+ stay()
case Event(TakeOverFromMe, _) =>
log.debug("Ignoring TakeOver request in [{}] from [{}].", stateName, sender().path.address)
- stay
+ stay()
case Event(Cleanup, _) =>
cleanupOverdueNotMemberAnyMore()
- stay
+ stay()
case Event(MemberDowned(m), _) =>
if (m.uniqueAddress == cluster.selfUniqueAddress)
logInfo("Self downed, waiting for removal")
- stay
+ stay()
case Event(ReleaseLeaseFailure(t), _) =>
log.error(
t,
"Failed to release lease. Singleton may not be able to run on another node until lease timeout occurs")
- stay
+ stay()
case Event(ReleaseLeaseResult(released), _) =>
if (released) {
logInfo("Lease released")
@@ -1084,7 +1089,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
log.error(
"Failed to release lease. Singleton may not be able to run on another node until lease timeout occurs")
}
- stay
+ stay()
}
onTransition {
diff --git a/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonProxy.scala b/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonProxy.scala
index da116f5044..4d7718528c 100644
--- a/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonProxy.scala
+++ b/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonProxy.scala
@@ -4,25 +4,25 @@
package akka.cluster.singleton
-import akka.actor._
-import akka.cluster.{ Cluster, Member, MemberStatus }
-
import scala.collection.immutable
-import akka.cluster.ClusterEvent._
-import akka.cluster.ClusterEvent.MemberRemoved
-import akka.cluster.ClusterEvent.MemberUp
+import scala.concurrent.duration._
+
+import com.typesafe.config.Config
+
+import akka.actor._
+import akka.actor.NoSerializationVerificationNeeded
import akka.actor.RootActorPath
+import akka.cluster.{ Cluster, Member, MemberStatus }
+import akka.cluster.ClusterEvent._
import akka.cluster.ClusterEvent.CurrentClusterState
import akka.cluster.ClusterEvent.MemberExited
-
-import scala.concurrent.duration._
-import com.typesafe.config.Config
-import akka.actor.NoSerializationVerificationNeeded
-import akka.event.Logging
-import akka.util.MessageBuffer
+import akka.cluster.ClusterEvent.MemberRemoved
+import akka.cluster.ClusterEvent.MemberUp
import akka.cluster.ClusterSettings
import akka.cluster.ClusterSettings.DataCenter
import akka.dispatch.Dispatchers
+import akka.event.Logging
+import akka.util.MessageBuffer
object ClusterSingletonProxySettings {
@@ -66,8 +66,11 @@ object ClusterSingletonProxySettings {
/**
* @param singletonName The actor name of the singleton actor that is started by the [[ClusterSingletonManager]].
- * @param role The role of the cluster nodes where the singleton can be deployed. If None, then any node will do.
- * @param dataCenter The data center of the cluster nodes where the singleton is running. If None then the same data center as current node.
+ * @param role The role of the cluster nodes where the singleton can be deployed. Corresponding to the `role`
+ * used by the `ClusterSingletonManager`. If the role is not specified it's a singleton among all
+ * nodes in the cluster, and the `ClusterSingletonManager` must then also be configured in
+ * same way.
+ * @param dataCenter The data center of the cluster nodes where the singleton is running. If None then the same data center as current node.
* @param singletonIdentificationInterval Interval at which the proxy will try to resolve the singleton instance.
* @param bufferSize If the location of the singleton is unknown the proxy will buffer this number of messages
* and deliver them when the singleton is identified. When the buffer is full old messages will be dropped
diff --git a/akka-cluster-tools/src/main/scala/akka/cluster/singleton/protobuf/ClusterSingletonMessageSerializer.scala b/akka-cluster-tools/src/main/scala/akka/cluster/singleton/protobuf/ClusterSingletonMessageSerializer.scala
index 65211af633..74dd045382 100644
--- a/akka-cluster-tools/src/main/scala/akka/cluster/singleton/protobuf/ClusterSingletonMessageSerializer.scala
+++ b/akka-cluster-tools/src/main/scala/akka/cluster/singleton/protobuf/ClusterSingletonMessageSerializer.scala
@@ -4,6 +4,8 @@
package akka.cluster.singleton.protobuf
+import java.io.NotSerializableException
+
import akka.actor.ExtendedActorSystem
import akka.cluster.singleton.ClusterSingletonManager.Internal.HandOverDone
import akka.cluster.singleton.ClusterSingletonManager.Internal.HandOverInProgress
@@ -11,7 +13,6 @@ import akka.cluster.singleton.ClusterSingletonManager.Internal.HandOverToMe
import akka.cluster.singleton.ClusterSingletonManager.Internal.TakeOverFromMe
import akka.serialization.BaseSerializer
import akka.serialization.SerializerWithStringManifest
-import java.io.NotSerializableException
/**
* INTERNAL API: Serializer of ClusterSingleton messages.
diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientHandoverSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientHandoverSpec.scala
index 2f01207eb5..001056b9ce 100644
--- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientHandoverSpec.scala
+++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientHandoverSpec.scala
@@ -4,15 +4,16 @@
package akka.cluster.client
+import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
import akka.actor.{ ActorPath, ActorRef }
import akka.cluster.{ Cluster, MultiNodeClusterSpec }
import akka.remote.testconductor.RoleName
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
import akka.testkit.{ ImplicitSender, TestActors }
-import com.typesafe.config.ConfigFactory
-import scala.concurrent.duration._
-
-import com.github.ghik.silencer.silent
object ClusterClientHandoverSpec extends MultiNodeConfig {
val client = role("client")
diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientSpec.scala
index 81a758d2d1..f144abdb77 100644
--- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientSpec.scala
+++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientSpec.scala
@@ -4,10 +4,13 @@
package akka.cluster.client
-import language.postfixOps
+import scala.concurrent.Await
import scala.concurrent.duration._
+import com.github.ghik.silencer.silent
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
import akka.actor.{
Actor,
ActorPath,
@@ -21,18 +24,15 @@ import akka.actor.{
import akka.cluster.Cluster
import akka.cluster.client.ClusterClientSpec.TestClientListener.LatestContactPoints
import akka.cluster.client.ClusterClientSpec.TestReceptionistListener.LatestClusterClients
+import akka.cluster.pubsub._
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
-import akka.testkit._
-import akka.cluster.pubsub._
import akka.remote.transport.ThrottlerTransportAdapter.Direction
+import akka.testkit._
import akka.util.Timeout
import akka.util.unused
-import scala.concurrent.Await
-
-import com.github.ghik.silencer.silent
object ClusterClientSpec extends MultiNodeConfig {
val client = role("client")
@@ -266,12 +266,12 @@ class ClusterClientSpec extends MultiNodeSpec(ClusterClientSpec) with STMultiNod
//#server
runOn(host1) {
- val serviceA = system.actorOf(Props[Service], "serviceA")
+ val serviceA = system.actorOf(Props[Service](), "serviceA")
ClusterClientReceptionist(system).registerService(serviceA)
}
runOn(host2, host3) {
- val serviceB = system.actorOf(Props[Service], "serviceB")
+ val serviceB = system.actorOf(Props[Service](), "serviceB")
ClusterClientReceptionist(system).registerService(serviceB)
}
//#server
diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientStopSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientStopSpec.scala
index 0eb0db64d0..f13370417a 100644
--- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientStopSpec.scala
+++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientStopSpec.scala
@@ -4,17 +4,18 @@
package akka.cluster.client
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
import akka.actor.{ Actor, Props }
import akka.cluster.Cluster
import akka.cluster.pubsub.{ DistributedPubSub, DistributedPubSubMediator }
import akka.remote.testconductor.RoleName
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
import akka.testkit.{ EventFilter, ImplicitSender }
-import com.typesafe.config.ConfigFactory
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
-import com.github.ghik.silencer.silent
object ClusterClientStopSpec extends MultiNodeConfig {
val client = role("client")
diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubMediatorSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubMediatorSpec.scala
index f410eceb18..6adf1ed39e 100644
--- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubMediatorSpec.scala
+++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubMediatorSpec.scala
@@ -4,22 +4,24 @@
package akka.cluster.pubsub
-import language.postfixOps
import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
import akka.actor.Actor
+import akka.actor.ActorLogging
import akka.actor.ActorRef
import akka.actor.PoisonPill
import akka.actor.Props
import akka.cluster.Cluster
+import akka.cluster.pubsub.DistributedPubSubMediator.Internal.Delta
+import akka.cluster.pubsub.DistributedPubSubMediator.Internal.Status
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
import akka.testkit._
-import akka.actor.ActorLogging
-import akka.cluster.pubsub.DistributedPubSubMediator.Internal.Status
-import akka.cluster.pubsub.DistributedPubSubMediator.Internal.Delta
object DistributedPubSubMediatorSpec extends MultiNodeConfig {
val first = role("first")
@@ -46,8 +48,8 @@ object DistributedPubSubMediatorSpec extends MultiNodeConfig {
}
class TestChatUser(mediator: ActorRef, testActor: ActorRef) extends Actor {
- import TestChatUser._
import DistributedPubSubMediator._
+ import TestChatUser._
def receive = {
case Whisper(path, msg) => mediator ! Send(path, msg, localAffinity = true)
@@ -129,9 +131,9 @@ class DistributedPubSubMediatorSpec
extends MultiNodeSpec(DistributedPubSubMediatorSpec)
with STMultiNodeSpec
with ImplicitSender {
+ import DistributedPubSubMediator._
import DistributedPubSubMediatorSpec._
import DistributedPubSubMediatorSpec.TestChatUser._
- import DistributedPubSubMediator._
override def initialParticipants = roles.size
@@ -344,17 +346,17 @@ class DistributedPubSubMediatorSpec
//#start-subscribers
runOn(first) {
- system.actorOf(Props[Subscriber], "subscriber1")
+ system.actorOf(Props[Subscriber](), "subscriber1")
}
runOn(second) {
- system.actorOf(Props[Subscriber], "subscriber2")
- system.actorOf(Props[Subscriber], "subscriber3")
+ system.actorOf(Props[Subscriber](), "subscriber2")
+ system.actorOf(Props[Subscriber](), "subscriber3")
}
//#start-subscribers
//#publish-message
runOn(third) {
- val publisher = system.actorOf(Props[Publisher], "publisher")
+ val publisher = system.actorOf(Props[Publisher](), "publisher")
later()
// after a while the subscriptions are replicated
publisher ! "hello"
@@ -371,16 +373,16 @@ class DistributedPubSubMediatorSpec
//#start-send-destinations
runOn(first) {
- system.actorOf(Props[Destination], "destination")
+ system.actorOf(Props[Destination](), "destination")
}
runOn(second) {
- system.actorOf(Props[Destination], "destination")
+ system.actorOf(Props[Destination](), "destination")
}
//#start-send-destinations
//#send-message
runOn(third) {
- val sender = system.actorOf(Props[Sender], "sender")
+ val sender = system.actorOf(Props[Sender](), "sender")
later()
// after a while the destinations are replicated
sender ! "hello"
diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubRestartSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubRestartSpec.scala
index 4ad1b6f7c4..1a0f7c472f 100644
--- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubRestartSpec.scala
+++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubRestartSpec.scala
@@ -4,24 +4,25 @@
package akka.cluster.pubsub
-import language.postfixOps
+import scala.concurrent.Await
import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
import akka.actor.Actor
+import akka.actor.ActorIdentity
import akka.actor.ActorRef
+import akka.actor.ActorSystem
+import akka.actor.Identify
import akka.actor.Props
+import akka.actor.RootActorPath
import akka.cluster.Cluster
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
import akka.testkit._
-import akka.actor.ActorSystem
-
-import scala.concurrent.Await
-import akka.actor.Identify
-import akka.actor.RootActorPath
-import akka.actor.ActorIdentity
object DistributedPubSubRestartSpec extends MultiNodeConfig {
val first = role("first")
@@ -55,8 +56,8 @@ class DistributedPubSubRestartSpec
extends MultiNodeSpec(DistributedPubSubRestartSpec)
with STMultiNodeSpec
with ImplicitSender {
- import DistributedPubSubRestartSpec._
import DistributedPubSubMediator._
+ import DistributedPubSubRestartSpec._
override def initialParticipants = roles.size
@@ -161,7 +162,7 @@ class DistributedPubSubRestartSpec
newMediator.tell(Internal.DeltaCount, probe.ref)
probe.expectMsg(0L)
- newSystem.actorOf(Props[Shutdown], "shutdown")
+ newSystem.actorOf(Props[Shutdown](), "shutdown")
Await.ready(newSystem.whenTerminated, 20.seconds)
} finally newSystem.terminate()
}
diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerChaosSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerChaosSpec.scala
index 3b4884ca91..883104cb6d 100644
--- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerChaosSpec.scala
+++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerChaosSpec.scala
@@ -4,13 +4,16 @@
package akka.cluster.singleton
-import language.postfixOps
import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
import akka.actor.Actor
import akka.actor.ActorRef
-import akka.actor.Props
+import akka.actor.ActorSelection
import akka.actor.PoisonPill
+import akka.actor.Props
import akka.actor.RootActorPath
import akka.cluster.Cluster
import akka.cluster.ClusterEvent._
@@ -20,7 +23,6 @@ import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
import akka.testkit._
import akka.testkit.TestEvent._
-import akka.actor.ActorSelection
object ClusterSingletonManagerChaosSpec extends MultiNodeConfig {
val controller = role("controller")
diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerDownedSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerDownedSpec.scala
index 90f8e85f41..f0b6b51a94 100644
--- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerDownedSpec.scala
+++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerDownedSpec.scala
@@ -5,6 +5,9 @@
package akka.cluster.singleton
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.PoisonPill
@@ -18,7 +21,6 @@ import akka.remote.testkit.STMultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter
import akka.testkit._
import akka.util.ccompat._
-import com.typesafe.config.ConfigFactory
@ccompatUsedUntil213
object ClusterSingletonManagerDownedSpec extends MultiNodeConfig {
diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeaseSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeaseSpec.scala
index 30819d1189..4ee6fbb708 100644
--- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeaseSpec.scala
+++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeaseSpec.scala
@@ -4,16 +4,19 @@
package akka.cluster.singleton
-import akka.actor.{ Actor, ActorIdentity, ActorLogging, ActorRef, Address, Identify, PoisonPill, Props }
-import akka.cluster.MemberStatus.Up
-import akka.cluster.TestLeaseActor._
-import akka.cluster.singleton.ClusterSingletonManagerLeaseSpec.ImportantSingleton.Response
-import akka.cluster._
-import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
-import akka.testkit._
+import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
-import scala.concurrent.duration._
+import akka.actor.{ Actor, ActorIdentity, ActorLogging, ActorRef, Address, Identify, PoisonPill, Props }
+import akka.cluster._
+import akka.cluster.MemberStatus.Up
+import akka.cluster.singleton.ClusterSingletonManagerLeaseSpec.ImportantSingleton.Response
+import akka.coordination.lease.TestLeaseActor
+import akka.coordination.lease.TestLeaseActorClient
+import akka.coordination.lease.TestLeaseActorClientExt
+import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
+import akka.testkit._
object ClusterSingletonManagerLeaseSpec extends MultiNodeConfig {
val controller = role("controller")
@@ -24,14 +27,14 @@ object ClusterSingletonManagerLeaseSpec extends MultiNodeConfig {
testTransport(true)
- commonConfig(ConfigFactory.parseString("""
+ commonConfig(ConfigFactory.parseString(s"""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off
akka.cluster.downing-provider-class = akka.cluster.testkit.AutoDowning
akka.cluster.testkit.auto-down-unreachable-after = 0s
test-lease {
- lease-class = akka.cluster.TestLeaseActorClient
+ lease-class = ${classOf[TestLeaseActorClient].getName}
heartbeat-interval = 1s
heartbeat-timeout = 120s
lease-operation-timeout = 3s
@@ -76,8 +79,9 @@ class ClusterSingletonManagerLeaseSpec
with ImplicitSender
with MultiNodeClusterSpec {
- import ClusterSingletonManagerLeaseSpec.ImportantSingleton._
import ClusterSingletonManagerLeaseSpec._
+ import ClusterSingletonManagerLeaseSpec.ImportantSingleton._
+ import TestLeaseActor._
override def initialParticipants = roles.size
@@ -127,10 +131,11 @@ class ClusterSingletonManagerLeaseSpec
}
"Start singleton and ping from all nodes" in {
- runOn(first, second, third, fourth) {
+ // fourth doesn't have the worker role
+ runOn(first, second, third) {
system.actorOf(
ClusterSingletonManager
- .props(props(), PoisonPill, ClusterSingletonManagerSettings(system).withRole("worker")),
+ .props(ImportantSingleton.props(), PoisonPill, ClusterSingletonManagerSettings(system).withRole("worker")),
"important")
}
enterBarrier("singleton-started")
diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeave2Spec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeave2Spec.scala
index aaaee797ef..54a0c512da 100644
--- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeave2Spec.scala
+++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeave2Spec.scala
@@ -6,6 +6,8 @@ package akka.cluster.singleton
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.ActorLogging
@@ -20,7 +22,6 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
import akka.testkit._
-import com.typesafe.config.ConfigFactory
object ClusterSingletonManagerLeave2Spec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeaveSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeaveSpec.scala
index a4e6f4ba42..538abee0b1 100644
--- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeaveSpec.scala
+++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeaveSpec.scala
@@ -5,17 +5,19 @@
package akka.cluster.singleton
import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
import akka.cluster.Cluster
+import akka.cluster.MemberStatus
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
import akka.testkit._
-import akka.cluster.MemberStatus
object ClusterSingletonManagerLeaveSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerSpec.scala
index 29d8d7c52f..b73133e5d8 100644
--- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerSpec.scala
+++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerSpec.scala
@@ -4,13 +4,17 @@
package akka.cluster.singleton
-import language.postfixOps
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
import akka.actor.Actor
+import akka.actor.ActorIdentity
import akka.actor.ActorLogging
import akka.actor.ActorRef
+import akka.actor.ActorSelection
+import akka.actor.Identify
import akka.actor.Props
import akka.actor.RootActorPath
import akka.cluster.Cluster
@@ -19,12 +23,9 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
+import akka.serialization.jackson.CborSerializable
import akka.testkit._
import akka.testkit.TestEvent._
-import akka.actor.Identify
-import akka.actor.ActorIdentity
-import akka.actor.ActorSelection
-import akka.serialization.jackson.CborSerializable
object ClusterSingletonManagerSpec extends MultiNodeConfig {
val controller = role("controller")
@@ -172,8 +173,8 @@ class ClusterSingletonManagerSpec
with ImplicitSender {
import ClusterSingletonManagerSpec._
- import ClusterSingletonManagerSpec.PointToPointChannel._
import ClusterSingletonManagerSpec.Consumer._
+ import ClusterSingletonManagerSpec.PointToPointChannel._
override def initialParticipants = roles.size
@@ -333,7 +334,7 @@ class ClusterSingletonManagerSpec
runOn(controller) {
// watch that it is not terminated, which would indicate misbehavior
- watch(system.actorOf(Props[PointToPointChannel], "queue"))
+ watch(system.actorOf(Props[PointToPointChannel](), "queue"))
}
enterBarrier("queue-started")
diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerStartupSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerStartupSpec.scala
index 913ae322e9..503bc5b57c 100644
--- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerStartupSpec.scala
+++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerStartupSpec.scala
@@ -5,18 +5,20 @@
package akka.cluster.singleton
import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorRef
-import akka.actor.Props
import akka.actor.PoisonPill
+import akka.actor.Props
import akka.cluster.Cluster
+import akka.cluster.MemberStatus
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
import akka.testkit._
-import akka.cluster.MemberStatus
object ClusterSingletonManagerStartupSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/MultiDcSingletonManagerSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/MultiDcSingletonManagerSpec.scala
index 521841a45f..c3345a5622 100644
--- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/MultiDcSingletonManagerSpec.scala
+++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/MultiDcSingletonManagerSpec.scala
@@ -7,12 +7,13 @@ package akka.cluster.singleton
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
+
import akka.actor.{ Actor, ActorLogging, Address, PoisonPill, Props }
import akka.cluster.Cluster
-import akka.testkit.ImplicitSender
-import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
import akka.cluster.ClusterSettings
+import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
import akka.serialization.jackson.CborSerializable
+import akka.testkit.ImplicitSender
object MultiDcSingletonManagerSpec extends MultiNodeConfig {
val controller = role("controller")
diff --git a/akka-cluster-tools/src/test/scala/akka/cluster/client/protobuf/ClusterClientMessageSerializerSpec.scala b/akka-cluster-tools/src/test/scala/akka/cluster/client/protobuf/ClusterClientMessageSerializerSpec.scala
index 97a4195498..42a145e6e4 100644
--- a/akka-cluster-tools/src/test/scala/akka/cluster/client/protobuf/ClusterClientMessageSerializerSpec.scala
+++ b/akka-cluster-tools/src/test/scala/akka/cluster/client/protobuf/ClusterClientMessageSerializerSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster.client.protobuf
-import akka.actor.ExtendedActorSystem
-import akka.testkit.AkkaSpec
-import akka.cluster.client.ClusterReceptionist.Internal._
import com.github.ghik.silencer.silent
+import akka.actor.ExtendedActorSystem
+import akka.cluster.client.ClusterReceptionist.Internal._
+import akka.testkit.AkkaSpec
+
@silent("deprecated")
class ClusterClientMessageSerializerSpec extends AkkaSpec {
diff --git a/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/DistributedPubSubMediatorDeadLettersSpec.scala b/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/DistributedPubSubMediatorDeadLettersSpec.scala
index 37bb7835f0..e7726c3189 100644
--- a/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/DistributedPubSubMediatorDeadLettersSpec.scala
+++ b/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/DistributedPubSubMediatorDeadLettersSpec.scala
@@ -4,10 +4,11 @@
package akka.cluster.pubsub
+import scala.concurrent.duration._
+
import akka.actor.DeadLetter
import akka.cluster.pubsub.DistributedPubSubMediator.{ Subscribe, _ }
import akka.testkit._
-import scala.concurrent.duration._
object DistributedPubSubMediatorDeadLettersSpec {
def config(sendToDeadLettersWhenNoSubscribers: Boolean) =
diff --git a/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/DistributedPubSubMediatorRouterSpec.scala b/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/DistributedPubSubMediatorRouterSpec.scala
index 45d129a51b..d5db6004a9 100644
--- a/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/DistributedPubSubMediatorRouterSpec.scala
+++ b/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/DistributedPubSubMediatorRouterSpec.scala
@@ -4,12 +4,13 @@
package akka.cluster.pubsub
-import akka.testkit._
-import akka.routing.{ ConsistentHashingRoutingLogic, RouterEnvelope }
-import akka.actor.ActorRef
import com.typesafe.config.ConfigFactory
import org.scalatest.wordspec.AnyWordSpecLike
+import akka.actor.ActorRef
+import akka.routing.{ ConsistentHashingRoutingLogic, RouterEnvelope }
+import akka.testkit._
+
case class WrappedMessage(msg: String) extends RouterEnvelope {
override def message = msg
}
diff --git a/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/protobuf/DistributedPubSubMessageSerializerSpec.scala b/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/protobuf/DistributedPubSubMessageSerializerSpec.scala
index f85fbcbd97..c60e47eac3 100644
--- a/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/protobuf/DistributedPubSubMessageSerializerSpec.scala
+++ b/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/protobuf/DistributedPubSubMessageSerializerSpec.scala
@@ -4,12 +4,13 @@
package akka.cluster.pubsub.protobuf
+import scala.collection.immutable.TreeMap
+
import akka.actor.{ Address, ExtendedActorSystem }
-import akka.testkit.AkkaSpec
+import akka.actor.Props
import akka.cluster.pubsub.DistributedPubSubMediator._
import akka.cluster.pubsub.DistributedPubSubMediator.Internal._
-import akka.actor.Props
-import scala.collection.immutable.TreeMap
+import akka.testkit.AkkaSpec
class DistributedPubSubMessageSerializerSpec extends AkkaSpec {
diff --git a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonLeaseSpec.scala b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonLeaseSpec.scala
index 23a3616e7c..63bc4e4eab 100644
--- a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonLeaseSpec.scala
+++ b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonLeaseSpec.scala
@@ -10,6 +10,8 @@ import scala.concurrent.Promise
import scala.concurrent.duration._
import scala.util.Success
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorRef
@@ -18,14 +20,11 @@ import akka.actor.PoisonPill
import akka.actor.Props
import akka.cluster.Cluster
import akka.cluster.MemberStatus
-import akka.cluster.TestLease
-import akka.cluster.TestLease.AcquireReq
-import akka.cluster.TestLease.ReleaseReq
-import akka.cluster.TestLeaseExt
+import akka.coordination.lease.TestLease
+import akka.coordination.lease.TestLeaseExt
import akka.testkit.AkkaSpec
import akka.testkit.TestException
import akka.testkit.TestProbe
-import com.typesafe.config.ConfigFactory
class ImportantSingleton(lifeCycleProbe: ActorRef) extends Actor with ActorLogging {
@@ -54,6 +53,7 @@ class ClusterSingletonLeaseSpec extends AkkaSpec(ConfigFactory.parseString("""
lease-retry-interval = 2000ms
}
""").withFallback(TestLease.config)) {
+ import TestLease.{ AcquireReq, ReleaseReq }
val cluster = Cluster(system)
val testLeaseExt = TestLeaseExt(system)
@@ -121,7 +121,7 @@ class ClusterSingletonLeaseSpec extends AkkaSpec(ConfigFactory.parseString("""
} // allow singleton manager to create the lease
testLease.probe.expectMsg(AcquireReq(leaseOwner))
singletonProbe.expectNoMessage(shortDuration)
- val nextResponse = Promise[Boolean]
+ val nextResponse = Promise[Boolean]()
testLease.setNextAcquireResult(nextResponse.future)
testLease.initialPromise.complete(Success(false))
testLease.probe.expectMsg(AcquireReq(leaseOwner))
@@ -155,7 +155,7 @@ class ClusterSingletonLeaseSpec extends AkkaSpec(ConfigFactory.parseString("""
} // allow singleton manager to create the lease
testLease.probe.expectMsg(AcquireReq(leaseOwner))
singletonProbe.expectNoMessage(shortDuration)
- val nextResponse = Promise[Boolean]
+ val nextResponse = Promise[Boolean]()
testLease.setNextAcquireResult(nextResponse.future)
testLease.initialPromise.failure(TestException("no lease for you"))
testLease.probe.expectMsg(AcquireReq(leaseOwner))
diff --git a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala
index c250187461..d2bb0e8b03 100644
--- a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala
+++ b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala
@@ -6,6 +6,8 @@ package akka.cluster.singleton
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.ActorSystem
@@ -17,7 +19,6 @@ import akka.cluster.MemberStatus
import akka.cluster.singleton.ClusterSingletonLeavingSpeedSpec.TheSingleton
import akka.testkit.AkkaSpec
import akka.testkit.TestProbe
-import com.typesafe.config.ConfigFactory
object ClusterSingletonLeavingSpeedSpec {
diff --git a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonProxySpec.scala b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonProxySpec.scala
index 56cba45dd8..d682ebc3ff 100644
--- a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonProxySpec.scala
+++ b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonProxySpec.scala
@@ -4,15 +4,17 @@
package akka.cluster.singleton
-import akka.testkit.{ TestKit, TestProbe }
-import akka.actor._
-import com.typesafe.config.ConfigFactory
-import akka.cluster.Cluster
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfterAll
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
+import akka.actor._
+import akka.cluster.Cluster
+import akka.testkit.{ TestKit, TestProbe }
+
class ClusterSingletonProxySpec extends AnyWordSpecLike with Matchers with BeforeAndAfterAll {
import ClusterSingletonProxySpec._
@@ -47,7 +49,7 @@ object ClusterSingletonProxySpec {
cluster.registerOnMemberUp {
system.actorOf(
ClusterSingletonManager.props(
- singletonProps = Props[Singleton],
+ singletonProps = Props[Singleton](),
terminationMessage = PoisonPill,
settings = ClusterSingletonManagerSettings(system).withRemovalMargin(5.seconds)),
name = "singletonManager")
diff --git a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonRestart2Spec.scala b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonRestart2Spec.scala
index 9ae476a3a3..bd22055c90 100644
--- a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonRestart2Spec.scala
+++ b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonRestart2Spec.scala
@@ -5,6 +5,9 @@
package akka.cluster.singleton
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorSystem
import akka.actor.PoisonPill
@@ -14,7 +17,6 @@ import akka.cluster.MemberStatus
import akka.cluster.UniqueAddress
import akka.testkit.AkkaSpec
import akka.testkit.TestProbe
-import com.typesafe.config.ConfigFactory
object ClusterSingletonRestart2Spec {
def singletonActorProps: Props = Props(new Singleton)
diff --git a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonRestartSpec.scala b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonRestartSpec.scala
index 9484866194..34cc68bbd7 100644
--- a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonRestartSpec.scala
+++ b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonRestartSpec.scala
@@ -5,6 +5,9 @@
package akka.cluster.singleton
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ActorSystem
import akka.actor.PoisonPill
import akka.cluster.Cluster
@@ -12,7 +15,6 @@ import akka.cluster.MemberStatus
import akka.testkit.AkkaSpec
import akka.testkit.TestActors
import akka.testkit.TestProbe
-import com.typesafe.config.ConfigFactory
class ClusterSingletonRestartSpec
extends AkkaSpec("""
diff --git a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/protobuf/ClusterSingletonMessageSerializerSpec.scala b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/protobuf/ClusterSingletonMessageSerializerSpec.scala
index 0d7f1e6991..80295c7fc3 100644
--- a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/protobuf/ClusterSingletonMessageSerializerSpec.scala
+++ b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/protobuf/ClusterSingletonMessageSerializerSpec.scala
@@ -5,11 +5,11 @@
package akka.cluster.singleton.protobuf
import akka.actor.ExtendedActorSystem
-import akka.testkit.AkkaSpec
import akka.cluster.singleton.ClusterSingletonManager.Internal.HandOverDone
import akka.cluster.singleton.ClusterSingletonManager.Internal.HandOverInProgress
import akka.cluster.singleton.ClusterSingletonManager.Internal.HandOverToMe
import akka.cluster.singleton.ClusterSingletonManager.Internal.TakeOverFromMe
+import akka.testkit.AkkaSpec
class ClusterSingletonMessageSerializerSpec extends AkkaSpec {
diff --git a/akka-cluster-typed/src/main/java/akka/cluster/typed/internal/protobuf/ClusterMessages.java b/akka-cluster-typed/src/main/java/akka/cluster/typed/internal/protobuf/ClusterMessages.java
index 609da28988..99e40644c0 100644
--- a/akka-cluster-typed/src/main/java/akka/cluster/typed/internal/protobuf/ClusterMessages.java
+++ b/akka-cluster-typed/src/main/java/akka/cluster/typed/internal/protobuf/ClusterMessages.java
@@ -49,6 +49,17 @@ public final class ClusterMessages {
* @return The systemUid.
*/
long getSystemUid();
+
+ /**
+ * optional int64 createdTimestamp = 3;
+ * @return Whether the createdTimestamp field is set.
+ */
+ boolean hasCreatedTimestamp();
+ /**
+ * optional int64 createdTimestamp = 3;
+ * @return The createdTimestamp.
+ */
+ long getCreatedTimestamp();
}
/**
* Protobuf type {@code akka.cluster.typed.ReceptionistEntry}
@@ -108,6 +119,11 @@ public final class ClusterMessages {
systemUid_ = input.readUInt64();
break;
}
+ case 24: {
+ bitField0_ |= 0x00000004;
+ createdTimestamp_ = input.readInt64();
+ break;
+ }
default: {
if (!parseUnknownField(
input, unknownFields, extensionRegistry, tag)) {
@@ -203,6 +219,23 @@ public final class ClusterMessages {
return systemUid_;
}
+ public static final int CREATEDTIMESTAMP_FIELD_NUMBER = 3;
+ private long createdTimestamp_;
+ /**
+ * optional int64 createdTimestamp = 3;
+ * @return Whether the createdTimestamp field is set.
+ */
+ public boolean hasCreatedTimestamp() {
+ return ((bitField0_ & 0x00000004) != 0);
+ }
+ /**
+ * optional int64 createdTimestamp = 3;
+ * @return The createdTimestamp.
+ */
+ public long getCreatedTimestamp() {
+ return createdTimestamp_;
+ }
+
private byte memoizedIsInitialized = -1;
@java.lang.Override
public final boolean isInitialized() {
@@ -231,6 +264,9 @@ public final class ClusterMessages {
if (((bitField0_ & 0x00000002) != 0)) {
output.writeUInt64(2, systemUid_);
}
+ if (((bitField0_ & 0x00000004) != 0)) {
+ output.writeInt64(3, createdTimestamp_);
+ }
unknownFields.writeTo(output);
}
@@ -247,6 +283,10 @@ public final class ClusterMessages {
size += akka.protobufv3.internal.CodedOutputStream
.computeUInt64Size(2, systemUid_);
}
+ if (((bitField0_ & 0x00000004) != 0)) {
+ size += akka.protobufv3.internal.CodedOutputStream
+ .computeInt64Size(3, createdTimestamp_);
+ }
size += unknownFields.getSerializedSize();
memoizedSize = size;
return size;
@@ -272,6 +312,11 @@ public final class ClusterMessages {
if (getSystemUid()
!= other.getSystemUid()) return false;
}
+ if (hasCreatedTimestamp() != other.hasCreatedTimestamp()) return false;
+ if (hasCreatedTimestamp()) {
+ if (getCreatedTimestamp()
+ != other.getCreatedTimestamp()) return false;
+ }
if (!unknownFields.equals(other.unknownFields)) return false;
return true;
}
@@ -292,6 +337,11 @@ public final class ClusterMessages {
hash = (53 * hash) + akka.protobufv3.internal.Internal.hashLong(
getSystemUid());
}
+ if (hasCreatedTimestamp()) {
+ hash = (37 * hash) + CREATEDTIMESTAMP_FIELD_NUMBER;
+ hash = (53 * hash) + akka.protobufv3.internal.Internal.hashLong(
+ getCreatedTimestamp());
+ }
hash = (29 * hash) + unknownFields.hashCode();
memoizedHashCode = hash;
return hash;
@@ -429,6 +479,8 @@ public final class ClusterMessages {
bitField0_ = (bitField0_ & ~0x00000001);
systemUid_ = 0L;
bitField0_ = (bitField0_ & ~0x00000002);
+ createdTimestamp_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000004);
return this;
}
@@ -465,6 +517,10 @@ public final class ClusterMessages {
result.systemUid_ = systemUid_;
to_bitField0_ |= 0x00000002;
}
+ if (((from_bitField0_ & 0x00000004) != 0)) {
+ result.createdTimestamp_ = createdTimestamp_;
+ to_bitField0_ |= 0x00000004;
+ }
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@@ -522,6 +578,9 @@ public final class ClusterMessages {
if (other.hasSystemUid()) {
setSystemUid(other.getSystemUid());
}
+ if (other.hasCreatedTimestamp()) {
+ setCreatedTimestamp(other.getCreatedTimestamp());
+ }
this.mergeUnknownFields(other.unknownFields);
onChanged();
return this;
@@ -678,6 +737,43 @@ public final class ClusterMessages {
onChanged();
return this;
}
+
+ private long createdTimestamp_ ;
+ /**
+ * optional int64 createdTimestamp = 3;
+ * @return Whether the createdTimestamp field is set.
+ */
+ public boolean hasCreatedTimestamp() {
+ return ((bitField0_ & 0x00000004) != 0);
+ }
+ /**
+ * optional int64 createdTimestamp = 3;
+ * @return The createdTimestamp.
+ */
+ public long getCreatedTimestamp() {
+ return createdTimestamp_;
+ }
+ /**
+ * optional int64 createdTimestamp = 3;
+ * @param value The createdTimestamp to set.
+ * @return This builder for chaining.
+ */
+ public Builder setCreatedTimestamp(long value) {
+ bitField0_ |= 0x00000004;
+ createdTimestamp_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * optional int64 createdTimestamp = 3;
+ * @return This builder for chaining.
+ */
+ public Builder clearCreatedTimestamp() {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ createdTimestamp_ = 0L;
+ onChanged();
+ return this;
+ }
@java.lang.Override
public final Builder setUnknownFields(
final akka.protobufv3.internal.UnknownFieldSet unknownFields) {
@@ -1394,11 +1490,12 @@ public final class ClusterMessages {
static {
java.lang.String[] descriptorData = {
"\n\025ClusterMessages.proto\022\022akka.cluster.ty" +
- "ped\032\026ContainerFormats.proto\"8\n\021Reception" +
+ "ped\032\026ContainerFormats.proto\"R\n\021Reception" +
"istEntry\022\020\n\010actorRef\030\001 \002(\t\022\021\n\tsystemUid\030" +
- "\002 \002(\004\"3\n\026PubSubMessagePublished\022\031\n\007messa" +
- "ge\030\001 \002(\0132\010.PayloadB(\n$akka.cluster.typed" +
- ".internal.protobufH\001"
+ "\002 \002(\004\022\030\n\020createdTimestamp\030\003 \001(\003\"3\n\026PubSu" +
+ "bMessagePublished\022\031\n\007message\030\001 \002(\0132\010.Pay" +
+ "loadB(\n$akka.cluster.typed.internal.prot" +
+ "obufH\001"
};
descriptor = akka.protobufv3.internal.Descriptors.FileDescriptor
.internalBuildGeneratedFileFrom(descriptorData,
@@ -1410,7 +1507,7 @@ public final class ClusterMessages {
internal_static_akka_cluster_typed_ReceptionistEntry_fieldAccessorTable = new
akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable(
internal_static_akka_cluster_typed_ReceptionistEntry_descriptor,
- new java.lang.String[] { "ActorRef", "SystemUid", });
+ new java.lang.String[] { "ActorRef", "SystemUid", "CreatedTimestamp", });
internal_static_akka_cluster_typed_PubSubMessagePublished_descriptor =
getDescriptor().getMessageTypes().get(1);
internal_static_akka_cluster_typed_PubSubMessagePublished_fieldAccessorTable = new
diff --git a/akka-cluster-typed/src/main/mima-filters/2.6.4.backwards.excludes/issue-28792-ClusterReceptionist.excludes b/akka-cluster-typed/src/main/mima-filters/2.6.4.backwards.excludes/issue-28792-ClusterReceptionist.excludes
new file mode 100644
index 0000000000..7c62477881
--- /dev/null
+++ b/akka-cluster-typed/src/main/mima-filters/2.6.4.backwards.excludes/issue-28792-ClusterReceptionist.excludes
@@ -0,0 +1,3 @@
+# #28792 Changes to internals of ClusterReceptionist
+ProblemFilters.exclude[Problem]("akka.cluster.typed.internal.receptionist.*")
+ProblemFilters.exclude[Problem]("akka.cluster.typed.internal.protobuf.*")
diff --git a/akka-cluster-typed/src/main/protobuf/ClusterMessages.proto b/akka-cluster-typed/src/main/protobuf/ClusterMessages.proto
index 7b77eafc47..6127847db6 100644
--- a/akka-cluster-typed/src/main/protobuf/ClusterMessages.proto
+++ b/akka-cluster-typed/src/main/protobuf/ClusterMessages.proto
@@ -14,8 +14,9 @@ import "ContainerFormats.proto";
message ReceptionistEntry {
required string actorRef = 1;
required uint64 systemUid = 2;
+ optional int64 createdTimestamp = 3;
}
message PubSubMessagePublished {
required Payload message = 1;
-}
\ No newline at end of file
+}
diff --git a/akka-cluster-typed/src/main/resources/reference.conf b/akka-cluster-typed/src/main/resources/reference.conf
index 45d36c6eca..4cd45a5d24 100644
--- a/akka-cluster-typed/src/main/resources/reference.conf
+++ b/akka-cluster-typed/src/main/resources/reference.conf
@@ -14,6 +14,11 @@ akka.cluster.typed.receptionist {
# in case of abrupt termination.
pruning-interval = 3 s
+ # The periodic task to remove actor references that are hosted by removed nodes
+ # will only remove entries older than this duration. The reason for this
+ # is to avoid removing entries of nodes that haven't been visible as joining.
+ prune-removed-older-than = 60 s
+
# Shard the services over this many Distributed Data keys, with large amounts of different
# service keys storing all of them in the same Distributed Data entry would lead to large updates
# etc. instead the keys are sharded across this number of keys. This must be the same on all nodes
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/internal/ReplicatorBehavior.scala b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/internal/ReplicatorBehavior.scala
index c9fa464205..63490c927b 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/internal/ReplicatorBehavior.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/internal/ReplicatorBehavior.scala
@@ -6,17 +6,17 @@ package akka.cluster.ddata.typed.internal
import scala.concurrent.duration._
-import akka.annotation.InternalApi
-import akka.cluster.{ ddata => dd }
-import akka.pattern.ask
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
+import akka.actor.typed.Terminated
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
-import akka.util.Timeout
-import akka.util.JavaDurationConverters._
+import akka.annotation.InternalApi
+import akka.cluster.{ ddata => dd }
import akka.cluster.ddata.ReplicatedData
-import akka.actor.typed.Terminated
+import akka.pattern.ask
+import akka.util.JavaDurationConverters._
+import akka.util.Timeout
/**
* INTERNAL API
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/DistributedData.scala b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/DistributedData.scala
index 7637a34c53..d60d740736 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/DistributedData.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/DistributedData.scala
@@ -6,11 +6,11 @@ package akka.cluster.ddata.typed.javadsl
import java.util.function.{ Function => JFunction }
+import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
+import akka.actor.typed.Behavior
import akka.actor.typed.Extension
import akka.actor.typed.ExtensionId
-import akka.actor.typed.ActorRef
-import akka.actor.typed.Behavior
import akka.actor.typed.ExtensionSetup
import akka.actor.typed.javadsl.Behaviors
import akka.annotation.DoNotInherit
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/Replicator.scala b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/Replicator.scala
index ef5092160a..8aeeea410d 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/Replicator.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/Replicator.scala
@@ -7,16 +7,16 @@ package akka.cluster.ddata.typed.javadsl
import java.time.Duration
import java.util.function.{ Function => JFunction }
-import akka.actor.typed.ActorRef
-import akka.actor.typed.Behavior
import akka.actor.DeadLetterSuppression
import akka.actor.NoSerializationVerificationNeeded
+import akka.actor.typed.ActorRef
+import akka.actor.typed.Behavior
import akka.annotation.DoNotInherit
import akka.annotation.InternalApi
+import akka.cluster.{ ddata => dd }
import akka.cluster.ddata.Key
import akka.cluster.ddata.ReplicatedData
import akka.cluster.ddata.typed.internal.ReplicatorBehavior
-import akka.cluster.{ ddata => dd }
import akka.util.JavaDurationConverters._
/**
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/ReplicatorMessageAdapter.scala b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/ReplicatorMessageAdapter.scala
index 077a18e922..6f8065496d 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/ReplicatorMessageAdapter.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/ReplicatorMessageAdapter.scala
@@ -10,13 +10,14 @@ import java.util.function.{ Function => JFunction }
import scala.util.Failure
import scala.util.Success
-import akka.util.JavaDurationConverters._
+import com.github.ghik.silencer.silent
+
import akka.actor.typed.ActorRef
import akka.actor.typed.javadsl.ActorContext
import akka.cluster.ddata.Key
import akka.cluster.ddata.ReplicatedData
+import akka.util.JavaDurationConverters._
import akka.util.Timeout
-import com.github.ghik.silencer.silent
/**
* When interacting with the `Replicator` from an actor this class provides convenient
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/ReplicatorSettings.scala b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/ReplicatorSettings.scala
index 1c6009e6e5..47899bad65 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/ReplicatorSettings.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/javadsl/ReplicatorSettings.scala
@@ -4,10 +4,11 @@
package akka.cluster.ddata.typed.javadsl
-import akka.cluster.{ ddata => dd }
+import com.typesafe.config.Config
+
import akka.actor.typed.ActorSystem
import akka.actor.typed.scaladsl.adapter._
-import com.typesafe.config.Config
+import akka.cluster.{ ddata => dd }
object ReplicatorSettings {
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/scaladsl/DistributedData.scala b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/scaladsl/DistributedData.scala
index 7675b08d51..6571716b35 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/scaladsl/DistributedData.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/scaladsl/DistributedData.scala
@@ -6,18 +6,19 @@ package akka.cluster.ddata.typed.scaladsl
import scala.concurrent.duration.FiniteDuration
-import akka.actor.typed.{ ActorRef, ActorSystem, Extension, ExtensionId, Props }
+import org.slf4j.LoggerFactory
+
import akka.actor.ExtendedActorSystem
+import akka.actor.typed.{ ActorRef, ActorSystem, Extension, ExtensionId, Props }
import akka.actor.typed.Behavior
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.LoggerOps
import akka.annotation.InternalApi
+import akka.cluster.{ ddata => dd }
import akka.cluster.Cluster
import akka.cluster.ddata.ReplicatedData
-import akka.cluster.{ ddata => dd }
import akka.cluster.ddata.SelfUniqueAddress
import akka.util.JavaDurationConverters._
-import org.slf4j.LoggerFactory
object DistributedData extends ExtensionId[DistributedData] {
def get(system: ActorSystem[_]): DistributedData = apply(system)
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/scaladsl/Replicator.scala b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/scaladsl/Replicator.scala
index 81237c57f0..87007a2312 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/scaladsl/Replicator.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/scaladsl/Replicator.scala
@@ -6,11 +6,11 @@ package akka.cluster.ddata.typed.scaladsl
import scala.concurrent.duration.FiniteDuration
+import akka.actor.typed.ActorRef
+import akka.actor.typed.Behavior
import akka.cluster.{ ddata => dd }
import akka.cluster.ddata.Key
import akka.cluster.ddata.ReplicatedData
-import akka.actor.typed.ActorRef
-import akka.actor.typed.Behavior
import akka.cluster.ddata.typed.internal.ReplicatorBehavior
/**
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/scaladsl/ReplicatorSettings.scala b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/scaladsl/ReplicatorSettings.scala
index f282192322..7520c4722c 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/scaladsl/ReplicatorSettings.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/ddata/typed/scaladsl/ReplicatorSettings.scala
@@ -4,11 +4,12 @@
package akka.cluster.ddata.typed.scaladsl
-import akka.cluster.{ ddata => dd }
+import com.typesafe.config.Config
+
import akka.actor.typed.ActorSystem
import akka.actor.typed.scaladsl.adapter._
import akka.annotation.InternalApi
-import com.typesafe.config.Config
+import akka.cluster.{ ddata => dd }
/**
* @see [[akka.cluster.ddata.ReplicatorSettings]].
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/typed/Cluster.scala b/akka-cluster-typed/src/main/scala/akka/cluster/typed/Cluster.scala
index 3adcbe680a..914746d1ec 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/typed/Cluster.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/typed/Cluster.scala
@@ -4,16 +4,16 @@
package akka.cluster.typed
+import scala.collection.immutable
+
import akka.actor.Address
-import akka.annotation.DoNotInherit
-import akka.cluster.ClusterEvent.{ ClusterDomainEvent, CurrentClusterState }
-import akka.cluster._
-import akka.japi.Util
import akka.actor.typed.{ ActorRef, ActorSystem, Extension, ExtensionId }
import akka.actor.typed.ExtensionSetup
+import akka.annotation.DoNotInherit
+import akka.cluster._
+import akka.cluster.ClusterEvent.{ ClusterDomainEvent, CurrentClusterState }
import akka.cluster.typed.internal.AdapterClusterImpl
-
-import scala.collection.immutable
+import akka.japi.Util
/**
* Messages for subscribing to changes in the cluster state
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/typed/ClusterSingleton.scala b/akka-cluster-typed/src/main/scala/akka/cluster/typed/ClusterSingleton.scala
index 9653026c8b..77c4d61045 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/typed/ClusterSingleton.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/typed/ClusterSingleton.scala
@@ -4,6 +4,13 @@
package akka.cluster.typed
+import scala.concurrent.duration._
+import scala.concurrent.duration.{ Duration, FiniteDuration }
+
+import com.typesafe.config.Config
+
+import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Extension, ExtensionId, Props }
+import akka.actor.typed.ExtensionSetup
import akka.annotation.{ DoNotInherit, InternalApi }
import akka.cluster.ClusterSettings.DataCenter
import akka.cluster.singleton.{
@@ -11,13 +18,7 @@ import akka.cluster.singleton.{
ClusterSingletonManagerSettings => ClassicClusterSingletonManagerSettings
}
import akka.cluster.typed.internal.AdaptedClusterSingletonImpl
-import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Extension, ExtensionId, Props }
import akka.util.JavaDurationConverters._
-import com.typesafe.config.Config
-import scala.concurrent.duration._
-import scala.concurrent.duration.{ Duration, FiniteDuration }
-
-import akka.actor.typed.ExtensionSetup
object ClusterSingletonSettings {
def apply(system: ActorSystem[_]): ClusterSingletonSettings =
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/AdaptedClusterImpl.scala b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/AdaptedClusterImpl.scala
index 363a38a50d..705c553a23 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/AdaptedClusterImpl.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/AdaptedClusterImpl.scala
@@ -4,17 +4,17 @@
package akka.cluster.typed.internal
+import akka.actor.typed.{ ActorRef, ActorSystem, Terminated }
import akka.actor.typed.Behavior
import akka.actor.typed.Props
import akka.actor.typed.SupervisorStrategy
-import akka.annotation.InternalApi
-import akka.cluster.ClusterEvent.MemberEvent
-import akka.cluster.{ ClusterEvent, Member, MemberStatus }
-import akka.actor.typed.{ ActorRef, ActorSystem, Terminated }
-import akka.cluster.typed._
import akka.actor.typed.internal.adapter.ActorSystemAdapter
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
+import akka.annotation.InternalApi
+import akka.cluster.{ ClusterEvent, Member, MemberStatus }
+import akka.cluster.ClusterEvent.MemberEvent
+import akka.cluster.typed._
/**
* INTERNAL API:
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/AdaptedClusterSingletonImpl.scala b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/AdaptedClusterSingletonImpl.scala
index df907d3ed1..fcafcaad63 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/AdaptedClusterSingletonImpl.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/AdaptedClusterSingletonImpl.scala
@@ -7,16 +7,16 @@ package akka.cluster.typed.internal
import java.util.concurrent.ConcurrentHashMap
import java.util.function.{ Function => JFunction }
-import akka.actor.typed.internal.{ PoisonPill, PoisonPillInterceptor }
import akka.actor.{ ExtendedActorSystem, InvalidActorNameException }
-import akka.annotation.InternalApi
-import akka.cluster.singleton.{ ClusterSingletonProxy, ClusterSingletonManager => OldSingletonManager }
-import akka.cluster.typed.{ Cluster, ClusterSingleton, ClusterSingletonImpl, ClusterSingletonSettings }
+import akka.actor.typed.{ ActorRef, ActorSystem, Behavior }
+import akka.actor.typed.internal.{ PoisonPill, PoisonPillInterceptor }
import akka.actor.typed.internal.adapter.ActorSystemAdapter
import akka.actor.typed.scaladsl.Behaviors
-import akka.actor.typed.{ ActorRef, ActorSystem, Behavior }
+import akka.annotation.InternalApi
import akka.cluster.ClusterSettings.DataCenter
+import akka.cluster.singleton.{ ClusterSingletonProxy, ClusterSingletonManager => OldSingletonManager }
import akka.cluster.typed
+import akka.cluster.typed.{ Cluster, ClusterSingleton, ClusterSingletonImpl, ClusterSingletonSettings }
/**
* INTERNAL API:
@@ -28,6 +28,7 @@ private[akka] final class AdaptedClusterSingletonImpl(system: ActorSystem[_]) ex
"only adapted actor systems can be used for the typed cluster singleton")
import ClusterSingletonImpl._
+
import akka.actor.typed.scaladsl.adapter._
private lazy val cluster = Cluster(system)
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/AkkaClusterTypedSerializer.scala b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/AkkaClusterTypedSerializer.scala
index 38d19b9661..2c882a6eb5 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/AkkaClusterTypedSerializer.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/AkkaClusterTypedSerializer.scala
@@ -9,12 +9,12 @@ import java.io.NotSerializableException
import akka.actor.ExtendedActorSystem
import akka.actor.typed.ActorRefResolver
import akka.actor.typed.internal.pubsub.TopicImpl
-import akka.annotation.InternalApi
-import akka.serialization.{ BaseSerializer, SerializerWithStringManifest }
import akka.actor.typed.scaladsl.adapter._
+import akka.annotation.InternalApi
import akka.cluster.typed.internal.protobuf.ClusterMessages
import akka.cluster.typed.internal.receptionist.ClusterReceptionist.Entry
import akka.remote.serialization.WrappedPayloadSupport
+import akka.serialization.{ BaseSerializer, SerializerWithStringManifest }
/**
* INTERNAL API
@@ -61,13 +61,17 @@ private[akka] final class AkkaClusterTypedSerializer(override val system: Extend
.toByteArray
}
- private def receptionistEntryToBinary(e: Entry): Array[Byte] =
- ClusterMessages.ReceptionistEntry
+ private def receptionistEntryToBinary(e: Entry): Array[Byte] = {
+ val b = ClusterMessages.ReceptionistEntry
.newBuilder()
.setActorRef(resolver.toSerializationFormat(e.ref))
.setSystemUid(e.systemUid)
- .build()
- .toByteArray
+
+ if (e.createdTimestamp != 0L)
+ b.setCreatedTimestamp(e.createdTimestamp)
+
+ b.build().toByteArray
+ }
private def pubSubMessageFromBinary(bytes: Array[Byte]): TopicImpl.MessagePublished[_] = {
val parsed = ClusterMessages.PubSubMessagePublished.parseFrom(bytes)
@@ -77,6 +81,7 @@ private[akka] final class AkkaClusterTypedSerializer(override val system: Extend
private def receptionistEntryFromBinary(bytes: Array[Byte]): Entry = {
val re = ClusterMessages.ReceptionistEntry.parseFrom(bytes)
- Entry(resolver.resolveActorRef(re.getActorRef), re.getSystemUid)
+ val createdTimestamp = if (re.hasCreatedTimestamp) re.getCreatedTimestamp else 0L
+ Entry(resolver.resolveActorRef(re.getActorRef), re.getSystemUid)(createdTimestamp)
}
}
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/delivery/ReliableDeliverySerializer.scala b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/delivery/ReliableDeliverySerializer.scala
index ce26c2b675..ff0cd75ba3 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/delivery/ReliableDeliverySerializer.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/delivery/ReliableDeliverySerializer.scala
@@ -6,7 +6,6 @@ package akka.cluster.typed.internal.delivery
import java.io.NotSerializableException
-import akka.util.ccompat.JavaConverters._
import akka.actor.typed.ActorRefResolver
import akka.actor.typed.delivery.ConsumerController
import akka.actor.typed.delivery.DurableProducerQueue
@@ -19,6 +18,7 @@ import akka.cluster.typed.internal.protobuf.ReliableDelivery.Confirmed
import akka.remote.serialization.WrappedPayloadSupport
import akka.serialization.BaseSerializer
import akka.serialization.SerializerWithStringManifest
+import akka.util.ccompat.JavaConverters._
/**
* INTERNAL API
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionist.scala b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionist.scala
index 8f9a70f225..11d87471d0 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionist.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionist.scala
@@ -4,30 +4,30 @@
package akka.cluster.typed.internal.receptionist
+import scala.concurrent.duration._
+
+import akka.actor.Address
+import akka.actor.typed.{ ActorRef, Behavior }
import akka.actor.typed.internal.receptionist.{ AbstractServiceKey, ReceptionistBehaviorProvider, ReceptionistMessages }
import akka.actor.typed.receptionist.Receptionist.Command
import akka.actor.typed.receptionist.ServiceKey
-import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.scaladsl.{ ActorContext, Behaviors, LoggerOps }
-import akka.actor.typed.{ ActorRef, Behavior }
+import akka.actor.typed.scaladsl.adapter._
import akka.annotation.InternalApi
-import akka.cluster.ClusterEvent.MemberRemoved
-import akka.cluster.ddata.{ ORMultiMap, ORMultiMapKey, Replicator }
import akka.cluster.{ Cluster, ClusterEvent, UniqueAddress }
-import akka.remote.AddressUidExtension
-import akka.util.TypedMultiMap
-
-import scala.concurrent.duration._
-import akka.actor.Address
import akka.cluster.ClusterEvent.ClusterDomainEvent
import akka.cluster.ClusterEvent.ClusterShuttingDown
import akka.cluster.ClusterEvent.MemberJoined
+import akka.cluster.ClusterEvent.MemberRemoved
import akka.cluster.ClusterEvent.MemberUp
import akka.cluster.ClusterEvent.MemberWeaklyUp
import akka.cluster.ClusterEvent.ReachabilityEvent
import akka.cluster.ClusterEvent.ReachableMember
import akka.cluster.ClusterEvent.UnreachableMember
+import akka.cluster.ddata.{ ORMultiMap, ORMultiMapKey, Replicator }
import akka.cluster.ddata.SelfUniqueAddress
+import akka.remote.AddressUidExtension
+import akka.util.TypedMultiMap
// just to provide a log class
/** INTERNAL API */
@@ -48,13 +48,13 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
// values contain system uid to make it possible to discern actors at the same
// path in different incarnations of a cluster node
- final case class Entry(ref: ActorRef[_], systemUid: Long) {
+ final case class Entry(ref: ActorRef[_], systemUid: Long)(val createdTimestamp: Long) {
def uniqueAddress(selfAddress: Address): UniqueAddress =
if (ref.path.address.hasLocalScope) UniqueAddress(selfAddress, systemUid)
else UniqueAddress(ref.path.address, systemUid)
+
override def toString: String =
s"${ref.path.toString}#${ref.path.uid} @ $systemUid"
-
}
private sealed trait InternalCommand extends Command
@@ -106,7 +106,7 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
tombstones.foldLeft(tombstones) {
case (acc, (actorRef, entries)) =>
val entriesToKeep = entries.filter {
- case (_, deadline) => deadline.hasTimeLeft
+ case (_, deadline) => deadline.hasTimeLeft()
}
if (entriesToKeep.size == entries.size) acc
else if (entriesToKeep.isEmpty) acc - actorRef
@@ -184,7 +184,7 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
val cluster = Cluster(classicSystem)
// don't use DistributedData.selfUniqueAddress here, because that will initialize extension, which
// isn't used otherwise by the ClusterReceptionist
- implicit val selfNodeAddress = SelfUniqueAddress(cluster.selfUniqueAddress)
+ implicit val selfNodeAddress: SelfUniqueAddress = SelfUniqueAddress(cluster.selfUniqueAddress)
val replicator = ctx.actorOf(Replicator.props(settings.replicatorSettings), "replicator")
@@ -262,56 +262,67 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
cluster.state.leader.contains(cluster.selfAddress)
}
- def nodesRemoved(addresses: Set[UniqueAddress]): Unit = {
+ def nodesRemoved(addresses: Set[UniqueAddress], onlyRemoveOldEntries: Boolean): Unit = {
// ok to update from several nodes but more efficient to try to do it from one node
- if (isLeader) {
- def isOnRemovedNode(entry: Entry): Boolean = addresses(entry.uniqueAddress(setup.selfUniqueAddress.address))
+ def isOnRemovedNode(entry: Entry): Boolean = addresses(entry.uniqueAddress(setup.selfUniqueAddress.address))
- val removals = {
- state.registry.allServices.foldLeft(Map.empty[AbstractServiceKey, Set[Entry]]) {
- case (acc, (key, entries)) =>
- val removedEntries = entries.filter(isOnRemovedNode)
- if (removedEntries.isEmpty) acc // no change
- else acc + (key -> removedEntries)
- }
+ val now = System.currentTimeMillis()
+
+ // it possible that an entry is added before MemberJoined is visible and such entries should not be removed
+ def isOld(entry: Entry): Boolean = (now - entry.createdTimestamp) >= settings.pruneRemovedOlderThan.toMillis
+
+ val removals = {
+ state.registry.allServices.foldLeft(Map.empty[AbstractServiceKey, Set[Entry]]) {
+ case (acc, (key, entries)) =>
+ val removedEntries =
+ entries.filter(entry => isOnRemovedNode(entry) && (!onlyRemoveOldEntries || isOld(entry)))
+
+ if (removedEntries.isEmpty) acc // no change
+ else acc + (key -> removedEntries)
}
+ }
- if (removals.nonEmpty) {
- if (ctx.log.isDebugEnabled)
- ctx.log.debugN(
- "ClusterReceptionist [{}] - Node(s) removed [{}], updating registry removing entries: [{}]",
- cluster.selfAddress,
- addresses.mkString(","),
- removals
- .map {
- case (key, entries) => key.asServiceKey.id -> entries.mkString("[", ", ", "]")
- }
- .mkString(","))
-
- // shard changes over the ddata keys they belong to
- val removalsPerDdataKey = state.registry.entriesPerDdataKey(removals)
-
- removalsPerDdataKey.foreach {
- case (ddataKey, removalForKey) =>
- replicator ! Replicator.Update(ddataKey, EmptyORMultiMap, settings.writeConsistency) { registry =>
- ServiceRegistry(registry).removeAll(removalForKey).toORMultiMap
+ if (removals.nonEmpty) {
+ if (ctx.log.isDebugEnabled)
+ ctx.log.debugN(
+ "ClusterReceptionist [{}] - Node(s) removed [{}], updating registry removing entries: [{}]",
+ cluster.selfAddress,
+ addresses.mkString(","),
+ removals
+ .map {
+ case (key, entries) => key.asServiceKey.id -> entries.mkString("[", ", ", "]")
}
- }
+ .mkString(","))
+ // shard changes over the ddata keys they belong to
+ val removalsPerDdataKey = state.registry.entriesPerDdataKey(removals)
+
+ removalsPerDdataKey.foreach {
+ case (ddataKey, removalForKey) =>
+ replicator ! Replicator.Update(ddataKey, EmptyORMultiMap, settings.writeConsistency) { registry =>
+ ServiceRegistry(registry).removeAll(removalForKey).toORMultiMap
+ }
}
}
}
def reachabilityChanged(keysForNode: Set[AbstractServiceKey], newState: State): Unit = {
- keysForNode.foreach { changedKey =>
+ notifySubscribers(keysForNode, servicesWereAddedOrRemoved = false, newState)
+ }
+
+ def notifySubscribers(
+ changedKeys: Set[AbstractServiceKey],
+ servicesWereAddedOrRemoved: Boolean,
+ newState: State): Unit = {
+ changedKeys.foreach { changedKey =>
val serviceKey = changedKey.asServiceKey
- val subscribers = state.subscriptions.get(changedKey)
+ val subscribers = newState.subscriptions.get(changedKey)
if (subscribers.nonEmpty) {
val (reachable, all) = newState.activeActorRefsFor(serviceKey, selfUniqueAddress)
val listing =
- ReceptionistMessages.Listing(serviceKey, reachable, all, servicesWereAddedOrRemoved = false)
+ ReceptionistMessages.Listing(serviceKey, reachable, all, servicesWereAddedOrRemoved)
subscribers.foreach(_ ! listing)
}
}
@@ -320,7 +331,7 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
def onCommand(cmd: Command): Behavior[Command] = cmd match {
case ReceptionistMessages.Register(key, serviceInstance, maybeReplyTo) =>
if (serviceInstance.path.address.hasLocalScope) {
- val entry = Entry(serviceInstance, setup.selfSystemUid)
+ val entry = Entry(serviceInstance, setup.selfSystemUid)(System.currentTimeMillis())
ctx.log
.debugN("ClusterReceptionist [{}] - Actor was registered: [{}] [{}]", cluster.selfAddress, key, entry)
// actor already watched after one service key registration
@@ -343,7 +354,7 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
case ReceptionistMessages.Deregister(key, serviceInstance, maybeReplyTo) =>
if (serviceInstance.path.address.hasLocalScope) {
- val entry = Entry(serviceInstance, setup.selfSystemUid)
+ val entry = Entry(serviceInstance, setup.selfSystemUid)(0L)
ctx.log.debugN(
"ClusterReceptionist [{}] - Unregister actor: [{}] [{}]",
cluster.selfAddress,
@@ -400,7 +411,7 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
behavior(setup, state.removeSubscriber(subscriber))
case LocalServiceActorTerminated(serviceInstance) =>
- val entry = Entry(serviceInstance, setup.selfSystemUid)
+ val entry = Entry(serviceInstance, setup.selfSystemUid)(0L)
// could be empty if there was a race between termination and unregistration
val keys = state.servicesPerActor.getOrElse(serviceInstance, Set.empty)
@@ -439,17 +450,11 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
state.tombstones.mkString(", "))
}
+ notifySubscribers(changedKeys, servicesWereAddedOrRemoved = true, newState)
+
changedKeys.foreach { changedKey =>
val serviceKey = changedKey.asServiceKey
- val subscribers = state.subscriptions.get(changedKey)
- if (subscribers.nonEmpty) {
- val (reachable, all) = newState.activeActorRefsFor(serviceKey, selfUniqueAddress)
- val listing =
- ReceptionistMessages.Listing(serviceKey, reachable, all, servicesWereAddedOrRemoved = true)
- subscribers.foreach(_ ! listing)
- }
-
// because of how ORMultiMap/ORset works, we could have a case where an actor we removed
// is re-introduced because of a concurrent update, in that case we need to re-remove it
val tombstonedButReAdded = newRegistry.actorRefsFor(serviceKey).filter(state.hasTombstone(serviceKey))
@@ -463,7 +468,7 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
replicator ! Replicator.Update(ddataKey, EmptyORMultiMap, settings.writeConsistency) { registry =>
tombstonedButReAdded
.foldLeft(ServiceRegistry(registry)) { (acc, ref) =>
- acc.removeBinding(serviceKey, Entry(ref, setup.selfSystemUid))
+ acc.removeBinding(serviceKey, Entry(ref, setup.selfSystemUid)(0L))
}
.toORMultiMap
}
@@ -476,7 +481,23 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
}
case NodeAdded(uniqueAddress) =>
- behavior(setup, state.copy(registry = state.registry.addNode(uniqueAddress)))
+ if (state.registry.nodes.contains(uniqueAddress)) {
+ Behaviors.same
+ } else {
+ val newState = state.copy(registry = state.registry.addNode(uniqueAddress))
+ val keysForNode = newState.registry.keysFor(uniqueAddress)
+ if (keysForNode.nonEmpty) {
+ ctx.log.debug2(
+ "ClusterReceptionist [{}] - Node with registered services added [{}]",
+ cluster.selfAddress,
+ uniqueAddress)
+ notifySubscribers(keysForNode, servicesWereAddedOrRemoved = true, newState)
+ } else {
+ ctx.log.debug2("ClusterReceptionist [{}] - Node added [{}]", cluster.selfAddress, uniqueAddress)
+ }
+
+ behavior(setup, newState)
+ }
case NodeRemoved(uniqueAddress) =>
if (uniqueAddress == selfUniqueAddress) {
@@ -484,17 +505,30 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
// If self cluster node is shutting down our own entries should have been removed via
// watch-Terminated or will be removed by other nodes. This point is anyway too late.
Behaviors.stopped
- } else {
+ } else if (state.registry.nodes.contains(uniqueAddress)) {
+
+ val keysForNode = state.registry.keysFor(uniqueAddress)
+ val newState = state.copy(registry = state.registry.removeNode(uniqueAddress))
+ if (keysForNode.nonEmpty) {
+ ctx.log.debug2(
+ "ClusterReceptionist [{}] - Node with registered services removed [{}]",
+ cluster.selfAddress,
+ uniqueAddress)
+ notifySubscribers(keysForNode, servicesWereAddedOrRemoved = true, newState)
+ }
+
// Ok to update from several nodes but more efficient to try to do it from one node.
if (isLeader) {
ctx.log.debug2(
"ClusterReceptionist [{}] - Leader node observed removed node [{}]",
cluster.selfAddress,
uniqueAddress)
- nodesRemoved(Set(uniqueAddress))
+ nodesRemoved(Set(uniqueAddress), onlyRemoveOldEntries = false)
}
- behavior(setup, state.copy(registry = state.registry.removeNode(uniqueAddress)))
+ behavior(setup, newState)
+ } else {
+ Behaviors.same
}
case NodeUnreachable(uniqueAddress) =>
@@ -528,14 +562,13 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
state.registry.allUniqueAddressesInState(setup.selfUniqueAddress)
val notInCluster = allAddressesInState.diff(state.registry.nodes)
- if (notInCluster.isEmpty) Behaviors.same
- else {
+ if (notInCluster.nonEmpty) {
if (ctx.log.isDebugEnabled)
ctx.log.debug2(
"ClusterReceptionist [{}] - Leader node cleanup tick, removed nodes: [{}]",
cluster.selfAddress,
notInCluster.mkString(","))
- nodesRemoved(notInCluster)
+ nodesRemoved(notInCluster, onlyRemoveOldEntries = true)
}
}
Behaviors.same
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistConfigCompatChecker.scala b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistConfigCompatChecker.scala
index d8ab1237d4..26711c0db9 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistConfigCompatChecker.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistConfigCompatChecker.scala
@@ -4,9 +4,10 @@
package akka.cluster.typed.internal.receptionist
+import com.typesafe.config.Config
+
import akka.annotation.InternalApi
import akka.cluster.{ ConfigValidation, JoinConfigCompatChecker, Valid }
-import com.typesafe.config.Config
/**
* INTERNAL API
diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistSettings.scala b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistSettings.scala
index 9e7659c1b2..87ebe9980c 100644
--- a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistSettings.scala
+++ b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistSettings.scala
@@ -4,16 +4,17 @@
package akka.cluster.typed.internal.receptionist
+import scala.concurrent.duration._
+import scala.concurrent.duration.{ FiniteDuration, MILLISECONDS }
+
+import com.typesafe.config.Config
+
import akka.actor.typed.ActorSystem
import akka.annotation.InternalApi
import akka.cluster.ddata.Replicator
import akka.cluster.ddata.Replicator.WriteConsistency
-import akka.util.Helpers.toRootLowerCase
-import com.typesafe.config.Config
-import scala.concurrent.duration._
-import scala.concurrent.duration.{ FiniteDuration, MILLISECONDS }
-
import akka.cluster.ddata.ReplicatorSettings
+import akka.util.Helpers.toRootLowerCase
/**
* Internal API
@@ -40,6 +41,7 @@ private[akka] object ClusterReceptionistSettings {
ClusterReceptionistSettings(
writeConsistency,
pruningInterval = config.getDuration("pruning-interval", MILLISECONDS).millis,
+ pruneRemovedOlderThan = config.getDuration("prune-removed-older-than", MILLISECONDS).millis,
config.getInt("distributed-key-count"),
replicatorSettings)
}
@@ -52,5 +54,6 @@ private[akka] object ClusterReceptionistSettings {
private[akka] case class ClusterReceptionistSettings(
writeConsistency: WriteConsistency,
pruningInterval: FiniteDuration,
+ pruneRemovedOlderThan: FiniteDuration,
distributedKeyCount: Int,
replicatorSettings: ReplicatorSettings)
diff --git a/akka-cluster-typed/src/multi-jvm/resources/logback-test.xml b/akka-cluster-typed/src/multi-jvm/resources/logback-test.xml
index e2d3457e51..e36652a969 100644
--- a/akka-cluster-typed/src/multi-jvm/resources/logback-test.xml
+++ b/akka-cluster-typed/src/multi-jvm/resources/logback-test.xml
@@ -5,7 +5,7 @@
- %date{ISO8601} %-5level %logger %marker - %msg {%mdc}%n
+ %date{ISO8601} %-5level %logger %marker - %msg MDC: {%mdc}%n
diff --git a/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/MultiDcClusterSingletonSpec.scala b/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/MultiDcClusterSingletonSpec.scala
index 6afd7d2c4b..4160a56759 100644
--- a/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/MultiDcClusterSingletonSpec.scala
+++ b/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/MultiDcClusterSingletonSpec.scala
@@ -4,14 +4,15 @@
package akka.cluster.typed
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
+import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.typed.scaladsl.adapter._
import akka.cluster.{ MemberStatus, MultiNodeClusterSpec }
import akka.remote.testconductor.RoleName
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
-import akka.actor.testkit.typed.scaladsl.TestProbe
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.duration._
object MultiDcClusterSingletonSpecConfig extends MultiNodeConfig {
val first: RoleName = role("first")
@@ -41,8 +42,8 @@ abstract class MultiDcClusterSingletonSpec
extends MultiNodeSpec(MultiDcClusterSingletonSpecConfig)
with MultiNodeTypedClusterSpec {
- import MultiDcPinger._
import MultiDcClusterSingletonSpecConfig._
+ import MultiDcPinger._
"A cluster with multiple data centers" must {
"be able to form" in {
@@ -65,7 +66,7 @@ abstract class MultiDcClusterSingletonSpec
runOn(first) {
val singleton = ClusterSingleton(typedSystem)
val pinger = singleton.init(SingletonActor(MultiDcPinger(), "ping").withStopMessage(NoMore))
- val probe = TestProbe[Pong]
+ val probe = TestProbe[Pong]()
pinger ! Ping(probe.ref)
probe.expectMessage(Pong("dc1"))
enterBarrier("singleton-up")
@@ -82,7 +83,7 @@ abstract class MultiDcClusterSingletonSpec
SingletonActor(MultiDcPinger(), "ping")
.withStopMessage(NoMore)
.withSettings(ClusterSingletonSettings(typedSystem).withDataCenter("dc1")))
- val probe = TestProbe[Pong]
+ val probe = TestProbe[Pong]()
pinger ! Ping(probe.ref)
probe.expectMessage(Pong("dc1"))
}
@@ -94,7 +95,7 @@ abstract class MultiDcClusterSingletonSpec
runOn(second, third) {
val singleton = ClusterSingleton(typedSystem)
val pinger = singleton.init(SingletonActor(MultiDcPinger(), "ping").withStopMessage(NoMore))
- val probe = TestProbe[Pong]
+ val probe = TestProbe[Pong]()
pinger ! Ping(probe.ref)
probe.expectMessage(Pong("dc2"))
}
diff --git a/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/MultiNodeTypedClusterSpec.scala b/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/MultiNodeTypedClusterSpec.scala
index 9eb37b8e0a..368645d233 100644
--- a/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/MultiNodeTypedClusterSpec.scala
+++ b/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/MultiNodeTypedClusterSpec.scala
@@ -6,26 +6,27 @@ package akka.cluster.typed
import java.util.concurrent.ConcurrentHashMap
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.language.implicitConversions
+
+import org.scalatest.Suite
+import org.scalatest.matchers.should.Matchers
+
+import akka.actor.{ Address, Scheduler }
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.Props
import akka.actor.typed.SpawnProtocol
-import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.scaladsl.AskPattern._
-import akka.actor.{ Address, Scheduler }
+import akka.actor.typed.scaladsl.adapter._
import akka.cluster.{ ClusterEvent, MemberStatus }
import akka.remote.testconductor.RoleName
import akka.remote.testkit.{ MultiNodeSpec, STMultiNodeSpec }
import akka.testkit.WatchedByCoroner
import akka.util.Timeout
-import org.scalatest.Suite
-import org.scalatest.matchers.should.Matchers
-
-import scala.concurrent.Await
-import scala.concurrent.Future
-import scala.concurrent.duration._
-import scala.language.implicitConversions
trait MultiNodeTypedClusterSpec extends Suite with STMultiNodeSpec with WatchedByCoroner with Matchers {
self: MultiNodeSpec =>
diff --git a/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/PubSubSpec.scala b/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/PubSubSpec.scala
index a625a53794..53ae40e5f6 100644
--- a/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/PubSubSpec.scala
+++ b/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/PubSubSpec.scala
@@ -4,6 +4,8 @@
package akka.cluster.typed
+import com.typesafe.config.ConfigFactory
+
import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.typed.ActorRef
import akka.actor.typed.internal.pubsub.TopicImpl
@@ -14,7 +16,6 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.serialization.jackson.CborSerializable
-import com.typesafe.config.ConfigFactory
object PubSubSpecConfig extends MultiNodeConfig {
val first: RoleName = role("first")
diff --git a/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/internal/ClusterReceptionistUnreachabilitySpec.scala b/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/internal/ClusterReceptionistUnreachabilitySpec.scala
index 97561d1478..076a013f8f 100644
--- a/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/internal/ClusterReceptionistUnreachabilitySpec.scala
+++ b/akka-cluster-typed/src/multi-jvm/scala/akka/cluster/typed/internal/ClusterReceptionistUnreachabilitySpec.scala
@@ -4,6 +4,10 @@
package akka.cluster.typed.internal
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.typed.receptionist.Receptionist
import akka.actor.typed.receptionist.ServiceKey
@@ -16,9 +20,6 @@ import akka.cluster.typed.MultiNodeTypedClusterSpec
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.duration._
object ClusterReceptionistUnreachabilitySpecConfig extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster-typed/src/test/resources/logback-test.xml b/akka-cluster-typed/src/test/resources/logback-test.xml
index ad3473919e..22c45c93b6 100644
--- a/akka-cluster-typed/src/test/resources/logback-test.xml
+++ b/akka-cluster-typed/src/test/resources/logback-test.xml
@@ -5,7 +5,7 @@
- %date{ISO8601} %-5level %logger %marker - %msg {%mdc}%n
+ %date{ISO8601} %-5level %logger %marker - %msg MDC: {%mdc}%n
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/ddata/typed/scaladsl/ReplicatorSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/ddata/typed/scaladsl/ReplicatorSpec.scala
index 718aac7d96..167afbd916 100644
--- a/akka-cluster-typed/src/test/scala/akka/cluster/ddata/typed/scaladsl/ReplicatorSpec.scala
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/ddata/typed/scaladsl/ReplicatorSpec.scala
@@ -4,9 +4,10 @@
package akka.cluster.ddata.typed.scaladsl
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
-import org.scalatest.wordspec.AnyWordSpecLike
class ReplicatorSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing {
"Replicator" must {
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/typed/ActorRefIgnoreSerializationSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/typed/ActorRefIgnoreSerializationSpec.scala
index 01bee42daa..71b6dfd5ad 100644
--- a/akka-cluster-typed/src/test/scala/akka/cluster/typed/ActorRefIgnoreSerializationSpec.scala
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/typed/ActorRefIgnoreSerializationSpec.scala
@@ -4,16 +4,17 @@
package akka.cluster.typed
-import akka.actor.typed.scaladsl.Behaviors
-import akka.actor.typed.scaladsl.adapter._
-import akka.actor.typed.{ ActorRef, ActorRefResolver, ActorSystem }
-import akka.actor.{ ExtendedActorSystem, IgnoreActorRef }
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfterAll
import org.scalatest.concurrent.ScalaFutures
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+
import akka.{ actor => classic }
+import akka.actor.{ ExtendedActorSystem, IgnoreActorRef }
+import akka.actor.typed.{ ActorRef, ActorRefResolver, ActorSystem }
+import akka.actor.typed.scaladsl.Behaviors
+import akka.actor.typed.scaladsl.adapter._
class ActorRefIgnoreSerializationSpec extends AnyWordSpec with ScalaFutures with Matchers with BeforeAndAfterAll {
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/typed/ActorSystemSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/typed/ActorSystemSpec.scala
index ecd0f9f216..31bbf999a5 100644
--- a/akka-cluster-typed/src/test/scala/akka/cluster/typed/ActorSystemSpec.scala
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/typed/ActorSystemSpec.scala
@@ -11,27 +11,28 @@ import scala.concurrent.Promise
import scala.concurrent.duration._
import scala.util.control.NonFatal
+import com.typesafe.config.ConfigFactory
+import org.scalatest._
+import org.scalatest.concurrent.Eventually
+import org.scalatest.concurrent.ScalaFutures
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.time.Span
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.Done
import akka.actor.CoordinatedShutdown
import akka.actor.ExtendedActorSystem
import akka.actor.InvalidMessageException
-import akka.actor.testkit.typed.scaladsl.TestInbox
import akka.actor.testkit.typed.scaladsl.LogCapturing
+import akka.actor.testkit.typed.scaladsl.TestInbox
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorRefResolver
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.PostStop
-import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.scaladsl.Behaviors
+import akka.actor.typed.scaladsl.adapter._
import akka.serialization.SerializerWithStringManifest
-import com.typesafe.config.ConfigFactory
-import org.scalatest._
-import org.scalatest.concurrent.Eventually
-import org.scalatest.concurrent.ScalaFutures
-import org.scalatest.time.Span
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
object ActorSystemSpec {
@@ -186,7 +187,7 @@ class ActorSystemSpec
"have a working thread factory" in {
withSystem("thread", Behaviors.empty[String]) { sys =>
- val p = Promise[Int]
+ val p = Promise[Int]()
sys.threadFactory
.newThread(new Runnable {
def run(): Unit = p.success(42)
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterActorLoggingSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterActorLoggingSpec.scala
index 225a1e3074..d8ce526744 100644
--- a/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterActorLoggingSpec.scala
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterActorLoggingSpec.scala
@@ -4,15 +4,16 @@
package akka.cluster.typed
+import com.typesafe.config.ConfigFactory
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.ExtendedActorSystem
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.LoggingTestKit
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.typed.internal.ActorMdc
import akka.actor.typed.scaladsl.Behaviors
-import com.typesafe.config.ConfigFactory
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpecLike
object ClusterActorLoggingSpec {
def config = ConfigFactory.parseString("""
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterApiSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterApiSpec.scala
index d8f979dac2..056365431a 100644
--- a/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterApiSpec.scala
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterApiSpec.scala
@@ -4,17 +4,18 @@
package akka.cluster.typed
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.Address
+import akka.actor.testkit.typed.TestKitSettings
+import akka.actor.testkit.typed.scaladsl.ActorTestKit
+import akka.actor.testkit.typed.scaladsl.LogCapturing
+import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
+import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.typed.scaladsl.adapter._
import akka.cluster.ClusterEvent._
import akka.cluster.MemberStatus
-import akka.actor.testkit.typed.scaladsl.TestProbe
-import akka.actor.testkit.typed.TestKitSettings
-import akka.actor.testkit.typed.scaladsl.ActorTestKit
-import com.typesafe.config.ConfigFactory
-import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
-import akka.actor.testkit.typed.scaladsl.LogCapturing
-import org.scalatest.wordspec.AnyWordSpecLike
object ClusterApiSpec {
val config =
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterDispatcherSelectorSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterDispatcherSelectorSpec.scala
new file mode 100644
index 0000000000..a576731581
--- /dev/null
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterDispatcherSelectorSpec.scala
@@ -0,0 +1,18 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.cluster.typed
+
+import com.typesafe.config.ConfigFactory
+
+import akka.actor.typed.scaladsl.DispatcherSelectorSpec
+
+class ClusterDispatcherSelectorSpec
+ extends DispatcherSelectorSpec(ConfigFactory.parseString("""
+ akka.actor.provider = cluster
+ """).withFallback(DispatcherSelectorSpec.config)) {
+
+ // same tests as in DispatcherSelectorSpec
+
+}
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterSingletonApiSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterSingletonApiSpec.scala
index c8feaa0160..7c282f5d59 100644
--- a/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterSingletonApiSpec.scala
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterSingletonApiSpec.scala
@@ -7,6 +7,9 @@ package akka.cluster.typed
import scala.concurrent.Await
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
@@ -15,8 +18,6 @@ import akka.actor.typed.ActorRef
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
import akka.serialization.jackson.CborSerializable
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object ClusterSingletonApiSpec {
@@ -54,7 +55,7 @@ class ClusterSingletonApiSpec
with LogCapturing {
import ClusterSingletonApiSpec._
- implicit val testSettings = TestKitSettings(system)
+ implicit val testSettings: TestKitSettings = TestKitSettings(system)
val clusterNode1 = Cluster(system)
val classicSystem1 = system.toClassic
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterSingletonPoisonPillSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterSingletonPoisonPillSpec.scala
index 3cf7185080..7e83202fd8 100644
--- a/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterSingletonPoisonPillSpec.scala
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/typed/ClusterSingletonPoisonPillSpec.scala
@@ -4,17 +4,18 @@
package akka.cluster.typed
+import scala.concurrent.duration._
+
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.scaladsl.{ ScalaTestWithActorTestKit, TestProbe }
-import akka.actor.typed.internal.PoisonPill
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.{ ActorRef, Behavior }
+import akka.actor.typed.internal.PoisonPill
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
import akka.cluster.typed.ClusterSingletonPoisonPillSpec.GetSelf
-import scala.concurrent.duration._
-
-import akka.actor.testkit.typed.scaladsl.LogCapturing
-import org.scalatest.wordspec.AnyWordSpecLike
object ClusterSingletonPoisonPillSpec {
@@ -31,14 +32,14 @@ class ClusterSingletonPoisonPillSpec
with AnyWordSpecLike
with LogCapturing {
- implicit val testSettings = TestKitSettings(system)
+ implicit val testSettings: TestKitSettings = TestKitSettings(system)
val clusterNode1 = Cluster(system)
clusterNode1.manager ! Join(clusterNode1.selfMember.address)
val classicSystem1 = system.toClassic
"A typed cluster singleton" must {
"support using PoisonPill to stop" in {
- val probe = TestProbe[ActorRef[Any]]
+ val probe = TestProbe[ActorRef[Any]]()
val singleton =
ClusterSingleton(system).init(SingletonActor(ClusterSingletonPoisonPillSpec.sneakyBehavior, "sneaky"))
singleton ! GetSelf(probe.ref)
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/typed/GroupRouterSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/typed/GroupRouterSpec.scala
index da4cd5d396..3c08882490 100644
--- a/akka-cluster-typed/src/test/scala/akka/cluster/typed/GroupRouterSpec.scala
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/typed/GroupRouterSpec.scala
@@ -4,16 +4,17 @@
package akka.cluster.typed
+import scala.concurrent.Promise
+
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit }
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior }
import akka.actor.typed.receptionist.{ Receptionist, ServiceKey }
import akka.actor.typed.scaladsl.{ Behaviors, GroupRouter, Routers }
import akka.serialization.jackson.CborSerializable
import akka.util.Timeout
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.Promise
-import org.scalatest.wordspec.AnyWordSpecLike
object GroupRouterSpec {
def config = ConfigFactory.parseString(s"""
@@ -68,6 +69,7 @@ class GroupRouterSpec extends ScalaTestWithActorTestKit(GroupRouterSpec.config)
def checkGroupRouterBehavior[T](groupRouter: GroupRouter[Ping.type], settings: GroupRouterSpecSettings)(
resultCheck: (Seq[ActorRef[Ping.type]], Seq[ActorRef[Ping.type]]) => T): T = {
import scala.concurrent.duration._
+
import akka.actor.typed.scaladsl.AskPattern._
implicit val system1 =
createSystem(
@@ -105,7 +107,7 @@ class GroupRouterSpec extends ScalaTestWithActorTestKit(GroupRouterSpec.config)
val node2 = Cluster(system2)
node2.manager ! Join(node1.selfMember.address)
- val statsPromise = Promise[(Seq[ActorRef[Ping.type]], Seq[ActorRef[Ping.type]])]
+ val statsPromise = Promise[(Seq[ActorRef[Ping.type]], Seq[ActorRef[Ping.type]])]()
val cancelable = system.scheduler.scheduleAtFixedRate(200.millis, 200.millis)(() => {
implicit val timeout = Timeout(3.seconds)
val actorRefsInNode1 = system1.ask[Seq[ActorRef[Ping.type]]](ref => GetWorkers(ref)).futureValue
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/typed/RemoteContextAskSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/typed/RemoteContextAskSpec.scala
index 7be06e03f5..499ae24f58 100644
--- a/akka-cluster-typed/src/test/scala/akka/cluster/typed/RemoteContextAskSpec.scala
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/typed/RemoteContextAskSpec.scala
@@ -8,6 +8,9 @@ import scala.concurrent.duration._
import scala.util.Failure
import scala.util.Success
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
@@ -19,8 +22,6 @@ import akka.actor.typed.receptionist.ServiceKey
import akka.actor.typed.scaladsl.Behaviors
import akka.serialization.jackson.CborSerializable
import akka.util.Timeout
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object RemoteContextAskSpec {
def config = ConfigFactory.parseString(s"""
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/typed/RemoteDeployNotAllowedSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/typed/RemoteDeployNotAllowedSpec.scala
index 613c02977f..6fcbc84068 100644
--- a/akka-cluster-typed/src/test/scala/akka/cluster/typed/RemoteDeployNotAllowedSpec.scala
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/typed/RemoteDeployNotAllowedSpec.scala
@@ -4,17 +4,18 @@
package akka.cluster.typed
-import akka.actor.typed.ActorSystem
-import akka.actor.typed.scaladsl.Behaviors
-import akka.actor.testkit.typed.scaladsl.TestProbe
-import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._
-import akka.actor.testkit.typed.scaladsl.ActorTestKit
-import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
-import akka.actor.testkit.typed.scaladsl.LogCapturing
+import com.typesafe.config.ConfigFactory
import org.scalatest.wordspec.AnyWordSpecLike
+import akka.actor.testkit.typed.scaladsl.ActorTestKit
+import akka.actor.testkit.typed.scaladsl.LogCapturing
+import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
+import akka.actor.testkit.typed.scaladsl.TestProbe
+import akka.actor.typed.ActorSystem
+import akka.actor.typed.scaladsl.Behaviors
+
object RemoteDeployNotAllowedSpec {
def config = ConfigFactory.parseString(s"""
akka {
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/typed/RemoteMessageSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/typed/RemoteMessageSpec.scala
index 78f9c19c92..009e4d5809 100644
--- a/akka-cluster-typed/src/test/scala/akka/cluster/typed/RemoteMessageSpec.scala
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/typed/RemoteMessageSpec.scala
@@ -6,15 +6,16 @@ package akka.cluster.typed
import scala.concurrent.Promise
+import com.typesafe.config.ConfigFactory
+
import akka.Done
+import akka.actor.{ ActorSystem => ClassicActorSystem }
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorRefResolver
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
-import akka.actor.{ ActorSystem => ClassicActorSystem }
import akka.serialization.jackson.CborSerializable
import akka.testkit.AkkaSpec
-import com.typesafe.config.ConfigFactory
object RemoteMessageSpec {
def config = ConfigFactory.parseString(s"""
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/AkkaClusterTypedSerializerSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/AkkaClusterTypedSerializerSpec.scala
index 470c783c0c..68da21a4b6 100644
--- a/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/AkkaClusterTypedSerializerSpec.scala
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/AkkaClusterTypedSerializerSpec.scala
@@ -4,14 +4,15 @@
package akka.cluster.typed.internal
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.ExtendedActorSystem
+import akka.actor.testkit.typed.scaladsl.LogCapturing
+import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
+import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
import akka.cluster.typed.internal.receptionist.ClusterReceptionist
import akka.serialization.SerializationExtension
-import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
-import akka.actor.testkit.typed.scaladsl.LogCapturing
-import akka.actor.typed.scaladsl.Behaviors
-import org.scalatest.wordspec.AnyWordSpecLike
class AkkaClusterTypedSerializerSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing {
@@ -21,7 +22,7 @@ class AkkaClusterTypedSerializerSpec extends ScalaTestWithActorTestKit with AnyW
"AkkaClusterTypedSerializer" must {
- Seq("ReceptionistEntry" -> ClusterReceptionist.Entry(ref, 666L)).foreach {
+ Seq("ReceptionistEntry" -> ClusterReceptionist.Entry(ref, 666L)(System.currentTimeMillis())).foreach {
case (scenario, item) =>
s"resolve serializer for $scenario" in {
val serializer = SerializationExtension(classicSystem)
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/delivery/ReliableDeliverySerializerSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/delivery/ReliableDeliverySerializerSpec.scala
index 4c84b34f7a..19ddb50f68 100644
--- a/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/delivery/ReliableDeliverySerializerSpec.scala
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/delivery/ReliableDeliverySerializerSpec.scala
@@ -4,6 +4,8 @@
package akka.cluster.typed.internal.delivery
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.ExtendedActorSystem
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
@@ -14,7 +16,6 @@ import akka.actor.typed.delivery.internal.ProducerControllerImpl
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
import akka.serialization.SerializationExtension
-import org.scalatest.wordspec.AnyWordSpecLike
class ReliableDeliverySerializerSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing {
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistSpec.scala
index 365568228c..26b9974472 100644
--- a/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistSpec.scala
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistSpec.scala
@@ -4,9 +4,15 @@
package akka.cluster.typed.internal.receptionist
+import java.util.concurrent.ThreadLocalRandom
+
import scala.concurrent.Await
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.actor.RootActorPath
import akka.actor.testkit.typed.FishingOutcome
import akka.actor.testkit.typed.scaladsl.ActorTestKit
@@ -24,9 +30,6 @@ import akka.cluster.typed.Join
import akka.cluster.typed.JoinSeedNodes
import akka.cluster.typed.Leave
import akka.serialization.jackson.CborSerializable
-import com.typesafe.config.ConfigFactory
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
object ClusterReceptionistSpec {
val config = ConfigFactory.parseString(s"""
@@ -111,6 +114,33 @@ class ClusterReceptionistSpec extends AnyWordSpec with Matchers with LogCapturin
}
}
+ "handle registrations before joining" in {
+ val testKit1 = ActorTestKit("ClusterReceptionistSpec-test-2", ClusterReceptionistSpec.config)
+ val system1 = testKit1.system
+ val testKit2 = ActorTestKit(system1.name, system1.settings.config)
+ val system2 = testKit2.system
+ try {
+ val regProbe1 = TestProbe[Any]()(system1)
+ val regProbe2 = TestProbe[Any]()(system2)
+ val service = testKit1.spawn(pingPongBehavior)
+ testKit1.system.receptionist ! Register(PingKey, service, regProbe1.ref)
+ regProbe1.expectMessage(Registered(PingKey, service))
+ system2.receptionist ! Subscribe(PingKey, regProbe2.ref)
+ regProbe2.expectMessage(Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
+
+ val clusterNode1 = Cluster(system1)
+ clusterNode1.manager ! Join(clusterNode1.selfMember.address)
+ val clusterNode2 = Cluster(system2)
+ clusterNode2.manager ! Join(clusterNode1.selfMember.address)
+
+ val PingKey.Listing(remoteServiceRefs) = regProbe2.expectMessageType[Listing](10.seconds)
+ remoteServiceRefs.head.path.address should ===(Cluster(system1).selfMember.address)
+ } finally {
+ testKit1.shutdownTestKit()
+ testKit2.shutdownTestKit()
+ }
+ }
+
"remove registrations when node dies" in {
testNodeRemoval(down = true)
}
@@ -735,5 +765,68 @@ class ClusterReceptionistSpec extends AnyWordSpec with Matchers with LogCapturin
}
// Fixme concurrent registration and unregistration
+
+ "notify subscribers when registering and joining simultaneously" in {
+ // failing test reproducer for issue #28792
+ // It's possible that the registry entry from the ddata update arrives before MemberJoined.
+ val config = ConfigFactory.parseString("""
+ # quick dissemination to increase the chance of the race condition
+ akka.cluster.typed.receptionist.distributed-data.write-consistency = all
+ akka.cluster.typed.receptionist.distributed-data.gossip-interval = 500ms
+ # run the RemoveTick cleanup often to exercise that scenario
+ akka.cluster.typed.receptionist.pruning-interval = 50ms
+ """).withFallback(ClusterReceptionistSpec.config)
+ val numberOfNodes = 6 // use 9 or more to stress it more
+ val testKits = Vector.fill(numberOfNodes)(ActorTestKit("ClusterReceptionistSpec", config))
+ try {
+ val probes = testKits.map(t => TestProbe[Any]()(t.system))
+ testKits.zip(probes).foreach { case (t, p) => t.system.receptionist ! Subscribe(PingKey, p.ref) }
+
+ val clusterNode1 = Cluster(testKits.head.system)
+ // join 3 first
+ (0 until 3).foreach { i =>
+ val t = testKits(i)
+ Cluster(t.system).manager ! Join(clusterNode1.selfMember.address)
+ val ref = t.spawn(pingPongBehavior)
+ t.system.receptionist ! Register(PingKey, ref)
+ }
+ // wait until all those are Up
+ (0 until 3).foreach { i =>
+ probes(i).awaitAssert(
+ Cluster(testKits(i).system).state.members.count(_.status == MemberStatus.Up) should ===(3),
+ 10.seconds)
+ }
+
+ // then join the rest randomly to the first 3
+ // important to not join all to first to be able to reproduce the problem
+ testKits.drop(3).foreach { t =>
+ val i = ThreadLocalRandom.current().nextInt(3)
+ Cluster(t.system).manager ! Join(Cluster(testKits(i).system).selfMember.address)
+ val ref = t.spawn(pingPongBehavior)
+ Thread.sleep(100) // increase chance of the race condition
+ t.system.receptionist ! Register(PingKey, ref)
+ }
+
+ (0 until numberOfNodes).foreach { i =>
+ probes(i).awaitAssert(
+ Cluster(testKits(i).system).state.members.count(_.status == MemberStatus.Up) should ===(numberOfNodes),
+ 30.seconds)
+ }
+
+ // eventually, all should be included in the Listing
+ (0 until numberOfNodes).foreach { i =>
+ probes(i).fishForMessage(10.seconds, s"$i") {
+ case PingKey.Listing(actors) if actors.size == numberOfNodes => FishingOutcomes.complete
+ case PingKey.Listing(_) => FishingOutcomes.continue
+ }
+ }
+ testKits.head.system.log.debug("All expected listings found.")
+
+ } finally {
+ // faster to terminate all at the same time
+ testKits.foreach(_.system.terminate())
+ testKits.foreach(_.shutdownTestKit())
+ }
+ }
}
}
diff --git a/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistStateSpec.scala b/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistStateSpec.scala
index 7e524d7734..4d52e3ca27 100644
--- a/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistStateSpec.scala
+++ b/akka-cluster-typed/src/test/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionistStateSpec.scala
@@ -4,6 +4,12 @@
package akka.cluster.typed.internal.receptionist
+import scala.concurrent.duration._
+import scala.concurrent.duration.Deadline
+
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.Address
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.typed.internal.receptionist.AbstractServiceKey
@@ -11,11 +17,6 @@ import akka.actor.typed.receptionist.ServiceKey
import akka.cluster.UniqueAddress
import akka.cluster.typed.internal.receptionist.ClusterReceptionist.SubscriptionsKV
import akka.util.TypedMultiMap
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpecLike
-
-import scala.concurrent.duration.Deadline
-import scala.concurrent.duration._
class ClusterReceptionistStateSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with Matchers {
diff --git a/akka-cluster-typed/src/test/scala/docs/akka/cluster/ddata/typed/scaladsl/ReplicatorDocSpec.scala b/akka-cluster-typed/src/test/scala/docs/akka/cluster/ddata/typed/scaladsl/ReplicatorDocSpec.scala
index 4c7e845132..72424fcc8a 100644
--- a/akka-cluster-typed/src/test/scala/docs/akka/cluster/ddata/typed/scaladsl/ReplicatorDocSpec.scala
+++ b/akka-cluster-typed/src/test/scala/docs/akka/cluster/ddata/typed/scaladsl/ReplicatorDocSpec.scala
@@ -34,7 +34,7 @@ object ReplicatorDocSpec {
// #sample
object Counter {
sealed trait Command
- final case object Increment extends Command
+ case object Increment extends Command
final case class GetValue(replyTo: ActorRef[Int]) extends Command
final case class GetCachedValue(replyTo: ActorRef[Int]) extends Command
case object Unsubscribe extends Command
diff --git a/akka-cluster-typed/src/test/scala/docs/akka/cluster/typed/BasicClusterExampleSpec.scala b/akka-cluster-typed/src/test/scala/docs/akka/cluster/typed/BasicClusterExampleSpec.scala
index 09db42afec..594451cbbf 100644
--- a/akka-cluster-typed/src/test/scala/docs/akka/cluster/typed/BasicClusterExampleSpec.scala
+++ b/akka-cluster-typed/src/test/scala/docs/akka/cluster/typed/BasicClusterExampleSpec.scala
@@ -41,6 +41,8 @@ akka {
seed-nodes = [
"akka://ClusterSystem@127.0.0.1:2551",
"akka://ClusterSystem@127.0.0.1:2552"]
+
+ downing-provider-class = "akka.cluster.sbr.SplitBrainResolverProvider"
}
}
#config-seeds
diff --git a/akka-cluster-typed/src/test/scala/docs/akka/cluster/typed/ReceptionistExample.scala b/akka-cluster-typed/src/test/scala/docs/akka/cluster/typed/ReceptionistExample.scala
index 8b1e3343e9..9bc9296aef 100644
--- a/akka-cluster-typed/src/test/scala/docs/akka/cluster/typed/ReceptionistExample.scala
+++ b/akka-cluster-typed/src/test/scala/docs/akka/cluster/typed/ReceptionistExample.scala
@@ -18,7 +18,7 @@ object PingPongExample {
val PingServiceKey = ServiceKey[Ping]("pingService")
final case class Ping(replyTo: ActorRef[Pong.type])
- final case object Pong
+ case object Pong
def apply(): Behavior[Ping] = {
Behaviors.setup { context =>
diff --git a/akka-cluster/src/main/java/akka/cluster/protobuf/msg/ClusterMessages.java b/akka-cluster/src/main/java/akka/cluster/protobuf/msg/ClusterMessages.java
index 6af29aef04..f3a07fa363 100644
--- a/akka-cluster/src/main/java/akka/cluster/protobuf/msg/ClusterMessages.java
+++ b/akka-cluster/src/main/java/akka/cluster/protobuf/msg/ClusterMessages.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2019-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
diff --git a/akka-cluster/src/main/resources/reference.conf b/akka-cluster/src/main/resources/reference.conf
index 4317057bbc..53b29c2210 100644
--- a/akka-cluster/src/main/resources/reference.conf
+++ b/akka-cluster/src/main/resources/reference.conf
@@ -42,6 +42,10 @@ akka {
# This is useful if you implement downing strategies that handle network partitions,
# e.g. by keeping the larger side of the partition and shutting down the smaller side.
# Disable with "off" or specify a duration to enable.
+ #
+ # When using the `akka.cluster.sbr.SplitBrainResolver` as downing provider it will use
+ # the akka.cluster.split-brain-resolver.stable-after as the default down-removal-margin
+ # if this down-removal-margin is undefined.
down-removal-margin = off
# Pluggable support for downing of nodes in the cluster.
@@ -364,3 +368,113 @@ akka {
}
}
+
+#//#split-brain-resolver
+
+# To enable the split brain resolver you first need to enable the provider in your application.conf:
+# akka.cluster.downing-provider-class = "akka.cluster.sbr.SplitBrainResolverProvider"
+
+akka.cluster.split-brain-resolver {
+ # Select one of the available strategies (see descriptions below):
+ # static-quorum, keep-majority, keep-oldest, down-all, lease-majority
+ active-strategy = keep-majority
+
+ #//#stable-after
+ # Time margin after which shards or singletons that belonged to a downed/removed
+ # partition are created in surviving partition. The purpose of this margin is that
+ # in case of a network partition the persistent actors in the non-surviving partitions
+ # must be stopped before corresponding persistent actors are started somewhere else.
+ # This is useful if you implement downing strategies that handle network partitions,
+ # e.g. by keeping the larger side of the partition and shutting down the smaller side.
+ # Decision is taken by the strategy when there has been no membership or
+ # reachability changes for this duration, i.e. the cluster state is stable.
+ stable-after = 20s
+ #//#stable-after
+
+ # When reachability observations by the failure detector are changed the SBR decisions
+ # are deferred until there are no changes within the 'stable-after' duration.
+ # If this continues for too long it might be an indication of an unstable system/network
+ # and it could result in delayed or conflicting decisions on separate sides of a network
+ # partition.
+ # As a precaution for that scenario all nodes are downed if no decision is made within
+ # `stable-after + down-all-when-unstable` from the first unreachability event.
+ # The measurement is reset if all unreachable have been healed, downed or removed, or
+ # if there are no changes within `stable-after * 2`.
+ # The value can be on, off, or a duration.
+ # By default it is 'on' and then it is derived to be 3/4 of stable-after.
+ down-all-when-unstable = on
+
+}
+#//#split-brain-resolver
+
+# Down the unreachable nodes if the number of remaining nodes are greater than or equal to
+# the given 'quorum-size'. Otherwise down the reachable nodes, i.e. it will shut down that
+# side of the partition. In other words, the 'size' defines the minimum number of nodes
+# that the cluster must have to be operational. If there are unreachable nodes when starting
+# up the cluster, before reaching this limit, the cluster may shutdown itself immediately.
+# This is not an issue if you start all nodes at approximately the same time.
+#
+# Note that you must not add more members to the cluster than 'quorum-size * 2 - 1', because
+# then both sides may down each other and thereby form two separate clusters. For example,
+# quorum-size configured to 3 in a 6 node cluster may result in a split where each side
+# consists of 3 nodes each, i.e. each side thinks it has enough nodes to continue by
+# itself. A warning is logged if this recommendation is violated.
+#//#static-quorum
+akka.cluster.split-brain-resolver.static-quorum {
+ # minimum number of nodes that the cluster must have
+ quorum-size = undefined
+
+ # if the 'role' is defined the decision is based only on members with that 'role'
+ role = ""
+}
+#//#static-quorum
+
+# Down the unreachable nodes if the current node is in the majority part based the last known
+# membership information. Otherwise down the reachable nodes, i.e. the own part. If the
+# the parts are of equal size the part containing the node with the lowest address is kept.
+# Note that if there are more than two partitions and none is in majority each part
+# will shutdown itself, terminating the whole cluster.
+#//#keep-majority
+akka.cluster.split-brain-resolver.keep-majority {
+ # if the 'role' is defined the decision is based only on members with that 'role'
+ role = ""
+}
+#//#keep-majority
+
+# Down the part that does not contain the oldest member (current singleton).
+#
+# There is one exception to this rule if 'down-if-alone' is defined to 'on'.
+# Then, if the oldest node has partitioned from all other nodes the oldest
+# will down itself and keep all other nodes running. The strategy will not
+# down the single oldest node when it is the only remaining node in the cluster.
+#
+# Note that if the oldest node crashes the others will remove it from the cluster
+# when 'down-if-alone' is 'on', otherwise they will down themselves if the
+# oldest node crashes, i.e. shutdown the whole cluster together with the oldest node.
+#//#keep-oldest
+akka.cluster.split-brain-resolver.keep-oldest {
+ # Enable downing of the oldest node when it is partitioned from all other nodes
+ down-if-alone = on
+
+ # if the 'role' is defined the decision is based only on members with that 'role',
+ # i.e. using the oldest member (singleton) within the nodes with that role
+ role = ""
+}
+#//#keep-oldest
+
+# Keep the part that can acquire the lease, and down the other part.
+# Best effort is to keep the side that has most nodes, i.e. the majority side.
+# This is achieved by adding a delay before trying to acquire the lease on the
+# minority side.
+#//#lease-majority
+akka.cluster.split-brain-resolver.lease-majority {
+ lease-implementation = ""
+
+ # This delay is used on the minority side before trying to acquire the lease,
+ # as an best effort to try to keep the majority side.
+ acquire-lease-delay-for-minority = 2s
+
+ # If the 'role' is defined the majority/minority is based only on members with that 'role'.
+ role = ""
+}
+#//#lease-majority
diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala
index 1270c4e047..a03c8bd7ef 100644
--- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala
@@ -8,26 +8,27 @@ import java.io.Closeable
import java.util.concurrent.ThreadFactory
import java.util.concurrent.atomic.AtomicBoolean
+import scala.annotation.varargs
+import scala.collection.immutable
+import scala.concurrent.{ Await, ExecutionContext }
+import scala.concurrent.duration._
+import scala.util.control.NonFatal
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.{ Config, ConfigFactory }
+
import akka.ConfigurationException
import akka.actor._
import akka.annotation.InternalApi
import akka.cluster.ClusterSettings.DataCenter
import akka.dispatch.MonitorableThreadFactory
import akka.event.{ Logging, LoggingAdapter }
-import akka.japi.Util
-import akka.pattern._
-import akka.remote.{ UniqueAddress => _, _ }
-import com.typesafe.config.{ Config, ConfigFactory }
-import scala.annotation.varargs
-import scala.collection.immutable
-import scala.concurrent.duration._
-import scala.concurrent.{ Await, ExecutionContext }
-import scala.util.control.NonFatal
-
import akka.event.LogMarker
import akka.event.Logging.LogLevel
import akka.event.MarkerLoggingAdapter
-import com.github.ghik.silencer.silent
+import akka.japi.Util
+import akka.pattern._
+import akka.remote.{ UniqueAddress => _, _ }
/**
* Cluster Extension Id and factory for creating Cluster extension.
diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRefProvider.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRefProvider.scala
index 415f5655d0..3a14c9799a 100644
--- a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRefProvider.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRefProvider.scala
@@ -4,12 +4,16 @@
package akka.cluster
+import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+
import akka.ConfigurationException
import akka.actor.ActorPath
-import akka.actor.Address
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.ActorSystemImpl
+import akka.actor.Address
import akka.actor.Deploy
import akka.actor.DynamicAccess
import akka.actor.NoScopeGiven
@@ -20,14 +24,11 @@ import akka.cluster.routing.ClusterRouterGroupSettings
import akka.cluster.routing.ClusterRouterPool
import akka.cluster.routing.ClusterRouterPoolSettings
import akka.event.EventStream
-import akka.remote.routing.RemoteRouterConfig
import akka.remote.RemoteActorRefProvider
import akka.remote.RemoteDeployer
+import akka.remote.routing.RemoteRouterConfig
import akka.routing.Group
import akka.routing.Pool
-import com.github.ghik.silencer.silent
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
/**
* INTERNAL API
diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala
index d51a1235a4..7dd05fe381 100644
--- a/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala
@@ -5,26 +5,27 @@
package akka.cluster
import scala.collection.immutable
-import scala.concurrent.duration._
import scala.concurrent.Future
import scala.concurrent.Promise
+import scala.concurrent.duration._
import scala.util.control.NonFatal
-import akka.actor._
-import akka.annotation.InternalApi
-import akka.actor.SupervisorStrategy.Stop
-import akka.cluster.MemberStatus._
-import akka.cluster.ClusterEvent._
-import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
+import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+
import akka.Done
+import akka.actor._
+import akka.actor.SupervisorStrategy.Stop
+import akka.annotation.InternalApi
+import akka.cluster.ClusterEvent._
+import akka.cluster.MemberStatus._
+import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
+import akka.event.ActorWithLogClass
+import akka.event.Logging
import akka.pattern.ask
import akka.remote.{ QuarantinedEvent => ClassicQuarantinedEvent }
import akka.remote.artery.QuarantinedEvent
import akka.util.Timeout
-import akka.event.ActorWithLogClass
-import akka.event.Logging
-import com.github.ghik.silencer.silent
-import com.typesafe.config.Config
/**
* Base trait for all cluster messages. All ClusterMessage's are serializable.
@@ -177,7 +178,7 @@ private[cluster] object InternalClusterAction {
final case class PublishChanges(state: MembershipState) extends PublishMessage
final case class PublishEvent(event: ClusterDomainEvent) extends PublishMessage
- final case object ExitingCompleted
+ case object ExitingCompleted
}
@@ -268,7 +269,7 @@ private[cluster] final class ClusterCoreSupervisor(joinConfigCompatChecker: Join
def createChildren(): Unit = {
val publisher =
- context.actorOf(Props[ClusterDomainEventPublisher].withDispatcher(context.props.dispatcher), name = "publisher")
+ context.actorOf(Props[ClusterDomainEventPublisher]().withDispatcher(context.props.dispatcher), name = "publisher")
coreDaemon = Some(
context.watch(context.actorOf(
Props(classOf[ClusterCoreDaemon], publisher, joinConfigCompatChecker).withDispatcher(context.props.dispatcher),
@@ -311,13 +312,13 @@ private[cluster] object ClusterCoreDaemon {
private[cluster] class ClusterCoreDaemon(publisher: ActorRef, joinConfigCompatChecker: JoinConfigCompatChecker)
extends Actor
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
- import InternalClusterAction._
import ClusterCoreDaemon._
+ import InternalClusterAction._
import MembershipState._
val cluster = Cluster(context.system)
- import cluster.ClusterLogger._
import cluster.{ crossDcFailureDetector, failureDetector, scheduler, selfAddress, selfRoles }
+ import cluster.ClusterLogger._
import cluster.settings._
val selfDc = cluster.selfDataCenter
@@ -476,7 +477,7 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef, joinConfigCompatCh
case Welcome(from, gossip) =>
welcome(from.address, from, gossip)
case _: Tick =>
- if (joinSeedNodesDeadline.exists(_.isOverdue))
+ if (joinSeedNodesDeadline.exists(_.isOverdue()))
joinSeedNodesWasUnsuccessful()
}: Actor.Receive).orElse(receiveExitingCompleted)
@@ -496,9 +497,9 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef, joinConfigCompatCh
joinSeedNodes(newSeedNodes)
case msg: SubscriptionMessage => publisher.forward(msg)
case _: Tick =>
- if (joinSeedNodesDeadline.exists(_.isOverdue))
+ if (joinSeedNodesDeadline.exists(_.isOverdue()))
joinSeedNodesWasUnsuccessful()
- else if (deadline.exists(_.isOverdue)) {
+ else if (deadline.exists(_.isOverdue())) {
// join attempt failed, retry
becomeUninitialized()
if (seedNodes.nonEmpty) joinSeedNodes(seedNodes)
@@ -1082,10 +1083,10 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef, joinConfigCompatCh
if (statsEnabled) {
gossipStats = gossipType match {
- case Merge => gossipStats.incrementMergeCount
- case Same => gossipStats.incrementSameCount
- case Newer => gossipStats.incrementNewerCount
- case Older => gossipStats.incrementOlderCount
+ case Merge => gossipStats.incrementMergeCount()
+ case Same => gossipStats.incrementSameCount()
+ case Newer => gossipStats.incrementNewerCount()
+ case Older => gossipStats.incrementOlderCount()
case Ignored => gossipStats // included in receivedGossipCount
}
}
diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala
index 1381970088..a4786c7cbb 100644
--- a/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala
@@ -4,22 +4,22 @@
package akka.cluster
-import language.postfixOps
import scala.collection.immutable
import scala.collection.immutable.{ SortedSet, VectorBuilder }
-import akka.actor.{ Actor, ActorRef, Address }
-import akka.cluster.ClusterSettings.DataCenter
-import akka.cluster.ClusterEvent._
-import akka.cluster.MemberStatus._
-import akka.event.EventStream
-import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
-import akka.actor.DeadLetterSuppression
-import akka.annotation.{ DoNotInherit, InternalApi }
-import akka.util.ccompat._
-
import scala.runtime.AbstractFunction5
import com.github.ghik.silencer.silent
+import language.postfixOps
+
+import akka.actor.{ Actor, ActorRef, Address }
+import akka.actor.DeadLetterSuppression
+import akka.annotation.{ DoNotInherit, InternalApi }
+import akka.cluster.ClusterEvent._
+import akka.cluster.ClusterSettings.DataCenter
+import akka.cluster.MemberStatus._
+import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
+import akka.event.EventStream
+import akka.util.ccompat._
/**
* Domain events published to the event bus.
@@ -337,7 +337,7 @@ object ClusterEvent {
* This event is published when the cluster node is shutting down,
* before the final [[MemberRemoved]] events are published.
*/
- final case object ClusterShuttingDown extends ClusterDomainEvent
+ case object ClusterShuttingDown extends ClusterDomainEvent
/**
* Java API: get the singleton instance of `ClusterShuttingDown` event
diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterJmx.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterJmx.scala
index 8e7d16457e..1d3ad80003 100644
--- a/akka-cluster/src/main/scala/akka/cluster/ClusterJmx.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/ClusterJmx.scala
@@ -5,12 +5,13 @@
package akka.cluster
import java.lang.management.ManagementFactory
-import javax.management.StandardMBean
-import akka.event.LoggingAdapter
-import akka.actor.AddressFromURIString
-import javax.management.ObjectName
import javax.management.InstanceAlreadyExistsException
import javax.management.InstanceNotFoundException
+import javax.management.ObjectName
+import javax.management.StandardMBean
+
+import akka.actor.AddressFromURIString
+import akka.event.LoggingAdapter
/**
* Interface for the cluster JMX MBean.
diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala
index 6470ed3572..2958e8e172 100644
--- a/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala
@@ -10,16 +10,16 @@ import akka.actor._
import akka.cluster.ClusterEvent.CurrentClusterState
import akka.cluster.ClusterEvent.MemberEvent
import akka.cluster.ClusterEvent.MemberJoined
-import akka.cluster.ClusterEvent.MemberUp
import akka.cluster.ClusterEvent.MemberRemoved
+import akka.cluster.ClusterEvent.MemberUp
import akka.cluster.ClusterEvent.MemberWeaklyUp
import akka.dispatch.Dispatchers
import akka.event.ActorWithLogClass
import akka.event.Logging
import akka.remote.FailureDetectorRegistry
+import akka.remote.RARP
import akka.remote.RemoteSettings
import akka.remote.RemoteWatcher
-import akka.remote.RARP
/**
* INTERNAL API
diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala
index 17533acdbd..94bfdc5779 100644
--- a/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala
@@ -5,17 +5,17 @@
package akka.cluster
import scala.collection.immutable
+import scala.concurrent.duration.Duration
+import scala.concurrent.duration.FiniteDuration
+
import com.typesafe.config.Config
import com.typesafe.config.ConfigObject
-import scala.concurrent.duration.Duration
import akka.actor.Address
import akka.actor.AddressFromURIString
import akka.annotation.InternalApi
-import akka.util.Helpers.{ toRootLowerCase, ConfigOps, Requiring }
-
-import scala.concurrent.duration.FiniteDuration
import akka.japi.Util.immutableSeq
+import akka.util.Helpers.{ toRootLowerCase, ConfigOps, Requiring }
object ClusterSettings {
type DataCenter = String
diff --git a/akka-cluster/src/main/scala/akka/cluster/CoordinatedShutdownLeave.scala b/akka-cluster/src/main/scala/akka/cluster/CoordinatedShutdownLeave.scala
index 84672a8ff0..8594545e1d 100644
--- a/akka-cluster/src/main/scala/akka/cluster/CoordinatedShutdownLeave.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/CoordinatedShutdownLeave.scala
@@ -15,7 +15,7 @@ import akka.cluster.MemberStatus._
* INTERNAL API
*/
private[akka] object CoordinatedShutdownLeave {
- def props(): Props = Props[CoordinatedShutdownLeave]
+ def props(): Props = Props[CoordinatedShutdownLeave]()
case object LeaveReq
}
diff --git a/akka-cluster/src/main/scala/akka/cluster/CrossDcClusterHeartbeat.scala b/akka-cluster/src/main/scala/akka/cluster/CrossDcClusterHeartbeat.scala
index 07b9d3e0d0..cbdf490143 100644
--- a/akka-cluster/src/main/scala/akka/cluster/CrossDcClusterHeartbeat.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/CrossDcClusterHeartbeat.scala
@@ -4,18 +4,18 @@
package akka.cluster
+import scala.collection.SortedSet
+import scala.collection.immutable
+
import akka.actor.{ Actor, ActorSelection, Address, NoSerializationVerificationNeeded }
import akka.annotation.InternalApi
import akka.cluster.ClusterEvent._
import akka.cluster.ClusterSettings.DataCenter
+import akka.event.ActorWithLogClass
+import akka.event.Logging
import akka.remote.FailureDetectorRegistry
import akka.util.ConstantFun
import akka.util.ccompat._
-import scala.collection.SortedSet
-import scala.collection.immutable
-
-import akka.event.ActorWithLogClass
-import akka.event.Logging
/**
* INTERNAL API
@@ -42,8 +42,8 @@ private[cluster] class CrossDcHeartbeatSender extends Actor {
val cluster = Cluster(context.system)
val verboseHeartbeat = cluster.settings.Debug.VerboseHeartbeatLogging
- import cluster.settings._
import cluster.{ scheduler, selfAddress, selfDataCenter, selfUniqueAddress }
+ import cluster.settings._
import context.dispatcher
private val clusterLogger =
diff --git a/akka-cluster/src/main/scala/akka/cluster/DowningProvider.scala b/akka-cluster/src/main/scala/akka/cluster/DowningProvider.scala
index b3fe1dc505..f19ad8e3e5 100644
--- a/akka-cluster/src/main/scala/akka/cluster/DowningProvider.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/DowningProvider.scala
@@ -4,11 +4,11 @@
package akka.cluster
+import scala.concurrent.duration.FiniteDuration
+
import akka.ConfigurationException
import akka.actor.{ ActorSystem, ExtendedActorSystem, Props }
-import scala.concurrent.duration.FiniteDuration
-
/**
* INTERNAL API
*/
@@ -40,9 +40,7 @@ private[cluster] object DowningProvider {
* When implementing a downing provider you should make sure that it will not split the cluster into
* several separate clusters in case of network problems or system overload (long GC pauses). This
* is much more difficult than it might be perceived at first, so carefully read the concerns and scenarios
- * described in
- * https://doc.akka.io/docs/akka/current/typed/cluster.html#downing and
- * https://doc.akka.io/docs/akka-enhancements/current/split-brain-resolver.html
+ * described in https://doc.akka.io/docs/akka/current/split-brain-resolver.html
*/
abstract class DowningProvider {
diff --git a/akka-cluster/src/main/scala/akka/cluster/Gossip.scala b/akka-cluster/src/main/scala/akka/cluster/Gossip.scala
index 87b64baee8..d045b43296 100644
--- a/akka-cluster/src/main/scala/akka/cluster/Gossip.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/Gossip.scala
@@ -5,11 +5,12 @@
package akka.cluster
import scala.collection.immutable
+import scala.concurrent.duration.Deadline
+
import ClusterSettings.DataCenter
import MemberStatus._
-import akka.annotation.InternalApi
-import scala.concurrent.duration.Deadline
+import akka.annotation.InternalApi
/**
* INTERNAL API
@@ -315,7 +316,7 @@ private[cluster] class GossipEnvelope private (
private def deserialize(): Unit = {
if ((g eq null) && (ser ne null)) {
- if (serDeadline.hasTimeLeft)
+ if (serDeadline.hasTimeLeft())
g = ser()
else
g = Gossip.empty
diff --git a/akka-cluster/src/main/scala/akka/cluster/JoinConfigCompatCheckCluster.scala b/akka-cluster/src/main/scala/akka/cluster/JoinConfigCompatCheckCluster.scala
index 50ef8a2188..f7f79c75f9 100644
--- a/akka-cluster/src/main/scala/akka/cluster/JoinConfigCompatCheckCluster.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/JoinConfigCompatCheckCluster.scala
@@ -4,10 +4,11 @@
package akka.cluster
-import akka.annotation.InternalApi
+import scala.collection.{ immutable => im }
+
import com.typesafe.config.Config
-import scala.collection.{ immutable => im }
+import akka.annotation.InternalApi
/**
* INTERNAL API
diff --git a/akka-cluster/src/main/scala/akka/cluster/JoinConfigCompatChecker.scala b/akka-cluster/src/main/scala/akka/cluster/JoinConfigCompatChecker.scala
index 07e4b2b952..bf8f6090c0 100644
--- a/akka-cluster/src/main/scala/akka/cluster/JoinConfigCompatChecker.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/JoinConfigCompatChecker.scala
@@ -6,13 +6,14 @@ package akka.cluster
import java.util
+import scala.collection.{ immutable => im }
+
+import com.typesafe.config.{ Config, ConfigFactory, ConfigValue }
+
import akka.actor.ExtendedActorSystem
import akka.annotation.{ DoNotInherit, InternalApi }
import akka.util.ccompat._
-import com.typesafe.config.{ Config, ConfigFactory, ConfigValue }
-
import akka.util.ccompat.JavaConverters._
-import scala.collection.{ immutable => im }
abstract class JoinConfigCompatChecker {
diff --git a/akka-cluster/src/main/scala/akka/cluster/Member.scala b/akka-cluster/src/main/scala/akka/cluster/Member.scala
index 15e0a4f3ca..6af9765a09 100644
--- a/akka-cluster/src/main/scala/akka/cluster/Member.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/Member.scala
@@ -4,13 +4,14 @@
package akka.cluster
-import akka.actor.Address
+import scala.runtime.AbstractFunction2
+
import MemberStatus._
-import akka.annotation.InternalApi
-import akka.cluster.ClusterSettings.DataCenter
import com.github.ghik.silencer.silent
-import scala.runtime.AbstractFunction2
+import akka.actor.Address
+import akka.annotation.InternalApi
+import akka.cluster.ClusterSettings.DataCenter
/**
* Represents the address, current status, and roles of a cluster member node.
diff --git a/akka-cluster/src/main/scala/akka/cluster/MembershipState.scala b/akka-cluster/src/main/scala/akka/cluster/MembershipState.scala
index bc12d5ea3a..f40da163f5 100644
--- a/akka-cluster/src/main/scala/akka/cluster/MembershipState.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/MembershipState.scala
@@ -6,16 +6,16 @@ package akka.cluster
import java.util.concurrent.ThreadLocalRandom
-import scala.collection.immutable
+import scala.annotation.tailrec
import scala.collection.SortedSet
+import scala.collection.immutable
+import scala.util.Random
+
+import akka.annotation.InternalApi
import akka.cluster.ClusterSettings.DataCenter
import akka.cluster.MemberStatus._
-import akka.annotation.InternalApi
import akka.util.ccompat._
-import scala.annotation.tailrec
-import scala.util.Random
-
/**
* INTERNAL API
*/
diff --git a/akka-cluster/src/main/scala/akka/cluster/Reachability.scala b/akka-cluster/src/main/scala/akka/cluster/Reachability.scala
index 54f71eb3a2..f36f46aeff 100644
--- a/akka-cluster/src/main/scala/akka/cluster/Reachability.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/Reachability.scala
@@ -4,9 +4,9 @@
package akka.cluster
-import akka.annotation.InternalApi
-
import scala.collection.immutable
+
+import akka.annotation.InternalApi
import akka.util.ccompat._
/**
diff --git a/akka-cluster/src/main/scala/akka/cluster/SeedNodeProcess.scala b/akka-cluster/src/main/scala/akka/cluster/SeedNodeProcess.scala
index eea6a41d70..16cae77d75 100644
--- a/akka-cluster/src/main/scala/akka/cluster/SeedNodeProcess.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/SeedNodeProcess.scala
@@ -186,7 +186,7 @@ private[cluster] final class FirstSeedNodeProcess(
def receive: Receive = {
case JoinSeedNode =>
- if (timeout.hasTimeLeft) {
+ if (timeout.hasTimeLeft()) {
// send InitJoin to remaining seed nodes (except myself)
receiveJoinSeedNode(remainingSeedNodes)
} else {
diff --git a/akka-cluster/src/main/scala/akka/cluster/VectorClock.scala b/akka-cluster/src/main/scala/akka/cluster/VectorClock.scala
index fbb2086786..0dea6c779f 100644
--- a/akka-cluster/src/main/scala/akka/cluster/VectorClock.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/VectorClock.scala
@@ -5,8 +5,9 @@
package akka.cluster
import java.security.MessageDigest
-import scala.collection.immutable.TreeMap
+
import scala.annotation.tailrec
+import scala.collection.immutable.TreeMap
/**
* VectorClock module with helper classes and methods.
diff --git a/akka-cluster/src/main/scala/akka/cluster/protobuf/ClusterMessageSerializer.scala b/akka-cluster/src/main/scala/akka/cluster/protobuf/ClusterMessageSerializer.scala
index 43e0f5f049..869716685d 100644
--- a/akka-cluster/src/main/scala/akka/cluster/protobuf/ClusterMessageSerializer.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/protobuf/ClusterMessageSerializer.scala
@@ -7,43 +7,40 @@ package akka.cluster.protobuf
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream }
import java.util.zip.{ GZIPInputStream, GZIPOutputStream }
-import akka.actor.{ Address, ExtendedActorSystem }
-import akka.cluster._
-import akka.cluster.protobuf.msg.{ ClusterMessages => cm }
-import akka.serialization._
-import akka.protobufv3.internal.{ ByteString, MessageLite }
-
import scala.annotation.tailrec
import scala.collection.immutable
-import akka.util.ccompat.JavaConverters._
import scala.concurrent.duration.Deadline
-import akka.annotation.InternalApi
-import akka.cluster.InternalClusterAction._
-import akka.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings }
-import akka.routing.Pool
-import akka.util.ccompat._
+
import com.github.ghik.silencer.silent
import com.typesafe.config.{ Config, ConfigFactory, ConfigRenderOptions }
+import akka.actor.{ Address, ExtendedActorSystem }
+import akka.annotation.InternalApi
+import akka.cluster._
+import akka.cluster.InternalClusterAction._
+import akka.cluster.protobuf.msg.{ ClusterMessages => cm }
+import akka.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings }
+import akka.protobufv3.internal.{ ByteString, MessageLite }
+import akka.routing.Pool
+import akka.serialization._
+import akka.util.ccompat._
+import akka.util.ccompat.JavaConverters._
+
/**
* INTERNAL API
*/
@InternalApi
@ccompatUsedUntil213
private[akka] object ClusterMessageSerializer {
- // Kept for one version iteration from 2.6.2 to allow rolling migration to short manifests
- // will be removed in 2.6.3
- // needs to be full class names for backwards compatibility
+ // Kept for one version iteration from 2.6.4 to allow rolling migration to short manifests
+ // can be removed in 2.6.6 or later.
val OldJoinManifest = s"akka.cluster.InternalClusterAction$$Join"
val OldWelcomeManifest = s"akka.cluster.InternalClusterAction$$Welcome"
val OldLeaveManifest = s"akka.cluster.ClusterUserAction$$Leave"
val OldDownManifest = s"akka.cluster.ClusterUserAction$$Down"
- // #24622 wire compatibility
- // we need to use this object name rather than classname to be able to join a 2.5.9 cluster during rolling upgrades
val OldInitJoinManifest = s"akka.cluster.InternalClusterAction$$InitJoin$$"
val OldInitJoinAckManifest = s"akka.cluster.InternalClusterAction$$InitJoinAck"
val OldInitJoinNackManifest = s"akka.cluster.InternalClusterAction$$InitJoinNack"
- // FIXME, remove in a later version (2.6?) and make 2.5.24+ a mandatory step for rolling upgrade
val HeartBeatManifestPre2523 = s"akka.cluster.ClusterHeartbeatSender$$Heartbeat"
val HeartBeatRspManifest2523 = s"akka.cluster.ClusterHeartbeatSender$$HeartbeatRsp"
val OldExitingConfirmedManifest = s"akka.cluster.InternalClusterAction$$ExitingConfirmed"
@@ -82,26 +79,26 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem)
private lazy val GossipTimeToLive = Cluster(system).settings.GossipTimeToLive
def manifest(o: AnyRef): String = o match {
- case _: InternalClusterAction.Join => OldJoinManifest
- case _: InternalClusterAction.Welcome => OldWelcomeManifest
- case _: ClusterUserAction.Leave => OldLeaveManifest
- case _: ClusterUserAction.Down => OldDownManifest
- case _: InternalClusterAction.InitJoin => OldInitJoinManifest
- case _: InternalClusterAction.InitJoinAck => OldInitJoinAckManifest
- case _: InternalClusterAction.InitJoinNack => OldInitJoinNackManifest
- case _: ClusterHeartbeatSender.Heartbeat => HeartBeatManifestPre2523
- case _: ClusterHeartbeatSender.HeartbeatRsp => HeartBeatRspManifest2523
- case _: ExitingConfirmed => OldExitingConfirmedManifest
- case _: GossipStatus => OldGossipStatusManifest
- case _: GossipEnvelope => OldGossipEnvelopeManifest
- case _: ClusterRouterPool => OldClusterRouterPoolManifest
+ case _: InternalClusterAction.Join => JoinManifest
+ case _: InternalClusterAction.Welcome => WelcomeManifest
+ case _: ClusterUserAction.Leave => LeaveManifest
+ case _: ClusterUserAction.Down => DownManifest
+ case _: InternalClusterAction.InitJoin => InitJoinManifest
+ case _: InternalClusterAction.InitJoinAck => InitJoinAckManifest
+ case _: InternalClusterAction.InitJoinNack => InitJoinNackManifest
+ case _: ClusterHeartbeatSender.Heartbeat => HeartbeatManifest
+ case _: ClusterHeartbeatSender.HeartbeatRsp => HeartbeatRspManifest
+ case _: ExitingConfirmed => ExitingConfirmedManifest
+ case _: GossipStatus => GossipStatusManifest
+ case _: GossipEnvelope => GossipEnvelopeManifest
+ case _: ClusterRouterPool => ClusterRouterPoolManifest
case _ =>
throw new IllegalArgumentException(s"Can't serialize object of type ${o.getClass} in [${getClass.getName}]")
}
def toBinary(obj: AnyRef): Array[Byte] = obj match {
- case ClusterHeartbeatSender.Heartbeat(from, _, _) => addressToProtoByteArray(from)
- case ClusterHeartbeatSender.HeartbeatRsp(from, _, _) => uniqueAddressToProtoByteArray(from)
+ case hb: ClusterHeartbeatSender.Heartbeat => heartbeatToProtoByteArray(hb)
+ case hbr: ClusterHeartbeatSender.HeartbeatRsp => heartbeatRspToProtoByteArray(hbr)
case m: GossipEnvelope => gossipEnvelopeToProto(m).toByteArray
case m: GossipStatus => gossipStatusToProto(m).toByteArray
case InternalClusterAction.Join(node, roles) => joinToProto(node, roles).toByteArray
@@ -118,10 +115,22 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem)
}
def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = manifest match {
+ case HeartbeatManifest => deserializeHeartBeat(bytes)
+ case HeartbeatRspManifest => deserializeHeartBeatResponse(bytes)
+ case GossipStatusManifest => deserializeGossipStatus(bytes)
+ case GossipEnvelopeManifest => deserializeGossipEnvelope(bytes)
+ case InitJoinManifest => deserializeInitJoin(bytes)
+ case InitJoinAckManifest => deserializeInitJoinAck(bytes)
+ case InitJoinNackManifest => deserializeInitJoinNack(bytes)
+ case JoinManifest => deserializeJoin(bytes)
+ case WelcomeManifest => deserializeWelcome(bytes)
+ case LeaveManifest => deserializeLeave(bytes)
+ case DownManifest => deserializeDown(bytes)
+ case ExitingConfirmedManifest => deserializeExitingConfirmed(bytes)
+ case ClusterRouterPoolManifest => deserializeClusterRouterPool(bytes)
+ // needs to stay in 2.6.5 to be able to talk to a 2.5.{3,4} node during rolling upgrade
case HeartBeatManifestPre2523 => deserializeHeartBeatAsAddress(bytes)
case HeartBeatRspManifest2523 => deserializeHeartBeatRspAsUniqueAddress(bytes)
- case HeartbeatManifest => deserializeHeartBeat(bytes)
- case HeartbeatRspManifest => deserializeHeartBeatResponse(bytes)
case OldGossipStatusManifest => deserializeGossipStatus(bytes)
case OldGossipEnvelopeManifest => deserializeGossipEnvelope(bytes)
case OldInitJoinManifest => deserializeInitJoin(bytes)
@@ -133,17 +142,6 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem)
case OldDownManifest => deserializeDown(bytes)
case OldExitingConfirmedManifest => deserializeExitingConfirmed(bytes)
case OldClusterRouterPoolManifest => deserializeClusterRouterPool(bytes)
- case GossipStatusManifest => deserializeGossipStatus(bytes)
- case GossipEnvelopeManifest => deserializeGossipEnvelope(bytes)
- case InitJoinManifest => deserializeInitJoin(bytes)
- case InitJoinAckManifest => deserializeInitJoinAck(bytes)
- case InitJoinNackManifest => deserializeInitJoinNack(bytes)
- case JoinManifest => deserializeJoin(bytes)
- case WelcomeManifest => deserializeWelcome(bytes)
- case LeaveManifest => deserializeLeave(bytes)
- case DownManifest => deserializeDown(bytes)
- case ExitingConfirmedManifest => deserializeExitingConfirmed(bytes)
- case ClusterRouterPoolManifest => deserializeClusterRouterPool(bytes)
case _ => throw new IllegalArgumentException(s"Unknown manifest [${manifest}]")
}
@@ -172,6 +170,26 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem)
out.toByteArray
}
+ private def heartbeatToProtoByteArray(hb: ClusterHeartbeatSender.Heartbeat): Array[Byte] = {
+ cm.Heartbeat
+ .newBuilder()
+ .setFrom(addressToProto(hb.from))
+ .setSequenceNr(hb.sequenceNr)
+ .setCreationTime(hb.creationTimeNanos)
+ .build
+ .toByteArray
+ }
+
+ private def heartbeatRspToProtoByteArray(hbr: ClusterHeartbeatSender.HeartbeatRsp): Array[Byte] = {
+ cm.HeartBeatResponse
+ .newBuilder()
+ .setFrom(uniqueAddressToProto(hbr.from))
+ .setSequenceNr(hbr.sequenceNr)
+ .setCreationTime(hbr.creationTimeNanos)
+ .build
+ .toByteArray
+ }
+
private def addressFromBinary(bytes: Array[Byte]): Address =
addressFromProto(cm.Address.parseFrom(bytes))
diff --git a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala b/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala
index a2cb36d545..0baf1d1799 100644
--- a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala
+++ b/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala
@@ -6,6 +6,13 @@ package akka.cluster.routing
import java.util.concurrent.atomic.AtomicInteger
+import scala.annotation.{ tailrec, varargs }
+import scala.collection.immutable
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.cluster.Cluster
import akka.cluster.ClusterEvent._
@@ -24,12 +31,6 @@ import akka.routing.RouterActor
import akka.routing.RouterConfig
import akka.routing.RouterPoolActor
import akka.routing.RoutingLogic
-import com.github.ghik.silencer.silent
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-
-import scala.annotation.{ tailrec, varargs }
-import scala.collection.immutable
import akka.util.ccompat.JavaConverters._
object ClusterRouterGroupSettings {
diff --git a/akka-cluster/src/main/scala/akka/cluster/sbr/DowningStrategy.scala b/akka-cluster/src/main/scala/akka/cluster/sbr/DowningStrategy.scala
new file mode 100644
index 0000000000..72028b7a2e
--- /dev/null
+++ b/akka-cluster/src/main/scala/akka/cluster/sbr/DowningStrategy.scala
@@ -0,0 +1,625 @@
+/*
+ * Copyright (C) 2009-2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import scala.collection.immutable
+import scala.concurrent.duration.Duration
+import scala.concurrent.duration.FiniteDuration
+
+import akka.actor.Address
+import akka.annotation.InternalApi
+import akka.cluster.ClusterSettings.DataCenter
+import akka.cluster.Member
+import akka.cluster.MemberStatus
+import akka.cluster.Reachability
+import akka.cluster.UniqueAddress
+import akka.coordination.lease.scaladsl.Lease
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[sbr] object DowningStrategy {
+ sealed trait Decision {
+ def isIndirectlyConnected: Boolean
+ }
+ case object DownReachable extends Decision {
+ override def isIndirectlyConnected = false
+ }
+ case object DownUnreachable extends Decision {
+ override def isIndirectlyConnected = false
+ }
+ case object DownAll extends Decision {
+ override def isIndirectlyConnected = false
+ }
+ case object DownIndirectlyConnected extends Decision {
+ override def isIndirectlyConnected = true
+ }
+ sealed trait AcquireLeaseDecision extends Decision {
+ def acquireDelay: FiniteDuration
+ }
+ final case class AcquireLeaseAndDownUnreachable(acquireDelay: FiniteDuration) extends AcquireLeaseDecision {
+ override def isIndirectlyConnected = false
+ }
+ final case class AcquireLeaseAndDownIndirectlyConnected(acquireDelay: FiniteDuration) extends AcquireLeaseDecision {
+ override def isIndirectlyConnected = true
+ }
+ case object ReverseDownIndirectlyConnected extends Decision {
+ override def isIndirectlyConnected = true
+ }
+}
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[sbr] abstract class DowningStrategy(val selfDc: DataCenter) {
+ import DowningStrategy._
+
+ // may contain Joining and WeaklyUp
+ private var _unreachable: Set[UniqueAddress] = Set.empty[UniqueAddress]
+
+ def unreachable: Set[UniqueAddress] = _unreachable
+
+ def unreachable(m: Member): Boolean = _unreachable(m.uniqueAddress)
+
+ private var _reachability: Reachability = Reachability.empty
+
+ private var _seenBy: Set[Address] = Set.empty
+
+ protected def ordering: Ordering[Member] = Member.ordering
+
+ // all members in self DC, both joining and up.
+ private var _allMembers: immutable.SortedSet[Member] = immutable.SortedSet.empty(ordering)
+
+ def role: Option[String]
+
+ // all Joining and WeaklyUp members in self DC
+ def joining: immutable.SortedSet[Member] =
+ _allMembers.filter(m => m.status == MemberStatus.Joining || m.status == MemberStatus.WeaklyUp)
+
+ // all members in self DC, both joining and up.
+ def allMembersInDC: immutable.SortedSet[Member] = _allMembers
+
+ /**
+ * All members in self DC, but doesn't contain Joining, WeaklyUp, Down and Exiting.
+ */
+ def members: immutable.SortedSet[Member] =
+ members(includingPossiblyUp = false, excludingPossiblyExiting = false)
+
+ /**
+ * All members in self DC, but doesn't contain Joining, WeaklyUp, Down and Exiting.
+ *
+ * When `includingPossiblyUp=true` it also includes Joining and WeaklyUp members that could have been
+ * changed to Up on the other side of a partition.
+ *
+ * When `excludingPossiblyExiting=true` it doesn't include Leaving members that could have been
+ * changed to Exiting on the other side of the partition.
+ */
+ def members(includingPossiblyUp: Boolean, excludingPossiblyExiting: Boolean): immutable.SortedSet[Member] =
+ _allMembers.filterNot(
+ m =>
+ (!includingPossiblyUp && m.status == MemberStatus.Joining) ||
+ (!includingPossiblyUp && m.status == MemberStatus.WeaklyUp) ||
+ (excludingPossiblyExiting && m.status == MemberStatus.Leaving) ||
+ m.status == MemberStatus.Down ||
+ m.status == MemberStatus.Exiting)
+
+ def membersWithRole: immutable.SortedSet[Member] =
+ membersWithRole(includingPossiblyUp = false, excludingPossiblyExiting = false)
+
+ def membersWithRole(includingPossiblyUp: Boolean, excludingPossiblyExiting: Boolean): immutable.SortedSet[Member] =
+ role match {
+ case None => members(includingPossiblyUp, excludingPossiblyExiting)
+ case Some(r) => members(includingPossiblyUp, excludingPossiblyExiting).filter(_.hasRole(r))
+ }
+
+ def reachableMembers: immutable.SortedSet[Member] =
+ reachableMembers(includingPossiblyUp = false, excludingPossiblyExiting = false)
+
+ def reachableMembers(includingPossiblyUp: Boolean, excludingPossiblyExiting: Boolean): immutable.SortedSet[Member] = {
+ val mbrs = members(includingPossiblyUp, excludingPossiblyExiting)
+ if (unreachable.isEmpty) mbrs
+ else mbrs.filter(m => !unreachable(m))
+ }
+
+ def reachableMembersWithRole: immutable.SortedSet[Member] =
+ reachableMembersWithRole(includingPossiblyUp = false, excludingPossiblyExiting = false)
+
+ def reachableMembersWithRole(
+ includingPossiblyUp: Boolean,
+ excludingPossiblyExiting: Boolean): immutable.SortedSet[Member] =
+ role match {
+ case None => reachableMembers(includingPossiblyUp, excludingPossiblyExiting)
+ case Some(r) => reachableMembers(includingPossiblyUp, excludingPossiblyExiting).filter(_.hasRole(r))
+ }
+
+ def unreachableMembers: immutable.SortedSet[Member] =
+ unreachableMembers(includingPossiblyUp = false, excludingPossiblyExiting = false)
+
+ def unreachableMembers(
+ includingPossiblyUp: Boolean,
+ excludingPossiblyExiting: Boolean): immutable.SortedSet[Member] = {
+ if (unreachable.isEmpty) immutable.SortedSet.empty
+ else members(includingPossiblyUp, excludingPossiblyExiting).filter(unreachable)
+ }
+
+ def unreachableMembersWithRole: immutable.SortedSet[Member] =
+ unreachableMembersWithRole(includingPossiblyUp = false, excludingPossiblyExiting = false)
+
+ def unreachableMembersWithRole(
+ includingPossiblyUp: Boolean,
+ excludingPossiblyExiting: Boolean): immutable.SortedSet[Member] =
+ role match {
+ case None => unreachableMembers(includingPossiblyUp, excludingPossiblyExiting)
+ case Some(r) => unreachableMembers(includingPossiblyUp, excludingPossiblyExiting).filter(_.hasRole(r))
+ }
+
+ def addUnreachable(m: Member): Unit = {
+ require(m.dataCenter == selfDc)
+
+ add(m)
+ _unreachable = _unreachable + m.uniqueAddress
+ }
+
+ def addReachable(m: Member): Unit = {
+ require(m.dataCenter == selfDc)
+
+ add(m)
+ _unreachable = _unreachable - m.uniqueAddress
+ }
+
+ def add(m: Member): Unit = {
+ require(m.dataCenter == selfDc)
+
+ removeFromAllMembers(m)
+ _allMembers += m
+ }
+
+ def remove(m: Member): Unit = {
+ require(m.dataCenter == selfDc)
+
+ removeFromAllMembers(m)
+ _unreachable -= m.uniqueAddress
+ }
+
+ private def removeFromAllMembers(m: Member): Unit = {
+ if (ordering eq Member.ordering) {
+ _allMembers -= m
+ } else {
+ // must use filterNot for removals/replace in the SortedSet when
+ // ageOrdering is using upNumber and that will change when Joining -> Up
+ _allMembers = _allMembers.filterNot(_.uniqueAddress == m.uniqueAddress)
+ }
+ }
+
+ def reachability: Reachability =
+ _reachability
+
+ private def isInSelfDc(node: UniqueAddress): Boolean = {
+ _allMembers.exists(m => m.uniqueAddress == node && m.dataCenter == selfDc)
+ }
+
+ /**
+ * @return true if it was changed
+ */
+ private[sbr] def setReachability(r: Reachability): Boolean = {
+ // skip records with Reachability.Reachable, and skip records related to other DC
+ val newReachability = r.filterRecords(
+ record =>
+ (record.status == Reachability.Unreachable || record.status == Reachability.Terminated) &&
+ isInSelfDc(record.observer) && isInSelfDc(record.subject))
+ val oldReachability = _reachability
+
+ val changed =
+ if (oldReachability.records.size != newReachability.records.size)
+ true
+ else
+ oldReachability.records.map(r => r.observer -> r.subject).toSet !=
+ newReachability.records.map(r => r.observer -> r.subject).toSet
+
+ _reachability = newReachability
+ changed
+ }
+
+ def seenBy: Set[Address] =
+ _seenBy
+
+ def setSeenBy(s: Set[Address]): Unit =
+ _seenBy = s
+
+ /**
+ * Nodes that are marked as unreachable but can communicate with gossip via a 3rd party.
+ *
+ * Cycle in unreachability graph corresponds to that some node is both
+ * observing another node as unreachable, and is also observed as unreachable by someone
+ * else.
+ *
+ * Another indication of indirectly connected nodes is if a node is marked as unreachable,
+ * but it has still marked current gossip state as seen.
+ *
+ * Those cases will not happen for clean splits and crashed nodes.
+ */
+ def indirectlyConnected: Set[UniqueAddress] = {
+ indirectlyConnectedFromIntersectionOfObserversAndSubjects.union(indirectlyConnectedFromSeenCurrentGossip)
+ }
+
+ private def indirectlyConnectedFromIntersectionOfObserversAndSubjects: Set[UniqueAddress] = {
+ // cycle in unreachability graph
+ val observers = reachability.allObservers
+ observers.intersect(reachability.allUnreachableOrTerminated)
+ }
+
+ private def indirectlyConnectedFromSeenCurrentGossip: Set[UniqueAddress] = {
+ reachability.records.flatMap { r =>
+ if (seenBy(r.subject.address)) r.observer :: r.subject :: Nil
+ else Nil
+ }.toSet
+ }
+
+ def hasIndirectlyConnected: Boolean = indirectlyConnected.nonEmpty
+
+ def unreachableButNotIndirectlyConnected: Set[UniqueAddress] = unreachable.diff(indirectlyConnected)
+
+ def nodesToDown(decision: Decision = decide()): Set[UniqueAddress] = {
+ val downable = members
+ .union(joining)
+ .filterNot(m => m.status == MemberStatus.Down || m.status == MemberStatus.Exiting)
+ .map(_.uniqueAddress)
+ decision match {
+ case DownUnreachable | AcquireLeaseAndDownUnreachable(_) => downable.intersect(unreachable)
+ case DownReachable => downable.diff(unreachable)
+ case DownAll => downable
+ case DownIndirectlyConnected | AcquireLeaseAndDownIndirectlyConnected(_) =>
+ // Down nodes that have been marked as unreachable via some network links but they are still indirectly
+ // connected via other links. It will keep other "normal" nodes.
+ // If there is a combination of indirectly connected nodes and a clean network partition (or node crashes)
+ // it will combine the above decision with the ordinary decision, e.g. keep majority, after excluding
+ // failure detection observations between the indirectly connected nodes.
+ // Also include nodes that corresponds to the decision without the unreachability observations from
+ // the indirectly connected nodes
+ downable.intersect(indirectlyConnected.union(additionalNodesToDownWhenIndirectlyConnected))
+ case ReverseDownIndirectlyConnected =>
+ // indirectly connected + all reachable
+ downable.intersect(indirectlyConnected).union(downable.diff(unreachable))
+ }
+ }
+
+ private def additionalNodesToDownWhenIndirectlyConnected: Set[UniqueAddress] = {
+ if (unreachableButNotIndirectlyConnected.isEmpty)
+ Set.empty
+ else {
+ val originalUnreachable = _unreachable
+ val originalReachability = _reachability
+ try {
+ val intersectionOfObserversAndSubjects = indirectlyConnectedFromIntersectionOfObserversAndSubjects
+ val haveSeenCurrentGossip = indirectlyConnectedFromSeenCurrentGossip
+ // remove records between the indirectly connected
+ _reachability = reachability.filterRecords(
+ r =>
+ !((intersectionOfObserversAndSubjects(r.observer) && intersectionOfObserversAndSubjects(r.subject)) ||
+ (haveSeenCurrentGossip(r.observer) && haveSeenCurrentGossip(r.subject))))
+ _unreachable = reachability.allUnreachableOrTerminated
+ val additionalDecision = decide()
+
+ if (additionalDecision.isIndirectlyConnected)
+ throw new IllegalStateException(
+ s"SBR double $additionalDecision decision, downing all instead. " +
+ s"originalReachability: [$originalReachability], filtered reachability [$reachability], " +
+ s"still indirectlyConnected: [$indirectlyConnected], seenBy: [$seenBy]")
+
+ nodesToDown(additionalDecision)
+ } finally {
+ _unreachable = originalUnreachable
+ _reachability = originalReachability
+ }
+ }
+ }
+
+ def isAllUnreachableDownOrExiting: Boolean = {
+ _unreachable.isEmpty ||
+ unreachableMembers.forall(m => m.status == MemberStatus.Down || m.status == MemberStatus.Exiting)
+ }
+
+ def reverseDecision(decision: Decision): Decision = {
+ decision match {
+ case DownUnreachable => DownReachable
+ case AcquireLeaseAndDownUnreachable(_) => DownReachable
+ case DownReachable => DownUnreachable
+ case DownAll => DownAll
+ case DownIndirectlyConnected => ReverseDownIndirectlyConnected
+ case AcquireLeaseAndDownIndirectlyConnected(_) => ReverseDownIndirectlyConnected
+ case ReverseDownIndirectlyConnected => DownIndirectlyConnected
+ }
+ }
+
+ def decide(): Decision
+
+ def lease: Option[Lease] = None
+
+}
+
+/**
+ * INTERNAL API
+ *
+ * Down the unreachable nodes if the number of remaining nodes are greater than or equal to the
+ * given `quorumSize`. Otherwise down the reachable nodes, i.e. it will shut down that side of the partition.
+ * In other words, the `quorumSize` defines the minimum number of nodes that the cluster must have to be operational.
+ * If there are unreachable nodes when starting up the cluster, before reaching this limit,
+ * the cluster may shutdown itself immediately. This is not an issue if you start all nodes at
+ * approximately the same time.
+ *
+ * Note that you must not add more members to the cluster than `quorumSize * 2 - 1`, because then
+ * both sides may down each other and thereby form two separate clusters. For example,
+ * quorum quorumSize configured to 3 in a 6 node cluster may result in a split where each side
+ * consists of 3 nodes each, i.e. each side thinks it has enough nodes to continue by
+ * itself. A warning is logged if this recommendation is violated.
+ *
+ * If the `role` is defined the decision is based only on members with that `role`.
+ *
+ * It is only counting members within the own data center.
+ */
+@InternalApi private[sbr] final class StaticQuorum(
+ selfDc: DataCenter,
+ val quorumSize: Int,
+ override val role: Option[String])
+ extends DowningStrategy(selfDc) {
+ import DowningStrategy._
+
+ override def decide(): Decision = {
+ if (isTooManyMembers)
+ DownAll
+ else if (hasIndirectlyConnected)
+ DownIndirectlyConnected
+ else if (membersWithRole.size - unreachableMembersWithRole.size >= quorumSize)
+ DownUnreachable
+ else
+ DownReachable
+ }
+
+ def isTooManyMembers: Boolean =
+ membersWithRole.size > (quorumSize * 2 - 1)
+}
+
+/**
+ * INTERNAL API
+ *
+ * Down the unreachable nodes if the current node is in the majority part based the last known
+ * membership information. Otherwise down the reachable nodes, i.e. the own part. If the the
+ * parts are of equal size the part containing the node with the lowest address is kept.
+ *
+ * If the `role` is defined the decision is based only on members with that `role`.
+ *
+ * Note that if there are more than two partitions and none is in majority each part
+ * will shutdown itself, terminating the whole cluster.
+ *
+ * It is only counting members within the own data center.
+ */
+@InternalApi private[sbr] final class KeepMajority(selfDc: DataCenter, override val role: Option[String])
+ extends DowningStrategy(selfDc) {
+ import DowningStrategy._
+
+ override def decide(): Decision = {
+ if (hasIndirectlyConnected)
+ DownIndirectlyConnected
+ else {
+ val ms = membersWithRole
+ if (ms.isEmpty)
+ DownAll // no node with matching role
+ else {
+ val reachableSize = reachableMembersWithRole.size
+ val unreachableSize = unreachableMembersWithRole.size
+
+ majorityDecision(reachableSize, unreachableSize, ms.head) match {
+ case DownUnreachable =>
+ majorityDecisionWhenIncludingMembershipChangesEdgeCase() match {
+ case DownUnreachable => DownUnreachable // same conclusion
+ case _ => DownAll // different conclusion, safest to DownAll
+ }
+ case decision => decision
+ }
+
+ }
+ }
+ }
+
+ private def majorityDecision(thisSide: Int, otherSide: Int, lowest: Member): Decision = {
+ if (thisSide == otherSide) {
+ // equal size, keep the side with the lowest address (first in members)
+ if (unreachable(lowest)) DownReachable else DownUnreachable
+ } else if (thisSide > otherSide) {
+ // we are in majority
+ DownUnreachable
+ } else {
+ // we are in minority
+ DownReachable
+ }
+ }
+
+ /**
+ * Check for edge case when membership change happens at the same time as partition.
+ * Count Joining and WeaklyUp on other side since those might be Up on other side.
+ * Don't count Leaving on this side since those might be Exiting on other side.
+ * Note that the membership changes we are looking for will only be done when all
+ * members have seen previous state, i.e. when a member is moved to Up everybody
+ * has seen it joining.
+ */
+ private def majorityDecisionWhenIncludingMembershipChangesEdgeCase(): Decision = {
+ // for this side we count as few as could be possible (excluding joining, excluding leaving)
+ val ms = membersWithRole(includingPossiblyUp = false, excludingPossiblyExiting = true)
+ if (ms.isEmpty) {
+ DownAll
+ } else {
+ val thisSideReachableSize =
+ reachableMembersWithRole(includingPossiblyUp = false, excludingPossiblyExiting = true).size
+ // for other side we count as many as could be possible (including joining, including leaving)
+ val otherSideUnreachableSize =
+ unreachableMembersWithRole(includingPossiblyUp = true, excludingPossiblyExiting = false).size
+ majorityDecision(thisSideReachableSize, otherSideUnreachableSize, ms.head)
+ }
+ }
+
+}
+
+/**
+ * INTERNAL API
+ *
+ * Down the part that does not contain the oldest member (current singleton).
+ *
+ * There is one exception to this rule if `downIfAlone` is defined to `true`.
+ * Then, if the oldest node has partitioned from all other nodes the oldest will
+ * down itself and keep all other nodes running. The strategy will not down the
+ * single oldest node when it is the only remaining node in the cluster.
+ *
+ * Note that if the oldest node crashes the others will remove it from the cluster
+ * when `downIfAlone` is `true`, otherwise they will down themselves if the
+ * oldest node crashes, i.e. shutdown the whole cluster together with the oldest node.
+ *
+ * If the `role` is defined the decision is based only on members with that `role`,
+ * i.e. using the oldest member (singleton) within the nodes with that role.
+ *
+ * It is only using members within the own data center, i.e. oldest within the
+ * data center.
+ */
+@InternalApi private[sbr] final class KeepOldest(
+ selfDc: DataCenter,
+ val downIfAlone: Boolean,
+ override val role: Option[String])
+ extends DowningStrategy(selfDc) {
+ import DowningStrategy._
+
+ // sort by age, oldest first
+ override def ordering: Ordering[Member] = Member.ageOrdering
+
+ override def decide(): Decision = {
+ if (hasIndirectlyConnected)
+ DownIndirectlyConnected
+ else {
+ val ms = membersWithRole
+ if (ms.isEmpty)
+ DownAll // no node with matching role
+ else {
+ val oldest = ms.head
+ val oldestIsReachable = !unreachable(oldest)
+ val reachableCount = reachableMembersWithRole.size
+ val unreachableCount = unreachableMembersWithRole.size
+
+ oldestDecision(oldestIsReachable, reachableCount, unreachableCount) match {
+ case DownUnreachable =>
+ oldestDecisionWhenIncludingMembershipChangesEdgeCase() match {
+ case DownUnreachable => DownUnreachable // same conclusion
+ case _ => DownAll // different conclusion, safest to DownAll
+ }
+ case decision => decision
+ }
+
+ }
+ }
+ }
+
+ private def oldestDecision(oldestIsOnThisSide: Boolean, thisSide: Int, otherSide: Int): Decision = {
+ if (oldestIsOnThisSide) {
+ // if there are only 2 nodes in the cluster it is better to keep the oldest, even though it is alone
+ // E.g. 2 nodes: thisSide=1, otherSide=1 => DownUnreachable, i.e. keep the oldest
+ // even though it is alone (because the node on the other side is no better)
+ // E.g. 3 nodes: thisSide=1, otherSide=2 => DownReachable, i.e. shut down the
+ // oldest because it is alone
+ if (downIfAlone && thisSide == 1 && otherSide >= 2) DownReachable
+ else DownUnreachable
+ } else {
+ if (downIfAlone && otherSide == 1 && thisSide >= 2) DownUnreachable
+ else DownReachable
+ }
+ }
+
+ /**
+ * Check for edge case when membership change happens at the same time as partition.
+ * Exclude Leaving on this side because those could be Exiting on other side.
+ *
+ * When `downIfAlone` also consider Joining and WeaklyUp since those might be Up on other side,
+ * and thereby flip the alone test.
+ */
+ private def oldestDecisionWhenIncludingMembershipChangesEdgeCase(): Decision = {
+ val ms = membersWithRole(includingPossiblyUp = false, excludingPossiblyExiting = true)
+ if (ms.isEmpty) {
+ DownAll
+ } else {
+ val oldest = ms.head
+ val oldestIsReachable = !unreachable(oldest)
+ // Joining and WeaklyUp are only relevant when downIfAlone = true
+ val includingPossiblyUp = downIfAlone
+ val reachableCount = reachableMembersWithRole(includingPossiblyUp, excludingPossiblyExiting = true).size
+ val unreachableCount = unreachableMembersWithRole(includingPossiblyUp, excludingPossiblyExiting = true).size
+
+ oldestDecision(oldestIsReachable, reachableCount, unreachableCount)
+ }
+ }
+}
+
+/**
+ * INTERNAL API
+ *
+ * Down all nodes unconditionally.
+ */
+@InternalApi private[sbr] final class DownAllNodes(selfDc: DataCenter) extends DowningStrategy(selfDc) {
+ import DowningStrategy._
+
+ override def decide(): Decision =
+ DownAll
+
+ override def role: Option[String] = None
+}
+
+/**
+ * INTERNAL API
+ *
+ * Keep the part that can acquire the lease, and down the other part.
+ *
+ * Best effort is to keep the side that has most nodes, i.e. the majority side.
+ * This is achieved by adding a delay before trying to acquire the lease on the
+ * minority side.
+ *
+ * If the `role` is defined the majority/minority is based only on members with that `role`.
+ * It is only counting members within the own data center.
+ */
+@InternalApi private[sbr] final class LeaseMajority(
+ selfDc: DataCenter,
+ override val role: Option[String],
+ _lease: Lease,
+ acquireLeaseDelayForMinority: FiniteDuration)
+ extends DowningStrategy(selfDc) {
+ import DowningStrategy._
+
+ override val lease: Option[Lease] = Some(_lease)
+
+ override def decide(): Decision = {
+ if (hasIndirectlyConnected)
+ AcquireLeaseAndDownIndirectlyConnected(Duration.Zero)
+ else
+ AcquireLeaseAndDownUnreachable(acquireLeaseDelay)
+ }
+
+ private def acquireLeaseDelay: FiniteDuration =
+ if (isInMinority) acquireLeaseDelayForMinority else Duration.Zero
+
+ private def isInMinority: Boolean = {
+ val ms = membersWithRole
+ if (ms.isEmpty)
+ false // no node with matching role
+ else {
+ val unreachableSize = unreachableMembersWithRole.size
+ val membersSize = ms.size
+
+ if (unreachableSize * 2 == membersSize) {
+ // equal size, try to keep the side with the lowest address (first in members)
+ unreachable(ms.head)
+ } else if (unreachableSize * 2 < membersSize) {
+ // we are in majority
+ false
+ } else {
+ // we are in minority
+ true
+ }
+ }
+ }
+}
diff --git a/akka-cluster/src/main/scala/akka/cluster/sbr/SplitBrainResolver.scala b/akka-cluster/src/main/scala/akka/cluster/sbr/SplitBrainResolver.scala
new file mode 100644
index 0000000000..4cdb9137b8
--- /dev/null
+++ b/akka-cluster/src/main/scala/akka/cluster/sbr/SplitBrainResolver.scala
@@ -0,0 +1,577 @@
+/*
+ * Copyright (C) 2009-2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import java.time.Instant
+import java.time.temporal.ChronoUnit
+
+import scala.concurrent.ExecutionContext
+import scala.concurrent.duration._
+
+import akka.actor.Actor
+import akka.actor.ActorLogging
+import akka.actor.Address
+import akka.actor.ExtendedActorSystem
+import akka.actor.Props
+import akka.actor.Stash
+import akka.actor.Timers
+import akka.annotation.InternalApi
+import akka.cluster.Cluster
+import akka.cluster.ClusterEvent
+import akka.cluster.ClusterEvent._
+import akka.cluster.ClusterSettings.DataCenter
+import akka.cluster.Member
+import akka.cluster.Reachability
+import akka.cluster.UniqueAddress
+import akka.pattern.pipe
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[sbr] object SplitBrainResolver {
+
+ def props(stableAfter: FiniteDuration, strategy: DowningStrategy): Props =
+ Props(new SplitBrainResolver(stableAfter, strategy))
+
+ case object Tick
+
+ /**
+ * Response (result) of the acquire lease request.
+ */
+ final case class AcquireLeaseResult(holdingLease: Boolean)
+
+ /**
+ * Response (result) of the release lease request.
+ */
+ final case class ReleaseLeaseResult(released: Boolean)
+
+ /**
+ * For delayed acquire of the lease.
+ */
+ case object AcquireLease
+
+ sealed trait ReleaseLeaseCondition
+ object ReleaseLeaseCondition {
+ case object NoLease extends ReleaseLeaseCondition
+ final case class WhenMembersRemoved(nodes: Set[UniqueAddress]) extends ReleaseLeaseCondition
+ final case class WhenTimeElapsed(deadline: Deadline) extends ReleaseLeaseCondition
+ }
+
+ final case class ReachabilityChangedStats(
+ firstChangeTimestamp: Long,
+ latestChangeTimestamp: Long,
+ changeCount: Long) {
+
+ def isEmpty: Boolean =
+ changeCount == 0
+
+ override def toString: String = {
+ if (isEmpty)
+ "reachability unchanged"
+ else {
+ val now = System.nanoTime()
+ s"reachability changed $changeCount times since ${(now - firstChangeTimestamp).nanos.toMillis} ms ago, " +
+ s"latest change was ${(now - latestChangeTimestamp).nanos.toMillis} ms ago"
+ }
+ }
+ }
+
+}
+
+/**
+ * INTERNAL API
+ *
+ * Unreachable members will be downed by this actor according to the given strategy.
+ * It is active on the leader node in the cluster.
+ *
+ * The implementation is split into two classes SplitBrainResolver and SplitBrainResolverBase to be
+ * able to unit test the logic without running cluster.
+ */
+@InternalApi private[sbr] final class SplitBrainResolver(stableAfter: FiniteDuration, strategy: DowningStrategy)
+ extends SplitBrainResolverBase(stableAfter, strategy) {
+
+ private val cluster = Cluster(context.system)
+
+ log.info(
+ "SBR started. Config: stableAfter: {} ms, strategy: {}, selfUniqueAddress: {}, selfDc: {}",
+ stableAfter.toMillis,
+ strategy.getClass.getSimpleName,
+ selfUniqueAddress,
+ selfDc)
+
+ override def selfUniqueAddress: UniqueAddress = cluster.selfUniqueAddress
+ override def selfDc: DataCenter = cluster.selfDataCenter
+
+ // re-subscribe when restart
+ override def preStart(): Unit = {
+ cluster.subscribe(self, ClusterEvent.InitialStateAsEvents, classOf[ClusterDomainEvent])
+ super.preStart()
+ }
+ override def postStop(): Unit = {
+ cluster.unsubscribe(self)
+ super.postStop()
+ }
+
+ override def down(node: Address): Unit = {
+ cluster.down(node)
+ }
+
+}
+
+/**
+ * INTERNAL API
+ *
+ * The implementation is split into two classes SplitBrainResolver and SplitBrainResolverBase to be
+ * able to unit test the logic without running cluster.
+ */
+@InternalApi private[sbr] abstract class SplitBrainResolverBase(stableAfter: FiniteDuration, strategy: DowningStrategy)
+ extends Actor
+ with ActorLogging
+ with Stash
+ with Timers {
+
+ import DowningStrategy._
+ import SplitBrainResolver.ReleaseLeaseCondition.NoLease
+ import SplitBrainResolver._
+
+ def selfUniqueAddress: UniqueAddress
+
+ def selfDc: DataCenter
+
+ def down(node: Address): Unit
+
+ // would be better as constructor parameter, but don't want to break Cinnamon instrumentation
+ private val settings = new SplitBrainResolverSettings(context.system.settings.config)
+
+ def downAllWhenUnstable: FiniteDuration =
+ settings.DownAllWhenUnstable
+
+ private val releaseLeaseAfter = stableAfter * 2
+
+ def tickInterval: FiniteDuration = 1.second
+
+ timers.startTimerWithFixedDelay(Tick, Tick, tickInterval)
+
+ var leader = false
+ var selfMemberAdded = false
+
+ private def internalDispatcher: ExecutionContext =
+ context.system.asInstanceOf[ExtendedActorSystem].dispatchers.internalDispatcher
+
+ // overridden in tests
+ protected def newStableDeadline(): Deadline = Deadline.now + stableAfter
+ var stableDeadline: Deadline = _
+ def resetStableDeadline(): Unit = {
+ stableDeadline = newStableDeadline()
+ }
+
+ resetStableDeadline()
+
+ private var reachabilityChangedStats: ReachabilityChangedStats =
+ ReachabilityChangedStats(System.nanoTime(), System.nanoTime(), 0)
+
+ private def resetReachabilityChangedStats(): Unit = {
+ val now = System.nanoTime()
+ reachabilityChangedStats = ReachabilityChangedStats(now, now, 0)
+ }
+
+ private def resetReachabilityChangedStatsIfAllUnreachableDowned(): Unit = {
+ if (!reachabilityChangedStats.isEmpty && strategy.isAllUnreachableDownOrExiting) {
+ log.debug("SBR resetting reachability stats, after all unreachable healed, downed or removed")
+ resetReachabilityChangedStats()
+ }
+ }
+
+ private var releaseLeaseCondition: ReleaseLeaseCondition = NoLease
+
+ /** Helper to wrap updates to strategy info with, so that stable-after timer is reset and information is logged about state change */
+ def mutateMemberInfo(resetStable: Boolean)(f: () => Unit): Unit = {
+ val unreachableBefore = strategy.unreachable.size
+ f()
+ val unreachableAfter = strategy.unreachable.size
+
+ def earliestTimeOfDecision: String =
+ Instant.now().plus(stableAfter.toMillis, ChronoUnit.MILLIS).toString
+
+ if (resetStable) {
+ if (isResponsible) {
+ if (unreachableBefore == 0 && unreachableAfter > 0) {
+ log.info(
+ "SBR found unreachable members, waiting for stable-after = {} ms before taking downing decision. " +
+ "Now {} unreachable members found. Downing decision will not be made before {}.",
+ stableAfter.toMillis,
+ unreachableAfter,
+ earliestTimeOfDecision)
+ } else if (unreachableBefore > 0 && unreachableAfter == 0) {
+ log.info(
+ "SBR found all unreachable members healed during stable-after period, no downing decision necessary for now.")
+ } else if (unreachableAfter > 0) {
+ log.info(
+ "SBR found unreachable members changed during stable-after period. Resetting timer. " +
+ "Now {} unreachable members found. Downing decision will not be made before {}.",
+ unreachableAfter,
+ earliestTimeOfDecision)
+ }
+ // else no unreachable members found but set of members changed
+ }
+
+ log.debug("SBR reset stable deadline when members/unreachable changed")
+ resetStableDeadline()
+ }
+ }
+
+ /** Helper to wrap updates to `leader` and `selfMemberAdded` to log changes in responsibility status */
+ def mutateResponsibilityInfo(f: () => Unit): Unit = {
+ val responsibleBefore = isResponsible
+ f()
+ val responsibleAfter = isResponsible
+
+ if (!responsibleBefore && responsibleAfter)
+ log.info(
+ "This node is now the leader responsible for taking SBR decisions among the reachable nodes " +
+ "(more leaders may exist).")
+ else if (responsibleBefore && !responsibleAfter)
+ log.info("This node is not the leader any more and not responsible for taking SBR decisions.")
+
+ if (leader && !selfMemberAdded)
+ log.debug("This node is leader but !selfMemberAdded.")
+ }
+
+ private var unreachableDataCenters = Set.empty[DataCenter]
+
+ override def postStop(): Unit = {
+ if (releaseLeaseCondition != NoLease) {
+ log.info(
+ "SBR is stopped and owns the lease. The lease will not be released until after the " +
+ "lease heartbeat-timeout.")
+ }
+ super.postStop()
+ }
+
+ def receive: Receive = {
+ case SeenChanged(_, seenBy) => seenChanged(seenBy)
+ case MemberJoined(m) => addJoining(m)
+ case MemberWeaklyUp(m) => addWeaklyUp(m)
+ case MemberUp(m) => addUp(m)
+ case MemberLeft(m) => leaving(m)
+ case UnreachableMember(m) => unreachableMember(m)
+ case MemberDowned(m) => unreachableMember(m)
+ case MemberExited(m) => unreachableMember(m)
+ case ReachableMember(m) => reachableMember(m)
+ case ReachabilityChanged(r) => reachabilityChanged(r)
+ case MemberRemoved(m, _) => remove(m)
+ case UnreachableDataCenter(dc) => unreachableDataCenter(dc)
+ case ReachableDataCenter(dc) => reachableDataCenter(dc)
+ case LeaderChanged(leaderOption) => leaderChanged(leaderOption)
+ case ReleaseLeaseResult(released) => releaseLeaseResult(released)
+ case Tick => tick()
+ case _: ClusterDomainEvent => // not interested in other events
+ }
+
+ private def leaderChanged(leaderOption: Option[Address]): Unit = {
+ mutateResponsibilityInfo { () =>
+ leader = leaderOption.contains(selfUniqueAddress.address)
+ }
+ }
+
+ private def tick(): Unit = {
+ // note the DownAll due to instability is running on all nodes to make that decision as quickly and
+ // aggressively as possible if time is out
+ if (reachabilityChangedStats.changeCount > 0) {
+ val now = System.nanoTime()
+ val durationSinceLatestChange = (now - reachabilityChangedStats.latestChangeTimestamp).nanos
+ val durationSinceFirstChange = (now - reachabilityChangedStats.firstChangeTimestamp).nanos
+
+ if (durationSinceLatestChange > (stableAfter * 2)) {
+ log.debug("SBR no reachability changes within {} ms, resetting stats", (stableAfter * 2).toMillis)
+ resetReachabilityChangedStats()
+ } else if (downAllWhenUnstable > Duration.Zero &&
+ durationSinceFirstChange > (stableAfter + downAllWhenUnstable)) {
+ log.warning("SBR detected instability and will down all nodes: {}", reachabilityChangedStats)
+ actOnDecision(DownAll)
+ }
+ }
+
+ if (isResponsible && strategy.unreachable.nonEmpty && stableDeadline.isOverdue()) {
+ strategy.decide() match {
+ case decision: AcquireLeaseDecision =>
+ strategy.lease match {
+ case Some(lease) =>
+ if (lease.checkLease()) {
+ log.info("SBR has acquired lease for decision [{}]", decision)
+ actOnDecision(decision)
+ } else {
+ if (decision.acquireDelay == Duration.Zero)
+ acquireLease() // reply message is AcquireLeaseResult
+ else {
+ log.debug("SBR delayed attempt to acquire lease for [{} ms]", decision.acquireDelay.toMillis)
+ timers.startSingleTimer(AcquireLease, AcquireLease, decision.acquireDelay)
+ }
+ context.become(waitingForLease(decision))
+ }
+ case None =>
+ throw new IllegalStateException("Unexpected lease decision although lease is not configured")
+ }
+
+ case decision =>
+ actOnDecision(decision)
+ }
+ }
+
+ releaseLeaseCondition match {
+ case ReleaseLeaseCondition.WhenTimeElapsed(deadline) =>
+ if (deadline.isOverdue())
+ releaseLease() // reply message is ReleaseLeaseResult, which will update the releaseLeaseCondition
+ case _ =>
+ // no lease or first waiting for downed nodes to be removed
+ }
+ }
+
+ private def acquireLease(): Unit = {
+ log.debug("SBR trying to acquire lease")
+ implicit val ec: ExecutionContext = internalDispatcher
+ strategy.lease.foreach(
+ _.acquire()
+ .recover {
+ case t =>
+ log.error(t, "SBR acquire of lease failed")
+ false
+ }
+ .map(AcquireLeaseResult)
+ .pipeTo(self))
+ }
+
+ def waitingForLease(decision: Decision): Receive = {
+ case AcquireLease =>
+ acquireLease() // reply message is LeaseResult
+
+ case AcquireLeaseResult(holdingLease) =>
+ if (holdingLease) {
+ log.info("SBR acquired lease for decision [{}]", decision)
+ val downedNodes = actOnDecision(decision)
+ releaseLeaseCondition = releaseLeaseCondition match {
+ case ReleaseLeaseCondition.WhenMembersRemoved(nodes) =>
+ ReleaseLeaseCondition.WhenMembersRemoved(nodes.union(downedNodes))
+ case _ =>
+ if (downedNodes.isEmpty)
+ ReleaseLeaseCondition.WhenTimeElapsed(Deadline.now + releaseLeaseAfter)
+ else
+ ReleaseLeaseCondition.WhenMembersRemoved(downedNodes)
+ }
+ } else {
+ val reverseDecision = strategy.reverseDecision(decision)
+ log.info("SBR couldn't acquire lease, reverse decision [{}] to [{}]", decision, reverseDecision)
+ actOnDecision(reverseDecision)
+ releaseLeaseCondition = NoLease
+ }
+
+ unstashAll()
+ context.become(receive)
+
+ case ReleaseLeaseResult(_) => // superseded by new acquire release request
+ case Tick => // ignore ticks while waiting
+ case _ =>
+ stash()
+ }
+
+ private def releaseLeaseResult(released: Boolean): Unit = {
+ releaseLeaseCondition match {
+ case ReleaseLeaseCondition.WhenTimeElapsed(deadline) =>
+ if (released && deadline.isOverdue())
+ releaseLeaseCondition = NoLease // released successfully
+ case _ =>
+ // no lease or first waiting for downed nodes to be removed
+ }
+ }
+
+ /**
+ * @return the nodes that were downed
+ */
+ def actOnDecision(decision: Decision): Set[UniqueAddress] = {
+ val nodesToDown =
+ try {
+ strategy.nodesToDown(decision)
+ } catch {
+ case e: IllegalStateException =>
+ log.warning(e.getMessage)
+ strategy.nodesToDown(DownAll)
+ }
+
+ val downMyself = nodesToDown.contains(selfUniqueAddress)
+
+ val indirectlyConnectedLogMessage =
+ if (decision.isIndirectlyConnected)
+ s", indirectly connected [${strategy.indirectlyConnected.mkString(", ")}]"
+ else ""
+ val unreachableDataCentersLogMessage =
+ if (unreachableDataCenters.nonEmpty)
+ s", unreachable DCs [${unreachableDataCenters.mkString(", ")}]"
+ else ""
+
+ log.warning(
+ s"SBR took decision $decision and is downing [${nodesToDown.map(_.address).mkString(", ")}]${if (downMyself) " including myself,"
+ else ""}, " +
+ s"[${strategy.unreachable.size}] unreachable of [${strategy.members.size}] members" +
+ indirectlyConnectedLogMessage +
+ s", all members in DC [${strategy.allMembersInDC.mkString(", ")}], full reachability status: ${strategy.reachability}" +
+ unreachableDataCentersLogMessage)
+
+ if (nodesToDown.nonEmpty) {
+ // downing is idempotent, and we also avoid calling down on nodes with status Down
+ // down selfAddress last, since it may shutdown itself if down alone
+ nodesToDown.foreach(uniqueAddress => if (uniqueAddress != selfUniqueAddress) down(uniqueAddress.address))
+ if (downMyself)
+ down(selfUniqueAddress.address)
+
+ resetReachabilityChangedStats()
+ resetStableDeadline()
+ }
+ nodesToDown
+ }
+
+ def isResponsible: Boolean = leader && selfMemberAdded
+
+ def unreachableMember(m: Member): Unit = {
+ if (m.uniqueAddress != selfUniqueAddress && m.dataCenter == selfDc) {
+ log.debug("SBR unreachableMember [{}]", m)
+ mutateMemberInfo(resetStable = true) { () =>
+ strategy.addUnreachable(m)
+ resetReachabilityChangedStatsIfAllUnreachableDowned()
+ }
+ }
+ }
+
+ def reachableMember(m: Member): Unit = {
+ if (m.uniqueAddress != selfUniqueAddress && m.dataCenter == selfDc) {
+ log.debug("SBR reachableMember [{}]", m)
+ mutateMemberInfo(resetStable = true) { () =>
+ strategy.addReachable(m)
+ resetReachabilityChangedStatsIfAllUnreachableDowned()
+ }
+ }
+ }
+
+ private[sbr] def reachabilityChanged(r: Reachability): Unit = {
+ if (strategy.setReachability(r)) {
+ // resetStableDeadline is done from unreachableMember/reachableMember
+ updateReachabilityChangedStats()
+ // it may also change when members are removed and therefore the reset may be needed
+ resetReachabilityChangedStatsIfAllUnreachableDowned()
+ log.debug("SBR noticed {}", reachabilityChangedStats)
+ }
+ }
+
+ private def updateReachabilityChangedStats(): Unit = {
+ val now = System.nanoTime()
+ if (reachabilityChangedStats.changeCount == 0)
+ reachabilityChangedStats = ReachabilityChangedStats(now, now, 1)
+ else
+ reachabilityChangedStats = reachabilityChangedStats.copy(
+ latestChangeTimestamp = now,
+ changeCount = reachabilityChangedStats.changeCount + 1)
+ }
+
+ def unreachableDataCenter(dc: DataCenter): Unit = {
+ unreachableDataCenters += dc
+ log.warning(
+ "Data center [{}] observed as unreachable. " +
+ "Note that nodes in other data center will not be downed by SBR in this data center [{}]",
+ dc,
+ selfDc)
+ }
+
+ def reachableDataCenter(dc: DataCenter): Unit = {
+ unreachableDataCenters -= dc
+ log.info("Data center [] observed as reachable again", dc)
+ }
+
+ def seenChanged(seenBy: Set[Address]): Unit = {
+ strategy.setSeenBy(seenBy)
+ }
+
+ def addUp(m: Member): Unit = {
+ if (selfDc == m.dataCenter) {
+ log.debug("SBR add Up [{}]", m)
+ mutateMemberInfo(resetStable = true) { () =>
+ strategy.add(m)
+ if (m.uniqueAddress == selfUniqueAddress) mutateResponsibilityInfo { () =>
+ selfMemberAdded = true
+ }
+ }
+ strategy match {
+ case s: StaticQuorum =>
+ if (s.isTooManyMembers)
+ log.warning(
+ "The cluster size is [{}] and static-quorum.quorum-size is [{}]. You should not add " +
+ "more than [{}] (static-quorum.size * 2 - 1) members to the cluster. If the exceeded cluster size " +
+ "remains when a SBR decision is needed it will down all nodes.",
+ s.membersWithRole.size,
+ s.quorumSize,
+ s.quorumSize * 2 - 1)
+ case _ => // ok
+ }
+ }
+ }
+
+ def leaving(m: Member): Unit = {
+ if (selfDc == m.dataCenter) {
+ log.debug("SBR leaving [{}]", m)
+ mutateMemberInfo(resetStable = false) { () =>
+ strategy.add(m)
+ }
+ }
+ }
+
+ def addJoining(m: Member): Unit = {
+ if (selfDc == m.dataCenter) {
+ log.debug("SBR add Joining/WeaklyUp [{}]", m)
+ strategy.add(m)
+ }
+ }
+
+ def addWeaklyUp(m: Member): Unit = {
+ if (m.uniqueAddress == selfUniqueAddress) mutateResponsibilityInfo { () =>
+ selfMemberAdded = true
+ }
+ // treat WeaklyUp in same way as joining
+ addJoining(m)
+ }
+
+ def remove(m: Member): Unit = {
+ if (selfDc == m.dataCenter) {
+ if (m.uniqueAddress == selfUniqueAddress)
+ context.stop(self)
+ else
+ mutateMemberInfo(resetStable = false) { () =>
+ log.debug("SBR remove [{}]", m)
+ strategy.remove(m)
+
+ resetReachabilityChangedStatsIfAllUnreachableDowned()
+
+ releaseLeaseCondition = releaseLeaseCondition match {
+ case ReleaseLeaseCondition.WhenMembersRemoved(downedNodes) =>
+ val remainingDownedNodes = downedNodes - m.uniqueAddress
+ if (remainingDownedNodes.isEmpty)
+ ReleaseLeaseCondition.WhenTimeElapsed(Deadline.now + releaseLeaseAfter)
+ else
+ ReleaseLeaseCondition.WhenMembersRemoved(remainingDownedNodes)
+ case other =>
+ // no lease or not holding lease
+ other
+ }
+ }
+ }
+ }
+
+ private def releaseLease(): Unit = {
+ implicit val ec: ExecutionContext = internalDispatcher
+ strategy.lease.foreach { l =>
+ if (releaseLeaseCondition != NoLease) {
+ log.info("SBR releasing lease")
+ l.release().recover { case _ => false }.map(ReleaseLeaseResult.apply).pipeTo(self)
+ }
+ }
+ }
+}
diff --git a/akka-cluster/src/main/scala/akka/cluster/sbr/SplitBrainResolverProvider.scala b/akka-cluster/src/main/scala/akka/cluster/sbr/SplitBrainResolverProvider.scala
new file mode 100644
index 0000000000..24c63cce9c
--- /dev/null
+++ b/akka-cluster/src/main/scala/akka/cluster/sbr/SplitBrainResolverProvider.scala
@@ -0,0 +1,63 @@
+/*
+ * Copyright (C) 2009-2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import scala.concurrent.duration.Duration
+import scala.concurrent.duration.FiniteDuration
+
+import akka.actor.ActorSystem
+import akka.actor.Props
+import akka.cluster.Cluster
+import akka.cluster.DowningProvider
+import akka.coordination.lease.scaladsl.LeaseProvider
+
+/**
+ * See reference documentation: https://doc.akka.io/docs/akka/current/split-brain-resolver.html
+ *
+ * Enabled with configuration:
+ * {{{
+ * akka.cluster.downing-provider-class = "akka.cluster.sbr.SplitBrainResolverProvider"
+ * }}}
+ */
+final class SplitBrainResolverProvider(system: ActorSystem) extends DowningProvider {
+
+ private val settings = new SplitBrainResolverSettings(system.settings.config)
+
+ override def downRemovalMargin: FiniteDuration = {
+ // if down-removal-margin is defined we let it trump stable-after to allow
+ // for two different values for SBR downing and cluster tool stop/start after downing
+ val drm = Cluster(system).settings.DownRemovalMargin
+ if (drm != Duration.Zero) drm
+ else settings.DowningStableAfter
+ }
+
+ override def downingActorProps: Option[Props] = {
+ import SplitBrainResolverSettings._
+
+ val cluster = Cluster(system)
+ val selfDc = cluster.selfDataCenter
+ val strategy =
+ settings.DowningStrategy match {
+ case KeepMajorityName =>
+ new KeepMajority(selfDc, settings.keepMajorityRole)
+ case StaticQuorumName =>
+ val s = settings.staticQuorumSettings
+ new StaticQuorum(selfDc, s.size, s.role)
+ case KeepOldestName =>
+ val s = settings.keepOldestSettings
+ new KeepOldest(selfDc, s.downIfAlone, s.role)
+ case DownAllName =>
+ new DownAllNodes(selfDc)
+ case LeaseMajorityName =>
+ val s = settings.leaseMajoritySettings
+ val leaseOwnerName = cluster.selfUniqueAddress.address.hostPort
+ val lease = LeaseProvider(system).getLease(s"${system.name}-akka-sbr", s.leaseImplementation, leaseOwnerName)
+ new LeaseMajority(selfDc, s.role, lease, s.acquireLeaseDelayForMinority)
+ }
+
+ Some(SplitBrainResolver.props(settings.DowningStableAfter, strategy))
+ }
+
+}
diff --git a/akka-cluster/src/main/scala/akka/cluster/sbr/SplitBrainResolverSettings.scala b/akka-cluster/src/main/scala/akka/cluster/sbr/SplitBrainResolverSettings.scala
new file mode 100644
index 0000000000..723cc61164
--- /dev/null
+++ b/akka-cluster/src/main/scala/akka/cluster/sbr/SplitBrainResolverSettings.scala
@@ -0,0 +1,128 @@
+/*
+ * Copyright (C) 2009-2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import java.util.Locale
+import java.util.concurrent.TimeUnit
+
+import scala.concurrent.duration.Duration
+import scala.concurrent.duration.FiniteDuration
+
+import com.typesafe.config.Config
+
+import akka.ConfigurationException
+import akka.annotation.InternalApi
+import akka.util.Helpers
+import akka.util.Helpers.Requiring
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[sbr] object SplitBrainResolverSettings {
+ final val KeepMajorityName = "keep-majority"
+ final val LeaseMajorityName = "lease-majority"
+ final val StaticQuorumName = "static-quorum"
+ final val KeepOldestName = "keep-oldest"
+ final val DownAllName = "down-all"
+
+ def allStrategyNames =
+ Set(KeepMajorityName, LeaseMajorityName, StaticQuorumName, KeepOldestName, DownAllName)
+}
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[sbr] final class SplitBrainResolverSettings(config: Config) {
+
+ import SplitBrainResolverSettings._
+
+ private val cc = config.getConfig("akka.cluster.split-brain-resolver")
+
+ val DowningStableAfter: FiniteDuration = {
+ val key = "stable-after"
+ FiniteDuration(cc.getDuration(key).toMillis, TimeUnit.MILLISECONDS).requiring(_ >= Duration.Zero, key + " >= 0s")
+ }
+
+ val DowningStrategy: String =
+ cc.getString("active-strategy").toLowerCase(Locale.ROOT) match {
+ case strategyName if allStrategyNames(strategyName) => strategyName
+ case unknown =>
+ throw new ConfigurationException(
+ s"Unknown downing strategy [$unknown]. Select one of [${allStrategyNames.mkString(",")}]")
+ }
+
+ val DownAllWhenUnstable: FiniteDuration = {
+ val key = "down-all-when-unstable"
+ Helpers.toRootLowerCase(cc.getString("down-all-when-unstable")) match {
+ case "on" =>
+ // based on stable-after
+ DowningStableAfter * 3 / 4
+ case "off" =>
+ // disabled
+ Duration.Zero
+ case _ =>
+ FiniteDuration(cc.getDuration(key).toMillis, TimeUnit.MILLISECONDS)
+ .requiring(_ > Duration.Zero, key + " > 0s, or 'off' to disable")
+ }
+ }
+
+ // the individual sub-configs below should only be called when the strategy has been selected
+
+ def keepMajorityRole: Option[String] = role(strategyConfig(KeepMajorityName))
+
+ def staticQuorumSettings: StaticQuorumSettings = {
+ val c = strategyConfig(StaticQuorumName)
+ val size = c
+ .getInt("quorum-size")
+ .requiring(_ >= 1, s"akka.cluster.split-brain-resolver.$StaticQuorumName.quorum-size must be >= 1")
+ StaticQuorumSettings(size, role(c))
+ }
+
+ def keepOldestSettings: KeepOldestSettings = {
+ val c = strategyConfig(KeepOldestName)
+ val downIfAlone = c.getBoolean("down-if-alone")
+ KeepOldestSettings(downIfAlone, role(c))
+ }
+
+ def leaseMajoritySettings: LeaseMajoritySettings = {
+ val c = strategyConfig(LeaseMajorityName)
+
+ val leaseImplementation = c.getString("lease-implementation")
+ require(
+ leaseImplementation != "",
+ s"akka.cluster.split-brain-resolver.$LeaseMajorityName.lease-implementation must be defined")
+
+ val acquireLeaseDelayForMinority =
+ FiniteDuration(c.getDuration("acquire-lease-delay-for-minority").toMillis, TimeUnit.MILLISECONDS)
+
+ LeaseMajoritySettings(leaseImplementation, acquireLeaseDelayForMinority, role(c))
+ }
+
+ private def strategyConfig(strategyName: String): Config = cc.getConfig(strategyName)
+
+ private def role(c: Config): Option[String] = c.getString("role") match {
+ case "" => None
+ case r => Some(r)
+ }
+
+}
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[sbr] final case class StaticQuorumSettings(size: Int, role: Option[String])
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[sbr] final case class KeepOldestSettings(downIfAlone: Boolean, role: Option[String])
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[sbr] final case class LeaseMajoritySettings(
+ leaseImplementation: String,
+ acquireLeaseDelayForMinority: FiniteDuration,
+ role: Option[String])
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/AttemptSysMsgRedeliverySpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/AttemptSysMsgRedeliverySpec.scala
index a843df4863..437d8f87b8 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/AttemptSysMsgRedeliverySpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/AttemptSysMsgRedeliverySpec.scala
@@ -5,15 +5,16 @@
package akka.cluster
import scala.concurrent.duration._
+
import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.ActorRef
import akka.actor.Identify
+import akka.actor.PoisonPill
import akka.actor.Props
+import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._
-import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
-import akka.actor.PoisonPill
object AttemptSysMsgRedeliveryMultiJvmSpec extends MultiNodeConfig {
@@ -51,7 +52,7 @@ class AttemptSysMsgRedeliverySpec
}
"redeliver system message after inactivity" taggedAs LongRunningTest in {
- system.actorOf(Props[Echo], "echo")
+ system.actorOf(Props[Echo](), "echo")
enterBarrier("echo-started")
system.actorSelection(node(first) / "user" / "echo") ! Identify(None)
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterAccrualFailureDetectorSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterAccrualFailureDetectorSpec.scala
index 78e2050315..018fa6a253 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterAccrualFailureDetectorSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterAccrualFailureDetectorSpec.scala
@@ -4,11 +4,13 @@
package akka.cluster
+import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
-import scala.concurrent.duration._
import akka.testkit._
object ClusterAccrualFailureDetectorMultiJvmSpec extends MultiNodeConfig {
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterDeathWatchSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterDeathWatchSpec.scala
index 4f9a5c85f3..6cd94eda67 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterDeathWatchSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterDeathWatchSpec.scala
@@ -10,16 +10,17 @@ import scala.concurrent.Await
import scala.concurrent.duration._
import scala.language.postfixOps
+import com.typesafe.config.ConfigFactory
+import org.scalatest.concurrent.ScalaFutures
+
import akka.actor._
import akka.cluster.MultiNodeClusterSpec.EndActor
import akka.remote.RemoteActorRef
import akka.remote.RemoteWatcher
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
-import akka.testkit.TestEvent._
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-import org.scalatest.concurrent.ScalaFutures
+import akka.testkit.TestEvent._
object ClusterDeathWatchMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
@@ -91,10 +92,10 @@ abstract class ClusterDeathWatchSpec
def receive = {
case ActorIdentity(`path2`, Some(ref)) =>
context.watch(ref)
- watchEstablished.countDown
+ watchEstablished.countDown()
case ActorIdentity(`path3`, Some(ref)) =>
context.watch(ref)
- watchEstablished.countDown
+ watchEstablished.countDown()
case Terminated(actor) => testActor ! actor.path
}
}).withDeploy(Deploy.local), name = "observer1")
@@ -242,7 +243,7 @@ abstract class ClusterDeathWatchSpec
enterBarrier("end-actor-created")
runOn(fourth) {
- val hello = system.actorOf(Props[Hello], "hello")
+ val hello = system.actorOf(Props[Hello](), "hello")
hello.isInstanceOf[RemoteActorRef] should ===(true)
hello.path.address should ===(address(first))
watch(hello)
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterWatcherNoClusterWatcheeSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterWatcherNoClusterWatcheeSpec.scala
index 3854c00b18..5112f2514e 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterWatcherNoClusterWatcheeSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterWatcherNoClusterWatcheeSpec.scala
@@ -6,6 +6,9 @@ package akka.cluster
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+import org.scalatest.concurrent.ScalaFutures
+
import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.ActorRef
@@ -20,8 +23,6 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit.ImplicitSender
import akka.testkit.TestProbe
-import com.typesafe.config.ConfigFactory
-import org.scalatest.concurrent.ScalaFutures
class ClusterWatcherNoClusterWatcheeConfig(val useUnsafe: Boolean, artery: Boolean) extends MultiNodeConfig {
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ConvergenceSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ConvergenceSpec.scala
index eb2752aa72..0f1e5ffd74 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/ConvergenceSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ConvergenceSpec.scala
@@ -4,14 +4,15 @@
package akka.cluster
-import language.postfixOps
+import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
+import akka.actor.Address
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import scala.concurrent.duration._
-import akka.actor.Address
final case class ConvergenceMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/DeterministicOldestWhenJoiningSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/DeterministicOldestWhenJoiningSpec.scala
index afd76479ee..e9de1c319c 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/DeterministicOldestWhenJoiningSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/DeterministicOldestWhenJoiningSpec.scala
@@ -4,16 +4,17 @@
package akka.cluster
-import scala.concurrent.duration._
import scala.collection.immutable
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
import akka.actor.Address
-import akka.cluster.ClusterEvent.MemberUp
import akka.cluster.ClusterEvent.CurrentClusterState
+import akka.cluster.ClusterEvent.MemberUp
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import com.typesafe.config.ConfigFactory
object DeterministicOldestWhenJoiningMultiJvmSpec extends MultiNodeConfig {
val seed1 = role("seed1")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/InitialHeartbeatSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/InitialHeartbeatSpec.scala
index 2df8db8413..cce8960f01 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/InitialHeartbeatSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/InitialHeartbeatSpec.scala
@@ -4,9 +4,11 @@
package akka.cluster
-import language.postfixOps
import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
import akka.cluster.ClusterEvent.CurrentClusterState
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/InitialMembersOfNewDcSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/InitialMembersOfNewDcSpec.scala
index d775c8d9f6..e90a112887 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/InitialMembersOfNewDcSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/InitialMembersOfNewDcSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster
-import akka.remote.testkit._
-import akka.testkit.ImplicitSender
+import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
-import scala.concurrent.duration._
+import akka.remote.testkit._
+import akka.testkit.ImplicitSender
object InitialMembersOfNewDcSpec extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(s"""
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinInProgressSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinInProgressSpec.scala
index 96a06edbe8..d39274b254 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinInProgressSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinInProgressSpec.scala
@@ -4,11 +4,13 @@
package akka.cluster
+import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import scala.concurrent.duration._
object JoinInProgressMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
@@ -48,7 +50,7 @@ abstract class JoinInProgressSpec extends MultiNodeSpec(JoinInProgressMultiJvmSp
runOn(first) {
val until = Deadline.now + 5.seconds
- while (!until.isOverdue) {
+ while (!until.isOverdue()) {
Thread.sleep(200)
cluster.failureDetector.isAvailable(second) should ===(true)
}
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinSeedNodeSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinSeedNodeSpec.scala
index 8465f7ce2f..9eae814c00 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinSeedNodeSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinSeedNodeSpec.scala
@@ -5,10 +5,11 @@
package akka.cluster
import scala.collection.immutable
+
+import akka.actor.Address
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import akka.actor.Address
object JoinSeedNodeMultiJvmSpec extends MultiNodeConfig {
val seed1 = role("seed1")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeMessageClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeMessageClusterSpec.scala
index 794a585a97..3f0eab9692 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeMessageClusterSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeMessageClusterSpec.scala
@@ -6,6 +6,8 @@ package akka.cluster
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ActorIdentity
import akka.actor.ActorRef
import akka.actor.ExtendedActorSystem
@@ -19,7 +21,6 @@ import akka.remote.testkit.MultiNodeSpec
import akka.serialization.SerializerWithStringManifest
import akka.testkit._
import akka.util.unused
-import com.typesafe.config.ConfigFactory
object LargeMessageClusterMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderDowningAllOtherNodesSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderDowningAllOtherNodesSpec.scala
index aa6d1966f1..0680999b58 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderDowningAllOtherNodesSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderDowningAllOtherNodesSpec.scala
@@ -5,10 +5,12 @@
package akka.cluster
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import com.typesafe.config.ConfigFactory
object LeaderDowningAllOtherNodesMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderDowningNodeThatIsUnreachableSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderDowningNodeThatIsUnreachableSpec.scala
index 16d611e587..6c0105a6fd 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderDowningNodeThatIsUnreachableSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderDowningNodeThatIsUnreachableSpec.scala
@@ -4,13 +4,14 @@
package akka.cluster
-import language.postfixOps
+import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import scala.concurrent.duration._
final case class LeaderDowningNodeThatIsUnreachableMultiNodeConfig(failureDetectorPuppet: Boolean)
extends MultiNodeConfig {
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderElectionSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderElectionSpec.scala
index 165f230cb3..628ba09a9e 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderElectionSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderElectionSpec.scala
@@ -4,11 +4,13 @@
package akka.cluster
+import scala.concurrent.duration._
+
import language.postfixOps
+
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import scala.concurrent.duration._
final case class LeaderElectionMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
val controller = role("controller")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderLeavingSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderLeavingSpec.scala
index 2ad79f3917..f4df524dbb 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderLeavingSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderLeavingSpec.scala
@@ -4,15 +4,17 @@
package akka.cluster
+import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+
+import akka.actor.Actor
+import akka.actor.Deploy
+import akka.actor.Props
+import akka.cluster.MemberStatus._
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import scala.concurrent.duration._
-import akka.actor.Props
-import akka.actor.Actor
-import akka.cluster.MemberStatus._
-import akka.actor.Deploy
object LeaderLeavingMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
@@ -33,8 +35,8 @@ class LeaderLeavingMultiJvmNode3 extends LeaderLeavingSpec
abstract class LeaderLeavingSpec extends MultiNodeSpec(LeaderLeavingMultiJvmSpec) with MultiNodeClusterSpec {
- import LeaderLeavingMultiJvmSpec._
import ClusterEvent._
+ import LeaderLeavingMultiJvmSpec._
"A LEADER that is LEAVING" must {
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MBeanSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MBeanSpec.scala
index 806cefe7cb..e41faa3f90 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MBeanSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MBeanSpec.scala
@@ -4,12 +4,15 @@
package akka.cluster
-import language.postfixOps
-import com.typesafe.config.ConfigFactory
-import scala.concurrent.duration._
import java.lang.management.ManagementFactory
+
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
import javax.management.InstanceNotFoundException
import javax.management.ObjectName
+import language.postfixOps
+
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MemberWeaklyUpSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MemberWeaklyUpSpec.scala
index b7ea0bebac..eb3b1e2079 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MemberWeaklyUpSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MemberWeaklyUpSpec.scala
@@ -4,14 +4,16 @@
package akka.cluster
-import scala.language.postfixOps
import scala.concurrent.duration._
+import scala.language.postfixOps
+
+import com.typesafe.config.ConfigFactory
+
+import akka.cluster.MemberStatus.WeaklyUp
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-import akka.cluster.MemberStatus.WeaklyUp
object MemberWeaklyUpSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerExitingSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerExitingSpec.scala
index 9d46505e87..3375143384 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerExitingSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerExitingSpec.scala
@@ -4,13 +4,13 @@
package akka.cluster
+import akka.actor.Actor
+import akka.actor.Deploy
+import akka.actor.Props
+import akka.cluster.MemberStatus._
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import akka.actor.Props
-import akka.actor.Actor
-import akka.cluster.MemberStatus._
-import akka.actor.Deploy
object MembershipChangeListenerExitingMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
@@ -28,8 +28,8 @@ abstract class MembershipChangeListenerExitingSpec
extends MultiNodeSpec(MembershipChangeListenerExitingMultiJvmSpec)
with MultiNodeClusterSpec {
- import MembershipChangeListenerExitingMultiJvmSpec._
import ClusterEvent._
+ import MembershipChangeListenerExitingMultiJvmSpec._
"A registered MembershipChangeListener" must {
"be notified when new node is EXITING" taggedAs LongRunningTest in {
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerUpSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerUpSpec.scala
index b201ed9239..b23118c831 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerUpSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerUpSpec.scala
@@ -4,12 +4,12 @@
package akka.cluster
+import akka.actor.Actor
+import akka.actor.Deploy
+import akka.actor.Props
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import akka.actor.Props
-import akka.actor.Actor
-import akka.actor.Deploy
object MembershipChangeListenerUpMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
@@ -27,8 +27,8 @@ abstract class MembershipChangeListenerUpSpec
extends MultiNodeSpec(MembershipChangeListenerUpMultiJvmSpec)
with MultiNodeClusterSpec {
- import MembershipChangeListenerUpMultiJvmSpec._
import ClusterEvent._
+ import MembershipChangeListenerUpMultiJvmSpec._
"A set of connected cluster systems" must {
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MinMembersBeforeUpSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MinMembersBeforeUpSpec.scala
index 29322ab5cd..414f0a4c63 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MinMembersBeforeUpSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MinMembersBeforeUpSpec.scala
@@ -5,11 +5,12 @@
package akka.cluster
import com.typesafe.config.ConfigFactory
+
+import akka.cluster.MemberStatus._
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import akka.cluster.MemberStatus._
import akka.util.ccompat._
@ccompatUsedUntil213
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcClusterSpec.scala
index 83e67a4534..decf649470 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcClusterSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcClusterSpec.scala
@@ -4,12 +4,13 @@
package akka.cluster
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.cluster.MemberStatus.Up
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
import akka.remote.transport.ThrottlerTransportAdapter.Direction
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.duration._
class MultiDcSpecConfig(crossDcConnections: Int = 5) extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcHeartbeatTakingOverSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcHeartbeatTakingOverSpec.scala
index 82b1858f2d..3d4b088cf1 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcHeartbeatTakingOverSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcHeartbeatTakingOverSpec.scala
@@ -4,16 +4,17 @@
package akka.cluster
+import scala.collection.immutable
+import scala.collection.immutable.SortedSet
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ActorSelection
import akka.annotation.InternalApi
import akka.remote.testconductor.RoleName
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-
-import scala.collection.immutable
-import scala.collection.immutable.SortedSet
-import scala.concurrent.duration._
import akka.util.ccompat._
@ccompatUsedUntil213
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcLastNodeSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcLastNodeSpec.scala
index eb7a267c36..a0a9cbafd3 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcLastNodeSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcLastNodeSpec.scala
@@ -4,10 +4,11 @@
package akka.cluster
-import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
+import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
-import scala.concurrent.duration._
+import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
object MultiDcLastNodeSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcSplitBrainSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcSplitBrainSpec.scala
index 8fb381aa0c..a73746d177 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcSplitBrainSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcSplitBrainSpec.scala
@@ -4,16 +4,17 @@
package akka.cluster
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ActorSystem
import akka.cluster.ClusterEvent._
import akka.remote.testconductor.RoleName
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit.TestProbe
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
object MultiDcSplitBrainMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcSunnyWeatherSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcSunnyWeatherSpec.scala
index b9fd3be8be..b3bb3ddbfb 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcSunnyWeatherSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiDcSunnyWeatherSpec.scala
@@ -4,15 +4,16 @@
package akka.cluster
+import scala.collection.immutable
+import scala.collection.immutable.SortedSet
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.annotation.InternalApi
import akka.remote.testconductor.RoleName
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-
-import scala.collection.immutable
-import scala.collection.immutable.SortedSet
-import scala.concurrent.duration._
object MultiDcSunnyWeatherMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala
index 1135472e80..22bf90d590 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala
@@ -6,6 +6,15 @@ package akka.cluster
import java.util.concurrent.ConcurrentHashMap
+import scala.collection.immutable
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.language.implicitConversions
+
+import com.typesafe.config.{ Config, ConfigFactory }
+import org.scalatest.{ Canceled, Outcome, Suite }
+import org.scalatest.exceptions.TestCanceledException
+
import akka.actor.{ Actor, ActorRef, ActorSystem, Address, Deploy, PoisonPill, Props, RootActorPath }
import akka.cluster.ClusterEvent.{ MemberEvent, MemberRemoved }
import akka.event.Logging.ErrorLevel
@@ -13,17 +22,9 @@ import akka.remote.DefaultFailureDetectorRegistry
import akka.remote.testconductor.RoleName
import akka.remote.testkit.{ MultiNodeSpec, STMultiNodeSpec }
import akka.serialization.jackson.CborSerializable
-import akka.testkit.TestEvent._
import akka.testkit._
+import akka.testkit.TestEvent._
import akka.util.ccompat._
-import com.typesafe.config.{ Config, ConfigFactory }
-import org.scalatest.exceptions.TestCanceledException
-import org.scalatest.{ Canceled, Outcome, Suite }
-
-import scala.collection.immutable
-import scala.concurrent.Await
-import scala.concurrent.duration._
-import scala.language.implicitConversions
@ccompatUsedUntil213
object MultiNodeClusterSpec {
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeChurnSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeChurnSpec.scala
index e92f2a7901..941e6a528e 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeChurnSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeChurnSpec.scala
@@ -4,16 +4,17 @@
package akka.cluster
+import scala.collection.immutable
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.event.Logging.Info
import akka.remote.RARP
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
import akka.testkit._
import akka.testkit.TestKit
-import com.typesafe.config.ConfigFactory
-
-import scala.collection.immutable
-import scala.concurrent.duration._
object NodeChurnMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeDowningAndBeingRemovedSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeDowningAndBeingRemovedSpec.scala
index ac3c6a8916..137c9330e3 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeDowningAndBeingRemovedSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeDowningAndBeingRemovedSpec.scala
@@ -4,11 +4,13 @@
package akka.cluster
+import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import scala.concurrent.duration._
object NodeDowningAndBeingRemovedMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingAndExitingAndBeingRemovedSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingAndExitingAndBeingRemovedSpec.scala
index 31e6c1878b..cb2b2419e0 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingAndExitingAndBeingRemovedSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingAndExitingAndBeingRemovedSpec.scala
@@ -4,10 +4,11 @@
package akka.cluster
+import scala.concurrent.duration._
+
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import scala.concurrent.duration._
object NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingAndExitingSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingAndExitingSpec.scala
index dae020bb15..4a6bc56f1e 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingAndExitingSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingAndExitingSpec.scala
@@ -4,13 +4,13 @@
package akka.cluster
+import akka.actor.Actor
+import akka.actor.Deploy
+import akka.actor.Props
+import akka.cluster.MemberStatus._
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import akka.actor.Props
-import akka.actor.Actor
-import akka.cluster.MemberStatus._
-import akka.actor.Deploy
object NodeLeavingAndExitingMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
@@ -28,8 +28,8 @@ abstract class NodeLeavingAndExitingSpec
extends MultiNodeSpec(NodeLeavingAndExitingMultiJvmSpec)
with MultiNodeClusterSpec {
- import NodeLeavingAndExitingMultiJvmSpec._
import ClusterEvent._
+ import NodeLeavingAndExitingMultiJvmSpec._
"A node that is LEAVING a non-singleton cluster" must {
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeUpSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeUpSpec.scala
index 125b3945bc..e7107cd4be 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeUpSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeUpSpec.scala
@@ -4,14 +4,16 @@
package akka.cluster
+import java.util.concurrent.atomic.AtomicReference
+
+import scala.collection.immutable.SortedSet
+import scala.concurrent.duration._
+
+import akka.actor.Actor
+import akka.actor.Props
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import scala.concurrent.duration._
-import scala.collection.immutable.SortedSet
-import java.util.concurrent.atomic.AtomicReference
-import akka.actor.Props
-import akka.actor.Actor
object NodeUpMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
@@ -25,8 +27,8 @@ class NodeUpMultiJvmNode2 extends NodeUpSpec
abstract class NodeUpSpec extends MultiNodeSpec(NodeUpMultiJvmSpec) with MultiNodeClusterSpec {
- import NodeUpMultiJvmSpec._
import ClusterEvent._
+ import NodeUpMultiJvmSpec._
"A cluster node that is joining another cluster" must {
"not be able to join a node that is not a cluster member" in {
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/QuickRestartSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/QuickRestartSpec.scala
index a7be1b4d48..21ce92f9fe 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/QuickRestartSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/QuickRestartSpec.scala
@@ -6,13 +6,14 @@ package akka.cluster
import java.util.concurrent.ThreadLocalRandom
+import scala.collection.immutable
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.{ ActorSystem, Address }
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-
-import scala.collection.immutable
-import scala.concurrent.duration._
// This test was a reproducer for issue #20639
object QuickRestartMultiJvmSpec extends MultiNodeConfig {
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/RemoteFeaturesWithClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/RemoteFeaturesWithClusterSpec.scala
index 907c0bfe3e..ba6e60b3ff 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/RemoteFeaturesWithClusterSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/RemoteFeaturesWithClusterSpec.scala
@@ -4,6 +4,9 @@
package akka.cluster
+import com.typesafe.config.ConfigFactory
+import org.scalatest.concurrent.ScalaFutures
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.AddressFromURIString
@@ -17,8 +20,6 @@ import akka.remote.RemoteWatcher.Heartbeat
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit.ImplicitSender
-import com.typesafe.config.ConfigFactory
-import org.scalatest.concurrent.ScalaFutures
class ClusterRemoteFeaturesConfig(artery: Boolean) extends MultiNodeConfig {
val first = role("first")
@@ -98,7 +99,7 @@ abstract class ClusterRemoteFeaturesSpec(multiNodeConfig: ClusterRemoteFeaturesC
enterBarrier("cluster-up")
runOn(first) {
- val actor = system.actorOf(Props[AddressPing], "kattdjur")
+ val actor = system.actorOf(Props[AddressPing](), "kattdjur")
actor.isInstanceOf[RemoteActorRef] shouldBe true
actor.path.address shouldEqual node(second).address
actor.path.address.hasGlobalScope shouldBe true
@@ -110,7 +111,7 @@ abstract class ClusterRemoteFeaturesSpec(multiNodeConfig: ClusterRemoteFeaturesC
enterBarrier("CARP-in-cluster-remote-validated")
def assertIsLocalRef(): Unit = {
- val actor = system.actorOf(Props[AddressPing], "kattdjur")
+ val actor = system.actorOf(Props[AddressPing](), "kattdjur")
actor.isInstanceOf[RepointableActorRef] shouldBe true
val localAddress = AddressFromURIString(s"akka://${system.name}")
actor.path.address shouldEqual localAddress
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartFirstSeedNodeSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartFirstSeedNodeSpec.scala
index 9d428e1f4c..06fb8875aa 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartFirstSeedNodeSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartFirstSeedNodeSpec.scala
@@ -4,21 +4,22 @@
package akka.cluster
-import language.postfixOps
import scala.collection.immutable
+import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
+import akka.actor.Actor
+import akka.actor.ActorSystem
+import akka.actor.Address
+import akka.actor.Deploy
+import akka.actor.Props
+import akka.actor.RootActorPath
+import akka.cluster.MemberStatus._
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-
-import scala.concurrent.duration._
-import akka.actor.Address
-import akka.actor.ActorSystem
-import akka.actor.Props
-import akka.actor.Actor
-import akka.actor.RootActorPath
-import akka.cluster.MemberStatus._
-import akka.actor.Deploy
import akka.util.ccompat._
@ccompatUsedUntil213
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode2Spec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode2Spec.scala
index 5163808f13..5b9277e340 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode2Spec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode2Spec.scala
@@ -7,6 +7,8 @@ package akka.cluster
import scala.collection.immutable
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorSystem
import akka.actor.Address
@@ -17,7 +19,6 @@ import akka.cluster.MemberStatus._
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import com.typesafe.config.ConfigFactory
import akka.util.ccompat._
@ccompatUsedUntil213
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode3Spec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode3Spec.scala
index db751e7039..8bc342cf0a 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode3Spec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode3Spec.scala
@@ -6,6 +6,9 @@ package akka.cluster
import scala.collection.immutable
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorSystem
import akka.actor.Address
@@ -17,7 +20,6 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._
-import com.typesafe.config.ConfigFactory
import akka.util.ccompat._
@ccompatUsedUntil213
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNodeSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNodeSpec.scala
index 2fb0a2dfd4..06f3e88fb2 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNodeSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNodeSpec.scala
@@ -7,6 +7,8 @@ package akka.cluster
import scala.collection.immutable
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.Done
import akka.actor.Actor
import akka.actor.ActorIdentity
@@ -22,7 +24,6 @@ import akka.cluster.MemberStatus._
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import com.typesafe.config.ConfigFactory
import akka.util.ccompat._
@ccompatUsedUntil213
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/SharedMediaDriverSupport.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/SharedMediaDriverSupport.scala
index 938f1a4275..dc7f760e70 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/SharedMediaDriverSupport.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SharedMediaDriverSupport.scala
@@ -10,17 +10,19 @@ import java.util.function.Consumer
import scala.annotation.tailrec
import scala.util.control.NonFatal
+
+import com.typesafe.config.ConfigFactory
+import io.aeron.CommonContext
+import io.aeron.driver.MediaDriver
+import io.aeron.driver.ThreadingMode
+import org.agrona.IoUtil
+
import akka.remote.RemoteSettings
import akka.remote.artery.ArterySettings
import akka.remote.artery.ArterySettings.AeronUpd
import akka.remote.artery.aeron.TaskRunner
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
-import com.typesafe.config.ConfigFactory
-import io.aeron.driver.MediaDriver
-import io.aeron.driver.ThreadingMode
-import org.agrona.IoUtil
-import io.aeron.CommonContext
object SharedMediaDriverSupport {
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/SingletonClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/SingletonClusterSpec.scala
index a2136f71a2..b1bf24b278 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/SingletonClusterSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SingletonClusterSpec.scala
@@ -4,13 +4,15 @@
package akka.cluster
-import akka.actor.Address
+import scala.collection.immutable
+import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+
+import akka.actor.Address
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import scala.concurrent.duration._
-import scala.collection.immutable
final case class SingletonClusterMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/SplitBrainSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/SplitBrainSpec.scala
index 84601b3fb0..cb80356555 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/SplitBrainSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SplitBrainSpec.scala
@@ -4,15 +4,16 @@
package akka.cluster
-import language.postfixOps
+import scala.concurrent.duration._
+import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
-import akka.testkit._
-import scala.concurrent.duration._
-import scala.concurrent.duration._
import akka.remote.transport.ThrottlerTransportAdapter.Direction
+import akka.testkit._
final case class SplitBrainMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/StreamRefSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/StreamRefSpec.scala
index ff35f1c2f7..551a34c63c 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/StreamRefSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/StreamRefSpec.scala
@@ -4,6 +4,13 @@
package akka.cluster
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.util.Failure
+import scala.util.Success
+
+import com.typesafe.config.ConfigFactory
+
import akka.Done
import akka.actor.{ Actor, ActorIdentity, ActorLogging, ActorRef, Identify, Props }
import akka.remote.testkit.MultiNodeConfig
@@ -21,12 +28,6 @@ import akka.stream.scaladsl.StreamRefs
import akka.stream.testkit.TestSubscriber
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.Future
-import scala.concurrent.duration._
-import scala.util.Failure
-import scala.util.Success
import akka.util.JavaDurationConverters._
object StreamRefSpec extends MultiNodeConfig {
@@ -56,7 +57,7 @@ object StreamRefSpec extends MultiNodeConfig {
class DataSource(streamLifecycleProbe: ActorRef) extends Actor with ActorLogging {
import context.dispatcher
- implicit val mat = Materializer(context)
+ implicit val mat: Materializer = Materializer(context)
def receive = {
case RequestLogs(streamId) =>
@@ -102,7 +103,7 @@ object StreamRefSpec extends MultiNodeConfig {
class DataReceiver(streamLifecycleProbe: ActorRef) extends Actor with ActorLogging {
import context.dispatcher
- implicit val mat = Materializer(context)
+ implicit val mat: Materializer = Materializer(context)
def receive = {
case PrepareUpload(nodeId) =>
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala
index f794ad6f50..14591ebf83 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala
@@ -4,21 +4,26 @@
package akka.cluster
-import language.postfixOps
+import java.lang.management.ManagementFactory
+import java.util.concurrent.ThreadLocalRandom
+
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.duration._
-import java.util.concurrent.ThreadLocalRandom
-import org.scalatest.BeforeAndAfterEach
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+import org.scalatest.BeforeAndAfterEach
+
import akka.actor.Actor
+import akka.actor.ActorIdentity
import akka.actor.ActorLogging
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.Address
import akka.actor.Deploy
+import akka.actor.Identify
import akka.actor.OneForOneStrategy
import akka.actor.Props
import akka.actor.RootActorPath
@@ -29,20 +34,16 @@ import akka.cluster.ClusterEvent.CurrentInternalStats
import akka.cluster.ClusterEvent.MemberEvent
import akka.remote.DefaultFailureDetectorRegistry
import akka.remote.PhiAccrualFailureDetector
+import akka.remote.RARP
import akka.remote.RemoteScope
+import akka.remote.artery.ArterySettings.AeronUpd
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.routing.FromConfig
import akka.testkit._
import akka.testkit.TestEvent._
-import akka.actor.Identify
-import akka.actor.ActorIdentity
import akka.util.Helpers.ConfigOps
import akka.util.Helpers.Requiring
-import java.lang.management.ManagementFactory
-
-import akka.remote.RARP
-import akka.remote.artery.ArterySettings.AeronUpd
/**
* This test is intended to be used as long running stress test
@@ -454,7 +455,7 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
* itself.
*/
class Master(settings: StressMultiJvmSpec.Settings, batchInterval: FiniteDuration, tree: Boolean) extends Actor {
- val workers = context.actorOf(FromConfig.props(Props[Worker]), "workers")
+ val workers = context.actorOf(FromConfig.props(Props[Worker]()), "workers")
val payload = Array.fill(settings.payloadSize)(ThreadLocalRandom.current.nextInt(127).toByte)
val retryTimeout = 5.seconds.dilated(context.system)
val idCounter = Iterator.from(0)
@@ -528,7 +529,7 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
def resend(): Unit = {
outstanding.values.foreach { jobState =>
- if (jobState.deadline.isOverdue)
+ if (jobState.deadline.isOverdue())
send(jobState.job)
}
}
@@ -785,7 +786,7 @@ abstract class StressSpec
// always create one worker when the cluster is started
lazy val createWorker: Unit =
- system.actorOf(Props[Worker], "worker")
+ system.actorOf(Props[Worker](), "worker")
def createResultAggregator(title: String, expectedResults: Int, includeInHistory: Boolean): Unit = {
runOn(roles.head) {
@@ -809,12 +810,12 @@ abstract class StressSpec
}
lazy val clusterResultHistory =
- if (settings.infolog) system.actorOf(Props[ClusterResultHistory], "resultHistory")
+ if (settings.infolog) system.actorOf(Props[ClusterResultHistory](), "resultHistory")
else system.deadLetters
- lazy val phiObserver = system.actorOf(Props[PhiObserver], "phiObserver")
+ lazy val phiObserver = system.actorOf(Props[PhiObserver](), "phiObserver")
- lazy val statsObserver = system.actorOf(Props[StatsObserver], "statsObserver")
+ lazy val statsObserver = system.actorOf(Props[StatsObserver](), "statsObserver")
def awaitClusterResult(): Unit = {
runOn(roles.head) {
@@ -892,7 +893,7 @@ abstract class StressSpec
val removeRole = roles(nbrUsedRoles - 1)
val removeAddress = address(removeRole)
runOn(removeRole) {
- system.actorOf(Props[Watchee], "watchee")
+ system.actorOf(Props[Watchee](), "watchee")
if (!shutdown) cluster.leave(myself)
}
enterBarrier("watchee-created-" + step)
@@ -1100,7 +1101,7 @@ abstract class StressSpec
def exerciseSupervision(title: String, duration: FiniteDuration, oneIteration: Duration): Unit =
within(duration + 10.seconds) {
val rounds = (duration.toMillis / oneIteration.toMillis).max(1).toInt
- val supervisor = system.actorOf(Props[Supervisor], "supervisor")
+ val supervisor = system.actorOf(Props[Supervisor](), "supervisor")
for (_ <- 0 until rounds) {
createResultAggregator(title, expectedResults = nbrUsedRoles, includeInHistory = false)
@@ -1108,7 +1109,7 @@ abstract class StressSpec
runOn(masterRoles: _*) {
reportResult {
roles.take(nbrUsedRoles).foreach { r =>
- supervisor ! Props[RemoteChild].withDeploy(Deploy(scope = RemoteScope(address(r))))
+ supervisor ! Props[RemoteChild]().withDeploy(Deploy(scope = RemoteScope(address(r))))
}
supervisor ! GetChildrenCount
expectMsgType[ChildrenCount] should ===(ChildrenCount(nbrUsedRoles, 0))
@@ -1161,7 +1162,7 @@ abstract class StressSpec
"log settings" taggedAs LongRunningTest in {
if (infolog) {
- log.info("StressSpec JVM:\n{}", jvmInfo)
+ log.info("StressSpec JVM:\n{}", jvmInfo())
runOn(roles.head) {
log.info("StressSpec settings:\n{}", settings)
}
@@ -1369,7 +1370,7 @@ abstract class StressSpec
"log jvm info" taggedAs LongRunningTest in {
if (infolog) {
- log.info("StressSpec JVM:\n{}", jvmInfo)
+ log.info("StressSpec JVM:\n{}", jvmInfo())
}
enterBarrier("after-" + step)
}
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala
index 13792c4643..5a68b17bab 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala
@@ -4,14 +4,17 @@
package akka.cluster
+import java.util.concurrent.atomic.AtomicReference
+
+import scala.collection.immutable.SortedSet
+
import com.typesafe.config.ConfigFactory
+
+import akka.actor.Actor
+import akka.actor.Props
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import java.util.concurrent.atomic.AtomicReference
-import scala.collection.immutable.SortedSet
-import akka.actor.Props
-import akka.actor.Actor
object SunnyWeatherMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
@@ -28,7 +31,11 @@ object SunnyWeatherMultiJvmSpec extends MultiNodeConfig {
loggers = ["akka.testkit.TestEventListener"]
loglevel = INFO
remote.log-remote-lifecycle-events = off
- cluster.failure-detector.monitored-by-nr-of-members = 3
+ cluster {
+ failure-detector.monitored-by-nr-of-members = 3
+ downing-provider-class = "akka.cluster.sbr.SplitBrainResolverProvider"
+ split-brain-resolver.active-strategy = keep-majority
+ }
}
"""))
@@ -42,9 +49,8 @@ class SunnyWeatherMultiJvmNode5 extends SunnyWeatherSpec
abstract class SunnyWeatherSpec extends MultiNodeSpec(SunnyWeatherMultiJvmSpec) with MultiNodeClusterSpec {
- import SunnyWeatherMultiJvmSpec._
-
import ClusterEvent._
+ import SunnyWeatherMultiJvmSpec._
"A normal cluster" must {
"be healthy" taggedAs LongRunningTest in {
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/SurviveNetworkInstabilitySpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/SurviveNetworkInstabilitySpec.scala
index 32e91397b9..ffd1b60ea5 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/SurviveNetworkInstabilitySpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SurviveNetworkInstabilitySpec.scala
@@ -7,6 +7,9 @@ package akka.cluster
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Address
@@ -21,8 +24,6 @@ import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.serialization.jackson.CborSerializable
import akka.testkit._
-import com.github.ghik.silencer.silent
-import com.typesafe.config.ConfigFactory
object SurviveNetworkInstabilityMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
@@ -121,11 +122,11 @@ abstract class SurviveNetworkInstabilitySpec
awaitAssert(clusterView.unreachableMembers.map(_.address) should ===(expected))
}
- system.actorOf(Props[Echo], "echo")
+ system.actorOf(Props[Echo](), "echo")
def assertCanTalk(alive: RoleName*): Unit = {
runOn(alive: _*) {
- awaitAllReachable
+ awaitAllReachable()
}
enterBarrier("reachable-ok")
@@ -284,7 +285,7 @@ abstract class SurviveNetworkInstabilitySpec
val others = Vector(first, third, fourth, fifth, sixth, seventh)
runOn(third) {
- system.actorOf(Props[Watcher], "watcher")
+ system.actorOf(Props[Watcher](), "watcher")
// undelivered system messages in RemoteChild on third should trigger QuarantinedEvent
system.eventStream.subscribe(testActor, quarantinedEventClass)
@@ -292,7 +293,7 @@ abstract class SurviveNetworkInstabilitySpec
enterBarrier("watcher-created")
runOn(second) {
- val refs = Vector.fill(sysMsgBufferSize + 1)(system.actorOf(Props[Echo])).toSet
+ val refs = Vector.fill(sysMsgBufferSize + 1)(system.actorOf(Props[Echo]())).toSet
system.actorSelection(node(third) / "user" / "watcher") ! Targets(refs)
expectMsg(TargetsRegistered)
}
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/TransitionSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/TransitionSpec.scala
index 18d2c4e8f5..ee1f98ef9c 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/TransitionSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/TransitionSpec.scala
@@ -4,16 +4,16 @@
package akka.cluster
+import InternalClusterAction._
+import MemberStatus._
+import com.typesafe.config.ConfigFactory
import language.implicitConversions
-import com.typesafe.config.ConfigFactory
+import akka.actor.Address
+import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import akka.actor.Address
-import akka.remote.testconductor.RoleName
-import MemberStatus._
-import InternalClusterAction._
object TransitionMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeJoinsAgainSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeJoinsAgainSpec.scala
index c3fd52208c..76a817e01e 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeJoinsAgainSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeJoinsAgainSpec.scala
@@ -4,21 +4,23 @@
package akka.cluster
-import language.postfixOps
import scala.collection.immutable
import scala.concurrent.Await
import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
import akka.actor.ActorSystem
import akka.actor.ExtendedActorSystem
+import akka.actor.Props
+import akka.cluster.MultiNodeClusterSpec.EndActor
+import akka.remote.RARP
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._
-import akka.actor.Props
-import akka.cluster.MultiNodeClusterSpec.EndActor
-import akka.remote.RARP
import akka.util.ccompat._
@ccompatUsedUntil213
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingGroupSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingGroupSpec.scala
index 794dea532f..e35e283851 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingGroupSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingGroupSpec.scala
@@ -4,16 +4,16 @@
package akka.cluster.routing
+import scala.concurrent.Await
+
import akka.actor.{ Actor, ActorRef, Props }
import akka.cluster.MultiNodeClusterSpec
import akka.pattern.ask
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
-import akka.routing.ConsistentHashingRouter.ConsistentHashMapping
import akka.routing.{ Broadcast, ConsistentHashingGroup, GetRoutees, Routees }
+import akka.routing.ConsistentHashingRouter.ConsistentHashMapping
import akka.testkit._
-import scala.concurrent.Await
-
object ClusterConsistentHashingGroupMultiJvmSpec extends MultiNodeConfig {
// using Java serialization because of `Any` in `Collected` (don't want to spend time on rewriting test)
@@ -52,7 +52,7 @@ abstract class ClusterConsistentHashingGroupSpec
"A cluster router with a consistent hashing group" must {
"start cluster with 3 nodes" taggedAs LongRunningTest in {
- system.actorOf(Props[Destination], "dest")
+ system.actorOf(Props[Destination](), "dest")
awaitClusterUp(first, second, third)
enterBarrier("after-1")
}
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingRouterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingRouterSpec.scala
index d2f97b0494..e29cfe5fb5 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingRouterSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingRouterSpec.scala
@@ -5,7 +5,9 @@
package akka.cluster.routing
import scala.concurrent.Await
+
import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Address
@@ -14,14 +16,14 @@ import akka.cluster.MultiNodeClusterSpec
import akka.pattern.ask
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
-import akka.routing.ConsistentHashingRouter.ConsistentHashMapping
-import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope
-import akka.routing.GetRoutees
-import akka.routing.FromConfig
-import akka.testkit._
import akka.routing.ActorRefRoutee
import akka.routing.ConsistentHashingPool
+import akka.routing.ConsistentHashingRouter.ConsistentHashMapping
+import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope
+import akka.routing.FromConfig
+import akka.routing.GetRoutees
import akka.routing.Routees
+import akka.testkit._
object ClusterConsistentHashingRouterMultiJvmSpec extends MultiNodeConfig {
@@ -65,7 +67,7 @@ abstract class ClusterConsistentHashingRouterSpec
with DefaultTimeout {
import ClusterConsistentHashingRouterMultiJvmSpec._
- lazy val router1 = system.actorOf(FromConfig.props(Props[Echo]), "router1")
+ lazy val router1 = system.actorOf(FromConfig.props(Props[Echo]()), "router1")
def currentRoutees(router: ActorRef) =
Await.result(router ? GetRoutees, timeout.duration).asInstanceOf[Routees].routees
@@ -125,7 +127,7 @@ abstract class ClusterConsistentHashingRouterSpec
ClusterRouterPool(
local = ConsistentHashingPool(nrOfInstances = 0),
settings = ClusterRouterPoolSettings(totalInstances = 10, maxInstancesPerNode = 2, allowLocalRoutees = true))
- .props(Props[Echo]),
+ .props(Props[Echo]()),
"router2")
// it may take some time until router receives cluster member events
awaitAssert { currentRoutees(router2).size should ===(6) }
@@ -144,7 +146,7 @@ abstract class ClusterConsistentHashingRouterSpec
val router3 =
system.actorOf(
- ConsistentHashingPool(nrOfInstances = 0, hashMapping = hashMapping).props(Props[Echo]),
+ ConsistentHashingPool(nrOfInstances = 0, hashMapping = hashMapping).props(Props[Echo]()),
"router3")
assertHashMapping(router3)
@@ -165,7 +167,7 @@ abstract class ClusterConsistentHashingRouterSpec
local = ConsistentHashingPool(nrOfInstances = 0, hashMapping = hashMapping),
settings =
ClusterRouterPoolSettings(totalInstances = 10, maxInstancesPerNode = 1, allowLocalRoutees = true))
- .props(Props[Echo]),
+ .props(Props[Echo]()),
"router4")
assertHashMapping(router4)
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterRoundRobinSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterRoundRobinSpec.scala
index 5002367a01..8bae8bd06c 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterRoundRobinSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterRoundRobinSpec.scala
@@ -4,10 +4,12 @@
package akka.cluster.routing
-import language.postfixOps
import scala.concurrent.Await
import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Address
@@ -17,15 +19,15 @@ import akka.cluster.MultiNodeClusterSpec
import akka.pattern.ask
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
-import akka.testkit._
import akka.remote.transport.ThrottlerTransportAdapter.Direction
-import akka.routing.FromConfig
-import akka.routing.RoundRobinPool
import akka.routing.ActorRefRoutee
import akka.routing.ActorSelectionRoutee
-import akka.routing.RoutedActorRef
+import akka.routing.FromConfig
import akka.routing.GetRoutees
+import akka.routing.RoundRobinPool
+import akka.routing.RoutedActorRef
import akka.routing.Routees
+import akka.testkit._
object ClusterRoundRobinMultiJvmSpec extends MultiNodeConfig {
@@ -109,16 +111,16 @@ abstract class ClusterRoundRobinSpec
with DefaultTimeout {
import ClusterRoundRobinMultiJvmSpec._
- lazy val router1 = system.actorOf(FromConfig.props(Props[SomeActor]), "router1")
+ lazy val router1 = system.actorOf(FromConfig.props(Props[SomeActor]()), "router1")
lazy val router2 = system.actorOf(
ClusterRouterPool(
RoundRobinPool(nrOfInstances = 0),
ClusterRouterPoolSettings(totalInstances = 3, maxInstancesPerNode = 1, allowLocalRoutees = true))
- .props(Props[SomeActor]),
+ .props(Props[SomeActor]()),
"router2")
- lazy val router3 = system.actorOf(FromConfig.props(Props[SomeActor]), "router3")
+ lazy val router3 = system.actorOf(FromConfig.props(Props[SomeActor]()), "router3")
lazy val router4 = system.actorOf(FromConfig.props(), "router4")
- lazy val router5 = system.actorOf(RoundRobinPool(nrOfInstances = 0).props(Props[SomeActor]), "router5")
+ lazy val router5 = system.actorOf(RoundRobinPool(nrOfInstances = 0).props(Props[SomeActor]()), "router5")
def receiveReplies(routeeType: RouteeType, expectedReplies: Int): Map[Address, Int] = {
val zero = Map.empty[Address, Int] ++ roles.map(address(_) -> 0)
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/UseRoleIgnoredSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/UseRoleIgnoredSpec.scala
index 199840fde3..85ea845277 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/UseRoleIgnoredSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/UseRoleIgnoredSpec.scala
@@ -4,6 +4,12 @@
package akka.cluster.routing
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.cluster.MultiNodeClusterSpec
import akka.pattern.ask
@@ -13,14 +19,9 @@ import akka.routing.GetRoutees
import akka.routing.RoundRobinGroup
import akka.routing.RoundRobinPool
import akka.routing.Routees
+import akka.testkit._
import akka.testkit.DefaultTimeout
import akka.testkit.ImplicitSender
-import akka.testkit._
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
-import scala.language.postfixOps
object UseRoleIgnoredMultiJvmSpec extends MultiNodeConfig {
@@ -110,7 +111,7 @@ abstract class UseRoleIgnoredSpec
totalInstances = 6,
maxInstancesPerNode = 2,
allowLocalRoutees = false,
- useRoles = roles)).props(Props[SomeActor]),
+ useRoles = roles)).props(Props[SomeActor]()),
"router-2")
awaitAssert(currentRoutees(router).size should ===(4))
@@ -143,7 +144,7 @@ abstract class UseRoleIgnoredSpec
totalInstances = 6,
routeesPaths = List("/user/foo", "/user/bar"),
allowLocalRoutees = false,
- useRoles = roles)).props,
+ useRoles = roles)).props(),
"router-2b")
awaitAssert(currentRoutees(router).size should ===(4))
@@ -176,7 +177,7 @@ abstract class UseRoleIgnoredSpec
totalInstances = 6,
maxInstancesPerNode = 2,
allowLocalRoutees = true,
- useRoles = roles)).props(Props[SomeActor]),
+ useRoles = roles)).props(Props[SomeActor]()),
"router-3")
awaitAssert(currentRoutees(router).size should ===(4))
@@ -209,7 +210,7 @@ abstract class UseRoleIgnoredSpec
totalInstances = 6,
routeesPaths = List("/user/foo", "/user/bar"),
allowLocalRoutees = true,
- useRoles = roles)).props,
+ useRoles = roles)).props(),
"router-3b")
awaitAssert(currentRoutees(router).size should ===(4))
@@ -242,7 +243,7 @@ abstract class UseRoleIgnoredSpec
totalInstances = 6,
maxInstancesPerNode = 2,
allowLocalRoutees = true,
- useRoles = roles)).props(Props[SomeActor]),
+ useRoles = roles)).props(Props[SomeActor]()),
"router-4")
awaitAssert(currentRoutees(router).size should ===(2))
@@ -275,7 +276,7 @@ abstract class UseRoleIgnoredSpec
totalInstances = 6,
routeesPaths = List("/user/foo", "/user/bar"),
allowLocalRoutees = true,
- useRoles = roles)).props,
+ useRoles = roles)).props(),
"router-4b")
awaitAssert(currentRoutees(router).size should ===(2))
@@ -308,7 +309,7 @@ abstract class UseRoleIgnoredSpec
totalInstances = 6,
maxInstancesPerNode = 2,
allowLocalRoutees = true,
- useRoles = roles)).props(Props[SomeActor]),
+ useRoles = roles)).props(Props[SomeActor]()),
"router-5")
awaitAssert(currentRoutees(router).size should ===(6))
@@ -341,7 +342,7 @@ abstract class UseRoleIgnoredSpec
totalInstances = 6,
routeesPaths = List("/user/foo", "/user/bar"),
allowLocalRoutees = true,
- useRoles = roles)).props,
+ useRoles = roles)).props(),
"router-5b")
awaitAssert(currentRoutees(router).size should ===(6))
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/sbr/DownAllIndirectlyConnected5NodeSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/sbr/DownAllIndirectlyConnected5NodeSpec.scala
new file mode 100644
index 0000000000..a41bf1a553
--- /dev/null
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/sbr/DownAllIndirectlyConnected5NodeSpec.scala
@@ -0,0 +1,128 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
+import akka.cluster.Cluster
+import akka.cluster.MemberStatus
+import akka.cluster.MultiNodeClusterSpec
+import akka.remote.testkit.MultiNodeConfig
+import akka.remote.testkit.MultiNodeSpec
+import akka.remote.transport.ThrottlerTransportAdapter
+
+object DownAllIndirectlyConnected5NodeSpec extends MultiNodeConfig {
+ val node1 = role("node1")
+ val node2 = role("node2")
+ val node3 = role("node3")
+ val node4 = role("node4")
+ val node5 = role("node5")
+
+ commonConfig(ConfigFactory.parseString("""
+ akka {
+ loglevel = INFO
+ cluster {
+ downing-provider-class = "akka.cluster.sbr.SplitBrainResolverProvider"
+ split-brain-resolver.active-strategy = keep-majority
+ split-brain-resolver.stable-after = 6s
+
+ run-coordinated-shutdown-when-down = off
+ }
+
+ actor.provider = cluster
+
+ test.filter-leeway = 10s
+ }
+ """))
+
+ testTransport(on = true)
+}
+
+class DownAllIndirectlyConnected5NodeSpecMultiJvmNode1 extends DownAllIndirectlyConnected5NodeSpec
+class DownAllIndirectlyConnected5NodeSpecMultiJvmNode2 extends DownAllIndirectlyConnected5NodeSpec
+class DownAllIndirectlyConnected5NodeSpecMultiJvmNode3 extends DownAllIndirectlyConnected5NodeSpec
+class DownAllIndirectlyConnected5NodeSpecMultiJvmNode4 extends DownAllIndirectlyConnected5NodeSpec
+class DownAllIndirectlyConnected5NodeSpecMultiJvmNode5 extends DownAllIndirectlyConnected5NodeSpec
+
+class DownAllIndirectlyConnected5NodeSpec
+ extends MultiNodeSpec(DownAllIndirectlyConnected5NodeSpec)
+ with MultiNodeClusterSpec {
+ import DownAllIndirectlyConnected5NodeSpec._
+
+ "A 5-node cluster with keep-one-indirectly-connected = off" should {
+ "down all when indirectly connected combined with clean partition" in {
+ val cluster = Cluster(system)
+
+ runOn(node1) {
+ cluster.join(cluster.selfAddress)
+ }
+ enterBarrier("node1 joined")
+ runOn(node2, node3, node4, node5) {
+ cluster.join(node(node1).address)
+ }
+ within(10.seconds) {
+ awaitAssert {
+ cluster.state.members.size should ===(5)
+ cluster.state.members.foreach {
+ _.status should ===(MemberStatus.Up)
+ }
+ }
+ }
+ enterBarrier("Cluster formed")
+
+ runOn(node1) {
+ for (x <- List(node1, node2, node3); y <- List(node4, node5)) {
+ testConductor.blackhole(x, y, ThrottlerTransportAdapter.Direction.Both).await
+ }
+ }
+ enterBarrier("blackholed-clean-partition")
+
+ runOn(node1) {
+ testConductor.blackhole(node2, node3, ThrottlerTransportAdapter.Direction.Both).await
+ }
+ enterBarrier("blackholed-indirectly-connected")
+
+ within(10.seconds) {
+ awaitAssert {
+ runOn(node1) {
+ cluster.state.unreachable.map(_.address) should ===(Set(node2, node3, node4, node5).map(node(_).address))
+ }
+ runOn(node2) {
+ cluster.state.unreachable.map(_.address) should ===(Set(node3, node4, node5).map(node(_).address))
+ }
+ runOn(node3) {
+ cluster.state.unreachable.map(_.address) should ===(Set(node2, node4, node5).map(node(_).address))
+ }
+ runOn(node4, node5) {
+ cluster.state.unreachable.map(_.address) should ===(Set(node1, node2, node3).map(node(_).address))
+ }
+ }
+ }
+ enterBarrier("unreachable")
+
+ runOn(node1) {
+ within(15.seconds) {
+ awaitAssert {
+ cluster.state.members.map(_.address) should ===(Set(node(node1).address))
+ cluster.state.members.foreach {
+ _.status should ===(MemberStatus.Up)
+ }
+ }
+ }
+ }
+
+ runOn(node2, node3, node4, node5) {
+ // downed
+ awaitCond(cluster.isTerminated, max = 15.seconds)
+ }
+
+ enterBarrier("done")
+ }
+
+ }
+
+}
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/sbr/DownAllUnstable5NodeSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/sbr/DownAllUnstable5NodeSpec.scala
new file mode 100644
index 0000000000..e9bb9c7d58
--- /dev/null
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/sbr/DownAllUnstable5NodeSpec.scala
@@ -0,0 +1,133 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
+import akka.cluster.Cluster
+import akka.cluster.MemberStatus
+import akka.cluster.MultiNodeClusterSpec
+import akka.remote.testkit.MultiNodeConfig
+import akka.remote.testkit.MultiNodeSpec
+import akka.remote.transport.ThrottlerTransportAdapter
+
+object DownAllUnstable5NodeSpec extends MultiNodeConfig {
+ val node1 = role("node1")
+ val node2 = role("node2")
+ val node3 = role("node3")
+ val node4 = role("node4")
+ val node5 = role("node5")
+
+ commonConfig(ConfigFactory.parseString("""
+ akka {
+ loglevel = INFO
+ cluster {
+ downing-provider-class = "akka.cluster.sbr.SplitBrainResolverProvider"
+ failure-detector.acceptable-heartbeat-pause = 3s
+ split-brain-resolver.active-strategy = keep-majority
+ split-brain-resolver.stable-after = 10s
+ split-brain-resolver.down-all-when-unstable = 7s
+
+ run-coordinated-shutdown-when-down = off
+ }
+
+ # quicker reconnect
+ remote.retry-gate-closed-for = 1s
+ remote.netty.tcp.connection-timeout = 3 s
+
+ actor.provider = cluster
+
+ test.filter-leeway = 10s
+ }
+ """))
+
+ testTransport(on = true)
+}
+
+class DownAllUnstable5NodeSpecMultiJvmNode1 extends DownAllUnstable5NodeSpec
+class DownAllUnstable5NodeSpecMultiJvmNode2 extends DownAllUnstable5NodeSpec
+class DownAllUnstable5NodeSpecMultiJvmNode3 extends DownAllUnstable5NodeSpec
+class DownAllUnstable5NodeSpecMultiJvmNode4 extends DownAllUnstable5NodeSpec
+class DownAllUnstable5NodeSpecMultiJvmNode5 extends DownAllUnstable5NodeSpec
+
+class DownAllUnstable5NodeSpec extends MultiNodeSpec(DownAllUnstable5NodeSpec) with MultiNodeClusterSpec {
+ import DownAllUnstable5NodeSpec._
+
+ "A 5-node cluster with down-all-when-unstable" should {
+ "down all when instability continues" in {
+ val cluster = Cluster(system)
+
+ runOn(node1) {
+ cluster.join(cluster.selfAddress)
+ }
+ enterBarrier("node1 joined")
+ runOn(node2, node3, node4, node5) {
+ cluster.join(node(node1).address)
+ }
+ within(10.seconds) {
+ awaitAssert {
+ cluster.state.members.size should ===(5)
+ cluster.state.members.foreach {
+ _.status should ===(MemberStatus.Up)
+ }
+ }
+ }
+ enterBarrier("Cluster formed")
+
+ // acceptable-heartbeat-pause = 3s
+ // stable-after = 10s
+ // down-all-when-unstable = 7s
+
+ runOn(node1) {
+ for (x <- List(node1, node2, node3); y <- List(node4, node5)) {
+ testConductor.blackhole(x, y, ThrottlerTransportAdapter.Direction.Both).await
+ }
+ }
+ enterBarrier("blackholed-clean-partition")
+
+ within(10.seconds) {
+ awaitAssert {
+ runOn(node1, node2, node3) {
+ cluster.state.unreachable.map(_.address) should ===(Set(node4, node5).map(node(_).address))
+ }
+ runOn(node4, node5) {
+ cluster.state.unreachable.map(_.address) should ===(Set(node1, node2, node3).map(node(_).address))
+ }
+ }
+ }
+ enterBarrier("unreachable-clean-partition")
+
+ // no decision yet
+ Thread.sleep(2000)
+ cluster.state.members.size should ===(5)
+ cluster.state.members.foreach {
+ _.status should ===(MemberStatus.Up)
+ }
+
+ runOn(node1) {
+ testConductor.blackhole(node2, node3, ThrottlerTransportAdapter.Direction.Both).await
+ }
+ enterBarrier("blackhole-2")
+ // then it takes about 5 seconds for failure detector to observe that
+ Thread.sleep(7000)
+
+ runOn(node1) {
+ testConductor.passThrough(node2, node3, ThrottlerTransportAdapter.Direction.Both).await
+ }
+ enterBarrier("passThrough-2")
+
+ // now it should have been unstable for more than 17 seconds
+
+ // all downed
+ awaitCond(cluster.isTerminated, max = 15.seconds)
+
+ enterBarrier("done")
+ }
+
+ }
+
+}
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/sbr/IndirectlyConnected3NodeSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/sbr/IndirectlyConnected3NodeSpec.scala
new file mode 100644
index 0000000000..eebbd67f45
--- /dev/null
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/sbr/IndirectlyConnected3NodeSpec.scala
@@ -0,0 +1,111 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
+import akka.cluster.Cluster
+import akka.cluster.MemberStatus
+import akka.cluster.MultiNodeClusterSpec
+import akka.remote.testkit.MultiNodeConfig
+import akka.remote.testkit.MultiNodeSpec
+import akka.remote.transport.ThrottlerTransportAdapter
+
+object IndirectlyConnected3NodeSpec extends MultiNodeConfig {
+ val node1 = role("node1")
+ val node2 = role("node2")
+ val node3 = role("node3")
+
+ commonConfig(ConfigFactory.parseString("""
+ akka {
+ loglevel = INFO
+ cluster {
+ downing-provider-class = "akka.cluster.sbr.SplitBrainResolverProvider"
+ split-brain-resolver.active-strategy = keep-majority
+ split-brain-resolver.stable-after = 6s
+
+ run-coordinated-shutdown-when-down = off
+ }
+
+ actor.provider = cluster
+
+ test.filter-leeway = 10s
+ }
+ """))
+
+ testTransport(on = true)
+}
+
+class IndirectlyConnected3NodeSpecMultiJvmNode1 extends IndirectlyConnected3NodeSpec
+class IndirectlyConnected3NodeSpecMultiJvmNode2 extends IndirectlyConnected3NodeSpec
+class IndirectlyConnected3NodeSpecMultiJvmNode3 extends IndirectlyConnected3NodeSpec
+
+class IndirectlyConnected3NodeSpec extends MultiNodeSpec(IndirectlyConnected3NodeSpec) with MultiNodeClusterSpec {
+ import IndirectlyConnected3NodeSpec._
+
+ "A 3-node cluster" should {
+ "avoid a split brain when two unreachable but can talk via third" in {
+ val cluster = Cluster(system)
+
+ runOn(node1) {
+ cluster.join(cluster.selfAddress)
+ }
+ enterBarrier("node1 joined")
+ runOn(node2, node3) {
+ cluster.join(node(node1).address)
+ }
+ within(10.seconds) {
+ awaitAssert {
+ cluster.state.members.size should ===(3)
+ cluster.state.members.foreach {
+ _.status should ===(MemberStatus.Up)
+ }
+ }
+ }
+ enterBarrier("Cluster formed")
+
+ runOn(node1) {
+ testConductor.blackhole(node2, node3, ThrottlerTransportAdapter.Direction.Both).await
+ }
+ enterBarrier("Blackholed")
+
+ within(10.seconds) {
+ awaitAssert {
+ runOn(node3) {
+ cluster.state.unreachable.map(_.address) should ===(Set(node(node2).address))
+ }
+ runOn(node2) {
+ cluster.state.unreachable.map(_.address) should ===(Set(node(node3).address))
+ }
+ runOn(node1) {
+ cluster.state.unreachable.map(_.address) should ===(Set(node(node3).address, node(node2).address))
+ }
+ }
+ }
+ enterBarrier("unreachable")
+
+ runOn(node1) {
+ within(15.seconds) {
+ awaitAssert {
+ cluster.state.members.map(_.address) should ===(Set(node(node1).address))
+ cluster.state.members.foreach {
+ _.status should ===(MemberStatus.Up)
+ }
+ }
+ }
+ }
+
+ runOn(node2, node3) {
+ // downed
+ awaitCond(cluster.isTerminated, max = 15.seconds)
+ }
+
+ enterBarrier("done")
+ }
+ }
+
+}
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/sbr/IndirectlyConnected5NodeSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/sbr/IndirectlyConnected5NodeSpec.scala
new file mode 100644
index 0000000000..97ff66862e
--- /dev/null
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/sbr/IndirectlyConnected5NodeSpec.scala
@@ -0,0 +1,125 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
+import akka.cluster.Cluster
+import akka.cluster.MemberStatus
+import akka.cluster.MultiNodeClusterSpec
+import akka.remote.testkit.MultiNodeConfig
+import akka.remote.testkit.MultiNodeSpec
+import akka.remote.transport.ThrottlerTransportAdapter
+
+object IndirectlyConnected5NodeSpec extends MultiNodeConfig {
+ val node1 = role("node1")
+ val node2 = role("node2")
+ val node3 = role("node3")
+ val node4 = role("node4")
+ val node5 = role("node5")
+
+ commonConfig(ConfigFactory.parseString("""
+ akka {
+ loglevel = INFO
+ cluster {
+ downing-provider-class = "akka.cluster.sbr.SplitBrainResolverProvider"
+ split-brain-resolver.active-strategy = keep-majority
+ split-brain-resolver.stable-after = 6s
+
+ run-coordinated-shutdown-when-down = off
+ }
+
+ actor.provider = cluster
+
+ test.filter-leeway = 10s
+ }
+ """))
+
+ testTransport(on = true)
+}
+
+class IndirectlyConnected5NodeSpecMultiJvmNode1 extends IndirectlyConnected5NodeSpec
+class IndirectlyConnected5NodeSpecMultiJvmNode2 extends IndirectlyConnected5NodeSpec
+class IndirectlyConnected5NodeSpecMultiJvmNode3 extends IndirectlyConnected5NodeSpec
+class IndirectlyConnected5NodeSpecMultiJvmNode4 extends IndirectlyConnected5NodeSpec
+class IndirectlyConnected5NodeSpecMultiJvmNode5 extends IndirectlyConnected5NodeSpec
+
+class IndirectlyConnected5NodeSpec extends MultiNodeSpec(IndirectlyConnected5NodeSpec) with MultiNodeClusterSpec {
+ import IndirectlyConnected5NodeSpec._
+
+ "A 5-node cluster" should {
+ "avoid a split brain when indirectly connected combined with clean partition" in {
+ val cluster = Cluster(system)
+
+ runOn(node1) {
+ cluster.join(cluster.selfAddress)
+ }
+ enterBarrier("node1 joined")
+ runOn(node2, node3, node4, node5) {
+ cluster.join(node(node1).address)
+ }
+ within(10.seconds) {
+ awaitAssert {
+ cluster.state.members.size should ===(5)
+ cluster.state.members.foreach {
+ _.status should ===(MemberStatus.Up)
+ }
+ }
+ }
+ enterBarrier("Cluster formed")
+
+ runOn(node1) {
+ for (x <- List(node1, node2, node3); y <- List(node4, node5)) {
+ testConductor.blackhole(x, y, ThrottlerTransportAdapter.Direction.Both).await
+ }
+ }
+ enterBarrier("blackholed-clean-partition")
+
+ runOn(node1) {
+ testConductor.blackhole(node2, node3, ThrottlerTransportAdapter.Direction.Both).await
+ }
+ enterBarrier("blackholed-indirectly-connected")
+
+ within(10.seconds) {
+ awaitAssert {
+ runOn(node1) {
+ cluster.state.unreachable.map(_.address) should ===(Set(node2, node3, node4, node5).map(node(_).address))
+ }
+ runOn(node2) {
+ cluster.state.unreachable.map(_.address) should ===(Set(node3, node4, node5).map(node(_).address))
+ }
+ runOn(node3) {
+ cluster.state.unreachable.map(_.address) should ===(Set(node2, node4, node5).map(node(_).address))
+ }
+ runOn(node4, node5) {
+ cluster.state.unreachable.map(_.address) should ===(Set(node1, node2, node3).map(node(_).address))
+ }
+ }
+ }
+ enterBarrier("unreachable")
+
+ runOn(node1) {
+ within(15.seconds) {
+ awaitAssert {
+ cluster.state.members.map(_.address) should ===(Set(node(node1).address))
+ cluster.state.members.foreach {
+ _.status should ===(MemberStatus.Up)
+ }
+ }
+ }
+ }
+
+ runOn(node2, node3, node4, node5) {
+ // downed
+ awaitCond(cluster.isTerminated, max = 15.seconds)
+ }
+
+ enterBarrier("done")
+ }
+ }
+
+}
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/sbr/LeaseMajority5NodeSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/sbr/LeaseMajority5NodeSpec.scala
new file mode 100644
index 0000000000..0e1e13a718
--- /dev/null
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/sbr/LeaseMajority5NodeSpec.scala
@@ -0,0 +1,189 @@
+/*
+ * Copyright (C) 2019-2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
+import akka.cluster.MemberStatus
+import akka.remote.testconductor.RoleName
+import akka.remote.testkit.MultiNodeConfig
+import akka.remote.testkit.MultiNodeSpec
+import akka.remote.transport.ThrottlerTransportAdapter
+import com.typesafe.config.ConfigFactory
+
+import akka.cluster.MultiNodeClusterSpec
+import akka.coordination.lease.TestLease
+import akka.coordination.lease.TestLeaseExt
+
+object LeaseMajority5NodeSpec extends MultiNodeConfig {
+ val node1 = role("node1")
+ val node2 = role("node2")
+ val node3 = role("node3")
+ val node4 = role("node4")
+ val node5 = role("node5")
+
+ commonConfig(ConfigFactory.parseString(s"""
+ akka {
+ loglevel = INFO
+ cluster {
+ downing-provider-class = "akka.cluster.sbr.SplitBrainResolverProvider"
+ split-brain-resolver {
+ active-strategy = lease-majority
+ stable-after = 6s
+ lease-majority {
+ lease-implementation = test-lease
+ acquire-lease-delay-for-minority = 1s
+ }
+ }
+
+ run-coordinated-shutdown-when-down = off
+ }
+
+ actor.provider = cluster
+
+ test.filter-leeway = 10s
+ }
+
+ test-lease {
+ lease-class = ${classOf[TestLease].getName}
+ heartbeat-interval = 1s
+ heartbeat-timeout = 120s
+ lease-operation-timeout = 3s
+ }
+ """))
+
+ testTransport(on = true)
+}
+
+class LeaseMajority5NodeSpecMultiJvmNode1 extends LeaseMajority5NodeSpec
+class LeaseMajority5NodeSpecMultiJvmNode2 extends LeaseMajority5NodeSpec
+class LeaseMajority5NodeSpecMultiJvmNode3 extends LeaseMajority5NodeSpec
+class LeaseMajority5NodeSpecMultiJvmNode4 extends LeaseMajority5NodeSpec
+class LeaseMajority5NodeSpecMultiJvmNode5 extends LeaseMajority5NodeSpec
+
+class LeaseMajority5NodeSpec extends MultiNodeSpec(LeaseMajority5NodeSpec) with MultiNodeClusterSpec {
+ import LeaseMajority5NodeSpec._
+
+ private val testLeaseName = "LeaseMajority5NodeSpec-akka-sbr"
+
+ def sortByAddress(roles: RoleName*): List[RoleName] = {
+
+ /**
+ * Sort the roles in the address order used by the cluster node ring.
+ */
+ implicit val clusterOrdering: Ordering[RoleName] = new Ordering[RoleName] {
+ import akka.cluster.Member.addressOrdering
+ def compare(x: RoleName, y: RoleName): Int = addressOrdering.compare(node(x).address, node(y).address)
+ }
+ roles.toList.sorted
+ }
+
+ def leader(roles: RoleName*): RoleName =
+ sortByAddress(roles: _*).head
+
+ "LeaseMajority in a 5-node cluster" should {
+ "setup cluster" in {
+ runOn(node1) {
+ cluster.join(cluster.selfAddress)
+ }
+ enterBarrier("node1 joined")
+ runOn(node2, node3, node4, node5) {
+ cluster.join(node(node1).address)
+ }
+ within(10.seconds) {
+ awaitAssert {
+ cluster.state.members.size should ===(5)
+ cluster.state.members.foreach {
+ _.status should ===(MemberStatus.Up)
+ }
+ }
+ }
+ enterBarrier("Cluster formed")
+ }
+
+ "keep the side that can acquire the lease" in {
+ val lease = TestLeaseExt(system).getTestLease(testLeaseName)
+ val leaseProbe = lease.probe
+
+ runOn(node1, node2, node3) {
+ lease.setNextAcquireResult(Future.successful(true))
+ }
+ runOn(node4, node5) {
+ lease.setNextAcquireResult(Future.successful(false))
+ }
+ enterBarrier("lease-in-place")
+ runOn(node1) {
+ for (x <- List(node1, node2, node3); y <- List(node4, node5)) {
+ testConductor.blackhole(x, y, ThrottlerTransportAdapter.Direction.Both).await
+ }
+ }
+ enterBarrier("blackholed-clean-partition")
+
+ runOn(node1, node2, node3) {
+ within(20.seconds) {
+ awaitAssert {
+ cluster.state.members.size should ===(3)
+ }
+ }
+ runOn(leader(node1, node2, node3)) {
+ leaseProbe.expectMsgType[TestLease.AcquireReq]
+ // after 2 * stable-after
+ leaseProbe.expectMsgType[TestLease.ReleaseReq](14.seconds)
+ }
+ }
+ runOn(node4, node5) {
+ within(20.seconds) {
+ awaitAssert {
+ cluster.isTerminated should ===(true)
+ }
+ runOn(leader(node4, node5)) {
+ leaseProbe.expectMsgType[TestLease.AcquireReq]
+ }
+ }
+ }
+ enterBarrier("downed-and-removed")
+ leaseProbe.expectNoMessage(1.second)
+
+ enterBarrier("done-1")
+ }
+ }
+
+ "keep the side that can acquire the lease, round 2" in {
+ val lease = TestLeaseExt(system).getTestLease(testLeaseName)
+
+ runOn(node1) {
+ lease.setNextAcquireResult(Future.successful(true))
+ }
+ runOn(node2, node3) {
+ lease.setNextAcquireResult(Future.successful(false))
+ }
+ enterBarrier("lease-in-place-2")
+ runOn(node1) {
+ for (x <- List(node1); y <- List(node2, node3)) {
+ testConductor.blackhole(x, y, ThrottlerTransportAdapter.Direction.Both).await
+ }
+ }
+ enterBarrier("blackholed-clean-partition-2")
+
+ runOn(node1) {
+ within(20.seconds) {
+ awaitAssert {
+ cluster.state.members.size should ===(1)
+ }
+ }
+ }
+ runOn(node2, node3) {
+ within(20.seconds) {
+ awaitAssert {
+ cluster.isTerminated should ===(true)
+ }
+ }
+ }
+
+ enterBarrier("done-2")
+ }
+
+}
diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala
index 3bb7ab571d..718ce9ab69 100644
--- a/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala
@@ -4,16 +4,17 @@
package akka.cluster
-import language.postfixOps
import scala.concurrent.duration._
-import com.typesafe.config.ConfigFactory
-import akka.testkit.AkkaSpec
-import akka.dispatch.Dispatchers
-import akka.remote.PhiAccrualFailureDetector
-import akka.util.Helpers.ConfigOps
-import akka.actor.Address
import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
+import akka.actor.Address
+import akka.dispatch.Dispatchers
+import akka.remote.PhiAccrualFailureDetector
+import akka.testkit.AkkaSpec
+import akka.util.Helpers.ConfigOps
@silent
class ClusterConfigSpec extends AkkaSpec {
diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterDeployerSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterDeployerSpec.scala
index 6e8966d5a5..99519e7e18 100644
--- a/akka-cluster/src/test/scala/akka/cluster/ClusterDeployerSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/ClusterDeployerSpec.scala
@@ -4,14 +4,15 @@
package akka.cluster
-import akka.testkit._
-import akka.actor._
-import akka.routing._
import com.typesafe.config._
-import akka.cluster.routing.ClusterRouterPool
+
+import akka.actor._
import akka.cluster.routing.ClusterRouterGroup
-import akka.cluster.routing.ClusterRouterPoolSettings
import akka.cluster.routing.ClusterRouterGroupSettings
+import akka.cluster.routing.ClusterRouterPool
+import akka.cluster.routing.ClusterRouterPoolSettings
+import akka.routing._
+import akka.testkit._
object ClusterDeployerSpec {
val deployerConf = ConfigFactory.parseString(
diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala
index 6a800c2e7d..098d29d161 100644
--- a/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala
@@ -5,19 +5,21 @@
package akka.cluster
import scala.collection.immutable.SortedSet
+
import org.scalatest.BeforeAndAfterEach
+
+import akka.actor.ActorRef
import akka.actor.Address
import akka.actor.PoisonPill
import akka.actor.Props
-import akka.cluster.MemberStatus._
-import akka.cluster.InternalClusterAction._
import akka.cluster.ClusterEvent._
+import akka.cluster.ClusterSettings.DefaultDataCenter
+import akka.cluster.InternalClusterAction._
+import akka.cluster.MemberStatus._
+import akka.remote.RARP
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
-import akka.actor.ActorRef
-import akka.remote.RARP
import akka.testkit.TestProbe
-import akka.cluster.ClusterSettings.DefaultDataCenter
object ClusterDomainEventPublisherSpec {
val config = """
@@ -101,7 +103,7 @@ class ClusterDomainEventPublisherSpec
system.eventStream.subscribe(memberSubscriber.ref, classOf[LeaderChanged])
system.eventStream.subscribe(memberSubscriber.ref, ClusterShuttingDown.getClass)
- publisher = system.actorOf(Props[ClusterDomainEventPublisher])
+ publisher = system.actorOf(Props[ClusterDomainEventPublisher]())
publisher ! PublishChanges(state0)
memberSubscriber.expectMsg(MemberUp(aUp))
memberSubscriber.expectMsg(LeaderChanged(Some(aUp.address)))
diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventSpec.scala
index f56516eca5..3b9d5522dd 100644
--- a/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventSpec.scala
@@ -4,16 +4,17 @@
package akka.cluster
-import akka.actor.Address
-
import scala.collection.immutable.SortedSet
+
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.Address
+
class ClusterDomainEventSpec extends AnyWordSpec with Matchers {
- import MemberStatus._
import ClusterEvent._
+ import MemberStatus._
val aRoles = Set("AA", "AB")
val aJoining = TestMember(Address("akka", "sys", "a", 2552), Joining, aRoles)
diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterHeartbeatSenderStateSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterHeartbeatSenderStateSpec.scala
index 8a0b6608c5..1d3103a2b6 100644
--- a/akka-cluster/src/test/scala/akka/cluster/ClusterHeartbeatSenderStateSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/ClusterHeartbeatSenderStateSpec.scala
@@ -4,13 +4,15 @@
package akka.cluster
-import akka.actor.Address
-import akka.remote.FailureDetector
-import akka.remote.DefaultFailureDetectorRegistry
import java.util.concurrent.ThreadLocalRandom
+
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.Address
+import akka.remote.DefaultFailureDetectorRegistry
+import akka.remote.FailureDetector
+
object ClusterHeartbeatSenderStateSpec {
class FailureDetectorStub extends FailureDetector {
diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterLogSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterLogSpec.scala
index 230b518e0d..219358af5e 100644
--- a/akka-cluster/src/test/scala/akka/cluster/ClusterLogSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/ClusterLogSpec.scala
@@ -4,9 +4,10 @@
package akka.cluster
+import com.typesafe.config.{ Config, ConfigFactory }
+
import akka.actor.{ Address, ExtendedActorSystem }
import akka.testkit.{ AkkaSpec, EventFilter, ImplicitSender }
-import com.typesafe.config.{ Config, ConfigFactory }
object ClusterLogSpec {
val config = """
diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala
index d4a90f3dd8..512befffae 100644
--- a/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala
@@ -5,14 +5,20 @@
package akka.cluster
import java.lang.management.ManagementFactory
+import javax.management.ObjectName
+
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
import akka.actor.ActorSystem
import akka.actor.Address
import akka.actor.CoordinatedShutdown
import akka.actor.ExtendedActorSystem
import akka.actor.Props
-import akka.cluster.ClusterEvent.MemberEvent
import akka.cluster.ClusterEvent._
+import akka.cluster.ClusterEvent.MemberEvent
import akka.cluster.InternalClusterAction._
import akka.stream.Materializer
import akka.stream.scaladsl.Sink
@@ -21,11 +27,6 @@ import akka.stream.scaladsl.StreamRefs
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import akka.testkit.TestProbe
-import com.typesafe.config.ConfigFactory
-import javax.management.ObjectName
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
object ClusterSpec {
val config = """
diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterTestKit.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterTestKit.scala
index dce65ffc76..afa5ac90d0 100644
--- a/akka-cluster/src/test/scala/akka/cluster/ClusterTestKit.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/ClusterTestKit.scala
@@ -4,13 +4,14 @@
package akka.cluster
+import scala.concurrent.duration.{ FiniteDuration, _ }
+import scala.util.Random
+
+import com.typesafe.config.{ Config, ConfigFactory }
+
import akka.actor.ActorSystem
import akka.cluster.MemberStatus.Removed
import akka.testkit.{ AkkaSpec, TestKitBase }
-import com.typesafe.config.{ Config, ConfigFactory }
-
-import scala.concurrent.duration.{ FiniteDuration, _ }
-import scala.util.Random
/**
* Builds on TestKitBase to provide some extra utilities to run cluster test.
diff --git a/akka-cluster/src/test/scala/akka/cluster/CrossDcHeartbeatSenderSpec.scala b/akka-cluster/src/test/scala/akka/cluster/CrossDcHeartbeatSenderSpec.scala
index 5124357aee..032ef93816 100644
--- a/akka-cluster/src/test/scala/akka/cluster/CrossDcHeartbeatSenderSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/CrossDcHeartbeatSenderSpec.scala
@@ -4,14 +4,14 @@
package akka.cluster
+import scala.collection.immutable.SortedSet
+
import akka.actor.{ ActorSelection, Address, Props }
import akka.cluster.ClusterEvent.CurrentClusterState
import akka.cluster.ClusterHeartbeatSender.Heartbeat
import akka.cluster.CrossDcHeartbeatSenderSpec.TestCrossDcHeartbeatSender
import akka.testkit.{ AkkaSpec, ImplicitSender, TestProbe }
-import scala.collection.immutable.SortedSet
-
object CrossDcHeartbeatSenderSpec {
class TestCrossDcHeartbeatSender(probe: TestProbe) extends CrossDcHeartbeatSender {
// disable register for cluster events
diff --git a/akka-cluster/src/test/scala/akka/cluster/DowningProviderSpec.scala b/akka-cluster/src/test/scala/akka/cluster/DowningProviderSpec.scala
index 0bca5b5be3..57ae2c55b2 100644
--- a/akka-cluster/src/test/scala/akka/cluster/DowningProviderSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/DowningProviderSpec.scala
@@ -8,15 +8,16 @@ import java.util.concurrent.atomic.AtomicBoolean
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.ConfigurationException
import akka.actor.ActorSystem
import akka.actor.Props
import akka.testkit.TestKit.awaitCond
import akka.testkit.TestKit.shutdownActorSystem
import akka.util.unused
-import com.typesafe.config.ConfigFactory
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
class FailingDowningProvider(@unused system: ActorSystem) extends DowningProvider {
override val downRemovalMargin: FiniteDuration = 20.seconds
diff --git a/akka-cluster/src/test/scala/akka/cluster/FailureDetectorPuppet.scala b/akka-cluster/src/test/scala/akka/cluster/FailureDetectorPuppet.scala
index fba68c2c9e..8667b84f29 100644
--- a/akka-cluster/src/test/scala/akka/cluster/FailureDetectorPuppet.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/FailureDetectorPuppet.scala
@@ -6,9 +6,10 @@ package akka.cluster
import java.util.concurrent.atomic.AtomicReference
-import akka.remote.FailureDetector
import com.typesafe.config.Config
+
import akka.event.EventStream
+import akka.remote.FailureDetector
import akka.util.unused
/**
diff --git a/akka-cluster/src/test/scala/akka/cluster/GossipSpec.scala b/akka-cluster/src/test/scala/akka/cluster/GossipSpec.scala
index 34159b3106..b955604086 100644
--- a/akka-cluster/src/test/scala/akka/cluster/GossipSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/GossipSpec.scala
@@ -4,14 +4,15 @@
package akka.cluster
-import akka.actor.Address
-import akka.cluster.Gossip.vclockName
-import akka.cluster.ClusterSettings.DefaultDataCenter
-
import scala.collection.immutable.SortedSet
+
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.Address
+import akka.cluster.ClusterSettings.DefaultDataCenter
+import akka.cluster.Gossip.vclockName
+
class GossipSpec extends AnyWordSpec with Matchers {
import MemberStatus._
diff --git a/akka-cluster/src/test/scala/akka/cluster/GossipTargetSelectorSpec.scala b/akka-cluster/src/test/scala/akka/cluster/GossipTargetSelectorSpec.scala
index 80c99d8462..c23b91797b 100644
--- a/akka-cluster/src/test/scala/akka/cluster/GossipTargetSelectorSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/GossipTargetSelectorSpec.scala
@@ -4,14 +4,15 @@
package akka.cluster
+import scala.collection.immutable.SortedSet
+
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.actor.Address
import akka.cluster.ClusterSettings.DataCenter
import akka.cluster.MemberStatus.Up
-import scala.collection.immutable.SortedSet
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
-
class GossipTargetSelectorSpec extends AnyWordSpec with Matchers {
val aDc1 = TestMember(Address("akka", "sys", "a", 2552), Up, Set.empty, dataCenter = "dc1")
diff --git a/akka-cluster/src/test/scala/akka/cluster/HeartbeatNodeRingPerfSpec.scala b/akka-cluster/src/test/scala/akka/cluster/HeartbeatNodeRingPerfSpec.scala
index e2d7249385..3b5539338b 100644
--- a/akka-cluster/src/test/scala/akka/cluster/HeartbeatNodeRingPerfSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/HeartbeatNodeRingPerfSpec.scala
@@ -4,10 +4,11 @@
package akka.cluster
-import akka.actor.Address
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.Address
+
class HeartbeatNodeRingPerfSpec extends AnyWordSpec with Matchers {
val nodesSize = sys.props.get("akka.cluster.HeartbeatNodeRingPerfSpec.nodesSize").getOrElse("250").toInt
diff --git a/akka-cluster/src/test/scala/akka/cluster/HeartbeatNodeRingSpec.scala b/akka-cluster/src/test/scala/akka/cluster/HeartbeatNodeRingSpec.scala
index e476b92b42..8ce7ffe82f 100644
--- a/akka-cluster/src/test/scala/akka/cluster/HeartbeatNodeRingSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/HeartbeatNodeRingSpec.scala
@@ -4,10 +4,11 @@
package akka.cluster
-import akka.actor.Address
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.Address
+
class HeartbeatNodeRingSpec extends AnyWordSpec with Matchers {
val aa = UniqueAddress(Address("akka", "sys", "aa", 2552), 1L)
diff --git a/akka-cluster/src/test/scala/akka/cluster/JoinConfigCompatCheckerRollingUpdateSpec.scala b/akka-cluster/src/test/scala/akka/cluster/JoinConfigCompatCheckerRollingUpdateSpec.scala
index 0c31127928..8e96959458 100644
--- a/akka-cluster/src/test/scala/akka/cluster/JoinConfigCompatCheckerRollingUpdateSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/JoinConfigCompatCheckerRollingUpdateSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster
-import scala.concurrent.duration._
import scala.collection.{ immutable => im }
+import scala.concurrent.duration._
+
+import com.typesafe.config.{ Config, ConfigFactory }
import akka.testkit.LongRunningTest
-import com.typesafe.config.{ Config, ConfigFactory }
object JoinConfigCompatCheckerRollingUpdateSpec {
diff --git a/akka-cluster/src/test/scala/akka/cluster/JoinConfigCompatCheckerSpec.scala b/akka-cluster/src/test/scala/akka/cluster/JoinConfigCompatCheckerSpec.scala
index 86cfc5359a..e9180db302 100644
--- a/akka-cluster/src/test/scala/akka/cluster/JoinConfigCompatCheckerSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/JoinConfigCompatCheckerSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster
-import akka.testkit.{ AkkaSpec, LongRunningTest }
+import scala.collection.{ immutable => im }
+import scala.concurrent.duration._
+
import com.typesafe.config.{ Config, ConfigFactory }
-import scala.concurrent.duration._
-import scala.collection.{ immutable => im }
+import akka.testkit.{ AkkaSpec, LongRunningTest }
object JoinConfigCompatCheckerSpec {
diff --git a/akka-cluster/src/test/scala/akka/cluster/JoinConfigCompatPreDefinedChecksSpec.scala b/akka-cluster/src/test/scala/akka/cluster/JoinConfigCompatPreDefinedChecksSpec.scala
index 8adf879554..9bfe9ed39e 100644
--- a/akka-cluster/src/test/scala/akka/cluster/JoinConfigCompatPreDefinedChecksSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/JoinConfigCompatPreDefinedChecksSpec.scala
@@ -4,9 +4,9 @@
package akka.cluster
-import com.typesafe.config.{ Config, ConfigFactory }
-
import scala.collection.{ immutable => im }
+
+import com.typesafe.config.{ Config, ConfigFactory }
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
diff --git a/akka-cluster/src/test/scala/akka/cluster/MemberOrderingSpec.scala b/akka-cluster/src/test/scala/akka/cluster/MemberOrderingSpec.scala
index 33bc9eab7c..86abcd3651 100644
--- a/akka-cluster/src/test/scala/akka/cluster/MemberOrderingSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/MemberOrderingSpec.scala
@@ -4,15 +4,17 @@
package akka.cluster
-import akka.actor.{ Address, AddressFromURIString }
import scala.collection.immutable.SortedSet
import scala.util.Random
+
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.{ Address, AddressFromURIString }
+
class MemberOrderingSpec extends AnyWordSpec with Matchers {
- import Member.ordering
import Member.addressOrdering
+ import Member.ordering
import MemberStatus._
def m(address: Address, status: MemberStatus): Member = TestMember(address, status)
diff --git a/akka-cluster/src/test/scala/akka/cluster/MembershipStateSpec.scala b/akka-cluster/src/test/scala/akka/cluster/MembershipStateSpec.scala
index ed817f162e..4a454efd21 100644
--- a/akka-cluster/src/test/scala/akka/cluster/MembershipStateSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/MembershipStateSpec.scala
@@ -4,13 +4,14 @@
package akka.cluster
-import akka.actor.Address
-import akka.cluster.MemberStatus.Up
-
import scala.collection.immutable.SortedSet
+
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.Address
+import akka.cluster.MemberStatus.Up
+
class MembershipStateSpec extends AnyWordSpec with Matchers {
// DC-a is in reverse age order
val a1 = TestMember(Address("akka", "sys", "a4", 2552), Up, 1, "dc-a")
diff --git a/akka-cluster/src/test/scala/akka/cluster/ReachabilityPerfSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ReachabilityPerfSpec.scala
index 171303c7a8..d534e34fb5 100644
--- a/akka-cluster/src/test/scala/akka/cluster/ReachabilityPerfSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/ReachabilityPerfSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster
-import akka.actor.Address
import com.github.ghik.silencer.silent
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.Address
+
class ReachabilityPerfSpec extends AnyWordSpec with Matchers {
val nodesSize = sys.props.get("akka.cluster.ReachabilityPerfSpec.nodesSize").getOrElse("250").toInt
diff --git a/akka-cluster/src/test/scala/akka/cluster/ReachabilitySpec.scala b/akka-cluster/src/test/scala/akka/cluster/ReachabilitySpec.scala
index 699183ba77..c0c18146fd 100644
--- a/akka-cluster/src/test/scala/akka/cluster/ReachabilitySpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/ReachabilitySpec.scala
@@ -4,10 +4,11 @@
package akka.cluster
-import akka.actor.Address
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.Address
+
class ReachabilitySpec extends AnyWordSpec with Matchers {
import Reachability.{ Reachable, Record, Terminated, Unreachable }
diff --git a/akka-cluster/src/test/scala/akka/cluster/StartupWithOneThreadSpec.scala b/akka-cluster/src/test/scala/akka/cluster/StartupWithOneThreadSpec.scala
index 46682e1e72..81c964d9c1 100644
--- a/akka-cluster/src/test/scala/akka/cluster/StartupWithOneThreadSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/StartupWithOneThreadSpec.scala
@@ -5,12 +5,13 @@
package akka.cluster
import scala.concurrent.duration._
-import akka.testkit.AkkaSpec
-import akka.testkit.ImplicitSender
-import akka.actor.Address
-import akka.actor.Props
+
import akka.actor.Actor
import akka.actor.ActorLogging
+import akka.actor.Address
+import akka.actor.Props
+import akka.testkit.AkkaSpec
+import akka.testkit.ImplicitSender
object StartupWithOneThreadSpec {
val config = """
diff --git a/akka-cluster/src/test/scala/akka/cluster/VectorClockPerfSpec.scala b/akka-cluster/src/test/scala/akka/cluster/VectorClockPerfSpec.scala
index 0c3eb6a330..854022a99d 100644
--- a/akka-cluster/src/test/scala/akka/cluster/VectorClockPerfSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/VectorClockPerfSpec.scala
@@ -5,6 +5,7 @@
package akka.cluster
import scala.collection.immutable.{ SortedSet, TreeMap }
+
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
diff --git a/akka-cluster/src/test/scala/akka/cluster/VectorClockSpec.scala b/akka-cluster/src/test/scala/akka/cluster/VectorClockSpec.scala
index a18b4e3375..efb8259de3 100644
--- a/akka-cluster/src/test/scala/akka/cluster/VectorClockSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/VectorClockSpec.scala
@@ -4,9 +4,10 @@
package akka.cluster
-import akka.testkit.AkkaSpec
import scala.collection.immutable.TreeMap
+import akka.testkit.AkkaSpec
+
class VectorClockSpec extends AkkaSpec {
import VectorClock._
diff --git a/akka-cluster/src/test/scala/akka/cluster/protobuf/ClusterMessageSerializerSpec.scala b/akka-cluster/src/test/scala/akka/cluster/protobuf/ClusterMessageSerializerSpec.scala
index 925d733abc..e727a47c0e 100644
--- a/akka-cluster/src/test/scala/akka/cluster/protobuf/ClusterMessageSerializerSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/protobuf/ClusterMessageSerializerSpec.scala
@@ -4,17 +4,16 @@
package akka.cluster.protobuf
+import collection.immutable.SortedSet
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
+import akka.actor.{ Address, ExtendedActorSystem }
import akka.cluster._
-import akka.actor.{ ActorSystem, Address, ExtendedActorSystem }
import akka.cluster.InternalClusterAction.CompatibleConfig
import akka.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings }
import akka.routing.RoundRobinPool
-import akka.cluster.protobuf.msg.{ ClusterMessages => cm }
-
-import collection.immutable.SortedSet
-import akka.testkit.{ AkkaSpec, TestKit }
-import com.github.ghik.silencer.silent
-import com.typesafe.config.ConfigFactory
+import akka.testkit.AkkaSpec
@silent
class ClusterMessageSerializerSpec extends AkkaSpec("akka.actor.provider = cluster") {
@@ -105,38 +104,28 @@ class ClusterMessageSerializerSpec extends AkkaSpec("akka.actor.provider = clust
}
// can be removed in 2.6.3 only checks deserialization with new not yet in effect manifests for 2.6.2
- "be serializable with new manifests for 2.6.3" in {
+ "be de-serializable with class manifests from 2.6.4 and earlier nodes" in {
val address = Address("akka", "system", "some.host.org", 4711)
val uniqueAddress = UniqueAddress(address, 17L)
val address2 = Address("akka", "system", "other.host.org", 4711)
val uniqueAddress2 = UniqueAddress(address2, 18L)
checkDeserializationWithManifest(
InternalClusterAction.Join(uniqueAddress, Set("foo", "bar", "dc-A")),
- ClusterMessageSerializer.JoinManifest)
+ ClusterMessageSerializer.OldJoinManifest)
checkDeserializationWithManifest(ClusterUserAction.Leave(address), ClusterMessageSerializer.LeaveManifest)
checkDeserializationWithManifest(ClusterUserAction.Down(address), ClusterMessageSerializer.DownManifest)
checkDeserializationWithManifest(
InternalClusterAction.InitJoin(ConfigFactory.empty),
- ClusterMessageSerializer.InitJoinManifest)
+ ClusterMessageSerializer.OldInitJoinManifest)
checkDeserializationWithManifest(
InternalClusterAction.InitJoinAck(address, CompatibleConfig(ConfigFactory.empty)),
- ClusterMessageSerializer.InitJoinAckManifest)
+ ClusterMessageSerializer.OldInitJoinAckManifest)
checkDeserializationWithManifest(
InternalClusterAction.InitJoinNack(address),
- ClusterMessageSerializer.InitJoinNackManifest)
- /* this has changed in 2.5.23 but it seems we forgot to add the next step in 2.5.24
- so we can't do two-way like this, the new manifest actually expects an address + timestamp + seqnr only when the new manifest is used
- see test below.
- checkDeserializationWithManifest(
- ClusterHeartbeatSender.Heartbeat(address, -1, -1),
- ClusterMessageSerializer.HeartbeatManifest)
- checkDeserializationWithManifest(
- ClusterHeartbeatSender.HeartbeatRsp(uniqueAddress, -1, -1),
- ClusterMessageSerializer.HeartbeatRspManifest)
- */
+ ClusterMessageSerializer.OldInitJoinNackManifest)
checkDeserializationWithManifest(
InternalClusterAction.ExitingConfirmed(uniqueAddress),
- ClusterMessageSerializer.ExitingConfirmedManifest)
+ ClusterMessageSerializer.OldExitingConfirmedManifest)
val node1 = VectorClock.Node("node1")
val node2 = VectorClock.Node("node2")
@@ -149,85 +138,15 @@ class ClusterMessageSerializerSpec extends AkkaSpec("akka.actor.provider = clust
.unreachable(b1.uniqueAddress, e1.uniqueAddress)
checkDeserializationWithManifest(
GossipEnvelope(a1.uniqueAddress, uniqueAddress2, g1),
- ClusterMessageSerializer.GossipEnvelopeManifest)
+ ClusterMessageSerializer.OldGossipEnvelopeManifest)
checkDeserializationWithManifest(
GossipStatus(a1.uniqueAddress, g1.version),
- ClusterMessageSerializer.GossipStatusManifest)
+ ClusterMessageSerializer.OldGossipStatusManifest)
checkDeserializationWithManifest(
InternalClusterAction.Welcome(uniqueAddress, g2),
- ClusterMessageSerializer.WelcomeManifest)
- }
-
- "be compatible with wire format of version 2.5.9 (using InitJoin singleton instead of class)" in {
- // we must use the old singleton class name so that the other side will see an InitJoin
- // but discard the config as it does not know about the config check
- val oldClassName = "akka.cluster.InternalClusterAction$InitJoin$"
- serializer.manifest(InternalClusterAction.InitJoin(ConfigFactory.empty())) should ===(oldClassName)
-
- // in 2.5.9 and earlier, it was an object and serialized to empty byte array
- // and we should accept that
- val deserialized = serializer.fromBinary(Array.emptyByteArray, oldClassName)
- deserialized shouldBe an[InternalClusterAction.InitJoin]
- }
-
- "deserialize from wire format of version 2.5.9 (using serialized address for InitJoinAck)" in {
- // we must use the old singleton class name so that the other side will see an InitJoin
- // but discard the config as it does not know about the config check
- val initJoinAck = InternalClusterAction.InitJoinAck(
- Address("akka", "cluster", "127.0.0.1", 2552),
- InternalClusterAction.UncheckedConfig)
- val serializedInitJoinAckPre2510 = serializer.addressToProto(initJoinAck.address).build().toByteArray
-
- val deserialized =
- serializer.fromBinary(serializedInitJoinAckPre2510, ClusterMessageSerializer.OldInitJoinAckManifest)
- deserialized shouldEqual initJoinAck
- }
-
- "serialize to wire format of version 2.5.9 (using serialized address for InitJoinAck)" in {
- val initJoinAck = InternalClusterAction.InitJoinAck(
- Address("akka", "cluster", "127.0.0.1", 2552),
- InternalClusterAction.ConfigCheckUnsupportedByJoiningNode)
- val bytes = serializer.toBinary(initJoinAck)
-
- val expectedSerializedInitJoinAckPre2510 = serializer.addressToProto(initJoinAck.address).build().toByteArray
- bytes.toList should ===(expectedSerializedInitJoinAckPre2510.toList)
- }
-
- "be compatible with wire format of version 2.5.3 (using use-role instead of use-roles)" in {
- val system = ActorSystem("ClusterMessageSerializer-old-wire-format")
-
- try {
- val serializer = new ClusterMessageSerializer(system.asInstanceOf[ExtendedActorSystem])
-
- // the oldSnapshot was created with the version of ClusterRouterPoolSettings in Akka 2.5.3. See issue #23257.
- // It was created with:
- /*
- import org.apache.commons.codec.binary.Hex.encodeHex
- val bytes = serializer.toBinary(
- ClusterRouterPool(RoundRobinPool(nrOfInstances = 4), ClusterRouterPoolSettings(123, 345, true, Some("role ABC"))))
- println(String.valueOf(encodeHex(bytes)))
- */
-
- val oldBytesHex = "0a0f08101205524f5252501a04080418001211087b10d90218012208726f6c6520414243"
-
- import org.apache.commons.codec.binary.Hex.decodeHex
- val oldBytes = decodeHex(oldBytesHex.toCharArray)
- val result = serializer.fromBinary(oldBytes, classOf[ClusterRouterPool])
-
- result match {
- case pool: ClusterRouterPool =>
- pool.settings.totalInstances should ===(123)
- pool.settings.maxInstancesPerNode should ===(345)
- pool.settings.allowLocalRoutees should ===(true)
- pool.settings.useRole should ===(Some("role ABC"))
- pool.settings.useRoles should ===(Set("role ABC"))
- }
- } finally {
- TestKit.shutdownActorSystem(system)
- }
-
+ ClusterMessageSerializer.OldWelcomeManifest)
}
"add a default data center role to gossip if none is present" in {
@@ -242,43 +161,20 @@ class ClusterMessageSerializerSpec extends AkkaSpec("akka.actor.provider = clust
}
}
+ // support for deserializing a new format with a string based manifest was added in 2.5.23 but the next step
+ // was never done, meaning that 2.6.4 still emits the old format
"Rolling upgrades for heart beat message changes in 2.5.23" must {
- // FIXME, add issue for serializing this as the new message type
- "serialize heart beats as Address to support versions prior or 2.5.23" in {
- serializer.manifest(ClusterHeartbeatSender.Heartbeat(a1.address, -1, -1)) should ===(
- ClusterMessageSerializer.HeartBeatManifestPre2523)
+ "deserialize heart beats represented by just an address Address to support versions prior or 2.6.5" in {
+ val serialized = serializer.addressToProto(a1.address).build().toByteArray
+ val deserialized = serializer.fromBinary(serialized, ClusterMessageSerializer.HeartBeatManifestPre2523)
+ deserialized should ===(ClusterHeartbeatSender.Heartbeat(a1.address, -1, -1))
}
- "serialize heart beat responses as UniqueAddress to support versions prior to 2.5.23" in {
- serializer.manifest(ClusterHeartbeatSender.HeartbeatRsp(a1.uniqueAddress, -1, -1)) should ===(
- ClusterMessageSerializer.HeartBeatRspManifest2523)
- }
-
- "be able to deserialize HeartBeat protobuf message" in {
- val hbProtobuf = cm.Heartbeat
- .newBuilder()
- .setFrom(serializer.addressToProto(a1.address))
- .setSequenceNr(1)
- .setCreationTime(2)
- .build()
- .toByteArray
-
- serializer.fromBinary(hbProtobuf, ClusterMessageSerializer.HeartbeatManifest) should ===(
- ClusterHeartbeatSender.Heartbeat(a1.address, 1, 2))
- }
-
- "be able to deserialize HeartBeatRsp probuf message" in {
- val hbrProtobuf = cm.HeartBeatResponse
- .newBuilder()
- .setFrom(serializer.uniqueAddressToProto(a1.uniqueAddress))
- .setSequenceNr(1)
- .setCreationTime(2)
- .build()
- .toByteArray
-
- serializer.fromBinary(hbrProtobuf, ClusterMessageSerializer.HeartbeatRspManifest) should ===(
- ClusterHeartbeatSender.HeartbeatRsp(a1.uniqueAddress, 1, 2))
+ "deserialize heart beat responses as UniqueAddress to support versions prior to 2.5.23" in {
+ val serialized = serializer.uniqueAddressToProto(a1.uniqueAddress).build().toByteArray
+ val deserialized = serializer.fromBinary(serialized, ClusterMessageSerializer.HeartBeatRspManifest2523)
+ deserialized should ===(ClusterHeartbeatSender.HeartbeatRsp(a1.uniqueAddress, -1, -1))
}
}
diff --git a/akka-cluster/src/test/scala/akka/cluster/routing/ClusterRouterSupervisorSpec.scala b/akka-cluster/src/test/scala/akka/cluster/routing/ClusterRouterSupervisorSpec.scala
index 99011070af..08f1fa7813 100644
--- a/akka-cluster/src/test/scala/akka/cluster/routing/ClusterRouterSupervisorSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/routing/ClusterRouterSupervisorSpec.scala
@@ -4,10 +4,10 @@
package akka.cluster.routing
-import akka.testkit._
import akka.actor._
-import akka.routing.RoundRobinPool
import akka.actor.OneForOneStrategy
+import akka.routing.RoundRobinPool
+import akka.testkit._
object ClusterRouterSupervisorSpec {
diff --git a/akka-cluster/src/test/scala/akka/cluster/sbr/SplitBrainResolverSpec.scala b/akka-cluster/src/test/scala/akka/cluster/sbr/SplitBrainResolverSpec.scala
new file mode 100644
index 0000000000..a60cde4113
--- /dev/null
+++ b/akka-cluster/src/test/scala/akka/cluster/sbr/SplitBrainResolverSpec.scala
@@ -0,0 +1,1639 @@
+/*
+ * Copyright (C) 2015-2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+import org.scalatest.concurrent.Eventually
+
+import akka.actor.ActorRef
+import akka.actor.Address
+import akka.actor.ExtendedActorSystem
+import akka.actor.Props
+import akka.cluster.ClusterEvent.LeaderChanged
+import akka.cluster.ClusterEvent.MemberRemoved
+import akka.cluster.ClusterEvent.MemberUp
+import akka.cluster.ClusterEvent.MemberWeaklyUp
+import akka.cluster.ClusterEvent.ReachabilityChanged
+import akka.cluster.ClusterEvent.ReachableDataCenter
+import akka.cluster.ClusterEvent.ReachableMember
+import akka.cluster.ClusterEvent.UnreachableDataCenter
+import akka.cluster.ClusterEvent.UnreachableMember
+import akka.cluster.ClusterSettings.DataCenter
+import akka.cluster.MemberStatus._
+import akka.cluster._
+import akka.coordination.lease.LeaseSettings
+import akka.coordination.lease.TestLease
+import akka.coordination.lease.TimeoutSettings
+import akka.testkit.AkkaSpec
+import akka.testkit.EventFilter
+
+object SplitBrainResolverSpec {
+
+ final case class DownCalled(address: Address)
+
+ object DowningTestActor {
+ def props(
+ stableAfter: FiniteDuration,
+ strategy: DowningStrategy,
+ probe: ActorRef,
+ selfUniqueAddress: UniqueAddress,
+ selfDc: DataCenter,
+ downAllWhenUnstable: FiniteDuration,
+ tickInterval: FiniteDuration): Props =
+ Props(
+ new DowningTestActor(
+ stableAfter,
+ strategy,
+ probe,
+ selfUniqueAddress,
+ selfDc,
+ downAllWhenUnstable,
+ tickInterval))
+ }
+
+ class DowningTestActor(
+ stableAfter: FiniteDuration,
+ strategy: DowningStrategy,
+ probe: ActorRef,
+ override val selfUniqueAddress: UniqueAddress,
+ override val selfDc: DataCenter,
+ override val downAllWhenUnstable: FiniteDuration,
+ val tick: FiniteDuration)
+ extends SplitBrainResolverBase(stableAfter, strategy) {
+
+ // manual ticks used in this test
+ override def tickInterval: FiniteDuration =
+ if (tick == Duration.Zero) super.tickInterval else tick
+
+ // immediate overdue if Duration.Zero is used
+ override def newStableDeadline(): Deadline = super.newStableDeadline() - 1.nanos
+
+ var downed = Set.empty[Address]
+
+ override def down(node: Address): Unit = {
+ if (leader && !downed(node)) {
+ downed += node
+ probe ! DownCalled(node)
+ } else if (!leader)
+ probe ! "down must only be done by leader"
+ }
+ }
+}
+
+class SplitBrainResolverSpec
+ extends AkkaSpec("""
+ |akka {
+ | actor.provider = cluster
+ | cluster.downing-provider-class = "akka.cluster.sbr.SplitBrainResolverProvider"
+ | cluster.split-brain-resolver.active-strategy=keep-majority
+ | remote {
+ | netty.tcp {
+ | hostname = "127.0.0.1"
+ | port = 0
+ | }
+ | }
+ |}
+ """.stripMargin)
+ with Eventually {
+
+ import DowningStrategy._
+ import SplitBrainResolverSpec._
+ import TestAddresses._
+
+ private val selfDc = TestAddresses.defaultDataCenter
+
+ private val testLeaseSettings =
+ new LeaseSettings("akka-sbr", "test", new TimeoutSettings(1.second, 2.minutes, 3.seconds), ConfigFactory.empty)
+
+ def createReachability(unreachability: Seq[(Member, Member)]): Reachability = {
+ Reachability(unreachability.map {
+ case (from, to) => Reachability.Record(from.uniqueAddress, to.uniqueAddress, Reachability.Unreachable, 1)
+ }.toIndexedSeq, unreachability.map {
+ case (from, _) => from.uniqueAddress -> 1L
+ }.toMap)
+ }
+
+ def extSystem: ExtendedActorSystem = system.asInstanceOf[ExtendedActorSystem]
+
+ abstract class StrategySetup {
+ def createStrategy(): DowningStrategy
+
+ var side1: Set[Member] = Set.empty
+ var side2: Set[Member] = Set.empty
+ var side3: Set[Member] = Set.empty
+
+ def side1Nodes: Set[UniqueAddress] = side1.map(_.uniqueAddress)
+ def side2Nodes: Set[UniqueAddress] = side2.map(_.uniqueAddress)
+ def side3Nodes: Set[UniqueAddress] = side3.map(_.uniqueAddress)
+
+ var indirectlyConnected: Seq[(Member, Member)] = Nil
+
+ private def initStrategy(): DowningStrategy = {
+ val strategy = createStrategy()
+ (side1 ++ side2 ++ side3).foreach(strategy.add)
+ strategy
+ }
+
+ def assertDowning(members: Set[Member]): Unit = {
+ assertDowningSide(side1, members)
+ assertDowningSide(side2, members)
+ assertDowningSide(side3, members)
+ }
+
+ def assertDowningSide(side: Set[Member], members: Set[Member]): Unit = {
+ if (side.nonEmpty)
+ strategy(side).nodesToDown() should be(members.map(_.uniqueAddress))
+ }
+
+ def strategy(side: Set[Member]): DowningStrategy = {
+ val others = side1 ++ side2 ++ side3 -- side
+ (side -- others) should be(side)
+
+ if (side.nonEmpty) {
+ val strategy = initStrategy()
+ val unreachability = (indirectlyConnected ++ others.map(o => side.head -> o)).toSet.toList
+ val r = createReachability(unreachability)
+ strategy.setReachability(r)
+
+ unreachability.foreach { case (_, to) => strategy.addUnreachable(to) }
+
+ strategy.setSeenBy(side.map(_.address))
+
+ strategy
+ } else
+ createStrategy()
+ }
+
+ }
+
+ "StaticQuorum" must {
+ class Setup2(size: Int, role: Option[String]) extends StrategySetup {
+ override def createStrategy() =
+ new StaticQuorum(selfDc, size, role)
+ }
+
+ "down unreachable when enough reachable nodes" in new Setup2(3, None) {
+ side1 = Set(memberA, memberC, memberE)
+ side2 = Set(memberB, memberD)
+ assertDowning(side2)
+ }
+
+ "down reachable when not enough reachable nodes" in {
+ val setup = new Setup2(size = 3, None) {
+ side1 = Set(memberA, memberB)
+ side2 = Set(memberC, memberD)
+ }
+ import setup._
+ strategy(side1).decide() should ===(DownReachable)
+ strategy(side2).decide() should ===(DownReachable)
+ }
+
+ "down unreachable when enough reachable nodes with role" in new Setup2(2, Some("role3")) {
+ side1 = Set(memberA, memberB, memberC)
+ side2 = Set(memberD, memberE)
+ assertDowning(side2)
+ }
+
+ "down all if N > static-quorum.size * 2 - 1" in new Setup2(3, None) {
+ side1 = Set(memberA, memberB, memberC)
+ side2 = Set(memberD, memberE, memberF)
+ assertDowning(side1.union(side2))
+ }
+
+ "handle joining" in {
+ val setup = new Setup2(size = 3, None) {
+ side1 = Set(memberA, memberB, joining(memberC))
+ side2 = Set(memberD, memberE, joining(memberF))
+ }
+ import setup._
+ // Joining not counted
+ strategy(side1).decide() should ===(DownReachable)
+ strategy(side2).decide() should ===(DownReachable)
+
+ // if C becomes Up
+ side1 = Set(memberA, memberB, memberC)
+ strategy(side1).decide() should ===(DownUnreachable)
+ strategy(side2).decide() should ===(DownReachable)
+
+ // if F becomes Up, C still Joining
+ side1 = Set(memberA, memberB, joining(memberC))
+ side2 = Set(memberD, memberE, memberF)
+ strategy(side1).decide() should ===(DownReachable)
+ strategy(side2).decide() should ===(DownUnreachable)
+
+ // if both C and F become Up, too many
+ side1 = Set(memberA, memberB, memberC)
+ side2 = Set(memberD, memberE, memberF)
+ strategy(side1).decide() should ===(DownAll)
+ strategy(side2).decide() should ===(DownAll)
+ }
+
+ "handle leaving/exiting" in {
+ val setup = new Setup2(size = 3, None) {
+ side1 = Set(memberA, memberB, leaving(memberC))
+ side2 = Set(memberD, memberE)
+ }
+ import setup._
+ strategy(side1).decide() should ===(DownUnreachable)
+ strategy(side2).decide() should ===(DownReachable)
+
+ side1 = Set(memberA, memberB, exiting(memberC))
+ strategy(side1).decide() should ===(DownReachable)
+ strategy(side2).decide() should ===(DownReachable)
+ }
+ }
+
+ "KeepMajority" must {
+ class Setup2(role: Option[String]) extends StrategySetup {
+ override def createStrategy() =
+ new KeepMajority(selfDc, role)
+ }
+
+ "down minority partition: {A, C, E} | {B, D} => {A, C, E}" in new Setup2(role = None) {
+ side1 = Set(memberA, memberC, memberE)
+ side2 = Set(memberB, memberD)
+ assertDowning(side2)
+ }
+
+ "down minority partition: {A, B} | {C, D, E} => {C, D, E}" in new Setup2(role = None) {
+ side1 = Set(memberA, memberB)
+ side2 = Set(memberC, memberD, memberE)
+ assertDowning(side1)
+ }
+
+ "down self when alone: {B} | {A, C} => {A, C}" in new Setup2(role = None) {
+ side1 = Set(memberB)
+ side2 = Set(memberA, memberC)
+ assertDowning(side1)
+ }
+
+ "keep half with lowest address when equal size partition: {A, B} | {C, D} => {A, B}" in new Setup2(role = None) {
+ side1 = Set(memberA, memberB)
+ side2 = Set(memberC, memberD)
+ assertDowning(side2)
+ }
+
+ "keep node with lowest address in two node cluster: {A} | {B} => {A}" in new Setup2(role = None) {
+ side1 = Set(memberA)
+ side2 = Set(memberB)
+ assertDowning(side2)
+ }
+
+ "down minority partition with role: {A*, B*} | {C, D*, E} => {A*, B*}" in new Setup2(role = Some("role3")) {
+ side1 = Set(memberA, memberB)
+ side2 = Set(memberC, memberD, memberE)
+ assertDowning(side2)
+ }
+
+ "keep half with lowest address with role when equal size partition: {A, D*, E} | {B, C*} => {B, C*}" in
+ new Setup2(role = Some("role2")) {
+ side1 = Set(memberA, memberD, memberE)
+ side2 = Set(memberB, memberC)
+ // memberC is lowest with role2
+ assertDowning(side1)
+ }
+
+ "down all when no node with role: {C} | {E} => {}" in new Setup2(role = Some("role3")) {
+ side1 = Set(memberC)
+ side2 = Set(memberE)
+ assertDowning(side1 ++ side2)
+ }
+
+ "not count joining node, but down it: {B, D} | {Aj, C} => {B, D}" in new Setup2(role = None) {
+ side1 = Set(memberB, memberD)
+ side2 = Set(joining(memberA), memberC)
+ assertDowning(side2)
+ }
+
+ "down minority partition and joining node: {A, Bj} | {C, D, E} => {C, D, E}" in new Setup2(role = None) {
+ side1 = Set(memberA, joining(memberB))
+ side2 = Set(memberC, memberD, memberE)
+ assertDowning(side1)
+ }
+
+ "down each part when split in 3 too small parts: {A, B} | {C, D} | {E} => {}" in new Setup2(role = None) {
+ side1 = Set(memberA, memberB)
+ side2 = Set(memberC, memberD)
+ side3 = Set(memberE)
+ assertDowningSide(side1, side1)
+ assertDowningSide(side2, side2)
+ assertDowningSide(side3, side3)
+ }
+
+ "detect edge case of membership change: {A, B, F', G'} | {C, D, E} => {A, B, F, G}" in {
+ val setup = new Setup2(role = None) {
+ side1 = Set(memberA, memberB, memberF, memberG)
+ side2 = Set(memberC, memberD, memberE)
+ }
+ import setup._
+ val strategy1 = strategy(side1)
+ val decision1 = strategy1.decide()
+ decision1 should ===(DownUnreachable)
+ strategy1.nodesToDown(decision1) should ===(side2Nodes)
+
+ // F and G were moved to Up by side1 at the same time as the partition, and that has not been seen by
+ // side2 so they are still joining
+ side1 = Set(memberA, memberB, joining(memberF), joining(memberG))
+ val strategy2 = strategy(side2)
+ val decision2 = strategy2.decide()
+ decision2 should ===(DownAll)
+ strategy2.nodesToDown(decision2) should ===(side1Nodes.union(side2Nodes))
+ }
+
+ "detect edge case of membership change when equal size: {A, B, F'} | {C, D, E} => {A, B, F}" in {
+ val setup = new Setup2(role = None) {
+ side1 = Set(memberA, memberB, memberF)
+ side2 = Set(memberC, memberD, memberE)
+ }
+ import setup._
+ val strategy1 = strategy(side1)
+ val decision1 = strategy1.decide()
+ // memberA is lowest address
+ decision1 should ===(DownUnreachable)
+ strategy1.nodesToDown(decision1) should ===(side2Nodes)
+
+ // F was moved to Up by side1 at the same time as the partition, and that has not been seen by
+ // side2 so it is still joining
+ side1 = Set(memberA, memberB, joining(memberF))
+ val strategy2 = strategy(side2)
+ val decision2 = strategy2.decide()
+ // when counting the joining F it becomes equal size
+ decision2 should ===(DownAll)
+ strategy2.nodesToDown(decision2) should ===(side1Nodes.union(side2Nodes))
+ }
+
+ "detect safe edge case of membership change: {A, B} | {C, D, E, F'} => {C, D, E, F}" in {
+ val setup = new Setup2(role = None) {
+ side1 = Set(memberA, memberB)
+ side2 = Set(memberC, memberD, memberE, joining(memberF))
+ }
+ import setup._
+ val strategy1 = strategy(side1)
+ val decision1 = strategy1.decide()
+ decision1 should ===(DownReachable)
+ strategy1.nodesToDown(decision1) should ===(side1Nodes)
+
+ // F was moved to Up by side2 at the same time as the partition
+ side2 = Set(memberC, memberD, memberE, memberF)
+ val strategy2 = strategy(side2)
+ val decision2 = strategy2.decide()
+ decision2 should ===(DownUnreachable)
+ strategy2.nodesToDown(decision2) should ===(side1Nodes)
+ }
+
+ "detect edge case of leaving/exiting membership change: {A', B} | {C, D} => {C, D}" in {
+ val setup = new Setup2(role = None) {
+ side1 = Set(leaving(memberA), memberB, joining(memberE))
+ side2 = Set(memberC, memberD)
+ }
+ import setup._
+ val strategy1 = strategy(side1)
+ val decision1 = strategy1.decide()
+ decision1 should ===(DownAll)
+ strategy1.nodesToDown(decision1) should ===(side1Nodes.union(side2Nodes))
+
+ // A was moved to Exiting by side2 at the same time as the partition, and that has not been seen by
+ // side1 so it is still Leaving there
+ side1 = Set(exiting(memberA), memberB)
+ val strategy2 = strategy(side2)
+ val decision2 = strategy2.decide()
+ decision2 should ===(DownUnreachable)
+ // A is already Exiting so not downed
+ strategy2.nodesToDown(decision2) should ===(side1Nodes - memberA.uniqueAddress)
+ }
+
+ "down indirectly connected: {(A, B)} => {}" in new Setup2(role = None) {
+ side1 = Set(memberA, memberB)
+ indirectlyConnected = List(memberA -> memberB, memberB -> memberA)
+ assertDowning(Set(memberA, memberB))
+ }
+
+ "down indirectly connected: {(A, B), C} => {C}" in new Setup2(role = None) {
+ side1 = Set(memberA, memberB, memberC)
+ indirectlyConnected = List(memberA -> memberB, memberB -> memberA)
+ // keep fully connected memberC
+ assertDowning(Set(memberA, memberB))
+ }
+
+ "down indirectly connected: {(A, B, C)} => {}" in new Setup2(role = None) {
+ side1 = Set(memberA, memberB, memberC)
+ indirectlyConnected = List(memberA -> memberB, memberB -> memberC, memberC -> memberA)
+ assertDowning(Set(memberA, memberB, memberC))
+ }
+
+ "down indirectly connected: {(A, B, C, D)} => {}" in new Setup2(role = None) {
+ side1 = Set(memberA, memberB, memberC, memberD)
+ indirectlyConnected = List(memberA -> memberD, memberD -> memberA, memberB -> memberC, memberC -> memberB)
+ assertDowning(Set(memberA, memberB, memberC, memberD))
+ }
+
+ "down indirectly connected: {(A, B, C), D, E} => {D, E}" in new Setup2(role = None) {
+ side1 = Set(memberA, memberB, memberC, memberD, memberE)
+ indirectlyConnected = List(memberA -> memberB, memberB -> memberC, memberC -> memberA)
+ // keep fully connected memberD, memberE
+ assertDowning(Set(memberA, memberB, memberC))
+ }
+
+ "down indirectly connected{A, (B, C), D, (E, F), G} => {A, D, G}" in new Setup2(role = None) {
+ side1 = Set(memberA, memberB, memberC, memberD, memberE, memberF, memberG)
+ // two groups of indirectly connected, 4 in total
+ indirectlyConnected = List(memberB -> memberC, memberC -> memberB, memberE -> memberF, memberF -> memberE)
+ // keep fully connected memberA, memberD, memberG
+ assertDowning(Set(memberB, memberC, memberE, memberF))
+ }
+
+ "down indirectly connected, detected via seen: {(A, B, C)} => {}" in new Setup2(role = None) {
+ side1 = Set(memberA, memberB, memberC)
+ indirectlyConnected = List(memberA -> memberB, memberA -> memberC)
+ assertDowning(Set(memberA, memberB, memberC))
+ }
+
+ "down indirectly connected, detected via seen: {(A, B, C, D), E} => {E}" in new Setup2(role = None) {
+ side1 = Set(memberA, memberB, memberC, memberD, memberE)
+ indirectlyConnected = List(memberB -> memberC, memberC -> memberB, memberA -> memberD)
+ // keep fully connected memberE
+ assertDowning(Set(memberA, memberB, memberC, memberD))
+ }
+
+ "down indirectly connected when combined with crashed: {(A, B), D, E} | {C} => {D, E}" in new Setup2(role = None) {
+ side1 = Set(memberA, memberB, memberD, memberE)
+ side2 = Set(memberC)
+ indirectlyConnected = List(memberA -> memberB, memberB -> memberA)
+ // keep fully connected memberD, memberE
+ // note that crashed memberC is also downed
+ assertDowningSide(side1, Set(memberA, memberB, memberC))
+ }
+
+ "down indirectly connected when combined with clean partition: {A, (B, C)} | {D, E} => {A}" in new Setup2(
+ role = None) {
+ side1 = Set(memberA, memberB, memberC)
+ side2 = Set(memberD, memberE)
+ indirectlyConnected = List(memberB -> memberC, memberC -> memberB)
+
+ // from side1 of the partition
+ // keep fully connected memberA
+ // note that memberD and memberE on the other side of the partition are also downed because side1
+ // is majority of clean partition
+ assertDowningSide(side1, Set(memberB, memberC, memberD, memberE))
+
+ // from side2 of the partition
+ // indirectly connected not seen from this side, if clean partition happened first
+ indirectlyConnected = Nil
+ // Note that memberC is not downed, as on the other side, because those indirectly connected
+ // not seen from this side. That outcome is OK.
+ assertDowningSide(side2, Set(memberD, memberE))
+
+ // alternative scenario from side2 of the partition
+ // indirectly connected on side1 happens before the clean partition
+ indirectlyConnected = List(memberB -> memberC, memberC -> memberB)
+ assertDowningSide(side2, Set(memberB, memberC, memberD, memberE))
+ }
+
+ "down indirectly connected on minority side, when combined with clean partition: {A, (B, C)} | {D, E, F, G} => {D, E, F, G}" in new Setup2(
+ role = None) {
+ side1 = Set(memberA, memberB, memberC)
+ side2 = Set(memberD, memberE, memberF, memberG)
+ indirectlyConnected = List(memberB -> memberC, memberC -> memberB)
+
+ // from side1 of the partition, minority
+ assertDowningSide(side1, Set(memberA, memberB, memberC))
+
+ // from side2 of the partition, majority
+ // indirectly connected not seen from this side, if clean partition happened first
+ indirectlyConnected = Nil
+ assertDowningSide(side2, Set(memberA, memberB, memberC))
+
+ // alternative scenario from side2 of the partition
+ // indirectly connected on side1 happens before the clean partition
+ indirectlyConnected = List(memberB -> memberC, memberC -> memberB)
+ assertDowningSide(side2, Set(memberA, memberB, memberC))
+ }
+
+ "down indirectly connected on majority side, when combined with clean partition: {A, B, C} | {(D, E), F, G} => {F, G}" in new Setup2(
+ role = None) {
+ side1 = Set(memberA, memberB, memberC)
+ side2 = Set(memberD, memberE, memberF, memberG)
+
+ // from side1 of the partition, minority
+ // indirectly connected not seen from this side, if clean partition happened first
+ indirectlyConnected = Nil
+ assertDowningSide(side1, Set(memberA, memberB, memberC))
+
+ // alternative scenario from side1 of the partition
+ // indirectly connected on side2 happens before the clean partition
+ indirectlyConnected = List(memberD -> memberE, memberE -> memberD)
+ // note that indirectly connected memberD and memberE are also downed
+ assertDowningSide(side1, Set(memberA, memberB, memberC, memberD, memberE))
+
+ // from side2 of the partition, majority
+ indirectlyConnected = List(memberD -> memberE, memberE -> memberD)
+ assertDowningSide(side2, Set(memberA, memberB, memberC, memberD, memberE))
+ }
+
+ "down indirectly connected spanning across a clean partition: {A, (B), C} | {D, (E, F), G} => {D, G}" in new Setup2(
+ role = None) {
+ side1 = Set(memberA, memberB, memberC)
+ side2 = Set(memberD, memberE, memberF, memberG)
+ indirectlyConnected = List(memberB -> memberE, memberE -> memberF, memberF -> memberB)
+
+ // from side1 of the partition, minority
+ assertDowningSide(side1, Set(memberA, memberB, memberC, memberE, memberF))
+
+ // from side2 of the partition, majority
+ assertDowningSide(side2, Set(memberA, memberB, memberC, memberE, memberF))
+ }
+
+ "down indirectly connected, detected via seen, combined with clean partition: {A, B, C} | {(D, E), (F, G)} => {}" in new Setup2(
+ role = None) {
+ side1 = Set(memberA, memberB, memberC)
+ side2 = Set(memberD, memberE, memberF, memberG)
+
+ // from side1 of the partition, minority
+ assertDowningSide(side1, Set(memberA, memberB, memberC))
+
+ // from side2 of the partition, majority
+ indirectlyConnected = List(memberD -> memberE, memberG -> memberF)
+ assertDowningSide(side2, Set(memberA, memberB, memberC, memberD, memberE, memberF, memberG))
+ }
+
+ "double DownIndirectlyConnected when indirectly connected happens before clean partition: {A, B, C} | {(D, E), (F, G)} => {}" in new Setup2(
+ role = None) {
+ side1 = Set(memberA, memberB, memberC)
+ side2 = Set(memberD, memberE, memberF, memberG)
+ // trouble when indirectly connected happens before clean partition
+ indirectlyConnected = List(memberD -> memberE, memberG -> memberF)
+
+ // from side1 of the partition, minority
+ // D and G are observers and marked E and F as unreachable
+ // A has marked D and G as unreachable
+ // The records D->E, G->F are not removed in the second decision because they are not detected via seenB
+ // due to clean partition. That means that the second decision will also be DownIndirectlyConnected. To bail
+ // out from this situation the strategy will throw IllegalStateException, which is caught and translated to
+ // DownAll.
+ intercept[IllegalStateException] {
+ assertDowningSide(side1, Set(memberA, memberB, memberC))
+ }
+
+ // from side2 of the partition, majority
+ assertDowningSide(side2, Set(memberA, memberB, memberC, memberD, memberE, memberF, memberG))
+ }
+
+ }
+
+ "KeepOldest" must {
+ class Setup2(downIfAlone: Boolean = true, role: Option[String] = None) extends StrategySetup {
+ override def createStrategy() = new KeepOldest(selfDc, downIfAlone, role)
+ }
+
+ "keep partition with oldest" in new Setup2 {
+ // E is the oldest
+ side1 = Set(memberA, memberE)
+ side2 = Set(memberB, memberC, memberD)
+ assertDowning(side2)
+ }
+
+ "keep partition with oldest with role" in new Setup2(role = Some("role2")) {
+ // C and D have role2, D is the oldest
+ side1 = Set(memberA, memberE)
+ side2 = Set(memberB, memberC, memberD)
+ assertDowning(side1)
+ }
+
+ "keep partition with oldest unless alone" in new Setup2(downIfAlone = true) {
+ side1 = Set(memberE)
+ side2 = Set(memberA, memberB, memberC, memberD)
+ assertDowning(side1)
+ }
+
+ "keep partition with oldest in two nodes cluster" in new Setup2 {
+ side1 = Set(memberB)
+ side2 = Set(memberA)
+ assertDowning(side2)
+ }
+
+ "keep one single oldest" in new Setup2 {
+ side1 = Set.empty
+ side2 = Set(memberA)
+ assertDowning(side1)
+ }
+
+ "keep oldest even when alone when downIfAlone = false" in new Setup2(downIfAlone = false) {
+ side1 = Set(memberE)
+ side2 = Set(memberA, memberB, memberC, memberD)
+ assertDowning(side2)
+ }
+
+ "detect leaving/exiting edge case: keep partition with oldest, scenario 1" in {
+ val setup = new Setup2(role = None) {
+ side1 = Set(memberA, memberB, memberD)
+ side2 = Set(memberC, exiting(memberE))
+ }
+ import setup._
+ val strategy1 = strategy(side1)
+ val decision1 = strategy1.decide()
+ // E is Exiting so not counted as oldest, D is oldest
+ decision1 should ===(DownUnreachable)
+ // side2 is downed, but E is already exiting and therefore not downed
+ strategy1.nodesToDown(decision1) should ===(side2Nodes - memberE.uniqueAddress)
+
+ // E was changed to Exiting by side1 but that is not seen on side2 due to the partition, so still Leaving
+ side2 = Set(memberC, leaving(memberE))
+ val strategy2 = strategy(side2)
+ val decision2 = strategy2.decide()
+
+ decision2 should ===(DownAll)
+ strategy2.nodesToDown(decision2) should ===(side1Nodes.union(side2Nodes))
+ }
+
+ "detect leaving/exiting edge case: keep partition with oldest, scenario 2" in {
+ val setup = new Setup2(role = None) {
+ side1 = Set(memberA, leaving(memberE))
+ side2 = Set(memberB, memberC, memberD)
+ }
+ import setup._
+ strategy(side1).decide() should ===(DownAll)
+ strategy(side2).decide() should ===(DownReachable)
+ }
+
+ "detect leaving/exiting edge case: keep partition with oldest, scenario 3" in new Setup2 {
+ // E is the oldest
+ side1 = Set(memberA, memberE)
+ side2 = Set(leaving(memberB), leaving(memberC), memberD)
+ assertDowning(side2)
+ }
+
+ "detect leaving/exiting edge case: keep partition with oldest unless alone, scenario 1" in {
+ val setup = new Setup2(role = None) {
+ side1 = Set(leaving(memberD), memberE)
+ side2 = Set(memberA, memberB, memberC)
+ }
+ import setup._
+ strategy(side1).decide() should ===(DownAll)
+ strategy(side2).decide() should ===(DownReachable)
+ }
+
+ "detect leaving/exiting edge case: keep partition with oldest unless alone, scenario 4" in {
+ val setup = new Setup2(role = None) {
+ side1 = Set(memberE)
+ side2 = Set(memberA, memberB, leaving(memberC))
+ }
+ import setup._
+ strategy(side1).decide() should ===(DownReachable)
+ strategy(side2).decide() should ===(DownUnreachable)
+ }
+
+ "detect leaving/exiting edge case: keep partition with oldest unless alone, scenario 3" in {
+ val setup = new Setup2(role = None) {
+ side1 = Set(memberE)
+ side2 = Set(memberA, leaving(memberB), leaving(memberC), leaving(memberD))
+ }
+ import setup._
+ strategy(side1).decide() should ===(DownReachable)
+ strategy(side2).decide() should ===(DownAll)
+ }
+
+ "detect leaving/exiting edge case: DownReachable on both sides when oldest leaving/exiting is alone" in {
+ val setup = new Setup2(role = None) {
+ side1 = Set(memberD, exiting(memberE))
+ side2 = Set(memberA, memberB, memberC)
+ }
+ import setup._
+ // E is Exiting so not counted as oldest, D is oldest, but it's alone so keep side2 anyway
+ strategy(side1).decide() should ===(DownReachable)
+
+ // E was changed to Exiting by side1 but that is not seen on side2 due to the partition, so still Leaving
+ side1 = Set(memberD, leaving(memberE))
+ strategy(side2).decide() should ===(DownReachable)
+ }
+
+ "detect leaving/exiting edge case: when one single oldest" in {
+ val setup = new Setup2(role = None) {
+ side1 = Set(memberA)
+ side2 = Set(exiting(memberB))
+ }
+ import setup._
+ // B is Exiting so not counted as oldest, A is oldest
+ strategy(side1).decide() should ===(DownUnreachable)
+
+ // B was changed to Exiting by side1 but that is not seen on side2 due to the partition, so still Leaving
+ side2 = Set(leaving(memberB))
+ strategy(side2).decide() should ===(DownAll)
+ }
+
+ "detect joining/up edge case: keep partition with oldest unless alone, scenario 1" in {
+ val setup = new Setup2(role = None) {
+ side1 = Set(joining(memberA), memberE)
+ side2 = Set(memberB, memberC, memberD)
+ }
+ import setup._
+ // E alone when not counting joining A
+ strategy(side1).decide() should ===(DownReachable)
+ // but A could have been up on other side1 and therefore side2 has to down all
+ strategy(side2).decide() should ===(DownAll)
+ }
+
+ "detect joining/up edge case: keep oldest even when alone when downIfAlone = false" in {
+ val setup = new Setup2(downIfAlone = false) {
+ side1 = Set(joining(memberA), memberE)
+ side2 = Set(memberB, memberC, memberD)
+ }
+ import setup._
+ // joining A shouldn't matter when downIfAlone = false
+ strategy(side1).decide() should ===(DownUnreachable)
+ strategy(side2).decide() should ===(DownReachable)
+ }
+
+ "down indirectly connected: {(A, B), C} => {C}" in new Setup2 {
+ side1 = Set(memberA, memberB, memberC)
+ indirectlyConnected = List(memberA -> memberB, memberB -> memberA)
+ assertDowning(Set(memberA, memberB))
+ }
+
+ "down indirectly connected on younger side, when combined with clean partition: {A, (B, C)} | {D, E, F, G} => {D, E, F, G}" in new Setup2 {
+ side1 = Set(memberA, memberB, memberC)
+ side2 = Set(memberD, memberE, memberF, memberG)
+ indirectlyConnected = List(memberB -> memberC, memberC -> memberB)
+
+ // from side1 of the partition, younger
+ assertDowningSide(side1, Set(memberA, memberB, memberC))
+
+ // from side2 of the partition, oldest
+ // indirectly connected not seen from this side, if clean partition happened first
+ indirectlyConnected = Nil
+ assertDowningSide(side2, Set(memberA, memberB, memberC))
+
+ // alternative scenario from side2 of the partition
+ // indirectly connected on side1 happens before the clean partition
+ indirectlyConnected = List(memberB -> memberC, memberC -> memberB)
+ assertDowningSide(side2, Set(memberA, memberB, memberC))
+ }
+
+ "down indirectly connected on oldest side, when combined with clean partition: {A, B, C} | {(D, E), F, G} => {F, G}" in new Setup2 {
+ side1 = Set(memberA, memberB, memberC)
+ side2 = Set(memberD, memberE, memberF, memberG)
+
+ // from side1 of the partition, younger
+ // indirectly connected not seen from this side, if clean partition happened first
+ indirectlyConnected = Nil
+ assertDowningSide(side1, Set(memberA, memberB, memberC))
+
+ // alternative scenario from side1 of the partition
+ // indirectly connected on side2 happens before the clean partition
+ indirectlyConnected = List(memberD -> memberE, memberE -> memberD)
+ // note that indirectly connected memberD and memberE are also downed
+ assertDowningSide(side1, Set(memberA, memberB, memberC, memberD, memberE))
+
+ // from side2 of the partition, oldest
+ indirectlyConnected = List(memberD -> memberE, memberE -> memberD)
+ assertDowningSide(side2, Set(memberA, memberB, memberC, memberD, memberE))
+ }
+
+ }
+
+ "DownAllNodes" must {
+ class Setup2 extends StrategySetup {
+ override def createStrategy() = new DownAllNodes(selfDc)
+ }
+
+ "down all" in new Setup2 {
+ side1 = Set(memberA, memberB, memberC)
+ side2 = Set(memberD, memberE)
+ assertDowning(side1.union(side2))
+ }
+
+ "down all when any indirectly connected: {(A, B), C} => {}" in new Setup2 {
+ side1 = Set(memberA, memberB, memberC)
+ indirectlyConnected = List(memberA -> memberB, memberB -> memberA)
+ assertDowning(side1)
+ }
+ }
+
+ "LeaseMajority" must {
+ class Setup2(role: Option[String]) extends StrategySetup {
+ val testLease: TestLease = new TestLease(testLeaseSettings, extSystem)
+
+ val acquireLeaseDelayForMinority: FiniteDuration = 2.seconds
+
+ override def createStrategy() =
+ new LeaseMajority(selfDc, role, testLease, acquireLeaseDelayForMinority)
+ }
+
+ "decide AcquireLeaseAndDownUnreachable, and DownReachable as reverse decision" in {
+ val setup = new Setup2(role = None) {
+ side1 = Set(memberA, memberC, memberE)
+ side2 = Set(memberB, memberD)
+ }
+ import setup._
+ val strategy1 = strategy(side1)
+ val decision1 = strategy1.decide()
+ decision1 should ===(AcquireLeaseAndDownUnreachable(Duration.Zero))
+ strategy1.nodesToDown(decision1) should ===(side2Nodes)
+ val reverseDecision1 = strategy1.reverseDecision(decision1)
+ reverseDecision1 should ===(DownReachable)
+ strategy1.nodesToDown(reverseDecision1) should ===(side1Nodes)
+
+ val strategy2 = strategy(side2)
+ val decision2 = strategy2.decide()
+ decision2 should ===(AcquireLeaseAndDownUnreachable(acquireLeaseDelayForMinority))
+ strategy2.nodesToDown(decision2) should ===(side1Nodes)
+ val reverseDecision2 = strategy2.reverseDecision(decision2)
+ reverseDecision2 should ===(DownReachable)
+ strategy2.nodesToDown(reverseDecision2) should ===(side2Nodes)
+ }
+
+ "try to keep half with lowest address when equal size partition" in {
+ val setup = new Setup2(role = Some("role2")) {
+ side1 = Set(memberA, memberD, memberE)
+ side2 = Set(memberB, memberC)
+ // memberC is lowest with role2
+ }
+ import setup._
+ val strategy1 = strategy(side1)
+ val decision1 = strategy1.decide()
+ // delay on side1 because memberC is lowest address with role2
+ decision1 should ===(AcquireLeaseAndDownUnreachable(acquireLeaseDelayForMinority))
+ strategy1.nodesToDown(decision1) should ===(side2Nodes)
+
+ val strategy2 = strategy(side2)
+ val decision2 = strategy2.decide()
+ decision2 should ===(AcquireLeaseAndDownUnreachable(Duration.Zero))
+ strategy2.nodesToDown(decision2) should ===(side1Nodes)
+ }
+
+ "down indirectly connected: {(A, B), C} => {C}" in {
+ val setup = new Setup2(role = None) {
+ side1 = Set(memberA, memberB, memberC)
+ indirectlyConnected = List(memberA -> memberB, memberB -> memberA)
+ }
+ import setup._
+ val strategy1 = strategy(side1)
+ val decision1 = strategy1.decide()
+ decision1 should ===(AcquireLeaseAndDownIndirectlyConnected(Duration.Zero))
+ strategy1.nodesToDown(decision1) should ===(Set(memberA.uniqueAddress, memberB.uniqueAddress))
+ val reverseDecision1 = strategy1.reverseDecision(decision1)
+ reverseDecision1 should ===(ReverseDownIndirectlyConnected)
+ strategy1.nodesToDown(reverseDecision1) should ===(side1Nodes)
+ }
+
+ "down indirectly connected when combined with clean partition: {A, (B, C)} | {D, E} => {A}" in {
+ val setup = new Setup2(role = None) {
+ side1 = Set(memberA, memberB, memberC)
+ side2 = Set(memberD, memberE)
+ indirectlyConnected = List(memberB -> memberC, memberC -> memberB)
+ }
+ import setup._
+
+ // from side1 of the partition
+ // keep fully connected memberA
+ // note that memberD and memberE on the other side of the partition are also downed
+ val strategy1 = strategy(side1)
+ val decision1 = strategy1.decide()
+ decision1 should ===(AcquireLeaseAndDownIndirectlyConnected(Duration.Zero))
+ strategy1.nodesToDown(decision1) should ===(Set(memberB, memberC, memberD, memberE).map(_.uniqueAddress))
+ val reverseDecision1 = strategy1.reverseDecision(decision1)
+ reverseDecision1 should ===(ReverseDownIndirectlyConnected)
+ strategy1.nodesToDown(reverseDecision1) should ===(side1Nodes)
+
+ // from side2 of the partition
+ // indirectly connected not seen from this side, if clean partition happened first
+ indirectlyConnected = Nil
+ // Note that memberC is not downed, as on the other side, because those indirectly connected
+ // not seen from this side. That outcome is OK.
+ val strategy2 = strategy(side2)
+ val decision2 = strategy2.decide()
+ decision2 should ===(AcquireLeaseAndDownUnreachable(acquireLeaseDelayForMinority))
+ strategy2.nodesToDown(decision2) should ===(side1Nodes)
+ val reverseDecision2 = strategy2.reverseDecision(decision2)
+ reverseDecision2 should ===(DownReachable)
+ strategy2.nodesToDown(reverseDecision2) should ===(side2Nodes)
+
+ // alternative scenario from side2 of the partition
+ // indirectly connected on side1 happens before the clean partition
+ indirectlyConnected = List(memberB -> memberC, memberC -> memberB)
+ val strategy3 = strategy(side2)
+ val decision3 = strategy3.decide()
+ decision3 should ===(AcquireLeaseAndDownIndirectlyConnected(Duration.Zero))
+ strategy3.nodesToDown(decision3) should ===(side1Nodes)
+ val reverseDecision3 = strategy3.reverseDecision(decision3)
+ reverseDecision3 should ===(ReverseDownIndirectlyConnected)
+ strategy3.nodesToDown(reverseDecision3) should ===(Set(memberB, memberC, memberD, memberE).map(_.uniqueAddress))
+
+ }
+ }
+
+ "Strategy" must {
+
+ class MajoritySetup(role: Option[String] = None) extends StrategySetup {
+ override def createStrategy() = new KeepMajority(selfDc, role)
+ }
+
+ class OldestSetup(role: Option[String] = None) extends StrategySetup {
+ override def createStrategy() = new KeepOldest(selfDc, downIfAlone = true, role)
+ }
+
+ "add and remove members with default Member ordering" in {
+ val setup = new MajoritySetup(role = None) {
+ side1 = Set.empty
+ side2 = Set.empty
+ }
+ import setup._
+ val strategy1 = strategy(side1)
+ testAddRemove(strategy1)
+ }
+
+ "add and remove members with oldest Member ordering" in {
+ val setup = new OldestSetup(role = None) {
+ side1 = Set.empty
+ side2 = Set.empty
+ }
+ testAddRemove(setup.strategy(setup.side1))
+ }
+
+ def testAddRemove(strategy: DowningStrategy) = {
+ strategy.add(joining(memberA))
+ strategy.add(joining(memberB))
+ strategy.allMembersInDC.size should ===(2)
+ strategy.allMembersInDC.foreach {
+ _.status should ===(MemberStatus.Joining)
+ }
+ strategy.add(memberA)
+ strategy.add(memberB)
+ strategy.allMembersInDC.size should ===(2)
+ strategy.allMembersInDC.foreach {
+ _.status should ===(MemberStatus.Up)
+ }
+ strategy.add(leaving(memberB))
+ strategy.allMembersInDC.size should ===(2)
+ strategy.allMembersInDC.toList.map(_.status).toSet should ===(Set(MemberStatus.Up, MemberStatus.Leaving))
+ strategy.add(exiting(memberB))
+ strategy.allMembersInDC.size should ===(2)
+ strategy.allMembersInDC.toList.map(_.status).toSet should ===(Set(MemberStatus.Up, MemberStatus.Exiting))
+ strategy.remove(memberA)
+ strategy.allMembersInDC.size should ===(1)
+ strategy.allMembersInDC.head.status should ===(MemberStatus.Exiting)
+ }
+
+ "collect and filter members with default Member ordering" in {
+ val setup = new MajoritySetup(role = None) {
+ side1 = Set.empty
+ side2 = Set.empty
+ }
+
+ testCollectAndFilter(setup)
+ }
+
+ "collect and filter members with oldest Member ordering" in {
+ val setup = new OldestSetup(role = None) {
+ side1 = Set.empty
+ side2 = Set.empty
+ }
+
+ testCollectAndFilter(setup)
+ }
+
+ def testCollectAndFilter(setup: StrategySetup): Unit = {
+ import setup._
+
+ side1 = Set(memberAWeaklyUp, memberB, joining(memberC))
+ side2 = Set(memberD, leaving(memberE), downed(memberF), exiting(memberG))
+
+ val strategy1 = strategy(side1)
+
+ strategy1.membersWithRole should ===(Set(memberB, memberD, leaving(memberE)))
+ strategy1.membersWithRole(includingPossiblyUp = true, excludingPossiblyExiting = false) should ===(
+ Set(memberAWeaklyUp, memberB, joining(memberC), memberD, leaving(memberE)))
+ strategy1.membersWithRole(includingPossiblyUp = false, excludingPossiblyExiting = true) should ===(
+ Set(memberB, memberD))
+ strategy1.membersWithRole(includingPossiblyUp = true, excludingPossiblyExiting = true) should ===(
+ Set(memberAWeaklyUp, memberB, joining(memberC), memberD))
+
+ strategy1.reachableMembersWithRole should ===(Set(memberB))
+ strategy1.reachableMembersWithRole(includingPossiblyUp = true, excludingPossiblyExiting = false) should ===(
+ Set(memberAWeaklyUp, memberB, joining(memberC)))
+ strategy1.reachableMembersWithRole(includingPossiblyUp = false, excludingPossiblyExiting = true) should ===(
+ Set(memberB))
+ strategy1.reachableMembersWithRole(includingPossiblyUp = true, excludingPossiblyExiting = true) should ===(
+ Set(memberAWeaklyUp, memberB, joining(memberC)))
+
+ strategy1.unreachableMembersWithRole should ===(Set(memberD, leaving(memberE)))
+ strategy1.unreachableMembers(includingPossiblyUp = true, excludingPossiblyExiting = false) should ===(
+ Set(memberD, leaving(memberE)))
+ strategy1.unreachableMembers(includingPossiblyUp = false, excludingPossiblyExiting = true) should ===(
+ Set(memberD))
+ strategy1.unreachableMembers(includingPossiblyUp = true, excludingPossiblyExiting = true) should ===(Set(memberD))
+
+ strategy1.unreachable(memberAWeaklyUp) should ===(false)
+ strategy1.unreachable(memberB) should ===(false)
+ strategy1.unreachable(memberD) should ===(true)
+ strategy1.unreachable(leaving(memberE)) should ===(true)
+ strategy1.unreachable(downed(memberF)) should ===(true)
+ strategy1.joining should ===(Set(memberAWeaklyUp, joining(memberC)))
+
+ val strategy2 = strategy(side2)
+
+ strategy2.membersWithRole should ===(Set(memberB, memberD, leaving(memberE)))
+ strategy2.membersWithRole(includingPossiblyUp = true, excludingPossiblyExiting = false) should ===(
+ Set(memberAWeaklyUp, memberB, joining(memberC), memberD, leaving(memberE)))
+ strategy2.membersWithRole(includingPossiblyUp = false, excludingPossiblyExiting = true) should ===(
+ Set(memberB, memberD))
+ strategy2.membersWithRole(includingPossiblyUp = true, excludingPossiblyExiting = true) should ===(
+ Set(memberAWeaklyUp, memberB, joining(memberC), memberD))
+
+ strategy2.unreachableMembersWithRole should ===(Set(memberB))
+ strategy2.unreachableMembersWithRole(includingPossiblyUp = true, excludingPossiblyExiting = false) should ===(
+ Set(memberAWeaklyUp, memberB, joining(memberC)))
+ strategy2.unreachableMembersWithRole(includingPossiblyUp = false, excludingPossiblyExiting = true) should ===(
+ Set(memberB))
+ strategy2.unreachableMembersWithRole(includingPossiblyUp = true, excludingPossiblyExiting = true) should ===(
+ Set(memberAWeaklyUp, memberB, joining(memberC)))
+
+ strategy2.reachableMembersWithRole should ===(Set(memberD, leaving(memberE)))
+ strategy2.reachableMembers(includingPossiblyUp = true, excludingPossiblyExiting = false) should ===(
+ Set(memberD, leaving(memberE)))
+ strategy2.reachableMembers(includingPossiblyUp = false, excludingPossiblyExiting = true) should ===(Set(memberD))
+ strategy2.reachableMembers(includingPossiblyUp = true, excludingPossiblyExiting = true) should ===(Set(memberD))
+
+ strategy2.unreachable(memberAWeaklyUp) should ===(true)
+ strategy2.unreachable(memberB) should ===(true)
+ strategy2.unreachable(memberD) should ===(false)
+ strategy2.unreachable(leaving(memberE)) should ===(false)
+ strategy2.unreachable(downed(memberF)) should ===(false)
+ strategy2.joining should ===(Set(memberAWeaklyUp, joining(memberC)))
+ }
+ }
+
+ "Split Brain Resolver" must {
+
+ class SetupKeepMajority(
+ stableAfter: FiniteDuration,
+ selfUniqueAddress: UniqueAddress,
+ role: Option[String],
+ downAllWhenUnstable: FiniteDuration = Duration.Zero,
+ tickInterval: FiniteDuration = Duration.Zero)
+ extends Setup(stableAfter, new KeepMajority(selfDc, role), selfUniqueAddress, downAllWhenUnstable, tickInterval)
+
+ class SetupKeepOldest(
+ stableAfter: FiniteDuration,
+ selfUniqueAddress: UniqueAddress,
+ downIfAlone: Boolean,
+ role: Option[String])
+ extends Setup(stableAfter, new KeepOldest(selfDc, downIfAlone, role), selfUniqueAddress)
+
+ class SetupStaticQuorum(
+ stableAfter: FiniteDuration,
+ selfUniqueAddress: UniqueAddress,
+ size: Int,
+ role: Option[String])
+ extends Setup(stableAfter, new StaticQuorum(selfDc, size, role), selfUniqueAddress)
+
+ class SetupDownAllNodes(stableAfter: FiniteDuration, selfUniqueAddress: UniqueAddress)
+ extends Setup(stableAfter, new DownAllNodes(selfDc), selfUniqueAddress)
+
+ class SetupLeaseMajority(
+ stableAfter: FiniteDuration,
+ selfUniqueAddress: UniqueAddress,
+ role: Option[String],
+ val testLease: TestLease,
+ downAllWhenUnstable: FiniteDuration = Duration.Zero,
+ tickInterval: FiniteDuration = Duration.Zero)
+ extends Setup(
+ stableAfter,
+ new LeaseMajority(selfDc, role, testLease, acquireLeaseDelayForMinority = 20.millis),
+ selfUniqueAddress,
+ downAllWhenUnstable,
+ tickInterval)
+
+ abstract class Setup(
+ stableAfter: FiniteDuration,
+ strategy: DowningStrategy,
+ selfUniqueAddress: UniqueAddress,
+ downAllWhenUnstable: FiniteDuration = Duration.Zero,
+ tickInterval: FiniteDuration = Duration.Zero) {
+
+ val a = system.actorOf(
+ DowningTestActor
+ .props(stableAfter, strategy, testActor, selfUniqueAddress, selfDc, downAllWhenUnstable, tickInterval))
+
+ def memberUp(members: Member*): Unit =
+ members.foreach(m => a ! MemberUp(m))
+
+ def memberWeaklyUp(members: Member*): Unit =
+ members.foreach(m => a ! MemberWeaklyUp(m))
+
+ def leader(member: Member): Unit =
+ a ! LeaderChanged(Some(member.address))
+
+ def unreachable(members: Member*): Unit =
+ members.foreach(m => a ! UnreachableMember(m))
+
+ def reachabilityChanged(unreachability: (Member, Member)*): Unit = {
+ unreachable(unreachability.map { case (_, to) => to }: _*)
+
+ val r = createReachability(unreachability)
+ a ! ReachabilityChanged(r)
+ }
+
+ def remove(members: Member*): Unit =
+ members.foreach(m => a ! MemberRemoved(m.copy(Removed), previousStatus = Exiting))
+
+ def dcUnreachable(members: Member*): Unit =
+ members.map(_.dataCenter).toSet[DataCenter].foreach(dc => a ! UnreachableDataCenter(dc))
+
+ def dcReachable(members: Member*): Unit =
+ members.map(_.dataCenter).toSet[DataCenter].foreach(dc => a ! ReachableDataCenter(dc))
+
+ def reachable(members: Member*): Unit =
+ members.foreach(m => a ! ReachableMember(m))
+
+ def tick(): Unit = a ! SplitBrainResolver.Tick
+
+ def expectDownCalled(members: Member*): Unit =
+ receiveN(members.length).toSet should be(members.map(m => DownCalled(m.address)).toSet)
+
+ def expectNoDecision(max: FiniteDuration): Unit =
+ expectNoMessage(max)
+
+ def stop(): Unit = {
+ system.stop(a)
+ expectNoMessage(100.millis)
+ }
+ }
+
+ "have downRemovalMargin equal to stable-after" in {
+ val cluster = Cluster(system)
+ val sbrSettings = new SplitBrainResolverSettings(system.settings.config)
+ cluster.downingProvider.downRemovalMargin should be(sbrSettings.DowningStableAfter)
+ }
+
+ "down unreachable when leader" in new SetupKeepMajority(Duration.Zero, memberA.uniqueAddress, role = None) {
+ memberUp(memberA, memberB, memberC)
+ leader(memberA)
+ unreachable(memberB)
+ tick()
+ expectDownCalled(memberB)
+ stop()
+ }
+
+ "not down unreachable when not leader" in new SetupKeepMajority(Duration.Zero, memberB.uniqueAddress, role = None) {
+ memberUp(memberA, memberB, memberC)
+ leader(memberA)
+ unreachable(memberC)
+ tick()
+ expectNoMessage(500.millis)
+ stop()
+ }
+
+ "down unreachable when becoming leader" in new SetupKeepMajority(
+ stableAfter = Duration.Zero,
+ memberA.uniqueAddress,
+ role = None) {
+ memberUp(memberA, memberB, memberC)
+ leader(memberB)
+ unreachable(memberC)
+ leader(memberA)
+ tick()
+ expectDownCalled(memberC)
+ stop()
+ }
+
+ "down unreachable after specified duration" in new SetupKeepMajority(
+ stableAfter = 2.seconds,
+ memberA.uniqueAddress,
+ role = None) {
+ memberUp(memberA, memberB, memberC)
+ leader(memberA)
+ unreachable(memberB)
+ expectNoMessage(1.second)
+ expectDownCalled(memberB)
+ stop()
+ }
+
+ "down unreachable when becoming leader inbetween detection and specified duration" in new SetupKeepMajority(
+ stableAfter = 2.seconds,
+ memberA.uniqueAddress,
+ role = None) {
+ memberUp(memberA, memberB, memberC)
+ leader(memberB)
+ unreachable(memberC)
+ leader(memberA)
+ tick()
+ expectNoMessage(1.second)
+ expectDownCalled(memberC)
+ stop()
+ }
+
+ "not down unreachable when loosing leadership inbetween detection and specified duration" in new SetupKeepMajority(
+ stableAfter = 1.seconds,
+ memberA.uniqueAddress,
+ role = None) {
+ memberUp(memberA, memberB, memberC)
+ leader(memberA)
+ unreachable(memberC)
+ leader(memberB)
+ tick()
+ expectNoMessage(1500.millis)
+ stop()
+ }
+
+ // reproducer of issue #436
+ "down when becoming Weakly-Up leader" in new SetupKeepMajority(
+ stableAfter = Duration.Zero,
+ memberAWeaklyUp.uniqueAddress,
+ role = None) {
+ memberUp(memberC)
+ memberWeaklyUp(memberAWeaklyUp, memberBWeaklyUp)
+ unreachable(memberC)
+ leader(memberAWeaklyUp)
+ tick()
+ expectDownCalled(memberAWeaklyUp, memberBWeaklyUp)
+ stop()
+ }
+
+ "not down when unreachable become reachable inbetween detection and specified duration" in new SetupKeepMajority(
+ stableAfter = 1.seconds,
+ memberA.uniqueAddress,
+ role = None) {
+ memberUp(memberA, memberB, memberC)
+ leader(memberA)
+ unreachable(memberB)
+ reachable(memberB)
+ tick()
+ expectNoMessage(1500.millis)
+ stop()
+ }
+
+ "not down when unreachable is removed inbetween detection and specified duration" in new SetupKeepMajority(
+ stableAfter = 1.seconds,
+ memberA.uniqueAddress,
+ role = None) {
+ memberUp(memberA, memberB, memberC)
+ leader(memberA)
+ unreachable(memberB)
+ a ! MemberRemoved(memberB.copy(Removed), previousStatus = Exiting)
+ tick()
+ expectNoMessage(1500.millis)
+ stop()
+ }
+
+ "not down when unreachable is already Down" in new SetupKeepMajority(
+ stableAfter = Duration.Zero,
+ memberA.uniqueAddress,
+ role = None) {
+ memberUp(memberA, memberB, memberC)
+ leader(memberA)
+ unreachable(memberB.copy(Down))
+ tick()
+ expectNoMessage(1000.millis)
+ stop()
+ }
+
+ "down minority partition" in new SetupKeepMajority(stableAfter = Duration.Zero, memberA.uniqueAddress, role = None) {
+ memberUp(memberA, memberB, memberC, memberD, memberE)
+ leader(memberA)
+ reachabilityChanged(memberA -> memberB, memberC -> memberD)
+ tick()
+ expectDownCalled(memberB, memberD)
+ stop()
+ }
+
+ "keep partition with oldest" in new SetupKeepOldest(
+ stableAfter = Duration.Zero,
+ memberA.uniqueAddress,
+ downIfAlone = true,
+ role = None) {
+ memberUp(memberA, memberB, memberC, memberD, memberE)
+ leader(memberA)
+ reachabilityChanged(memberA -> memberB, memberA -> memberC, memberE -> memberD)
+ tick()
+ expectDownCalled(memberB, memberC, memberD)
+ stop()
+ }
+
+ "log warning if N > static-quorum.size * 2 - 1" in new SetupStaticQuorum(
+ stableAfter = Duration.Zero,
+ memberA.uniqueAddress,
+ size = 2,
+ role = None) {
+ EventFilter.warning(pattern = "cluster size is \\[4\\].*not add more than \\[3\\]", occurrences = 1).intercept {
+ memberUp(memberA, memberB, memberC, memberD)
+ }
+ leader(memberA)
+ unreachable(memberC, memberD)
+ tick()
+ // down all
+ expectDownCalled(memberA, memberB, memberC, memberD)
+ stop()
+ }
+
+ "not care about partition across data centers" in new SetupKeepMajority(
+ stableAfter = Duration.Zero,
+ memberA.uniqueAddress,
+ role = None) {
+ memberUp(dataCenter(selfDc, memberA, memberB, memberC).toList: _*)
+ memberUp(dataCenter("other", memberD, memberE).toList: _*)
+ leader(memberA)
+ unreachable(memberB)
+ dcUnreachable(dataCenter("other", memberD, memberE).toList: _*)
+ tick()
+ expectDownCalled(memberB)
+ stop()
+ }
+
+ "not count members from other data centers" in new SetupKeepMajority(
+ stableAfter = Duration.Zero,
+ memberA.uniqueAddress,
+ role = None) {
+ memberUp(dataCenter(selfDc, memberA, memberB, memberC).toList: _*)
+ memberUp(dataCenter("other", memberD, memberE).toList: _*)
+ leader(memberA)
+ unreachable(memberB, memberC)
+ tick()
+ // if memberD and memberE would be counted then memberA would be in majority side
+ expectDownCalled(memberA)
+ stop()
+ }
+
+ "keep oldest in self data centers" in new SetupKeepOldest(
+ // note that this is now on B
+ stableAfter = Duration.Zero,
+ memberB.uniqueAddress,
+ downIfAlone = true,
+ role = None) {
+ memberUp(dataCenter(selfDc, memberA, memberB, memberC).toList: _*)
+ // D and E have lower upNumber (older), but not in self DC
+ memberUp(dataCenter("other", memberD, memberE).toList: _*)
+ leader(memberB)
+ unreachable(memberA, memberC)
+ tick()
+ // if memberD and memberE would be counted then memberC would not oldest
+ // C is oldest in selfDc, so keep C and B and down self
+ expectDownCalled(memberB)
+ stop()
+ }
+
+ "log warning for data center unreachability" in new SetupKeepMajority(
+ stableAfter = Duration.Zero,
+ memberA.uniqueAddress,
+ role = None) {
+ memberUp(dataCenter(selfDc, memberA, memberB, memberC).toList: _*)
+ memberUp(dataCenter("other", memberD, memberE).toList: _*)
+ leader(memberA)
+ EventFilter.warning(start = "Data center [other] observed as unreachable", occurrences = 1).intercept {
+ dcUnreachable(dataCenter("other", memberD, memberE).toList: _*)
+ }
+ stop()
+ }
+
+ "down indirectly connected: {(A, B), C} => {C}" in new SetupKeepMajority(
+ stableAfter = Duration.Zero,
+ memberA.uniqueAddress,
+ role = None) {
+ memberUp(memberA, memberB, memberC)
+ leader(memberA)
+ reachabilityChanged(memberA -> memberB, memberB -> memberA)
+ tick()
+ // keep fully connected memberC
+ expectDownCalled(memberA, memberB)
+ stop()
+ }
+
+ "down indirectly connected when combined with crashed: {(A, B), D, E} | {C} => {D, E}" in new SetupKeepMajority(
+ stableAfter = Duration.Zero,
+ memberA.uniqueAddress,
+ role = None) {
+ memberUp(memberA, memberB, memberC, memberD, memberE)
+ leader(memberA)
+ reachabilityChanged(memberA -> memberB, memberB -> memberA, memberB -> memberC)
+ tick()
+ // keep fully connected memberD, memberE
+ // note that crashed memberC is also downed
+ expectDownCalled(memberA, memberB, memberC)
+ stop()
+ }
+
+ "down indirectly connected when combined with clean partition: {A, (B, C)} | {D, E} => {A}" in {
+ // from left side of the partition, memberA, memberB, memberC
+ new SetupKeepMajority(stableAfter = Duration.Zero, memberA.uniqueAddress, role = None) {
+ memberUp(memberA, memberB, memberC, memberD, memberE)
+ leader(memberA)
+ // indirectly connected: memberB, memberC
+ // clean partition: memberA, memberB, memberC | memeberD, memberE
+ reachabilityChanged(
+ memberB -> memberC,
+ memberC -> memberB,
+ memberA -> memberD,
+ memberB -> memberD,
+ memberB -> memberE,
+ memberC -> memberE)
+ tick()
+ // keep fully connected memberA
+ // note that memberD and memberE on the other side of the partition are also downed
+ expectDownCalled(memberB, memberC, memberD, memberE)
+ stop()
+ }
+
+ // from right side of the partition, memberD, memberE
+ new SetupKeepMajority(stableAfter = Duration.Zero, memberD.uniqueAddress, role = None) {
+ memberUp(memberA, memberB, memberC, memberD, memberE)
+ leader(memberD)
+ // indirectly connected not seen from this side
+ // clean partition: memberA, memberB, memberC | memeberD, memberE
+ reachabilityChanged(memberD -> memberA, memberD -> memberB, memberE -> memberB, memberE -> memberC)
+ tick()
+ // Note that memberC is not downed, as on the other side, because those indirectly connected
+ // not seen from this side. That outcome is OK.
+ expectDownCalled(memberD, memberE)
+ stop()
+ }
+
+ }
+
+ "down all in self data centers" in new SetupDownAllNodes(stableAfter = Duration.Zero, memberA.uniqueAddress) {
+ memberUp(dataCenter(selfDc, memberA, memberB, memberC).toList: _*)
+ // D and E not in self DC
+ memberUp(dataCenter("other", memberD, memberE).toList: _*)
+ leader(memberA)
+ unreachable(memberA, memberC)
+ tick()
+ expectDownCalled(memberA, memberB, memberC)
+ stop()
+ }
+
+ "down all when unstable" in new SetupKeepMajority(
+ stableAfter = 2.seconds,
+ downAllWhenUnstable = 1.second,
+ selfUniqueAddress = memberA.uniqueAddress,
+ role = None,
+ tickInterval = 100.seconds) {
+ memberUp(memberA, memberB, memberC, memberD, memberE)
+ leader(memberA)
+ reachabilityChanged(memberB -> memberD, memberB -> memberE)
+ tick()
+ expectNoDecision(100.millis)
+
+ Thread.sleep(1000)
+ reachabilityChanged(memberB -> memberD)
+ reachable(memberE)
+ tick()
+ expectNoDecision(100.millis)
+
+ Thread.sleep(1000)
+ reachabilityChanged(memberB -> memberD, memberB -> memberE)
+ tick()
+ expectNoDecision(100.millis)
+
+ Thread.sleep(1000)
+ reachabilityChanged(memberB -> memberD)
+ reachable(memberE)
+ tick()
+ expectDownCalled(memberA, memberB, memberC, memberD, memberE)
+ }
+
+ "not down all when becoming stable again" in new SetupKeepMajority(
+ stableAfter = 2.seconds,
+ downAllWhenUnstable = 1.second,
+ selfUniqueAddress = memberA.uniqueAddress,
+ role = None,
+ tickInterval = 100.seconds) {
+ memberUp(memberA, memberB, memberC, memberD, memberE)
+ leader(memberA)
+ reachabilityChanged(memberB -> memberD, memberB -> memberE)
+ tick()
+ expectNoDecision(100.millis)
+
+ Thread.sleep(1000)
+ reachabilityChanged(memberB -> memberD)
+ reachable(memberE)
+ tick()
+ expectNoDecision(100.millis)
+
+ // wait longer than stableAfter
+ Thread.sleep(500)
+ tick()
+ expectNoDecision(100.millis)
+ reachabilityChanged()
+ reachable(memberD)
+ Thread.sleep(500)
+ tick()
+ expectNoDecision(100.millis)
+
+ Thread.sleep(3000)
+ tick()
+ expectNoDecision(100.millis)
+ }
+
+ "down other side when lease can be acquired" in new SetupLeaseMajority(
+ Duration.Zero,
+ memberA.uniqueAddress,
+ role = None,
+ new TestLease(testLeaseSettings, extSystem)) {
+ memberUp(memberA, memberB, memberC)
+ leader(memberA)
+ unreachable(memberB)
+ testLease.setNextAcquireResult(Future.successful(true))
+ tick()
+ expectDownCalled(memberB)
+ stop()
+ }
+
+ "down own side when lease cannot be acquired" in new SetupLeaseMajority(
+ Duration.Zero,
+ memberA.uniqueAddress,
+ role = None,
+ new TestLease(testLeaseSettings, extSystem)) {
+ memberUp(memberA, memberB, memberC)
+ leader(memberA)
+ unreachable(memberB)
+ testLease.setNextAcquireResult(Future.successful(false))
+ tick()
+ expectDownCalled(memberA, memberC)
+ stop()
+ }
+
+ "down indirectly connected when lease can be acquired: {(A, B), C} => {C}" in new SetupLeaseMajority(
+ stableAfter = Duration.Zero,
+ memberA.uniqueAddress,
+ role = None,
+ new TestLease(testLeaseSettings, extSystem)) {
+ memberUp(memberA, memberB, memberC)
+ leader(memberA)
+ reachabilityChanged(memberA -> memberB, memberB -> memberA)
+ testLease.setNextAcquireResult(Future.successful(true))
+ tick()
+ // keep fully connected memberC
+ expectDownCalled(memberA, memberB)
+ stop()
+ }
+
+ "down indirectly connected when lease cannot be acquired: {(A, B), C} => {C}" in new SetupLeaseMajority(
+ stableAfter = Duration.Zero,
+ memberA.uniqueAddress,
+ role = None,
+ new TestLease(testLeaseSettings, extSystem)) {
+ memberUp(memberA, memberB, memberC)
+ leader(memberA)
+ reachabilityChanged(memberA -> memberB, memberB -> memberA)
+ testLease.setNextAcquireResult(Future.successful(false))
+ tick()
+ // all reachable + all indirectly connected
+ expectDownCalled(memberA, memberB, memberC)
+ stop()
+ }
+
+ }
+
+ "Split Brain Resolver downing provider" must {
+
+ "be loadable through the cluster extension" in {
+ Cluster(system).downingProvider shouldBe a[SplitBrainResolverProvider]
+ }
+ }
+
+ "Reachability changes" must {
+ val strategy = new KeepMajority(defaultDataCenter, None)
+ strategy.add(memberA)
+ strategy.add(memberB)
+ strategy.add(memberC)
+
+ val memberDInOtherDC = dcMember("otherDC", memberD)
+ val memberEInOtherDC = dcMember("otherDC", memberE)
+
+ "be noticed when records added" in {
+ strategy.setReachability(createReachability(List(memberA -> memberB)))
+ strategy.setReachability(createReachability(List(memberA -> memberB, memberA -> memberC))) should ===(true)
+ }
+
+ "be noticed when records removed" in {
+ strategy.setReachability(createReachability(List(memberA -> memberB, memberA -> memberC)))
+ strategy.setReachability(createReachability(List(memberA -> memberB))) should ===(true)
+ strategy.setReachability(Reachability.empty) should ===(true)
+ }
+
+ "be noticed when records change to Reachable" in {
+ val r = createReachability(List(memberA -> memberB, memberA -> memberC))
+ strategy.setReachability(r)
+ strategy.setReachability(r.reachable(memberA.uniqueAddress, memberC.uniqueAddress)) should ===(true)
+ }
+
+ "be noticed when records added and removed" in {
+ strategy.setReachability(createReachability(List(memberA -> memberB)))
+ strategy.setReachability(createReachability(List(memberC -> memberB))) should ===(true)
+ }
+
+ "be ignored when records for other DC added" in {
+ strategy.setReachability(createReachability(List(memberA -> memberB)))
+ strategy.setReachability(createReachability(List(memberA -> memberB, memberA -> memberDInOtherDC))) should ===(
+ false)
+ strategy.setReachability(createReachability(List(memberA -> memberB, memberDInOtherDC -> memberB))) should ===(
+ false)
+ strategy.setReachability(createReachability(List(memberA -> memberB, memberDInOtherDC -> memberEInOtherDC))) should ===(
+ false)
+ }
+ }
+
+}
diff --git a/akka-cluster/src/test/scala/akka/cluster/sbr/TestAddresses.scala b/akka-cluster/src/test/scala/akka/cluster/sbr/TestAddresses.scala
new file mode 100644
index 0000000000..3ac75dc830
--- /dev/null
+++ b/akka-cluster/src/test/scala/akka/cluster/sbr/TestAddresses.scala
@@ -0,0 +1,55 @@
+/*
+ * Copyright (C) 2016-2020 Lightbend Inc.
+ */
+
+package akka.cluster.sbr
+
+import akka.actor.Address
+import akka.cluster.ClusterSettings
+import akka.cluster.Member
+import akka.cluster.MemberStatus
+import akka.cluster.MemberStatus.Up
+import akka.cluster.MemberStatus.WeaklyUp
+import akka.cluster.UniqueAddress
+
+/**
+ * Needed since the Member constructor is akka private
+ */
+object TestAddresses {
+ private def dcRole(dc: ClusterSettings.DataCenter): String =
+ ClusterSettings.DcRolePrefix + dc
+ val defaultDataCenter = ClusterSettings.DefaultDataCenter
+ private def defaultDcRole = dcRole(defaultDataCenter)
+
+ val addressA = Address("akka.tcp", "sys", "a", 2552)
+ val memberA = new Member(UniqueAddress(addressA, 0L), 5, Up, Set("role3", defaultDcRole))
+ val memberB =
+ new Member(UniqueAddress(addressA.copy(host = Some("b")), 0L), 4, Up, Set("role1", "role3", defaultDcRole))
+ val memberC = new Member(UniqueAddress(addressA.copy(host = Some("c")), 0L), 3, Up, Set("role2", defaultDcRole))
+ val memberD =
+ new Member(UniqueAddress(addressA.copy(host = Some("d")), 0L), 2, Up, Set("role1", "role2", "role3", defaultDcRole))
+ val memberE = new Member(UniqueAddress(addressA.copy(host = Some("e")), 0L), 1, Up, Set(defaultDcRole))
+ val memberF = new Member(UniqueAddress(addressA.copy(host = Some("f")), 0L), 5, Up, Set(defaultDcRole))
+ val memberG = new Member(UniqueAddress(addressA.copy(host = Some("g")), 0L), 6, Up, Set(defaultDcRole))
+
+ val memberAWeaklyUp = new Member(memberA.uniqueAddress, Int.MaxValue, WeaklyUp, memberA.roles)
+ val memberBWeaklyUp = new Member(memberB.uniqueAddress, Int.MaxValue, WeaklyUp, memberB.roles)
+
+ def dcMember(dc: ClusterSettings.DataCenter, m: Member): Member =
+ new Member(
+ m.uniqueAddress,
+ m.upNumber,
+ m.status,
+ m.roles.filterNot(_.startsWith(ClusterSettings.DcRolePrefix)) + dcRole(dc))
+
+ def dataCenter(dc: ClusterSettings.DataCenter, members: Member*): Set[Member] =
+ members.toSet[Member].map(m => dcMember(dc, m))
+
+ def joining(m: Member): Member = Member(m.uniqueAddress, m.roles)
+
+ def leaving(m: Member): Member = m.copy(MemberStatus.Leaving)
+
+ def exiting(m: Member): Member = leaving(m).copy(MemberStatus.Exiting)
+
+ def downed(m: Member): Member = m.copy(MemberStatus.Down)
+}
diff --git a/akka-cluster/src/test/scala/akka/cluster/testkit/AutoDown.scala b/akka-cluster/src/test/scala/akka/cluster/testkit/AutoDown.scala
index 7d612cccea..92cbe34983 100644
--- a/akka-cluster/src/test/scala/akka/cluster/testkit/AutoDown.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/testkit/AutoDown.scala
@@ -155,7 +155,7 @@ private[cluster] abstract class AutoDownBase(autoDownUnreachableAfter: FiniteDur
var leader = false
override def postStop(): Unit = {
- scheduledUnreachable.values.foreach { _.cancel }
+ scheduledUnreachable.values.foreach { _.cancel() }
}
def receive = {
@@ -209,7 +209,7 @@ private[cluster] abstract class AutoDownBase(autoDownUnreachableAfter: FiniteDur
}
def remove(node: UniqueAddress): Unit = {
- scheduledUnreachable.get(node).foreach { _.cancel }
+ scheduledUnreachable.get(node).foreach { _.cancel() }
scheduledUnreachable -= node
pendingUnreachable -= node
}
diff --git a/akka-coordination/src/main/scala/akka/coordination/lease/LeaseUsageSettings.scala b/akka-coordination/src/main/scala/akka/coordination/lease/LeaseUsageSettings.scala
index 04d42dc940..23485c29f9 100644
--- a/akka-coordination/src/main/scala/akka/coordination/lease/LeaseUsageSettings.scala
+++ b/akka-coordination/src/main/scala/akka/coordination/lease/LeaseUsageSettings.scala
@@ -4,11 +4,11 @@
package akka.coordination.lease
+import scala.concurrent.duration.FiniteDuration
+
import akka.util.JavaDurationConverters._
import akka.util.PrettyDuration._
-import scala.concurrent.duration.FiniteDuration
-
final class LeaseUsageSettings private[akka] (val leaseImplementation: String, val leaseRetryInterval: FiniteDuration) {
def getLeaseRetryInterval(): java.time.Duration = leaseRetryInterval.asJava
diff --git a/akka-coordination/src/main/scala/akka/coordination/lease/TimeoutSettings.scala b/akka-coordination/src/main/scala/akka/coordination/lease/TimeoutSettings.scala
index 81b57642b7..16297c5d3e 100644
--- a/akka-coordination/src/main/scala/akka/coordination/lease/TimeoutSettings.scala
+++ b/akka-coordination/src/main/scala/akka/coordination/lease/TimeoutSettings.scala
@@ -4,10 +4,12 @@
package akka.coordination.lease
-import com.typesafe.config.{ Config, ConfigValueType }
-import akka.util.JavaDurationConverters._
import scala.concurrent.duration._
+import com.typesafe.config.{ Config, ConfigValueType }
+
+import akka.util.JavaDurationConverters._
+
object TimeoutSettings {
def apply(config: Config): TimeoutSettings = {
val heartBeatTimeout = config.getDuration("heartbeat-timeout").asScala
diff --git a/akka-coordination/src/main/scala/akka/coordination/lease/internal/LeaseAdapter.scala b/akka-coordination/src/main/scala/akka/coordination/lease/internal/LeaseAdapter.scala
index 2d4f4a7df1..02cc582f50 100644
--- a/akka-coordination/src/main/scala/akka/coordination/lease/internal/LeaseAdapter.scala
+++ b/akka-coordination/src/main/scala/akka/coordination/lease/internal/LeaseAdapter.scala
@@ -8,16 +8,16 @@ import java.util.Optional
import java.util.concurrent.CompletionStage
import java.util.function.Consumer
-import akka.annotation.InternalApi
-import akka.coordination.lease.LeaseSettings
-import akka.coordination.lease.scaladsl.{ Lease => ScalaLease }
-import akka.coordination.lease.javadsl.{ Lease => JavaLease }
-
import scala.compat.java8.FutureConverters._
import scala.compat.java8.OptionConverters._
import scala.concurrent.ExecutionContext
import scala.concurrent.Future
+import akka.annotation.InternalApi
+import akka.coordination.lease.LeaseSettings
+import akka.coordination.lease.javadsl.{ Lease => JavaLease }
+import akka.coordination.lease.scaladsl.{ Lease => ScalaLease }
+
/**
* INTERNAL API
*/
diff --git a/akka-coordination/src/main/scala/akka/coordination/lease/javadsl/LeaseProvider.scala b/akka-coordination/src/main/scala/akka/coordination/lease/javadsl/LeaseProvider.scala
index d774d00cf6..18029287f8 100644
--- a/akka-coordination/src/main/scala/akka/coordination/lease/javadsl/LeaseProvider.scala
+++ b/akka-coordination/src/main/scala/akka/coordination/lease/javadsl/LeaseProvider.scala
@@ -4,8 +4,8 @@
package akka.coordination.lease.javadsl
-import akka.actor.ClassicActorSystemProvider
import akka.actor.{ ActorSystem, ExtendedActorSystem, Extension, ExtensionId, ExtensionIdProvider }
+import akka.actor.ClassicActorSystemProvider
import akka.coordination.lease.internal.LeaseAdapter
import akka.coordination.lease.internal.LeaseAdapterToScala
import akka.coordination.lease.scaladsl.{ LeaseProvider => ScalaLeaseProvider }
diff --git a/akka-coordination/src/main/scala/akka/coordination/lease/scaladsl/Lease.scala b/akka-coordination/src/main/scala/akka/coordination/lease/scaladsl/Lease.scala
index c07b37ea24..d170b51d21 100644
--- a/akka-coordination/src/main/scala/akka/coordination/lease/scaladsl/Lease.scala
+++ b/akka-coordination/src/main/scala/akka/coordination/lease/scaladsl/Lease.scala
@@ -4,10 +4,10 @@
package akka.coordination.lease.scaladsl
-import akka.coordination.lease.LeaseSettings
-
import scala.concurrent.Future
+import akka.coordination.lease.LeaseSettings
+
abstract class Lease(val settings: LeaseSettings) {
/**
diff --git a/akka-coordination/src/main/scala/akka/coordination/lease/scaladsl/LeaseProvider.scala b/akka-coordination/src/main/scala/akka/coordination/lease/scaladsl/LeaseProvider.scala
index d9de061bec..45a5e5a51c 100644
--- a/akka-coordination/src/main/scala/akka/coordination/lease/scaladsl/LeaseProvider.scala
+++ b/akka-coordination/src/main/scala/akka/coordination/lease/scaladsl/LeaseProvider.scala
@@ -8,18 +8,18 @@ import java.util.concurrent.ConcurrentHashMap
import java.util.function.{ Function => JFunction }
import scala.collection.immutable
+import scala.reflect.ClassTag
import scala.util.{ Failure, Success, Try }
+
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
-import akka.event.Logging
import akka.coordination.lease.LeaseSettings
import akka.coordination.lease.internal.LeaseAdapterToScala
-
-import scala.reflect.ClassTag
+import akka.event.Logging
object LeaseProvider extends ExtensionId[LeaseProvider] with ExtensionIdProvider {
override def get(system: ActorSystem): LeaseProvider = super.get(system)
diff --git a/akka-cluster-tools/src/test/scala/akka/cluster/TestLease.scala b/akka-coordination/src/test/scala/akka/coordination/lease/TestLease.scala
similarity index 87%
rename from akka-cluster-tools/src/test/scala/akka/cluster/TestLease.scala
rename to akka-coordination/src/test/scala/akka/coordination/lease/TestLease.scala
index 681b135b97..bc9462524e 100644
--- a/akka-cluster-tools/src/test/scala/akka/cluster/TestLease.scala
+++ b/akka-coordination/src/test/scala/akka/coordination/lease/TestLease.scala
@@ -2,20 +2,25 @@
* Copyright (C) 2019-2020 Lightbend Inc.
*/
-package akka.cluster
+package akka.coordination.lease
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.atomic.AtomicReference
+import scala.concurrent.Future
+import scala.concurrent.Promise
+
+import com.typesafe.config.ConfigFactory
+
+import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
-import akka.actor.{ ActorSystem, ExtendedActorSystem, Extension, ExtensionId, ExtensionIdProvider }
-import akka.coordination.lease.LeaseSettings
+import akka.actor.ExtendedActorSystem
+import akka.actor.Extension
+import akka.actor.ExtensionId
+import akka.actor.ExtensionIdProvider
import akka.coordination.lease.scaladsl.Lease
import akka.event.Logging
import akka.testkit.TestProbe
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.{ Future, Promise }
import akka.util.ccompat.JavaConverters._
object TestLeaseExt extends ExtensionId[TestLeaseExt] with ExtensionIdProvider {
@@ -46,9 +51,9 @@ object TestLease {
final case class AcquireReq(owner: String)
final case class ReleaseReq(owner: String)
- val config = ConfigFactory.parseString("""
+ val config = ConfigFactory.parseString(s"""
test-lease {
- lease-class = akka.cluster.TestLease
+ lease-class = ${classOf[TestLease].getName}
}
""".stripMargin)
}
@@ -63,7 +68,7 @@ class TestLease(settings: LeaseSettings, system: ExtendedActorSystem) extends Le
TestLeaseExt(system).setTestLease(settings.leaseName, this)
- val initialPromise = Promise[Boolean]
+ val initialPromise = Promise[Boolean]()
private val nextAcquireResult = new AtomicReference[Future[Boolean]](initialPromise.future)
private val nextCheckLeaseResult = new AtomicReference[Boolean](false)
diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/TestLeaseActor.scala b/akka-coordination/src/test/scala/akka/coordination/lease/TestLeaseActor.scala
similarity index 93%
rename from akka-cluster-tools/src/multi-jvm/scala/akka/cluster/TestLeaseActor.scala
rename to akka-coordination/src/test/scala/akka/coordination/lease/TestLeaseActor.scala
index 76eacbd6a2..091027acac 100644
--- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/TestLeaseActor.scala
+++ b/akka-coordination/src/test/scala/akka/coordination/lease/TestLeaseActor.scala
@@ -2,12 +2,13 @@
* Copyright (C) 2019-2020 Lightbend Inc.
*/
-package akka.cluster
+package akka.coordination.lease
import java.util.concurrent.atomic.AtomicReference
import scala.concurrent.Future
import scala.concurrent.duration._
+
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorRef
@@ -18,10 +19,8 @@ import akka.actor.Extension
import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
import akka.actor.Props
-import akka.cluster.TestLeaseActor.{ Acquire, Create, Release }
-import akka.event.Logging
-import akka.coordination.lease.LeaseSettings
import akka.coordination.lease.scaladsl.Lease
+import akka.event.Logging
import akka.pattern.ask
import akka.testkit.JavaSerializable
import akka.util.Timeout
@@ -35,7 +34,7 @@ object TestLeaseActor {
final case class Release(owner: String) extends LeaseRequest
final case class Create(leaseName: String, ownerName: String) extends JavaSerializable
- final case object GetRequests extends JavaSerializable
+ case object GetRequests extends JavaSerializable
final case class LeaseRequests(requests: List[LeaseRequest]) extends JavaSerializable
final case class ActionRequest(request: LeaseRequest, result: Any) extends JavaSerializable // boolean of Failure
}
@@ -95,6 +94,9 @@ class TestLeaseActorClientExt(val system: ExtendedActorSystem) extends Extension
}
class TestLeaseActorClient(settings: LeaseSettings, system: ExtendedActorSystem) extends Lease(settings) {
+ import TestLeaseActor.Acquire
+ import TestLeaseActor.Create
+ import TestLeaseActor.Release
private val log = Logging(system, getClass)
val leaseActor = TestLeaseActorClientExt(system).getLeaseActor()
@@ -102,7 +104,7 @@ class TestLeaseActorClient(settings: LeaseSettings, system: ExtendedActorSystem)
log.info("lease created {}", settings)
leaseActor ! Create(settings.leaseName, settings.ownerName)
- private implicit val timeout = Timeout(100.seconds)
+ private implicit val timeout: Timeout = Timeout(100.seconds)
override def acquire(): Future[Boolean] = {
(leaseActor ? Acquire(settings.ownerName)).mapTo[Boolean]
diff --git a/akka-coordination/src/test/scala/akka/coordination/lease/TimeoutSettingsSpec.scala b/akka-coordination/src/test/scala/akka/coordination/lease/TimeoutSettingsSpec.scala
index a76b8173a2..27b6409b69 100644
--- a/akka-coordination/src/test/scala/akka/coordination/lease/TimeoutSettingsSpec.scala
+++ b/akka-coordination/src/test/scala/akka/coordination/lease/TimeoutSettingsSpec.scala
@@ -4,9 +4,9 @@
package akka.coordination.lease
-import com.typesafe.config.ConfigFactory
-
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
diff --git a/akka-coordination/src/test/scala/akka/coordination/lease/scaladsl/LeaseProviderSpec.scala b/akka-coordination/src/test/scala/akka/coordination/lease/scaladsl/LeaseProviderSpec.scala
index 0a7d391288..19ee98c2a3 100644
--- a/akka-coordination/src/test/scala/akka/coordination/lease/scaladsl/LeaseProviderSpec.scala
+++ b/akka-coordination/src/test/scala/akka/coordination/lease/scaladsl/LeaseProviderSpec.scala
@@ -5,12 +5,13 @@
package akka.coordination.lease.scaladsl
import scala.concurrent.Future
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ExtendedActorSystem
import akka.coordination.lease.LeaseSettings
import akka.testkit.{ AkkaSpec, EventFilter }
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.duration._
object LeaseProviderSpec {
class LeaseA(settings: LeaseSettings) extends Lease(settings) {
diff --git a/akka-discovery/src/main/scala/akka/discovery/Discovery.scala b/akka-discovery/src/main/scala/akka/discovery/Discovery.scala
index 380a0c6f61..822d6600b9 100644
--- a/akka-discovery/src/main/scala/akka/discovery/Discovery.scala
+++ b/akka-discovery/src/main/scala/akka/discovery/Discovery.scala
@@ -7,11 +7,11 @@ package akka.discovery
import java.util.concurrent.ConcurrentHashMap
import java.util.function.{ Function => JFunction }
+import scala.util.{ Failure, Success, Try }
+
import akka.actor._
import akka.annotation.InternalApi
-import scala.util.{ Failure, Success, Try }
-
final class Discovery(implicit system: ExtendedActorSystem) extends Extension {
Discovery.checkClassPathForOldDiscovery(system)
diff --git a/akka-discovery/src/main/scala/akka/discovery/ServiceDiscovery.scala b/akka-discovery/src/main/scala/akka/discovery/ServiceDiscovery.scala
index 9dc955f6a4..96557bee21 100644
--- a/akka-discovery/src/main/scala/akka/discovery/ServiceDiscovery.scala
+++ b/akka-discovery/src/main/scala/akka/discovery/ServiceDiscovery.scala
@@ -13,6 +13,7 @@ import scala.collection.immutable
import scala.compat.java8.OptionConverters._
import scala.concurrent.Future
import scala.concurrent.duration.FiniteDuration
+
import akka.actor.{ DeadLetterSuppression, NoSerializationVerificationNeeded }
import akka.util.HashCode
diff --git a/akka-discovery/src/main/scala/akka/discovery/aggregate/AggregateServiceDiscovery.scala b/akka-discovery/src/main/scala/akka/discovery/aggregate/AggregateServiceDiscovery.scala
index 94fcd8cf92..ced8bb91a1 100644
--- a/akka-discovery/src/main/scala/akka/discovery/aggregate/AggregateServiceDiscovery.scala
+++ b/akka-discovery/src/main/scala/akka/discovery/aggregate/AggregateServiceDiscovery.scala
@@ -4,20 +4,22 @@
package akka.discovery.aggregate
-import akka.actor.ExtendedActorSystem
-import akka.annotation.InternalApi
-import akka.discovery.ServiceDiscovery.Resolved
-import akka.discovery.aggregate.AggregateServiceDiscovery.Methods
-import akka.discovery.{ Discovery, Lookup, ServiceDiscovery }
-import akka.event.Logging
-import akka.util.Helpers.Requiring
-import com.typesafe.config.Config
-
-import akka.util.ccompat.JavaConverters._
import scala.concurrent.Future
import scala.concurrent.duration.FiniteDuration
import scala.util.control.NonFatal
+import com.typesafe.config.Config
+
+import akka.actor.ExtendedActorSystem
+import akka.annotation.InternalApi
+import akka.discovery.{ Discovery, Lookup, ServiceDiscovery }
+import akka.discovery.ServiceDiscovery.Resolved
+import akka.discovery.aggregate.AggregateServiceDiscovery.Methods
+import akka.dispatch.MessageDispatcher
+import akka.event.Logging
+import akka.util.Helpers.Requiring
+import akka.util.ccompat.JavaConverters._
+
/**
* INTERNAL API
*/
@@ -55,7 +57,7 @@ private[akka] final class AggregateServiceDiscovery(system: ExtendedActorSystem)
val serviceDiscovery = Discovery(system)
settings.discoveryMethods.map(mech => (mech, serviceDiscovery.loadServiceDiscovery(mech)))
}
- private implicit val ec = system.dispatchers.internalDispatcher
+ private implicit val ec: MessageDispatcher = system.dispatchers.internalDispatcher
/**
* Each discovery method is given the resolveTimeout rather than reducing it each time between methods.
diff --git a/akka-discovery/src/main/scala/akka/discovery/config/ConfigServiceDiscovery.scala b/akka-discovery/src/main/scala/akka/discovery/config/ConfigServiceDiscovery.scala
index 80e03e6ced..caaf2e84cf 100644
--- a/akka-discovery/src/main/scala/akka/discovery/config/ConfigServiceDiscovery.scala
+++ b/akka-discovery/src/main/scala/akka/discovery/config/ConfigServiceDiscovery.scala
@@ -4,16 +4,17 @@
package akka.discovery.config
+import scala.concurrent.Future
+import scala.concurrent.duration.FiniteDuration
+
+import com.typesafe.config.Config
+
import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi
import akka.discovery.{ Lookup, ServiceDiscovery }
import akka.discovery.ServiceDiscovery.{ Resolved, ResolvedTarget }
import akka.event.Logging
-import com.typesafe.config.Config
-
import akka.util.ccompat.JavaConverters._
-import scala.concurrent.Future
-import scala.concurrent.duration.FiniteDuration
/**
* INTERNAL API
diff --git a/akka-discovery/src/main/scala/akka/discovery/dns/DnsServiceDiscovery.scala b/akka-discovery/src/main/scala/akka/discovery/dns/DnsServiceDiscovery.scala
index 2ff2d20406..472a7b1462 100644
--- a/akka-discovery/src/main/scala/akka/discovery/dns/DnsServiceDiscovery.scala
+++ b/akka-discovery/src/main/scala/akka/discovery/dns/DnsServiceDiscovery.scala
@@ -6,28 +6,27 @@ package akka.discovery.dns
import java.net.InetAddress
-import akka.actor.ActorRef
-
-import scala.concurrent.duration._
-import akka.actor.ExtendedActorSystem
-import akka.annotation.InternalApi
-import akka.discovery.ServiceDiscovery.{ Resolved, ResolvedTarget }
-import akka.event.Logging
-import akka.io.{ Dns, IO }
-import akka.pattern.ask
-
-import scala.concurrent.Future
-import scala.concurrent.duration.FiniteDuration
-import akka.discovery._
-import akka.io.SimpleDnsCache
-import akka.io.dns.DnsProtocol.{ Ip, Srv }
-import akka.io.dns.{ AAAARecord, ARecord, DnsProtocol, SRVRecord }
-
import scala.collection.{ immutable => im }
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.concurrent.duration.FiniteDuration
import scala.util.Failure
import scala.util.Success
+
+import akka.actor.ActorRef
+import akka.actor.ExtendedActorSystem
+import akka.annotation.InternalApi
+import akka.discovery._
+import akka.discovery.ServiceDiscovery.{ Resolved, ResolvedTarget }
+import akka.dispatch.MessageDispatcher
+import akka.event.Logging
+import akka.io.{ Dns, IO }
+import akka.io.SimpleDnsCache
+import akka.io.dns.{ AAAARecord, ARecord, DnsProtocol, SRVRecord }
+import akka.io.dns.DnsProtocol.{ Ip, Srv }
import akka.io.dns.internal.AsyncDnsManager
import akka.pattern.AskTimeoutException
+import akka.pattern.ask
import akka.util.OptionVal
import akka.util.Timeout
@@ -70,7 +69,6 @@ private object DnsServiceDiscovery {
private[akka] class DnsServiceDiscovery(system: ExtendedActorSystem) extends ServiceDiscovery {
import DnsServiceDiscovery._
-
import ServiceDiscovery._
private val log = Logging(system, getClass)
@@ -91,7 +89,7 @@ private[akka] class DnsServiceDiscovery(system: ExtendedActorSystem) extends Ser
// (eventually visible)
private var asyncDnsCache: OptionVal[SimpleDnsCache] = OptionVal.None
- private implicit val ec = system.dispatchers.internalDispatcher
+ private implicit val ec: MessageDispatcher = system.dispatchers.internalDispatcher
dns.ask(AsyncDnsManager.GetCache)(Timeout(30.seconds)).onComplete {
case Success(cache: SimpleDnsCache) =>
diff --git a/akka-discovery/src/test/scala/akka/discovery/DiscoveryConfigurationSpec.scala b/akka-discovery/src/test/scala/akka/discovery/DiscoveryConfigurationSpec.scala
index 64aa1e393f..4f17e084b1 100644
--- a/akka-discovery/src/test/scala/akka/discovery/DiscoveryConfigurationSpec.scala
+++ b/akka-discovery/src/test/scala/akka/discovery/DiscoveryConfigurationSpec.scala
@@ -6,13 +6,15 @@ package akka.discovery
import scala.concurrent.Future
import scala.concurrent.duration.FiniteDuration
-import akka.actor.ActorSystem
-import akka.discovery.ServiceDiscovery.Resolved
-import akka.testkit.TestKit
+
import com.typesafe.config.ConfigFactory
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.ActorSystem
+import akka.discovery.ServiceDiscovery.Resolved
+import akka.testkit.TestKit
+
class DiscoveryConfigurationSpec extends AnyWordSpec with Matchers {
"ServiceDiscovery" should {
diff --git a/akka-discovery/src/test/scala/akka/discovery/aggregate/AggregateServiceDiscoverySpec.scala b/akka-discovery/src/test/scala/akka/discovery/aggregate/AggregateServiceDiscoverySpec.scala
index 7b7d4e73c0..1e07ca4428 100644
--- a/akka-discovery/src/test/scala/akka/discovery/aggregate/AggregateServiceDiscoverySpec.scala
+++ b/akka-discovery/src/test/scala/akka/discovery/aggregate/AggregateServiceDiscoverySpec.scala
@@ -4,21 +4,22 @@
package akka.discovery.aggregate
-import akka.actor.{ ActorSystem, ExtendedActorSystem }
-import akka.discovery.ServiceDiscovery.{ Resolved, ResolvedTarget }
-import akka.discovery.{ Discovery, Lookup, ServiceDiscovery }
-import akka.testkit.TestKit
-import akka.util.unused
-import com.typesafe.config.{ Config, ConfigFactory }
-import org.scalatest.concurrent.ScalaFutures
-import org.scalatest.BeforeAndAfterAll
-
+import scala.collection.immutable
import scala.concurrent.Future
import scala.concurrent.duration._
-import scala.collection.immutable
+
+import com.typesafe.config.{ Config, ConfigFactory }
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.concurrent.ScalaFutures
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
+import akka.actor.{ ActorSystem, ExtendedActorSystem }
+import akka.discovery.{ Discovery, Lookup, ServiceDiscovery }
+import akka.discovery.ServiceDiscovery.{ Resolved, ResolvedTarget }
+import akka.testkit.TestKit
+import akka.util.unused
+
class StubbedServiceDiscovery(@unused system: ExtendedActorSystem) extends ServiceDiscovery {
override def lookup(query: Lookup, resolveTimeout: FiniteDuration): Future[Resolved] = {
diff --git a/akka-discovery/src/test/scala/akka/discovery/config/ConfigServiceDiscoverySpec.scala b/akka-discovery/src/test/scala/akka/discovery/config/ConfigServiceDiscoverySpec.scala
index ea9ae725ce..cc12c5c256 100644
--- a/akka-discovery/src/test/scala/akka/discovery/config/ConfigServiceDiscoverySpec.scala
+++ b/akka-discovery/src/test/scala/akka/discovery/config/ConfigServiceDiscoverySpec.scala
@@ -4,18 +4,19 @@
package akka.discovery.config
+import scala.collection.immutable
+import scala.concurrent.duration._
+
+import com.typesafe.config.{ Config, ConfigFactory }
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.concurrent.ScalaFutures
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.ActorSystem
import akka.discovery.Discovery
import akka.discovery.ServiceDiscovery.ResolvedTarget
import akka.testkit.TestKit
-import com.typesafe.config.{ Config, ConfigFactory }
-import org.scalatest.concurrent.ScalaFutures
-import org.scalatest.BeforeAndAfterAll
-
-import scala.concurrent.duration._
-import scala.collection.immutable
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpecLike
object ConfigServiceDiscoverySpec {
diff --git a/akka-discovery/src/test/scala/akka/discovery/config/ConfigServicesParserSpec.scala b/akka-discovery/src/test/scala/akka/discovery/config/ConfigServicesParserSpec.scala
index e7560e1010..1fcb83d966 100644
--- a/akka-discovery/src/test/scala/akka/discovery/config/ConfigServicesParserSpec.scala
+++ b/akka-discovery/src/test/scala/akka/discovery/config/ConfigServicesParserSpec.scala
@@ -4,14 +4,15 @@
package akka.discovery.config
-import akka.discovery.ServiceDiscovery.{ Resolved, ResolvedTarget }
-import akka.discovery.config.ConfigServicesParserSpec._
-import com.typesafe.config.{ Config, ConfigFactory }
-
import scala.collection.immutable
+
+import com.typesafe.config.{ Config, ConfigFactory }
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.discovery.ServiceDiscovery.{ Resolved, ResolvedTarget }
+import akka.discovery.config.ConfigServicesParserSpec._
+
object ConfigServicesParserSpec {
val exampleConfig: Config = ConfigFactory.parseString("""
services {
diff --git a/akka-discovery/src/test/scala/akka/discovery/dns/DnsDiscoverySpec.scala b/akka-discovery/src/test/scala/akka/discovery/dns/DnsDiscoverySpec.scala
index 8eacff8f29..d497d42931 100644
--- a/akka-discovery/src/test/scala/akka/discovery/dns/DnsDiscoverySpec.scala
+++ b/akka-discovery/src/test/scala/akka/discovery/dns/DnsDiscoverySpec.scala
@@ -6,15 +6,16 @@ package akka.discovery.dns
import java.net.InetAddress
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ActorSystem
import akka.discovery.{ Discovery, Lookup }
+import akka.discovery.ServiceDiscovery
import akka.discovery.ServiceDiscovery.ResolvedTarget
import akka.io.dns.DockerBindDnsService
import akka.testkit.{ AkkaSpec, SocketUtil, TestKit }
-import com.typesafe.config.ConfigFactory
-import scala.concurrent.duration._
-
-import akka.discovery.ServiceDiscovery
object DnsDiscoverySpec {
diff --git a/akka-discovery/src/test/scala/akka/discovery/dns/DnsServiceDiscoverySpec.scala b/akka-discovery/src/test/scala/akka/discovery/dns/DnsServiceDiscoverySpec.scala
index 0e4d2ff9d1..0695e8f4f4 100644
--- a/akka-discovery/src/test/scala/akka/discovery/dns/DnsServiceDiscoverySpec.scala
+++ b/akka-discovery/src/test/scala/akka/discovery/dns/DnsServiceDiscoverySpec.scala
@@ -6,22 +6,23 @@ package akka.discovery.dns
import java.net.{ Inet6Address, InetAddress }
+import scala.collection.{ immutable => im }
+import scala.concurrent.duration._
+
+import org.scalatest.concurrent.ScalaFutures
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.ActorRef
import akka.actor.ExtendedActorSystem
import akka.discovery
import akka.discovery.ServiceDiscovery
-import akka.discovery.ServiceDiscovery.DiscoveryTimeoutException
import akka.discovery.ServiceDiscovery.{ Resolved, ResolvedTarget }
-import akka.io.dns.CachePolicy.Ttl
+import akka.discovery.ServiceDiscovery.DiscoveryTimeoutException
import akka.io.dns.{ AAAARecord, ARecord, DnsProtocol, SRVRecord }
+import akka.io.dns.CachePolicy.Ttl
import akka.testkit.AkkaSpec
import akka.testkit.TestProbe
-import org.scalatest.wordspec.AnyWordSpecLike
-import org.scalatest.concurrent.ScalaFutures
-import org.scalatest.matchers.should.Matchers
-
-import scala.collection.{ immutable => im }
-import scala.concurrent.duration._
class DnsServiceDiscoverySpec extends AkkaSpec with AnyWordSpecLike with Matchers with ScalaFutures {
diff --git a/akka-distributed-data/src/main/mima-filters/2.6.5.backwards.excludes/pr-29041-immutable-cancellable.excludes b/akka-distributed-data/src/main/mima-filters/2.6.5.backwards.excludes/pr-29041-immutable-cancellable.excludes
new file mode 100644
index 0000000000..ffa12fe001
--- /dev/null
+++ b/akka-distributed-data/src/main/mima-filters/2.6.5.backwards.excludes/pr-29041-immutable-cancellable.excludes
@@ -0,0 +1,6 @@
+# Change internal methods of the Replicator actor
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.ReadWriteAggregator.sendToSecondarySchedule")
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.ReadWriteAggregator.sendToSecondarySchedule_=")
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.ReadWriteAggregator.timeoutSchedule")
+ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.ReadWriteAggregator.timeoutSchedule_=")
+
diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/DurableStore.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/DurableStore.scala
index c696a16024..4b38fea365 100644
--- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/DurableStore.scala
+++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/DurableStore.scala
@@ -8,11 +8,17 @@ import java.io.File
import java.nio.ByteBuffer
import java.util.concurrent.TimeUnit
-import akka.util.ccompat.JavaConverters._
import scala.concurrent.duration._
import scala.util.Try
import scala.util.control.NonFatal
+import com.typesafe.config.Config
+import org.lmdbjava.Dbi
+import org.lmdbjava.DbiFlags
+import org.lmdbjava.Env
+import org.lmdbjava.EnvFlags
+import org.lmdbjava.Txn
+
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorRef
@@ -27,12 +33,7 @@ import akka.serialization.SerializationExtension
import akka.serialization.SerializerWithStringManifest
import akka.util.ByteString
import akka.util.OptionVal
-import com.typesafe.config.Config
-import org.lmdbjava.DbiFlags
-import org.lmdbjava.Env
-import org.lmdbjava.EnvFlags
-import org.lmdbjava.Txn
-import org.lmdbjava.Dbi
+import akka.util.ccompat.JavaConverters._
/**
* An actor implementing the durable store for the Distributed Data `Replicator`
@@ -112,8 +113,8 @@ object LmdbDurableStore {
final class LmdbDurableStore(config: Config) extends Actor with ActorLogging {
import DurableStore._
- import LmdbDurableStore.WriteBehind
import LmdbDurableStore.Lmdb
+ import LmdbDurableStore.WriteBehind
val serialization = SerializationExtension(context.system)
val serializer = serialization.serializerFor(classOf[DurableDataEnvelope]).asInstanceOf[SerializerWithStringManifest]
@@ -166,7 +167,7 @@ final class LmdbDurableStore(config: Config) extends Actor with ActorLogging {
val valueBuffer = lmdb().valueBuffer
if (valueBuffer.remaining < size) {
DirectByteBufferPool.tryCleanDirectByteBuffer(valueBuffer)
- _lmdb = OptionVal.Some(lmdb.copy(valueBuffer = ByteBuffer.allocateDirect(size * 2)))
+ _lmdb = OptionVal.Some(lmdb().copy(valueBuffer = ByteBuffer.allocateDirect(size * 2)))
}
}
diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/GCounter.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/GCounter.scala
index 016b79b29d..e242c585a8 100644
--- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/GCounter.scala
+++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/GCounter.scala
@@ -4,10 +4,11 @@
package akka.cluster.ddata
+import java.math.BigInteger
+
+import akka.annotation.InternalApi
import akka.cluster.Cluster
import akka.cluster.UniqueAddress
-import java.math.BigInteger
-import akka.annotation.InternalApi
object GCounter {
val empty: GCounter = new GCounter
diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/ORMap.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/ORMap.scala
index 7da7fd803b..4a5b7b1225 100644
--- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/ORMap.scala
+++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/ORMap.scala
@@ -4,13 +4,13 @@
package akka.cluster.ddata
+import scala.collection.immutable
+
+import akka.annotation.InternalApi
import akka.cluster.Cluster
import akka.cluster.UniqueAddress
-import akka.util.HashCode
-import akka.annotation.InternalApi
import akka.cluster.ddata.ORMap.ZeroTag
-
-import scala.collection.immutable
+import akka.util.HashCode
object ORMap {
private val _empty: ORMap[Any, ReplicatedData] = new ORMap(ORSet.empty, Map.empty, VanillaORMapTag)
diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/ORMultiMap.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/ORMultiMap.scala
index deb50d6ec6..da6875fb90 100644
--- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/ORMultiMap.scala
+++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/ORMultiMap.scala
@@ -5,8 +5,8 @@
package akka.cluster.ddata
import akka.annotation.InternalApi
-import akka.cluster.ddata.ORMap._
import akka.cluster.{ Cluster, UniqueAddress }
+import akka.cluster.ddata.ORMap._
object ORMultiMap {
diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/ORSet.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/ORSet.scala
index a67ac0986f..affe898569 100644
--- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/ORSet.scala
+++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/ORSet.scala
@@ -6,10 +6,11 @@ package akka.cluster.ddata
import scala.annotation.tailrec
import scala.collection.immutable
+
+import akka.annotation.InternalApi
import akka.cluster.Cluster
import akka.cluster.UniqueAddress
import akka.util.{ unused, HashCode }
-import akka.annotation.InternalApi
object ORSet {
private val _empty: ORSet[Any] = new ORSet(Map.empty, VersionVector.empty)
diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/PNCounter.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/PNCounter.scala
index 84965f82d6..dc93dd59a9 100644
--- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/PNCounter.scala
+++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/PNCounter.scala
@@ -4,12 +4,12 @@
package akka.cluster.ddata
-import akka.cluster.Cluster
-import akka.cluster.UniqueAddress
-import akka.util.HashCode
import java.math.BigInteger
import akka.annotation.InternalApi
+import akka.cluster.Cluster
+import akka.cluster.UniqueAddress
+import akka.util.HashCode
object PNCounter {
val empty: PNCounter = new PNCounter(GCounter.empty, GCounter.empty)
diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/PruningState.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/PruningState.scala
index 506ff437ba..60e8904517 100644
--- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/PruningState.scala
+++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/PruningState.scala
@@ -5,9 +5,9 @@
package akka.cluster.ddata
import akka.actor.Address
+import akka.annotation.InternalApi
import akka.cluster.Member
import akka.cluster.UniqueAddress
-import akka.annotation.InternalApi
import akka.util.unused
/**
diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/ReplicatedData.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/ReplicatedData.scala
index 3a2593f44e..503df856d4 100644
--- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/ReplicatedData.scala
+++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/ReplicatedData.scala
@@ -4,10 +4,12 @@
package akka.cluster.ddata
-import akka.cluster.UniqueAddress
-import scala.compat.java8.OptionConverters._
import java.util.Optional
+import scala.compat.java8.OptionConverters._
+
+import akka.cluster.UniqueAddress
+
/**
* Interface for implementing a state based convergent
* replicated data type (CvRDT).
diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/Replicator.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/Replicator.scala
index 6ad23b52bd..8cbfa04501 100644
--- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/Replicator.scala
+++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/Replicator.scala
@@ -5,66 +5,60 @@
package akka.cluster.ddata
import java.security.MessageDigest
+import java.util.Optional
+import java.util.concurrent.ThreadLocalRandom
+import java.util.concurrent.TimeUnit
+import java.util.function.{ Function => JFunction }
+import scala.annotation.varargs
import scala.collection.immutable
+import scala.collection.immutable.TreeSet
import scala.collection.mutable
import scala.concurrent.duration._
import scala.concurrent.duration.FiniteDuration
-import java.util.concurrent.ThreadLocalRandom
-
import scala.util.Failure
import scala.util.Success
import scala.util.Try
import scala.util.control.NoStackTrace
+import scala.util.control.NonFatal
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
import akka.actor.Actor
+import akka.actor.ActorInitializationException
import akka.actor.ActorLogging
import akka.actor.ActorRef
import akka.actor.ActorSelection
import akka.actor.ActorSystem
import akka.actor.Address
-import akka.actor.NoSerializationVerificationNeeded
+import akka.actor.Cancellable
+import akka.actor.DeadLetterSuppression
import akka.actor.Deploy
+import akka.actor.ExtendedActorSystem
+import akka.actor.NoSerializationVerificationNeeded
+import akka.actor.OneForOneStrategy
import akka.actor.Props
import akka.actor.ReceiveTimeout
+import akka.actor.SupervisorStrategy
import akka.actor.Terminated
+import akka.annotation.InternalApi
import akka.cluster.Cluster
import akka.cluster.ClusterEvent._
import akka.cluster.ClusterEvent.InitialStateAsEvents
import akka.cluster.Member
+import akka.cluster.MemberStatus
import akka.cluster.UniqueAddress
+import akka.cluster.ddata.DurableStore._
+import akka.cluster.ddata.Key.KeyId
+import akka.cluster.ddata.Key.KeyR
+import akka.dispatch.Dispatchers
+import akka.event.Logging
import akka.serialization.SerializationExtension
import akka.util.ByteString
-import com.typesafe.config.Config
-import java.util.function.{ Function => JFunction }
-
-import akka.dispatch.Dispatchers
-import akka.actor.DeadLetterSuppression
-import akka.cluster.ddata.Key.KeyR
-import java.util.Optional
-
-import akka.cluster.ddata.DurableStore._
-import akka.actor.ExtendedActorSystem
-import akka.actor.SupervisorStrategy
-import akka.actor.OneForOneStrategy
-import akka.actor.ActorInitializationException
-import java.util.concurrent.TimeUnit
-
import akka.util.Helpers.toRootLowerCase
-import akka.actor.Cancellable
-import scala.util.control.NonFatal
-
-import akka.cluster.ddata.Key.KeyId
-import akka.annotation.InternalApi
-import scala.collection.immutable.TreeSet
-
-import akka.cluster.MemberStatus
-import scala.annotation.varargs
-
-import akka.event.Logging
import akka.util.JavaDurationConverters._
import akka.util.ccompat._
-import com.github.ghik.silencer.silent
@ccompatUsedUntil213
object ReplicatorSettings {
@@ -845,7 +839,7 @@ object Replicator {
* Get current number of replicas, including the local replica.
* Will reply to sender with [[ReplicaCount]].
*/
- final case object GetReplicaCount
+ case object GetReplicaCount
/**
* Java API: The `GetReplicaCount` instance
@@ -1301,10 +1295,10 @@ object Replicator {
*/
final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLogging {
+ import PruningState._
import Replicator._
import Replicator.Internal._
import Replicator.Internal.DeltaPropagation.NoDeltaPlaceholder
- import PruningState._
import settings._
val cluster = Cluster(context.system)
@@ -2375,8 +2369,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def shuffle: Boolean
import context.dispatcher
- var sendToSecondarySchedule = context.system.scheduler.scheduleOnce(timeout / 5, self, SendToSecondary)
- var timeoutSchedule = context.system.scheduler.scheduleOnce(timeout, self, ReceiveTimeout)
+ private val sendToSecondarySchedule = context.system.scheduler.scheduleOnce(timeout / 5, self, SendToSecondary)
+ private val timeoutSchedule = context.system.scheduler.scheduleOnce(timeout, self, ReceiveTimeout)
var remaining = nodes.iterator.map(_.address).toSet
@@ -2460,9 +2454,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
extends ReadWriteAggregator
with ActorLogging {
+ import ReadWriteAggregator._
import Replicator._
import Replicator.Internal._
- import ReadWriteAggregator._
override def timeout: FiniteDuration = consistency.timeout
@@ -2605,9 +2599,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
extends ReadWriteAggregator
with ActorLogging {
+ import ReadWriteAggregator._
import Replicator._
import Replicator.Internal._
- import ReadWriteAggregator._
override def timeout: FiniteDuration = consistency.timeout
diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/VersionVector.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/VersionVector.scala
index dde0e94bed..75466e1a8f 100644
--- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/VersionVector.scala
+++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/VersionVector.scala
@@ -5,11 +5,13 @@
package akka.cluster.ddata
import java.util.concurrent.atomic.AtomicLong
+
import scala.annotation.tailrec
import scala.collection.immutable.TreeMap
+
+import akka.annotation.InternalApi
import akka.cluster.Cluster
import akka.cluster.UniqueAddress
-import akka.annotation.InternalApi
/**
* VersionVector module with helper classes and methods.
diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/ReplicatedDataSerializer.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/ReplicatedDataSerializer.scala
index e07c8bce9b..19da5d8a29 100644
--- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/ReplicatedDataSerializer.scala
+++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/ReplicatedDataSerializer.scala
@@ -5,32 +5,32 @@
package akka.cluster.ddata.protobuf
import java.{ util, lang => jl }
+import java.io.NotSerializableException
+import java.util
import java.util.ArrayList
import java.util.Collections
import java.util.Comparator
import scala.annotation.tailrec
-import akka.util.ccompat.JavaConverters._
-
import scala.collection.immutable
+
+import com.github.ghik.silencer.silent
+
+import akka.actor.ActorRef
import akka.actor.ExtendedActorSystem
import akka.cluster.ddata._
import akka.cluster.ddata.Replicator.Internal._
import akka.cluster.ddata.protobuf.msg.{ ReplicatedDataMessages => rd }
import akka.cluster.ddata.protobuf.msg.{ ReplicatorMessages => dm }
-import akka.serialization.SerializerWithStringManifest
-import akka.serialization.BaseSerializer
-import akka.protobufv3.internal.{ ByteString, GeneratedMessageV3 }
-import akka.util.ByteString.UTF_8
-import java.io.NotSerializableException
-import java.util
-
-import com.github.ghik.silencer.silent
-import akka.actor.ActorRef
import akka.cluster.ddata.protobuf.msg.ReplicatorMessages.OtherMessage
+import akka.protobufv3.internal.{ ByteString, GeneratedMessageV3 }
import akka.protobufv3.internal.GeneratedMessageV3
+import akka.serialization.BaseSerializer
import akka.serialization.Serialization
+import akka.serialization.SerializerWithStringManifest
+import akka.util.ByteString.UTF_8
import akka.util.ccompat._
+import akka.util.ccompat.JavaConverters._
@ccompatUsedUntil213
private object ReplicatedDataSerializer {
diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala
index 21ff8c5209..4b37f8ca16 100644
--- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala
+++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala
@@ -4,35 +4,37 @@
package akka.cluster.ddata.protobuf
-import scala.concurrent.duration._
+import java.io.NotSerializableException
import java.util.concurrent.TimeUnit
-import akka.util.ccompat.JavaConverters._
+import java.util.concurrent.atomic.AtomicInteger
+
+import scala.annotation.tailrec
import scala.collection.immutable
+import scala.concurrent.duration._
import scala.concurrent.duration.Duration
+import scala.concurrent.duration.FiniteDuration
+
+import akka.actor.Address
import akka.actor.ExtendedActorSystem
+import akka.annotation.InternalApi
import akka.cluster.Member
import akka.cluster.UniqueAddress
+import akka.cluster.ddata.DurableStore.DurableDataEnvelope
+import akka.cluster.ddata.Key.KeyR
import akka.cluster.ddata.PruningState
+import akka.cluster.ddata.PruningState.PruningPerformed
import akka.cluster.ddata.ReplicatedData
import akka.cluster.ddata.Replicator._
import akka.cluster.ddata.Replicator.Internal._
+import akka.cluster.ddata.VersionVector
import akka.cluster.ddata.protobuf.msg.{ ReplicatorMessages => dm }
+import akka.protobufv3.internal.ByteString
+import akka.serialization.BaseSerializer
import akka.serialization.Serialization
import akka.serialization.SerializerWithStringManifest
-import akka.serialization.BaseSerializer
import akka.util.{ ByteString => AkkaByteString }
-import akka.protobufv3.internal.ByteString
-import akka.cluster.ddata.Key.KeyR
-import java.util.concurrent.atomic.AtomicInteger
-import scala.annotation.tailrec
-import scala.concurrent.duration.FiniteDuration
-import akka.cluster.ddata.DurableStore.DurableDataEnvelope
-import java.io.NotSerializableException
-import akka.actor.Address
-import akka.cluster.ddata.VersionVector
-import akka.annotation.InternalApi
-import akka.cluster.ddata.PruningState.PruningPerformed
import akka.util.ccompat._
+import akka.util.ccompat.JavaConverters._
/**
* INTERNAL API
diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/SerializationSupport.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/SerializationSupport.scala
index e3e68f4bfa..e146e313fc 100644
--- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/SerializationSupport.scala
+++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/SerializationSupport.scala
@@ -8,19 +8,21 @@ import java.io.ByteArrayInputStream
import java.io.ByteArrayOutputStream
import java.util.zip.GZIPInputStream
import java.util.zip.GZIPOutputStream
+
import scala.annotation.tailrec
import scala.collection.immutable.TreeMap
-import akka.util.ccompat.JavaConverters._
+
import akka.actor.ActorRef
import akka.actor.Address
import akka.actor.ExtendedActorSystem
import akka.cluster.UniqueAddress
+import akka.cluster.ddata.VersionVector
import akka.cluster.ddata.protobuf.msg.{ ReplicatorMessages => dm }
-import akka.serialization._
import akka.protobufv3.internal.ByteString
import akka.protobufv3.internal.MessageLite
-import akka.cluster.ddata.VersionVector
+import akka.serialization._
import akka.util.ccompat._
+import akka.util.ccompat.JavaConverters._
/**
* Some useful serialization helper methods.
diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/DurableDataSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/DurableDataSpec.scala
index 13cdc917d3..cf2b2d34db 100644
--- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/DurableDataSpec.scala
+++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/DurableDataSpec.scala
@@ -7,6 +7,10 @@ package akka.cluster.ddata
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
+
+import com.typesafe.config.ConfigFactory
+import org.scalatest.CancelAfterFailure
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.ActorSystem
@@ -16,8 +20,6 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-import org.scalatest.CancelAfterFailure
final case class DurableDataSpecConfig(writeBehind: Boolean) extends MultiNodeConfig {
val first = role("first")
@@ -84,7 +86,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
override def initialParticipants = roles.size
val cluster = Cluster(system)
- implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
+ implicit val selfUniqueAddress: SelfUniqueAddress = DistributedData(system).selfUniqueAddress
val timeout = 14.seconds.dilated // initialization of lmdb can be very slow in CI environment
val writeTwo = WriteTo(2, timeout)
val readTwo = ReadFrom(2, timeout)
diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/DurablePruningSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/DurablePruningSpec.scala
index dbff51aeae..730df3fd4e 100644
--- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/DurablePruningSpec.scala
+++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/DurablePruningSpec.scala
@@ -4,18 +4,19 @@
package akka.cluster.ddata
+import scala.concurrent.Await
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
+import akka.actor.ActorRef
+import akka.actor.ActorSystem
import akka.cluster.Cluster
+import akka.cluster.MemberStatus
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-import akka.actor.ActorSystem
-import akka.actor.ActorRef
-import scala.concurrent.Await
-import akka.cluster.MemberStatus
import akka.util.ccompat._
@ccompatUsedUntil213
@@ -46,7 +47,7 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN
override def initialParticipants = roles.size
val cluster = Cluster(system)
- implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
+ implicit val selfUniqueAddress: SelfUniqueAddress = DistributedData(system).selfUniqueAddress
val maxPruningDissemination = 3.seconds
def startReplicator(sys: ActorSystem): ActorRef =
diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/JepsenInspiredInsertSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/JepsenInspiredInsertSpec.scala
index a4db728c6b..6e8d0cc008 100644
--- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/JepsenInspiredInsertSpec.scala
+++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/JepsenInspiredInsertSpec.scala
@@ -4,16 +4,18 @@
package akka.cluster.ddata
-import scala.concurrent.duration._
import java.util.concurrent.ThreadLocalRandom
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.cluster.Cluster
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._
-import com.typesafe.config.ConfigFactory
object JepsenInspiredInsertSpec extends MultiNodeConfig {
val controller = role("controller")
@@ -53,7 +55,7 @@ class JepsenInspiredInsertSpec
override def initialParticipants = roles.size
val cluster = Cluster(system)
- implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
+ implicit val selfUniqueAddress: SelfUniqueAddress = DistributedData(system).selfUniqueAddress
val replicator = DistributedData(system).replicator
val nodes = roles.drop(1) // controller not part of active nodes
val nodeCount = nodes.size
diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/PerformanceSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/PerformanceSpec.scala
index 47036ba068..1337b699a6 100644
--- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/PerformanceSpec.scala
+++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/PerformanceSpec.scala
@@ -6,6 +6,9 @@ package akka.cluster.ddata
import scala.concurrent.Await
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Deploy
@@ -15,7 +18,6 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import com.typesafe.config.ConfigFactory
object PerformanceSpec extends MultiNodeConfig {
val n1 = role("n1")
@@ -69,7 +71,7 @@ class PerformanceSpec extends MultiNodeSpec(PerformanceSpec) with STMultiNodeSpe
override def initialParticipants = roles.size
val cluster = Cluster(system)
- implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
+ implicit val selfUniqueAddress: SelfUniqueAddress = DistributedData(system).selfUniqueAddress
val replicator = DistributedData(system).replicator
val timeout = 3.seconds.dilated
val factor = 1 // use 3 here for serious tuning
diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorChaosSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorChaosSpec.scala
index 3263eb751b..88fffacd60 100644
--- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorChaosSpec.scala
+++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorChaosSpec.scala
@@ -6,13 +6,14 @@ package akka.cluster.ddata
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.cluster.Cluster
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._
-import com.typesafe.config.ConfigFactory
object ReplicatorChaosSpec extends MultiNodeConfig {
val first = role("first")
@@ -38,13 +39,13 @@ class ReplicatorChaosSpecMultiJvmNode4 extends ReplicatorChaosSpec
class ReplicatorChaosSpecMultiJvmNode5 extends ReplicatorChaosSpec
class ReplicatorChaosSpec extends MultiNodeSpec(ReplicatorChaosSpec) with STMultiNodeSpec with ImplicitSender {
- import ReplicatorChaosSpec._
import Replicator._
+ import ReplicatorChaosSpec._
override def initialParticipants = roles.size
val cluster = Cluster(system)
- implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
+ implicit val selfUniqueAddress: SelfUniqueAddress = DistributedData(system).selfUniqueAddress
val replicator = system.actorOf(
Replicator.props(ReplicatorSettings(system).withRole("backend").withGossipInterval(1.second)),
"replicator")
diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorDeltaSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorDeltaSpec.scala
index 918c0c009b..4b2ebd7d1a 100644
--- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorDeltaSpec.scala
+++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorDeltaSpec.scala
@@ -8,6 +8,8 @@ import java.util.concurrent.ThreadLocalRandom
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.cluster.Cluster
import akka.cluster.ddata.Replicator._
import akka.remote.testconductor.RoleName
@@ -15,7 +17,6 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.serialization.jackson.CborSerializable
import akka.testkit._
-import com.typesafe.config.ConfigFactory
object ReplicatorDeltaSpec extends MultiNodeConfig {
val first = role("first")
@@ -146,7 +147,7 @@ class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMult
override def initialParticipants = roles.size
val cluster = Cluster(system)
- implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
+ implicit val selfUniqueAddress: SelfUniqueAddress = DistributedData(system).selfUniqueAddress
val fullStateReplicator = system.actorOf(
Replicator.props(ReplicatorSettings(system).withGossipInterval(1.second).withDeltaCrdtEnabled(false)),
"fullStateReplicator")
diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorMapDeltaSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorMapDeltaSpec.scala
index d5da64e92b..4991522558 100644
--- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorMapDeltaSpec.scala
+++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorMapDeltaSpec.scala
@@ -7,14 +7,16 @@ package akka.cluster.ddata
import java.util.concurrent.ThreadLocalRandom
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.cluster.Cluster
import akka.cluster.ddata.Replicator._
+import akka.event.Logging.Error
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-import akka.event.Logging.Error
object ReplicatorMapDeltaSpec extends MultiNodeConfig {
val first = role("first")
@@ -186,7 +188,7 @@ class ReplicatorMapDeltaSpec extends MultiNodeSpec(ReplicatorMapDeltaSpec) with
override def initialParticipants = roles.size
val cluster = Cluster(system)
- implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
+ implicit val selfUniqueAddress: SelfUniqueAddress = DistributedData(system).selfUniqueAddress
val fullStateReplicator = system.actorOf(
Replicator.props(ReplicatorSettings(system).withGossipInterval(1.second).withDeltaCrdtEnabled(false)),
"fullStateReplicator")
diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorORSetDeltaSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorORSetDeltaSpec.scala
index a531d123a0..83def61b72 100644
--- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorORSetDeltaSpec.scala
+++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorORSetDeltaSpec.scala
@@ -6,13 +6,14 @@ package akka.cluster.ddata
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.cluster.Cluster
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._
-import com.typesafe.config.ConfigFactory
object ReplicatorORSetDeltaSpec extends MultiNodeConfig {
val first = role("first")
@@ -42,7 +43,7 @@ class ReplicatorORSetDeltaSpec
override def initialParticipants = roles.size
val cluster = Cluster(system)
- implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
+ implicit val selfUniqueAddress: SelfUniqueAddress = DistributedData(system).selfUniqueAddress
val replicator =
system.actorOf(Replicator.props(ReplicatorSettings(system).withGossipInterval(1.second)), "replicator")
val timeout = 3.seconds.dilated
diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorPruningSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorPruningSpec.scala
index d40afe9b39..e0230ddccc 100644
--- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorPruningSpec.scala
+++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorPruningSpec.scala
@@ -6,6 +6,8 @@ package akka.cluster.ddata
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.cluster.Cluster
import akka.cluster.ClusterEvent.InitialStateAsEvents
import akka.cluster.ClusterEvent.MemberUp
@@ -13,7 +15,6 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
-import com.typesafe.config.ConfigFactory
object ReplicatorPruningSpec extends MultiNodeConfig {
val first = role("first")
@@ -36,13 +37,13 @@ class ReplicatorPruningSpecMultiJvmNode2 extends ReplicatorPruningSpec
class ReplicatorPruningSpecMultiJvmNode3 extends ReplicatorPruningSpec
class ReplicatorPruningSpec extends MultiNodeSpec(ReplicatorPruningSpec) with STMultiNodeSpec with ImplicitSender {
- import ReplicatorPruningSpec._
import Replicator._
+ import ReplicatorPruningSpec._
override def initialParticipants = roles.size
val cluster = Cluster(system)
- implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
+ implicit val selfUniqueAddress: SelfUniqueAddress = DistributedData(system).selfUniqueAddress
val maxPruningDissemination = 3.seconds
val replicator = system.actorOf(
Replicator.props(
diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorSpec.scala
index a6060bb027..caaf45267d 100644
--- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorSpec.scala
+++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorSpec.scala
@@ -7,15 +7,16 @@ package akka.cluster.ddata
import scala.concurrent.Await
import scala.concurrent.duration._
-import akka.pattern.ask
+import com.typesafe.config.ConfigFactory
+
import akka.cluster.Cluster
+import akka.pattern.ask
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._
import akka.util.Timeout
-import com.typesafe.config.ConfigFactory
object ReplicatorSpec extends MultiNodeConfig {
val first = role("first")
@@ -38,13 +39,13 @@ class ReplicatorSpecMultiJvmNode2 extends ReplicatorSpec
class ReplicatorSpecMultiJvmNode3 extends ReplicatorSpec
class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec with ImplicitSender {
- import ReplicatorSpec._
import Replicator._
+ import ReplicatorSpec._
override def initialParticipants = roles.size
val cluster = Cluster(system)
- implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
+ implicit val selfUniqueAddress: SelfUniqueAddress = DistributedData(system).selfUniqueAddress
val replicator = system.actorOf(
Replicator.props(ReplicatorSettings(system).withGossipInterval(1.second).withMaxDeltaElements(10)),
"replicator")
diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/STMultiNodeSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/STMultiNodeSpec.scala
index 605d0045fb..15ce9cfbc2 100644
--- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/STMultiNodeSpec.scala
+++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/STMultiNodeSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster.ddata
-import akka.remote.testkit.MultiNodeSpecCallbacks
import org.scalatest.BeforeAndAfterAll
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
+import akka.remote.testkit.MultiNodeSpecCallbacks
+
/**
* Hooks up MultiNodeSpec with ScalaTest
*/
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/DataEnvelopeSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/DataEnvelopeSpec.scala
index bd8bc85176..49d4a10661 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/DataEnvelopeSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/DataEnvelopeSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster.ddata
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Replicator.Internal.DataEnvelope
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
class DataEnvelopeSpec extends AnyWordSpec with Matchers {
import PruningState._
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/DeltaPropagationSelectorSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/DeltaPropagationSelectorSpec.scala
index 4b1a22bee3..4f08aa64cb 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/DeltaPropagationSelectorSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/DeltaPropagationSelectorSpec.scala
@@ -4,6 +4,10 @@
package akka.cluster.ddata
+import org.scalactic.TypeCheckedTripleEquals
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Key.KeyId
@@ -11,9 +15,6 @@ import akka.cluster.ddata.Replicator.Internal.DataEnvelope
import akka.cluster.ddata.Replicator.Internal.Delta
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholder
-import org.scalactic.TypeCheckedTripleEquals
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
object DeltaPropagationSelectorSpec {
class TestSelector(val selfUniqueAddress: UniqueAddress, override val allNodes: Vector[UniqueAddress])
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/FlagSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/FlagSpec.scala
index 1099f132eb..fc481fba70 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/FlagSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/FlagSpec.scala
@@ -4,10 +4,11 @@
package akka.cluster.ddata
-import akka.cluster.ddata.Replicator.Changed
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.cluster.ddata.Replicator.Changed
+
class FlagSpec extends AnyWordSpec with Matchers {
"A Flag" must {
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/GCounterSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/GCounterSpec.scala
index feb0fd310b..e61becd8af 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/GCounterSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/GCounterSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster.ddata
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Replicator.Changed
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
class GCounterSpec extends AnyWordSpec with Matchers {
val node1 = UniqueAddress(Address("akka", "Sys", "localhost", 2551), 1L)
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/GSetSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/GSetSpec.scala
index 5095b1b343..d0b148a8de 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/GSetSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/GSetSpec.scala
@@ -4,10 +4,11 @@
package akka.cluster.ddata
-import akka.cluster.ddata.Replicator.Changed
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.cluster.ddata.Replicator.Changed
+
class GSetSpec extends AnyWordSpec with Matchers {
val user1 = """{"username":"john","password":"coltrane"}"""
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/LWWMapSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/LWWMapSpec.scala
index 14cfb99eb7..1a4f58be14 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/LWWMapSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/LWWMapSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster.ddata
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Replicator.Changed
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
class LWWMapSpec extends AnyWordSpec with Matchers {
import LWWRegister.defaultClock
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/LWWRegisterSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/LWWRegisterSpec.scala
index 6637ca485f..47f049d209 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/LWWRegisterSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/LWWRegisterSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster.ddata
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Replicator.Changed
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
class LWWRegisterSpec extends AnyWordSpec with Matchers {
import LWWRegister.defaultClock
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/LocalConcurrencySpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/LocalConcurrencySpec.scala
index a1daaee9f0..1a293a1661 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/LocalConcurrencySpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/LocalConcurrencySpec.scala
@@ -4,16 +4,17 @@
package akka.cluster.ddata
+import com.typesafe.config.ConfigFactory
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.Actor
import akka.actor.ActorSystem
import akka.actor.Props
import akka.actor.Stash
import akka.testkit.ImplicitSender
import akka.testkit.TestKit
-import com.typesafe.config.ConfigFactory
-import org.scalatest.BeforeAndAfterAll
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpecLike
object LocalConcurrencySpec {
@@ -25,7 +26,7 @@ object LocalConcurrencySpec {
class Updater extends Actor with Stash {
- implicit val selfUniqueAddress = DistributedData(context.system).selfUniqueAddress
+ implicit val selfUniqueAddress: SelfUniqueAddress = DistributedData(context.system).selfUniqueAddress
val replicator = DistributedData(context.system).replicator
@@ -45,7 +46,7 @@ class LocalConcurrencySpec(_system: ActorSystem)
with ImplicitSender {
import LocalConcurrencySpec._
- def this() {
+ def this() =
this(
ActorSystem(
"LocalConcurrencySpec",
@@ -54,7 +55,6 @@ class LocalConcurrencySpec(_system: ActorSystem)
akka.remote.classic.netty.tcp.port=0
akka.remote.artery.canonical.port = 0
""")))
- }
override def afterAll(): Unit = {
shutdown(system)
@@ -65,8 +65,8 @@ class LocalConcurrencySpec(_system: ActorSystem)
"Updates from same node" must {
"be possible to do from two actors" in {
- val updater1 = system.actorOf(Props[Updater], "updater1")
- val updater2 = system.actorOf(Props[Updater], "updater2")
+ val updater1 = system.actorOf(Props[Updater](), "updater1")
+ val updater2 = system.actorOf(Props[Updater](), "updater2")
val numMessages = 100
for (n <- 1 to numMessages) {
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/LotsOfDataBot.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/LotsOfDataBot.scala
index 56dc42a36b..a448dd7e6d 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/LotsOfDataBot.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/LotsOfDataBot.scala
@@ -8,11 +8,12 @@ import java.util.concurrent.ThreadLocalRandom
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorSystem
import akka.actor.Props
-import com.typesafe.config.ConfigFactory
/**
* This "sample" simulates lots of data entries, and can be used for
@@ -57,7 +58,7 @@ object LotsOfDataBot {
// Create an Akka system
val system = ActorSystem("ClusterSystem", config)
// Create an actor that handles cluster domain events
- system.actorOf(Props[LotsOfDataBot], name = "dataBot")
+ system.actorOf(Props[LotsOfDataBot](), name = "dataBot")
}
}
@@ -69,7 +70,7 @@ class LotsOfDataBot extends Actor with ActorLogging {
import LotsOfDataBot._
import Replicator._
- implicit val selfUniqueAddress = DistributedData(context.system).selfUniqueAddress
+ implicit val selfUniqueAddress: SelfUniqueAddress = DistributedData(context.system).selfUniqueAddress
val replicator = DistributedData(context.system).replicator
import context.dispatcher
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/ORMapSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/ORMapSpec.scala
index d1a7294953..69dc7eef77 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/ORMapSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/ORMapSpec.scala
@@ -4,12 +4,13 @@
package akka.cluster.ddata
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.ORSet.AddDeltaOp
import akka.cluster.ddata.Replicator.Changed
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
class ORMapSpec extends AnyWordSpec with Matchers {
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/ORMultiMapSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/ORMultiMapSpec.scala
index e004071069..aba0c8546a 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/ORMultiMapSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/ORMultiMapSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster.ddata
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Replicator.Changed
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
class ORMultiMapSpec extends AnyWordSpec with Matchers {
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/ORSetSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/ORSetSpec.scala
index 3e273b4bbd..535c4c770b 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/ORSetSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/ORSetSpec.scala
@@ -6,11 +6,12 @@ package akka.cluster.ddata
import scala.collection.immutable.TreeMap
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Replicator.Changed
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
class ORSetSpec extends AnyWordSpec with Matchers {
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/PNCounterMapSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/PNCounterMapSpec.scala
index 983ad22bae..f4b8467e42 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/PNCounterMapSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/PNCounterMapSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster.ddata
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Replicator.Changed
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
class PNCounterMapSpec extends AnyWordSpec with Matchers {
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/PNCounterSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/PNCounterSpec.scala
index 88fee0fba1..8f3e44f298 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/PNCounterSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/PNCounterSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster.ddata
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Replicator.Changed
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
class PNCounterSpec extends AnyWordSpec with Matchers {
val node1 = UniqueAddress(Address("akka", "Sys", "localhost", 2551), 1L)
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/PruningStateSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/PruningStateSpec.scala
index ceedee1077..5a92443671 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/PruningStateSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/PruningStateSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster.ddata
-import akka.actor.Address
-import akka.cluster.UniqueAddress
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.Address
+import akka.cluster.UniqueAddress
+
class PruningStateSpec extends AnyWordSpec with Matchers {
import PruningState._
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/ReplicatorSettingsSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/ReplicatorSettingsSpec.scala
index 83f311aecf..a5d8039348 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/ReplicatorSettingsSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/ReplicatorSettingsSpec.scala
@@ -4,11 +4,12 @@
package akka.cluster.ddata
-import akka.testkit.AkkaSpec
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfterAll
import org.scalatest.wordspec.AnyWordSpecLike
+import akka.testkit.AkkaSpec
+
object ReplicatorSettingsSpec {
val config = ConfigFactory.parseString("""
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/VersionVectorSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/VersionVectorSpec.scala
index 9f236d7eaa..145610ef5d 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/VersionVectorSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/VersionVectorSpec.scala
@@ -4,13 +4,14 @@
package akka.cluster.ddata
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.ActorSystem
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.testkit.TestKit
-import org.scalatest.BeforeAndAfterAll
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpecLike
class VersionVectorSpec
extends TestKit(ActorSystem("VersionVectorSpec"))
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/WriteAggregatorSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/WriteAggregatorSpec.scala
index b17ed3794f..0b233fd755 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/WriteAggregatorSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/WriteAggregatorSpec.scala
@@ -4,6 +4,7 @@
package akka.cluster.ddata
+import scala.concurrent.Future
import scala.concurrent.duration._
import akka.actor.Actor
@@ -12,14 +13,12 @@ import akka.actor.ActorSelection
import akka.actor.ActorSystem
import akka.actor.Address
import akka.actor.Props
-import akka.testkit._
-import akka.cluster.ddata.Replicator.Internal._
-import akka.cluster.ddata.Replicator._
-import akka.remote.RARP
-import scala.concurrent.Future
-
import akka.cluster.Cluster
import akka.cluster.UniqueAddress
+import akka.cluster.ddata.Replicator._
+import akka.cluster.ddata.Replicator.Internal._
+import akka.remote.RARP
+import akka.testkit._
object WriteAggregatorSpec {
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatedDataSerializerSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatedDataSerializerSpec.scala
index 761a3807c4..2ebf8ec347 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatedDataSerializerSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatedDataSerializerSpec.scala
@@ -6,25 +6,26 @@ package akka.cluster.ddata.protobuf
import java.util.Base64
-import akka.actor.ActorIdentity
-import akka.actor.ActorRef
+import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfterAll
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
+
+import akka.actor.ActorIdentity
+import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.Address
import akka.actor.ExtendedActorSystem
import akka.actor.Identify
-import akka.cluster.ddata._
-import akka.cluster.ddata.Replicator.Internal._
-import akka.testkit.TestKit
-import akka.cluster.UniqueAddress
-import akka.remote.RARP
-import com.typesafe.config.ConfigFactory
import akka.actor.Props
import akka.actor.RootActorPath
import akka.cluster.Cluster
+import akka.cluster.UniqueAddress
+import akka.cluster.ddata._
+import akka.cluster.ddata.Replicator.Internal._
+import akka.remote.RARP
import akka.testkit.TestActors
+import akka.testkit.TestKit
class ReplicatedDataSerializerSpec
extends TestKit(
diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializerSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializerSpec.scala
index b2884b8bd5..a8f0ea018f 100644
--- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializerSpec.scala
+++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializerSpec.scala
@@ -5,29 +5,31 @@
package akka.cluster.ddata.protobuf
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfterAll
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.ActorSystem
import akka.actor.Address
import akka.actor.ExtendedActorSystem
import akka.actor.Props
+import akka.cluster.UniqueAddress
+import akka.cluster.ddata.DurableStore.DurableDataEnvelope
+import akka.cluster.ddata.GCounter
import akka.cluster.ddata.GSet
import akka.cluster.ddata.GSetKey
+import akka.cluster.ddata.ORMultiMap
+import akka.cluster.ddata.ORSet
import akka.cluster.ddata.PruningState.PruningInitialized
import akka.cluster.ddata.PruningState.PruningPerformed
import akka.cluster.ddata.Replicator._
import akka.cluster.ddata.Replicator.Internal._
+import akka.cluster.ddata.VersionVector
+import akka.remote.RARP
import akka.testkit.TestKit
import akka.util.{ unused, ByteString }
-import akka.cluster.UniqueAddress
-import akka.remote.RARP
-import com.typesafe.config.ConfigFactory
-import akka.cluster.ddata.DurableStore.DurableDataEnvelope
-import akka.cluster.ddata.GCounter
-import akka.cluster.ddata.VersionVector
-import akka.cluster.ddata.ORSet
-import akka.cluster.ddata.ORMultiMap
class ReplicatorMessageSerializerSpec
extends TestKit(
diff --git a/akka-docs/src/main/paradox/actors.md b/akka-docs/src/main/paradox/actors.md
index 0e4c4cd997..2952150f13 100644
--- a/akka-docs/src/main/paradox/actors.md
+++ b/akka-docs/src/main/paradox/actors.md
@@ -8,7 +8,7 @@ To use Classic Actors, add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor_$scala.binary_version$"
+ artifact="akka-actor_$scala.binary.version$"
version="$akka.version$"
}
@@ -16,7 +16,7 @@ To use Classic Actors, add the following dependency in your project:
## Introduction
-The [Actor Model](http://en.wikipedia.org/wiki/Actor_model) provides a higher level of abstraction for writing concurrent
+The [Actor Model](https://en.wikipedia.org/wiki/Actor_model) provides a higher level of abstraction for writing concurrent
and distributed systems. It alleviates the developer from having to deal with
explicit locking and thread management, making it easier to write correct
concurrent and parallel systems. Actors were defined in the 1973 paper by Carl
@@ -294,7 +294,7 @@ singleton scope.
Techniques for dependency injection and integration with dependency injection frameworks
are described in more depth in the
-[Using Akka with Dependency Injection](http://letitcrash.com/post/55958814293/akka-dependency-injection)
+[Using Akka with Dependency Injection](https://letitcrash.com/post/55958814293/akka-dependency-injection)
guideline and the [Akka Java Spring](https://github.com/typesafehub/activator-akka-java-spring) tutorial.
## Actor API
@@ -832,7 +832,7 @@ That has benefits such as:
The `Receive` can be implemented in other ways than using the `ReceiveBuilder` since it in the
end is just a wrapper around a Scala `PartialFunction`. In Java, you can implement `PartialFunction` by
extending `AbstractPartialFunction`. For example, one could implement an adapter
-to [Vavr Pattern Matching DSL](http://www.vavr.io/vavr-docs/#_pattern_matching). See the [Akka Vavr sample project](https://github.com/akka/akka-samples/tree/2.5/akka-sample-vavr) for more details.
+to [Vavr Pattern Matching DSL](https://www.vavr.io/vavr-docs/#_pattern_matching). See the [Akka Vavr sample project](https://github.com/akka/akka-samples/tree/2.5/akka-sample-vavr) for more details.
If the validation of the `ReceiveBuilder` match logic turns out to be a bottleneck for some of your
actors you can consider to implement it at lower level by extending `UntypedAbstractActor` instead
diff --git a/akka-docs/src/main/paradox/additional/books.md b/akka-docs/src/main/paradox/additional/books.md
index 6b40f1fe7d..bd1311e9ca 100644
--- a/akka-docs/src/main/paradox/additional/books.md
+++ b/akka-docs/src/main/paradox/additional/books.md
@@ -4,16 +4,16 @@
### Recommended reads
* [Reactive Design Patterns](https://www.reactivedesignpatterns.com/), by Roland Kuhn with Jamie Allen and Brian Hanafee, Manning Publications Co., ISBN 9781617291807, Feb 2017
- * [Akka in Action](http://www.lightbend.com/resources/e-book/akka-in-action), by Raymond Roestenburg and Rob Bakker, Manning Publications Co., ISBN: 9781617291012, September 2016
+ * [Akka in Action](https://www.lightbend.com/resources/e-book/akka-in-action), by Raymond Roestenburg and Rob Bakker, Manning Publications Co., ISBN: 9781617291012, September 2016
### Other reads about Akka and the Actor model
* [Akka Cookbook](https://www.packtpub.com/application-development/akka-cookbook), by Héctor Veiga Ortiz & Piyush Mishra, PACKT Publishing, ISBN: 9781785288180, May 2017
* [Mastering Akka](https://www.packtpub.com/application-development/mastering-akka), by Christian Baxter, PACKT Publishing, ISBN: 9781786465023, October 2016
* [Learning Akka](https://www.packtpub.com/application-development/learning-akka), by Jason Goodwin, PACKT Publishing, ISBN: 9781784393007, December 2015
- * [Reactive Messaging Patterns with the Actor Model](http://www.informit.com/store/reactive-messaging-patterns-with-the-actor-model-applications-9780133846836), by Vaughn Vernon, Addison-Wesley Professional, ISBN: 0133846830, August 2015
- * [Developing an Akka Edge](http://bleedingedgepress.com/our-books/developing-an-akka-edge/), by Thomas Lockney and Raymond Tay, Bleeding Edge Press, ISBN: 9781939902054, April 2014
- * [Effective Akka](http://shop.oreilly.com/product/0636920028789.do), by Jamie Allen, O'Reilly Media, ISBN: 1449360076, August 2013
- * [Akka Concurrency](http://www.artima.com/shop/akka_concurrency), by Derek Wyatt, artima developer, ISBN: 0981531660, May 2013
+ * [Reactive Messaging Patterns with the Actor Model](https://www.informit.com/store/reactive-messaging-patterns-with-the-actor-model-applications-9780133846836), by Vaughn Vernon, Addison-Wesley Professional, ISBN: 0133846830, August 2015
+ * [Developing an Akka Edge](https://bleedingedgepress.com/developing-an-akka-edge/), by Thomas Lockney and Raymond Tay, Bleeding Edge Press, ISBN: 9781939902054, April 2014
+ * [Effective Akka](https://shop.oreilly.com/product/0636920028789.do), by Jamie Allen, O'Reilly Media, ISBN: 1449360076, August 2013
+ * [Akka Concurrency](https://www.artima.com/shop/akka_concurrency), by Derek Wyatt, artima developer, ISBN: 0981531660, May 2013
* [Akka Essentials](https://www.packtpub.com/application-development/akka-essentials), by Munish K. Gupta, PACKT Publishing, ISBN: 1849518289, October 2012
* [Start Building RESTful Microservices using Akka HTTP with Scala](https://www.amazon.com/dp/1976762545/), by Ayush Kumar Mishra, Knoldus Software LLP, ISBN: 9781976762543, December 2017
@@ -23,3 +23,7 @@
* [Zen of Akka](https://www.youtube.com/watch?v=vgFoKOxrTzg) - an overview of good and bad practices in Akka, by Konrad Malawski, ScalaDays New York, June 2016
* [Learning Akka Videos](https://www.packtpub.com/application-development/learning-akka-video), by Salma Khater, PACKT Publishing, ISBN: 9781784391836, January 2016
* [Building Microservice with AKKA HTTP (Video)](https://www.packtpub.com/application-development/building-microservice-akka-http-video), by Tomasz Lelek, PACKT Publishing, ISBN: 9781788298582, March 2017
+
+## Blogs
+
+A list of [blogs and presentations](https://akka.io/blog/external-archive.html) curated by the Akka team.
diff --git a/akka-docs/src/main/paradox/additional/faq.md b/akka-docs/src/main/paradox/additional/faq.md
index c995f27a1a..e783a66a70 100644
--- a/akka-docs/src/main/paradox/additional/faq.md
+++ b/akka-docs/src/main/paradox/additional/faq.md
@@ -4,7 +4,7 @@
### Where does the name Akka come from?
-It is the name of a beautiful Swedish [mountain](https://lh4.googleusercontent.com/-z28mTALX90E/UCOsd249TdI/AAAAAAAAAB0/zGyNNZla-zY/w442-h331/akka-beautiful-panorama.jpg)
+It is the name of a beautiful Swedish [mountain](https://en.wikipedia.org/wiki/%C3%81hkk%C3%A1)
up in the northern part of Sweden called Laponia. The mountain is also sometimes
called 'The Queen of Laponia'.
@@ -16,9 +16,9 @@ Also, the name AKKA is a palindrome of the letters A and K as in Actor Kernel.
Akka is also:
- * the name of the goose that Nils traveled across Sweden on in [The Wonderful Adventures of Nils](http://en.wikipedia.org/wiki/The_Wonderful_Adventures_of_Nils) by the Swedish writer Selma Lagerlöf.
+ * the name of the goose that Nils traveled across Sweden on in [The Wonderful Adventures of Nils](https://en.wikipedia.org/wiki/The_Wonderful_Adventures_of_Nils) by the Swedish writer Selma Lagerlöf.
* the Finnish word for 'nasty elderly woman' and the word for 'elder sister' in the Indian languages Tamil, Telugu, Kannada and Marathi.
- * a [font](http://www.dafont.com/akka.font)
+ * a [font](https://www.dafont.com/akka.font)
* a town in Morocco
* a near-earth asteroid
diff --git a/akka-docs/src/main/paradox/additional/operations.md b/akka-docs/src/main/paradox/additional/operations.md
index 4adc231285..0f29afbf0b 100644
--- a/akka-docs/src/main/paradox/additional/operations.md
+++ b/akka-docs/src/main/paradox/additional/operations.md
@@ -14,7 +14,7 @@ When starting clusters on cloud systems such as Kubernetes, AWS, Google Cloud, A
you may want to automate the discovery of nodes for the cluster joining process, using your cloud providers,
cluster orchestrator, or other form of service discovery (such as managed DNS).
-The open source Akka Management library includes the [Cluster Bootstrap](https://doc.akka.io/docs/akka-management/current/bootstrap/index.html)
+The open source Akka Management library includes the @extref:[Cluster Bootstrap](akka-management:bootstrap/index.html)
module which handles just that. Please refer to its documentation for more details.
@@@ note
@@ -32,13 +32,13 @@ See @ref:[Rolling Updates, Cluster Shutdown and Coordinated Shutdown](../additio
There are several management tools for the cluster.
Complete information on running and managing Akka applications can be found in
-the [Akka Management](https://doc.akka.io/docs/akka-management/current/) project documentation.
+the @exref:[Akka Management](akka-management:) project documentation.
### HTTP
Information and management of the cluster is available with a HTTP API.
-See documentation of [Akka Management](http://developer.lightbend.com/docs/akka-management/current/).
+See documentation of @extref:[Akka Management](akka-management:).
### JMX
@@ -60,6 +60,6 @@ Member nodes are identified by their address, in format *`akka://actor-system-na
## Monitoring and Observability
Aside from log monitoring and the monitoring provided by your APM or platform provider, [Lightbend Telemetry](https://developer.lightbend.com/docs/telemetry/current/instrumentations/akka/akka.html),
-available through a [Lightbend Platform Subscription](https://www.lightbend.com/lightbend-platform-subscription),
+available through a [Lightbend Subscription](https://www.lightbend.com/lightbend-subscription),
can provide additional insights in the run-time characteristics of your application, including metrics, events,
and distributed tracing for Akka Actors, Cluster, HTTP, and more.
diff --git a/akka-docs/src/main/paradox/additional/osgi.md b/akka-docs/src/main/paradox/additional/osgi.md
index 55abb960d5..01e7aaaca5 100644
--- a/akka-docs/src/main/paradox/additional/osgi.md
+++ b/akka-docs/src/main/paradox/additional/osgi.md
@@ -6,13 +6,13 @@ To use Akka in OSGi, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-osgi_$scala.binary_version$
+ artifact=akka-osgi_$scala.binary.version$
version=$akka.version$
}
## Background
-[OSGi](http://www.osgi.org/developer) is a mature packaging and deployment standard for component-based systems. It
+[OSGi](https://www.osgi.org/developer/where-to-start/) is a mature packaging and deployment standard for component-based systems. It
has similar capabilities as Project Jigsaw (originally scheduled for JDK 1.8), but has far stronger facilities to
support legacy Java code. This is to say that while Jigsaw-ready modules require significant changes to most source files
and on occasion to the structure of the overall application, OSGi can be used to modularize almost any Java code as far
diff --git a/akka-docs/src/main/paradox/additional/packaging.md b/akka-docs/src/main/paradox/additional/packaging.md
index b57a962f3e..9bb4e8c360 100644
--- a/akka-docs/src/main/paradox/additional/packaging.md
+++ b/akka-docs/src/main/paradox/additional/packaging.md
@@ -33,12 +33,12 @@ Add [sbt-native-packager](https://github.com/sbt/sbt-native-packager) in `projec
addSbtPlugin("com.typesafe.sbt" % "sbt-native-packager" % "1.1.5")
```
-Follow the instructions for the `JavaAppPackaging` in the [sbt-native-packager plugin documentation](http://sbt-native-packager.readthedocs.io/en/latest/archetypes/java_app/index.html).
+Follow the instructions for the `JavaAppPackaging` in the [sbt-native-packager plugin documentation](https://sbt-native-packager.readthedocs.io/en/latest/archetypes/java_app/index.html).
## Maven: jarjar, onejar or assembly
-You can use the [Apache Maven Shade Plugin](http://maven.apache.org/plugins/maven-shade-plugin)
-support for [Resource Transformers](http://maven.apache.org/plugins/maven-shade-plugin/examples/resource-transformers.html#AppendingTransformer)
+You can use the [Apache Maven Shade Plugin](https://maven.apache.org/plugins/maven-shade-plugin/)
+support for [Resource Transformers](https://maven.apache.org/plugins/maven-shade-plugin/examples/resource-transformers.html#AppendingTransformer)
to merge all the reference.confs on the build classpath into one.
The plugin configuration might look like this:
diff --git a/akka-docs/src/main/paradox/additional/rolling-updates.md b/akka-docs/src/main/paradox/additional/rolling-updates.md
index 3f50c6b81b..ee52862ac4 100644
--- a/akka-docs/src/main/paradox/additional/rolling-updates.md
+++ b/akka-docs/src/main/paradox/additional/rolling-updates.md
@@ -67,7 +67,7 @@ Environments such as Kubernetes send a SIGTERM, however if the JVM is wrapped wi
In case of network failures it may still be necessary to set the node's status to Down in order to complete the removal.
@ref:[Cluster Downing](../typed/cluster.md#downing) details downing nodes and downing providers.
-[Split Brain Resolver](https://doc.akka.io/docs/akka-enhancements/current/split-brain-resolver.html) can be used to ensure
+@ref:[Split Brain Resolver](../split-brain-resolver.md) can be used to ensure
the cluster continues to function during network partitions and node failures. For example
if there is an unreachability problem Split Brain Resolver would make a decision based on the configured downing strategy.
diff --git a/akka-docs/src/main/paradox/cluster-client.md b/akka-docs/src/main/paradox/cluster-client.md
index 37d737f167..2c00ea9de8 100644
--- a/akka-docs/src/main/paradox/cluster-client.md
+++ b/akka-docs/src/main/paradox/cluster-client.md
@@ -15,7 +15,7 @@ To use Cluster Client, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-tools_$scala.binary_version$
+ artifact=akka-cluster-tools_$scala.binary.version$
version=$akka.version$
}
@@ -231,7 +231,7 @@ contacts can be fetched and a new cluster client started.
## Migration to Akka gRPC
Cluster Client is deprecated and it is not advised to build new applications with it.
-As a replacement we recommend using [Akka gRPC](https://doc.akka.io/docs/akka-grpc/current/index.html)
+As a replacement we recommend using [Akka gRPC](https://doc.akka.io/docs/akka-grpc/current/)
with an application-specific protocol. The benefits of this approach are:
* Improved security by using TLS for gRPC (HTTP/2) versus exposing Akka Remoting outside the Akka Cluster
@@ -244,7 +244,7 @@ with an application-specific protocol. The benefits of this approach are:
### Migrating directly
Existing users of Cluster Client may migrate directly to Akka gRPC and use it
-as documented in [its documentation](https://doc.akka.io/docs/akka-grpc/current).
+as documented in [its documentation](https://doc.akka.io/docs/akka-grpc/current/).
### Migrating gradually
diff --git a/akka-docs/src/main/paradox/cluster-metrics.md b/akka-docs/src/main/paradox/cluster-metrics.md
index 4efabc446d..7bec024ae6 100644
--- a/akka-docs/src/main/paradox/cluster-metrics.md
+++ b/akka-docs/src/main/paradox/cluster-metrics.md
@@ -6,7 +6,7 @@ To use Cluster Metrics Extension, you must add the following dependency in your
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-metrics_$scala.binary_version$
+ artifact=akka-cluster-metrics_$scala.binary.version$
version=$akka.version$
}
@@ -112,7 +112,7 @@ To enable usage of Sigar you can add the following dependency to the user projec
version="$sigar_loader.version$"
}
-You can download Kamon sigar-loader from [Maven Central](http://search.maven.org/#search%7Cga%7C1%7Csigar-loader)
+You can download Kamon sigar-loader from [Maven Central](https://search.maven.org/search?q=sigar-loader)
## Adaptive Load Balancing
@@ -126,7 +126,7 @@ It can be configured to use a specific MetricsSelector to produce the probabilit
* `mix` / `MixMetricsSelector` - Combines heap, cpu and load. Weights based on mean of remaining capacity of the combined selectors.
* Any custom implementation of `akka.cluster.metrics.MetricsSelector`
-The collected metrics values are smoothed with [exponential weighted moving average](http://en.wikipedia.org/wiki/Moving_average#Exponential_moving_average). In the @ref:[Cluster configuration](cluster-usage.md#cluster-configuration) you can adjust how quickly past data is decayed compared to new data.
+The collected metrics values are smoothed with [exponential weighted moving average](https://en.wikipedia.org/wiki/Moving_average#Exponential_moving_average). In the @ref:[Cluster configuration](cluster-usage.md#cluster-configuration) you can adjust how quickly past data is decayed compared to new data.
Let's take a look at this router in action. What can be more demanding than calculating factorials?
diff --git a/akka-docs/src/main/paradox/cluster-routing.md b/akka-docs/src/main/paradox/cluster-routing.md
index 1b1c515c3b..6dc7e56f45 100644
--- a/akka-docs/src/main/paradox/cluster-routing.md
+++ b/akka-docs/src/main/paradox/cluster-routing.md
@@ -33,7 +33,7 @@ To use Cluster aware routers, you must add the following dependency in your proj
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-cluster_$scala.binary_version$"
+ artifact="akka-cluster_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/cluster-sharding.md b/akka-docs/src/main/paradox/cluster-sharding.md
index 351808691a..a908bd7585 100644
--- a/akka-docs/src/main/paradox/cluster-sharding.md
+++ b/akka-docs/src/main/paradox/cluster-sharding.md
@@ -9,7 +9,7 @@ To use Cluster Sharding, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-sharding_$scala.binary_version$
+ artifact=akka-cluster-sharding_$scala.binary.version$
version=$akka.version$
}
diff --git a/akka-docs/src/main/paradox/cluster-singleton.md b/akka-docs/src/main/paradox/cluster-singleton.md
index 2b49211696..94fc0bfa65 100644
--- a/akka-docs/src/main/paradox/cluster-singleton.md
+++ b/akka-docs/src/main/paradox/cluster-singleton.md
@@ -9,7 +9,7 @@ To use Cluster Singleton, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-tools_$scala.binary_version$
+ artifact=akka-cluster-tools_$scala.binary.version$
version=$akka.version$
}
@@ -107,7 +107,7 @@ There are two actors that could potentially be supervised. For the `consumer` si
The Cluster singleton manager actor should not have its supervision strategy changed as it should always be running.
However it is sometimes useful to add supervision for the user actor.
To accomplish this add a parent supervisor actor which will be used to create the 'real' singleton instance.
-Below is an example implementation (credit to [this StackOverflow answer](https://stackoverflow.com/a/36716708/779513))
+Below is an example implementation (credit to [this StackOverflow answer](https://stackoverflow.com/questions/36701898/how-to-supervise-cluster-singleton-in-akka/36716708#36716708))
Scala
: @@snip [ClusterSingletonSupervision.scala](/akka-docs/src/test/scala/docs/cluster/singleton/ClusterSingletonSupervision.scala) { #singleton-supervisor-actor }
diff --git a/akka-docs/src/main/paradox/cluster-usage.md b/akka-docs/src/main/paradox/cluster-usage.md
index d45c049209..51467fd5b4 100644
--- a/akka-docs/src/main/paradox/cluster-usage.md
+++ b/akka-docs/src/main/paradox/cluster-usage.md
@@ -25,7 +25,7 @@ To use Akka Cluster add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-cluster_$scala.binary_version$"
+ artifact="akka-cluster_$scala.binary.version$"
version="$akka.version$"
}
@@ -414,7 +414,7 @@ Examples: ./akka-cluster localhost 9999 is-available
```
To be able to use the script you must enable remote monitoring and management when starting the JVMs of the cluster nodes,
-as described in [Monitoring and Management Using JMX Technology](http://docs.oracle.com/javase/8/docs/technotes/guides/management/agent.html).
+as described in [Monitoring and Management Using JMX Technology](https://docs.oracle.com/javase/8/docs/technotes/guides/management/agent.html).
Make sure you understand the security implications of enabling remote monitoring and management.
diff --git a/akka-docs/src/main/paradox/common/other-modules.md b/akka-docs/src/main/paradox/common/other-modules.md
index 04680bb242..ed3e1046ed 100644
--- a/akka-docs/src/main/paradox/common/other-modules.md
+++ b/akka-docs/src/main/paradox/common/other-modules.md
@@ -8,7 +8,7 @@ A full server- and client-side HTTP stack on top of akka-actor and akka-stream.
Alpakka is a Reactive Enterprise Integration library for Java and Scala, based on Reactive Streams and Akka.
-## [Alpakka Kafka Connector](https://doc.akka.io/docs/akka-stream-kafka/current/)
+## [Alpakka Kafka Connector](https://doc.akka.io/docs/alpakka-kafka/current/)
The Alpakka Kafka Connector connects Apache Kafka with Akka Streams.
@@ -26,6 +26,7 @@ An Akka Persistence journal and snapshot store backed by Couchbase.
* [Akka Cluster Bootstrap](https://doc.akka.io/docs/akka-management/current/bootstrap/) helps bootstrapping an Akka cluster using Akka Discovery.
* [Akka Management Cluster HTTP](https://doc.akka.io/docs/akka-management/current/cluster-http-management.html) provides HTTP endpoints for introspecting and managing Akka clusters.
* [Akka Discovery for Kubernetes, Consul, Marathon, and AWS](https://doc.akka.io/docs/akka-management/current/discovery/)
+* [Kubernetes Lease](https://doc.akka.io/docs/akka-management/current/kubernetes-lease.html)
## [Akka gRPC](https://doc.akka.io/docs/akka-grpc/current/)
@@ -33,8 +34,6 @@ Akka gRPC provides support for building streaming gRPC servers and clients on to
## Akka Resilience Enhancements
-* [Akka Split Brain Resolver](https://doc.akka.io/docs/akka-enhancements/current/split-brain-resolver.html)
-* [Kubernetes Lease](https://doc.akka.io/docs/akka-enhancements/current/kubernetes-lease.html)
* [Akka Thread Starvation Detector](https://doc.akka.io/docs/akka-enhancements/current/starvation-detector.html)
* [Akka Configuration Checker](https://doc.akka.io/docs/akka-enhancements/current/config-checker.html)
* [Akka Diagnostics Recorder](https://doc.akka.io/docs/akka-enhancements/current/diagnostics-recorder.html)
@@ -44,7 +43,6 @@ Akka gRPC provides support for building streaming gRPC servers and clients on to
* [Akka Multi-DC Persistence](https://doc.akka.io/docs/akka-enhancements/current/persistence-dc/index.html)
* [Akka GDPR for Persistence](https://doc.akka.io/docs/akka-enhancements/current/gdpr/index.html)
-
## Community Projects
Akka has a vibrant and passionate user community, the members of which have created many independent projects using Akka as well as extensions to it. See [Community Projects](https://akka.io/community/).
diff --git a/akka-docs/src/main/paradox/coordinated-shutdown.md b/akka-docs/src/main/paradox/coordinated-shutdown.md
index 6fb5e5808d..28b366275b 100644
--- a/akka-docs/src/main/paradox/coordinated-shutdown.md
+++ b/akka-docs/src/main/paradox/coordinated-shutdown.md
@@ -1,37 +1,39 @@
# Coordinated Shutdown
-Under normal conditions when `ActorSystem` is terminated or the JVM process is shut down certain
+Under normal conditions, when an `ActorSystem` is terminated or the JVM process is shut down, certain
actors and services will be stopped in a specific order.
-This is handled by an extension named `CoordinatedShutdown`. It will run the registered tasks
-during the shutdown process. The order of the shutdown phases is defined in configuration `akka.coordinated-shutdown.phases`.
-The default phases are defined as:
+The @apidoc[CoordinatedShutdown$] extension registers internal and user-defined tasks to be executed during the shutdown process. The tasks are grouped in configuration-defined "phases" which define the shutdown order.
-@@snip [reference.conf](/akka-actor/src/main/resources/reference.conf) { #coordinated-shutdown-phases }
-
-More phases can be added in the application's configuration if needed by overriding a phase with an
-additional `depends-on`. Especially the phases `before-service-unbind`, `before-cluster-shutdown` and
+Especially the phases `before-service-unbind`, `before-cluster-shutdown` and
`before-actor-system-terminate` are intended for application specific phases or tasks.
+The order of the shutdown phases is defined in configuration `akka.coordinated-shutdown.phases`. See the default phases in the `reference.conf` tab:
+
+Most relevant default phases
+: | Phase | Description |
+|-------------|----------------------------------------------|
+| before-service-unbind | The first pre-defined phase during shutdown. |
+| before-cluster-shutdown | Phase for custom application tasks that are to be run after service shutdown and before cluster shutdown. |
+| before-actor-system-terminate | Phase for custom application tasks that are to be run after cluster shutdown and before `ActorSystem` termination. |
+
+reference.conf (HOCON)
+: @@snip [reference.conf](/akka-actor/src/main/resources/reference.conf) { #coordinated-shutdown-phases }
+
+More phases can be added in the application's `application.conf` if needed by overriding a phase with an
+additional `depends-on`.
+
The default phases are defined in a single linear order, but the phases can be ordered as a
directed acyclic graph (DAG) by defining the dependencies between the phases.
The phases are ordered with [topological](https://en.wikipedia.org/wiki/Topological_sorting) sort of the DAG.
-Tasks can be added to a phase with:
+Tasks can be added to a phase like in this example which allows a certain actor to react before termination starts:
Scala
-: @@snip [ActorDocSpec.scala](/akka-docs/src/test/scala/docs/actor/ActorDocSpec.scala) { #coordinated-shutdown-addTask }
+: @@snip [snip](/akka-docs/src/test/scala/docs/actor/typed/CoordinatedActorShutdownSpec.scala) { #coordinated-shutdown-addTask }
Java
-: @@snip [ActorDocTest.java](/akka-docs/src/test/java/jdocs/actor/ActorDocTest.java) { #coordinated-shutdown-addTask }
-
-If cancellation of previously added tasks is required:
-
-Scala
-: @@snip [ActorDocSpec.scala](/akka-docs/src/test/scala/docs/actor/ActorDocSpec.scala) { #coordinated-shutdown-cancellable }
-
-Java
-: @@snip [ActorDocTest.java](/akka-docs/src/test/java/jdocs/actor/ActorDocTest.java) { #coordinated-shutdown-cancellable }
+: @@snip [snip](/akka-docs/src/test/java/jdocs/actor/typed/CoordinatedActorShutdownTest.java) { #coordinated-shutdown-addTask }
The returned @scala[`Future[Done]`] @java[`CompletionStage`] should be completed when the task is completed. The task name parameter
is only used for debugging/logging.
@@ -43,9 +45,17 @@ If tasks are not completed within a configured timeout (see @ref:[reference.conf
the next phase will be started anyway. It is possible to configure `recover=off` for a phase
to abort the rest of the shutdown process if a task fails or is not completed within the timeout.
+If cancellation of previously added tasks is required:
+
+Scala
+: @@snip [snip](/akka-docs/src/test/scala/docs/actor/typed/CoordinatedActorShutdownSpec.scala) { #coordinated-shutdown-cancellable }
+
+Java
+: @@snip [snip](/akka-docs/src/test/java/jdocs/actor/typed/CoordinatedActorShutdownTest.java) { #coordinated-shutdown-cancellable }
+
In the above example, it may be more convenient to simply stop the actor when it's done shutting down, rather than send back a done message,
and for the shutdown task to not complete until the actor is terminated. A convenience method is provided that adds a task that sends
-a message to the actor and then watches its termination:
+a message to the actor and then watches its termination (there is currently no corresponding functionality for the new actors API @github[see #29056](#29056)):
Scala
: @@snip [ActorDocSpec.scala](/akka-docs/src/test/scala/docs/actor/ActorDocSpec.scala) { #coordinated-shutdown-addActorTerminationTask }
@@ -57,14 +67,14 @@ Tasks should typically be registered as early as possible after system startup.
the coordinated shutdown tasks that have been registered will be performed but tasks that are
added too late will not be run.
-To start the coordinated shutdown process you can invoke @scala[`run`] @java[`runAll`] on the `CoordinatedShutdown`
-extension:
+To start the coordinated shutdown process you can either invoke `terminate()` on the `ActorSystem`, or @scala[`run`] @java[`runAll`] on the `CoordinatedShutdown`
+extension and pass it a class implementing @apidoc[CoordinatedShutdown.Reason] for informational purposes:
Scala
-: @@snip [ActorDocSpec.scala](/akka-docs/src/test/scala/docs/actor/ActorDocSpec.scala) { #coordinated-shutdown-run }
+: @@snip [snip](/akka-docs/src/test/scala/docs/actor/typed/CoordinatedActorShutdownSpec.scala) { #coordinated-shutdown-run }
Java
-: @@snip [ActorDocTest.java](/akka-docs/src/test/java/jdocs/actor/ActorDocTest.java) { #coordinated-shutdown-run }
+: @@snip [snip](/akka-docs/src/test/java/jdocs/actor/typed/CoordinatedActorShutdownTest.java) { #coordinated-shutdown-run }
It's safe to call the @scala[`run`] @java[`runAll`] method multiple times. It will only run once.
@@ -76,7 +86,7 @@ To enable a hard `System.exit` as a final action you can configure:
akka.coordinated-shutdown.exit-jvm = on
```
-The coordinated shutdown process can also be started by calling `ActorSystem.terminate()`.
+The coordinated shutdown process is also started once the actor system's root actor is stopped.
When using @ref:[Akka Cluster](cluster-usage.md) the `CoordinatedShutdown` will automatically run
when the cluster node sees itself as `Exiting`, i.e. leaving from another node will trigger
@@ -96,10 +106,10 @@ If you have application specific JVM shutdown hooks it's recommended that you re
those shutting down Akka Remoting (Artery).
Scala
-: @@snip [ActorDocSpec.scala](/akka-docs/src/test/scala/docs/actor/ActorDocSpec.scala) { #coordinated-shutdown-jvm-hook }
+: @@snip [snip](/akka-docs/src/test/scala/docs/actor/typed/CoordinatedActorShutdownSpec.scala) { #coordinated-shutdown-jvm-hook }
Java
-: @@snip [ActorDocTest.java](/akka-docs/src/test/java/jdocs/actor/ActorDocTest.java) { #coordinated-shutdown-jvm-hook }
+: @@snip [snip](/akka-docs/src/test/java/jdocs/actor/typed/CoordinatedActorShutdownTest.java) { #coordinated-shutdown-jvm-hook }
For some tests it might be undesired to terminate the `ActorSystem` via `CoordinatedShutdown`.
You can disable that by adding the following to the configuration of the `ActorSystem` that is
diff --git a/akka-docs/src/main/paradox/coordination.md b/akka-docs/src/main/paradox/coordination.md
index d1dcfb5685..e2266ad2ab 100644
--- a/akka-docs/src/main/paradox/coordination.md
+++ b/akka-docs/src/main/paradox/coordination.md
@@ -9,7 +9,7 @@ Akka Coordination is a set of tools for distributed coordination.
@@dependency[sbt,Gradle,Maven] {
group="com.typesafe.akka"
- artifact="akka-coordination_$scala.binary_version$"
+ artifact="akka-coordination_$scala.binary.version$"
version="$akka.version$"
}
@@ -35,10 +35,10 @@ Any lease implementation should provide the following guarantees:
To acquire a lease:
Scala
-: @@snip [LeaseDocSpec.scala](/akka-coordination/src/test/scala/docs/akka/coordination/LeaseDocSpec.scala) { #lease-usage }
+: @@snip [LeaseDocSpec.scala](/akka-docs/src/test/scala/docs/coordination/LeaseDocSpec.scala) { #lease-usage }
Java
-: @@snip [LeaseDocTest.java](/akka-coordination/src/test/java/jdocs/akka/coordination/lease/LeaseDocTest.java) { #lease-usage }
+: @@snip [LeaseDocTest.java](/akka-docs/src/test/java/jdocs/coordination/LeaseDocTest.java) { #lease-usage }
Acquiring a lease returns a @scala[Future]@java[CompletionStage] as lease implementations typically are implemented
via a third party system such as the Kubernetes API server or Zookeeper.
@@ -53,10 +53,10 @@ It is important to pick a lease name that will be unique for your use case. If a
in a Cluster the cluster host port can be use:
Scala
-: @@snip [LeaseDocSpec.scala](/akka-coordination/src/test/scala/docs/akka/coordination/LeaseDocSpec.scala) { #cluster-owner }
+: @@snip [LeaseDocSpec.scala](/akka-docs/src/test/scala/docs/coordination/LeaseDocSpec.scala) { #cluster-owner }
Java
-: @@snip [LeaseDocTest.scala](/akka-coordination/src/test/java/jdocs/akka/coordination/lease/LeaseDocTest.java) { #cluster-owner }
+: @@snip [LeaseDocTest.scala](/akka-docs/src/test/java/jdocs/coordination/LeaseDocTest.java) { #cluster-owner }
For use cases where multiple different leases on the same node then something unique must be added to the name. For example
a lease can be used with Cluster Sharding and in this case the shard Id is included in the lease name for each shard.
@@ -77,7 +77,7 @@ Leases can be used for @ref[Cluster Singletons](cluster-singleton.md#lease) and
## Lease implementations
-* [Kubernetes API](https://doc.akka.io/docs/akka-enhancements/current/kubernetes-lease.html)
+* [Kubernetes API](https://doc.akka.io/docs/akka-management/current/kubernetes-lease.html)
## Implementing a lease
@@ -85,10 +85,10 @@ Implementations should extend
the @scala[`akka.coordination.lease.scaladsl.Lease`]@java[`akka.coordination.lease.javadsl.Lease`]
Scala
-: @@snip [LeaseDocSpec.scala](/akka-coordination/src/test/scala/docs/akka/coordination/LeaseDocSpec.scala) { #lease-example }
+: @@snip [LeaseDocSpec.scala](/akka-docs/src/test/scala/docs/coordination/LeaseDocSpec.scala) { #lease-example }
Java
-: @@snip [LeaseDocTest.scala](/akka-coordination/src/test/java/jdocs/akka/coordination/lease/LeaseDocTest.java) { #lease-example }
+: @@snip [LeaseDocTest.java](/akka-docs/src/test/java/jdocs/coordination/LeaseDocTest.java) { #lease-example }
The methods should provide the following guarantees:
@@ -109,10 +109,10 @@ The lease implementation should have support for the following properties where
This configuration location is passed into `getLease`.
Scala
-: @@snip [LeaseDocSpec.scala](/akka-coordination/src/test/scala/docs/akka/coordination/LeaseDocSpec.scala) { #lease-config }
+: @@snip [LeaseDocSpec.scala](/akka-docs/src/test/scala/docs/coordination/LeaseDocSpec.scala) { #lease-config }
Java
-: @@snip [LeaseDocSpec.scala](/akka-coordination/src/test/scala/docs/akka/coordination/LeaseDocSpec.scala) { #lease-config }
+: @@snip [LeaseDocSpec.scala](/akka-docs/src/test/scala/docs/coordination/LeaseDocSpec.scala) { #lease-config }
diff --git a/akka-docs/src/main/paradox/discovery/index.md b/akka-docs/src/main/paradox/discovery/index.md
index 5fb4cf93b8..51c13594f4 100644
--- a/akka-docs/src/main/paradox/discovery/index.md
+++ b/akka-docs/src/main/paradox/discovery/index.md
@@ -35,7 +35,7 @@ See @ref:[Migration hints](#migrating-from-akka-management-discovery-before-1-0-
@@dependency[sbt,Gradle,Maven] {
group="com.typesafe.akka"
- artifact="akka-discovery_$scala.binary_version$"
+ artifact="akka-discovery_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/dispatchers.md b/akka-docs/src/main/paradox/dispatchers.md
index 8146f0e7e3..5e33a5e9bb 100644
--- a/akka-docs/src/main/paradox/dispatchers.md
+++ b/akka-docs/src/main/paradox/dispatchers.md
@@ -9,7 +9,7 @@ Dispatchers are part of core Akka, which means that they are part of the akka-ac
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor_$scala.binary_version$"
+ artifact="akka-actor_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/distributed-data.md b/akka-docs/src/main/paradox/distributed-data.md
index f3594bd4a7..c28563c09d 100644
--- a/akka-docs/src/main/paradox/distributed-data.md
+++ b/akka-docs/src/main/paradox/distributed-data.md
@@ -9,7 +9,7 @@ To use Akka Distributed Data, you must add the following dependency in your proj
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-distributed-data_$scala.binary_version$"
+ artifact="akka-distributed-data_$scala.binary.version$"
version="$akka.version$"
}
@@ -268,11 +268,9 @@ For the full documentation of this feature and for new projects see @ref:[Limita
## Learn More about CRDTs
- * [Eventually Consistent Data Structures](https://vimeo.com/43903960)
-talk by Sean Cribbs
* [Strong Eventual Consistency and Conflict-free Replicated Data Types (video)](https://www.youtube.com/watch?v=oyUHd894w18&feature=youtu.be)
talk by Mark Shapiro
- * [A comprehensive study of Convergent and Commutative Replicated Data Types](http://hal.upmc.fr/file/index/docid/555588/filename/techreport.pdf)
+ * [A comprehensive study of Convergent and Commutative Replicated Data Types](https://hal.inria.fr/file/index/docid/555588/filename/techreport.pdf)
paper by Mark Shapiro et. al.
## Configuration
diff --git a/akka-docs/src/main/paradox/distributed-pub-sub.md b/akka-docs/src/main/paradox/distributed-pub-sub.md
index 2f1d8167ea..fd998e8fde 100644
--- a/akka-docs/src/main/paradox/distributed-pub-sub.md
+++ b/akka-docs/src/main/paradox/distributed-pub-sub.md
@@ -9,7 +9,7 @@ To use Distributed Publish Subscribe you must add the following dependency in yo
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-cluster-tools_$scala.binary_version$"
+ artifact="akka-cluster-tools_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/fault-tolerance.md b/akka-docs/src/main/paradox/fault-tolerance.md
index a80c246d20..8c75abcb6c 100644
--- a/akka-docs/src/main/paradox/fault-tolerance.md
+++ b/akka-docs/src/main/paradox/fault-tolerance.md
@@ -9,7 +9,7 @@ The concept of fault tolerance relates to actors, so in order to use these make
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor_$scala.binary_version$"
+ artifact="akka-actor_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/fsm.md b/akka-docs/src/main/paradox/fsm.md
index 7902a16b4d..01c0bfc0a3 100644
--- a/akka-docs/src/main/paradox/fsm.md
+++ b/akka-docs/src/main/paradox/fsm.md
@@ -9,14 +9,14 @@ To use Finite State Machine actors, you must add the following dependency in you
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor_$scala.binary_version$"
+ artifact="akka-actor_$scala.binary.version$"
version="$akka.version$"
}
## Overview
The FSM (Finite State Machine) is available as @scala[a mixin for the] @java[an abstract base class that implements an] Akka Actor and
-is best described in the [Erlang design principles](http://www.erlang.org/documentation/doc-4.8.2/doc/design_principles/fsm.html)
+is best described in the [Erlang design principles](https://www.erlang.org/documentation/doc-4.8.2/doc/design_principles/fsm.html)
A FSM can be described as a set of relations of the form:
diff --git a/akka-docs/src/main/paradox/general/actors.md b/akka-docs/src/main/paradox/general/actors.md
index 386d47710e..89ef1d0d61 100644
--- a/akka-docs/src/main/paradox/general/actors.md
+++ b/akka-docs/src/main/paradox/general/actors.md
@@ -9,7 +9,7 @@ section looks at one such actor in isolation, explaining the concepts you
encounter while implementing it. For a more in depth reference with all the
details please refer to @ref:[Introduction to Actors](../typed/actors.md).
-The [Actor Model](http://en.wikipedia.org/wiki/Actor_model) as defined by
+The [Actor Model](https://en.wikipedia.org/wiki/Actor_model) as defined by
Hewitt, Bishop and Steiger in 1973 is a computational model that expresses
exactly what it means for computation to be distributed. The processing
units—Actors—can only communicate by exchanging messages and upon reception of a
diff --git a/akka-docs/src/main/paradox/general/message-delivery-reliability.md b/akka-docs/src/main/paradox/general/message-delivery-reliability.md
index 5e54816157..825d906697 100644
--- a/akka-docs/src/main/paradox/general/message-delivery-reliability.md
+++ b/akka-docs/src/main/paradox/general/message-delivery-reliability.md
@@ -87,7 +87,7 @@ mailbox would interact with the third point, or even what it would mean to
decide upon the “successfully” part of point five.
Along those same lines goes the reasoning in [Nobody Needs Reliable
-Messaging](http://www.infoq.com/articles/no-reliable-messaging). The only meaningful way for a sender to know whether an
+Messaging](https://www.infoq.com/articles/no-reliable-messaging/). The only meaningful way for a sender to know whether an
interaction was successful is by receiving a business-level acknowledgement
message, which is not something Akka could make up on its own (neither are we
writing a “do what I mean” framework nor would you want us to).
@@ -96,7 +96,7 @@ Akka embraces distributed computing and makes the fallibility of communication
explicit through message passing, therefore it does not try to lie and emulate
a leaky abstraction. This is a model that has been used with great success in
Erlang and requires the users to design their applications around it. You can
-read more about this approach in the [Erlang documentation](http://www.erlang.org/faq/academic.html) (section 10.9 and
+read more about this approach in the [Erlang documentation](https://erlang.org/faq/academic.html) (section 10.9 and
10.10), Akka follows it closely.
Another angle on this issue is that by providing only basic guarantees those
diff --git a/akka-docs/src/main/paradox/general/stream/stream-design.md b/akka-docs/src/main/paradox/general/stream/stream-design.md
index f7c5e9301f..45416068c8 100644
--- a/akka-docs/src/main/paradox/general/stream/stream-design.md
+++ b/akka-docs/src/main/paradox/general/stream/stream-design.md
@@ -104,7 +104,7 @@ A source that emits a stream of streams is still a normal Source, the kind of el
## The difference between Error and Failure
-The starting point for this discussion is the [definition given by the Reactive Manifesto](http://www.reactivemanifesto.org/glossary#Failure). Translated to streams this means that an error is accessible within the stream as a normal data element, while a failure means that the stream itself has failed and is collapsing. In concrete terms, on the Reactive Streams interface level data elements (including errors) are signaled via `onNext` while failures raise the `onError` signal.
+The starting point for this discussion is the [definition given by the Reactive Manifesto](https://www.reactivemanifesto.org/glossary#Failure). Translated to streams this means that an error is accessible within the stream as a normal data element, while a failure means that the stream itself has failed and is collapsing. In concrete terms, on the Reactive Streams interface level data elements (including errors) are signaled via `onNext` while failures raise the `onError` signal.
@@@ note
diff --git a/akka-docs/src/main/paradox/includes/cluster.md b/akka-docs/src/main/paradox/includes/cluster.md
index 6227e376a1..97233be6f9 100644
--- a/akka-docs/src/main/paradox/includes/cluster.md
+++ b/akka-docs/src/main/paradox/includes/cluster.md
@@ -33,6 +33,14 @@ i.e. the sender does not have to know on which node the destination actor is run
+
+### Cluster aware routers
+
+Distribute messages to actors on different nodes in the cluster with routing strategies
+like round-robin and consistent hashing.
+
+
+
### Cluster across multiple data centers
diff --git a/akka-docs/src/main/paradox/index-actors.md b/akka-docs/src/main/paradox/index-actors.md
index f5327d4c94..1b6d49aa4b 100644
--- a/akka-docs/src/main/paradox/index-actors.md
+++ b/akka-docs/src/main/paradox/index-actors.md
@@ -8,7 +8,7 @@ To use Classic Akka Actors, you must add the following dependency in your projec
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor_$scala.binary_version$"
+ artifact="akka-actor_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/index-utilities-classic.md b/akka-docs/src/main/paradox/index-utilities-classic.md
index f6b98d9699..76d88a281d 100644
--- a/akka-docs/src/main/paradox/index-utilities-classic.md
+++ b/akka-docs/src/main/paradox/index-utilities-classic.md
@@ -6,7 +6,7 @@ To use Utilities, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor_$scala.binary_version$"
+ artifact="akka-actor_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/io-tcp.md b/akka-docs/src/main/paradox/io-tcp.md
index 25e04221a7..56a132e032 100644
--- a/akka-docs/src/main/paradox/io-tcp.md
+++ b/akka-docs/src/main/paradox/io-tcp.md
@@ -9,7 +9,7 @@ To use TCP, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor_$scala.binary_version$"
+ artifact="akka-actor_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/io-udp.md b/akka-docs/src/main/paradox/io-udp.md
index ee0c125349..f9ea24ab1a 100644
--- a/akka-docs/src/main/paradox/io-udp.md
+++ b/akka-docs/src/main/paradox/io-udp.md
@@ -9,7 +9,7 @@ To use UDP, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor_$scala.binary_version$"
+ artifact="akka-actor_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/io.md b/akka-docs/src/main/paradox/io.md
index cfb509cdda..6eced33888 100644
--- a/akka-docs/src/main/paradox/io.md
+++ b/akka-docs/src/main/paradox/io.md
@@ -6,7 +6,7 @@ To use I/O, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor_$scala.binary_version$"
+ artifact="akka-actor_$scala.binary.version$"
version="$akka.version$"
}
@@ -93,7 +93,7 @@ To maintain isolation, actors should communicate with immutable objects only. `B
immutable container for bytes. It is used by Akka's I/O system as an efficient, immutable alternative
the traditional byte containers used for I/O on the JVM, such as @scala[`Array[Byte]`]@java[`byte[]`] and `ByteBuffer`.
-`ByteString` is a [rope-like](http://en.wikipedia.org/wiki/Rope_\(computer_science\)) data structure that is immutable
+`ByteString` is a [rope-like](https://en.wikipedia.org/wiki/Rope_\(computer_science\)) data structure that is immutable
and provides fast concatenation and slicing operations (perfect for I/O). When two `ByteString`s are concatenated
together they are both stored within the resulting `ByteString` instead of copying both to a new @scala[`Array`]@java[array]. Operations
such as `drop` and `take` return `ByteString`s that still reference the original @scala[`Array`]@java[array], but just change the
diff --git a/akka-docs/src/main/paradox/logging.md b/akka-docs/src/main/paradox/logging.md
index 4f48a1714e..1b63bb1b86 100644
--- a/akka-docs/src/main/paradox/logging.md
+++ b/akka-docs/src/main/paradox/logging.md
@@ -9,7 +9,7 @@ To use Logging, you must at least use the Akka actors dependency in your project
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor_$scala.binary_version$"
+ artifact="akka-actor_$scala.binary.version$"
version="$akka.version$"
}
@@ -339,7 +339,7 @@ It has a single dependency: the slf4j-api jar. In your runtime, you also need a
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-slf4j_$scala.binary_version$"
+ artifact="akka-slf4j_$scala.binary.version$"
version="$akka.version$"
group2="ch.qos.logback"
artifact2="logback-classic"
@@ -445,7 +445,7 @@ All MDC properties as key-value entries can be included with `%mdc`:
```
- %date{ISO8601} %-5level %logger{36} - %msg {%mdc}%n
+ %date{ISO8601} %-5level %logger{36} - %msg MDC: {%mdc}%n
```
@@ -544,7 +544,7 @@ All MDC properties as key-value entries can be included with `%mdc`:
```
- %date{ISO8601} %-5level %logger{36} - %msg {%mdc}%n
+ %date{ISO8601} %-5level %logger{36} - %msg MDC: {%mdc}%n
```
@@ -574,7 +574,7 @@ The marker can be included in the Logback output with `%marker` and all MDC prop
```
- [%date{ISO8601}] [%level] [%logger] [%marker] [%thread] - %msg {%mdc}%n
+ [%date{ISO8601}] [%level] [%logger] [%marker] [%thread] - %msg MDC: {%mdc}%n
```
diff --git a/akka-docs/src/main/paradox/mailboxes.md b/akka-docs/src/main/paradox/mailboxes.md
index 53267dbd55..e1b110d59b 100644
--- a/akka-docs/src/main/paradox/mailboxes.md
+++ b/akka-docs/src/main/paradox/mailboxes.md
@@ -9,7 +9,7 @@ To use Mailboxes, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor_$scala.binary_version$"
+ artifact="akka-actor_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/multi-node-testing.md b/akka-docs/src/main/paradox/multi-node-testing.md
index ec165631da..db6a786303 100644
--- a/akka-docs/src/main/paradox/multi-node-testing.md
+++ b/akka-docs/src/main/paradox/multi-node-testing.md
@@ -9,7 +9,7 @@ To use Multi Node Testing, you must add the following dependency in your project
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-multi-node-testkit_$scala.binary_version$
+ artifact=akka-multi-node-testkit_$scala.binary.version$
version=$akka.version$
}
diff --git a/akka-docs/src/main/paradox/persistence-fsm.md b/akka-docs/src/main/paradox/persistence-fsm.md
index 073cc6cbf9..a675054a8a 100644
--- a/akka-docs/src/main/paradox/persistence-fsm.md
+++ b/akka-docs/src/main/paradox/persistence-fsm.md
@@ -8,7 +8,7 @@ Persistent FSMs are part of Akka persistence, you must add the following depende
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-persistence_$scala.binary_version$"
+ artifact="akka-persistence_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/persistence-journals.md b/akka-docs/src/main/paradox/persistence-journals.md
index b7411052f9..c73fe34874 100644
--- a/akka-docs/src/main/paradox/persistence-journals.md
+++ b/akka-docs/src/main/paradox/persistence-journals.md
@@ -97,13 +97,13 @@ Don't run snapshot store tasks/futures on the system default dispatcher, since t
## Plugin TCK
-In order to help developers build correct and high quality storage plugins, we provide a Technology Compatibility Kit ([TCK](http://en.wikipedia.org/wiki/Technology_Compatibility_Kit) for short).
+In order to help developers build correct and high quality storage plugins, we provide a Technology Compatibility Kit ([TCK](https://en.wikipedia.org/wiki/Technology_Compatibility_Kit) for short).
The TCK is usable from Java as well as Scala projects. To test your implementation (independently of language) you need to include the akka-persistence-tck dependency:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-persistence-tck_$scala.binary_version$"
+ artifact="akka-persistence-tck_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/persistence-plugins.md b/akka-docs/src/main/paradox/persistence-plugins.md
index 508bc8b04f..0cc2aafbf2 100644
--- a/akka-docs/src/main/paradox/persistence-plugins.md
+++ b/akka-docs/src/main/paradox/persistence-plugins.md
@@ -4,10 +4,11 @@ Storage backends for journals and snapshot stores are pluggable in the Akka pers
A directory of persistence journal and snapshot store plugins is available at the Akka Community Projects page, see [Community plugins](https://akka.io/community/)
-Two popular plugins are:
+Plugins maintained within the Akka organization are:
* [akka-persistence-cassandra](https://doc.akka.io/docs/akka-persistence-cassandra/current/)
-* [akka-persistence-jdbc](https://github.com/dnvriend/akka-persistence-jdbc)
+* [akka-persistence-couchbase](https://doc.akka.io/docs/akka-persistence-couchbase/current/)
+* [akka-persistence-jdbc](https://doc.akka.io/docs/akka-persistence-jdbc/current/)
Plugins can be selected either by "default" for all persistent actors,
or "individually", when a persistent actor defines its own set of plugins.
diff --git a/akka-docs/src/main/paradox/persistence-query-leveldb.md b/akka-docs/src/main/paradox/persistence-query-leveldb.md
index 6066b5c87e..e7b9e0a9a5 100644
--- a/akka-docs/src/main/paradox/persistence-query-leveldb.md
+++ b/akka-docs/src/main/paradox/persistence-query-leveldb.md
@@ -6,7 +6,7 @@ To use Persistence Query, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-persistence-query_$scala.binary_version$
+ artifact=akka-persistence-query_$scala.binary.version$
version=$akka.version$
}
diff --git a/akka-docs/src/main/paradox/persistence-query.md b/akka-docs/src/main/paradox/persistence-query.md
index d29d9d09ac..fc86fe8cf5 100644
--- a/akka-docs/src/main/paradox/persistence-query.md
+++ b/akka-docs/src/main/paradox/persistence-query.md
@@ -9,7 +9,7 @@ To use Persistence Query, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-persistence-query_$scala.binary_version$
+ artifact=akka-persistence-query_$scala.binary.version$
version=$akka.version$
}
@@ -197,7 +197,7 @@ Java
## Performance and denormalization
-When building systems using @ref:[Event sourcing](typed/persistence.md#event-sourcing-concepts) and CQRS ([Command & Query Responsibility Segregation](https://msdn.microsoft.com/en-us/library/jj554200.aspx)) techniques
+When building systems using @ref:[Event sourcing](typed/persistence.md#event-sourcing-concepts) and CQRS ([Command & Query Responsibility Segregation](https://docs.microsoft.com/en-us/previous-versions/msp-n-p/jj554200(v=pandp.10)?redirectedfrom=MSDN)) techniques
it is tremendously important to realise that the write-side has completely different needs from the read-side,
and separating those concerns into datastores that are optimised for either side makes it possible to offer the best
experience for the write and read sides independently.
@@ -221,7 +221,7 @@ it may be more efficient or interesting to query it (instead of the source event
### Materialize view to Reactive Streams compatible datastore
-If the read datastore exposes a [Reactive Streams](http://reactive-streams.org) interface then implementing a simple projection
+If the read datastore exposes a [Reactive Streams](https://www.reactive-streams.org) interface then implementing a simple projection
is as simple as, using the read-journal and feeding it into the databases driver interface, for example like so:
Scala
diff --git a/akka-docs/src/main/paradox/persistence-schema-evolution.md b/akka-docs/src/main/paradox/persistence-schema-evolution.md
index abc9372b5a..bef012ac34 100644
--- a/akka-docs/src/main/paradox/persistence-schema-evolution.md
+++ b/akka-docs/src/main/paradox/persistence-schema-evolution.md
@@ -6,7 +6,7 @@ This documentation page touches upon @ref[Akka Persistence](persistence.md), so
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-persistence_$scala.binary_version$"
+ artifact="akka-persistence_$scala.binary.version$"
version="$akka.version$"
}
@@ -87,13 +87,18 @@ be able to replay events that were persisted using the old serialization scheme.
an event-log from one serialization format to another one, however it may be a more involved process if you need
to perform this on a live system.
-Binary serialization formats that we have seen work well for long-lived applications include the very flexible IDL based:
-[Google Protobuf](https://developers.google.com/protocol-buffers), [Apache Thrift](https://thrift.apache.org/) or [Apache Avro](https://avro.apache.org). Avro schema evolution is more "entire schema" based, instead of
-single fields focused like in protobuf or thrift, and usually requires using some kind of schema registry.
+@ref:[Serialization with Jackson](serialization-jackson.md) is a good choice in many cases and our
+recommendation if you don't have other preference. It also has support for
+@ref:[Schema Evolution](serialization-jackson.md#schema-evolution).
-Users who want their data to be human-readable directly in the write-side
-datastore may opt to use plain-old [JSON](https://json.org) as the storage format, though that comes at a cost of lacking support for schema
-evolution and relatively large marshalling latency.
+[Google Protocol Buffers](https://developers.google.com/protocol-buffers/) is good if you want
+more control over the schema evolution of your messages, but it requires more work to develop and
+maintain the mapping between serialized representation and domain representation.
+
+Binary serialization formats that we have seen work well for long-lived applications include the very flexible IDL based:
+[Google Protocol Buffers](https://developers.google.com/protocol-buffers), [Apache Thrift](https://thrift.apache.org/)
+or [Apache Avro](https://avro.apache.org). Avro schema evolution is more "entire schema" based, instead of
+single fields focused like in protobuf or thrift, and usually requires using some kind of schema registry.
There are plenty excellent blog posts explaining the various trade-offs between popular serialization formats,
one post we would like to highlight is the very well illustrated [Schema evolution in Avro, Protocol Buffers and Thrift](https://martin.kleppmann.com/2012/12/05/schema-evolution-in-avro-protocol-buffers-thrift.html)
@@ -143,9 +148,9 @@ However, once you move to production you should really *pick a different seriali
@@@ warning
-Do not rely on Java serialization (which will be picked by Akka by default if you don't specify any serializers)
-for *serious* application development! It does not lean itself well to evolving schemas over long periods of time,
-and its performance is also not very high (it never was designed for high-throughput scenarios).
+Do not rely on Java serialization for *serious* application development! It does not lean itself well to evolving
+schemas over long periods of time, and its performance is also not very high (it never was designed for high-throughput
+scenarios).
@@@
@@ -157,8 +162,7 @@ it to work with your event classes.
@@@ note
-Read the @ref:[Akka Serialization](serialization.md) docs to learn more about defining custom serializers,
-to improve performance and maintainability of your system. Do not depend on Java serialization for production deployments.
+Read the @ref:[Akka Serialization](serialization.md) docs to learn more about defining custom serializers.
@@@
@@ -199,6 +203,14 @@ some of the various options one might go about handling the described situation.
a complete guide, so feel free to adapt these techniques depending on your serializer's capabilities
and/or other domain specific limitations.
+@@@ note
+
+@ref:[Serialization with Jackson](serialization-jackson.md) has good support for
+@ref:[Schema Evolution](serialization-jackson.md#schema-evolution) and many of the scenarios described here
+can be solved with that Jackson transformation technique instead.
+
+@@@
+
### Add fields
@@ -209,10 +221,8 @@ needs to have an associated code which indicates if it is a window or aisle seat
**Solution:**
Adding fields is the most common change you'll need to apply to your messages so make sure the serialization format
you picked for your payloads can handle it apropriately, i.e. such changes should be *binary compatible*.
-This is achieved using the right serializer toolkit – we recommend something like [Google Protocol Buffers](https://developers.google.com/protocol-buffers/) or
-[Apache Thrift](https://thrift.apache.org/) however other tools may fit your needs just as well – picking a serializer backend is something
-you should research before picking one to run with. In the following examples we will be using protobuf, mostly because
-we are familiar with it, it does its job well and Akka is using it internally as well.
+This is achieved using the right serializer toolkit. In the following examples we will be using protobuf.
+See also @ref:[how to add fields with Jackson](serialization-jackson.md#add-field).
While being able to read messages with missing fields is half of the solution, you also need to deal with the missing
values somehow. This is usually modeled as some kind of default value, or by representing the field as an @scala[`Option[T]`]@java[`Optional`]
@@ -286,8 +296,8 @@ which was set to `1` (because it was the initial schema), and once you change th
and write an adapter which can perform the rename.
This approach is popular when your serialization format is something like JSON, where renames can not be performed
-automatically by the serializer. You can do these kinds of "promotions" either manually (as shown in the example below)
-or using a library like @scala[[Stamina](https://github.com/scalapenos/stamina)]@java[[Stamina](https://github.com/javapenos/stamina)] which helps to create those `V1->V2->V3->...->Vn` promotion chains without much boilerplate.
+automatically by the serializer. See also @ref:[how to rename fields with Jackson](serialization-jackson.md#rename-field),
+which is using this kind of versioning approach.

diff --git a/akka-docs/src/main/paradox/persistence.md b/akka-docs/src/main/paradox/persistence.md
index d586f61f3f..e3abde3dd8 100644
--- a/akka-docs/src/main/paradox/persistence.md
+++ b/akka-docs/src/main/paradox/persistence.md
@@ -12,7 +12,7 @@ To use Akka Persistence, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-persistence_$scala.binary_version$"
+ artifact="akka-persistence_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/project/examples.md b/akka-docs/src/main/paradox/project/examples.md
index 966ffe5182..af9e879ffe 100644
--- a/akka-docs/src/main/paradox/project/examples.md
+++ b/akka-docs/src/main/paradox/project/examples.md
@@ -5,8 +5,8 @@ of how to run.
## Quickstart
-@scala[[Quickstart Guide](https://developer.lightbend.com/guides/akka-quickstart-scala)]
-@java[[Quickstart Guide](https://developer.lightbend.com/guides/akka-quickstart-java)]
+@scala[[Quickstart Guide](https://developer.lightbend.com/guides/akka-quickstart-scala/)]
+@java[[Quickstart Guide](https://developer.lightbend.com/guides/akka-quickstart-java/)]
The *Quickstart* guide walks you through example code that introduces how to define actor systems, actors, and
messages as well as how to use the test module and logging.
diff --git a/akka-docs/src/main/paradox/project/immutable.md b/akka-docs/src/main/paradox/project/immutable.md
index 8ad4668755..957ef91f31 100644
--- a/akka-docs/src/main/paradox/project/immutable.md
+++ b/akka-docs/src/main/paradox/project/immutable.md
@@ -83,5 +83,5 @@ getter, toString, hashCode, equals.
### Integrating Lombok with an IDE
Lombok integrates with popular IDEs:
-* To use Lombok in IntelliJ IDEA you'll need the [Lombok Plugin for IntelliJ IDEA](https://plugins.jetbrains.com/idea/plugin/6317-lombok-plugin) and you'll also need to enable Annotation Processing (`Settings / Build,Execution,Deployment / Compiler / Annotation Processors` and tick `Enable annotation processing`)
+* To use Lombok in IntelliJ IDEA you'll need the [Lombok Plugin for IntelliJ IDEA](https://plugins.jetbrains.com/plugin/6317-lombok) and you'll also need to enable Annotation Processing (`Settings / Build,Execution,Deployment / Compiler / Annotation Processors` and tick `Enable annotation processing`)
* To Use Lombok in Eclipse, run `java -jar lombok.jar` (see the video at [Project Lombok](https://projectlombok.org/)).
diff --git a/akka-docs/src/main/paradox/project/licenses.md b/akka-docs/src/main/paradox/project/licenses.md
index 5f1c69df4c..5d18ca3f50 100644
--- a/akka-docs/src/main/paradox/project/licenses.md
+++ b/akka-docs/src/main/paradox/project/licenses.md
@@ -22,10 +22,10 @@ the License.
## Akka Committer License Agreement
-All committers have signed this [CLA](http://www.lightbend.com/contribute/current-cla).
-It can be [signed online](http://www.lightbend.com/contribute/cla).
+All committers have signed this [CLA](https://www.lightbend.com/contribute/current-cla).
+It can be [signed online](https://www.lightbend.com/contribute/cla).
## Licenses for Dependency Libraries
Each dependency and its license can be seen in the project build file (the comment on the side of each dependency):
-@extref[AkkaBuild.scala](github:project/AkkaBuild.scala#L1054)
\ No newline at end of file
+@extref[AkkaBuild.scala](github:project/AkkaBuild.scala#L1054)
diff --git a/akka-docs/src/main/paradox/project/links.md b/akka-docs/src/main/paradox/project/links.md
index 1c2ce078a5..d7f450ea91 100644
--- a/akka-docs/src/main/paradox/project/links.md
+++ b/akka-docs/src/main/paradox/project/links.md
@@ -2,18 +2,18 @@
## Commercial Support
-Commercial support is provided by [Lightbend](http://www.lightbend.com).
-Akka is part of the [Lightbend Platform](http://www.lightbend.com/platform).
+Commercial support is provided by [Lightbend](https://www.lightbend.com).
+Akka is part of the [Akka Platform](https://www.lightbend.com/akka-platform).
## Sponsors
**Lightbend** is the company behind the Akka Project, Scala Programming Language,
Play Web Framework, Lagom, sbt and many other open source projects.
-It also provides the Lightbend Reactive Platform, which is powered by an open source core and commercial Enterprise Suite for building scalable Reactive systems on the JVM. Learn more at [lightbend.com](http://www.lightbend.com).
+It also provides the Lightbend Reactive Platform, which is powered by an open source core and commercial Enterprise Suite for building scalable Reactive systems on the JVM. Learn more at [lightbend.com](https://www.lightbend.com).
## Akka Discuss Forums
-[Akka Discuss Forums](http://discuss.akka.io)
+[Akka Discuss Forums](https://discuss.akka.io)
## Gitter
@@ -28,7 +28,7 @@ Akka uses Git and is hosted at [Github akka/akka](https://github.com/akka/akka).
## Releases Repository
All Akka releases are published via Sonatype to Maven Central, see
-[search.maven.org](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22com.typesafe.akka%22)
+[search.maven.org](https://search.maven.org/search?q=g:com.typesafe.akka)
## Snapshots Repository
@@ -36,7 +36,7 @@ Nightly builds are available in [https://repo.akka.io/snapshots](https://repo.ak
timestamped versions.
For timestamped versions, pick a timestamp from
-[https://repo.akka.io/snapshots/com/typesafe/akka](https://repo.akka.io/snapshots/com/typesafe/akka).
+[https://repo.akka.io/snapshots/com/typesafe/akka/](https://repo.akka.io/snapshots/com/typesafe/akka/).
All Akka modules that belong to the same build have the same timestamp.
@@@ warning
@@ -57,7 +57,7 @@ Define the library dependencies with the timestamp as version. For example:
@@@vars
```
-libraryDependencies += "com.typesafe.akka" % "akka-remote_$scala.binary_version$" % "2.5-20170510-230859"
+libraryDependencies += "com.typesafe.akka" % "akka-remote_$scala.binary.version$" % "2.5-20170510-230859"
```
@@@
@@ -83,7 +83,7 @@ Define the library dependencies with the timestamp as version. For example:
com.typesafe.akka
- akka-remote_$scala.binary_version$
+ akka-remote_$scala.binary.version$
2.5-20170510-230859
diff --git a/akka-docs/src/main/paradox/project/migration-guide-2.5.x-2.6.x.md b/akka-docs/src/main/paradox/project/migration-guide-2.5.x-2.6.x.md
index ed07987267..6413f37f0f 100644
--- a/akka-docs/src/main/paradox/project/migration-guide-2.5.x-2.6.x.md
+++ b/akka-docs/src/main/paradox/project/migration-guide-2.5.x-2.6.x.md
@@ -280,7 +280,7 @@ Explicitly disable Artery by setting property `akka.remote.artery.enabled` to `f
specific to classic remoting needs to be moved to `akka.remote.classic`. To see which configuration options
are specific to classic search for them in: @ref:[`akka-remote/reference.conf`](../general/configuration-reference.md#config-akka-remote).
-If you have a [Lightbend Platform Subscription](https://www.lightbend.com/lightbend-platform-subscription) you can use our [Config Checker](https://doc.akka.io/docs/akka-enhancements/current/config-checker.html) enhancement to flag any settings that have not been properly migrated.
+If you have a [Lightbend Subscription](https://www.lightbend.com/lightbend-subscription) you can use our [Config Checker](https://doc.akka.io/docs/akka-enhancements/current/config-checker.html) enhancement to flag any settings that have not been properly migrated.
### Persistent mode for Cluster Sharding
diff --git a/akka-docs/src/main/paradox/project/migration-guide-old.md b/akka-docs/src/main/paradox/project/migration-guide-old.md
index 2d1e45e4fd..884d66179d 100644
--- a/akka-docs/src/main/paradox/project/migration-guide-old.md
+++ b/akka-docs/src/main/paradox/project/migration-guide-old.md
@@ -3,7 +3,7 @@
Migration from old versions:
* [2.3.x to 2.4.x](https://doc.akka.io/docs/akka/2.4/project/migration-guide-2.3.x-2.4.x.html)
-* [2.2.x to 2.3.x](https://doc.akka.io/docs/akka/2.3.12/project/migration-guide-2.2.x-2.3.x.html)
-* [2.1.x to 2.2.x](https://doc.akka.io/docs/akka/2.2.3/project/migration-guide-2.1.x-2.2.x.html)
-* [2.0.x to 2.1.x](https://doc.akka.io/docs/akka/2.1.4/project/migration-guide-2.0.x-2.1.x.html)
+* [2.2.x to 2.3.x](https://doc.akka.io/docs/akka/2.3/project/migration-guide-2.2.x-2.3.x.html)
+* [2.1.x to 2.2.x](https://doc.akka.io/docs/akka/2.2/project/migration-guide-2.1.x-2.2.x.html)
+* [2.0.x to 2.1.x](https://doc.akka.io/docs/akka/2.1/project/migration-guide-2.0.x-2.1.x.html)
* [1.3.x to 2.0.x](https://doc.akka.io/docs/akka/2.0.5/project/migration-guide-1.3.x-2.0.x.html).
diff --git a/akka-docs/src/main/paradox/project/rolling-update.md b/akka-docs/src/main/paradox/project/rolling-update.md
index ab7a067ccf..8c5cbbe5c8 100644
--- a/akka-docs/src/main/paradox/project/rolling-update.md
+++ b/akka-docs/src/main/paradox/project/rolling-update.md
@@ -82,7 +82,36 @@ In preparation of switching away from class based manifests to more efficient le
has been prepared to accept those shorter forms but still emits the old long manifests.
* 2.6.2 - shorter manifests accepted
-* 2.6.3 - shorter manifests emitted (not released yet)
+* 2.6.5 - shorter manifests emitted
-This means that a rolling upgrade will have to go through 2.6.2 and 2.6.3 when upgrading to 2.6.3 or higher or else
-cluster nodes will not be able to communicate during the rolling upgrade.
\ No newline at end of file
+This means that a rolling update will have to go through at least one of 2.6.2, 2.6.3 or 2.6.4 when upgrading to
+2.6.5 or higher or else cluster nodes will not be able to communicate during the rolling update.
+
+### 2.6.5 JacksonCborSerializer
+
+Issue: [#28918](https://github.com/akka/akka/issues/28918). JacksonCborSerializer was using plain JSON format
+instead of CBOR.
+
+If you have `jackson-cbor` in your `serialization-bindings` a rolling upgrade will have to go through 2.6.5 when
+upgrading to 2.6.5 or higher.
+
+In Akka 2.6.5 the `jackson-cbor` binding will still serialize to JSON format to support rolling update from 2.6.4.
+It also adds a new binding to be able to deserialize CBOR format when rolling update from 2.6.5 to 2.6.6.
+In Akka 2.6.6 the `jackson-cbor` binding will serialize to CBOR and that can be deserialized by 2.6.5. Old
+data, such as persistent events, can still be deserialized.
+
+You can start using CBOR format already with Akka 2.6.5 without waiting for the 2.6.6 release. First, perform
+a rolling update to Akka 2.6.5 using default configuration. Then change the configuration to:
+
+```
+akka.actor {
+ serializers {
+ jackson-cbor = "akka.serialization.jackson.JacksonCborSerializer"
+ }
+ serialization-identifiers {
+ jackson-cbor = 33
+ }
+}
+```
+
+Perform a second rolling update with the new configuration.
diff --git a/akka-docs/src/main/paradox/remoting-artery.md b/akka-docs/src/main/paradox/remoting-artery.md
index 683d71e95c..ef3a800c13 100644
--- a/akka-docs/src/main/paradox/remoting-artery.md
+++ b/akka-docs/src/main/paradox/remoting-artery.md
@@ -22,7 +22,7 @@ To use Artery Remoting, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-remote_$scala.binary_version$
+ artifact=akka-remote_$scala.binary.version$
version=$akka.version$
}
@@ -311,13 +311,13 @@ According to [RFC 7525](https://tools.ietf.org/html/rfc7525) the recommended alg
You should always check the latest information about security and algorithm recommendations though before you configure your system.
Creating and working with keystores and certificates is well documented in the
-[Generating X.509 Certificates](http://lightbend.github.io/ssl-config/CertificateGeneration.html#using-keytool)
+[Generating X.509 Certificates](https://lightbend.github.io/ssl-config/CertificateGeneration.html#using-keytool)
section of Lightbend's SSL-Config library.
Since an Akka remoting is inherently @ref:[peer-to-peer](general/remoting.md#symmetric-communication) both the key-store as well as trust-store
need to be configured on each remoting node participating in the cluster.
-The official [Java Secure Socket Extension documentation](http://docs.oracle.com/javase/7/docs/technotes/guides/security/jsse/JSSERefGuide.html)
+The official [Java Secure Socket Extension documentation](https://docs.oracle.com/javase/8/docs/technotes/guides/security/jsse/JSSERefGuide.html)
as well as the [Oracle documentation on creating KeyStore and TrustStores](https://docs.oracle.com/cd/E19509-01/820-3503/6nf1il6er/index.html)
are both great resources to research when setting up security on the JVM. Please consult those resources when troubleshooting
and configuring SSL.
@@ -717,7 +717,7 @@ The needed classpath:
Agrona-0.5.4.jar:aeron-driver-1.0.1.jar:aeron-client-1.0.1.jar
```
-You find those jar files on [Maven Central](http://search.maven.org/), or you can create a
+You find those jar files on [Maven Central](https://search.maven.org/), or you can create a
package with your preferred build tool.
You can pass [Aeron properties](https://github.com/real-logic/Aeron/wiki/Configuration-Options) as
diff --git a/akka-docs/src/main/paradox/remoting.md b/akka-docs/src/main/paradox/remoting.md
index fcc7174fe1..4e3597c17a 100644
--- a/akka-docs/src/main/paradox/remoting.md
+++ b/akka-docs/src/main/paradox/remoting.md
@@ -26,7 +26,7 @@ To use Akka Remoting, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-remote_$scala.binary_version$
+ artifact=akka-remote_$scala.binary.version$
version=$akka.version$
}
@@ -486,13 +486,13 @@ According to [RFC 7525](https://tools.ietf.org/html/rfc7525) the recommended alg
You should always check the latest information about security and algorithm recommendations though before you configure your system.
Creating and working with keystores and certificates is well documented in the
-[Generating X.509 Certificates](http://lightbend.github.io/ssl-config/CertificateGeneration.html#using-keytool)
+[Generating X.509 Certificates](https://lightbend.github.io/ssl-config/CertificateGeneration.html#using-keytool)
section of Lightbend's SSL-Config library.
Since an Akka remoting is inherently @ref:[peer-to-peer](general/remoting.md#symmetric-communication) both the key-store as well as trust-store
need to be configured on each remoting node participating in the cluster.
-The official [Java Secure Socket Extension documentation](http://docs.oracle.com/javase/7/docs/technotes/guides/security/jsse/JSSERefGuide.html)
+The official [Java Secure Socket Extension documentation](https://docs.oracle.com/javase/8/docs/technotes/guides/security/jsse/JSSERefGuide.html)
as well as the [Oracle documentation on creating KeyStore and TrustStores](https://docs.oracle.com/cd/E19509-01/820-3503/6nf1il6er/index.html)
are both great resources to research when setting up security on the JVM. Please consult those resources when troubleshooting
and configuring SSL.
diff --git a/akka-docs/src/main/paradox/routing.md b/akka-docs/src/main/paradox/routing.md
index ddd3d01f76..feb311203d 100644
--- a/akka-docs/src/main/paradox/routing.md
+++ b/akka-docs/src/main/paradox/routing.md
@@ -9,7 +9,7 @@ To use Routing, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor_$scala.binary_version$"
+ artifact="akka-actor_$scala.binary.version$"
version="$akka.version$"
}
@@ -257,7 +257,7 @@ Java
### RoundRobinPool and RoundRobinGroup
-Routes in a [round-robin](http://en.wikipedia.org/wiki/Round-robin) fashion to its routees.
+Routes in a [round-robin](https://en.wikipedia.org/wiki/Round-robin) fashion to its routees.
RoundRobinPool defined in configuration:
@@ -598,7 +598,7 @@ Java
### ConsistentHashingPool and ConsistentHashingGroup
-The ConsistentHashingPool uses [consistent hashing](http://en.wikipedia.org/wiki/Consistent_hashing)
+The ConsistentHashingPool uses [consistent hashing](https://en.wikipedia.org/wiki/Consistent_hashing)
to select a routee based on the sent message. This
[article](http://www.tom-e-white.com/2007/11/consistent-hashing.html) gives good
insight into how consistent hashing is implemented.
diff --git a/akka-docs/src/main/paradox/scheduler.md b/akka-docs/src/main/paradox/scheduler.md
index 3225e40b4f..600edeae22 100644
--- a/akka-docs/src/main/paradox/scheduler.md
+++ b/akka-docs/src/main/paradox/scheduler.md
@@ -12,7 +12,7 @@ To use Scheduler, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor_$scala.binary_version$"
+ artifact="akka-actor_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/security/2017-02-10-java-serialization.md b/akka-docs/src/main/paradox/security/2017-02-10-java-serialization.md
index 4fa590288f..82e40e849f 100644
--- a/akka-docs/src/main/paradox/security/2017-02-10-java-serialization.md
+++ b/akka-docs/src/main/paradox/security/2017-02-10-java-serialization.md
@@ -27,7 +27,7 @@ Please subscribe to the [akka-security](https://groups.google.com/forum/#!forum/
### Severity
-The [CVSS](https://en.wikipedia.org/wiki/CVSS) score of this vulnerability is 6.8 (Medium), based on vector [AV:A/AC:M/Au:N/C:C/I:C/A:C/E:F/RL:TF/RC:C](https://nvd.nist.gov/cvss.cfm?calculator&version=2&vector=\(AV:A/AC:M/Au:N/C:C/I:C/A:C/E:F/RL:TF/RC:C\)).
+The [CVSS](https://en.wikipedia.org/wiki/CVSS) score of this vulnerability is 6.8 (Medium), based on vector [AV:A/AC:M/Au:N/C:C/I:C/A:C/E:F/RL:TF/RC:C](https://nvd.nist.gov/vuln-metrics/cvss/v2-calculator?calculator&version=2&vector=%5C(AV:A/AC:M/Au:N/C:C/I:C/A:C/E:F/RL:TF/RC:C%5C)).
Rationale for the score:
diff --git a/akka-docs/src/main/paradox/serialization-classic.md b/akka-docs/src/main/paradox/serialization-classic.md
index 01debcb51e..03f324b707 100644
--- a/akka-docs/src/main/paradox/serialization-classic.md
+++ b/akka-docs/src/main/paradox/serialization-classic.md
@@ -11,7 +11,7 @@ To use Serialization, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor_$scala.binary_version$"
+ artifact="akka-actor_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/serialization-jackson.md b/akka-docs/src/main/paradox/serialization-jackson.md
index 7db9903329..775344ab92 100644
--- a/akka-docs/src/main/paradox/serialization-jackson.md
+++ b/akka-docs/src/main/paradox/serialization-jackson.md
@@ -9,7 +9,7 @@ To use Jackson Serialization, you must add the following dependency in your proj
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-serialization-jackson_$scala.binary_version$"
+ artifact="akka-serialization-jackson_$scala.binary.version$"
version="$akka.version$"
}
@@ -377,7 +377,9 @@ the `jackson-json` binding the default configuration is:
@@snip [reference.conf](/akka-serialization-jackson/src/main/resources/reference.conf) { #compression }
-Messages larger than the `compress-larger-than` property are compressed with GZIP.
+Supported compression algorithms are: gzip, lz4. Use 'off' to disable compression.
+Gzip is generally slower than lz4.
+Messages larger than the `compress-larger-than` property are compressed.
Compression can be disabled by setting the `algorithm` property to `off`. It will still be able to decompress
payloads that were compressed when serialized, e.g. if this configuration is changed.
diff --git a/akka-docs/src/main/paradox/serialization.md b/akka-docs/src/main/paradox/serialization.md
index 718fed94ab..8970b19d1f 100644
--- a/akka-docs/src/main/paradox/serialization.md
+++ b/akka-docs/src/main/paradox/serialization.md
@@ -9,7 +9,7 @@ To use Serialization, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor_$scala.binary_version$"
+ artifact="akka-actor_$scala.binary.version$"
version="$akka.version$"
}
@@ -59,7 +59,7 @@ you would need to reference it as `Wrapper$Message` instead of `Wrapper.Message`
@@@
-Akka provides serializers for several primitive types and [protobuf](http://code.google.com/p/protobuf/)
+Akka provides serializers for several primitive types and [protobuf](https://github.com/protocolbuffers/protobuf)
`com.google.protobuf.GeneratedMessage` (protobuf2) and `com.google.protobuf.GeneratedMessageV3` (protobuf3) by default (the latter only if
depending on the akka-remote module), so normally you don't need to add
configuration for that if you send raw protobuf messages as actor messages.
@@ -82,6 +82,7 @@ Scala
Java
: @@snip [SerializationDocTest.java](/akka-docs/src/test/java/jdocs/serialization/SerializationDocTest.java) { #programmatic }
+
The manifest is a type hint so that the same serializer can be used for different classes.
Note that when deserializing from bytes the manifest and the identifier of the serializer are needed.
@@ -119,6 +120,21 @@ Scala
Java
: @@snip [SerializationDocTest.java](/akka-docs/src/test/java/jdocs/serialization/SerializationDocTest.java) { #my-own-serializer }
+The `identifier` must be unique. The identifier is used when selecting which serializer to use for deserialization.
+If you have accidentally configured several serializers with the same identifier that will be detected and prevent
+the `ActorSystem` from being started. It can be a hardcoded value because it must remain the same value to support
+rolling updates.
+
+@@@ div { .group-scala }
+
+If you prefer to define the identifier in cofiguration that is supported by the `BaseSerializer` trait, which
+implements the `def identifier` by reading it from configuration based on the serializer's class name:
+
+Scala
+: @@snip [SerializationDocSpec.scala](/akka-docs/src/test/scala/docs/serialization/SerializationDocSpec.scala) { #serialization-identifiers-config }
+
+@@@
+
The manifest is a type hint so that the same serializer can be used for different
classes. The manifest parameter in @scala[`fromBinary`]@java[`fromBinaryJava`] is the class of the object that
was serialized. In `fromBinary` you can match on the class and deserialize the
diff --git a/akka-docs/src/main/paradox/split-brain-resolver.md b/akka-docs/src/main/paradox/split-brain-resolver.md
new file mode 100644
index 0000000000..e6886d22c6
--- /dev/null
+++ b/akka-docs/src/main/paradox/split-brain-resolver.md
@@ -0,0 +1,481 @@
+# Split Brain Resolver
+
+When operating an Akka cluster you must consider how to handle
+[network partitions](http://en.wikipedia.org/wiki/Network_partition) (a.k.a. split brain scenarios)
+and machine crashes (including JVM and hardware failures). This is crucial for correct behavior if
+you use @ref:[Cluster Singleton](typed/cluster-singleton.md) or @ref:[Cluster Sharding](typed/cluster-sharding.md),
+especially together with Akka Persistence.
+
+## Module info
+
+To use Akka Split Brain Resolver is part of `akka-cluster` and you probably already have that
+dependency included. Otherwise, add the following dependency in your project:
+
+@@dependency[sbt,Maven,Gradle] {
+ group=com.typesafe.akka
+ artifact=akka-cluster_$scala.binary.version$
+ version=$akka.version$
+}
+
+@@project-info{ projectId="akka-cluster" }
+
+## Enable the Split Brain Resolver
+
+You need to enable the Split Brain Resolver by configuring it as downing provider in the configuration of
+the `ActorSystem` (`application.conf`):
+
+```
+akka.cluster.downing-provider-class = "akka.cluster.sbr.SplitBrainResolverProvider"
+```
+
+You should also consider the different available @ref:[downing strategies](#strategies).
+
+## The Problem
+
+A fundamental problem in distributed systems is that network partitions (split brain scenarios) and
+machine crashes are indistinguishable for the observer, i.e. a node can observe that there is a problem
+with another node, but it cannot tell if it has crashed and will never be available again or if there is
+a network issue that might or might not heal again after a while. Temporary and permanent failures are
+indistinguishable because decisions must be made in finite time, and there always exists a temporary
+failure that lasts longer than the time limit for the decision.
+
+A third type of problem is if a process is unresponsive, e.g. because of overload, CPU starvation or
+long garbage collection pauses. This is also indistinguishable from network partitions and crashes.
+The only signal we have for decision is "no reply in given time for heartbeats" and this means that
+phenomena causing delays or lost heartbeats are indistinguishable from each other and must be
+handled in the same way.
+
+When there is a crash, we would like to remove the affected node immediately from the cluster membership.
+When there is a network partition or unresponsive process we would like to wait for a while in the hope
+that it is a transient problem that will heal again, but at some point, we must give up and continue with
+the nodes on one side of the partition and shut down nodes on the other side. Also, certain features are
+not fully available during partitions so it might not matter that the partition is transient or not if
+it just takes too long. Those two goals are in conflict with each other and there is a trade-off
+between how quickly we can remove a crashed node and premature action on transient network partitions.
+
+This is a difficult problem to solve given that the nodes on the different sides of the network partition
+cannot communicate with each other. We must ensure that both sides can make this decision by themselves and
+that they take the same decision about which part will keep running and which part will shut itself down.
+
+Another type of problem that makes it difficult to see the "right" picture is when some nodes are not fully
+connected and cannot communicate directly to each other but information can be disseminated between them via
+other nodes.
+
+The Akka cluster has a failure detector that will notice network partitions and machine crashes (but it
+cannot distinguish the two). It uses periodic heartbeat messages to check if other nodes are available
+and healthy. These observations by the failure detector are referred to as a node being *unreachable*
+and it may become *reachable* again if the failure detector observes that it can communicate with it again.
+
+The failure detector in itself is not enough for making the right decision in all situations.
+The naive approach is to remove an unreachable node from the cluster membership after a timeout.
+This works great for crashes and short transient network partitions, but not for long network
+partitions. Both sides of the network partition will see the other side as unreachable and
+after a while remove it from its cluster membership. Since this happens on both sides the result
+is that two separate disconnected clusters have been created.
+This approach is provided by the opt-in (off by default) auto-down feature in the OSS version of
+Akka Cluster.
+
+If you use the timeout based auto-down feature in combination with Cluster Singleton or Cluster Sharding
+that would mean that two singleton instances or two sharded entities with the same identifier would be running.
+One would be running: one in each cluster.
+For example when used together with Akka Persistence that could result in that two instances of a
+persistent actor with the same `persistenceId` are running and writing concurrently to the
+same stream of persistent events, which will have fatal consequences when replaying these events.
+
+The default setting in Akka Cluster is to not remove unreachable nodes automatically and
+the recommendation is that the decision of what to
+do should be taken by a human operator or an external monitoring system. This is a valid solution,
+but not very convenient if you do not have this staff or external system for other reasons.
+
+If the unreachable nodes are not downed at all they will still be part of the cluster membership.
+Meaning that Cluster Singleton and Cluster Sharding will not failover to another node. While there
+are unreachable nodes new nodes that are joining the cluster will not be promoted to full worthy
+members (with status Up). Similarly, leaving members will not be removed until all unreachable
+nodes have been resolved. In other words, keeping unreachable members for an unbounded time is
+undesirable.
+
+With that introduction of the problem domain, it is time to look at the provided strategies for
+handling network partition, unresponsive nodes and crashed nodes.
+
+## Strategies
+
+By default the @ref:[Keep Majority](#keep-majority) strategy will be used because it works well for
+most systems. However, it's wort considering the other available strategies and pick a strategy that fits
+the characteristics of your system. For example, in a Kubernetes environment the @ref:[Lease](#lease) strategy
+can be a good choice.
+
+Every strategy has a failure scenario where it makes a "wrong" decision. This section describes the different
+strategies and guidelines of when to use what.
+
+When there is uncertainty it selects to down more nodes than necessary, or even downing of all nodes.
+Therefore Split Brain Resolver should always be combined with a mechanism to automatically start up nodes that
+have been shutdown, and join them to the existing cluster or form a new cluster again.
+
+You enable a strategy with the configuration property `akka.cluster.split-brain-resolver.active-strategy`.
+
+### Stable after
+
+All strategies are inactive until the cluster membership and the information about unreachable nodes
+have been stable for a certain time period. Continuously adding more nodes while there is a network
+partition does not influence this timeout, since the status of those nodes will not be changed to Up
+while there are unreachable nodes. Joining nodes are not counted in the logic of the strategies.
+
+@@snip [reference.conf](/akka-cluster/src/main/resources/reference.conf) { #split-brain-resolver }
+
+Set `akka.cluster.split-brain-resolver.stable-after` to a shorter duration to have quicker removal of crashed nodes,
+at the price of risking too early action on transient network partitions that otherwise would have healed. Do not
+set this to a shorter duration than the membership dissemination time in the cluster, which depends
+on the cluster size. Recommended minimum duration for different cluster sizes:
+
+|cluster size | stable-after|
+|-------------|-------------|
+|5 | 7 s |
+|10 | 10 s|
+|20 | 13 s|
+|50 | 17 s|
+|100 | 20 s|
+|1000 | 30 s|
+
+The different strategies may have additional settings that are described below.
+
+@@@ note
+
+It is important that you use the same configuration on all nodes.
+
+@@@
+
+The side of the split that decides to shut itself down will use the cluster *down* command
+to initiate the removal of a cluster member. When that has been spread among the reachable nodes
+it will be removed from the cluster membership.
+
+It's good to terminate the `ActorSystem` and exit the JVM when the node is removed from the cluster.
+
+That is handled by @ref:[Coordinated Shutdown](coordinated-shutdown.md)
+but to exit the JVM it's recommended that you enable:
+
+```
+akka.coordinated-shutdown.exit-jvm = on
+```
+
+@@@ note
+
+Some legacy containers may block calls to System.exit(..) and you may have to find an alternate
+way to shut the app down. For example, when running Akka on top of a Spring / Tomcat setup, you
+could replace the call to `System.exit(..)` with a call to Spring's ApplicationContext .close() method
+(or with a HTTP call to Tomcat Manager's API to un-deploy the app).
+
+@@@
+
+### Keep Majority
+
+The strategy named `keep-majority` will down the unreachable nodes if the current node is in
+the majority part based on the last known membership information. Otherwise down the reachable nodes,
+i.e. the own part. If the parts are of equal size the part containing the node with the lowest
+address is kept.
+
+This strategy is a good choice when the number of nodes in the cluster change dynamically and you can
+therefore not use `static-quorum`.
+
+This strategy also handles the edge case that may occur when there are membership changes at the same
+time as the network partition occurs. For example, the status of two members are changed to `Up`
+on one side but that information is not disseminated to the other side before the connection is broken.
+Then one side sees two more nodes and both sides might consider themselves having a majority. It will
+detect this situation and make the safe decision to down all nodes on the side that could be in minority
+if the joining nodes were changed to `Up` on the other side. Note that this has the drawback that
+if the joining nodes were not changed to `Up` and becoming a majority on the other side then each part
+will shut down itself, terminating the whole cluster.
+
+Note that if there are more than two partitions and none is in majority each part will shut down
+itself, terminating the whole cluster.
+
+If more than half of the nodes crash at the same time the other running nodes will down themselves
+because they think that they are not in majority, and thereby the whole cluster is terminated.
+
+The decision can be based on nodes with a configured `role` instead of all nodes in the cluster.
+This can be useful when some types of nodes are more valuable than others. You might for example
+have some nodes responsible for persistent data and some nodes with stateless worker services.
+Then it probably more important to keep as many persistent data nodes as possible even though
+it means shutting down more worker nodes.
+
+Configuration:
+
+```
+akka.cluster.split-brain-resolver.active-strategy=keep-majority
+```
+
+@@snip [reference.conf](/akka-cluster/src/main/resources/reference.conf) { #keep-majority }
+
+### Static Quorum
+
+The strategy named `static-quorum` will down the unreachable nodes if the number of remaining
+nodes are greater than or equal to a configured `quorum-size`. Otherwise, it will down the reachable nodes,
+i.e. it will shut down that side of the partition. In other words, the `quorum-size` defines the minimum
+number of nodes that the cluster must have to be operational.
+
+This strategy is a good choice when you have a fixed number of nodes in the cluster, or when you can
+define a fixed number of nodes with a certain role.
+
+For example, in a 9 node cluster you will configure the `quorum-size` to 5. If there is a network split
+of 4 and 5 nodes the side with 5 nodes will survive and the other 4 nodes will be downed. After that,
+in the 5 node cluster, no more failures can be handled, because the remaining cluster size would be
+less than 5. In the case of another failure in that 5 node cluster all nodes will be downed.
+
+Therefore it is important that you join new nodes when old nodes have been removed.
+
+Another consequence of this is that if there are unreachable nodes when starting up the cluster,
+before reaching this limit, the cluster may shut itself down immediately. This is not an issue
+if you start all nodes at approximately the same time or use the `akka.cluster.min-nr-of-members`
+to define required number of members before the leader changes member status of 'Joining' members to 'Up'
+You can tune the timeout after which downing decisions are made using the `stable-after` setting.
+
+You should not add more members to the cluster than **quorum-size * 2 - 1**. A warning is logged
+if this recommendation is violated. If the exceeded cluster size remains when a SBR decision is
+needed it will down all nodes because otherwise there is a risk that both sides may down each
+other and thereby form two separate clusters.
+
+For rolling updates it's best to leave the cluster gracefully via
+@ref:[Coordinated Shutdown](coordinated-shutdown.md) (SIGTERM).
+For successful leaving SBR will not be used (no downing) but if there is an unreachability problem
+at the same time as the rolling update is in progress there could be an SBR decision. To avoid that
+the total number of members limit is not exceeded during the rolling update it's recommended to
+leave and fully remove one node before adding a new one, when using `static-quorum`.
+
+If the cluster is split into 3 (or more) parts each part that is smaller than then configured `quorum-size`
+will down itself and possibly shutdown the whole cluster.
+
+If more nodes than the configured `quorum-size` crash at the same time the other running nodes
+will down themselves because they think that they are not in the majority, and thereby the whole
+cluster is terminated.
+
+The decision can be based on nodes with a configured `role` instead of all nodes in the cluster.
+This can be useful when some types of nodes are more valuable than others. You might, for example,
+have some nodes responsible for persistent data and some nodes with stateless worker services.
+Then it probably more important to keep as many persistent data nodes as possible even though
+it means shutting down more worker nodes.
+
+There is another use of the `role` as well. By defining a `role` for a few (e.g. 7) stable
+nodes in the cluster and using that in the configuration of `static-quorum` you will be able
+to dynamically add and remove other nodes without this role and still have good decisions of what
+nodes to keep running and what nodes to shut down in the case of network partitions. The advantage
+of this approach compared to `keep-majority` (described below) is that you *do not* risk splitting
+the cluster into two separate clusters, i.e. *a split brain**. You must still obey the rule of not
+starting too many nodes with this `role` as described above. It also suffers the risk of shutting
+down all nodes if there is a failure when there are not enough nodes with this `role` remaining
+in the cluster, as described above.
+
+Configuration:
+
+```
+akka.cluster.split-brain-resolver.active-strategy=static-quorum
+```
+
+@@snip [reference.conf](/akka-cluster/src/main/resources/reference.conf) { #static-quorum }
+
+### Keep Oldest
+
+The strategy named `keep-oldest` will down the part that does not contain the oldest
+member. The oldest member is interesting because the active Cluster Singleton instance
+is running on the oldest member.
+
+There is one exception to this rule if `down-if-alone` is configured to `on`.
+Then, if the oldest node has partitioned from all other nodes the oldest will down itself
+and keep all other nodes running. The strategy will not down the single oldest node when
+it is the only remaining node in the cluster.
+
+Note that if the oldest node crashes the others will remove it from the cluster
+when `down-if-alone` is `on`, otherwise they will down themselves if the
+oldest node crashes, i.e. shut down the whole cluster together with the oldest node.
+
+This strategy is good to use if you use Cluster Singleton and do not want to shut down the node
+where the singleton instance runs. If the oldest node crashes a new singleton instance will be
+started on the next oldest node. The drawback is that the strategy may keep only a few nodes
+in a large cluster. For example, if one part with the oldest consists of 2 nodes and the
+other part consists of 98 nodes then it will keep 2 nodes and shut down 98 nodes.
+
+This strategy also handles the edge case that may occur when there are membership changes at the same
+time as the network partition occurs. For example, the status of the oldest member is changed to `Exiting`
+on one side but that information is not disseminated to the other side before the connection is broken.
+It will detect this situation and make the safe decision to down all nodes on the side that sees the oldest as
+`Leaving`. Note that this has the drawback that if the oldest was `Leaving` and not changed to `Exiting` then
+each part will shut down itself, terminating the whole cluster.
+
+The decision can be based on nodes with a configured `role` instead of all nodes in the cluster,
+i.e. using the oldest member (singleton) within the nodes with that role.
+
+Configuration:
+
+```
+akka.cluster.split-brain-resolver.active-strategy=keep-oldest
+```
+
+@@snip [reference.conf](/akka-cluster/src/main/resources/reference.conf) { #keep-oldest }
+
+### Down All
+
+The strategy named `down-all` will down all nodes.
+
+This strategy can be a safe alternative if the network environment is highly unstable with unreachability observations
+that can't be fully trusted, and including frequent occurrences of @ref:[indirectly connected nodes](#indirectly-connected-nodes).
+Due to the instability there is an increased risk of different information on different sides of partitions and
+therefore the other strategies may result in conflicting decisions. In such environments it can be better to shutdown
+all nodes and start up a new fresh cluster.
+
+Shutting down all nodes means that the system will be completely unavailable until nodes have been restarted and
+formed a new cluster. This strategy is not recommended for large clusters (> 10 nodes) because any minor problem
+will shutdown all nodes, and that is more likely to happen in larger clusters since there are more nodes that
+may fail.
+
+See also @ref[Down all when unstable](#down-all-when-unstable) and @ref:[indirectly connected nodes](#indirectly-connected-nodes).
+
+### Lease
+
+The strategy named `lease-majority` is using a distributed lease (lock) to decide what nodes that are allowed to
+survive. Only one SBR instance can acquire the lease make the decision to remain up. The other side will
+not be able to aquire the lease and will therefore down itself.
+
+Best effort is to keep the side that has most nodes, i.e. the majority side. This is achieved by adding a delay
+before trying to acquire the lease on the minority side.
+
+There is currently one supported implementation of the lease which is backed by a
+[Custom Resource Definition (CRD)](https://kubernetes.io/docs/concepts/extend-kubernetes/api-extension/custom-resources/)
+in Kubernetes. It is described in the [Kubernetes Lease](https://doc.akka.io/docs/akka-management/current/kubernetes-lease.html)
+documentation.
+
+This strategy is very safe since coordination is added by an external arbiter. The trade-off compared to other
+strategies is that it requires additional infrastructure for implementing the lease and it reduces the availability
+of a decision to that of the system backing the lease store.
+
+Similar to other strategies it is important that decisions are not deferred for too because the nodes that couldn't
+acquire the lease must decide to down themselves, see @ref[Down all when unstable](#down-all-when-unstable).
+
+In some cases the lease will be unavailable when needed for a decision from all SBR instances, e.g. because it is
+on another side of a network partition, and then all nodes will be downed.
+
+Configuration:
+
+```
+akka {
+ cluster {
+ downing-provider-class = "akka.cluster.sbr.SplitBrainResolverProvider"
+ split-brain-resolver {
+ active-strategy = "lease-majority"
+ lease-majority {
+ lease-implementation = "akka.lease.kubernetes"
+ }
+ }
+ }
+}
+```
+
+@@snip [reference.conf](/akka-cluster/src/main/resources/reference.conf) { #lease-majority }
+
+See also configuration and additional dependency in [Kubernetes Lease](https://doc.akka.io/docs/akka-management/current/kubernetes-lease.html)
+
+## Indirectly connected nodes
+
+In a malfunctional network there can be situations where nodes are observed as unreachable via some network
+links but they are still indirectly connected via other nodes, i.e. it's not a clean network partition (or node crash).
+
+When this situation is detected the Split Brain Resolvers will keep fully connected nodes and down all the indirectly
+connected nodes.
+
+If there is a combination of indirectly connected nodes and a clean network partition it will combine the
+above decision with the ordinary decision, e.g. keep majority, after excluding suspicious failure detection
+observations.
+
+## Down all when unstable
+
+When reachability observations by the failure detector are changed the SBR decisions
+are deferred until there are no changes within the `stable-after` duration.
+If this continues for too long it might be an indication of an unstable system/network
+and it could result in delayed or conflicting decisions on separate sides of a network
+partition.
+
+As a precaution for that scenario all nodes are downed if no decision is made within
+`stable-after + down-all-when-unstable` from the first unreachability event.
+The measurement is reset if all unreachable have been healed, downed or removed, or
+if there are no changes within `stable-after * 2`.
+
+This is enabled by default for all strategies and by default the duration is derived to
+be 3/4 of `stable-after`.
+
+The below property can be defined as a duration of for how long the changes are acceptable to
+continue after the `stable-after` or it can be set to `off` to disable this feature.
+
+
+```
+akka.cluster.split-brain-resolver {
+ down-all-when-unstable = 15s
+ stable-after = 20s
+}
+```
+
+@@@ warning
+
+It is recommended to keep `down-all-when-unstable` enabled and not set it to a longer duration than `stable-after`
+(`down-removal-margin`) because that can result in delayed decisions on the side that should have been downed, e.g.
+in the case of a clean network partition followed by continued instability on the side that should be downed.
+That could result in that members are removed from one side but are still running on the other side.
+
+@@@
+
+## Multiple data centers
+
+Akka Cluster has @ref:[support for multiple data centers](cluster-dc.md), where the cluster
+membership is managed by each data center separately and independently of network partitions across different
+data centers. The Split Brain Resolver is embracing that strategy and will not count nodes or down nodes in
+another data center.
+
+When there is a network partition across data centers the typical solution is to wait the partition out until it heals, i.e.
+do nothing. Other decisions should be performed by an external monitoring tool or human operator.
+
+## Cluster Singleton and Cluster Sharding
+
+The purpose of Cluster Singleton and Cluster Sharding is to run at most one instance
+of a given actor at any point in time. When such an instance is shut down a new instance
+is supposed to be started elsewhere in the cluster. It is important that the new instance is
+not started before the old instance has been stopped. This is especially important when the
+singleton or the sharded instance is persistent, since there must only be one active
+writer of the journaled events of a persistent actor instance.
+
+Since the strategies on different sides of a network partition cannot communicate with each other
+and they may take the decision at slightly different points in time there must be a time based
+margin that makes sure that the new instance is not started before the old has been stopped.
+
+You would like to configure this to a short duration to have quick failover, but that will increase the
+risk of having multiple singleton/sharded instances running at the same time and it may take a different
+amount of time to act on the decision (dissemination of the down/removal). The duration is by default
+the same as the `stable-after` property (see @ref:[Stable after](#stable-after) above). It is recommended to
+leave this value as is, but it can also be separately overriden with the `akka.cluster.down-removal-margin` property.
+
+Another concern for setting this `stable-after`/`akka.cluster.down-removal-margin` is dealing with JVM pauses e.g.
+garbage collection. When a node is unresponsive it is not known if it is due to a pause, overload, a crash or a
+network partition. If it is pause that lasts longer than `stable-after` * 2 it gives time for SBR to down the node
+and for singletons and shards to be started on other nodes. When the node un-pauses there will be a short time before
+it sees its self as down where singletons and sharded actors are still running. It is therefore important to understand
+the max pause time your application is likely to incur and make sure it is smaller than `stable-margin`.
+
+If you choose to set a separate value for `down-removal-margin`, the recommended minimum duration for different cluster sizes are:
+
+|cluster size | down-removal-margin|
+|-------------|--------------------|
+|5 | 7 s |
+|10 | 10 s|
+|20 | 13 s|
+|50 | 17 s|
+|100 | 20 s|
+|1000 | 30 s|
+
+### Expected Failover Time
+
+As you have seen, there are several configured timeouts that add to the total failover latency.
+With default configuration those are:
+
+ * failure detection 5 seconds
+ * stable-after 20 seconds
+ * down-removal-margin (by default the same as stable-after) 20 seconds
+
+In total, you can expect the failover time of a singleton or sharded instance to be around 45 seconds
+with default configuration. The default configuration is sized for a cluster of 100 nodes. If you have
+around 10 nodes you can reduce the `stable-after` to around 10 seconds,
+resulting in an expected failover time of around 25 seconds.
diff --git a/akka-docs/src/main/paradox/stream/actor-interop.md b/akka-docs/src/main/paradox/stream/actor-interop.md
index 2e8386e4b6..6b52523b43 100644
--- a/akka-docs/src/main/paradox/stream/actor-interop.md
+++ b/akka-docs/src/main/paradox/stream/actor-interop.md
@@ -6,7 +6,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/stream/futures-interop.md b/akka-docs/src/main/paradox/stream/futures-interop.md
index 9ec22c8dd9..4901ec5422 100644
--- a/akka-docs/src/main/paradox/stream/futures-interop.md
+++ b/akka-docs/src/main/paradox/stream/futures-interop.md
@@ -6,7 +6,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/stream/index.md b/akka-docs/src/main/paradox/stream/index.md
index 3d51561f0c..1dfce673b4 100644
--- a/akka-docs/src/main/paradox/stream/index.md
+++ b/akka-docs/src/main/paradox/stream/index.md
@@ -9,7 +9,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/stream/operators/ActorFlow/ask.md b/akka-docs/src/main/paradox/stream/operators/ActorFlow/ask.md
index b70f1a4b43..0eb9276624 100644
--- a/akka-docs/src/main/paradox/stream/operators/ActorFlow/ask.md
+++ b/akka-docs/src/main/paradox/stream/operators/ActorFlow/ask.md
@@ -1,6 +1,6 @@
# ActorFlow.ask
-Use the `AskPattern` to send each element as an `ask` to the target actor, and expect a reply back that will be sent further downstream.
+Use the "Ask Pattern" to send each stream element as an `ask` to the target actor (of the new actors API), and expect a reply back that will be emitted downstream.
@ref[Actor interop operators](../index.md#actor-interop-operators)
@@ -10,29 +10,52 @@ This operator is included in:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream-typed_$scala.binary_version$"
+ artifact="akka-stream-typed_$scala.binary.version$"
version="$akka.version$"
}
-@@@div { .group-scala }
-
## Signature
-@@signature [ActorFlow.scala](/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorFlow.scala) { #ask }
-
-@@@
+@apidoc[ActorFlow.ask](ActorFlow$) { scala="#ask%5BI,Q,A](ref:akka.actor.typed.ActorRef%5BQ])(makeMessage:(I,akka.actor.typed.ActorRef%5BA])=%3EQ)(implicittimeout:akka.util.Timeout):akka.stream.scaladsl.Flow%5BI,A,akka.NotUsed]" java="#ask(akka.actor.typed.ActorRef,java.time.Duration,java.util.function.BiFunction)" }
## Description
-Emit the contents of a file, as `ByteString`s, materializes into a @scala[`Future`] @java[`CompletionStage`] which will be completed with
-a `IOResult` upon reaching the end of the file or if there is a failure.
+Use the @ref[Ask pattern](../../../typed/interaction-patterns.md#request-response-with-ask-from-outside-an-actor) to send a request-reply message to the target `ref` actor.
+If any of the asks times out it will fail the stream with an @apidoc[AskTimeoutException].
+
+The `ask` operator requires
+
+* the actor `ref`,
+* a `makeMessage` function to create the message sent to the actor from the incoming element and the actor ref accepting the actor's reply message,
+* and a timeout.
+
+See also:
+
+* @ref[Flow.ask](../Source-or-Flow/ask.md) for the classic actors variant
## Examples
+The `ActorFlow.ask` sends a message to the actor which expects `Asking` messages which contain the actor ref for replies of type `Reply`. The replies are emitted when received and the `map` extracts the message `String`.
Scala
-: @@snip [ask.scala](/akka-stream-typed/src/test/scala/akka/stream/typed/scaladsl/ActorFlowSpec.scala) { #imports #ask-actor #ask }
+: @@snip [ask.scala](/akka-stream-typed/src/test/scala/docs/scaladsl/ActorFlowSpec.scala) { #imports #ask-actor #ask }
Java
-: @@snip [ask.java](/akka-stream-typed/src/test/java/akka/stream/typed/javadsl/ActorFlowCompileTest.java) { #ask-actor #ask }
+: @@snip [ask.java](/akka-stream-typed/src/test/java/docs/javadsl/ActorFlowCompileTest.java) { #ask-actor #ask }
+
+## Reactive Streams semantics
+
+@@@div { .callout }
+
+**emits** when the futures (in submission order) created by the ask pattern internally are completed
+
+**backpressures** when the number of futures reaches the configured parallelism and the downstream backpressures
+
+**completes** when upstream completes and all futures have been completed and all elements have been emitted
+
+**fails** when the passed in actor terminates, or a timeout is exceeded in any of the asks performed
+
+**cancels** when downstream cancels
+
+@@@
diff --git a/akka-docs/src/main/paradox/stream/operators/ActorSink/actorRef.md b/akka-docs/src/main/paradox/stream/operators/ActorSink/actorRef.md
index 924f1f1ccf..f7b84e3cfb 100644
--- a/akka-docs/src/main/paradox/stream/operators/ActorSink/actorRef.md
+++ b/akka-docs/src/main/paradox/stream/operators/ActorSink/actorRef.md
@@ -10,17 +10,13 @@ This operator is included in:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream-typed_$scala.binary_version$"
+ artifact="akka-stream-typed_$scala.binary.version$"
version="$akka.version$"
}
-@@@div { .group-scala }
-
## Signature
-@@signature [ActorSink.scala](/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorSink.scala) { #actorRef }
-
-@@@
+@apidoc[ActorSink.actorRef](ActorSink$) { scala="#actorRef[T](ref:akka.actor.typed.ActorRef[T],onCompleteMessage:T,onFailureMessage:Throwable=>T):akka.stream.scaladsl.Sink[T,akka.NotUsed]" java="#actorRef(akka.actor.typed.ActorRef,java.lang.Object,akka.japi.function.Function)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/ActorSink/actorRefWithBackpressure.md b/akka-docs/src/main/paradox/stream/operators/ActorSink/actorRefWithBackpressure.md
index 8f7c828fc0..0b601a8f1f 100644
--- a/akka-docs/src/main/paradox/stream/operators/ActorSink/actorRefWithBackpressure.md
+++ b/akka-docs/src/main/paradox/stream/operators/ActorSink/actorRefWithBackpressure.md
@@ -10,17 +10,13 @@ This operator is included in:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream-typed_$scala.binary_version$"
+ artifact="akka-stream-typed_$scala.binary.version$"
version="$akka.version$"
}
-@@@div { .group-scala }
-
## Signature
-@@signature [ActorSink.scala](/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorSink.scala) { #actorRefWithBackpressure }
-
-@@@
+@apidoc[ActorSink.actorRefWithBackpressure](ActorSink$) { scala="#actorRefWithBackpressure[T,M,A](ref:akka.actor.typed.ActorRef[M],messageAdapter:(akka.actor.typed.ActorRef[A],T)=>M,onInitMessage:akka.actor.typed.ActorRef[A]=>M,ackMessage:A,onCompleteMessage:M,onFailureMessage:Throwable=>M):akka.stream.scaladsl.Sink[T,akka.NotUsed]" java="#actorRefWithBackpressure(akka.actor.typed.ActorRef,akka.japi.function.Function2,akka.japi.function.Function,java.lang.Object,java.lang.Object,akka.japi.function.Function)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/ActorSource/actorRef.md b/akka-docs/src/main/paradox/stream/operators/ActorSource/actorRef.md
index ed16e84aee..c08e569dd7 100644
--- a/akka-docs/src/main/paradox/stream/operators/ActorSource/actorRef.md
+++ b/akka-docs/src/main/paradox/stream/operators/ActorSource/actorRef.md
@@ -10,17 +10,13 @@ This operator is included in:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream-typed_$scala.binary_version$"
+ artifact="akka-stream-typed_$scala.binary.version$"
version="$akka.version$"
}
-@@@div { .group-scala }
-
## Signature
-@@signature [ActorSource.scala](/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorSource.scala) { #actorRef }
-
-@@@
+@apidoc[ActorSource.actorRef](ActorSource$) { scala="#actorRef[T](completionMatcher:PartialFunction[T,Unit],failureMatcher:PartialFunction[T,Throwable],bufferSize:Int,overflowStrategy:akka.stream.OverflowStrategy):akka.stream.scaladsl.Source[T,akka.actor.typed.ActorRef[T]]" java="#actorRef(java.util.function.Predicate,akka.japi.function.Function,int,akka.stream.OverflowStrategy)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/ActorSource/actorRefWithBackpressure.md b/akka-docs/src/main/paradox/stream/operators/ActorSource/actorRefWithBackpressure.md
index b2c52d4eb6..e6988389ad 100644
--- a/akka-docs/src/main/paradox/stream/operators/ActorSource/actorRefWithBackpressure.md
+++ b/akka-docs/src/main/paradox/stream/operators/ActorSource/actorRefWithBackpressure.md
@@ -10,16 +10,13 @@ This operator is included in:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream-typed_$scala.binary_version$"
+ artifact="akka-stream-typed_$scala.binary.version$"
version="$akka.version$"
}
-@@@div { .group-scala }
-
## Signature
-@@signature [ActorSource.scala](/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorSource.scala) { #actorRefWithBackpressure }
-@@@
+@apidoc[ActorSource.actorRefWithBackpressure](ActorSource$) { scala="#actorRefWithBackpressure[T,Ack](ackTo:akka.actor.typed.ActorRef[Ack],ackMessage:Ack,completionMatcher:PartialFunction[T,akka.stream.CompletionStrategy],failureMatcher:PartialFunction[T,Throwable]):akka.stream.scaladsl.Source[T,akka.actor.typed.ActorRef[T]]" java="#actorRefWithBackpressure(akka.actor.typed.ActorRef,java.lang.Object,akka.japi.function.Function,akka.japi.function.Function)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/FileIO/fromFile.md b/akka-docs/src/main/paradox/stream/operators/FileIO/fromFile.md
index 67d2ccc384..cf4f822b8b 100644
--- a/akka-docs/src/main/paradox/stream/operators/FileIO/fromFile.md
+++ b/akka-docs/src/main/paradox/stream/operators/FileIO/fromFile.md
@@ -10,13 +10,10 @@ The `fromFile` operator has been deprecated, use @ref:[fromPath](./fromPath.md)
@@@
-@@@div { .group-scala }
-
## Signature
-@@signature [FileIO.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/FileIO.scala) { #fromFile }
+@apidoc[FileIO.fromFile](FileIO$) { scala="#fromFile(f:java.io.File,chunkSize:Int):akka.stream.scaladsl.Source[akka.util.ByteString,scala.concurrent.Future[akka.stream.IOResult]]" java="#fromFile(java.io.File)" java="#fromFile(java.io.File,int)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/FileIO/fromPath.md b/akka-docs/src/main/paradox/stream/operators/FileIO/fromPath.md
index ee2eac2223..a0b178d0a3 100644
--- a/akka-docs/src/main/paradox/stream/operators/FileIO/fromPath.md
+++ b/akka-docs/src/main/paradox/stream/operators/FileIO/fromPath.md
@@ -4,13 +4,10 @@ Emits the contents of a file from the given path.
@ref[File IO Sinks and Sources](../index.md#file-io-sinks-and-sources)
-@@@div { .group-scala }
-
## Signature
-@@signature [FileIO.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/FileIO.scala) { #fromPath }
+@apidoc[FileIO.fromPath](FileIO$) { scala="#fromPath(f:java.nio.file.Path,chunkSize:Int,startPosition:Long):akka.stream.scaladsl.Source[akka.util.ByteString,scala.concurrent.Future[akka.stream.IOResult]]" java="#fromPath(java.nio.file.Path,int,long)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/FileIO/toFile.md b/akka-docs/src/main/paradox/stream/operators/FileIO/toFile.md
index c53703320f..6d88608364 100644
--- a/akka-docs/src/main/paradox/stream/operators/FileIO/toFile.md
+++ b/akka-docs/src/main/paradox/stream/operators/FileIO/toFile.md
@@ -10,13 +10,10 @@ The `toFile` operator has been deprecated, use @ref:[toPath](./toPath.md) instea
@@@
-@@@div { .group-scala }
-
## Signature
-@@signature [FileIO.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/FileIO.scala) { #toFile }
+@apidoc[FileIO.toFile](FileIO$) { scala="#toFile(f:java.io.File,options:Set[java.nio.file.OpenOption]):akka.stream.scaladsl.Sink[akka.util.ByteString,scala.concurrent.Future[akka.stream.IOResult]]" java="#toFile(java.io.File,java.util.Set)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/FileIO/toPath.md b/akka-docs/src/main/paradox/stream/operators/FileIO/toPath.md
index 07c4bb1588..7657c46bf0 100644
--- a/akka-docs/src/main/paradox/stream/operators/FileIO/toPath.md
+++ b/akka-docs/src/main/paradox/stream/operators/FileIO/toPath.md
@@ -4,13 +4,10 @@ Create a sink which will write incoming `ByteString` s to a given file path.
@ref[File IO Sinks and Sources](../index.md#file-io-sinks-and-sources)
-@@@div { .group-scala }
-
## Signature
-@@signature [FileIO.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/FileIO.scala) { #toPath }
+@apidoc[FileIO.toPath](FileIO$) { scala="#toPath(f:java.nio.file.Path,options:Set[java.nio.file.OpenOption],startPosition:Long):akka.stream.scaladsl.Sink[akka.util.ByteString,scala.concurrent.Future[akka.stream.IOResult]]" java="#toPath(java.nio.file.Path,java.util.Set,long)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Flow/asFlowWithContext.md b/akka-docs/src/main/paradox/stream/operators/Flow/asFlowWithContext.md
index e2932c5c15..0745f131c7 100644
--- a/akka-docs/src/main/paradox/stream/operators/Flow/asFlowWithContext.md
+++ b/akka-docs/src/main/paradox/stream/operators/Flow/asFlowWithContext.md
@@ -4,13 +4,10 @@ Turns a Flow into a FlowWithContext which can propagate a context per element al
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #asFlowWithContext }
+@apidoc[Flow.asFlowWithContext](Flow) { scala="#asFlowWithContext[U,CtxU,CtxOut](collapseContext:(U,CtxU)=>In)(extractContext:Out=>CtxOut):akka.stream.scaladsl.FlowWithContext[U,CtxU,Out,CtxOut,Mat]" java="#asFlowWithContext(akka.japi.function.Function2,akka.japi.function.Function)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Flow/completionStageFlow.md b/akka-docs/src/main/paradox/stream/operators/Flow/completionStageFlow.md
index 7cfda7e3ff..8a2c98c0a9 100644
--- a/akka-docs/src/main/paradox/stream/operators/Flow/completionStageFlow.md
+++ b/akka-docs/src/main/paradox/stream/operators/Flow/completionStageFlow.md
@@ -4,13 +4,10 @@ Streams the elements through the given future flow once it successfully complete
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #futureFlow }
+@apidoc[Flow.completionStageFlow](Flow$) { java="#completionStageFlow(java.util.concurrent.CompletionStage)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Flow/fromSinkAndSource.md b/akka-docs/src/main/paradox/stream/operators/Flow/fromSinkAndSource.md
index b7d08ed73b..aae3771eaf 100644
--- a/akka-docs/src/main/paradox/stream/operators/Flow/fromSinkAndSource.md
+++ b/akka-docs/src/main/paradox/stream/operators/Flow/fromSinkAndSource.md
@@ -34,7 +34,7 @@ Java
With this server running you could use `telnet 127.0.0.1 9999` to see a stream of timestamps being printed, one every second.
-The following sample is a little bit more advanced and uses the @apidoc[MergeHub] to dynamically merge incoming messages to a single stream which is then fed into a @apidoc[BroadcastHub] which emits elements over a dynamic set of downstreams allowing us to create a simplistic little TCP chat server in which a text entered from one client is emitted to all connected clients.
+The following sample is a little bit more advanced and uses the @apidoc[MergeHub$] to dynamically merge incoming messages to a single stream which is then fed into a @apidoc[BroadcastHub$] which emits elements over a dynamic set of downstreams allowing us to create a simplistic little TCP chat server in which a text entered from one client is emitted to all connected clients.
Scala
: @@snip [FromSinkAndSource.scala](/akka-docs/src/test/scala/docs/stream/operators/flow/FromSinkAndSource.scala) { #chat }
diff --git a/akka-docs/src/main/paradox/stream/operators/Flow/futureFlow.md b/akka-docs/src/main/paradox/stream/operators/Flow/futureFlow.md
index 205709624c..55a2ba4b5f 100644
--- a/akka-docs/src/main/paradox/stream/operators/Flow/futureFlow.md
+++ b/akka-docs/src/main/paradox/stream/operators/Flow/futureFlow.md
@@ -4,13 +4,10 @@ Streams the elements through the given future flow once it successfully complete
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #futureFlow }
+@apidoc[Flow.futureFlow](Flow$) { scala="#futureFlow[I,O,M](flow:scala.concurrent.Future[akka.stream.scaladsl.Flow[I,O,M]]):akka.stream.scaladsl.Flow[I,O,scala.concurrent.Future[M]]" }
-@@@
## Description
@@ -39,5 +36,9 @@ Scala
**completes** when upstream completes and all futures have been completed and all elements have been emitted
+**cancels** when downstream cancels (keep reading)
+ The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
+ This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
+ this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
@@@
diff --git a/akka-docs/src/main/paradox/stream/operators/Flow/lazyFlow.md b/akka-docs/src/main/paradox/stream/operators/Flow/lazyFlow.md
index 3a63c8b98b..950ac5d625 100644
--- a/akka-docs/src/main/paradox/stream/operators/Flow/lazyFlow.md
+++ b/akka-docs/src/main/paradox/stream/operators/Flow/lazyFlow.md
@@ -4,21 +4,52 @@ Defers creation and materialization of a `Flow` until there is a first element.
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #lazyFlow }
+@apidoc[Flow.lazyFlow](Flow$) { scala="#lazyFlow[I,O,M](create:()=>akka.stream.scaladsl.Flow[I,O,M]):akka.stream.scaladsl.Flow[I,O,scala.concurrent.Future[M]]" java="#lazyFlow(akka.japi.function.Creator)" }
-@@@
## Description
-When the first element comes from upstream the actual `Flow` is created and materialized.
-The internal `Flow` will not be created if there are no elements on completion or failure of up or downstream.
+Defers `Flow` creation and materialization until when the first element arrives at the `lazyFlow` from upstream. After
+that the stream behaves as if the nested flow replaced the `lazyFlow`.
+The nested `Flow` will not be created if the outer flow completes or fails before any elements arrive.
-The materialized value of the `Flow` will be the materialized value of the created internal flow if it is materialized
-and failed with a `akka.stream.NeverMaterializedException` if the stream fails or completes without the flow being materialized.
+Note that asynchronous boundaries and many other operators in the stream may do pre-fetching or trigger demand and thereby making an early element come throught the stream leading to creation of the inner flow earlier than you would expect.
+
+The materialized value of the `Flow` is a @scala[`Future`]@java[`CompletionStage`] that is completed with the
+materialized value of the nested flow once that is constructed.
+
+See also:
+
+ * @ref:[flatMapPrefix](../Source-or-Flow/flatMapPrefix.md)
+ * @ref:[Flow.lazyFutureFlow](lazyFutureFlow.md) and @ref:[Flow.lazyCompletionStageFlow](lazyCompletionStageFlow.md)
+ * @ref:[Source.lazySource](../Source/lazySource.md)
+ * @ref:[Sink.lazySink](../Sink/lazySink.md)
+
+## Examples
+
+In this sample we produce a short sequence of numbers, mostly to side effect and write to standard out to see in which
+order things happen. Note how producing the first value in the `Source` happens before the creation of the flow:
+
+Scala
+: @@snip [Lazy.scala](/akka-docs/src/test/scala/docs/stream/operators/flow/Lazy.scala) { #simple-example }
+
+Java
+: @@snip [Lazy.java](/akka-docs/src/test/java/jdocs/stream/operators/flow/Lazy.java) { #simple-example }
+
+Since the factory is called once per stream materialization it can be used to safely construct a mutable object to
+use with the actual deferred `Flow`. In this example we fold elements into an `ArrayList` created inside the lazy
+flow factory:
+
+Scala
+: @@snip [Lazy.scala](/akka-docs/src/test/scala/docs/stream/operators/flow/Lazy.scala) { #mutable-example }
+
+Java
+: @@snip [Lazy.java](/akka-docs/src/test/java/jdocs/stream/operators/flow/Lazy.java) { #mutable-example }
+
+If we instead had used `fold` directly with an `ArrayList` we would have shared the same list across
+all materialization and what is even worse, unsafely across threads.
## Reactive Streams semantics
@@ -32,5 +63,8 @@ and failed with a `akka.stream.NeverMaterializedException` if the stream fails o
**completes** when upstream completes and all futures have been completed and all elements have been emitted
+**cancels** when downstream cancels (keep reading)
+ The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
+ This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
+ this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
@@@
-
diff --git a/akka-docs/src/main/paradox/stream/operators/Flow/lazyFutureFlow.md b/akka-docs/src/main/paradox/stream/operators/Flow/lazyFutureFlow.md
index 1be4a6af28..fe8841464f 100644
--- a/akka-docs/src/main/paradox/stream/operators/Flow/lazyFutureFlow.md
+++ b/akka-docs/src/main/paradox/stream/operators/Flow/lazyFutureFlow.md
@@ -4,13 +4,10 @@ Defers creation and materialization of a `Flow` until there is a first element.
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #lazyFlow }
+@apidoc[Flow.lazyFutureFlow](Flow$) { scala="#lazyFutureFlow[I,O,M](create:()=>scala.concurrent.Future[akka.stream.scaladsl.Flow[I,O,M]]):akka.stream.scaladsl.Flow[I,O,scala.concurrent.Future[M]]" }
-@@@
## Description
@@ -38,5 +35,9 @@ See @ref:[lazyFlow](lazyFlow.md) for sample.
**completes** when upstream completes and all futures have been completed and all elements have been emitted
+**cancels** when downstream cancels (keep reading)
+ The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
+ This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
+ this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
@@@
diff --git a/akka-docs/src/main/paradox/stream/operators/Flow/lazyInitAsync.md b/akka-docs/src/main/paradox/stream/operators/Flow/lazyInitAsync.md
index 45b5ca320a..2dd60ca08a 100644
--- a/akka-docs/src/main/paradox/stream/operators/Flow/lazyInitAsync.md
+++ b/akka-docs/src/main/paradox/stream/operators/Flow/lazyInitAsync.md
@@ -4,13 +4,9 @@ Deprecated by @ref:[`Flow.lazyFutureFlow`](lazyFutureFlow.md) in combination wit
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #lazyInitAsync }
-
-@@@
+@apidoc[Flow.lazyInitAsync](Flow$) { scala="#lazyInitAsync[I,O,M](flowFactory:()=>scala.concurrent.Future[akka.stream.scaladsl.Flow[I,O,M]]):akka.stream.scaladsl.Flow[I,O,scala.concurrent.Future[Option[M]]]" java="#lazyInitAsync(akka.japi.function.Creator)" }
## Description
@@ -30,5 +26,9 @@ Defers creation until a first element arrives.
**completes** when upstream completes and all futures have been completed and all elements have been emitted
+**cancels** when downstream cancels (keep reading)
+ The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
+ This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
+ this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
@@@
diff --git a/akka-docs/src/main/paradox/stream/operators/RestartFlow/onFailuresWithBackoff.md b/akka-docs/src/main/paradox/stream/operators/RestartFlow/onFailuresWithBackoff.md
index fcadffac0c..83172e232f 100644
--- a/akka-docs/src/main/paradox/stream/operators/RestartFlow/onFailuresWithBackoff.md
+++ b/akka-docs/src/main/paradox/stream/operators/RestartFlow/onFailuresWithBackoff.md
@@ -4,13 +4,10 @@ Wrap the given @apidoc[Flow] with a @apidoc[Flow] that will restart it when it f
@ref[Error handling](../index.md#error-handling)
-@@@div { .group-scala }
-
## Signature
-@@signature [RestartFlow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/RestartFlow.scala) { #onFailuresWithBackoff }
+@apidoc[RestartFlow.onFailuresWithBackoff](RestartFlow$) { scala="#onFailuresWithBackoff[In,Out](minBackoff:scala.concurrent.duration.FiniteDuration,maxBackoff:scala.concurrent.duration.FiniteDuration,randomFactor:Double,maxRestarts:Int)(flowFactory:()=>akka.stream.scaladsl.Flow[In,Out,_]):akka.stream.scaladsl.Flow[In,Out,akka.NotUsed]" java="#onFailuresWithBackoff(java.time.Duration,java.time.Duration,double,int,akka.japi.function.Creator)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/RestartFlow/withBackoff.md b/akka-docs/src/main/paradox/stream/operators/RestartFlow/withBackoff.md
index b998f4391c..a8c8f0855e 100644
--- a/akka-docs/src/main/paradox/stream/operators/RestartFlow/withBackoff.md
+++ b/akka-docs/src/main/paradox/stream/operators/RestartFlow/withBackoff.md
@@ -4,13 +4,9 @@ Wrap the given @apidoc[Flow] with a @apidoc[Flow] that will restart it when it f
@ref[Error handling](../index.md#error-handling)
-@@@div { .group-scala }
-
## Signature
-@@signature [RestartFlow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/RestartFlow.scala) { #withBackoff }
-
-@@@
+@apidoc[RestartFlow.withBackoff](RestartFlow$) { scala="#withBackoff[In,Out](minBackoff:scala.concurrent.duration.FiniteDuration,maxBackoff:scala.concurrent.duration.FiniteDuration,randomFactor:Double)(flowFactory:()=>akka.stream.scaladsl.Flow[In,Out,_]):akka.stream.scaladsl.Flow[In,Out,akka.NotUsed]" java="#withBackoff(java.time.Duration,java.time.Duration,double,int,akka.japi.function.Creator)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/RestartSink/withBackoff.md b/akka-docs/src/main/paradox/stream/operators/RestartSink/withBackoff.md
index 64bb8272e2..7e34cb445e 100644
--- a/akka-docs/src/main/paradox/stream/operators/RestartSink/withBackoff.md
+++ b/akka-docs/src/main/paradox/stream/operators/RestartSink/withBackoff.md
@@ -4,13 +4,10 @@ Wrap the given @apidoc[Sink] with a @apidoc[Sink] that will restart it when it f
@ref[Error handling](../index.md#error-handling)
-@@@div { .group-scala }
-
## Signature
-@@signature [RestartSink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/RestartSink.scala) { #withBackoff }
+@apidoc[RestartSink.withBackoff](RestartSink$) { scala="#withBackoff[T](minBackoff:scala.concurrent.duration.FiniteDuration,maxBackoff:scala.concurrent.duration.FiniteDuration,randomFactor:Double,maxRestarts:Int)(sinkFactory:()=>akka.stream.scaladsl.Sink[T,_]):akka.stream.scaladsl.Sink[T,akka.NotUsed]" java="#withBackoff(java.time.Duration,java.time.Duration,double,int,akka.japi.function.Creator)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/RestartSource/onFailuresWithBackoff.md b/akka-docs/src/main/paradox/stream/operators/RestartSource/onFailuresWithBackoff.md
index 72e33853d3..48423c6e19 100644
--- a/akka-docs/src/main/paradox/stream/operators/RestartSource/onFailuresWithBackoff.md
+++ b/akka-docs/src/main/paradox/stream/operators/RestartSource/onFailuresWithBackoff.md
@@ -4,13 +4,9 @@ Wrap the given @apidoc[Source] with a @apidoc[Source] that will restart it when
@ref[Error handling](../index.md#error-handling)
-@@@div { .group-scala }
-
## Signature
-@@signature [RestartSource.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/RestartSource.scala) { #onFailuresWithBackoff }
-
-@@@
+@apidoc[RestartSource.onFailuresWithBackoff](RestartSource$) { scala="#onFailuresWithBackoff[T](minBackoff:scala.concurrent.duration.FiniteDuration,maxBackoff:scala.concurrent.duration.FiniteDuration,randomFactor:Double)(sourceFactory:()=>akka.stream.scaladsl.Source[T,_]):akka.stream.scaladsl.Source[T,akka.NotUsed]" java="#onFailuresWithBackoff(java.time.Duration,java.time.Duration,double,int,akka.japi.function.Creator)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/RestartSource/withBackoff.md b/akka-docs/src/main/paradox/stream/operators/RestartSource/withBackoff.md
index 55a034dd0c..45a0afddfb 100644
--- a/akka-docs/src/main/paradox/stream/operators/RestartSource/withBackoff.md
+++ b/akka-docs/src/main/paradox/stream/operators/RestartSource/withBackoff.md
@@ -4,13 +4,9 @@ Wrap the given @apidoc[Source] with a @apidoc[Source] that will restart it when
@ref[Error handling](../index.md#error-handling)
-@@@div { .group-scala }
-
## Signature
-@@signature [RestartSource.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/RestartSource.scala) { #withBackoff }
-
-@@@
+@apidoc[RestartSource.withBackoff](RestartSource$) { scala="#withBackoff[T](minBackoff:scala.concurrent.duration.FiniteDuration,maxBackoff:scala.concurrent.duration.FiniteDuration,randomFactor:Double,maxRestarts:Int)(sourceFactory:()=>akka.stream.scaladsl.Source[T,_]):akka.stream.scaladsl.Source[T,akka.NotUsed]" java="#withBackoff(java.time.Duration,java.time.Duration,double,int,akka.japi.function.Creator)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/RetryFlow/withBackoff.md b/akka-docs/src/main/paradox/stream/operators/RetryFlow/withBackoff.md
index 5860939341..285b328228 100644
--- a/akka-docs/src/main/paradox/stream/operators/RetryFlow/withBackoff.md
+++ b/akka-docs/src/main/paradox/stream/operators/RetryFlow/withBackoff.md
@@ -6,15 +6,7 @@ Wrap the given @apidoc[Flow] and retry individual elements in that stream with a
## Signature
-Scala
-: @@signature [RetryFlow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/RetryFlow.scala) { #withBackoff }
-
-Java
-: @@snip [RetryFlowTest.java](/akka-stream-tests/src/test/java/akka/stream/javadsl/RetryFlowTest.java) { #withBackoff-signature }
-
-## API documentation
-
-@apidoc[RetryFlow$]
+@apidoc[RetryFlow.withBackoff](RetryFlow$) { scala="#withBackoff[In,Out,Mat](minBackoff:scala.concurrent.duration.FiniteDuration,maxBackoff:scala.concurrent.duration.FiniteDuration,randomFactor:Double,maxRetries:Int,flow:akka.stream.scaladsl.Flow[In,Out,Mat])(decideRetry:(In,Out)=>Option[In]):akka.stream.scaladsl.Flow[In,Out,Mat]" java="#withBackoff(java.time.Duration,java.time.Duration,double,int,akka.stream.javadsl.Flow,akka.japi.function.Function2)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/RetryFlow/withBackoffAndContext.md b/akka-docs/src/main/paradox/stream/operators/RetryFlow/withBackoffAndContext.md
index ffb1e6e54c..c6133adfae 100644
--- a/akka-docs/src/main/paradox/stream/operators/RetryFlow/withBackoffAndContext.md
+++ b/akka-docs/src/main/paradox/stream/operators/RetryFlow/withBackoffAndContext.md
@@ -6,11 +6,7 @@ Wrap the given @apidoc[FlowWithContext] and retry individual elements in that st
## Signature
-Scala
-: @@signature [RetryFlow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/RetryFlow.scala) { #withBackoffAndContext }
-
-Java
-: @@snip [RetryFlowTest.java](/akka-stream-tests/src/test/java/akka/stream/javadsl/RetryFlowTest.java) { #signature }
+@apidoc[RetryFlow.withBackoffAndContext](RetryFlow$) { scala="#withBackoffAndContext[In,CtxIn,Out,CtxOut,Mat](minBackoff:scala.concurrent.duration.FiniteDuration,maxBackoff:scala.concurrent.duration.FiniteDuration,randomFactor:Double,maxRetries:Int,flow:akka.stream.scaladsl.FlowWithContext[In,CtxIn,Out,CtxOut,Mat])(decideRetry:((In,CtxIn),(Out,CtxOut))=>Option[(In,CtxIn)]):akka.stream.scaladsl.FlowWithContext[In,CtxIn,Out,CtxOut,Mat]" java="#withBackoffAndContext(java.time.Duration,java.time.Duration,double,int,akka.stream.javadsl.FlowWithContext,akka.japi.function.Function2)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/actorRef.md b/akka-docs/src/main/paradox/stream/operators/Sink/actorRef.md
index 45ea9a2434..9aacf693d8 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/actorRef.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/actorRef.md
@@ -1,14 +1,12 @@
-# actorRef
+# Sink.actorRef
Send the elements from the stream to an `ActorRef`.
@ref[Sink operators](../index.md#sink-operators)
-@@@ div { .group-scala }
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #actorRef }
-@@@
+@apidoc[Sink.actorRef](Sink$) { scala="#actorRef[T](ref:akka.actor.ActorRef,onCompleteMessage:Any,onFailureMessage:Throwable=>Any):akka.stream.scaladsl.Sink[T,akka.NotUsed]" java="#actorRef(akka.actor.ActorRef,java.lang.Object)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/actorRefWithBackpressure.md b/akka-docs/src/main/paradox/stream/operators/Sink/actorRefWithBackpressure.md
index 6f9d26da68..b243a03d5b 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/actorRefWithBackpressure.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/actorRefWithBackpressure.md
@@ -4,6 +4,10 @@ Send the elements from the stream to an `ActorRef` which must then acknowledge r
@ref[Sink operators](../index.md#sink-operators)
+## Signature
+
+@apidoc[Sink.actorRefWithBackpressure](Sink$) { scala="#actorRefWithBackpressure[T](ref:akka.actor.ActorRef,onInitMessage:Any,ackMessage:Any,onCompleteMessage:Any,onFailureMessage:Throwable=>Any):akka.stream.scaladsl.Sink[T,akka.NotUsed]" java="#actorRefWithBackpressure(akka.actor.ActorRef,java.lang.Object,java.lang.Object,java.lang.Object,akka.japi.function.Function)" }
+
## Description
Send the elements from the stream to an `ActorRef` which must then acknowledge reception after completing a message,
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/asPublisher.md b/akka-docs/src/main/paradox/stream/operators/Sink/asPublisher.md
index d41338ff24..8217c2bdf5 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/asPublisher.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/asPublisher.md
@@ -1,16 +1,13 @@
-# asPublisher
+# Sink.asPublisher
Integration with Reactive Streams, materializes into a `org.reactivestreams.Publisher`.
@ref[Sink operators](../index.md#sink-operators)
-@@@ div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #asPublisher }
+@apidoc[Sink.asPublisher](Sink$) { scala="#asPublisher[T](fanout:Boolean):akka.stream.scaladsl.Sink[T,org.reactivestreams.Publisher[T]]" java="#asPublisher(akka.stream.javadsl.AsPublisher)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/cancelled.md b/akka-docs/src/main/paradox/stream/operators/Sink/cancelled.md
index 3f2678c6ca..900fa47e8c 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/cancelled.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/cancelled.md
@@ -1,16 +1,13 @@
-# cancelled
+# Sink.cancelled
Immediately cancel the stream
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #cancelled }
+@apidoc[Sink.cancelled](Sink$) { scala="#cancelled[T]:akka.stream.scaladsl.Sink[T,akka.NotUsed]" java="#cancelled()" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/collection.md b/akka-docs/src/main/paradox/stream/operators/Sink/collection.md
index 176058df1e..21cc0b0cd7 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/collection.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/collection.md
@@ -4,11 +4,10 @@
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #collection }
+@apidoc[Sink.collection](Sink$) { scala="#collection[T,That](implicitcbf:akka.util.ccompat.Factory[T,Thatwithscala.collection.immutable.Iterable[_]]):akka.stream.scaladsl.Sink[T,scala.concurrent.Future[That]]" }
+
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/combine.md b/akka-docs/src/main/paradox/stream/operators/Sink/combine.md
index ef101caa55..cde70625be 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/combine.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/combine.md
@@ -1,16 +1,12 @@
-# combine
+# Sink.combine
Combine several sinks into one using a user specified strategy
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #combine }
-
-@@@
+@apidoc[Sink.combine](Sink$) { scala="#combine[T,U](first:akka.stream.scaladsl.Sink[U,_],second:akka.stream.scaladsl.Sink[U,_],rest:akka.stream.scaladsl.Sink[U,_]*)(strategy:Int=>akka.stream.Graph[akka.stream.UniformFanOutShape[T,U],akka.NotUsed]):akka.stream.scaladsl.Sink[T,akka.NotUsed]" java="#combine(akka.stream.javadsl.Sink,akka.stream.javadsl.Sink,java.util.List,akka.japi.function.Function)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/fold.md b/akka-docs/src/main/paradox/stream/operators/Sink/fold.md
index d936581a32..818a8fecba 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/fold.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/fold.md
@@ -1,16 +1,12 @@
-# fold
+# Sink.fold
Fold over emitted element with a function, where each invocation will get the new element and the result from the previous fold invocation.
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #fold }
-
-@@@
+@apidoc[Sink.fold](Sink$) { scala="#fold[U,T](zero:U)(f:(U,T)=>U):akka.stream.scaladsl.Sink[T,scala.concurrent.Future[U]]" java="#fold(java.lang.Object,akka.japi.function.Function2)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/foreach.md b/akka-docs/src/main/paradox/stream/operators/Sink/foreach.md
index 7eabfba126..e0970e3e6b 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/foreach.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/foreach.md
@@ -1,4 +1,4 @@
-# foreach
+# Sink.foreach
Invoke a given procedure for each element received.
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/foreachAsync.md b/akka-docs/src/main/paradox/stream/operators/Sink/foreachAsync.md
index ec38ddf497..aadb562331 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/foreachAsync.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/foreachAsync.md
@@ -1,16 +1,13 @@
-# foreachAsync
+# Sink.foreachAsync
Invoke a given procedure asynchronously for each element received.
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #foreachAsync }
+@apidoc[Sink.foreachAsync](Sink$) { scala="#foreachAsync[T](parallelism:Int)(f:T=>scala.concurrent.Future[Unit]):akka.stream.scaladsl.Sink[T,scala.concurrent.Future[akka.Done]]" java="#foreachAsync(int,akka.japi.function.Function)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/foreachParallel.md b/akka-docs/src/main/paradox/stream/operators/Sink/foreachParallel.md
index 6ff2a7be0e..04ecb5b083 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/foreachParallel.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/foreachParallel.md
@@ -1,4 +1,4 @@
-# foreachParallel
+# Sink.foreachParallel
Like `foreach` but allows up to `parallellism` procedure calls to happen in parallel.
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/fromMaterializer.md b/akka-docs/src/main/paradox/stream/operators/Sink/fromMaterializer.md
index 59d12c969a..7485e657b8 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/fromMaterializer.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/fromMaterializer.md
@@ -4,14 +4,11 @@ Defer the creation of a `Sink` until materialization and access `Materializer` a
@ref[Sink operators](../index.md#sink-operators)
-@@@ div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #fromMaterializer }
-@@@
+@apidoc[Sink.fromMaterializer](Sink$) { scala="#fromMaterializer[T,M](factory:(akka.stream.Materializer,akka.stream.Attributes)=>akka.stream.scaladsl.Sink[T,M]):akka.stream.scaladsl.Sink[T,scala.concurrent.Future[M]]" java="#fromMaterializer(java.util.function.BiFunction)" }
## Description
Typically used when access to materializer is needed to run a different stream during the construction of a sink.
-Can also be used to access the underlying `ActorSystem` from `Materializer`.
\ No newline at end of file
+Can also be used to access the underlying `ActorSystem` from `Materializer`.
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/fromSubscriber.md b/akka-docs/src/main/paradox/stream/operators/Sink/fromSubscriber.md
index e0ab946157..b49c10ac8e 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/fromSubscriber.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/fromSubscriber.md
@@ -1,16 +1,13 @@
-# fromSubscriber
+# Sink.fromSubscriber
Integration with Reactive Streams, wraps a `org.reactivestreams.Subscriber` as a sink.
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #fromSubscriber }
+@apidoc[Sink.fromSubscriber](Sink$) { scala="#fromSubscriber[T](subscriber:org.reactivestreams.Subscriber[T]):akka.stream.scaladsl.Sink[T,akka.NotUsed]" java="#fromSubscriber(org.reactivestreams.Subscriber)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/futureSink.md b/akka-docs/src/main/paradox/stream/operators/Sink/futureSink.md
index 24784fe624..512e08cdfc 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/futureSink.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/futureSink.md
@@ -4,13 +4,10 @@ Streams the elements to the given future sink once it successfully completes.
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #futureSink }
+@apidoc[Sink.futureSink](Sink$) { scala="#futureSink[T,M](future:scala.concurrent.Future[akka.stream.scaladsl.Sink[T,M]]):akka.stream.scaladsl.Sink[T,scala.concurrent.Future[M]]" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/head.md b/akka-docs/src/main/paradox/stream/operators/Sink/head.md
index 026ef08460..eb2dc3c503 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/head.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/head.md
@@ -1,16 +1,13 @@
-# head
+# Sink.head
Materializes into a @scala[`Future`] @java[`CompletionStage`] which completes with the first value arriving, after this the stream is canceled.
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #head }
+@apidoc[Sink.head](Sink$) { scala="#head[T]:akka.stream.scaladsl.Sink[T,scala.concurrent.Future[T]]" java="#head()" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/headOption.md b/akka-docs/src/main/paradox/stream/operators/Sink/headOption.md
index 90cd37def3..e952021779 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/headOption.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/headOption.md
@@ -4,13 +4,10 @@ Materializes into a @scala[`Future[Option[T]]`] @java[`CompletionStage>`]
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #lastOption }
+@apidoc[Sink.lastOption](Sink$) { scala="#lastOption[T]:akka.stream.scaladsl.Sink[T,scala.concurrent.Future[Option[T]]]" java="#lastOption()" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/lazyFutureSink.md b/akka-docs/src/main/paradox/stream/operators/Sink/lazyFutureSink.md
index a56a5ae683..54ca625de8 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/lazyFutureSink.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/lazyFutureSink.md
@@ -4,13 +4,10 @@ Defers creation and materialization of a `Sink` until there is a first element.
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #lazySink }
+@apidoc[Sink.lazyFutureSink](Sink$) { scala="#lazyFutureSink[T,M](create:()=>scala.concurrent.Future[akka.stream.scaladsl.Sink[T,M]]):akka.stream.scaladsl.Sink[T,scala.concurrent.Future[M]]" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/lazyInitAsync.md b/akka-docs/src/main/paradox/stream/operators/Sink/lazyInitAsync.md
index c85d14560e..9b171f3563 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/lazyInitAsync.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/lazyInitAsync.md
@@ -4,13 +4,11 @@ Deprecated by @ref:[`Sink.lazyFutureSink`](lazyFutureSink.md).
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #lazyInitAsync }
+@apidoc[Flow.lazyInitAsync](Flow$) { scala="#lazyInitAsync[I,O,M](flowFactory:()=>scala.concurrent.Future[akka.stream.scaladsl.Flow[I,O,M]]):akka.stream.scaladsl.Flow[I,O,scala.concurrent.Future[Option[M]]]" java="#lazyInitAsync(akka.japi.function.Creator)" }
+@apidoc[Sink.lazyInitAsync](Sink$) { scala="#lazyInitAsync[T,M](sinkFactory:()=>scala.concurrent.Future[akka.stream.scaladsl.Sink[T,M]]):akka.stream.scaladsl.Sink[T,scala.concurrent.Future[Option[M]]]" java="#lazyInitAsync(akka.japi.function.Creator)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/lazySink.md b/akka-docs/src/main/paradox/stream/operators/Sink/lazySink.md
index 3c54d5968a..7639b3c675 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/lazySink.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/lazySink.md
@@ -4,25 +4,40 @@ Defers creation and materialization of a `Sink` until there is a first element.
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #lazySink }
+@apidoc[Sink.lazySink](Sink$) { scala="#lazySink[T,M](create:()=>akka.stream.scaladsl.Sink[T,M]):akka.stream.scaladsl.Sink[T,scala.concurrent.Future[M]]" java="#lazySink(akka.japi.function.Creator)" }
-@@@
## Description
-When the first element comes from upstream the actual `Sink` is created and materialized.
-The internal `Sink` will not be created if the stream completes of fails before any element got through.
+Defers `Sink` creation and materialization until when the first element arrives from upstream to the `lazySink`. After
+that the stream behaves as if the nested sink replaced the `lazySink`.
+The nested `Sink` will not be created if upstream completes or fails without any elements arriving at the sink.
-The materialized value of the `Sink` will be the materialized value of the created internal flow if it is materialized
-and failed with a `akka.stream.NeverMaterializedException` if the stream fails or completes without the flow being materialized.
+The materialized value of the `Sink` is a @scala[`Future`]@java[`CompletionStage`] that is completed with the
+materialized value of the nested sink once that is constructed.
Can be combined with @ref[prefixAndTail](../Source-or-Flow/prefixAndTail.md) to base the sink on the first element.
-See also @ref:[lazyFutureSink](lazyFutureSink.md) and @ref:[lazyCompletionStageSink](lazyCompletionStageSink.md).
+See also:
+
+ * @ref:[Sink.lazyFutureSink](lazyFutureSink.md) and @ref:[lazyCompletionStageSink](lazyCompletionStageSink.md).
+ * @ref:[Source.lazySource](../Source/lazySource.md)
+ * @ref:[Flow.lazyFlow](../Flow/lazyFlow.md)
+
+## Examples
+
+In this example we side effect from `Flow.map`, the sink factory and `Sink.foreach` so that the order becomes visible,
+the nested sink is only created once the element has passed `map`:
+
+Scala
+: @@snip [Lazy.scala](/akka-docs/src/test/scala/docs/stream/operators/sink/Lazy.scala) { #simple-example }
+
+Java
+: @@snip [Lazy.java](/akka-docs/src/test/java/jdocs/stream/operators/sink/Lazy.java) { #simple-example }
+
+
## Reactive Streams semantics
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/onComplete.md b/akka-docs/src/main/paradox/stream/operators/Sink/onComplete.md
index aba84c8594..61869a6708 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/onComplete.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/onComplete.md
@@ -4,13 +4,10 @@ Invoke a callback when the stream has completed or failed.
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #onComplete }
+@apidoc[Sink.onComplete](Sink$) { scala="#onComplete[T](callback:scala.util.Try[akka.Done]=>Unit):akka.stream.scaladsl.Sink[T,akka.NotUsed]" java="#onComplete(akka.japi.function.Procedure)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/preMaterialize.md b/akka-docs/src/main/paradox/stream/operators/Sink/preMaterialize.md
index a82781ca3d..2b749796b4 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/preMaterialize.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/preMaterialize.md
@@ -4,13 +4,10 @@ Materializes this Sink, immediately returning (1) its materialized value, and (2
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #preMaterialize }
+@apidoc[Sink.preMaterialize](Sink) { scala="#preMaterialize()(implicitmaterializer:akka.stream.Materializer):(Mat,akka.stream.scaladsl.Sink[In,akka.NotUsed])" java="#preMaterialize(akka.actor.ClassicActorSystemProvider)" java="#preMaterialize(akka.stream.Materializer)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/queue.md b/akka-docs/src/main/paradox/stream/operators/Sink/queue.md
index 710a38529b..8e48ebe3b0 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/queue.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/queue.md
@@ -4,13 +4,10 @@ Materialize a `SinkQueue` that can be pulled to trigger demand through the sink.
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #queue }
+@apidoc[Sink.queue](Sink$) { scala="#queue[T](maxConcurrentPulls:Int):akka.stream.scaladsl.Sink[T,akka.stream.scaladsl.SinkQueueWithCancel[T]]" java="#queue(int)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/reduce.md b/akka-docs/src/main/paradox/stream/operators/Sink/reduce.md
index 0937aeb308..f5ce8a3536 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/reduce.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/reduce.md
@@ -4,13 +4,10 @@ Apply a reduction function on the incoming elements and pass the result to the n
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #reduce }
+@apidoc[Sink.reduce](Sink$) { scala="#reduce[T](f:(T,T)=>T):akka.stream.scaladsl.Sink[T,scala.concurrent.Future[T]]" java="#reduce(akka.japi.function.Function2)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/seq.md b/akka-docs/src/main/paradox/stream/operators/Sink/seq.md
index 662e764504..7f89f0f160 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/seq.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/seq.md
@@ -4,13 +4,10 @@ Collect values emitted from the stream into a collection.
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #seq }
+@apidoc[Sink.seq](Sink$) { scala="#seq[T]:akka.stream.scaladsl.Sink[T,scala.concurrent.Future[scala.collection.immutable.Seq[T]]]" java="#seq()" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/setup.md b/akka-docs/src/main/paradox/stream/operators/Sink/setup.md
index 5415b7f2d0..d6bb76a37c 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/setup.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/setup.md
@@ -10,14 +10,11 @@ The `setup` operator has been deprecated, use @ref:[fromMaterializer](./fromMate
@@@
-@@@ div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #setup }
-@@@
+@apidoc[Sink.setup](Sink$) { scala="#setup[T,M](factory:(akka.stream.ActorMaterializer,akka.stream.Attributes)=>akka.stream.scaladsl.Sink[T,M]):akka.stream.scaladsl.Sink[T,scala.concurrent.Future[M]]" java="#setup(java.util.function.BiFunction)" }
## Description
Typically used when access to materializer is needed to run a different stream during the construction of a sink.
-Can also be used to access the underlying `ActorSystem` from `ActorMaterializer`.
\ No newline at end of file
+Can also be used to access the underlying `ActorSystem` from `ActorMaterializer`.
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/takeLast.md b/akka-docs/src/main/paradox/stream/operators/Sink/takeLast.md
index 3db64e0504..cd8ed89586 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/takeLast.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/takeLast.md
@@ -4,13 +4,10 @@ Collect the last `n` values emitted from the stream into a collection.
@ref[Sink operators](../index.md#sink-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #takeLast }
+@apidoc[Sink.takeLast](Sink$) { scala="#takeLast[T](n:Int):akka.stream.scaladsl.Sink[T,scala.concurrent.Future[scala.collection.immutable.Seq[T]]]" java="#takeLast(int)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/alsoTo.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/alsoTo.md
index 773323ee3d..281a8e9329 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/alsoTo.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/alsoTo.md
@@ -4,11 +4,11 @@ Attaches the given `Sink` to this `Flow`, meaning that elements that pass throug
@ref[Fan-out operators](../index.md#fan-out-operators)
-@@@ div { .group-scala }
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #alsoTo }
-@@@
+@apidoc[Source.alsoTo](Source) { scala="#alsoTo(that:akka.stream.Graph[akka.stream.SinkShape[Out],_]):FlowOps.this.Repr[Out]" java="#alsoTo(akka.stream.Graph)" }
+@apidoc[Flow.alsoTo](Flow) { scala="#alsoTo(that:akka.stream.Graph[akka.stream.SinkShape[Out],_]):FlowOps.this.Repr[Out]" java="#alsoTo(akka.stream.Graph)" }
+
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/ask.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/ask.md
index 09f3d343ad..abdf060481 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/ask.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/ask.md
@@ -1,27 +1,29 @@
-# Flow.ask
+# ask
-Use the `ask` pattern to send a request-reply message to the target `ref` actor.
+Use the "Ask Pattern" to send a request-reply message to the target `ref` actor (of the classic actors API).
-@ref[Asynchronous operators](../index.md#asynchronous-operators)
+@ref[Actor interop operators](../index.md#actor-interop-operators)
-@@@ div { .group-scala }
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #ask }
-@@@
+@apidoc[Source.ask](Source) {scala="#ask[S](ref:akka.actor.ActorRef)(implicittimeout:akka.util.Timeout,implicittag:scala.reflect.ClassTag[S]):FlowOps.this.Repr[S]" java="#ask(akka.actor.ActorRef,java.lang.Class,akka.util.Timeout)" }
+@apidoc[Flow.ask](Flow$) { scala="#ask%5BS](ref:akka.actor.ActorRef)(implicittimeout:akka.util.Timeout,implicittag:scala.reflect.ClassTag%5BS]):FlowOps.this.Repr%5BS]" java="#ask(akka.actor.ActorRef,java.lang.Class,akka.util.Timeout)" }
## Description
-Use the `ask` pattern to send a request-reply message to the target `ref` actor.
+Use the @ref[Ask Pattern](../../../actors.md#ask-send-and-receive-future) to send a request-reply message to the target `ref` actor.
If any of the asks times out it will fail the stream with a @apidoc[AskTimeoutException].
The @java[`mapTo` class]@scala[`S` generic] parameter is used to cast the responses from the actor to the expected outgoing flow type.
-Similar to the plain ask pattern, the target actor is allowed to reply with `akka.util.Status`.
-An `akka.util.Status#Failure` will cause the operator to fail with the cause carried in the `Failure` message.
+Similar to the plain ask pattern, the target actor is allowed to reply with @apidoc[akka.actor.Status$].
+An @apidoc[akka.actor.Status.Failure] will cause the operator to fail with the cause carried in the `Failure` message.
-Adheres to the @scala[@scaladoc[`ActorAttributes.SupervisionStrategy`](akka.stream.ActorAttributes$$SupervisionStrategy)]
-@java[`ActorAttributes.SupervisionStrategy`] attribute.
+Adheres to the @apidoc[ActorAttributes.SupervisionStrategy] attribute.
+
+See also:
+
+* @ref[ActorFlow.ask](../ActorFlow/ask.md) for the `akka.actor.typed.ActorRef[_]` variant
## Reactive Streams semantics
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/backpressureTimeout.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/backpressureTimeout.md
index a46704f468..69555f7c3d 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/backpressureTimeout.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/backpressureTimeout.md
@@ -4,11 +4,11 @@ If the time between the emission of an element and the following downstream dema
@ref[Time aware operators](../index.md#time-aware-operators)
-@@@ div { .group-scala }
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #backpressureTimeout }
-@@@
+@apidoc[Source.backpressureTimeout](Source) { scala="#backpressureTimeout(timeout:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[Out]" java="#backpressureTimeout(java.time.Duration)" }
+@apidoc[Flow.backpressureTimeout](Flow) { scala="#backpressureTimeout(timeout:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[Out]" java="#backpressureTimeout(java.time.Duration)" }
+
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/batch.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/batch.md
index d265130cb2..264d79dfd3 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/batch.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/batch.md
@@ -4,11 +4,11 @@ Allow for a slower downstream by passing incoming elements and a summary into an
@ref[Backpressure aware operators](../index.md#backpressure-aware-operators)
-@@@ div { .group-scala }
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #batch }
-@@@
+@apidoc[Source.batch](Source) { scala="#batch[S](max:Long,seed:Out=>S)(aggregate:(S,Out)=>S):FlowOps.this.Repr[S]" java="#batch(long,akka.japi.function.Function,akka.japi.function.Function2)" }
+@apidoc[Flow.batch](Flow) { scala="#batch[S](max:Long,seed:Out=>S)(aggregate:(S,Out)=>S):FlowOps.this.Repr[S]" java="#batch(long,akka.japi.function.Function,akka.japi.function.Function2)" }
+
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/batchWeighted.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/batchWeighted.md
index 7d6fe3904f..54986246b9 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/batchWeighted.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/batchWeighted.md
@@ -4,11 +4,11 @@ Allow for a slower downstream by passing incoming elements and a summary into an
@ref[Backpressure aware operators](../index.md#backpressure-aware-operators)
-@@@ div { .group-scala }
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #batchWeighted }
-@@@
+@apidoc[Source.batchWeighted](Source) { scala="#batchWeighted[S](max:Long,costFn:Out=>Long,seed:Out=>S)(aggregate:(S,Out)=>S):FlowOps.this.Repr[S]" java="#batchWeighted(long,akka.japi.function.Function,akka.japi.function.Function,akka.japi.function.Function2)" }
+@apidoc[Flow.batchWeighted](Flow) { scala="#batchWeighted[S](max:Long,costFn:Out=>Long,seed:Out=>S)(aggregate:(S,Out)=>S):FlowOps.this.Repr[S]" java="#batchWeighted(long,akka.japi.function.Function,akka.japi.function.Function,akka.japi.function.Function2)" }
+
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/buffer.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/buffer.md
index 694f17630c..f01fc5813e 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/buffer.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/buffer.md
@@ -4,12 +4,10 @@ Allow for a temporarily faster upstream events by buffering `size` elements.
@ref[Backpressure aware operators](../index.md#backpressure-aware-operators)
-@@@ div { .group-scala }
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #buffer }
-@@@
-
+@apidoc[Source.buffer](Source) { scala"#buffer(size:Int,overflowStrategy:akka.stream.OverflowStrategy):FlowOps.this.Repr[Out]" java="#buffer(int,akka.stream.OverflowStrategy)" }
+@apidoc[Flow.buffer](Flow) { scala="#buffer(size:Int,overflowStrategy:akka.stream.OverflowStrategy):FlowOps.this.Repr[Out]" java="#buffer(int,akka.stream.OverflowStrategy)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/collect.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/collect.md
index cc7e3d0e8d..631c71b3a2 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/collect.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/collect.md
@@ -4,13 +4,10 @@ Apply a partial function to each incoming element, if the partial function is de
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #collect }
-
-@@@
+@apidoc[Source.collect](Source) { scala="#collect[T](pf:PartialFunction[Out,T]):FlowOps.this.Repr[T]" java="#collect(scala.PartialFunction)" }
+@apidoc[Flow.collect](Flow) { scala="#collect[T](pf:PartialFunction[Out,T]):FlowOps.this.Repr[T]" java="#collect(scala.PartialFunction)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/collectType.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/collectType.md
index 5601481615..5773d59398 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/collectType.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/collectType.md
@@ -4,13 +4,11 @@ Transform this stream by testing the type of each of the elements on which the e
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #collectType }
+@apidoc[Source.collectType](Source) { scala="#collectType[T](implicittag:scala.reflect.ClassTag[T]):FlowOps.this.Repr[T]" java="#collectType(java.lang.Class)" }
+@apidoc[Flow.collectType](Flow) { scala="#collectType[T](implicittag:scala.reflect.ClassTag[T]):FlowOps.this.Repr[T]" java="#collectType(java.lang.Class)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/completionTimeout.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/completionTimeout.md
index dd0f4da832..ebdcffe0a5 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/completionTimeout.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/completionTimeout.md
@@ -4,13 +4,11 @@ If the completion of the stream does not happen until the provided timeout, the
@ref[Time aware operators](../index.md#time-aware-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #completionTimeout }
+@apidoc[Source.completionTimeout](Source) { scala="#completionTimeout(timeout:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[Out]" java="#completionTimeout(java.time.Duration)" }
+@apidoc[Flow.completionTimeout](Flow) { scala="#completionTimeout(timeout:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[Out]" java="#completionTimeout(java.time.Duration)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/concat.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/concat.md
index 7ab2d97926..dd0ef601f8 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/concat.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/concat.md
@@ -4,13 +4,11 @@ After completion of the original upstream the elements of the given source will
@ref[Fan-in operators](../index.md#fan-in-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #concat }
+@apidoc[Source.concat](Source) { scala="#concat[U>:Out,Mat2](that:akka.stream.Graph[akka.stream.SourceShape[U],Mat2]):FlowOps.this.Repr[U]" java="#concat(akka.stream.Graph)" }
+@apidoc[Flow.concat](Flow) { scala="#concat[U>:Out,Mat2](that:akka.stream.Graph[akka.stream.SourceShape[U],Mat2]):FlowOps.this.Repr[U]" java="#concat(akka.stream.Graph)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/conflate.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/conflate.md
index 7a15fd4582..845db40df4 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/conflate.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/conflate.md
@@ -4,13 +4,11 @@ Allow for a slower downstream by passing incoming elements and a summary into an
@ref[Backpressure aware operators](../index.md#backpressure-aware-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #conflate }
+@apidoc[Source.conflate](Source) { scala="#conflate[O2>:Out](aggregate:(O2,O2)=>O2):FlowOps.this.Repr[O2]" java="#conflate(akka.japi.function.Function2)" }
+@apidoc[Flow.conflate](Flow) { scala="#conflate[O2>:Out](aggregate:(O2,O2)=>O2):FlowOps.this.Repr[O2]" java="#conflate(akka.japi.function.Function2)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/conflateWithSeed.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/conflateWithSeed.md
index d25ecf8e3d..7ff00f1fff 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/conflateWithSeed.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/conflateWithSeed.md
@@ -4,13 +4,11 @@ Allow for a slower downstream by passing incoming elements and a summary into an
@ref[Backpressure aware operators](../index.md#backpressure-aware-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #conflateWithSeed }
+@apidoc[Source.conflateWithSeed](Source) { scala="#conflateWithSeed[S](seed:Out=>S)(aggregate:(S,Out)=>S):FlowOps.this.Repr[S]" java="#conflateWithSeed(akka.japi.function.Function,akka.japi.function.Function2)" }
+@apidoc[Flow.conflateWithSeed](Flow) { scala="#conflateWithSeed[S](seed:Out=>S)(aggregate:(S,Out)=>S):FlowOps.this.Repr[S]" java="#conflateWithSeed(akka.japi.function.Function,akka.japi.function.Function2)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/delay.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/delay.md
index 7d88f2e7cd..b445f7c796 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/delay.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/delay.md
@@ -4,13 +4,11 @@ Delay every element passed through with a specific duration.
@ref[Timer driven operators](../index.md#timer-driven-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #delay }
+@apidoc[Source.delay](Source) { scala="#delay(of:scala.concurrent.duration.FiniteDuration,strategy:akka.stream.DelayOverflowStrategy):FlowOps.this.Repr[Out]" java="#delay(java.time.Duration,akka.stream.DelayOverflowStrategy)" }
+@apidoc[Flow.delay](Flow) { scala="#delay(of:scala.concurrent.duration.FiniteDuration,strategy:akka.stream.DelayOverflowStrategy):FlowOps.this.Repr[Out]" java="#delay(java.time.Duration,akka.stream.DelayOverflowStrategy)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/delayWith.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/delayWith.md
index bdb7312ce7..baadcd9a88 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/delayWith.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/delayWith.md
@@ -4,13 +4,11 @@ Delay every element passed through with a duration that can be controlled dynami
@ref[Timer driven operators](../index.md#timer-driven-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #delayWith }
+@apidoc[Source.delayWith](Source) { scala="#delayWith(delayStrategySupplier:()=>akka.stream.scaladsl.DelayStrategy[Out],overFlowStrategy:akka.stream.DelayOverflowStrategy):FlowOps.this.Repr[Out]" java="#delayWith(java.util.function.Supplier,akka.stream.DelayOverflowStrategy)" }
+@apidoc[Flow.delayWith](Flow) { scala="#delayWith(delayStrategySupplier:()=>akka.stream.scaladsl.DelayStrategy[Out],overFlowStrategy:akka.stream.DelayOverflowStrategy):FlowOps.this.Repr[Out]" java="#delayWith(java.util.function.Supplier,akka.stream.DelayOverflowStrategy)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/detach.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/detach.md
index d63d932906..3172bf7838 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/detach.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/detach.md
@@ -4,13 +4,10 @@ Detach upstream demand from downstream demand without detaching the stream rates
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #detach }
-
-@@@
+@apidoc[Source.detach](Source) { scala="#detach:FlowOps.this.Repr[Out]" java="#detach()" }
+@apidoc[Flow.detach](Flow) { scala="#detach:FlowOps.this.Repr[Out]" java="#detach()" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/divertTo.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/divertTo.md
index 0daccddd15..175ba7e6b3 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/divertTo.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/divertTo.md
@@ -4,13 +4,11 @@ Each upstream element will either be diverted to the given sink, or the downstre
@ref[Fan-out operators](../index.md#fan-out-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #divertTo }
+@apidoc[Source.divertTo](Source) { scala="#divertTo(that:akka.stream.Graph[akka.stream.SinkShape[Out],_],when:Out=>Boolean):FlowOps.this.Repr[Out]" java="#divertTo(akka.stream.Graph,akka.japi.function.Predicate)" }
+@apidoc[Flow.divertTo](Flow) { scala="#divertTo(that:akka.stream.Graph[akka.stream.SinkShape[Out],_],when:Out=>Boolean):FlowOps.this.Repr[Out]" java="#divertTo(akka.stream.Graph,akka.japi.function.Predicate)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/drop.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/drop.md
index 174b21251a..440c433750 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/drop.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/drop.md
@@ -4,13 +4,11 @@ Drop `n` elements and then pass any subsequent element downstream.
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #drop }
+@apidoc[Source.drop](Source) { scala="#drop(n:Long):FlowOps.this.Repr[Out]" java="#drop(long)" }
+@apidoc[Flow.drop](Flow) { scala="#drop(n:Long):FlowOps.this.Repr[Out]" java="#drop(long)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/dropWhile.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/dropWhile.md
index d27f83d586..b19a445e15 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/dropWhile.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/dropWhile.md
@@ -4,13 +4,11 @@ Drop elements as long as a predicate function return true for the element
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #dropWhile }
+@apidoc[Source.dropWhile](Source) { scala="#dropWhile(p:Out=>Boolean):FlowOps.this.Repr[Out]" java="#dropWhile(akka.japi.function.Predicate)" }
+@apidoc[Flow.dropWhile](Flow) { scala="#dropWhile(p:Out=>Boolean):FlowOps.this.Repr[Out]" java="#dropWhile(akka.japi.function.Predicate)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/dropWithin.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/dropWithin.md
index e4d5743c1b..51031dfd36 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/dropWithin.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/dropWithin.md
@@ -4,13 +4,11 @@ Drop elements until a timeout has fired
@ref[Timer driven operators](../index.md#timer-driven-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #dropWithin }
+@apidoc[Source.dropWithin](Source) { scala="#dropWithin(d:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[Out]" java="#dropWithin(java.time.Duration)" }
+@apidoc[Flow.dropWithin](Flow) { scala="#dropWithin(d:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[Out]" java="#dropWithin(java.time.Duration)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/expand.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/expand.md
index 72238e0740..b081eecef3 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/expand.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/expand.md
@@ -4,13 +4,10 @@ Like `extrapolate`, but does not have the `initial` argument, and the `Iterator`
@ref[Backpressure aware operators](../index.md#backpressure-aware-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #expand }
-
-@@@
+@apidoc[Source.expand](Source) { scala="#expand[U](expander:Out=>Iterator[U]):FlowOps.this.Repr[U]" java="#expand(akka.japi.function.Function)" }
+@apidoc[Flow.expand](Flow) { scala="#expand[U](expander:Out=>Iterator[U]):FlowOps.this.Repr[U]" java="#expand(akka.japi.function.Function)" }
## Description
@@ -20,6 +17,21 @@ element, allowing for it to be rewritten and/or filtered.
See @ref:[Understanding extrapolate and expand](../../stream-rate.md#understanding-extrapolate-and-expand) for more information
and examples.
+## Example
+
+Imagine a streaming client decoding a video. It is possible the network bandwidth is a bit
+unreliable. It's fine, as long as the audio remains fluent, it doesn't matter if we can't decode
+a frame or two (or more). But we also want to watermark every decoded frame with the name of
+our colleague. `expand` provides access to the element flowing through the stream
+and let's us create extra frames in case the producer slows down:
+
+Scala
+: @@snip [ExtrapolateAndExpand.scala](/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/ExtrapolateAndExpand.scala) { #expand }
+
+Java
+: @@snip [ExtrapolateAndExpand.java](/akka-docs/src/test/java/jdocs/stream/operators/sourceorflow/ExtrapolateAndExpand.java) { #expand }
+
+
## Reactive Streams semantics
@@@div { .callout }
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/extrapolate.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/extrapolate.md
index 10a85db494..cb5b38b6f4 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/extrapolate.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/extrapolate.md
@@ -4,13 +4,10 @@ Allow for a faster downstream by expanding the last emitted element to an `Itera
@ref[Backpressure aware operators](../index.md#backpressure-aware-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #extrapolate }
-
-@@@
+@apidoc[Source.extrapolate](Source) { scala="#extrapolate[U>:Out](extrapolator:U=>Iterator[U],initial:Option[U]):FlowOps.this.Repr[U]" java="#extrapolate(akka.japi.function.Function,java.lang.Object)" }
+@apidoc[Flow.extrapolate](Flow) { scala="#extrapolate[U>:Out](extrapolator:U=>Iterator[U],initial:Option[U]):FlowOps.this.Repr[U]" java="#extrapolate(akka.japi.function.Function,java.lang.Object)" }
## Description
@@ -25,6 +22,19 @@ Includes an optional `initial` argument to prevent blocking the entire stream wh
See @ref:[Understanding extrapolate and expand](../../stream-rate.md#understanding-extrapolate-and-expand) for more information
and examples.
+## Example
+
+Imagine a videoconference client decoding a video feed from a colleague working remotely. It is possible
+the network bandwidth is a bit unreliable. It's fine, as long as the audio remains fluent, it doesn't matter
+if we can't decode a frame or two (or more). When a frame is dropped, though, we want the UI to show the last
+frame decoded:
+
+Scala
+: @@snip [ExtrapolateAndExpand.scala](/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/ExtrapolateAndExpand.scala) { #extrapolate }
+
+Java
+: @@snip [ExtrapolateAndExpand.java](/akka-docs/src/test/java/jdocs/stream/operators/sourceorflow/ExtrapolateAndExpand.java) { #extrapolate }
+
## Reactive Streams semantics
@@@div { .callout }
@@ -36,4 +46,3 @@ and examples.
**completes** when upstream completes
@@@
-
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/filter.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/filter.md
index 2598a7325d..8e3d06e864 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/filter.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/filter.md
@@ -4,13 +4,11 @@ Filter the incoming elements using a predicate.
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #filter }
+@apidoc[Source.filter](Source) { scala="#filter(p:Out=>Boolean):FlowOps.this.Repr[Out]" java="#filter(akka.japi.function.Predicate)" }
+@apidoc[Flow.filter](Flow) { scala="#filter(p:Out=>Boolean):FlowOps.this.Repr[Out]" java="#filter(akka.japi.function.Predicate)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/filterNot.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/filterNot.md
index 8ea399269e..8b22f92e22 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/filterNot.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/filterNot.md
@@ -4,13 +4,11 @@ Filter the incoming elements using a predicate.
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #filterNot }
+@apidoc[Source.filterNot](Source) { scala="#filterNot(p:Out=>Boolean):FlowOps.this.Repr[Out]" java="#filterNot(akka.japi.function.Predicate)" }
+@apidoc[Flow.filterNot](Flow) { scala="#filterNot(p:Out=>Boolean):FlowOps.this.Repr[Out]" java="#filterNot(akka.japi.function.Predicate)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/flatMapConcat.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/flatMapConcat.md
index 7b71e962d5..27f7bcd030 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/flatMapConcat.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/flatMapConcat.md
@@ -4,18 +4,32 @@ Transform each input element into a `Source` whose elements are then flattened i
@ref[Nesting and flattening operators](../index.md#nesting-and-flattening-operators)
-@@@div { .group-scala }
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #flatMapConcat }
+@apidoc[Flow.flatMapConcat](Flow) { scala="#flatMapConcat[T,M](f:Out=%3akka.stream.Graph[akka.stream.SourceShape[T],M]):FlowOps.this.Repr[T]" java="#flatMapConcat(akka.japi.function.Function)" }
-@@@
+See also: @ref:[flatMapMerge](flatMapMerge.md)
## Description
Transform each input element into a `Source` whose elements are then flattened into the output stream through
-concatenation. This means each source is fully consumed before consumption of the next source starts.
+concatenation. This means each source is fully consumed before consumption of the next source starts.
+
+## Example
+
+In the following example `flatMapConcat` is used to create a `Source` for each incoming customerId. This could be, for example,
+ a calculation or a query to a database. Each customer is then passed to `lookupCustomerEvents` which returns
+a `Source`. All the events for a customer are delivered before moving to the next customer.
+
+Scala
+: @@snip [FlatMapConcat.scala](/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/FlatMapConcat.scala) { #flatmap-concat}
+
+Java
+: @@snip [FlatMapConcat.java](/akka-docs/src/test/java/jdocs/stream/operators/sourceorflow/FlatMapConcat.java) { #flatmap-concat }
+
+
+
## Reactive Streams semantics
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/flatMapPrefix.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/flatMapPrefix.md
index e2dff32a5c..a064b1243e 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/flatMapPrefix.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/flatMapPrefix.md
@@ -4,13 +4,11 @@ Use the first `n` elements from the stream to determine how to process the rest.
@ref[Nesting and flattening operators](../index.md#nesting-and-flattening-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #flatMapPrefix }
+@apidoc[Source.flatMapPrefix](Source) { scala="#flatMapPrefix[Out2,Mat2](n:Int)(f:scala.collection.immutable.Seq[Out]=>akka.stream.scaladsl.Flow[Out,Out2,Mat2]):FlowOps.this.Repr[Out2]" java="#flatMapPrefix(int,akka.japi.function.Function)" }
+@apidoc[Flow.flatMapPrefix](Flow) { scala="#flatMapPrefix[Out2,Mat2](n:Int)(f:scala.collection.immutable.Seq[Out]=>akka.stream.scaladsl.Flow[Out,Out2,Mat2]):FlowOps.this.Repr[Out2]" java="#flatMapPrefix(int,akka.japi.function.Function)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/fold.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/fold.md
index 12628fe8dc..7724eb4ff6 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/fold.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/fold.md
@@ -4,13 +4,10 @@ Start with current value `zero` and then apply the current and next value to the
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #fold }
-
-@@@
+@apidoc[Source.fold](Source) { scala="#fold[T](zero:T)(f:(T,Out)=>T):FlowOps.this.Repr[T]" java="#fold(java.lang.Object,akka.japi.function.Function2)" }
+@apidoc[Flow.fold](Flow) { scala="#fold[T](zero:T)(f:(T,Out)=>T):FlowOps.this.Repr[T]" java="#fold(java.lang.Object,akka.japi.function.Function2)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/foldAsync.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/foldAsync.md
index f83d712c99..983fa349a8 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/foldAsync.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/foldAsync.md
@@ -4,13 +4,10 @@ Just like `fold` but receives a function that results in a @scala[`Future`] @jav
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #foldAsync }
-
-@@@
+@apidoc[Source.foldAsync](Source) { scala="#foldAsync[T](zero:T)(f:(T,Out)=>scala.concurrent.Future[T]):FlowOps.this.Repr[T]" java="#foldAsync(java.lang.Object,akka.japi.function.Function2)" }
+@apidoc[Flow.foldAsync](Flow) { scala="#foldAsync[T](zero:T)(f:(T,Out)=>scala.concurrent.Future[T]):FlowOps.this.Repr[T]" java="#foldAsync(java.lang.Object,akka.japi.function.Function2)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/fromMaterializer.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/fromMaterializer.md
index abae21cb13..ecfdeeee3e 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/fromMaterializer.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/fromMaterializer.md
@@ -1,19 +1,16 @@
-# Source/Flow.fromMaterializer
+# fromMaterializer
Defer the creation of a `Source/Flow` until materialization and access `Materializer` and `Attributes`
@ref[Simple operators](../index.md#simple-operators)
-@@@ div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #fromMaterializer }
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #fromMaterializer }
+@apidoc[Source.fromMaterializer](Source$) { scala="#fromMaterializer[T,M](factory:(akka.stream.Materializer,akka.stream.Attributes)=>akka.stream.scaladsl.Source[T,M]):akka.stream.scaladsl.Source[T,scala.concurrent.Future[M]]" java="#fromMaterializer(java.util.function.BiFunction)" }
+@apidoc[Flow.fromMaterializer](Flow$) { scala="#fromMaterializer[T,U,M](factory:(akka.stream.Materializer,akka.stream.Attributes)=>akka.stream.scaladsl.Flow[T,U,M]):akka.stream.scaladsl.Flow[T,U,scala.concurrent.Future[M]]" java="#fromMaterializer(java.util.function.BiFunction)" }
-@@@
## Description
Typically used when access to materializer is needed to run a different stream during the construction of a source/flow.
-Can also be used to access the underlying `ActorSystem` from `Materializer`.
\ No newline at end of file
+Can also be used to access the underlying `ActorSystem` from `Materializer`.
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/groupBy.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/groupBy.md
index c7d4ee7f28..137a1bf52f 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/groupBy.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/groupBy.md
@@ -4,13 +4,11 @@ Demultiplex the incoming stream into separate output streams.
@ref[Nesting and flattening operators](../index.md#nesting-and-flattening-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #groupBy }
+@apidoc[Source.groupBy](Source) { scala="#groupBy[K](maxSubstreams:Int,f:Out=>K):akka.stream.scaladsl.SubFlow[Out,Mat,FlowOps.this.Repr,FlowOps.this.Closed]" java="#groupBy(int,akka.japi.function.Function,boolean)" }
+@apidoc[Flow.groupBy](Flow) { scala="#groupBy[K](maxSubstreams:Int,f:Out=>K):akka.stream.scaladsl.SubFlow[Out,Mat,FlowOps.this.Repr,FlowOps.this.Closed]" java="#groupBy(int,akka.japi.function.Function,boolean)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/grouped.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/grouped.md
index 1003913a42..d3c5d4822a 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/grouped.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/grouped.md
@@ -4,13 +4,11 @@ Accumulate incoming events until the specified number of elements have been accu
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #grouped }
+@apidoc[Source.grouped](Source) { scala="#grouped(n:Int):FlowOps.this.Repr[scala.collection.immutable.Seq[Out]]" java="#grouped(int)" }
+@apidoc[Flow.grouped](Flow) { scala="#grouped(n:Int):FlowOps.this.Repr[scala.collection.immutable.Seq[Out]]" java="#grouped(int)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/groupedWeightedWithin.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/groupedWeightedWithin.md
index 6aa5cb6061..afe45cd9ef 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/groupedWeightedWithin.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/groupedWeightedWithin.md
@@ -4,13 +4,11 @@ Chunk up this stream into groups of elements received within a time window, or l
@ref[Timer driven operators](../index.md#timer-driven-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #groupedWeightedWithin }
+@apidoc[Source.groupedWeightedWithin](Source) { scala="#groupedWeightedWithin(maxWeight:Long,d:scala.concurrent.duration.FiniteDuration)(costFn:Out=>Long):FlowOps.this.Repr[scala.collection.immutable.Seq[Out]]" java="#groupedWeightedWithin(long,akka.japi.function.Function,java.time.Duration)" }
+@apidoc[Flow.groupedWeightedWithin](Flow) { scala="#groupedWeightedWithin(maxWeight:Long,d:scala.concurrent.duration.FiniteDuration)(costFn:Out=>Long):FlowOps.this.Repr[scala.collection.immutable.Seq[Out]]" java="#groupedWeightedWithin(long,akka.japi.function.Function,java.time.Duration)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/groupedWithin.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/groupedWithin.md
index e06517255f..1cefe4fcd1 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/groupedWithin.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/groupedWithin.md
@@ -4,13 +4,11 @@ Chunk up this stream into groups of elements received within a time window, or l
@ref[Timer driven operators](../index.md#timer-driven-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #groupedWithin }
+@apidoc[Source.groupedWithin](Source) { scala="#groupedWithin(n:Int,d:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[scala.collection.immutable.Seq[Out]]" java="#groupedWithin(int,java.time.Duration)" }
+@apidoc[Flow.groupedWithin](Flow) { scala="#groupedWithin(n:Int,d:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[scala.collection.immutable.Seq[Out]]" java="#groupedWithin(int,java.time.Duration)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/idleTimeout.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/idleTimeout.md
index 295aec1cf0..574c243a90 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/idleTimeout.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/idleTimeout.md
@@ -4,13 +4,10 @@ If the time between two processed elements exceeds the provided timeout, the str
@ref[Time aware operators](../index.md#time-aware-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #idleTimeout }
-
-@@@
+@apidoc[Source.idleTimeout](Source) { scala="#idleTimeout(timeout:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[Out]" java="#idleTimeout(java.time.Duration)" }
+@apidoc[Flow.idleTimeout](Flow) { scala="#idleTimeout(timeout:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[Out]" java="#idleTimeout(java.time.Duration)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/initialDelay.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/initialDelay.md
index c9e9d0b7b2..a0c6ce9871 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/initialDelay.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/initialDelay.md
@@ -4,13 +4,11 @@ Delays the initial element by the specified duration.
@ref[Timer driven operators](../index.md#timer-driven-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #initialDelay }
+@apidoc[Source.initialDelay](Source$) { scala="#initialDelay(delay:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[Out]" java="#initialDelay(java.time.Duration)" }
+@apidoc[Flow.initialDelay](Flow) { scala="#initialDelay(delay:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[Out]" java="#initialDelay(java.time.Duration)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/initialTimeout.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/initialTimeout.md
index ef52d058b4..151a25d866 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/initialTimeout.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/initialTimeout.md
@@ -4,13 +4,11 @@ If the first element has not passed through this operators before the provided t
@ref[Time aware operators](../index.md#time-aware-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #initialTimeout }
+@apidoc[Source.initialTimeout](Source) { scala="#initialTimeout(timeout:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[Out]" java="#initialTimeout(java.time.Duration)" }
+@apidoc[Flow.initialTimeout](Flow) { scala="#initialTimeout(timeout:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[Out]" java="#initialTimeout(java.time.Duration)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/interleave.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/interleave.md
index 227b1b7b20..18983af6a7 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/interleave.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/interleave.md
@@ -4,13 +4,11 @@ Emits a specifiable number of elements from the original source, then from the p
@ref[Fan-in operators](../index.md#fan-in-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #interleave }
+@apidoc[Source.interleave](Source) { scala="#interleave[U>:Out](that:akka.stream.Graph[akka.stream.SourceShape[U],_],segmentSize:Int,eagerClose:Boolean):FlowOps.this.Repr[U]" java="#interleave(akka.stream.Graph,int,boolean)" }
+@apidoc[Flow.interleave](Flow) { scala="#interleave[U>:Out](that:akka.stream.Graph[akka.stream.SourceShape[U],_],segmentSize:Int,eagerClose:Boolean):FlowOps.this.Repr[U]" java="#interleave(akka.stream.Graph,int,boolean)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/intersperse.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/intersperse.md
index 7bceb1ec19..7b0b843640 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/intersperse.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/intersperse.md
@@ -4,13 +4,11 @@ Intersperse stream with provided element similar to `List.mkString`.
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #intersperse }
+@apidoc[Source.intersperse](Source) { scala="#intersperse[T>:Out](start:T,inject:T,end:T):FlowOps.this.Repr[T]" java="#intersperse(java.lang.Object,java.lang.Object,java.lang.Object)" }
+@apidoc[Flow.intersperse](Flow) { scala="#intersperse[T>:Out](start:T,inject:T,end:T):FlowOps.this.Repr[T]" java="#intersperse(java.lang.Object,java.lang.Object,java.lang.Object)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/keepAlive.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/keepAlive.md
index 1f36a967bb..974a9025d3 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/keepAlive.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/keepAlive.md
@@ -4,13 +4,11 @@ Injects additional (configured) elements if upstream does not emit for a configu
@ref[Time aware operators](../index.md#time-aware-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #keepAlive }
+@apidoc[Source.keepAlive](Source) { scala="#keepAlive[U>:Out](maxIdle:scala.concurrent.duration.FiniteDuration,injectedElem:()=>U):FlowOps.this.Repr[U]" java="#keepAlive(java.time.Duration,akka.japi.function.Creator)" }
+@apidoc[Flow.keepAlive](Flow) { scala="#keepAlive[U>:Out](maxIdle:scala.concurrent.duration.FiniteDuration,injectedElem:()=>U):FlowOps.this.Repr[U]" java="#keepAlive(java.time.Duration,akka.japi.function.Creator)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/log.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/log.md
index 2d9fa25346..f0233810cf 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/log.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/log.md
@@ -4,13 +4,10 @@ Log elements flowing through the stream as well as completion and erroring.
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #log }
-
-@@@
+@apidoc[Source.log](Source) { scala="#log(name:String,extract:Out=>Any)(implicitlog:akka.event.LoggingAdapter):FlowOps.this.Repr[Out]" java="#log(java.lang.String)" }
+@apidoc[Flow.log](Flow) { scala="#log(name:String,extract:Out=>Any)(implicitlog:akka.event.LoggingAdapter):FlowOps.this.Repr[Out]" java="#log(java.lang.String)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/logWithMarker.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/logWithMarker.md
index 6f7d5a80ce..7364bbd4d3 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/logWithMarker.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/logWithMarker.md
@@ -6,12 +6,9 @@ Log elements flowing through the stream as well as completion and erroring.
## Signature
+@apidoc[Source.logWithMarker](Source) { scala="#logWithMarker(name:String,marker:Out=>akka.event.LogMarker,extract:Out=>Any)(implicitlog:akka.event.MarkerLoggingAdapter):FlowOps.this.Repr[Out]" java="#logWithMarker(java.lang.String,akka.japi.function.Function)" }
+@apidoc[Flow.logWithMarker](Flow) { scala="#logWithMarker(name:String,marker:Out=>akka.event.LogMarker,extract:Out=>Any)(implicitlog:akka.event.MarkerLoggingAdapter):FlowOps.this.Repr[Out]" java="#logWithMarker(java.lang.String,akka.japi.function.Function)" }
-Scala
-: @@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #logWithMarker }
-
-Java
-: @@snip [FlowLogWithMarkerTest.java](/akka-stream-tests/src/test/java/akka/stream/javadsl/FlowLogWithMarkerTest.java) { #signature }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/map.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/map.md
index 49fa7e3ccc..d1001d8ab7 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/map.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/map.md
@@ -4,13 +4,10 @@ Transform each element in the stream by calling a mapping function with it and p
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #map }
-
-@@@
+@apidoc[Source.map](Source) { scala="#map[T](f:Out=>T):FlowOps.this.Repr[T]" java="#map(akka.japi.function.Function)" }
+@apidoc[Flow.map](Flow) { scala="#map[T](f:Out=>T):FlowOps.this.Repr[T]" java="#map(akka.japi.function.Function)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapAsync.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapAsync.md
index 4bce170c46..8b6e40119f 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapAsync.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapAsync.md
@@ -4,13 +4,11 @@ Pass incoming elements to a function that return a @scala[`Future`] @java[`Compl
@ref[Asynchronous operators](../index.md#asynchronous-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #mapAsync }
+@apidoc[Source.mapAsync](Source) { scala="#mapAsync[T](parallelism:Int)(f:Out=>scala.concurrent.Future[T]):FlowOps.this.Repr[T]" java="#mapAsync(int,akka.japi.function.Function)" }
+@apidoc[Flow.mapAsync](Flow) { scala="#mapAsync[T](parallelism:Int)(f:Out=>scala.concurrent.Future[T]):FlowOps.this.Repr[T]" java="#mapAsync(int,akka.japi.function.Function)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapAsyncUnordered.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapAsyncUnordered.md
index f83a1ccb66..9eaad32aa4 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapAsyncUnordered.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapAsyncUnordered.md
@@ -4,13 +4,11 @@ Like `mapAsync` but @scala[`Future`] @java[`CompletionStage`] results are passed
@ref[Asynchronous operators](../index.md#asynchronous-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #mapAsyncUnordered }
+@apidoc[Source.mapAsyncUnordered](Source) { scala="#mapAsyncUnordered[T](parallelism:Int)(f:Out=>scala.concurrent.Future[T]):FlowOps.this.Repr[T]" java="#mapAsyncUnordered(int,akka.japi.function.Function)" }
+@apidoc[Flow.mapAsyncUnordered](Flow) { scala="#mapAsyncUnordered[T](parallelism:Int)(f:Out=>scala.concurrent.Future[T]):FlowOps.this.Repr[T]" java="#mapAsyncUnordered(int,akka.japi.function.Function)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapConcat.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapConcat.md
index 8b02dd8b46..c204a089b2 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapConcat.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapConcat.md
@@ -4,13 +4,11 @@ Transform each element into zero or more elements that are individually passed d
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #mapConcat }
+@apidoc[Source.mapConcat](Source) { scala="#mapConcat[T](f:Out=>scala.collection.immutable.Iterable[T]):FlowOps.this.Repr[T]" java="#mapConcat(akka.japi.function.Function)" }
+@apidoc[Flow.mapConcat](Flow) { scala="#mapConcat[T](f:Out=>scala.collection.immutable.Iterable[T]):FlowOps.this.Repr[T]" java="#mapConcat(akka.japi.function.Function)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapError.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapError.md
index 259d8b789e..b20e3a2a01 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapError.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mapError.md
@@ -4,13 +4,11 @@ While similar to `recover` this operators can be used to transform an error sign
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #mapError }
+@apidoc[Source.mapError](Source) { scala="#mapError(pf:PartialFunction[Throwable,Throwable]):FlowOps.this.Repr[Out]" java="#mapError(java.lang.Class,akka.japi.function.Function)" }
+@apidoc[Flow.mapError](Flow) { scala="#mapError(pf:PartialFunction[Throwable,Throwable]):FlowOps.this.Repr[Out]" java="#mapError(java.lang.Class,akka.japi.function.Function)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/merge.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/merge.md
index a62ec24c4d..93a56600fb 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/merge.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/merge.md
@@ -4,13 +4,11 @@ Merge multiple sources.
@ref[Fan-in operators](../index.md#fan-in-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #merge }
+@apidoc[Source.merge](Source) { scala="#merge[U>:Out,M](that:akka.stream.Graph[akka.stream.SourceShape[U],M],eagerComplete:Boolean):FlowOps.this.Repr[U]" java="#merge(akka.stream.Graph)" java="#merge(akka.stream.Graph,boolean)" }
+@apidoc[Flow.merge](Flow) { scala="#merge[U>:Out,M](that:akka.stream.Graph[akka.stream.SourceShape[U],M],eagerComplete:Boolean):FlowOps.this.Repr[U]" java="#merge(akka.stream.Graph)" java="#merge(akka.stream.Graph,boolean)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mergeLatest.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mergeLatest.md
index 897564e1d6..28af71722f 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mergeLatest.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mergeLatest.md
@@ -4,20 +4,30 @@ Merge multiple sources.
@ref[Fan-in operators](../index.md#fan-in-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #mergeLatest }
-
-@@@
+@apidoc[Flow.mergeLatest](Flow) { scala="#mergeLatest[U%3E:Out,M](that:akka.stream.Graph[akka.stream.SourceShape[U],M],eagerComplete:Boolean):FlowOps.this.Repr[scala.collection.immutable.Seq[U]]" java="#mergeLatest(akka.stream.Graph,boolean)" }
## Description
MergeLatest joins elements from N input streams into stream of lists of size N.
-i-th element in list is the latest emitted element from i-th input stream.
+The i-th element in list is the latest emitted element from i-th input stream.
MergeLatest emits list for each element emitted from some input stream,
but only after each input stream emitted at least one element
+If `eagerComplete` is set to true then it completes as soon as the first upstream
+completes otherwise when all upstreams complete.
+
+## Example
+
+This example takes a stream of prices and quantities and emits the price each time the
+price of quantity changes:
+
+Scala
+: @@snip [MergeLatest.scala](/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/MergeLatest.scala) { #mergeLatest }
+
+Java
+: @@snip [MergeLatest.java](/akka-docs/src/test/java/jdocs/stream/operators/sourceorflow/MergeLatest.java) { #mergeLatest }
+
## Reactive Streams semantics
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mergeSorted.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mergeSorted.md
index 72e3cb77e7..0856a004b5 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mergeSorted.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/mergeSorted.md
@@ -4,13 +4,11 @@ Merge multiple sources.
@ref[Fan-in operators](../index.md#fan-in-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #mergeSorted }
+@apidoc[Source.mergeSorted](Source) { scala="#mergeSorted[U>:Out,M](that:akka.stream.Graph[akka.stream.SourceShape[U],M])(implicitord:Ordering[U]):FlowOps.this.Repr[U]" java="#mergeSorted(akka.stream.Graph,java.util.Comparator)" }
+@apidoc[Flow.mergeSorted](Flow) { scala="#mergeSorted[U>:Out,M](that:akka.stream.Graph[akka.stream.SourceShape[U],M])(implicitord:Ordering[U]):FlowOps.this.Repr[U]" java="#mergeSorted(akka.stream.Graph,java.util.Comparator)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/monitor.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/monitor.md
index b188b8016b..e8930f8119 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/monitor.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/monitor.md
@@ -4,13 +4,11 @@ Materializes to a `FlowMonitor` that monitors messages flowing through or comple
@ref[Watching status operators](../index.md#watching-status-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #monitor }
+@apidoc[Source.monitor](Source) { scala="#monitor[Mat2]()(combine:(Mat,akka.stream.FlowMonitor[Out])=>Mat2):FlowOpsMat.this.ReprMat[Out,Mat2]" java="#monitor(akka.japi.function.Function2)" java="#monitor()" }
+@apidoc[Flow.monitor](Flow) { scala="#monitor[Mat2]()(combine:(Mat,akka.stream.FlowMonitor[Out])=>Mat2):FlowOpsMat.this.ReprMat[Out,Mat2]" java="#monitor(akka.japi.function.Function2)" java="#monitor()" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/orElse.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/orElse.md
index f26ab03112..a60805e105 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/orElse.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/orElse.md
@@ -4,13 +4,11 @@ If the primary source completes without emitting any elements, the elements from
@ref[Fan-in operators](../index.md#fan-in-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #orElse }
+@apidoc[Source.orElse](Source) { scala="#orElse[U>:Out,Mat2](secondary:akka.stream.Graph[akka.stream.SourceShape[U],Mat2]):FlowOps.this.Repr[U]" java="#orElse(akka.stream.Graph)" }
+@apidoc[Flow.orElse](Flow) { scala="#orElse[U>:Out,Mat2](secondary:akka.stream.Graph[akka.stream.SourceShape[U],Mat2]):FlowOps.this.Repr[U]" java="#orElse(akka.stream.Graph)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/prefixAndTail.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/prefixAndTail.md
index 4bc12bff27..b68beb770c 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/prefixAndTail.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/prefixAndTail.md
@@ -4,13 +4,11 @@ Take up to *n* elements from the stream (less than *n* only if the upstream comp
@ref[Nesting and flattening operators](../index.md#nesting-and-flattening-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #prefixAndTail }
+@apidoc[Source.prefixAndTail](Source) { scala="#prefixAndTail[U>:Out](n:Int):FlowOps.this.Repr[(scala.collection.immutable.Seq[Out],akka.stream.scaladsl.Source[U,akka.NotUsed])]" java="#prefixAndTail(int)" }
+@apidoc[Flow.prefixAndTail](Flow) { scala="#prefixAndTail[U>:Out](n:Int):FlowOps.this.Repr[(scala.collection.immutable.Seq[Out],akka.stream.scaladsl.Source[U,akka.NotUsed])]" java="#prefixAndTail(int)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/prepend.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/prepend.md
index cd738f3e3f..dab4264eb7 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/prepend.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/prepend.md
@@ -4,13 +4,11 @@ Prepends the given source to the flow, consuming it until completion before the
@ref[Fan-in operators](../index.md#fan-in-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #prepend }
+@apidoc[Source.prepend](Source) { scala="#prepend[U>:Out,Mat2](that:akka.stream.Graph[akka.stream.SourceShape[U],Mat2]):FlowOps.this.Repr[U]" java="#prepend(akka.stream.Graph)" }
+@apidoc[Flow.prepend](Flow) { scala="#prepend[U>:Out,Mat2](that:akka.stream.Graph[akka.stream.SourceShape[U],Mat2]):FlowOps.this.Repr[U]" java="#prepend(akka.stream.Graph)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/recover.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/recover.md
index 55dd917fc7..1737d63087 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/recover.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/recover.md
@@ -4,13 +4,11 @@ Allow sending of one last element downstream when a failure has happened upstrea
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #recover }
+@apidoc[Source.recover](Source) { scala="#recover[T>:Out](pf:PartialFunction[Throwable,T]):FlowOps.this.Repr[T]" java="#recover(scala.PartialFunction)" java="#recover(java.lang.Class,java.util.function.Supplier)" }
+@apidoc[Flow.recover](Flow) { scala="#recover[T>:Out](pf:PartialFunction[Throwable,T]):FlowOps.this.Repr[T]" java="#recover(scala.PartialFunction)" java="#recover(java.lang.Class,java.util.function.Supplier)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/recoverWith.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/recoverWith.md
index 084f122d25..5ceb776f4e 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/recoverWith.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/recoverWith.md
@@ -4,13 +4,11 @@ Allow switching to alternative Source when a failure has happened upstream.
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #recoverWith }
+@apidoc[Source.recoverWith](Source) { scala="#recoverWith[T>:Out](pf:PartialFunction[Throwable,akka.stream.Graph[akka.stream.SourceShape[T],akka.NotUsed]]):FlowOps.this.Repr[T]" java="#recoverWith(java.lang.Class,java.util.function.Supplier)" }
+@apidoc[Flow.recoverWith](Flow) { scala="#recoverWith[T>:Out](pf:PartialFunction[Throwable,akka.stream.Graph[akka.stream.SourceShape[T],akka.NotUsed]]):FlowOps.this.Repr[T]" java="#recoverWith(java.lang.Class,java.util.function.Supplier)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/recoverWithRetries.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/recoverWithRetries.md
index 4dfeddad29..2c88da08d3 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/recoverWithRetries.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/recoverWithRetries.md
@@ -4,13 +4,11 @@ RecoverWithRetries allows to switch to alternative Source on flow failure.
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #recoverWithRetries }
+@apidoc[Source.recoverWithRetries](Source) { scala="#recoverWithRetries[T>:Out](attempts:Int,pf:PartialFunction[Throwable,akka.stream.Graph[akka.stream.SourceShape[T],akka.NotUsed]]):FlowOps.this.Repr[T]" java="#recoverWithRetries(int,java.lang.Class,java.util.function.Supplier)" }
+@apidoc[Flow.recoverWithRetries](Flow) { scala="#recoverWithRetries[T>:Out](attempts:Int,pf:PartialFunction[Throwable,akka.stream.Graph[akka.stream.SourceShape[T],akka.NotUsed]]):FlowOps.this.Repr[T]" java="#recoverWithRetries(int,java.lang.Class,java.util.function.Supplier)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/reduce.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/reduce.md
index ef411f954f..7e2ba0b9a4 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/reduce.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/reduce.md
@@ -4,13 +4,11 @@ Start with first element and then apply the current and next value to the given
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #reduce }
+@apidoc[Source.reduce](Source) { scala="#reduce[T>:Out](f:(T,T)=>T):FlowOps.this.Repr[T]" java="#reduce(akka.japi.function.Function2)" }
+@apidoc[Flow.reduce](Flow) { scala="#reduce[T>:Out](f:(T,T)=>T):FlowOps.this.Repr[T]" java="#reduce(akka.japi.function.Function2)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/scan.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/scan.md
index c300126634..9d94ececf0 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/scan.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/scan.md
@@ -4,13 +4,11 @@ Emit its current value, which starts at `zero`, and then apply the current and n
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #scan }
+@apidoc[Source.scan](Source) { scala="#scan[T](zero:T)(f:(T,Out)=>T):FlowOps.this.Repr[T]" java="#scan(java.lang.Object,akka.japi.function.Function2)" }
+@apidoc[Flow.scan](Flow) { scala="#scan[T](zero:T)(f:(T,Out)=>T):FlowOps.this.Repr[T]" java="#scan(java.lang.Object,akka.japi.function.Function2)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/scanAsync.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/scanAsync.md
index 8b43b7865e..c1d7e18686 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/scanAsync.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/scanAsync.md
@@ -4,13 +4,11 @@ Just like @ref[`scan`](./scan.md) but receives a function that results in a @sca
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #scanAsync }
+@apidoc[Source.scanAsync](Source) { scala="#scanAsync[T](zero:T)(f:(T,Out)=>scala.concurrent.Future[T]):FlowOps.this.Repr[T]" java="#scanAsync(java.lang.Object,akka.japi.function.Function2)" }
+@apidoc[Flow.scanAsync](Flow) { scala="#scanAsync[T](zero:T)(f:(T,Out)=>scala.concurrent.Future[T]):FlowOps.this.Repr[T]" java="#scanAsync(java.lang.Object,akka.japi.function.Function2)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/setup.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/setup.md
index 82bce88eed..32d8e7f042 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/setup.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/setup.md
@@ -1,4 +1,4 @@
-# Source/Flow.setup
+# setup
Defer the creation of a `Source/Flow` until materialization and access `Materializer` and `Attributes`
@@ -10,16 +10,12 @@ The `setup` operator has been deprecated, use @ref:[fromMaterializer](./fromMate
@@@
-@@@ div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #setup }
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #setup }
-
-@@@
+@apidoc[Source.setup](Source$) { scala="#setup[T,M](factory:(akka.stream.ActorMaterializer,akka.stream.Attributes)=>akka.stream.scaladsl.Source[T,M]):akka.stream.scaladsl.Source[T,scala.concurrent.Future[M]]" java="#setup(java.util.function.BiFunction)" }
+@apidoc[Flow.setup](Flow$) { scala="#setup[T,U,M](factory:(akka.stream.ActorMaterializer,akka.stream.Attributes)=>akka.stream.scaladsl.Flow[T,U,M]):akka.stream.scaladsl.Flow[T,U,scala.concurrent.Future[M]]" java="#setup(java.util.function.BiFunction)" }
## Description
Typically used when access to materializer is needed to run a different stream during the construction of a source/flow.
-Can also be used to access the underlying `ActorSystem` from `ActorMaterializer`.
\ No newline at end of file
+Can also be used to access the underlying `ActorSystem` from `ActorMaterializer`.
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/splitAfter.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/splitAfter.md
index a71b8b3eb2..31ecedc25f 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/splitAfter.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/splitAfter.md
@@ -4,13 +4,11 @@ End the current substream whenever a predicate returns `true`, starting a new su
@ref[Nesting and flattening operators](../index.md#nesting-and-flattening-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #splitAfter }
+@apidoc[Source.splitAfter](Source) { scala="#splitAfter(substreamCancelStrategy:akka.stream.SubstreamCancelStrategy)(p:Out=>Boolean):akka.stream.scaladsl.SubFlow[Out,Mat,FlowOps.this.Repr,FlowOps.this.Closed]" java="#splitAfter(akka.stream.SubstreamCancelStrategy,akka.japi.function.Predicate)" }
+@apidoc[Flow.splitAfter](Flow) { scala="#splitAfter(substreamCancelStrategy:akka.stream.SubstreamCancelStrategy)(p:Out=>Boolean):akka.stream.scaladsl.SubFlow[Out,Mat,FlowOps.this.Repr,FlowOps.this.Closed]" java="#splitAfter(akka.stream.SubstreamCancelStrategy,akka.japi.function.Predicate)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/splitWhen.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/splitWhen.md
index 609afe8ca2..27345b68f4 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/splitWhen.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/splitWhen.md
@@ -4,13 +4,11 @@ Split off elements into a new substream whenever a predicate function return `tr
@ref[Nesting and flattening operators](../index.md#nesting-and-flattening-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #splitWhen }
+@apidoc[Source.splitWhen](Source) { scala="#splitWhen(substreamCancelStrategy:akka.stream.SubstreamCancelStrategy)(p:Out=>Boolean):akka.stream.scaladsl.SubFlow[Out,Mat,FlowOps.this.Repr,FlowOps.this.Closed]" java="#splitWhen(akka.stream.SubstreamCancelStrategy,akka.japi.function.Predicate)" }
+@apidoc[Flow.splitWhen](Flow) { scala="#splitWhen(substreamCancelStrategy:akka.stream.SubstreamCancelStrategy)(p:Out=>Boolean):akka.stream.scaladsl.SubFlow[Out,Mat,FlowOps.this.Repr,FlowOps.this.Closed]" java="#splitWhen(akka.stream.SubstreamCancelStrategy,akka.japi.function.Predicate)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/take.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/take.md
index 8f24acca3c..62249b453a 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/take.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/take.md
@@ -4,13 +4,11 @@ Pass `n` incoming elements downstream and then complete
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #take }
+@apidoc[Source.take](Source) { scala="#take(n:Long):FlowOps.this.Repr[Out]" java="#take(long)" }
+@apidoc[Flow.take](Flow) { scala="#take(n:Long):FlowOps.this.Repr[Out]" java="#take(long)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/takeWhile.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/takeWhile.md
index 8cdf517409..ac2660bdd5 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/takeWhile.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/takeWhile.md
@@ -4,13 +4,11 @@ Pass elements downstream as long as a predicate function returns true and then c
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #takeWhile }
+@apidoc[Source.takeWhile](Source) { scala="#takeWhile(p:Out=>Boolean,inclusive:Boolean):FlowOps.this.Repr[Out]" java="#takeWhile(akka.japi.function.Predicate)" }
+@apidoc[Flow.takeWhile](Flow) { scala="#takeWhile(p:Out=>Boolean,inclusive:Boolean):FlowOps.this.Repr[Out]" java="#takeWhile(akka.japi.function.Predicate)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/takeWithin.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/takeWithin.md
index ffcabd3d55..51696e81dd 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/takeWithin.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/takeWithin.md
@@ -4,13 +4,11 @@ Pass elements downstream within a timeout and then complete.
@ref[Timer driven operators](../index.md#timer-driven-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #takeWithin }
+@apidoc[Source.takeWithin](Source) { scala="#takeWithin(d:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[Out]" java="#takeWithin(java.time.Duration)" }
+@apidoc[Flow.takeWithin](Flow) { scala="#takeWithin(d:scala.concurrent.duration.FiniteDuration):FlowOps.this.Repr[Out]" java="#takeWithin(java.time.Duration)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/throttle.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/throttle.md
index 77ad31c338..8dc1dbc961 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/throttle.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/throttle.md
@@ -4,13 +4,10 @@ Limit the throughput to a specific number of elements per time unit, or a specif
@ref[Simple operators](../index.md#simple-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #throttle }
-
-@@@
+@apidoc[Source.throttle](Source) { scala="#throttle(cost:Int,per:scala.concurrent.duration.FiniteDuration,maximumBurst:Int,costCalculation:Out=>Int,mode:akka.stream.ThrottleMode):FlowOps.this.Repr[Out]" java="#throttle(int,java.time.Duration,int,akka.japi.function.Function,akka.stream.ThrottleMode)" }
+@apidoc[Flow.throttle](Flow) { scala="#throttle(cost:Int,per:scala.concurrent.duration.FiniteDuration,maximumBurst:Int,costCalculation:Out=>Int,mode:akka.stream.ThrottleMode):FlowOps.this.Repr[Out]" java="#throttle(int,java.time.Duration,int,akka.japi.function.Function,akka.stream.ThrottleMode)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/watch.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/watch.md
index 6004b5d23b..63f06c5851 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/watch.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/watch.md
@@ -2,15 +2,12 @@
Watch a specific `ActorRef` and signal a failure downstream once the actor terminates.
-@ref[Simple operators](../index.md#simple-operators)
-
-@@@div { .group-scala }
+@ref[Actor interop operators](../index.md#actor-interop-operators)
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #watch }
-
-@@@
+@apidoc[Source.watch](Source) { scala="#watch(ref:akka.actor.ActorRef):FlowOps.this.Repr[Out]" java="#watch(akka.actor.ActorRef)" }
+@apidoc[Flow.watch](Flow) { scala="#watch(ref:akka.actor.ActorRef):FlowOps.this.Repr[Out]" java="#watch(akka.actor.ActorRef)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/watchTermination.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/watchTermination.md
index 809aaee03f..ec863081f4 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/watchTermination.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/watchTermination.md
@@ -4,13 +4,11 @@ Materializes to a @scala[`Future`] @java[`CompletionStage`] that will be complet
@ref[Watching status operators](../index.md#watching-status-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #watchTermination }
+@apidoc[Source.watchTermination](Source) { scala="#watchTermination[Mat2]()(matF:(Mat,scala.concurrent.Future[akka.Done])=>Mat2):FlowOpsMat.this.ReprMat[Out,Mat2]" java="#watchTermination(akka.japi.function.Function2)" }
+@apidoc[Flow.watchTermination](Flow) { scala="#watchTermination[Mat2]()(matF:(Mat,scala.concurrent.Future[akka.Done])=>Mat2):FlowOpsMat.this.ReprMat[Out,Mat2]" java="#watchTermination(akka.japi.function.Function2)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/wireTap.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/wireTap.md
index d5acd9d378..9d9b92d094 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/wireTap.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/wireTap.md
@@ -4,13 +4,11 @@ Attaches the given `Sink` to this `Flow` as a wire tap, meaning that elements th
@ref[Fan-out operators](../index.md#fan-out-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #wireTap }
+@apidoc[Source.wireTap](Source) { scala="#wireTap(f:Out=>Unit):FlowOps.this.Repr[Out]" java="#wireTap(akka.japi.function.Procedure)" }
+@apidoc[Flow.wireTap](Flow) { scala="#wireTap(f:Out=>Unit):FlowOps.this.Repr[Out]" java="#wireTap(akka.japi.function.Procedure)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zip.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zip.md
index dd4a64a100..bbb76837ed 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zip.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zip.md
@@ -4,13 +4,10 @@ Combines elements from each of multiple sources into @scala[tuples] @java[*Pair*
@ref[Fan-in operators](../index.md#fan-in-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #zip }
-
-@@@
+@apidoc[Source.zip](Source$) { scala="#zip[U](that:akka.stream.Graph[akka.stream.SourceShape[U],_]):FlowOps.this.Repr[(Out,U)]" java="#zip(akka.stream.Graph)" }
+@apidoc[Flow.zip](Flow) { scala="#zip[U](that:akka.stream.Graph[akka.stream.SourceShape[U],_]):FlowOps.this.Repr[(Out,U)]" java="#zip(akka.stream.Graph)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipAll.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipAll.md
index 9effd50ca4..f78cfdc9a9 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipAll.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipAll.md
@@ -4,13 +4,11 @@ Combines elements from two sources into @scala[tuples] @java[*Pair*] handling ea
@ref[Fan-in operators](../index.md#fan-in-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #zipAll }
+@apidoc[Source.zipAll](Source) { scala="#zipAll[U,A>:Out](that:akka.stream.Graph[akka.stream.SourceShape[U],_],thisElem:A,thatElem:U):FlowOps.this.Repr[(A,U)]" java="#zipAll(akka.stream.Graph,java.lang.Object,java.lang.Object)" }
+@apidoc[Flow.zipAll](Flow) { scala="#zipAll[U,A>:Out](that:akka.stream.Graph[akka.stream.SourceShape[U],_],thisElem:A,thatElem:U):FlowOps.this.Repr[(A,U)]" java="#zipAll(akka.stream.Graph,java.lang.Object,java.lang.Object)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipLatest.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipLatest.md
index a6e374df29..9db446a6d1 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipLatest.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipLatest.md
@@ -4,13 +4,11 @@ Combines elements from each of multiple sources into @scala[tuples] @java[*Pair*
@ref[Fan-in operators](../index.md#fan-in-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #zipLatest }
+@apidoc[Source.zipLatest](Source) { scala="#zipLatest[U](that:akka.stream.Graph[akka.stream.SourceShape[U],_]):FlowOps.this.Repr[(Out,U)]" java="#zipLatest(akka.stream.Graph)" }
+@apidoc[Flow.zipLatest](Flow) { scala="#zipLatest[U](that:akka.stream.Graph[akka.stream.SourceShape[U],_]):FlowOps.this.Repr[(Out,U)]" java="#zipLatest(akka.stream.Graph)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipLatestWith.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipLatestWith.md
index 61b4fc18cc..678c67c812 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipLatestWith.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipLatestWith.md
@@ -4,13 +4,11 @@ Combines elements from multiple sources through a `combine` function and passes
@ref[Fan-in operators](../index.md#fan-in-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #zipLatestWith }
+@apidoc[Source.zipLatestWith](Source) { scala="#zipLatestWith[Out2,Out3](that:akka.stream.Graph[akka.stream.SourceShape[Out2],_])(combine:(Out,Out2)=>Out3):FlowOps.this.Repr[Out3]" java="#zipLatestWith(akka.stream.Graph,akka.japi.function.Function2)" }
+@apidoc[Flow.zipLatestWith](Flow) { scala="#zipLatestWith[Out2,Out3](that:akka.stream.Graph[akka.stream.SourceShape[Out2],_])(combine:(Out,Out2)=>Out3):FlowOps.this.Repr[Out3]" java="#zipLatestWith(akka.stream.Graph,akka.japi.function.Function2)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipWith.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipWith.md
index f04ccd2dad..9995f36383 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipWith.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipWith.md
@@ -4,13 +4,11 @@ Combines elements from multiple sources through a `combine` function and passes
@ref[Fan-in operators](../index.md#fan-in-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #zipWith }
+@apidoc[Source.zipWith](Source) { scala="#zipWith[Out2,Out3](that:akka.stream.Graph[akka.stream.SourceShape[Out2],_])(combine:(Out,Out2)=>Out3):FlowOps.this.Repr[Out3]" java="#zipWith(akka.stream.Graph,akka.japi.function.Function2)" }
+@apidoc[Flow.zipWith](Flow) { scala="#zipWith[Out2,Out3](that:akka.stream.Graph[akka.stream.SourceShape[Out2],_])(combine:(Out,Out2)=>Out3):FlowOps.this.Repr[Out3]" java="#zipWith(akka.stream.Graph,akka.japi.function.Function2)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipWithIndex.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipWithIndex.md
index 6c0e0ff53a..5a6e27dc77 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipWithIndex.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/zipWithIndex.md
@@ -4,13 +4,10 @@ Zips elements of current flow with its indices.
@ref[Fan-in operators](../index.md#fan-in-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #zipWithIndex }
-
-@@@
+@apidoc[Source.zipWithIndex](Source) { scala="#zipWithIndex:FlowOps.this.Repr[(Out,Long)]" java="#zipWithIndex()" }
+@apidoc[Flow.zipWithIndex](Flow) { scala="#zipWithIndex:FlowOps.this.Repr[(Out,Long)]" java="#zipWithIndex()" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/actorRef.md b/akka-docs/src/main/paradox/stream/operators/Source/actorRef.md
index bd2108d63c..b13ce70aaf 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/actorRef.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/actorRef.md
@@ -1,4 +1,4 @@
-# actorRef
+# Source.actorRef
Materialize an `ActorRef`; sending messages to it will emit them on the stream.
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/actorRefWithBackpressure.md b/akka-docs/src/main/paradox/stream/operators/Source/actorRefWithBackpressure.md
index 7256445021..4806a22fe1 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/actorRefWithBackpressure.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/actorRefWithBackpressure.md
@@ -1,14 +1,12 @@
-# actorRefWithBackpressure
+# Source.actorRefWithBackpressure
Materialize an `ActorRef`; sending messages to it will emit them on the stream. The source acknowledges reception after emitting a message, to provide back pressure from the source.
@ref[Source operators](../index.md#source-operators)
-@@@ div { .group-scala }
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #actorRefWithBackpressure }
-@@@
+@apidoc[Source.actorRefWithBackpressure](Source$) { scala="#actorRefWithBackpressure[T](ackMessage:Any,completionMatcher:PartialFunction[Any,akka.stream.CompletionStrategy],failureMatcher:PartialFunction[Any,Throwable]):akka.stream.scaladsl.Source[T,akka.actor.ActorRef]" java="#actorRefWithBackpressure(java.lang.Object,akka.japi.function.Function,akka.japi.function.Function)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/asSourceWithContext.md b/akka-docs/src/main/paradox/stream/operators/Source/asSourceWithContext.md
index 1d18ddf496..9c04c0f902 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/asSourceWithContext.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/asSourceWithContext.md
@@ -4,13 +4,10 @@ Turns a Source into a SourceWithContext which can propagate a context per elemen
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #asSourceWithContext }
+@apidoc[Source.asSourceWithContext](Source) { scala="#asSourceWithContext[Ctx](f:Out=>Ctx):akka.stream.scaladsl.SourceWithContext[Out,Ctx,Mat]" java="#asSourceWithContext(akka.japi.function.Function)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/asSubscriber.md b/akka-docs/src/main/paradox/stream/operators/Source/asSubscriber.md
index bdacf6cc6b..59c803c2cb 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/asSubscriber.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/asSubscriber.md
@@ -1,4 +1,4 @@
-# asSubscriber
+# Source.asSubscriber
Integration with Reactive Streams, materializes into a @javadoc[Subscriber](java.util.concurrent.Flow.Subscriber).
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/combine.md b/akka-docs/src/main/paradox/stream/operators/Source/combine.md
index 5792b0110b..599508f40e 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/combine.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/combine.md
@@ -1,16 +1,13 @@
-# combine
+# Source.combine
Combine several sources, using a given strategy such as merge or concat, into one source.
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #combine }
+@apidoc[Source.combine](Source$) { scala="#combine[T,U](first:akka.stream.scaladsl.Source[T,_],second:akka.stream.scaladsl.Source[T,_],rest:akka.stream.scaladsl.Source[T,_]*)(strategy:Int=>akka.stream.Graph[akka.stream.UniformFanInShape[T,U],akka.NotUsed]):akka.stream.scaladsl.Source[U,akka.NotUsed]" java="#combine(akka.stream.javadsl.Source,akka.stream.javadsl.Source,java.util.List,akka.japi.function.Function)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/completionStage.md b/akka-docs/src/main/paradox/stream/operators/Source/completionStage.md
index f9af46feeb..10d0d65bc5 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/completionStage.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/completionStage.md
@@ -1,16 +1,13 @@
-# completionStage
+# Source.completionStage
Send the single value of the `CompletionStage` when it completes and there is demand.
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-java }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #completionStage }
+@apidoc[Source.completionStage](Source$) { scala="#completionStage[T](completionStage:java.util.concurrent.CompletionStage[T]):akka.stream.scaladsl.Source[T,akka.NotUsed]" java="#completionStage(java.util.concurrent.CompletionStage)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/completionStageSource.md b/akka-docs/src/main/paradox/stream/operators/Source/completionStageSource.md
index a7a9edffac..d68e66b426 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/completionStageSource.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/completionStageSource.md
@@ -1,4 +1,4 @@
-# completionStageSource
+# Source.completionStageSource
Streams the elements of an asynchronous source once its given *completion* operator completes.
@@ -35,4 +35,4 @@ Java
**completes** after the asynchronous source completes
-@@@
\ No newline at end of file
+@@@
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/cycle.md b/akka-docs/src/main/paradox/stream/operators/Source/cycle.md
index db71dd0118..57698ddbcf 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/cycle.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/cycle.md
@@ -1,16 +1,13 @@
-# cycle
+# Source.cycle
Stream iterator in cycled manner.
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #cycle }
+@apidoc[Source.cycle](Source$) { scala="#cycle[T](f:()=>Iterator[T]):akka.stream.scaladsl.Source[T,akka.NotUsed]" java="#cycle(akka.japi.function.Creator)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/empty.md b/akka-docs/src/main/paradox/stream/operators/Source/empty.md
index 6ba6480723..59f3c643cb 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/empty.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/empty.md
@@ -1,16 +1,15 @@
-# empty
+# Source.empty
Complete right away without ever emitting any elements.
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
+@ref:[`Source.never`](never.md) a source which emits nothing and never completes.
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #empty }
+@apidoc[Source.empty](Source$) { scala="#empty[T]:akka.stream.scaladsl.Source[T,akka.NotUsed]" java="#empty()" java="#empty(java.lang.Class)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/failed.md b/akka-docs/src/main/paradox/stream/operators/Source/failed.md
index 6084cf7b5f..a1e8e10419 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/failed.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/failed.md
@@ -1,16 +1,13 @@
-# failed
+# Source.failed
Fail directly with a user specified exception.
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #failed }
+@apidoc[Source.failed](Source$) { scala="#failed[T](cause:Throwable):akka.stream.scaladsl.Source[T,akka.NotUsed]" java="#failed(java.lang.Throwable)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/from.md b/akka-docs/src/main/paradox/stream/operators/Source/from.md
index 1cc3308c30..b546e1b70a 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/from.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/from.md
@@ -1,17 +1,22 @@
-# @scala[apply]@java[from]
+# @scala[Source.apply]@java[Source.from]
Stream the values of an @scala[`immutable.Seq`]@java[`Iterable`].
@ref[Source operators](../index.md#source-operators)
+## Signature
@@@div { .group-scala }
-## Signature
+@apidoc[Source.apply](Source$) { scala="#apply[T](iterable:scala.collection.immutable.Iterable[T]):akka.stream.scaladsl.Source[T,akka.NotUsed]" }
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #apply }
+@@@
-@@@
+@@@div { .group-java }
+
+@apidoc[Source.from](Source$) { java="#from(java.lang.Iterable)" }
+
+@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/fromCompletionStage.md b/akka-docs/src/main/paradox/stream/operators/Source/fromCompletionStage.md
index a56304c887..adb811316f 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/fromCompletionStage.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/fromCompletionStage.md
@@ -1,16 +1,13 @@
-# fromCompletionStage
+# Source.fromCompletionStage
Deprecated by @ref:[`Source.completionStage`](completionStage.md).
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #fromCompletionStage }
+@apidoc[Source.fromCompletionStage](Source$) { scala="#fromCompletionStage[T](future:java.util.concurrent.CompletionStage[T]):akka.stream.scaladsl.Source[T,akka.NotUsed]" java="#fromCompletionStage(java.util.concurrent.CompletionStage)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/fromFuture.md b/akka-docs/src/main/paradox/stream/operators/Source/fromFuture.md
index 99290228c8..76b9da58c6 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/fromFuture.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/fromFuture.md
@@ -1,16 +1,13 @@
-# fromFuture
+# Source.fromFuture
Deprecated by @ref[`Source.future`](future.md).
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #fromFuture }
+@apidoc[Source.fromFuture](Source$) { scala="#fromFuture[T](future:scala.concurrent.Future[T]):akka.stream.scaladsl.Source[T,akka.NotUsed]" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/fromFutureSource.md b/akka-docs/src/main/paradox/stream/operators/Source/fromFutureSource.md
index 5da9d1d291..06a2434df8 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/fromFutureSource.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/fromFutureSource.md
@@ -1,16 +1,13 @@
-# fromFutureSource
+# Source.fromFutureSource
Deprecated by @ref:[`Source.futureSource`](futureSource.md).
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #fromFutureSource }
+@apidoc[Source.fromFutureSource](Source$) { scala="#fromFutureSource[T,M](future:scala.concurrent.Future[akka.stream.Graph[akka.stream.SourceShape[T],M]]):akka.stream.scaladsl.Source[T,scala.concurrent.Future[M]]" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/fromIterator.md b/akka-docs/src/main/paradox/stream/operators/Source/fromIterator.md
index f46fec5a52..a4730669be 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/fromIterator.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/fromIterator.md
@@ -1,16 +1,13 @@
-# fromIterator
+# Source.fromIterator
Stream the values from an `Iterator`, requesting the next value when there is demand.
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #fromIterator }
+@apidoc[Source.fromIterator](Source$) { scala="#fromIterator[T](f:()=>Iterator[T]):akka.stream.scaladsl.Source[T,akka.NotUsed]" java="#fromIterator(akka.japi.function.Creator)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/fromJavaStream.md b/akka-docs/src/main/paradox/stream/operators/Source/fromJavaStream.md
new file mode 100644
index 0000000000..99921335d5
--- /dev/null
+++ b/akka-docs/src/main/paradox/stream/operators/Source/fromJavaStream.md
@@ -0,0 +1,36 @@
+# fromJavaStream
+
+Stream the values from a Java 8 `Stream`, requesting the next value when there is demand.
+
+@ref[Source operators](../index.md#source-operators)
+
+## Signature
+
+@apidoc[StreamConverters.fromJavaStream](StreamConverters$) { scala="#fromJavaStream[T,S<:java.util.stream.BaseStream[T,S]](stream:()=>java.util.stream.BaseStream[T,S]):akka.stream.scaladsl.Source[T,akka.NotUsed]" java="#fromJavaStream(akka.japi.function.Creator)" }
+
+
+## Description
+
+Stream the values from a Java 8 `Stream`, requesting the next value when there is demand. The iterator will be created anew
+for each materialization, which is the reason the @scala[`method`] @java[`factory`] takes a @scala[`function`] @java[`Creator`] rather than an `Stream` directly.
+
+ You can use [[Source.async]] to create asynchronous boundaries between synchronous java stream and the rest of flow.
+## Example
+
+Scala
+: @@snip [From.scala](/akka-docs/src/test/scala/docs/stream/operators/source/From.scala) { #from-javaStream }
+
+Java
+: @@snip [From.java](/akka-docs/src/test/java/jdocs/stream/operators/source/From.java) { #from-javaStream }
+
+
+## Reactive Streams semantics
+
+@@@div { .callout }
+
+**emits** the next value returned from the iterator
+
+**completes** when the iterator reaches its end
+
+@@@
+
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/fromPublisher.md b/akka-docs/src/main/paradox/stream/operators/Source/fromPublisher.md
index f59510c55f..701ae47511 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/fromPublisher.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/fromPublisher.md
@@ -1,4 +1,4 @@
-# fromPublisher
+# Source.fromPublisher
Integration with Reactive Streams, subscribes to a @javadoc[Publisher](java.util.concurrent.Flow.Publisher).
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/fromSourceCompletionStage.md b/akka-docs/src/main/paradox/stream/operators/Source/fromSourceCompletionStage.md
index 636d5ac9cb..c8c5516898 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/fromSourceCompletionStage.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/fromSourceCompletionStage.md
@@ -1,4 +1,4 @@
-# fromSourceCompletionStage
+# Source.fromSourceCompletionStage
Deprecated by @ref:[`Source.completionStageSource`](completionStageSource.md).
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/future.md b/akka-docs/src/main/paradox/stream/operators/Source/future.md
index f39bb5ca37..2e6b9d94fc 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/future.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/future.md
@@ -1,16 +1,12 @@
-# future
+# Source.future
Send the single value of the `Future` when it completes and there is demand.
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #future }
-
-@@@
+@apidoc[Source.future](Source$) { scala="#future[T](futureElement:scala.concurrent.Future[T]):akka.stream.scaladsl.Source[T,akka.NotUsed]" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/futureSource.md b/akka-docs/src/main/paradox/stream/operators/Source/futureSource.md
index 83bdc4abc5..ddf35d324c 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/futureSource.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/futureSource.md
@@ -1,16 +1,13 @@
-# futureSource
+# Source.futureSource
Streams the elements of the given future source once it successfully completes.
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #futureSource }
+@apidoc[Source.futureSource](Source$) { scala="#futureSource[T,M](futureSource:scala.concurrent.Future[akka.stream.scaladsl.Source[T,M]]):akka.stream.scaladsl.Source[T,scala.concurrent.Future[M]]" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/lazily.md b/akka-docs/src/main/paradox/stream/operators/Source/lazily.md
index b5424286f5..8a5bfdedbd 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/lazily.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/lazily.md
@@ -1,16 +1,13 @@
-# lazily
+# Source.lazily
Deprecated by @ref:[`Source.lazySource`](lazySource.md).
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #lazily }
+@apidoc[Source.lazily](Source$) { scala="#lazily[T,M](create:()=>akka.stream.scaladsl.Source[T,M]):akka.stream.scaladsl.Source[T,scala.concurrent.Future[M]]" java="#lazily(akka.japi.function.Creator)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/lazilyAsync.md b/akka-docs/src/main/paradox/stream/operators/Source/lazilyAsync.md
index 7ac25ac62a..c20b8c1ef6 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/lazilyAsync.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/lazilyAsync.md
@@ -1,4 +1,4 @@
-# lazilyAsync
+# Source.lazilyAsync
Deprecated by @ref:[`Source.lazyFutureSource`](lazyFutureSource.md).
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/lazyCompletionStage.md b/akka-docs/src/main/paradox/stream/operators/Source/lazyCompletionStage.md
index 45281e24fe..86240c126e 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/lazyCompletionStage.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/lazyCompletionStage.md
@@ -1,4 +1,4 @@
-# lazyCompletionStage
+# Source.lazyCompletionStage
Defers creation of a future of a single element source until there is demand.
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/lazyCompletionStageSource.md b/akka-docs/src/main/paradox/stream/operators/Source/lazyCompletionStageSource.md
index f364b5d09e..32b1e0c761 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/lazyCompletionStageSource.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/lazyCompletionStageSource.md
@@ -1,4 +1,4 @@
-# lazyCompletionStageSource
+# Source.lazyCompletionStageSource
Defers creation of a future source until there is demand.
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/lazyFuture.md b/akka-docs/src/main/paradox/stream/operators/Source/lazyFuture.md
index 5032a54eeb..39cea31791 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/lazyFuture.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/lazyFuture.md
@@ -1,16 +1,12 @@
-# lazyFuture
+# Source.lazyFuture
Defers creation of a future of a single element source until there is demand.
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #lazyFuture }
-
-@@@
+@apidoc[Source.lazyFuture](Source$) { scala="#lazyFuture[T](create:()=>scala.concurrent.Future[T]):akka.stream.scaladsl.Source[T,akka.NotUsed]" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/lazyFutureSource.md b/akka-docs/src/main/paradox/stream/operators/Source/lazyFutureSource.md
index 0a14ef20cb..f08bac4cd4 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/lazyFutureSource.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/lazyFutureSource.md
@@ -1,16 +1,13 @@
-# lazyFutureSource
+# Source.lazyFutureSource
Defers creation and materialization of a `Source` until there is demand.
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #lazyFutureSource }
+@apidoc[Source.lazyFutureSource](Source$) { scala="#lazyFutureSource[T,M](create:()=>scala.concurrent.Future[akka.stream.scaladsl.Source[T,M]]):akka.stream.scaladsl.Source[T,scala.concurrent.Future[M]]" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/lazySingle.md b/akka-docs/src/main/paradox/stream/operators/Source/lazySingle.md
index 8432842d46..7a7f24c248 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/lazySingle.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/lazySingle.md
@@ -1,16 +1,13 @@
-# lazySingle
+# Source.lazySingle
Defers creation of a single element source until there is demand.
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #lazySingle }
+@apidoc[Source.lazySingle](Source$) { scala="#lazySingle[T](create:()=>T):akka.stream.scaladsl.Source[T,akka.NotUsed]" java="#lazySingle(akka.japi.function.Creator)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/lazySource.md b/akka-docs/src/main/paradox/stream/operators/Source/lazySource.md
index 21c3280b1a..42585bb5a3 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/lazySource.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/lazySource.md
@@ -1,26 +1,61 @@
-# lazySource
+# Source.lazySource
Defers creation and materialization of a `Source` until there is demand.
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #lazySource }
+@apidoc[Source.lazySource](Source$) { scala="#lazySource[T,M](create:()=>akka.stream.scaladsl.Source[T,M]):akka.stream.scaladsl.Source[T,scala.concurrent.Future[M]]" java="#lazySource(akka.japi.function.Creator)" }
-@@@
## Description
Defers creation and materialization of a `Source` until there is demand, then emits the elements from the source
-downstream just like if it had been created up front.
+downstream just like if it had been created up front. If the stream fails or cancels before there is demand the factory will not be invoked.
-See also @ref:[lazyFutureSource](lazyFutureSource.md).
+Note that asynchronous boundaries and many other operators in the stream may do pre-fetching or trigger demand earlier
+than you would expect.
-Note that asynchronous boundaries (and other operators) in the stream may do pre-fetching which counter acts
-the laziness and will trigger the factory immediately.
+The materialized value of the `lazy` is a @scala[`Future`]@java[`CompletionStage`] that is completed with the
+materialized value of the nested source once that is constructed.
+
+See also:
+
+ * @ref:[Source.lazyFutureSource](lazyFutureSource.md) and @ref:[Source.lazyCompletionStageSource](lazyCompletionStageSource.md)
+ * @ref:[Flow.lazyFlow](../Flow/lazyFlow.md)
+ * @ref:[Sink.lazySink](../Sink/lazySink.md)
+
+## Example
+
+In this example you might expect this sample to not construct the expensive source until `.pull` is called. However,
+since `Sink.queue` has a buffer and will ask for that immediately on materialization the expensive source is in created
+quickly after the stream has been materialized:
+
+Scala
+: @@snip [Lazy.scala](/akka-docs/src/test/scala/docs/stream/operators/source/Lazy.scala) { #not-a-good-example }
+
+Java
+: @@snip [Lazy.java](/akka-docs/src/test/java/jdocs/stream/operators/source/Lazy.java) { #not-a-good-example }
+
+Instead the most useful aspect of the operator is that the factory is called once per stream materialization
+which means that it can be used to safely construct a mutable object to use with the actual deferred source.
+
+In this example we make use of that by unfolding a mutable object that works like an iterator with a method to say if
+there are more elements and one that produces the next and moves to the next element.
+
+If the `IteratorLikeThing` was used directly in a `Source.unfold` the same instance would end up being unsafely shared
+across all three materializations of the stream, but wrapping it with `Source.lazy` ensures we create a separate instance
+for each of the started streams:
+
+Scala
+: @@snip [Lazy.scala](/akka-docs/src/test/scala/docs/stream/operators/source/Lazy.scala) { #one-per-materialization }
+
+Java
+: @@snip [Lazy.java](/akka-docs/src/test/java/jdocs/stream/operators/source/Lazy.java) { #one-per-materialization }
+
+Note though that you can often also achieve the same using @ref:[unfoldResource](unfoldResource.md). If you have an actual `Iterator`
+you should prefer @ref:[fromIterator](fromIterator.md).
## Reactive Streams semantics
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/maybe.md b/akka-docs/src/main/paradox/stream/operators/Source/maybe.md
index af8e4e52ed..ea26848388 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/maybe.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/maybe.md
@@ -1,4 +1,4 @@
-# maybe
+# Source.maybe
Create a source that emits once the materialized @scala[`Promise`] @java[`CompletableFuture`] is completed with a value.
@@ -6,11 +6,8 @@ Create a source that emits once the materialized @scala[`Promise`] @java[`Comple
## Signature
-Scala
- : @@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #maybe }
+@apidoc[Source.maybe](Source$) { scala="#maybe[T]:akka.stream.scaladsl.Source[T,scala.concurrent.Promise[Option[T]]]" java="#maybe()" }
-Java
-: @@snip [SourceDocExamples.java](/akka-docs/src/test/java/jdocs/stream/operators/SourceDocExamples.java) { #maybe-signature }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/never.md b/akka-docs/src/main/paradox/stream/operators/Source/never.md
new file mode 100644
index 0000000000..4b07141417
--- /dev/null
+++ b/akka-docs/src/main/paradox/stream/operators/Source/never.md
@@ -0,0 +1,27 @@
+# never
+
+Never emit any elements, never complete and never fail.
+
+@ref[Source operators](../index.md#source-operators)
+
+@ref:[`Source.empty`](empty.md), a source which emits nothing and completes immediately.
+
+## Signature
+
+@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #never }
+
+@@@
+
+## Description
+
+Create a source which never emits any elements, never completes and never failes. Useful for tests.
+
+## Reactive Streams semantics
+
+@@@div { .callout }
+
+**emits** never
+
+**completes** never
+
+@@@
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/queue.md b/akka-docs/src/main/paradox/stream/operators/Source/queue.md
index 9486364f13..7b851c9039 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/queue.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/queue.md
@@ -4,13 +4,10 @@ Materialize a `SourceQueue` onto which elements can be pushed for emitting from
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #queue }
+@apidoc[Source.queue](Source$) { scala="#queue[T](bufferSize:Int,overflowStrategy:akka.stream.OverflowStrategy):akka.stream.scaladsl.Source[T,akka.stream.scaladsl.SourceQueueWithComplete[T]]" java="#queue(int,akka.stream.OverflowStrategy)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/range.md b/akka-docs/src/main/paradox/stream/operators/Source/range.md
index 573f41dd2d..412ecc1f82 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/range.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/range.md
@@ -8,7 +8,7 @@ Emit each integer in a range, with an option to take bigger steps than 1.
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/tick.md b/akka-docs/src/main/paradox/stream/operators/Source/tick.md
index 45d2c17dbf..56f52c03ce 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/tick.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/tick.md
@@ -4,13 +4,10 @@ A periodical repetition of an arbitrary object.
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #tick }
+@apidoc[Source.tick](Source$) { scala="#tick[T](initialDelay:scala.concurrent.duration.FiniteDuration,interval:scala.concurrent.duration.FiniteDuration,tick:T):akka.stream.scaladsl.Source[T,akka.actor.Cancellable]" java="#tick(java.time.Duration,java.time.Duration,java.lang.Object)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/unfold.md b/akka-docs/src/main/paradox/stream/operators/Source/unfold.md
index a2640d1d8c..c0f75a7fbc 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/unfold.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/unfold.md
@@ -6,11 +6,8 @@ Stream the result of a function as long as it returns a @scala[`Some`] @java[non
## Signature
-Scala
-: @@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #unfold }
+@apidoc[Source.unfold](Source$) { scala="#unfold[S,E](s:S)(f:S=>Option[(S,E)]):akka.stream.scaladsl.Source[E,akka.NotUsed]" java="#unfold(java.lang.Object,akka.japi.function.Function)" }
-Java
-: @@snip [SourceUnfoldTest.java](/akka-stream-tests/src/test/java/akka/stream/javadsl/SourceUnfoldTest.java) { #signature }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/unfoldAsync.md b/akka-docs/src/main/paradox/stream/operators/Source/unfoldAsync.md
index 87cc13ee14..631d98932d 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/unfoldAsync.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/unfoldAsync.md
@@ -4,13 +4,10 @@ Just like `unfold` but the fold function returns a @scala[`Future`] @java[`Compl
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #unfoldAsync }
+@apidoc[Source.unfoldAsync](Source$) { scala="#unfoldAsync[S,E](s:S)(f:S=>scala.concurrent.Future[Option[(S,E)]]):akka.stream.scaladsl.Source[E,akka.NotUsed]" java="#unfoldAsync(java.lang.Object,akka.japi.function.Function)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/unfoldResource.md b/akka-docs/src/main/paradox/stream/operators/Source/unfoldResource.md
index b09e7d2561..b56659bc79 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/unfoldResource.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/unfoldResource.md
@@ -4,13 +4,10 @@ Wrap any resource that can be opened, queried for next element (in a blocking wa
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #unfoldResource }
+@apidoc[Source.unfoldResource](Source$) { scala="#unfoldResource[T,S](create:()=>S,read:S=>Option[T],close:S=>Unit):akka.stream.scaladsl.Source[T,akka.NotUsed]" java="#unfoldResource(akka.japi.function.Creator,akka.japi.function.Function,akka.japi.function.Procedure)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/Source/unfoldResourceAsync.md b/akka-docs/src/main/paradox/stream/operators/Source/unfoldResourceAsync.md
index c7afbf94dd..ec31ae624a 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source/unfoldResourceAsync.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source/unfoldResourceAsync.md
@@ -4,13 +4,10 @@ Wrap any resource that can be opened, queried for next element and closed in an
@ref[Source operators](../index.md#source-operators)
-@@@div { .group-scala }
-
## Signature
-@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #unfoldResourceAsync }
+@apidoc[Source.unfoldResourceAsync](Source$) { scala="#unfoldResourceAsync[T,S](create:()=>scala.concurrent.Future[S],read:S=>scala.concurrent.Future[Option[T]],close:S=>scala.concurrent.Future[akka.Done]):akka.stream.scaladsl.Source[T,akka.NotUsed]" java="#unfoldResourceAsync(akka.japi.function.Creator,akka.japi.function.Function,akka.japi.function.Function)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/StreamConverters/asInputStream.md b/akka-docs/src/main/paradox/stream/operators/StreamConverters/asInputStream.md
index db822c673e..7467230b1c 100644
--- a/akka-docs/src/main/paradox/stream/operators/StreamConverters/asInputStream.md
+++ b/akka-docs/src/main/paradox/stream/operators/StreamConverters/asInputStream.md
@@ -4,11 +4,10 @@ Create a sink which materializes into an `InputStream` that can be read to trigg
@ref[Additional Sink and Source converters](../index.md#additional-sink-and-source-converters)
-@@@ div { .group-scala }
## Signature
-@@signature [StreamConverters.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/StreamConverters.scala) { #asInputStream }
-@@@
+@apidoc[StreamConverters.asInputStream](StreamConverters$) { scala="#asInputStream(readTimeout:scala.concurrent.duration.FiniteDuration):akka.stream.scaladsl.Sink[akka.util.ByteString,java.io.InputStream]" java="#asInputStream()" }
+
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/StreamConverters/asOutputStream.md b/akka-docs/src/main/paradox/stream/operators/StreamConverters/asOutputStream.md
index afd7a4ff41..8385da91b4 100644
--- a/akka-docs/src/main/paradox/stream/operators/StreamConverters/asOutputStream.md
+++ b/akka-docs/src/main/paradox/stream/operators/StreamConverters/asOutputStream.md
@@ -4,11 +4,9 @@ Create a source that materializes into an `OutputStream`.
@ref[Additional Sink and Source converters](../index.md#additional-sink-and-source-converters)
-@@@ div { .group-scala }
## Signature
-@@signature [StreamConverters.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/StreamConverters.scala) { #asOutputStream }
-@@@
+@apidoc[StreamConverters.asOutputStream](StreamConverters$) { scala="#asOutputStream(writeTimeout:scala.concurrent.duration.FiniteDuration):akka.stream.scaladsl.Source[akka.util.ByteString,java.io.OutputStream]" java="#asOutputStream(java.time.Duration)" }
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/StreamConverters/javaCollector.md b/akka-docs/src/main/paradox/stream/operators/StreamConverters/javaCollector.md
index 0094584b63..2930c4769c 100644
--- a/akka-docs/src/main/paradox/stream/operators/StreamConverters/javaCollector.md
+++ b/akka-docs/src/main/paradox/stream/operators/StreamConverters/javaCollector.md
@@ -4,13 +4,10 @@ Create a sink which materializes into a @scala[`Future`] @java[`CompletionStage`
@ref[Additional Sink and Source converters](../index.md#additional-sink-and-source-converters)
-@@@div { .group-scala }
-
## Signature
-@@signature [StreamConverters.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/StreamConverters.scala) { #javaCollector }
+@apidoc[StreamConverters.javaCollector](StreamConverters$) { scala="#javaCollector[T,R](collectorFactory:()=>java.util.stream.Collector[T,_,R]):akka.stream.scaladsl.Sink[T,scala.concurrent.Future[R]]" java="#javaCollector(akka.japi.function.Creator)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/StreamConverters/javaCollectorParallelUnordered.md b/akka-docs/src/main/paradox/stream/operators/StreamConverters/javaCollectorParallelUnordered.md
index c8e0bf3b3c..089d64754f 100644
--- a/akka-docs/src/main/paradox/stream/operators/StreamConverters/javaCollectorParallelUnordered.md
+++ b/akka-docs/src/main/paradox/stream/operators/StreamConverters/javaCollectorParallelUnordered.md
@@ -4,13 +4,10 @@ Create a sink which materializes into a @scala[`Future`] @java[`CompletionStage`
@ref[Additional Sink and Source converters](../index.md#additional-sink-and-source-converters)
-@@@div { .group-scala }
-
## Signature
-@@signature [StreamConverters.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/StreamConverters.scala) { #javaCollectorParallelUnordered }
+@apidoc[StreamConverters.javaCollectorParallelUnordered](StreamConverters$) { scala="#javaCollectorParallelUnordered[T,R](parallelism:Int)(collectorFactory:()=>java.util.stream.Collector[T,_,R]):akka.stream.scaladsl.Sink[T,scala.concurrent.Future[R]]" java="#javaCollectorParallelUnordered(int,akka.japi.function.Creator)" }
-@@@
## Description
diff --git a/akka-docs/src/main/paradox/stream/operators/index.md b/akka-docs/src/main/paradox/stream/operators/index.md
index 8fcd2c18e2..649b321b9d 100644
--- a/akka-docs/src/main/paradox/stream/operators/index.md
+++ b/akka-docs/src/main/paradox/stream/operators/index.md
@@ -22,6 +22,7 @@ These built-in sources are available from @scala[`akka.stream.scaladsl.Source`]
|Source|@ref[fromFuture](Source/fromFuture.md)|Deprecated by @ref[`Source.future`](Source/future.md).|
|Source|@ref[fromFutureSource](Source/fromFutureSource.md)|Deprecated by @ref[`Source.futureSource`](Source/futureSource.md).|
|Source|@ref[fromIterator](Source/fromIterator.md)|Stream the values from an `Iterator`, requesting the next value when there is demand.|
+|Source|@ref[fromJavaStream](Source/fromJavaStream.md)|Stream the values from a Java 8 `Stream`, requesting the next value when there is demand.|
|Source|@ref[fromPublisher](Source/fromPublisher.md)|Integration with Reactive Streams, subscribes to a @javadoc[Publisher](java.util.concurrent.Flow.Publisher).|
|Source|@ref[fromSourceCompletionStage](Source/fromSourceCompletionStage.md)|Deprecated by @ref[`Source.completionStageSource`](Source/completionStageSource.md).|
|Source|@ref[future](Source/future.md)|Send the single value of the `Future` when it completes and there is demand.|
@@ -35,6 +36,7 @@ These built-in sources are available from @scala[`akka.stream.scaladsl.Source`]
|Source|@ref[lazySingle](Source/lazySingle.md)|Defers creation of a single element source until there is demand.|
|Source|@ref[lazySource](Source/lazySource.md)|Defers creation and materialization of a `Source` until there is demand.|
|Source|@ref[maybe](Source/maybe.md)|Create a source that emits once the materialized @scala[`Promise`] @java[`CompletableFuture`] is completed with a value.|
+|Source|@ref[never](Source/never.md)|Never emit any elements, never complete and never fail.|
|Source|@ref[queue](Source/queue.md)|Materialize a `SourceQueue` onto which elements can be pushed for emitting from the source. |
|Source|@ref[range](Source/range.md)|Emit each integer in a range, with an option to take bigger steps than 1.|
|Source|@ref[repeat](Source/repeat.md)|Stream a single object repeatedly.|
@@ -180,7 +182,6 @@ depending on being backpressured by downstream or not.
|Source/Flow|@ref[take](Source-or-Flow/take.md)|Pass `n` incoming elements downstream and then complete|
|Source/Flow|@ref[takeWhile](Source-or-Flow/takeWhile.md)|Pass elements downstream as long as a predicate function returns true and then complete. |
|Source/Flow|@ref[throttle](Source-or-Flow/throttle.md)|Limit the throughput to a specific number of elements per time unit, or a specific total cost per time unit, where a function has to be provided to calculate the individual cost of each element.|
-|Source/Flow|@ref[watch](Source-or-Flow/watch.md)|Watch a specific `ActorRef` and signal a failure downstream once the actor terminates.|
## Flow operators composed of Sinks and Sources
@@ -198,7 +199,6 @@ operation at the same time (usually handling the completion of a @scala[`Future`
| |Operator|Description|
|--|--|--|
-|Source/Flow|@ref[ask](Source-or-Flow/ask.md)|Use the `ask` pattern to send a request-reply message to the target `ref` actor.|
|Source/Flow|@ref[mapAsync](Source-or-Flow/mapAsync.md)|Pass incoming elements to a function that return a @scala[`Future`] @java[`CompletionStage`] result.|
|Source/Flow|@ref[mapAsyncUnordered](Source-or-Flow/mapAsyncUnordered.md)|Like `mapAsync` but @scala[`Future`] @java[`CompletionStage`] results are passed downstream as they arrive regardless of the order of the elements that triggered them.|
@@ -322,7 +322,9 @@ Operators meant for inter-operating between Akka Streams and Actors:
|ActorSink|@ref[actorRef](ActorSink/actorRef.md)|Sends the elements of the stream to the given @java[`ActorRef`]@scala[`ActorRef[T]`], without considering backpressure.|
|ActorSource|@ref[actorRefWithBackpressure](ActorSource/actorRefWithBackpressure.md)|Materialize an @java[`ActorRef`]@scala[`ActorRef[T]`]; sending messages to it will emit them on the stream. The source acknowledges reception after emitting a message, to provide back pressure from the source.|
|ActorSink|@ref[actorRefWithBackpressure](ActorSink/actorRefWithBackpressure.md)|Sends the elements of the stream to the given @java[`ActorRef`]@scala[`ActorRef[T]`] with backpressure, to be able to signal demand when the actor is ready to receive more elements.|
-|ActorFlow|@ref[ask](ActorFlow/ask.md)|Use the `AskPattern` to send each element as an `ask` to the target actor, and expect a reply back that will be sent further downstream.|
+|Source/Flow|@ref[ask](Source-or-Flow/ask.md)|Use the "Ask Pattern" to send a request-reply message to the target `ref` actor (of the classic actors API).|
+|ActorFlow|@ref[ask](ActorFlow/ask.md)|Use the "Ask Pattern" to send each stream element as an `ask` to the target actor (of the new actors API), and expect a reply back that will be emitted downstream.|
+|Source/Flow|@ref[watch](Source-or-Flow/watch.md)|Watch a specific `ActorRef` and signal a failure downstream once the actor terminates.|
## Compression operators
@@ -349,195 +351,197 @@ For more background see the @ref[Error Handling in Streams](../stream-error.md)
@@@ index
-* [combine](Source/combine.md)
+* [actorRef](Source/actorRef.md)
+* [actorRef](Sink/actorRef.md)
+* [actorRef](ActorSource/actorRef.md)
+* [actorRef](ActorSink/actorRef.md)
+* [actorRefWithBackpressure](Source/actorRefWithBackpressure.md)
+* [actorRefWithBackpressure](Sink/actorRefWithBackpressure.md)
+* [actorRefWithBackpressure](ActorSource/actorRefWithBackpressure.md)
+* [actorRefWithBackpressure](ActorSink/actorRefWithBackpressure.md)
+* [alsoTo](Source-or-Flow/alsoTo.md)
+* [asFlowWithContext](Flow/asFlowWithContext.md)
+* [asInputStream](StreamConverters/asInputStream.md)
+* [asJavaStream](StreamConverters/asJavaStream.md)
+* [ask](Source-or-Flow/ask.md)
+* [ask](ActorFlow/ask.md)
+* [asOutputStream](StreamConverters/asOutputStream.md)
+* [asPublisher](Sink/asPublisher.md)
* [asSourceWithContext](Source/asSourceWithContext.md)
-* [fromPublisher](Source/fromPublisher.md)
-* [fromIterator](Source/fromIterator.md)
+* [asSubscriber](Source/asSubscriber.md)
+* [backpressureTimeout](Source-or-Flow/backpressureTimeout.md)
+* [Balance](Balance.md)
+* [batch](Source-or-Flow/batch.md)
+* [batchWeighted](Source-or-Flow/batchWeighted.md)
+* [Broadcast](Broadcast.md)
+* [buffer](Source-or-Flow/buffer.md)
+* [cancelled](Sink/cancelled.md)
+* [collect](Source-or-Flow/collect.md)
+* [collection](Sink/collection.md)
+* [collectType](Source-or-Flow/collectType.md)
+* [combine](Source/combine.md)
+* [combine](Sink/combine.md)
+* [completionStage](Source/completionStage.md)
+* [completionStageFlow](Flow/completionStageFlow.md)
+* [completionStageSink](Sink/completionStageSink.md)
+* [completionStageSource](Source/completionStageSource.md)
+* [completionTimeout](Source-or-Flow/completionTimeout.md)
+* [concat](Source-or-Flow/concat.md)
+* [conflate](Source-or-Flow/conflate.md)
+* [conflateWithSeed](Source-or-Flow/conflateWithSeed.md)
* [cycle](Source/cycle.md)
-* [fromMaterializer](Source-or-Flow/fromMaterializer.md)
-* [setup](Source-or-Flow/setup.md)
-* [fromFuture](Source/fromFuture.md)
-* [fromCompletionStage](Source/fromCompletionStage.md)
-* [fromFutureSource](Source/fromFutureSource.md)
-* [fromSourceCompletionStage](Source/fromSourceCompletionStage.md)
-* [tick](Source/tick.md)
-* [single](Source/single.md)
-* [repeat](Source/repeat.md)
-* [unfold](Source/unfold.md)
-* [unfoldAsync](Source/unfoldAsync.md)
+* [deflate](Compression/deflate.md)
+* [delay](Source-or-Flow/delay.md)
+* [delayWith](Source-or-Flow/delayWith.md)
+* [detach](Source-or-Flow/detach.md)
+* [divertTo](Source-or-Flow/divertTo.md)
+* [drop](Source-or-Flow/drop.md)
+* [dropWhile](Source-or-Flow/dropWhile.md)
+* [dropWithin](Source-or-Flow/dropWithin.md)
* [empty](Source/empty.md)
-* [maybe](Source/maybe.md)
+* [expand](Source-or-Flow/expand.md)
+* [extrapolate](Source-or-Flow/extrapolate.md)
* [failed](Source/failed.md)
+* [filter](Source-or-Flow/filter.md)
+* [filterNot](Source-or-Flow/filterNot.md)
+* [flatMapConcat](Source-or-Flow/flatMapConcat.md)
+* [flatMapMerge](Source-or-Flow/flatMapMerge.md)
+* [flatMapPrefix](Source-or-Flow/flatMapPrefix.md)
+* [fold](Source-or-Flow/fold.md)
+* [fold](Sink/fold.md)
+* [foldAsync](Source-or-Flow/foldAsync.md)
+* [foreach](Sink/foreach.md)
+* [foreachAsync](Sink/foreachAsync.md)
+* [foreachParallel](Sink/foreachParallel.md)
+* [from](Source/from.md)
+* [fromCompletionStage](Source/fromCompletionStage.md)
+* [fromFile](FileIO/fromFile.md)
+* [fromFuture](Source/fromFuture.md)
+* [fromFutureSource](Source/fromFutureSource.md)
+* [fromInputStream](StreamConverters/fromInputStream.md)
+* [fromIterator](Source/fromIterator.md)
+* [fromJavaStream](Source/fromJavaStream.md)
+* [fromJavaStream](StreamConverters/fromJavaStream.md)
+* [fromMaterializer](Source-or-Flow/fromMaterializer.md)
+* [fromMaterializer](Sink/fromMaterializer.md)
+* [fromOutputStream](StreamConverters/fromOutputStream.md)
+* [fromPath](FileIO/fromPath.md)
+* [fromPublisher](Source/fromPublisher.md)
+* [fromSinkAndSource](Flow/fromSinkAndSource.md)
+* [fromSinkAndSourceCoupled](Flow/fromSinkAndSourceCoupled.md)
+* [fromSourceCompletionStage](Source/fromSourceCompletionStage.md)
+* [fromSubscriber](Sink/fromSubscriber.md)
+* [future](Source/future.md)
+* [futureFlow](Flow/futureFlow.md)
+* [futureSink](Sink/futureSink.md)
+* [futureSource](Source/futureSource.md)
+* [groupBy](Source-or-Flow/groupBy.md)
+* [grouped](Source-or-Flow/grouped.md)
+* [groupedWeightedWithin](Source-or-Flow/groupedWeightedWithin.md)
+* [groupedWithin](Source-or-Flow/groupedWithin.md)
+* [gzip](Compression/gzip.md)
+* [head](Sink/head.md)
+* [headOption](Sink/headOption.md)
+* [idleTimeout](Source-or-Flow/idleTimeout.md)
+* [ignore](Sink/ignore.md)
+* [initialDelay](Source-or-Flow/initialDelay.md)
+* [initialTimeout](Source-or-Flow/initialTimeout.md)
+* [interleave](Source-or-Flow/interleave.md)
+* [intersperse](Source-or-Flow/intersperse.md)
+* [javaCollector](StreamConverters/javaCollector.md)
+* [javaCollectorParallelUnordered](StreamConverters/javaCollectorParallelUnordered.md)
+* [keepAlive](Source-or-Flow/keepAlive.md)
+* [last](Sink/last.md)
+* [lastOption](Sink/lastOption.md)
* [lazily](Source/lazily.md)
* [lazilyAsync](Source/lazilyAsync.md)
-* [future](Source/future.md)
-* [completionStage](Source/completionStage.md)
-* [futureSource](Source/futureSource.md)
-* [lazySingle](Source/lazySingle.md)
-* [lazyFuture](Source/lazyFuture.md)
-* [lazySource](Source/lazySource.md)
-* [lazyFutureSource](Source/lazyFutureSource.md)
-* [asSubscriber](Source/asSubscriber.md)
-* [actorRef](Source/actorRef.md)
-* [actorRefWithBackpressure](Source/actorRefWithBackpressure.md)
-* [zipN](Source/zipN.md)
-* [zipWithN](Source/zipWithN.md)
-* [queue](Source/queue.md)
-* [unfoldResource](Source/unfoldResource.md)
-* [unfoldResourceAsync](Source/unfoldResourceAsync.md)
-* [@scala[apply]@java[from]](Source/from.md)
-* [range](Source/range.md)
-* [completionStageSource](Source/completionStageSource.md)
* [lazyCompletionStage](Source/lazyCompletionStage.md)
+* [lazyCompletionStageFlow](Flow/lazyCompletionStageFlow.md)
+* [lazyCompletionStageSink](Sink/lazyCompletionStageSink.md)
* [lazyCompletionStageSource](Source/lazyCompletionStageSource.md)
-* [concat](Source-or-Flow/concat.md)
-* [prepend](Source-or-Flow/prepend.md)
-* [orElse](Source-or-Flow/orElse.md)
-* [alsoTo](Source-or-Flow/alsoTo.md)
-* [divertTo](Source-or-Flow/divertTo.md)
-* [wireTap](Source-or-Flow/wireTap.md)
-* [interleave](Source-or-Flow/interleave.md)
+* [lazyFlow](Flow/lazyFlow.md)
+* [lazyFuture](Source/lazyFuture.md)
+* [lazyFutureFlow](Flow/lazyFutureFlow.md)
+* [lazyFutureSink](Sink/lazyFutureSink.md)
+* [lazyFutureSource](Source/lazyFutureSource.md)
+* [lazyInitAsync](Flow/lazyInitAsync.md)
+* [lazyInitAsync](Sink/lazyInitAsync.md)
+* [lazySingle](Source/lazySingle.md)
+* [lazySink](Sink/lazySink.md)
+* [lazySource](Source/lazySource.md)
+* [limit](Source-or-Flow/limit.md)
+* [limitWeighted](Source-or-Flow/limitWeighted.md)
+* [log](Source-or-Flow/log.md)
+* [logWithMarker](Source-or-Flow/logWithMarker.md)
+* [map](Source-or-Flow/map.md)
+* [mapAsync](Source-or-Flow/mapAsync.md)
+* [mapAsyncUnordered](Source-or-Flow/mapAsyncUnordered.md)
+* [mapConcat](Source-or-Flow/mapConcat.md)
+* [mapError](Source-or-Flow/mapError.md)
+* [maybe](Source/maybe.md)
* [merge](Source-or-Flow/merge.md)
* [mergeLatest](Source-or-Flow/mergeLatest.md)
* [mergePreferred](Source-or-Flow/mergePreferred.md)
* [mergePrioritized](Source-or-Flow/mergePrioritized.md)
* [mergeSorted](Source-or-Flow/mergeSorted.md)
-* [zip](Source-or-Flow/zip.md)
-* [zipAll](Source-or-Flow/zipAll.md)
-* [zipLatest](Source-or-Flow/zipLatest.md)
-* [zipWith](Source-or-Flow/zipWith.md)
-* [zipLatestWith](Source-or-Flow/zipLatestWith.md)
-* [zipWithIndex](Source-or-Flow/zipWithIndex.md)
-* [map](Source-or-Flow/map.md)
+* [monitor](Source-or-Flow/monitor.md)
+* [never](Source/never.md)
+* [onComplete](Sink/onComplete.md)
+* [onFailuresWithBackoff](RestartSource/onFailuresWithBackoff.md)
+* [onFailuresWithBackoff](RestartFlow/onFailuresWithBackoff.md)
+* [orElse](Source-or-Flow/orElse.md)
+* [Partition](Partition.md)
+* [prefixAndTail](Source-or-Flow/prefixAndTail.md)
+* [preMaterialize](Sink/preMaterialize.md)
+* [prepend](Source-or-Flow/prepend.md)
+* [queue](Source/queue.md)
+* [queue](Sink/queue.md)
+* [range](Source/range.md)
* [recover](Source-or-Flow/recover.md)
-* [mapError](Source-or-Flow/mapError.md)
* [recoverWith](Source-or-Flow/recoverWith.md)
* [recoverWithRetries](Source-or-Flow/recoverWithRetries.md)
-* [mapConcat](Source-or-Flow/mapConcat.md)
-* [statefulMapConcat](Source-or-Flow/statefulMapConcat.md)
-* [mapAsync](Source-or-Flow/mapAsync.md)
-* [mapAsyncUnordered](Source-or-Flow/mapAsyncUnordered.md)
-* [ask](Source-or-Flow/ask.md)
-* [watch](Source-or-Flow/watch.md)
-* [filter](Source-or-Flow/filter.md)
-* [filterNot](Source-or-Flow/filterNot.md)
-* [collect](Source-or-Flow/collect.md)
-* [collectType](Source-or-Flow/collectType.md)
-* [grouped](Source-or-Flow/grouped.md)
-* [limit](Source-or-Flow/limit.md)
-* [limitWeighted](Source-or-Flow/limitWeighted.md)
-* [sliding](Source-or-Flow/sliding.md)
+* [reduce](Source-or-Flow/reduce.md)
+* [reduce](Sink/reduce.md)
+* [repeat](Source/repeat.md)
* [scan](Source-or-Flow/scan.md)
* [scanAsync](Source-or-Flow/scanAsync.md)
-* [fold](Source-or-Flow/fold.md)
-* [foldAsync](Source-or-Flow/foldAsync.md)
-* [reduce](Source-or-Flow/reduce.md)
-* [intersperse](Source-or-Flow/intersperse.md)
-* [groupedWithin](Source-or-Flow/groupedWithin.md)
-* [groupedWeightedWithin](Source-or-Flow/groupedWeightedWithin.md)
-* [delay](Source-or-Flow/delay.md)
-* [delayWith](Source-or-Flow/delayWith.md)
-* [drop](Source-or-Flow/drop.md)
-* [dropWithin](Source-or-Flow/dropWithin.md)
-* [takeWhile](Source-or-Flow/takeWhile.md)
-* [dropWhile](Source-or-Flow/dropWhile.md)
-* [take](Source-or-Flow/take.md)
-* [takeWithin](Source-or-Flow/takeWithin.md)
-* [conflateWithSeed](Source-or-Flow/conflateWithSeed.md)
-* [conflate](Source-or-Flow/conflate.md)
-* [batch](Source-or-Flow/batch.md)
-* [batchWeighted](Source-or-Flow/batchWeighted.md)
-* [expand](Source-or-Flow/expand.md)
-* [extrapolate](Source-or-Flow/extrapolate.md)
-* [buffer](Source-or-Flow/buffer.md)
-* [prefixAndTail](Source-or-Flow/prefixAndTail.md)
-* [flatMapPrefix](Source-or-Flow/flatMapPrefix.md)
-* [groupBy](Source-or-Flow/groupBy.md)
-* [splitWhen](Source-or-Flow/splitWhen.md)
-* [splitAfter](Source-or-Flow/splitAfter.md)
-* [flatMapConcat](Source-or-Flow/flatMapConcat.md)
-* [flatMapMerge](Source-or-Flow/flatMapMerge.md)
-* [initialTimeout](Source-or-Flow/initialTimeout.md)
-* [completionTimeout](Source-or-Flow/completionTimeout.md)
-* [idleTimeout](Source-or-Flow/idleTimeout.md)
-* [backpressureTimeout](Source-or-Flow/backpressureTimeout.md)
-* [keepAlive](Source-or-Flow/keepAlive.md)
-* [throttle](Source-or-Flow/throttle.md)
-* [detach](Source-or-Flow/detach.md)
-* [watchTermination](Source-or-Flow/watchTermination.md)
-* [monitor](Source-or-Flow/monitor.md)
-* [initialDelay](Source-or-Flow/initialDelay.md)
-* [log](Source-or-Flow/log.md)
-* [logWithMarker](Source-or-Flow/logWithMarker.md)
-* [asFlowWithContext](Flow/asFlowWithContext.md)
-* [fromSinkAndSource](Flow/fromSinkAndSource.md)
-* [fromSinkAndSourceCoupled](Flow/fromSinkAndSourceCoupled.md)
-* [lazyInitAsync](Flow/lazyInitAsync.md)
-* [futureFlow](Flow/futureFlow.md)
-* [lazyFlow](Flow/lazyFlow.md)
-* [lazyFutureFlow](Flow/lazyFutureFlow.md)
-* [completionStageFlow](Flow/completionStageFlow.md)
-* [lazyCompletionStageFlow](Flow/lazyCompletionStageFlow.md)
-* [preMaterialize](Sink/preMaterialize.md)
-* [fromMaterializer](Sink/fromMaterializer.md)
-* [setup](Sink/setup.md)
-* [fromSubscriber](Sink/fromSubscriber.md)
-* [cancelled](Sink/cancelled.md)
-* [head](Sink/head.md)
-* [headOption](Sink/headOption.md)
-* [last](Sink/last.md)
-* [lastOption](Sink/lastOption.md)
-* [takeLast](Sink/takeLast.md)
* [seq](Sink/seq.md)
-* [collection](Sink/collection.md)
-* [asPublisher](Sink/asPublisher.md)
-* [ignore](Sink/ignore.md)
-* [foreach](Sink/foreach.md)
-* [foreachAsync](Sink/foreachAsync.md)
-* [combine](Sink/combine.md)
-* [foreachParallel](Sink/foreachParallel.md)
-* [fold](Sink/fold.md)
-* [reduce](Sink/reduce.md)
-* [onComplete](Sink/onComplete.md)
-* [actorRef](Sink/actorRef.md)
-* [actorRefWithBackpressure](Sink/actorRefWithBackpressure.md)
-* [queue](Sink/queue.md)
-* [lazyInitAsync](Sink/lazyInitAsync.md)
-* [futureSink](Sink/futureSink.md)
-* [lazySink](Sink/lazySink.md)
-* [lazyFutureSink](Sink/lazyFutureSink.md)
-* [completionStageSink](Sink/completionStageSink.md)
-* [lazyCompletionStageSink](Sink/lazyCompletionStageSink.md)
-* [fromInputStream](StreamConverters/fromInputStream.md)
-* [asOutputStream](StreamConverters/asOutputStream.md)
-* [fromOutputStream](StreamConverters/fromOutputStream.md)
-* [asInputStream](StreamConverters/asInputStream.md)
-* [javaCollector](StreamConverters/javaCollector.md)
-* [javaCollectorParallelUnordered](StreamConverters/javaCollectorParallelUnordered.md)
-* [asJavaStream](StreamConverters/asJavaStream.md)
-* [fromJavaStream](StreamConverters/fromJavaStream.md)
-* [fromFile](FileIO/fromFile.md)
-* [fromPath](FileIO/fromPath.md)
+* [setup](Source-or-Flow/setup.md)
+* [setup](Sink/setup.md)
+* [single](Source/single.md)
+* [sliding](Source-or-Flow/sliding.md)
+* [splitAfter](Source-or-Flow/splitAfter.md)
+* [splitWhen](Source-or-Flow/splitWhen.md)
+* [statefulMapConcat](Source-or-Flow/statefulMapConcat.md)
+* [take](Source-or-Flow/take.md)
+* [takeLast](Sink/takeLast.md)
+* [takeWhile](Source-or-Flow/takeWhile.md)
+* [takeWithin](Source-or-Flow/takeWithin.md)
+* [throttle](Source-or-Flow/throttle.md)
+* [tick](Source/tick.md)
* [toFile](FileIO/toFile.md)
* [toPath](FileIO/toPath.md)
+* [unfold](Source/unfold.md)
+* [unfoldAsync](Source/unfoldAsync.md)
+* [unfoldResource](Source/unfoldResource.md)
+* [unfoldResourceAsync](Source/unfoldResourceAsync.md)
+* [Unzip](Unzip.md)
+* [UnzipWith](UnzipWith.md)
+* [watch](Source-or-Flow/watch.md)
+* [watchTermination](Source-or-Flow/watchTermination.md)
+* [wireTap](Source-or-Flow/wireTap.md)
* [withBackoff](RestartSource/withBackoff.md)
-* [onFailuresWithBackoff](RestartSource/onFailuresWithBackoff.md)
* [withBackoff](RestartFlow/withBackoff.md)
-* [onFailuresWithBackoff](RestartFlow/onFailuresWithBackoff.md)
* [withBackoff](RestartSink/withBackoff.md)
* [withBackoff](RetryFlow/withBackoff.md)
* [withBackoffAndContext](RetryFlow/withBackoffAndContext.md)
-* [gzip](Compression/gzip.md)
-* [deflate](Compression/deflate.md)
-* [actorRef](ActorSource/actorRef.md)
-* [actorRefWithBackpressure](ActorSource/actorRefWithBackpressure.md)
-* [ask](ActorFlow/ask.md)
-* [actorRef](ActorSink/actorRef.md)
-* [actorRefWithBackpressure](ActorSink/actorRefWithBackpressure.md)
-* [Partition](Partition.md)
-* [Broadcast](Broadcast.md)
-* [Balance](Balance.md)
-* [Unzip](Unzip.md)
-* [UnzipWith](UnzipWith.md)
+* [zip](Source-or-Flow/zip.md)
+* [zipAll](Source-or-Flow/zipAll.md)
+* [zipLatest](Source-or-Flow/zipLatest.md)
+* [zipLatestWith](Source-or-Flow/zipLatestWith.md)
+* [zipN](Source/zipN.md)
+* [zipWith](Source-or-Flow/zipWith.md)
+* [zipWithIndex](Source-or-Flow/zipWithIndex.md)
+* [zipWithN](Source/zipWithN.md)
@@@
diff --git a/akka-docs/src/main/paradox/stream/reactive-streams-interop.md b/akka-docs/src/main/paradox/stream/reactive-streams-interop.md
index 55ae578240..0afebd4ac6 100644
--- a/akka-docs/src/main/paradox/stream/reactive-streams-interop.md
+++ b/akka-docs/src/main/paradox/stream/reactive-streams-interop.md
@@ -6,14 +6,14 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
## Overview
-Akka Streams implements the [Reactive Streams](http://reactive-streams.org/) standard for asynchronous stream processing with non-blocking
+Akka Streams implements the [Reactive Streams](https://www.reactive-streams.org/) standard for asynchronous stream processing with non-blocking
back pressure.
Since Java 9 the APIs of Reactive Streams has been included in the Java Standard library, under the `java.util.concurrent.Flow`
@@ -133,5 +133,5 @@ An incomplete list of other implementations:
* [Reactor (1.1+)](https://github.com/reactor/reactor)
* [RxJava](https://github.com/ReactiveX/RxJavaReactiveStreams)
- * [Ratpack](http://www.ratpack.io/manual/current/streams.html)
- * [Slick](http://slick.lightbend.com)
+ * [Ratpack](https://www.ratpack.io/manual/current/streams.html)
+ * [Slick](https://scala-slick.org/)
diff --git a/akka-docs/src/main/paradox/stream/stream-composition.md b/akka-docs/src/main/paradox/stream/stream-composition.md
index daba8c933d..07148745db 100644
--- a/akka-docs/src/main/paradox/stream/stream-composition.md
+++ b/akka-docs/src/main/paradox/stream/stream-composition.md
@@ -6,7 +6,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/stream/stream-cookbook.md b/akka-docs/src/main/paradox/stream/stream-cookbook.md
index 39c2a53973..e1f9693968 100644
--- a/akka-docs/src/main/paradox/stream/stream-cookbook.md
+++ b/akka-docs/src/main/paradox/stream/stream-cookbook.md
@@ -6,7 +6,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/stream/stream-customize.md b/akka-docs/src/main/paradox/stream/stream-customize.md
index 9381c4571c..c06caec49b 100644
--- a/akka-docs/src/main/paradox/stream/stream-customize.md
+++ b/akka-docs/src/main/paradox/stream/stream-customize.md
@@ -6,7 +6,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
@@ -35,7 +35,7 @@ smaller ones, and allows state to be maintained inside it in a safe way.
As a first motivating example, we will build a new `Source` that will emit numbers from 1 until it is
cancelled. To start, we need to define the "interface" of our operator, which is called *shape* in Akka Streams terminology
-(this is explained in more detail in the section @ref:[Modularity, Composition and Hierarchy](stream-composition.md)). This is how this looks like:
+(this is explained in more detail in the section @ref:[Modularity, Composition and Hierarchy](stream-composition.md)). This is how it looks:
Scala
: @@snip [GraphStageDocSpec.scala](/akka-docs/src/test/scala/docs/stream/GraphStageDocSpec.scala) { #boilerplate-example }
@@ -516,7 +516,7 @@ allow nicer syntax. The short answer is that Scala 2 does not support this in a
that it is impossible to abstract over the kind of stream that is being extended because `Source`, `Flow`
and `SubFlow` differ in the number and kind of their type parameters. While it would be possible to write
an implicit class that enriches them generically, this class would require explicit instantiation with all type
-parameters due to [SI-2712](https://issues.scala-lang.org/browse/SI-2712). For a partial workaround that unifies
+parameters due to [SI-2712](https://github.com/scala/bug/issues/2712). For a partial workaround that unifies
extensions to `Source` and `Flow` see [this sketch by R. Kuhn](https://gist.github.com/rkuhn/2870fcee4937dda2cad5).
A lot simpler is the task of adding an extension method to `Source` as shown below:
diff --git a/akka-docs/src/main/paradox/stream/stream-dynamic.md b/akka-docs/src/main/paradox/stream/stream-dynamic.md
index b1aa256251..3d0693554d 100644
--- a/akka-docs/src/main/paradox/stream/stream-dynamic.md
+++ b/akka-docs/src/main/paradox/stream/stream-dynamic.md
@@ -6,7 +6,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/stream/stream-error.md b/akka-docs/src/main/paradox/stream/stream-error.md
index 589379b5ff..dc250b2984 100644
--- a/akka-docs/src/main/paradox/stream/stream-error.md
+++ b/akka-docs/src/main/paradox/stream/stream-error.md
@@ -6,7 +6,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/stream/stream-flows-and-basics.md b/akka-docs/src/main/paradox/stream/stream-flows-and-basics.md
index 6a390d3599..b8c50ecbfb 100644
--- a/akka-docs/src/main/paradox/stream/stream-flows-and-basics.md
+++ b/akka-docs/src/main/paradox/stream/stream-flows-and-basics.md
@@ -6,7 +6,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
@@ -77,7 +77,7 @@ RunnableGraph
It is possible to attach a `Flow` to a `Source` resulting in a composite source, and it is also possible to prepend
-a `Flow` to a `Sink` to get a new sink. After a stream is properly terminated by having both a source and a sink,
+a `Flow` to a `Sink` to get a new sink. After a stream is properly constructed by having both a source and a sink,
it will be represented by the `RunnableGraph` type, indicating that it is ready to be executed.
It is important to remember that even after constructing the `RunnableGraph` by connecting all the source, sink and
@@ -190,7 +190,7 @@ of absence of a value we recommend using @scala[`scala.Option` or `scala.util.Ei
## Back-pressure explained
-Akka Streams implement an asynchronous non-blocking back-pressure protocol standardised by the [Reactive Streams](http://reactive-streams.org/)
+Akka Streams implement an asynchronous non-blocking back-pressure protocol standardised by the [Reactive Streams](https://www.reactive-streams.org/)
specification, which Akka is a founding member of.
The user of the library does not have to write any explicit back-pressure handling code — it is built in
diff --git a/akka-docs/src/main/paradox/stream/stream-graphs.md b/akka-docs/src/main/paradox/stream/stream-graphs.md
index e8d7497459..979e7da9bf 100644
--- a/akka-docs/src/main/paradox/stream/stream-graphs.md
+++ b/akka-docs/src/main/paradox/stream/stream-graphs.md
@@ -6,7 +6,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/stream/stream-introduction.md b/akka-docs/src/main/paradox/stream/stream-introduction.md
index 58d94ad6a9..9e4ea40388 100644
--- a/akka-docs/src/main/paradox/stream/stream-introduction.md
+++ b/akka-docs/src/main/paradox/stream/stream-introduction.md
@@ -19,9 +19,7 @@ measures in order to achieve stable streaming between actors, since in addition
to sending and receiving we also need to take care to not overflow any buffers
or mailboxes in the process. Another pitfall is that Actor messages can be lost
and must be retransmitted in that case. Failure to do so would lead to holes at
-the receiving side. When dealing with streams of elements of a fixed given type,
-Actors also do not currently offer good static guarantees that no wiring errors
-are made: type-safety could be improved in this case.
+the receiving side.
For these reasons we decided to bundle up a solution to these problems as an
Akka Streams API. The purpose is to offer an intuitive and safe way to
@@ -30,7 +28,7 @@ efficiently and with bounded resource usage—no more OutOfMemoryErrors. In orde
to achieve this our streams need to be able to limit the buffering that they
employ, they need to be able to slow down producers if the consumers cannot
keep up. This feature is called back-pressure and is at the core of the
-[Reactive Streams](http://reactive-streams.org/) initiative of which Akka is a
+[Reactive Streams](https://www.reactive-streams.org/) initiative of which Akka is a
founding member. For you this means that the hard problem of propagating and
reacting to back-pressure has been incorporated in the design of Akka Streams
already, so you have one less thing to worry about; it also means that Akka
diff --git a/akka-docs/src/main/paradox/stream/stream-io.md b/akka-docs/src/main/paradox/stream/stream-io.md
index 8032611783..88cc3df83f 100644
--- a/akka-docs/src/main/paradox/stream/stream-io.md
+++ b/akka-docs/src/main/paradox/stream/stream-io.md
@@ -6,7 +6,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/stream/stream-parallelism.md b/akka-docs/src/main/paradox/stream/stream-parallelism.md
index 5b44054e72..2f6e78de06 100644
--- a/akka-docs/src/main/paradox/stream/stream-parallelism.md
+++ b/akka-docs/src/main/paradox/stream/stream-parallelism.md
@@ -6,7 +6,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/stream/stream-quickstart.md b/akka-docs/src/main/paradox/stream/stream-quickstart.md
index bd8d400769..f06453d3f2 100644
--- a/akka-docs/src/main/paradox/stream/stream-quickstart.md
+++ b/akka-docs/src/main/paradox/stream/stream-quickstart.md
@@ -6,7 +6,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/stream/stream-rate.md b/akka-docs/src/main/paradox/stream/stream-rate.md
index fb4e405066..67846c17d0 100644
--- a/akka-docs/src/main/paradox/stream/stream-rate.md
+++ b/akka-docs/src/main/paradox/stream/stream-rate.md
@@ -6,7 +6,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
@@ -234,7 +234,7 @@ Scala
Java
: @@snip [RateTransformationDocTest.java](/akka-docs/src/test/java/jdocs/stream/RateTransformationDocTest.java) { #extrapolate-seed }
-`extrapolate` and `expand` also allow to produce metainformation based on demand signalled from the downstream.
+`extrapolate` and `expand` also allow to produce meta-information based on demand signalled from the downstream.
Leveraging this, here is a flow that tracks and reports a drift between a fast consumer and a slow producer.
Scala
@@ -260,3 +260,5 @@ This makes `expand` able to transform or even filter out (by providing an empty
Regardless, since we provide a non-empty `Iterator` in both examples, this means that the
output of this flow is going to report a drift of zero if the producer is fast enough - or a larger drift otherwise.
+
+See also @ref:[`extrapolate`](operators/Source-or-Flow/extrapolate.md) and @ref:[`expand`](operators/Source-or-Flow/expand.md) for more information and examples.
diff --git a/akka-docs/src/main/paradox/stream/stream-refs.md b/akka-docs/src/main/paradox/stream/stream-refs.md
index c9d6009afb..c13ba1f08b 100644
--- a/akka-docs/src/main/paradox/stream/stream-refs.md
+++ b/akka-docs/src/main/paradox/stream/stream-refs.md
@@ -6,7 +6,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
@@ -29,7 +29,7 @@ distributed processing framework or to introduce such capabilities in specific p
Stream refs are trivial to use in existing clustered Akka applications and require no additional configuration
or setup. They automatically maintain flow-control / back-pressure over the network and employ Akka's failure detection
mechanisms to fail-fast ("let it crash!") in the case of failures of remote nodes. They can be seen as an implementation
-of the [Work Pulling Pattern](http://www.michaelpollmeier.com/akka-work-pulling-pattern), which one would otherwise
+of the [Work Pulling Pattern](https://www.michaelpollmeier.com/akka-work-pulling-pattern), which one would otherwise
implement manually.
@@@ note
diff --git a/akka-docs/src/main/paradox/stream/stream-substream.md b/akka-docs/src/main/paradox/stream/stream-substream.md
index bfdc093fda..13026497b8 100644
--- a/akka-docs/src/main/paradox/stream/stream-substream.md
+++ b/akka-docs/src/main/paradox/stream/stream-substream.md
@@ -6,7 +6,7 @@ To use Akka Streams, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream_$scala.binary_version$"
+ artifact="akka-stream_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/stream/stream-testkit.md b/akka-docs/src/main/paradox/stream/stream-testkit.md
index a8115bfd73..2b132c7359 100644
--- a/akka-docs/src/main/paradox/stream/stream-testkit.md
+++ b/akka-docs/src/main/paradox/stream/stream-testkit.md
@@ -6,7 +6,7 @@ To use Akka Stream TestKit, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-stream-testkit_$scala.binary_version$"
+ artifact="akka-stream-testkit_$scala.binary.version$"
version="$akka.version$"
scope="test"
}
diff --git a/akka-docs/src/main/paradox/testing.md b/akka-docs/src/main/paradox/testing.md
index 712d53db5c..8d1aeb979b 100644
--- a/akka-docs/src/main/paradox/testing.md
+++ b/akka-docs/src/main/paradox/testing.md
@@ -9,7 +9,7 @@ To use Akka Testkit, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-testkit_$scala.binary_version$"
+ artifact="akka-testkit_$scala.binary.version$"
version="$akka.version$"
scope="test"
}
@@ -758,7 +758,7 @@ akka {
## Different Testing Frameworks
-Akka’s own test suite is written using [ScalaTest](http://scalatest.org),
+Akka’s own test suite is written using [ScalaTest](http://www.scalatest.org),
which also shines through in documentation examples. However, the TestKit and
its facilities do not depend on that framework, you can essentially use
whichever suits your development style best.
@@ -783,7 +783,7 @@ backwards compatibility in the future, use at own risk.
### Specs2
-Some [Specs2](http://specs2.org) users have contributed examples of how to work around some clashes which may arise:
+Some [Specs2](https://etorreborre.github.io/specs2/) users have contributed examples of how to work around some clashes which may arise:
* Mixing TestKit into `org.specs2.mutable.Specification` results in a
name clash involving the `end` method (which is a private variable in
diff --git a/akka-docs/src/main/paradox/typed/actor-discovery.md b/akka-docs/src/main/paradox/typed/actor-discovery.md
index 902939d803..6a50e3986f 100644
--- a/akka-docs/src/main/paradox/typed/actor-discovery.md
+++ b/akka-docs/src/main/paradox/typed/actor-discovery.md
@@ -1,6 +1,6 @@
# Actor discovery
-For the Akka Classic documentation of this feature see @ref:[Classic Actors](../actors.md#actorselection).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Actors](../actors.md#actorselection).
## Dependency
@@ -8,7 +8,7 @@ To use Akka Actor Typed, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-actor-typed_$scala.binary_version$
+ artifact=akka-actor-typed_$scala.binary.version$
version=$akka.version$
}
@@ -36,7 +36,7 @@ registered to the same key.
The registry is dynamic. New actors can be registered during the lifecycle of the system. Entries are removed when
registered actors are stopped, manually deregistered or the node they live on is removed from the @ref:[Cluster](cluster.md).
To facilitate this dynamic aspect you can also subscribe to changes with the `Receptionist.Subscribe` message. It will send
-`Listing` messages to the subscriber when entries for a key are changed.
+`Listing` messages to the subscriber, first with the set of entries upon subscription, then whenever the entries for a key are changed.
These imports are used in the following example:
diff --git a/akka-docs/src/main/paradox/typed/actor-lifecycle.md b/akka-docs/src/main/paradox/typed/actor-lifecycle.md
index 23a5af3b6a..7835178972 100644
--- a/akka-docs/src/main/paradox/typed/actor-lifecycle.md
+++ b/akka-docs/src/main/paradox/typed/actor-lifecycle.md
@@ -3,7 +3,7 @@ project.description: The Akka Actor lifecycle.
---
# Actor lifecycle
-For the Akka Classic documentation of this feature see @ref:[Classic Actors](../actors.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Actors](../actors.md).
## Dependency
@@ -11,7 +11,7 @@ To use Akka Actor Typed, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-actor-typed_$scala.binary_version$
+ artifact=akka-actor-typed_$scala.binary.version$
version=$akka.version$
}
diff --git a/akka-docs/src/main/paradox/typed/actors.md b/akka-docs/src/main/paradox/typed/actors.md
index 2c3d16b9e0..6e323ff8b0 100644
--- a/akka-docs/src/main/paradox/typed/actors.md
+++ b/akka-docs/src/main/paradox/typed/actors.md
@@ -3,7 +3,7 @@ project.description: The Actor model, managing internal state and changing behav
---
# Introduction to Actors
-For the Akka Classic documentation of this feature see @ref:[Classic Actors](../actors.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Actors](../actors.md).
## Module info
@@ -11,7 +11,7 @@ To use Akka Actors, add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-actor-typed_$scala.binary_version$
+ artifact=akka-actor-typed_$scala.binary.version$
version=$akka.version$
}
@@ -23,7 +23,7 @@ imports when working in Scala, or viceversa. See @ref:[IDE Tips](../additional/i
## Akka Actors
-The [Actor Model](http://en.wikipedia.org/wiki/Actor_model) provides a higher level of abstraction for writing concurrent
+The [Actor Model](https://en.wikipedia.org/wiki/Actor_model) provides a higher level of abstraction for writing concurrent
and distributed systems. It alleviates the developer from having to deal with
explicit locking and thread management, making it easier to write correct
concurrent and parallel systems. Actors were defined in the 1973 paper by Carl
diff --git a/akka-docs/src/main/paradox/typed/choosing-cluster.md b/akka-docs/src/main/paradox/typed/choosing-cluster.md
index 5a594cb274..f56b200c9f 100644
--- a/akka-docs/src/main/paradox/typed/choosing-cluster.md
+++ b/akka-docs/src/main/paradox/typed/choosing-cluster.md
@@ -10,7 +10,7 @@ Microservices has many attractive properties, such as the independent nature of
multiple smaller and more focused teams that can deliver new functionality more frequently and can
respond quicker to business opportunities. Reactive Microservices should be isolated, autonomous, and have
a single responsibility as identified by Jonas Bonér in the book
-[Reactive Microsystems: The Evolution of Microservices at Scale](https://info.lightbend.com/ebook-reactive-microservices-the-evolution-of-microservices-at-scale-register.html).
+[Reactive Microsystems: The Evolution of Microservices at Scale](https://www.lightbend.com/ebooks/reactive-microsystems-evolution-of-microservices-scalability-oreilly).
In a microservices architecture, you should consider communication within a service and between services.
@@ -29,9 +29,9 @@ during a rolling deployment, but deployment of the entire set has a single point
intra-service communication can take advantage of Akka Cluster, failure management and actor messaging, which
is convenient to use and has great performance.
-Between different services [Akka HTTP](https://doc.akka.io/docs/akka-http/current) or
+Between different services [Akka HTTP](https://doc.akka.io/docs/akka-http/current/) or
[Akka gRPC](https://doc.akka.io/docs/akka-grpc/current/) can be used for synchronous (yet non-blocking)
-communication and [Akka Streams Kafka](https://doc.akka.io/docs/akka-stream-kafka/current/home.html) or other
+communication and [Akka Streams Kafka](https://doc.akka.io/docs/alpakka-kafka/current/) or other
[Alpakka](https://doc.akka.io/docs/alpakka/current/) connectors for integration asynchronous communication.
All those communication mechanisms work well with streaming of messages with end-to-end back-pressure, and the
synchronous communication tools can also be used for single request response interactions. It is also important
diff --git a/akka-docs/src/main/paradox/typed/cluster-concepts.md b/akka-docs/src/main/paradox/typed/cluster-concepts.md
index a142898987..1153f782ab 100644
--- a/akka-docs/src/main/paradox/typed/cluster-concepts.md
+++ b/akka-docs/src/main/paradox/typed/cluster-concepts.md
@@ -30,15 +30,15 @@ and membership state transitions.
### Gossip
-The cluster membership used in Akka is based on Amazon's [Dynamo](http://www.allthingsdistributed.com/files/amazon-dynamo-sosp2007.pdf) system and
-particularly the approach taken in Basho's' [Riak](http://basho.com/technology/architecture/) distributed database.
-Cluster membership is communicated using a [Gossip Protocol](http://en.wikipedia.org/wiki/Gossip_protocol), where the current
+The cluster membership used in Akka is based on Amazon's [Dynamo](https://www.allthingsdistributed.com/files/amazon-dynamo-sosp2007.pdf) system and
+particularly the approach taken in Basho's' [Riak](https://riak.com/technology/architecture/) distributed database.
+Cluster membership is communicated using a [Gossip Protocol](https://en.wikipedia.org/wiki/Gossip_protocol), where the current
state of the cluster is gossiped randomly through the cluster, with preference to
members that have not seen the latest version.
#### Vector Clocks
-[Vector clocks](http://en.wikipedia.org/wiki/Vector_clock) are a type of data structure and algorithm for generating a partial
+[Vector clocks](https://en.wikipedia.org/wiki/Vector_clock) are a type of data structure and algorithm for generating a partial
ordering of events in a distributed system and detecting causality violations.
We use vector clocks to reconcile and merge differences in cluster state
@@ -48,7 +48,7 @@ to the cluster state has an accompanying update to the vector clock.
#### Gossip Convergence
Information about the cluster converges locally at a node at certain points in time.
-This is when a node can prove that the cluster state he is observing has been observed
+This is when a node can prove that the cluster state it is observing has been observed
by all other nodes in the cluster. Convergence is implemented by passing a set of nodes
that have seen current state version during gossip. This information is referred to as the
seen set in the gossip overview. When all nodes are included in the seen set there is
@@ -175,5 +175,5 @@ The periodic nature of the gossip has a nice batching effect of state changes,
e.g. joining several nodes quickly after each other to one node will result in only
one state change to be spread to other members in the cluster.
-The gossip messages are serialized with [protobuf](https://code.google.com/p/protobuf/) and also gzipped to reduce payload
+The gossip messages are serialized with [protobuf](https://github.com/protocolbuffers/protobuf) and also gzipped to reduce payload
size.
diff --git a/akka-docs/src/main/paradox/typed/cluster-dc.md b/akka-docs/src/main/paradox/typed/cluster-dc.md
index 315c1e4c8c..544bfd3ebd 100644
--- a/akka-docs/src/main/paradox/typed/cluster-dc.md
+++ b/akka-docs/src/main/paradox/typed/cluster-dc.md
@@ -1,6 +1,6 @@
# Multi-DC Cluster
-For the Akka Classic documentation of this feature see @ref:[Classic Multi-DC Cluster](../cluster-dc.md)
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Multi-DC Cluster](../cluster-dc.md)
This chapter describes how @ref[Akka Cluster](cluster.md) can be used across
multiple data centers, availability zones or regions.
@@ -20,7 +20,7 @@ To use Akka Cluster add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-typed_$scala.binary_version$
+ artifact=akka-cluster-typed_$scala.binary.version$
version=$akka.version$
}
diff --git a/akka-docs/src/main/paradox/typed/cluster-membership.md b/akka-docs/src/main/paradox/typed/cluster-membership.md
index 327408916e..dbd53512cc 100644
--- a/akka-docs/src/main/paradox/typed/cluster-membership.md
+++ b/akka-docs/src/main/paradox/typed/cluster-membership.md
@@ -29,7 +29,7 @@ UID.
## Member States
-The cluster membership state is a specialized [CRDT](http://hal.upmc.fr/docs/00/55/55/88/PDF/techreport.pdf), which means that it has a monotonic
+The cluster membership state is a specialized [CRDT](https://hal.inria.fr/file/index/docid/555588/filename/techreport.pdf), which means that it has a monotonic
merge function. When concurrent changes occur on different nodes the updates can always be
merged and converge to the same end result.
@@ -108,8 +108,7 @@ performs in such a case must be designed in a way that all concurrent leaders wo
might be impossible in general and only feasible under additional constraints). The most important case of that kind is a split
brain scenario where nodes need to be downed, either manually or automatically, to bring the cluster back to convergence.
-See the [Lightbend Split Brain Resolver](https://doc.akka.io/docs/akka-enhancements/current/split-brain-resolver.html)
-for an implementation of that.
+The @ref:[Split Brain Resolver](../split-brain-resolver.md) is the built-in implementation of that.
Another transition that is possible without convergence is marking members as `WeaklyUp` as described in the next section.
@@ -140,7 +139,7 @@ startup if a node to join have been specified in the configuration
* **leave** - tell a node to leave the cluster gracefully, normally triggered by ActorSystem or JVM shutdown through @ref[coordinated shutdown](../coordinated-shutdown.md)
- * **down** - mark a node as down. This action is required to remove crashed nodes (that did not 'leave') from the cluster. It can be triggered manually, through [Cluster HTTP Management](https://doc.akka.io/docs/akka-management/current/cluster-http-management.html#put-cluster-members-address-responses), or automatically by a @ref[downing provider](cluster.md#downing) like [Split Brain Resolver](https://doc.akka.io/docs/akka-enhancements/current/split-brain-resolver.html)
+ * **down** - mark a node as down. This action is required to remove crashed nodes (that did not 'leave') from the cluster. It can be triggered manually, through [Cluster HTTP Management](https://doc.akka.io/docs/akka-management/current/cluster-http-management.html#put-cluster-members-address-responses), or automatically by a @ref[downing provider](cluster.md#downing) like @ref:[Split Brain Resolver](../split-brain-resolver.md)
#### Leader Actions
diff --git a/akka-docs/src/main/paradox/typed/cluster-sharded-daemon-process.md b/akka-docs/src/main/paradox/typed/cluster-sharded-daemon-process.md
index 9ed36d620f..1d145ab8bd 100644
--- a/akka-docs/src/main/paradox/typed/cluster-sharded-daemon-process.md
+++ b/akka-docs/src/main/paradox/typed/cluster-sharded-daemon-process.md
@@ -15,7 +15,7 @@ To use Akka Sharded Daemon Process, you must add the following dependency in you
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-sharding-typed_$scala.binary_version$
+ artifact=akka-cluster-sharding-typed_$scala.binary.version$
version=$akka.version$
}
@@ -55,4 +55,4 @@ either with a single `ServiceKey` which all daemon process actors register theme
## Scalability
This cluster tool is intended for small numbers of consumers and will not scale well to a large set. In large clusters
-it is recommended to limit the nodes the sharded daemon process will run on using a role.
\ No newline at end of file
+it is recommended to limit the nodes the sharded daemon process will run on using a role.
diff --git a/akka-docs/src/main/paradox/typed/cluster-sharding.md b/akka-docs/src/main/paradox/typed/cluster-sharding.md
index 2d5eb70780..7ba3b5265b 100644
--- a/akka-docs/src/main/paradox/typed/cluster-sharding.md
+++ b/akka-docs/src/main/paradox/typed/cluster-sharding.md
@@ -3,7 +3,7 @@ project.description: Shard a clustered compute process across the network with l
---
# Cluster Sharding
-For the Akka Classic documentation of this feature see @ref:[Classic Cluster Sharding](../cluster-sharding.md)
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Cluster Sharding](../cluster-sharding.md)
## Module info
@@ -11,7 +11,7 @@ To use Akka Cluster Sharding, you must add the following dependency in your proj
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-sharding-typed_$scala.binary_version$
+ artifact=akka-cluster-sharding-typed_$scala.binary.version$
version=$akka.version$
}
@@ -423,6 +423,36 @@ rebalanced to other nodes.
See @ref:[How To Startup when Cluster Size Reached](cluster.md#how-to-startup-when-a-cluster-size-is-reached)
for more information about `min-nr-of-members`.
+## Inspecting cluster sharding state
+
+Two requests to inspect the cluster state are available:
+
+@apidoc[akka.cluster.sharding.typed.GetShardRegionState] which will reply with a
+@apidoc[ShardRegion.CurrentShardRegionState] that contains the identifiers of the shards running in
+a Region and what entities are alive for each of them.
+
+Scala
+: @@snip [ShardingCompileOnlySpec.scala](/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/ShardingCompileOnlySpec.scala) { #get-shard-region-state }
+
+Java
+: @@snip [ShardingCompileOnlyTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ShardingCompileOnlyTest.java) { #get-shard-region-state }
+
+@apidoc[akka.cluster.sharding.typed.GetClusterShardingStats] which will query all the regions in the cluster and reply with a
+@apidoc[ShardRegion.ClusterShardingStats] containing the identifiers of the shards running in each region and a count
+of entities that are alive in each shard.
+
+Scala
+: @@snip [ShardingCompileOnlySpec.scala](/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/ShardingCompileOnlySpec.scala) { #get-cluster-sharding-stats }
+
+Java
+: @@snip [ShardingCompileOnlyTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ShardingCompileOnlyTest.java) { #get-cluster-sharding-stats }
+
+If any shard queries failed, for example due to timeout if a shard was too busy to reply within the configured `akka.cluster.sharding.shard-region-query-timeout`,
+`ShardRegion.CurrentShardRegionState` and `ShardRegion.ClusterShardingStats` will also include the set of shard identifiers by region that failed.
+
+The purpose of these messages is testing and monitoring, they are not provided to give access to
+directly sending messages to the individual entities.
+
## Lease
A @ref[lease](../coordination.md) can be used as an additional safety measure to ensure a shard
diff --git a/akka-docs/src/main/paradox/typed/cluster-singleton.md b/akka-docs/src/main/paradox/typed/cluster-singleton.md
index 65c004743d..0ac134bfaa 100644
--- a/akka-docs/src/main/paradox/typed/cluster-singleton.md
+++ b/akka-docs/src/main/paradox/typed/cluster-singleton.md
@@ -1,6 +1,6 @@
# Cluster Singleton
-For the Akka Classic documentation of this feature see @ref:[Classic Cluster Singleton](../cluster-singleton.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Cluster Singleton](../cluster-singleton.md).
## Module info
@@ -8,7 +8,7 @@ To use Cluster Singleton, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-typed_$scala.binary_version$
+ artifact=akka-cluster-typed_$scala.binary.version$
version=$akka.version$
}
diff --git a/akka-docs/src/main/paradox/typed/cluster.md b/akka-docs/src/main/paradox/typed/cluster.md
index c17bf83d15..fa007ae31e 100644
--- a/akka-docs/src/main/paradox/typed/cluster.md
+++ b/akka-docs/src/main/paradox/typed/cluster.md
@@ -13,7 +13,7 @@ For specific documentation topics see:
* @ref:[Rolling Updates](../additional/rolling-updates.md)
* @ref:[Operating, Managing, Observability](../additional/operations.md)
-For the Akka Classic documentation of this feature see @ref:[Classic Cluster](../cluster-usage.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Cluster](../cluster-usage.md).
You have to enable @ref:[serialization](../serialization.md) to send messages between ActorSystems (nodes) in the Cluster.
@ref:[Serialization with Jackson](../serialization-jackson.md) is a good choice in many cases, and our
@@ -25,7 +25,7 @@ To use Akka Cluster add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-typed_$scala.binary_version$
+ artifact=akka-cluster-typed_$scala.binary.version$
version=$akka.version$
}
@@ -275,23 +275,22 @@ new joining members to 'Up'. The node must first become `reachable` again, or th
status of the unreachable member must be changed to `Down`. Changing status to `Down`
can be performed automatically or manually.
-By default, downing must be performed manually using @ref:[HTTP](../additional/operations.md#http) or @ref:[JMX](../additional/operations.md#jmx).
+We recommend that you enable the @ref:[Split Brain Resolver](../split-brain-resolver.md) that is part of the
+Akka Cluster module. You enable it with configuration:
+
+```
+akka.cluster.downing-provider-class = "akka.cluster.sbr.SplitBrainResolverProvider"
+```
+
+You should also consider the different available @ref:[downing strategies](../split-brain-resolver.md#strategies).
+
+If a downing provider is not configured downing must be performed manually using
+@ref:[HTTP](../additional/operations.md#http) or @ref:[JMX](../additional/operations.md#jmx).
Note that @ref:[Cluster Singleton](cluster-singleton.md) or @ref:[Cluster Sharding entities](cluster-sharding.md) that
are running on a crashed (unreachable) node will not be started on another node until the previous node has
been removed from the Cluster. Removal of crashed (unreachable) nodes is performed after a downing decision.
-A production solution for downing is provided by
-[Split Brain Resolver](https://doc.akka.io/docs/akka-enhancements/current/split-brain-resolver.html),
-which is part of the [Lightbend Platform](http://www.lightbend.com/platform).
-If you don’t have a Lightbend Platform Subscription, you should still carefully read the
-[documentation](https://doc.akka.io/docs/akka-enhancements/current/split-brain-resolver.html)
-of the Split Brain Resolver and make sure that the solution you are using handles the concerns and scenarios
-described there.
-
-A custom downing strategy can be implemented with a @apidoc[akka.cluster.DowningProvider] and enabled with
-configuration `akka.cluster.downing-provider-class`.
-
Downing can also be performed programmatically with @scala[`Cluster(system).manager ! Down(address)`]@java[`Cluster.get(system).manager().tell(Down(address))`],
but that is mostly useful from tests and when implementing a `DowningProvider`.
@@ -439,7 +438,11 @@ See @ref:[Cluster Sharding](cluster-sharding.md).
@@include[cluster.md](../includes/cluster.md) { #cluster-ddata }
See @ref:[Distributed Data](distributed-data.md).
-@@include[cluster.md](../includes/cluster.md) { #cluster-pubsub }
+@@include[cluster.md](../includes/cluster.md) { #cluster-pubsub }
+See @ref:[Distributed Publish Subscribe](distributed-pub-sub.md).
+
+@@include[cluster.md](../includes/cluster.md) { #cluster-router }
+See @ref:[Group Routers](routers.md#group-router).
@@include[cluster.md](../includes/cluster.md) { #cluster-multidc }
See @ref:[Cluster Multi-DC](cluster-dc.md).
diff --git a/akka-docs/src/main/paradox/typed/coexisting.md b/akka-docs/src/main/paradox/typed/coexisting.md
index 3922313bfe..c79716e6ff 100644
--- a/akka-docs/src/main/paradox/typed/coexisting.md
+++ b/akka-docs/src/main/paradox/typed/coexisting.md
@@ -6,7 +6,7 @@ To use Akka Actor Typed, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-actor-typed_$scala.binary_version$
+ artifact=akka-actor-typed_$scala.binary.version$
version=$akka.version$
}
diff --git a/akka-docs/src/main/paradox/typed/dispatchers.md b/akka-docs/src/main/paradox/typed/dispatchers.md
index 1f3fc1d615..8d825cd214 100644
--- a/akka-docs/src/main/paradox/typed/dispatchers.md
+++ b/akka-docs/src/main/paradox/typed/dispatchers.md
@@ -3,7 +3,7 @@ project.description: Akka dispatchers and how to choose the right ones.
---
# Dispatchers
-For the Akka Classic documentation of this feature see @ref:[Classic Dispatchers](../dispatchers.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Dispatchers](../dispatchers.md).
## Dependency
@@ -12,7 +12,7 @@ page describes how to use dispatchers with `akka-actor-typed`, which has depende
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor-typed_$scala.binary_version$"
+ artifact="akka-actor-typed_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/typed/distributed-data.md b/akka-docs/src/main/paradox/typed/distributed-data.md
index 573f289816..510c6f0e0e 100644
--- a/akka-docs/src/main/paradox/typed/distributed-data.md
+++ b/akka-docs/src/main/paradox/typed/distributed-data.md
@@ -3,7 +3,7 @@ project.description: Share data between nodes and perform updates without coordi
---
# Distributed Data
-For the Akka Classic documentation of this feature see @ref:[Classic Distributed Data](../distributed-data.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Distributed Data](../distributed-data.md).
## Module info
@@ -11,7 +11,7 @@ To use Akka Cluster Distributed Data, you must add the following dependency in y
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-typed_$scala.binary_version$
+ artifact=akka-cluster-typed_$scala.binary.version$
version=$akka.version$
}
@@ -270,8 +270,8 @@ better safety for small clusters.
#### Consistency and response types
When using `ReadLocal`, you will never receive a `GetFailure` response, since the local replica is always available to
-local readers. `WriteLocal` however may still reply with `UpdateFailure` messages, in the event that the `modify` function
-threw an exception, or, if using @ref:[durable storage](#durable-storage), if storing failed.
+local readers. `WriteLocal` however may still reply with `UpdateFailure` messages if the `modify` function
+throws an exception, or if it fails to persist to @ref:[durable storage](#durable-storage).
#### Examples
@@ -534,7 +534,7 @@ akka.cluster.distributed-data.prefer-oldest = on
### Delta-CRDT
-[Delta State Replicated Data Types](http://arxiv.org/abs/1603.01529)
+[Delta State Replicated Data Types](https://arxiv.org/abs/1603.01529)
are supported. A delta-CRDT is a way to reduce the need for sending the full state
for updates. For example adding element `'c'` and `'d'` to set `{'a', 'b'}` would
result in sending the delta `{'c', 'd'}` and merge that with the state on the
@@ -665,7 +665,7 @@ All entries can be made durable by specifying:
akka.cluster.distributed-data.durable.keys = ["*"]
```
-@scala[[LMDB](https://symas.com/products/lightning-memory-mapped-database/)]@java[[LMDB](https://github.com/lmdbjava/lmdbjava/)] is the default storage implementation. It is
+@scala[[LMDB](https://symas.com/lmdb/technical/)]@java[[LMDB](https://github.com/lmdbjava/lmdbjava/)] is the default storage implementation. It is
possible to replace that with another implementation by implementing the actor protocol described in
`akka.cluster.ddata.DurableStore` and defining the `akka.cluster.distributed-data.durable.store-actor-class`
property for the new implementation.
@@ -761,11 +761,9 @@ API documentation of the `Replicator` for details.
## Learn More about CRDTs
- * [Eventually Consistent Data Structures](https://vimeo.com/43903960)
-talk by Sean Cribbs
* [Strong Eventual Consistency and Conflict-free Replicated Data Types (video)](https://www.youtube.com/watch?v=oyUHd894w18&feature=youtu.be)
talk by Mark Shapiro
- * [A comprehensive study of Convergent and Commutative Replicated Data Types](http://hal.upmc.fr/file/index/docid/555588/filename/techreport.pdf)
+ * [A comprehensive study of Convergent and Commutative Replicated Data Types](https://hal.inria.fr/file/index/docid/555588/filename/techreport.pdf)
paper by Mark Shapiro et. al.
## Configuration
diff --git a/akka-docs/src/main/paradox/typed/distributed-pub-sub.md b/akka-docs/src/main/paradox/typed/distributed-pub-sub.md
index 813dfe8593..6e2f20cd27 100644
--- a/akka-docs/src/main/paradox/typed/distributed-pub-sub.md
+++ b/akka-docs/src/main/paradox/typed/distributed-pub-sub.md
@@ -1,6 +1,6 @@
# Distributed Publish Subscribe in Cluster
-For the Akka Classic documentation of this feature see @ref:[Classic Distributed Publish Subscribe](../distributed-pub-sub.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Distributed Publish Subscribe](../distributed-pub-sub.md).
## Module info
@@ -9,7 +9,7 @@ when used in a clustered application:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-cluster-typed_$scala.binary_version$"
+ artifact="akka-cluster-typed_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/typed/fault-tolerance.md b/akka-docs/src/main/paradox/typed/fault-tolerance.md
index 0fa90388bf..cccd7db2a8 100644
--- a/akka-docs/src/main/paradox/typed/fault-tolerance.md
+++ b/akka-docs/src/main/paradox/typed/fault-tolerance.md
@@ -1,6 +1,6 @@
# Fault Tolerance
-For the Akka Classic documentation of this feature see @ref:[Classic Fault Tolerance](../fault-tolerance.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Fault Tolerance](../fault-tolerance.md).
When an actor throws an unexpected exception, a failure, while processing a message or during initialization, the actor
will by default be stopped.
diff --git a/akka-docs/src/main/paradox/typed/from-classic.md b/akka-docs/src/main/paradox/typed/from-classic.md
index 710cb53da3..844db9d0d1 100644
--- a/akka-docs/src/main/paradox/typed/from-classic.md
+++ b/akka-docs/src/main/paradox/typed/from-classic.md
@@ -27,7 +27,7 @@ For example `akka-cluster-typed`:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-typed_$scala.binary_version$
+ artifact=akka-cluster-typed_$scala.binary.version$
version=$akka.version$
}
@@ -56,12 +56,12 @@ APIs, which is familiar from Akka Streams.
Examples of a few package names:
-| Classic | Typed for Scala | Typed for Java |
-|-----------------------|---------------------------------|--------------------------------|
-| akka.actor | akka.actor.typed.scaladsl | akka.actor.typed.javadsl |
-| akka.cluster | akka.cluster.typed | akka.cluster.typed |
-| akka.cluster.sharding | akka.cluster.sharding.scaladsl | akka.cluster.sharding.javadsl |
-| akka.persistence | akka.persistence.typed.scaladsl | akka.persistence.typed.javadsl |
+| Classic | Typed for Scala | Typed for Java |
+|-----------------------|---------------------------------------|--------------------------------------|
+| akka.actor | akka.actor.typed.scaladsl | akka.actor.typed.javadsl |
+| akka.cluster | akka.cluster.typed | akka.cluster.typed |
+| akka.cluster.sharding | akka.cluster.sharding.typed.scaladsl | akka.cluster.sharding.typed.javadsl |
+| akka.persistence | akka.persistence.typed.scaladsl | akka.persistence.typed.javadsl |
## Actor definition
@@ -251,8 +251,8 @@ Links to reference documentation:
## Stopping
-Classic actors can be stopped with the `stop` method of `ActorContext` or `ActorSystem`. In Typed an actor is
-stopping itself by returning `Behaviors.stopped`. There is also a `stop` method in the `ActorContext` but it
+Classic actors can be stopped with the `stop` method of `ActorContext` or `ActorSystem`. In Typed an actor stops
+itself by returning `Behaviors.stopped`. There is also a `stop` method in the `ActorContext` but it
can only be used for stopping direct child actors and not any arbitrary actor.
`PoisonPill` is not supported in Typed. Instead, if you need to request an actor to stop you should
diff --git a/akka-docs/src/main/paradox/typed/fsm.md b/akka-docs/src/main/paradox/typed/fsm.md
index a9d28486c1..cc954e223d 100644
--- a/akka-docs/src/main/paradox/typed/fsm.md
+++ b/akka-docs/src/main/paradox/typed/fsm.md
@@ -3,7 +3,7 @@ project.description: Finite State Machines (FSM) with Akka Actors.
---
# Behaviors as finite state machines
-For the Akka Classic documentation of this feature see @ref:[Classic FSM](../fsm.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic FSM](../fsm.md).
An actor can be used to model a Finite State Machine (FSM).
diff --git a/akka-docs/src/main/paradox/typed/guide/introduction.md b/akka-docs/src/main/paradox/typed/guide/introduction.md
index ab2580b737..86576d8271 100644
--- a/akka-docs/src/main/paradox/typed/guide/introduction.md
+++ b/akka-docs/src/main/paradox/typed/guide/introduction.md
@@ -31,7 +31,7 @@ efficiently.
## How to get started
If this is your first experience with Akka, we recommend that you start by
-running a simple Hello World project. See the @scala[[Quickstart Guide](https://developer.lightbend.com/guides/akka-quickstart-scala)] @java[[Quickstart Guide](https://developer.lightbend.com/guides/akka-quickstart-java)] for
+running a simple Hello World project. See the @scala[[Quickstart Guide](https://developer.lightbend.com/guides/akka-quickstart-scala/)] @java[[Quickstart Guide](https://developer.lightbend.com/guides/akka-quickstart-java/)] for
instructions on downloading and running the Hello World example. The *Quickstart* guide walks you through example code that introduces how to define actor systems, actors, and messages as well as how to use the test module and logging. Within 30 minutes, you should be able to run the Hello World example and learn how it is constructed.
This *Getting Started* guide provides the next level of information. It covers why the actor model fits the needs of modern distributed systems and includes a tutorial that will help further your knowledge of Akka. Topics include:
diff --git a/akka-docs/src/main/paradox/typed/guide/modules.md b/akka-docs/src/main/paradox/typed/guide/modules.md
index bc7effcf0d..dd626f07c6 100644
--- a/akka-docs/src/main/paradox/typed/guide/modules.md
+++ b/akka-docs/src/main/paradox/typed/guide/modules.md
@@ -1,6 +1,6 @@
# Overview of Akka libraries and modules
-Before delving into some best practices for writing actors, it will be helpful to preview the most commonly used Akka libraries. This will help you start thinking about the functionality you want to use in your system. All core Akka functionality is available as Open Source Software (OSS). Lightbend sponsors Akka development but can also help you with [commercial offerings ](https://www.lightbend.com/platform/subscription) such as training, consulting, support, and [Enterprise Suite](https://www.lightbend.com/platform/production) — a comprehensive set of tools for managing Akka systems.
+Before delving into some best practices for writing actors, it will be helpful to preview the most commonly used Akka libraries. This will help you start thinking about the functionality you want to use in your system. All core Akka functionality is available as Open Source Software (OSS). Lightbend sponsors Akka development but can also help you with [commercial offerings ](https://www.lightbend.com/lightbend-subscription) such as training, consulting, support, and [Enterprise capabilities](https://www.lightbend.com/why-lightbend#enterprise-capabilities) — a comprehensive set of tools for managing Akka systems.
The following capabilities are included with Akka OSS and are introduced later on this page:
@@ -14,11 +14,10 @@ The following capabilities are included with Akka OSS and are introduced later o
* @ref:[Streams](#streams)
* @ref:[HTTP](#http)
-With a [Lightbend Platform Subscription](https://www.lightbend.com/platform/subscription), you can use [Akka Enhancements](https://doc.akka.io/docs/akka-enhancements/current/) that includes:
+With a [Lightbend Platform Subscription](https://www.lightbend.com/lightbend-subscription), you can use [Akka Enhancements](https://doc.akka.io/docs/akka-enhancements/current/) that includes:
[Akka Resilience Enhancements](https://doc.akka.io/docs/akka-enhancements/current/akka-resilience-enhancements.html):
-* [Split Brain Resolver](https://doc.akka.io/docs/akka-enhancements/current/split-brain-resolver.html) — Detects and recovers from network partitions, eliminating data inconsistencies and possible downtime.
* [Configuration Checker](https://doc.akka.io/docs/akka-enhancements/current/config-checker.html) — Checks for potential configuration issues and logs suggestions.
* [Diagnostics Recorder](https://doc.akka.io/docs/akka-enhancements/current/diagnostics-recorder.html) — Captures configuration and system information in a format that makes it easy to troubleshoot issues during development and production.
* [Thread Starvation Detector](https://doc.akka.io/docs/akka-enhancements/current/starvation-detector.html) — Monitors an Akka system dispatcher and logs warnings if it becomes unresponsive.
@@ -36,7 +35,7 @@ This page does not list all available modules, but overviews the main functional
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-actor-typed_$scala.binary_version$
+ artifact=akka-actor-typed_$scala.binary.version$
version=$akka.version$
}
@@ -61,7 +60,7 @@ Challenges that actors solve include the following:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-remote_$scala.binary_version$
+ artifact=akka-remote_$scala.binary.version$
version=$akka.version$
}
@@ -84,7 +83,7 @@ Challenges Remoting solves include the following:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-typed_$scala.binary_version$
+ artifact=akka-cluster-typed_$scala.binary.version$
version=$akka.version$
}
@@ -107,7 +106,7 @@ Challenges the Cluster module solves include the following:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-sharding-typed_$scala.binary_version$
+ artifact=akka-cluster-sharding-typed_$scala.binary.version$
version=$akka.version$
}
@@ -126,7 +125,7 @@ Challenges that Sharding solves include the following:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-singleton_$scala.binary_version$
+ artifact=akka-cluster-singleton_$scala.binary.version$
version=$akka.version$
}
@@ -147,7 +146,7 @@ The Singleton module can be used to solve these challenges:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-persistence-typed_$scala.binary_version$
+ artifact=akka-persistence-typed_$scala.binary.version$
version=$akka.version$
}
@@ -160,7 +159,7 @@ cluster for example) or alternate views (like reports).
Persistence tackles the following challenges:
* How to restore the state of an entity/actor when system restarts or crashes.
-* How to implement a [CQRS system](https://msdn.microsoft.com/en-us/library/jj591573.aspx).
+* How to implement a [CQRS system](https://docs.microsoft.com/en-us/previous-versions/msp-n-p/jj591573(v=pandp.10)?redirectedfrom=MSDN).
* How to ensure reliable delivery of messages in face of network errors and system crashes.
* How to introspect domain events that have led an entity to its current state.
* How to leverage [Event Sourcing](https://martinfowler.com/eaaDev/EventSourcing.html) in your application to support long-running processes while the project continues to evolve.
@@ -169,7 +168,7 @@ Persistence tackles the following challenges:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-typed_$scala.binary_version$
+ artifact=akka-cluster-typed_$scala.binary.version$
version=$akka.version$
}
@@ -188,7 +187,7 @@ Distributed Data is intended to solve the following challenges:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-stream-typed_$scala.binary_version$
+ artifact=akka-stream-typed_$scala.binary.version$
version=$akka.version$
}
@@ -198,7 +197,7 @@ process a potentially large, or infinite, stream of sequential events and proper
faster processing stages do not overwhelm slower ones in the chain or graph. Streams provide a higher-level
abstraction on top of actors that simplifies writing such processing networks, handling all the fine details in the
background and providing a safe, typed, composable programming model. Streams is also an implementation
-of the [Reactive Streams standard](http://www.reactive-streams.org) which enables integration with all third
+of the [Reactive Streams standard](https://www.reactive-streams.org) which enables integration with all third
party implementations of that standard.
Streams solve the following challenges:
@@ -210,7 +209,7 @@ Streams solve the following challenges:
### HTTP
-[Akka HTTP](https://doc.akka.io/docs/akka-http/current) is a separate module from Akka.
+[Akka HTTP](https://doc.akka.io/docs/akka-http/current/) is a separate module from Akka.
The de facto standard for providing APIs remotely, internal or external, is [HTTP](https://en.wikipedia.org/wiki/Hypertext_Transfer_Protocol). Akka provides a library to construct or consume such HTTP services by giving a set of tools to create HTTP services (and serve them) and a client that can be
used to consume other services. These tools are particularly suited to streaming in and out a large set of data or real-time events by leveraging the underlying model of Akka Streams.
diff --git a/akka-docs/src/main/paradox/typed/guide/tutorial_1.md b/akka-docs/src/main/paradox/typed/guide/tutorial_1.md
index 2db15c4aea..17e8251d54 100644
--- a/akka-docs/src/main/paradox/typed/guide/tutorial_1.md
+++ b/akka-docs/src/main/paradox/typed/guide/tutorial_1.md
@@ -6,7 +6,7 @@ Add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor-typed_$scala.binary_version$"
+ artifact="akka-actor-typed_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/typed/guide/tutorial_5.md b/akka-docs/src/main/paradox/typed/guide/tutorial_5.md
index 0d2bfe38ab..0dd8d8ed06 100644
--- a/akka-docs/src/main/paradox/typed/guide/tutorial_5.md
+++ b/akka-docs/src/main/paradox/typed/guide/tutorial_5.md
@@ -228,7 +228,7 @@ In the context of the IoT system, this guide introduced the following concepts,
To continue your journey with Akka, we recommend:
-* Start building your own applications with Akka, make sure you [get involved in our amazing community](https://akka.io/get-involved) for help if you get stuck.
+* Start building your own applications with Akka, make sure you [get involved in our amazing community](https://akka.io/get-involved/) for help if you get stuck.
* If you’d like some additional background, and detail, read the rest of the @ref:[reference documentation](../actors.md) and check out some of the @ref:[books and videos](../../additional/books.md) on Akka.
* If you are interested in functional programming, read how actors can be defined in a @ref:[functional style](../actors.md#functional-style). In this guide the object-oriented style was used, but you can mix both as you like.
diff --git a/akka-docs/src/main/paradox/typed/index-cluster.md b/akka-docs/src/main/paradox/typed/index-cluster.md
index a17e0dca73..4712692cac 100644
--- a/akka-docs/src/main/paradox/typed/index-cluster.md
+++ b/akka-docs/src/main/paradox/typed/index-cluster.md
@@ -25,6 +25,7 @@ project.description: Akka Cluster concepts, node membership service, CRDT Distri
* [multi-node-testing](../multi-node-testing.md)
* [remoting-artery](../remoting-artery.md)
* [remoting](../remoting.md)
+* [split-brain-resolver](../split-brain-resolver.md)
* [coordination](../coordination.md)
* [choosing-cluster](choosing-cluster.md)
diff --git a/akka-docs/src/main/paradox/typed/interaction-patterns.md b/akka-docs/src/main/paradox/typed/interaction-patterns.md
index e6b25ac4aa..41e79c98d2 100644
--- a/akka-docs/src/main/paradox/typed/interaction-patterns.md
+++ b/akka-docs/src/main/paradox/typed/interaction-patterns.md
@@ -1,6 +1,6 @@
# Interaction Patterns
-For the Akka Classic documentation of this feature see @ref:[Classic Actors](../actors.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Actors](../actors.md).
## Dependency
@@ -8,7 +8,7 @@ To use Akka Actor Typed, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-actor-typed_$scala.binary_version$
+ artifact=akka-actor-typed_$scala.binary.version$
version=$akka.version$
}
@@ -192,7 +192,10 @@ The response adapting function is running in the receiving actor and can safely
Sometimes you need to interact with actors from the outside of the actor system, this can be done with fire-and-forget as described above or through another version of `ask` that returns a @scala[`Future[Response]`]@java[`CompletionStage`] that is either completed with a successful response or failed with a `TimeoutException` if there was no response within the specified timeout.
-To do this we use @scala[`ActorRef.ask` (or the symbolic `ActorRef.?`) implicitly provided by `akka.actor.typed.scaladsl.AskPattern`]@java[`akka.actor.typed.javadsl.AskPattern.ask`] to send a message to an actor and get a @scala[`Future[Response]`]@java[`CompletionState[Response]`] back.
+@scala[To do this we use `ask` (or the symbolic `?`) implicitly added to `ActorRef` by `akka.actor.typed.scaladsl.AskPattern._`
+to send a message to an actor and get a `Future[Response]` back. `ask` takes implicit `Timeout` and `ActorSystem` parameters.]
+@java[To do this we use `akka.actor.typed.javadsl.AskPattern.ask` to send a message to an actor and get a
+`CompletionState[Response]` back.]
**Example:**
@@ -246,6 +249,7 @@ Java
**Problems:**
The returned `ActorRef` ignores all messages sent to it, therefore it should be used carefully.
+
* Passing it around inadvertently as if it was a normal `ActorRef` may result in broken actor-to-actor interactions.
* Using it when performing an `ask` from outside the Actor System will cause the @scala[`Future`]@java[`CompletionStage`] returned by the `ask` to timeout since it will never complete.
* Finally, it's legal to `watch` it, but since it's of a special kind, it never terminates and therefore you will never receive a `Terminated` signal from it.
diff --git a/akka-docs/src/main/paradox/typed/logging.md b/akka-docs/src/main/paradox/typed/logging.md
index e4c34d86e2..c6b376df45 100644
--- a/akka-docs/src/main/paradox/typed/logging.md
+++ b/akka-docs/src/main/paradox/typed/logging.md
@@ -3,7 +3,7 @@ project.description: Logging options with Akka.
---
# Logging
-For the Akka Classic documentation of this feature see @ref:[Classic Logging](../logging.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Logging](../logging.md).
## Dependency
@@ -12,7 +12,7 @@ via the SLF4J backend, such as Logback configuration.
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor-typed_$scala.binary_version$"
+ artifact="akka-actor-typed_$scala.binary.version$"
version="$akka.version$"
}
@@ -241,7 +241,7 @@ All MDC properties as key-value entries can be included with `%mdc`:
```
- %date{ISO8601} %-5level %logger{36} - %msg {%mdc}%n
+ %date{ISO8601} %-5level %logger{36} - %msg MDC: {%mdc}%n
```
@@ -426,7 +426,7 @@ All MDC properties as key-value entries can be included with `%mdc`:
```
- %date{ISO8601} %-5level %logger{36} - %msg {%mdc}%n
+ %date{ISO8601} %-5level %logger{36} - %msg MDC: {%mdc}%n
```
@@ -446,7 +446,7 @@ The marker can be included in the Logback output with `%marker` and all MDC prop
```
- [%date{ISO8601}] [%level] [%logger] [%marker] [%thread] - %msg {%mdc}%n
+ [%date{ISO8601}] [%level] [%logger] [%marker] [%thread] - %msg MDC: {%mdc}%n
```
diff --git a/akka-docs/src/main/paradox/typed/mailboxes.md b/akka-docs/src/main/paradox/typed/mailboxes.md
index c95c13347b..3b73270591 100644
--- a/akka-docs/src/main/paradox/typed/mailboxes.md
+++ b/akka-docs/src/main/paradox/typed/mailboxes.md
@@ -1,6 +1,6 @@
# Mailboxes
-For the Akka Classic documentation of this feature see @ref:[Classic Mailboxes](../mailboxes.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Mailboxes](../mailboxes.md).
## Dependency
@@ -9,7 +9,7 @@ page describes how to use mailboxes with `akka-actor-typed`, which has dependenc
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-actor-typed_$scala.binary_version$"
+ artifact="akka-actor-typed_$scala.binary.version$"
version="$akka.version$"
}
diff --git a/akka-docs/src/main/paradox/typed/persistence-snapshot.md b/akka-docs/src/main/paradox/typed/persistence-snapshot.md
index 6bdba2c68c..227609871c 100644
--- a/akka-docs/src/main/paradox/typed/persistence-snapshot.md
+++ b/akka-docs/src/main/paradox/typed/persistence-snapshot.md
@@ -3,7 +3,7 @@ project.description: Append only event logs, snapshots and recovery with Akka ev
---
# Snapshotting
-For the Akka Classic documentation of this feature see @ref:[Classic Akka Persistence](../persistence.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Akka Persistence](../persistence.md).
## Snapshots
diff --git a/akka-docs/src/main/paradox/typed/persistence-testing.md b/akka-docs/src/main/paradox/typed/persistence-testing.md
index 8261e92d3f..3f471dd46a 100644
--- a/akka-docs/src/main/paradox/typed/persistence-testing.md
+++ b/akka-docs/src/main/paradox/typed/persistence-testing.md
@@ -1,49 +1,38 @@
# Testing
-## Dependency
+## Module info
-To use Akka Persistence and Actor TestKit, add the module to your project:
+To use Akka Persistence TestKit, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group1=com.typesafe.akka
- artifact1=akka-persistence-typed_$scala.binary_version$
+ artifact1=akka-persistence-typed_$scala.binary.version$
version1=$akka.version$
group2=com.typesafe.akka
- artifact2=akka-actor-testkit-typed_$scala.binary_version$
+ artifact2=akka-persistence-testkit_$scala.binary.version$
version2=$akka.version$
scope2=test
}
+@@project-info{ projectId="akka-persistence-testkit" }
+
## Unit testing
-Unit testing of `EventSourcedBehavior` can be done with the @ref:[ActorTestKit](testing-async.md)
-in the same way as other behaviors.
+**Note!** The `EventSourcedBehaviorTestKit` is a new feature, api may have changes breaking source compatibility in future versions.
-@ref:[Synchronous behavior testing](testing-sync.md) for `EventSourcedBehavior` is not supported yet, but
-tracked in @github[issue #23712](#23712).
+Unit testing of `EventSourcedBehavior` can be done with the @apidoc[EventSourcedBehaviorTestKit]. It supports running
+one command at a time and you can assert that the synchronously returned result is as expected. The result contains the
+events emitted by the command and the new state after applying the events. It also has support for verifying the reply
+to a command.
-You need to configure a journal, and the in-memory journal is sufficient for unit testing. To enable the
-in-memory journal you need to pass the following configuration to the @scala[`ScalaTestWithActorTestKit`]@java[`TestKitJunitResource`].
+You need to configure the `ActorSystem` with the `EventSourcedBehaviorTestKit.config`. The configuration enables
+the in-memory journal and snapshot storage.
Scala
-: @@snip [AccountExampleDocSpec.scala](/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleDocSpec.scala) { #inmem-config }
+: @@snip [AccountExampleDocSpec.scala](/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleDocSpec.scala) { #testkit }
Java
-: @@snip [AccountExampleDocTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleDocTest.java) { #inmem-config }
-
-The `test-serialization = on` configuration of the `InmemJournal` will verify that persisted events can be serialized and deserialized.
-
-Optionally you can also configure a snapshot store. To enable the file based snapshot store you need to pass the
-following configuration to the @scala[`ScalaTestWithActorTestKit`]@java[`TestKitJunitResource`].
-
-Scala
-: @@snip [AccountExampleDocSpec.scala](/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleDocSpec.scala) { #snapshot-store-config }
-
-Java
-: @@snip [AccountExampleDocTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleDocTest.java) { #snapshot-store-config }
-
-Then you can `spawn` the `EventSourcedBehavior` and verify the outcome of sending commands to the actor using
-the facilities of the @ref:[ActorTestKit](testing-async.md).
+: @@snip [AccountExampleDocTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleDocTest.java) { #testkit }
A full test for the `AccountEntity`, which is shown in the @ref:[Persistence Style Guide](persistence-style.md), may look like this:
@@ -53,28 +42,26 @@ Scala
Java
: @@snip [AccountExampleDocTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleDocTest.java) { #test }
-Note that each test case is using a different `PersistenceId` to not interfere with each other.
+Serialization of commands, events and state are verified automatically. The serialization checks can be
+customized with the `SerializationSettings` when creating the `EventSourcedBehaviorTestKit`. By default,
+the serialization roundtrip is checked but the equality of the result of the serialization is not checked.
+`equals` must be implemented @scala[(or using `case class`)] in the commands, events and state if `verifyEquality`
+is enabled.
-The @apidoc[akka.persistence.journal.inmem.InmemJournal$] publishes `Write` and `Delete` operations to the
-`eventStream`, which makes it possible to verify that the expected events have been emitted and stored by the
-`EventSourcedBehavior`. You can subscribe to to the `eventStream` with a `TestProbe` like this:
-
-Scala
-: @@snip [AccountExampleDocSpec.scala](/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleDocSpec.scala) { #test-events }
-
-Java
-: @@snip [AccountExampleDocTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleDocTest.java) { #test-events }
+To test recovery the `restart` method of the `EventSourcedBehaviorTestKit` can be used. It will restart the
+behavior, which will then recover from stored snapshot and events from previous commands. It's also possible
+to populate the storage with events or simulate failures by using the underlying @apidoc[PersistenceTestKit].
## Persistence TestKit
-**Note!** The testkit is a new feature, api may have changes breaking source compatibility in future versions.
+**Note!** The `PersistenceTestKit` is a new feature, api may have changes breaking source compatibility in future versions.
Persistence testkit allows to check events saved in a storage, emulate storage operations and exceptions.
To use the testkit you need to add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group="com.typesafe.akka"
- artifact="akka-persistence-testkit_$scala.binary_version$"
+ artifact="akka-persistence-testkit_$scala.binary.version$"
version="$akka.version$"
}
@@ -107,10 +94,10 @@ Java
A typical scenario is to create a persistent actor, send commands to it and check that it persists events as it is expected:
Scala
-: @@snip [TestKitExamples.scala](/akka-docs/src/test/scala/docs/persistence/testkit/TestKitExamples.scala) { #testkit-typed-usecase }
+: @@snip [TestKitExamples.scala](/akka-docs/src/test/scala/docs/persistence/testkit/TestKitExamples.scala) { #test }
Java
-: @@snip [TestKitExamples.java](/akka-docs/src/test/java/jdocs/persistence/testkit/TestKitExamples.java) { #testkit-typed-usecase }
+: @@snip [PersistenceTestKitSampleTest.java](/akka-docs/src/test/java/jdocs/persistence/testkit/PersistenceTestKitSampleTest.java) { #test }
You can safely use persistence testkit in combination with main akka testkit.
@@ -134,6 +121,12 @@ Implement the @apidoc[ProcessingPolicy[EventStorage.JournalOperation]] @scala[tr
or @apidoc[ProcessingPolicy[SnapshotStorage.SnapshotOperation]] @scala[trait]@java[interface] for snapshot storage,
and set it with `withPolicy()` method.
+Scala
+: @@snip [TestKitExamples.scala](/akka-docs/src/test/scala/docs/persistence/testkit/TestKitExamples.scala) { #policy-test }
+
+Java
+: @@snip [PersistenceTestKitPolicySampleTest.java](/akka-docs/src/test/java/jdocs/persistence/testkit/PersistenceTestKitPolicySampleTest.java) { #policy-test }
+
`tryProcess()` method of the @apidoc[ProcessingPolicy] has two arguments: persistence id and the storage operation.
Event storage has the following operations:
@@ -181,11 +174,34 @@ to the @ref:[reference configuration](../general/configuration-reference.md#conf
## Integration testing
-The in-memory journal and file based snapshot store can be used also for integration style testing of a single
-`ActorSystem`, for example when using Cluster Sharding with a single Cluster node.
+`EventSourcedBehavior` actors can be tested with the @ref:[ActorTestKit](testing-async.md) together with
+other actors. The in-memory journal and snapshot storage from the @ref:[Persistence TestKit](#persistence-testkit)
+can be used also for integration style testing of a single `ActorSystem`, for example when using Cluster Sharding
+with a single Cluster node.
For tests that involve more than one Cluster node you have to use another journal and snapshot store.
While it's possible to use the @ref:[Persistence Plugin Proxy](../persistence-plugins.md#persistence-plugin-proxy)
it's often better and more realistic to use a real database.
-See [akka-samples issue #128](https://github.com/akka/akka-samples/issues/128).
+The @ref:[CQRS example](../project/examples.md#cqrs) includes tests that are using Akka Persistence Cassandra.
+
+### Plugin initialization
+
+Some Persistence plugins create tables automatically, but has the limitation that it can't be done concurrently
+from several ActorSystems. That can be a problem if the test creates a Cluster and all nodes tries to initialize
+the plugins at the same time. To coordinate initialization you can use the `PersistenceInit` utility.
+
+`PersistenceInit` is part of `akka-persistence-testkit` and you need to add the dependency to your project:
+
+@@dependency[sbt,Maven,Gradle] {
+ group="com.typesafe.akka"
+ artifact="akka-persistence-testkit_$scala.binary.version$"
+ version="$akka.version$"
+}
+
+Scala
+: @@snip [PersistenceInitSpec.scala](/akka-docs/src/test/scala/docs/persistence/testkit/PersistenceInitSpec.scala) { #imports #init }
+
+Java
+: @@snip [PersistenceInitTest.java](/akka-docs/src/test/java/jdocs/persistence/testkit/PersistenceInitTest.java) { #imports #init }
+
diff --git a/akka-docs/src/main/paradox/typed/persistence.md b/akka-docs/src/main/paradox/typed/persistence.md
index 7e0bd2da1f..0b2e2dd5ae 100644
--- a/akka-docs/src/main/paradox/typed/persistence.md
+++ b/akka-docs/src/main/paradox/typed/persistence.md
@@ -3,7 +3,7 @@ project.description: Event Sourcing with Akka Persistence enables actors to pers
---
# Event Sourcing
-For the Akka Classic documentation of this feature see @ref:[Classic Akka Persistence](../persistence.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Akka Persistence](../persistence.md).
## Module info
@@ -11,7 +11,7 @@ To use Akka Persistence, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-persistence-typed_$scala.binary_version$
+ artifact=akka-persistence-typed_$scala.binary.version$
version=$akka.version$
}
@@ -47,7 +47,7 @@ provides tools to facilitate in building GDPR capable systems.
### Event sourcing concepts
-See an [introduction to EventSourcing](https://msdn.microsoft.com/en-us/library/jj591559.aspx) at MSDN.
+See an [introduction to EventSourcing](https://docs.microsoft.com/en-us/previous-versions/msp-n-p/jj591559(v=pandp.10)?redirectedfrom=MSDN) at MSDN.
Another excellent article about "thinking in Events" is [Events As First-Class Citizens](https://hackernoon.com/events-as-first-class-citizens-8633e8479493)
by Randy Shoup. It is a short and recommended read if you're starting developing Events based applications.
@@ -516,6 +516,20 @@ akka.persistence.journal.leveldb.replay-filter {
}
```
+### Disable recovery
+
+You can also completely disable the recovery of events and snapshots:
+
+Scala
+: @@snip [BasicPersistentBehaviorCompileOnly.scala](/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/BasicPersistentBehaviorCompileOnly.scala) { #recovery-disabled }
+
+Java
+: @@snip [BasicPersistentBehaviorTest.java](/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/BasicPersistentBehaviorTest.java) { #recovery-disabled }
+
+Please refer to @ref[snapshots](persistence-snapshot.md#snapshots) if you need to disable only the snapshot recovery, or you need to select specific snapshots.
+
+In any case, the highest sequence number will always be recovered so you can keep persisting new events without corrupting your event log.
+
## Tagging
Persistence allows you to use event tags without using an @ref[`EventAdapter`](../persistence.md#event-adapters):
diff --git a/akka-docs/src/main/paradox/typed/reliable-delivery.md b/akka-docs/src/main/paradox/typed/reliable-delivery.md
index e86d30a43c..3f263ec9a0 100644
--- a/akka-docs/src/main/paradox/typed/reliable-delivery.md
+++ b/akka-docs/src/main/paradox/typed/reliable-delivery.md
@@ -3,7 +3,7 @@ project.description: Reliable delivery and flow control of messages between acto
---
# Reliable delivery
-For the Akka Classic documentation of this feature see @ref:[Classic At-Least-Once Delivery](../persistence.md#at-least-once-delivery).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic At-Least-Once Delivery](../persistence.md#at-least-once-delivery).
@@@ warning
@@ -19,20 +19,20 @@ To use reliable delivery, add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-actor-typed_$scala.binary_version$
+ artifact=akka-actor-typed_$scala.binary.version$
version=$akka.version$
}
## Introduction
-Normal @ref:[message delivery reliability](../general/message-delivery-reliability.md) is at-most once delivery, which
+Normal @ref:[message delivery reliability](../general/message-delivery-reliability.md) is at-most-once delivery, which
means that messages may be lost. That should be rare, but still possible.
-For interactions between some actors that is not acceptable and at-least once delivery or effectively once processing
+For interactions between some actors, that is not acceptable and at-least-once delivery or effectively-once processing
is needed. The tools for reliable delivery described here help with implementing that. It can't be achieved
-automatically under the hood without collaboration from the application because confirming when a message has been
+automatically under the hood without collaboration from the application. This is because confirming when a message has been
fully processed is a business level concern. Only ensuring that it was transferred over the network or delivered to
-the mailbox of the actor would not be enough, since it may crash right after without being processed.
+the mailbox of the actor would not be enough, since the actor may crash right before being able to process the message.
Lost messages are detected, resent and deduplicated as needed. In addition, it also includes flow control for
the sending of messages to avoid that a fast producer overwhelms a slower consumer or sends messages at
@@ -50,10 +50,10 @@ There are 3 supported patterns, which are described in the following sections:
## Point-to-point
-Point-to-point reliable delivery between a single producer actor sending messages and a single consumer actor
+This pattern implements point-to-point reliable delivery between a single producer actor sending messages and a single consumer actor
receiving the messages.
-Messages are sent from the producer to @apidoc[ProducerController] and via @apidoc[ConsumerController] actors, which
+Messages are sent from the producer to @apidoc[ProducerController$] and via @apidoc[ConsumerController$] actors, which
handle the delivery and confirmation of the processing in the destination consumer actor.

@@ -65,33 +65,34 @@ The `ProducerController` sends `RequestNext` to the producer, which is then allo
message to the `ProducerController`. Thereafter the producer will receive a new `RequestNext`
when it's allowed to send one more message.
-The producer and `ProducerController` actors are supposed to be local so that these messages are
-fast and not lost. This is enforced by a runtime check.
+The producer and `ProducerController` actors are required to be local so that message delivery is
+both fast and guaranteed. This requirement is enforced by a runtime check.
Similarly, on the consumer side the destination consumer actor will start the flow by sending an
initial `ConsumerController.Start` message to the `ConsumerController`.
-For the `ProducerController` to know where to send the messages it must be connected with the
-`ConsumerController`. You do this is with `ProducerController.RegisterConsumer` or
-`ConsumerController.RegisterToProducerController` messages. When using the the point-to-point pattern
-it is the application's responsibility to connect them together. For example, by sending the `ActorRef`
-in an ordinary message to the other side, or register the `ActorRef` in the @ref:[Receptionist](actor-discovery.md)
-and find it on the other side.
+For the `ProducerController` to know where to send the messages, it must be connected with the
+`ConsumerController`. This can be done with the `ProducerController.RegisterConsumer` or
+`ConsumerController.RegisterToProducerController` messages. When using the point-to-point pattern,
+it is the application's responsibility to connect them together. For example, this can be done by sending the `ActorRef`
+in an ordinary message to the other side, or by registering the `ActorRef` in the @ref:[Receptionist](actor-discovery.md)
+so it can be found on the other side.
You must also take measures to reconnect them if any of the sides crashes, for example by watching it
for termination.
-Received messages from the producer are wrapped in `ConsumerController.Delivery` when sent to the consumer,
-which is supposed to reply with `ConsumerController.Confirmed` when it has processed the message.
-Next message is not delivered until the previous is confirmed. More messages from the producer that arrive
+Messages sent by the producer are wrapped in `ConsumerController.Delivery` when received by a consumer and the consumer
+should reply with `ConsumerController.Confirmed` when it has processed the message.
+
+The next message is not delivered until the previous one is confirmed. Any messages from the producer that arrive
while waiting for the confirmation are stashed by the `ConsumerController` and delivered when the previous
message is confirmed.
-The consumer and the `ConsumerController` actors are supposed to be local so that these messages are fast
-and not lost. This is enforced by a runtime check.
+Similar to the producer side, the consumer and the `ConsumerController` actors are required to be local so that message delivery is
+both fast and guaranteed. This requirement is enforced by a runtime check.
Many unconfirmed messages can be in flight between the `ProducerController` and `ConsumerController`, but
-it is limited by a flow control window. The flow control is driven by the consumer side, which means that
+their number is limited by a flow control window. The flow control is driven by the consumer side, which means that
the `ProducerController` will not send faster than the demand requested by the `ConsumerController`.
### Point-to-point example
@@ -129,25 +130,25 @@ Java
### Point-to-point delivery semantics
-As long as neither producer nor consumer crash the messages are delivered to the consumer actor in the same order
-as they were sent as they were sent to the `ProducerController`, without loss or duplicates. Meaning effectively
-once processing without any business level deduplication.
+As long as neither producer nor consumer crash, the messages are delivered to the consumer actor in the same order
+as they were sent to the `ProducerController`, without loss or duplicates. This means effectively-once
+processing without any business level deduplication.
-Unconfirmed messages may be lost if the producer crashes. To avoid that you need to enable the @ref:[durable
-queue](#durable-producer) on the producer side. The stored unconfirmed messages will be redelivered when the
-corresponding producer is started again. Even if the same `ConsumerController` instance is used there may be
+Unconfirmed messages may be lost if the producer crashes. To avoid that, you need to enable the @ref:[durable
+queue](#durable-producer) on the producer side. By doing so, any stored unconfirmed messages will be redelivered when the
+corresponding producer is started again. Even if the same `ConsumerController` instance is used, there may be
delivery of messages that had already been processed but the fact that they were confirmed had not been stored yet.
-Meaning at-least once delivery.
+This means that we have at-least-once delivery.
-If the consumer crashes a new `ConsumerController` can be connected to the original `ProducerConsumer`
+If the consumer crashes, a new `ConsumerController` can be connected to the original `ProducerConsumer`
without restarting it. The `ProducerConsumer` will then redeliver all unconfirmed messages. In that case
-the unconfirmed messages will be delivered to the new consumer, and some of these may already have been
+the unconfirmed messages will be delivered to the new consumer and some of these may already have been
processed by the previous consumer.
-Meaning at-least once delivery.
+Again, this means that we have at-least-once delivery.
## Work pulling
-Work pulling is a pattern where several worker actors pull tasks in their own pace from
+Work pulling is a pattern where several worker actors pull tasks at their own pace from
a shared work manager instead of that the manager pushes work to the workers blindly
without knowing their individual capacity and current availability.
@@ -155,7 +156,7 @@ One important property is that the order of the messages should not matter, beca
message is routed randomly to one of the workers with demand. In other words, two subsequent
messages may be routed to two different workers and processed independent of each other.
-Messages are sent from the producer to @apidoc[WorkPullingProducerController] and via @apidoc[ConsumerController]
+Messages are sent from the producer to @apidoc[WorkPullingProducerController$] and via @apidoc[ConsumerController$]
actors, which handle the delivery and confirmation of the processing in the destination worker (consumer) actor.

@@ -230,17 +231,17 @@ For work pulling the order of the messages should not matter, because each messa
to one of the workers with demand and can therefore be processed in any order.
As long as neither producers nor workers crash (or workers being removed for other reasons) the messages are
-delivered to the workers without loss or duplicates. Meaning effectively once processing without any
+delivered to the workers without loss or duplicates. Meaning effectively-once processing without any
business level deduplication.
Unconfirmed messages may be lost if the producer crashes. To avoid that you need to enable the @ref:[durable
queue](#durable-producer) on the producer side. The stored unconfirmed messages will be redelivered when the
corresponding producer is started again. Those messages may be routed to different workers than before
and some of them may have already been processed but the fact that they were confirmed had not been stored
-yet. Meaning at-least once delivery.
+yet. Meaning at-least-once delivery.
If a worker crashes or is stopped gracefully the unconfirmed messages will be redelivered to other workers.
-In that case some of these may already have been processed by the previous worker. Meaning at-least once delivery.
+In that case some of these may already have been processed by the previous worker. Meaning at-least-once delivery.
## Sharding
@@ -248,7 +249,7 @@ To use reliable delivery with Cluster Sharding, add the following module to your
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-cluster-sharding-typed_$scala.binary_version$
+ artifact=akka-cluster-sharding-typed_$scala.binary.version$
version=$akka.version$
}
@@ -265,7 +266,7 @@ and sending from another producer (different node)

-The @apidoc[ShardingProducerController] should be used together with @apidoc[ShardingConsumerController].
+The @apidoc[ShardingProducerController$] should be used together with @apidoc[ShardingConsumerController$].
A producer can send messages via a `ShardingProducerController` to any `ShardingConsumerController`
identified by an `entityId`. A single `ShardingProducerController` per `ActorSystem` (node) can be
@@ -328,13 +329,13 @@ Java
### Sharding delivery semantics
As long as neither producer nor consumer crash the messages are delivered to the consumer actor in the same order
-as they were sent to the `ShardingProducerController`, without loss or duplicates. Meaning effectively once
+as they were sent to the `ShardingProducerController`, without loss or duplicates. Meaning effectively-once
processing without any business level deduplication.
Unconfirmed messages may be lost if the producer crashes. To avoid that you need to enable the @ref:[durable
queue](#durable-producer) on the producer side. The stored unconfirmed messages will be redelivered when the
corresponding producer is started again. In that case there may be delivery of messages that had already been
-processed but the fact that they were confirmed had not been stored yet. Meaning at-least once delivery.
+processed but the fact that they were confirmed had not been stored yet. Meaning at-least-once delivery.
If the consumer crashes or the shard is rebalanced the unconfirmed messages will be redelivered. In that case
some of these may already have been processed by the previous consumer.
@@ -343,7 +344,7 @@ some of these may already have been processed by the previous consumer.
Until sent messages have been confirmed the producer side keeps them in memory to be able to
resend them. If the JVM of the producer side crashes those unconfirmed messages are lost.
-To make sure the messages can be delivered also in that scenario a @apidoc[DurableProducerQueue] can be used.
+To make sure the messages can be delivered also in that scenario a @apidoc[DurableProducerQueue$] can be used.
Then the unconfirmed messages are stored in a durable way so that they can be redelivered when the producer
is started again. An implementation of the `DurableProducerQueue` is provided by @apidoc[EventSourcedProducerQueue]
in `akka-persistence-typed`.
@@ -354,7 +355,7 @@ When using the `EventSourcedProducerQueue` the following dependency is needed:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-persistence-typed_$scala.binary_version$
+ artifact=akka-persistence-typed_$scala.binary.version$
version=$akka.version$
}
diff --git a/akka-docs/src/main/paradox/typed/routers.md b/akka-docs/src/main/paradox/typed/routers.md
index 989d12cb7a..d6302afa30 100644
--- a/akka-docs/src/main/paradox/typed/routers.md
+++ b/akka-docs/src/main/paradox/typed/routers.md
@@ -1,6 +1,6 @@
# Routers
-For the Akka Classic documentation of this feature see @ref:[Classic Routing](../routing.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Routing](../routing.md).
## Dependency
@@ -8,7 +8,7 @@ To use Akka Actor Typed, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-actor-typed_$scala.binary_version$
+ artifact=akka-actor-typed_$scala.binary.version$
version=$akka.version$
}
@@ -73,7 +73,7 @@ Java
## Routing strategies
-There are two different strategies for selecting what routee a message is forwarded to that can be selected
+There are three different strategies for selecting which routee a message is forwarded to that can be selected
from the router before spawning it:
Scala
@@ -105,13 +105,18 @@ An optional parameter `preferLocalRoutees` can be used for this strategy. Router
### Consistent Hashing
-Uses [consistent hashing](http://en.wikipedia.org/wiki/Consistent_hashing) to select a routee based
+Uses [consistent hashing](https://en.wikipedia.org/wiki/Consistent_hashing) to select a routee based
on the sent message. This [article](http://www.tom-e-white.com/2007/11/consistent-hashing.html)
gives good insight into how consistent hashing is implemented.
Currently you have to define hashMapping of the router to map incoming messages to their consistent
hash key. This makes the decision transparent for the sender.
+Consistent hashing makes messages with the same hash routee to the same routee as long as the set of routees stays the same.
+When the set of routees changes, consistent hashing tries to make sure, but does not guarantee, that messages with the same hash are routed to the same routee.
+
+See also @ref[Akka Cluster Sharding](cluster-sharding.md) which provides stable routing and rebalancing of the routee actors.
+
## Routers and performance
Note that if the routees are sharing a resource, the resource will determine if increasing the number of
diff --git a/akka-docs/src/main/paradox/typed/stash.md b/akka-docs/src/main/paradox/typed/stash.md
index 2a7b77f7f8..d43675ffe0 100644
--- a/akka-docs/src/main/paradox/typed/stash.md
+++ b/akka-docs/src/main/paradox/typed/stash.md
@@ -1,6 +1,6 @@
# Stash
-For the Akka Classic documentation of this feature see @ref:[Classic Actors](../actors.md#stash).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Actors](../actors.md#stash).
## Dependency
@@ -8,7 +8,7 @@ To use Akka Actor Typed, you must add the following dependency in your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-actor-typed_$scala.binary_version$
+ artifact=akka-actor-typed_$scala.binary.version$
version=$akka.version$
}
diff --git a/akka-docs/src/main/paradox/typed/style-guide.md b/akka-docs/src/main/paradox/typed/style-guide.md
index 49a600268c..ac42c38477 100644
--- a/akka-docs/src/main/paradox/typed/style-guide.md
+++ b/akka-docs/src/main/paradox/typed/style-guide.md
@@ -446,8 +446,8 @@ be good to know that it's optional in case you would prefer a different approach
* direct processing because there is only one message type
* if or switch statements
* annotation processor
-* [Vavr Pattern Matching DSL](http://www.vavr.io/vavr-docs/#_pattern_matching)
-* future pattern matching in Java ([JEP 305](http://openjdk.java.net/jeps/305))
+* [Vavr Pattern Matching DSL](https://www.vavr.io/vavr-docs/#_pattern_matching)
+* pattern matching since JDK 14 ([JEP 305](https://openjdk.java.net/jeps/305))
In `Behaviors` there are `receive`, `receiveMessage` and `receiveSignal` factory methods that takes functions
instead of using the `ReceiveBuilder`, which is the `receive` with the class parameter.
diff --git a/akka-docs/src/main/paradox/typed/testing-async.md b/akka-docs/src/main/paradox/typed/testing-async.md
index 1213c5efab..d8b24596b6 100644
--- a/akka-docs/src/main/paradox/typed/testing-async.md
+++ b/akka-docs/src/main/paradox/typed/testing-async.md
@@ -1,6 +1,6 @@
## Asynchronous testing
-For the Akka Classic documentation of this feature see @ref:[Classic Testing](../testing.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Testing](../testing.md).
Asynchronous testing uses a real @apidoc[akka.actor.typed.ActorSystem] that allows you to test your Actors in a more realistic environment.
diff --git a/akka-docs/src/main/paradox/typed/testing-sync.md b/akka-docs/src/main/paradox/typed/testing-sync.md
index ae9fb96f68..f5459fcfe3 100644
--- a/akka-docs/src/main/paradox/typed/testing-sync.md
+++ b/akka-docs/src/main/paradox/typed/testing-sync.md
@@ -1,6 +1,6 @@
## Synchronous behavior testing
-For the Akka Classic documentation of this feature see @ref:[Classic Testing](../testing.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Testing](../testing.md).
The `BehaviorTestKit` provides a very nice way of unit testing a `Behavior` in a deterministic way, but it has
some limitations to be aware of.
diff --git a/akka-docs/src/main/paradox/typed/testing.md b/akka-docs/src/main/paradox/typed/testing.md
index 17c2770d49..2e60e08de2 100644
--- a/akka-docs/src/main/paradox/typed/testing.md
+++ b/akka-docs/src/main/paradox/typed/testing.md
@@ -1,6 +1,6 @@
# Testing
-For the Akka Classic documentation of this feature see @ref:[Classic Testing](../testing.md).
+You are viewing the documentation for the new actor APIs, to view the Akka Classic documentation, see @ref:[Classic Testing](../testing.md).
## Module info
@@ -8,7 +8,7 @@ To use Actor TestKit add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
- artifact=akka-actor-testkit-typed_$scala.binary_version$
+ artifact=akka-actor-testkit-typed_$scala.binary.version$
version=$akka.version$
scope=test
}
@@ -19,7 +19,7 @@ We recommend using Akka TestKit with ScalaTest:
@@dependency[sbt,Maven,Gradle] {
group=org.scalatest
- artifact=scalatest_$scala.binary_version$
+ artifact=scalatest_$scala.binary.version$
version=$scalatest.version$
scope=test
}
diff --git a/akka-docs/src/test/java/jdocs/actor/ActorDocTest.java b/akka-docs/src/test/java/jdocs/actor/ActorDocTest.java
index c31b1d191a..ebe238eefc 100644
--- a/akka-docs/src/test/java/jdocs/actor/ActorDocTest.java
+++ b/akka-docs/src/test/java/jdocs/actor/ActorDocTest.java
@@ -847,51 +847,11 @@ public class ActorDocTest extends AbstractJavaTest {
};
}
- private CompletionStage cleanup() {
- return null;
- }
-
- @Test
- public void coordinatedShutdown() {
- final ActorRef someActor = system.actorOf(Props.create(FirstActor.class));
- // #coordinated-shutdown-addTask
- CoordinatedShutdown.get(system)
- .addTask(
- CoordinatedShutdown.PhaseBeforeServiceUnbind(),
- "someTaskName",
- () -> {
- return akka.pattern.Patterns.ask(someActor, "stop", Duration.ofSeconds(5))
- .thenApply(reply -> Done.getInstance());
- });
- // #coordinated-shutdown-addTask
-
- // #coordinated-shutdown-cancellable
- Cancellable cancellable =
- CoordinatedShutdown.get(system)
- .addCancellableTask(
- CoordinatedShutdown.PhaseBeforeServiceUnbind(), "someTaskCleanup", () -> cleanup());
- // much later...
- cancellable.cancel();
- // #coordinated-shutdown-cancellable
-
- // #coordinated-shutdown-jvm-hook
- CoordinatedShutdown.get(system)
- .addJvmShutdownHook(() -> System.out.println("custom JVM shutdown hook..."));
- // #coordinated-shutdown-jvm-hook
-
- // don't run this
- if (false) {
- // #coordinated-shutdown-run
- CompletionStage done =
- CoordinatedShutdown.get(system).runAll(CoordinatedShutdown.unknownReason());
- // #coordinated-shutdown-run
- }
- }
-
@Test
public void coordinatedShutdownActorTermination() {
ActorRef someActor = system.actorOf(Props.create(FirstActor.class));
someActor.tell(PoisonPill.getInstance(), ActorRef.noSender());
+ // https://github.com/akka/akka/issues/29056
// #coordinated-shutdown-addActorTerminationTask
CoordinatedShutdown.get(system)
.addActorTerminationTask(
diff --git a/akka-docs/src/test/java/jdocs/actor/typed/CoordinatedActorShutdownTest.java b/akka-docs/src/test/java/jdocs/actor/typed/CoordinatedActorShutdownTest.java
new file mode 100644
index 0000000000..c4690305eb
--- /dev/null
+++ b/akka-docs/src/test/java/jdocs/actor/typed/CoordinatedActorShutdownTest.java
@@ -0,0 +1,133 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package jdocs.actor.typed;
+
+import akka.Done;
+import akka.actor.Cancellable;
+import akka.actor.CoordinatedShutdown;
+import akka.actor.typed.ActorRef;
+import akka.actor.typed.ActorSystem;
+import akka.actor.typed.Behavior;
+import akka.actor.typed.javadsl.*;
+// #coordinated-shutdown-addTask
+import static akka.actor.typed.javadsl.AskPattern.ask;
+
+// #coordinated-shutdown-addTask
+
+import java.time.Duration;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionStage;
+
+public class CoordinatedActorShutdownTest {
+
+ // #coordinated-shutdown-addTask
+ public static class MyActor extends AbstractBehavior {
+ interface Messages {}
+
+ // ...
+
+ static final class Stop implements Messages {
+ final ActorRef replyTo;
+
+ Stop(ActorRef replyTo) {
+ this.replyTo = replyTo;
+ }
+ }
+ // #coordinated-shutdown-addTask
+
+ public static Behavior create() {
+ return Behaviors.setup(MyActor::new);
+ }
+
+ private MyActor(ActorContext context) {
+ super(context);
+ }
+
+ // #coordinated-shutdown-addTask
+ @Override
+ public Receive createReceive() {
+ return newReceiveBuilder().onMessage(Stop.class, this::stop).build();
+ }
+
+ private Behavior stop(Stop stop) {
+ // shut down the actor internal
+ // ...
+ stop.replyTo.tell(Done.done());
+ return Behaviors.stopped();
+ }
+ }
+
+ // #coordinated-shutdown-addTask
+
+ public static class Root extends AbstractBehavior {
+
+ public static Behavior create() {
+ return Behaviors.setup(
+ context -> {
+ ActorRef myActor = context.spawn(MyActor.create(), "my-actor");
+ ActorSystem system = context.getSystem();
+ // #coordinated-shutdown-addTask
+ CoordinatedShutdown.get(system)
+ .addTask(
+ CoordinatedShutdown.PhaseBeforeServiceUnbind(),
+ "someTaskName",
+ () ->
+ ask(myActor, MyActor.Stop::new, Duration.ofSeconds(5), system.scheduler()));
+ // #coordinated-shutdown-addTask
+ return Behaviors.empty();
+ });
+ }
+
+ private Root(ActorContext context) {
+ super(context);
+ }
+
+ @Override
+ public Receive createReceive() {
+ return newReceiveBuilder().build();
+ }
+ }
+
+ private CompletionStage cleanup() {
+ return CompletableFuture.completedFuture(Done.done());
+ }
+
+ public void mount() {
+ ActorSystem system = ActorSystem.create(Root.create(), "main");
+
+ // #coordinated-shutdown-cancellable
+ Cancellable cancellable =
+ CoordinatedShutdown.get(system)
+ .addCancellableTask(
+ CoordinatedShutdown.PhaseBeforeServiceUnbind(), "someTaskCleanup", () -> cleanup());
+ // much later...
+ cancellable.cancel();
+ // #coordinated-shutdown-cancellable
+
+ // #coordinated-shutdown-jvm-hook
+ CoordinatedShutdown.get(system)
+ .addJvmShutdownHook(() -> System.out.println("custom JVM shutdown hook..."));
+ // #coordinated-shutdown-jvm-hook
+
+ // don't run this
+ if (false) {
+ // #coordinated-shutdown-run
+ // shut down with `ActorSystemTerminateReason`
+ system.terminate();
+
+ // or define a specific reason
+ class UserInitiatedShutdown implements CoordinatedShutdown.Reason {
+ @Override
+ public String toString() {
+ return "UserInitiatedShutdown";
+ }
+ }
+
+ CompletionStage done =
+ CoordinatedShutdown.get(system).runAll(new UserInitiatedShutdown());
+ // #coordinated-shutdown-run
+ }
+ }
+}
diff --git a/akka-coordination/src/test/java/jdocs/akka/coordination/lease/LeaseDocTest.java b/akka-docs/src/test/java/jdocs/coordination/LeaseDocTest.java
similarity index 97%
rename from akka-coordination/src/test/java/jdocs/akka/coordination/lease/LeaseDocTest.java
rename to akka-docs/src/test/java/jdocs/coordination/LeaseDocTest.java
index c1c2d53b22..49ede28cae 100644
--- a/akka-coordination/src/test/java/jdocs/akka/coordination/lease/LeaseDocTest.java
+++ b/akka-docs/src/test/java/jdocs/coordination/LeaseDocTest.java
@@ -2,14 +2,14 @@
* Copyright (C) 2019-2020 Lightbend Inc.
*/
-package jdocs.akka.coordination.lease;
+package jdocs.coordination;
import akka.actor.ActorSystem;
import akka.coordination.lease.LeaseSettings;
import akka.coordination.lease.javadsl.Lease;
import akka.coordination.lease.javadsl.LeaseProvider;
import akka.testkit.javadsl.TestKit;
-import docs.akka.coordination.LeaseDocSpec;
+import docs.coordination.LeaseDocSpec;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
diff --git a/akka-docs/src/test/java/jdocs/persistence/testkit/PersistenceInitTest.java b/akka-docs/src/test/java/jdocs/persistence/testkit/PersistenceInitTest.java
new file mode 100644
index 0000000000..710cb0a35e
--- /dev/null
+++ b/akka-docs/src/test/java/jdocs/persistence/testkit/PersistenceInitTest.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package jdocs.persistence.testkit;
+
+import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
+
+import com.typesafe.config.ConfigFactory;
+import jdocs.AbstractJavaTest;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.UUID;
+
+// #imports
+import akka.persistence.testkit.javadsl.PersistenceInit;
+import akka.Done;
+
+import java.time.Duration;
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.TimeUnit;
+
+// #imports
+
+public class PersistenceInitTest extends AbstractJavaTest {
+ @ClassRule
+ public static final TestKitJunitResource testKit =
+ new TestKitJunitResource(
+ ConfigFactory.parseString(
+ "akka.persistence.journal.plugin = \"akka.persistence.journal.inmem\" \n"
+ + "akka.persistence.journal.inmem.test-serialization = on \n"
+ + "akka.persistence.snapshot-store.plugin = \"akka.persistence.snapshot-store.local\" \n"
+ + "akka.persistence.snapshot-store.local.dir = \"target/snapshot-"
+ + UUID.randomUUID().toString()
+ + "\" \n")
+ .withFallback(ConfigFactory.defaultApplication()));
+
+ @Test
+ public void testInit() throws Exception {
+ // #init
+ Duration timeout = Duration.ofSeconds(5);
+ CompletionStage done =
+ PersistenceInit.initializeDefaultPlugins(testKit.system(), timeout);
+ done.toCompletableFuture().get(timeout.getSeconds(), TimeUnit.SECONDS);
+ // #init
+ }
+}
diff --git a/akka-docs/src/test/java/jdocs/persistence/testkit/PersistenceTestKitPolicySampleTest.java b/akka-docs/src/test/java/jdocs/persistence/testkit/PersistenceTestKitPolicySampleTest.java
new file mode 100644
index 0000000000..2d61d16da7
--- /dev/null
+++ b/akka-docs/src/test/java/jdocs/persistence/testkit/PersistenceTestKitPolicySampleTest.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package jdocs.persistence.testkit;
+
+import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
+import akka.actor.typed.ActorRef;
+import akka.persistence.testkit.JournalOperation;
+import akka.persistence.testkit.PersistenceTestKitPlugin;
+import akka.persistence.testkit.ProcessingPolicy;
+import akka.persistence.testkit.ProcessingResult;
+import akka.persistence.testkit.ProcessingSuccess;
+import akka.persistence.testkit.StorageFailure;
+import akka.persistence.testkit.WriteEvents;
+import akka.persistence.testkit.javadsl.PersistenceTestKit;
+import akka.persistence.typed.PersistenceId;
+import com.typesafe.config.ConfigFactory;
+import jdocs.AbstractJavaTest;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+// #policy-test
+public class PersistenceTestKitPolicySampleTest extends AbstractJavaTest {
+
+ @ClassRule
+ public static final TestKitJunitResource testKit =
+ new TestKitJunitResource(
+ PersistenceTestKitPlugin.getInstance()
+ .config()
+ .withFallback(ConfigFactory.defaultApplication()));
+
+ PersistenceTestKit persistenceTestKit = PersistenceTestKit.create(testKit.system());
+
+ @Before
+ public void beforeEach() {
+ persistenceTestKit.clearAll();
+ persistenceTestKit.resetPolicy();
+ }
+
+ @Test
+ public void test() {
+ SampleEventStoragePolicy policy = new SampleEventStoragePolicy();
+ persistenceTestKit.withPolicy(policy);
+
+ PersistenceId persistenceId = PersistenceId.ofUniqueId("some-id");
+ ActorRef ref =
+ testKit.spawn(YourPersistentBehavior.create(persistenceId));
+
+ YourPersistentBehavior.Cmd cmd = new YourPersistentBehavior.Cmd("data");
+ ref.tell(cmd);
+
+ persistenceTestKit.expectNothingPersisted(persistenceId.id());
+ }
+
+ static class SampleEventStoragePolicy implements ProcessingPolicy {
+ @Override
+ public ProcessingResult tryProcess(String persistenceId, JournalOperation processingUnit) {
+ if (processingUnit instanceof WriteEvents) {
+ return StorageFailure.create();
+ } else {
+ return ProcessingSuccess.getInstance();
+ }
+ }
+ }
+}
+// #policy-test
diff --git a/akka-docs/src/test/java/jdocs/persistence/testkit/PersistenceTestKitSampleTest.java b/akka-docs/src/test/java/jdocs/persistence/testkit/PersistenceTestKitSampleTest.java
new file mode 100644
index 0000000000..ec6c01994b
--- /dev/null
+++ b/akka-docs/src/test/java/jdocs/persistence/testkit/PersistenceTestKitSampleTest.java
@@ -0,0 +1,125 @@
+/*
+ * Copyright (C) 2009-2020 Lightbend Inc.
+ */
+
+package jdocs.persistence.testkit;
+
+import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
+import akka.actor.typed.ActorRef;
+import akka.actor.typed.Behavior;
+import akka.actor.typed.javadsl.Behaviors;
+import akka.persistence.testkit.PersistenceTestKitPlugin;
+import akka.persistence.testkit.javadsl.PersistenceTestKit;
+import akka.persistence.typed.PersistenceId;
+import akka.persistence.typed.javadsl.CommandHandler;
+import akka.persistence.typed.javadsl.EventHandler;
+import akka.persistence.typed.javadsl.EventSourcedBehavior;
+import akka.serialization.jackson.CborSerializable;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.typesafe.config.ConfigFactory;
+import jdocs.AbstractJavaTest;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+// #test
+public class PersistenceTestKitSampleTest extends AbstractJavaTest {
+
+ @ClassRule
+ public static final TestKitJunitResource testKit =
+ new TestKitJunitResource(
+ PersistenceTestKitPlugin.getInstance()
+ .config()
+ .withFallback(ConfigFactory.defaultApplication()));
+
+ PersistenceTestKit persistenceTestKit = PersistenceTestKit.create(testKit.system());
+
+ @Before
+ public void beforeEach() {
+ persistenceTestKit.clearAll();
+ }
+
+ @Test
+ public void test() {
+ PersistenceId persistenceId = PersistenceId.ofUniqueId("some-id");
+ ActorRef ref =
+ testKit.spawn(YourPersistentBehavior.create(persistenceId));
+
+ YourPersistentBehavior.Cmd cmd = new YourPersistentBehavior.Cmd("data");
+ ref.tell(cmd);
+ YourPersistentBehavior.Evt expectedEventPersisted = new YourPersistentBehavior.Evt(cmd.data);
+
+ persistenceTestKit.expectNextPersisted(persistenceId.id(), expectedEventPersisted);
+ }
+}
+
+class YourPersistentBehavior
+ extends EventSourcedBehavior<
+ YourPersistentBehavior.Cmd, YourPersistentBehavior.Evt, YourPersistentBehavior.State> {
+
+ static final class Cmd implements CborSerializable {
+
+ public final String data;
+
+ @JsonCreator
+ public Cmd(String data) {
+ this.data = data;
+ }
+ }
+
+ static final class Evt implements CborSerializable {
+
+ public final String data;
+
+ @JsonCreator
+ public Evt(String data) {
+ this.data = data;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ Evt evt = (Evt) o;
+
+ return data.equals(evt.data);
+ }
+
+ @Override
+ public int hashCode() {
+ return data.hashCode();
+ }
+ }
+
+ static final class State implements CborSerializable {}
+
+ static Behavior create(PersistenceId persistenceId) {
+ return Behaviors.setup(context -> new YourPersistentBehavior(persistenceId));
+ }
+
+ private YourPersistentBehavior(PersistenceId persistenceId) {
+ super(persistenceId);
+ }
+
+ @Override
+ public State emptyState() {
+ // some state
+ return new State();
+ }
+
+ @Override
+ public CommandHandler commandHandler() {
+ return newCommandHandlerBuilder()
+ .forAnyState()
+ .onCommand(Cmd.class, command -> Effect().persist(new Evt(command.data)))
+ .build();
+ }
+
+ @Override
+ public EventHandler eventHandler() {
+ // TODO handle events
+ return newEventHandlerBuilder().forAnyState().onEvent(Evt.class, (state, evt) -> state).build();
+ }
+}
+// #test
diff --git a/akka-docs/src/test/java/jdocs/persistence/testkit/TestKitExamples.java b/akka-docs/src/test/java/jdocs/persistence/testkit/TestKitExamples.java
index 8a78ba0968..dcc6c19b9e 100644
--- a/akka-docs/src/test/java/jdocs/persistence/testkit/TestKitExamples.java
+++ b/akka-docs/src/test/java/jdocs/persistence/testkit/TestKitExamples.java
@@ -4,13 +4,10 @@
package jdocs.persistence.testkit;
-import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
-import akka.actor.typed.ActorRef;
import akka.persistence.testkit.DeleteEvents;
import akka.persistence.testkit.DeleteSnapshotByMeta;
import akka.persistence.testkit.DeleteSnapshotsByCriteria;
import akka.persistence.testkit.JournalOperation;
-import akka.persistence.testkit.PersistenceTestKitPlugin;
import akka.persistence.testkit.ProcessingPolicy;
import akka.persistence.testkit.ProcessingResult;
import akka.persistence.testkit.ProcessingSuccess;
@@ -22,15 +19,6 @@ import akka.persistence.testkit.SnapshotOperation;
import akka.persistence.testkit.StorageFailure;
import akka.persistence.testkit.WriteEvents;
import akka.persistence.testkit.WriteSnapshot;
-import akka.persistence.testkit.javadsl.PersistenceTestKit;
-import akka.persistence.typed.PersistenceId;
-import akka.persistence.typed.javadsl.CommandHandler;
-import akka.persistence.typed.javadsl.EventHandler;
-import akka.persistence.typed.javadsl.EventSourcedBehavior;
-import com.typesafe.config.ConfigFactory;
-import org.junit.Before;
-import org.junit.ClassRule;
-import org.junit.Test;
public class TestKitExamples {
@@ -105,81 +93,3 @@ public class TestKitExamples {
// #set-snapshot-storage-policy
}
-
-// #testkit-typed-usecase
-class SampleTest {
-
- @ClassRule
- public static final TestKitJunitResource testKit =
- new TestKitJunitResource(
- PersistenceTestKitPlugin.getInstance()
- .config()
- .withFallback(ConfigFactory.defaultApplication()));
-
- PersistenceTestKit persistenceTestKit = PersistenceTestKit.create(testKit.system());
-
- @Before
- void beforeAll() {
- persistenceTestKit.clearAll();
- }
-
- @Test
- void test() {
- ActorRef ref =
- testKit.spawn(new YourPersistentBehavior(PersistenceId.ofUniqueId("some-id")));
-
- Cmd cmd = new Cmd("data");
- ref.tell(cmd);
- Evt expectedEventPersisted = new Evt(cmd.data);
-
- persistenceTestKit.expectNextPersisted("your-persistence-id", expectedEventPersisted);
- }
-}
-
-final class Cmd {
-
- public final String data;
-
- public Cmd(String data) {
- this.data = data;
- }
-}
-
-final class Evt {
-
- public final String data;
-
- public Evt(String data) {
- this.data = data;
- }
-}
-
-final class State {}
-
-class YourPersistentBehavior extends EventSourcedBehavior {
-
- public YourPersistentBehavior(PersistenceId persistenceId) {
- super(persistenceId);
- }
-
- @Override
- public State emptyState() {
- // some state
- return new State();
- }
-
- @Override
- public CommandHandler commandHandler() {
- return newCommandHandlerBuilder()
- .forAnyState()
- .onCommand(Cmd.class, command -> Effect().persist(new Evt(command.data)))
- .build();
- }
-
- @Override
- public EventHandler eventHandler() {
- // TODO handle events
- return newEventHandlerBuilder().build();
- }
-}
-// #testkit-typed-usecase
diff --git a/akka-docs/src/test/java/jdocs/stream/operators/flow/Lazy.java b/akka-docs/src/test/java/jdocs/stream/operators/flow/Lazy.java
new file mode 100644
index 0000000000..b503475e5c
--- /dev/null
+++ b/akka-docs/src/test/java/jdocs/stream/operators/flow/Lazy.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package jdocs.stream.operators.flow;
+/*
+ * Copyright (C) 2009-2020 Lightbend Inc.
+ */
+
+import akka.NotUsed;
+import akka.actor.ActorSystem;
+import akka.japi.Pair;
+import akka.stream.javadsl.Flow;
+import akka.stream.javadsl.RunnableGraph;
+import akka.stream.javadsl.Sink;
+import akka.stream.javadsl.Source;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletionStage;
+
+public class Lazy {
+ private ActorSystem system = null;
+
+ void example() {
+ // #simple-example
+ Source numbers =
+ Source.unfold(
+ 0,
+ n -> {
+ int next = n + 1;
+ System.out.println("Source producing " + next);
+ return Optional.of(Pair.create(next, next));
+ })
+ .take(3);
+
+ Flow> flow =
+ Flow.lazyFlow(
+ () -> {
+ System.out.println("Creating the actual flow");
+ return Flow.fromFunction(
+ element -> {
+ System.out.println("Actual flow mapped " + element);
+ return element;
+ });
+ });
+
+ numbers.via(flow).run(system);
+ // prints:
+ // Source producing 1
+ // Creating the actual flow
+ // Actual flow mapped 1
+ // Source producing 2
+ // Actual flow mapped 2
+ // #simple-example
+ }
+
+ void statefulMap() {
+ // #mutable-example
+ Flow, CompletionStage> mutableFold =
+ Flow.lazyFlow(
+ () -> {
+ List zero = new ArrayList<>();
+
+ return Flow.of(Integer.class)
+ .fold(
+ zero,
+ (list, element) -> {
+ list.add(element);
+ return list;
+ });
+ });
+
+ RunnableGraph stream =
+ Source.range(1, 3).via(mutableFold).to(Sink.foreach(System.out::println));
+
+ stream.run(system);
+ stream.run(system);
+ stream.run(system);
+ // prints:
+ // [1, 2, 3]
+ // [1, 2, 3]
+ // [1, 2, 3]
+ // #mutable-example
+ }
+}
diff --git a/akka-docs/src/test/java/jdocs/stream/operators/sink/Lazy.java b/akka-docs/src/test/java/jdocs/stream/operators/sink/Lazy.java
new file mode 100644
index 0000000000..f3b30f0f17
--- /dev/null
+++ b/akka-docs/src/test/java/jdocs/stream/operators/sink/Lazy.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package jdocs.stream.operators.sink;
+/*
+ * Copyright (C) 2009-2020 Lightbend Inc.
+ */
+
+import akka.actor.ActorSystem;
+import akka.stream.javadsl.Keep;
+import akka.stream.javadsl.Sink;
+import akka.stream.javadsl.Source;
+
+import java.util.Optional;
+import java.util.concurrent.CompletionStage;
+
+public class Lazy {
+
+ private ActorSystem system = null;
+
+ void example() {
+ // #simple-example
+ CompletionStage> matVal =
+ Source.maybe()
+ .map(
+ element -> {
+ System.out.println("mapped " + element);
+ return element;
+ })
+ .toMat(
+ Sink.lazySink(
+ () -> {
+ System.out.println("Sink created");
+ return Sink.foreach(elem -> System.out.println("foreach " + elem));
+ }),
+ Keep.left())
+ .run(system);
+
+ // some time passes
+ // nothing has been printed
+ matVal.toCompletableFuture().complete(Optional.of("one"));
+ // now prints:
+ // mapped one
+ // Sink created
+ // foreach one
+
+ // #simple-example
+ }
+}
diff --git a/akka-docs/src/test/java/jdocs/stream/operators/source/From.java b/akka-docs/src/test/java/jdocs/stream/operators/source/From.java
index b1dbef700f..ee917d5dcb 100644
--- a/akka-docs/src/test/java/jdocs/stream/operators/source/From.java
+++ b/akka-docs/src/test/java/jdocs/stream/operators/source/From.java
@@ -8,6 +8,7 @@ import akka.actor.ActorSystem;
import akka.stream.javadsl.Source;
import java.util.Arrays;
+import java.util.stream.IntStream;
public class From {
@@ -23,4 +24,15 @@ public class From {
// 3
// #from-iterator
}
+
+ void fromJavaStreamSample() {
+ // #from-javaStream
+ Source.fromJavaStream(() -> IntStream.rangeClosed(1, 3))
+ .runForeach(System.out::println, system);
+ // could print
+ // 1
+ // 2
+ // 3
+ // #from-javaStream
+ }
}
diff --git a/akka-docs/src/test/java/jdocs/stream/operators/source/Lazy.java b/akka-docs/src/test/java/jdocs/stream/operators/source/Lazy.java
new file mode 100644
index 0000000000..b8695b102f
--- /dev/null
+++ b/akka-docs/src/test/java/jdocs/stream/operators/source/Lazy.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright (C) 2009-2020 Lightbend Inc.
+ */
+
+package jdocs.stream.operators.source;
+
+import akka.Done;
+import akka.NotUsed;
+import akka.actor.ActorSystem;
+import akka.japi.Pair;
+import akka.stream.javadsl.RunnableGraph;
+import akka.stream.javadsl.Sink;
+import akka.stream.javadsl.SinkQueueWithCancel;
+import akka.stream.javadsl.Source;
+
+import java.util.Optional;
+import java.util.concurrent.CompletionStage;
+
+public class Lazy {
+
+ private ActorSystem system = null;
+
+ private Source createExpensiveSource() {
+ throw new UnsupportedOperationException("Not implemented in sample");
+ }
+
+ void notReallyThatLazy() {
+ // #not-a-good-example
+ Source> source =
+ Source.lazySource(
+ () -> {
+ System.out.println("Creating the actual source");
+ return createExpensiveSource();
+ });
+
+ SinkQueueWithCancel queue = source.runWith(Sink.queue(), system);
+
+ // ... time passes ...
+ // at some point in time we pull the first time
+ // but the source creation may already have been triggered
+ queue.pull();
+ // #not-a-good-example
+ }
+
+ static class IteratorLikeThing {
+ boolean thereAreMore() {
+ throw new UnsupportedOperationException("Not implemented in sample");
+ }
+
+ String extractNext() {
+ throw new UnsupportedOperationException("Not implemented in sample");
+ }
+ }
+
+ void safeMutableSource() {
+ // #one-per-materialization
+ RunnableGraph> stream =
+ Source.lazySource(
+ () -> {
+ IteratorLikeThing instance = new IteratorLikeThing();
+ return Source.unfold(
+ instance,
+ sameInstance -> {
+ if (sameInstance.thereAreMore())
+ return Optional.of(Pair.create(sameInstance, sameInstance.extractNext()));
+ else return Optional.empty();
+ });
+ })
+ .to(Sink.foreach(System.out::println));
+
+ // each of the three materializations will have their own instance of IteratorLikeThing
+ stream.run(system);
+ stream.run(system);
+ stream.run(system);
+ // #one-per-materialization
+ }
+}
diff --git a/akka-docs/src/test/java/jdocs/stream/operators/sourceorflow/ExtrapolateAndExpand.java b/akka-docs/src/test/java/jdocs/stream/operators/sourceorflow/ExtrapolateAndExpand.java
new file mode 100644
index 0000000000..4593d5ded3
--- /dev/null
+++ b/akka-docs/src/test/java/jdocs/stream/operators/sourceorflow/ExtrapolateAndExpand.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package jdocs.stream.operators.sourceorflow;
+
+import akka.NotUsed;
+import akka.actor.ActorSystem;
+import akka.actor.Cancellable;
+import akka.japi.Pair;
+import akka.japi.function.Function;
+import akka.stream.javadsl.Flow;
+import akka.stream.javadsl.Sink;
+import akka.stream.javadsl.Source;
+import akka.util.ByteString;
+import docs.stream.operators.sourceorflow.ExtrapolateAndExpandCommon;
+import docs.stream.operators.sourceorflow.ExtrapolateAndExpandCommon.Frame;
+
+import java.time.Duration;
+import java.util.stream.Stream;
+
+/** */
+public class ExtrapolateAndExpand {
+ public static Function decodeAsFrame =
+ ExtrapolateAndExpandCommon.Frame$.MODULE$::decode;
+
+ public static Frame BLACK_FRAME = ExtrapolateAndExpandCommon.Frame$.MODULE$.blackFrame();
+
+ public static long nowInSeconds() {
+ return ExtrapolateAndExpand.nowInSeconds();
+ }
+
+ public static void main(String[] args) {
+ ActorSystem actorSystem = ActorSystem.create("25fps-stream");
+
+ Source networkSource = ExtrapolateAndExpandCommon.networkSource().asJava();
+
+ Flow decode = Flow.of(ByteString.class).map(decodeAsFrame);
+
+ // #extrapolate
+ // if upstream is too slow, produce copies of the last frame but grayed out.
+ Flow rateControl =
+ Flow.of(Frame.class)
+ .extrapolate(
+ lastFrame -> {
+ Frame gray =
+ new Frame(
+ ByteString.fromString(
+ "gray frame!! - " + lastFrame.pixels().utf8String()));
+ return Stream.iterate(gray, i -> i).iterator();
+ },
+ BLACK_FRAME // initial value
+ );
+
+ Source videoSource = networkSource.via(decode).via(rateControl);
+
+ // let's create a 25fps stream (a Frame every 40.millis)
+ Source tickSource =
+ Source.tick(Duration.ZERO, Duration.ofMillis(40), "tick");
+
+ Source videoAt25Fps = tickSource.zip(videoSource).map(Pair::second);
+
+ // #extrapolate
+
+ // #expand
+ // each element flowing through the stream is expanded to a watermark copy
+ // of the upstream frame and grayed out copies. The grayed out copies should
+ // only be used downstream if the producer is too slow.
+ Flow watermarkerRateControl =
+ Flow.of(Frame.class)
+ .expand(
+ lastFrame -> {
+ Frame watermarked =
+ new Frame(
+ lastFrame.pixels().$plus$plus(ByteString.fromString(" - watermark")));
+ Frame gray =
+ new Frame(lastFrame.pixels().$plus$plus(ByteString.fromString(" - gray")));
+ return Stream.concat(Stream.of(watermarked), Stream.iterate(gray, i -> i))
+ .iterator();
+ });
+
+ Source watermakedVideoSource =
+ networkSource.via(decode).via(watermarkerRateControl);
+
+ // let's create a 25fps stream (a Frame every 40.millis)
+ Source ticks = Source.tick(Duration.ZERO, Duration.ofMillis(40), "tick");
+
+ Source watermarkedVideoAt25Fps =
+ ticks.zip(watermakedVideoSource).map(Pair::second);
+
+ // #expand
+ videoAt25Fps
+ .map(Frame::pixels)
+ .map(ByteString::utf8String)
+ .map(pixels -> nowInSeconds() + " - " + pixels)
+ .to(Sink.foreach(System.out::println))
+ .run(actorSystem);
+ }
+}
diff --git a/akka-docs/src/test/java/jdocs/stream/operators/sourceorflow/FlatMapConcat.java b/akka-docs/src/test/java/jdocs/stream/operators/sourceorflow/FlatMapConcat.java
new file mode 100644
index 0000000000..b0ebadeaf6
--- /dev/null
+++ b/akka-docs/src/test/java/jdocs/stream/operators/sourceorflow/FlatMapConcat.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright (C) 2019-2020 Lightbend Inc.
+ */
+
+package jdocs.stream.operators.sourceorflow;
+
+import akka.NotUsed;
+import akka.actor.ActorSystem;
+import akka.stream.javadsl.Source;
+
+import java.util.Arrays;
+
+public class FlatMapConcat {
+ private static ActorSystem system = null;
+
+ // #flatmap-concat
+ // e.g. could be a query to a database
+ private Source lookupCustomerEvents(String customerId) {
+ return Source.from(Arrays.asList(customerId + "-event-1", customerId + "-event-2"));
+ }
+ // #flatmap-concat
+
+ void example() {
+ // #flatmap-concat
+ Source.from(Arrays.asList("customer-1", "customer-2"))
+ .flatMapConcat(this::lookupCustomerEvents)
+ .runForeach(System.out::println, system);
+ // prints - events from each customer consecutively
+ // customer-1-event-1
+ // customer-1-event-2
+ // customer-2-event-1
+ // customer-2-event-2
+ // #flatmap-concat
+ }
+}
diff --git a/akka-docs/src/test/java/jdocs/stream/operators/sourceorflow/MergeLatest.java b/akka-docs/src/test/java/jdocs/stream/operators/sourceorflow/MergeLatest.java
new file mode 100644
index 0000000000..41ca921ce2
--- /dev/null
+++ b/akka-docs/src/test/java/jdocs/stream/operators/sourceorflow/MergeLatest.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package jdocs.stream.operators.sourceorflow;
+
+import akka.NotUsed;
+import akka.actor.typed.ActorSystem;
+import akka.stream.javadsl.Source;
+
+import java.util.Arrays;
+
+public class MergeLatest {
+
+ private static final ActorSystem system = null;
+
+ public static void example() {
+ // #mergeLatest
+ Source prices = Source.from(Arrays.asList(100, 101, 99, 103));
+ Source quantities = Source.from(Arrays.asList(1, 3, 4, 2));
+
+ prices
+ .mergeLatest(quantities, true)
+ .map(priceAndQuantity -> priceAndQuantity.get(0) * priceAndQuantity.get(1))
+ .runForeach(System.out::println, system);
+
+ // prints something like:
+ // 100
+ // 101
+ // 303
+ // 297
+ // 396
+ // 412
+ // 206
+ // #mergeLatest
+ }
+}
diff --git a/akka-docs/src/test/resources/logback-test.xml b/akka-docs/src/test/resources/logback-test.xml
index 5e0b801edc..90f202b250 100644
--- a/akka-docs/src/test/resources/logback-test.xml
+++ b/akka-docs/src/test/resources/logback-test.xml
@@ -8,7 +8,7 @@
INFO
- %date{ISO8601} %-5level %logger %marker - %msg {%mdc}%n
+ %date{ISO8601} %-5level %logger %marker - %msg MDC: {%mdc}%n
diff --git a/akka-docs/src/test/scala/docs/actor/ActorDocSpec.scala b/akka-docs/src/test/scala/docs/actor/ActorDocSpec.scala
index 49f8176e9b..8362b3f01f 100644
--- a/akka-docs/src/test/scala/docs/actor/ActorDocSpec.scala
+++ b/akka-docs/src/test/scala/docs/actor/ActorDocSpec.scala
@@ -724,34 +724,8 @@ class ActorDocSpec extends AkkaSpec("""
}
"using CoordinatedShutdown" in {
- val someActor = system.actorOf(Props(classOf[Replier], this))
- //#coordinated-shutdown-addTask
- CoordinatedShutdown(system).addTask(CoordinatedShutdown.PhaseBeforeServiceUnbind, "someTaskName") { () =>
- import akka.pattern.ask
- import system.dispatcher
- implicit val timeout = Timeout(5.seconds)
- (someActor ? "stop").map(_ => Done)
- }
- //#coordinated-shutdown-addTask
-
- {
- def cleanup(): Unit = {}
- import system.dispatcher
- //#coordinated-shutdown-cancellable
- val c = CoordinatedShutdown(system).addCancellableTask(CoordinatedShutdown.PhaseBeforeServiceUnbind, "cleanup") {
- () =>
- Future {
- cleanup()
- Done
- }
- }
-
- // much later...
- c.cancel()
- //#coordinated-shutdown-cancellable
- }
-
- {
+ // other snippets moved to docs.actor.typed.CoordinatedActorShutdownSpec
+ { // https://github.com/akka/akka/issues/29056
val someActor = system.actorOf(Props(classOf[Replier], this))
someActor ! PoisonPill
//#coordinated-shutdown-addActorTerminationTask
@@ -762,19 +736,6 @@ class ActorDocSpec extends AkkaSpec("""
Some("stop"))
//#coordinated-shutdown-addActorTerminationTask
}
-
- //#coordinated-shutdown-jvm-hook
- CoordinatedShutdown(system).addJvmShutdownHook {
- println("custom JVM shutdown hook...")
- }
- //#coordinated-shutdown-jvm-hook
-
- // don't run this
- def dummy(): Unit = {
- //#coordinated-shutdown-run
- val done: Future[Done] = CoordinatedShutdown(system).run(CoordinatedShutdown.UnknownReason)
- //#coordinated-shutdown-run
- }
}
}
diff --git a/akka-docs/src/test/scala/docs/actor/typed/CoordinatedActorShutdownSpec.scala b/akka-docs/src/test/scala/docs/actor/typed/CoordinatedActorShutdownSpec.scala
new file mode 100644
index 0000000000..c8728c27ed
--- /dev/null
+++ b/akka-docs/src/test/scala/docs/actor/typed/CoordinatedActorShutdownSpec.scala
@@ -0,0 +1,91 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package docs.actor.typed
+
+import akka.Done
+import akka.actor.{ Cancellable, CoordinatedShutdown }
+import akka.actor.typed.{ ActorRef, ActorSystem, Behavior }
+import akka.actor.typed.scaladsl.Behaviors
+import akka.actor.typed.scaladsl.AskPattern._
+import akka.util.Timeout
+
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
+class CoordinatedActorShutdownSpec {
+
+ //#coordinated-shutdown-addTask
+ object MyActor {
+
+ trait Messages
+ case class Stop(replyTo: ActorRef[Done]) extends Messages
+
+ def behavior: Behavior[Messages] =
+ Behaviors.receiveMessage {
+ // ...
+ case Stop(replyTo) =>
+ // shut down the actor internals
+ // ..
+ replyTo.tell(Done)
+ Behaviors.stopped
+ }
+ }
+
+ //#coordinated-shutdown-addTask
+
+ trait Message
+
+ def root: Behavior[Message] = Behaviors.setup[Message] { context =>
+ implicit val system = context.system
+ val myActor = context.spawn(MyActor.behavior, "my-actor")
+ //#coordinated-shutdown-addTask
+ CoordinatedShutdown(context.system).addTask(CoordinatedShutdown.PhaseBeforeServiceUnbind, "someTaskName") { () =>
+ implicit val timeout = Timeout(5.seconds)
+ myActor.ask(MyActor.Stop)
+ }
+ //#coordinated-shutdown-addTask
+
+ Behaviors.empty
+
+ }
+
+ def showCancel: Unit = {
+ val system = ActorSystem(root, "main")
+
+ def cleanup(): Unit = {}
+ import system.executionContext
+ //#coordinated-shutdown-cancellable
+ val c: Cancellable =
+ CoordinatedShutdown(system).addCancellableTask(CoordinatedShutdown.PhaseBeforeServiceUnbind, "cleanup") { () =>
+ Future {
+ cleanup()
+ Done
+ }
+ }
+
+ // much later...
+ c.cancel()
+ //#coordinated-shutdown-cancellable
+
+ //#coordinated-shutdown-jvm-hook
+ CoordinatedShutdown(system).addJvmShutdownHook {
+ println("custom JVM shutdown hook...")
+ }
+ //#coordinated-shutdown-jvm-hook
+
+ // don't run this
+ def dummy(): Unit = {
+ //#coordinated-shutdown-run
+ // shut down with `ActorSystemTerminateReason`
+ system.terminate()
+
+ // or define a specific reason
+ case object UserInitiatedShutdown extends CoordinatedShutdown.Reason
+
+ val done: Future[Done] = CoordinatedShutdown(system).run(UserInitiatedShutdown)
+ //#coordinated-shutdown-run
+ }
+ }
+}
diff --git a/akka-coordination/src/test/scala/docs/akka/coordination/LeaseDocSpec.scala b/akka-docs/src/test/scala/docs/coordination/LeaseDocSpec.scala
similarity index 90%
rename from akka-coordination/src/test/scala/docs/akka/coordination/LeaseDocSpec.scala
rename to akka-docs/src/test/scala/docs/coordination/LeaseDocSpec.scala
index 4e6fa168f4..f1f2116081 100644
--- a/akka-coordination/src/test/scala/docs/akka/coordination/LeaseDocSpec.scala
+++ b/akka-docs/src/test/scala/docs/coordination/LeaseDocSpec.scala
@@ -2,15 +2,17 @@
* Copyright (C) 2019-2020 Lightbend Inc.
*/
-package docs.akka.coordination
+package docs.coordination
+
+import scala.concurrent.Future
+
+import com.typesafe.config.ConfigFactory
import akka.cluster.Cluster
import akka.coordination.lease.LeaseSettings
-import akka.coordination.lease.scaladsl.{ Lease, LeaseProvider }
+import akka.coordination.lease.scaladsl.Lease
+import akka.coordination.lease.scaladsl.LeaseProvider
import akka.testkit.AkkaSpec
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.Future
//#lease-example
class SampleLease(settings: LeaseSettings) extends Lease(settings) {
@@ -37,11 +39,11 @@ object LeaseDocSpec {
def config() =
ConfigFactory.parseString("""
- jdocs-lease.lease-class = "jdocs.akka.coordination.lease.LeaseDocTest$SampleLease"
+ jdocs-lease.lease-class = "jdocs.coordination.LeaseDocTest$SampleLease"
#lease-config
akka.actor.provider = cluster
docs-lease {
- lease-class = "docs.akka.coordination.SampleLease"
+ lease-class = "docs.coordination.SampleLease"
heartbeat-timeout = 100s
heartbeat-interval = 1s
lease-operation-timeout = 1s
diff --git a/akka-docs/src/test/scala/docs/persistence/testkit/PersistenceInitSpec.scala b/akka-docs/src/test/scala/docs/persistence/testkit/PersistenceInitSpec.scala
new file mode 100644
index 0000000000..24227b270e
--- /dev/null
+++ b/akka-docs/src/test/scala/docs/persistence/testkit/PersistenceInitSpec.scala
@@ -0,0 +1,38 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package docs.persistence.testkit
+
+import akka.Done
+import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
+import org.scalatest.wordspec.AnyWordSpecLike
+
+//#imports
+import akka.persistence.testkit.scaladsl.PersistenceInit
+
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
+//#imports
+
+class PersistenceInitSpec
+ extends ScalaTestWithActorTestKit(
+ """
+ akka.persistence.journal.plugin = "akka.persistence.journal.inmem"
+ akka.persistence.snapshot-store.plugin = "akka.persistence.snapshot-store.local"
+ akka.persistence.snapshot-store.local.dir = "target/snapshot-${UUID.randomUUID().toString}"
+ """)
+ with AnyWordSpecLike {
+
+ "PersistenceInit" should {
+ "initialize plugins" in {
+ //#init
+ val timeout = 5.seconds
+ val done: Future[Done] = PersistenceInit.initializeDefaultPlugins(system, timeout)
+ Await.result(done, timeout)
+ //#init
+ }
+ }
+}
diff --git a/akka-docs/src/test/scala/docs/persistence/testkit/TestKitExamples.scala b/akka-docs/src/test/scala/docs/persistence/testkit/TestKitExamples.scala
index 8d443e757e..a10029da9a 100644
--- a/akka-docs/src/test/scala/docs/persistence/testkit/TestKitExamples.scala
+++ b/akka-docs/src/test/scala/docs/persistence/testkit/TestKitExamples.scala
@@ -4,104 +4,160 @@
package docs.persistence.testkit
-import akka.actor.typed.ActorSystem
-import akka.persistence.testkit._
-import akka.persistence.testkit.scaladsl.PersistenceTestKit
-import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior }
+import akka.persistence.typed.PersistenceId
+import akka.persistence.typed.scaladsl.Effect
+import akka.persistence.typed.scaladsl.EventSourcedBehavior
+import akka.serialization.jackson.CborSerializable
import com.typesafe.config.ConfigFactory
-import org.scalatest.BeforeAndAfterAll
+import docs.persistence.testkit.PersistenceTestKitSampleSpec.{ Cmd, Evt, _ }
+import org.scalatest.BeforeAndAfterEach
import org.scalatest.wordspec.AnyWordSpecLike
-class TestKitExamples {
+object PersistenceTestKitSampleSpec {
+ final case class Cmd(data: String) extends CborSerializable
+ final case class Evt(data: String) extends CborSerializable
+ object State {
+ val empty: State = new State
+ }
+ final class State extends CborSerializable {
+ def updated(event: Evt): State = this
+ }
+}
- //#testkit-typed-usecase
- class TypedSampleSpec extends AnyWordSpecLike with BeforeAndAfterAll {
+//#test
+import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
+import akka.persistence.testkit.PersistenceTestKitPlugin
+import akka.persistence.testkit.scaladsl.PersistenceTestKit
- val system: ActorSystem[Cmd] = ActorSystem(
- EventSourcedBehavior[Cmd, Evt, State](
- persistenceId = ???,
- eventHandler = ???,
- commandHandler = (_, cmd) => Effect.persist(Evt(cmd.data)),
- emptyState = ???),
- "name",
- PersistenceTestKitPlugin.config.withFallback(ConfigFactory.defaultApplication()))
- val persistenceTestKit = PersistenceTestKit(system)
+class PersistenceTestKitSampleSpec
+ extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config.withFallback(ConfigFactory.defaultApplication()))
+ with AnyWordSpecLike
+ with BeforeAndAfterEach {
- override def beforeAll(): Unit =
- persistenceTestKit.clearAll()
+ val persistenceTestKit = PersistenceTestKit(system)
- "Persistent actor" should {
+ override def beforeEach(): Unit = {
+ persistenceTestKit.clearAll()
+ }
- "persist all events" in {
+ "Persistent actor" should {
- val persistentActor = system
- val cmd = Cmd("data")
+ "persist all events" in {
- persistentActor ! cmd
+ val persistenceId = PersistenceId.ofUniqueId("your-persistence-id")
+ val persistentActor = spawn(
+ EventSourcedBehavior[Cmd, Evt, State](
+ persistenceId,
+ emptyState = State.empty,
+ commandHandler = (_, cmd) => Effect.persist(Evt(cmd.data)),
+ eventHandler = (state, evt) => state.updated(evt)))
+ val cmd = Cmd("data")
- val expectedPersistedEvent = Evt(cmd.data)
- persistenceTestKit.expectNextPersisted("your-persistence-id", expectedPersistedEvent)
+ persistentActor ! cmd
+
+ val expectedPersistedEvent = Evt(cmd.data)
+ persistenceTestKit.expectNextPersisted(persistenceId.id, expectedPersistedEvent)
+ }
+
+ }
+}
+//#test
+
+//#set-event-storage-policy
+import akka.persistence.testkit._
+
+class SampleEventStoragePolicy extends EventStorage.JournalPolicies.PolicyType {
+
+ //you can use internal state, it does not need to be thread safe
+ var count = 1
+
+ override def tryProcess(persistenceId: String, processingUnit: JournalOperation): ProcessingResult =
+ if (count < 10) {
+ count += 1
+ //check the type of operation and react with success or with reject or with failure.
+ //if you return ProcessingSuccess the operation will be performed, otherwise not.
+ processingUnit match {
+ case ReadEvents(batch) if batch.nonEmpty => ProcessingSuccess
+ case WriteEvents(batch) if batch.size > 1 =>
+ ProcessingSuccess
+ case ReadSeqNum => StorageFailure()
+ case DeleteEvents(_) => Reject()
+ case _ => StorageFailure()
}
+ } else {
+ ProcessingSuccess
+ }
+
+}
+//#set-event-storage-policy
+
+//#set-snapshot-storage-policy
+class SampleSnapshotStoragePolicy extends SnapshotStorage.SnapshotPolicies.PolicyType {
+
+ //you can use internal state, it does not need to be thread safe
+ var count = 1
+
+ override def tryProcess(persistenceId: String, processingUnit: SnapshotOperation): ProcessingResult =
+ if (count < 10) {
+ count += 1
+ //check the type of operation and react with success or with reject or with failure.
+ //if you return ProcessingSuccess the operation will be performed, otherwise not.
+ processingUnit match {
+ case ReadSnapshot(_, payload) if payload.nonEmpty =>
+ ProcessingSuccess
+ case WriteSnapshot(meta, payload) if meta.sequenceNr > 10 =>
+ ProcessingSuccess
+ case DeleteSnapshotsByCriteria(_) => StorageFailure()
+ case DeleteSnapshotByMeta(meta) if meta.sequenceNr < 10 =>
+ ProcessingSuccess
+ case _ => StorageFailure()
+ }
+ } else {
+ ProcessingSuccess
+ }
+}
+//#set-snapshot-storage-policy
+
+//#policy-test
+class PersistenceTestKitSampleSpecWithPolicy
+ extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config.withFallback(ConfigFactory.defaultApplication()))
+ with AnyWordSpecLike
+ with BeforeAndAfterEach {
+
+ val persistenceTestKit = PersistenceTestKit(system)
+
+ override def beforeEach(): Unit = {
+ persistenceTestKit.clearAll()
+ persistenceTestKit.resetPolicy()
+ }
+
+ "Testkit policy" should {
+
+ "fail all operations with custom exception" in {
+ val policy = new EventStorage.JournalPolicies.PolicyType {
+
+ class CustomFailure extends RuntimeException
+
+ override def tryProcess(persistenceId: String, processingUnit: JournalOperation): ProcessingResult =
+ processingUnit match {
+ case WriteEvents(_) => StorageFailure(new CustomFailure)
+ case _ => ProcessingSuccess
+ }
+ }
+ persistenceTestKit.withPolicy(policy)
+
+ val persistenceId = PersistenceId.ofUniqueId("your-persistence-id")
+ val persistentActor = spawn(
+ EventSourcedBehavior[Cmd, Evt, State](
+ persistenceId,
+ emptyState = State.empty,
+ commandHandler = (_, cmd) => Effect.persist(Evt(cmd.data)),
+ eventHandler = (state, evt) => state.updated(evt)))
+
+ persistentActor ! Cmd("data")
+ persistenceTestKit.expectNothingPersisted(persistenceId.id)
}
}
- //#testkit-typed-usecase
-
- //#set-event-storage-policy
- class SampleEventStoragePolicy extends EventStorage.JournalPolicies.PolicyType {
-
- //you can use internal state, it does not need to be thread safe
- var count = 1
-
- override def tryProcess(persistenceId: String, processingUnit: JournalOperation): ProcessingResult =
- if (count < 10) {
- count += 1
- //check the type of operation and react with success or with reject or with failure.
- //if you return ProcessingSuccess the operation will be performed, otherwise not.
- processingUnit match {
- case ReadEvents(batch) if batch.nonEmpty => ProcessingSuccess
- case WriteEvents(batch) if batch.size > 1 =>
- ProcessingSuccess
- case ReadSeqNum => StorageFailure()
- case DeleteEvents(_) => Reject()
- case _ => StorageFailure()
- }
- } else {
- ProcessingSuccess
- }
-
- }
- //#set-event-storage-policy
-
- //#set-snapshot-storage-policy
- class SampleSnapshotStoragePolicy extends SnapshotStorage.SnapshotPolicies.PolicyType {
-
- //you can use internal state, it does not need to be thread safe
- var count = 1
-
- override def tryProcess(persistenceId: String, processingUnit: SnapshotOperation): ProcessingResult =
- if (count < 10) {
- count += 1
- //check the type of operation and react with success or with reject or with failure.
- //if you return ProcessingSuccess the operation will be performed, otherwise not.
- processingUnit match {
- case ReadSnapshot(_, payload) if payload.nonEmpty =>
- ProcessingSuccess
- case WriteSnapshot(meta, payload) if meta.sequenceNr > 10 =>
- ProcessingSuccess
- case DeleteSnapshotsByCriteria(_) => StorageFailure()
- case DeleteSnapshotByMeta(meta) if meta.sequenceNr < 10 =>
- ProcessingSuccess
- case _ => StorageFailure()
- }
- } else {
- ProcessingSuccess
- }
- }
- //#set-snapshot-storage-policy
-
}
-
-case class Cmd(data: String)
-case class Evt(data: String)
-trait State
+//#policy-test
diff --git a/akka-docs/src/test/scala/docs/serialization/SerializationDocSpec.scala b/akka-docs/src/test/scala/docs/serialization/SerializationDocSpec.scala
index 2a7a90ab2f..81c304af47 100644
--- a/akka-docs/src/test/scala/docs/serialization/SerializationDocSpec.scala
+++ b/akka-docs/src/test/scala/docs/serialization/SerializationDocSpec.scala
@@ -109,6 +109,21 @@ package docs.serialization {
*/
trait JsonSerializable
+ object SerializerIdConfig {
+ val config =
+ """
+ #//#serialization-identifiers-config
+ akka {
+ actor {
+ serialization-identifiers {
+ "docs.serialization.MyOwnSerializer" = 1234567
+ }
+ }
+ }
+ #//#serialization-identifiers-config
+ """
+ }
+
class SerializationDocSpec extends AkkaSpec {
"demonstrate configuration of serialize messages" in {
val config = ConfigFactory.parseString("""
diff --git a/akka-docs/src/test/scala/docs/stream/FlowDocSpec.scala b/akka-docs/src/test/scala/docs/stream/FlowDocSpec.scala
index 9f28d49e2d..8e7e0feb25 100644
--- a/akka-docs/src/test/scala/docs/stream/FlowDocSpec.scala
+++ b/akka-docs/src/test/scala/docs/stream/FlowDocSpec.scala
@@ -156,7 +156,7 @@ class FlowDocSpec extends AkkaSpec with CompileOnlySpec {
})
//#flow-mat-combine
- // An source that can be signalled explicitly from the outside
+ // A source that can be signalled explicitly from the outside
val source: Source[Int, Promise[Option[Int]]] = Source.maybe[Int]
// A flow that internally throttles elements to 1/second, and returns a Cancellable
diff --git a/akka-docs/src/test/scala/docs/stream/IntegrationDocSpec.scala b/akka-docs/src/test/scala/docs/stream/IntegrationDocSpec.scala
index f55a52fe85..98265be633 100644
--- a/akka-docs/src/test/scala/docs/stream/IntegrationDocSpec.scala
+++ b/akka-docs/src/test/scala/docs/stream/IntegrationDocSpec.scala
@@ -85,7 +85,7 @@ object IntegrationDocSpec {
}
final case class Save(tweet: Tweet)
- final case object SaveDone
+ case object SaveDone
class DatabaseService(probe: ActorRef) extends Actor {
override def receive = {
diff --git a/akka-docs/src/test/scala/docs/stream/operators/flow/Lazy.scala b/akka-docs/src/test/scala/docs/stream/operators/flow/Lazy.scala
new file mode 100644
index 0000000000..a619b03418
--- /dev/null
+++ b/akka-docs/src/test/scala/docs/stream/operators/flow/Lazy.scala
@@ -0,0 +1,69 @@
+/*
+ * Copyright (C) 2009-2020 Lightbend Inc.
+ */
+
+package docs.stream.operators.flow
+
+import java.util
+
+import akka.actor.ActorSystem
+import akka.stream.scaladsl.Flow
+import akka.stream.scaladsl.Sink
+import akka.stream.scaladsl.Source
+
+object Lazy {
+
+ implicit val system: ActorSystem = ???
+
+ def example(): Unit = {
+ // #simple-example
+ val numbers = Source
+ .unfold(0) { n =>
+ val next = n + 1
+ println(s"Source producing $next")
+ Some((next, next))
+ }
+ .take(3)
+
+ val flow = Flow.lazyFlow { () =>
+ println("Creating the actual flow")
+ Flow[Int].map { element =>
+ println(s"Actual flow mapped $element")
+ element
+ }
+ }
+
+ numbers.via(flow).run()
+ // prints:
+ // Source producing 1
+ // Creating the actual flow
+ // Actual flow mapped 1
+ // Source producing 2
+ // Actual flow mapped 2
+ // #simple-example
+ }
+
+ def statefulMap(): Unit = {
+ // #mutable-example
+ val mutableFold = Flow.lazyFlow { () =>
+ val zero = new util.ArrayList[Int]()
+ Flow[Int].fold(zero) { (list, element) =>
+ list.add(element)
+ list
+ }
+ }
+ val stream =
+ Source(1 to 3).via(mutableFold).to(Sink.foreach(println))
+
+ stream.run()
+ stream.run()
+ stream.run()
+ // prints:
+ // [1, 2, 3]
+ // [1, 2, 3]
+ // [1, 2, 3]
+
+ // #mutable-example
+ }
+
+}
diff --git a/akka-docs/src/test/scala/docs/stream/operators/sink/Lazy.scala b/akka-docs/src/test/scala/docs/stream/operators/sink/Lazy.scala
new file mode 100644
index 0000000000..6587325cbc
--- /dev/null
+++ b/akka-docs/src/test/scala/docs/stream/operators/sink/Lazy.scala
@@ -0,0 +1,41 @@
+/*
+ * Copyright (C) 2009-2020 Lightbend Inc.
+ */
+
+package docs.stream.operators.sink
+
+import akka.actor.ActorSystem
+import akka.stream.scaladsl.Keep
+import akka.stream.scaladsl.Sink
+import akka.stream.scaladsl.Source
+
+object Lazy {
+
+ implicit val system: ActorSystem = ???
+
+ def example(): Unit = {
+ // #simple-example
+ val matVal =
+ Source
+ .maybe[String]
+ .map { element =>
+ println(s"mapped $element")
+ element
+ }
+ .toMat(Sink.lazySink { () =>
+ println("Sink created")
+ Sink.foreach(elem => println(s"foreach $elem"))
+ })(Keep.left)
+ .run()
+
+ // some time passes
+ // nothing has been printed
+ matVal.success(Some("one"))
+ // now prints:
+ // mapped one
+ // Sink created
+ // foreach one
+
+ // #simple-example
+ }
+}
diff --git a/akka-docs/src/test/scala/docs/stream/operators/source/From.scala b/akka-docs/src/test/scala/docs/stream/operators/source/From.scala
index 7d034a75d6..be29fe5573 100644
--- a/akka-docs/src/test/scala/docs/stream/operators/source/From.scala
+++ b/akka-docs/src/test/scala/docs/stream/operators/source/From.scala
@@ -4,6 +4,8 @@
package docs.stream.operators.source
+import java.util.stream.IntStream
+
import akka.actor.ActorSystem
import akka.stream.scaladsl.Source
@@ -21,4 +23,14 @@ object From {
//#from-iterator
}
+ def fromJavaStreamSample(): Unit = {
+ //#from-javaStream
+ Source.fromJavaStream(() => IntStream.rangeClosed(1, 3)).runForeach(println)
+ // could print
+ // 1
+ // 2
+ // 3
+ //#from-javaStream
+ }
+
}
diff --git a/akka-docs/src/test/scala/docs/stream/operators/source/Lazy.scala b/akka-docs/src/test/scala/docs/stream/operators/source/Lazy.scala
new file mode 100644
index 0000000000..568ce3829e
--- /dev/null
+++ b/akka-docs/src/test/scala/docs/stream/operators/source/Lazy.scala
@@ -0,0 +1,56 @@
+/*
+ * Copyright (C) 2009-2020 Lightbend Inc.
+ */
+
+package docs.stream.operators.source
+
+import akka.NotUsed
+import akka.actor.ActorSystem
+import akka.stream.scaladsl.Sink
+import akka.stream.scaladsl.Source
+
+object Lazy {
+
+ implicit val system: ActorSystem = ???
+
+ def createExpensiveSource(): Source[String, NotUsed] = ???
+
+ def notReallyThatLazy(): Unit = {
+ // #not-a-good-example
+ val source = Source.lazySource { () =>
+ println("Creating the actual source")
+ createExpensiveSource()
+ }
+
+ val queue = source.runWith(Sink.queue())
+
+ // ... time passes ...
+ // at some point in time we pull the first time
+ // but the source creation may already have been triggered
+ queue.pull()
+ // #not-a-good-example
+ }
+
+ class IteratorLikeThing {
+ def thereAreMore: Boolean = ???
+ def extractNext: String = ???
+ }
+ def safeMutableSource(): Unit = {
+ // #one-per-materialization
+ val stream = Source
+ .lazySource { () =>
+ val iteratorLike = new IteratorLikeThing
+ Source.unfold(iteratorLike) { iteratorLike =>
+ if (iteratorLike.thereAreMore) Some((iteratorLike, iteratorLike.extractNext))
+ else None
+ }
+ }
+ .to(Sink.foreach(println))
+
+ // each of the three materializations will have their own instance of IteratorLikeThing
+ stream.run()
+ stream.run()
+ stream.run()
+ // #one-per-materialization
+ }
+}
diff --git a/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/ExtrapolateAndExpand.scala b/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/ExtrapolateAndExpand.scala
new file mode 100644
index 0000000000..59fc31e473
--- /dev/null
+++ b/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/ExtrapolateAndExpand.scala
@@ -0,0 +1,123 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package docs.stream.operators.sourceorflow
+
+import akka.NotUsed
+import akka.actor.ActorSystem
+import akka.actor.Cancellable
+import akka.stream.DelayOverflowStrategy
+import akka.stream.scaladsl.DelayStrategy
+import akka.stream.scaladsl.Flow
+import akka.stream.scaladsl.Sink
+import akka.stream.scaladsl.Source
+import akka.util.ByteString
+import docs.stream.operators.sourceorflow.ExtrapolateAndExpand.fps
+import docs.stream.operators.sourceorflow.ExtrapolateAndExpand.nowInSeconds
+import docs.stream.operators.sourceorflow.ExtrapolateAndExpand.periodInMillis
+import docs.stream.operators.sourceorflow.ExtrapolateAndExpand.videoAt25Fps
+
+import scala.concurrent.duration._
+import scala.util.Random
+
+/**
+ *
+ */
+object ExtrapolateAndExpandMain extends App {
+ implicit val sys = ActorSystem("25fps-stream")
+ videoAt25Fps.map(_.pixels.utf8String).map(frame => s"$nowInSeconds - $frame").to(Sink.foreach(println)).run()
+
+}
+object ExtrapolateAndExpand {
+
+ val periodInMillis = 40
+ val fps = 1000 / periodInMillis
+
+ import ExtrapolateAndExpandCommon._
+
+ val decode: Flow[ByteString, Frame, NotUsed] =
+ Flow[ByteString].map(Frame.decode)
+
+ // #extrapolate
+ // if upstream is too slow, produce copies of the last frame but grayed out.
+ val rateControl: Flow[Frame, Frame, NotUsed] =
+ Flow[Frame].extrapolate((frame: Frame) => {
+ val grayedOut = frame.withFilter(Gray)
+ Iterator.continually(grayedOut)
+ }, Some(Frame.blackFrame))
+
+ val videoSource: Source[Frame, NotUsed] = networkSource.via(decode).via(rateControl)
+
+ // let's create a 25fps stream (a Frame every 40.millis)
+ val tickSource: Source[Tick.type, Cancellable] = Source.tick(0.seconds, 40.millis, Tick)
+
+ val videoAt25Fps: Source[Frame, Cancellable] =
+ tickSource.zip(videoSource).map(_._2)
+ // #extrapolate
+
+ // #expand
+ // each element flowing through the stream is expanded to a watermark copy
+ // of the upstream frame and grayed out copies. The grayed out copies should
+ // only be used downstream if the producer is too slow.
+ val watermarkerRateControl: Flow[Frame, Frame, NotUsed] =
+ Flow[Frame].expand((frame: Frame) => {
+ val watermarked = frame.withFilter(Watermark)
+ val grayedOut = frame.withFilter(Gray)
+ (Iterator.single(watermarked) ++ Iterator.continually(grayedOut))
+ })
+
+ val watermarkedVideoSource: Source[Frame, NotUsed] =
+ networkSource.via(decode).via(rateControl)
+
+ // let's create a 25fps stream (a Frame every 40.millis)
+ val ticks: Source[Tick.type, Cancellable] = Source.tick(0.seconds, 40.millis, Tick)
+
+ val watermarkedVideoAt25Fps: Source[Frame, Cancellable] =
+ ticks.zip(watermarkedVideoSource).map(_._2)
+
+ // #expand
+
+ def nowInSeconds = System.nanoTime() / 1000000000
+}
+
+object ExtrapolateAndExpandCommon {
+ // This `networkSource` simulates a client sending frames over the network. There's a
+ // stage throttling the elements at 24fps and then a `delayWith` that randomly delays
+ // frames simulating network latency and bandwidth limitations (uses buffer of
+ // default capacity).
+ val networkSource: Source[ByteString, NotUsed] =
+ Source
+ .fromIterator(() => Iterator.from(0)) // produce frameIds
+ .throttle(fps, 1.second)
+ .map(i => ByteString.fromString(s"fakeFrame-$i"))
+ .delayWith(
+ () =>
+ new DelayStrategy[ByteString] {
+ override def nextDelay(elem: ByteString): FiniteDuration =
+ Random.nextInt(periodInMillis * 10).millis
+ },
+ DelayOverflowStrategy.dropBuffer)
+
+ case object Tick
+
+ sealed trait Filter {
+ def filter(fr: Frame): Frame
+ }
+ object Gray extends Filter {
+ override def filter(fr: Frame): Frame =
+ Frame(ByteString.fromString(s"gray frame!! - ${fr.pixels.utf8String}"))
+ }
+ object Watermark extends Filter {
+ override def filter(fr: Frame): Frame =
+ Frame(fr.pixels.++(ByteString.fromString(" - watermark")))
+ }
+
+ case class Frame(pixels: ByteString) {
+ def withFilter(f: Filter): Frame = f.filter(this)
+ }
+ object Frame {
+ val blackFrame: Frame = Frame(ByteString.empty)
+ def decode(bs: ByteString): Frame = Frame(bs)
+ }
+}
diff --git a/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/FlatMapConcat.scala b/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/FlatMapConcat.scala
new file mode 100644
index 0000000000..0438ed4713
--- /dev/null
+++ b/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/FlatMapConcat.scala
@@ -0,0 +1,31 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package docs.stream.operators.sourceorflow
+import akka.NotUsed
+import akka.actor.ActorSystem
+import akka.stream.scaladsl.Source
+
+object FlatMapConcat {
+
+ implicit val system: ActorSystem = ActorSystem()
+
+ // #flatmap-concat
+ val source: Source[String, NotUsed] = Source(List("customer-1", "customer-2"))
+
+ // e.g. could b a query to a database
+ def lookupCustomerEvents(customerId: String): Source[String, NotUsed] = {
+ Source(List(s"$customerId-event-1", s"$customerId-event-2"))
+ }
+
+ source.flatMapConcat(customerId => lookupCustomerEvents(customerId)).runForeach(println)
+
+ // prints - events from each customer consecutively
+ // customer-1-event-1
+ // customer-1-event-2
+ // customer-2-event-1
+ // customer-2-event-2
+ // #flatmap-concat
+
+}
diff --git a/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/MergeLatest.scala b/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/MergeLatest.scala
new file mode 100644
index 0000000000..7da08732b7
--- /dev/null
+++ b/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/MergeLatest.scala
@@ -0,0 +1,32 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package docs.stream.operators.sourceorflow
+import akka.actor.ActorSystem
+import akka.stream.scaladsl.Source
+
+object MergeLatest extends App {
+ implicit val system = ActorSystem()
+
+ //#mergeLatest
+ val prices = Source(List(100, 101, 99, 103))
+ val quantity = Source(List(1, 3, 4, 2))
+
+ prices
+ .mergeLatest(quantity)
+ .map {
+ case price :: quantity :: Nil => price * quantity
+ }
+ .runForeach(println)
+
+ // prints something like:
+ // 100
+ // 101
+ // 303
+ // 297
+ // 396
+ // 412
+ // 206
+ //#mergeLatest
+}
diff --git a/akka-multi-node-testkit/src/main/java/akka/remote/testconductor/TestConductorProtocol.java b/akka-multi-node-testkit/src/main/java/akka/remote/testconductor/TestConductorProtocol.java
index ddda2348b8..b9825acb94 100644
--- a/akka-multi-node-testkit/src/main/java/akka/remote/testconductor/TestConductorProtocol.java
+++ b/akka-multi-node-testkit/src/main/java/akka/remote/testconductor/TestConductorProtocol.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2019-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
diff --git a/akka-multi-node-testkit/src/main/mima-filters/2.6.4.backwards.excludes/RemoteConnection-shutdown.excludes b/akka-multi-node-testkit/src/main/mima-filters/2.6.4.backwards.excludes/RemoteConnection-shutdown.excludes
new file mode 100644
index 0000000000..f3000e0786
--- /dev/null
+++ b/akka-multi-node-testkit/src/main/mima-filters/2.6.4.backwards.excludes/RemoteConnection-shutdown.excludes
@@ -0,0 +1 @@
+ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.remote.testconductor.RemoteConnection.shutdown")
diff --git a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Conductor.scala b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Conductor.scala
index 1f2c62af1f..310fab7bb6 100644
--- a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Conductor.scala
+++ b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Conductor.scala
@@ -4,8 +4,27 @@
package akka.remote.testconductor
-import language.postfixOps
+import java.net.InetSocketAddress
+import java.util.concurrent.ConcurrentHashMap
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.reflect.classTag
+import scala.util.control.NoStackTrace
+
+import RemoteConnection.getAddrString
+import language.postfixOps
+import org.jboss.netty.channel.{
+ Channel,
+ ChannelHandlerContext,
+ ChannelStateEvent,
+ MessageEvent,
+ SimpleChannelUpstreamHandler
+}
+
+import akka.AkkaException
+import akka.ConfigurationException
import akka.actor.{
Actor,
ActorRef,
@@ -19,28 +38,11 @@ import akka.actor.{
Status,
SupervisorStrategy
}
-import akka.AkkaException
-import akka.ConfigurationException
-import akka.event.LoggingReceive
import akka.event.{ Logging, LoggingAdapter }
+import akka.event.LoggingReceive
import akka.pattern.ask
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.util.Timeout
-import java.net.InetSocketAddress
-import java.util.concurrent.ConcurrentHashMap
-import org.jboss.netty.channel.{
- Channel,
- ChannelHandlerContext,
- ChannelStateEvent,
- MessageEvent,
- SimpleChannelUpstreamHandler
-}
-import RemoteConnection.getAddrString
-import scala.concurrent.Await
-import scala.concurrent.duration._
-import scala.concurrent.Future
-import scala.reflect.classTag
-import scala.util.control.NoStackTrace
/**
* The conductor is the one orchestrating the test: it governs the
@@ -277,7 +279,7 @@ trait Conductor { this: TestConductorExt =>
private[akka] class ConductorHandler(_createTimeout: Timeout, controller: ActorRef, log: LoggingAdapter)
extends SimpleChannelUpstreamHandler {
- implicit val createTimeout = _createTimeout
+ implicit val createTimeout: Timeout = _createTimeout
val clients = new ConcurrentHashMap[Channel, ActorRef]()
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
@@ -337,8 +339,8 @@ private[akka] object ServerFSM {
private[akka] class ServerFSM(val controller: ActorRef, val channel: Channel)
extends Actor
with LoggingFSM[ServerFSM.State, Option[ActorRef]] {
- import ServerFSM._
import Controller._
+ import ServerFSM._
var roleName: RoleName = null
@@ -368,7 +370,7 @@ private[akka] class ServerFSM(val controller: ActorRef, val channel: Channel)
stop()
case Event(ToClient(msg), _) =>
log.warning("cannot send {} in state Initial", msg)
- stay
+ stay()
case Event(StateTimeout, _) =>
log.info("closing channel to {} because of Hello timeout", getAddrString(channel))
channel.close()
@@ -378,22 +380,22 @@ private[akka] class ServerFSM(val controller: ActorRef, val channel: Channel)
when(Ready) {
case Event(d: Done, Some(s)) =>
s ! d
- stay.using(None)
+ stay().using(None)
case Event(op: ServerOp, _) =>
controller ! op
- stay
+ stay()
case Event(msg: NetworkOp, _) =>
log.warning("client {} sent unsupported message {}", getAddrString(channel), msg)
stop()
case Event(ToClient(msg: UnconfirmedClientOp), _) =>
channel.write(msg)
- stay
+ stay()
case Event(ToClient(msg), None) =>
channel.write(msg)
- stay.using(Some(sender()))
+ stay().using(Some(sender()))
case Event(ToClient(msg), _) =>
log.warning("cannot send {} while waiting for previous ACK", msg)
- stay
+ stay()
}
initialize()
@@ -420,8 +422,8 @@ private[akka] object Controller {
* INTERNAL API.
*/
private[akka] class Controller(private var initialParticipants: Int, controllerPort: InetSocketAddress) extends Actor {
- import Controller._
import BarrierCoordinator._
+ import Controller._
val settings = TestConductor().Settings
val connection = RemoteConnection(
@@ -451,7 +453,7 @@ private[akka] class Controller(private var initialParticipants: Int, controllerP
SupervisorStrategy.Restart
}
- val barrier = context.actorOf(Props[BarrierCoordinator], "barriers")
+ val barrier = context.actorOf(Props[BarrierCoordinator](), "barriers")
var nodes = Map[RoleName, NodeInfo]()
// map keeping unanswered queries for node addresses (enqueued upon GetAddress, serviced upon NodeInfo)
@@ -463,7 +465,7 @@ private[akka] class Controller(private var initialParticipants: Int, controllerP
val (ip, port) = channel.getRemoteAddress match {
case s: InetSocketAddress => (s.getAddress.getHostAddress, s.getPort)
}
- val name = ip + ":" + port + "-server" + generation.next
+ val name = ip + ":" + port + "-server" + generation.next()
sender() ! context.actorOf(Props(classOf[ServerFSM], self, channel).withDeploy(Deploy.local), name)
case c @ NodeInfo(name, address, fsm) =>
barrier.forward(c)
@@ -593,13 +595,13 @@ private[akka] class BarrierCoordinator
whenUnhandled {
case Event(n: NodeInfo, d @ Data(clients, _, _, _)) =>
if (clients.find(_.name == n.name).isDefined) throw new DuplicateNode(d, n)
- stay.using(d.copy(clients = clients + n))
+ stay().using(d.copy(clients = clients + n))
case Event(ClientDisconnected(name), d @ Data(clients, _, arrived, _)) =>
if (arrived.isEmpty)
- stay.using(d.copy(clients = clients.filterNot(_.name == name)))
+ stay().using(d.copy(clients = clients.filterNot(_.name == name)))
else {
clients.find(_.name == name) match {
- case None => stay
+ case None => stay()
case Some(c) => throw ClientLost(d.copy(clients = clients - c, arrived = arrived.filterNot(_ == c.fsm)), name)
}
}
@@ -608,17 +610,17 @@ private[akka] class BarrierCoordinator
when(Idle) {
case Event(EnterBarrier(name, timeout), d @ Data(clients, _, _, _)) =>
if (failed)
- stay.replying(ToClient(BarrierResult(name, false)))
+ stay().replying(ToClient(BarrierResult(name, false)))
else if (clients.map(_.fsm) == Set(sender()))
- stay.replying(ToClient(BarrierResult(name, true)))
+ stay().replying(ToClient(BarrierResult(name, true)))
else if (clients.find(_.fsm == sender()).isEmpty)
- stay.replying(ToClient(BarrierResult(name, false)))
+ stay().replying(ToClient(BarrierResult(name, false)))
else {
goto(Waiting).using(d.copy(barrier = name, arrived = sender() :: Nil, deadline = getDeadline(timeout)))
}
case Event(RemoveClient(name), d @ Data(clients, _, _, _)) =>
if (clients.isEmpty) throw BarrierEmpty(d, "cannot remove " + name + ": no client to remove")
- stay.using(d.copy(clients = clients.filterNot(_.name == name)))
+ stay().using(d.copy(clients = clients.filterNot(_.name == name)))
}
onTransition {
@@ -639,7 +641,7 @@ private[akka] class BarrierCoordinator
handleBarrier(d.copy(arrived = together))
case Event(RemoveClient(name), d @ Data(clients, _, arrived, _)) =>
clients.find(_.name == name) match {
- case None => stay
+ case None => stay()
case Some(client) =>
handleBarrier(d.copy(clients = clients - client, arrived = arrived.filterNot(_ == client.fsm)))
}
@@ -660,7 +662,7 @@ private[akka] class BarrierCoordinator
data.arrived.foreach(_ ! ToClient(BarrierResult(data.barrier, true)))
goto(Idle).using(data.copy(barrier = "", arrived = Nil))
} else {
- stay.using(data)
+ stay().using(data)
}
}
diff --git a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/DataTypes.scala b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/DataTypes.scala
index e243008237..0537efaaa2 100644
--- a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/DataTypes.scala
+++ b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/DataTypes.scala
@@ -4,15 +4,16 @@
package akka.remote.testconductor
-import language.implicitConversions
-
-import org.jboss.netty.handler.codec.oneone.OneToOneEncoder
-import org.jboss.netty.channel.ChannelHandlerContext
-import org.jboss.netty.channel.Channel
-import akka.remote.testconductor.{ TestConductorProtocol => TCP }
-import akka.actor.Address
-import org.jboss.netty.handler.codec.oneone.OneToOneDecoder
import scala.concurrent.duration._
+
+import language.implicitConversions
+import org.jboss.netty.channel.Channel
+import org.jboss.netty.channel.ChannelHandlerContext
+import org.jboss.netty.handler.codec.oneone.OneToOneDecoder
+import org.jboss.netty.handler.codec.oneone.OneToOneEncoder
+
+import akka.actor.Address
+import akka.remote.testconductor.{ TestConductorProtocol => TCP }
import akka.remote.testconductor.TestConductorProtocol.BarrierOp
import akka.remote.transport.ThrottlerTransportAdapter.Direction
diff --git a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Extension.scala b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Extension.scala
index fa890120b3..8f4aeb7e74 100644
--- a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Extension.scala
+++ b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Extension.scala
@@ -4,12 +4,13 @@
package akka.remote.testconductor
-import akka.actor.ClassicActorSystemProvider
+import com.typesafe.config.Config
+
import akka.actor.{ ActorContext, ActorSystem, ExtendedActorSystem, Extension, ExtensionId, ExtensionIdProvider }
+import akka.actor.ClassicActorSystemProvider
+import akka.dispatch.ThreadPoolConfig
import akka.remote.RemoteActorRefProvider
import akka.util.Timeout
-import com.typesafe.config.Config
-import akka.dispatch.ThreadPoolConfig
/**
* Access to the [[akka.remote.testconductor.TestConductorExt]] extension:
@@ -62,8 +63,8 @@ class TestConductorExt(val system: ExtendedActorSystem) extends Extension with C
val ClientReconnects = config.getInt("client-reconnects")
val ReconnectBackoff = config.getMillisDuration("reconnect-backoff")
- implicit val BarrierTimeout = Timeout(config.getMillisDuration("barrier-timeout"))
- implicit val QueryTimeout = Timeout(config.getMillisDuration("query-timeout"))
+ implicit val BarrierTimeout: Timeout = Timeout(config.getMillisDuration("barrier-timeout"))
+ implicit val QueryTimeout: Timeout = Timeout(config.getMillisDuration("query-timeout"))
val PacketSplitThreshold = config.getMillisDuration("packet-split-threshold")
private def computeWPS(config: Config): Int =
diff --git a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Player.scala b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Player.scala
index 003cdc4898..a8f466c81e 100644
--- a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Player.scala
+++ b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Player.scala
@@ -4,17 +4,16 @@
package akka.remote.testconductor
+import java.net.{ ConnectException, InetSocketAddress }
import java.util.concurrent.TimeoutException
-import akka.actor._
-import akka.remote.testconductor.RemoteConnection.getAddrString
-
import scala.collection.immutable
import scala.concurrent.{ Await, ExecutionContext, Future }
import scala.concurrent.duration._
-import scala.util.control.NoStackTrace
import scala.reflect.classTag
-import akka.util.Timeout
+import scala.util.control.NoStackTrace
+import scala.util.control.NonFatal
+
import org.jboss.netty.channel.{
Channel,
ChannelHandlerContext,
@@ -24,23 +23,23 @@ import org.jboss.netty.channel.{
SimpleChannelUpstreamHandler,
WriteCompletionEvent
}
-import akka.pattern.{ ask, AskTimeoutException }
-import akka.event.{ Logging, LoggingAdapter }
-import java.net.{ ConnectException, InetSocketAddress }
-import akka.remote.transport.ThrottlerTransportAdapter.{ Blackhole, SetThrottle, TokenBucket, Unthrottled }
+import akka.actor._
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
+import akka.event.{ Logging, LoggingAdapter }
+import akka.pattern.{ ask, AskTimeoutException }
+import akka.remote.testconductor.RemoteConnection.getAddrString
+import akka.remote.transport.ThrottlerTransportAdapter.{ Blackhole, SetThrottle, TokenBucket, Unthrottled }
+import akka.util.Timeout
import akka.util.ccompat._
-import scala.util.control.NonFatal
-
@ccompatUsedUntil213
object Player {
final class Waiter extends Actor with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
- import FSM._
import ClientFSM._
+ import FSM._
var waiting: ActorRef = _
@@ -60,7 +59,7 @@ object Player {
}
- def waiterProps = Props[Waiter]
+ def waiterProps = Props[Waiter]()
}
/**
@@ -191,7 +190,7 @@ private[akka] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress)
when(Connecting, stateTimeout = settings.ConnectTimeout) {
case Event(_: ClientOp, _) =>
- stay.replying(Status.Failure(new IllegalStateException("not connected yet")))
+ stay().replying(Status.Failure(new IllegalStateException("not connected yet")))
case Event(Connected(channel), _) =>
channel.write(Hello(name.name, TestConductor().address))
goto(AwaitDone).using(Data(Some(channel), None))
@@ -211,7 +210,7 @@ private[akka] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress)
log.error("received {} instead of Done", msg)
goto(Failed)
case Event(_: ServerOp, _) =>
- stay.replying(Status.Failure(new IllegalStateException("not connected yet")))
+ stay().replying(Status.Failure(new IllegalStateException("not connected yet")))
case Event(StateTimeout, _) =>
log.error("connect timeout to TestConductor")
goto(Failed)
@@ -223,7 +222,7 @@ private[akka] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress)
throw new ConnectionFailure("disconnect")
case Event(ToServer(_: Done), Data(Some(channel), _)) =>
channel.write(Done)
- stay
+ stay()
case Event(ToServer(msg), d @ Data(Some(channel), None)) =>
channel.write(msg)
val token = msg match {
@@ -231,10 +230,10 @@ private[akka] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress)
case GetAddress(node) => Some(node.name -> sender())
case _ => None
}
- stay.using(d.copy(runningOp = token))
+ stay().using(d.copy(runningOp = token))
case Event(ToServer(op), Data(_, Some((token, _)))) =>
log.error("cannot write {} while waiting for {}", op, token)
- stay
+ stay()
case Event(op: ClientOp, d @ Data(Some(channel @ _), runningOp)) =>
op match {
case BarrierResult(b, success) =>
@@ -249,13 +248,13 @@ private[akka] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress)
case None =>
log.warning("did not expect {}", op)
}
- stay.using(d.copy(runningOp = None))
+ stay().using(d.copy(runningOp = None))
case AddressReply(_, address) =>
runningOp match {
case Some((_, requester)) => requester ! address
case None => log.warning("did not expect {}", op)
}
- stay.using(d.copy(runningOp = None))
+ stay().using(d.copy(runningOp = None))
case t: ThrottleMsg =>
import context.dispatcher // FIXME is this the right EC for the future below?
val mode =
@@ -278,10 +277,10 @@ private[akka] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress)
throw new RuntimeException("Throttle was requested from the TestConductor, but no transport " +
"adapters available that support throttling. Specify `testTransport(on = true)` in your MultiNodeConfig")
}
- stay
+ stay()
case _: DisconnectMsg =>
// FIXME: Currently ignoring, needs support from Remoting
- stay
+ stay()
case TerminateMsg(Left(false)) =>
context.system.terminate()
stop()
@@ -290,17 +289,17 @@ private[akka] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress)
stop()
case TerminateMsg(Right(exitValue)) =>
System.exit(exitValue)
- stay // needed because Java doesn’t have Nothing
- case _: Done => stay //FIXME what should happen?
+ stay() // needed because Java doesn’t have Nothing
+ case _: Done => stay() //FIXME what should happen?
}
}
when(Failed) {
case Event(msg: ClientOp, _) =>
- stay.replying(Status.Failure(new RuntimeException("cannot do " + msg + " while Failed")))
+ stay().replying(Status.Failure(new RuntimeException("cannot do " + msg + " while Failed")))
case Event(msg: NetworkOp, _) =>
log.warning("ignoring network message {} while Failed", msg)
- stay
+ stay()
}
onTermination {
diff --git a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/RemoteConnection.scala b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/RemoteConnection.scala
index 6cdde8ef22..49cf4c7d7a 100644
--- a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/RemoteConnection.scala
+++ b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/RemoteConnection.scala
@@ -4,6 +4,13 @@
package akka.remote.testconductor
+import java.net.InetSocketAddress
+import java.util.concurrent.Executors
+
+import scala.util.control.NonFatal
+
+import org.jboss.netty.bootstrap.{ ClientBootstrap, ServerBootstrap }
+import org.jboss.netty.buffer.ChannelBuffer
import org.jboss.netty.channel.{
Channel,
ChannelPipeline,
@@ -11,17 +18,14 @@ import org.jboss.netty.channel.{
ChannelUpstreamHandler,
DefaultChannelPipeline
}
-import org.jboss.netty.channel.socket.nio.{ NioClientSocketChannelFactory, NioServerSocketChannelFactory }
-import org.jboss.netty.bootstrap.{ ClientBootstrap, ServerBootstrap }
-import org.jboss.netty.handler.codec.frame.{ LengthFieldBasedFrameDecoder, LengthFieldPrepender }
-import java.net.InetSocketAddress
-import java.util.concurrent.Executors
-import akka.event.Logging
-import akka.util.Helpers
-import org.jboss.netty.handler.codec.oneone.{ OneToOneDecoder, OneToOneEncoder }
import org.jboss.netty.channel.ChannelHandlerContext
+import org.jboss.netty.channel.socket.nio.{ NioClientSocketChannelFactory, NioServerSocketChannelFactory }
+import org.jboss.netty.handler.codec.frame.{ LengthFieldBasedFrameDecoder, LengthFieldPrepender }
+import org.jboss.netty.handler.codec.oneone.{ OneToOneDecoder, OneToOneEncoder }
+
+import akka.event.Logging
import akka.protobufv3.internal.Message
-import org.jboss.netty.buffer.ChannelBuffer
+import akka.util.Helpers
/**
* INTERNAL API.
@@ -109,8 +113,14 @@ private[akka] object RemoteConnection {
case _ => "[unknown]"
}
- def shutdown(channel: Channel) =
- try channel.close()
- finally try channel.getFactory.shutdown()
- finally channel.getFactory.releaseExternalResources()
+ def shutdown(channel: Channel): Unit = {
+ try {
+ try channel.close()
+ finally try channel.getFactory.shutdown()
+ finally channel.getFactory.releaseExternalResources()
+ } catch {
+ case NonFatal(_) =>
+ // silence this one to not make tests look like they failed, it's not really critical
+ }
+ }
}
diff --git a/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala b/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala
index ff16044268..edffba336b 100644
--- a/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala
+++ b/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala
@@ -4,27 +4,27 @@
package akka.remote.testkit
-import language.implicitConversions
import java.net.{ InetAddress, InetSocketAddress }
-import com.typesafe.config.{ Config, ConfigFactory, ConfigObject }
-
-import scala.concurrent.{ Await, Awaitable }
-import scala.util.control.NonFatal
import scala.collection.immutable
-import akka.actor._
-import akka.util.Timeout
-import akka.remote.testconductor.{ TestConductor, TestConductorExt }
-import akka.testkit._
-import akka.testkit.TestKit
-import akka.testkit.TestEvent._
-
+import scala.concurrent.{ Await, Awaitable }
import scala.concurrent.duration._
-import akka.remote.testconductor.RoleName
+import scala.util.control.NonFatal
+
+import com.typesafe.config.{ Config, ConfigFactory, ConfigObject }
+import language.implicitConversions
+import org.jboss.netty.channel.ChannelException
+
+import akka.actor._
import akka.actor.RootActorPath
import akka.event.{ Logging, LoggingAdapter }
import akka.remote.RemoteTransportException
-import org.jboss.netty.channel.ChannelException
+import akka.remote.testconductor.{ TestConductor, TestConductorExt }
+import akka.remote.testconductor.RoleName
+import akka.testkit._
+import akka.testkit.TestEvent._
+import akka.testkit.TestKit
+import akka.util.Timeout
import akka.util.ccompat._
/**
@@ -245,15 +245,6 @@ object MultiNodeSpec {
ConfigFactory.parseMap(map.asJava)
}
- private def getCallerName(clazz: Class[_]): String = {
- val pattern = s"(akka\\.remote\\.testkit\\.MultiNodeSpec.*|akka\\.remote\\.RemotingMultiNodeSpec)"
- val s = Thread.currentThread.getStackTrace.map(_.getClassName).drop(1).dropWhile(_.matches(pattern))
- val reduced = s.lastIndexWhere(_ == clazz.getName) match {
- case -1 => s
- case z => s.drop(z + 1)
- }
- reduced.head.replaceFirst(""".*\.""", "").replaceAll("[^a-zA-Z_0-9]", "_")
- }
}
/**
@@ -284,7 +275,7 @@ abstract class MultiNodeSpec(
def this(config: MultiNodeConfig) =
this(config, {
- val name = MultiNodeSpec.getCallerName(classOf[MultiNodeSpec])
+ val name = TestKitUtils.testNameFromCallStack(classOf[MultiNodeSpec], "".r)
config =>
try {
ActorSystem(name, config)
diff --git a/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/PerfFlamesSupport.scala b/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/PerfFlamesSupport.scala
index a144b6d6de..d96f253fac 100644
--- a/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/PerfFlamesSupport.scala
+++ b/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/PerfFlamesSupport.scala
@@ -6,11 +6,11 @@ package akka.remote.testkit
import java.io.File
-import akka.remote.testconductor.RoleName
-
import scala.concurrent.Future
import scala.concurrent.duration._
+import akka.remote.testconductor.RoleName
+
/**
* INTERNAL API: Support trait allowing trivially recording perf metrics from [[MultiNodeSpec]]s
*/
diff --git a/akka-osgi/src/main/scala/akka/osgi/ActorSystemActivator.scala b/akka-osgi/src/main/scala/akka/osgi/ActorSystemActivator.scala
index 034e686677..2c2aa71a91 100644
--- a/akka-osgi/src/main/scala/akka/osgi/ActorSystemActivator.scala
+++ b/akka-osgi/src/main/scala/akka/osgi/ActorSystemActivator.scala
@@ -4,13 +4,14 @@
package akka.osgi
-import akka.actor.ActorSystem
import java.util.{ Dictionary, Properties }
-import akka.util.unused
+import com.typesafe.config.{ Config, ConfigFactory }
import org.osgi.framework._
import org.osgi.service.log.LogService
-import com.typesafe.config.{ Config, ConfigFactory }
+
+import akka.actor.{ ActorRefFactory, ActorSystem }
+import akka.util.unused
/**
* Abstract bundle activator implementation to bootstrap and configure an actor system in an
@@ -104,7 +105,10 @@ abstract class ActorSystemActivator extends BundleActivator {
val properties = new Properties()
properties.put("name", system.name)
registration = Some(
- context.registerService(classOf[ActorSystem].getName, system, properties.asInstanceOf[Dictionary[String, Any]]))
+ context.registerService(
+ Array(classOf[ActorSystem].getName, classOf[ActorRefFactory].getName),
+ system,
+ properties.asInstanceOf[Dictionary[String, Any]]))
}
/**
diff --git a/akka-osgi/src/main/scala/akka/osgi/BundleDelegatingClassLoader.scala b/akka-osgi/src/main/scala/akka/osgi/BundleDelegatingClassLoader.scala
index cfa381d216..384c4ca342 100644
--- a/akka-osgi/src/main/scala/akka/osgi/BundleDelegatingClassLoader.scala
+++ b/akka-osgi/src/main/scala/akka/osgi/BundleDelegatingClassLoader.scala
@@ -6,13 +6,16 @@ package akka.osgi
import java.net.URL
import java.util.Enumeration
-import org.osgi.framework.{ Bundle, BundleContext }
-import scala.util.Try
-import org.osgi.framework.wiring.{ BundleRevision, BundleWire, BundleWiring }
-import akka.util.ccompat.JavaConverters._
-import scala.util.Success
-import scala.util.Failure
+
import scala.annotation.tailrec
+import scala.util.Failure
+import scala.util.Success
+import scala.util.Try
+
+import org.osgi.framework.{ Bundle, BundleContext }
+import org.osgi.framework.wiring.{ BundleRevision, BundleWire, BundleWiring }
+
+import akka.util.ccompat.JavaConverters._
/*
* Companion object to create bundle delegating ClassLoader instances
diff --git a/akka-osgi/src/main/scala/akka/osgi/DefaultOSGiLogger.scala b/akka-osgi/src/main/scala/akka/osgi/DefaultOSGiLogger.scala
index 6ca8fc5bf6..5cd7ff107d 100644
--- a/akka-osgi/src/main/scala/akka/osgi/DefaultOSGiLogger.scala
+++ b/akka-osgi/src/main/scala/akka/osgi/DefaultOSGiLogger.scala
@@ -4,8 +4,9 @@
package akka.osgi
-import akka.event.Logging
import org.osgi.service.log.LogService
+
+import akka.event.Logging
import akka.event.Logging.{ DefaultLogger, LogEvent }
import akka.event.Logging.Error.NoCause
diff --git a/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala b/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala
index fda8b653d0..d6a4a96670 100644
--- a/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala
+++ b/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala
@@ -4,11 +4,12 @@
package akka.osgi
-import akka.actor.ActorSystem
-import akka.util.unused
import com.typesafe.config.{ Config, ConfigFactory }
import org.osgi.framework.BundleContext
+import akka.actor.ActorSystem
+import akka.util.unused
+
/**
* Factory class to create ActorSystem implementations in an OSGi environment. This mainly involves dealing with
* bundle classloaders appropriately to ensure that configuration files and classes get loaded properly
diff --git a/akka-osgi/src/test/resources/logback-test.xml b/akka-osgi/src/test/resources/logback-test.xml
index b932a3d8f8..2b9953fb9b 100644
--- a/akka-osgi/src/test/resources/logback-test.xml
+++ b/akka-osgi/src/test/resources/logback-test.xml
@@ -4,7 +4,7 @@
- %date{ISO8601} %-5level %logger %marker - %msg {%mdc}%n
+ %date{ISO8601} %-5level %logger %marker - %msg MDC: {%mdc}%n
@@ -12,7 +12,7 @@
target/akka-osgi.log
true
- %date{ISO8601} %-5level %logger %marker - %msg {%mdc}%n
+ %date{ISO8601} %-5level %logger %marker - %msg MDC: {%mdc}%n
diff --git a/akka-osgi/src/test/scala/akka/osgi/ActorSystemActivatorTest.scala b/akka-osgi/src/test/scala/akka/osgi/ActorSystemActivatorTest.scala
index 2bccfb466f..546bddc51b 100644
--- a/akka-osgi/src/test/scala/akka/osgi/ActorSystemActivatorTest.scala
+++ b/akka-osgi/src/test/scala/akka/osgi/ActorSystemActivatorTest.scala
@@ -4,20 +4,21 @@
package akka.osgi
+import scala.collection.immutable
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import PojoSRTestSupport.bundle
+import de.kalpatec.pojosr.framework.launch.BundleDescriptor
import language.postfixOps
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+import test.{ PingPongActorSystemActivator, RuntimeNameActorSystemActivator, TestActivators }
+import test.PingPong._
import akka.actor.ActorSystem
import akka.pattern.ask
-import scala.concurrent.Await
-import scala.concurrent.duration._
-import scala.collection.immutable
import akka.util.Timeout
-import de.kalpatec.pojosr.framework.launch.BundleDescriptor
-import test.{ PingPongActorSystemActivator, RuntimeNameActorSystemActivator, TestActivators }
-import test.PingPong._
-import PojoSRTestSupport.bundle
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
/**
* Test cases for [[akka.osgi.ActorSystemActivator]] in 2 different scenarios:
diff --git a/akka-osgi/src/test/scala/akka/osgi/PojoSRTestSupport.scala b/akka-osgi/src/test/scala/akka/osgi/PojoSRTestSupport.scala
index f55b417a29..7445f87d86 100644
--- a/akka-osgi/src/test/scala/akka/osgi/PojoSRTestSupport.scala
+++ b/akka-osgi/src/test/scala/akka/osgi/PojoSRTestSupport.scala
@@ -4,21 +4,22 @@
package akka.osgi
-import de.kalpatec.pojosr.framework.launch.{ BundleDescriptor, ClasspathScanner, PojoServiceRegistryFactory }
-
-import akka.util.ccompat.JavaConverters._
-import org.apache.commons.io.IOUtils.copy
-
-import org.osgi.framework._
-import java.net.URL
-import java.util.jar.JarInputStream
import java.io._
-import org.scalatest.{ BeforeAndAfterAll, Suite }
+import java.net.URL
import java.util.{ Date, HashMap, ServiceLoader, UUID }
-import scala.reflect.ClassTag
+import java.util.jar.JarInputStream
+
+import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.duration._
-import scala.annotation.tailrec
+import scala.reflect.ClassTag
+
+import de.kalpatec.pojosr.framework.launch.{ BundleDescriptor, ClasspathScanner, PojoServiceRegistryFactory }
+import org.apache.commons.io.IOUtils.copy
+import org.osgi.framework._
+import org.scalatest.{ BeforeAndAfterAll, Suite }
+
+import akka.util.ccompat.JavaConverters._
/**
* Trait that provides support for building akka-osgi tests using PojoSR
diff --git a/akka-osgi/src/test/scala/akka/osgi/test/TestActivators.scala b/akka-osgi/src/test/scala/akka/osgi/test/TestActivators.scala
index c67f721a7f..04be8bd200 100644
--- a/akka-osgi/src/test/scala/akka/osgi/test/TestActivators.scala
+++ b/akka-osgi/src/test/scala/akka/osgi/test/TestActivators.scala
@@ -4,11 +4,12 @@
package akka.osgi.test
-import akka.osgi.ActorSystemActivator
-import akka.actor.{ ActorSystem, Props }
import PingPong._
import org.osgi.framework.BundleContext
+import akka.actor.{ ActorSystem, Props }
+import akka.osgi.ActorSystemActivator
+
/**
* A set of [[akka.osgi.ActorSystemActivator]]s for testing purposes
*/
@@ -24,7 +25,7 @@ object TestActivators {
class PingPongActorSystemActivator extends ActorSystemActivator {
def configure(context: BundleContext, system: ActorSystem): Unit = {
- system.actorOf(Props[PongActor], name = "pong")
+ system.actorOf(Props[PongActor](), name = "pong")
registerService(context, system)
}
diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/Offset.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/Offset.scala
index fc6fc57dd2..7d845fc5ec 100644
--- a/akka-persistence-query/src/main/scala/akka/persistence/query/Offset.scala
+++ b/akka-persistence-query/src/main/scala/akka/persistence/query/Offset.scala
@@ -52,7 +52,7 @@ final case class TimeBasedUUID(value: UUID) extends Offset with Ordered[TimeBase
/**
* Used when retrieving all events.
*/
-final case object NoOffset extends Offset {
+case object NoOffset extends Offset {
/**
* Java API:
diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/PersistenceQuery.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/PersistenceQuery.scala
index 9aa68c5170..11c45f3fb1 100644
--- a/akka-persistence-query/src/main/scala/akka/persistence/query/PersistenceQuery.scala
+++ b/akka-persistence-query/src/main/scala/akka/persistence/query/PersistenceQuery.scala
@@ -4,14 +4,15 @@
package akka.persistence.query
-import akka.actor._
-import akka.annotation.InternalApi
-import akka.persistence.query.scaladsl.ReadJournal
-import akka.persistence.{ PersistencePlugin, PluginProvider }
-import akka.util.unused
+import scala.reflect.ClassTag
+
import com.typesafe.config.{ Config, ConfigFactory }
-import scala.reflect.ClassTag
+import akka.actor._
+import akka.annotation.InternalApi
+import akka.persistence.{ PersistencePlugin, PluginProvider }
+import akka.persistence.query.scaladsl.ReadJournal
+import akka.util.unused
/**
* Persistence extension for queries.
diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/javadsl/CurrentEventsByPersistenceIdQuery.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/javadsl/CurrentEventsByPersistenceIdQuery.scala
index 9299818c08..050130f65d 100644
--- a/akka-persistence-query/src/main/scala/akka/persistence/query/javadsl/CurrentEventsByPersistenceIdQuery.scala
+++ b/akka-persistence-query/src/main/scala/akka/persistence/query/javadsl/CurrentEventsByPersistenceIdQuery.scala
@@ -5,8 +5,8 @@
package akka.persistence.query.javadsl
import akka.NotUsed
-import akka.stream.javadsl.Source
import akka.persistence.query.EventEnvelope
+import akka.stream.javadsl.Source
/**
* A plugin may optionally support this query by implementing this interface.
diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/javadsl/EventsByPersistenceIdQuery.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/javadsl/EventsByPersistenceIdQuery.scala
index a0b6f5973e..857b7525c3 100644
--- a/akka-persistence-query/src/main/scala/akka/persistence/query/javadsl/EventsByPersistenceIdQuery.scala
+++ b/akka-persistence-query/src/main/scala/akka/persistence/query/javadsl/EventsByPersistenceIdQuery.scala
@@ -5,8 +5,8 @@
package akka.persistence.query.javadsl
import akka.NotUsed
-import akka.stream.javadsl.Source
import akka.persistence.query.EventEnvelope
+import akka.stream.javadsl.Source
/**
* A plugin may optionally support this query by implementing this interface.
diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdStage.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdStage.scala
index 3018924e3d..1fbfcfbc06 100644
--- a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdStage.scala
+++ b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdStage.scala
@@ -4,6 +4,8 @@
package akka.persistence.query.journal.leveldb
+import scala.concurrent.duration.FiniteDuration
+
import akka.NotUsed
import akka.actor.ActorRef
import akka.annotation.InternalApi
@@ -26,8 +28,6 @@ import akka.stream.stage.GraphStageLogic
import akka.stream.stage.OutHandler
import akka.stream.stage.TimerGraphStageLogicWithLogging
-import scala.concurrent.duration.FiniteDuration
-
/**
* INTERNAL API
*/
diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagStage.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagStage.scala
index d500b1cb8d..79569a587a 100644
--- a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagStage.scala
+++ b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagStage.scala
@@ -4,6 +4,8 @@
package akka.persistence.query.journal.leveldb
+import scala.concurrent.duration.FiniteDuration
+
import akka.NotUsed
import akka.actor.ActorRef
import akka.annotation.InternalApi
@@ -14,19 +16,17 @@ import akka.persistence.journal.leveldb.LeveldbJournal
import akka.persistence.journal.leveldb.LeveldbJournal.ReplayTaggedMessages
import akka.persistence.journal.leveldb.LeveldbJournal.ReplayedTaggedMessage
import akka.persistence.journal.leveldb.LeveldbJournal.TaggedEventAppended
-import akka.persistence.query.journal.leveldb.EventsByTagStage.Continue
import akka.persistence.query.EventEnvelope
import akka.persistence.query.Sequence
+import akka.persistence.query.journal.leveldb.EventsByTagStage.Continue
+import akka.stream.Attributes
import akka.stream.Materializer
+import akka.stream.Outlet
+import akka.stream.SourceShape
import akka.stream.stage.GraphStage
import akka.stream.stage.GraphStageLogic
import akka.stream.stage.OutHandler
import akka.stream.stage.TimerGraphStageLogicWithLogging
-import akka.stream.Attributes
-import akka.stream.Outlet
-import akka.stream.SourceShape
-
-import scala.concurrent.duration.FiniteDuration
/**
* INTERNAL API
diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/LeveldbReadJournalProvider.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/LeveldbReadJournalProvider.scala
index 738bfb764d..1c5fc7a1af 100644
--- a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/LeveldbReadJournalProvider.scala
+++ b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/LeveldbReadJournalProvider.scala
@@ -4,9 +4,10 @@
package akka.persistence.query.journal.leveldb
+import com.typesafe.config.Config
+
import akka.actor.ExtendedActorSystem
import akka.persistence.query.ReadJournalProvider
-import com.typesafe.config.Config
class LeveldbReadJournalProvider(system: ExtendedActorSystem, config: Config) extends ReadJournalProvider {
diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/scaladsl/LeveldbReadJournal.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/scaladsl/LeveldbReadJournal.scala
index 5b888e4c42..0e3a81d19c 100644
--- a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/scaladsl/LeveldbReadJournal.scala
+++ b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/scaladsl/LeveldbReadJournal.scala
@@ -6,24 +6,25 @@ package akka.persistence.query.journal.leveldb.scaladsl
import java.net.URLEncoder
+import scala.concurrent.duration._
+
+import com.typesafe.config.Config
+
import akka.NotUsed
import akka.actor.ExtendedActorSystem
import akka.event.Logging
-import akka.persistence.query.journal.leveldb.AllPersistenceIdsStage
-import akka.persistence.query.journal.leveldb.EventsByPersistenceIdStage
-import akka.persistence.query.journal.leveldb.EventsByTagStage
-import akka.persistence.query.scaladsl.ReadJournal
-import akka.persistence.query.scaladsl._
import akka.persistence.query.EventEnvelope
import akka.persistence.query.NoOffset
import akka.persistence.query.Offset
import akka.persistence.query.Sequence
+import akka.persistence.query.journal.leveldb.AllPersistenceIdsStage
+import akka.persistence.query.journal.leveldb.EventsByPersistenceIdStage
+import akka.persistence.query.journal.leveldb.EventsByTagStage
+import akka.persistence.query.scaladsl._
+import akka.persistence.query.scaladsl.ReadJournal
import akka.stream.scaladsl.Source
import akka.util.ByteString
import akka.util.unused
-import com.typesafe.config.Config
-
-import scala.concurrent.duration._
/**
* Scala API [[akka.persistence.query.scaladsl.ReadJournal]] implementation for LevelDB.
diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/scaladsl/CurrentEventsByPersistenceIdQuery.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/scaladsl/CurrentEventsByPersistenceIdQuery.scala
index d2626fa5e2..d27f56889d 100644
--- a/akka-persistence-query/src/main/scala/akka/persistence/query/scaladsl/CurrentEventsByPersistenceIdQuery.scala
+++ b/akka-persistence-query/src/main/scala/akka/persistence/query/scaladsl/CurrentEventsByPersistenceIdQuery.scala
@@ -5,8 +5,8 @@
package akka.persistence.query.scaladsl
import akka.NotUsed
-import akka.stream.scaladsl.Source
import akka.persistence.query.EventEnvelope
+import akka.stream.scaladsl.Source
/**
* A plugin may optionally support this query by implementing this trait.
diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/scaladsl/EventsByPersistenceIdQuery.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/scaladsl/EventsByPersistenceIdQuery.scala
index c72d428699..96b0e31562 100644
--- a/akka-persistence-query/src/main/scala/akka/persistence/query/scaladsl/EventsByPersistenceIdQuery.scala
+++ b/akka-persistence-query/src/main/scala/akka/persistence/query/scaladsl/EventsByPersistenceIdQuery.scala
@@ -5,8 +5,8 @@
package akka.persistence.query.scaladsl
import akka.NotUsed
-import akka.stream.scaladsl.Source
import akka.persistence.query.EventEnvelope
+import akka.stream.scaladsl.Source
/**
* A plugin may optionally support this query by implementing this trait.
diff --git a/akka-persistence-query/src/test/scala/akka/persistence/query/DummyReadJournal.scala b/akka-persistence-query/src/test/scala/akka/persistence/query/DummyReadJournal.scala
index 3edc22a36a..75a6107941 100644
--- a/akka-persistence-query/src/test/scala/akka/persistence/query/DummyReadJournal.scala
+++ b/akka-persistence-query/src/test/scala/akka/persistence/query/DummyReadJournal.scala
@@ -4,10 +4,11 @@
package akka.persistence.query
-import akka.NotUsed
-import akka.stream.scaladsl.Source
import com.typesafe.config.{ Config, ConfigFactory }
+
+import akka.NotUsed
import akka.actor.ExtendedActorSystem
+import akka.stream.scaladsl.Source
import akka.util.unused
/**
diff --git a/akka-persistence-query/src/test/scala/akka/persistence/query/OffsetSpec.scala b/akka-persistence-query/src/test/scala/akka/persistence/query/OffsetSpec.scala
index 8178974829..54b15f46e4 100644
--- a/akka-persistence-query/src/test/scala/akka/persistence/query/OffsetSpec.scala
+++ b/akka-persistence-query/src/test/scala/akka/persistence/query/OffsetSpec.scala
@@ -7,6 +7,7 @@ package akka.persistence.query
import java.util.UUID
import scala.util.Random
+
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
diff --git a/akka-persistence-query/src/test/scala/akka/persistence/query/PersistenceQuerySpec.scala b/akka-persistence-query/src/test/scala/akka/persistence/query/PersistenceQuerySpec.scala
index c4b89db255..d88d857369 100644
--- a/akka-persistence-query/src/test/scala/akka/persistence/query/PersistenceQuerySpec.scala
+++ b/akka-persistence-query/src/test/scala/akka/persistence/query/PersistenceQuerySpec.scala
@@ -6,15 +6,16 @@ package akka.persistence.query
import java.util.concurrent.atomic.AtomicInteger
-import akka.actor.ActorSystem
-import akka.persistence.journal.{ EventSeq, ReadEventAdapter }
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
import com.typesafe.config.{ Config, ConfigFactory }
import org.scalatest.BeforeAndAfterAll
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
-import scala.concurrent.Await
-import scala.concurrent.duration._
+import akka.actor.ActorSystem
+import akka.persistence.journal.{ EventSeq, ReadEventAdapter }
class PersistenceQuerySpec extends AnyWordSpecLike with Matchers with BeforeAndAfterAll {
diff --git a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/AllPersistenceIdsSpec.scala b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/AllPersistenceIdsSpec.scala
index 41ef933f88..257744adbc 100644
--- a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/AllPersistenceIdsSpec.scala
+++ b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/AllPersistenceIdsSpec.scala
@@ -4,6 +4,8 @@
package akka.persistence.query.journal.leveldb
+import scala.concurrent.duration._
+
import akka.persistence.query.PersistenceQuery
import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal
import akka.persistence.query.scaladsl.PersistenceIdsQuery
@@ -11,8 +13,6 @@ import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
-import scala.concurrent.duration._
-
object AllPersistenceIdsSpec {
val config = """
akka.loglevel = INFO
diff --git a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/Cleanup.scala b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/Cleanup.scala
index 28e60dfb4c..8e0f9d3e2f 100644
--- a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/Cleanup.scala
+++ b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/Cleanup.scala
@@ -4,10 +4,12 @@
package akka.persistence.query.journal.leveldb
-import akka.testkit.AkkaSpec
import java.io.File
+
import org.apache.commons.io.FileUtils
+import akka.testkit.AkkaSpec
+
trait Cleanup { this: AkkaSpec =>
val storageLocations =
List(
diff --git a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdSpec.scala b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdSpec.scala
index 70db3ba583..3c72e8e0ee 100644
--- a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdSpec.scala
+++ b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdSpec.scala
@@ -4,16 +4,16 @@
package akka.persistence.query.journal.leveldb
+import scala.concurrent.duration._
+
import akka.actor.ActorRef
+import akka.persistence.query.EventEnvelope
import akka.persistence.query.PersistenceQuery
import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal
import akka.persistence.query.scaladsl.EventsByTagQuery
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
-import scala.concurrent.duration._
-
-import akka.persistence.query.EventEnvelope
object EventsByPersistenceIdSpec {
val config = """
diff --git a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByTagSpec.scala b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByTagSpec.scala
index c64466e18b..cb8520e8a6 100644
--- a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByTagSpec.scala
+++ b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByTagSpec.scala
@@ -4,20 +4,20 @@
package akka.persistence.query.journal.leveldb
+import scala.concurrent.duration._
+
import akka.persistence.journal.Tagged
import akka.persistence.journal.WriteEventAdapter
-import akka.persistence.query.NoOffset
-import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal
-import akka.persistence.query.scaladsl.EventsByTagQuery
import akka.persistence.query.EventEnvelope
+import akka.persistence.query.NoOffset
import akka.persistence.query.PersistenceQuery
import akka.persistence.query.Sequence
+import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal
+import akka.persistence.query.scaladsl.EventsByTagQuery
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
-import scala.concurrent.duration._
-
object EventsByTagSpec {
val config = s"""
akka.loglevel = INFO
diff --git a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/TestActor.scala b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/TestActor.scala
index 314db7e650..06c3f6ff57 100644
--- a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/TestActor.scala
+++ b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/TestActor.scala
@@ -4,8 +4,8 @@
package akka.persistence.query.journal.leveldb
-import akka.persistence.PersistentActor
import akka.actor.Props
+import akka.persistence.PersistentActor
object TestActor {
def props(persistenceId: String): Props =
diff --git a/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/PersistencePluginProxySpec.scala b/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/PersistencePluginProxySpec.scala
index b1d2da558f..449410f838 100644
--- a/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/PersistencePluginProxySpec.scala
+++ b/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/PersistencePluginProxySpec.scala
@@ -4,11 +4,12 @@
package akka.persistence.journal.leveldb
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.persistence._
import akka.persistence.journal.PersistencePluginProxy
import akka.testkit.{ AkkaSpec, TestProbe }
-import com.typesafe.config.ConfigFactory
object PersistencePluginProxySpec {
lazy val config =
diff --git a/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala b/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala
index 454be1f371..e29b66d8fc 100644
--- a/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala
+++ b/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala
@@ -4,10 +4,11 @@
package akka.persistence.journal.leveldb
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.persistence._
import akka.testkit.{ AkkaSpec, TestProbe }
-import com.typesafe.config.ConfigFactory
object SharedLeveldbJournalSpec {
val config = ConfigFactory.parseString(s"""
diff --git a/akka-persistence-shared/src/test/scala/akka/persistence/serialization/SerializerSpec.scala b/akka-persistence-shared/src/test/scala/akka/persistence/serialization/SerializerSpec.scala
index 30d6a0f31a..40f3003700 100644
--- a/akka-persistence-shared/src/test/scala/akka/persistence/serialization/SerializerSpec.scala
+++ b/akka-persistence-shared/src/test/scala/akka/persistence/serialization/SerializerSpec.scala
@@ -6,16 +6,19 @@ package akka.persistence.serialization
import java.io.NotSerializableException
import java.util.UUID
+
+import scala.concurrent.Await
+import scala.concurrent.duration.Duration
+
+import com.typesafe.config._
+import org.apache.commons.codec.binary.Hex.{ decodeHex, encodeHex }
+
import akka.actor._
-import akka.persistence.AtLeastOnceDelivery.{ AtLeastOnceDeliverySnapshot, UnconfirmedDelivery }
import akka.persistence._
+import akka.persistence.AtLeastOnceDelivery.{ AtLeastOnceDeliverySnapshot, UnconfirmedDelivery }
import akka.serialization._
import akka.testkit._
import akka.util.ByteString.UTF_8
-import com.typesafe.config._
-import org.apache.commons.codec.binary.Hex.{ decodeHex, encodeHex }
-import scala.concurrent.Await
-import scala.concurrent.duration.Duration
object SerializerSpecConfigs {
val customSerializers =
@@ -330,7 +333,7 @@ class MessageSerializerRemotingSpec extends AkkaSpec(remote.withFallback(customS
val serialization = SerializationExtension(system)
override protected def atStartup(): Unit = {
- remoteSystem.actorOf(Props[RemoteActor], "remote")
+ remoteSystem.actorOf(Props[RemoteActor](), "remote")
}
override def afterTermination(): Unit = {
diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/PluginSpec.scala b/akka-persistence-tck/src/main/scala/akka/persistence/PluginSpec.scala
index 674f73c10e..3241f90217 100644
--- a/akka-persistence-tck/src/main/scala/akka/persistence/PluginSpec.scala
+++ b/akka-persistence-tck/src/main/scala/akka/persistence/PluginSpec.scala
@@ -4,16 +4,19 @@
package akka.persistence
+import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
+
import scala.reflect.ClassTag
-import akka.actor._
-import akka.testkit._
+
import com.typesafe.config._
import org.scalatest._
-import java.util.UUID
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
+import akka.actor._
+import akka.testkit._
+
abstract class PluginSpec(val config: Config)
extends TestKitBase
with AnyWordSpecLike
diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalPerfSpec.scala b/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalPerfSpec.scala
index a9ab591801..303ba45870 100644
--- a/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalPerfSpec.scala
+++ b/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalPerfSpec.scala
@@ -4,12 +4,13 @@
package akka.persistence.japi.journal
-import akka.persistence.CapabilityFlag
-import akka.persistence.journal.JournalPerfSpec
import com.typesafe.config.Config
import org.scalactic.source.Position
import org.scalatest.Informer
+import akka.persistence.CapabilityFlag
+import akka.persistence.journal.JournalPerfSpec
+
/**
* JAVA API
*
@@ -49,7 +50,7 @@ class JavaJournalPerfSpec(config: Config) extends JournalPerfSpec(config) {
System.out.println(message)
}
- override protected def supportsRejectingNonSerializableObjects: CapabilityFlag = CapabilityFlag.on
+ override protected def supportsRejectingNonSerializableObjects: CapabilityFlag = CapabilityFlag.on()
- override protected def supportsSerialization: CapabilityFlag = CapabilityFlag.on
+ override protected def supportsSerialization: CapabilityFlag = CapabilityFlag.on()
}
diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalSpec.scala b/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalSpec.scala
index a9fade9961..093f5a3f5c 100644
--- a/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalSpec.scala
+++ b/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalSpec.scala
@@ -4,9 +4,10 @@
package akka.persistence.japi.journal
+import com.typesafe.config.Config
+
import akka.persistence.CapabilityFlag
import akka.persistence.journal.JournalSpec
-import com.typesafe.config.Config
/**
* JAVA API
@@ -21,7 +22,7 @@ import com.typesafe.config.Config
* @param config configures the Journal plugin to be tested
*/
class JavaJournalSpec(config: Config) extends JournalSpec(config) {
- override protected def supportsRejectingNonSerializableObjects: CapabilityFlag = CapabilityFlag.on
+ override protected def supportsRejectingNonSerializableObjects: CapabilityFlag = CapabilityFlag.on()
- override protected def supportsSerialization: CapabilityFlag = CapabilityFlag.on
+ override protected def supportsSerialization: CapabilityFlag = CapabilityFlag.on()
}
diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/japi/snapshot/JavaSnapshotStoreSpec.scala b/akka-persistence-tck/src/main/scala/akka/persistence/japi/snapshot/JavaSnapshotStoreSpec.scala
index a89282aad9..b8b92cdc84 100644
--- a/akka-persistence-tck/src/main/scala/akka/persistence/japi/snapshot/JavaSnapshotStoreSpec.scala
+++ b/akka-persistence-tck/src/main/scala/akka/persistence/japi/snapshot/JavaSnapshotStoreSpec.scala
@@ -4,9 +4,10 @@
package akka.persistence.japi.snapshot
+import com.typesafe.config.Config
+
import akka.persistence.CapabilityFlag
import akka.persistence.snapshot.SnapshotStoreSpec
-import com.typesafe.config.Config
/**
* JAVA API
@@ -20,5 +21,5 @@ import com.typesafe.config.Config
* @see [[akka.persistence.snapshot.SnapshotStoreSpec]]
*/
class JavaSnapshotStoreSpec(config: Config) extends SnapshotStoreSpec(config) {
- override protected def supportsSerialization: CapabilityFlag = CapabilityFlag.on
+ override protected def supportsSerialization: CapabilityFlag = CapabilityFlag.on()
}
diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalPerfSpec.scala b/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalPerfSpec.scala
index 24a8f4ad92..37f23fd9c2 100644
--- a/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalPerfSpec.scala
+++ b/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalPerfSpec.scala
@@ -9,6 +9,9 @@ import java.nio.charset.StandardCharsets
import scala.collection.immutable
import scala.concurrent.duration._
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ActorLogging
import akka.actor.ActorRef
import akka.actor.Props
@@ -19,8 +22,6 @@ import akka.persistence.journal.JournalPerfSpec.Cmd
import akka.persistence.journal.JournalPerfSpec.ResetCounter
import akka.serialization.SerializerWithStringManifest
import akka.testkit.TestProbe
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
object JournalPerfSpec {
class BenchActor(override val persistenceId: String, replyTo: ActorRef, replyAfter: Int)
diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalSpec.scala b/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalSpec.scala
index 76d35d92fd..854b7faa70 100644
--- a/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalSpec.scala
+++ b/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalSpec.scala
@@ -4,15 +4,16 @@
package akka.persistence.journal
-import akka.persistence.scalatest.{ MayVerb, OptionalTests }
-
import scala.concurrent.duration._
+
+import com.typesafe.config._
+
import akka.actor._
import akka.persistence._
import akka.persistence.JournalProtocol._
+import akka.persistence.scalatest.{ MayVerb, OptionalTests }
import akka.testkit._
import akka.util.unused
-import com.typesafe.config._
object JournalSpec {
val config: Config = ConfigFactory.parseString(s"""
diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/scalatest/OptionalTests.scala b/akka-persistence-tck/src/main/scala/akka/persistence/scalatest/OptionalTests.scala
index dea5c32694..316e885e9f 100644
--- a/akka-persistence-tck/src/main/scala/akka/persistence/scalatest/OptionalTests.scala
+++ b/akka-persistence-tck/src/main/scala/akka/persistence/scalatest/OptionalTests.scala
@@ -4,9 +4,10 @@
package akka.persistence.scalatest
-import akka.persistence.CapabilityFlag
import org.scalatest.Informing
+import akka.persistence.CapabilityFlag
+
trait OptionalTests {
this: Informing =>
diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/snapshot/SnapshotStoreSpec.scala b/akka-persistence-tck/src/main/scala/akka/persistence/snapshot/SnapshotStoreSpec.scala
index 667cf50d68..702bedb37f 100644
--- a/akka-persistence-tck/src/main/scala/akka/persistence/snapshot/SnapshotStoreSpec.scala
+++ b/akka-persistence-tck/src/main/scala/akka/persistence/snapshot/SnapshotStoreSpec.scala
@@ -4,15 +4,16 @@
package akka.persistence.snapshot
-import akka.persistence.scalatest.{ MayVerb, OptionalTests }
-
import scala.collection.immutable.Seq
+
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.persistence._
import akka.persistence.SnapshotProtocol._
+import akka.persistence.scalatest.{ MayVerb, OptionalTests }
import akka.testkit.TestProbe
-import com.typesafe.config.ConfigFactory
-import com.typesafe.config.Config
object SnapshotStoreSpec {
val config: Config = ConfigFactory.parseString(s"""
@@ -44,7 +45,8 @@ abstract class SnapshotStoreSpec(config: Config)
with MayVerb
with OptionalTests
with SnapshotStoreCapabilityFlags {
- implicit lazy val system = ActorSystem("SnapshotStoreSpec", config.withFallback(SnapshotStoreSpec.config))
+ implicit lazy val system: ActorSystem =
+ ActorSystem("SnapshotStoreSpec", config.withFallback(SnapshotStoreSpec.config))
private var senderProbe: TestProbe = _
private var metadata: Seq[SnapshotMetadata] = Nil
diff --git a/akka-persistence-tck/src/test/scala/akka/persistence/PluginCleanup.scala b/akka-persistence-tck/src/test/scala/akka/persistence/PluginCleanup.scala
index f9214b4bd0..7176fe5a58 100644
--- a/akka-persistence-tck/src/test/scala/akka/persistence/PluginCleanup.scala
+++ b/akka-persistence-tck/src/test/scala/akka/persistence/PluginCleanup.scala
@@ -5,6 +5,7 @@
package akka.persistence
import java.io.File
+
import org.apache.commons.io.FileUtils
import org.scalatest.BeforeAndAfterAll
diff --git a/akka-persistence-tck/src/test/scala/akka/persistence/japi/JavaJournalSpecSpec.scala b/akka-persistence-tck/src/test/scala/akka/persistence/japi/JavaJournalSpecSpec.scala
index b118ae940d..567f784d60 100644
--- a/akka-persistence-tck/src/test/scala/akka/persistence/japi/JavaJournalSpecSpec.scala
+++ b/akka-persistence-tck/src/test/scala/akka/persistence/japi/JavaJournalSpecSpec.scala
@@ -4,10 +4,11 @@
package akka.persistence.japi
-import akka.persistence.japi.journal.JavaJournalSpec
import com.typesafe.config.ConfigFactory
import org.scalatest.DoNotDiscover
+import akka.persistence.japi.journal.JavaJournalSpec
+
/* Only checking that compilation works with the constructor here as expected (no other abstract fields leaked) */
@DoNotDiscover
class JavaJournalSpecSpec extends JavaJournalSpec(ConfigFactory.parseString(""))
diff --git a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalJavaSpec.scala b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalJavaSpec.scala
index 8e31fca109..216b42fe57 100644
--- a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalJavaSpec.scala
+++ b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalJavaSpec.scala
@@ -4,8 +4,8 @@
package akka.persistence.journal.leveldb
-import akka.persistence.journal.JournalSpec
import akka.persistence.{ PersistenceSpec, PluginCleanup }
+import akka.persistence.journal.JournalSpec
class LeveldbJournalJavaSpec
extends JournalSpec(
diff --git a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativePerfSpec.scala b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativePerfSpec.scala
index ee658500b9..06bbf34363 100644
--- a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativePerfSpec.scala
+++ b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativePerfSpec.scala
@@ -4,9 +4,9 @@
package akka.persistence.journal.leveldb
-import akka.persistence.journal.JournalPerfSpec
import akka.persistence.PersistenceSpec
import akka.persistence.PluginCleanup
+import akka.persistence.journal.JournalPerfSpec
class LeveldbJournalNativePerfSpec
extends JournalPerfSpec(
diff --git a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativeSpec.scala b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativeSpec.scala
index b4eb285a34..d5fa8b35d2 100644
--- a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativeSpec.scala
+++ b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativeSpec.scala
@@ -4,8 +4,8 @@
package akka.persistence.journal.leveldb
-import akka.persistence.journal.JournalSpec
import akka.persistence.{ PersistenceSpec, PluginCleanup }
+import akka.persistence.journal.JournalSpec
class LeveldbJournalNativeSpec
extends JournalSpec(
diff --git a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNoAtomicPersistMultipleEventsSpec.scala b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNoAtomicPersistMultipleEventsSpec.scala
index 2a9e6cf254..22c0e988fd 100644
--- a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNoAtomicPersistMultipleEventsSpec.scala
+++ b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNoAtomicPersistMultipleEventsSpec.scala
@@ -4,8 +4,8 @@
package akka.persistence.journal.leveldb
-import akka.persistence.journal.JournalSpec
import akka.persistence.{ PersistenceSpec, PluginCleanup }
+import akka.persistence.journal.JournalSpec
class LeveldbJournalNoAtomicPersistMultipleEventsSpec
extends JournalSpec(
diff --git a/akka-persistence-tck/src/test/scala/akka/persistence/snapshot/local/LocalSnapshotStoreSpec.scala b/akka-persistence-tck/src/test/scala/akka/persistence/snapshot/local/LocalSnapshotStoreSpec.scala
index 726bc9ad83..27d13cb38e 100644
--- a/akka-persistence-tck/src/test/scala/akka/persistence/snapshot/local/LocalSnapshotStoreSpec.scala
+++ b/akka-persistence-tck/src/test/scala/akka/persistence/snapshot/local/LocalSnapshotStoreSpec.scala
@@ -4,8 +4,9 @@
package akka.persistence.snapshot.local
-import akka.persistence.CapabilityFlag
import com.typesafe.config.ConfigFactory
+
+import akka.persistence.CapabilityFlag
import akka.persistence.PluginCleanup
import akka.persistence.snapshot.SnapshotStoreSpec
@@ -19,5 +20,5 @@ class LocalSnapshotStoreSpec
"""))
with PluginCleanup {
- override protected def supportsSerialization: CapabilityFlag = CapabilityFlag.on
+ override protected def supportsSerialization: CapabilityFlag = CapabilityFlag.on()
}
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/EventStorage.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/EventStorage.scala
index ee730e279f..070e39901f 100644
--- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/EventStorage.scala
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/EventStorage.scala
@@ -4,15 +4,16 @@
package akka.persistence.testkit
+import java.util.{ List => JList }
+
+import scala.collection.immutable
+import scala.util.{ Failure, Success, Try }
+
import akka.annotation.InternalApi
import akka.persistence.PersistentRepr
import akka.persistence.testkit.ProcessingPolicy.DefaultPolicies
import akka.persistence.testkit.internal.TestKitStorage
import akka.util.ccompat.JavaConverters._
-import java.util.{ List => JList }
-
-import scala.collection.immutable
-import scala.util.{ Failure, Success, Try }
/**
* INTERNAL API
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/PersistenceTestKitPlugin.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/PersistenceTestKitPlugin.scala
index c619776057..9730e8a918 100644
--- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/PersistenceTestKitPlugin.scala
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/PersistenceTestKitPlugin.scala
@@ -4,16 +4,17 @@
package akka.persistence.testkit
+import scala.collection.immutable
+import scala.concurrent.Future
+import scala.util.Try
+
+import com.typesafe.config.{ Config, ConfigFactory }
+
import akka.annotation.InternalApi
import akka.persistence._
import akka.persistence.journal.{ AsyncWriteJournal, Tagged }
import akka.persistence.snapshot.SnapshotStore
import akka.persistence.testkit.internal.{ InMemStorageExtension, SnapshotStorageEmulatorExtension }
-import com.typesafe.config.{ Config, ConfigFactory }
-
-import scala.collection.immutable
-import scala.concurrent.Future
-import scala.util.Try
/**
* INTERNAL API
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/SnapshotStorage.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/SnapshotStorage.scala
index 3ab098f97a..1c66d012b8 100644
--- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/SnapshotStorage.scala
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/SnapshotStorage.scala
@@ -4,13 +4,13 @@
package akka.persistence.testkit
+import scala.util.Success
+
import akka.annotation.InternalApi
import akka.persistence.{ SelectedSnapshot, SnapshotMetadata, SnapshotSelectionCriteria }
import akka.persistence.testkit.ProcessingPolicy.DefaultPolicies
import akka.persistence.testkit.internal.TestKitStorage
-import scala.util.Success
-
/**
* INTERNAL API
*/
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/EventSourcedBehaviorTestKitImpl.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/EventSourcedBehaviorTestKitImpl.scala
new file mode 100644
index 0000000000..e520b22a4d
--- /dev/null
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/EventSourcedBehaviorTestKitImpl.scala
@@ -0,0 +1,187 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.persistence.testkit.internal
+
+import scala.collection.immutable
+import scala.concurrent.duration._
+import scala.reflect.ClassTag
+import scala.util.control.NonFatal
+
+import akka.actor.testkit.typed.scaladsl.ActorTestKit
+import akka.actor.testkit.typed.scaladsl.SerializationTestKit
+import akka.actor.typed.ActorRef
+import akka.actor.typed.ActorSystem
+import akka.actor.typed.Behavior
+import akka.annotation.InternalApi
+import akka.persistence.testkit.scaladsl.EventSourcedBehaviorTestKit
+import akka.persistence.testkit.scaladsl.EventSourcedBehaviorTestKit.CommandResult
+import akka.persistence.testkit.scaladsl.EventSourcedBehaviorTestKit.CommandResultWithReply
+import akka.persistence.testkit.scaladsl.EventSourcedBehaviorTestKit.RestartResult
+import akka.persistence.testkit.scaladsl.EventSourcedBehaviorTestKit.SerializationSettings
+import akka.persistence.testkit.scaladsl.PersistenceTestKit
+import akka.persistence.typed.PersistenceId
+import akka.persistence.typed.internal.EventSourcedBehaviorImpl
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[akka] object EventSourcedBehaviorTestKitImpl {
+ final case class CommandResultImpl[Command, Event, State, Reply](
+ command: Command,
+ events: immutable.Seq[Event],
+ state: State,
+ replyOption: Option[Reply])
+ extends CommandResultWithReply[Command, Event, State, Reply] {
+
+ override def hasNoEvents: Boolean = events.isEmpty
+
+ override def event: Event = {
+ if (events.nonEmpty) events.head else throw new AssertionError("No events")
+ }
+
+ override def eventOfType[E <: Event: ClassTag]: E =
+ ofType(event, "event")
+
+ override def stateOfType[S <: State: ClassTag]: S =
+ ofType(state, "state")
+
+ override def reply: Reply = replyOption.getOrElse(throw new AssertionError("No reply"))
+
+ override def replyOfType[R <: Reply: ClassTag]: R =
+ ofType(reply, "reply")
+
+ // cast with nice error message
+ private def ofType[A: ClassTag](obj: Any, errorParam: String): A = {
+ obj match {
+ case a: A => a
+ case other =>
+ val expectedClass = implicitly[ClassTag[A]].runtimeClass
+ throw new AssertionError(
+ s"Expected $errorParam class [${expectedClass.getName}], " +
+ s"but was [${other.getClass.getName}]")
+ }
+ }
+ }
+
+ final case class RestartResultImpl[State](state: State) extends RestartResult[State]
+}
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[akka] class EventSourcedBehaviorTestKitImpl[Command, Event, State](
+ actorTestKit: ActorTestKit,
+ behavior: Behavior[Command],
+ serializationSettings: SerializationSettings)
+ extends EventSourcedBehaviorTestKit[Command, Event, State] {
+
+ import EventSourcedBehaviorTestKitImpl._
+
+ private def system: ActorSystem[_] = actorTestKit.system
+
+ override val persistenceTestKit: PersistenceTestKit = PersistenceTestKit(system)
+
+ private val probe = actorTestKit.createTestProbe[Any]()
+ private val stateProbe = actorTestKit.createTestProbe[State]()
+ private var actor: ActorRef[Command] = actorTestKit.spawn(behavior)
+ private def internalActor = actor.unsafeUpcast[Any]
+ private val persistenceId: PersistenceId = {
+ internalActor ! EventSourcedBehaviorImpl.GetPersistenceId(probe.ref)
+ try {
+ probe.expectMessageType[PersistenceId]
+ } catch {
+ case NonFatal(_) =>
+ throw new IllegalArgumentException("Only EventSourcedBehavior, or nested EventSourcedBehavior allowed.")
+ }
+ }
+ private val serializationTestKit = new SerializationTestKit(system)
+
+ private var emptyStateVerified = false
+
+ persistenceTestKit.clearByPersistenceId(persistenceId.id)
+
+ override def runCommand(command: Command): CommandResult[Command, Event, State] = {
+ if (serializationSettings.enabled && serializationSettings.verifyCommands)
+ verifySerializationAndThrow(command, "Command")
+
+ if (serializationSettings.enabled && !emptyStateVerified) {
+ internalActor ! EventSourcedBehaviorImpl.GetState(stateProbe.ref)
+ val emptyState = stateProbe.receiveMessage()
+ verifySerializationAndThrow(emptyState, "Empty State")
+ emptyStateVerified = true
+ }
+
+ // FIXME we can expand the api of persistenceTestKit to read from storage from a seqNr instead
+ val oldEvents =
+ persistenceTestKit.persistedInStorage(persistenceId.id).map(_.asInstanceOf[Event])
+
+ actor ! command
+
+ internalActor ! EventSourcedBehaviorImpl.GetState(stateProbe.ref)
+ val newState = stateProbe.receiveMessage()
+
+ val newEvents =
+ persistenceTestKit.persistedInStorage(persistenceId.id).map(_.asInstanceOf[Event]).drop(oldEvents.size)
+
+ if (serializationSettings.enabled) {
+ if (serializationSettings.verifyEvents) {
+ newEvents.foreach(verifySerializationAndThrow(_, "Event"))
+ }
+
+ if (serializationSettings.verifyState)
+ verifySerializationAndThrow(newState, "State")
+ }
+
+ CommandResultImpl[Command, Event, State, Nothing](command, newEvents, newState, None)
+ }
+
+ override def runCommand[R](creator: ActorRef[R] => Command): CommandResultWithReply[Command, Event, State, R] = {
+ val replyProbe = actorTestKit.createTestProbe[R]()
+ val command = creator(replyProbe.ref)
+ val result = runCommand(command)
+
+ val reply = try {
+ replyProbe.receiveMessage(Duration.Zero)
+ } catch {
+ case NonFatal(_) =>
+ throw new AssertionError(s"Missing expected reply for command [$command].")
+ } finally {
+ replyProbe.stop()
+ }
+
+ if (serializationSettings.enabled && serializationSettings.verifyCommands)
+ verifySerializationAndThrow(reply, "Reply")
+
+ CommandResultImpl[Command, Event, State, R](result.command, result.events, result.state, Some(reply))
+ }
+
+ override def restart(): RestartResult[State] = {
+ actorTestKit.stop(actor)
+ actor = actorTestKit.spawn(behavior)
+ internalActor ! EventSourcedBehaviorImpl.GetState(stateProbe.ref)
+ try {
+ val state = stateProbe.receiveMessage()
+ RestartResultImpl(state)
+ } catch {
+ case NonFatal(_) =>
+ throw new IllegalStateException("Could not restart. Maybe exception from event handler. See logs.")
+ }
+ }
+
+ override def clear(): Unit = {
+ persistenceTestKit.clearByPersistenceId(persistenceId.id)
+ restart()
+ }
+
+ private def verifySerializationAndThrow(obj: Any, errorMessagePrefix: String): Unit = {
+ try {
+ serializationTestKit.verifySerialization(obj, serializationSettings.verifyEquality)
+ } catch {
+ case NonFatal(exc) =>
+ throw new IllegalArgumentException(s"$errorMessagePrefix [$obj] isn't serializable.", exc)
+ }
+ }
+
+}
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/InMemStorageExtension.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/InMemStorageExtension.scala
index 7861fd6f3f..fb55631be1 100644
--- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/InMemStorageExtension.scala
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/InMemStorageExtension.scala
@@ -6,8 +6,8 @@ package akka.persistence.testkit.internal
import akka.actor.{ ActorSystem, ExtendedActorSystem, ExtensionId, ExtensionIdProvider }
import akka.annotation.InternalApi
-import akka.persistence.testkit.scaladsl.PersistenceTestKit
import akka.persistence.testkit.EventStorage
+import akka.persistence.testkit.scaladsl.PersistenceTestKit
/**
* INTERNAL API
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/PersistenceInitImpl.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/PersistenceInitImpl.scala
new file mode 100644
index 0000000000..bde769571d
--- /dev/null
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/PersistenceInitImpl.scala
@@ -0,0 +1,56 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.persistence.testkit.internal
+
+import java.util.concurrent.TimeUnit
+
+import akka.actor.ActorLogging
+import akka.actor.Props
+import akka.annotation.InternalApi
+import akka.persistence.PersistentActor
+import akka.persistence.RecoveryCompleted
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[akka] object PersistenceInitImpl {
+
+ def props(journalPluginId: String, snapshotPluginId: String, persistenceId: String): Props = {
+ Props(new PersistenceInitImpl(journalPluginId, snapshotPluginId, persistenceId))
+ }
+}
+
+/**
+ * INTERNAL API: Initialize a journal and snapshot plugin by starting this `PersistentActor`
+ * and send any message to it. It will reply to the `sender()` with the same message when
+ * recovery has completed.
+ */
+@InternalApi private[akka] class PersistenceInitImpl(
+ override val journalPluginId: String,
+ override val snapshotPluginId: String,
+ override val persistenceId: String)
+ extends PersistentActor
+ with ActorLogging {
+
+ private val startTime = System.nanoTime()
+
+ def receiveRecover: Receive = {
+ case RecoveryCompleted =>
+ log.debug(
+ "Initialization completed for journal [{}] and snapshot [{}] plugins, with persistenceId [{}], in [{} ms]",
+ journalPluginId,
+ snapshotPluginId,
+ persistenceId,
+ TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime))
+ case _ =>
+ }
+
+ def receiveCommand: Receive = {
+ case msg =>
+ // recovery has completed
+ sender() ! msg
+ context.stop(self)
+ }
+}
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SerializedEventStorageImpl.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SerializedEventStorageImpl.scala
index 88cd10c76d..9d25479dba 100644
--- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SerializedEventStorageImpl.scala
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SerializedEventStorageImpl.scala
@@ -4,14 +4,14 @@
package akka.persistence.testkit.internal
+import scala.util.Try
+
import akka.actor.{ ActorSystem, ExtendedActorSystem }
import akka.annotation.InternalApi
import akka.persistence.PersistentRepr
import akka.persistence.testkit.EventStorage
import akka.serialization.{ Serialization, SerializationExtension }
-import scala.util.Try
-
/**
* INTERNAL API
*/
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SnapshotStorageEmulatorExtension.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SnapshotStorageEmulatorExtension.scala
index cab5938335..276a72897a 100644
--- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SnapshotStorageEmulatorExtension.scala
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SnapshotStorageEmulatorExtension.scala
@@ -6,8 +6,8 @@ package akka.persistence.testkit.internal
import akka.actor.{ ActorSystem, ExtendedActorSystem, Extension, ExtensionId, ExtensionIdProvider }
import akka.annotation.InternalApi
-import akka.persistence.testkit.scaladsl.SnapshotTestKit
import akka.persistence.testkit.SnapshotStorage
+import akka.persistence.testkit.scaladsl.SnapshotTestKit
/**
* INTERNAL API
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/TestKitStorage.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/TestKitStorage.scala
index 76ab2a0c9c..0b3fd01981 100644
--- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/TestKitStorage.scala
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/TestKitStorage.scala
@@ -7,12 +7,12 @@ package akka.persistence.testkit.internal
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.atomic.AtomicReference
+import scala.collection.immutable
+
import akka.actor.Extension
import akka.annotation.InternalApi
import akka.persistence.testkit.ProcessingPolicy
-import scala.collection.immutable
-
/**
* INTERNAL API
*/
@@ -143,7 +143,7 @@ sealed trait PolicyOps[U] {
def setPolicy(policy: Policy): Unit = _processingPolicy.set(policy)
- def returnDefaultPolicy(): Unit = setPolicy(DefaultPolicy)
+ def resetPolicy(): Unit = setPolicy(DefaultPolicy)
}
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/javadsl/EventSourcedBehaviorTestKit.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/javadsl/EventSourcedBehaviorTestKit.scala
new file mode 100644
index 0000000000..3446225e76
--- /dev/null
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/javadsl/EventSourcedBehaviorTestKit.scala
@@ -0,0 +1,246 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.persistence.testkit.javadsl
+
+import java.util.{ List => JList }
+import java.util.function.{ Function => JFunction }
+
+import scala.reflect.ClassTag
+
+import com.typesafe.config.Config
+
+import akka.actor.typed.ActorRef
+import akka.actor.typed.ActorSystem
+import akka.actor.typed.Behavior
+import akka.annotation.ApiMayChange
+import akka.annotation.DoNotInherit
+import akka.persistence.testkit.scaladsl
+import akka.util.ccompat.JavaConverters._
+
+/**
+ * Testing of [[akka.persistence.typed.javadsl.EventSourcedBehavior]] implementations.
+ * It supports running one command at a time and you can assert that the synchronously returned result is as expected.
+ * The result contains the events emitted by the command and the new state after applying the events.
+ * It also has support for verifying the reply to a command.
+ *
+ * Serialization of commands, events and state are verified automatically.
+ */
+@ApiMayChange
+object EventSourcedBehaviorTestKit {
+
+ /**
+ * The configuration to be included in the configuration of the `ActorSystem`. Typically used as
+ * constructor parameter to `TestKitJunitResource`. The configuration enables the in-memory
+ * journal and snapshot storage.
+ */
+ val config: Config = scaladsl.EventSourcedBehaviorTestKit.config
+
+ val enabledSerializationSettings: SerializationSettings = new SerializationSettings(
+ enabled = true,
+ verifyEquality = false,
+ verifyCommands = true,
+ verifyEvents = true,
+ verifyState = true)
+
+ val disabledSerializationSettings: SerializationSettings = new SerializationSettings(
+ enabled = false,
+ verifyEquality = false,
+ verifyCommands = false,
+ verifyEvents = false,
+ verifyState = false)
+
+ /**
+ * Customization of which serialization checks that are performed.
+ * `equals` must be implemented (or using `case class`) when `verifyEquality` is enabled.
+ */
+ final class SerializationSettings(
+ val enabled: Boolean,
+ val verifyEquality: Boolean,
+ val verifyCommands: Boolean,
+ val verifyEvents: Boolean,
+ val verifyState: Boolean) {
+
+ def withEnabled(value: Boolean): SerializationSettings =
+ copy(enabled = value)
+
+ def withVerifyEquality(value: Boolean): SerializationSettings =
+ copy(verifyEquality = value)
+
+ def withVerifyCommands(value: Boolean): SerializationSettings =
+ copy(verifyCommands = value)
+
+ def withVerifyEvents(value: Boolean): SerializationSettings =
+ copy(verifyEvents = value)
+
+ def withVerifyState(value: Boolean): SerializationSettings =
+ copy(verifyState = value)
+
+ private def copy(
+ enabled: Boolean = this.enabled,
+ verifyEquality: Boolean = this.verifyEquality,
+ verifyCommands: Boolean = this.verifyCommands,
+ verifyEvents: Boolean = this.verifyEvents,
+ verifyState: Boolean = this.verifyState): SerializationSettings = {
+ new SerializationSettings(enabled, verifyEquality, verifyCommands, verifyEvents, verifyState)
+ }
+ }
+
+ /**
+ * Factory method to create a new EventSourcedBehaviorTestKit.
+ */
+ def create[Command, Event, State](
+ system: ActorSystem[_],
+ behavior: Behavior[Command]): EventSourcedBehaviorTestKit[Command, Event, State] =
+ create(system, behavior, enabledSerializationSettings)
+
+ /**
+ * Factory method to create a new EventSourcedBehaviorTestKit with custom [[SerializationSettings]].
+ *
+ * Note that `equals` must be implemented in the commands, events and state if `verifyEquality` is enabled.
+ */
+ def create[Command, Event, State](
+ system: ActorSystem[_],
+ behavior: Behavior[Command],
+ serializationSettings: SerializationSettings): EventSourcedBehaviorTestKit[Command, Event, State] = {
+ val scaladslSettings = new scaladsl.EventSourcedBehaviorTestKit.SerializationSettings(
+ enabled = serializationSettings.enabled,
+ verifyEquality = serializationSettings.verifyEquality,
+ verifyCommands = serializationSettings.verifyCommands,
+ verifyEvents = serializationSettings.verifyEvents,
+ verifyState = serializationSettings.verifyState)
+ new EventSourcedBehaviorTestKit(scaladsl.EventSourcedBehaviorTestKit(system, behavior, scaladslSettings))
+ }
+
+ /**
+ * The result of running a command.
+ */
+ @DoNotInherit class CommandResult[Command, Event, State](
+ delegate: scaladsl.EventSourcedBehaviorTestKit.CommandResult[Command, Event, State]) {
+
+ /**
+ * The command that was run.
+ */
+ def command: Command =
+ delegate.command
+
+ /**
+ * The events that were emitted by the command, and persisted.
+ * In many cases only one event is emitted and then it's more convenient to use [[CommandResult.event]]
+ * or [[CommandResult.eventOfType]].
+ */
+ def events: JList[Event] =
+ delegate.events.asJava
+
+ /**
+ * `true` if no events were emitted by the command.
+ */
+ def hasNoEvents: Boolean =
+ delegate.hasNoEvents
+
+ /**
+ * The first event. It will throw `AssertionError` if there is no event.
+ */
+ def event: Event =
+ delegate.event
+
+ /**
+ * The first event as a given expected type. It will throw `AssertionError` if there is no event or
+ * if the event is of a different type.
+ */
+ def eventOfType[E <: Event](eventClass: Class[E]): E =
+ delegate.eventOfType(ClassTag[E](eventClass))
+
+ /**
+ * The state after applying the events.
+ */
+ def state: State =
+ delegate.state
+
+ /**
+ * The state as a given expected type. It will throw `AssertionError` if the state is of a different type.
+ */
+ def stateOfType[S <: State](stateClass: Class[S]): S =
+ delegate.stateOfType(ClassTag[S](stateClass))
+ }
+
+ /**
+ * The result of running a command with a `ActorRef replyTo`, i.e. the `runCommand` with a
+ * `Function, Command>` parameter.
+ */
+ final class CommandResultWithReply[Command, Event, State, Reply](
+ delegate: scaladsl.EventSourcedBehaviorTestKit.CommandResultWithReply[Command, Event, State, Reply])
+ extends CommandResult[Command, Event, State](delegate) {
+
+ /**
+ * The reply. It will throw `AssertionError` if there was no reply.
+ */
+ def reply: Reply =
+ delegate.reply
+
+ /**
+ * The reply as a given expected type. It will throw `AssertionError` if there is no reply or
+ * if the reply is of a different type.
+ */
+ def replyOfType[R <: Reply](replyClass: Class[R]): R =
+ delegate.replyOfType(ClassTag[R](replyClass))
+ }
+
+ /**
+ * The result of restarting the behavior.
+ */
+ final class RestartResult[State](delegate: scaladsl.EventSourcedBehaviorTestKit.RestartResult[State]) {
+
+ /**
+ * The state after recovery.
+ */
+ def state: State =
+ delegate.state
+ }
+
+}
+
+@ApiMayChange
+final class EventSourcedBehaviorTestKit[Command, Event, State](
+ delegate: scaladsl.EventSourcedBehaviorTestKit[Command, Event, State]) {
+
+ import EventSourcedBehaviorTestKit._
+
+ private val _persistenceTestKit = new PersistenceTestKit(delegate.persistenceTestKit)
+
+ /**
+ * Run one command through the behavior. The returned result contains emitted events and the state
+ * after applying the events.
+ */
+ def runCommand(command: Command): CommandResult[Command, Event, State] =
+ new CommandResult(delegate.runCommand(command))
+
+ /**
+ * Run one command with a `replyTo: ActorRef` through the behavior. The returned result contains emitted events,
+ * the state after applying the events, and the reply.
+ */
+ def runCommand[R](creator: JFunction[ActorRef[R], Command]): CommandResultWithReply[Command, Event, State, R] =
+ new CommandResultWithReply(delegate.runCommand(replyTo => creator.apply(replyTo)))
+
+ /**
+ * Restart the behavior, which will then recover from stored snapshot and events. Can be used for testing
+ * that the recovery is correct.
+ */
+ def restart(): RestartResult[State] =
+ new RestartResult(delegate.restart())
+
+ /**
+ * Clears the in-memory journal and snapshot storage and restarts the behavior.
+ */
+ def clear(): Unit =
+ delegate.clear()
+
+ /**
+ * The underlying `PersistenceTestKit` for the in-memory journal and snapshot storage.
+ * Can be useful for advanced testing scenarios, such as simulating failures or
+ * populating the journal with events that are used for replay.
+ */
+ def persistenceTestKit: PersistenceTestKit =
+ _persistenceTestKit
+}
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/javadsl/PersistenceInit.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/javadsl/PersistenceInit.scala
new file mode 100644
index 0000000000..7f1c62dc7a
--- /dev/null
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/javadsl/PersistenceInit.scala
@@ -0,0 +1,45 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.persistence.testkit.javadsl
+
+import java.time.Duration
+import java.util.concurrent.CompletionStage
+
+import scala.compat.java8.FutureConverters._
+
+import akka.Done
+import akka.actor.ClassicActorSystemProvider
+import akka.persistence.testkit.scaladsl
+import akka.util.JavaDurationConverters._
+
+/**
+ * Test utility to initialize persistence plugins. Useful when initialization order or coordination
+ * is needed. For example to avoid creating tables concurrently.
+ */
+object PersistenceInit {
+
+ /**
+ * Initialize the default journal and snapshot plugins.
+ *
+ * @return a `CompletionStage` that is completed when the initialization has completed
+ */
+ def initializeDefaultPlugins(system: ClassicActorSystemProvider, timeout: Duration): CompletionStage[Done] =
+ initializePlugins(system, journalPluginId = "", snapshotPluginId = "", timeout)
+
+ /**
+ * Initialize the given journal and snapshot plugins.
+ *
+ * The `snapshotPluginId` can be empty (`""`) if snapshot plugin isn't used.
+ *
+ * @return a `CompletionStage` that is completed when the initialization has completed
+ */
+ def initializePlugins(
+ system: ClassicActorSystemProvider,
+ journalPluginId: String,
+ snapshotPluginId: String,
+ timeout: Duration): CompletionStage[Done] =
+ scaladsl.PersistenceInit.initializePlugins(system, journalPluginId, snapshotPluginId, timeout.asScala).toJava
+
+}
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/javadsl/PersistenceTestKit.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/javadsl/PersistenceTestKit.scala
index f540b1d143..cd91ff12c4 100644
--- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/javadsl/PersistenceTestKit.scala
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/javadsl/PersistenceTestKit.scala
@@ -4,24 +4,24 @@
package akka.persistence.testkit.javadsl
-import akka.actor.ActorSystem
-import akka.persistence.testkit.scaladsl.{ PersistenceTestKit => ScalaTestKit }
-import akka.util.JavaDurationConverters._
-import akka.util.ccompat.JavaConverters._
import java.time.Duration
import java.util.{ List => JList }
import java.util.{ function => jf }
+import akka.actor.ActorSystem
import akka.annotation.ApiMayChange
import akka.persistence.testkit.{ EventStorage, ExpectedFailure, ExpectedRejection, JournalOperation }
+import akka.persistence.testkit.scaladsl.{ PersistenceTestKit => ScalaTestKit }
+import akka.util.JavaDurationConverters._
+import akka.util.ccompat.JavaConverters._
/**
* Class for testing persisted events in persistent actors.
*/
@ApiMayChange
-class PersistenceTestKit(system: ActorSystem) {
+class PersistenceTestKit(scalaTestkit: ScalaTestKit) {
- private val scalaTestkit = new ScalaTestKit(system)
+ def this(system: ActorSystem) = this(new ScalaTestKit(system))
/**
* Check that nothing has been saved in the storage.
@@ -431,7 +431,7 @@ class PersistenceTestKit(system: ActorSystem) {
/**
* Returns default policy if it was changed by [[PersistenceTestKit.withPolicy()]].
*/
- def returnDefaultPolicy(): Unit = scalaTestkit.returnDefaultPolicy()
+ def resetPolicy(): Unit = scalaTestkit.resetPolicy()
}
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/javadsl/SnapshotTestKit.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/javadsl/SnapshotTestKit.scala
index 4e3b0f4e50..275ce5282a 100644
--- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/javadsl/SnapshotTestKit.scala
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/javadsl/SnapshotTestKit.scala
@@ -4,17 +4,17 @@
package akka.persistence.testkit.javadsl
-import akka.actor.ActorSystem
-import akka.persistence.testkit.scaladsl.{ SnapshotTestKit => ScalaTestKit }
-import akka.persistence.testkit.{ ExpectedFailure, SnapshotMeta, SnapshotOperation, SnapshotStorage }
-import akka.util.JavaDurationConverters._
-import akka.util.ccompat.JavaConverters._
import java.time.Duration
import java.util.{ List => JList }
import java.util.{ function => jf }
+import akka.actor.ActorSystem
import akka.annotation.ApiMayChange
import akka.japi.Pair
+import akka.persistence.testkit.{ ExpectedFailure, SnapshotMeta, SnapshotOperation, SnapshotStorage }
+import akka.persistence.testkit.scaladsl.{ SnapshotTestKit => ScalaTestKit }
+import akka.util.JavaDurationConverters._
+import akka.util.ccompat.JavaConverters._
/**
* Class for testing persisted snapshots in persistent actors.
@@ -258,7 +258,7 @@ class SnapshotTestKit(system: ActorSystem) {
/**
* Returns default policy if it was changed by [[SnapshotTestKit.withPolicy()]].
*/
- def returnDefaultPolicy(): Unit = scalaTestkit.returnDefaultPolicy()
+ def resetPolicy(): Unit = scalaTestkit.resetPolicy()
}
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/EventSourcedBehaviorTestKit.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/EventSourcedBehaviorTestKit.scala
new file mode 100644
index 0000000000..2aed301b88
--- /dev/null
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/EventSourcedBehaviorTestKit.scala
@@ -0,0 +1,224 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.persistence.testkit.scaladsl
+
+import scala.collection.immutable
+import scala.reflect.ClassTag
+
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+
+import akka.actor.testkit.typed.scaladsl.ActorTestKit
+import akka.actor.typed.ActorRef
+import akka.actor.typed.ActorSystem
+import akka.actor.typed.Behavior
+import akka.annotation.ApiMayChange
+import akka.annotation.DoNotInherit
+import akka.persistence.testkit.PersistenceTestKitPlugin
+import akka.persistence.testkit.internal.EventSourcedBehaviorTestKitImpl
+
+/**
+ * Testing of [[akka.persistence.typed.scaladsl.EventSourcedBehavior]] implementations.
+ * It supports running one command at a time and you can assert that the synchronously returned result is as expected.
+ * The result contains the events emitted by the command and the new state after applying the events.
+ * It also has support for verifying the reply to a command.
+ *
+ * Serialization of commands, events and state are verified automatically.
+ */
+@ApiMayChange
+object EventSourcedBehaviorTestKit {
+
+ /**
+ * The configuration to be included in the configuration of the `ActorSystem`. Typically used as
+ * constructor parameter to `ScalaTestWithActorTestKit`. The configuration enables the in-memory
+ * journal and snapshot storage.
+ */
+ val config: Config = ConfigFactory.parseString("""
+ akka.persistence.testkit.events.serialize = off
+ """).withFallback(PersistenceTestKitPlugin.config)
+
+ object SerializationSettings {
+ val enabled: SerializationSettings = new SerializationSettings(
+ enabled = true,
+ verifyEquality = false,
+ verifyCommands = true,
+ verifyEvents = true,
+ verifyState = true)
+
+ val disabled: SerializationSettings = new SerializationSettings(
+ enabled = false,
+ verifyEquality = false,
+ verifyCommands = false,
+ verifyEvents = false,
+ verifyState = false)
+ }
+
+ /**
+ * Customization of which serialization checks that are performed.
+ * `equals` must be implemented (or using `case class`) when `verifyEquality` is enabled.
+ */
+ final class SerializationSettings private[akka] (
+ val enabled: Boolean,
+ val verifyEquality: Boolean,
+ val verifyCommands: Boolean,
+ val verifyEvents: Boolean,
+ val verifyState: Boolean) {
+
+ def withEnabled(value: Boolean): SerializationSettings =
+ copy(enabled = value)
+
+ def withVerifyEquality(value: Boolean): SerializationSettings =
+ copy(verifyEquality = value)
+
+ def withVerifyCommands(value: Boolean): SerializationSettings =
+ copy(verifyCommands = value)
+
+ def withVerifyEvents(value: Boolean): SerializationSettings =
+ copy(verifyEvents = value)
+
+ def withVerifyState(value: Boolean): SerializationSettings =
+ copy(verifyState = value)
+
+ private def copy(
+ enabled: Boolean = this.enabled,
+ verifyEquality: Boolean = this.verifyEquality,
+ verifyCommands: Boolean = this.verifyCommands,
+ verifyEvents: Boolean = this.verifyEvents,
+ verifyState: Boolean = this.verifyState): SerializationSettings = {
+ new SerializationSettings(enabled, verifyEquality, verifyCommands, verifyEvents, verifyState)
+ }
+ }
+
+ /**
+ * Factory method to create a new EventSourcedBehaviorTestKit.
+ */
+ def apply[Command, Event, State](
+ system: ActorSystem[_],
+ behavior: Behavior[Command]): EventSourcedBehaviorTestKit[Command, Event, State] =
+ apply(system, behavior, SerializationSettings.enabled)
+
+ /**
+ * Factory method to create a new EventSourcedBehaviorTestKit with custom [[SerializationSettings]].
+ *
+ * Note that `equals` must be implemented (or using `case class`) in the commands, events and state if
+ * `verifyEquality` is enabled.
+ */
+ def apply[Command, Event, State](
+ system: ActorSystem[_],
+ behavior: Behavior[Command],
+ serializationSettings: SerializationSettings): EventSourcedBehaviorTestKit[Command, Event, State] =
+ new EventSourcedBehaviorTestKitImpl(ActorTestKit(system), behavior, serializationSettings)
+
+ /**
+ * The result of running a command.
+ */
+ @DoNotInherit trait CommandResult[Command, Event, State] {
+
+ /**
+ * The command that was run.
+ */
+ def command: Command
+
+ /**
+ * The events that were emitted by the command, and persisted.
+ * In many cases only one event is emitted and then it's more convenient to use [[CommandResult.event]]
+ * or [[CommandResult.eventOfType]].
+ */
+ def events: immutable.Seq[Event]
+
+ /**
+ * `true` if no events were emitted by the command.
+ */
+ def hasNoEvents: Boolean
+
+ /**
+ * The first event. It will throw `AssertionError` if there is no event.
+ */
+ def event: Event
+
+ /**
+ * The first event as a given expected type. It will throw `AssertionError` if there is no event or
+ * if the event is of a different type.
+ */
+ def eventOfType[E <: Event: ClassTag]: E
+
+ /**
+ * The state after applying the events.
+ */
+ def state: State
+
+ /**
+ * The state as a given expected type. It will throw `AssertionError` if the state is of a different type.
+ */
+ def stateOfType[S <: State: ClassTag]: S
+ }
+
+ /**
+ * The result of running a command with a `replyTo: ActorRef[R]`, i.e. the `runCommand` with a
+ * `ActorRef[R] => Command` parameter.
+ */
+ @DoNotInherit trait CommandResultWithReply[Command, Event, State, Reply]
+ extends CommandResult[Command, Event, State] {
+
+ /**
+ * The reply. It will throw `AssertionError` if there was no reply.
+ */
+ def reply: Reply
+
+ /**
+ * The reply as a given expected type. It will throw `AssertionError` if there is no reply or
+ * if the reply is of a different type.
+ */
+ def replyOfType[R <: Reply: ClassTag]: R
+ }
+
+ /**
+ * The result of restarting the behavior.
+ */
+ @DoNotInherit trait RestartResult[State] {
+
+ /**
+ * The state after recovery.
+ */
+ def state: State
+ }
+
+}
+
+@ApiMayChange
+@DoNotInherit trait EventSourcedBehaviorTestKit[Command, Event, State] {
+
+ import EventSourcedBehaviorTestKit._
+
+ /**
+ * Run one command through the behavior. The returned result contains emitted events and the state
+ * after applying the events.
+ */
+ def runCommand(command: Command): CommandResult[Command, Event, State]
+
+ /**
+ * Run one command with a `replyTo: ActorRef[R]` through the behavior. The returned result contains emitted events,
+ * the state after applying the events, and the reply.
+ */
+ def runCommand[R](creator: ActorRef[R] => Command): CommandResultWithReply[Command, Event, State, R]
+
+ /**
+ * Restart the behavior, which will then recover from stored snapshot and events. Can be used for testing
+ * that the recovery is correct.
+ */
+ def restart(): RestartResult[State]
+
+ /**
+ * Clears the in-memory journal and snapshot storage and restarts the behavior.
+ */
+ def clear(): Unit
+
+ /**
+ * The underlying `PersistenceTestKit` for the in-memory journal and snapshot storage.
+ * Can be useful for advanced testing scenarios, such as simulating failures or
+ * populating the journal with events that are used for replay.
+ */
+ def persistenceTestKit: PersistenceTestKit
+}
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/PersistenceInit.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/PersistenceInit.scala
new file mode 100644
index 0000000000..4216894ce8
--- /dev/null
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/PersistenceInit.scala
@@ -0,0 +1,56 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.persistence.testkit.scaladsl
+
+import java.util.UUID
+
+import scala.concurrent.Future
+import scala.concurrent.duration.FiniteDuration
+
+import akka.Done
+import akka.actor.ClassicActorSystemProvider
+import akka.actor.ExtendedActorSystem
+import akka.persistence.testkit.internal.PersistenceInitImpl
+import akka.util.Timeout
+
+/**
+ * Test utility to initialize persistence plugins. Useful when initialization order or coordination
+ * is needed. For example to avoid creating tables concurrently.
+ */
+object PersistenceInit {
+
+ /**
+ * Initialize the default journal and snapshot plugins.
+ *
+ * @return a `Future` that is completed when the initialization has completed
+ */
+ def initializeDefaultPlugins(system: ClassicActorSystemProvider, timeout: FiniteDuration): Future[Done] =
+ initializePlugins(system, journalPluginId = "", snapshotPluginId = "", timeout)
+
+ /**
+ * Initialize the given journal and snapshot plugins.
+ *
+ * The `snapshotPluginId` can be empty (`""`) if snapshot plugin isn't used.
+ *
+ * @return a `Future` that is completed when the initialization has completed
+ */
+ def initializePlugins(
+ system: ClassicActorSystemProvider,
+ journalPluginId: String,
+ snapshotPluginId: String,
+ timeout: FiniteDuration): Future[Done] = {
+ val persistenceId: String = s"persistenceInit-${UUID.randomUUID()}"
+ val extSystem = system.classicSystem.asInstanceOf[ExtendedActorSystem]
+ val ref =
+ extSystem.systemActorOf(
+ PersistenceInitImpl.props(journalPluginId, snapshotPluginId, persistenceId),
+ persistenceId)
+ import extSystem.dispatcher
+
+ import akka.pattern.ask
+ implicit val askTimeout: Timeout = timeout
+ (ref ? "start").map(_ => Done)
+ }
+}
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/PersistenceTestKit.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/PersistenceTestKit.scala
index cbdca4a262..aae6f4de07 100644
--- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/PersistenceTestKit.scala
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/PersistenceTestKit.scala
@@ -4,19 +4,27 @@
package akka.persistence.testkit.scaladsl
-import akka.actor.{ ActorSystem, ExtendedActorSystem, Extension, ExtensionId }
-import akka.actor.typed.{ ActorSystem => TypedActorSystem }
-import akka.annotation.ApiMayChange
-import akka.persistence.testkit._
-import akka.persistence.testkit.internal.{ InMemStorageExtension, SnapshotStorageEmulatorExtension }
-import akka.persistence.{ Persistence, PersistentRepr, SnapshotMetadata }
-import akka.testkit.TestProbe
-import com.typesafe.config.Config
-
import scala.collection.immutable
import scala.concurrent.duration.FiniteDuration
import scala.util.Try
+import com.typesafe.config.Config
+
+import akka.actor.ActorSystem
+import akka.actor.ClassicActorSystemProvider
+import akka.actor.ExtendedActorSystem
+import akka.actor.Extension
+import akka.actor.ExtensionId
+import akka.actor.typed.{ ActorSystem => TypedActorSystem }
+import akka.annotation.ApiMayChange
+import akka.persistence.Persistence
+import akka.persistence.PersistentRepr
+import akka.persistence.SnapshotMetadata
+import akka.persistence.testkit._
+import akka.persistence.testkit.internal.InMemStorageExtension
+import akka.persistence.testkit.internal.SnapshotStorageEmulatorExtension
+import akka.testkit.TestProbe
+
private[testkit] trait CommonTestKitOps[S, P] extends ClearOps with PolicyOpsTestKit[P] {
this: HasStorage[P, S] =>
@@ -292,7 +300,7 @@ private[testkit] trait PersistenceTestKitOps[S, P]
/**
* Persist `snapshots` into storage in order.
*/
- def persistForRecovery(persistenceId: String, snapshots: immutable.Seq[Any]): Unit
+ def persistForRecovery(persistenceId: String, events: immutable.Seq[Any]): Unit
/**
* Retrieve all snapshots saved in storage by persistence id.
@@ -479,9 +487,9 @@ class PersistenceTestKit(system: ActorSystem)
override def failNextNDeletes(persistenceId: String, n: Int, cause: Throwable): Unit =
failNextNOpsCond((pid, op) => pid == persistenceId && op.isInstanceOf[DeleteEvents], n, cause)
- def persistForRecovery(persistenceId: String, snapshots: immutable.Seq[Any]): Unit = {
- storage.addAny(persistenceId, snapshots)
- addToIndex(persistenceId, snapshots.size)
+ def persistForRecovery(persistenceId: String, events: immutable.Seq[Any]): Unit = {
+ storage.addAny(persistenceId, events)
+ addToIndex(persistenceId, events.size)
}
def persistedInStorage(persistenceId: String): immutable.Seq[Any] =
@@ -493,9 +501,7 @@ class PersistenceTestKit(system: ActorSystem)
@ApiMayChange
object PersistenceTestKit {
- def apply(system: ActorSystem): PersistenceTestKit = new PersistenceTestKit(system)
-
- def apply(system: TypedActorSystem[_]): PersistenceTestKit = apply(system.classicSystem)
+ def apply(system: ClassicActorSystemProvider): PersistenceTestKit = new PersistenceTestKit(system.classicSystem)
object Settings extends ExtensionId[Settings] {
diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/TestOps.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/TestOps.scala
index 4028b84e5e..89b6251c1d 100644
--- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/TestOps.scala
+++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/TestOps.scala
@@ -4,17 +4,17 @@
package akka.persistence.testkit.scaladsl
-import akka.persistence.testkit.ProcessingPolicy.DefaultPolicies
-import akka.persistence.testkit.internal.TestKitStorage
-import akka.persistence.testkit.{ ExpectedFailure, ExpectedRejection }
-import akka.testkit.TestKitBase
-import akka.util
-import akka.util.BoxedType
-
import scala.collection.immutable
import scala.concurrent.duration.FiniteDuration
import scala.reflect.ClassTag
+import akka.persistence.testkit.{ ExpectedFailure, ExpectedRejection }
+import akka.persistence.testkit.ProcessingPolicy.DefaultPolicies
+import akka.persistence.testkit.internal.TestKitStorage
+import akka.testkit.TestKitBase
+import akka.util
+import akka.util.BoxedType
+
private[testkit] trait RejectSupport[U] {
this: PolicyOpsTestKit[U] with HasStorage[U, _] =>
@@ -109,7 +109,7 @@ private[testkit] trait PolicyOpsTestKit[P] extends {
/**
* Returns default policy if it was changed by [[PolicyOpsTestKit.this.withPolicy()]].
*/
- def returnDefaultPolicy(): Unit = storage.returnDefaultPolicy()
+ def resetPolicy(): Unit = storage.resetPolicy()
}
@@ -119,6 +119,7 @@ private[testkit] trait ExpectOps[U] {
private[testkit] val probe: TestKitBase
import probe._
+
import akka.testkit._
private[testkit] def pollInterval: FiniteDuration
diff --git a/akka-persistence-testkit/src/test/resources/logback-test.xml b/akka-persistence-testkit/src/test/resources/logback-test.xml
new file mode 100644
index 0000000000..3d08c776bd
--- /dev/null
+++ b/akka-persistence-testkit/src/test/resources/logback-test.xml
@@ -0,0 +1,31 @@
+
+
+
+
+
+
+
+ %date{ISO8601} %-5level %logger %marker - %msg MDC: {%mdc}%n
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/CommonUtils.scala b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/CommonUtils.scala
index 3c6694c0e9..f8a47a60e9 100644
--- a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/CommonUtils.scala
+++ b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/CommonUtils.scala
@@ -6,11 +6,12 @@ package akka.persistence.testkit
import java.util.UUID
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.{ ActorRef, ActorSystem }
import akka.persistence._
import akka.testkit.TestKitBase
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
trait CommonUtils extends AnyWordSpecLike with TestKitBase {
diff --git a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/javadsl/CommonSnapshotTests.scala b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/javadsl/CommonSnapshotTests.scala
index b5a4d15c13..40028e9ff8 100644
--- a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/javadsl/CommonSnapshotTests.scala
+++ b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/javadsl/CommonSnapshotTests.scala
@@ -4,14 +4,15 @@
package akka.persistence.testkit.javadsl
+import org.scalatest.matchers.should.Matchers._
+
import akka.actor.Props
+import akka.actor.typed.javadsl.Adapter
+import akka.japi.Pair
import akka.persistence._
import akka.persistence.testkit._
import akka.testkit.EventFilter
-import org.scalatest.matchers.should.Matchers._
import akka.util.ccompat.JavaConverters._
-import akka.japi.Pair
-import akka.actor.typed.javadsl.Adapter
trait CommonSnapshotTests extends JavaDslUtils {
@@ -91,7 +92,7 @@ trait CommonSnapshotTests extends JavaDslUtils {
expectMsgPF() { case SaveSnapshotSuccess(_) => }
expectNextPersisted(pid, 777)
- returnDefaultPolicy()
+ resetPolicy()
}
diff --git a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/javadsl/CommonTestKitTests.scala b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/javadsl/CommonTestKitTests.scala
index 1712bb37b2..1ceb61f662 100644
--- a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/javadsl/CommonTestKitTests.scala
+++ b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/javadsl/CommonTestKitTests.scala
@@ -4,13 +4,14 @@
package akka.persistence.testkit.javadsl
+import org.scalatest.matchers.should.Matchers._
+
import akka.actor.Props
+import akka.actor.typed.javadsl.Adapter
import akka.persistence._
import akka.persistence.testkit._
import akka.testkit.EventFilter
-import org.scalatest.matchers.should.Matchers._
import akka.util.ccompat.JavaConverters._
-import akka.actor.typed.javadsl.Adapter
trait CommonTestKitTests extends JavaDslUtils {
@@ -134,7 +135,7 @@ trait CommonTestKitTests extends JavaDslUtils {
expectNextPersisted(pid, B(666))
- returnDefaultPolicy()
+ resetPolicy()
}
diff --git a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/javadsl/JavaDslUtils.scala b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/javadsl/JavaDslUtils.scala
index 3bd6c127b0..640416af9d 100644
--- a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/javadsl/JavaDslUtils.scala
+++ b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/javadsl/JavaDslUtils.scala
@@ -7,9 +7,9 @@ package akka.persistence.testkit.javadsl
import java.util
import akka.actor.typed.ActorRef
-import akka.persistence.typed.javadsl.{ CommandHandler, EventHandler, EventSourcedBehavior, SignalHandler }
import akka.persistence.testkit.{ Cmd, CommonUtils, EmptyState, Evt, Passivate, Recovered, Stopped, TestCommand }
import akka.persistence.typed.{ PersistenceId, RecoveryCompleted }
+import akka.persistence.typed.javadsl.{ CommandHandler, EventHandler, EventSourcedBehavior, SignalHandler }
trait JavaDslUtils extends CommonUtils {
diff --git a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/CommonSnapshotTests.scala b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/CommonSnapshotTests.scala
index 5336d6ba5d..08bb5ed021 100644
--- a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/CommonSnapshotTests.scala
+++ b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/CommonSnapshotTests.scala
@@ -4,12 +4,13 @@
package akka.persistence.testkit.scaladsl
+import org.scalatest.matchers.should.Matchers._
+
import akka.actor.Props
+import akka.actor.typed.scaladsl.adapter._
import akka.persistence._
import akka.persistence.testkit._
import akka.testkit.EventFilter
-import org.scalatest.matchers.should.Matchers._
-import akka.actor.typed.scaladsl.adapter._
trait CommonSnapshotTests extends ScalaDslUtils {
@@ -111,7 +112,7 @@ trait CommonSnapshotTests extends ScalaDslUtils {
expectMsgPF() { case SaveSnapshotSuccess(_) => }
expectNextPersisted(pid, 777)
- returnDefaultPolicy()
+ resetPolicy()
}
diff --git a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/CommonTestKitTests.scala b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/CommonTestKitTests.scala
index 5558b6a63a..6f0acc9c7c 100644
--- a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/CommonTestKitTests.scala
+++ b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/CommonTestKitTests.scala
@@ -4,12 +4,13 @@
package akka.persistence.testkit.scaladsl
-import akka.actor.Props
-import akka.persistence.{ DeleteMessagesFailure, DeleteMessagesSuccess }
-import akka.testkit.EventFilter
-import akka.persistence.testkit._
import org.scalatest.matchers.should.Matchers._
+
+import akka.actor.Props
import akka.actor.typed.scaladsl.adapter._
+import akka.persistence.{ DeleteMessagesFailure, DeleteMessagesSuccess }
+import akka.persistence.testkit._
+import akka.testkit.EventFilter
trait CommonTestKitTests extends ScalaDslUtils {
@@ -150,7 +151,7 @@ trait CommonTestKitTests extends ScalaDslUtils {
aa ! B(666)
expectNextPersisted(pid, B(666))
- returnDefaultPolicy()
+ resetPolicy()
}
diff --git a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/EventSourcedBehaviorTestKitSpec.scala b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/EventSourcedBehaviorTestKitSpec.scala
new file mode 100644
index 0000000000..39869b475d
--- /dev/null
+++ b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/EventSourcedBehaviorTestKitSpec.scala
@@ -0,0 +1,302 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.persistence.testkit.scaladsl
+
+import java.io.NotSerializableException
+
+import org.scalatest.wordspec.AnyWordSpecLike
+
+import akka.Done
+import akka.actor.testkit.typed.TestException
+import akka.actor.testkit.typed.scaladsl.LogCapturing
+import akka.actor.testkit.typed.scaladsl.LoggingTestKit
+import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
+import akka.actor.typed.ActorRef
+import akka.actor.typed.Behavior
+import akka.actor.typed.scaladsl.ActorContext
+import akka.actor.typed.scaladsl.Behaviors
+import akka.persistence.testkit.scaladsl.EventSourcedBehaviorTestKitSpec.TestCounter.NotSerializableState
+import akka.persistence.typed.PersistenceId
+import akka.persistence.typed.internal.JournalFailureException
+import akka.persistence.typed.scaladsl.Effect
+import akka.persistence.typed.scaladsl.EventSourcedBehavior
+import akka.serialization.DisabledJavaSerializer
+import akka.serialization.jackson.CborSerializable
+import akka.util.unused
+
+object EventSourcedBehaviorTestKitSpec {
+
+ object TestCounter {
+ sealed trait Command
+ case object Increment extends Command with CborSerializable
+ final case class IncrementWithConfirmation(replyTo: ActorRef[Done]) extends Command with CborSerializable
+ case class IncrementSeveral(n: Int) extends Command with CborSerializable
+ final case class GetValue(replyTo: ActorRef[State]) extends Command with CborSerializable
+
+ sealed trait Event
+ final case class Incremented(delta: Int) extends Event with CborSerializable
+
+ sealed trait State
+ final case class RealState(value: Int, history: Vector[Int]) extends State with CborSerializable
+
+ case object IncrementWithNotSerializableEvent extends Command with CborSerializable
+ final case class NotSerializableEvent(delta: Int) extends Event
+
+ case object IncrementWithNotSerializableState extends Command with CborSerializable
+ final case class IncrementedWithNotSerializableState(delta: Int) extends Event with CborSerializable
+ final case class NotSerializableState(value: Int, history: Vector[Int]) extends State
+
+ case object NotSerializableCommand extends Command
+
+ final case class IncrementWithNotSerializableReply(replyTo: ActorRef[NotSerializableReply.type])
+ extends Command
+ with CborSerializable
+ object NotSerializableReply
+
+ def apply(persistenceId: PersistenceId): Behavior[Command] =
+ apply(persistenceId, RealState(0, Vector.empty))
+
+ def apply(persistenceId: PersistenceId, emptyState: State): Behavior[Command] =
+ Behaviors.setup(ctx => counter(ctx, persistenceId, emptyState))
+
+ private def counter(
+ @unused ctx: ActorContext[Command],
+ persistenceId: PersistenceId,
+ emptyState: State): EventSourcedBehavior[Command, Event, State] = {
+ EventSourcedBehavior.withEnforcedReplies[Command, Event, State](
+ persistenceId,
+ emptyState,
+ commandHandler = (state, command) =>
+ command match {
+ case Increment =>
+ Effect.persist(Incremented(1)).thenNoReply()
+
+ case IncrementWithConfirmation(replyTo) =>
+ Effect.persist(Incremented(1)).thenReply(replyTo)(_ => Done)
+
+ case IncrementSeveral(n: Int) =>
+ val events = (1 to n).map(_ => Incremented(1))
+ Effect.persist(events).thenNoReply()
+
+ case IncrementWithNotSerializableEvent =>
+ Effect.persist(NotSerializableEvent(1)).thenNoReply()
+
+ case IncrementWithNotSerializableState =>
+ Effect.persist(IncrementedWithNotSerializableState(1)).thenNoReply()
+
+ case IncrementWithNotSerializableReply(replyTo) =>
+ Effect.persist(Incremented(1)).thenReply(replyTo)(_ => NotSerializableReply)
+
+ case NotSerializableCommand =>
+ Effect.noReply
+
+ case GetValue(replyTo) =>
+ Effect.reply(replyTo)(state)
+
+ },
+ eventHandler = {
+ case (RealState(value, history), Incremented(delta)) =>
+ if (delta <= 0)
+ throw new IllegalStateException("Delta must be positive")
+ RealState(value + delta, history :+ value)
+ case (RealState(value, history), NotSerializableEvent(delta)) =>
+ RealState(value + delta, history :+ value)
+ case (RealState(value, history), IncrementedWithNotSerializableState(delta)) =>
+ NotSerializableState(value + delta, history :+ value)
+ case (state: NotSerializableState, _) =>
+ throw new IllegalStateException(state.toString)
+ })
+ }
+ }
+}
+
+class EventSourcedBehaviorTestKitSpec
+ extends ScalaTestWithActorTestKit(EventSourcedBehaviorTestKit.config)
+ with AnyWordSpecLike
+ with LogCapturing {
+ import EventSourcedBehaviorTestKitSpec._
+
+ private val persistenceId = PersistenceId.ofUniqueId("test")
+ private val behavior = TestCounter(persistenceId)
+
+ private def createTestKit() = {
+ EventSourcedBehaviorTestKit[TestCounter.Command, TestCounter.Event, TestCounter.State](system, behavior)
+ }
+
+ "EventSourcedBehaviorTestKit" must {
+
+ "run commands" in {
+ val eventSourcedTestKit = createTestKit()
+
+ val result1 = eventSourcedTestKit.runCommand(TestCounter.Increment)
+ result1.event should ===(TestCounter.Incremented(1))
+ result1.state should ===(TestCounter.RealState(1, Vector(0)))
+
+ val result2 = eventSourcedTestKit.runCommand(TestCounter.Increment)
+ result2.event should ===(TestCounter.Incremented(1))
+ result2.state should ===(TestCounter.RealState(2, Vector(0, 1)))
+
+ result2.eventOfType[TestCounter.Incremented].delta should ===(1)
+ intercept[AssertionError] {
+ // wrong event type
+ result2.eventOfType[TestCounter.NotSerializableEvent].delta should ===(1)
+ }
+ }
+
+ "run command emitting several events" in {
+ val eventSourcedTestKit = createTestKit()
+
+ val result = eventSourcedTestKit.runCommand(TestCounter.IncrementSeveral(3))
+ result.events should ===(List(TestCounter.Incremented(1), TestCounter.Incremented(1), TestCounter.Incremented(1)))
+ result.state should ===(TestCounter.RealState(3, Vector(0, 1, 2)))
+ }
+
+ "run commands with reply" in {
+ val eventSourcedTestKit = createTestKit()
+
+ val result1 = eventSourcedTestKit.runCommand[Done](TestCounter.IncrementWithConfirmation(_))
+ result1.event should ===(TestCounter.Incremented(1))
+ result1.state should ===(TestCounter.RealState(1, Vector(0)))
+ result1.reply should ===(Done)
+
+ val result2 = eventSourcedTestKit.runCommand[Done](TestCounter.IncrementWithConfirmation(_))
+ result2.event should ===(TestCounter.Incremented(1))
+ result2.state should ===(TestCounter.RealState(2, Vector(0, 1)))
+ result2.reply should ===(Done)
+ }
+
+ "detect missing reply" in {
+ val eventSourcedTestKit = createTestKit()
+
+ intercept[AssertionError] {
+ eventSourcedTestKit.runCommand[Done](_ => TestCounter.Increment)
+ }.getMessage should include("Missing expected reply")
+ }
+
+ "run command with reply that is not emitting events" in {
+ val eventSourcedTestKit = createTestKit()
+
+ eventSourcedTestKit.runCommand(TestCounter.Increment)
+ val result = eventSourcedTestKit.runCommand[TestCounter.State](TestCounter.GetValue(_))
+ result.hasNoEvents should ===(true)
+ intercept[AssertionError] {
+ result.event
+ }
+ result.state should ===(TestCounter.RealState(1, Vector(0)))
+ }
+
+ "detect non-serializable events" in {
+ val eventSourcedTestKit = createTestKit()
+
+ val exc = intercept[IllegalArgumentException] {
+ eventSourcedTestKit.runCommand(TestCounter.IncrementWithNotSerializableEvent)
+ }
+ (exc.getMessage should startWith).regex("Event.*isn't serializable")
+ exc.getCause.getClass should ===(classOf[DisabledJavaSerializer.JavaSerializationException])
+ }
+
+ "detect non-serializable state" in {
+ val eventSourcedTestKit = createTestKit()
+
+ val exc = intercept[IllegalArgumentException] {
+ eventSourcedTestKit.runCommand(TestCounter.IncrementWithNotSerializableState)
+ }
+ (exc.getMessage should include).regex("State.*isn't serializable")
+ exc.getCause.getClass should ===(classOf[DisabledJavaSerializer.JavaSerializationException])
+ }
+
+ "detect non-serializable empty state" in {
+ val eventSourcedTestKit =
+ EventSourcedBehaviorTestKit[TestCounter.Command, TestCounter.Event, TestCounter.State](
+ system,
+ TestCounter(persistenceId, NotSerializableState(0, Vector.empty)))
+
+ val exc = intercept[IllegalArgumentException] {
+ eventSourcedTestKit.runCommand(TestCounter.Increment)
+ }
+ (exc.getMessage should include).regex("Empty State.*isn't serializable")
+ exc.getCause.getClass should ===(classOf[DisabledJavaSerializer.JavaSerializationException])
+ }
+
+ "detect non-serializable command" in {
+ val eventSourcedTestKit = createTestKit()
+
+ val exc = intercept[IllegalArgumentException] {
+ eventSourcedTestKit.runCommand(TestCounter.NotSerializableCommand)
+ }
+ (exc.getMessage should include).regex("Command.*isn't serializable")
+ exc.getCause.getClass should ===(classOf[DisabledJavaSerializer.JavaSerializationException])
+ }
+
+ "detect non-serializable reply" in {
+ val eventSourcedTestKit = createTestKit()
+
+ val exc = intercept[IllegalArgumentException] {
+ eventSourcedTestKit.runCommand(TestCounter.IncrementWithNotSerializableReply(_))
+ }
+ (exc.getMessage should include).regex("Reply.*isn't serializable")
+ exc.getCause.getClass should ===(classOf[NotSerializableException])
+ }
+
+ "support test of replay" in {
+ val eventSourcedTestKit = createTestKit()
+
+ eventSourcedTestKit.runCommand(TestCounter.Increment)
+ eventSourcedTestKit.runCommand(TestCounter.Increment)
+ val expectedState = TestCounter.RealState(3, Vector(0, 1, 2))
+ eventSourcedTestKit.runCommand(TestCounter.Increment).state should ===(expectedState)
+ }
+
+ "support test of replay from stored events" in {
+ val eventSourcedTestKit = createTestKit()
+ eventSourcedTestKit.persistenceTestKit
+ .persistForRecovery(persistenceId.id, List(TestCounter.Incremented(1), TestCounter.Incremented(1)))
+ eventSourcedTestKit.restart().state should ===(TestCounter.RealState(2, Vector(0, 1)))
+ }
+
+ "support test of invalid events" in {
+ val eventSourcedTestKit = createTestKit()
+ eventSourcedTestKit.persistenceTestKit
+ .persistForRecovery(persistenceId.id, List(TestCounter.Incremented(1), TestCounter.Incremented(-1)))
+ intercept[IllegalStateException] {
+ eventSourcedTestKit.restart()
+ }
+
+ }
+
+ "only allow EventSourcedBehavior" in {
+ intercept[IllegalArgumentException] {
+ EventSourcedBehaviorTestKit[TestCounter.Command, TestCounter.Event, TestCounter.State](
+ system,
+ Behaviors.empty[TestCounter.Command])
+ }
+ }
+
+ "support test of failures" in {
+ val eventSourcedTestKit = createTestKit()
+ eventSourcedTestKit.runCommand(TestCounter.Increment)
+ eventSourcedTestKit.runCommand(TestCounter.Increment)
+ eventSourcedTestKit.persistenceTestKit.failNextPersisted(persistenceId.id, TestException("DB err"))
+ LoggingTestKit.error[JournalFailureException].expect {
+ intercept[AssertionError] {
+ eventSourcedTestKit.runCommand(TestCounter.Increment)
+ }
+ }
+
+ eventSourcedTestKit.restart().state should ===(TestCounter.RealState(2, Vector(0, 1)))
+ eventSourcedTestKit.runCommand(TestCounter.Increment).state should ===(TestCounter.RealState(3, Vector(0, 1, 2)))
+ }
+
+ "have possibility to clear" in {
+ val eventSourcedTestKit = createTestKit()
+ eventSourcedTestKit.runCommand(TestCounter.Increment)
+ eventSourcedTestKit.runCommand(TestCounter.Increment).state should ===(TestCounter.RealState(2, Vector(0, 1)))
+
+ eventSourcedTestKit.clear()
+ eventSourcedTestKit.runCommand(TestCounter.Increment).state should ===(TestCounter.RealState(1, Vector(0)))
+ }
+ }
+
+}
diff --git a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/ScalaDslUtils.scala b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/ScalaDslUtils.scala
index 432f5fe4d5..ba55a0f45b 100644
--- a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/ScalaDslUtils.scala
+++ b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/ScalaDslUtils.scala
@@ -5,10 +5,10 @@
package akka.persistence.testkit.scaladsl
import akka.actor.typed.ActorRef
-import akka.persistence.typed.RecoveryCompleted
-import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior }
import akka.persistence.testkit.{ Cmd, CommonUtils, EmptyState, Evt, Passivate, Recovered, Stopped, TestCommand }
import akka.persistence.typed.PersistenceId
+import akka.persistence.typed.RecoveryCompleted
+import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior }
trait ScalaDslUtils extends CommonUtils {
diff --git a/akka-persistence-typed/src/main/mima-filters/2.6.4.backwards.excludes/issue-23712-testkit.excludes b/akka-persistence-typed/src/main/mima-filters/2.6.4.backwards.excludes/issue-23712-testkit.excludes
new file mode 100644
index 0000000000..1e6c5bab15
--- /dev/null
+++ b/akka-persistence-typed/src/main/mima-filters/2.6.4.backwards.excludes/issue-23712-testkit.excludes
@@ -0,0 +1,6 @@
+# 23712 Internals for EventSourcedBehaviorTestKit
+ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.persistence.typed.internal.RequestingRecoveryPermit.stashInternal")
+ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.persistence.typed.internal.ReplayingSnapshot.stashInternal")
+ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.persistence.typed.internal.ReplayingEvents.stashInternal")
+ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.persistence.typed.internal.StashManagement.stashInternal")
+ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.persistence.typed.internal.Running.stashInternal")
diff --git a/akka-persistence-typed/src/main/mima-filters/2.6.4.backwards.excludes/issue-25216-typed-persistence-recovery-strategies.excludes b/akka-persistence-typed/src/main/mima-filters/2.6.4.backwards.excludes/issue-25216-typed-persistence-recovery-strategies.excludes
new file mode 100644
index 0000000000..40b68765b4
--- /dev/null
+++ b/akka-persistence-typed/src/main/mima-filters/2.6.4.backwards.excludes/issue-25216-typed-persistence-recovery-strategies.excludes
@@ -0,0 +1,3 @@
+# #25216 customize recovery strategy for typed persistence behaviors
+# abstract method withRecovery(akka.persistence.Recovery)akka.persistence.typed.scaladsl.EventSourcedBehavior in interface akka.persistence.typed.scaladsl.EventSourcedBehavior is present only in current version
+ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.persistence.typed.scaladsl.EventSourcedBehavior.withRecovery")
\ No newline at end of file
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/delivery/EventSourcedProducerQueue.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/delivery/EventSourcedProducerQueue.scala
index 2a091c5c49..c923343570 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/delivery/EventSourcedProducerQueue.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/delivery/EventSourcedProducerQueue.scala
@@ -8,7 +8,8 @@ import java.time.{ Duration => JavaDuration }
import scala.concurrent.duration._
-import akka.util.JavaDurationConverters._
+import com.typesafe.config.Config
+
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.SupervisorStrategy
@@ -21,7 +22,7 @@ import akka.persistence.typed.delivery.EventSourcedProducerQueue.CleanupTick
import akka.persistence.typed.scaladsl.Effect
import akka.persistence.typed.scaladsl.EventSourcedBehavior
import akka.persistence.typed.scaladsl.RetentionCriteria
-import com.typesafe.config.Config
+import akka.util.JavaDurationConverters._
/**
* [[DurableProducerQueue]] that can be used with [[akka.actor.typed.delivery.ProducerController]]
@@ -219,14 +220,17 @@ private class EventSourcedProducerQueue[A](
cleanupUnusedAfter: FiniteDuration) {
import DurableProducerQueue._
+ private val traceEnabled = context.log.isTraceEnabled
+
def onCommand(state: State[A], command: Command[A]): Effect[Event, State[A]] = {
command match {
case StoreMessageSent(sent, replyTo) =>
if (sent.seqNr == state.currentSeqNr) {
- context.log.trace(
- "StoreMessageSent seqNr [{}], confirmationQualifier [{}]",
- sent.seqNr,
- sent.confirmationQualifier)
+ if (traceEnabled)
+ context.log.trace(
+ "StoreMessageSent seqNr [{}], confirmationQualifier [{}]",
+ sent.seqNr,
+ sent.confirmationQualifier)
Effect.persist(sent).thenReply(replyTo)(_ => StoreMessageSentAck(sent.seqNr))
} else if (sent.seqNr == state.currentSeqNr - 1) {
// already stored, could be a retry after timout
@@ -239,7 +243,11 @@ private class EventSourcedProducerQueue[A](
}
case StoreMessageConfirmed(seqNr, confirmationQualifier, timestampMillis) =>
- context.log.trace("StoreMessageConfirmed seqNr [{}], confirmationQualifier [{}]", seqNr, confirmationQualifier)
+ if (traceEnabled)
+ context.log.trace(
+ "StoreMessageConfirmed seqNr [{}], confirmationQualifier [{}]",
+ seqNr,
+ confirmationQualifier)
val previousConfirmedSeqNr = state.confirmedSeqNr.get(confirmationQualifier) match {
case Some((nr, _)) => nr
case None => 0L
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/BehaviorSetup.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/BehaviorSetup.scala
index b338cd8f83..1ff5521cfd 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/BehaviorSetup.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/BehaviorSetup.scala
@@ -4,19 +4,20 @@
package akka.persistence.typed.internal
-import akka.actor.typed.Signal
-import akka.actor.typed.scaladsl.ActorContext
-import akka.actor.{ ActorRef, Cancellable }
-import akka.annotation.InternalApi
-import akka.persistence._
-import akka.persistence.typed.scaladsl.{ EventSourcedBehavior, RetentionCriteria }
-import akka.persistence.typed.{ EventAdapter, PersistenceId, SnapshotAdapter }
-import akka.util.OptionVal
-import org.slf4j.{ Logger, MDC }
-
import scala.concurrent.ExecutionContext
import scala.util.control.NonFatal
+import org.slf4j.{ Logger, MDC }
+
+import akka.actor.{ ActorRef, Cancellable }
+import akka.actor.typed.Signal
+import akka.actor.typed.scaladsl.ActorContext
+import akka.annotation.InternalApi
+import akka.persistence._
+import akka.persistence.typed.{ EventAdapter, PersistenceId, SnapshotAdapter }
+import akka.persistence.typed.scaladsl.{ EventSourcedBehavior, RetentionCriteria }
+import akka.util.OptionVal
+
/**
* INTERNAL API
*/
@@ -51,6 +52,7 @@ private[akka] final class BehaviorSetup[C, E, S](
import BehaviorSetup._
import InternalProtocol.RecoveryTickEvent
+
import akka.actor.typed.scaladsl.adapter._
val persistence: Persistence = Persistence(context.system.toClassic)
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EffectImpl.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EffectImpl.scala
index 3bbe0aed65..c5682151d2 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EffectImpl.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EffectImpl.scala
@@ -6,10 +6,10 @@ package akka.persistence.typed.internal
import scala.collection.immutable
+import akka.actor.typed.ActorRef
import akka.annotation.InternalApi
import akka.persistence.typed.javadsl
import akka.persistence.typed.scaladsl
-import akka.actor.typed.ActorRef
/** INTERNAL API */
@InternalApi
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EventSourcedBehaviorImpl.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EventSourcedBehaviorImpl.scala
index 995e4921c7..e989af5bca 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EventSourcedBehaviorImpl.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EventSourcedBehaviorImpl.scala
@@ -8,6 +8,7 @@ import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
import akka.actor.typed
+import akka.actor.typed.ActorRef
import akka.actor.typed.BackoffSupervisorStrategy
import akka.actor.typed.Behavior
import akka.actor.typed.BehaviorInterceptor
@@ -29,14 +30,15 @@ import akka.persistence.typed.DeleteSnapshotsCompleted
import akka.persistence.typed.DeleteSnapshotsFailed
import akka.persistence.typed.DeletionTarget
import akka.persistence.typed.EventAdapter
-import akka.persistence.typed.SnapshotAdapter
import akka.persistence.typed.NoOpEventAdapter
import akka.persistence.typed.PersistenceId
+import akka.persistence.typed.SnapshotAdapter
import akka.persistence.typed.SnapshotCompleted
import akka.persistence.typed.SnapshotFailed
import akka.persistence.typed.SnapshotSelectionCriteria
-import akka.persistence.typed.scaladsl.RetentionCriteria
import akka.persistence.typed.scaladsl._
+import akka.persistence.typed.scaladsl.{ Recovery => TypedRecovery }
+import akka.persistence.typed.scaladsl.RetentionCriteria
import akka.util.ConstantFun
import akka.util.unused
@@ -56,6 +58,17 @@ private[akka] object EventSourcedBehaviorImpl {
}
final case class WriterIdentity(instanceId: Int, writerUuid: String)
+ /**
+ * Used by EventSourcedBehaviorTestKit to retrieve the `persistenceId`.
+ */
+ final case class GetPersistenceId(replyTo: ActorRef[PersistenceId]) extends Signal
+
+ /**
+ * Used by EventSourcedBehaviorTestKit to retrieve the state.
+ * Can't be a Signal because those are not stashed.
+ */
+ final case class GetState[State](replyTo: ActorRef[State]) extends InternalProtocol
+
}
@InternalApi
@@ -112,6 +125,7 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State](
ctx.log.debug("Events successfully deleted to sequence number [{}].", toSequenceNr)
case (_, DeleteEventsFailed(toSequenceNr, failure)) =>
ctx.log.warn2("Failed to delete events to sequence number [{}] due to: {}", toSequenceNr, failure.getMessage)
+ case (_, EventSourcedBehaviorImpl.GetPersistenceId(replyTo)) => replyTo ! persistenceId
}
// do this once, even if the actor is restarted
@@ -220,6 +234,9 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State](
backoffStrategy: BackoffSupervisorStrategy): EventSourcedBehavior[Command, Event, State] =
copy(supervisionStrategy = backoffStrategy)
+ override def withRecovery(recovery: TypedRecovery): EventSourcedBehavior[Command, Event, State] = {
+ copy(recovery = recovery.toClassic)
+ }
}
/** Protocol used internally by the eventsourced behaviors. */
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EventSourcedSettings.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EventSourcedSettings.scala
index 88a002436a..8ca98cb068 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EventSourcedSettings.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EventSourcedSettings.scala
@@ -8,10 +8,11 @@ import java.util.concurrent.TimeUnit
import scala.concurrent.duration._
+import com.typesafe.config.Config
+
import akka.actor.typed.ActorSystem
import akka.annotation.InternalApi
import akka.persistence.Persistence
-import com.typesafe.config.Config
/**
* INTERNAL API
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ExternalInteractions.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ExternalInteractions.scala
index c82937481a..abe744434d 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ExternalInteractions.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ExternalInteractions.scala
@@ -16,11 +16,9 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.LoggerOps
import akka.annotation.InternalApi
import akka.annotation.InternalStableApi
-
+import akka.persistence._
import akka.persistence.JournalProtocol.ReplayMessages
import akka.persistence.SnapshotProtocol.LoadSnapshot
-import akka.persistence._
-
import akka.util.unused
/** INTERNAL API */
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/RecoveryImpl.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/RecoveryImpl.scala
new file mode 100644
index 0000000000..fce4beb9ab
--- /dev/null
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/RecoveryImpl.scala
@@ -0,0 +1,50 @@
+/*
+ * Copyright (C) 2019-2020 Lightbend Inc.
+ */
+
+package akka.persistence.typed.internal
+
+import akka.annotation.InternalApi
+import akka.persistence.typed.{ javadsl, scaladsl, SnapshotSelectionCriteria }
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[akka] case object DefaultRecovery extends javadsl.Recovery with scaladsl.Recovery {
+ override def asScala: scaladsl.Recovery = this
+ override def asJava: javadsl.Recovery = this
+
+ /**
+ * INTERNAL API
+ */
+ override private[akka] def toClassic = akka.persistence.Recovery()
+}
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[akka] case object DisabledRecovery extends javadsl.Recovery with scaladsl.Recovery {
+ override def asScala: scaladsl.Recovery = this
+ override def asJava: javadsl.Recovery = this
+
+ /**
+ * INTERNAL API
+ */
+ override private[akka] def toClassic = akka.persistence.Recovery.none
+}
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[akka] case class RecoveryWithSnapshotSelectionCriteria(
+ snapshotSelectionCriteria: SnapshotSelectionCriteria)
+ extends javadsl.Recovery
+ with scaladsl.Recovery {
+ override def asScala: scaladsl.Recovery = this
+ override def asJava: javadsl.Recovery = this
+
+ /**
+ * INTERNAL API
+ */
+ override private[akka] def toClassic = akka.persistence.Recovery(snapshotSelectionCriteria.toClassic)
+}
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingEvents.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingEvents.scala
index dc538923ad..da06fb1294 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingEvents.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingEvents.scala
@@ -4,8 +4,8 @@
package akka.persistence.typed.internal
-import scala.util.control.NonFatal
import scala.concurrent.duration._
+import scala.util.control.NonFatal
import akka.actor.typed.{ Behavior, Signal }
import akka.actor.typed.internal.PoisonPill
@@ -13,18 +13,19 @@ import akka.actor.typed.internal.UnstashException
import akka.actor.typed.scaladsl.{ AbstractBehavior, ActorContext, Behaviors, LoggerOps }
import akka.annotation.{ InternalApi, InternalStableApi }
import akka.event.Logging
-import akka.persistence.JournalProtocol._
import akka.persistence._
+import akka.persistence.JournalProtocol._
import akka.persistence.typed.EmptyEventSeq
import akka.persistence.typed.EventsSeq
-import akka.persistence.typed.RecoveryFailed
import akka.persistence.typed.RecoveryCompleted
+import akka.persistence.typed.RecoveryFailed
import akka.persistence.typed.SingleEventSeq
+import akka.persistence.typed.internal.EventSourcedBehaviorImpl.GetState
import akka.persistence.typed.internal.ReplayingEvents.ReplayingState
import akka.persistence.typed.internal.Running.WithSeqNrAccessible
import akka.util.OptionVal
-import akka.util.unused
import akka.util.PrettyDuration._
+import akka.util.unused
/***
* INTERNAL API
@@ -87,11 +88,12 @@ private[akka] final class ReplayingEvents[C, E, S](
override def onMessage(msg: InternalProtocol): Behavior[InternalProtocol] = {
msg match {
- case JournalResponse(r) => onJournalResponse(r)
- case SnapshotterResponse(r) => onSnapshotterResponse(r)
- case RecoveryTickEvent(snap) => onRecoveryTick(snap)
- case cmd: IncomingCommand[C] => onCommand(cmd)
- case RecoveryPermitGranted => Behaviors.unhandled // should not happen, we already have the permit
+ case JournalResponse(r) => onJournalResponse(r)
+ case SnapshotterResponse(r) => onSnapshotterResponse(r)
+ case RecoveryTickEvent(snap) => onRecoveryTick(snap)
+ case cmd: IncomingCommand[C] => onCommand(cmd)
+ case get: GetState[S @unchecked] => stashInternal(get)
+ case RecoveryPermitGranted => Behaviors.unhandled // should not happen, we already have the permit
}
}
@@ -162,7 +164,6 @@ private[akka] final class ReplayingEvents[C, E, S](
Behaviors.unhandled
} else {
stashInternal(cmd)
- Behaviors.same
}
}
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingSnapshot.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingSnapshot.scala
index 2052e003a0..e71ffec70b 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingSnapshot.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingSnapshot.scala
@@ -8,10 +8,11 @@ import akka.actor.typed.Behavior
import akka.actor.typed.internal.PoisonPill
import akka.actor.typed.scaladsl.{ ActorContext, Behaviors }
import akka.annotation.{ InternalApi, InternalStableApi }
+import akka.persistence._
import akka.persistence.SnapshotProtocol.LoadSnapshotFailed
import akka.persistence.SnapshotProtocol.LoadSnapshotResult
-import akka.persistence._
import akka.persistence.typed.RecoveryFailed
+import akka.persistence.typed.internal.EventSourcedBehaviorImpl.GetState
import akka.util.unused
/**
@@ -66,7 +67,8 @@ private[akka] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetup
Behaviors.unhandled
} else
onCommand(cmd)
- case RecoveryPermitGranted => Behaviors.unhandled // should not happen, we already have the permit
+ case get: GetState[S @unchecked] => stashInternal(get)
+ case RecoveryPermitGranted => Behaviors.unhandled // should not happen, we already have the permit
}
.receiveSignal(returnPermitOnStop.orElse {
case (_, PoisonPill) =>
@@ -118,7 +120,6 @@ private[akka] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetup
def onCommand(cmd: IncomingCommand[C]): Behavior[InternalProtocol] = {
// during recovery, stash all incoming commands
stashInternal(cmd)
- Behaviors.same
}
def onJournalResponse(response: JournalProtocol.Response): Behavior[InternalProtocol] = {
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/RequestingRecoveryPermit.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/RequestingRecoveryPermit.scala
index fc8552e93b..93cd47e653 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/RequestingRecoveryPermit.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/RequestingRecoveryPermit.scala
@@ -53,7 +53,6 @@ private[akka] class RequestingRecoveryPermit[C, E, S](override val setup: Behavi
Behaviors.unhandled
} else {
stashInternal(other)
- Behaviors.same
}
}
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/RetentionCriteriaImpl.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/RetentionCriteriaImpl.scala
index 15a97f7986..5ca922fde0 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/RetentionCriteriaImpl.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/RetentionCriteriaImpl.scala
@@ -5,8 +5,8 @@
package akka.persistence.typed.internal
import akka.annotation.InternalApi
-import akka.persistence.typed.scaladsl
import akka.persistence.typed.javadsl
+import akka.persistence.typed.scaladsl
/**
* INTERNAL API
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/Running.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/Running.scala
index a0545c9615..a9174e5440 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/Running.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/Running.scala
@@ -6,6 +6,7 @@ package akka.persistence.typed.internal
import scala.annotation.tailrec
import scala.collection.immutable
+
import akka.actor.UnhandledMessage
import akka.actor.typed.Behavior
import akka.actor.typed.Signal
@@ -25,17 +26,18 @@ import akka.persistence.SaveSnapshotFailure
import akka.persistence.SaveSnapshotSuccess
import akka.persistence.SnapshotProtocol
import akka.persistence.journal.Tagged
-import akka.persistence.typed.DeleteSnapshotsCompleted
-import akka.persistence.typed.DeleteSnapshotsFailed
import akka.persistence.typed.DeleteEventsCompleted
import akka.persistence.typed.DeleteEventsFailed
+import akka.persistence.typed.DeleteSnapshotsCompleted
+import akka.persistence.typed.DeleteSnapshotsFailed
import akka.persistence.typed.DeletionTarget
import akka.persistence.typed.EventRejectedException
import akka.persistence.typed.SnapshotCompleted
import akka.persistence.typed.SnapshotFailed
-import akka.persistence.typed.internal.Running.WithSeqNrAccessible
import akka.persistence.typed.SnapshotMetadata
import akka.persistence.typed.SnapshotSelectionCriteria
+import akka.persistence.typed.internal.EventSourcedBehaviorImpl.GetState
+import akka.persistence.typed.internal.Running.WithSeqNrAccessible
import akka.persistence.typed.scaladsl.Effect
import akka.util.unused
@@ -91,9 +93,9 @@ private[akka] object Running {
extends JournalInteractions[C, E, S]
with SnapshotInteractions[C, E, S]
with StashManagement[C, E, S] {
+ import BehaviorSetup._
import InternalProtocol._
import Running.RunningState
- import BehaviorSetup._
final class HandlingCommands(state: RunningState[S])
extends AbstractBehavior[InternalProtocol](setup.context)
@@ -103,6 +105,7 @@ private[akka] object Running {
case IncomingCommand(c: C @unchecked) => onCommand(state, c)
case JournalResponse(r) => onDeleteEventsJournalResponse(r, state.state)
case SnapshotterResponse(r) => onDeleteSnapshotResponse(r, state.state)
+ case get: GetState[S @unchecked] => onGetState(get)
case _ => Behaviors.unhandled
}
@@ -120,6 +123,12 @@ private[akka] object Running {
applyEffects(cmd, state, effect.asInstanceOf[EffectImpl[E, S]]) // TODO can we avoid the cast?
}
+ // Used by EventSourcedBehaviorTestKit to retrieve the state.
+ def onGetState(get: GetState[S]): Behavior[InternalProtocol] = {
+ get.replyTo ! state.state
+ this
+ }
+
@tailrec def applyEffects(
msg: Any,
state: RunningState[S],
@@ -235,6 +244,7 @@ private[akka] object Running {
msg match {
case JournalResponse(r) => onJournalResponse(r)
case in: IncomingCommand[C @unchecked] => onCommand(in)
+ case get: GetState[S @unchecked] => stashInternal(get)
case SnapshotterResponse(r) => onDeleteSnapshotResponse(r, visibleState.state)
case RecoveryTickEvent(_) => Behaviors.unhandled
case RecoveryPermitGranted => Behaviors.unhandled
@@ -248,7 +258,6 @@ private[akka] object Running {
Behaviors.unhandled
} else {
stashInternal(cmd)
- this
}
}
@@ -347,7 +356,6 @@ private[akka] object Running {
Behaviors.unhandled
} else {
stashInternal(cmd)
- Behaviors.same
}
}
@@ -405,6 +413,8 @@ private[akka] object Running {
case _ =>
onDeleteSnapshotResponse(response, state.state)
}
+ case get: GetState[S @unchecked] =>
+ stashInternal(get)
case _ =>
Behaviors.unhandled
}
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/StashManagement.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/StashManagement.scala
index d11506bbbf..49df6ae06b 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/StashManagement.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/StashManagement.scala
@@ -4,13 +4,14 @@
package akka.persistence.typed.internal
-import akka.actor.typed.Behavior
import akka.actor.Dropped
-import akka.actor.typed.scaladsl.adapter._
-import akka.actor.typed.scaladsl.StashOverflowException
+import akka.actor.typed.Behavior
import akka.actor.typed.scaladsl.ActorContext
+import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.LoggerOps
import akka.actor.typed.scaladsl.StashBuffer
+import akka.actor.typed.scaladsl.StashOverflowException
+import akka.actor.typed.scaladsl.adapter._
import akka.annotation.InternalApi
import akka.util.ConstantFun
@@ -29,8 +30,10 @@ private[akka] trait StashManagement[C, E, S] {
/**
* Stash a command to the internal stash buffer, which is used while waiting for persist to be completed.
*/
- protected def stashInternal(msg: InternalProtocol): Unit =
+ protected def stashInternal(msg: InternalProtocol): Behavior[InternalProtocol] = {
stash(msg, stashState.internalStashBuffer)
+ Behaviors.same
+ }
/**
* Stash a command to the user stash buffer, which is used when `Stash` effect is used.
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/CommandHandler.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/CommandHandler.scala
index 21800d0f51..2c77aef16f 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/CommandHandler.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/CommandHandler.scala
@@ -7,12 +7,12 @@ package akka.persistence.typed.javadsl
import java.util.Objects
import java.util.function.{ BiFunction, Predicate, Supplier, Function => JFunction }
+import scala.compat.java8.FunctionConverters._
+
import akka.annotation.InternalApi
import akka.persistence.typed.internal._
import akka.util.OptionVal
-import scala.compat.java8.FunctionConverters._
-
/**
* FunctionalInterface for reacting on commands
*
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/CommandHandlerWithReply.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/CommandHandlerWithReply.scala
index b8c6270b50..4fc7db9d59 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/CommandHandlerWithReply.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/CommandHandlerWithReply.scala
@@ -7,12 +7,12 @@ package akka.persistence.typed.javadsl
import java.util.Objects
import java.util.function.{ BiFunction, Predicate, Supplier, Function => JFunction }
+import scala.compat.java8.FunctionConverters._
+
import akka.annotation.InternalApi
import akka.persistence.typed.internal._
import akka.util.OptionVal
-import scala.compat.java8.FunctionConverters._
-
/* Note that this is a copy of CommandHandler.scala to support ReplyEffect
* s/Effect/ReplyEffect/
* s/CommandHandler/CommandHandlerWithReply/
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/Effect.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/Effect.scala
index 418de29531..2fe8d4e750 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/Effect.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/Effect.scala
@@ -4,13 +4,13 @@
package akka.persistence.typed.javadsl
-import akka.util.ccompat.JavaConverters._
+import akka.actor.typed.ActorRef
import akka.annotation.DoNotInherit
import akka.annotation.InternalApi
import akka.japi.function
-import akka.persistence.typed.internal.SideEffect
import akka.persistence.typed.internal._
-import akka.actor.typed.ActorRef
+import akka.persistence.typed.internal.SideEffect
+import akka.util.ccompat.JavaConverters._
/**
* INTERNAL API: see `class EffectFactories`
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/EventHandler.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/EventHandler.scala
index 113c2b9368..d5b9ce4433 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/EventHandler.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/EventHandler.scala
@@ -7,11 +7,11 @@ package akka.persistence.typed.javadsl
import java.util.Objects
import java.util.function.{ BiFunction, Predicate, Supplier, Function => JFunction }
+import scala.compat.java8.FunctionConverters._
+
import akka.annotation.InternalApi
import akka.util.OptionVal
-import scala.compat.java8.FunctionConverters._
-
/**
* FunctionalInterface for reacting on events having been persisted
*
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/EventSourcedBehavior.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/EventSourcedBehavior.scala
index 15976bded4..0b896c4b0e 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/EventSourcedBehavior.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/EventSourcedBehavior.scala
@@ -13,8 +13,8 @@ import akka.actor.typed.Behavior
import akka.actor.typed.internal.BehaviorImpl.DeferredBehavior
import akka.actor.typed.javadsl.ActorContext
import akka.annotation.InternalApi
-import akka.persistence.typed.EventAdapter
import akka.persistence.typed._
+import akka.persistence.typed.EventAdapter
import akka.persistence.typed.internal._
import akka.util.unused
@@ -124,6 +124,7 @@ abstract class EventSourcedBehavior[Command, Event, State] private[akka] (
* You may configure the behavior to skip replaying snapshots completely, in which case the recovery will be
* performed by replaying all events -- which may take a long time.
*/
+ @deprecated("override recovery instead", "2.6.5")
def snapshotSelectionCriteria: SnapshotSelectionCriteria = SnapshotSelectionCriteria.latest
/**
@@ -151,6 +152,12 @@ abstract class EventSourcedBehavior[Command, Event, State] private[akka] (
*/
def retentionCriteria: RetentionCriteria = RetentionCriteria.disabled
+ /**
+ * Override to change the strategy for recovery of snapshots and events.
+ * By default, snapshots and events are recovered.
+ */
+ def recovery: Recovery = Recovery.default
+
/**
* The `tagger` function should give event tags, which will be used in persistence query
*/
@@ -194,7 +201,7 @@ abstract class EventSourcedBehavior[Command, Event, State] private[akka] (
.snapshotAdapter(snapshotAdapter())
.withJournalPluginId(journalPluginId)
.withSnapshotPluginId(snapshotPluginId)
- .withSnapshotSelectionCriteria(snapshotSelectionCriteria)
+ .withRecovery(recovery.asScala)
val handler = signalHandler()
val behaviorWithSignalHandler =
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/PersistentFSMMigration.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/PersistentFSMMigration.scala
index 6fbaaf846f..f645a352ec 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/PersistentFSMMigration.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/PersistentFSMMigration.scala
@@ -6,11 +6,12 @@ package akka.persistence.typed.javadsl
import java.time.Duration
import java.util.Optional
-import akka.japi.function.Function3
-import akka.util.JavaDurationConverters._
+
import scala.compat.java8.OptionConverters._
+import akka.japi.function.Function3
import akka.persistence.typed.SnapshotAdapter
+import akka.util.JavaDurationConverters._
/**
* Helper functions for migration from PersistentFSM to Persistence Typed
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/Recovery.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/Recovery.scala
new file mode 100644
index 0000000000..326f8dac14
--- /dev/null
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/Recovery.scala
@@ -0,0 +1,50 @@
+/*
+ * Copyright (C) 2019-2020 Lightbend Inc.
+ */
+
+package akka.persistence.typed.javadsl
+
+import akka.annotation.InternalApi
+import akka.persistence.typed.SnapshotSelectionCriteria
+import akka.persistence.typed.internal.{ DefaultRecovery, DisabledRecovery, RecoveryWithSnapshotSelectionCriteria }
+
+/**
+ * Strategy for recovery of snapshots and events.
+ */
+abstract class Recovery {
+ def asScala: akka.persistence.typed.scaladsl.Recovery
+
+ /**
+ * INTERNAL API
+ */
+ @InternalApi private[akka] def toClassic: akka.persistence.Recovery
+}
+
+/**
+ * Strategy for recovery of snapshots and events.
+ */
+object Recovery {
+
+ /**
+ * Snapshots and events are recovered
+ */
+ val default: Recovery = DefaultRecovery
+
+ /**
+ * Neither snapshots nor events are recovered
+ */
+ val disabled: Recovery = DisabledRecovery
+
+ /**
+ * Changes the snapshot selection criteria used for the recovery.
+ *
+ * By default the most recent snapshot is used, and the remaining state updates are recovered by replaying events
+ * from the sequence number up until which the snapshot reached.
+ *
+ * You may configure the behavior to skip replaying snapshots completely, in which case the recovery will be
+ * performed by replaying all events -- which may take a long time.
+ */
+ def withSnapshotSelectionCriteria(snapshotSelectionCriteria: SnapshotSelectionCriteria) =
+ RecoveryWithSnapshotSelectionCriteria(snapshotSelectionCriteria)
+
+}
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/Effect.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/Effect.scala
index ea33b17591..65cc3cf839 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/Effect.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/Effect.scala
@@ -5,10 +5,11 @@
package akka.persistence.typed.scaladsl
import scala.collection.{ immutable => im }
-import akka.annotation.DoNotInherit
-import akka.persistence.typed.internal.SideEffect
-import akka.persistence.typed.internal._
+
import akka.actor.typed.ActorRef
+import akka.annotation.DoNotInherit
+import akka.persistence.typed.internal._
+import akka.persistence.typed.internal.SideEffect
/**
* Factory methods for creating [[Effect]] directives - how an event sourced actor reacts on a command.
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/EventSourcedBehavior.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/EventSourcedBehavior.scala
index 44fddfd5ea..7583715b9e 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/EventSourcedBehavior.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/EventSourcedBehavior.scala
@@ -5,17 +5,18 @@
package akka.persistence.typed.scaladsl
import scala.annotation.tailrec
+
import akka.actor.typed.BackoffSupervisorStrategy
import akka.actor.typed.Behavior
-import akka.actor.typed.internal.BehaviorImpl.DeferredBehavior
import akka.actor.typed.Signal
+import akka.actor.typed.internal.BehaviorImpl.DeferredBehavior
import akka.actor.typed.internal.InterceptorImpl
import akka.actor.typed.internal.LoggerClass
import akka.actor.typed.scaladsl.ActorContext
import akka.annotation.DoNotInherit
import akka.persistence.typed.EventAdapter
-import akka.persistence.typed.SnapshotAdapter
import akka.persistence.typed.PersistenceId
+import akka.persistence.typed.SnapshotAdapter
import akka.persistence.typed.SnapshotSelectionCriteria
import akka.persistence.typed.internal._
@@ -45,9 +46,9 @@ object EventSourcedBehavior {
* Create a `Behavior` for a persistent actor.
*
* @param persistenceId stable unique identifier for the event sourced behavior
- * @param emtpyState the intial state for the entity before any events have been processed
+ * @param emptyState the intial state for the entity before any events have been processed
* @param commandHandler map commands to effects e.g. persisting events, replying to commands
- * @param evnetHandler compute the new state given the current state when an event has been persisted
+ * @param eventHandler compute the new state given the current state when an event has been persisted
*/
def apply[Command, Event, State](
persistenceId: PersistenceId,
@@ -157,6 +158,7 @@ object EventSourcedBehavior {
* You may configure the behavior to skip replaying snapshots completely, in which case the recovery will be
* performed by replaying all events -- which may take a long time.
*/
+ @deprecated("use withRecovery(Recovery.withSnapshotSelectionCriteria(...))", "2.6.5")
def withSnapshotSelectionCriteria(selection: SnapshotSelectionCriteria): EventSourcedBehavior[Command, Event, State]
/**
@@ -207,4 +209,10 @@ object EventSourcedBehavior {
* If not specified the actor will be stopped on failure.
*/
def onPersistFailure(backoffStrategy: BackoffSupervisorStrategy): EventSourcedBehavior[Command, Event, State]
+
+ /**
+ * Change the recovery strategy.
+ * By default, snapshots and events are recovered.
+ */
+ def withRecovery(recovery: Recovery): EventSourcedBehavior[Command, Event, State]
}
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/PersistentFSMMigration.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/PersistentFSMMigration.scala
index 9a002989da..f939e1f8bb 100644
--- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/PersistentFSMMigration.scala
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/PersistentFSMMigration.scala
@@ -4,11 +4,11 @@
package akka.persistence.typed.scaladsl
+import scala.concurrent.duration.FiniteDuration
+
import akka.persistence.fsm.PersistentFSM.PersistentFSMSnapshot
import akka.persistence.typed.SnapshotAdapter
-import scala.concurrent.duration.FiniteDuration
-
/**
* Helper functions for migration from PersistentFSM to Persistence Typed
*/
diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/Recovery.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/Recovery.scala
new file mode 100644
index 0000000000..c3f43a15e6
--- /dev/null
+++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/Recovery.scala
@@ -0,0 +1,51 @@
+/*
+ * Copyright (C) 2019-2020 Lightbend Inc.
+ */
+
+package akka.persistence.typed.scaladsl
+
+import akka.annotation.InternalApi
+import akka.persistence.typed.SnapshotSelectionCriteria
+import akka.persistence.typed.internal.{ DefaultRecovery, DisabledRecovery, RecoveryWithSnapshotSelectionCriteria }
+
+/**
+ * Strategy for recovery of snapshots and events.
+ */
+trait Recovery {
+ def asJava: akka.persistence.typed.javadsl.Recovery
+
+ /**
+ * INTERNAL API
+ */
+ @InternalApi private[akka] def toClassic: akka.persistence.Recovery
+
+}
+
+/**
+ * Strategy for recovery of snapshots and events.
+ */
+object Recovery {
+
+ /**
+ * Snapshots and events are recovered
+ */
+ val default: Recovery = DefaultRecovery
+
+ /**
+ * Neither snapshots nor events are recovered
+ */
+ val disabled: Recovery = DisabledRecovery
+
+ /**
+ * Changes the snapshot selection criteria used for the recovery.
+ *
+ * By default the most recent snapshot is used, and the remaining state updates are recovered by replaying events
+ * from the sequence number up until which the snapshot reached.
+ *
+ * You may configure the behavior to skip replaying snapshots completely, in which case the recovery will be
+ * performed by replaying all events -- which may take a long time.
+ */
+ def withSnapshotSelectionCriteria(snapshotSelectionCriteria: SnapshotSelectionCriteria) =
+ RecoveryWithSnapshotSelectionCriteria(snapshotSelectionCriteria)
+
+}
diff --git a/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/BasicPersistentBehaviorTest.java b/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/BasicPersistentBehaviorTest.java
index 92274cefa6..6aee3284c9 100644
--- a/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/BasicPersistentBehaviorTest.java
+++ b/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/BasicPersistentBehaviorTest.java
@@ -16,6 +16,7 @@ import akka.persistence.typed.SnapshotFailed;
import akka.persistence.typed.SnapshotSelectionCriteria;
import akka.persistence.typed.javadsl.CommandHandler;
import akka.persistence.typed.javadsl.Effect;
+import akka.persistence.typed.javadsl.Recovery;
import akka.persistence.typed.javadsl.EventHandler;
// #behavior
import akka.persistence.typed.javadsl.EventSourcedBehavior;
@@ -344,6 +345,13 @@ public class BasicPersistentBehaviorTest {
}
// #recovery
+ // #recovery-disabled
+ @Override
+ public Recovery recovery() {
+ return Recovery.disabled();
+ }
+ // #recovery-disabled
+
// #tagging
@Override
public Set tagsFor(Event event) {
@@ -555,8 +563,8 @@ public class BasicPersistentBehaviorTest {
// #snapshotSelection
@Override
- public SnapshotSelectionCriteria snapshotSelectionCriteria() {
- return SnapshotSelectionCriteria.none();
+ public Recovery recovery() {
+ return Recovery.withSnapshotSelectionCriteria(SnapshotSelectionCriteria.none());
}
// #snapshotSelection
}
diff --git a/akka-persistence-typed/src/test/resources/logback-test.xml b/akka-persistence-typed/src/test/resources/logback-test.xml
index ad3473919e..22c45c93b6 100644
--- a/akka-persistence-typed/src/test/resources/logback-test.xml
+++ b/akka-persistence-typed/src/test/resources/logback-test.xml
@@ -5,7 +5,7 @@
- %date{ISO8601} %-5level %logger %marker - %msg {%mdc}%n
+ %date{ISO8601} %-5level %logger %marker - %msg MDC: {%mdc}%n
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/ClusterSingletonPersistenceSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/ClusterSingletonPersistenceSpec.scala
index fd5890c64a..69940c21ba 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/ClusterSingletonPersistenceSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/ClusterSingletonPersistenceSpec.scala
@@ -4,17 +4,20 @@
package akka.persistence.typed
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
+import akka.actor
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.typed.ActorRef
+import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.cluster.typed.ClusterSingleton
import akka.cluster.typed.SingletonActor
import akka.persistence.typed.scaladsl.Effect
import akka.persistence.typed.scaladsl.EventSourcedBehavior
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object ClusterSingletonPersistenceSpec {
val config = ConfigFactory.parseString("""
@@ -32,7 +35,7 @@ object ClusterSingletonPersistenceSpec {
sealed trait Command
final case class Add(s: String) extends Command
final case class Get(replyTo: ActorRef[String]) extends Command
- private final case object StopPlz extends Command
+ private case object StopPlz extends Command
val persistentActor: Behavior[Command] =
EventSourcedBehavior[Command, String, String](
@@ -55,11 +58,12 @@ class ClusterSingletonPersistenceSpec
with AnyWordSpecLike
with LogCapturing {
import ClusterSingletonPersistenceSpec._
+
import akka.actor.typed.scaladsl.adapter._
- implicit val s = system
+ implicit val s: ActorSystem[Nothing] = system
- implicit val classicSystem = system.toClassic
+ implicit val classicSystem: actor.ActorSystem = system.toClassic
private val classicCluster = akka.cluster.Cluster(classicSystem)
"A typed cluster singleton with persistent actor" must {
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/ManyRecoveriesSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/ManyRecoveriesSpec.scala
index e343fa64ca..908e237556 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/ManyRecoveriesSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/ManyRecoveriesSpec.scala
@@ -4,19 +4,20 @@
package akka.persistence.typed
-import akka.actor.typed.scaladsl.Behaviors
-import akka.actor.typed.scaladsl.adapter.TypedActorSystemOps
-import akka.persistence.typed.scaladsl.EventSourcedBehavior.CommandHandler
-import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior }
-import akka.testkit.TestLatch
-import akka.actor.testkit.typed.scaladsl.TestProbe
import scala.concurrent.Await
import scala.concurrent.duration._
-import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
-import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.wordspec.AnyWordSpecLike
+import akka.actor.testkit.typed.scaladsl.LogCapturing
+import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
+import akka.actor.testkit.typed.scaladsl.TestProbe
+import akka.actor.typed.scaladsl.Behaviors
+import akka.actor.typed.scaladsl.adapter.TypedActorSystemOps
+import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior }
+import akka.persistence.typed.scaladsl.EventSourcedBehavior.CommandHandler
+import akka.testkit.TestLatch
+
object ManyRecoveriesSpec {
sealed case class Cmd(s: String)
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/PersistenceIdSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/PersistenceIdSpec.scala
index 26673fd55d..1a417b6750 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/PersistenceIdSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/PersistenceIdSpec.scala
@@ -4,10 +4,11 @@
package akka.persistence.typed
-import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.testkit.typed.scaladsl.LogCapturing
+
class PersistenceIdSpec extends AnyWordSpec with Matchers with LogCapturing {
"PersistenceId" must {
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/StashingWhenSnapshottingSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/StashingWhenSnapshottingSpec.scala
index ec44854444..a6d2455878 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/StashingWhenSnapshottingSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/StashingWhenSnapshottingSpec.scala
@@ -6,25 +6,26 @@ package akka.persistence.typed
import java.util.concurrent.CyclicBarrier
+import scala.concurrent.Future
+import scala.concurrent.Promise
+import scala.util.Success
+
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.typed.ActorSystem
import akka.actor.typed.Extension
import akka.actor.typed.ExtensionId
+import akka.actor.typed.scaladsl.adapter._
import akka.persistence
import akka.persistence.SelectedSnapshot
import akka.persistence.snapshot.SnapshotStore
-import com.typesafe.config.ConfigFactory
-import akka.actor.typed.scaladsl.adapter._
import akka.persistence.typed.StashingWhenSnapshottingSpec.ControllableSnapshotStoreExt
import akka.persistence.typed.scaladsl.Effect
import akka.persistence.typed.scaladsl.EventSourcedBehavior
-import scala.concurrent.Future
-import scala.concurrent.Promise
-import scala.util.Success
-
-import akka.actor.testkit.typed.scaladsl.LogCapturing
-import org.scalatest.wordspec.AnyWordSpecLike
object StashingWhenSnapshottingSpec {
object ControllableSnapshotStoreExt extends ExtensionId[ControllableSnapshotStoreExt] {
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/delivery/EventSourcedProducerQueueSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/delivery/EventSourcedProducerQueueSpec.scala
index b3e620b0a2..472a2f5e98 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/delivery/EventSourcedProducerQueueSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/delivery/EventSourcedProducerQueueSpec.scala
@@ -6,10 +6,14 @@ package akka.persistence.typed.delivery
import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
+
import scala.concurrent.duration._
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl._
-import akka.actor.typed.eventstream.EventStream
import akka.actor.typed.delivery.DurableProducerQueue.Confirmed
import akka.actor.typed.delivery.DurableProducerQueue.LoadState
import akka.actor.typed.delivery.DurableProducerQueue.MessageSent
@@ -18,11 +22,9 @@ import akka.actor.typed.delivery.DurableProducerQueue.State
import akka.actor.typed.delivery.DurableProducerQueue.StoreMessageConfirmed
import akka.actor.typed.delivery.DurableProducerQueue.StoreMessageSent
import akka.actor.typed.delivery.DurableProducerQueue.StoreMessageSentAck
+import akka.actor.typed.eventstream.EventStream
import akka.persistence.journal.inmem.InmemJournal
import akka.persistence.typed.PersistenceId
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object EventSourcedProducerQueueSpec {
def conf: Config =
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/delivery/ReliableDeliveryWithEventSourcedProducerQueueSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/delivery/ReliableDeliveryWithEventSourcedProducerQueueSpec.scala
index 0e019dcc35..9b719c337f 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/delivery/ReliableDeliveryWithEventSourcedProducerQueueSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/delivery/ReliableDeliveryWithEventSourcedProducerQueueSpec.scala
@@ -6,13 +6,14 @@ package akka.persistence.typed.delivery
import java.util.UUID
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl._
import akka.actor.typed.delivery.ConsumerController
import akka.actor.typed.delivery.ProducerController
import akka.persistence.typed.PersistenceId
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object ReliableDeliveryWithEventSourcedProducerQueueSpec {
def conf: Config =
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/delivery/WorkPullingWithEventSourcedProducerQueueSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/delivery/WorkPullingWithEventSourcedProducerQueueSpec.scala
index 82d88ddf6d..5cc2949c16 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/delivery/WorkPullingWithEventSourcedProducerQueueSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/delivery/WorkPullingWithEventSourcedProducerQueueSpec.scala
@@ -7,15 +7,16 @@ package akka.persistence.typed.delivery
import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.FishingOutcome
import akka.actor.testkit.typed.scaladsl._
import akka.actor.typed.delivery.ConsumerController
import akka.actor.typed.delivery.WorkPullingProducerController
import akka.actor.typed.receptionist.ServiceKey
import akka.persistence.typed.PersistenceId
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object WorkPullingWithEventSourcedProducerQueueSpec {
def conf: Config =
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/internal/RecoveryPermitterSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/internal/RecoveryPermitterSpec.scala
index afa7624024..666a42c234 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/internal/RecoveryPermitterSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/internal/RecoveryPermitterSpec.scala
@@ -4,24 +4,26 @@
package akka.persistence.typed.internal
-import akka.actor.PoisonPill
-import akka.actor.testkit.typed.scaladsl.TestProbe
-import akka.actor.typed.scaladsl.Behaviors
-import akka.actor.typed.scaladsl.adapter.{ TypedActorRefOps, TypedActorSystemOps }
-import akka.actor.typed.{ ActorRef, Behavior }
-import akka.persistence.Persistence
-import akka.persistence.RecoveryPermitter.{ RecoveryPermitGranted, RequestRecoveryPermit, ReturnRecoveryPermit }
-import akka.persistence.typed.scaladsl.EventSourcedBehavior.CommandHandler
-import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior }
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
+import org.scalatest.wordspec.AnyWordSpecLike
+
+import akka.actor.ActorSystem
+import akka.actor.PoisonPill
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.LoggingTestKit
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
-import akka.actor.testkit.typed.scaladsl.LogCapturing
+import akka.actor.testkit.typed.scaladsl.TestProbe
+import akka.actor.typed.{ ActorRef, Behavior }
+import akka.actor.typed.scaladsl.Behaviors
+import akka.actor.typed.scaladsl.adapter.{ TypedActorRefOps, TypedActorSystemOps }
+import akka.persistence.Persistence
+import akka.persistence.RecoveryPermitter.{ RecoveryPermitGranted, RequestRecoveryPermit, ReturnRecoveryPermit }
import akka.persistence.typed.PersistenceId
import akka.persistence.typed.RecoveryCompleted
-import org.scalatest.wordspec.AnyWordSpecLike
+import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior }
+import akka.persistence.typed.scaladsl.EventSourcedBehavior.CommandHandler
object RecoveryPermitterSpec {
@@ -76,7 +78,7 @@ class RecoveryPermitterSpec extends ScalaTestWithActorTestKit(s"""
import RecoveryPermitterSpec._
- implicit val classicSystem = system.toClassic
+ implicit val classicSystem: ActorSystem = system.toClassic
private val permitter = Persistence(classicSystem).recoveryPermitter
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/internal/RetentionCriteriaSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/internal/RetentionCriteriaSpec.scala
index 81dca58b1e..9ee081a609 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/internal/RetentionCriteriaSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/internal/RetentionCriteriaSpec.scala
@@ -4,12 +4,13 @@
package akka.persistence.typed.internal
-import akka.actor.testkit.typed.scaladsl.LogCapturing
-import akka.persistence.typed.scaladsl.RetentionCriteria
import org.scalatest.TestSuite
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
+import akka.actor.testkit.typed.scaladsl.LogCapturing
+import akka.persistence.typed.scaladsl.RetentionCriteria
+
class RetentionCriteriaSpec extends TestSuite with Matchers with AnyWordSpecLike with LogCapturing {
"RetentionCriteria" must {
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/internal/StashStateSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/internal/StashStateSpec.scala
index 40bd752dfa..9db11c23d9 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/internal/StashStateSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/internal/StashStateSpec.scala
@@ -6,14 +6,15 @@ package akka.persistence.typed.internal
import scala.concurrent.duration._
+import org.scalatest.wordspec.AnyWordSpecLike
+
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
-import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.Behavior
import akka.actor.typed.scaladsl.Behaviors
import akka.persistence.typed.internal.InternalProtocol.IncomingCommand
import akka.persistence.typed.internal.InternalProtocol.RecoveryPermitGranted
-import org.scalatest.wordspec.AnyWordSpecLike
class StashStateSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing {
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorFailureSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorFailureSpec.scala
index a5732632f6..3cf4ee20ac 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorFailureSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorFailureSpec.scala
@@ -9,6 +9,10 @@ import scala.concurrent.Future
import scala.concurrent.duration._
import scala.util.Try
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.TestException
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.scaladsl._
@@ -27,9 +31,6 @@ import akka.persistence.typed.RecoveryCompleted
import akka.persistence.typed.RecoveryCompleted
import akka.persistence.typed.RecoveryFailed
import akka.persistence.typed.internal.JournalFailureException
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
class ChaosJournal extends InmemJournal {
var counts = Map.empty[String, Int]
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorInterceptorSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorInterceptorSpec.scala
index 7a3a330d30..4be6ba60e4 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorInterceptorSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorInterceptorSpec.scala
@@ -6,6 +6,10 @@ package akka.persistence.typed.scaladsl
import java.util.concurrent.atomic.AtomicInteger
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl._
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
@@ -13,9 +17,6 @@ import akka.actor.typed.BehaviorInterceptor
import akka.actor.typed.TypedActorContext
import akka.actor.typed.scaladsl.Behaviors
import akka.persistence.typed.PersistenceId
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object EventSourcedBehaviorInterceptorSpec {
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorRecoveryTimeoutSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorRecoveryTimeoutSpec.scala
index c7b37b79c8..f045ef793f 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorRecoveryTimeoutSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorRecoveryTimeoutSpec.scala
@@ -8,6 +8,10 @@ import java.util.concurrent.atomic.AtomicInteger
import scala.concurrent.duration._
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl._
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
@@ -17,9 +21,6 @@ import akka.persistence.journal.SteppingInmemJournal
import akka.persistence.typed.PersistenceId
import akka.persistence.typed.RecoveryFailed
import akka.persistence.typed.internal.JournalFailureException
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object EventSourcedBehaviorRecoveryTimeoutSpec {
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorReplySpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorReplySpec.scala
index ac65750a10..50133e4666 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorReplySpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorReplySpec.scala
@@ -7,6 +7,10 @@ package akka.persistence.typed.scaladsl
import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.Done
import akka.actor.testkit.typed.scaladsl._
import akka.actor.typed.ActorRef
@@ -15,9 +19,6 @@ import akka.actor.typed.scaladsl.ActorContext
import akka.actor.typed.scaladsl.Behaviors
import akka.persistence.typed.PersistenceId
import akka.serialization.jackson.CborSerializable
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object EventSourcedBehaviorReplySpec {
def conf: Config = ConfigFactory.parseString(s"""
@@ -87,7 +88,7 @@ class EventSourcedBehaviorReplySpec
"persist an event thenReply" in {
val c = spawn(counter(nextPid()))
- val probe = TestProbe[Done]
+ val probe = TestProbe[Done]()
c ! IncrementWithConfirmation(probe.ref)
probe.expectMessage(Done)
@@ -99,17 +100,17 @@ class EventSourcedBehaviorReplySpec
"persist an event thenReply later" in {
val c = spawn(counter(nextPid()))
- val probe = TestProbe[Done]
+ val probe = TestProbe[Done]()
c ! IncrementReplyLater(probe.ref)
probe.expectMessage(Done)
}
"reply to query command" in {
val c = spawn(counter(nextPid()))
- val updateProbe = TestProbe[Done]
+ val updateProbe = TestProbe[Done]()
c ! IncrementWithConfirmation(updateProbe.ref)
- val queryProbe = TestProbe[State]
+ val queryProbe = TestProbe[State]()
c ! GetValue(queryProbe.ref)
queryProbe.expectMessage(State(1, Vector(0)))
}
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorRetentionSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorRetentionSpec.scala
index b0ada72fc7..6e47cd74f8 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorRetentionSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorRetentionSpec.scala
@@ -7,8 +7,15 @@ package akka.persistence.typed.scaladsl
import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
+import scala.concurrent.duration._
import scala.util.Success
import scala.util.Try
+
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl._
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
@@ -26,11 +33,6 @@ import akka.persistence.typed.SnapshotFailed
import akka.persistence.typed.SnapshotSelectionCriteria
import akka.serialization.jackson.CborSerializable
import akka.util.unused
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import scala.concurrent.duration._
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpecLike
object EventSourcedBehaviorRetentionSpec extends Matchers {
@@ -44,10 +46,10 @@ object EventSourcedBehaviorRetentionSpec extends Matchers {
""")
sealed trait Command extends CborSerializable
- final case object Increment extends Command
+ case object Increment extends Command
final case class IncrementWithPersistAll(nr: Int) extends Command
final case class GetValue(replyTo: ActorRef[State]) extends Command
- final case object StopIt extends Command
+ case object StopIt extends Command
final case class WrappedSignal(signal: EventSourcedSignal)
@@ -207,7 +209,7 @@ class EventSourcedBehaviorRetentionSpec
"snapshot via predicate" in {
val pid = nextPid()
- val snapshotSignalProbe = TestProbe[WrappedSignal]
+ val snapshotSignalProbe = TestProbe[WrappedSignal]()
val alwaysSnapshot: Behavior[Command] =
Behaviors.setup { ctx =>
counter(ctx, pid, snapshotSignalProbe = Some(snapshotSignalProbe.ref)).snapshotWhen { (_, _, _) =>
@@ -237,7 +239,7 @@ class EventSourcedBehaviorRetentionSpec
"check all events for snapshot in PersistAll" in {
val pid = nextPid()
- val snapshotSignalProbe = TestProbe[WrappedSignal]
+ val snapshotSignalProbe = TestProbe[WrappedSignal]()
val snapshotAtTwo = Behaviors.setup[Command](ctx =>
counter(ctx, pid, snapshotSignalProbe = Some(snapshotSignalProbe.ref)).snapshotWhen { (s, _, _) =>
s.value == 2
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorSpec.scala
index 263c3d039d..25f4f9042c 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorSpec.scala
@@ -14,6 +14,10 @@ import scala.util.Failure
import scala.util.Success
import scala.util.Try
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.Done
import akka.actor.ActorInitializationException
import akka.actor.testkit.typed.TestException
@@ -25,6 +29,8 @@ import akka.actor.typed.SupervisorStrategy
import akka.actor.typed.Terminated
import akka.actor.typed.scaladsl.ActorContext
import akka.actor.typed.scaladsl.Behaviors
+import akka.persistence.{ SnapshotMetadata => ClassicSnapshotMetadata }
+import akka.persistence.{ SnapshotSelectionCriteria => ClassicSnapshotSelectionCriteria }
import akka.persistence.SelectedSnapshot
import akka.persistence.journal.inmem.InmemJournal
import akka.persistence.query.EventEnvelope
@@ -37,13 +43,9 @@ import akka.persistence.typed.RecoveryCompleted
import akka.persistence.typed.SnapshotCompleted
import akka.persistence.typed.SnapshotFailed
import akka.persistence.typed.SnapshotMetadata
-import akka.persistence.{ SnapshotMetadata => ClassicSnapshotMetadata }
-import akka.persistence.{ SnapshotSelectionCriteria => ClassicSnapshotSelectionCriteria }
+import akka.persistence.typed.SnapshotSelectionCriteria
import akka.serialization.jackson.CborSerializable
import akka.stream.scaladsl.Sink
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object EventSourcedBehaviorSpec {
@@ -91,22 +93,22 @@ object EventSourcedBehaviorSpec {
""")
sealed trait Command extends CborSerializable
- final case object Increment extends Command
- final case object IncrementThenLogThenStop extends Command
- final case object IncrementTwiceThenLogThenStop extends Command
+ case object Increment extends Command
+ case object IncrementThenLogThenStop extends Command
+ case object IncrementTwiceThenLogThenStop extends Command
final case class IncrementWithPersistAll(nr: Int) extends Command
- final case object IncrementLater extends Command
- final case object IncrementAfterReceiveTimeout extends Command
- final case object IncrementTwiceAndThenLog extends Command
+ case object IncrementLater extends Command
+ case object IncrementAfterReceiveTimeout extends Command
+ case object IncrementTwiceAndThenLog extends Command
final case class IncrementWithConfirmation(replyTo: ActorRef[Done]) extends Command
- final case object DoNothingAndThenLog extends Command
- final case object EmptyEventsListAndThenLog extends Command
+ case object DoNothingAndThenLog extends Command
+ case object EmptyEventsListAndThenLog extends Command
final case class GetValue(replyTo: ActorRef[State]) extends Command
- final case object DelayFinished extends Command
+ case object DelayFinished extends Command
private case object Timeout extends Command
- final case object LogThenStop extends Command
- final case object Fail extends Command
- final case object StopIt extends Command
+ case object LogThenStop extends Command
+ case object Fail extends Command
+ case object StopIt extends Command
sealed trait Event extends CborSerializable
final case class Incremented(delta: Int) extends Event
@@ -130,9 +132,9 @@ object EventSourcedBehaviorSpec {
counter(
ctx,
persistenceId,
- loggingActor = TestProbe[String].ref,
- probe = TestProbe[(State, Event)].ref,
- snapshotProbe = TestProbe[Try[SnapshotMetadata]].ref)
+ loggingActor = TestProbe[String]().ref,
+ probe = TestProbe[(State, Event)]().ref,
+ snapshotProbe = TestProbe[Try[SnapshotMetadata]]().ref)
def counter(ctx: ActorContext[Command], persistenceId: PersistenceId, logging: ActorRef[String])(
implicit system: ActorSystem[_]): EventSourcedBehavior[Command, Event, State] =
@@ -140,8 +142,8 @@ object EventSourcedBehaviorSpec {
ctx,
persistenceId,
loggingActor = logging,
- probe = TestProbe[(State, Event)].ref,
- TestProbe[Try[SnapshotMetadata]].ref)
+ probe = TestProbe[(State, Event)]().ref,
+ TestProbe[Try[SnapshotMetadata]]().ref)
def counterWithProbe(
ctx: ActorContext[Command],
@@ -149,18 +151,18 @@ object EventSourcedBehaviorSpec {
probe: ActorRef[(State, Event)],
snapshotProbe: ActorRef[Try[SnapshotMetadata]])(
implicit system: ActorSystem[_]): EventSourcedBehavior[Command, Event, State] =
- counter(ctx, persistenceId, TestProbe[String].ref, probe, snapshotProbe)
+ counter(ctx, persistenceId, TestProbe[String]().ref, probe, snapshotProbe)
def counterWithProbe(ctx: ActorContext[Command], persistenceId: PersistenceId, probe: ActorRef[(State, Event)])(
implicit system: ActorSystem[_]): EventSourcedBehavior[Command, Event, State] =
- counter(ctx, persistenceId, TestProbe[String].ref, probe, TestProbe[Try[SnapshotMetadata]].ref)
+ counter(ctx, persistenceId, TestProbe[String]().ref, probe, TestProbe[Try[SnapshotMetadata]]().ref)
def counterWithSnapshotProbe(
ctx: ActorContext[Command],
persistenceId: PersistenceId,
probe: ActorRef[Try[SnapshotMetadata]])(
implicit system: ActorSystem[_]): EventSourcedBehavior[Command, Event, State] =
- counter(ctx, persistenceId, TestProbe[String].ref, TestProbe[(State, Event)].ref, snapshotProbe = probe)
+ counter(ctx, persistenceId, TestProbe[String]().ref, TestProbe[(State, Event)]().ref, snapshotProbe = probe)
def counter(
ctx: ActorContext[Command],
@@ -182,7 +184,7 @@ object EventSourcedBehaviorSpec {
.thenRun { (_: State) =>
loggingActor ! firstLogging
}
- .thenStop
+ .thenStop()
case IncrementTwiceThenLogThenStop =>
Effect
@@ -190,7 +192,7 @@ object EventSourcedBehaviorSpec {
.thenRun { (_: State) =>
loggingActor ! firstLogging
}
- .thenStop
+ .thenStop()
case IncrementWithPersistAll(n) =>
Effect.persist((0 until n).map(_ => Incremented(1)))
@@ -253,7 +255,7 @@ object EventSourcedBehaviorSpec {
.thenRun { _ =>
loggingActor ! firstLogging
}
- .thenStop
+ .thenStop()
case Fail =>
throw new TestException("boom!")
@@ -293,18 +295,18 @@ class EventSourcedBehaviorSpec
"A typed persistent actor" must {
"persist an event" in {
- val c = spawn(counter(nextPid))
- val probe = TestProbe[State]
+ val c = spawn(counter(nextPid()))
+ val probe = TestProbe[State]()
c ! Increment
c ! GetValue(probe.ref)
probe.expectMessage(State(1, Vector(0)))
}
"replay stored events" in {
- val pid = nextPid
+ val pid = nextPid()
val c = spawn(counter(pid))
- val probe = TestProbe[State]
+ val probe = TestProbe[State]()
c ! Increment
c ! Increment
c ! Increment
@@ -320,8 +322,8 @@ class EventSourcedBehaviorSpec
}
"handle Terminated signal" in {
- val c = spawn(counter(nextPid))
- val probe = TestProbe[State]
+ val c = spawn(counter(nextPid()))
+ val probe = TestProbe[State]()
c ! Increment
c ! IncrementLater
eventually {
@@ -331,9 +333,9 @@ class EventSourcedBehaviorSpec
}
"handle receive timeout" in {
- val c = spawn(counter(nextPid))
+ val c = spawn(counter(nextPid()))
- val probe = TestProbe[State]
+ val probe = TestProbe[State]()
c ! Increment
c ! IncrementAfterReceiveTimeout
// let it timeout
@@ -344,15 +346,82 @@ class EventSourcedBehaviorSpec
}
}
+ "adhere default and disabled Recovery strategies" in {
+ val pid = nextPid()
+ val probe = TestProbe[State]
+
+ def counterWithRecoveryStrategy(recoveryStrategy: Recovery) =
+ Behaviors.setup[Command](counter(_, pid).withRecovery(recoveryStrategy))
+
+ val counterSetup = spawn(counterWithRecoveryStrategy(Recovery.default))
+ counterSetup ! Increment
+ counterSetup ! Increment
+ counterSetup ! Increment
+ counterSetup ! GetValue(probe.ref)
+ probe.expectMessage(State(3, Vector(0, 1, 2)))
+
+ val counterDefaultRecoveryStrategy = spawn(counterWithRecoveryStrategy(Recovery.default))
+ counterSetup ! Increment
+ counterDefaultRecoveryStrategy ! GetValue(probe.ref)
+ probe.expectMessage(State(4, Vector(0, 1, 2, 3)))
+
+ val counterDisabledRecoveryStrategy = spawn(counterWithRecoveryStrategy(Recovery.disabled))
+ counterDisabledRecoveryStrategy ! Increment
+ counterDisabledRecoveryStrategy ! Increment
+ counterDisabledRecoveryStrategy ! GetValue(probe.ref)
+ probe.expectMessage(State(2, Vector(0, 1)))
+ }
+
+ "adhere Recovery strategy with SnapshotSelectionCriteria" in {
+ val pid = nextPid()
+ val eventProbe = TestProbe[(State, Event)]
+ val commandProbe = TestProbe[State]
+ val snapshotProbe = TestProbe[Try[SnapshotMetadata]]
+
+ def counterWithSnapshotSelectionCriteria(recoveryStrategy: Recovery) =
+ Behaviors.setup[Command](
+ counterWithProbe(_, pid, eventProbe.ref, snapshotProbe.ref).withRecovery(recoveryStrategy).snapshotWhen {
+ case (_, _, _) => true
+ })
+
+ val counterSetup = spawn(counterWithSnapshotSelectionCriteria(Recovery.default))
+ counterSetup ! Increment
+ counterSetup ! Increment
+ counterSetup ! Increment
+ eventProbe.receiveMessages(3)
+ snapshotProbe.receiveMessages(3)
+ counterSetup ! GetValue(commandProbe.ref)
+ commandProbe.expectMessage(State(3, Vector(0, 1, 2)))
+
+ val counterWithSnapshotSelectionCriteriaNone = spawn(
+ counterWithSnapshotSelectionCriteria(Recovery.withSnapshotSelectionCriteria(SnapshotSelectionCriteria.none)))
+ // replay all events, no snapshot
+ eventProbe.expectMessage(State(0, Vector.empty) -> Incremented(1))
+ eventProbe.expectMessage(State(1, Vector(0)) -> Incremented(1))
+ eventProbe.expectMessage(State(2, Vector(0, 1)) -> Incremented(1))
+ counterWithSnapshotSelectionCriteriaNone ! Increment
+ eventProbe.expectMessage(State(3, Vector(0, 1, 2)) -> Incremented(1))
+ counterWithSnapshotSelectionCriteriaNone ! GetValue(commandProbe.ref)
+ commandProbe.expectMessage(State(4, Vector(0, 1, 2, 3)))
+
+ val counterWithSnapshotSelectionCriteriaLatest = spawn(
+ counterWithSnapshotSelectionCriteria(Recovery.withSnapshotSelectionCriteria(SnapshotSelectionCriteria.latest)))
+ // replay no events, only latest snapshot
+ eventProbe.expectNoMessage()
+ counterWithSnapshotSelectionCriteriaLatest ! Increment
+ counterWithSnapshotSelectionCriteriaLatest ! GetValue(commandProbe.ref)
+ commandProbe.expectMessage(State(5, Vector(0, 1, 2, 3, 4)))
+ }
+
/**
* Verify that all side-effects callbacks are called (in order) and only once.
* The [[IncrementTwiceAndThenLog]] command will emit two Increment events
*/
"chainable side effects with events" in {
- val loggingProbe = TestProbe[String]
- val c = spawn(counter(nextPid, loggingProbe.ref))
+ val loggingProbe = TestProbe[String]()
+ val c = spawn(counter(nextPid(), loggingProbe.ref))
- val probe = TestProbe[State]
+ val probe = TestProbe[State]()
c ! IncrementTwiceAndThenLog
c ! GetValue(probe.ref)
@@ -363,8 +432,8 @@ class EventSourcedBehaviorSpec
}
"persist then stop" in {
- val loggingProbe = TestProbe[String]
- val c = spawn(counter(nextPid, loggingProbe.ref))
+ val loggingProbe = TestProbe[String]()
+ val c = spawn(counter(nextPid(), loggingProbe.ref))
val watchProbe = watcher(c)
c ! IncrementThenLogThenStop
@@ -373,8 +442,8 @@ class EventSourcedBehaviorSpec
}
"persist(All) then stop" in {
- val loggingProbe = TestProbe[String]
- val c = spawn(counter(nextPid, loggingProbe.ref))
+ val loggingProbe = TestProbe[String]()
+ val c = spawn(counter(nextPid(), loggingProbe.ref))
val watchProbe = watcher(c)
c ! IncrementTwiceThenLogThenStop
@@ -384,8 +453,8 @@ class EventSourcedBehaviorSpec
}
"persist an event thenReply" in {
- val c = spawn(counter(nextPid))
- val probe = TestProbe[Done]
+ val c = spawn(counter(nextPid()))
+ val probe = TestProbe[Done]()
c ! IncrementWithConfirmation(probe.ref)
probe.expectMessage(Done)
@@ -397,10 +466,10 @@ class EventSourcedBehaviorSpec
/** Proves that side-effects are called when emitting an empty list of events */
"chainable side effects without events" in {
- val loggingProbe = TestProbe[String]
- val c = spawn(counter(nextPid, loggingProbe.ref))
+ val loggingProbe = TestProbe[String]()
+ val c = spawn(counter(nextPid(), loggingProbe.ref))
- val probe = TestProbe[State]
+ val probe = TestProbe[State]()
c ! EmptyEventsListAndThenLog
c ! GetValue(probe.ref)
probe.expectMessage(State(0, Vector.empty))
@@ -409,10 +478,10 @@ class EventSourcedBehaviorSpec
/** Proves that side-effects are called when explicitly calling Effect.none */
"chainable side effects when doing nothing (Effect.none)" in {
- val loggingProbe = TestProbe[String]
- val c = spawn(counter(nextPid, loggingProbe.ref))
+ val loggingProbe = TestProbe[String]()
+ val c = spawn(counter(nextPid(), loggingProbe.ref))
- val probe = TestProbe[State]
+ val probe = TestProbe[State]()
c ! DoNothingAndThenLog
c ! GetValue(probe.ref)
probe.expectMessage(State(0, Vector.empty))
@@ -420,9 +489,9 @@ class EventSourcedBehaviorSpec
}
"work when wrapped in other behavior" in {
- val probe = TestProbe[State]
+ val probe = TestProbe[State]()
val behavior = Behaviors
- .supervise[Command](counter(nextPid))
+ .supervise[Command](counter(nextPid()))
.onFailure(SupervisorStrategy.restartWithBackoff(1.second, 10.seconds, 0.1))
val c = spawn(behavior)
c ! Increment
@@ -431,8 +500,8 @@ class EventSourcedBehaviorSpec
}
"stop after logging (no persisting)" in {
- val loggingProbe = TestProbe[String]
- val c: ActorRef[Command] = spawn(counter(nextPid, loggingProbe.ref))
+ val loggingProbe = TestProbe[String]()
+ val c: ActorRef[Command] = spawn(counter(nextPid(), loggingProbe.ref))
val watchProbe = watcher(c)
c ! LogThenStop
loggingProbe.expectMessage(firstLogging)
@@ -440,8 +509,8 @@ class EventSourcedBehaviorSpec
}
"wrap persistent behavior in tap" in {
- val probe = TestProbe[Command]
- val wrapped: Behavior[Command] = Behaviors.monitor(probe.ref, counter(nextPid))
+ val probe = TestProbe[Command]()
+ val wrapped: Behavior[Command] = Behaviors.monitor(probe.ref, counter(nextPid()))
val c = spawn(wrapped)
c ! Increment
@@ -452,7 +521,7 @@ class EventSourcedBehaviorSpec
}
"tag events" in {
- val pid = nextPid
+ val pid = nextPid()
val c = spawn(Behaviors.setup[Command](ctx => counter(ctx, pid).withTagger(_ => Set("tag1", "tag2"))))
val replyProbe = TestProbe[State]()
@@ -468,10 +537,10 @@ class EventSourcedBehaviorSpec
val c = spawn(Behaviors.withTimers[Command] { timers =>
timers.startSingleTimer(Increment, 1.millis)
Thread.sleep(30) // now it's probably already in the mailbox, and will be stashed
- counter(nextPid)
+ counter(nextPid())
})
- val probe = TestProbe[State]
+ val probe = TestProbe[State]()
c ! Increment
probe.awaitAssert {
c ! GetValue(probe.ref)
@@ -483,10 +552,10 @@ class EventSourcedBehaviorSpec
val c = spawn(Behaviors.withTimers[Command] { timers =>
// probably arrives after recovery completed
timers.startSingleTimer(Increment, 200.millis)
- counter(nextPid)
+ counter(nextPid())
})
- val probe = TestProbe[State]
+ val probe = TestProbe[State]()
c ! Increment
probe.awaitAssert {
c ! GetValue(probe.ref)
@@ -498,11 +567,11 @@ class EventSourcedBehaviorSpec
LoggingTestKit.error("Exception during recovery from snapshot").expect {
val c = spawn(
Behaviors.setup[Command](ctx =>
- counter(ctx, nextPid)
+ counter(ctx, nextPid())
.withSnapshotPluginId("slow-snapshot-store")
.withJournalPluginId("short-recovery-timeout")))
- val probe = TestProbe[State]
+ val probe = TestProbe[State]()
probe.expectTerminated(c, probe.remainingOrDefault)
}
@@ -510,7 +579,7 @@ class EventSourcedBehaviorSpec
"not wrap a failure caused by command stashed while recovering in a journal failure" in {
val pid = nextPid()
- val probe = TestProbe[AnyRef]
+ val probe = TestProbe[AnyRef]()
// put some events in there, so that recovering takes a little time
val c = spawn(Behaviors.setup[Command](counter(_, pid)))
@@ -532,7 +601,7 @@ class EventSourcedBehaviorSpec
intercept[IllegalArgumentException] {
PersistenceId.ofUniqueId(null)
}
- val probe = TestProbe[AnyRef]
+ val probe = TestProbe[AnyRef]()
LoggingTestKit.error[ActorInitializationException].withMessageContains("persistenceId must not be null").expect {
val ref = spawn(Behaviors.setup[Command](counter(_, persistenceId = PersistenceId.ofUniqueId(null))))
probe.expectTerminated(ref)
@@ -547,7 +616,7 @@ class EventSourcedBehaviorSpec
intercept[IllegalArgumentException] {
PersistenceId.ofUniqueId("")
}
- val probe = TestProbe[AnyRef]
+ val probe = TestProbe[AnyRef]()
LoggingTestKit.error[ActorInitializationException].withMessageContains("persistenceId must not be empty").expect {
val ref = spawn(Behaviors.setup[Command](counter(_, persistenceId = PersistenceId.ofUniqueId(""))))
probe.expectTerminated(ref)
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorStashSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorStashSpec.scala
index eab7f98fa6..654025151f 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorStashSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorStashSpec.scala
@@ -10,13 +10,17 @@ import java.util.concurrent.atomic.AtomicInteger
import scala.concurrent.duration._
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.NotUsed
+import akka.actor.Dropped
+import akka.actor.UnhandledMessage
import akka.actor.testkit.typed.TestException
import akka.actor.testkit.typed.scaladsl._
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
-import akka.actor.Dropped
-import akka.actor.UnhandledMessage
import akka.actor.typed.PostStop
import akka.actor.typed.SupervisorStrategy
import akka.actor.typed.eventstream.EventStream
@@ -26,9 +30,6 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
import akka.persistence.typed.PersistenceId
import akka.persistence.typed.RecoveryCompleted
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object EventSourcedBehaviorStashSpec {
def conf: Config = ConfigFactory.parseString(s"""
@@ -177,8 +178,8 @@ class EventSourcedBehaviorStashSpec
"stash and unstash" in {
val c = spawn(counter(nextPid()))
- val ackProbe = TestProbe[Ack]
- val stateProbe = TestProbe[State]
+ val ackProbe = TestProbe[Ack]()
+ val stateProbe = TestProbe[State]()
c ! Increment("1", ackProbe.ref)
ackProbe.expectMessage(Ack("1"))
@@ -205,8 +206,8 @@ class EventSourcedBehaviorStashSpec
"handle mix of stash, persist and unstash" in {
val c = spawn(counter(nextPid()))
- val ackProbe = TestProbe[Ack]
- val stateProbe = TestProbe[State]
+ val ackProbe = TestProbe[Ack]()
+ val stateProbe = TestProbe[State]()
c ! Increment("1", ackProbe.ref)
ackProbe.expectMessage(Ack("1"))
@@ -233,8 +234,8 @@ class EventSourcedBehaviorStashSpec
"unstash in right order" in {
val c = spawn(counter(nextPid()))
- val ackProbe = TestProbe[Ack]
- val stateProbe = TestProbe[State]
+ val ackProbe = TestProbe[Ack]()
+ val stateProbe = TestProbe[State]()
c ! Increment(s"inc-1", ackProbe.ref)
@@ -270,9 +271,9 @@ class EventSourcedBehaviorStashSpec
"handle many stashed" in {
val c = spawn(counter(nextPid()))
- val ackProbe = TestProbe[Ack]
- val stateProbe = TestProbe[State]
- val notUsedProbe = TestProbe[NotUsed]
+ val ackProbe = TestProbe[Ack]()
+ val stateProbe = TestProbe[State]()
+ val notUsedProbe = TestProbe[NotUsed]()
val unhandledProbe = createTestProbe[UnhandledMessage]()
system.eventStream ! EventStream.Subscribe(unhandledProbe.ref)
@@ -383,8 +384,8 @@ class EventSourcedBehaviorStashSpec
"discard user stash when restarted due to thrown exception" in {
val c = spawn(counter(nextPid()))
- val ackProbe = TestProbe[Ack]
- val stateProbe = TestProbe[State]
+ val ackProbe = TestProbe[Ack]()
+ val stateProbe = TestProbe[State]()
c ! Increment("inc-1", ackProbe.ref)
ackProbe.expectMessage(Ack("inc-1"))
@@ -418,8 +419,8 @@ class EventSourcedBehaviorStashSpec
"discard internal stash when restarted due to thrown exception" in {
val c = spawn(counter(nextPid()))
- val ackProbe = TestProbe[Ack]
- val stateProbe = TestProbe[State]
+ val ackProbe = TestProbe[Ack]()
+ val stateProbe = TestProbe[State]()
val latch = new CountDownLatch(1)
// make first command slow to ensure that all subsequent commands are enqueued first
@@ -448,8 +449,8 @@ class EventSourcedBehaviorStashSpec
"preserve internal stash when persist failed" in {
val c = spawn(counter(PersistenceId.ofUniqueId("fail-fifth-a")))
- val ackProbe = TestProbe[Ack]
- val stateProbe = TestProbe[State]
+ val ackProbe = TestProbe[Ack]()
+ val stateProbe = TestProbe[State]()
(1 to 10).foreach { n =>
c ! Increment(s"inc-$n", ackProbe.ref)
@@ -466,8 +467,8 @@ class EventSourcedBehaviorStashSpec
"preserve user stash when persist failed" in {
val c = spawn(counter(PersistenceId.ofUniqueId("fail-fifth-b")))
- val ackProbe = TestProbe[Ack]
- val stateProbe = TestProbe[State]
+ val ackProbe = TestProbe[Ack]()
+ val stateProbe = TestProbe[State]()
c ! Increment("inc-1", ackProbe.ref)
ackProbe.expectMessage(Ack("inc-1"))
@@ -604,7 +605,7 @@ class EventSourcedBehaviorStashSpec
"stop from PoisonPill even though user stash is not empty" in {
val c = spawn(counter(nextPid()))
- val ackProbe = TestProbe[Ack]
+ val ackProbe = TestProbe[Ack]()
c ! Increment("1", ackProbe.ref)
ackProbe.expectMessage(Ack("1"))
@@ -622,7 +623,7 @@ class EventSourcedBehaviorStashSpec
"stop from PoisonPill after unstashing completed" in {
val c = spawn(counter(nextPid()))
- val ackProbe = TestProbe[Ack]
+ val ackProbe = TestProbe[Ack]()
val unhandledProbe = createTestProbe[UnhandledMessage]()
system.eventStream ! EventStream.Subscribe(unhandledProbe.ref)
@@ -658,7 +659,7 @@ class EventSourcedBehaviorStashSpec
"stop from PoisonPill after recovery completed" in {
val pid = nextPid()
val c = spawn(counter(pid))
- val ackProbe = TestProbe[Ack]
+ val ackProbe = TestProbe[Ack]()
c ! Increment("1", ackProbe.ref)
c ! Increment("2", ackProbe.ref)
@@ -667,7 +668,7 @@ class EventSourcedBehaviorStashSpec
ackProbe.expectMessage(Ack("2"))
ackProbe.expectMessage(Ack("3"))
- val signalProbe = TestProbe[String]
+ val signalProbe = TestProbe[String]()
val c2 = spawn(counter(pid, Some(signalProbe.ref)))
// this PoisonPill will most likely be received in RequestingRecoveryPermit since it's sent immediately
c2.toClassic ! PoisonPill
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorTimersSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorTimersSpec.scala
index e012ea1383..c6ec83b203 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorTimersSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorTimersSpec.scala
@@ -9,14 +9,15 @@ import java.util.concurrent.atomic.AtomicInteger
import scala.concurrent.duration._
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl._
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
import akka.actor.typed.scaladsl.Behaviors
import akka.persistence.typed.PersistenceId
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object EventSourcedBehaviorTimersSpec {
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorWatchSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorWatchSpec.scala
index b2afca57ba..b6d7bfe066 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorWatchSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorWatchSpec.scala
@@ -6,11 +6,14 @@ package akka.persistence.typed.scaladsl
import java.util.concurrent.atomic.AtomicInteger
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.TestException
import akka.actor.testkit.typed.scaladsl.{ LogCapturing, LoggingTestKit, ScalaTestWithActorTestKit, TestProbe }
import akka.actor.typed._
import akka.actor.typed.scaladsl.{ ActorContext, Behaviors }
-import akka.persistence.Recovery
+import akka.persistence.{ Recovery => ClassicRecovery }
+import akka.persistence.typed.{ NoOpEventAdapter, PersistenceId, RecoveryCompleted }
import akka.persistence.typed.internal.{
BehaviorSetup,
EventSourcedSettings,
@@ -19,10 +22,8 @@ import akka.persistence.typed.internal.{
StashState
}
import akka.persistence.typed.internal.EventSourcedBehaviorImpl.WriterIdentity
-import akka.persistence.typed.{ NoOpEventAdapter, PersistenceId, RecoveryCompleted }
import akka.serialization.jackson.CborSerializable
import akka.util.ConstantFun
-import org.scalatest.wordspec.AnyWordSpecLike
object EventSourcedBehaviorWatchSpec {
sealed trait Command extends CborSerializable
@@ -61,7 +62,7 @@ class EventSourcedBehaviorWatchSpec
NoOpEventAdapter.instance[String],
NoOpSnapshotAdapter.instance[String],
snapshotWhen = ConstantFun.scalaAnyThreeToFalse,
- Recovery(),
+ ClassicRecovery(),
RetentionCriteria.disabled,
holdingRecoveryPermit = false,
settings = settings,
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedEventAdapterSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedEventAdapterSpec.scala
index 41391ce6a2..2ac9907a5e 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedEventAdapterSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedEventAdapterSpec.scala
@@ -7,9 +7,12 @@ package akka.persistence.typed.scaladsl
import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
-import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.ActorRef
import akka.actor.typed.scaladsl.Behaviors
import akka.persistence.query.EventEnvelope
@@ -22,8 +25,6 @@ import akka.persistence.typed.PersistenceId
import akka.serialization.jackson.CborSerializable
import akka.stream.scaladsl.Sink
import akka.testkit.JavaSerializable
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object EventSourcedEventAdapterSpec {
@@ -84,7 +85,6 @@ class EventSourcedEventAdapterSpec
extends ScalaTestWithActorTestKit(EventSourcedEventAdapterSpec.conf)
with AnyWordSpecLike
with LogCapturing {
- import EventSourcedEventAdapterSpec._
import EventSourcedBehaviorSpec.{
counter,
Command,
@@ -95,6 +95,7 @@ class EventSourcedEventAdapterSpec
Incremented,
State
}
+ import EventSourcedEventAdapterSpec._
val pidCounter = new AtomicInteger(0)
private def nextPid(): PersistenceId = PersistenceId.ofUniqueId(s"c${pidCounter.incrementAndGet()})")
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedSequenceNumberSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedSequenceNumberSpec.scala
index 768e8febe4..3c3947ab30 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedSequenceNumberSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedSequenceNumberSpec.scala
@@ -4,14 +4,15 @@
package akka.persistence.typed.scaladsl
-import akka.actor.testkit.typed.scaladsl.LogCapturing
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl.{ ScalaTestWithActorTestKit, TestProbe }
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.{ ActorRef, Behavior }
import akka.actor.typed.scaladsl.Behaviors
import akka.persistence.typed.PersistenceId
import akka.persistence.typed.RecoveryCompleted
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object EventSourcedSequenceNumberSpec {
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedSnapshotAdapterSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedSnapshotAdapterSpec.scala
index 31bbc7a8ca..7d75da5e2b 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedSnapshotAdapterSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedSnapshotAdapterSpec.scala
@@ -7,6 +7,10 @@ package akka.persistence.typed.scaladsl
import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
@@ -16,9 +20,6 @@ import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal
import akka.persistence.typed.PersistenceId
import akka.persistence.typed.SnapshotAdapter
import akka.serialization.jackson.CborSerializable
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object EventSourcedSnapshotAdapterSpec {
private val conf: Config = ConfigFactory.parseString(s"""
@@ -38,6 +39,7 @@ class EventSourcedSnapshotAdapterSpec
with AnyWordSpecLike
with LogCapturing {
import EventSourcedSnapshotAdapterSpec._
+
import akka.actor.typed.scaladsl.adapter._
val pidCounter = new AtomicInteger(0)
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/LoggerSourceSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/LoggerSourceSpec.scala
index 296bb42bf8..707117b162 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/LoggerSourceSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/LoggerSourceSpec.scala
@@ -5,17 +5,18 @@
package akka.persistence.typed.scaladsl
import java.util.concurrent.atomic.AtomicInteger
+import org.scalatest.wordspec.AnyWordSpecLike
+import org.slf4j.event.Level
+
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.LoggingTestKit
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
-import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.Behavior
import akka.actor.typed.scaladsl.Behaviors
import akka.persistence.typed.PersistenceId
import akka.persistence.typed.RecoveryCompleted
import akka.persistence.typed.SnapshotCompleted
import akka.persistence.typed.SnapshotFailed
-import org.slf4j.event.Level
-import org.scalatest.wordspec.AnyWordSpecLike
// Note that the spec name here is important since there are heuristics in place to avoid names
// starting with EventSourcedBehavior
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/NullEmptyStateSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/NullEmptyStateSpec.scala
index b98de12bbc..433f10b57f 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/NullEmptyStateSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/NullEmptyStateSpec.scala
@@ -4,14 +4,15 @@
package akka.persistence.typed.scaladsl
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.scaladsl._
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
import akka.persistence.typed.PersistenceId
import akka.persistence.typed.RecoveryCompleted
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object NullEmptyStateSpec {
@@ -26,7 +27,7 @@ class NullEmptyStateSpec
with AnyWordSpecLike
with LogCapturing {
- implicit val testSettings = TestKitSettings(system)
+ implicit val testSettings: TestKitSettings = TestKitSettings(system)
def primitiveState(persistenceId: PersistenceId, probe: ActorRef[String]): Behavior[String] =
EventSourcedBehavior[String, String, String](
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/OptionalSnapshotStoreSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/OptionalSnapshotStoreSpec.scala
index 856d2945fb..13939e2f07 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/OptionalSnapshotStoreSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/OptionalSnapshotStoreSpec.scala
@@ -6,14 +6,15 @@ package akka.persistence.typed.scaladsl
import java.util.UUID
+import org.scalatest.wordspec.AnyWordSpecLike
+
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.LoggingTestKit
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
-import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.persistence.typed.PersistenceId
import akka.persistence.typed.scaladsl.EventSourcedBehavior.CommandHandler
import akka.serialization.jackson.CborSerializable
-import org.scalatest.wordspec.AnyWordSpecLike
object OptionalSnapshotStoreSpec {
@@ -73,7 +74,7 @@ class OptionalSnapshotStoreSpec extends ScalaTestWithActorTestKit(s"""
}
"successfully save a snapshot when no default snapshot-store configured, yet PersistentActor picked one explicitly" in {
- val stateProbe = TestProbe[State]
+ val stateProbe = TestProbe[State]()
val persistentActor = spawn(persistentBehaviorWithSnapshotPlugin(stateProbe))
persistentActor ! AnyCommand
stateProbe.expectMessageType[State]
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PerformanceSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PerformanceSpec.scala
index c86123a3a1..ff63266e5d 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PerformanceSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PerformanceSpec.scala
@@ -8,18 +8,19 @@ import java.util.UUID
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.TestException
+import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
-import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.ActorRef
import akka.actor.typed.SupervisorStrategy
import akka.actor.typed.scaladsl.Behaviors
import akka.persistence.typed.PersistenceId
import akka.persistence.typed.RecoveryCompleted
import akka.persistence.typed.scaladsl.EventSourcedBehavior.CommandHandler
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object PerformanceSpec {
@@ -145,7 +146,7 @@ class PerformanceSpec extends ScalaTestWithActorTestKit(ConfigFactory.parseStrin
}
def stressEventSourcedPersistentActor(failAt: Option[Long]): Unit = {
- val probe = TestProbe[Reply]
+ val probe = TestProbe[Reply]()
val name = s"${this.getClass.getSimpleName}-${UUID.randomUUID().toString}"
val persistentActor = spawn(eventSourcedTestPersistenceBehavior(name, probe), name)
stressPersistentActor(persistentActor, probe, failAt, "persistent events")
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PersistentActorCompileOnlyTest.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PersistentActorCompileOnlyTest.scala
index 45908c7f75..bbb6d6741e 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PersistentActorCompileOnlyTest.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PersistentActorCompileOnlyTest.scala
@@ -4,18 +4,18 @@
package akka.persistence.typed.scaladsl
-import akka.actor.typed.ActorSystem
-
import scala.concurrent.ExecutionContext
+import scala.concurrent.Future
import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
+
import akka.actor.typed.{ ActorRef, Behavior }
+import akka.actor.typed.ActorSystem
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.TimerScheduler
import akka.persistence.typed.PersistenceId
import akka.persistence.typed.RecoveryCompleted
-import com.github.ghik.silencer.silent
-
-import scala.concurrent.Future
// unused names in pattern match can be useful in the docs
@silent
@@ -290,7 +290,7 @@ object PersistentActorCompileOnlyTest {
private val commandHandler: CommandHandler[Command, Event, State] = CommandHandler.command {
case Enough =>
- Effect.persist(Done).thenRun((_: State) => println("yay")).thenStop
+ Effect.persist(Done).thenRun((_: State) => println("yay")).thenStop()
}
private val eventHandler: (State, Event) => State = {
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PrimitiveStateSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PrimitiveStateSpec.scala
index 073810d06e..1fa472d7e3 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PrimitiveStateSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PrimitiveStateSpec.scala
@@ -4,13 +4,14 @@
package akka.persistence.typed.scaladsl
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl._
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
import akka.persistence.typed.PersistenceId
import akka.persistence.typed.RecoveryCompleted
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object PrimitiveStateSpec {
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/SlowInMemorySnapshotStore.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/SlowInMemorySnapshotStore.scala
index e145e8d19f..7d77e1717e 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/SlowInMemorySnapshotStore.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/SlowInMemorySnapshotStore.scala
@@ -4,13 +4,13 @@
package akka.persistence.typed.scaladsl
+import scala.concurrent.Future
+
+import akka.persistence.{ SnapshotMetadata => ClassicSnapshotMetadata }
+import akka.persistence.{ SnapshotSelectionCriteria => ClassicSnapshotSelectionCriteria }
import akka.persistence.SelectedSnapshot
import akka.persistence.snapshot.SnapshotStore
import akka.persistence.typed.scaladsl.SnapshotMutableStateSpec.MutableState
-import akka.persistence.{ SnapshotSelectionCriteria => ClassicSnapshotSelectionCriteria }
-import akka.persistence.{ SnapshotMetadata => ClassicSnapshotMetadata }
-
-import scala.concurrent.Future
class SlowInMemorySnapshotStore extends SnapshotStore {
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/SnapshotMutableStateSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/SnapshotMutableStateSpec.scala
index 15a3ad6aa2..c05108fb59 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/SnapshotMutableStateSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/SnapshotMutableStateSpec.scala
@@ -7,6 +7,10 @@ package akka.persistence.typed.scaladsl
import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl._
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
@@ -14,9 +18,6 @@ import akka.persistence.typed.PersistenceId
import akka.persistence.typed.SnapshotCompleted
import akka.persistence.typed.SnapshotFailed
import akka.serialization.jackson.CborSerializable
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.scalatest.wordspec.AnyWordSpecLike
object SnapshotMutableStateSpec {
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/SnapshotRecoveryWithEmptyJournalSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/SnapshotRecoveryWithEmptyJournalSpec.scala
index 68deea04c3..4ed257cc2d 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/SnapshotRecoveryWithEmptyJournalSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/SnapshotRecoveryWithEmptyJournalSpec.scala
@@ -7,6 +7,11 @@ package akka.persistence.typed.scaladsl
import java.io.File
import java.util.UUID
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.apache.commons.io.FileUtils
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.typed.ActorRef
@@ -16,10 +21,6 @@ import akka.persistence.serialization.Snapshot
import akka.persistence.typed.PersistenceId
import akka.serialization.Serialization
import akka.serialization.SerializationExtension
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.apache.commons.io.FileUtils
-import org.scalatest.wordspec.AnyWordSpecLike
object SnapshotRecoveryWithEmptyJournalSpec {
val survivingSnapshotPath = s"target/survivingSnapshotPath-${UUID.randomUUID().toString}"
diff --git a/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/BasicPersistentBehaviorCompileOnly.scala b/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/BasicPersistentBehaviorCompileOnly.scala
index 79f62e0223..3891bafa5b 100644
--- a/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/BasicPersistentBehaviorCompileOnly.scala
+++ b/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/BasicPersistentBehaviorCompileOnly.scala
@@ -15,6 +15,7 @@ import akka.persistence.typed.DeleteEventsFailed
import akka.persistence.typed.DeleteSnapshotsFailed
import akka.persistence.typed.EventAdapter
import akka.persistence.typed.EventSeq
+import akka.persistence.typed.scaladsl.Recovery
//#structure
//#behavior
import akka.persistence.typed.scaladsl.EventSourcedBehavior
@@ -121,6 +122,18 @@ object BasicPersistentBehaviorCompileOnly {
//#recovery
}
+ object RecoveryDisabledBehavior {
+ def apply(): Behavior[Command] =
+ //#recovery-disabled
+ EventSourcedBehavior[Command, Event, State](
+ persistenceId = PersistenceId.ofUniqueId("abc"),
+ emptyState = State(),
+ commandHandler = (state, cmd) => throw new NotImplementedError("TODO: process the command & return an Effect"),
+ eventHandler = (state, evt) => throw new NotImplementedError("TODO: process the event return the next state"))
+ .withRecovery(Recovery.disabled)
+ //#recovery-disabled
+ }
+
object TaggingBehavior {
def apply(): Behavior[Command] =
//#tagging
@@ -241,7 +254,7 @@ object BasicPersistentBehaviorCompileOnly {
emptyState = State(),
commandHandler = (state, cmd) => throw new NotImplementedError("TODO: process the command & return an Effect"),
eventHandler = (state, evt) => throw new NotImplementedError("TODO: process the event return the next state"))
- .withSnapshotSelectionCriteria(SnapshotSelectionCriteria.none)
+ .withRecovery(Recovery.withSnapshotSelectionCriteria(SnapshotSelectionCriteria.none))
//#snapshotSelection
//#retentionCriteria
diff --git a/akka-persistence/src/main/java/akka/persistence/serialization/MessageFormats.java b/akka-persistence/src/main/java/akka/persistence/serialization/MessageFormats.java
index 027ddb6b6f..9cb54077c0 100644
--- a/akka-persistence/src/main/java/akka/persistence/serialization/MessageFormats.java
+++ b/akka-persistence/src/main/java/akka/persistence/serialization/MessageFormats.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2019-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
diff --git a/akka-persistence/src/main/scala/akka/persistence/AtLeastOnceDelivery.scala b/akka-persistence/src/main/scala/akka/persistence/AtLeastOnceDelivery.scala
index dffbbd5fb0..166633e1ae 100644
--- a/akka-persistence/src/main/scala/akka/persistence/AtLeastOnceDelivery.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/AtLeastOnceDelivery.scala
@@ -8,11 +8,11 @@ import scala.collection.immutable
import scala.concurrent.duration.FiniteDuration
import akka.actor.{ ActorPath, ActorSelection, NotInfluenceReceiveTimeout }
-import akka.persistence.serialization.Message
import akka.actor.Cancellable
import akka.actor.DeadLetterSuppression
import akka.annotation.InternalApi
import akka.persistence.AtLeastOnceDelivery.Internal.Delivery
+import akka.persistence.serialization.Message
import akka.util.ccompat._
@ccompatUsedUntil213
diff --git a/akka-persistence/src/main/scala/akka/persistence/Eventsourced.scala b/akka-persistence/src/main/scala/akka/persistence/Eventsourced.scala
index 3245cf99b6..c4397bc78d 100644
--- a/akka-persistence/src/main/scala/akka/persistence/Eventsourced.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/Eventsourced.scala
@@ -7,17 +7,18 @@ package akka.persistence
import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
+import scala.collection.immutable
+import scala.concurrent.duration.FiniteDuration
+import scala.util.control.NonFatal
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
import akka.actor.{ Actor, ActorCell, DeadLetter, StashOverflowException }
import akka.annotation.{ InternalApi, InternalStableApi }
import akka.dispatch.Envelope
import akka.event.{ Logging, LoggingAdapter }
import akka.util.Helpers.ConfigOps
-import com.github.ghik.silencer.silent
-import com.typesafe.config.ConfigFactory
-
-import scala.collection.immutable
-import scala.concurrent.duration.FiniteDuration
-import scala.util.control.NonFatal
/** INTERNAL API */
@InternalApi
diff --git a/akka-persistence/src/main/scala/akka/persistence/JournalProtocol.scala b/akka-persistence/src/main/scala/akka/persistence/JournalProtocol.scala
index a748593c8e..a0bfd4748e 100644
--- a/akka-persistence/src/main/scala/akka/persistence/JournalProtocol.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/JournalProtocol.scala
@@ -4,10 +4,10 @@
package akka.persistence
-import akka.actor._
-
import scala.collection.immutable
+import akka.actor._
+
/**
* INTERNAL API.
*
diff --git a/akka-persistence/src/main/scala/akka/persistence/Persistence.scala b/akka-persistence/src/main/scala/akka/persistence/Persistence.scala
index 1c168e357c..317c5f32be 100644
--- a/akka-persistence/src/main/scala/akka/persistence/Persistence.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/Persistence.scala
@@ -7,20 +7,20 @@ package akka.persistence
import java.util.concurrent.atomic.AtomicReference
import java.util.function.Consumer
+import scala.annotation.tailrec
+import scala.concurrent.duration._
+import scala.util.control.NonFatal
+
+import com.typesafe.config.{ Config, ConfigFactory }
+
import akka.actor._
+import akka.annotation.InternalApi
+import akka.annotation.InternalStableApi
import akka.event.{ Logging, LoggingAdapter }
import akka.persistence.journal.{ EventAdapters, IdentityEventAdapters }
import akka.util.Collections.EmptyImmutableSeq
import akka.util.Helpers.ConfigOps
-import com.typesafe.config.{ Config, ConfigFactory }
-import scala.annotation.tailrec
-import scala.concurrent.duration._
-
import akka.util.Reflect
-import scala.util.control.NonFatal
-
-import akka.annotation.InternalApi
-import akka.annotation.InternalStableApi
/**
* Persistence configuration.
diff --git a/akka-persistence/src/main/scala/akka/persistence/PersistencePlugin.scala b/akka-persistence/src/main/scala/akka/persistence/PersistencePlugin.scala
index f8e70ac3d2..2515054ce3 100644
--- a/akka-persistence/src/main/scala/akka/persistence/PersistencePlugin.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/PersistencePlugin.scala
@@ -6,15 +6,16 @@ package akka.persistence
import java.util.concurrent.atomic.AtomicReference
+import scala.annotation.tailrec
+import scala.reflect.ClassTag
+import scala.util.Failure
+
+import com.typesafe.config.Config
+
import akka.actor.{ ExtendedActorSystem, Extension, ExtensionId }
import akka.annotation.InternalApi
import akka.event.Logging
import akka.persistence.PersistencePlugin.PluginHolder
-import com.typesafe.config.Config
-
-import scala.annotation.tailrec
-import scala.reflect.ClassTag
-import scala.util.Failure
/**
* INTERNAL API
diff --git a/akka-persistence/src/main/scala/akka/persistence/Persistent.scala b/akka-persistence/src/main/scala/akka/persistence/Persistent.scala
index 20164a9314..f260f79124 100644
--- a/akka-persistence/src/main/scala/akka/persistence/Persistent.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/Persistent.scala
@@ -4,11 +4,11 @@
package akka.persistence
-import akka.actor.{ ActorRef, NoSerializationVerificationNeeded }
-import akka.persistence.serialization.Message
import scala.collection.immutable
+import akka.actor.{ ActorRef, NoSerializationVerificationNeeded }
import akka.annotation.DoNotInherit
+import akka.persistence.serialization.Message
import akka.util.HashCode
/**
diff --git a/akka-persistence/src/main/scala/akka/persistence/PersistentActor.scala b/akka-persistence/src/main/scala/akka/persistence/PersistentActor.scala
index 9b45ef855c..474b2a9641 100644
--- a/akka-persistence/src/main/scala/akka/persistence/PersistentActor.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/PersistentActor.scala
@@ -9,11 +9,12 @@ import java.lang.{ Iterable => JIterable }
import scala.collection.immutable
import scala.util.control.NoStackTrace
+import com.typesafe.config.Config
+
import akka.actor._
import akka.annotation.InternalApi
import akka.japi.Procedure
import akka.japi.Util
-import com.typesafe.config.Config
abstract class RecoveryCompleted
diff --git a/akka-persistence/src/main/scala/akka/persistence/RecoveryPermitter.scala b/akka-persistence/src/main/scala/akka/persistence/RecoveryPermitter.scala
index a9a596b280..4f07df4ec3 100644
--- a/akka-persistence/src/main/scala/akka/persistence/RecoveryPermitter.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/RecoveryPermitter.scala
@@ -4,12 +4,12 @@
package akka.persistence
-import akka.annotation.{ InternalApi, InternalStableApi }
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorRef
import akka.actor.Props
import akka.actor.Terminated
+import akka.annotation.{ InternalApi, InternalStableApi }
import akka.util.MessageBuffer
/**
diff --git a/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSM.scala b/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSM.scala
index f520059791..510c8b9134 100644
--- a/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSM.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSM.scala
@@ -4,19 +4,21 @@
package akka.persistence.fsm
-import akka.actor._
-import akka.annotation.InternalApi
-import akka.persistence.fsm.PersistentFSM.FSMState
-import akka.persistence.serialization.Message
-import akka.persistence.{ PersistentActor, RecoveryCompleted, SnapshotOffer }
-import akka.util.JavaDurationConverters
+import scala.annotation.varargs
+import scala.collection.immutable
+import scala.concurrent.ExecutionContextExecutor
+import scala.concurrent.duration._
+import scala.reflect.ClassTag
+
import com.github.ghik.silencer.silent
import com.typesafe.config.Config
-import scala.annotation.varargs
-import scala.collection.immutable
-import scala.concurrent.duration._
-import scala.reflect.ClassTag
+import akka.actor._
+import akka.annotation.InternalApi
+import akka.persistence.{ PersistentActor, RecoveryCompleted, SnapshotOffer }
+import akka.persistence.fsm.PersistentFSM.FSMState
+import akka.persistence.serialization.Message
+import akka.util.JavaDurationConverters
/**
* SnapshotAfter Extension Id and factory for creating SnapshotAfter extension
@@ -331,7 +333,7 @@ object PersistentFSM {
extends NoSerializationVerificationNeeded {
private var ref: Option[Cancellable] = _
private val scheduler = context.system.scheduler
- private implicit val executionContext = context.dispatcher
+ private implicit val executionContext: ExecutionContextExecutor = context.dispatcher
def schedule(actor: ActorRef, timeout: FiniteDuration): Unit = {
val timerMsg = msg match {
diff --git a/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSMBase.scala b/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSMBase.scala
index a2e40d01ca..6f325d9dc0 100644
--- a/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSMBase.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSMBase.scala
@@ -4,15 +4,16 @@
package akka.persistence.fsm
+import scala.collection.mutable
+import scala.concurrent.duration.FiniteDuration
+
+import language.implicitConversions
+
import akka.actor._
import akka.japi.pf.{ FSMTransitionHandlerBuilder, UnitMatch, UnitPFBuilder }
-import language.implicitConversions
-import scala.collection.mutable
-
import akka.routing.{ Deafen, Listen, Listeners }
-import akka.util.unused
import akka.util.JavaDurationConverters._
-import scala.concurrent.duration.FiniteDuration
+import akka.util.unused
/**
* Finite State Machine actor trait. Use as follows:
@@ -192,7 +193,7 @@ trait PersistentFSMBase[S, D, E] extends Actor with Listeners with ActorLogging
/**
* Produce change descriptor to stop this FSM actor including specified reason.
*/
- final def stop(reason: Reason, stateData: D): State = stay.copy(stopReason = Some(reason), stateData = stateData)
+ final def stop(reason: Reason, stateData: D): State = stay().copy(stopReason = Some(reason), stateData = stateData)
final class TransformHelper(func: StateFunction) {
def using(andThen: PartialFunction[State, State]): StateFunction =
@@ -283,9 +284,9 @@ trait PersistentFSMBase[S, D, E] extends Actor with Listeners with ActorLogging
if (debugEvent)
log.debug("setting " + (if (mode.repeat) "repeating " else "") + "timer '" + name + "'/" + timeout + ": " + msg)
if (timers contains name) {
- timers(name).cancel
+ timers(name).cancel()
}
- val timer = Timer(name, msg, mode, timerGen.next, this)(context)
+ val timer = Timer(name, msg, mode, timerGen.next(), this)(context)
timer.schedule(self, timeout)
timers(name) = timer
}
@@ -298,7 +299,7 @@ trait PersistentFSMBase[S, D, E] extends Actor with Listeners with ActorLogging
if (debugEvent)
log.debug("canceling timer '" + name + "'")
if (timers contains name) {
- timers(name).cancel
+ timers(name).cancel()
timers -= name
}
}
@@ -458,7 +459,7 @@ trait PersistentFSMBase[S, D, E] extends Actor with Listeners with ActorLogging
private val handleEventDefault: StateFunction = {
case Event(value, _) =>
log.warning("unhandled event " + value + " in state " + stateName)
- stay
+ stay()
}
private var handleEvent: StateFunction = handleEventDefault
@@ -551,7 +552,7 @@ trait PersistentFSMBase[S, D, E] extends Actor with Listeners with ActorLogging
private[akka] def makeTransition(nextState: State): Unit = {
if (!stateFunctions.contains(nextState.stateName)) {
- terminate(stay.withStopReason(Failure("Next state %s does not exist".format(nextState.stateName))))
+ terminate(stay().withStopReason(Failure("Next state %s does not exist".format(nextState.stateName))))
} else {
nextState.replies.reverse.foreach { r =>
sender() ! r
@@ -594,7 +595,7 @@ trait PersistentFSMBase[S, D, E] extends Actor with Listeners with ActorLogging
* setting this instance’s state to terminated does no harm during restart
* since the new instance will initialize fresh using startWith()
*/
- terminate(stay.withStopReason(Shutdown))
+ terminate(stay().withStopReason(Shutdown))
super.postStop()
}
@@ -720,11 +721,13 @@ object AbstractPersistentFSMBase {
*/
@deprecated("Use EventSourcedBehavior", "2.6.0")
abstract class AbstractPersistentFSMBase[S, D, E] extends PersistentFSMBase[S, D, E] {
+ import java.util.{ List => JList }
+
+ import PersistentFSM._
+
+ import akka.japi.pf.FI._
import akka.persistence.fsm.japi.pf.FSMStateFunctionBuilder
import akka.persistence.fsm.japi.pf.FSMStopBuilder
- import akka.japi.pf.FI._
- import java.util.{ List => JList }
- import PersistentFSM._
/**
* Returns this AbstractActor's ActorContext
diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/AsyncWriteJournal.scala b/akka-persistence/src/main/scala/akka/persistence/journal/AsyncWriteJournal.scala
index f47cea2c46..33426702fe 100644
--- a/akka-persistence/src/main/scala/akka/persistence/journal/AsyncWriteJournal.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/journal/AsyncWriteJournal.scala
@@ -4,19 +4,18 @@
package akka.persistence.journal
-import scala.concurrent.duration._
-
-import akka.actor._
-import akka.pattern.pipe
-import akka.persistence._
-import akka.util.Helpers.toRootLowerCase
import scala.collection.immutable
import scala.concurrent.ExecutionContext
import scala.concurrent.Future
+import scala.concurrent.duration._
import scala.util.{ Failure, Success, Try }
import scala.util.control.NonFatal
+import akka.actor._
import akka.pattern.CircuitBreaker
+import akka.pattern.pipe
+import akka.persistence._
+import akka.util.Helpers.toRootLowerCase
/**
* Abstract journal, optimized for asynchronous, non-blocking writes.
diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/AsyncWriteProxy.scala b/akka-persistence/src/main/scala/akka/persistence/journal/AsyncWriteProxy.scala
index 42f2bdd06a..4834552630 100644
--- a/akka-persistence/src/main/scala/akka/persistence/journal/AsyncWriteProxy.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/journal/AsyncWriteProxy.scala
@@ -4,16 +4,16 @@
package akka.persistence.journal
+import scala.collection.immutable
+import scala.concurrent._
+import scala.concurrent.duration.Duration
+import scala.util.Try
+
import akka.AkkaException
import akka.actor._
import akka.pattern.ask
import akka.persistence._
import akka.util._
-import scala.util.Try
-
-import scala.collection.immutable
-import scala.concurrent._
-import scala.concurrent.duration.Duration
/**
* INTERNAL API.
@@ -98,7 +98,7 @@ private[persistence] trait AsyncWriteProxy extends AsyncWriteJournal with Stash
*/
private[persistence] object AsyncWriteProxy {
final case class SetStore(ref: ActorRef)
- final case object InitTimeout
+ case object InitTimeout
}
/**
diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/EventAdapter.scala b/akka-persistence/src/main/scala/akka/persistence/journal/EventAdapter.scala
index 2134420873..6fcf4d45f6 100644
--- a/akka-persistence/src/main/scala/akka/persistence/journal/EventAdapter.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/journal/EventAdapter.scala
@@ -105,14 +105,14 @@ final case class SingleEventSeq(event: Any) extends EventSeq { // TODO try to ma
}
sealed trait EmptyEventSeq extends EventSeq
-final object EmptyEventSeq extends EmptyEventSeq {
+object EmptyEventSeq extends EmptyEventSeq {
override def events = Nil
}
final case class EventsSeq[E](events: immutable.Seq[E]) extends EventSeq
/** No-op model adapter which passes through the incoming events as-is. */
-final case object IdentityEventAdapter extends EventAdapter {
+case object IdentityEventAdapter extends EventAdapter {
override def toJournal(event: Any): Any = event
override def fromJournal(event: Any, manifest: String): EventSeq = EventSeq.single(event)
override def manifest(event: Any): String = ""
diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/EventAdapters.scala b/akka-persistence/src/main/scala/akka/persistence/journal/EventAdapters.scala
index 46cfd25953..de13145a93 100644
--- a/akka-persistence/src/main/scala/akka/persistence/journal/EventAdapters.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/journal/EventAdapters.scala
@@ -7,16 +7,17 @@ package akka.persistence.journal
import java.util
import java.util.concurrent.ConcurrentHashMap
-import akka.actor.ExtendedActorSystem
-import akka.event.{ Logging, LoggingAdapter }
-import akka.util.ccompat._
-import com.typesafe.config.Config
-
import scala.collection.immutable
import scala.collection.mutable.ArrayBuffer
import scala.reflect.ClassTag
import scala.util.Try
+import com.typesafe.config.Config
+
+import akka.actor.ExtendedActorSystem
+import akka.event.{ Logging, LoggingAdapter }
+import akka.util.ccompat._
+
/**
* `EventAdapters` serves as a per-journal collection of bound event adapters.
*/
diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/PersistencePluginProxy.scala b/akka-persistence/src/main/scala/akka/persistence/journal/PersistencePluginProxy.scala
index 2fb0d9e12f..e33296cec1 100644
--- a/akka-persistence/src/main/scala/akka/persistence/journal/PersistencePluginProxy.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/journal/PersistencePluginProxy.scala
@@ -7,6 +7,10 @@ package akka.persistence.journal
import java.net.URISyntaxException
import java.util.concurrent.TimeoutException
+import scala.concurrent.duration._
+
+import com.typesafe.config.Config
+
import akka.actor._
import akka.persistence.{
AtomicWrite,
@@ -20,9 +24,6 @@ import akka.persistence.{
SnapshotProtocol
}
import akka.util.Helpers.Requiring
-import com.typesafe.config.Config
-
-import scala.concurrent.duration._
object PersistencePluginProxy {
final case class TargetLocation(address: Address)
@@ -72,8 +73,8 @@ object PersistencePluginProxyExtension
}
final class PersistencePluginProxy(config: Config) extends Actor with Stash with ActorLogging {
- import PersistencePluginProxy._
import JournalProtocol._
+ import PersistencePluginProxy._
import SnapshotProtocol._
private val pluginId = self.path.name
@@ -190,31 +191,31 @@ final class PersistencePluginProxy(config: Config) extends Actor with Stash with
req match { // exhaustive match
case WriteMessages(messages, persistentActor, actorInstanceId) =>
val atomicWriteCount = messages.count(_.isInstanceOf[AtomicWrite])
- persistentActor ! WriteMessagesFailed(timeoutException, atomicWriteCount)
+ persistentActor ! WriteMessagesFailed(timeoutException(), atomicWriteCount)
messages.foreach {
case a: AtomicWrite =>
a.payload.foreach { p =>
- persistentActor ! WriteMessageFailure(p, timeoutException, actorInstanceId)
+ persistentActor ! WriteMessageFailure(p, timeoutException(), actorInstanceId)
}
case r: NonPersistentRepr =>
persistentActor ! LoopMessageSuccess(r.payload, actorInstanceId)
}
case ReplayMessages(_, _, _, _, persistentActor) =>
- persistentActor ! ReplayMessagesFailure(timeoutException)
+ persistentActor ! ReplayMessagesFailure(timeoutException())
case DeleteMessagesTo(_, toSequenceNr, persistentActor) =>
- persistentActor ! DeleteMessagesFailure(timeoutException, toSequenceNr)
+ persistentActor ! DeleteMessagesFailure(timeoutException(), toSequenceNr)
}
case req: SnapshotProtocol.Request =>
req match { // exhaustive match
case _: LoadSnapshot =>
- sender() ! LoadSnapshotFailed(timeoutException)
+ sender() ! LoadSnapshotFailed(timeoutException())
case SaveSnapshot(metadata, _) =>
- sender() ! SaveSnapshotFailure(metadata, timeoutException)
+ sender() ! SaveSnapshotFailure(metadata, timeoutException())
case DeleteSnapshot(metadata) =>
- sender() ! DeleteSnapshotFailure(metadata, timeoutException)
+ sender() ! DeleteSnapshotFailure(metadata, timeoutException())
case DeleteSnapshots(_, criteria) =>
- sender() ! DeleteSnapshotsFailure(criteria, timeoutException)
+ sender() ! DeleteSnapshotsFailure(criteria, timeoutException())
}
case TargetLocation(address) =>
diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/ReplayFilter.scala b/akka-persistence/src/main/scala/akka/persistence/journal/ReplayFilter.scala
index 8134c06392..f23e8608ea 100644
--- a/akka-persistence/src/main/scala/akka/persistence/journal/ReplayFilter.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/journal/ReplayFilter.scala
@@ -4,14 +4,16 @@
package akka.persistence.journal
-import akka.actor.ActorRef
-import akka.actor.Actor
-import akka.persistence.JournalProtocol
import java.util.LinkedList
-import akka.actor.Props
-import akka.actor.ActorLogging
+
import scala.collection.mutable.LinkedHashSet
+import akka.actor.Actor
+import akka.actor.ActorLogging
+import akka.actor.ActorRef
+import akka.actor.Props
+import akka.persistence.JournalProtocol
+
/**
* INTERNAL API
*
diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/WriteJournalBase.scala b/akka-persistence/src/main/scala/akka/persistence/journal/WriteJournalBase.scala
index 79e223a648..bc97417ef6 100644
--- a/akka-persistence/src/main/scala/akka/persistence/journal/WriteJournalBase.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/journal/WriteJournalBase.scala
@@ -4,9 +4,10 @@
package akka.persistence.journal
+import scala.collection.immutable
+
import akka.actor.Actor
import akka.persistence.{ Persistence, PersistentEnvelope, PersistentRepr }
-import scala.collection.immutable
import akka.persistence.AtomicWrite
private[akka] trait WriteJournalBase {
diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/inmem/InmemJournal.scala b/akka-persistence/src/main/scala/akka/persistence/journal/inmem/InmemJournal.scala
index 4dcf15c24b..be2f0122ef 100644
--- a/akka-persistence/src/main/scala/akka/persistence/journal/inmem/InmemJournal.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/journal/inmem/InmemJournal.scala
@@ -8,16 +8,18 @@ import scala.collection.immutable
import scala.concurrent.Future
import scala.util.Try
import scala.util.control.NonFatal
-import akka.annotation.ApiMayChange
-import akka.annotation.InternalApi
-import akka.persistence.journal.{ AsyncWriteJournal, Tagged }
-import akka.persistence.PersistentRepr
-import akka.persistence.AtomicWrite
-import akka.serialization.SerializationExtension
-import akka.serialization.Serializers
+
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
+import akka.annotation.ApiMayChange
+import akka.annotation.InternalApi
+import akka.persistence.AtomicWrite
+import akka.persistence.PersistentRepr
+import akka.persistence.journal.{ AsyncWriteJournal, Tagged }
+import akka.serialization.SerializationExtension
+import akka.serialization.Serializers
+
/**
* The InmemJournal publishes writes and deletes to the `eventStream`, which tests may use to
* verify that expected events have been persisted or deleted.
diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/japi/AsyncRecovery.scala b/akka-persistence/src/main/scala/akka/persistence/journal/japi/AsyncRecovery.scala
index 0ce692b075..d0c8ed2e93 100644
--- a/akka-persistence/src/main/scala/akka/persistence/journal/japi/AsyncRecovery.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/journal/japi/AsyncRecovery.scala
@@ -5,11 +5,12 @@
package akka.persistence.journal.japi
import java.util.function.Consumer
+
import scala.concurrent.Future
import akka.actor.Actor
-import akka.persistence.journal.{ AsyncRecovery => SAsyncReplay }
import akka.persistence.PersistentRepr
+import akka.persistence.journal.{ AsyncRecovery => SAsyncReplay }
/**
* Java API: asynchronous message replay and sequence number recovery interface.
diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/japi/AsyncWriteJournal.scala b/akka-persistence/src/main/scala/akka/persistence/journal/japi/AsyncWriteJournal.scala
index 1b3ddfb3df..8b3eac5f0d 100644
--- a/akka-persistence/src/main/scala/akka/persistence/journal/japi/AsyncWriteJournal.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/journal/japi/AsyncWriteJournal.scala
@@ -5,13 +5,14 @@
package akka.persistence.journal.japi
import scala.collection.immutable
-import akka.util.ccompat.JavaConverters._
+import scala.concurrent.Future
+import scala.util.Failure
+import scala.util.Try
+
import akka.persistence._
import akka.persistence.journal.{ AsyncWriteJournal => SAsyncWriteJournal }
import akka.util.ccompat._
-import scala.concurrent.Future
-import scala.util.Try
-import scala.util.Failure
+import akka.util.ccompat.JavaConverters._
/**
* Java API: abstract journal, optimized for asynchronous, non-blocking writes.
diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbIdMapping.scala b/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbIdMapping.scala
index a97fada79e..67c6d6f9f7 100644
--- a/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbIdMapping.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbIdMapping.scala
@@ -5,6 +5,7 @@
package akka.persistence.journal.leveldb
import org.iq80.leveldb.DBIterator
+
import akka.actor.Actor
import akka.util.ByteString.UTF_8
import akka.util.unused
diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbJournal.scala b/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbJournal.scala
index c0a2fe5b50..7129b96e8a 100644
--- a/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbJournal.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbJournal.scala
@@ -4,20 +4,21 @@
package akka.persistence.journal.leveldb
-import akka.actor._
-import akka.persistence.Persistence
-import akka.persistence.journal._
-import akka.util.Timeout
-import akka.util.Helpers.ConfigOps
-import akka.persistence.PersistentRepr
import scala.concurrent.Future
-import akka.persistence.JournalProtocol.RecoverySuccess
-import akka.persistence.JournalProtocol.ReplayMessagesFailure
-import akka.pattern.pipe
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
+import akka.actor._
+import akka.pattern.pipe
+import akka.persistence.JournalProtocol.RecoverySuccess
+import akka.persistence.JournalProtocol.ReplayMessagesFailure
+import akka.persistence.Persistence
+import akka.persistence.PersistentRepr
+import akka.persistence.journal._
+import akka.util.Helpers.ConfigOps
+import akka.util.Timeout
+
/**
* INTERNAL API.
*
@@ -92,7 +93,7 @@ private[persistence] object LeveldbJournal {
* subscriber followed by [[PersistenceIdAdded]] messages when new persistenceIds
* are created.
*/
- final case object SubscribeAllPersistenceIds extends SubscriptionCommand
+ case object SubscribeAllPersistenceIds extends SubscriptionCommand
final case class CurrentPersistenceIds(allPersistenceIds: Set[String]) extends DeadLetterSuppression
final case class PersistenceIdAdded(persistenceId: String) extends DeadLetterSuppression
diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbRecovery.scala b/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbRecovery.scala
index 1a8700761c..8bca84e264 100644
--- a/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbRecovery.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbRecovery.scala
@@ -5,9 +5,11 @@
package akka.persistence.journal.leveldb
import scala.concurrent.Future
+
+import org.iq80.leveldb.DBIterator
+
import akka.persistence._
import akka.persistence.journal.AsyncRecovery
-import org.iq80.leveldb.DBIterator
import akka.persistence.journal.leveldb.LeveldbJournal.ReplayedTaggedMessage
/**
diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbStore.scala b/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbStore.scala
index 11caeee05c..64328f8183 100644
--- a/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbStore.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/LeveldbStore.scala
@@ -6,20 +6,21 @@ package akka.persistence.journal.leveldb
import java.io.File
+import scala.collection.immutable
import scala.collection.mutable
-import akka.actor._
-import akka.persistence._
-import akka.persistence.journal.WriteJournalBase
-import akka.serialization.SerializationExtension
+import scala.concurrent.Future
+import scala.util._
+import scala.util.control.NonFatal
+
+import com.typesafe.config.{ Config, ConfigFactory, ConfigObject }
import org.iq80.leveldb._
-import scala.collection.immutable
-import akka.util.ccompat.JavaConverters._
-import scala.util._
-import scala.concurrent.Future
-import scala.util.control.NonFatal
+import akka.actor._
+import akka.persistence._
import akka.persistence.journal.Tagged
-import com.typesafe.config.{ Config, ConfigFactory, ConfigObject }
+import akka.persistence.journal.WriteJournalBase
+import akka.serialization.SerializationExtension
+import akka.util.ccompat.JavaConverters._
private[persistence] object LeveldbStore {
val emptyConfig = ConfigFactory.empty()
diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/SharedLeveldbStore.scala b/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/SharedLeveldbStore.scala
index 2d003cac9d..48cbeb9086 100644
--- a/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/SharedLeveldbStore.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/journal/leveldb/SharedLeveldbStore.scala
@@ -4,15 +4,17 @@
package akka.persistence.journal.leveldb
-import akka.persistence.journal.AsyncWriteTarget
-import akka.pattern.pipe
-import scala.util.Try
-import scala.util.Success
-import scala.util.Failure
-import scala.util.control.NonFatal
-import akka.persistence.AtomicWrite
-import com.typesafe.config.Config
import scala.concurrent.Future
+import scala.util.Failure
+import scala.util.Success
+import scala.util.Try
+import scala.util.control.NonFatal
+
+import com.typesafe.config.Config
+
+import akka.pattern.pipe
+import akka.persistence.AtomicWrite
+import akka.persistence.journal.AsyncWriteTarget
/**
* A LevelDB store that can be shared by multiple actor systems. The shared store must be
diff --git a/akka-persistence/src/main/scala/akka/persistence/serialization/MessageSerializer.scala b/akka-persistence/src/main/scala/akka/persistence/serialization/MessageSerializer.scala
index f4ca47e722..8916b0bb7a 100644
--- a/akka-persistence/src/main/scala/akka/persistence/serialization/MessageSerializer.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/serialization/MessageSerializer.scala
@@ -4,24 +4,25 @@
package akka.persistence.serialization
-import akka.actor.{ ActorPath, ExtendedActorSystem }
-import akka.persistence.AtLeastOnceDelivery._
-import akka.persistence._
-import akka.persistence.fsm.PersistentFSM.{ PersistentFSMSnapshot, StateChangeEvent }
-import akka.persistence.serialization.{ MessageFormats => mf }
-import akka.serialization._
-import akka.protobufv3.internal.ByteString
+import java.io.NotSerializableException
+
import scala.collection.immutable
import scala.collection.immutable.VectorBuilder
import scala.concurrent.duration
-import akka.actor.Actor
-import akka.util.ccompat._
-
import scala.concurrent.duration.Duration
-import java.io.NotSerializableException
import com.github.ghik.silencer.silent
+import akka.actor.{ ActorPath, ExtendedActorSystem }
+import akka.actor.Actor
+import akka.persistence._
+import akka.persistence.AtLeastOnceDelivery._
+import akka.persistence.fsm.PersistentFSM.{ PersistentFSMSnapshot, StateChangeEvent }
+import akka.persistence.serialization.{ MessageFormats => mf }
+import akka.protobufv3.internal.ByteString
+import akka.serialization._
+import akka.util.ccompat._
+
/**
* Marker trait for all protobuf-serializable messages in `akka.persistence`.
*/
diff --git a/akka-persistence/src/main/scala/akka/persistence/serialization/SnapshotSerializer.scala b/akka-persistence/src/main/scala/akka/persistence/serialization/SnapshotSerializer.scala
index c556d9e9e6..66da0f6a0c 100644
--- a/akka-persistence/src/main/scala/akka/persistence/serialization/SnapshotSerializer.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/serialization/SnapshotSerializer.scala
@@ -5,6 +5,7 @@
package akka.persistence.serialization
import java.io._
+
import akka.actor._
import akka.serialization._
import akka.util.ByteString.UTF_8
diff --git a/akka-persistence/src/main/scala/akka/persistence/snapshot/NoSnapshotStore.scala b/akka-persistence/src/main/scala/akka/persistence/snapshot/NoSnapshotStore.scala
index 81883603d4..6229f21f76 100644
--- a/akka-persistence/src/main/scala/akka/persistence/snapshot/NoSnapshotStore.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/snapshot/NoSnapshotStore.scala
@@ -4,10 +4,10 @@
package akka.persistence.snapshot
-import akka.persistence.{ SelectedSnapshot, SnapshotMetadata, SnapshotSelectionCriteria }
-
import scala.concurrent.Future
+import akka.persistence.{ SelectedSnapshot, SnapshotMetadata, SnapshotSelectionCriteria }
+
/**
* Used as default snapshot-store in case no other store was configured.
*
diff --git a/akka-persistence/src/main/scala/akka/persistence/snapshot/SnapshotStore.scala b/akka-persistence/src/main/scala/akka/persistence/snapshot/SnapshotStore.scala
index c4e7db49ca..31094b8100 100644
--- a/akka-persistence/src/main/scala/akka/persistence/snapshot/SnapshotStore.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/snapshot/SnapshotStore.scala
@@ -5,13 +5,13 @@
package akka.persistence.snapshot
import scala.concurrent.ExecutionContext
-import scala.concurrent.duration._
import scala.concurrent.Future
+import scala.concurrent.duration._
import akka.actor._
+import akka.pattern.CircuitBreaker
import akka.pattern.pipe
import akka.persistence._
-import akka.pattern.CircuitBreaker
/**
* Abstract snapshot store.
diff --git a/akka-persistence/src/main/scala/akka/persistence/snapshot/japi/SnapshotStore.scala b/akka-persistence/src/main/scala/akka/persistence/snapshot/japi/SnapshotStore.scala
index af7d1f5cab..adf9064bb7 100644
--- a/akka-persistence/src/main/scala/akka/persistence/snapshot/japi/SnapshotStore.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/snapshot/japi/SnapshotStore.scala
@@ -4,11 +4,11 @@
package akka.persistence.snapshot.japi
+import scala.concurrent.Future
+
+import akka.japi.Util._
import akka.persistence._
import akka.persistence.snapshot.{ SnapshotStore => SSnapshotStore }
-import akka.japi.Util._
-
-import scala.concurrent.Future
/**
* Java API: abstract snapshot store.
diff --git a/akka-persistence/src/main/scala/akka/persistence/snapshot/local/LocalSnapshotStore.scala b/akka-persistence/src/main/scala/akka/persistence/snapshot/local/LocalSnapshotStore.scala
index b662345c48..de860b1479 100644
--- a/akka-persistence/src/main/scala/akka/persistence/snapshot/local/LocalSnapshotStore.scala
+++ b/akka-persistence/src/main/scala/akka/persistence/snapshot/local/LocalSnapshotStore.scala
@@ -6,6 +6,13 @@ package akka.persistence.snapshot.local
import java.io._
import java.net.{ URLDecoder, URLEncoder }
+import java.nio.file.Files
+
+import scala.collection.immutable
+import scala.concurrent.Future
+import scala.util._
+
+import com.typesafe.config.Config
import akka.actor.ActorLogging
import akka.persistence._
@@ -14,12 +21,6 @@ import akka.persistence.snapshot._
import akka.serialization.SerializationExtension
import akka.util.ByteString.UTF_8
import akka.util.ccompat._
-import com.typesafe.config.Config
-
-import scala.collection.immutable
-import scala.concurrent.Future
-import scala.util._
-import java.nio.file.Files
/**
* INTERNAL API
@@ -92,7 +93,7 @@ private[persistence] class LocalSnapshotStore(config: Config) extends SnapshotSt
}
private def snapshotFiles(metadata: SnapshotMetadata): immutable.Seq[File] = {
- snapshotDir.listFiles(new SnapshotSeqNrFilenameFilter(metadata)).toVector
+ snapshotDir().listFiles(new SnapshotSeqNrFilenameFilter(metadata)).toVector
}
@scala.annotation.tailrec
@@ -144,13 +145,13 @@ private[persistence] class LocalSnapshotStore(config: Config) extends SnapshotSt
/** Only by persistenceId and sequenceNr, timestamp is informational - accommodates for 2.13.x series files */
protected def snapshotFileForWrite(metadata: SnapshotMetadata, extension: String = ""): File =
new File(
- snapshotDir,
+ snapshotDir(),
s"snapshot-${URLEncoder.encode(metadata.persistenceId, UTF_8)}-${metadata.sequenceNr}-${metadata.timestamp}${extension}")
private def snapshotMetadatas(
persistenceId: String,
criteria: SnapshotSelectionCriteria): immutable.Seq[SnapshotMetadata] = {
- val files = snapshotDir.listFiles(new SnapshotFilenameFilter(persistenceId))
+ val files = snapshotDir().listFiles(new SnapshotFilenameFilter(persistenceId))
if (files eq null) Nil // if the dir was removed
else {
files
diff --git a/akka-persistence/src/test/scala/akka/persistence/AtLeastOnceDeliveryCrashSpec.scala b/akka-persistence/src/test/scala/akka/persistence/AtLeastOnceDeliveryCrashSpec.scala
index 7615990621..9c923a618a 100644
--- a/akka-persistence/src/test/scala/akka/persistence/AtLeastOnceDeliveryCrashSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/AtLeastOnceDeliveryCrashSpec.scala
@@ -4,12 +4,12 @@
package akka.persistence
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
import akka.actor._
import akka.actor.SupervisorStrategy.{ Escalate, Stop }
import akka.testkit.{ AkkaSpec, ImplicitSender, TestProbe }
-import scala.concurrent.duration._
-
-import scala.util.control.NoStackTrace
object AtLeastOnceDeliveryCrashSpec {
diff --git a/akka-persistence/src/test/scala/akka/persistence/AtLeastOnceDeliveryFailureSpec.scala b/akka-persistence/src/test/scala/akka/persistence/AtLeastOnceDeliveryFailureSpec.scala
index 2f07447d7f..e78004c5fa 100644
--- a/akka-persistence/src/test/scala/akka/persistence/AtLeastOnceDeliveryFailureSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/AtLeastOnceDeliveryFailureSpec.scala
@@ -4,8 +4,9 @@
package akka.persistence
-import scala.concurrent.duration._
import java.util.concurrent.ThreadLocalRandom
+
+import scala.concurrent.duration._
import scala.language.postfixOps
import com.typesafe.config.ConfigFactory
diff --git a/akka-persistence/src/test/scala/akka/persistence/AtLeastOnceDeliverySpec.scala b/akka-persistence/src/test/scala/akka/persistence/AtLeastOnceDeliverySpec.scala
index 332f32365a..d3ef5e7e96 100644
--- a/akka-persistence/src/test/scala/akka/persistence/AtLeastOnceDeliverySpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/AtLeastOnceDeliverySpec.scala
@@ -4,15 +4,16 @@
package akka.persistence
-import akka.actor._
-import akka.persistence.AtLeastOnceDelivery.{ AtLeastOnceDeliverySnapshot, UnconfirmedWarning }
-import akka.testkit._
-import com.typesafe.config._
-
import scala.concurrent.duration._
import scala.util.Failure
import scala.util.control.NoStackTrace
+import com.typesafe.config._
+
+import akka.actor._
+import akka.persistence.AtLeastOnceDelivery.{ AtLeastOnceDeliverySnapshot, UnconfirmedWarning }
+import akka.testkit._
+
object AtLeastOnceDeliverySpec {
case class Req(payload: String)
diff --git a/akka-persistence/src/test/scala/akka/persistence/EndToEndEventAdapterSpec.scala b/akka-persistence/src/test/scala/akka/persistence/EndToEndEventAdapterSpec.scala
index 4f09f88366..8b6a66a0e8 100644
--- a/akka-persistence/src/test/scala/akka/persistence/EndToEndEventAdapterSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/EndToEndEventAdapterSpec.scala
@@ -6,18 +6,19 @@ package akka.persistence
import java.io.File
-import akka.actor._
-import akka.persistence.journal.{ EventAdapter, EventSeq }
-import akka.testkit.TestProbe
-import akka.util.unused
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
import com.typesafe.config.{ Config, ConfigFactory }
import org.apache.commons.io.FileUtils
import org.scalatest.BeforeAndAfterAll
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
-import scala.concurrent.Await
-import scala.concurrent.duration._
+import akka.actor._
+import akka.persistence.journal.{ EventAdapter, EventSeq }
+import akka.testkit.TestProbe
+import akka.util.unused
object EndToEndEventAdapterSpec {
diff --git a/akka-persistence/src/test/scala/akka/persistence/EventAdapterSpec.scala b/akka-persistence/src/test/scala/akka/persistence/EventAdapterSpec.scala
index 7ec7adeb35..7e1e338f2c 100644
--- a/akka-persistence/src/test/scala/akka/persistence/EventAdapterSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/EventAdapterSpec.scala
@@ -4,14 +4,15 @@
package akka.persistence
+import scala.collection.immutable
+
+import com.typesafe.config.{ Config, ConfigFactory }
+
import akka.actor._
import akka.event.Logging
import akka.persistence.EventAdapterSpec.{ Tagged, UserDataChanged }
import akka.persistence.journal.{ EventAdapter, EventSeq, SingleEventSeq }
import akka.testkit.ImplicitSender
-import com.typesafe.config.{ Config, ConfigFactory }
-
-import scala.collection.immutable
object EventAdapterSpec {
@@ -101,7 +102,7 @@ abstract class EventAdapterSpec(journalName: String, journalConfig: Config, adap
import EventAdapterSpec._
- def this(journalName: String) {
+ def this(journalName: String) =
this(
"inmem",
PersistenceSpec.config("inmem", "InmemPersistentTaggingSpec"),
@@ -150,7 +151,6 @@ abstract class EventAdapterSpec(journalName: String, journalConfig: Config, adap
| }
|}
""".stripMargin))
- }
def persister(name: String, journalId: String = journalName) =
system.actorOf(Props(classOf[PersistAllIncomingActor], name, "akka.persistence.journal." + journalId))
diff --git a/akka-persistence/src/test/scala/akka/persistence/EventSourcedActorDeleteFailureSpec.scala b/akka-persistence/src/test/scala/akka/persistence/EventSourcedActorDeleteFailureSpec.scala
index 04af3df211..8160ac5d81 100644
--- a/akka-persistence/src/test/scala/akka/persistence/EventSourcedActorDeleteFailureSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/EventSourcedActorDeleteFailureSpec.scala
@@ -4,16 +4,16 @@
package akka.persistence
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
import akka.actor._
import akka.event.Logging
import akka.event.Logging.Warning
import akka.persistence.journal.inmem.InmemJournal
import akka.testkit.{ EventFilter, ImplicitSender, TestEvent }
-import scala.concurrent.Future
-import scala.concurrent.duration._
-import scala.util.control.NoStackTrace
-
object EventSourcedActorDeleteFailureSpec {
case class DeleteTo(n: Long)
diff --git a/akka-persistence/src/test/scala/akka/persistence/EventSourcedActorFailureSpec.scala b/akka-persistence/src/test/scala/akka/persistence/EventSourcedActorFailureSpec.scala
index d81fc9ecda..0712f9af4c 100644
--- a/akka-persistence/src/test/scala/akka/persistence/EventSourcedActorFailureSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/EventSourcedActorFailureSpec.scala
@@ -4,17 +4,16 @@
package akka.persistence
+import scala.collection.immutable
+import scala.concurrent.Future
+import scala.util.{ Failure, Try }
+import scala.util.control.NoStackTrace
+
import akka.actor.{ OneForOneStrategy, _ }
import akka.persistence.journal.AsyncWriteJournal
import akka.persistence.journal.inmem.InmemJournal
import akka.testkit.{ EventFilter, ImplicitSender, TestEvent, TestProbe }
-import scala.collection.immutable
-import scala.util.control.NoStackTrace
-import scala.util.{ Failure, Try }
-
-import scala.concurrent.Future
-
object EventSourcedActorFailureSpec {
import PersistentActorSpec.{ Cmd, Evt, ExamplePersistentActor }
diff --git a/akka-persistence/src/test/scala/akka/persistence/LoadPluginSpec.scala b/akka-persistence/src/test/scala/akka/persistence/LoadPluginSpec.scala
index 99e66ad8c3..ecca8b476a 100644
--- a/akka-persistence/src/test/scala/akka/persistence/LoadPluginSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/LoadPluginSpec.scala
@@ -4,10 +4,11 @@
package akka.persistence
-import akka.persistence.journal.inmem.InmemJournal
import com.typesafe.config.Config
-import akka.testkit.ImplicitSender
+
import akka.actor.Actor
+import akka.persistence.journal.inmem.InmemJournal
+import akka.testkit.ImplicitSender
import akka.util.unused
object LoadPluginSpec {
diff --git a/akka-persistence/src/test/scala/akka/persistence/ManyRecoveriesSpec.scala b/akka-persistence/src/test/scala/akka/persistence/ManyRecoveriesSpec.scala
index 6c4b73cf14..d834ca7e7f 100644
--- a/akka-persistence/src/test/scala/akka/persistence/ManyRecoveriesSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/ManyRecoveriesSpec.scala
@@ -4,11 +4,13 @@
package akka.persistence
+import scala.concurrent.Await
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-import scala.concurrent.Await
object ManyRecoveriesSpec {
diff --git a/akka-persistence/src/test/scala/akka/persistence/OptimizedRecoverySpec.scala b/akka-persistence/src/test/scala/akka/persistence/OptimizedRecoverySpec.scala
index 1a7d0dc29d..a544427444 100644
--- a/akka-persistence/src/test/scala/akka/persistence/OptimizedRecoverySpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/OptimizedRecoverySpec.scala
@@ -14,7 +14,7 @@ object OptimizedRecoverySpec {
case object TakeSnapshot
final case class Save(s: String)
final case class Saved(s: String, seqNr: Long)
- final case object PersistFromRecoveryCompleted
+ case object PersistFromRecoveryCompleted
def props(name: String, recovery: Recovery, probe: ActorRef): Props = {
Props(new TestPersistentActor(name, recovery, probe))
diff --git a/akka-persistence/src/test/scala/akka/persistence/OptionalSnapshotStoreSpec.scala b/akka-persistence/src/test/scala/akka/persistence/OptionalSnapshotStoreSpec.scala
index 2a9e67a021..b5fd531491 100644
--- a/akka-persistence/src/test/scala/akka/persistence/OptionalSnapshotStoreSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/OptionalSnapshotStoreSpec.scala
@@ -4,11 +4,12 @@
package akka.persistence
+import com.typesafe.config.ConfigFactory
+
import akka.actor.{ Actor, Props }
import akka.event.Logging
import akka.event.Logging.Warning
import akka.testkit.{ EventFilter, ImplicitSender, TestEvent }
-import com.typesafe.config.ConfigFactory
object OptionalSnapshotStoreSpec {
diff --git a/akka-persistence/src/test/scala/akka/persistence/PersistenceSpec.scala b/akka-persistence/src/test/scala/akka/persistence/PersistenceSpec.scala
index 35e9082b75..4acf2339e3 100644
--- a/akka-persistence/src/test/scala/akka/persistence/PersistenceSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/PersistenceSpec.scala
@@ -7,16 +7,14 @@ package akka.persistence
import java.io.File
import java.util.concurrent.atomic.AtomicInteger
-import org.scalatest.matchers.{ MatchResult, Matcher }
-
import scala.collection.immutable
import scala.reflect.ClassTag
import scala.util.control.NoStackTrace
import com.typesafe.config.{ Config, ConfigFactory }
-
import org.apache.commons.io.FileUtils
import org.scalatest.BeforeAndAfterEach
+import org.scalatest.matchers.{ MatchResult, Matcher }
import akka.actor.Props
import akka.testkit.AkkaSpec
diff --git a/akka-persistence/src/test/scala/akka/persistence/PersistentActorBoundedStashingSpec.scala b/akka-persistence/src/test/scala/akka/persistence/PersistentActorBoundedStashingSpec.scala
index fd01ed848c..8f2cc14948 100644
--- a/akka-persistence/src/test/scala/akka/persistence/PersistentActorBoundedStashingSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/PersistentActorBoundedStashingSpec.scala
@@ -4,16 +4,17 @@
package akka.persistence
-import akka.actor.DeadLetter
-import akka.persistence.PersistentActorBoundedStashingSpec._
-import akka.persistence.journal.SteppingInmemJournal
-import akka.testkit.TestEvent.Mute
-import akka.testkit.EventFilter
-import akka.testkit.ImplicitSender
+import scala.concurrent.duration._
+
import com.typesafe.config.Config
import org.scalatest.BeforeAndAfterEach
-import scala.concurrent.duration._
+import akka.actor.DeadLetter
+import akka.persistence.PersistentActorBoundedStashingSpec._
+import akka.persistence.journal.SteppingInmemJournal
+import akka.testkit.EventFilter
+import akka.testkit.ImplicitSender
+import akka.testkit.TestEvent.Mute
object PersistentActorBoundedStashingSpec {
final case class Cmd(data: Any)
diff --git a/akka-persistence/src/test/scala/akka/persistence/PersistentActorJournalProtocolSpec.scala b/akka-persistence/src/test/scala/akka/persistence/PersistentActorJournalProtocolSpec.scala
index 939416522f..e14a74bd7c 100644
--- a/akka-persistence/src/test/scala/akka/persistence/PersistentActorJournalProtocolSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/PersistentActorJournalProtocolSpec.scala
@@ -4,11 +4,13 @@
package akka.persistence
-import akka.actor._
-import akka.testkit._
import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+
+import akka.actor._
import akka.persistence.JournalProtocol._
+import akka.testkit._
object PersistentActorJournalProtocolSpec {
diff --git a/akka-persistence/src/test/scala/akka/persistence/PersistentActorRecoveryTimeoutSpec.scala b/akka-persistence/src/test/scala/akka/persistence/PersistentActorRecoveryTimeoutSpec.scala
index 3672c25fec..d8b8699a16 100644
--- a/akka-persistence/src/test/scala/akka/persistence/PersistentActorRecoveryTimeoutSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/PersistentActorRecoveryTimeoutSpec.scala
@@ -4,13 +4,14 @@
package akka.persistence
-import akka.actor.Status.Failure
-import akka.actor.{ Actor, ActorLogging, ActorRef, Props }
-import akka.persistence.journal.SteppingInmemJournal
-import akka.testkit.{ AkkaSpec, ImplicitSender, TestProbe }
+import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
-import scala.concurrent.duration._
+import akka.actor.{ Actor, ActorLogging, ActorRef, Props }
+import akka.actor.Status.Failure
+import akka.persistence.journal.SteppingInmemJournal
+import akka.testkit.{ AkkaSpec, ImplicitSender, TestProbe }
object PersistentActorRecoveryTimeoutSpec {
val journalId = "persistent-actor-recovery-timeout-spec"
diff --git a/akka-persistence/src/test/scala/akka/persistence/PersistentActorSpec.scala b/akka-persistence/src/test/scala/akka/persistence/PersistentActorSpec.scala
index 1f4feb50c4..fdf692f668 100644
--- a/akka-persistence/src/test/scala/akka/persistence/PersistentActorSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/PersistentActorSpec.scala
@@ -6,18 +6,19 @@ package akka.persistence
import java.util.concurrent.atomic.AtomicInteger
-import akka.actor._
-import akka.persistence.PersistentActorSpec._
-import akka.testkit.{ EventFilter, ImplicitSender, TestLatch, TestProbe }
-import com.github.ghik.silencer.silent
-import com.typesafe.config.{ Config, ConfigFactory }
-
import scala.collection.immutable.Seq
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.Random
import scala.util.control.NoStackTrace
+import com.github.ghik.silencer.silent
+import com.typesafe.config.{ Config, ConfigFactory }
+
+import akka.actor._
+import akka.persistence.PersistentActorSpec._
+import akka.testkit.{ EventFilter, ImplicitSender, TestLatch, TestProbe }
+
object PersistentActorSpec {
final case class Cmd(data: Any)
diff --git a/akka-persistence/src/test/scala/akka/persistence/PersistentActorStashingSpec.scala b/akka-persistence/src/test/scala/akka/persistence/PersistentActorStashingSpec.scala
index 3ee8544a64..3f12c57af3 100644
--- a/akka-persistence/src/test/scala/akka/persistence/PersistentActorStashingSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/PersistentActorStashingSpec.scala
@@ -4,15 +4,16 @@
package akka.persistence
-import akka.actor.SupervisorStrategy.Resume
+import scala.concurrent.duration._
+import scala.reflect.ClassTag
+
+import com.typesafe.config.Config
+
import akka.actor.{ Actor, ActorRef, OneForOneStrategy, Props }
+import akka.actor.SupervisorStrategy.Resume
import akka.persistence.journal.SteppingInmemJournal
import akka.testkit.ImplicitSender
import akka.util.unused
-import com.typesafe.config.Config
-
-import scala.concurrent.duration._
-import scala.reflect.ClassTag
object PersistentActorStashingSpec {
final case class Cmd(data: Any)
diff --git a/akka-persistence/src/test/scala/akka/persistence/RecoveryPermitterSpec.scala b/akka-persistence/src/test/scala/akka/persistence/RecoveryPermitterSpec.scala
index c47d6b647e..47e3d71bcc 100644
--- a/akka-persistence/src/test/scala/akka/persistence/RecoveryPermitterSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/RecoveryPermitterSpec.scala
@@ -7,11 +7,12 @@ package akka.persistence
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.testkit.{ EventFilter, ImplicitSender, TestEvent }
-import com.typesafe.config.ConfigFactory
-import akka.testkit.TestProbe
import akka.testkit.TestActors
+import akka.testkit.TestProbe
object RecoveryPermitterSpec {
@@ -48,8 +49,8 @@ class RecoveryPermitterSpec extends PersistenceSpec(ConfigFactory.parseString(s"
akka.persistence.journal.plugin = "akka.persistence.journal.inmem"
akka.actor.warn-about-java-serializer-usage = off
""")) with ImplicitSender {
- import RecoveryPermitterSpec._
import RecoveryPermitter._
+ import RecoveryPermitterSpec._
system.eventStream.publish(TestEvent.Mute(EventFilter[TestExc]()))
diff --git a/akka-persistence/src/test/scala/akka/persistence/SnapshotFailureRobustnessSpec.scala b/akka-persistence/src/test/scala/akka/persistence/SnapshotFailureRobustnessSpec.scala
index 727cf0a470..e7123d6d2b 100644
--- a/akka-persistence/src/test/scala/akka/persistence/SnapshotFailureRobustnessSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/SnapshotFailureRobustnessSpec.scala
@@ -6,15 +6,16 @@ package akka.persistence
import java.io.IOException
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+import com.typesafe.config.Config
+
import akka.actor.{ ActorRef, Props }
import akka.event.Logging
import akka.persistence.snapshot.local.LocalSnapshotStore
import akka.testkit.{ EventFilter, ImplicitSender, TestEvent }
-import com.typesafe.config.Config
-
-import scala.concurrent.Future
-import scala.concurrent.duration._
-import scala.language.postfixOps
object SnapshotFailureRobustnessSpec {
diff --git a/akka-persistence/src/test/scala/akka/persistence/SnapshotRecoveryWithEmptyJournalSpec.scala b/akka-persistence/src/test/scala/akka/persistence/SnapshotRecoveryWithEmptyJournalSpec.scala
index 85264b4839..e0e628f0da 100644
--- a/akka-persistence/src/test/scala/akka/persistence/SnapshotRecoveryWithEmptyJournalSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/SnapshotRecoveryWithEmptyJournalSpec.scala
@@ -6,11 +6,12 @@ package akka.persistence
import java.io.File
+import org.apache.commons.io.FileUtils
+
import akka.actor._
import akka.persistence.serialization.Snapshot
import akka.serialization.{ Serialization, SerializationExtension }
import akka.testkit._
-import org.apache.commons.io.FileUtils
object SnapshotRecoveryWithEmptyJournalSpec {
val survivingSnapshotPath = "target/survivingSnapshotPath"
diff --git a/akka-persistence/src/test/scala/akka/persistence/SnapshotSerializationSpec.scala b/akka-persistence/src/test/scala/akka/persistence/SnapshotSerializationSpec.scala
index 54d2c7d30e..8fcda64722 100644
--- a/akka-persistence/src/test/scala/akka/persistence/SnapshotSerializationSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/SnapshotSerializationSpec.scala
@@ -4,10 +4,11 @@
package akka.persistence
+import java.io._
+
import akka.actor.{ ActorRef, Props }
import akka.serialization.Serializer
-import akka.testkit.{ ImplicitSender }
-import java.io._
+import akka.testkit.ImplicitSender
object SnapshotSerializationSpec {
trait SerializationMarker
diff --git a/akka-persistence/src/test/scala/akka/persistence/SnapshotSpec.scala b/akka-persistence/src/test/scala/akka/persistence/SnapshotSpec.scala
index 7ef097537c..a4702b3eb9 100644
--- a/akka-persistence/src/test/scala/akka/persistence/SnapshotSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/SnapshotSpec.scala
@@ -84,8 +84,8 @@ object SnapshotSpec {
}
class SnapshotSpec extends PersistenceSpec(PersistenceSpec.config("leveldb", "SnapshotSpec")) with ImplicitSender {
- import SnapshotSpec._
import SnapshotProtocol._
+ import SnapshotSpec._
override protected def beforeEach(): Unit = {
super.beforeEach()
diff --git a/akka-persistence/src/test/scala/akka/persistence/TimerPersistentActorSpec.scala b/akka-persistence/src/test/scala/akka/persistence/TimerPersistentActorSpec.scala
index ba97086936..fad5a7b62d 100644
--- a/akka-persistence/src/test/scala/akka/persistence/TimerPersistentActorSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/TimerPersistentActorSpec.scala
@@ -8,10 +8,11 @@ import scala.concurrent.duration._
import scala.runtime.BoxedUnit
import scala.runtime.BoxedUnit
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.japi.Procedure
import akka.testkit.{ EventFilter, ImplicitSender }
-import com.typesafe.config.ConfigFactory
import akka.testkit.TestEvent.Mute
object TimerPersistentActorSpec {
@@ -101,7 +102,7 @@ class TimerPersistentActorSpec extends PersistenceSpec(ConfigFactory.parseString
}
"reject wrong order of traits, PersistentActor with Timer" in {
- val pa = system.actorOf(Props[WrongOrder])
+ val pa = system.actorOf(Props[WrongOrder]())
watch(pa)
expectTerminated(pa)
}
diff --git a/akka-persistence/src/test/scala/akka/persistence/fsm/PersistentFSMSpec.scala b/akka-persistence/src/test/scala/akka/persistence/fsm/PersistentFSMSpec.scala
index c03e1ce2f7..b9e91051ed 100644
--- a/akka-persistence/src/test/scala/akka/persistence/fsm/PersistentFSMSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/fsm/PersistentFSMSpec.scala
@@ -6,17 +6,18 @@ package akka.persistence.fsm
import java.io.File
-import akka.actor.{ ActorSystem, _ }
-import akka.persistence._
-import akka.persistence.fsm.PersistentFSM._
-import akka.testkit._
+import scala.concurrent.duration._
+import scala.language.postfixOps
+import scala.reflect.ClassTag
+
import com.github.ghik.silencer.silent
import com.typesafe.config.{ Config, ConfigFactory }
import org.apache.commons.io.FileUtils
-import scala.concurrent.duration._
-import scala.language.postfixOps
-import scala.reflect.ClassTag
+import akka.actor.{ ActorSystem, _ }
+import akka.persistence._
+import akka.persistence.fsm.PersistentFSM._
+import akka.testkit._
@silent("deprecated")
abstract class PersistentFSMSpec(config: Config) extends PersistenceSpec(config) with ImplicitSender {
@@ -464,7 +465,7 @@ object PersistentFSMSpec {
startWith(LookingAround, EmptyShoppingCart)
when(LookingAround) {
- case Event("stay", _) => stay
+ case Event("stay", _) => stay()
case Event(_, _) => goto(LookingAround)
}
@@ -493,12 +494,12 @@ object PersistentFSMSpec {
case Event(AddItem(item), _) =>
goto(Shopping).applying(ItemAdded(item)).forMax(1 seconds)
case Event(GetCurrentCart, data) =>
- stay.replying(data)
+ stay().replying(data)
}
when(Shopping) {
case Event(AddItem(item), _) =>
- stay.applying(ItemAdded(item)).forMax(1 seconds)
+ stay().applying(ItemAdded(item)).forMax(1 seconds)
case Event(Buy, _) =>
//#customer-andthen-example
goto(Paid).applying(OrderExecuted).andThen {
@@ -512,14 +513,14 @@ object PersistentFSMSpec {
//#customer-andthen-example
case Event(Leave, _) =>
//#customer-snapshot-example
- stop.applying(OrderDiscarded).andThen {
+ stop().applying(OrderDiscarded).andThen {
case _ =>
reportActor ! ShoppingCardDiscarded
saveStateSnapshot()
}
//#customer-snapshot-example
case Event(GetCurrentCart, data) =>
- stay.replying(data)
+ stay().replying(data)
case Event(StateTimeout, _) =>
goto(Inactive).forMax(2 seconds)
}
@@ -528,7 +529,7 @@ object PersistentFSMSpec {
case Event(AddItem(item), _) =>
goto(Shopping).applying(ItemAdded(item)).forMax(1 seconds)
case Event(StateTimeout, _) =>
- stop.applying(OrderDiscarded).andThen {
+ stop().applying(OrderDiscarded).andThen {
case _ => reportActor ! ShoppingCardDiscarded
}
}
@@ -536,7 +537,7 @@ object PersistentFSMSpec {
when(Paid) {
case Event(Leave, _) => stop()
case Event(GetCurrentCart, data) =>
- stay.replying(data)
+ stay().replying(data)
}
//#customer-fsm-body
@@ -633,7 +634,7 @@ object PersistentFSMSpec {
when(PersistSingleAtOnce) {
case Event(i: Int, _) =>
- stay.applying(IntAdded(i))
+ stay().applying(IntAdded(i))
case Event("4x", _) =>
goto(Persist4xAtOnce)
case Event(SaveSnapshotSuccess(metadata), _) =>
@@ -643,7 +644,7 @@ object PersistentFSMSpec {
when(Persist4xAtOnce) {
case Event(i: Int, _) =>
- stay.applying(IntAdded(i), IntAdded(i), IntAdded(i), IntAdded(i))
+ stay().applying(IntAdded(i), IntAdded(i), IntAdded(i), IntAdded(i))
case Event(SaveSnapshotSuccess(metadata), _) =>
probe ! s"SeqNo=${metadata.sequenceNr}, StateData=${stateData}"
stay()
diff --git a/akka-persistence/src/test/scala/akka/persistence/journal/InmemEventAdaptersSpec.scala b/akka-persistence/src/test/scala/akka/persistence/journal/InmemEventAdaptersSpec.scala
index 9320f24265..3f00e957ad 100644
--- a/akka-persistence/src/test/scala/akka/persistence/journal/InmemEventAdaptersSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/journal/InmemEventAdaptersSpec.scala
@@ -4,9 +4,10 @@
package akka.persistence.journal
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ExtendedActorSystem
import akka.testkit.AkkaSpec
-import com.typesafe.config.ConfigFactory
class InmemEventAdaptersSpec extends AkkaSpec {
diff --git a/akka-persistence/src/test/scala/akka/persistence/journal/ReplayFilterSpec.scala b/akka-persistence/src/test/scala/akka/persistence/journal/ReplayFilterSpec.scala
index 94b9597df1..55df93ab2f 100644
--- a/akka-persistence/src/test/scala/akka/persistence/journal/ReplayFilterSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/journal/ReplayFilterSpec.scala
@@ -5,9 +5,9 @@
package akka.persistence.journal
import akka.actor._
-import akka.testkit._
import akka.persistence.JournalProtocol
import akka.persistence.PersistentRepr
+import akka.testkit._
class ReplayFilterSpec extends AkkaSpec with ImplicitSender {
import JournalProtocol._
diff --git a/akka-persistence/src/test/scala/akka/persistence/journal/SteppingInmemJournal.scala b/akka-persistence/src/test/scala/akka/persistence/journal/SteppingInmemJournal.scala
index 6052058a38..e9fb2b5359 100644
--- a/akka-persistence/src/test/scala/akka/persistence/journal/SteppingInmemJournal.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/journal/SteppingInmemJournal.scala
@@ -4,17 +4,19 @@
package akka.persistence.journal
+import scala.collection.immutable.Seq
+import scala.concurrent.{ Await, Future, Promise }
+import scala.concurrent.duration._
+import scala.util.Try
+
+import com.typesafe.config.{ Config, ConfigFactory }
+
import akka.actor.{ ActorRef, ActorSystem }
import akka.pattern.ask
-import akka.persistence.journal.inmem.InmemJournal
import akka.persistence.{ AtomicWrite, PersistentRepr }
-import akka.util.Timeout
+import akka.persistence.journal.inmem.InmemJournal
import akka.testkit._
-import com.typesafe.config.{ Config, ConfigFactory }
-import scala.collection.immutable.Seq
-import scala.concurrent.duration._
-import scala.concurrent.{ Await, Future, Promise }
-import scala.util.Try
+import akka.util.Timeout
object SteppingInmemJournal {
diff --git a/akka-persistence/src/test/scala/akka/persistence/journal/chaos/ChaosJournal.scala b/akka-persistence/src/test/scala/akka/persistence/journal/chaos/ChaosJournal.scala
index 62744b8209..1381222971 100644
--- a/akka-persistence/src/test/scala/akka/persistence/journal/chaos/ChaosJournal.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/journal/chaos/ChaosJournal.scala
@@ -4,14 +4,16 @@
package akka.persistence.journal.chaos
+import java.util.concurrent.ThreadLocalRandom
+
import scala.collection.immutable
import scala.concurrent.Future
-import java.util.concurrent.ThreadLocalRandom
+import scala.util.Try
+import scala.util.control.NonFatal
+
import akka.persistence._
import akka.persistence.journal.AsyncWriteJournal
import akka.persistence.journal.inmem.InmemMessages
-import scala.util.Try
-import scala.util.control.NonFatal
class WriteFailedException(ps: Seq[PersistentRepr])
extends TestException(s"write failed for payloads = [${ps.map(_.payload)}]")
diff --git a/akka-persistence/src/test/scala/akka/persistence/journal/leveldb/JournalCompactionSpec.scala b/akka-persistence/src/test/scala/akka/persistence/journal/leveldb/JournalCompactionSpec.scala
index 9112c95563..bfaa5ab890 100644
--- a/akka-persistence/src/test/scala/akka/persistence/journal/leveldb/JournalCompactionSpec.scala
+++ b/akka-persistence/src/test/scala/akka/persistence/journal/leveldb/JournalCompactionSpec.scala
@@ -6,15 +6,16 @@ package akka.persistence.journal.leveldb
import java.io.File
-import akka.actor.{ ActorLogging, ActorRef, ActorSystem, Props }
-import akka.persistence.journal.leveldb.JournalCompactionSpec.EventLogger._
-import akka.persistence.journal.leveldb.JournalCompactionSpec.SpecComponentBuilder
-import akka.persistence.{ DeleteMessagesSuccess, PersistenceSpec, PersistentActor }
-import akka.testkit.TestProbe
+import scala.util.Random
+
import com.typesafe.config.Config
import org.apache.commons.io.FileUtils
-import scala.util.Random
+import akka.actor.{ ActorLogging, ActorRef, ActorSystem, Props }
+import akka.persistence.{ DeleteMessagesSuccess, PersistenceSpec, PersistentActor }
+import akka.persistence.journal.leveldb.JournalCompactionSpec.EventLogger._
+import akka.persistence.journal.leveldb.JournalCompactionSpec.SpecComponentBuilder
+import akka.testkit.TestProbe
class JournalNoCompactionSpec
extends JournalCompactionSpecBase(SpecComponentBuilder("leveldb-JournalNoCompactionSpec")) {
diff --git a/akka-pki/src/main/scala/akka/pki/pem/DERPrivateKeyLoader.scala b/akka-pki/src/main/scala/akka/pki/pem/DERPrivateKeyLoader.scala
new file mode 100644
index 0000000000..68f3f58b3e
--- /dev/null
+++ b/akka-pki/src/main/scala/akka/pki/pem/DERPrivateKeyLoader.scala
@@ -0,0 +1,130 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.pki.pem
+
+import java.math.BigInteger
+import java.security.KeyFactory
+import java.security.PrivateKey
+import java.security.spec.PKCS8EncodedKeySpec
+import java.security.spec.RSAMultiPrimePrivateCrtKeySpec
+import java.security.spec.RSAOtherPrimeInfo
+import java.security.spec.RSAPrivateCrtKeySpec
+
+import akka.annotation.ApiMayChange
+import akka.pki.pem.PEMDecoder.DERData
+import com.hierynomus.asn1.ASN1InputStream
+import com.hierynomus.asn1.encodingrules.der.DERDecoder
+import com.hierynomus.asn1.types.constructed.ASN1Sequence
+import com.hierynomus.asn1.types.primitive.ASN1Integer
+
+final class PEMLoadingException(message: String, cause: Throwable) extends RuntimeException(message, cause) {
+ def this(msg: String) = this(msg, null)
+}
+
+object DERPrivateKeyLoader {
+
+ /**
+ * Converts the DER payload in [[PEMDecoder.DERData]] into a [[java.security.PrivateKey]]. The received DER
+ * data must be a valid PKCS#1 (identified in PEM as "RSA PRIVATE KEY") or non-ecnrypted PKCS#8 (identified
+ * in PEM as "PRIVATE KEY").
+ * @throws PEMLoadingException when the `derData` is for an unsupported format
+ */
+ @ApiMayChange
+ @throws[PEMLoadingException]("when the `derData` is for an unsupported format")
+ def load(derData: DERData): PrivateKey = {
+ derData.label match {
+ case "RSA PRIVATE KEY" =>
+ loadPkcs1PrivateKey(derData.bytes)
+ case "PRIVATE KEY" =>
+ loadPkcs8PrivateKey(derData.bytes)
+ case unknown =>
+ throw new PEMLoadingException(s"Don't know how to read a private key from PEM data with label [$unknown]")
+ }
+ }
+
+ private def loadPkcs1PrivateKey(bytes: Array[Byte]) = {
+ val derInputStream = new ASN1InputStream(new DERDecoder, bytes)
+ // Here's the specification: https://tools.ietf.org/html/rfc3447#appendix-A.1.2
+ val sequence = {
+ try {
+ derInputStream.readObject[ASN1Sequence]()
+ } finally {
+ derInputStream.close()
+ }
+ }
+ val version = getInteger(sequence, 0, "version").intValueExact()
+ if (version < 0 || version > 1) {
+ throw new IllegalArgumentException(s"Unsupported PKCS1 version: $version")
+ }
+ val modulus = getInteger(sequence, 1, "modulus")
+ val publicExponent = getInteger(sequence, 2, "publicExponent")
+ val privateExponent = getInteger(sequence, 3, "privateExponent")
+ val prime1 = getInteger(sequence, 4, "prime1")
+ val prime2 = getInteger(sequence, 5, "prime2")
+ val exponent1 = getInteger(sequence, 6, "exponent1")
+ val exponent2 = getInteger(sequence, 7, "exponent2")
+ val coefficient = getInteger(sequence, 8, "coefficient")
+
+ val keySpec = if (version == 0) {
+ new RSAPrivateCrtKeySpec(
+ modulus,
+ publicExponent,
+ privateExponent,
+ prime1,
+ prime2,
+ exponent1,
+ exponent2,
+ coefficient)
+ } else {
+ // Does anyone even use multi-primes? Who knows, maybe this code will never be used. Anyway, I guess it will work,
+ // the spec isn't exactly complicated.
+ val otherPrimeInfosSequence = getSequence(sequence, 9, "otherPrimeInfos")
+ val otherPrimeInfos = (for (i <- 0 until otherPrimeInfosSequence.size()) yield {
+ val name = s"otherPrimeInfos[$i]"
+ val seq = getSequence(otherPrimeInfosSequence, i, name)
+ val prime = getInteger(seq, 0, s"$name.prime")
+ val exponent = getInteger(seq, 1, s"$name.exponent")
+ val coefficient = getInteger(seq, 2, s"$name.coefficient")
+ new RSAOtherPrimeInfo(prime, exponent, coefficient)
+ }).toArray
+ new RSAMultiPrimePrivateCrtKeySpec(
+ modulus,
+ publicExponent,
+ privateExponent,
+ prime1,
+ prime2,
+ exponent1,
+ exponent2,
+ coefficient,
+ otherPrimeInfos)
+ }
+
+ val keyFactory = KeyFactory.getInstance("RSA")
+ keyFactory.generatePrivate(keySpec)
+ }
+
+ private def getInteger(sequence: ASN1Sequence, index: Int, name: String): BigInteger = {
+ sequence.get(index) match {
+ case integer: ASN1Integer => integer.getValue
+ case other =>
+ throw new IllegalArgumentException(s"Expected integer tag for $name at index $index, but got: ${other.getTag}")
+ }
+ }
+
+ private def getSequence(sequence: ASN1Sequence, index: Int, name: String): ASN1Sequence = {
+ sequence.get(index) match {
+ case seq: ASN1Sequence => seq
+ case other =>
+ throw new IllegalArgumentException(s"Expected sequence tag for $name at index $index, but got: ${other.getTag}")
+ }
+ }
+
+ private def loadPkcs8PrivateKey(bytes: Array[Byte]) = {
+ val keySpec = new PKCS8EncodedKeySpec(bytes)
+ val keyFactory = KeyFactory.getInstance("RSA")
+ keyFactory.generatePrivate(keySpec)
+ }
+
+}
diff --git a/akka-pki/src/main/scala/akka/pki/pem/PEMDecoder.scala b/akka-pki/src/main/scala/akka/pki/pem/PEMDecoder.scala
new file mode 100644
index 0000000000..1f6023a9ed
--- /dev/null
+++ b/akka-pki/src/main/scala/akka/pki/pem/PEMDecoder.scala
@@ -0,0 +1,76 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.pki.pem
+
+import java.util.Base64
+
+import akka.annotation.ApiMayChange
+
+/**
+ * Decodes lax PEM encoded data, according to
+ *
+ * https://tools.ietf.org/html/rfc7468
+ */
+object PEMDecoder {
+
+ // I believe this regex matches the RFC7468 Lax ABNF semantics jkhdft exactly.
+ private val PEMRegex = {
+ // Luckily, Java Pattern's \s matches the RFCs W ABNF expression perfectly
+ // (space, tab, carriage return, line feed, form feed, vertical tab)
+
+ // The variables here are named to match the expressions in the RFC7468 ABNF
+ // description. The content of the regex may not match the structure of the
+ // expression because sometimes there are nicer way to do things in regexes.
+
+ // All printable ASCII characters minus hyphen
+ val labelchar = """[\p{Print}&&[^-]]"""
+ // Starts and finishes with a labelchar, with as many label chars and hyphens or
+ // spaces in between, but no double spaces or hyphens, also may be empty.
+ val label = raw"""(?:$labelchar(?:[\- ]?$labelchar)*)?"""
+ // capturing group so we can extract the label
+ val preeb = raw"""-----BEGIN ($label)-----"""
+ // we don't extract the end label because the RFC says we can ignore it (it
+ // doesn't have to match the begin label)
+ val posteb = raw"""-----END $label-----"""
+ // Any of the base64 chars (alphanum, +, /) and whitespace, followed by at most 2
+ // padding characters, separated by zero to many whitespace characters
+ val laxbase64text = """[A-Za-z0-9\+/\s]*(?:=\s*){0,2}"""
+
+ val laxtextualmessage = raw"""\s*$preeb($laxbase64text)$posteb\s*"""
+
+ laxtextualmessage.r
+ }
+
+ /**
+ * Decodes a PEM String into an identifier and the DER bytes of the content.
+ *
+ * See https://tools.ietf.org/html/rfc7468 and https://en.wikipedia.org/wiki/Privacy-Enhanced_Mail
+ *
+ * @param pemData the PEM data (pre-eb, base64-MIME data and ponst-eb)
+ * @return the decoded bytes and the content type.
+ */
+ @throws[PEMLoadingException](
+ "If the `pemData` is not valid PEM format (according to https://tools.ietf.org/html/rfc7468).")
+ @ApiMayChange
+ def decode(pemData: String): DERData = {
+ pemData match {
+ case PEMRegex(label, base64) =>
+ try {
+ new DERData(label, Base64.getMimeDecoder.decode(base64))
+ } catch {
+ case iae: IllegalArgumentException =>
+ throw new PEMLoadingException(
+ s"Error decoding base64 data from PEM data (note: expected MIME-formatted Base64)",
+ iae)
+ }
+
+ case _ => throw new PEMLoadingException("Not a PEM encoded data.")
+ }
+ }
+
+ @ApiMayChange
+ final class DERData(val label: String, val bytes: Array[Byte])
+
+}
diff --git a/akka-pki/src/test/resources/certificate.pem b/akka-pki/src/test/resources/certificate.pem
new file mode 100644
index 0000000000..4fb0ea7b86
--- /dev/null
+++ b/akka-pki/src/test/resources/certificate.pem
@@ -0,0 +1,19 @@
+-----BEGIN CERTIFICATE-----
+MIIDCzCCAfOgAwIBAgIQfEHPfR1p1xuW9TQlfxAugjANBgkqhkiG9w0BAQsFADAv
+MS0wKwYDVQQDEyQwZDIwN2I2OC05YTIwLTRlZTgtOTJjYi1iZjk2OTk1ODFjZjgw
+HhcNMTkxMDExMTMyODUzWhcNMjQxMDA5MTQyODUzWjAvMS0wKwYDVQQDEyQwZDIw
+N2I2OC05YTIwLTRlZTgtOTJjYi1iZjk2OTk1ODFjZjgwggEiMA0GCSqGSIb3DQEB
+AQUAA4IBDwAwggEKAoIBAQDhD0BxlDzEOzcp45lPHL60lnM6k3puEGb2lKHL5/nR
+F94FCnZL0FH8EdxWzzAYgys+kUwSdo4QMuWuvjY2Km4Wob6k4uAeYEFTCfBdi4/z
+r4kpWzu8xLz+uZWimLQrjqVytNNK3DMv6ebWUJ/92VTDS4yzWk4YV0MVr2b2OgMK
+SgMvaFQ8L/CwyML72PBWIqU67+MMvvcTLxQdyEgQTTjP0bbiXMLDvfZDarLJojsW
+SNBz7AIkznhGkzIGGdhAa41PnPu9XaBFhaqx9Qe3+MG2/k1l/46eHtmxCqhOUde1
+i0vy6ZfgcGifua1tg1UBI/oT4S0dsq24dq7K1MYLyHTrAgMBAAGjIzAhMA4GA1Ud
+DwEB/wQEAwICBDAPBgNVHRMBAf8EBTADAQH/MA0GCSqGSIb3DQEBCwUAA4IBAQAa
+5YOlvob4wqps3HLaOIi7VzDihNHciP+BI0mzxHa7D3bGaecRPSeG3xEoD/Uxs9o4
+8cByPpYx1Wl8LLRx14wDcK0H+UPpo4gCI6h6q92cJj0YRjcSUUt8EIu3qnFMjtM+
+sl/uc21fGlq6cvMRZXqtVYENoSNTDHi5a5eEXRa2eZ8XntjvOanOhIKWmxvr8r4a
+Voz4WdnXx1C8/BzB62UBoMu4QqMGMLk5wXP0D6hECUuespMest+BeoJAVhTq7wZs
+rSP9q08n1stZFF4+bEBaxcqIqdhOLQdHcYELN+a5v0Mcwdsy7jJMagmNPfsKoOKC
+hLOsmNYKHdmWg37Jib5o
+-----END CERTIFICATE-----
\ No newline at end of file
diff --git a/akka-pki/src/test/resources/multi-prime-pkcs1.pem b/akka-pki/src/test/resources/multi-prime-pkcs1.pem
new file mode 100644
index 0000000000..2ad888e5b6
--- /dev/null
+++ b/akka-pki/src/test/resources/multi-prime-pkcs1.pem
@@ -0,0 +1,28 @@
+-----BEGIN RSA PRIVATE KEY-----
+MIIE2AIBAQKCAQEAqFD3rwpvkqgkxSCIKyO2M///6CnJz/YqYycSxeUXJSeC+sf0
+Svu4mzSNyx9mglH6ubVJ1x01XlK7GDCAOuqi7Dgay23m+qRq+MF89gYZY9YuVPBr
+jsFPK3XguIOLTIV2VCdskBLbW0G6b6VVDOVLCffD6fRuy/H43BI2l9+nuTAdYpCE
+sX7IliRi2HR09nv1THSMrdrcq48EdLK+Qzj3f+9gedqeJP7ABY29eIBAOasUY356
+So/2dqwhfmMbXswHqIFVQIBd+y1F4Gwf4HxTN06bHhs5rOg4fAREmr1SU25CnhDA
+SBw8c5zRFNGgekkzIxSZLFw27ezYGxZyIjnKawIDAQABAoIBAFrcKniRV52BqyfG
+4fr3sjnr7gcz17+tkUApLZcqjg3+gFREcHmx3PvbqNeHwdyDyKdLV+sJ129tlZX/
+SJmFZCHEP6KlV1TiQOS7/msI69fbHPO5PTaCjTzkbA4WOAgv+M4XjR82RM1EusO1
+tPegWfLhj5dvdAfgTpodW1XDFs3QHoQbKkLLMREOb3j+LuK38npxN1UmtDNSRlII
+xv77KywOij0LMG4CjIeXmjGdL9BWzlbHv5Zk0wuWHtGFvkoQO6EPn3NVb1uY1ZdX
+IekdvA71qssliVFi30/3ZiFTwt3fXfNBawN7t1s0pJlsPAOz7iiucHWrUCvtaXD+
+miFFN4ECVgez3BQqe6o3lmHG7LQKy/RWBfDXLCc1JziipTeu+e1piNd7WnJcpsbl
+31kTA3JBp7VU41EEXEBtWz8cW3I9y6AhukAyeWBnww9FSet8bKGAuYj1siCXAlYH
+L+pEeS8NuGO1iigEEZzZrOMC3Yg9/evwhjXxbtwst9NewE22UdJGB7tm/2v7osaE
+/DVQk4xrNe/AEDRcQf73XXEEq/wsoJjHp/Be8lvH6bLHpx0rBQJWA6Lxy3+HJQTb
+dcwx4oqaYf/fDCa6TMR2gPGg6RouGt1RYtJ1EDsX7h2HBgnI/b9ri1vJem2BlFVM
+2A3uSpBMO4zQ3xA1Z8N1PxdqX8g9jT412iiNcX8CVWC3atQH8iuwKhnSEqyuVw7s
+e/kTMVcIsQMNENzn+/3JwaBQNXZFnGLJqhdhIMAkuVbcopiu9+/E4462gecCNRSX
+XUyBHANxJIWicAtDYdXaDP6u3NUCVgSPOp+63cxSzMo7FLsYIAFH14VFEXKaw9s9
+JbMI4+dIUMfgs2fvCK6ibFpIrhESv4kaQ+uRjkTjyGv+OkKvP5jxoDUjKRA4WcLU
+OHH2wrajfrFJ0sfhMIIBDDCCAQgCVgMKWFv+kAwf55zhJjiP/+dSyL7wnEFw5gcw
+F8tBg1M7LR4BgTn+j/p/8qEg4scL0eDdDcxGNPaMNlYAtFjO56Qv/oAHJ7EdwfYR
+knMUDxYZV2LU+aGpAlYCMhBIrnWbK9b3xOby5ZnolDF/IQXVhA+4lRQ5pR+OlScp
+ifClzpxuSsMNdFAPaQuwlDEImJJakDoUtQGHODKysC3ailAxaMnORjY5f/y8+qPO
+LPnvsQJWAYm7meCYk9a89TGWQfFl4l1W1dYlI/4FDYOirOvFd+/LICMk+9E43Qff
+lqeAL45/QoXNEDpSVNy507ggtBbxqcEehjspYGqp76NRTvRMKgK9/XH4u2E=
+-----END RSA PRIVATE KEY-----
\ No newline at end of file
diff --git a/akka-pki/src/test/resources/pkcs1.pem b/akka-pki/src/test/resources/pkcs1.pem
new file mode 100644
index 0000000000..5f23b81867
--- /dev/null
+++ b/akka-pki/src/test/resources/pkcs1.pem
@@ -0,0 +1,27 @@
+-----BEGIN RSA PRIVATE KEY-----
+MIIEpQIBAAKCAQEA0IQFs9KpS6fpm7Bq5JcAzRzdZnYub7qGBJ9+QZX8F6qUYiXf
+jbVPAZSIksNg6G5yMkzBLZ8r0UOm5WJs6sAHKGJOQk9DcwZEt3XpGbYXAnM5V1sb
+xd5oNcbXLcHouU8jrEj+O2KvmgCzDDCUOf3SjGnF4dWqhsTT9tMJZvWVB0OjpKnT
+zcoxFKO/XCz8Spb1+FgKUgt3afA+JTRpQWtaZJ41fuTg0rm0qtUSeZXPUEYkqqK4
+msoSe7dbu7uiEOkUPoeiP7wzSrwihQSCxOzwdphV2XtF5Xfs24/Ad4WKXTqRVUK/
+yldcQy2orFSsX41KBzS8PI1hnOC6uRA0PiGd/QIDAQABAoIBAQCDbxShGuK326m3
+B2b5m+1XXSB5m3j92FbtxxMwiDgVOuK5UyItEuIwHs5PpHQLTsMQzaze8vwNtlUX
+Ngltl4lrfTvTNF9Ru9vIwLwkBtFOLA8y7yz8doq9iw7LuvTVCft0d7Y4/KWvr00t
+G9nzC/mRpIKlLaeFt7/cT34XtikwH+Ez8uWGidYnrqkKZ0bsIZvD+e7gae+veohN
+BnTcyDIk8P5nXG0vM4hxtjLo3KstemwOt6vCtiKzL2Vq/JAVD3nlec8WPBzft79I
+k5tb3Qm/OnxIQaWF5AhAVkXgMsLL3ddJoAn/K6NZ6NtRGZozkwdP+m4nacrKFJVJ
+6ew7OdAJAoGBAO65GvteHLXQ3d1NfU+X6RSBatJdpuf2S4Hc7P+sTkPjLDlQPDKL
+ZFLVigPlCehMvgGASPOxL8UqPZugwEo83ywp5t/iOUccXZCPSISreSzWJPfOJ+dl
+aKP2cSHHPNC/mISDpp/NF+xfgEAUQQ6AdOKwHGlsFWBvkkw810d4zmXvAoGBAN+b
+QYv32wNa2IHC1Ri3VgtfHpQ20fMz+UO6TMefBbgkcgBk4L+O7gSCpThaqy9R7UnX
+IEH0QyaBEXzQxB7qvCo1pczWiEUcG+vAyyz9U9oO9Hee/UTMOWL4Sj7qoavau2Be
+5PFOO6qA+19JTnStuNb3swNrMmxDQpyNDvUkYAbTAoGBALuYkSCJ84vZaBBZrajX
+mt13WieYWuocPXf+0euVTyfAJOehKr0ZlywVDNFEssVvUT1Cv5FpYz3QlPtwlsuA
+DGzbPMghMZu1Kb3JK1a+nYnjeseVpPwNT+7RYlQGCr+MYOF5x336oNsqrVEt2XX4
+8mGVva4GtsHCy7fHc/GBeMjXAoGBALhEYkytER//okG0xBUdKFwwo6tyTavEndpx
+UUqDwpvP9N5cQ1W4vG6dFviMx1s0gX4DOQMA/sFhRX79L1FnEW8bTKmz9RI2qs+p
+zgUiMhKVlmJpc79ZKMVlZRHaGybbFuTA7pvoY4ULy5rndy7x5kvITg44LZJID0Gh
+gL0Fn9ifAoGAEaWA7yxTA7phDIt0U91HZEVhNSM4cZDurE7614qWhKveSP8f0J4c
+3d9y/re4RcCwmss/FtQWSkB+32WbD3qk+QB7hV1oFqJ5ObcfwevGYR8m6vOz1h2L
+3pQNi4PcH3U8eeGG1laFKUQ295rBLNqIbOo2y+4hxMnC4tka1X118Ec=
+-----END RSA PRIVATE KEY-----
\ No newline at end of file
diff --git a/akka-pki/src/test/resources/pkcs8.pem b/akka-pki/src/test/resources/pkcs8.pem
new file mode 100644
index 0000000000..bd63aa1dc0
--- /dev/null
+++ b/akka-pki/src/test/resources/pkcs8.pem
@@ -0,0 +1,28 @@
+-----BEGIN PRIVATE KEY-----
+MIIEvwIBADANBgkqhkiG9w0BAQEFAASCBKkwggSlAgEAAoIBAQDQhAWz0qlLp+mb
+sGrklwDNHN1mdi5vuoYEn35BlfwXqpRiJd+NtU8BlIiSw2DobnIyTMEtnyvRQ6bl
+YmzqwAcoYk5CT0NzBkS3dekZthcCczlXWxvF3mg1xtctwei5TyOsSP47Yq+aALMM
+MJQ5/dKMacXh1aqGxNP20wlm9ZUHQ6OkqdPNyjEUo79cLPxKlvX4WApSC3dp8D4l
+NGlBa1pknjV+5ODSubSq1RJ5lc9QRiSqoriayhJ7t1u7u6IQ6RQ+h6I/vDNKvCKF
+BILE7PB2mFXZe0Xld+zbj8B3hYpdOpFVQr/KV1xDLaisVKxfjUoHNLw8jWGc4Lq5
+EDQ+IZ39AgMBAAECggEBAINvFKEa4rfbqbcHZvmb7VddIHmbeP3YVu3HEzCIOBU6
+4rlTIi0S4jAezk+kdAtOwxDNrN7y/A22VRc2CW2XiWt9O9M0X1G728jAvCQG0U4s
+DzLvLPx2ir2LDsu69NUJ+3R3tjj8pa+vTS0b2fML+ZGkgqUtp4W3v9xPfhe2KTAf
+4TPy5YaJ1ieuqQpnRuwhm8P57uBp7696iE0GdNzIMiTw/mdcbS8ziHG2Mujcqy16
+bA63q8K2IrMvZWr8kBUPeeV5zxY8HN+3v0iTm1vdCb86fEhBpYXkCEBWReAywsvd
+10mgCf8ro1no21EZmjOTB0/6bidpysoUlUnp7Ds50AkCgYEA7rka+14ctdDd3U19
+T5fpFIFq0l2m5/ZLgdzs/6xOQ+MsOVA8MotkUtWKA+UJ6Ey+AYBI87EvxSo9m6DA
+SjzfLCnm3+I5RxxdkI9IhKt5LNYk984n52Voo/ZxIcc80L+YhIOmn80X7F+AQBRB
+DoB04rAcaWwVYG+STDzXR3jOZe8CgYEA35tBi/fbA1rYgcLVGLdWC18elDbR8zP5
+Q7pMx58FuCRyAGTgv47uBIKlOFqrL1HtSdcgQfRDJoERfNDEHuq8KjWlzNaIRRwb
+68DLLP1T2g70d579RMw5YvhKPuqhq9q7YF7k8U47qoD7X0lOdK241vezA2sybENC
+nI0O9SRgBtMCgYEAu5iRIInzi9loEFmtqNea3XdaJ5ha6hw9d/7R65VPJ8Ak56Eq
+vRmXLBUM0USyxW9RPUK/kWljPdCU+3CWy4AMbNs8yCExm7UpvckrVr6dieN6x5Wk
+/A1P7tFiVAYKv4xg4XnHffqg2yqtUS3ZdfjyYZW9rga2wcLLt8dz8YF4yNcCgYEA
+uERiTK0RH/+iQbTEFR0oXDCjq3JNq8Sd2nFRSoPCm8/03lxDVbi8bp0W+IzHWzSB
+fgM5AwD+wWFFfv0vUWcRbxtMqbP1Ejaqz6nOBSIyEpWWYmlzv1koxWVlEdobJtsW
+5MDum+hjhQvLmud3LvHmS8hODjgtkkgPQaGAvQWf2J8CgYARpYDvLFMDumEMi3RT
+3UdkRWE1IzhxkO6sTvrXipaEq95I/x/Qnhzd33L+t7hFwLCayz8W1BZKQH7fZZsP
+eqT5AHuFXWgWonk5tx/B68ZhHybq87PWHYvelA2Lg9wfdTx54YbWVoUpRDb3msEs
+2ohs6jbL7iHEycLi2RrVfXXwRw==
+-----END PRIVATE KEY-----
diff --git a/akka-pki/src/test/scala/akka/pki/pem/DERPrivateKeyLoaderSpec.scala b/akka-pki/src/test/scala/akka/pki/pem/DERPrivateKeyLoaderSpec.scala
new file mode 100644
index 0000000000..c10eb11930
--- /dev/null
+++ b/akka-pki/src/test/scala/akka/pki/pem/DERPrivateKeyLoaderSpec.scala
@@ -0,0 +1,54 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.pki.pem
+
+import java.io.File
+import java.nio.charset.Charset
+import java.nio.file.Files
+import java.security.PrivateKey
+
+import org.scalatest.EitherValues
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
+class DERPrivateKeyLoaderSpec extends AnyWordSpec with Matchers with EitherValues {
+
+ "The DER Private Key loader" should {
+ "decode the same key in PKCS#1 and PKCS#8 formats" in {
+ val pkcs1 = load("pkcs1.pem")
+ val pkcs8 = load("pkcs8.pem")
+ pkcs1 should ===(pkcs8)
+ }
+
+ "parse multi primes" in {
+ load("multi-prime-pkcs1.pem")
+ // Not much we can verify here - I actually think the default JDK security implementation ignores the extra
+ // primes, and it fails to parse a multi-prime PKCS#8 key.
+ }
+
+ "fail on unsupported PEM contents (Certificates are not private keys)" in {
+ assertThrows[PEMLoadingException] {
+ load("certificate.pem")
+ }
+ }
+
+ }
+
+ private def load(resource: String): PrivateKey = {
+ val derData: PEMDecoder.DERData = loadDerData(resource)
+ DERPrivateKeyLoader.load(derData)
+ }
+
+ private def loadDerData(resource: String) = {
+ val resourceUrl = getClass.getClassLoader.getResource(resource)
+ resourceUrl.getProtocol should ===("file")
+ val path = new File(resourceUrl.toURI).toPath
+ val bytes = Files.readAllBytes(path)
+ val str = new String(bytes, Charset.forName("UTF-8"))
+ val derData = PEMDecoder.decode(str)
+ derData
+ }
+
+}
diff --git a/akka-pki/src/test/scala/akka/pki/pem/PEMDecoderSpec.scala b/akka-pki/src/test/scala/akka/pki/pem/PEMDecoderSpec.scala
new file mode 100644
index 0000000000..6716e782e0
--- /dev/null
+++ b/akka-pki/src/test/scala/akka/pki/pem/PEMDecoderSpec.scala
@@ -0,0 +1,93 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.pki.pem
+
+import java.util.Base64
+
+import org.scalatest.EitherValues
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
+class PEMDecoderSpec extends AnyWordSpec with Matchers with EitherValues {
+
+ private val cert =
+ """-----BEGIN CERTIFICATE-----
+ |MIIDCzCCAfOgAwIBAgIQfEHPfR1p1xuW9TQlfxAugjANBgkqhkiG9w0BAQsFADAv
+ |MS0wKwYDVQQDEyQwZDIwN2I2OC05YTIwLTRlZTgtOTJjYi1iZjk2OTk1ODFjZjgw
+ |HhcNMTkxMDExMTMyODUzWhcNMjQxMDA5MTQyODUzWjAvMS0wKwYDVQQDEyQwZDIw
+ |N2I2OC05YTIwLTRlZTgtOTJjYi1iZjk2OTk1ODFjZjgwggEiMA0GCSqGSIb3DQEB
+ |AQUAA4IBDwAwggEKAoIBAQDhD0BxlDzEOzcp45lPHL60lnM6k3puEGb2lKHL5/nR
+ |F94FCnZL0FH8EdxWzzAYgys+kUwSdo4QMuWuvjY2Km4Wob6k4uAeYEFTCfBdi4/z
+ |r4kpWzu8xLz+uZWimLQrjqVytNNK3DMv6ebWUJ/92VTDS4yzWk4YV0MVr2b2OgMK
+ |SgMvaFQ8L/CwyML72PBWIqU67+MMvvcTLxQdyEgQTTjP0bbiXMLDvfZDarLJojsW
+ |SNBz7AIkznhGkzIGGdhAa41PnPu9XaBFhaqx9Qe3+MG2/k1l/46eHtmxCqhOUde1
+ |i0vy6ZfgcGifua1tg1UBI/oT4S0dsq24dq7K1MYLyHTrAgMBAAGjIzAhMA4GA1Ud
+ |DwEB/wQEAwICBDAPBgNVHRMBAf8EBTADAQH/MA0GCSqGSIb3DQEBCwUAA4IBAQAa
+ |5YOlvob4wqps3HLaOIi7VzDihNHciP+BI0mzxHa7D3bGaecRPSeG3xEoD/Uxs9o4
+ |8cByPpYx1Wl8LLRx14wDcK0H+UPpo4gCI6h6q92cJj0YRjcSUUt8EIu3qnFMjtM+
+ |sl/uc21fGlq6cvMRZXqtVYENoSNTDHi5a5eEXRa2eZ8XntjvOanOhIKWmxvr8r4a
+ |Voz4WdnXx1C8/BzB62UBoMu4QqMGMLk5wXP0D6hECUuespMest+BeoJAVhTq7wZs
+ |rSP9q08n1stZFF4+bEBaxcqIqdhOLQdHcYELN+a5v0Mcwdsy7jJMagmNPfsKoOKC
+ |hLOsmNYKHdmWg37Jib5o
+ |-----END CERTIFICATE-----""".stripMargin
+
+ "The PEM decoder" should {
+ "decode a real world certificate" in {
+ PEMDecoder.decode(cert).label should ===("CERTIFICATE")
+ }
+
+ "decode data with no spaces" in {
+ val result = PEMDecoder.decode(
+ "-----BEGIN FOO-----" + Base64.getEncoder.encodeToString("abc".getBytes()) + "-----END FOO-----")
+ result.label should ===("FOO")
+ new String(result.bytes) should ===("abc")
+ }
+
+ "decode data with lots of spaces" in {
+ val result = PEMDecoder.decode(
+ "\n \t \r -----BEGIN FOO-----\n" +
+ Base64.getEncoder.encodeToString("abc".getBytes()).flatMap(c => s"$c\n\r \t\n") + "-----END FOO-----\n \t \r ")
+ result.label should ===("FOO")
+ new String(result.bytes) should ===("abc")
+ }
+
+ "decode data with two padding characters" in {
+ // A 4 byte input results in a 6 character output with 2 padding characters
+ val result = PEMDecoder.decode(
+ "-----BEGIN FOO-----" + Base64.getEncoder.encodeToString("abcd".getBytes()) + "-----END FOO-----")
+ result.label should ===("FOO")
+ new String(result.bytes) should ===("abcd")
+ }
+
+ "decode data with one padding character" in {
+ // A 5 byte input results in a 7 character output with 1 padding character1
+ val result = PEMDecoder.decode(
+ "-----BEGIN FOO-----" + Base64.getEncoder.encodeToString("abcde".getBytes()) + "-----END FOO-----")
+ result.label should ===("FOO")
+ new String(result.bytes) should ===("abcde")
+ }
+
+ "fail decode when the format is wrong (not MIME BASE64, lines too long)" in {
+ val input = """-----BEGIN CERTIFICATE-----
+ |MIIDCzCCAfOgAwIBAgIQfEHPfR1p1xuW9TQlfxAugjANBgkqhkiG9w0BAQsFADAviZjk2OTk1ODFjZjgw
+ |HhcNMTkxMDExMTMyODUzWhcNMjQxMDA5MTQyODUzWjAvMS0wKwYDVQQDEyQwZDIwhLOsmNYKHdmWg37Jib5o
+ |-----END CERTIFICATE-----""".stripMargin
+
+ assertThrows[PEMLoadingException] {
+ PEMDecoder.decode(input)
+ }
+ }
+
+ "fail decode when the format is wrong (not PEM, invalid per/post-EB)" in {
+ val input = cert.replace("BEGIN", "BGN").replace("END ", "GLGLGL ")
+
+ assertThrows[PEMLoadingException] {
+ PEMDecoder.decode(input)
+ }
+ }
+
+ }
+
+}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/io/DnsSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/io/DnsSpec.scala
index e0a02f8819..3ec2c989b7 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/io/DnsSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/io/DnsSpec.scala
@@ -8,9 +8,10 @@ import java.net.Inet4Address
import java.net.Inet6Address
import java.net.InetAddress
+import com.github.ghik.silencer.silent
+
import akka.remote.RemotingMultiNodeSpec
import akka.remote.testkit.MultiNodeConfig
-import com.github.ghik.silencer.silent
object DnsSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/AttemptSysMsgRedeliverySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/AttemptSysMsgRedeliverySpec.scala
index 3b5e8311b7..3d6bba19c2 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/AttemptSysMsgRedeliverySpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/AttemptSysMsgRedeliverySpec.scala
@@ -5,16 +5,18 @@
package akka.remote
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+import testkit.MultiNodeConfig
+
import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.ActorRef
import akka.actor.Identify
+import akka.actor.PoisonPill
import akka.actor.Props
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._
-import testkit.MultiNodeConfig
-import akka.actor.PoisonPill
-import com.typesafe.config.ConfigFactory
class AttemptSysMsgRedeliveryMultiJvmSpec(artery: Boolean) extends MultiNodeConfig {
@@ -55,14 +57,14 @@ object AttemptSysMsgRedeliverySpec {
abstract class AttemptSysMsgRedeliverySpec(multiNodeConfig: AttemptSysMsgRedeliveryMultiJvmSpec)
extends RemotingMultiNodeSpec(multiNodeConfig) {
- import multiNodeConfig._
import AttemptSysMsgRedeliverySpec._
+ import multiNodeConfig._
def initialParticipants = roles.size
"AttemptSysMsgRedelivery" must {
"redeliver system message after inactivity" taggedAs LongRunningTest in {
- system.actorOf(Props[Echo], "echo")
+ system.actorOf(Props[Echo](), "echo")
enterBarrier("echo-started")
system.actorSelection(node(first) / "user" / "echo") ! Identify(None)
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/LookupRemoteActorSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/LookupRemoteActorSpec.scala
index 6c986ac655..d7e07d85cc 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/LookupRemoteActorSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/LookupRemoteActorSpec.scala
@@ -4,15 +4,16 @@
package akka.remote
+import com.typesafe.config.ConfigFactory
+import testkit.MultiNodeConfig
+
import akka.actor.Actor
+import akka.actor.ActorIdentity
import akka.actor.ActorRef
+import akka.actor.Identify
import akka.actor.Props
import akka.pattern.ask
-import testkit.MultiNodeConfig
import akka.testkit._
-import akka.actor.Identify
-import akka.actor.ActorIdentity
-import com.typesafe.config.ConfigFactory
class LookupRemoteActorMultiJvmSpec(artery: Boolean) extends MultiNodeConfig {
@@ -43,13 +44,13 @@ object LookupRemoteActorSpec {
abstract class LookupRemoteActorSpec(multiNodeConfig: LookupRemoteActorMultiJvmSpec)
extends RemotingMultiNodeSpec(multiNodeConfig) {
- import multiNodeConfig._
import LookupRemoteActorSpec._
+ import multiNodeConfig._
def initialParticipants = 2
runOn(master) {
- system.actorOf(Props[SomeActor], "service-hello")
+ system.actorOf(Props[SomeActor](), "service-hello")
}
"Remoting" must {
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/NewRemoteActorSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/NewRemoteActorSpec.scala
index 922bc7d0cf..0d0595820e 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/NewRemoteActorSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/NewRemoteActorSpec.scala
@@ -4,17 +4,17 @@
package akka.remote
-import akka.actor.Terminated
+import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
import language.postfixOps
+import testkit.MultiNodeConfig
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
+import akka.actor.Terminated
import akka.util.unused
-import testkit.MultiNodeConfig
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.duration._
class NewRemoteActorMultiJvmSpec(artery: Boolean) extends MultiNodeConfig {
@@ -58,8 +58,8 @@ object NewRemoteActorSpec {
abstract class NewRemoteActorSpec(multiNodeConfig: NewRemoteActorMultiJvmSpec)
extends RemotingMultiNodeSpec(multiNodeConfig) {
- import multiNodeConfig._
import NewRemoteActorSpec._
+ import multiNodeConfig._
def initialParticipants = roles.size
@@ -70,7 +70,7 @@ abstract class NewRemoteActorSpec(multiNodeConfig: NewRemoteActorMultiJvmSpec)
"be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" in {
runOn(master) {
- val actor = system.actorOf(Props[SomeActor], "service-hello")
+ val actor = system.actorOf(Props[SomeActor](), "service-hello")
actor.isInstanceOf[RemoteActorRef] should ===(true)
actor.path.address should ===(node(slave).address)
@@ -100,7 +100,7 @@ abstract class NewRemoteActorSpec(multiNodeConfig: NewRemoteActorMultiJvmSpec)
"be locally instantiated on a remote node and be able to communicate through its RemoteActorRef (with deployOnAll)" in {
runOn(master) {
- val actor = system.actorOf(Props[SomeActor], "service-hello2")
+ val actor = system.actorOf(Props[SomeActor](), "service-hello2")
actor.isInstanceOf[RemoteActorRef] should ===(true)
actor.path.address should ===(node(slave).address)
@@ -114,7 +114,7 @@ abstract class NewRemoteActorSpec(multiNodeConfig: NewRemoteActorMultiJvmSpec)
"be able to shutdown system when using remote deployed actor" in within(20 seconds) {
runOn(master) {
- val actor = system.actorOf(Props[SomeActor], "service-hello3")
+ val actor = system.actorOf(Props[SomeActor](), "service-hello3")
actor.isInstanceOf[RemoteActorRef] should ===(true)
actor.path.address should ===(node(slave).address)
// This watch is in race with the shutdown of the watched system. This race should remain, as the test should
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/PiercingShouldKeepQuarantineSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/PiercingShouldKeepQuarantineSpec.scala
index 77157f20ae..3b1e80a505 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/PiercingShouldKeepQuarantineSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/PiercingShouldKeepQuarantineSpec.scala
@@ -5,10 +5,12 @@
package akka.remote
import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+
import akka.actor._
-import akka.testkit._
import akka.remote.testkit.MultiNodeConfig
+import akka.testkit._
class PiercingShouldKeepQuarantineConfig(artery: Boolean) extends MultiNodeConfig {
val first = role("first")
@@ -41,8 +43,8 @@ object PiercingShouldKeepQuarantineSpec {
abstract class PiercingShouldKeepQuarantineSpec(multiNodeConfig: PiercingShouldKeepQuarantineConfig)
extends RemotingMultiNodeSpec(multiNodeConfig) {
- import multiNodeConfig._
import PiercingShouldKeepQuarantineSpec._
+ import multiNodeConfig._
override def initialParticipants = roles.size
@@ -74,7 +76,7 @@ abstract class PiercingShouldKeepQuarantineSpec(multiNodeConfig: PiercingShouldK
}
runOn(second) {
- system.actorOf(Props[Subject], "subject")
+ system.actorOf(Props[Subject](), "subject")
enterBarrier("actors-started")
enterBarrier("actor-identified")
enterBarrier("quarantine-intact")
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeliverySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeliverySpec.scala
index 515175cbb1..5b94848963 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeliverySpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeliverySpec.scala
@@ -4,17 +4,18 @@
package akka.remote
-import scala.language.postfixOps
import scala.concurrent.duration._
+import scala.language.postfixOps
import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
+import akka.actor.ActorIdentity
import akka.actor.ActorRef
+import akka.actor.Identify
import akka.actor.Props
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
-import akka.actor.ActorIdentity
-import akka.actor.Identify
import akka.serialization.jackson.CborSerializable
class RemoteDeliveryConfig(artery: Boolean) extends MultiNodeConfig {
@@ -47,8 +48,8 @@ object RemoteDeliverySpec {
abstract class RemoteDeliverySpec(multiNodeConfig: RemoteDeliveryConfig)
extends RemotingMultiNodeSpec(multiNodeConfig) {
- import multiNodeConfig._
import RemoteDeliverySpec._
+ import multiNodeConfig._
override def initialParticipants = roles.size
@@ -60,7 +61,7 @@ abstract class RemoteDeliverySpec(multiNodeConfig: RemoteDeliveryConfig)
"Remote message delivery" must {
"not drop messages under normal circumstances" in {
- system.actorOf(Props[Postman], "postman-" + myself.name)
+ system.actorOf(Props[Postman](), "postman-" + myself.name)
enterBarrier("actors-started")
runOn(first) {
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeploymentDeathWatchSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeploymentDeathWatchSpec.scala
index 3bb60ceeae..46f0ce0ab4 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeploymentDeathWatchSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeploymentDeathWatchSpec.scala
@@ -4,11 +4,14 @@
package akka.remote
-import language.postfixOps
import java.util.concurrent.TimeoutException
+
import scala.concurrent.Await
import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
import akka.actor.Actor
import akka.actor.ActorSystemImpl
import akka.actor.Props
@@ -68,8 +71,8 @@ object RemoteDeploymentDeathWatchSpec {
abstract class RemoteDeploymentDeathWatchSpec(multiNodeConfig: RemoteDeploymentDeathWatchMultiJvmSpec)
extends RemotingMultiNodeSpec(multiNodeConfig) {
- import multiNodeConfig._
import RemoteDeploymentDeathWatchSpec._
+ import multiNodeConfig._
def scenario: String
// Possible to override to let them heartbeat for a while.
@@ -82,7 +85,7 @@ abstract class RemoteDeploymentDeathWatchSpec(multiNodeConfig: RemoteDeploymentD
"be able to shutdown when remote node crash" taggedAs LongRunningTest in within(20 seconds) {
runOn(second) {
// remote deployment to third
- val hello = system.actorOf(Props[Hello], "hello")
+ val hello = system.actorOf(Props[Hello](), "hello")
hello.path.address should ===(node(third).address)
enterBarrier("hello-deployed")
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteFeaturesSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteFeaturesSpec.scala
index 7ec34b395a..77873f2dac 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteFeaturesSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteFeaturesSpec.scala
@@ -6,6 +6,8 @@ package akka.remote
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ActorIdentity
import akka.actor.ActorRef
import akka.actor.ActorSystemImpl
@@ -27,7 +29,6 @@ import akka.routing.RoundRobinGroup
import akka.routing.RoundRobinPool
import akka.routing.RoutedActorRef
import akka.testkit.TestProbe
-import com.typesafe.config.ConfigFactory
class RemotingFeaturesConfig(val useUnsafe: Boolean, artery: Boolean) extends MultiNodeConfig {
@@ -229,9 +230,10 @@ abstract class RemotingFeaturesSpec(val multiNodeConfig: RemotingFeaturesConfig)
extends RemotingMultiNodeSpec(multiNodeConfig) {
import RemoteWatcher._
- import akka.remote.routing.RemoteRoundRobinSpec._
import multiNodeConfig._
+ import akka.remote.routing.RemoteRoundRobinSpec._
+
override def initialParticipants: Int = roles.size
muteDeadLetters(Heartbeat.getClass)()
@@ -272,7 +274,7 @@ abstract class RemotingFeaturesSpec(val multiNodeConfig: RemotingFeaturesConfig)
"send messages to remote paths" in {
runOn(first, second, third) {
- system.actorOf(Props[SomeActor], name = "target-" + myself.name)
+ system.actorOf(Props[SomeActor](), name = "target-" + myself.name)
enterBarrier("start", "end")
}
@@ -310,7 +312,7 @@ abstract class RemotingFeaturesSpec(val multiNodeConfig: RemotingFeaturesConfig)
runOn(fourth) {
enterBarrier("start")
- val actor = system.actorOf(RoundRobinPool(nrOfInstances = 0).props(Props[SomeActor]), "service-hello")
+ val actor = system.actorOf(RoundRobinPool(nrOfInstances = 0).props(Props[SomeActor]()), "service-hello")
actor.isInstanceOf[RoutedActorRef] should ===(true)
for (_ <- 0 until iterationCount; _ <- 0 until workerInstances) {
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeDeathWatchSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeDeathWatchSpec.scala
index 2aede454a3..39174b4bd2 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeDeathWatchSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeDeathWatchSpec.scala
@@ -4,9 +4,11 @@
package akka.remote
-import language.postfixOps
import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.ActorRef
@@ -92,9 +94,9 @@ object RemoteNodeDeathWatchSpec {
abstract class RemoteNodeDeathWatchSpec(multiNodeConfig: RemoteNodeDeathWatchConfig)
extends RemotingMultiNodeSpec(multiNodeConfig) {
- import multiNodeConfig._
import RemoteNodeDeathWatchSpec._
import RemoteWatcher._
+ import multiNodeConfig._
def scenario: String
// Possible to override to let them heartbeat for a while.
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeRestartDeathWatchSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeRestartDeathWatchSpec.scala
index bfc583575d..09399ce6c1 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeRestartDeathWatchSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeRestartDeathWatchSpec.scala
@@ -6,19 +6,21 @@ package akka.remote
import scala.concurrent.Await
import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.ActorRef
+import akka.actor.ActorSystem
+import akka.actor.ExtendedActorSystem
import akka.actor.Identify
import akka.actor.Props
-import akka.remote.testconductor.RoleName
-import akka.remote.transport.ThrottlerTransportAdapter.Direction
-import akka.remote.testkit.MultiNodeConfig
-import akka.testkit._
-import akka.actor.ExtendedActorSystem
-import akka.actor.ActorSystem
import akka.actor.RootActorPath
+import akka.remote.testconductor.RoleName
+import akka.remote.testkit.MultiNodeConfig
+import akka.remote.transport.ThrottlerTransportAdapter.Direction
+import akka.testkit._
class RemoteNodeRestartDeathWatchConfig(artery: Boolean) extends MultiNodeConfig {
val first = role("first")
@@ -61,8 +63,8 @@ object RemoteNodeRestartDeathWatchSpec {
abstract class RemoteNodeRestartDeathWatchSpec(multiNodeConfig: RemoteNodeRestartDeathWatchConfig)
extends RemotingMultiNodeSpec(multiNodeConfig) {
- import multiNodeConfig._
import RemoteNodeRestartDeathWatchSpec._
+ import multiNodeConfig._
override def initialParticipants = roles.size
@@ -101,7 +103,7 @@ abstract class RemoteNodeRestartDeathWatchSpec(multiNodeConfig: RemoteNodeRestar
runOn(second) {
val address = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
- system.actorOf(Props[Subject], "subject")
+ system.actorOf(Props[Subject](), "subject")
enterBarrier("actors-started")
enterBarrier("watch-established")
@@ -114,7 +116,7 @@ abstract class RemoteNodeRestartDeathWatchSpec(multiNodeConfig: RemoteNodeRestar
akka.remote.classic.netty.tcp.port = ${address.port.get}
akka.remote.artery.canonical.port = ${address.port.get}
""").withFallback(system.settings.config))
- freshSystem.actorOf(Props[Subject], "subject")
+ freshSystem.actorOf(Props[Subject](), "subject")
Await.ready(freshSystem.whenTerminated, 30.seconds)
}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteQuarantinePiercingSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteQuarantinePiercingSpec.scala
index c08276588f..f9306a49fa 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteQuarantinePiercingSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteQuarantinePiercingSpec.scala
@@ -4,14 +4,16 @@
package akka.remote
+import scala.concurrent.Await
import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.remote.testconductor.RoleName
+import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.testkit._
-import akka.remote.testconductor.RoleName
-import scala.concurrent.Await
class RemoteQuarantinePiercingConfig(artery: Boolean) extends MultiNodeConfig {
val first = role("first")
@@ -49,8 +51,8 @@ object RemoteQuarantinePiercingSpec {
abstract class RemoteQuarantinePiercingSpec(multiNodeConfig: RemoteQuarantinePiercingConfig)
extends RemotingMultiNodeSpec(multiNodeConfig) {
- import multiNodeConfig._
import RemoteQuarantinePiercingSpec._
+ import multiNodeConfig._
override def initialParticipants = roles.size
@@ -104,7 +106,7 @@ abstract class RemoteQuarantinePiercingSpec(multiNodeConfig: RemoteQuarantinePie
runOn(second) {
val address = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
- system.actorOf(Props[Subject], "subject")
+ system.actorOf(Props[Subject](), "subject")
enterBarrier("actors-started")
enterBarrier("actor-identified")
@@ -117,7 +119,7 @@ abstract class RemoteQuarantinePiercingSpec(multiNodeConfig: RemoteQuarantinePie
akka.remote.classic.netty.tcp.port = ${address.port.get}
akka.remote.artery.canonical.port = ${address.port.get}
""").withFallback(system.settings.config))
- freshSystem.actorOf(Props[Subject], "subject")
+ freshSystem.actorOf(Props[Subject](), "subject")
Await.ready(freshSystem.whenTerminated, 30.seconds)
}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteReDeploymentSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteReDeploymentSpec.scala
index 36d2eb1759..2586a24f0c 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteReDeploymentSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteReDeploymentSpec.scala
@@ -4,19 +4,20 @@
package akka.remote
-import akka.remote.testkit.MultiNodeConfig
-import akka.actor.Actor
-import akka.actor.ActorRef
-import akka.actor.Props
-import akka.remote.transport.ThrottlerTransportAdapter.Direction._
-import com.typesafe.config.ConfigFactory
-import akka.actor.ActorSystem
import scala.concurrent.Await
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
+import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.ActorLogging
+import akka.actor.ActorRef
+import akka.actor.ActorSystem
import akka.actor.Identify
+import akka.actor.Props
+import akka.remote.testkit.MultiNodeConfig
+import akka.remote.transport.ThrottlerTransportAdapter.Direction._
import akka.testkit.TestProbe
class RemoteReDeploymentConfig(artery: Boolean) extends MultiNodeConfig {
@@ -118,8 +119,8 @@ abstract class RemoteReDeploymentMultiJvmSpec(multiNodeConfig: RemoteReDeploymen
def initialParticipants = roles.size
- import multiNodeConfig._
import RemoteReDeploymentMultiJvmSpec._
+ import multiNodeConfig._
"A remote deployment target system" must {
@@ -131,7 +132,7 @@ abstract class RemoteReDeploymentMultiJvmSpec(multiNodeConfig: RemoteReDeploymen
runOn(second) {
// Create a 'Parent' actor on the 'second' node
// have it create a 'Hello' child (which will be on the 'first' node due to the deployment config):
- system.actorOf(Props[Parent], "parent") ! ((Props[Hello], "hello"))
+ system.actorOf(Props[Parent](), "parent") ! ((Props[Hello](), "hello"))
// The 'Hello' child will send "HelloParent" to the 'Parent', which will pass it to the 'echo' monitor:
expectMsg(15.seconds, "HelloParent")
}
@@ -189,7 +190,7 @@ abstract class RemoteReDeploymentMultiJvmSpec(multiNodeConfig: RemoteReDeploymen
runOn(second) {
val p = TestProbe()(sys)
sys.actorOf(echoProps(p.ref), "echo")
- p.send(sys.actorOf(Props[Parent], "parent"), (Props[Hello], "hello"))
+ p.send(sys.actorOf(Props[Parent](), "parent"), (Props[Hello](), "hello"))
p.expectMsg(15.seconds, "HelloParent")
}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemotingMultiNodeSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemotingMultiNodeSpec.scala
index 8ce347325a..50987aa872 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemotingMultiNodeSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemotingMultiNodeSpec.scala
@@ -4,11 +4,12 @@
package akka.remote
+import com.typesafe.config.ConfigFactory
+import org.scalatest.Suite
+
import akka.remote.artery.ArterySpecSupport
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
import akka.testkit.{ DefaultTimeout, ImplicitSender }
-import com.typesafe.config.ConfigFactory
-import org.scalatest.Suite
object RemotingMultiNodeSpec {
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/TransportFailSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/TransportFailSpec.scala
index dcded25ffe..39c49fea3b 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/TransportFailSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/TransportFailSpec.scala
@@ -8,6 +8,10 @@ import java.util.concurrent.atomic.AtomicBoolean
import scala.concurrent.duration._
+import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.ActorRef
@@ -19,9 +23,6 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.testkit._
import akka.util.unused
-import com.github.ghik.silencer.silent
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
object TransportFailConfig extends MultiNodeConfig {
val first = role("first")
@@ -123,7 +124,7 @@ abstract class TransportFailSpec extends RemotingMultiNodeSpec(TransportFailConf
}
runOn(second) {
- system.actorOf(Props[Subject], "subject")
+ system.actorOf(Props[Subject](), "subject")
enterBarrier("actors-started")
}
@@ -155,7 +156,7 @@ abstract class TransportFailSpec extends RemotingMultiNodeSpec(TransportFailConf
}
runOn(second) {
- val subject2 = system.actorOf(Props[Subject], "subject2")
+ val subject2 = system.actorOf(Props[Subject](), "subject2")
enterBarrier("actors-started2")
enterBarrier("watch-established2")
subject2 ! PoisonPill
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/BenchmarkFileReporter.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/BenchmarkFileReporter.scala
index 0b08e157d3..fc25ba800e 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/BenchmarkFileReporter.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/BenchmarkFileReporter.scala
@@ -11,10 +11,10 @@ import java.time.LocalDateTime
import java.time.ZoneId
import java.time.format.DateTimeFormatter
-import akka.actor.ActorSystem
-
import scala.util.Try
+import akka.actor.ActorSystem
+
/**
* Simple to file logger for benchmark results. Will log relevant settings first to make sure
* results can be understood later.
@@ -33,7 +33,27 @@ object BenchmarkFileReporter {
val formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd-HH-mm-ss")
- def apply(test: String, system: ActorSystem): BenchmarkFileReporter =
+ def apply(test: String, system: ActorSystem): BenchmarkFileReporter = {
+ val settingsToReport =
+ Seq(
+ "akka.test.MaxThroughputSpec.totalMessagesFactor",
+ "akka.test.MaxThroughputSpec.real-message",
+ "akka.test.LatencySpec.totalMessagesFactor",
+ "akka.test.LatencySpec.repeatCount",
+ "akka.test.LatencySpec.real-message",
+ "akka.remote.artery.enabled",
+ "akka.remote.artery.advanced.inbound-lanes",
+ "akka.remote.artery.advanced.buffer-pool-size",
+ "akka.remote.artery.advanced.aeron.idle-cpu-level",
+ "akka.remote.artery.advanced.aeron.embedded-media-driver",
+ "akka.remote.default-remote-dispatcher.throughput",
+ "akka.remote.default-remote-dispatcher.fork-join-executor.parallelism-factor",
+ "akka.remote.default-remote-dispatcher.fork-join-executor.parallelism-min",
+ "akka.remote.default-remote-dispatcher.fork-join-executor.parallelism-max")
+ apply(test, system, settingsToReport)
+ }
+
+ def apply(test: String, system: ActorSystem, settingsToReport: Seq[String]): BenchmarkFileReporter =
new BenchmarkFileReporter {
override val testName = test
@@ -51,22 +71,6 @@ object BenchmarkFileReporter {
val fos = Files.newOutputStream(testResultFile.toPath)
reportResults(s"Git commit: $gitCommit")
- val settingsToReport =
- Seq(
- "akka.test.MaxThroughputSpec.totalMessagesFactor",
- "akka.test.MaxThroughputSpec.real-message",
- "akka.test.LatencySpec.totalMessagesFactor",
- "akka.test.LatencySpec.repeatCount",
- "akka.test.LatencySpec.real-message",
- "akka.remote.artery.enabled",
- "akka.remote.artery.advanced.inbound-lanes",
- "akka.remote.artery.advanced.buffer-pool-size",
- "akka.remote.artery.advanced.aeron.idle-cpu-level",
- "akka.remote.artery.advanced.aeron.embedded-media-driver",
- "akka.remote.default-remote-dispatcher.throughput",
- "akka.remote.default-remote-dispatcher.fork-join-executor.parallelism-factor",
- "akka.remote.default-remote-dispatcher.fork-join-executor.parallelism-min",
- "akka.remote.default-remote-dispatcher.fork-join-executor.parallelism-max")
settingsToReport.foreach(reportSetting)
def reportResults(result: String): Unit = synchronized {
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/DirectMemorySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/DirectMemorySpec.scala
index 62648ed8a2..352e248382 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/DirectMemorySpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/DirectMemorySpec.scala
@@ -6,12 +6,13 @@ package akka.remote.artery
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.actor.{ Actor, ActorPath, ActorRef, Props }
import akka.remote.RemotingMultiNodeSpec
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
import akka.testkit.ImplicitSender
import akka.testkit.JavaSerializable
-import com.typesafe.config.ConfigFactory
object DirectMemorySpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/FanInThrougputSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/FanInThrougputSpec.scala
index aaae74cd68..e4e878d66b 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/FanInThrougputSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/FanInThrougputSpec.scala
@@ -6,15 +6,16 @@ package akka.remote.artery
import java.util.concurrent.Executors
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
+import akka.remote.{ RemoteActorRefProvider, RemotingMultiNodeSpec }
import akka.remote.artery.MaxThroughputSpec._
import akka.remote.testconductor.RoleName
import akka.remote.testkit.{ MultiNodeConfig, PerfFlamesSupport }
-import akka.remote.{ RemoteActorRefProvider, RemotingMultiNodeSpec }
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.duration._
object FanInThroughputSpec extends MultiNodeConfig {
val totalNumberOfNodes =
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/FanOutThrougputSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/FanOutThrougputSpec.scala
index 560478e59a..dbb2852c52 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/FanOutThrougputSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/FanOutThrougputSpec.scala
@@ -7,14 +7,16 @@ package akka.remote.artery
import java.util.concurrent.Executors
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.remote.{ RemoteActorRefProvider, RemotingMultiNodeSpec }
+import akka.remote.artery.MaxThroughputSpec._
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.PerfFlamesSupport
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-import akka.remote.artery.MaxThroughputSpec._
object FanOutThroughputSpec extends MultiNodeConfig {
val totalNumberOfNodes =
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/HandshakeRestartReceiverSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/HandshakeRestartReceiverSpec.scala
index c79c71cea4..d52f44d543 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/HandshakeRestartReceiverSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/HandshakeRestartReceiverSpec.scala
@@ -9,6 +9,8 @@ import scala.concurrent.Future
import scala.concurrent.Promise
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.remote.AddressUidExtension
import akka.remote.RARP
@@ -17,7 +19,6 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
import akka.testkit._
-import com.typesafe.config.ConfigFactory
object HandshakeRestartReceiverSpec extends MultiNodeConfig {
val first = role("first")
@@ -81,7 +82,7 @@ abstract class HandshakeRestartReceiverSpec
"detect restarted receiver and initiate new handshake" in {
runOn(second) {
- system.actorOf(Props[Subject], "subject")
+ system.actorOf(Props[Subject](), "subject")
}
enterBarrier("subject-started")
@@ -124,7 +125,7 @@ abstract class HandshakeRestartReceiverSpec
ConfigFactory.parseString(s"""
akka.remote.artery.canonical.port = ${address.port.get}
""").withFallback(system.settings.config))
- freshSystem.actorOf(Props[Subject], "subject2")
+ freshSystem.actorOf(Props[Subject](), "subject2")
Await.result(freshSystem.whenTerminated, 45.seconds)
}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/LatencySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/LatencySpec.scala
index 548a283731..96c96dec1a 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/LatencySpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/LatencySpec.scala
@@ -8,6 +8,11 @@ import java.util.concurrent.Executors
import java.util.concurrent.atomic.AtomicLongArray
import java.util.concurrent.locks.LockSupport
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+import org.HdrHistogram.Histogram
+
import akka.actor._
import akka.dispatch.Dispatchers
import akka.remote.RemotingMultiNodeSpec
@@ -17,10 +22,6 @@ import akka.serialization.jackson.CborSerializable
import akka.stream.ThrottleMode
import akka.stream.scaladsl.Source
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-import org.HdrHistogram.Histogram
-
-import scala.concurrent.duration._
object LatencySpec extends MultiNodeConfig {
val first = role("first")
@@ -63,7 +64,7 @@ object LatencySpec extends MultiNodeConfig {
}
""")).withFallback(RemotingMultiNodeSpec.commonConfig))
- final case object Reset extends CborSerializable
+ case object Reset extends CborSerializable
def echoProps(): Props =
Props(new Echo).withDispatcher(Dispatchers.InternalDispatcherId)
@@ -341,7 +342,7 @@ abstract class LatencySpec extends RemotingMultiNodeSpec(LatencySpec) {
"start echo" in {
runOn(second) {
// just echo back
- system.actorOf(echoProps, "echo")
+ system.actorOf(echoProps(), "echo")
}
enterBarrier("echo-started")
}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/MaxThroughputSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/MaxThroughputSpec.scala
index 7d99753e89..ae5dfbeaeb 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/MaxThroughputSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/MaxThroughputSpec.scala
@@ -8,17 +8,18 @@ import java.nio.ByteBuffer
import java.util.concurrent.Executors
import java.util.concurrent.TimeUnit.NANOSECONDS
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
+import akka.remote.{ RARP, RemoteActorRefProvider, RemotingMultiNodeSpec }
import akka.remote.artery.compress.CompressionProtocol.Events.ReceivedActorRefCompressionTable
import akka.remote.testconductor.RoleName
import akka.remote.testkit.{ MultiNodeConfig, PerfFlamesSupport }
-import akka.remote.{ RARP, RemoteActorRefProvider, RemotingMultiNodeSpec }
-import akka.serialization.jackson.CborSerializable
import akka.serialization.{ ByteBufferSerializer, SerializerWithStringManifest }
+import akka.serialization.jackson.CborSerializable
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.duration._
object MaxThroughputSpec extends MultiNodeConfig {
val first = role("first")
@@ -86,9 +87,9 @@ object MaxThroughputSpec extends MultiNodeConfig {
case object Run
sealed trait Echo extends DeadLetterSuppression with CborSerializable
- final case object StartAck extends Echo
+ case object StartAck extends Echo
final case class Start(correspondingReceiver: ActorRef) extends Echo
- final case object End extends Echo
+ case object End extends Echo
final case class Warmup(payload: AnyRef) extends CborSerializable
final case class EndResult(totalReceived: Long) extends CborSerializable
final case class FlowControl(id: Int, burstStartTime: Long) extends Echo
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteRestartedQuarantinedSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteRestartedQuarantinedSpec.scala
index b4ec1e510c..b90db2830c 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteRestartedQuarantinedSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteRestartedQuarantinedSpec.scala
@@ -4,17 +4,18 @@
package akka.remote.artery
-import akka.actor.{ ActorIdentity, Identify, _ }
-import akka.remote.testconductor.RoleName
-import akka.remote.testkit.MultiNodeConfig
-import akka.remote.{ AddressUidExtension, RARP, RemotingMultiNodeSpec }
-import akka.testkit._
-import com.typesafe.config.ConfigFactory
-
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.language.postfixOps
+import com.typesafe.config.ConfigFactory
+
+import akka.actor.{ ActorIdentity, Identify, _ }
+import akka.remote.{ AddressUidExtension, RARP, RemotingMultiNodeSpec }
+import akka.remote.testconductor.RoleName
+import akka.remote.testkit.MultiNodeConfig
+import akka.testkit._
+
object RemoteRestartedQuarantinedSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
@@ -63,7 +64,7 @@ abstract class RemoteRestartedQuarantinedSpec extends RemotingMultiNodeSpec(Remo
"should not crash the other system (#17213)" taggedAs LongRunningTest in {
- system.actorOf(Props[Subject], "subject")
+ system.actorOf(Props[Subject](), "subject")
enterBarrier("subject-started")
runOn(first) {
@@ -128,7 +129,7 @@ abstract class RemoteRestartedQuarantinedSpec extends RemotingMultiNodeSpec(Remo
probe.expectMsgType[ActorIdentity](5.seconds).ref should not be (None)
// Now the other system will be able to pass, too
- freshSystem.actorOf(Props[Subject], "subject")
+ freshSystem.actorOf(Props[Subject](), "subject")
Await.ready(freshSystem.whenTerminated, 10.seconds)
}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/SurviveInboundStreamRestartWithCompressionInFlightSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/SurviveInboundStreamRestartWithCompressionInFlightSpec.scala
index 054c427c34..ed32c60611 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/SurviveInboundStreamRestartWithCompressionInFlightSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/SurviveInboundStreamRestartWithCompressionInFlightSpec.scala
@@ -4,16 +4,17 @@
package akka.remote.artery
+import scala.util.control.NoStackTrace
+
+import com.typesafe.config.ConfigFactory
+import org.scalatest.concurrent.ScalaFutures
+
import akka.actor._
import akka.actor.ActorIdentity
import akka.actor.Identify
import akka.remote.{ RARP, RemotingMultiNodeSpec }
import akka.remote.testkit.MultiNodeConfig
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-import org.scalatest.concurrent.ScalaFutures
-
-import scala.util.control.NoStackTrace
object SurviveInboundStreamRestartWithCompressionInFlightSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/SurviveNetworkPartitionSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/SurviveNetworkPartitionSpec.scala
index 0aa0314f48..c61323423e 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/SurviveNetworkPartitionSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/SurviveNetworkPartitionSpec.scala
@@ -6,6 +6,9 @@ package akka.remote.artery
import scala.concurrent.duration._
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.actor.ActorIdentity
import akka.actor.Identify
@@ -13,8 +16,6 @@ import akka.remote.{ RARP, RemotingMultiNodeSpec }
import akka.remote.testkit.MultiNodeConfig
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._
-import com.github.ghik.silencer.silent
-import com.typesafe.config.ConfigFactory
object SurviveNetworkPartitionSpec extends MultiNodeConfig {
val first = role("first")
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestMessage.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestMessage.scala
index 3bfc1c46fe..3535982f42 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestMessage.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestMessage.scala
@@ -5,9 +5,9 @@
package akka.remote.artery
import akka.actor.ExtendedActorSystem
-import akka.serialization.SerializerWithStringManifest
-import akka.remote.artery.protobuf.{ TestMessages => proto }
import akka.protobufv3.internal.ByteString
+import akka.remote.artery.protobuf.{ TestMessages => proto }
+import akka.serialization.SerializerWithStringManifest
object TestMessage {
final case class Item(id: Long, name: String)
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestRateReporter.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestRateReporter.scala
index 17a60dbb65..a7fd631664 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestRateReporter.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestRateReporter.scala
@@ -13,9 +13,16 @@ class TestRateReporter(name: String)
bytesPerSec: Double,
totalMessages: Long,
totalBytes: Long): Unit = {
- println(
- name +
- f": ${messagesPerSec}%,.0f msgs/sec, ${bytesPerSec}%,.0f bytes/sec, " +
- f"totals ${totalMessages}%,d messages ${totalBytes / (1024 * 1024)}%,d MB")
+ if (totalBytes > 0) {
+ println(
+ name +
+ f": ${messagesPerSec}%,.0f msgs/sec, ${bytesPerSec}%,.0f bytes/sec, " +
+ f"totals ${totalMessages}%,d messages ${totalBytes / (1024 * 1024)}%,d MB")
+ } else {
+ println(
+ name +
+ f": ${messagesPerSec}%,.0f msgs/sec " +
+ f"total ${totalMessages}%,d messages")
+ }
}
}) {}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamConcistencySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamConcistencySpec.scala
index 594975891e..899b59652e 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamConcistencySpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamConcistencySpec.scala
@@ -8,6 +8,14 @@ package aeron
import java.io.File
import java.util.concurrent.atomic.AtomicInteger
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+import io.aeron.Aeron
+import io.aeron.driver.MediaDriver
+import org.agrona.IoUtil
+
import akka.Done
import akka.actor.ExtendedActorSystem
import akka.actor.Props
@@ -20,13 +28,6 @@ import akka.stream.ThrottleMode
import akka.stream.scaladsl.Source
import akka.testkit._
import akka.util.ByteString
-import com.typesafe.config.ConfigFactory
-import io.aeron.Aeron
-import io.aeron.driver.MediaDriver
-import org.agrona.IoUtil
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
object AeronStreamConsistencySpec extends MultiNodeConfig {
val first = role("first")
@@ -97,7 +98,7 @@ abstract class AeronStreamConsistencySpec
"Message consistency of Aeron Streams" must {
"start upd port" in {
- system.actorOf(Props[UdpPortActor], "updPort")
+ system.actorOf(Props[UdpPortActor](), "updPort")
enterBarrier("udp-port-started")
}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamLatencySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamLatencySpec.scala
index 7a859f8cd4..17046d1f9a 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamLatencySpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamLatencySpec.scala
@@ -16,6 +16,14 @@ import java.util.concurrent.locks.LockSupport
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+import io.aeron.Aeron
+import io.aeron.CncFileDescriptor
+import io.aeron.driver.MediaDriver
+import org.HdrHistogram.Histogram
+import org.agrona.IoUtil
+import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
+
import akka.Done
import akka.actor._
import akka.remote.testconductor.RoleName
@@ -28,13 +36,6 @@ import akka.stream.scaladsl.Flow
import akka.stream.scaladsl.Source
import akka.testkit._
import akka.util.ByteString
-import com.typesafe.config.ConfigFactory
-import io.aeron.Aeron
-import io.aeron.CncFileDescriptor
-import io.aeron.driver.MediaDriver
-import org.HdrHistogram.Histogram
-import org.agrona.IoUtil
-import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
object AeronStreamLatencySpec extends MultiNodeConfig {
val first = role("first")
@@ -305,7 +306,7 @@ abstract class AeronStreamLatencySpec
"Latency of Aeron Streams" must {
"start upd port" in {
- system.actorOf(Props[UdpPortActor], "updPort")
+ system.actorOf(Props[UdpPortActor](), "updPort")
enterBarrier("udp-port-started")
}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamMaxThroughputSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamMaxThroughputSpec.scala
index 938fa40c3d..6d6cbd8f88 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamMaxThroughputSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamMaxThroughputSpec.scala
@@ -5,28 +5,28 @@
package akka.remote.artery
package aeron
+import java.io.File
import java.util.concurrent.Executors
import scala.collection.AbstractIterator
import scala.concurrent.Await
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+import io.aeron.Aeron
+import io.aeron.CncFileDescriptor
+import io.aeron.driver.MediaDriver
+import org.agrona.IoUtil
+
import akka.actor._
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
+import akka.stream.KillSwitches
import akka.stream.scaladsl.Source
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-import io.aeron.Aeron
-import io.aeron.driver.MediaDriver
-import akka.stream.KillSwitches
-import java.io.File
-
import akka.util.ByteString
-import io.aeron.CncFileDescriptor
-import org.agrona.IoUtil
object AeronStreamMaxThroughputSpec extends MultiNodeConfig {
val first = role("first")
@@ -226,7 +226,7 @@ abstract class AeronStreamMaxThroughputSpec
"Max throughput of Aeron Streams" must {
"start upd port" in {
- system.actorOf(Props[UdpPortActor], "updPort")
+ system.actorOf(Props[UdpPortActor](), "updPort")
enterBarrier("udp-port-started")
}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteGatePiercingSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteGatePiercingSpec.scala
index 1319098a4b..fa304daac9 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteGatePiercingSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteGatePiercingSpec.scala
@@ -4,18 +4,19 @@
package akka.remote.classic
-import akka.actor.{ ActorIdentity, Identify, _ }
-import akka.remote.testconductor.RoleName
-import akka.remote.testkit.MultiNodeConfig
-import akka.remote.transport.AssociationHandle
-import akka.remote.transport.ThrottlerTransportAdapter.ForceDisassociateExplicitly
-import akka.remote.{ RARP, RemotingMultiNodeSpec }
-import akka.testkit._
-import com.typesafe.config.ConfigFactory
import scala.concurrent.Await
import scala.concurrent.duration._
import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
+import akka.actor.{ ActorIdentity, Identify, _ }
+import akka.remote.{ RARP, RemotingMultiNodeSpec }
+import akka.remote.testconductor.RoleName
+import akka.remote.testkit.MultiNodeConfig
+import akka.remote.transport.AssociationHandle
+import akka.remote.transport.ThrottlerTransportAdapter.ForceDisassociateExplicitly
+import akka.testkit._
object RemoteGatePiercingSpec extends MultiNodeConfig {
val first = role("first")
@@ -62,7 +63,7 @@ abstract class RemoteGatePiercingSpec extends RemotingMultiNodeSpec(RemoteGatePi
"RemoteGatePiercing" must {
"allow restarted node to pass through gate" taggedAs LongRunningTest in {
- system.actorOf(Props[Subject], "subject")
+ system.actorOf(Props[Subject](), "subject")
enterBarrier("actors-started")
runOn(first) {
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteNodeRestartGateSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteNodeRestartGateSpec.scala
index 12b2fb0164..06d7262518 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteNodeRestartGateSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteNodeRestartGateSpec.scala
@@ -4,18 +4,19 @@
package akka.remote.classic
-import akka.actor.{ ActorIdentity, Identify, _ }
-import akka.remote.testconductor.RoleName
-import akka.remote.testkit.MultiNodeConfig
-import akka.remote.transport.AssociationHandle
-import akka.remote.transport.ThrottlerTransportAdapter.ForceDisassociateExplicitly
-import akka.remote.{ RARP, RemotingMultiNodeSpec }
-import akka.testkit._
-import com.typesafe.config.ConfigFactory
import scala.concurrent.Await
import scala.concurrent.duration._
import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
+import akka.actor.{ ActorIdentity, Identify, _ }
+import akka.remote.{ RARP, RemotingMultiNodeSpec }
+import akka.remote.testconductor.RoleName
+import akka.remote.testkit.MultiNodeConfig
+import akka.remote.transport.AssociationHandle
+import akka.remote.transport.ThrottlerTransportAdapter.ForceDisassociateExplicitly
+import akka.testkit._
object RemoteNodeRestartGateSpec extends MultiNodeConfig {
val first = role("first")
@@ -60,7 +61,7 @@ abstract class RemoteNodeRestartGateSpec extends RemotingMultiNodeSpec(RemoteNod
"allow restarted node to pass through gate" taggedAs LongRunningTest in {
- system.actorOf(Props[Subject], "subject")
+ system.actorOf(Props[Subject](), "subject")
enterBarrier("subject-started")
runOn(first) {
@@ -120,7 +121,7 @@ abstract class RemoteNodeRestartGateSpec extends RemotingMultiNodeSpec(RemoteNod
}
// Now the other system will be able to pass, too
- freshSystem.actorOf(Props[Subject], "subject")
+ freshSystem.actorOf(Props[Subject](), "subject")
Await.ready(freshSystem.whenTerminated, 30.seconds)
}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteNodeShutdownAndComesBackSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteNodeShutdownAndComesBackSpec.scala
index 2494d092f0..b608186581 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteNodeShutdownAndComesBackSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteNodeShutdownAndComesBackSpec.scala
@@ -4,18 +4,19 @@
package akka.remote.classic
-import akka.actor.{ ActorIdentity, Identify, _ }
-import akka.remote.testconductor.RoleName
-import akka.remote.testkit.MultiNodeConfig
-import akka.remote.transport.ThrottlerTransportAdapter.{ Direction, ForceDisassociate }
-import akka.remote.{ RARP, RemotingMultiNodeSpec }
-import akka.testkit._
-import com.typesafe.config.ConfigFactory
-
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.language.postfixOps
+import com.typesafe.config.ConfigFactory
+
+import akka.actor.{ ActorIdentity, Identify, _ }
+import akka.remote.{ RARP, RemotingMultiNodeSpec }
+import akka.remote.testconductor.RoleName
+import akka.remote.testkit.MultiNodeConfig
+import akka.remote.transport.ThrottlerTransportAdapter.{ Direction, ForceDisassociate }
+import akka.testkit._
+
object RemoteNodeShutdownAndComesBackSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
@@ -63,7 +64,7 @@ abstract class RemoteNodeShutdownAndComesBackSpec extends RemotingMultiNodeSpec(
"properly reset system message buffer state when new system with same Address comes up" taggedAs LongRunningTest in {
runOn(first) {
val secondAddress = node(second).address
- system.actorOf(Props[Subject], "subject1")
+ system.actorOf(Props[Subject](), "subject1")
enterBarrier("actors-started")
val subject = identify(second, "subject")
@@ -127,8 +128,8 @@ abstract class RemoteNodeShutdownAndComesBackSpec extends RemotingMultiNodeSpec(
runOn(second) {
val address = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
- system.actorOf(Props[Subject], "subject")
- system.actorOf(Props[Subject], "sysmsgBarrier")
+ system.actorOf(Props[Subject](), "subject")
+ system.actorOf(Props[Subject](), "sysmsgBarrier")
enterBarrier("actors-started")
enterBarrier("watch-established")
@@ -141,7 +142,7 @@ abstract class RemoteNodeShutdownAndComesBackSpec extends RemotingMultiNodeSpec(
akka.remote.classic.netty.tcp.port = ${address.port.get}
akka.remote.artery.canonical.port = ${address.port.get}
""").withFallback(system.settings.config))
- freshSystem.actorOf(Props[Subject], "subject")
+ freshSystem.actorOf(Props[Subject](), "subject")
Await.ready(freshSystem.whenTerminated, 30.seconds)
}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteRestartedQuarantinedSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteRestartedQuarantinedSpec.scala
index 4446c22464..b3bf1cdf74 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteRestartedQuarantinedSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/RemoteRestartedQuarantinedSpec.scala
@@ -4,17 +4,18 @@
package akka.remote.classic
-import akka.actor.{ ActorIdentity, Identify, _ }
-import akka.remote.testconductor.RoleName
-import akka.remote.testkit.MultiNodeConfig
-import akka.remote.{ AddressUidExtension, RARP, RemotingMultiNodeSpec, ThisActorSystemQuarantinedEvent }
-import akka.testkit._
-import com.typesafe.config.ConfigFactory
-
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.language.postfixOps
+import com.typesafe.config.ConfigFactory
+
+import akka.actor.{ ActorIdentity, Identify, _ }
+import akka.remote.{ AddressUidExtension, RARP, RemotingMultiNodeSpec, ThisActorSystemQuarantinedEvent }
+import akka.remote.testconductor.RoleName
+import akka.remote.testkit.MultiNodeConfig
+import akka.testkit._
+
object RemoteRestartedQuarantinedSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
@@ -67,7 +68,7 @@ abstract class RemoteRestartedQuarantinedSpec extends RemotingMultiNodeSpec(Remo
"should not crash the other system (#17213)" taggedAs LongRunningTest in {
- system.actorOf(Props[Subject], "subject")
+ system.actorOf(Props[Subject](), "subject")
enterBarrier("subject-started")
runOn(first) {
@@ -140,7 +141,7 @@ abstract class RemoteRestartedQuarantinedSpec extends RemotingMultiNodeSpec(Remo
30.seconds)
// Now the other system will be able to pass, too
- freshSystem.actorOf(Props[Subject], "subject")
+ freshSystem.actorOf(Props[Subject](), "subject")
Await.ready(freshSystem.whenTerminated, 10.seconds)
}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/Ticket15109Spec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/Ticket15109Spec.scala
index f98bd09fdd..16d7adabb6 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/Ticket15109Spec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/classic/Ticket15109Spec.scala
@@ -4,18 +4,19 @@
package akka.remote.classic
-import akka.actor.{ ActorIdentity, Identify, _ }
-import akka.remote.testconductor.RoleName
-import akka.remote.testkit.MultiNodeConfig
-import akka.remote.transport.AssociationHandle
-import akka.remote.transport.ThrottlerTransportAdapter.ForceDisassociateExplicitly
-import akka.remote.{ RARP, RemotingMultiNodeSpec }
-import akka.testkit._
-import com.typesafe.config.ConfigFactory
import scala.concurrent.Await
import scala.concurrent.duration._
import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
+import akka.actor.{ ActorIdentity, Identify, _ }
+import akka.remote.{ RARP, RemotingMultiNodeSpec }
+import akka.remote.testconductor.RoleName
+import akka.remote.testkit.MultiNodeConfig
+import akka.remote.transport.AssociationHandle
+import akka.remote.transport.ThrottlerTransportAdapter.ForceDisassociateExplicitly
+import akka.testkit._
object Ticket15109Spec extends MultiNodeConfig {
val first = role("first")
@@ -73,7 +74,7 @@ abstract class Ticket15109Spec extends RemotingMultiNodeSpec(Ticket15109Spec) {
var subject: ActorRef = system.deadLetters
runOn(second) {
- system.actorOf(Props[Subject], "subject")
+ system.actorOf(Props[Subject](), "subject")
}
enterBarrier("actors-started")
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRandomSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRandomSpec.scala
index f6b797f5e3..509cca48c3 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRandomSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRandomSpec.scala
@@ -5,6 +5,9 @@
package akka.remote.routing
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Address
@@ -16,7 +19,6 @@ import akka.routing.Broadcast
import akka.routing.RandomPool
import akka.routing.RoutedActorRef
import akka.testkit._
-import com.typesafe.config.ConfigFactory
class RemoteRandomConfig(artery: Boolean) extends MultiNodeConfig {
@@ -60,8 +62,8 @@ object RemoteRandomSpec {
class RemoteRandomSpec(multiNodeConfig: RemoteRandomConfig)
extends RemotingMultiNodeSpec(multiNodeConfig)
with DefaultTimeout {
- import multiNodeConfig._
import RemoteRandomSpec._
+ import multiNodeConfig._
def initialParticipants = roles.size
@@ -74,7 +76,7 @@ class RemoteRandomSpec(multiNodeConfig: RemoteRandomConfig)
runOn(fourth) {
enterBarrier("start")
- val actor = system.actorOf(RandomPool(nrOfInstances = 0).props(Props[SomeActor]), "service-hello")
+ val actor = system.actorOf(RandomPool(nrOfInstances = 0).props(Props[SomeActor]()), "service-hello")
actor.isInstanceOf[RoutedActorRef] should ===(true)
val connectionCount = 3
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRoundRobinSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRoundRobinSpec.scala
index feaf67d5f4..a4dd754c30 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRoundRobinSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRoundRobinSpec.scala
@@ -4,18 +4,19 @@
package akka.remote.routing
+import scala.collection.immutable
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.{ Actor, ActorRef, Address, PoisonPill, Props }
import akka.pattern.ask
import akka.remote.RemotingMultiNodeSpec
import akka.remote.testkit.MultiNodeConfig
import akka.routing._
import akka.testkit._
-import com.typesafe.config.ConfigFactory
-
-import scala.collection.immutable
-import scala.concurrent.Await
-import scala.concurrent.duration._
-import scala.language.postfixOps
class RemoteRoundRobinConfig(artery: Boolean) extends MultiNodeConfig {
@@ -91,7 +92,7 @@ class RemoteRoundRobinSpec(multiNodeConfig: RemoteRoundRobinConfig)
runOn(fourth) {
enterBarrier("start")
- val actor = system.actorOf(RoundRobinPool(nrOfInstances = 0).props(Props[SomeActor]), "service-hello")
+ val actor = system.actorOf(RoundRobinPool(nrOfInstances = 0).props(Props[SomeActor]()), "service-hello")
actor.isInstanceOf[RoutedActorRef] should ===(true)
val connectionCount = 3
@@ -136,7 +137,7 @@ class RemoteRoundRobinSpec(multiNodeConfig: RemoteRoundRobinConfig)
enterBarrier("start")
val actor =
system.actorOf(
- RoundRobinPool(nrOfInstances = 1, resizer = Some(new TestResizer)).props(Props[SomeActor]),
+ RoundRobinPool(nrOfInstances = 1, resizer = Some(new TestResizer)).props(Props[SomeActor]()),
"service-hello2")
actor.isInstanceOf[RoutedActorRef] should ===(true)
@@ -173,7 +174,7 @@ class RemoteRoundRobinSpec(multiNodeConfig: RemoteRoundRobinConfig)
"send messages with actor selection to remote paths" in {
runOn(first, second, third) {
- system.actorOf(Props[SomeActor], name = "target-" + myself.name)
+ system.actorOf(Props[SomeActor](), name = "target-" + myself.name)
enterBarrier("start", "end")
}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteScatterGatherSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteScatterGatherSpec.scala
index 367ba21279..bb1c755609 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteScatterGatherSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteScatterGatherSpec.scala
@@ -5,6 +5,9 @@
package akka.remote.routing
import scala.concurrent.duration._
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Address
@@ -13,11 +16,10 @@ import akka.actor.Props
import akka.remote.RemotingMultiNodeSpec
import akka.remote.testkit.MultiNodeConfig
import akka.routing.Broadcast
-import akka.routing.ScatterGatherFirstCompletedPool
import akka.routing.RoutedActorRef
+import akka.routing.ScatterGatherFirstCompletedPool
import akka.testkit._
import akka.testkit.TestEvent._
-import com.typesafe.config.ConfigFactory
class RemoteScatterGatherConfig(artery: Boolean) extends MultiNodeConfig {
@@ -65,8 +67,8 @@ object RemoteScatterGatherSpec {
class RemoteScatterGatherSpec(multiNodeConfig: RemoteScatterGatherConfig)
extends RemotingMultiNodeSpec(multiNodeConfig)
with DefaultTimeout {
- import multiNodeConfig._
import RemoteScatterGatherSpec._
+ import multiNodeConfig._
def initialParticipants = roles.size
@@ -82,7 +84,7 @@ class RemoteScatterGatherSpec(multiNodeConfig: RemoteScatterGatherConfig)
runOn(fourth) {
enterBarrier("start")
val actor = system.actorOf(
- ScatterGatherFirstCompletedPool(nrOfInstances = 1, within = 10.seconds).props(Props[SomeActor]),
+ ScatterGatherFirstCompletedPool(nrOfInstances = 1, within = 10.seconds).props(Props[SomeActor]()),
"service-hello")
actor.isInstanceOf[RoutedActorRef] should ===(true)
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/sample/MultiNodeSample.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/sample/MultiNodeSample.scala
index 45be1f0133..981a2341e5 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/sample/MultiNodeSample.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/sample/MultiNodeSample.scala
@@ -55,7 +55,7 @@ class MultiNodeSample extends MultiNodeSpec(MultiNodeSampleConfig) with STMultiN
}
runOn(node2) {
- system.actorOf(Props[Ponger], "ponger")
+ system.actorOf(Props[Ponger](), "ponger")
enterBarrier("deployed")
}
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/testconductor/TestConductorSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/testconductor/TestConductorSpec.scala
index 662011ae7b..bd3ae7abcc 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/testconductor/TestConductorSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/testconductor/TestConductorSpec.scala
@@ -4,15 +4,16 @@
package akka.remote.testconductor
-import language.postfixOps
-import akka.actor.{ Actor, ActorIdentity, Deploy, Identify, Props }
-
import scala.concurrent.duration._
-import akka.testkit.LongRunningTest
+
+import com.typesafe.config.ConfigFactory
+import language.postfixOps
+
+import akka.actor.{ Actor, ActorIdentity, Deploy, Identify, Props }
import akka.remote.RemotingMultiNodeSpec
import akka.remote.testkit.MultiNodeConfig
import akka.remote.transport.ThrottlerTransportAdapter.Direction
-import com.typesafe.config.ConfigFactory
+import akka.testkit.LongRunningTest
object TestConductorMultiJvmSpec extends MultiNodeConfig {
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString("""
diff --git a/akka-remote-tests/src/test/java/akka/remote/artery/protobuf/TestMessages.java b/akka-remote-tests/src/test/java/akka/remote/artery/protobuf/TestMessages.java
index 25542e4f19..408015da8a 100644
--- a/akka-remote-tests/src/test/java/akka/remote/artery/protobuf/TestMessages.java
+++ b/akka-remote-tests/src/test/java/akka/remote/artery/protobuf/TestMessages.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2019-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
diff --git a/akka-remote-tests/src/test/scala/akka/remote/artery/ArteryFailedToBindSpec.scala b/akka-remote-tests/src/test/scala/akka/remote/artery/ArteryFailedToBindSpec.scala
index 9b1e9d08cb..43f3293eb1 100644
--- a/akka-remote-tests/src/test/scala/akka/remote/artery/ArteryFailedToBindSpec.scala
+++ b/akka-remote-tests/src/test/scala/akka/remote/artery/ArteryFailedToBindSpec.scala
@@ -4,14 +4,15 @@
package akka.remote.artery
+import com.typesafe.config.ConfigFactory
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.actor.ActorSystem
import akka.remote.RARP
import akka.remote.RemoteTransportException
import akka.testkit.SocketUtil
import akka.testkit.TestKit
-import com.typesafe.config.ConfigFactory
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
class ArteryFailedToBindSpec extends AnyWordSpec with Matchers {
diff --git a/akka-remote-tests/src/test/scala/akka/remote/classic/RemotingFailedToBindSpec.scala b/akka-remote-tests/src/test/scala/akka/remote/classic/RemotingFailedToBindSpec.scala
index a1c4de0276..b9a499bab8 100644
--- a/akka-remote-tests/src/test/scala/akka/remote/classic/RemotingFailedToBindSpec.scala
+++ b/akka-remote-tests/src/test/scala/akka/remote/classic/RemotingFailedToBindSpec.scala
@@ -4,13 +4,14 @@
package akka.remote.classic
-import akka.actor.ActorSystem
-import akka.testkit.SocketUtil
import com.typesafe.config.ConfigFactory
import org.jboss.netty.channel.ChannelException
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.ActorSystem
+import akka.testkit.SocketUtil
+
class RemotingFailedToBindSpec extends AnyWordSpec with Matchers {
"an ActorSystem" must {
diff --git a/akka-remote-tests/src/test/scala/akka/remote/testconductor/BarrierSpec.scala b/akka-remote-tests/src/test/scala/akka/remote/testconductor/BarrierSpec.scala
index b4e3d3bf26..365a2023f8 100644
--- a/akka-remote-tests/src/test/scala/akka/remote/testconductor/BarrierSpec.scala
+++ b/akka-remote-tests/src/test/scala/akka/remote/testconductor/BarrierSpec.scala
@@ -4,12 +4,14 @@
package akka.remote.testconductor
+import java.net.{ InetAddress, InetSocketAddress }
+
+import scala.concurrent.duration._
+
import language.postfixOps
import akka.actor._
import akka.testkit.{ AkkaSpec, EventFilter, ImplicitSender, TestProbe, TimingTest }
-import scala.concurrent.duration._
-import java.net.{ InetAddress, InetSocketAddress }
object BarrierSpec {
final case class Failed(ref: ActorRef, thr: Throwable)
@@ -23,9 +25,9 @@ object BarrierSpec {
class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender {
+ import BarrierCoordinator._
import BarrierSpec._
import Controller._
- import BarrierCoordinator._
val A = RoleName("a")
val B = RoleName("b")
@@ -564,7 +566,7 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender {
*/
private def getBarrier(): ActorRef = {
system.actorOf(Props(new Actor {
- val barrier = context.actorOf(Props[BarrierCoordinator])
+ val barrier = context.actorOf(Props[BarrierCoordinator]())
override def supervisorStrategy = OneForOneStrategy() {
case x => testActor ! Failed(barrier, x); SupervisorStrategy.Restart
}
diff --git a/akka-remote-tests/src/test/scala/akka/remote/testconductor/ControllerSpec.scala b/akka-remote-tests/src/test/scala/akka/remote/testconductor/ControllerSpec.scala
index 323daf012b..33815ca1df 100644
--- a/akka-remote-tests/src/test/scala/akka/remote/testconductor/ControllerSpec.scala
+++ b/akka-remote-tests/src/test/scala/akka/remote/testconductor/ControllerSpec.scala
@@ -4,12 +4,13 @@
package akka.remote.testconductor
-import akka.testkit.AkkaSpec
-import akka.actor.{ AddressFromURIString, PoisonPill, Props }
-import akka.testkit.ImplicitSender
-import akka.remote.testconductor.Controller.NodeInfo
-import java.net.InetSocketAddress
import java.net.InetAddress
+import java.net.InetSocketAddress
+
+import akka.actor.{ AddressFromURIString, PoisonPill, Props }
+import akka.remote.testconductor.Controller.NodeInfo
+import akka.testkit.AkkaSpec
+import akka.testkit.ImplicitSender
object ControllerSpec {
val config = """
diff --git a/akka-remote-tests/src/test/scala/akka/remote/testkit/LogRoleReplace.scala b/akka-remote-tests/src/test/scala/akka/remote/testkit/LogRoleReplace.scala
index 5c331bb2e9..e8697fca5e 100644
--- a/akka-remote-tests/src/test/scala/akka/remote/testkit/LogRoleReplace.scala
+++ b/akka-remote-tests/src/test/scala/akka/remote/testkit/LogRoleReplace.scala
@@ -18,6 +18,7 @@ import java.io.OutputStreamWriter
import java.io.PrintWriter
import java.io.StringReader
import java.io.StringWriter
+
import scala.annotation.tailrec
/**
diff --git a/akka-remote-tests/src/test/scala/org/scalatest/extra/QuietReporter.scala b/akka-remote-tests/src/test/scala/org/scalatest/extra/QuietReporter.scala
index d44004d138..a1bcb5b67b 100644
--- a/akka-remote-tests/src/test/scala/org/scalatest/extra/QuietReporter.scala
+++ b/akka-remote-tests/src/test/scala/org/scalatest/extra/QuietReporter.scala
@@ -4,10 +4,11 @@
package org.scalatest.extra
-import org.scalatest.tools.StandardOutReporter
-import org.scalatest.events._
import java.lang.Boolean.getBoolean
+import org.scalatest.events._
+import org.scalatest.tools.StandardOutReporter
+
class QuietReporter(inColor: Boolean, withDurations: Boolean = false)
extends StandardOutReporter(withDurations, inColor, false, true, false, false, false, false, false, false, false) {
diff --git a/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java b/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java
index 7122dbc0e9..4fff204269 100644
--- a/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java
+++ b/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2019-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
diff --git a/akka-remote/src/main/java/akka/remote/ContainerFormats.java b/akka-remote/src/main/java/akka/remote/ContainerFormats.java
index 5bae17b263..c15a92afbe 100644
--- a/akka-remote/src/main/java/akka/remote/ContainerFormats.java
+++ b/akka-remote/src/main/java/akka/remote/ContainerFormats.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2019-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
diff --git a/akka-remote/src/main/java/akka/remote/SystemMessageFormats.java b/akka-remote/src/main/java/akka/remote/SystemMessageFormats.java
index 53a327d4ad..b810dc83db 100644
--- a/akka-remote/src/main/java/akka/remote/SystemMessageFormats.java
+++ b/akka-remote/src/main/java/akka/remote/SystemMessageFormats.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2019-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
diff --git a/akka-remote/src/main/java/akka/remote/WireFormats.java b/akka-remote/src/main/java/akka/remote/WireFormats.java
index 87237aa035..1104a931ee 100644
--- a/akka-remote/src/main/java/akka/remote/WireFormats.java
+++ b/akka-remote/src/main/java/akka/remote/WireFormats.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2019-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
@@ -7671,19 +7671,25 @@ public final class WireFormats {
/**
* repeated string tags = 12;
+ * @return A list containing the tags.
*/
java.util.List
getTagsList();
/**
* repeated string tags = 12;
+ * @return The count of tags.
*/
int getTagsCount();
/**
* repeated string tags = 12;
+ * @param index The index of the element to return.
+ * @return The tags at the given index.
*/
java.lang.String getTags(int index);
/**
* repeated string tags = 12;
+ * @param index The index of the value to return.
+ * @return The bytes of the tags at the given index.
*/
akka.protobufv3.internal.ByteString
getTagsBytes(int index);
@@ -8194,6 +8200,7 @@ public final class WireFormats {
private akka.protobufv3.internal.LazyStringList tags_;
/**
* repeated string tags = 12;
+ * @return A list containing the tags.
*/
public akka.protobufv3.internal.ProtocolStringList
getTagsList() {
@@ -8201,18 +8208,23 @@ public final class WireFormats {
}
/**
* repeated string tags = 12;
+ * @return The count of tags.
*/
public int getTagsCount() {
return tags_.size();
}
/**
* repeated string tags = 12;
+ * @param index The index of the element to return.
+ * @return The tags at the given index.
*/
public java.lang.String getTags(int index) {
return tags_.get(index);
}
/**
* repeated string tags = 12;
+ * @param index The index of the value to return.
+ * @return The bytes of the tags at the given index.
*/
public akka.protobufv3.internal.ByteString
getTagsBytes(int index) {
@@ -9514,6 +9526,7 @@ public final class WireFormats {
}
/**
* repeated string tags = 12;
+ * @return A list containing the tags.
*/
public akka.protobufv3.internal.ProtocolStringList
getTagsList() {
@@ -9521,18 +9534,23 @@ public final class WireFormats {
}
/**
* repeated string tags = 12;
+ * @return The count of tags.
*/
public int getTagsCount() {
return tags_.size();
}
/**
* repeated string tags = 12;
+ * @param index The index of the element to return.
+ * @return The tags at the given index.
*/
public java.lang.String getTags(int index) {
return tags_.get(index);
}
/**
* repeated string tags = 12;
+ * @param index The index of the value to return.
+ * @return The bytes of the tags at the given index.
*/
public akka.protobufv3.internal.ByteString
getTagsBytes(int index) {
@@ -9540,6 +9558,9 @@ public final class WireFormats {
}
/**
* repeated string tags = 12;
+ * @param index The index to set the value at.
+ * @param value The tags to set.
+ * @return This builder for chaining.
*/
public Builder setTags(
int index, java.lang.String value) {
@@ -9553,6 +9574,8 @@ public final class WireFormats {
}
/**
* repeated string tags = 12;
+ * @param value The tags to add.
+ * @return This builder for chaining.
*/
public Builder addTags(
java.lang.String value) {
@@ -9566,6 +9589,8 @@ public final class WireFormats {
}
/**
* repeated string tags = 12;
+ * @param values The tags to add.
+ * @return This builder for chaining.
*/
public Builder addAllTags(
java.lang.Iterable values) {
@@ -9577,6 +9602,7 @@ public final class WireFormats {
}
/**
* repeated string tags = 12;
+ * @return This builder for chaining.
*/
public Builder clearTags() {
tags_ = akka.protobufv3.internal.LazyStringArrayList.EMPTY;
@@ -9586,6 +9612,8 @@ public final class WireFormats {
}
/**
* repeated string tags = 12;
+ * @param value The bytes of the tags to add.
+ * @return This builder for chaining.
*/
public Builder addTagsBytes(
akka.protobufv3.internal.ByteString value) {
diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf
index 09d2981a21..8e74e151c8 100644
--- a/akka-remote/src/main/resources/reference.conf
+++ b/akka-remote/src/main/resources/reference.conf
@@ -661,12 +661,18 @@ akka {
# https://blogs.oracle.com/java-platform-group/entry/java_8_will_use_tls
protocol = "TLSv1.2"
- # Example: ["TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA"]
- # You need to install the JCE Unlimited Strength Jurisdiction Policy
- # Files to use AES 256.
+ # Example: ["TLS_DHE_RSA_WITH_AES_128_GCM_SHA256",
+ # "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256",
+ # "TLS_DHE_RSA_WITH_AES_256_GCM_SHA384",
+ # "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384"]
+ # When doing rolling upgrades, make sure to include both the algorithm used
+ # by old nodes and the preferred algorithm.
+ # If you use a JDK 8 prior to 8u161 you need to install
+ # the JCE Unlimited Strength Jurisdiction Policy Files to use AES 256.
# More info here:
- # http://docs.oracle.com/javase/7/docs/technotes/guides/security/SunProviders.html#SunJCEProvider
- enabled-algorithms = ["TLS_RSA_WITH_AES_128_CBC_SHA"]
+ # https://www.oracle.com/java/technologies/javase-jce-all-downloads.html
+ enabled-algorithms = ["TLS_RSA_WITH_AES_128_CBC_SHA",
+ "TLS_DHE_RSA_WITH_AES_256_GCM_SHA384"]
# There are two options, and the default SecureRandom is recommended:
# "" or "SecureRandom" => (default)
@@ -994,7 +1000,7 @@ akka {
# Note that compression tables are "rolling" (i.e. a new table replaces the old
# compression table once in a while), and this setting is only about the total number
# of compressions within a single such table.
- # Must be a positive natural number.
+ # Must be a positive natural number. Can be disabled with "off".
max = 256
# interval between new table compression advertisements.
@@ -1006,7 +1012,7 @@ akka {
# Note that compression tables are "rolling" (i.e. a new table replaces the old
# compression table once in a while), and this setting is only about the total number
# of compressions within a single such table.
- # Must be a positive natural number.
+ # Must be a positive natural number. Can be disabled with "off".
max = 256
# interval between new table compression advertisements.
@@ -1133,12 +1139,18 @@ akka {
# https://blogs.oracle.com/java-platform-group/entry/java_8_will_use_tls
protocol = "TLSv1.2"
- # Example: ["TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA"]
- # You need to install the JCE Unlimited Strength Jurisdiction Policy
- # Files to use AES 256.
+ # Example: ["TLS_DHE_RSA_WITH_AES_128_GCM_SHA256",
+ # "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256",
+ # "TLS_DHE_RSA_WITH_AES_256_GCM_SHA384",
+ # "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384"]
+ # When doing rolling upgrades, make sure to include both the algorithm used
+ # by old nodes and the preferred algorithm.
+ # If you use a JDK 8 prior to 8u161 you need to install
+ # the JCE Unlimited Strength Jurisdiction Policy Files to use AES 256.
# More info here:
- # http://docs.oracle.com/javase/7/docs/technotes/guides/security/SunProviders.html#SunJCEProvider
- enabled-algorithms = ["TLS_RSA_WITH_AES_128_CBC_SHA"]
+ # https://www.oracle.com/java/technologies/javase-jce-all-downloads.html
+ enabled-algorithms = ["TLS_RSA_WITH_AES_128_CBC_SHA",
+ "TLS_DHE_RSA_WITH_AES_256_GCM_SHA384"]
# There are two options, and the default SecureRandom is recommended:
# "" or "SecureRandom" => (default)
diff --git a/akka-remote/src/main/scala-jdk-9/akka/remote/artery/jfr/Events.scala b/akka-remote/src/main/scala-jdk-9/akka/remote/artery/jfr/Events.scala
index f71b15fcd8..cb9013da27 100644
--- a/akka-remote/src/main/scala-jdk-9/akka/remote/artery/jfr/Events.scala
+++ b/akka-remote/src/main/scala-jdk-9/akka/remote/artery/jfr/Events.scala
@@ -6,16 +6,17 @@ package akka.remote.artery.jfr
import java.net.InetSocketAddress
+import jdk.jfr.Category
+import jdk.jfr.DataAmount
+import jdk.jfr.Enabled
+import jdk.jfr.Event
+import jdk.jfr.Label
+import jdk.jfr.StackTrace
+import jdk.jfr.Timespan
+
import akka.actor.Address
import akka.annotation.InternalApi
import akka.remote.UniqueAddress
-import jdk.jfr.StackTrace
-import jdk.jfr.Category
-import jdk.jfr.Label
-import jdk.jfr.Event
-import jdk.jfr.DataAmount
-import jdk.jfr.Enabled
-import jdk.jfr.Timespan
// requires jdk9+ to compile
// for editing these in IntelliJ, open module settings, change JDK dependency to 11 for only this module
diff --git a/akka-remote/src/main/scala/akka/remote/AckedDelivery.scala b/akka-remote/src/main/scala/akka/remote/AckedDelivery.scala
index 8f3519271c..15c507987e 100644
--- a/akka-remote/src/main/scala/akka/remote/AckedDelivery.scala
+++ b/akka-remote/src/main/scala/akka/remote/AckedDelivery.scala
@@ -5,6 +5,7 @@
package akka.remote
import scala.collection.immutable._
+
import akka.AkkaException
@deprecated("Classic remoting is deprecated, use Artery", "2.6.0")
diff --git a/akka-remote/src/main/scala/akka/remote/DeadlineFailureDetector.scala b/akka-remote/src/main/scala/akka/remote/DeadlineFailureDetector.scala
index 09931d6215..ea67c146f1 100644
--- a/akka-remote/src/main/scala/akka/remote/DeadlineFailureDetector.scala
+++ b/akka-remote/src/main/scala/akka/remote/DeadlineFailureDetector.scala
@@ -7,7 +7,9 @@ package akka.remote
import scala.concurrent.duration._
import scala.concurrent.duration.Duration
import scala.concurrent.duration.FiniteDuration
+
import com.typesafe.config.Config
+
import akka.event.EventStream
import akka.remote.FailureDetector.Clock
import akka.util.Helpers.ConfigOps
diff --git a/akka-remote/src/main/scala/akka/remote/DefaultFailureDetectorRegistry.scala b/akka-remote/src/main/scala/akka/remote/DefaultFailureDetectorRegistry.scala
index 1582480778..c474d3e407 100644
--- a/akka-remote/src/main/scala/akka/remote/DefaultFailureDetectorRegistry.scala
+++ b/akka-remote/src/main/scala/akka/remote/DefaultFailureDetectorRegistry.scala
@@ -5,9 +5,10 @@
package akka.remote
import java.util.concurrent.atomic.AtomicReference
+import java.util.concurrent.locks.{ Lock, ReentrantLock }
+
import scala.annotation.tailrec
import scala.collection.immutable.Map
-import java.util.concurrent.locks.{ Lock, ReentrantLock }
/**
* A lock-less thread-safe implementation of [[akka.remote.FailureDetectorRegistry]].
diff --git a/akka-remote/src/main/scala/akka/remote/Endpoint.scala b/akka-remote/src/main/scala/akka/remote/Endpoint.scala
index 91cc088a67..a468d55015 100644
--- a/akka-remote/src/main/scala/akka/remote/Endpoint.scala
+++ b/akka-remote/src/main/scala/akka/remote/Endpoint.scala
@@ -4,16 +4,30 @@
package akka.remote
+import java.io.NotSerializableException
+import java.util.concurrent.{ ConcurrentHashMap, TimeoutException }
+import java.util.concurrent.locks.LockSupport
+
+import scala.annotation.tailrec
+import scala.concurrent.Future
+import scala.concurrent.duration.Deadline
+import scala.concurrent.duration.Duration
+import scala.util.control.NonFatal
+
+import com.github.ghik.silencer.silent
+
+import akka.{ AkkaException, OnlyCauseStackTrace }
+import akka.actor._
import akka.actor.OneForOneStrategy
import akka.actor.SupervisorStrategy._
import akka.actor.Terminated
-import akka.actor._
import akka.dispatch.sysmsg.SystemMessage
import akka.event.{ LogMarker, Logging, MarkerLoggingAdapter }
import akka.pattern.pipe
import akka.remote.EndpointManager.{ Link, ResendState, Send }
import akka.remote.EndpointWriter.{ FlushAndStop, StoppedReading }
import akka.remote.WireFormats.SerializedMessage
+import akka.remote.transport._
import akka.remote.transport.AkkaPduCodec.Message
import akka.remote.transport.AssociationHandle.{
ActorHandleEventListener,
@@ -22,23 +36,9 @@ import akka.remote.transport.AssociationHandle.{
InboundPayload
}
import akka.remote.transport.Transport.InvalidAssociationException
-import akka.remote.transport._
import akka.serialization.Serialization
import akka.util.ByteString
-import akka.{ AkkaException, OnlyCauseStackTrace }
-import java.io.NotSerializableException
-import java.util.concurrent.{ ConcurrentHashMap, TimeoutException }
-
-import scala.annotation.tailrec
-import scala.concurrent.duration.Deadline
-import scala.util.control.NonFatal
-import java.util.concurrent.locks.LockSupport
-
-import scala.concurrent.Future
-import scala.concurrent.duration.Duration
-
import akka.util.OptionVal
-import com.github.ghik.silencer.silent
/**
* INTERNAL API
@@ -899,9 +899,15 @@ private[remote] class EndpointWriter(
remoteMetrics.logPayloadBytes(s.message, pduSize)
if (pduSize > transport.maximumPayloadBytes) {
- val reason = new OversizedPayloadException(
- s"Discarding oversized payload sent to ${s.recipient}: max allowed size ${transport.maximumPayloadBytes} bytes, actual size of encoded ${s.message.getClass} was ${pdu.size} bytes.")
- log.error(reason, "Transient association error (association remains live)")
+ val reasonText =
+ s"Discarding oversized payload sent to ${s.recipient}: max allowed size ${transport.maximumPayloadBytes} bytes, actual size of encoded ${s.message.getClass} was ${pdu.size} bytes."
+ log.error(
+ new OversizedPayloadException(reasonText),
+ "Transient association error (association remains live)")
+ extendedSystem.eventStream.publish(s.senderOption match {
+ case OptionVal.Some(msgSender) => Dropped(s.message, reasonText, msgSender, s.recipient)
+ case OptionVal.None => Dropped(s.message, reasonText, s.recipient)
+ })
true
} else {
val ok = h.write(pdu)
diff --git a/akka-remote/src/main/scala/akka/remote/FailureDetector.scala b/akka-remote/src/main/scala/akka/remote/FailureDetector.scala
index a30b15ff20..bd157af2c9 100644
--- a/akka-remote/src/main/scala/akka/remote/FailureDetector.scala
+++ b/akka-remote/src/main/scala/akka/remote/FailureDetector.scala
@@ -40,7 +40,7 @@ object FailureDetector {
// Abstract class to be able to extend it from Java
abstract class Clock extends (() => Long)
- implicit val defaultClock = new Clock {
+ implicit val defaultClock: Clock = new Clock {
def apply() = NANOSECONDS.toMillis(System.nanoTime)
}
}
diff --git a/akka-remote/src/main/scala/akka/remote/FailureDetectorRegistry.scala b/akka-remote/src/main/scala/akka/remote/FailureDetectorRegistry.scala
index 2e7ddf0d73..a5840e2c52 100644
--- a/akka-remote/src/main/scala/akka/remote/FailureDetectorRegistry.scala
+++ b/akka-remote/src/main/scala/akka/remote/FailureDetectorRegistry.scala
@@ -4,10 +4,11 @@
package akka.remote
-import akka.actor.{ ActorContext, ActorSystem, ExtendedActorSystem }
import com.typesafe.config.Config
-import akka.event.EventStream
+
import akka.ConfigurationException
+import akka.actor.{ ActorContext, ActorSystem, ExtendedActorSystem }
+import akka.event.EventStream
/**
* Interface for a registry of Akka failure detectors. New resources are implicitly registered when heartbeat is first
diff --git a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala
index e85dde7226..c6a6f5cac2 100644
--- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala
+++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala
@@ -4,16 +4,16 @@
package akka.remote
-import akka.remote.WireFormats._
-import akka.protobufv3.internal.ByteString
+import scala.util.control.NonFatal
+
import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi
+import akka.protobufv3.internal.ByteString
+import akka.remote.WireFormats._
import akka.remote.artery.{ EnvelopeBuffer, HeaderBuilder, OutboundEnvelope }
import akka.serialization._
import akka.util.unused
-import scala.util.control.NonFatal
-
/**
* INTERNAL API
*
diff --git a/akka-remote/src/main/scala/akka/remote/PhiAccrualFailureDetector.scala b/akka-remote/src/main/scala/akka/remote/PhiAccrualFailureDetector.scala
index 7a71393771..8a74c11057 100644
--- a/akka-remote/src/main/scala/akka/remote/PhiAccrualFailureDetector.scala
+++ b/akka-remote/src/main/scala/akka/remote/PhiAccrualFailureDetector.scala
@@ -4,18 +4,19 @@
package akka.remote
-import akka.event.Logging.Warning
-import akka.remote.FailureDetector.Clock
import java.util.concurrent.atomic.AtomicReference
import scala.annotation.tailrec
+import scala.collection.immutable
import scala.concurrent.duration.Duration
import scala.concurrent.duration.FiniteDuration
-import scala.collection.immutable
import com.typesafe.config.Config
+
import akka.event.EventStream
import akka.event.Logging
+import akka.event.Logging.Warning
+import akka.remote.FailureDetector.Clock
import akka.util.Helpers.ConfigOps
/**
diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala
index 81c55658c0..b83ab570d5 100644
--- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala
+++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala
@@ -9,12 +9,14 @@ import scala.util.Failure
import scala.util.control.Exception.Catcher
import scala.util.control.NonFatal
+import com.github.ghik.silencer.silent
+
import akka.ConfigurationException
import akka.Done
+import akka.actor._
import akka.actor.SystemGuardian.RegisterTerminationHook
import akka.actor.SystemGuardian.TerminationHook
import akka.actor.SystemGuardian.TerminationHookDone
-import akka.actor._
import akka.annotation.InternalApi
import akka.dispatch.RequiresMessageQueue
import akka.dispatch.UnboundedMessageQueueSemantics
@@ -36,7 +38,6 @@ import akka.serialization.Serialization
import akka.util.ErrorMessages
import akka.util.OptionVal
import akka.util.unused
-import com.github.ghik.silencer.silent
/**
* INTERNAL API
@@ -372,9 +373,6 @@ private[akka] class RemoteActorRefProvider(
if (systemService) local.actorOf(system, props, supervisor, path, systemService, deploy, lookupDeploy, async)
else {
- if (!system.dispatchers.hasDispatcher(props.dispatcher))
- throw new ConfigurationException(s"Dispatcher [${props.dispatcher}] not configured for path $path")
-
/*
* This needs to deal with “mangled” paths, which are created by remote
* deployment, also in this method. The scheme is the following:
diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala
index 3309be0072..0ec0664ce9 100644
--- a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala
+++ b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala
@@ -4,8 +4,12 @@
package akka.remote
+import java.util.concurrent.ConcurrentHashMap
+
import scala.annotation.tailrec
+import scala.collection.immutable
import scala.util.control.NonFatal
+
import akka.actor.{
Actor,
ActorPath,
@@ -19,22 +23,19 @@ import akka.actor.{
Props,
VirtualPathContainer
}
-import akka.event.{ AddressTerminatedTopic, LogMarker, MarkerLoggingAdapter }
-import akka.dispatch.sysmsg.{ DeathWatchNotification, SystemMessage, Watch }
-import akka.actor.ActorRefWithCell
+import akka.actor.ActorIdentity
import akka.actor.ActorRefScope
-import akka.util.Switch
+import akka.actor.ActorRefWithCell
import akka.actor.ActorSelectionMessage
-import akka.actor.SelectParent
+import akka.actor.EmptyLocalActorRef
+import akka.actor.Identify
import akka.actor.SelectChildName
import akka.actor.SelectChildPattern
-import akka.actor.Identify
-import akka.actor.ActorIdentity
-import akka.actor.EmptyLocalActorRef
-import java.util.concurrent.ConcurrentHashMap
-
-import scala.collection.immutable
+import akka.actor.SelectParent
+import akka.dispatch.sysmsg.{ DeathWatchNotification, SystemMessage, Watch }
import akka.dispatch.sysmsg.Unwatch
+import akka.event.{ AddressTerminatedTopic, LogMarker, MarkerLoggingAdapter }
+import akka.util.Switch
/**
* INTERNAL API
@@ -169,13 +170,13 @@ private[akka] class RemoteSystemDaemon(
doCreateActor(message, props, deploy, path, supervisor)
else {
val ex =
- new NotWhitelistedClassRemoteDeploymentAttemptException(props.actorClass, remoteDeploymentWhitelist)
+ new NotWhitelistedClassRemoteDeploymentAttemptException(props.actorClass(), remoteDeploymentWhitelist)
log.error(
LogMarker.Security,
ex,
"Received command to create remote Actor, but class [{}] is not white-listed! " +
"Target path: [{}]",
- props.actorClass,
+ props.actorClass(),
path)
}
case DaemonMsgCreate(props, deploy, path, supervisor) =>
diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala
index b08bf8ae8e..0a5b4f9d30 100644
--- a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala
+++ b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala
@@ -4,14 +4,15 @@
package akka.remote
-import akka.actor._
-import akka.routing._
-import akka.remote.routing._
-import akka.ConfigurationException
-import akka.japi.Util.immutableSeq
import com.typesafe.config._
-import akka.routing.Pool
+
+import akka.ConfigurationException
+import akka.actor._
+import akka.japi.Util.immutableSeq
+import akka.remote.routing._
import akka.remote.routing.RemoteRouterConfig
+import akka.routing._
+import akka.routing.Pool
@SerialVersionUID(1L)
final case class RemoteScope(node: Address) extends Scope {
diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDeploymentWatcher.scala b/akka-remote/src/main/scala/akka/remote/RemoteDeploymentWatcher.scala
index c764794264..2b8a70e77d 100644
--- a/akka-remote/src/main/scala/akka/remote/RemoteDeploymentWatcher.scala
+++ b/akka-remote/src/main/scala/akka/remote/RemoteDeploymentWatcher.scala
@@ -4,12 +4,12 @@
package akka.remote
-import akka.actor.InternalActorRef
-import akka.actor.Terminated
import akka.actor.Actor
import akka.actor.ActorRef
-import akka.dispatch.sysmsg.DeathWatchNotification
+import akka.actor.InternalActorRef
+import akka.actor.Terminated
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
+import akka.dispatch.sysmsg.DeathWatchNotification
/**
* INTERNAL API
diff --git a/akka-remote/src/main/scala/akka/remote/RemoteMetricsExtension.scala b/akka-remote/src/main/scala/akka/remote/RemoteMetricsExtension.scala
index e1d8802b92..07a7b45ede 100644
--- a/akka-remote/src/main/scala/akka/remote/RemoteMetricsExtension.scala
+++ b/akka-remote/src/main/scala/akka/remote/RemoteMetricsExtension.scala
@@ -7,6 +7,9 @@ package akka.remote
import java.util.concurrent.ConcurrentHashMap
import scala.annotation.tailrec
+
+import com.github.ghik.silencer.silent
+
import akka.actor.ActorSelectionMessage
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
@@ -16,7 +19,6 @@ import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
import akka.event.Logging
import akka.routing.RouterEnvelope
-import com.github.ghik.silencer.silent
/**
* INTERNAL API
diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala
index 42ff37bf32..ebc1716190 100644
--- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala
+++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala
@@ -4,23 +4,25 @@
package akka.remote
-import com.typesafe.config.Config
-import scala.concurrent.duration._
import scala.collection.immutable
+import scala.concurrent.duration._
-import akka.util.Timeout
-import akka.util.Helpers.{ toRootLowerCase, ConfigOps, Requiring }
-import akka.japi.Util._
+import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+
+import akka.ConfigurationException
import akka.actor.Props
+import akka.annotation.InternalApi
import akka.event.Logging
import akka.event.Logging.LogLevel
-import akka.ConfigurationException
-import akka.annotation.InternalApi
+import akka.japi.Util._
import akka.remote.artery.ArterySettings
-import com.github.ghik.silencer.silent
+import akka.util.Helpers.{ toRootLowerCase, ConfigOps, Requiring }
+import akka.util.Timeout
final class RemoteSettings(val config: Config) {
import config._
+
import akka.util.ccompat.JavaConverters._
val Artery = ArterySettings(getConfig("akka.remote.artery"))
diff --git a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala
index 117b9a322f..860f63cf57 100644
--- a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala
+++ b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala
@@ -4,15 +4,15 @@
package akka.remote
+import scala.collection.immutable
+import scala.concurrent.Future
+import scala.util.control.NoStackTrace
+
import akka.AkkaException
import akka.Done
import akka.actor._
import akka.annotation.InternalStableApi
import akka.event.LoggingAdapter
-
-import scala.collection.immutable
-import scala.concurrent.Future
-import scala.util.control.NoStackTrace
import akka.util.{ unused, OptionVal }
/**
diff --git a/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala b/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala
index ede5aecf4c..a727ef21aa 100644
--- a/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala
+++ b/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala
@@ -7,16 +7,17 @@ package akka.remote
import scala.collection.mutable
import scala.concurrent.duration._
+import com.github.ghik.silencer.silent
+
import akka.actor._
import akka.annotation.InternalApi
-import akka.dispatch.sysmsg.{ DeathWatchNotification, Watch }
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
+import akka.dispatch.Dispatchers
+import akka.dispatch.sysmsg.{ DeathWatchNotification, Watch }
import akka.event.AddressTerminatedTopic
import akka.remote.artery.ArteryMessage
-import akka.dispatch.Dispatchers
import akka.remote.artery.ArteryTransport
import akka.util.unused
-import com.github.ghik.silencer.silent
/**
* INTERNAL API
diff --git a/akka-remote/src/main/scala/akka/remote/Remoting.scala b/akka-remote/src/main/scala/akka/remote/Remoting.scala
index 1169e15a01..41dff1bad7 100644
--- a/akka-remote/src/main/scala/akka/remote/Remoting.scala
+++ b/akka-remote/src/main/scala/akka/remote/Remoting.scala
@@ -4,36 +4,37 @@
package akka.remote
+import java.net.URLEncoder
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.TimeoutException
+
+import scala.collection.immutable
+import scala.collection.immutable.{ HashMap, Seq }
+import scala.concurrent.{ Await, Future, Promise }
+import scala.concurrent.duration._
+import scala.util.{ Failure, Success }
+import scala.util.control.NonFatal
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+
import akka.Done
-import akka.actor.SupervisorStrategy._
import akka.actor._
+import akka.actor.ActorInitializationException
+import akka.actor.SupervisorStrategy._
+import akka.annotation.InternalStableApi
+import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
+import akka.dispatch.MessageDispatcher
import akka.event.{ Logging, LoggingAdapter }
import akka.pattern.{ ask, gracefulStop, pipe }
import akka.remote.EndpointManager._
import akka.remote.Remoting.TransportSupervisor
-import akka.remote.transport.Transport.{ ActorAssociationEventListener, AssociationEventListener, InboundAssociation }
import akka.remote.transport._
-import com.typesafe.config.Config
-import java.net.URLEncoder
-import java.util.concurrent.TimeoutException
-
-import scala.collection.immutable.{ HashMap, Seq }
-import scala.concurrent.duration._
-import scala.concurrent.{ Await, Future, Promise }
-import scala.util.control.NonFatal
-import scala.util.{ Failure, Success }
import akka.remote.transport.AkkaPduCodec.Message
-import java.util.concurrent.ConcurrentHashMap
-
-import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
+import akka.remote.transport.Transport.{ ActorAssociationEventListener, AssociationEventListener, InboundAssociation }
import akka.util.ByteString.UTF_8
import akka.util.OptionVal
-
-import scala.collection.immutable
-import akka.actor.ActorInitializationException
-import akka.annotation.InternalStableApi
import akka.util.ccompat._
-import com.github.ghik.silencer.silent
/**
* INTERNAL API
@@ -149,9 +150,9 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc
import provider.remoteSettings._
- private implicit val ec = system.dispatchers.lookup(Dispatcher)
+ private implicit val ec: MessageDispatcher = system.dispatchers.lookup(Dispatcher)
- val transportSupervisor = system.systemActorOf(configureDispatcher(Props[TransportSupervisor]), "transports")
+ val transportSupervisor = system.systemActorOf(configureDispatcher(Props[TransportSupervisor]()), "transports")
override def localAddressForRemote(remote: Address): Address =
Remoting.localAddressForRemote(transportMapping, remote)
@@ -459,17 +460,17 @@ private[remote] object EndpointManager {
def prune(): Unit = {
addressToWritable = addressToWritable.collect {
- case entry @ (_, Gated(timeOfRelease)) if timeOfRelease.hasTimeLeft =>
+ case entry @ (_, Gated(timeOfRelease)) if timeOfRelease.hasTimeLeft() =>
// Gated removed when no time left
entry
- case entry @ (_, Quarantined(_, timeOfRelease)) if timeOfRelease.hasTimeLeft =>
+ case entry @ (_, Quarantined(_, timeOfRelease)) if timeOfRelease.hasTimeLeft() =>
// Quarantined removed when no time left
entry
case entry @ (_, _: Pass) => entry
}
addressToRefuseUid = addressToRefuseUid.collect {
- case entry @ (_, (_, timeOfRelease)) if timeOfRelease.hasTimeLeft =>
+ case entry @ (_, (_, timeOfRelease)) if timeOfRelease.hasTimeLeft() =>
// // Quarantined/refuseUid removed when no time left
entry
}
diff --git a/akka-remote/src/main/scala/akka/remote/RemotingLifecycleEvent.scala b/akka-remote/src/main/scala/akka/remote/RemotingLifecycleEvent.scala
index 99c7df4e18..05595d6098 100644
--- a/akka-remote/src/main/scala/akka/remote/RemotingLifecycleEvent.scala
+++ b/akka-remote/src/main/scala/akka/remote/RemotingLifecycleEvent.scala
@@ -4,12 +4,13 @@
package akka.remote
-import akka.event.Logging.LogLevel
-import akka.event.{ Logging, LoggingAdapter }
-import akka.actor.{ ActorSystem, Address }
+import scala.runtime.AbstractFunction2
+
import com.github.ghik.silencer.silent
-import scala.runtime.AbstractFunction2
+import akka.actor.{ ActorSystem, Address }
+import akka.event.{ Logging, LoggingAdapter }
+import akka.event.Logging.LogLevel
@silent("@SerialVersionUID has no effect")
@SerialVersionUID(1L)
diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala
index d5fa667c81..9cf6e20beb 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala
@@ -6,9 +6,12 @@ package akka.remote.artery
import java.net.InetAddress
-import akka.util.ccompat.JavaConverters._
-
import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+
import akka.NotUsed
import akka.japi.Util.immutableSeq
import akka.stream.ActorMaterializerSettings
@@ -16,9 +19,7 @@ import akka.util.Helpers.ConfigOps
import akka.util.Helpers.Requiring
import akka.util.Helpers.toRootLowerCase
import akka.util.WildcardIndex
-import com.github.ghik.silencer.silent
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
+import akka.util.ccompat.JavaConverters._
/** INTERNAL API */
private[akka] final class ArterySettings private (config: Config) {
@@ -27,8 +28,8 @@ private[akka] final class ArterySettings private (config: Config) {
def withDisabledCompression(): ArterySettings =
ArterySettings(ConfigFactory.parseString("""|akka.remote.artery.advanced.compression {
- | actor-refs.max = 0
- | manifests.max = 0
+ | actor-refs.max = off
+ | manifests.max = off
|}""".stripMargin).withFallback(config))
val Enabled: Boolean = getBoolean("enabled")
@@ -242,21 +243,29 @@ private[akka] object ArterySettings {
private[remote] final class Compression private[ArterySettings] (config: Config) {
import config._
- private[akka] final val Enabled = ActorRefs.Max > 0 || Manifests.Max > 0
+ private[akka] final val Enabled = ActorRefs.Enabled || Manifests.Enabled
object ActorRefs {
val config: Config = getConfig("actor-refs")
import config._
val AdvertisementInterval: FiniteDuration = config.getMillisDuration("advertisement-interval")
- val Max: Int = getInt("max")
+ val Max: Int = toRootLowerCase(getString("max")) match {
+ case "off" => 0
+ case _ => getInt("max")
+ }
+ final val Enabled = Max > 0
}
object Manifests {
val config: Config = getConfig("manifests")
import config._
val AdvertisementInterval: FiniteDuration = config.getMillisDuration("advertisement-interval")
- val Max: Int = getInt("max")
+ val Max: Int = toRootLowerCase(getString("max")) match {
+ case "off" => 0
+ case _ => getInt("max")
+ }
+ final val Enabled = Max > 0
}
}
object Compression {
diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala
index 5c28b73c8b..78a608aec4 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala
@@ -18,11 +18,13 @@ import scala.util.Try
import scala.util.control.NoStackTrace
import scala.util.control.NonFatal
+import com.github.ghik.silencer.silent
+
import akka.Done
import akka.NotUsed
+import akka.actor._
import akka.actor.Actor
import akka.actor.Props
-import akka.actor._
import akka.annotation.InternalStableApi
import akka.dispatch.Dispatchers
import akka.event.Logging
@@ -37,8 +39,8 @@ import akka.remote.artery.Encoder.OutboundCompressionAccess
import akka.remote.artery.InboundControlJunction.ControlMessageObserver
import akka.remote.artery.InboundControlJunction.ControlMessageSubject
import akka.remote.artery.OutboundControlJunction.OutboundControlIngress
-import akka.remote.artery.compress.CompressionProtocol.CompressionMessage
import akka.remote.artery.compress._
+import akka.remote.artery.compress.CompressionProtocol.CompressionMessage
import akka.remote.transport.ThrottlerTransportAdapter.Blackhole
import akka.remote.transport.ThrottlerTransportAdapter.SetThrottle
import akka.remote.transport.ThrottlerTransportAdapter.Unthrottled
@@ -49,7 +51,6 @@ import akka.stream.scaladsl.Sink
import akka.util.OptionVal
import akka.util.WildcardIndex
import akka.util.unused
-import com.github.ghik.silencer.silent
/**
* INTERNAL API
@@ -471,7 +472,8 @@ private[remote] abstract class ArteryTransport(_system: ExtendedActorSystem, _pr
bindAddress.uid)
else {
log.info(
- s"Remoting started with transport [Artery ${settings.Transport}]; listening on address [{}] and bound to [{}] with UID [{}]",
+ "Remoting started with transport [Artery {}]; listening on address [{}] and bound to [{}] with UID [{}]",
+ settings.Transport,
localAddress.address,
bindAddress.address,
localAddress.uid)
diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala
index 2f3919842e..63d5e6a0bd 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala
@@ -17,6 +17,9 @@ import scala.concurrent.Future
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
+import com.github.ghik.silencer.silent
+import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
+
import akka.Done
import akka.NotUsed
import akka.actor.ActorRef
@@ -53,8 +56,6 @@ import akka.util.PrettyDuration._
import akka.util.Unsafe
import akka.util.WildcardIndex
import akka.util.ccompat._
-import com.github.ghik.silencer.silent
-import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
/**
* INTERNAL API
diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala
index 7da4099870..f505c89cf0 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala
@@ -6,28 +6,29 @@ package akka.remote.artery
import java.util.concurrent.TimeUnit
-import scala.concurrent.duration._
import scala.concurrent.{ Future, Promise }
+import scala.concurrent.duration._
import scala.util.control.NonFatal
+
import akka.Done
import akka.actor.{ EmptyLocalActorRef, _ }
import akka.event.Logging
import akka.pattern.PromiseActorRef
+import akka.remote.{ MessageSerializer, OversizedPayloadException, RemoteActorRefProvider, UniqueAddress }
import akka.remote.artery.Decoder.{
AdvertiseActorRefsCompressionTable,
AdvertiseClassManifestsCompressionTable,
InboundCompressionAccess,
InboundCompressionAccessImpl
}
+import akka.remote.artery.OutboundHandshake.HandshakeReq
import akka.remote.artery.SystemMessageDelivery.SystemMessageEnvelope
-import akka.remote.artery.compress.CompressionProtocol._
import akka.remote.artery.compress._
-import akka.remote.{ MessageSerializer, OversizedPayloadException, RemoteActorRefProvider, UniqueAddress }
+import akka.remote.artery.compress.CompressionProtocol._
import akka.serialization.{ Serialization, SerializationExtension, Serializers }
import akka.stream._
import akka.stream.stage._
import akka.util.{ unused, OptionVal, Unsafe }
-import akka.remote.artery.OutboundHandshake.HandshakeReq
/**
* INTERNAL API
@@ -165,14 +166,26 @@ private[remote] class Encoder(
Logging.messageClassName(outboundEnvelope.message))
throw e
case _ if e.isInstanceOf[java.nio.BufferOverflowException] =>
- val reason = new OversizedPayloadException(
- "Discarding oversized payload sent to " +
+ val reasonText = "Discarding oversized payload sent to " +
s"${outboundEnvelope.recipient}: max allowed size ${envelope.byteBuffer.limit()} " +
- s"bytes. Message type [${Logging.messageClassName(outboundEnvelope.message)}].")
+ s"bytes. Message type [${Logging.messageClassName(outboundEnvelope.message)}]."
log.error(
- reason,
+ new OversizedPayloadException(reasonText),
"Failed to serialize oversized message [{}].",
Logging.messageClassName(outboundEnvelope.message))
+ system.eventStream.publish(outboundEnvelope.sender match {
+ case OptionVal.Some(msgSender) =>
+ Dropped(
+ outboundEnvelope.message,
+ reasonText,
+ msgSender,
+ outboundEnvelope.recipient.getOrElse(ActorRef.noSender))
+ case OptionVal.None =>
+ Dropped(
+ outboundEnvelope.message,
+ reasonText,
+ outboundEnvelope.recipient.getOrElse(ActorRef.noSender))
+ })
pull(in)
case _ =>
log.error(e, "Failed to serialize message [{}].", Logging.messageClassName(outboundEnvelope.message))
@@ -304,7 +317,7 @@ private[remote] object Decoder {
* External call from ChangeInboundCompression materialized value
*/
override def currentCompressionOriginUids: Future[Set[Long]] = {
- val p = Promise[Set[Long]]
+ val p = Promise[Set[Long]]()
currentCompressionOriginUidsCb.invoke(p)
p.future
}
@@ -387,16 +400,13 @@ private[remote] class Decoder(
val tickDelay = 1.seconds
scheduleWithFixedDelay(Tick, tickDelay, tickDelay)
- if (settings.Advanced.Compression.Enabled) {
- settings.Advanced.Compression.ActorRefs.AdvertisementInterval match {
- case d: FiniteDuration => scheduleWithFixedDelay(AdvertiseActorRefsCompressionTable, d, d)
- case _ => // not advertising actor ref compressions
- }
- settings.Advanced.Compression.Manifests.AdvertisementInterval match {
- case d: FiniteDuration =>
- scheduleWithFixedDelay(AdvertiseClassManifestsCompressionTable, d, d)
- case _ => // not advertising class manifest compressions
- }
+ if (settings.Advanced.Compression.ActorRefs.Enabled) {
+ val d = settings.Advanced.Compression.ActorRefs.AdvertisementInterval
+ scheduleWithFixedDelay(AdvertiseActorRefsCompressionTable, d, d)
+ }
+ if (settings.Advanced.Compression.Manifests.Enabled) {
+ val d = settings.Advanced.Compression.Manifests.AdvertisementInterval
+ scheduleWithFixedDelay(AdvertiseClassManifestsCompressionTable, d, d)
}
}
override def onPush(): Unit =
diff --git a/akka-remote/src/main/scala/akka/remote/artery/Control.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala
index 771833b6b2..38b3b93e5e 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/Control.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala
@@ -11,14 +11,14 @@ import scala.concurrent.Promise
import scala.util.Try
import akka.Done
+import akka.event.Logging
+import akka.remote.UniqueAddress
import akka.stream.Attributes
import akka.stream.FlowShape
import akka.stream.Inlet
import akka.stream.Outlet
import akka.stream.stage._
-import akka.remote.UniqueAddress
import akka.util.OptionVal
-import akka.event.Logging
/** INTERNAL API: marker trait for protobuf-serializable artery messages */
private[remote] trait ArteryMessage extends Serializable
diff --git a/akka-remote/src/main/scala/akka/remote/artery/EnvelopeBufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/EnvelopeBufferPool.scala
index 676881d674..3dc46e4bdd 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/EnvelopeBufferPool.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/EnvelopeBufferPool.scala
@@ -6,14 +6,14 @@ package akka.remote.artery
import java.nio.{ ByteBuffer, ByteOrder }
+import org.agrona.concurrent.{ ManyToManyConcurrentArrayQueue, UnsafeBuffer }
+
import akka.actor.ActorRef
import akka.io.DirectByteBufferPool
import akka.remote.artery.compress.{ CompressionTable, InboundCompressions, NoInboundCompressions }
import akka.serialization.Serialization
import akka.util.{ OptionVal, Unsafe }
-import org.agrona.concurrent.{ ManyToManyConcurrentArrayQueue, UnsafeBuffer }
-
/**
* INTERNAL API
*/
diff --git a/akka-remote/src/main/scala/akka/remote/artery/FixedSizePartitionHub.scala b/akka-remote/src/main/scala/akka/remote/artery/FixedSizePartitionHub.scala
index 698791a948..4cfbaff7f6 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/FixedSizePartitionHub.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/FixedSizePartitionHub.scala
@@ -4,9 +4,10 @@
package akka.remote.artery
+import org.agrona.concurrent.OneToOneConcurrentArrayQueue
+
import akka.annotation.InternalApi
import akka.stream.scaladsl.PartitionHub
-import org.agrona.concurrent.OneToOneConcurrentArrayQueue
/**
* INTERNAL API
diff --git a/akka-remote/src/main/scala/akka/remote/artery/ImmutableLongMap.scala b/akka-remote/src/main/scala/akka/remote/artery/ImmutableLongMap.scala
index 42fa03f950..92e1074a53 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/ImmutableLongMap.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/ImmutableLongMap.scala
@@ -4,11 +4,13 @@
package akka.remote.artery
-import scala.annotation.tailrec
-import akka.util.OptionVal
-import scala.reflect.ClassTag
import java.util.Arrays
+
+import scala.annotation.tailrec
+import scala.reflect.ClassTag
+
import akka.util.HashCode
+import akka.util.OptionVal
/**
* INTERNAL API
diff --git a/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala
index 192113e502..9cd704d300 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala
@@ -4,10 +4,10 @@
package akka.remote.artery
-import akka.actor.InternalActorRef
-import akka.util.OptionVal
import akka.actor.ActorRef
+import akka.actor.InternalActorRef
import akka.actor.NoSerializationVerificationNeeded
+import akka.util.OptionVal
/**
* INTERNAL API
diff --git a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala
index 438c5c13e3..0a8748e26c 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala
@@ -4,16 +4,16 @@
package akka.remote.artery
+import akka.actor.ActorSelectionMessage
+import akka.event.Logging
+import akka.remote.HeartbeatMessage
+import akka.remote.UniqueAddress
import akka.stream.Attributes
import akka.stream.FlowShape
import akka.stream.Inlet
import akka.stream.Outlet
import akka.stream.stage._
-import akka.remote.UniqueAddress
import akka.util.OptionVal
-import akka.event.Logging
-import akka.remote.HeartbeatMessage
-import akka.actor.ActorSelectionMessage
/**
* INTERNAL API
diff --git a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala
index fbb3e7b1a1..1bc647f144 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala
@@ -27,8 +27,8 @@ private[remote] class MessageDispatcher(system: ExtendedActorSystem, provider: R
private val debugLogEnabled: Boolean = log.isDebugEnabled
def dispatch(inboundEnvelope: InboundEnvelope): Unit = {
- import provider.remoteSettings.Artery._
import Logging.messageClassName
+ import provider.remoteSettings.Artery._
val recipient = inboundEnvelope.recipient.get
val message = inboundEnvelope.message
diff --git a/akka-remote/src/main/scala/akka/remote/artery/RemoteInstrument.scala b/akka-remote/src/main/scala/akka/remote/artery/RemoteInstrument.scala
index 0755b64d92..ed31509434 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/RemoteInstrument.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/RemoteInstrument.scala
@@ -9,6 +9,7 @@ import java.nio.ByteBuffer
import scala.annotation.tailrec
import scala.collection.immutable
import scala.util.control.NonFatal
+
import akka.actor.{ ActorRef, ExtendedActorSystem }
import akka.annotation.InternalStableApi
import akka.event.{ Logging, LoggingAdapter }
diff --git a/akka-remote/src/main/scala/akka/remote/artery/RemotingFlightRecorder.scala b/akka-remote/src/main/scala/akka/remote/artery/RemotingFlightRecorder.scala
index d468dcf060..cb4a1c5ff0 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/RemotingFlightRecorder.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/RemotingFlightRecorder.scala
@@ -6,6 +6,9 @@ package akka.remote.artery
import java.net.InetSocketAddress
+import scala.util.Failure
+import scala.util.Success
+
import akka.actor.Address
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
@@ -15,9 +18,6 @@ import akka.annotation.InternalApi
import akka.remote.UniqueAddress
import akka.util.JavaVersion
-import scala.util.Failure
-import scala.util.Success
-
/**
* INTERNAL API
*/
diff --git a/akka-remote/src/main/scala/akka/remote/artery/RestartCounter.scala b/akka-remote/src/main/scala/akka/remote/artery/RestartCounter.scala
index b5c56c7a84..aa1f7879e1 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/RestartCounter.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/RestartCounter.scala
@@ -4,10 +4,11 @@
package akka.remote.artery
-import scala.concurrent.duration.Deadline
import java.util.concurrent.atomic.AtomicReference
-import scala.concurrent.duration.FiniteDuration
+
import scala.annotation.tailrec
+import scala.concurrent.duration.Deadline
+import scala.concurrent.duration.FiniteDuration
/**
* INTERNAL API
diff --git a/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala b/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala
index d2ee9011ad..b4ad17cdfb 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala
@@ -6,18 +6,18 @@ package akka.remote.artery
import java.util.Queue
-import akka.stream.stage.OutHandler
+import scala.annotation.tailrec
+import scala.concurrent.Promise
+import scala.util.Failure
+import scala.util.Success
+import scala.util.Try
+
import akka.stream.Attributes
import akka.stream.Outlet
import akka.stream.SourceShape
import akka.stream.stage.GraphStageLogic
import akka.stream.stage.GraphStageWithMaterializedValue
-
-import scala.annotation.tailrec
-import scala.concurrent.Promise
-import scala.util.Try
-import scala.util.Success
-import scala.util.Failure
+import akka.stream.stage.OutHandler
/**
* INTERNAL API
diff --git a/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala
index 3c735ec2e8..9c365ff8ef 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala
@@ -7,7 +7,9 @@ package akka.remote.artery
import java.util.concurrent.atomic.AtomicReference
import scala.annotation.tailrec
+
import akka.actor.Address
+import akka.event.Logging
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.stream.Attributes
import akka.stream.FlowShape
@@ -15,7 +17,6 @@ import akka.stream.Inlet
import akka.stream.Outlet
import akka.stream.stage._
import akka.util.OptionVal
-import akka.event.Logging
object TestManagementCommands {
diff --git a/akka-remote/src/main/scala/akka/remote/artery/aeron/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/aeron/AeronSink.scala
index cf796e3a01..8169c710fa 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/aeron/AeronSink.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/aeron/AeronSink.scala
@@ -5,7 +5,6 @@
package akka.remote.artery
package aeron
-import akka.util.PrettyDuration.PrettyPrintableDuration
import scala.annotation.tailrec
import scala.concurrent.Future
import scala.concurrent.Promise
@@ -14,6 +13,12 @@ import scala.util.Failure
import scala.util.Success
import scala.util.Try
import scala.util.control.NoStackTrace
+
+import io.aeron.Aeron
+import io.aeron.Publication
+import org.agrona.concurrent.UnsafeBuffer
+import org.agrona.hints.ThreadHints
+
import akka.Done
import akka.stream.Attributes
import akka.stream.Inlet
@@ -22,10 +27,7 @@ import akka.stream.stage.AsyncCallback
import akka.stream.stage.GraphStageLogic
import akka.stream.stage.GraphStageWithMaterializedValue
import akka.stream.stage.InHandler
-import io.aeron.Aeron
-import io.aeron.Publication
-import org.agrona.concurrent.UnsafeBuffer
-import org.agrona.hints.ThreadHints
+import akka.util.PrettyDuration.PrettyPrintableDuration
/**
* INTERNAL API
diff --git a/akka-remote/src/main/scala/akka/remote/artery/aeron/AeronSource.scala b/akka-remote/src/main/scala/akka/remote/artery/aeron/AeronSource.scala
index 60b72006fb..7a1d16244d 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/aeron/AeronSource.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/aeron/AeronSource.scala
@@ -6,24 +6,24 @@ package akka.remote.artery
package aeron
import scala.annotation.tailrec
+import scala.concurrent.{ Future, Promise }
+import scala.util.control.NonFatal
+
+import io.aeron.{ Aeron, FragmentAssembler, Subscription }
+import io.aeron.exceptions.DriverTimeoutException
+import io.aeron.logbuffer.FragmentHandler
+import io.aeron.logbuffer.Header
+import org.agrona.DirectBuffer
+import org.agrona.hints.ThreadHints
+
import akka.stream.Attributes
import akka.stream.Outlet
import akka.stream.SourceShape
import akka.stream.stage.AsyncCallback
import akka.stream.stage.GraphStageLogic
-import akka.stream.stage.OutHandler
-import io.aeron.{ Aeron, FragmentAssembler, Subscription }
-import io.aeron.logbuffer.FragmentHandler
-import io.aeron.logbuffer.Header
-import org.agrona.DirectBuffer
-import org.agrona.hints.ThreadHints
import akka.stream.stage.GraphStageWithMaterializedValue
-
-import scala.util.control.NonFatal
+import akka.stream.stage.OutHandler
import akka.stream.stage.StageLogging
-import io.aeron.exceptions.DriverTimeoutException
-
-import scala.concurrent.{ Future, Promise }
/**
* INTERNAL API
@@ -35,7 +35,7 @@ private[remote] object AeronSource {
handler: MessageHandler,
onMessage: AsyncCallback[EnvelopeBuffer]): () => Boolean = { () =>
{
- handler.reset
+ handler.reset()
sub.poll(handler.fragmentsHandler, 1)
val msg = handler.messageReceived
handler.reset() // for GC
@@ -167,7 +167,7 @@ private[remote] class AeronSource(
}
override def channelEndpointStatus(): Future[Long] = {
- val promise = Promise[Long]
+ val promise = Promise[Long]()
getStatusCb.invoke(promise)
promise.future
}
diff --git a/akka-remote/src/main/scala/akka/remote/artery/aeron/ArteryAeronUdpTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/aeron/ArteryAeronUdpTransport.scala
index e156553ea8..0062d15850 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/aeron/ArteryAeronUdpTransport.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/aeron/ArteryAeronUdpTransport.scala
@@ -16,20 +16,7 @@ import scala.collection.immutable
import scala.concurrent.{ Await, Future }
import scala.concurrent.duration._
import scala.util.control.NonFatal
-import akka.Done
-import akka.actor.Address
-import akka.actor.Cancellable
-import akka.actor.ExtendedActorSystem
-import akka.event.Logging
-import akka.remote.RemoteActorRefProvider
-import akka.remote.RemoteTransportException
-import akka.remote.artery.compress._
-import akka.stream.KillSwitches
-import akka.stream.scaladsl.Flow
-import akka.stream.scaladsl.Keep
-import akka.stream.scaladsl.Sink
-import akka.stream.scaladsl.Source
-import akka.util.ccompat._
+
import io.aeron.Aeron
import io.aeron.AvailableImageHandler
import io.aeron.CncFileDescriptor
@@ -47,6 +34,21 @@ import org.agrona.IoUtil
import org.agrona.concurrent.BackoffIdleStrategy
import org.agrona.concurrent.status.CountersReader.MetaData
+import akka.Done
+import akka.actor.Address
+import akka.actor.Cancellable
+import akka.actor.ExtendedActorSystem
+import akka.event.Logging
+import akka.remote.RemoteActorRefProvider
+import akka.remote.RemoteTransportException
+import akka.remote.artery.compress._
+import akka.stream.KillSwitches
+import akka.stream.scaladsl.Flow
+import akka.stream.scaladsl.Keep
+import akka.stream.scaladsl.Sink
+import akka.stream.scaladsl.Source
+import akka.util.ccompat._
+
/**
* INTERNAL API
*/
@@ -464,8 +466,8 @@ private[remote] class ArteryAeronUdpTransport(_system: ExtendedActorSystem, _pro
}
def autoSelectPort(hostname: String): Int = {
- import java.nio.channels.DatagramChannel
import java.net.InetSocketAddress
+ import java.nio.channels.DatagramChannel
val socket = DatagramChannel.open().socket()
socket.bind(new InetSocketAddress(hostname, 0))
diff --git a/akka-remote/src/main/scala/akka/remote/artery/aeron/TaskRunner.scala b/akka-remote/src/main/scala/akka/remote/artery/aeron/TaskRunner.scala
index 99ec887347..475e0ce343 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/aeron/TaskRunner.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/aeron/TaskRunner.scala
@@ -7,17 +7,18 @@ package aeron
import java.util.concurrent.TimeUnit.{ MICROSECONDS, MILLISECONDS }
-import akka.Done
-import akka.actor.ExtendedActorSystem
-import akka.dispatch.{ AbstractNodeQueue, MonitorableThreadFactory }
-import akka.event.Logging
-import org.agrona.concurrent.{ BackoffIdleStrategy, BusySpinIdleStrategy, IdleStrategy, SleepingIdleStrategy }
-
import scala.annotation.tailrec
import scala.concurrent.{ Future, Promise }
import scala.reflect.ClassTag
import scala.util.control.NonFatal
+import org.agrona.concurrent.{ BackoffIdleStrategy, BusySpinIdleStrategy, IdleStrategy, SleepingIdleStrategy }
+
+import akka.Done
+import akka.actor.ExtendedActorSystem
+import akka.dispatch.{ AbstractNodeQueue, MonitorableThreadFactory }
+import akka.event.Logging
+
/**
* INTERNAL API
*/
diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala
index 7f5695b77b..2908f5e719 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala
@@ -7,10 +7,11 @@ package akka.remote.artery.compress
import java.util
import java.util.Comparator
-import akka.util.HashCode
import org.agrona.collections.Hashing
import org.agrona.collections.Object2IntHashMap
+import akka.util.HashCode
+
/**
* INTERNAL API: Versioned compression table to be advertised between systems
*
diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala
index 73d5a4d3d6..7a57c9c3ac 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala
@@ -7,13 +7,15 @@ package akka.remote.artery.compress
import java.util.function.LongFunction
import scala.annotation.tailrec
+
+import org.agrona.collections.Long2ObjectHashMap
+
import akka.actor.{ ActorRef, ActorSystem, Address, InternalActorRef }
import akka.event.Logging
import akka.event.LoggingAdapter
import akka.pattern.PromiseActorRef
import akka.remote.artery._
import akka.util.{ unused, OptionVal }
-import org.agrona.collections.Long2ObjectHashMap
/**
* INTERNAL API
diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/TopHeavyHitters.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/TopHeavyHitters.scala
index 2e62fd17b9..d829d9bf60 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/compress/TopHeavyHitters.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/compress/TopHeavyHitters.scala
@@ -25,9 +25,12 @@ import scala.reflect.ClassTag
*/
private[remote] final class TopHeavyHitters[T >: Null](val max: Int)(implicit classTag: ClassTag[T]) { self =>
- require((max & (max - 1)) == 0, "Maximum numbers of heavy hitters should be in form of 2^k for any natural k")
+ private val adjustedMax = if (max == 0) 1 else max // need at least one
+ require(
+ (adjustedMax & (adjustedMax - 1)) == 0,
+ "Maximum numbers of heavy hitters should be in form of 2^k for any natural k")
- val capacity = max * 2
+ val capacity = adjustedMax * 2
val mask = capacity - 1
import TopHeavyHitters._
@@ -44,7 +47,7 @@ private[remote] final class TopHeavyHitters[T >: Null](val max: Int)(implicit cl
private[this] val weights: Array[Long] = new Array(capacity)
// Heap structure containing indices to slots in the hashmap
- private[this] val heap: Array[Int] = Array.fill(max)(-1)
+ private[this] val heap: Array[Int] = Array.fill(adjustedMax)(-1)
/*
* Invariants (apart from heap and hashmap invariants):
@@ -104,8 +107,10 @@ private[remote] final class TopHeavyHitters[T >: Null](val max: Int)(implicit cl
new Iterator[T] {
var i = 0
- @tailrec override final def hasNext: Boolean =
+ @tailrec override final def hasNext: Boolean = {
+ // note that this is using max and not adjustedMax so will be empty if disabled (max=0)
(i < self.max) && ((value != null) || { next(); hasNext })
+ }
override final def next(): T = {
val v = value
@@ -258,7 +263,7 @@ private[remote] final class TopHeavyHitters[T >: Null](val max: Int)(implicit cl
val leftIndex = index * 2 + 1
val rightIndex = index * 2 + 2
val currentWeight: Long = weights(heap(index))
- if (rightIndex < max) {
+ if (rightIndex < adjustedMax) {
val leftValueIndex: Int = heap(leftIndex)
val rightValueIndex: Int = heap(rightIndex)
if (leftValueIndex < 0) {
@@ -282,7 +287,7 @@ private[remote] final class TopHeavyHitters[T >: Null](val max: Int)(implicit cl
}
}
}
- } else if (leftIndex < max) {
+ } else if (leftIndex < adjustedMax) {
val leftValueIndex: Int = heap(leftIndex)
if (leftValueIndex < 0) {
swapHeapNode(index, leftIndex)
diff --git a/akka-remote/src/main/scala/akka/remote/artery/tcp/ArteryTcpTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/tcp/ArteryTcpTransport.scala
index 487c262da3..4397b88458 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/tcp/ArteryTcpTransport.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/tcp/ArteryTcpTransport.scala
@@ -72,8 +72,8 @@ private[remote] class ArteryTcpTransport(
_provider: RemoteActorRefProvider,
tlsEnabled: Boolean)
extends ArteryTransport(_system, _provider) {
- import ArteryTransport._
import ArteryTcpTransport._
+ import ArteryTransport._
override type LifeCycle = NotUsed
diff --git a/akka-remote/src/main/scala/akka/remote/artery/tcp/SSLEngineProvider.scala b/akka-remote/src/main/scala/akka/remote/artery/tcp/SSLEngineProvider.scala
index aa73eaba09..29767e1f1d 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/tcp/SSLEngineProvider.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/tcp/SSLEngineProvider.scala
@@ -12,9 +12,18 @@ import java.nio.file.Paths
import java.security.GeneralSecurityException
import java.security.KeyStore
import java.security.SecureRandom
+import javax.net.ssl.KeyManager
+import javax.net.ssl.KeyManagerFactory
+import javax.net.ssl.SSLContext
+import javax.net.ssl.SSLEngine
+import javax.net.ssl.SSLSession
+import javax.net.ssl.TrustManager
+import javax.net.ssl.TrustManagerFactory
import scala.util.Try
+import com.typesafe.config.Config
+
import akka.actor.ActorSystem
import akka.actor.ExtendedActorSystem
import akka.actor.setup.Setup
@@ -25,14 +34,6 @@ import akka.event.MarkerLoggingAdapter
import akka.japi.Util.immutableSeq
import akka.stream.TLSRole
import akka.util.ccompat._
-import com.typesafe.config.Config
-import javax.net.ssl.KeyManager
-import javax.net.ssl.KeyManagerFactory
-import javax.net.ssl.SSLContext
-import javax.net.ssl.SSLEngine
-import javax.net.ssl.SSLSession
-import javax.net.ssl.TrustManager
-import javax.net.ssl.TrustManagerFactory
@ccompatUsedUntil213
trait SSLEngineProvider {
diff --git a/akka-remote/src/main/scala/akka/remote/routing/RemoteRouterConfig.scala b/akka-remote/src/main/scala/akka/remote/routing/RemoteRouterConfig.scala
index b43ab858ae..c7937878c3 100644
--- a/akka-remote/src/main/scala/akka/remote/routing/RemoteRouterConfig.scala
+++ b/akka-remote/src/main/scala/akka/remote/routing/RemoteRouterConfig.scala
@@ -6,6 +6,9 @@ package akka.remote.routing
import java.util.concurrent.atomic.AtomicInteger
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ActorCell
import akka.actor.ActorContext
import akka.actor.ActorSystem
@@ -22,8 +25,6 @@ import akka.routing.Routee
import akka.routing.Router
import akka.routing.RouterActor
import akka.routing.RouterConfig
-import com.github.ghik.silencer.silent
-import com.typesafe.config.ConfigFactory
/**
* [[akka.routing.RouterConfig]] implementation for remote deployment on defined
@@ -53,7 +54,7 @@ final case class RemoteRouterConfig(local: Pool, nodes: Iterable[Address]) exten
val deploy = Deploy(
config = ConfigFactory.empty(),
routerConfig = routeeProps.routerConfig,
- scope = RemoteScope(nodeAddressIter.next))
+ scope = RemoteScope(nodeAddressIter.next()))
// attachChild means that the provider will treat this call as if possibly done out of the wrong
// context and use RepointableActorRef instead of LocalActorRef. Seems like a slightly sub-optimal
diff --git a/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala
index 2560d0eaa1..f035f08272 100644
--- a/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala
+++ b/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala
@@ -8,12 +8,12 @@ import java.io.NotSerializableException
import akka.actor.{ ActorRef, Address, ExtendedActorSystem }
import akka.protobufv3.internal.MessageLite
-import akka.remote.RemoteWatcher.ArteryHeartbeatRsp
-import akka.remote.artery.OutboundHandshake.{ HandshakeReq, HandshakeRsp }
-import akka.remote.artery.compress.CompressionProtocol._
-import akka.remote.artery.compress.{ CompressionProtocol, CompressionTable }
-import akka.remote.artery.{ ActorSystemTerminating, ActorSystemTerminatingAck, Quarantined, SystemMessageDelivery }
import akka.remote._
+import akka.remote.RemoteWatcher.ArteryHeartbeatRsp
+import akka.remote.artery.{ ActorSystemTerminating, ActorSystemTerminatingAck, Quarantined, SystemMessageDelivery }
+import akka.remote.artery.OutboundHandshake.{ HandshakeReq, HandshakeRsp }
+import akka.remote.artery.compress.{ CompressionProtocol, CompressionTable }
+import akka.remote.artery.compress.CompressionProtocol._
import akka.serialization.{ BaseSerializer, Serialization, SerializationExtension, SerializerWithStringManifest }
/** INTERNAL API */
diff --git a/akka-remote/src/main/scala/akka/remote/serialization/DaemonMsgCreateSerializer.scala b/akka-remote/src/main/scala/akka/remote/serialization/DaemonMsgCreateSerializer.scala
index af1f682de0..0f393f796f 100644
--- a/akka-remote/src/main/scala/akka/remote/serialization/DaemonMsgCreateSerializer.scala
+++ b/akka-remote/src/main/scala/akka/remote/serialization/DaemonMsgCreateSerializer.scala
@@ -5,18 +5,18 @@
package akka.remote.serialization
import scala.collection.immutable
+import scala.reflect.ClassTag
+
+import com.typesafe.config.{ Config, ConfigFactory }
+import util.{ Failure, Success }
-import akka.serialization.{ BaseSerializer, SerializationExtension, SerializerWithStringManifest }
-import akka.protobufv3.internal.ByteString
import akka.actor.{ Deploy, ExtendedActorSystem, NoScopeGiven, Props, Scope }
+import akka.protobufv3.internal.ByteString
import akka.remote.DaemonMsgCreate
import akka.remote.WireFormats.{ DaemonMsgCreateData, DeployData, PropsData }
import akka.routing.{ NoRouter, RouterConfig }
-import com.typesafe.config.{ Config, ConfigFactory }
+import akka.serialization.{ BaseSerializer, SerializationExtension, SerializerWithStringManifest }
import akka.util.ccompat._
-
-import scala.reflect.ClassTag
-import util.{ Failure, Success }
import akka.util.ccompat.JavaConverters._
/**
@@ -29,9 +29,9 @@ import akka.util.ccompat.JavaConverters._
*/
@ccompatUsedUntil213
private[akka] final class DaemonMsgCreateSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
- import ProtobufSerializer.serializeActorRef
- import ProtobufSerializer.deserializeActorRef
import Deploy.NoDispatcherGiven
+ import ProtobufSerializer.deserializeActorRef
+ import ProtobufSerializer.serializeActorRef
private lazy val serialization = SerializationExtension(system)
diff --git a/akka-remote/src/main/scala/akka/remote/serialization/MessageContainerSerializer.scala b/akka-remote/src/main/scala/akka/remote/serialization/MessageContainerSerializer.scala
index f71279d2a1..7f0f7a1e8d 100644
--- a/akka-remote/src/main/scala/akka/remote/serialization/MessageContainerSerializer.scala
+++ b/akka-remote/src/main/scala/akka/remote/serialization/MessageContainerSerializer.scala
@@ -5,13 +5,14 @@
package akka.remote.serialization
import scala.collection.immutable
-import akka.protobufv3.internal.ByteString
+
import akka.actor.ActorSelectionMessage
import akka.actor.ExtendedActorSystem
import akka.actor.SelectChildName
import akka.actor.SelectChildPattern
import akka.actor.SelectParent
import akka.actor.SelectionPathElement
+import akka.protobufv3.internal.ByteString
import akka.remote.ContainerFormats
import akka.serialization.{ BaseSerializer, SerializationExtension, Serializers }
import akka.util.ccompat._
diff --git a/akka-remote/src/main/scala/akka/remote/serialization/MiscMessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/serialization/MiscMessageSerializer.scala
index 45458d2022..89ac982fd0 100644
--- a/akka-remote/src/main/scala/akka/remote/serialization/MiscMessageSerializer.scala
+++ b/akka-remote/src/main/scala/akka/remote/serialization/MiscMessageSerializer.scala
@@ -9,17 +9,19 @@ import java.nio.charset.StandardCharsets
import java.util.Optional
import java.util.concurrent.TimeUnit
+import scala.concurrent.duration.{ FiniteDuration, TimeUnit }
+
+import com.typesafe.config.{ Config, ConfigFactory, ConfigRenderOptions }
+
import akka.{ Done, NotUsed }
import akka.actor._
import akka.dispatch.Dispatchers
+import akka.remote._
import akka.remote.WireFormats.AddressData
import akka.remote.routing.RemoteRouterConfig
-import akka.remote._
import akka.routing._
import akka.serialization.{ BaseSerializer, Serialization, SerializationExtension, SerializerWithStringManifest }
-import com.typesafe.config.{ Config, ConfigFactory, ConfigRenderOptions }
import akka.util.ccompat.JavaConverters._
-import scala.concurrent.duration.{ FiniteDuration, TimeUnit }
class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer {
diff --git a/akka-remote/src/main/scala/akka/remote/serialization/ProtobufSerializer.scala b/akka-remote/src/main/scala/akka/remote/serialization/ProtobufSerializer.scala
index 6e8b5c1e95..1cfe7317ae 100644
--- a/akka-remote/src/main/scala/akka/remote/serialization/ProtobufSerializer.scala
+++ b/akka-remote/src/main/scala/akka/remote/serialization/ProtobufSerializer.scala
@@ -7,14 +7,14 @@ package akka.remote.serialization
import java.lang.reflect.Method
import java.util.concurrent.atomic.AtomicReference
-import akka.actor.{ ActorRef, ExtendedActorSystem }
-import akka.remote.WireFormats.ActorRefData
-import akka.serialization.{ BaseSerializer, Serialization }
import scala.annotation.tailrec
import scala.util.control.NonFatal
+import akka.actor.{ ActorRef, ExtendedActorSystem }
import akka.event.LogMarker
import akka.event.Logging
+import akka.remote.WireFormats.ActorRefData
+import akka.serialization.{ BaseSerializer, Serialization }
import akka.serialization.SerializationExtension
object ProtobufSerializer {
diff --git a/akka-remote/src/main/scala/akka/remote/serialization/WrappedPayloadSupport.scala b/akka-remote/src/main/scala/akka/remote/serialization/WrappedPayloadSupport.scala
index 5da837c9c6..d8f4135951 100644
--- a/akka-remote/src/main/scala/akka/remote/serialization/WrappedPayloadSupport.scala
+++ b/akka-remote/src/main/scala/akka/remote/serialization/WrappedPayloadSupport.scala
@@ -6,9 +6,9 @@ package akka.remote.serialization
import akka.actor.ExtendedActorSystem
import akka.event.Logging
+import akka.protobufv3.internal.ByteString
import akka.remote.ContainerFormats
import akka.serialization.{ SerializationExtension, Serializers }
-import akka.protobufv3.internal.ByteString
import akka.serialization.DisabledJavaSerializer
/**
diff --git a/akka-remote/src/main/scala/akka/remote/transport/AbstractTransportAdapter.scala b/akka-remote/src/main/scala/akka/remote/transport/AbstractTransportAdapter.scala
index 9ce2c6dd76..6136b6dabc 100644
--- a/akka-remote/src/main/scala/akka/remote/transport/AbstractTransportAdapter.scala
+++ b/akka-remote/src/main/scala/akka/remote/transport/AbstractTransportAdapter.scala
@@ -4,18 +4,19 @@
package akka.remote.transport
-import akka.actor._
-import akka.pattern.{ ask, gracefulStop, pipe }
-import akka.remote.Remoting.RegisterTransportActor
-import akka.remote.transport.Transport._
-import akka.remote.RARP
-import akka.util.Timeout
import scala.collection.immutable
-import scala.concurrent.duration._
import scala.concurrent.{ ExecutionContext, Future, Promise }
-import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
-import akka.remote.transport.AssociationHandle.DisassociateInfo
+import scala.concurrent.duration._
+
+import akka.actor._
import akka.actor.DeadLetterSuppression
+import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
+import akka.pattern.{ ask, gracefulStop, pipe }
+import akka.remote.RARP
+import akka.remote.Remoting.RegisterTransportActor
+import akka.remote.transport.AssociationHandle.DisassociateInfo
+import akka.remote.transport.Transport._
+import akka.util.Timeout
@deprecated("Classic remoting is deprecated, use Artery", "2.6.0")
trait TransportAdapterProvider {
@@ -160,7 +161,7 @@ object ActorTransportAdapter {
extends TransportOperation
with DeadLetterSuppression
- implicit val AskTimeout = Timeout(5.seconds)
+ implicit val AskTimeout: Timeout = Timeout(5.seconds)
}
@deprecated("Classic remoting is deprecated, use Artery", "2.6.0")
diff --git a/akka-remote/src/main/scala/akka/remote/transport/AkkaPduCodec.scala b/akka-remote/src/main/scala/akka/remote/transport/AkkaPduCodec.scala
index 269b4108e1..7afdb41b6a 100644
--- a/akka-remote/src/main/scala/akka/remote/transport/AkkaPduCodec.scala
+++ b/akka-remote/src/main/scala/akka/remote/transport/AkkaPduCodec.scala
@@ -4,15 +4,16 @@
package akka.remote.transport
+import com.github.ghik.silencer.silent
+
import akka.AkkaException
import akka.actor.{ ActorRef, Address, AddressFromURIString, InternalActorRef }
-import akka.remote.WireFormats._
-import akka.remote._
-import akka.util.ByteString
-import akka.protobufv3.internal.InvalidProtocolBufferException
import akka.protobufv3.internal.{ ByteString => PByteString }
+import akka.protobufv3.internal.InvalidProtocolBufferException
+import akka.remote._
+import akka.remote.WireFormats._
+import akka.util.ByteString
import akka.util.OptionVal
-import com.github.ghik.silencer.silent
/**
* INTERNAL API
diff --git a/akka-remote/src/main/scala/akka/remote/transport/AkkaProtocolTransport.scala b/akka-remote/src/main/scala/akka/remote/transport/AkkaProtocolTransport.scala
index a7ff6b7fa4..1f8cdedce7 100644
--- a/akka-remote/src/main/scala/akka/remote/transport/AkkaProtocolTransport.scala
+++ b/akka-remote/src/main/scala/akka/remote/transport/AkkaProtocolTransport.scala
@@ -6,8 +6,18 @@ package akka.remote.transport
import java.util.concurrent.TimeoutException
-import akka.actor.SupervisorStrategy.Stop
+import scala.collection.immutable
+import scala.concurrent.{ Future, Promise }
+import scala.concurrent.duration._
+import scala.util.control.NonFatal
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+
+import akka.{ AkkaException, OnlyCauseStackTrace }
import akka.actor._
+import akka.actor.SupervisorStrategy.Stop
+import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
import akka.pattern.pipe
import akka.remote._
import akka.remote.transport.ActorTransportAdapter._
@@ -18,15 +28,6 @@ import akka.remote.transport.ProtocolStateActor._
import akka.remote.transport.Transport._
import akka.util.ByteString
import akka.util.Helpers.Requiring
-import akka.{ AkkaException, OnlyCauseStackTrace }
-import com.typesafe.config.Config
-
-import scala.collection.immutable
-import scala.concurrent.duration._
-import scala.concurrent.{ Future, Promise }
-import scala.util.control.NonFatal
-import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
-import com.github.ghik.silencer.silent
@SerialVersionUID(1L)
class AkkaProtocolException(msg: String, cause: Throwable) extends AkkaException(msg, cause) with OnlyCauseStackTrace {
@@ -35,9 +36,10 @@ class AkkaProtocolException(msg: String, cause: Throwable) extends AkkaException
private[remote] class AkkaProtocolSettings(config: Config) {
- import akka.util.Helpers.ConfigOps
import config._
+ import akka.util.Helpers.ConfigOps
+
val TransportFailureDetectorConfig: Config = getConfig("akka.remote.classic.transport-failure-detector")
val TransportFailureDetectorImplementationClass: String =
TransportFailureDetectorConfig.getString("implementation-class")
diff --git a/akka-remote/src/main/scala/akka/remote/transport/FailureInjectorTransportAdapter.scala b/akka-remote/src/main/scala/akka/remote/transport/FailureInjectorTransportAdapter.scala
index c6f23bb504..af414ac25d 100644
--- a/akka-remote/src/main/scala/akka/remote/transport/FailureInjectorTransportAdapter.scala
+++ b/akka-remote/src/main/scala/akka/remote/transport/FailureInjectorTransportAdapter.scala
@@ -4,20 +4,21 @@
package akka.remote.transport
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.ThreadLocalRandom
+
+import scala.concurrent.{ Future, Promise }
+import scala.util.control.NoStackTrace
+
import FailureInjectorTransportAdapter._
+import com.github.ghik.silencer.silent
+
import akka.AkkaException
import akka.actor.{ Address, ExtendedActorSystem }
import akka.event.{ Logging, LoggingAdapter }
import akka.remote.transport.AssociationHandle.{ HandleEvent, HandleEventListener }
import akka.remote.transport.Transport._
import akka.util.ByteString
-import java.util.concurrent.ConcurrentHashMap
-import java.util.concurrent.ThreadLocalRandom
-
-import com.github.ghik.silencer.silent
-
-import scala.concurrent.{ Future, Promise }
-import scala.util.control.NoStackTrace
@SerialVersionUID(1L)
@deprecated("Classic remoting is deprecated, use Artery", "2.6.0")
diff --git a/akka-remote/src/main/scala/akka/remote/transport/TestTransport.scala b/akka-remote/src/main/scala/akka/remote/transport/TestTransport.scala
index 7c170e5138..3b3f8eb9e2 100644
--- a/akka-remote/src/main/scala/akka/remote/transport/TestTransport.scala
+++ b/akka-remote/src/main/scala/akka/remote/transport/TestTransport.scala
@@ -6,16 +6,17 @@ package akka.remote.transport
import java.util.concurrent.{ ConcurrentHashMap, CopyOnWriteArrayList }
+import scala.concurrent.{ Await, Future, Promise }
+import scala.concurrent.ExecutionContext.Implicits.global
+import scala.concurrent.duration._
+
+import TestTransport._
+import com.typesafe.config.Config
+
import akka.actor._
import akka.remote.transport.AssociationHandle._
import akka.remote.transport.Transport._
import akka.util.ByteString
-import com.typesafe.config.Config
-import TestTransport._
-
-import scala.concurrent.duration._
-import scala.concurrent.{ Await, Future, Promise }
-import scala.concurrent.ExecutionContext.Implicits.global
/**
* Transport implementation to be used for testing.
diff --git a/akka-remote/src/main/scala/akka/remote/transport/ThrottlerTransportAdapter.scala b/akka-remote/src/main/scala/akka/remote/transport/ThrottlerTransportAdapter.scala
index 05cdeb879a..8a41d7f264 100644
--- a/akka-remote/src/main/scala/akka/remote/transport/ThrottlerTransportAdapter.scala
+++ b/akka-remote/src/main/scala/akka/remote/transport/ThrottlerTransportAdapter.scala
@@ -4,8 +4,25 @@
package akka.remote.transport
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicReference
+
+import scala.annotation.tailrec
+import scala.collection.immutable.Queue
+import scala.concurrent.{ Future, Promise }
+import scala.concurrent.duration._
+import scala.math.min
+import scala.util.{ Failure, Success }
+import scala.util.control.NonFatal
+
+import com.github.ghik.silencer.silent
+
import akka.actor._
+import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
+import akka.dispatch.ExecutionContexts
+import akka.dispatch.sysmsg.{ Unwatch, Watch }
import akka.pattern.{ ask, pipe, PromiseActorRef }
+import akka.remote.RARP
import akka.remote.transport.ActorTransportAdapter.AssociateUnderlying
import akka.remote.transport.AkkaPduCodec.Associate
import akka.remote.transport.AssociationHandle.{
@@ -19,22 +36,6 @@ import akka.remote.transport.ThrottlerManager.{ Checkin, Handle, Listener, Liste
import akka.remote.transport.ThrottlerTransportAdapter._
import akka.remote.transport.Transport._
import akka.util.{ ByteString, Timeout }
-import java.util.concurrent.TimeUnit
-import java.util.concurrent.atomic.AtomicReference
-
-import akka.dispatch.ExecutionContexts
-
-import scala.annotation.tailrec
-import scala.collection.immutable.Queue
-import scala.concurrent.{ Future, Promise }
-import scala.concurrent.duration._
-import scala.math.min
-import scala.util.{ Failure, Success }
-import scala.util.control.NonFatal
-import akka.dispatch.sysmsg.{ Unwatch, Watch }
-import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
-import akka.remote.RARP
-import com.github.ghik.silencer.silent
@deprecated("Classic remoting is deprecated, use Artery", "2.6.0")
class ThrottlerProvider extends TransportAdapterProvider {
diff --git a/akka-remote/src/main/scala/akka/remote/transport/Transport.scala b/akka-remote/src/main/scala/akka/remote/transport/Transport.scala
index 13c48dcdbd..c1eb228f42 100644
--- a/akka-remote/src/main/scala/akka/remote/transport/Transport.scala
+++ b/akka-remote/src/main/scala/akka/remote/transport/Transport.scala
@@ -6,13 +6,15 @@ package akka.remote.transport
import scala.concurrent.{ Future, Promise }
import scala.util.control.NoStackTrace
-import akka.actor.{ ActorRef, Address, NoSerializationVerificationNeeded }
-import akka.util.{ unused, ByteString }
-import akka.remote.transport.AssociationHandle.HandleEventListener
+
+import com.github.ghik.silencer.silent
+
import akka.AkkaException
+import akka.actor.{ ActorRef, Address, NoSerializationVerificationNeeded }
import akka.actor.DeadLetterSuppression
import akka.event.LoggingAdapter
-import com.github.ghik.silencer.silent
+import akka.remote.transport.AssociationHandle.HandleEventListener
+import akka.util.{ unused, ByteString }
@deprecated("Classic remoting is deprecated, use Artery", "2.6.0")
object Transport {
diff --git a/akka-remote/src/main/scala/akka/remote/transport/netty/NettyHelpers.scala b/akka-remote/src/main/scala/akka/remote/transport/netty/NettyHelpers.scala
index 5a8d20967d..7b4a4d79c4 100644
--- a/akka-remote/src/main/scala/akka/remote/transport/netty/NettyHelpers.scala
+++ b/akka-remote/src/main/scala/akka/remote/transport/netty/NettyHelpers.scala
@@ -4,13 +4,14 @@
package akka.remote.transport.netty
-import akka.AkkaException
import java.nio.channels.ClosedChannelException
-import akka.util.unused
+import scala.util.control.NonFatal
+
import org.jboss.netty.channel._
-import scala.util.control.NonFatal
+import akka.AkkaException
+import akka.util.unused
/**
* INTERNAL API
diff --git a/akka-remote/src/main/scala/akka/remote/transport/netty/NettySSLSupport.scala b/akka-remote/src/main/scala/akka/remote/transport/netty/NettySSLSupport.scala
index e3552016ee..cdffb91022 100644
--- a/akka-remote/src/main/scala/akka/remote/transport/netty/NettySSLSupport.scala
+++ b/akka-remote/src/main/scala/akka/remote/transport/netty/NettySSLSupport.scala
@@ -4,11 +4,12 @@
package akka.remote.transport.netty
-import akka.japi.Util._
+import com.github.ghik.silencer.silent
import com.typesafe.config.Config
import org.jboss.netty.handler.ssl.SslHandler
+
+import akka.japi.Util._
import akka.util.ccompat._
-import com.github.ghik.silencer.silent
/**
* INTERNAL API
diff --git a/akka-remote/src/main/scala/akka/remote/transport/netty/NettyTransport.scala b/akka-remote/src/main/scala/akka/remote/transport/netty/NettyTransport.scala
index ad6f8c826b..90f2f918a3 100644
--- a/akka-remote/src/main/scala/akka/remote/transport/netty/NettyTransport.scala
+++ b/akka-remote/src/main/scala/akka/remote/transport/netty/NettyTransport.scala
@@ -7,34 +7,20 @@ package akka.remote.transport.netty
import java.net.InetAddress
import java.net.InetSocketAddress
import java.net.SocketAddress
-import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.CancellationException
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.Executors
+import java.util.concurrent.atomic.AtomicInteger
-import scala.concurrent.duration.FiniteDuration
import scala.concurrent.ExecutionContext
import scala.concurrent.Future
import scala.concurrent.Promise
import scala.concurrent.blocking
+import scala.concurrent.duration.FiniteDuration
import scala.util.Try
import scala.util.control.NoStackTrace
import scala.util.control.NonFatal
-import akka.actor.ActorSystem
-import akka.actor.Address
-import akka.actor.ExtendedActorSystem
-import akka.dispatch.ThreadPoolConfig
-import akka.event.Logging
-import akka.remote.RARP
-import akka.remote.transport.AssociationHandle.HandleEventListener
-import akka.remote.transport.Transport._
-import akka.remote.transport.AssociationHandle
-import akka.remote.transport.Transport
-import akka.util.Helpers
-import akka.util.Helpers.Requiring
-import akka.util.OptionVal
-import akka.ConfigurationException
-import akka.OnlyCauseStackTrace
+
import com.github.ghik.silencer.silent
import com.typesafe.config.Config
import org.jboss.netty.bootstrap.Bootstrap
@@ -56,6 +42,22 @@ import org.jboss.netty.handler.codec.frame.LengthFieldPrepender
import org.jboss.netty.handler.ssl.SslHandler
import org.jboss.netty.util.HashedWheelTimer
+import akka.ConfigurationException
+import akka.OnlyCauseStackTrace
+import akka.actor.ActorSystem
+import akka.actor.Address
+import akka.actor.ExtendedActorSystem
+import akka.dispatch.ThreadPoolConfig
+import akka.event.Logging
+import akka.remote.RARP
+import akka.remote.transport.AssociationHandle
+import akka.remote.transport.AssociationHandle.HandleEventListener
+import akka.remote.transport.Transport
+import akka.remote.transport.Transport._
+import akka.util.Helpers
+import akka.util.Helpers.Requiring
+import akka.util.OptionVal
+
@deprecated("Classic remoting is deprecated, use Artery", "2.6.0")
object NettyFutureBridge {
def apply(nettyFuture: ChannelFuture): Future[Channel] = {
@@ -73,7 +75,7 @@ object NettyFutureBridge {
def apply(nettyFuture: ChannelGroupFuture): Future[ChannelGroup] = {
import akka.util.ccompat.JavaConverters._
- val p = Promise[ChannelGroup]
+ val p = Promise[ChannelGroup]()
nettyFuture.addListener(new ChannelGroupFutureListener {
def operationComplete(future: ChannelGroupFuture): Unit =
p.complete(
@@ -111,9 +113,10 @@ class NettyTransportExceptionNoStack(msg: String, cause: Throwable)
@deprecated("Classic remoting is deprecated, use Artery", "2.6.0")
class NettyTransportSettings(config: Config) {
- import akka.util.Helpers.ConfigOps
import config._
+ import akka.util.Helpers.ConfigOps
+
val EnableSsl: Boolean = getBoolean("enable-ssl")
val SSLEngineProviderClassName: String = if (EnableSsl) getString("ssl-engine-provider") else ""
diff --git a/akka-remote/src/main/scala/akka/remote/transport/netty/SSLEngineProvider.scala b/akka-remote/src/main/scala/akka/remote/transport/netty/SSLEngineProvider.scala
index 5a3af16a53..c00bf02224 100644
--- a/akka-remote/src/main/scala/akka/remote/transport/netty/SSLEngineProvider.scala
+++ b/akka-remote/src/main/scala/akka/remote/transport/netty/SSLEngineProvider.scala
@@ -11,6 +11,12 @@ import java.nio.file.Paths
import java.security.GeneralSecurityException
import java.security.KeyStore
import java.security.SecureRandom
+import javax.net.ssl.KeyManager
+import javax.net.ssl.KeyManagerFactory
+import javax.net.ssl.SSLContext
+import javax.net.ssl.SSLEngine
+import javax.net.ssl.TrustManager
+import javax.net.ssl.TrustManagerFactory
import scala.util.Try
@@ -20,12 +26,6 @@ import akka.event.MarkerLoggingAdapter
import akka.remote.RemoteTransportException
import akka.remote.artery.tcp.SecureRandomFactory
import akka.stream.TLSRole
-import javax.net.ssl.KeyManager
-import javax.net.ssl.KeyManagerFactory
-import javax.net.ssl.SSLContext
-import javax.net.ssl.SSLEngine
-import javax.net.ssl.TrustManager
-import javax.net.ssl.TrustManagerFactory
@deprecated("Classic remoting is deprecated, use Artery", "2.6.0")
trait SSLEngineProvider {
diff --git a/akka-remote/src/main/scala/akka/remote/transport/netty/TcpSupport.scala b/akka-remote/src/main/scala/akka/remote/transport/netty/TcpSupport.scala
index 87be2b21e9..a51fa80f24 100644
--- a/akka-remote/src/main/scala/akka/remote/transport/netty/TcpSupport.scala
+++ b/akka-remote/src/main/scala/akka/remote/transport/netty/TcpSupport.scala
@@ -4,19 +4,20 @@
package akka.remote.transport.netty
+import java.net.InetSocketAddress
+
+import scala.concurrent.{ Future, Promise }
+
+import com.github.ghik.silencer.silent
+import org.jboss.netty.buffer.{ ChannelBuffer, ChannelBuffers }
+import org.jboss.netty.channel._
+
import akka.actor.Address
+import akka.event.LoggingAdapter
import akka.remote.transport.AssociationHandle
import akka.remote.transport.AssociationHandle.{ Disassociated, HandleEvent, HandleEventListener, InboundPayload }
import akka.remote.transport.Transport.AssociationEventListener
import akka.util.ByteString
-import java.net.InetSocketAddress
-
-import akka.event.LoggingAdapter
-import org.jboss.netty.buffer.{ ChannelBuffer, ChannelBuffers }
-import org.jboss.netty.channel._
-import scala.concurrent.{ Future, Promise }
-
-import com.github.ghik.silencer.silent
/**
* INTERNAL API
diff --git a/akka-remote/src/test/java/akka/remote/ProtobufProtocol.java b/akka-remote/src/test/java/akka/remote/ProtobufProtocol.java
index 9f9050ad42..a7b7c4fa03 100644
--- a/akka-remote/src/test/java/akka/remote/ProtobufProtocol.java
+++ b/akka-remote/src/test/java/akka/remote/ProtobufProtocol.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2019-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
diff --git a/akka-remote/src/test/java/akka/remote/protobuf/v3/ProtobufProtocolV3.java b/akka-remote/src/test/java/akka/remote/protobuf/v3/ProtobufProtocolV3.java
index 87ef7ef8cb..6fc6c9a302 100644
--- a/akka-remote/src/test/java/akka/remote/protobuf/v3/ProtobufProtocolV3.java
+++ b/akka-remote/src/test/java/akka/remote/protobuf/v3/ProtobufProtocolV3.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2019-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
diff --git a/akka-remote/src/test/scala-jdk9-only/akka/remote/artery/jfr/JFRRemotingFlightRecorderSpec.scala b/akka-remote/src/test/scala-jdk9-only/akka/remote/artery/jfr/JFRRemotingFlightRecorderSpec.scala
index 3ec93ee2c3..7cbb45a29f 100644
--- a/akka-remote/src/test/scala-jdk9-only/akka/remote/artery/jfr/JFRRemotingFlightRecorderSpec.scala
+++ b/akka-remote/src/test/scala-jdk9-only/akka/remote/artery/jfr/JFRRemotingFlightRecorderSpec.scala
@@ -4,12 +4,13 @@
package akka.remote.artery.jfr
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ActorSystem
import akka.remote.artery.NoOpRemotingFlightRecorder
import akka.remote.artery.RemotingFlightRecorder
import akka.testkit.AkkaSpec
import akka.testkit.TestKit
-import com.typesafe.config.ConfigFactory
class JFRRemotingFlightRecorderSpec extends AkkaSpec {
diff --git a/akka-remote/src/test/scala/akka/remote/AccrualFailureDetectorSpec.scala b/akka-remote/src/test/scala/akka/remote/AccrualFailureDetectorSpec.scala
index 417ddb6740..cef462e3d6 100644
--- a/akka-remote/src/test/scala/akka/remote/AccrualFailureDetectorSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/AccrualFailureDetectorSpec.scala
@@ -4,13 +4,14 @@
package akka.remote
-import akka.testkit.AkkaSpec
-
import scala.collection.immutable.TreeMap
import scala.concurrent.duration._
-import akka.remote.FailureDetector.Clock
+
import com.github.ghik.silencer.silent
+import akka.remote.FailureDetector.Clock
+import akka.testkit.AkkaSpec
+
@silent
class AccrualFailureDetectorSpec extends AkkaSpec("akka.loglevel = INFO") {
diff --git a/akka-remote/src/test/scala/akka/remote/AckedDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/AckedDeliverySpec.scala
index c972aa0760..bbe4b90531 100644
--- a/akka-remote/src/test/scala/akka/remote/AckedDeliverySpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/AckedDeliverySpec.scala
@@ -4,12 +4,14 @@
package akka.remote
-import akka.testkit.AkkaSpec
-import scala.annotation.tailrec
import java.util.concurrent.ThreadLocalRandom
+import scala.annotation.tailrec
+
import com.github.ghik.silencer.silent
+import akka.testkit.AkkaSpec
+
@silent("deprecated")
object AckedDeliverySpec {
diff --git a/akka-remote/src/test/scala/akka/remote/DaemonicSpec.scala b/akka-remote/src/test/scala/akka/remote/DaemonicSpec.scala
index 25df7f236b..fa5eb16679 100644
--- a/akka-remote/src/test/scala/akka/remote/DaemonicSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/DaemonicSpec.scala
@@ -4,13 +4,13 @@
package akka.remote
-import akka.testkit._
-
import scala.concurrent.duration._
-import akka.actor.{ ActorSystem, Address }
-import akka.util.ccompat._
+
import com.typesafe.config.ConfigFactory
+import akka.actor.{ ActorSystem, Address }
+import akka.testkit._
+import akka.util.ccompat._
import akka.util.ccompat.JavaConverters._
@ccompatUsedUntil213
diff --git a/akka-remote/src/test/scala/akka/remote/DeadlineFailureDetectorSpec.scala b/akka-remote/src/test/scala/akka/remote/DeadlineFailureDetectorSpec.scala
index 4d3bedc6cc..f749c2f33c 100644
--- a/akka-remote/src/test/scala/akka/remote/DeadlineFailureDetectorSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/DeadlineFailureDetectorSpec.scala
@@ -4,9 +4,10 @@
package akka.remote
-import akka.testkit.AkkaSpec
import scala.concurrent.duration._
+
import akka.remote.FailureDetector.Clock
+import akka.testkit.AkkaSpec
class DeadlineFailureDetectorSpec extends AkkaSpec {
diff --git a/akka-remote/src/test/scala/akka/remote/EndpointRegistrySpec.scala b/akka-remote/src/test/scala/akka/remote/EndpointRegistrySpec.scala
index 48624b9049..3da7ac0345 100644
--- a/akka-remote/src/test/scala/akka/remote/EndpointRegistrySpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/EndpointRegistrySpec.scala
@@ -4,10 +4,11 @@
package akka.remote
-import akka.testkit.AkkaSpec
+import scala.concurrent.duration._
+
import akka.actor.{ Address, Props }
import akka.remote.EndpointManager._
-import scala.concurrent.duration._
+import akka.testkit.AkkaSpec
class EndpointRegistrySpec extends AkkaSpec {
diff --git a/akka-remote/src/test/scala/akka/remote/FailureDetectorRegistrySpec.scala b/akka-remote/src/test/scala/akka/remote/FailureDetectorRegistrySpec.scala
index 0d3955edcb..4670637694 100644
--- a/akka-remote/src/test/scala/akka/remote/FailureDetectorRegistrySpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/FailureDetectorRegistrySpec.scala
@@ -4,8 +4,9 @@
package akka.remote
-import akka.remote.FailureDetector.Clock
import scala.concurrent.duration._
+
+import akka.remote.FailureDetector.Clock
import akka.testkit.AkkaSpec
class FailureDetectorRegistrySpec extends AkkaSpec("akka.loglevel = INFO") {
diff --git a/akka-remote/src/test/scala/akka/remote/LogSourceSpec.scala b/akka-remote/src/test/scala/akka/remote/LogSourceSpec.scala
index 4333ebca6d..becd0bfabc 100644
--- a/akka-remote/src/test/scala/akka/remote/LogSourceSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/LogSourceSpec.scala
@@ -4,15 +4,15 @@
package akka.remote
-import akka.testkit.AkkaSpec
import akka.actor.Actor
import akka.actor.ActorLogging
+import akka.actor.Deploy
+import akka.actor.ExtendedActorSystem
import akka.actor.Props
import akka.event.Logging
-import akka.testkit.TestProbe
-import akka.actor.Deploy
import akka.event.Logging.Info
-import akka.actor.ExtendedActorSystem
+import akka.testkit.AkkaSpec
+import akka.testkit.TestProbe
object LogSourceSpec {
class Reporter extends Actor with ActorLogging {
@@ -31,7 +31,7 @@ class LogSourceSpec extends AkkaSpec("""
import LogSourceSpec._
- val reporter = system.actorOf(Props[Reporter], "reporter")
+ val reporter = system.actorOf(Props[Reporter](), "reporter")
val logProbe = TestProbe()
system.eventStream.subscribe(system.actorOf(Props(new Actor {
def receive = {
diff --git a/akka-remote/src/test/scala/akka/remote/MessageLoggingSpec.scala b/akka-remote/src/test/scala/akka/remote/MessageLoggingSpec.scala
index fe936c0541..5b93726c89 100644
--- a/akka-remote/src/test/scala/akka/remote/MessageLoggingSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/MessageLoggingSpec.scala
@@ -4,11 +4,12 @@
package akka.remote
-import akka.actor.{ Actor, ActorIdentity, ActorSystem, ExtendedActorSystem, Identify, Props, RootActorPath }
-import akka.testkit.{ AkkaSpec, ImplicitSender, TestKit }
-import com.typesafe.config.{ Config, ConfigFactory }
import MessageLoggingSpec._
+import com.typesafe.config.{ Config, ConfigFactory }
+
+import akka.actor.{ Actor, ActorIdentity, ActorSystem, ExtendedActorSystem, Identify, Props, RootActorPath }
import akka.serialization.jackson.CborSerializable
+import akka.testkit.{ AkkaSpec, ImplicitSender, TestKit }
object MessageLoggingSpec {
def config(artery: Boolean) = ConfigFactory.parseString(s"""
@@ -60,7 +61,7 @@ abstract class MessageLoggingSpec(config: Config) extends AkkaSpec(config) with
"Message logging" must {
"not be on if debug logging not enabled" in {
- remoteSystem.actorOf(Props[BadActor], "bad")
+ remoteSystem.actorOf(Props[BadActor](), "bad")
val as = system.actorSelection(RootActorPath(remoteAddress) / "user" / "bad")
as ! Identify("bad")
val ref = expectMsgType[ActorIdentity].ref.get
diff --git a/akka-remote/src/test/scala/akka/remote/NetworkFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/NetworkFailureSpec.scala
index b55ed00ba1..e3dce33b50 100644
--- a/akka-remote/src/test/scala/akka/remote/NetworkFailureSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/NetworkFailureSpec.scala
@@ -4,12 +4,13 @@
package akka.remote
+import java.util.concurrent.atomic.AtomicBoolean
+
+import scala.concurrent.Future
+
import akka.testkit.AkkaSpec
import akka.testkit.DefaultTimeout
-import java.util.concurrent.atomic.AtomicBoolean
-import scala.concurrent.{ Future }
-
trait NetworkFailureSpec extends DefaultTimeout { self: AkkaSpec =>
import scala.concurrent.duration.Duration
@@ -25,7 +26,7 @@ trait NetworkFailureSpec extends DefaultTimeout { self: AkkaSpec =>
enableTcpReset()
println("===>>> Reply with [TCP RST] for [" + duration + "]")
Thread.sleep(duration.toMillis)
- restoreIP
+ restoreIP()
} catch {
case e: Throwable =>
dead.set(true)
@@ -40,7 +41,7 @@ trait NetworkFailureSpec extends DefaultTimeout { self: AkkaSpec =>
enableNetworkThrottling()
println("===>>> Throttling network with [" + BytesPerSecond + ", " + DelayMillis + "] for [" + duration + "]")
Thread.sleep(duration.toMillis)
- restoreIP
+ restoreIP()
} catch {
case e: Throwable =>
dead.set(true)
@@ -55,7 +56,7 @@ trait NetworkFailureSpec extends DefaultTimeout { self: AkkaSpec =>
enableNetworkDrop()
println("===>>> Blocking network [TCP DENY] for [" + duration + "]")
Thread.sleep(duration.toMillis)
- restoreIP
+ restoreIP()
} catch {
case e: Throwable =>
dead.set(true)
diff --git a/akka-remote/src/test/scala/akka/remote/RemoteActorMailboxSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteActorMailboxSpec.scala
index d006628b08..9efb03d389 100644
--- a/akka-remote/src/test/scala/akka/remote/RemoteActorMailboxSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/RemoteActorMailboxSpec.scala
@@ -4,9 +4,10 @@
package akka.remote
-import akka.actor.ActorMailboxSpec
import com.typesafe.config.ConfigFactory
+import akka.actor.ActorMailboxSpec
+
class RemoteActorMailboxSpec
extends ActorMailboxSpec(
ConfigFactory.parseString("""akka.actor.provider = remote""").withFallback(ActorMailboxSpec.mailboxConf)) {}
diff --git a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala
index 6a87a4bb21..321ba259a7 100644
--- a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala
@@ -4,16 +4,16 @@
package akka.remote
-import language.postfixOps
-
-import akka.testkit.AkkaSpec
import scala.concurrent.duration._
+import com.github.ghik.silencer.silent
+import language.postfixOps
+
import akka.remote.transport.AkkaProtocolSettings
+import akka.remote.transport.netty.{ NettyTransportSettings, SSLSettings }
+import akka.testkit.AkkaSpec
import akka.util.Helpers
import akka.util.Helpers.ConfigOps
-import akka.remote.transport.netty.{ NettyTransportSettings, SSLSettings }
-import com.github.ghik.silencer.silent
@silent // classic deprecated
class RemoteConfigSpec extends AkkaSpec("""
@@ -130,7 +130,8 @@ class RemoteConfigSpec extends AkkaSpec("""
sslSettings.SSLTrustStore should ===("truststore")
sslSettings.SSLTrustStorePassword should ===("changeme")
sslSettings.SSLProtocol should ===("TLSv1.2")
- sslSettings.SSLEnabledAlgorithms should ===(Set("TLS_RSA_WITH_AES_128_CBC_SHA"))
+ sslSettings.SSLEnabledAlgorithms should ===(
+ Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_DHE_RSA_WITH_AES_256_GCM_SHA384"))
sslSettings.SSLRandomNumberGenerator should ===("")
}
diff --git a/akka-remote/src/test/scala/akka/remote/RemoteConsistentHashingRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteConsistentHashingRouterSpec.scala
index 5731d5a833..fb44ce02db 100644
--- a/akka-remote/src/test/scala/akka/remote/RemoteConsistentHashingRouterSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/RemoteConsistentHashingRouterSpec.scala
@@ -4,11 +4,11 @@
package akka.remote
-import akka.testkit.AkkaSpec
import akka.actor.Address
import akka.routing.ActorSelectionRoutee
-import akka.routing.ConsistentRoutee
import akka.routing.ConsistentHash
+import akka.routing.ConsistentRoutee
+import akka.testkit.AkkaSpec
class RemoteConsistentHashingRouterSpec
extends AkkaSpec("""
diff --git a/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala
index 0beb52ec95..4bb46458a9 100644
--- a/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala
@@ -4,11 +4,12 @@
package akka.remote
-import akka.testkit._
+import com.typesafe.config._
+
+import akka.ConfigurationException
import akka.actor._
import akka.routing._
-import com.typesafe.config._
-import akka.ConfigurationException
+import akka.testkit._
object RemoteDeployerSpec {
val deployerConf = ConfigFactory.parseString(
diff --git a/akka-remote/src/test/scala/akka/remote/RemoteFeaturesSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteFeaturesSpec.scala
index a818e18e3b..93e65370d5 100644
--- a/akka-remote/src/test/scala/akka/remote/RemoteFeaturesSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/RemoteFeaturesSpec.scala
@@ -6,6 +6,10 @@ package akka.remote
import scala.concurrent.duration._
+import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.AddressFromURIString
@@ -22,9 +26,6 @@ import akka.remote.artery.RemoteDeploymentSpec
import akka.testkit.EventFilter
import akka.testkit.ImplicitSender
import akka.testkit.TestProbe
-import com.github.ghik.silencer.silent
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
object RemoteFeaturesSpec {
@@ -144,7 +145,7 @@ class RemoteFeaturesDisabledSpec extends RemoteFeaturesSpec(RemoteFeaturesSpec.d
}
"""))
- val masterRef = masterSystem.actorOf(Props[RemoteDeploymentSpec.Echo1], actorName)
+ val masterRef = masterSystem.actorOf(Props[RemoteDeploymentSpec.Echo1](), actorName)
masterRef.path shouldEqual RootActorPath(AddressFromURIString(s"akka://${masterSystem.name}")) / "user" / actorName
masterRef.path.address.hasLocalScope shouldBe true
diff --git a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala
index a77106f6dc..830a58bbbc 100644
--- a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala
@@ -5,11 +5,13 @@
package akka.remote
import scala.collection.immutable
-import akka.testkit._
-import akka.routing._
+
+import com.typesafe.config._
+
import akka.actor._
import akka.remote.routing._
-import com.typesafe.config._
+import akka.routing._
+import akka.testkit._
import akka.testkit.TestActors.echoActorProps
object RemoteRouterSpec {
@@ -259,7 +261,7 @@ class RemoteRouterSpec extends AkkaSpec(s"""
// we don't really support deployment configuration of system actors, but
// it's used for the pool of the SimpleDnsManager "/IO-DNS/inet-address"
val probe = TestProbe()(masterSystem)
- val parent = masterSystem.asInstanceOf[ExtendedActorSystem].systemActorOf(Props[Parent], "sys-parent")
+ val parent = masterSystem.asInstanceOf[ExtendedActorSystem].systemActorOf(Props[Parent](), "sys-parent")
parent.tell((FromConfig.props(echoActorProps), "round"), probe.ref)
val router = probe.expectMsgType[ActorRef]
val replies = collectRouteePaths(probe, router, 10)
diff --git a/akka-remote/src/test/scala/akka/remote/Ticket1978CommunicationSpec.scala b/akka-remote/src/test/scala/akka/remote/Ticket1978CommunicationSpec.scala
index b47020cd39..82e94275c2 100644
--- a/akka-remote/src/test/scala/akka/remote/Ticket1978CommunicationSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/Ticket1978CommunicationSpec.scala
@@ -8,20 +8,21 @@ import java.io.ByteArrayOutputStream
import java.security.NoSuchAlgorithmException
import java.util.zip.GZIPOutputStream
-import akka.actor._
-import akka.event.NoMarkerLogging
-import akka.pattern.ask
-import akka.remote.Configuration.{ getCipherConfig, CipherConfig }
-import akka.remote.transport.netty.SSLSettings
-import akka.testkit._
-import akka.util.Timeout
-import com.typesafe.config._
import scala.concurrent.{ Await, Future }
import scala.concurrent.duration._
import scala.reflect.classTag
-import akka.remote.transport.netty.ConfigSSLEngineProvider
import com.github.ghik.silencer.silent
+import com.typesafe.config._
+
+import akka.actor._
+import akka.event.NoMarkerLogging
+import akka.pattern.ask
+import akka.remote.Configuration.{ getCipherConfig, CipherConfig }
+import akka.remote.transport.netty.ConfigSSLEngineProvider
+import akka.remote.transport.netty.SSLSettings
+import akka.testkit._
+import akka.util.Timeout
@silent("deprecated")
object Configuration {
@@ -110,10 +111,12 @@ object Configuration {
}
class Ticket1978SHA1PRNGSpec
- extends Ticket1978CommunicationSpec(getCipherConfig("SHA1PRNG", "TLS_RSA_WITH_AES_128_CBC_SHA"))
+ extends Ticket1978CommunicationSpec(
+ getCipherConfig("SHA1PRNG", "TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_DHE_RSA_WITH_AES_256_GCM_SHA384"))
class Ticket1978DefaultRNGSecureSpec
- extends Ticket1978CommunicationSpec(getCipherConfig("", "TLS_RSA_WITH_AES_128_CBC_SHA"))
+ extends Ticket1978CommunicationSpec(
+ getCipherConfig("", "TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_DHE_RSA_WITH_AES_256_GCM_SHA384"))
class Ticket1978CrappyRSAWithMD5OnlyHereToMakeSureThingsWorkSpec
extends Ticket1978CommunicationSpec(getCipherConfig("", "SSL_RSA_WITH_NULL_MD5"))
diff --git a/akka-remote/src/test/scala/akka/remote/Ticket1978ConfigSpec.scala b/akka-remote/src/test/scala/akka/remote/Ticket1978ConfigSpec.scala
index c51fde105d..e2114219e2 100644
--- a/akka-remote/src/test/scala/akka/remote/Ticket1978ConfigSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/Ticket1978ConfigSpec.scala
@@ -4,8 +4,8 @@
package akka.remote
-import akka.testkit._
import akka.remote.transport.netty.SSLSettings
+import akka.testkit._
class Ticket1978ConfigSpec extends AkkaSpec("""
akka.remote.classic.netty.ssl.security {
@@ -23,7 +23,8 @@ class Ticket1978ConfigSpec extends AkkaSpec("""
settings.SSLTrustStore should ===("truststore")
settings.SSLTrustStorePassword should ===("changeme")
settings.SSLProtocol should ===("TLSv1.2")
- settings.SSLEnabledAlgorithms should ===(Set("TLS_RSA_WITH_AES_128_CBC_SHA"))
+ settings.SSLEnabledAlgorithms should ===(
+ Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_DHE_RSA_WITH_AES_256_GCM_SHA384"))
settings.SSLRandomNumberGenerator should ===("SecureRandom")
}
}
diff --git a/akka-remote/src/test/scala/akka/remote/TransientSerializationErrorSpec.scala b/akka-remote/src/test/scala/akka/remote/TransientSerializationErrorSpec.scala
index 825a3619fd..6d0c316cff 100644
--- a/akka-remote/src/test/scala/akka/remote/TransientSerializationErrorSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/TransientSerializationErrorSpec.scala
@@ -6,11 +6,12 @@ package akka.remote
import java.io.NotSerializableException
+import com.typesafe.config.{ Config, ConfigFactory }
+
import akka.actor.{ ActorSystem, ExtendedActorSystem, RootActorPath }
import akka.serialization.SerializerWithStringManifest
import akka.testkit.{ AkkaSpec, TestActors, TestKit }
import akka.util.unused
-import com.typesafe.config.{ Config, ConfigFactory }
object TransientSerializationErrorSpec {
object ManifestNotSerializable
diff --git a/akka-remote/src/test/scala/akka/remote/TypedActorRemoteDeploySpec.scala b/akka-remote/src/test/scala/akka/remote/TypedActorRemoteDeploySpec.scala
index f71484d8d1..5b1e806d83 100644
--- a/akka-remote/src/test/scala/akka/remote/TypedActorRemoteDeploySpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/TypedActorRemoteDeploySpec.scala
@@ -7,12 +7,12 @@ package akka.remote
import scala.concurrent.{ Await, Future }
import scala.concurrent.duration._
+import TypedActorRemoteDeploySpec._
+import com.github.ghik.silencer.silent
+import com.typesafe.config._
+
import akka.actor.{ ActorSystem, Deploy, TypedActor, TypedProps }
import akka.testkit.AkkaSpec
-import TypedActorRemoteDeploySpec._
-
-import com.typesafe.config._
-import com.github.ghik.silencer.silent
object TypedActorRemoteDeploySpec {
val conf = ConfigFactory.parseString("""
@@ -47,7 +47,7 @@ class TypedActorRemoteDeploySpec extends AkkaSpec(conf) {
def verify[T](f: RemoteNameService => Future[T], expected: T) = {
val ts = TypedActor(system)
val echoService: RemoteNameService =
- ts.typedActorOf(TypedProps[RemoteNameServiceImpl].withDeploy(Deploy(scope = RemoteScope(remoteAddress))))
+ ts.typedActorOf(TypedProps[RemoteNameServiceImpl]().withDeploy(Deploy(scope = RemoteScope(remoteAddress))))
Await.result(f(echoService), 3.seconds) should ===(expected)
val actor = ts.getActorRefFor(echoService)
system.stop(actor)
diff --git a/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala
index c340579464..188f947599 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala
@@ -4,13 +4,14 @@
package akka.remote.artery
-import akka.actor.setup.ActorSystemSetup
-import akka.actor.{ ActorSystem, Address, BootstrapSetup, RootActorPath }
-import akka.remote.RARP
-import akka.testkit.{ AkkaSpec, SocketUtil }
import com.typesafe.config.{ Config, ConfigFactory }
import org.scalatest.{ Outcome, Pending }
+import akka.actor.{ ActorSystem, Address, BootstrapSetup, RootActorPath }
+import akka.actor.setup.ActorSystemSetup
+import akka.remote.RARP
+import akka.testkit.{ AkkaSpec, SocketUtil }
+
/**
* Base class for remoting tests what needs to test interaction between a "local" actor system
* which is always created (the usual AkkaSpec system), and multiple additional actor systems over artery
diff --git a/akka-remote/src/test/scala/akka/remote/artery/BindCanonicalAddressSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/BindCanonicalAddressSpec.scala
index 5529e8576d..4976d031a4 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/BindCanonicalAddressSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/BindCanonicalAddressSpec.scala
@@ -4,18 +4,19 @@
package akka.remote.artery
-import com.typesafe.config.ConfigFactory
-import akka.actor.{ ActorSystem, Address }
-import akka.remote.classic.transport.netty.NettyTransportSpec._
+import java.net.InetAddress
import scala.concurrent.Await
-
import scala.concurrent.duration.Duration
-import akka.testkit.SocketUtil
-import java.net.InetAddress
+
+import com.typesafe.config.ConfigFactory
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.{ ActorSystem, Address }
+import akka.remote.classic.transport.netty.NettyTransportSpec._
+import akka.testkit.SocketUtil
+
trait BindCanonicalAddressBehaviors {
this: AnyWordSpec with Matchers =>
def arteryConnectionTest(transport: String, isUDP: Boolean): Unit = {
@@ -29,7 +30,7 @@ trait BindCanonicalAddressBehaviors {
implicit val sys = ActorSystem("sys", config.withFallback(commonConfig))
- getInternal should contain(getExternal)
+ getInternal should contain(getExternal())
Await.result(sys.terminate(), Duration.Inf)
}
@@ -46,13 +47,13 @@ trait BindCanonicalAddressBehaviors {
getExternal should ===(address.toAkkaAddress("akka"))
// May have selected the same random port - bind another in that case while the other still has the canonical port
val internals =
- if (getInternal.collect { case Address(_, _, _, Some(port)) => port }.toSeq.contains(address.getPort)) {
+ if (getInternal().collect { case Address(_, _, _, Some(port)) => port }.toSeq.contains(address.getPort)) {
val sys2 = ActorSystem("sys", config.withFallback(commonConfig))
val secondInternals = getInternal()(sys2)
Await.result(sys2.terminate(), Duration.Inf)
secondInternals
} else {
- getInternal
+ getInternal()
}
internals should not contain address.toAkkaAddress("akka")
Await.result(sys.terminate(), Duration.Inf)
@@ -92,8 +93,8 @@ trait BindCanonicalAddressBehaviors {
implicit val sys = ActorSystem("sys", config.withFallback(commonConfig))
- getInternal.flatMap(_.port) should contain(getExternal.port.get)
- getInternal.map(x => (x.host.get should include).regex("0.0.0.0".r)) // regexp dot is intentional to match IPv4 and 6 addresses
+ getInternal().flatMap(_.port) should contain(getExternal().port.get)
+ getInternal().map(x => (x.host.get should include).regex("0.0.0.0".r)) // regexp dot is intentional to match IPv4 and 6 addresses
Await.result(sys.terminate(), Duration.Inf)
}
diff --git a/akka-remote/src/test/scala/akka/remote/artery/FlushOnShutdownSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/FlushOnShutdownSpec.scala
index 8b9cb59263..fd32ef4e51 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/FlushOnShutdownSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/FlushOnShutdownSpec.scala
@@ -4,12 +4,12 @@
package akka.remote.artery
-import akka.actor.{ Actor, ActorIdentity, Identify, Props }
-import akka.testkit.TestProbe
-
import scala.concurrent.Await
import scala.concurrent.duration._
+import akka.actor.{ Actor, ActorIdentity, Identify, Props }
+import akka.testkit.TestProbe
+
class FlushOnShutdownSpec extends ArteryMultiNodeSpec(ArterySpecSupport.defaultConfig) {
val remoteSystem = newRemoteSystem()
diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeDenySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeDenySpec.scala
index 9a9eb343ba..cc09045bb6 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeDenySpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeDenySpec.scala
@@ -6,10 +6,11 @@ package akka.remote.artery
import scala.concurrent.duration._
-import akka.actor.{ ActorIdentity, Identify }
-import akka.testkit._
import com.typesafe.config.ConfigFactory
+
+import akka.actor.{ ActorIdentity, Identify }
import akka.actor.RootActorPath
+import akka.testkit._
object HandshakeDenySpec {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala
index d4198b9df9..a64916833e 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala
@@ -6,10 +6,11 @@ package akka.remote.artery
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.actor.{ ActorIdentity, Identify }
import akka.testkit.ImplicitSender
import akka.testkit.TestActors
-import com.typesafe.config.ConfigFactory
import akka.testkit.TestProbe
object HandshakeFailureSpec {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala
index 406effc5f0..585a450e1e 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala
@@ -6,10 +6,11 @@ package akka.remote.artery
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.testkit.ImplicitSender
import akka.testkit.TestActors
-import com.typesafe.config.ConfigFactory
object HandshakeRetrySpec {
val commonConfig = ConfigFactory.parseString(s"""
diff --git a/akka-remote/src/test/scala/akka/remote/artery/ImmutableLongMapSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/ImmutableLongMapSpec.scala
index 2bf65cffe2..33815a6d2f 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/ImmutableLongMapSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/ImmutableLongMapSpec.scala
@@ -4,11 +4,13 @@
package akka.remote.artery
-import akka.util.OptionVal
import scala.util.Random
+
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.util.OptionVal
+
class ImmutableLongMapSpec extends AnyWordSpec with Matchers {
"ImmutableLongMap" must {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala
index b0461c443f..794c2aea75 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala
@@ -4,6 +4,8 @@
package akka.remote.artery
+import scala.util.Try
+
import akka.Done
import akka.actor.Address
import akka.remote.UniqueAddress
@@ -16,8 +18,6 @@ import akka.testkit.ImplicitSender
import akka.testkit.TestProbe
import akka.util.OptionVal
-import scala.util.Try
-
object InboundControlJunctionSpec {
trait TestControlMessage extends ControlMessage
diff --git a/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala
index fb3009311f..81a7dbd4a2 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala
@@ -4,14 +4,14 @@
package akka.remote.artery
-import akka.actor.{ Actor, ActorRef, ActorSelection, Props, RootActorPath }
-import akka.remote.{ RARP, RemoteActorRef }
-import akka.testkit.TestProbe
-import akka.util.ByteString
-
import scala.concurrent.Await
import scala.concurrent.duration._
+
+import akka.actor.{ Actor, ActorRef, ActorSelection, Props, RootActorPath }
+import akka.remote.{ RARP, RemoteActorRef }
import akka.testkit.JavaSerializable
+import akka.testkit.TestProbe
+import akka.util.ByteString
object LargeMessagesStreamSpec {
case class Ping(payload: ByteString = ByteString.empty) extends JavaSerializable
diff --git a/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala
index 694e21dc86..68f8b53c34 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala
@@ -6,12 +6,13 @@ package akka.remote.artery
import scala.concurrent.duration._
+import com.typesafe.config.ConfigFactory
+
import akka.actor.RootActorPath
import akka.remote.RARP
import akka.testkit.ImplicitSender
import akka.testkit.TestActors
import akka.testkit.TestProbe
-import com.typesafe.config.ConfigFactory
object LateConnectSpec {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/LruBoundedCacheSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/LruBoundedCacheSpec.scala
index b4b00be932..898dc1902d 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/LruBoundedCacheSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/LruBoundedCacheSpec.scala
@@ -4,11 +4,12 @@
package akka.remote.artery
-import akka.testkit.AkkaSpec
-import akka.util.Unsafe
+import scala.util.Random
+
import com.github.ghik.silencer.silent
-import scala.util.Random
+import akka.testkit.AkkaSpec
+import akka.util.Unsafe
@silent
class LruBoundedCacheSpec extends AkkaSpec {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/MetadataCarryingSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/MetadataCarryingSpec.scala
index 0d5c1a09bc..6dd1617bf2 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/MetadataCarryingSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/MetadataCarryingSpec.scala
@@ -4,17 +4,16 @@
package akka.remote.artery
+import java.nio.{ ByteBuffer, CharBuffer }
+import java.nio.charset.Charset
import java.util.concurrent.atomic.AtomicReference
import akka.actor._
import akka.testkit.ImplicitSender
+import akka.testkit.JavaSerializable
import akka.testkit.TestActors
import akka.testkit.TestProbe
import akka.util.ByteString
-import java.nio.{ ByteBuffer, CharBuffer }
-import java.nio.charset.Charset
-
-import akka.testkit.JavaSerializable
object MetadataCarryingSpy extends ExtensionId[MetadataCarryingSpy] with ExtensionIdProvider {
override def get(system: ActorSystem): MetadataCarryingSpy = super.get(system)
diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala
index cae9383e1b..9428523cf0 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala
@@ -4,6 +4,8 @@
package akka.remote.artery
+import scala.concurrent.duration._
+
import akka.actor.Address
import akka.remote.UniqueAddress
import akka.remote.artery.OutboundHandshake.HandshakeReq
@@ -17,8 +19,6 @@ import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import akka.util.OptionVal
-import scala.concurrent.duration._
-
class OutboundHandshakeSpec extends AkkaSpec("""
akka.stream.materializer.debug.fuzzing-mode = on
""") with ImplicitSender {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundIdleShutdownSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundIdleShutdownSpec.scala
index 44e4cec54c..31c3200ace 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/OutboundIdleShutdownSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundIdleShutdownSpec.scala
@@ -8,6 +8,9 @@ import scala.concurrent.Future
import scala.concurrent.Promise
import scala.concurrent.duration._
+import org.scalatest.concurrent.Eventually
+import org.scalatest.time.Span
+
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.Address
@@ -17,8 +20,6 @@ import akka.remote.UniqueAddress
import akka.testkit.ImplicitSender
import akka.testkit.TestActors
import akka.testkit.TestProbe
-import org.scalatest.concurrent.Eventually
-import org.scalatest.time.Span
class OutboundIdleShutdownSpec extends ArteryMultiNodeSpec(s"""
akka.loglevel=INFO
diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala
index 8f334850da..dd0cdbad46 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala
@@ -5,10 +5,10 @@
package akka.remote.artery
import akka.actor.{ EmptyLocalActorRef, InternalActorRef }
+import akka.actor.ActorRefScope
+import akka.actor.ExtendedActorSystem
import akka.remote.RemoteActorRef
import akka.testkit.{ EventFilter, TestActors }
-import akka.actor.ExtendedActorSystem
-import akka.actor.ActorRefScope
class RemoteActorRefProviderSpec extends ArteryMultiNodeSpec {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala
index fb78a17922..9a3a0e9c83 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala
@@ -4,6 +4,8 @@
package akka.remote.artery
+import scala.concurrent.duration._
+
import akka.actor.{
Actor,
ActorIdentity,
@@ -17,8 +19,6 @@ import akka.actor.{
Terminated
}
import akka.testkit.{ ImplicitSender, TestActors }
-
-import scala.concurrent.duration._
import akka.testkit.JavaSerializable
object RemoteActorSelectionSpec {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala
index b04aaedab0..9c21f9e580 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala
@@ -4,11 +4,11 @@
package akka.remote.artery
+import scala.concurrent.duration._
+
import akka.actor.ActorSystem
import akka.testkit.{ EventFilter, ImplicitSender, TestActors, TestEvent, TestProbe }
-import scala.concurrent.duration._
-
class RemoteConnectionSpec extends ArteryMultiNodeSpec("akka.remote.retry-gate-closed-for = 5s") with ImplicitSender {
def muteSystem(system: ActorSystem): Unit = {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala
index 2e3e517c29..7d552ddcbb 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala
@@ -4,15 +4,16 @@
package akka.remote.artery
-import akka.testkit._
-import akka.actor._
-import com.typesafe.config.ConfigFactory
-import akka.actor.RootActorPath
-
import scala.concurrent.duration._
-import akka.testkit.SocketUtil
-import akka.remote.RARP
+
import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
+import akka.actor._
+import akka.actor.RootActorPath
+import akka.remote.RARP
+import akka.testkit._
+import akka.testkit.SocketUtil
object RemoteDeathWatchSpec {
val otherPort = ArteryMultiNodeSpec.freePort(ConfigFactory.load())
diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala
index 095bc78c84..3013e8c7bd 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala
@@ -4,12 +4,13 @@
package akka.remote.artery
+import com.typesafe.config._
+
import akka.ConfigurationException
import akka.actor._
import akka.remote.RemoteScope
import akka.routing._
import akka.testkit._
-import com.typesafe.config._
object RemoteDeployerSpec {
val deployerConf = ConfigFactory.parseString("""
diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala
index 3668e9a6fb..f3734272ab 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala
@@ -4,10 +4,11 @@
package akka.remote.artery
-import akka.testkit._
-import akka.actor._
import com.typesafe.config._
+
+import akka.actor._
import akka.remote.RARP
+import akka.testkit._
object RemoteDeploymentSpec {
class Echo1 extends Actor {
@@ -93,7 +94,7 @@ class RemoteDeploymentSpec
"create and supervise children on remote node" in {
val senderProbe = TestProbe()(masterSystem)
- val r = masterSystem.actorOf(Props[Echo1], "blub")
+ val r = masterSystem.actorOf(Props[Echo1](), "blub")
r.path.toString should ===(
s"akka://${system.name}@localhost:${port}/remote/akka/${masterSystem.name}@localhost:${masterPort}/user/blub")
@@ -111,7 +112,7 @@ class RemoteDeploymentSpec
"create and supervise children on remote node for unknown exception" in {
val senderProbe = TestProbe()(masterSystem)
- val r = masterSystem.actorOf(Props[Echo1], "blub2")
+ val r = masterSystem.actorOf(Props[Echo1](), "blub2")
r.path.toString should ===(
s"akka://${system.name}@localhost:${port}/remote/akka/${masterSystem.name}@localhost:${masterPort}/user/blub2")
@@ -130,7 +131,7 @@ class RemoteDeploymentSpec
"notice immediate death" in {
val parent = masterSystem.actorOf(parentProps(testActor), "parent")
EventFilter[ActorInitializationException](occurrences = 1).intercept {
- parent.tell(Props[DeadOnArrival], testActor)
+ parent.tell(Props[DeadOnArrival](), testActor)
val child = expectMsgType[ActorRef]
expectMsgType[ActorInitializationException]
@@ -149,7 +150,7 @@ class RemoteDeploymentSpec
}.toVector
val probes = Vector.fill(numParents, numChildren)(TestProbe()(masterSystem))
- val childProps = Props[Echo1]
+ val childProps = Props[Echo1]()
for (p <- (0 until numParents); c <- (0 until numChildren)) {
parents(p).tell((childProps, numMessages), probes(p)(c).ref)
}
diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteFailureSpec.scala
index 7cf392bd27..3ae73bfd71 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/RemoteFailureSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteFailureSpec.scala
@@ -4,11 +4,11 @@
package akka.remote.artery
-import akka.remote.EndpointDisassociatedException
-import akka.testkit.{ EventFilter, ImplicitSender, TestActors, TestEvent }
import scala.concurrent.duration._
+import akka.remote.EndpointDisassociatedException
import akka.serialization.jackson.CborSerializable
+import akka.testkit.{ EventFilter, ImplicitSender, TestActors, TestEvent }
import akka.testkit.DeadLettersFilter
import akka.testkit.TestEvent.Mute
diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteInstrumentsSerializationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteInstrumentsSerializationSpec.scala
index 45e0178329..8be036db78 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/RemoteInstrumentsSerializationSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteInstrumentsSerializationSpec.scala
@@ -4,16 +4,17 @@
package akka.remote.artery
-import akka.actor.{ ActorRef, ActorSystem, ExtendedActorSystem, InternalActorRef }
-import akka.event._
-import akka.testkit.TestEvent.Mute
-import akka.testkit.{ AkkaSpec, EventFilter, TestProbe }
-import akka.util.{ unused, OptionVal }
import java.nio.{ ByteBuffer, CharBuffer }
import java.nio.charset.Charset
import scala.concurrent.duration._
+import akka.actor.{ ActorRef, ActorSystem, ExtendedActorSystem, InternalActorRef }
+import akka.event._
+import akka.testkit.{ AkkaSpec, EventFilter, TestProbe }
+import akka.testkit.TestEvent.Mute
+import akka.util.{ unused, OptionVal }
+
class RemoteInstrumentsSerializationSpec extends AkkaSpec("akka.loglevel = DEBUG") {
import RemoteInstrumentsSerializationSpec._
diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteInstrumentsSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteInstrumentsSpec.scala
index f76145b9eb..3354608939 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/RemoteInstrumentsSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteInstrumentsSpec.scala
@@ -5,9 +5,9 @@
package akka.remote.artery
import org.scalacheck.{ Arbitrary, Gen }
-import org.scalatestplus.scalacheck.Checkers
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import org.scalatestplus.scalacheck.Checkers
class RemoteInstrumentsSpec extends AnyWordSpec with Matchers with Checkers {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala
index 610250b6ee..1cf702b63b 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala
@@ -7,13 +7,12 @@ package akka.remote.artery
import java.io.NotSerializableException
import java.util.concurrent.ThreadLocalRandom
-import akka.actor.{ Actor, ActorRef, PoisonPill, Props }
-import akka.remote.{ AssociationErrorEvent, DisassociatedEvent, OversizedPayloadException, RARP }
-import akka.testkit.{ EventFilter, ImplicitSender, TestActors }
-import akka.util.ByteString
-import com.github.ghik.silencer.silent
-
import scala.concurrent.duration._
+import com.github.ghik.silencer.silent
+import akka.actor.{ Actor, ActorRef, Dropped, PoisonPill, Props }
+import akka.remote.{ AssociationErrorEvent, DisassociatedEvent, OversizedPayloadException, RARP }
+import akka.testkit.{ EventFilter, ImplicitSender, TestActors, TestProbe }
+import akka.util.ByteString
object RemoteMessageSerializationSpec {
class ProxyActor(val one: ActorRef, val another: ActorRef) extends Actor {
@@ -51,6 +50,8 @@ class RemoteMessageSerializationSpec extends ArteryMultiNodeSpec with ImplicitSe
}
"drop sent messages over payload size" in {
+ val droppedProbe = TestProbe()
+ system.eventStream.subscribe(droppedProbe.ref, classOf[Dropped])
val oversized = byteStringOfSize(maxPayloadBytes + 1)
EventFilter[OversizedPayloadException](start = "Failed to serialize oversized message", occurrences = 1)
.intercept {
@@ -58,6 +59,7 @@ class RemoteMessageSerializationSpec extends ArteryMultiNodeSpec with ImplicitSe
expectNoMessage(1.second) // No AssocitionErrorEvent should be published
}
}
+ droppedProbe.expectMsgType[Dropped].message should ===(oversized)
}
// TODO max payload size is not configurable yet, so we cannot send a too big message, it fails no sending side
diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala
index d4c90b810d..497051c6bb 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala
@@ -4,15 +4,16 @@
package akka.remote.artery
-import akka.actor._
-import akka.remote.routing._
-import akka.remote.{ RARP, RemoteScope }
-import akka.routing._
-import akka.testkit.TestActors.echoActorProps
-import akka.testkit._
+import scala.collection.immutable
+
import com.typesafe.config._
-import scala.collection.immutable
+import akka.actor._
+import akka.remote.{ RARP, RemoteScope }
+import akka.remote.routing._
+import akka.routing._
+import akka.testkit._
+import akka.testkit.TestActors.echoActorProps
object RemoteRouterSpec {
class Parent extends Actor {
@@ -251,7 +252,7 @@ class RemoteRouterSpec
// we don't really support deployment configuration of system actors, but
// it's used for the pool of the SimpleDnsManager "/IO-DNS/inet-address"
val probe = TestProbe()(masterSystem)
- val parent = masterSystem.asInstanceOf[ExtendedActorSystem].systemActorOf(Props[Parent], "sys-parent")
+ val parent = masterSystem.asInstanceOf[ExtendedActorSystem].systemActorOf(Props[Parent](), "sys-parent")
parent.tell((FromConfig.props(echoActorProps), "round"), probe.ref)
val router = probe.expectMsgType[ActorRef]
val replies = collectRouteePaths(probe, router, 10)
diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala
index 93afbad1b5..e98c2b6a66 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala
@@ -4,12 +4,23 @@
package akka.remote.artery
-import akka.actor.{ Actor, ActorIdentity, ActorRef, Deploy, Identify, PoisonPill, Props, RootActorPath }
-import akka.testkit.{ ImplicitSender, TestActors, TestProbe }
-import com.typesafe.config.{ Config, ConfigFactory }
+import java.util.UUID
import scala.concurrent.duration._
+
+import com.typesafe.config.{ Config, ConfigFactory }
+
+import akka.actor.Actor
+import akka.actor.ActorIdentity
+import akka.actor.ActorPath
+import akka.actor.ActorRef
import akka.actor.ActorSelection
+import akka.actor.Deploy
+import akka.actor.Identify
+import akka.actor.PoisonPill
+import akka.actor.Props
+import akka.actor.RootActorPath
+import akka.testkit.{ ImplicitSender, TestActors, TestProbe }
class ArteryUpdSendConsistencyWithOneLaneSpec
extends AbstractRemoteSendConsistencySpec(ConfigFactory.parseString("""
@@ -61,6 +72,17 @@ abstract class AbstractRemoteSendConsistencySpec(config: Config)
val addressB = address(systemB)
val rootB = RootActorPath(addressB)
+ private def actorRefBySelection(path: ActorPath) = {
+
+ val correlationId = Some(UUID.randomUUID().toString)
+ system.actorSelection(path) ! Identify(correlationId)
+
+ val actorIdentity = expectMsgType[ActorIdentity](5.seconds)
+ actorIdentity.correlationId shouldBe correlationId
+
+ actorIdentity.ref.get
+ }
+
"Artery" must {
"be able to identify a remote actor and ping it" in {
@@ -70,11 +92,9 @@ abstract class AbstractRemoteSendConsistencySpec(config: Config)
}
}), "echo")
- val echoSel = system.actorSelection(rootB / "user" / "echo")
- val echoRef = {
- system.actorSelection(rootB / "user" / "echo") ! Identify(None)
- expectMsgType[ActorIdentity](5.seconds).ref.get
- }
+ val actorPath = rootB / "user" / "echo"
+ val echoSel = system.actorSelection(actorPath)
+ val echoRef = actorRefBySelection(actorPath)
echoRef ! "ping"
expectMsg("pong")
@@ -118,18 +138,9 @@ abstract class AbstractRemoteSendConsistencySpec(config: Config)
systemB.actorOf(TestActors.echoActorProps, "echoB")
systemB.actorOf(TestActors.echoActorProps, "echoC")
- val remoteRefA = {
- system.actorSelection(rootB / "user" / "echoA") ! Identify(None)
- expectMsgType[ActorIdentity].ref.get
- }
- val remoteRefB = {
- system.actorSelection(rootB / "user" / "echoB") ! Identify(None)
- expectMsgType[ActorIdentity].ref.get
- }
- val remoteRefC = {
- system.actorSelection(rootB / "user" / "echoC") ! Identify(None)
- expectMsgType[ActorIdentity].ref.get
- }
+ val remoteRefA = actorRefBySelection(rootB / "user" / "echoA")
+ val remoteRefB = actorRefBySelection(rootB / "user" / "echoB")
+ val remoteRefC = actorRefBySelection(rootB / "user" / "echoC")
def senderProps(remoteRef: ActorRef) =
Props(new Actor {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala
index 96315afa57..364bd407e5 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala
@@ -7,10 +7,11 @@ package akka.remote.artery
import scala.concurrent.duration._
import scala.language.postfixOps
+import com.typesafe.config.ConfigFactory
+
import akka.actor._
import akka.remote._
import akka.testkit._
-import com.typesafe.config.ConfigFactory
object RemoteWatcherSpec {
@@ -46,7 +47,7 @@ object RemoteWatcherSpec {
class TestRemoteWatcher(heartbeatExpectedResponseAfter: FiniteDuration)
extends RemoteWatcher(
- createFailureDetector,
+ createFailureDetector(),
heartbeatInterval = TurnOff,
unreachableReaperInterval = TurnOff,
heartbeatExpectedResponseAfter = heartbeatExpectedResponseAfter) {
@@ -99,13 +100,13 @@ class RemoteWatcherSpec
"A RemoteWatcher" must {
"have correct interaction when watching" in {
- val monitorA = system.actorOf(Props[TestRemoteWatcher], "monitor1")
+ val monitorA = system.actorOf(Props[TestRemoteWatcher](), "monitor1")
val monitorB = createRemoteActor(Props(classOf[TestActorProxy], testActor), "monitor1")
- val a1 = system.actorOf(Props[MyActor], "a1").asInstanceOf[InternalActorRef]
- val a2 = system.actorOf(Props[MyActor], "a2").asInstanceOf[InternalActorRef]
- val b1 = createRemoteActor(Props[MyActor], "b1")
- val b2 = createRemoteActor(Props[MyActor], "b2")
+ val a1 = system.actorOf(Props[MyActor](), "a1").asInstanceOf[InternalActorRef]
+ val a2 = system.actorOf(Props[MyActor](), "a2").asInstanceOf[InternalActorRef]
+ val b1 = createRemoteActor(Props[MyActor](), "b1")
+ val b2 = createRemoteActor(Props[MyActor](), "b2")
monitorA ! WatchRemote(b1, a1)
monitorA ! WatchRemote(b2, a1)
@@ -163,11 +164,11 @@ class RemoteWatcherSpec
system.eventStream.subscribe(p.ref, classOf[TestRemoteWatcher.AddressTerm])
system.eventStream.subscribe(q.ref, classOf[TestRemoteWatcher.Quarantined])
- val monitorA = system.actorOf(Props[TestRemoteWatcher], "monitor4")
+ val monitorA = system.actorOf(Props[TestRemoteWatcher](), "monitor4")
val monitorB = createRemoteActor(Props(classOf[TestActorProxy], testActor), "monitor4")
- val a = system.actorOf(Props[MyActor], "a4").asInstanceOf[InternalActorRef]
- val b = createRemoteActor(Props[MyActor], "b4")
+ val a = system.actorOf(Props[MyActor](), "a4").asInstanceOf[InternalActorRef]
+ val b = createRemoteActor(Props[MyActor](), "b4")
monitorA ! WatchRemote(b, a)
@@ -204,8 +205,8 @@ class RemoteWatcherSpec
val monitorA = system.actorOf(Props(classOf[TestRemoteWatcher], heartbeatExpectedResponseAfter), "monitor5")
createRemoteActor(Props(classOf[TestActorProxy], testActor), "monitor5")
- val a = system.actorOf(Props[MyActor], "a5").asInstanceOf[InternalActorRef]
- val b = createRemoteActor(Props[MyActor], "b5")
+ val a = system.actorOf(Props[MyActor](), "a5").asInstanceOf[InternalActorRef]
+ val b = createRemoteActor(Props[MyActor](), "b5")
monitorA ! WatchRemote(b, a)
@@ -235,11 +236,11 @@ class RemoteWatcherSpec
system.eventStream.subscribe(p.ref, classOf[TestRemoteWatcher.AddressTerm])
system.eventStream.subscribe(q.ref, classOf[TestRemoteWatcher.Quarantined])
- val monitorA = system.actorOf(Props[TestRemoteWatcher], "monitor6")
+ val monitorA = system.actorOf(Props[TestRemoteWatcher](), "monitor6")
val monitorB = createRemoteActor(Props(classOf[TestActorProxy], testActor), "monitor6")
- val a = system.actorOf(Props[MyActor], "a6").asInstanceOf[InternalActorRef]
- val b = createRemoteActor(Props[MyActor], "b6")
+ val a = system.actorOf(Props[MyActor](), "a6").asInstanceOf[InternalActorRef]
+ val b = createRemoteActor(Props[MyActor](), "b6")
monitorA ! WatchRemote(b, a)
@@ -271,7 +272,7 @@ class RemoteWatcherSpec
expectNoMessage(2 seconds)
// assume that connection comes up again, or remote system is restarted
- val c = createRemoteActor(Props[MyActor], "c6")
+ val c = createRemoteActor(Props[MyActor](), "c6")
monitorA ! WatchRemote(c, a)
diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemotingFlightRecorderSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemotingFlightRecorderSpec.scala
index 5a769db8d1..deef13db0c 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/RemotingFlightRecorderSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/RemotingFlightRecorderSpec.scala
@@ -4,9 +4,10 @@
package akka.remote.artery
+import org.scalatest.matchers.should.Matchers
+
import akka.testkit.AkkaSpec
import akka.util.JavaVersion
-import org.scalatest.matchers.should.Matchers
class RemotingFlightRecorderSpec extends AkkaSpec with Matchers {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/RollingEventLogSimulationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RollingEventLogSimulationSpec.scala
index 1dac7d4e1b..c72db6ae4a 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/RollingEventLogSimulationSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/RollingEventLogSimulationSpec.scala
@@ -4,12 +4,12 @@
package akka.remote.artery
-import akka.testkit.AkkaSpec
-
import scala.annotation.tailrec
import scala.util.Random
import scala.util.control.NonFatal
+import akka.testkit.AkkaSpec
+
/*
* This test is a simulation of the actual concurrent rolling log implemented in SnapshottableRollingEventLog. It
* is not possible to test the implementation to such extent than this simulation allows, however, the two implementations
diff --git a/akka-remote/src/test/scala/akka/remote/artery/SendQueueSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SendQueueSpec.scala
index 752d29e46e..0c6aa84926 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/SendQueueSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/SendQueueSpec.scala
@@ -6,6 +6,10 @@ package akka.remote.artery
import java.util.Queue
+import scala.concurrent.duration._
+
+import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
+
import akka.actor.Actor
import akka.actor.Props
import akka.stream.scaladsl.Keep
@@ -14,9 +18,6 @@ import akka.stream.testkit.TestSubscriber
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
-import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
-
-import scala.concurrent.duration._
object SendQueueSpec {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala
index 8193ac71cf..790dab8a6d 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala
@@ -5,9 +5,9 @@
package akka.remote.artery
import akka.actor.{ ActorIdentity, Identify, RootActorPath }
+import akka.testkit.EventFilter
import akka.testkit.ImplicitSender
import akka.testkit.TestActors
-import akka.testkit.EventFilter
object SerializationErrorSpec {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala
index 085a5aa14a..60ceb4490e 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala
@@ -6,6 +6,12 @@ package akka.remote.artery
import java.util.concurrent.ThreadLocalRandom
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+
import akka.NotUsed
import akka.actor.ActorIdentity
import akka.actor.ActorSystem
@@ -26,11 +32,6 @@ import akka.testkit.TestActors
import akka.testkit.TestEvent
import akka.testkit.TestProbe
import akka.util.OptionVal
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
object SystemMessageDeliverySpec {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala
index b180368cee..76bffceaaf 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala
@@ -10,6 +10,8 @@ import java.util.concurrent.ThreadLocalRandom
import scala.concurrent.Future
+import com.typesafe.config.ConfigFactory
+
import akka.Done
import akka.actor.ActorRef
import akka.actor.Address
@@ -18,7 +20,6 @@ import akka.remote.UniqueAddress
import akka.remote.artery.InboundControlJunction.ControlMessageObserver
import akka.remote.artery.InboundControlJunction.ControlMessageSubject
import akka.util.OptionVal
-import com.typesafe.config.ConfigFactory
private[remote] class TestInboundContext(
override val localAddress: UniqueAddress,
diff --git a/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala
index a3ac7e8004..fad12b330a 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala
@@ -7,23 +7,24 @@ package akka.remote.artery
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.ActorRef
+import akka.actor.ActorSelection
import akka.actor.Deploy
import akka.actor.Identify
import akka.actor.PoisonPill
import akka.actor.Props
import akka.actor.RootActorPath
import akka.actor.Terminated
-import akka.testkit.ImplicitSender
-import akka.testkit.TestProbe
-import akka.actor.ActorSelection
-import akka.testkit.TestEvent
import akka.event.Logging
import akka.remote.RARP
import akka.serialization.jackson.CborSerializable
import akka.testkit.EventFilter
+import akka.testkit.ImplicitSender
+import akka.testkit.TestEvent
+import akka.testkit.TestProbe
object UntrustedSpec {
final case class IdentifyReq(path: String) extends CborSerializable
diff --git a/akka-remote/src/test/scala/akka/remote/artery/aeron/AeronSinkSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/aeron/AeronSinkSpec.scala
index 98f6c4909d..dc603334a8 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/aeron/AeronSinkSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/aeron/AeronSinkSpec.scala
@@ -7,6 +7,14 @@ package aeron
import java.io.File
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
+import io.aeron.Aeron
+import io.aeron.driver.MediaDriver
+import org.agrona.IoUtil
+
import akka.actor.ExtendedActorSystem
import akka.remote.artery.aeron.AeronSink.GaveUpMessageException
import akka.stream.scaladsl.Sink
@@ -14,13 +22,6 @@ import akka.stream.scaladsl.Source
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import akka.testkit.SocketUtil
-import io.aeron.Aeron
-import io.aeron.driver.MediaDriver
-import org.agrona.IoUtil
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
-import scala.util.control.NoStackTrace
class AeronSinkSpec extends AkkaSpec("""
akka.stream.materializer.debug.fuzzing-mode = on
diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala
index 28025a6b49..199591739e 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala
@@ -4,16 +4,17 @@
package akka.remote.artery.compress
+import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
-import akka.actor._
-import akka.remote.artery.compress.CompressionProtocol.Events
-import akka.testkit._
import com.typesafe.config.ConfigFactory
-import scala.concurrent.duration._
+import akka.actor._
import akka.actor.ExtendedActorSystem
-import akka.serialization.SerializerWithStringManifest
import akka.remote.artery.ArteryMultiNodeSpec
+import akka.remote.artery.compress.CompressionProtocol.Events
+import akka.serialization.SerializerWithStringManifest
+import akka.testkit._
object CompressionIntegrationSpec {
@@ -134,6 +135,72 @@ class CompressionIntegrationSpec
}
}
+ "not be advertised if ActorRef compression disabled" in {
+ val config = """
+ akka.remote.artery.advanced.compression.actor-refs.max = off
+ akka.remote.artery.advanced.compression {
+ actor-refs.advertisement-interval = 50 ms
+ manifests.advertisement-interval = 50 ms
+ }
+ """
+ val systemC = newRemoteSystem(Some(config))
+ val systemD = newRemoteSystem(Some(config))
+ val cRefProbe = TestProbe()(systemC)
+ val dRefProbe = TestProbe()(systemD)
+ systemC.eventStream.subscribe(cRefProbe.ref, classOf[CompressionProtocol.Events.ReceivedActorRefCompressionTable])
+ systemD.eventStream.subscribe(dRefProbe.ref, classOf[CompressionProtocol.Events.ReceivedActorRefCompressionTable])
+
+ systemD.actorOf(TestActors.echoActorProps, "echo")
+
+ val cProbe = TestProbe()(systemC)
+ systemC.actorSelection(rootActorPath(systemD) / "user" / "echo").tell(Identify(None), cProbe.ref)
+ val echoRefD = cProbe.expectMsgType[ActorIdentity].ref.get
+
+ (1 to messagesToExchange).foreach { _ =>
+ echoRefD.tell(TestMessage("hello"), cProbe.ref)
+ }
+ cProbe.receiveN(messagesToExchange) // the replies
+ cRefProbe.expectNoMessage(100.millis)
+ dRefProbe.expectNoMessage(100.millis)
+
+ shutdown(systemC)
+ shutdown(systemD)
+ }
+
+ "not be advertised if manifest compression disabled" in {
+ val config = """
+ akka.remote.artery.advanced.compression.manifests.max = off
+ akka.remote.artery.advanced.compression {
+ actor-refs.advertisement-interval = 50 ms
+ manifests.advertisement-interval = 50 ms
+ }
+ """
+ val systemC = newRemoteSystem(Some(config))
+ val systemD = newRemoteSystem(Some(config))
+ val cManifestProbe = TestProbe()(systemC)
+ val dManifestProbe = TestProbe()(systemD)
+ systemC.eventStream
+ .subscribe(cManifestProbe.ref, classOf[CompressionProtocol.Events.ReceivedClassManifestCompressionTable])
+ systemD.eventStream
+ .subscribe(dManifestProbe.ref, classOf[CompressionProtocol.Events.ReceivedClassManifestCompressionTable])
+
+ systemD.actorOf(TestActors.echoActorProps, "echo")
+
+ val cProbe = TestProbe()(systemC)
+ systemC.actorSelection(rootActorPath(systemD) / "user" / "echo").tell(Identify(None), cProbe.ref)
+ val echoRefD = cProbe.expectMsgType[ActorIdentity].ref.get
+
+ (1 to messagesToExchange).foreach { _ =>
+ echoRefD.tell(TestMessage("hello"), cProbe.ref)
+ }
+ cProbe.receiveN(messagesToExchange) // the replies
+ cManifestProbe.expectNoMessage(100.millis)
+ dManifestProbe.expectNoMessage(100.millis)
+
+ shutdown(systemC)
+ shutdown(systemD)
+ }
+
}
"handle noSender sender" in {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala
index 764c9d2587..f046cc1632 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala
@@ -4,18 +4,19 @@
package akka.remote.artery.compress
-import akka.actor.{ ActorIdentity, ActorSystem, Identify }
-import akka.pattern.ask
-import akka.remote.RARP
-import akka.remote.artery.compress.CompressionProtocol.Events.{ Event, ReceivedActorRefCompressionTable }
-import akka.remote.artery.{ ArteryMultiNodeSpec, ArterySpecSupport, ArteryTransport }
-import akka.testkit._
-import akka.util.Timeout
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
-import scala.concurrent.Await
-import scala.concurrent.duration._
+import akka.actor.{ ActorIdentity, ActorSystem, Identify }
+import akka.pattern.ask
+import akka.remote.RARP
+import akka.remote.artery.{ ArteryMultiNodeSpec, ArterySpecSupport, ArteryTransport }
+import akka.remote.artery.compress.CompressionProtocol.Events.{ Event, ReceivedActorRefCompressionTable }
+import akka.testkit._
+import akka.util.Timeout
object HandshakeShouldDropCompressionTableSpec {
val commonConfig = ConfigFactory.parseString(s"""
@@ -39,7 +40,7 @@ class HandshakeShouldDropCompressionTableSpec
with ImplicitSender
with BeforeAndAfter {
- implicit val t = Timeout(3.seconds)
+ implicit val t: Timeout = Timeout(3.seconds)
var systemB: ActorSystem = null
val portB = freePort()
diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/HeavyHittersSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/HeavyHittersSpec.scala
index 2ea9aec76d..55ed187ea9 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/compress/HeavyHittersSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HeavyHittersSpec.scala
@@ -161,5 +161,15 @@ class HeavyHittersSpec extends AnyWordSpecLike with Matchers {
hitters.lowestHitterWeight should ===(3)
}
+ "be disabled with max=0" in {
+ val hitters = new TopHeavyHitters[String](0)
+ hitters.update("A", 10) shouldBe true
+ hitters.iterator.toSet should ===(Set.empty)
+
+ hitters.update("B", 5) shouldBe false
+ hitters.update("C", 15) shouldBe true
+ hitters.iterator.toSet should ===(Set.empty)
+ }
+
}
}
diff --git a/akka-remote/src/test/scala/akka/remote/artery/tcp/TcpFramingSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/tcp/TcpFramingSpec.scala
index 5c622e4e36..2d3f738e97 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/tcp/TcpFramingSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/tcp/TcpFramingSpec.scala
@@ -5,17 +5,17 @@
package akka.remote.artery
package tcp
+import scala.util.Random
+
import akka.stream.impl.io.ByteStringParser.ParsingException
-import akka.stream.scaladsl.Framing.FramingException
import akka.stream.scaladsl.Flow
+import akka.stream.scaladsl.Framing.FramingException
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import akka.util.ByteString
-import scala.util.Random
-
class TcpFramingSpec extends AkkaSpec("""
akka.stream.materializer.debug.fuzzing-mode = on
""") with ImplicitSender {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/tcp/TlsTcpSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/tcp/TlsTcpSpec.scala
index bcaed57aa1..4925446996 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/tcp/TlsTcpSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/tcp/TlsTcpSpec.scala
@@ -8,24 +8,24 @@ package tcp
import java.io.ByteArrayOutputStream
import java.security.NoSuchAlgorithmException
import java.util.zip.GZIPOutputStream
+import javax.net.ssl.SSLEngine
import scala.concurrent.duration._
-import akka.actor.ActorRef
-import akka.actor.ActorPath
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ActorIdentity
+import akka.actor.ActorPath
+import akka.actor.ActorRef
import akka.actor.ExtendedActorSystem
import akka.actor.Identify
import akka.actor.RootActorPath
import akka.actor.setup.ActorSystemSetup
+import akka.testkit.EventFilter
import akka.testkit.ImplicitSender
import akka.testkit.TestActors
import akka.testkit.TestProbe
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import javax.net.ssl.SSLEngine
-
-import akka.testkit.EventFilter
class TlsTcpWithDefaultConfigSpec extends TlsTcpSpec(ConfigFactory.empty())
@@ -33,7 +33,7 @@ class TlsTcpWithSHA1PRNGSpec
extends TlsTcpSpec(ConfigFactory.parseString("""
akka.remote.artery.ssl.config-ssl-engine {
random-number-generator = "SHA1PRNG"
- enabled-algorithms = ["TLS_RSA_WITH_AES_128_CBC_SHA"]
+ enabled-algorithms = ["TLS_DHE_RSA_WITH_AES_256_GCM_SHA384"]
}
"""))
@@ -41,7 +41,7 @@ class TlsTcpWithDefaultRNGSecureSpec
extends TlsTcpSpec(ConfigFactory.parseString("""
akka.remote.artery.ssl.config-ssl-engine {
random-number-generator = ""
- enabled-algorithms = ["TLS_RSA_WITH_AES_128_CBC_SHA"]
+ enabled-algorithms = ["TLS_DHE_RSA_WITH_AES_256_GCM_SHA384"]
}
"""))
diff --git a/akka-remote/src/test/scala/akka/remote/classic/ActorsLeakSpec.scala b/akka-remote/src/test/scala/akka/remote/classic/ActorsLeakSpec.scala
index 80bf9b5f96..304d111f3b 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/ActorsLeakSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/ActorsLeakSpec.scala
@@ -6,19 +6,20 @@ package akka.remote.classic
import java.util.concurrent.TimeoutException
-import akka.actor._
-import akka.actor.dungeon.ChildrenContainer
-import akka.remote.transport.ThrottlerTransportAdapter.ForceDisassociate
-import akka.remote.{ AddressUidExtension, RARP }
-import akka.testkit.TestActors.EchoActor
-import akka.testkit._
-import com.github.ghik.silencer.silent
-import com.typesafe.config.ConfigFactory
-
import scala.collection.immutable
import scala.concurrent.Await
import scala.concurrent.duration._
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
+import akka.actor._
+import akka.actor.dungeon.ChildrenContainer
+import akka.remote.{ AddressUidExtension, RARP }
+import akka.remote.transport.ThrottlerTransportAdapter.ForceDisassociate
+import akka.testkit._
+import akka.testkit.TestActors.EchoActor
+
object ActorsLeakSpec {
val config = ConfigFactory.parseString("""
@@ -75,7 +76,7 @@ class ActorsLeakSpec extends AkkaSpec(ActorsLeakSpec.config) with ImplicitSender
"Remoting" must {
"not leak actors" in {
- system.actorOf(Props[EchoActor], "echo")
+ system.actorOf(Props[EchoActor](), "echo")
val echoPath = RootActorPath(RARP(system).provider.getDefaultAddress) / "user" / "echo"
val targets = List("/system/endpointManager", "/system/transports").map { path =>
@@ -120,7 +121,7 @@ class ActorsLeakSpec extends AkkaSpec(ActorsLeakSpec.config) with ImplicitSender
try {
val remoteAddress = RARP(remoteSystem).provider.getDefaultAddress
- remoteSystem.actorOf(Props[StoppableActor], "stoppable")
+ remoteSystem.actorOf(Props[StoppableActor](), "stoppable")
// the message from remote to local will cause inbound connection established
val probe = TestProbe()(remoteSystem)
@@ -185,7 +186,7 @@ class ActorsLeakSpec extends AkkaSpec(ActorsLeakSpec.config) with ImplicitSender
ActorSystem("remote", ConfigFactory.parseString("akka.remote.classic.netty.tcp.port = 0").withFallback(config))
val remoteAddress = RARP(remoteSystem).provider.getDefaultAddress
- remoteSystem.actorOf(Props[StoppableActor], "stoppable")
+ remoteSystem.actorOf(Props[StoppableActor](), "stoppable")
try {
val probe = TestProbe()(remoteSystem)
diff --git a/akka-remote/src/test/scala/akka/remote/classic/RemoteDeathWatchSpec.scala b/akka-remote/src/test/scala/akka/remote/classic/RemoteDeathWatchSpec.scala
index e173b21852..8cc6326ce4 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/RemoteDeathWatchSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/RemoteDeathWatchSpec.scala
@@ -4,14 +4,15 @@
package akka.remote.classic
+import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
import akka.actor.{ RootActorPath, _ }
import akka.event.Logging.Warning
import akka.remote.{ QuarantinedEvent, RARP, RemoteActorRef }
import akka.testkit.{ SocketUtil, _ }
-import com.github.ghik.silencer.silent
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.duration._
@silent // classic deprecated
class RemoteDeathWatchSpec
diff --git a/akka-remote/src/test/scala/akka/remote/classic/RemoteDeploymentWhitelistSpec.scala b/akka-remote/src/test/scala/akka/remote/classic/RemoteDeploymentWhitelistSpec.scala
index 7dae281431..9fdfd384f2 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/RemoteDeploymentWhitelistSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/RemoteDeploymentWhitelistSpec.scala
@@ -4,14 +4,15 @@
package akka.remote.classic
+import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config._
+
import akka.actor._
import akka.remote.EndpointException
import akka.remote.transport._
import akka.testkit._
-import com.typesafe.config._
-import scala.concurrent.duration._
-
-import com.github.ghik.silencer.silent
// relies on test transport
object RemoteDeploymentWhitelistSpec {
@@ -149,7 +150,7 @@ class RemoteDeploymentWhitelistSpec
"RemoteDeployment Whitelist" must {
"allow deploying Echo actor (included in whitelist)" in {
- val r = system.actorOf(Props[EchoWhitelisted], "blub")
+ val r = system.actorOf(Props[EchoWhitelisted](), "blub")
r.path.toString should ===(
s"akka.test://remote-sys@localhost:12346/remote/akka.test/${getClass.getSimpleName}@localhost:12345/user/blub")
r ! 42
@@ -165,7 +166,7 @@ class RemoteDeploymentWhitelistSpec
}
"not deploy actor not listed in whitelist" in {
- val r = system.actorOf(Props[EchoNotWhitelisted], "danger-mouse")
+ val r = system.actorOf(Props[EchoNotWhitelisted](), "danger-mouse")
r.path.toString should ===(
s"akka.test://remote-sys@localhost:12346/remote/akka.test/${getClass.getSimpleName}@localhost:12345/user/danger-mouse")
r ! 42
diff --git a/akka-remote/src/test/scala/akka/remote/classic/RemoteInitErrorSpec.scala b/akka-remote/src/test/scala/akka/remote/classic/RemoteInitErrorSpec.scala
index 0a778fcf7e..b2676596b5 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/RemoteInitErrorSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/RemoteInitErrorSpec.scala
@@ -4,18 +4,19 @@
package akka.remote.classic
-import akka.actor.ActorSystem
-import com.typesafe.config.ConfigFactory
-import org.scalatest.concurrent.Eventually._
-
-import akka.util.ccompat.JavaConverters._
import scala.collection.mutable.Set
import scala.concurrent.duration._
import scala.language.postfixOps
import scala.util.control.NonFatal
+
+import com.typesafe.config.ConfigFactory
+import org.scalatest.concurrent.Eventually._
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.ActorSystem
+import akka.util.ccompat.JavaConverters._
+
/**
* The 192.0.2.1 is a Documentation IP-address and should not be used at all
* by any network node. Therefore we assume here that the initialization of
diff --git a/akka-remote/src/test/scala/akka/remote/classic/RemoteSettingsSpec.scala b/akka-remote/src/test/scala/akka/remote/classic/RemoteSettingsSpec.scala
index 1dd61e12eb..fc4be679f9 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/RemoteSettingsSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/RemoteSettingsSpec.scala
@@ -4,12 +4,13 @@
package akka.remote.classic
-import akka.remote.RemoteSettings
import com.github.ghik.silencer.silent
import com.typesafe.config.ConfigFactory
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.remote.RemoteSettings
+
@silent("deprecated")
class RemoteSettingsSpec extends AnyWordSpec with Matchers {
diff --git a/akka-remote/src/test/scala/akka/remote/classic/RemoteWatcherSpec.scala b/akka-remote/src/test/scala/akka/remote/classic/RemoteWatcherSpec.scala
index 64cb8cb37c..7aa04ff9ba 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/RemoteWatcherSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/RemoteWatcherSpec.scala
@@ -4,13 +4,14 @@
package akka.remote.classic
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+import com.github.ghik.silencer.silent
+
import akka.actor._
import akka.remote._
import akka.testkit._
-import com.github.ghik.silencer.silent
-
-import scala.concurrent.duration._
-import scala.language.postfixOps
object RemoteWatcherSpec {
@@ -46,7 +47,7 @@ object RemoteWatcherSpec {
class TestRemoteWatcher(heartbeatExpectedResponseAfter: FiniteDuration)
extends RemoteWatcher(
- createFailureDetector,
+ createFailureDetector(),
heartbeatInterval = TurnOff,
unreachableReaperInterval = TurnOff,
heartbeatExpectedResponseAfter = heartbeatExpectedResponseAfter) {
@@ -115,13 +116,13 @@ class RemoteWatcherSpec extends AkkaSpec("""
"A RemoteWatcher" must {
"have correct interaction when watching" in {
- val monitorA = system.actorOf(Props[TestRemoteWatcher], "monitor1")
+ val monitorA = system.actorOf(Props[TestRemoteWatcher](), "monitor1")
val monitorB = createRemoteActor(Props(classOf[TestActorProxy], testActor), "monitor1")
- val a1 = system.actorOf(Props[MyActor], "a1").asInstanceOf[InternalActorRef]
- val a2 = system.actorOf(Props[MyActor], "a2").asInstanceOf[InternalActorRef]
- val b1 = createRemoteActor(Props[MyActor], "b1")
- val b2 = createRemoteActor(Props[MyActor], "b2")
+ val a1 = system.actorOf(Props[MyActor](), "a1").asInstanceOf[InternalActorRef]
+ val a2 = system.actorOf(Props[MyActor](), "a2").asInstanceOf[InternalActorRef]
+ val b1 = createRemoteActor(Props[MyActor](), "b1")
+ val b2 = createRemoteActor(Props[MyActor](), "b2")
monitorA ! WatchRemote(b1, a1)
monitorA ! WatchRemote(b2, a1)
@@ -179,11 +180,11 @@ class RemoteWatcherSpec extends AkkaSpec("""
system.eventStream.subscribe(p.ref, classOf[TestRemoteWatcher.AddressTerm])
system.eventStream.subscribe(q.ref, classOf[TestRemoteWatcher.Quarantined])
- val monitorA = system.actorOf(Props[TestRemoteWatcher], "monitor4")
+ val monitorA = system.actorOf(Props[TestRemoteWatcher](), "monitor4")
val monitorB = createRemoteActor(Props(classOf[TestActorProxy], testActor), "monitor4")
- val a = system.actorOf(Props[MyActor], "a4").asInstanceOf[InternalActorRef]
- val b = createRemoteActor(Props[MyActor], "b4")
+ val a = system.actorOf(Props[MyActor](), "a4").asInstanceOf[InternalActorRef]
+ val b = createRemoteActor(Props[MyActor](), "b4")
monitorA ! WatchRemote(b, a)
@@ -220,8 +221,8 @@ class RemoteWatcherSpec extends AkkaSpec("""
val monitorA = system.actorOf(Props(classOf[TestRemoteWatcher], heartbeatExpectedResponseAfter), "monitor5")
createRemoteActor(Props(classOf[TestActorProxy], testActor), "monitor5")
- val a = system.actorOf(Props[MyActor], "a5").asInstanceOf[InternalActorRef]
- val b = createRemoteActor(Props[MyActor], "b5")
+ val a = system.actorOf(Props[MyActor](), "a5").asInstanceOf[InternalActorRef]
+ val b = createRemoteActor(Props[MyActor](), "b5")
monitorA ! WatchRemote(b, a)
@@ -251,11 +252,11 @@ class RemoteWatcherSpec extends AkkaSpec("""
system.eventStream.subscribe(p.ref, classOf[TestRemoteWatcher.AddressTerm])
system.eventStream.subscribe(q.ref, classOf[TestRemoteWatcher.Quarantined])
- val monitorA = system.actorOf(Props[TestRemoteWatcher], "monitor6")
+ val monitorA = system.actorOf(Props[TestRemoteWatcher](), "monitor6")
val monitorB = createRemoteActor(Props(classOf[TestActorProxy], testActor), "monitor6")
- val a = system.actorOf(Props[MyActor], "a6").asInstanceOf[InternalActorRef]
- val b = createRemoteActor(Props[MyActor], "b6")
+ val a = system.actorOf(Props[MyActor](), "a6").asInstanceOf[InternalActorRef]
+ val b = createRemoteActor(Props[MyActor](), "b6")
monitorA ! WatchRemote(b, a)
@@ -287,7 +288,7 @@ class RemoteWatcherSpec extends AkkaSpec("""
expectNoMessage(2 seconds)
// assume that connection comes up again, or remote system is restarted
- val c = createRemoteActor(Props[MyActor], "c6")
+ val c = createRemoteActor(Props[MyActor](), "c6")
monitorA ! WatchRemote(c, a)
diff --git a/akka-remote/src/test/scala/akka/remote/classic/RemotingSpec.scala b/akka-remote/src/test/scala/akka/remote/classic/RemotingSpec.scala
index 35a06d74a4..8ac42971c0 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/RemotingSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/RemotingSpec.scala
@@ -7,23 +7,23 @@ package akka.remote.classic
import java.io.NotSerializableException
import java.util.concurrent.ThreadLocalRandom
-import akka.actor._
-import akka.event.AddressTerminatedTopic
-import akka.pattern.ask
-import akka.remote.transport.AssociationHandle.{ HandleEvent, HandleEventListener }
-import akka.remote.transport.Transport.InvalidAssociationException
-import akka.remote.transport._
-import akka.remote._
-import akka.testkit.SocketUtil.temporaryServerAddress
-import akka.testkit._
-import akka.util.ByteString
+import scala.concurrent.{ Await, Future }
+import scala.concurrent.duration._
+import scala.util.control.NonFatal
+
import com.github.ghik.silencer.silent
import com.typesafe.config._
-import scala.concurrent.{ Await, Future }
-import scala.concurrent.duration._
-
-import scala.util.control.NonFatal
+import akka.actor._
+import akka.event.AddressTerminatedTopic
+import akka.pattern.ask
+import akka.remote._
+import akka.remote.transport._
+import akka.remote.transport.AssociationHandle.{ HandleEvent, HandleEventListener }
+import akka.remote.transport.Transport.InvalidAssociationException
+import akka.testkit._
+import akka.testkit.SocketUtil.temporaryServerAddress
+import akka.util.ByteString
object RemotingSpec {
@@ -33,7 +33,7 @@ object RemotingSpec {
var target: ActorRef = context.system.deadLetters
def receive = {
- case (_: Props, n: String) => sender() ! context.actorOf(Props[Echo1], n)
+ case (_: Props, n: String) => sender() ! context.actorOf(Props[Echo1](), n)
case ex: Exception => throw ex
case ActorSelReq(s) => sender() ! context.actorSelection(s)
case x => target = sender(); sender() ! x
@@ -73,7 +73,7 @@ object RemotingSpec {
key-password = "changeme"
trust-store-password = "changeme"
protocol = "TLSv1.2"
- enabled-algorithms = [TLS_RSA_WITH_AES_128_CBC_SHA]
+ enabled-algorithms = [TLS_RSA_WITH_AES_128_CBC_SHA, TLS_DHE_RSA_WITH_AES_256_GCM_SHA384]
}
common-netty-settings {
@@ -158,7 +158,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
sys.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].deployer.deploy(d)
}
- val remote = remoteSystem.actorOf(Props[Echo2], "echo")
+ val remote = remoteSystem.actorOf(Props[Echo2](), "echo")
val here = RARP(system).provider.resolveActorRef("akka.test://remote-sys@localhost:12346/user/echo")
@@ -226,7 +226,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
"support ask" in {
Await.result(here ? "ping", timeout.duration) match {
case ("pong", _: akka.pattern.PromiseActorRef) => // good
- case m => fail(m + " was not (pong, AskActorRef)")
+ case m => fail("" + m + " was not (pong, AskActorRef)")
}
}
@@ -246,7 +246,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
moreSystems.foreach { sys =>
sys.eventStream.publish(TestEvent
.Mute(EventFilter[EndpointDisassociatedException](), EventFilter.warning(pattern = "received dead letter.*")))
- sys.actorOf(Props[Echo2], name = "echo")
+ sys.actorOf(Props[Echo2](), name = "echo")
}
val moreRefs =
moreSystems.map(sys => system.actorSelection(RootActorPath(getOtherAddress(sys, "tcp")) / "user" / "echo"))
@@ -284,7 +284,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
}
"create and supervise children on remote node" in {
- val r = system.actorOf(Props[Echo1], "blub")
+ val r = system.actorOf(Props[Echo1](), "blub")
r.path.toString should ===(
"akka.test://remote-sys@localhost:12346/remote/akka.test/RemotingSpec@localhost:12345/user/blub")
r ! 42
@@ -300,7 +300,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
}
"not send to remote re-created actor with same name" in {
- val echo = remoteSystem.actorOf(Props[Echo1], "otherEcho1")
+ val echo = remoteSystem.actorOf(Props[Echo1](), "otherEcho1")
echo ! 71
expectMsg(71)
echo ! PoisonPill
@@ -308,7 +308,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
echo ! 72
expectNoMessage(1.second)
- val echo2 = remoteSystem.actorOf(Props[Echo1], "otherEcho1")
+ val echo2 = remoteSystem.actorOf(Props[Echo1](), "otherEcho1")
echo2 ! 73
expectMsg(73)
// msg to old ActorRef (different uid) should not get through
@@ -337,10 +337,10 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
}
}), "looker2")
// child is configured to be deployed on remoteSystem
- l ! ((Props[Echo1], "child"))
+ l ! ((Props[Echo1](), "child"))
val child = expectMsgType[ActorRef]
// grandchild is configured to be deployed on RemotingSpec (system)
- child ! ((Props[Echo1], "grandchild"))
+ child ! ((Props[Echo1](), "grandchild"))
val grandchild = expectMsgType[ActorRef]
grandchild.asInstanceOf[ActorRefScope].isLocal should ===(true)
grandchild ! 53
@@ -362,7 +362,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
expectMsgType[ActorSelection] ! Identify(None)
(expectMsgType[ActorIdentity].ref.get should be).theSameInstanceAs(l)
- grandchild ! ((Props[Echo1], "grandgrandchild"))
+ grandchild ! ((Props[Echo1](), "grandgrandchild"))
val grandgrandchild = expectMsgType[ActorRef]
system.actorSelection("/user/looker2/child") ! Identify("idReq1")
@@ -404,7 +404,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
child ! PoisonPill
expectMsg("postStop")
expectMsgType[Terminated].actor should ===(child)
- l ! ((Props[Echo1], "child"))
+ l ! ((Props[Echo1](), "child"))
val child2 = expectMsgType[ActorRef]
child2 ! Identify("idReq15")
expectMsg(ActorIdentity("idReq15", Some(child2)))
@@ -430,7 +430,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
}
"be able to use multiple transports and use the appropriate one (TCP)" in {
- val r = system.actorOf(Props[Echo1], "gonk")
+ val r = system.actorOf(Props[Echo1](), "gonk")
r.path.toString should ===(
s"akka.tcp://remote-sys@localhost:${port(remoteSystem, "tcp")}/remote/akka.tcp/RemotingSpec@localhost:${port(system, "tcp")}/user/gonk")
r ! 42
@@ -446,7 +446,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
}
"be able to use multiple transports and use the appropriate one (SSL)" in {
- val r = system.actorOf(Props[Echo1], "roghtaar")
+ val r = system.actorOf(Props[Echo1](), "roghtaar")
r.path.toString should ===(
s"akka.ssl.tcp://remote-sys@localhost:${port(remoteSystem, "ssl.tcp")}/remote/akka.ssl.tcp/RemotingSpec@localhost:${port(system, "ssl.tcp")}/user/roghtaar")
r ! 42
@@ -479,6 +479,8 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
}
"drop sent messages over payload size" in {
+ val droppedProbe = TestProbe()
+ system.eventStream.subscribe(droppedProbe.ref, classOf[Dropped])
val oversized = byteStringOfSize(maxPayloadBytes + 1)
EventFilter[OversizedPayloadException](pattern = ".*Discarding oversized payload sent.*", occurrences = 1)
.intercept {
@@ -486,6 +488,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
expectNoMessage(1.second) // No AssocitionErrorEvent should be published
}
}
+ droppedProbe.expectMsgType[Dropped].message should ===(oversized)
}
"drop received messages over payload size" in {
@@ -508,7 +511,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
.withFallback(remoteSystem.settings.config)
val otherSystem = ActorSystem("other-system", config)
try {
- val otherGuy = otherSystem.actorOf(Props[Echo2], "other-guy")
+ val otherGuy = otherSystem.actorOf(Props[Echo2](), "other-guy")
// check that we use the specified transport address instead of the default
val otherGuyRemoteTcp = otherGuy.path.toSerializationFormatWithAddress(getOtherAddress(otherSystem, "tcp"))
val remoteEchoHereTcp =
@@ -795,7 +798,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
try {
muteSystem(otherSystem)
probe.expectNoMessage(2.seconds)
- otherSystem.actorOf(Props[Echo2], "echo")
+ otherSystem.actorOf(Props[Echo2](), "echo")
within(5.seconds) {
awaitAssert {
otherSelection.tell("ping", probe.ref)
@@ -821,7 +824,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
muteSystem(thisSystem)
val thisProbe = new TestProbe(thisSystem)
val thisSender = thisProbe.ref
- thisSystem.actorOf(Props[Echo2], "echo")
+ thisSystem.actorOf(Props[Echo2](), "echo")
val otherAddress = temporaryServerAddress()
val otherConfig = ConfigFactory.parseString(s"""
akka.remote.classic.netty.tcp.port = ${otherAddress.getPort}
diff --git a/akka-remote/src/test/scala/akka/remote/classic/UntrustedSpec.scala b/akka-remote/src/test/scala/akka/remote/classic/UntrustedSpec.scala
index 3a9322c8f8..9bb86dfd56 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/UntrustedSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/UntrustedSpec.scala
@@ -7,9 +7,11 @@ package akka.remote.classic
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.ActorRef
+import akka.actor.ActorSelection
import akka.actor.ActorSystem
import akka.actor.Deploy
import akka.actor.ExtendedActorSystem
@@ -18,14 +20,13 @@ import akka.actor.PoisonPill
import akka.actor.Props
import akka.actor.RootActorPath
import akka.actor.Terminated
-import akka.testkit.AkkaSpec
-import akka.testkit.ImplicitSender
-import akka.testkit.TestProbe
-import akka.actor.ActorSelection
-import akka.testkit.TestEvent
import akka.event.Logging
+import akka.testkit.AkkaSpec
import akka.testkit.EventFilter
+import akka.testkit.ImplicitSender
import akka.testkit.JavaSerializable
+import akka.testkit.TestEvent
+import akka.testkit.TestProbe
object UntrustedSpec {
final case class IdentifyReq(path: String) extends JavaSerializable
diff --git a/akka-remote/src/test/scala/akka/remote/classic/transport/AkkaProtocolSpec.scala b/akka-remote/src/test/scala/akka/remote/classic/transport/AkkaProtocolSpec.scala
index c8ef455702..d229291759 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/transport/AkkaProtocolSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/transport/AkkaProtocolSpec.scala
@@ -6,9 +6,17 @@ package akka.remote.classic.transport
import java.util.concurrent.TimeoutException
+import scala.concurrent.{ Await, Promise }
+import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Address
import akka.protobufv3.internal.{ ByteString => PByteString }
+import akka.remote.{ FailureDetector, WireFormats }
import akka.remote.classic.transport.AkkaProtocolSpec.TestFailureDetector
+import akka.remote.transport.{ AssociationRegistry => _, _ }
import akka.remote.transport.AkkaPduCodec.{ Associate, Disassociate, Heartbeat }
import akka.remote.transport.AssociationHandle.{
ActorHandleEventListener,
@@ -19,15 +27,8 @@ import akka.remote.transport.AssociationHandle.{
import akka.remote.transport.ProtocolStateActor
import akka.remote.transport.TestTransport._
import akka.remote.transport.Transport._
-import akka.remote.transport.{ AssociationRegistry => _, _ }
-import akka.remote.{ FailureDetector, WireFormats }
import akka.testkit.{ AkkaSpec, ImplicitSender }
import akka.util.{ ByteString, OptionVal }
-import com.typesafe.config.ConfigFactory
-import scala.concurrent.duration._
-import scala.concurrent.{ Await, Promise }
-
-import com.github.ghik.silencer.silent
object AkkaProtocolSpec {
diff --git a/akka-remote/src/test/scala/akka/remote/classic/transport/AkkaProtocolStressTest.scala b/akka-remote/src/test/scala/akka/remote/classic/transport/AkkaProtocolStressTest.scala
index bfd6ba8b50..4a68f478b3 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/transport/AkkaProtocolStressTest.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/transport/AkkaProtocolStressTest.scala
@@ -4,16 +4,17 @@
package akka.remote.classic.transport
-import akka.actor._
-import akka.remote.classic.transport.AkkaProtocolStressTest._
-import akka.remote.transport.FailureInjectorTransportAdapter.{ Drop, One }
-import akka.remote.{ EndpointException, RARP }
-import akka.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender, TimingTest, _ }
-import com.typesafe.config.{ Config, ConfigFactory }
-
import scala.concurrent.Await
import scala.concurrent.duration._
+import com.typesafe.config.{ Config, ConfigFactory }
+
+import akka.actor._
+import akka.remote.{ EndpointException, RARP }
+import akka.remote.classic.transport.AkkaProtocolStressTest._
+import akka.remote.transport.FailureInjectorTransportAdapter.{ Drop, One }
+import akka.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender, TimingTest, _ }
+
object AkkaProtocolStressTest {
val configA: Config =
ConfigFactory.parseString("""
diff --git a/akka-remote/src/test/scala/akka/remote/classic/transport/GenericTransportSpec.scala b/akka-remote/src/test/scala/akka/remote/classic/transport/GenericTransportSpec.scala
index 8b4811fbdf..99342a486b 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/transport/GenericTransportSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/transport/GenericTransportSpec.scala
@@ -4,18 +4,19 @@
package akka.remote.classic.transport
-import akka.actor.{ Address, ExtendedActorSystem }
-import akka.remote.RemoteActorRefProvider
-import akka.remote.transport.AssociationHandle.{ ActorHandleEventListener, Disassociated, InboundPayload }
-import akka.remote.transport.TestTransport.{ AssociateAttempt, DisassociateAttempt, ListenAttempt, WriteAttempt, _ }
-import akka.remote.transport.Transport._
-import akka.remote.transport.{ AssociationRegistry => _, _ }
-import akka.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender }
-import akka.util.ByteString
import scala.concurrent.{ Await, Future }
import com.github.ghik.silencer.silent
+import akka.actor.{ Address, ExtendedActorSystem }
+import akka.remote.RemoteActorRefProvider
+import akka.remote.transport.{ AssociationRegistry => _, _ }
+import akka.remote.transport.AssociationHandle.{ ActorHandleEventListener, Disassociated, InboundPayload }
+import akka.remote.transport.TestTransport.{ AssociateAttempt, DisassociateAttempt, ListenAttempt, WriteAttempt, _ }
+import akka.remote.transport.Transport._
+import akka.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender }
+import akka.util.ByteString
+
@silent("deprecated")
abstract class GenericTransportSpec(withAkkaProtocol: Boolean = false)
extends AkkaSpec("""
diff --git a/akka-remote/src/test/scala/akka/remote/classic/transport/SwitchableLoggedBehaviorSpec.scala b/akka-remote/src/test/scala/akka/remote/classic/transport/SwitchableLoggedBehaviorSpec.scala
index d51c9325ba..6a96d0afb8 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/transport/SwitchableLoggedBehaviorSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/transport/SwitchableLoggedBehaviorSpec.scala
@@ -4,14 +4,14 @@
package akka.remote.classic.transport
-import akka.AkkaException
-import akka.remote.transport.TestTransport.SwitchableLoggedBehavior
-import akka.testkit.{ AkkaSpec, DefaultTimeout }
-
import scala.concurrent.{ Await, Future, Promise }
import scala.util.Failure
import scala.util.control.NoStackTrace
+import akka.AkkaException
+import akka.remote.transport.TestTransport.SwitchableLoggedBehavior
+import akka.testkit.{ AkkaSpec, DefaultTimeout }
+
object SwitchableLoggedBehaviorSpec {
object TestException extends AkkaException("Test exception") with NoStackTrace
}
diff --git a/akka-remote/src/test/scala/akka/remote/classic/transport/SystemMessageDeliveryStressTest.scala b/akka-remote/src/test/scala/akka/remote/classic/transport/SystemMessageDeliveryStressTest.scala
index 46eea7eb30..0f3d581287 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/transport/SystemMessageDeliveryStressTest.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/transport/SystemMessageDeliveryStressTest.scala
@@ -4,18 +4,19 @@
package akka.remote.classic.transport
-import akka.actor.{ Actor, ActorRef, ActorSystem, ExtendedActorSystem, Props, RootActorPath, _ }
-import akka.dispatch.sysmsg.{ Failed, SystemMessage }
-import akka.remote.transport.AssociationHandle
-import akka.remote.transport.FailureInjectorTransportAdapter.{ Drop, One }
-import akka.remote.transport.ThrottlerTransportAdapter._
-import akka.remote.{ EndpointException, QuarantinedEvent, RARP }
-import akka.testkit.{ AkkaSpec, DefaultTimeout, EventFilter, ImplicitSender, TestEvent, TimingTest, _ }
-import com.typesafe.config.{ Config, ConfigFactory }
import scala.concurrent.Await
import scala.concurrent.duration._
import com.github.ghik.silencer.silent
+import com.typesafe.config.{ Config, ConfigFactory }
+
+import akka.actor.{ Actor, ActorRef, ActorSystem, ExtendedActorSystem, Props, RootActorPath, _ }
+import akka.dispatch.sysmsg.{ Failed, SystemMessage }
+import akka.remote.{ EndpointException, QuarantinedEvent, RARP }
+import akka.remote.transport.AssociationHandle
+import akka.remote.transport.FailureInjectorTransportAdapter.{ Drop, One }
+import akka.remote.transport.ThrottlerTransportAdapter._
+import akka.testkit.{ AkkaSpec, DefaultTimeout, EventFilter, ImplicitSender, TestEvent, TimingTest, _ }
object SystemMessageDeliveryStressTest {
val msgCount = 5000
diff --git a/akka-remote/src/test/scala/akka/remote/classic/transport/TestTransportSpec.scala b/akka-remote/src/test/scala/akka/remote/classic/transport/TestTransportSpec.scala
index 172ecee96d..027fdf61d4 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/transport/TestTransportSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/transport/TestTransportSpec.scala
@@ -4,17 +4,18 @@
package akka.remote.classic.transport
-import akka.actor.Address
-import akka.remote.transport.AssociationHandle.{ ActorHandleEventListener, Disassociated, InboundPayload }
-import akka.remote.transport.TestTransport._
-import akka.remote.transport.Transport._
-import akka.remote.transport.{ AssociationHandle, TestTransport }
-import akka.testkit._
-import akka.util.ByteString
import scala.concurrent._
import com.github.ghik.silencer.silent
+import akka.actor.Address
+import akka.remote.transport.{ AssociationHandle, TestTransport }
+import akka.remote.transport.AssociationHandle.{ ActorHandleEventListener, Disassociated, InboundPayload }
+import akka.remote.transport.TestTransport._
+import akka.remote.transport.Transport._
+import akka.testkit._
+import akka.util.ByteString
+
@silent("deprecated")
class TestTransportSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
diff --git a/akka-remote/src/test/scala/akka/remote/classic/transport/ThrottlerTransportAdapterSpec.scala b/akka-remote/src/test/scala/akka/remote/classic/transport/ThrottlerTransportAdapterSpec.scala
index 39c2459ecc..1472f4e603 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/transport/ThrottlerTransportAdapterSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/transport/ThrottlerTransportAdapterSpec.scala
@@ -4,17 +4,18 @@
package akka.remote.classic.transport
-import akka.actor._
-import akka.remote.classic.transport.ThrottlerTransportAdapterSpec._
-import akka.remote.transport.ThrottlerTransportAdapter._
-import akka.remote.transport.{ TestTransport, ThrottlerTransportAdapter }
-import akka.remote.{ EndpointException, RemoteActorRefProvider }
-import akka.testkit.{ AkkaSpec, DefaultTimeout, EventFilter, ImplicitSender, TestEvent, TimingTest }
-import com.typesafe.config.{ Config, ConfigFactory }
import scala.concurrent.Await
import scala.concurrent.duration._
import com.github.ghik.silencer.silent
+import com.typesafe.config.{ Config, ConfigFactory }
+
+import akka.actor._
+import akka.remote.{ EndpointException, RemoteActorRefProvider }
+import akka.remote.classic.transport.ThrottlerTransportAdapterSpec._
+import akka.remote.transport.{ TestTransport, ThrottlerTransportAdapter }
+import akka.remote.transport.ThrottlerTransportAdapter._
+import akka.testkit.{ AkkaSpec, DefaultTimeout, EventFilter, ImplicitSender, TestEvent, TimingTest }
object ThrottlerTransportAdapterSpec {
val configA: Config =
@@ -77,7 +78,7 @@ object ThrottlerTransportAdapterSpec {
class ThrottlerTransportAdapterSpec extends AkkaSpec(configA) with ImplicitSender with DefaultTimeout {
val systemB = ActorSystem("systemB", system.settings.config)
- val remote = systemB.actorOf(Props[Echo], "echo")
+ val remote = systemB.actorOf(Props[Echo](), "echo")
val rootB = RootActorPath(systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress)
val here = {
diff --git a/akka-remote/src/test/scala/akka/remote/classic/transport/netty/NettyTransportSpec.scala b/akka-remote/src/test/scala/akka/remote/classic/transport/netty/NettyTransportSpec.scala
index 922c201af2..645fffb877 100644
--- a/akka-remote/src/test/scala/akka/remote/classic/transport/netty/NettyTransportSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/classic/transport/netty/NettyTransportSpec.scala
@@ -7,15 +7,16 @@ package akka.remote.classic.transport.netty
import java.net.{ InetAddress, InetSocketAddress }
import java.nio.channels.ServerSocketChannel
+import scala.concurrent.Await
+import scala.concurrent.duration.Duration
+
+import com.typesafe.config.ConfigFactory
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpec
+
import akka.actor.{ ActorSystem, Address, ExtendedActorSystem }
import akka.remote.BoundAddressesExtension
import akka.testkit.SocketUtil
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.Await
-import scala.concurrent.duration.Duration
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpec
object NettyTransportSpec {
val commonConfig = ConfigFactory.parseString("""
@@ -116,8 +117,8 @@ class NettyTransportSpec extends AnyWordSpec with Matchers with BindBehavior {
""")
implicit val sys = ActorSystem("sys", bindConfig.withFallback(commonConfig))
- getInternal.flatMap(_.port) should contain(getExternal.port.get)
- getInternal.map(x => (x.host.get should include).regex("0.0.0.0".r)) // regexp dot is intentional to match IPv4 and 6 addresses
+ getInternal().flatMap(_.port) should contain(getExternal().port.get)
+ getInternal().map(x => (x.host.get should include).regex("0.0.0.0".r)) // regexp dot is intentional to match IPv4 and 6 addresses
Await.result(sys.terminate(), Duration.Inf)
}
diff --git a/akka-remote/src/test/scala/akka/remote/serialization/AllowJavaSerializationOffSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/AllowJavaSerializationOffSpec.scala
index dbd0a7dcaf..2b49e499e9 100644
--- a/akka-remote/src/test/scala/akka/remote/serialization/AllowJavaSerializationOffSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/serialization/AllowJavaSerializationOffSpec.scala
@@ -7,14 +7,15 @@ package akka.remote.serialization
import java.util.{ BitSet => ProgrammaticJavaDummy }
import java.util.{ Date => SerializableDummy }
-import akka.actor.setup.ActorSystemSetup
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ActorSystem
import akka.actor.BootstrapSetup
import akka.actor.ExtendedActorSystem
+import akka.actor.setup.ActorSystemSetup
import akka.serialization._
import akka.testkit.AkkaSpec
import akka.testkit.TestKit
-import com.typesafe.config.ConfigFactory
class ConfigurationDummy
class ProgrammaticDummy
diff --git a/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala
index 9b63cd7859..a423abbda6 100644
--- a/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala
@@ -8,6 +8,7 @@ import java.io.NotSerializableException
import akka.actor._
import akka.remote.{ RemoteWatcher, UniqueAddress }
+import akka.remote.artery.{ ActorSystemTerminating, ActorSystemTerminatingAck, Quarantined, SystemMessageDelivery }
import akka.remote.artery.OutboundHandshake.{ HandshakeReq, HandshakeRsp }
import akka.remote.artery.compress.CompressionProtocol.{
ActorRefCompressionAdvertisement,
@@ -16,7 +17,6 @@ import akka.remote.artery.compress.CompressionProtocol.{
ClassManifestCompressionAdvertisementAck
}
import akka.remote.artery.compress.CompressionTable
-import akka.remote.artery.{ ActorSystemTerminating, ActorSystemTerminatingAck, Quarantined, SystemMessageDelivery }
import akka.serialization.SerializationExtension
import akka.testkit.AkkaSpec
diff --git a/akka-remote/src/test/scala/akka/remote/serialization/DaemonMsgCreateSerializerAllowJavaSerializationSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/DaemonMsgCreateSerializerAllowJavaSerializationSpec.scala
index d2a3323497..fe13670d7e 100644
--- a/akka-remote/src/test/scala/akka/remote/serialization/DaemonMsgCreateSerializerAllowJavaSerializationSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/serialization/DaemonMsgCreateSerializerAllowJavaSerializationSpec.scala
@@ -4,6 +4,8 @@
package akka.remote.serialization
+import com.typesafe.config.ConfigFactory
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Address
@@ -21,7 +23,6 @@ import akka.serialization.SerializationExtension
import akka.testkit.AkkaSpec
import akka.testkit.JavaSerializable
import akka.util.unused
-import com.typesafe.config.ConfigFactory
object DaemonMsgCreateSerializerAllowJavaSerializationSpec {
@@ -74,7 +75,7 @@ class DaemonMsgCreateSerializerAllowJavaSerializationSpec
import DaemonMsgCreateSerializerAllowJavaSerializationSpec._
val ser = SerializationExtension(system)
- val supervisor = system.actorOf(Props[MyActor], "supervisor")
+ val supervisor = system.actorOf(Props[MyActor](), "supervisor")
"Serialization" must {
@@ -116,7 +117,7 @@ class DaemonMsgCreateSerializerAllowJavaSerializationSpec
scope = RemoteScope(Address("akka", "Test", "host2", 1922)),
dispatcher = Deploy.NoDispatcherGiven)
DaemonMsgCreate(
- props = Props[MyActor].withDispatcher("my-disp").withDeploy(deploy1),
+ props = Props[MyActor]().withDispatcher("my-disp").withDeploy(deploy1),
deploy = deploy2,
path = "foo",
supervisor = supervisor)
@@ -132,12 +133,12 @@ class DaemonMsgCreateSerializerNoJavaSerializationSpec extends AkkaSpec("""
import DaemonMsgCreateSerializerAllowJavaSerializationSpec.MyActor
- val supervisor = system.actorOf(Props[MyActor], "supervisor")
+ val supervisor = system.actorOf(Props[MyActor](), "supervisor")
val ser = SerializationExtension(system)
"serialize and de-serialize DaemonMsgCreate with FromClassCreator" in {
verifySerialization {
- DaemonMsgCreate(props = Props[MyActor], deploy = Deploy(), path = "foo", supervisor = supervisor)
+ DaemonMsgCreate(props = Props[MyActor](), deploy = Deploy(), path = "foo", supervisor = supervisor)
}
}
@@ -166,7 +167,7 @@ class DaemonMsgCreateSerializerNoJavaSerializationSpec extends AkkaSpec("""
scope = RemoteScope(Address("akka", "Test", "host2", 1922)),
dispatcher = Deploy.NoDispatcherGiven)
DaemonMsgCreate(
- props = Props[MyActor].withDispatcher("my-disp").withDeploy(deploy1),
+ props = Props[MyActor]().withDispatcher("my-disp").withDeploy(deploy1),
deploy = deploy2,
path = "foo",
supervisor = supervisor)
diff --git a/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala
index 5aa973d543..cab843cce8 100644
--- a/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala
@@ -4,21 +4,23 @@
package akka.remote.serialization
-import akka.actor._
-import akka.remote.{ RemoteScope, RemoteWatcher }
-import akka.serialization.SerializationExtension
-import akka.testkit.AkkaSpec
-import com.typesafe.config.ConfigFactory
-import scala.util.control.NoStackTrace
-import scala.concurrent.duration._
-import java.util.Optional
import java.io.NotSerializableException
+import java.util.Optional
import java.util.concurrent.TimeoutException
-import akka.pattern.AskTimeoutException
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
+import com.typesafe.config.ConfigFactory
+
import akka.{ Done, NotUsed }
+import akka.actor._
+import akka.pattern.AskTimeoutException
+import akka.remote.{ RemoteScope, RemoteWatcher }
import akka.remote.routing.RemoteRouterConfig
import akka.routing._
+import akka.serialization.SerializationExtension
+import akka.testkit.AkkaSpec
import akka.testkit.JavaSerializable
object MiscMessageSerializerSpec {
diff --git a/akka-remote/src/test/scala/akka/remote/serialization/PrimitivesSerializationSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/PrimitivesSerializationSpec.scala
index 56b790c02d..7b710cb1f6 100644
--- a/akka-remote/src/test/scala/akka/remote/serialization/PrimitivesSerializationSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/serialization/PrimitivesSerializationSpec.scala
@@ -9,6 +9,8 @@ import java.nio.ByteOrder
import scala.util.Random
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ExtendedActorSystem
import akka.serialization.BaseSerializer
import akka.serialization.ByteBufferSerializer
@@ -16,7 +18,6 @@ import akka.serialization.SerializationExtension
import akka.serialization.Serializer
import akka.testkit.AkkaSpec
import akka.util.ByteString
-import com.typesafe.config.ConfigFactory
object PrimitivesSerializationSpec {
val serializationTestOverrides = ""
diff --git a/akka-remote/src/test/scala/akka/remote/serialization/ProtobufSerializerSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/ProtobufSerializerSpec.scala
index a87c2e04f5..af85760222 100644
--- a/akka-remote/src/test/scala/akka/remote/serialization/ProtobufSerializerSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/serialization/ProtobufSerializerSpec.scala
@@ -4,13 +4,13 @@
package akka.remote.serialization
+import akka.actor.ExtendedActorSystem
+import akka.remote.MessageSerializer
+import akka.remote.ProtobufProtocol.MyMessage
+import akka.remote.WireFormats.SerializedMessage
+import akka.remote.protobuf.v3.ProtobufProtocolV3.MyMessageV3
import akka.serialization.SerializationExtension
import akka.testkit.AkkaSpec
-import akka.remote.WireFormats.SerializedMessage
-import akka.remote.ProtobufProtocol.MyMessage
-import akka.remote.MessageSerializer
-import akka.actor.ExtendedActorSystem
-import akka.remote.protobuf.v3.ProtobufProtocolV3.MyMessageV3
import akka.util.unused
// those must be defined as top level classes, to have static parseFrom
diff --git a/akka-remote/src/test/scala/akka/remote/serialization/SerializationTransportInformationSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/SerializationTransportInformationSpec.scala
index aaef0246b3..88367ce0f4 100644
--- a/akka-remote/src/test/scala/akka/remote/serialization/SerializationTransportInformationSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/serialization/SerializationTransportInformationSpec.scala
@@ -6,21 +6,22 @@ package akka.remote.serialization
import java.nio.charset.StandardCharsets
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ActorIdentity
-import akka.serialization.Serialization
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.ExtendedActorSystem
import akka.actor.Identify
import akka.actor.RootActorPath
import akka.remote.RARP
+import akka.serialization.Serialization
import akka.serialization.SerializerWithStringManifest
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import akka.testkit.JavaSerializable
import akka.testkit.TestActors
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
object SerializationTransportInformationSpec {
diff --git a/akka-remote/src/test/scala/akka/remote/serialization/SystemMessageSerializationSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/SystemMessageSerializationSpec.scala
index af4cadeb4d..24b652c07c 100644
--- a/akka-remote/src/test/scala/akka/remote/serialization/SystemMessageSerializationSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/serialization/SystemMessageSerializationSpec.scala
@@ -4,12 +4,13 @@
package akka.remote.serialization
+import com.typesafe.config.ConfigFactory
+
import akka.actor.{ ActorInitializationException, ExtendedActorSystem, InternalActorRef }
import akka.dispatch.sysmsg._
import akka.serialization.SerializationExtension
-import akka.testkit.JavaSerializable
import akka.testkit.{ AkkaSpec, TestProbe }
-import com.typesafe.config.ConfigFactory
+import akka.testkit.JavaSerializable
object SystemMessageSerializationSpec {
val serializationTestOverrides =
diff --git a/akka-serialization-jackson/src/main/resources/reference.conf b/akka-serialization-jackson/src/main/resources/reference.conf
index b3d1f42a7d..04df3d9279 100644
--- a/akka-serialization-jackson/src/main/resources/reference.conf
+++ b/akka-serialization-jackson/src/main/resources/reference.conf
@@ -175,6 +175,10 @@ akka.serialization.jackson {
# override the settings in 'akka.serialization.jackson'
jackson-cbor {}
+ # Issue #28918 for compatibility with data serialized with JacksonCborSerializer in
+ # Akka 2.6.4 or earlier, which was plain JSON format.
+ jackson-cbor-264 = ${akka.serialization.jackson.jackson-cbor}
+
}
#//#features
@@ -184,6 +188,7 @@ akka.serialization.jackson.jackson-json.compression {
# Compression algorithm.
# - off : no compression
# - gzip : using common java gzip
+ # - lz4 : using lz4-java
algorithm = gzip
# If compression is enabled with the `algorithm` setting the payload is compressed
@@ -196,10 +201,18 @@ akka.actor {
serializers {
jackson-json = "akka.serialization.jackson.JacksonJsonSerializer"
jackson-cbor = "akka.serialization.jackson.JacksonCborSerializer"
+
+ # Issue #28918 for compatibility with data serialized with JacksonCborSerializer in
+ # Akka 2.6.4 or earlier, which was plain JSON format.
+ jackson-cbor-264 = "akka.serialization.jackson.JacksonJsonSerializer"
}
serialization-identifiers {
jackson-json = 31
- jackson-cbor = 32
+ jackson-cbor = 33
+
+ # Issue #28918 for compatibility with data serialized with JacksonCborSerializer in
+ # Akka 2.6.4 or earlier, which was plain JSON format.
+ jackson-cbor-264 = 32
}
serialization-bindings {
# Define bindings for classes or interfaces use Jackson serializer, e.g.
diff --git a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/ActorRefModule.scala b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/ActorRefModule.scala
index 5f14d1a80e..e3d2557802 100644
--- a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/ActorRefModule.scala
+++ b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/ActorRefModule.scala
@@ -6,8 +6,6 @@ package akka.serialization.jackson
// FIXME maybe move many things to `akka.serialization.jackson.internal` package?
-import akka.actor.ActorRef
-import akka.annotation.InternalApi
import com.fasterxml.jackson.core.JsonGenerator
import com.fasterxml.jackson.core.JsonParser
import com.fasterxml.jackson.core.JsonTokenId
@@ -16,6 +14,9 @@ import com.fasterxml.jackson.databind.SerializerProvider
import com.fasterxml.jackson.databind.deser.std.StdScalarDeserializer
import com.fasterxml.jackson.databind.ser.std.StdScalarSerializer
+import akka.actor.ActorRef
+import akka.annotation.InternalApi
+
/**
* INTERNAL API: Adds support for serializing and deserializing [[ActorRef]].
*/
diff --git a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/AddressModule.scala b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/AddressModule.scala
index 19b4aea32a..f61771b52a 100644
--- a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/AddressModule.scala
+++ b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/AddressModule.scala
@@ -4,9 +4,6 @@
package akka.serialization.jackson
-import akka.actor.Address
-import akka.actor.AddressFromURIString
-import akka.annotation.InternalApi
import com.fasterxml.jackson.core.JsonGenerator
import com.fasterxml.jackson.core.JsonParser
import com.fasterxml.jackson.core.JsonTokenId
@@ -15,6 +12,10 @@ import com.fasterxml.jackson.databind.SerializerProvider
import com.fasterxml.jackson.databind.deser.std.StdScalarDeserializer
import com.fasterxml.jackson.databind.ser.std.StdScalarSerializer
+import akka.actor.Address
+import akka.actor.AddressFromURIString
+import akka.annotation.InternalApi
+
/**
* INTERNAL API: Adds support for serializing and deserializing [[Address]].
*/
diff --git a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/FiniteDurationModule.scala b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/FiniteDurationModule.scala
index c5f3cb35f1..a99061615a 100644
--- a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/FiniteDurationModule.scala
+++ b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/FiniteDurationModule.scala
@@ -6,8 +6,6 @@ package akka.serialization.jackson
import scala.concurrent.duration.FiniteDuration
-import akka.annotation.InternalApi
-import akka.util.JavaDurationConverters._
import com.fasterxml.jackson.core.JsonGenerator
import com.fasterxml.jackson.core.JsonParser
import com.fasterxml.jackson.databind.DeserializationContext
@@ -17,6 +15,9 @@ import com.fasterxml.jackson.databind.ser.std.StdScalarSerializer
import com.fasterxml.jackson.datatype.jsr310.deser.DurationDeserializer
import com.fasterxml.jackson.datatype.jsr310.ser.DurationSerializer
+import akka.annotation.InternalApi
+import akka.util.JavaDurationConverters._
+
/**
* INTERNAL API: Adds support for serializing and deserializing [[FiniteDuration]].
*/
diff --git a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonMigration.scala b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonMigration.scala
index b6c903f2ed..c39b2a17e0 100644
--- a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonMigration.scala
+++ b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonMigration.scala
@@ -5,6 +5,7 @@
package akka.serialization.jackson
import com.fasterxml.jackson.databind.JsonNode
+
import akka.util.unused
/**
diff --git a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonModule.scala b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonModule.scala
index c3067dfc3c..f9a35d62da 100644
--- a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonModule.scala
+++ b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonModule.scala
@@ -4,7 +4,6 @@
package akka.serialization.jackson
-import akka.annotation.InternalApi
import com.fasterxml.jackson.core.Version
import com.fasterxml.jackson.core.util.VersionUtil
import com.fasterxml.jackson.databind.BeanDescription
@@ -20,6 +19,8 @@ import com.fasterxml.jackson.databind.deser.Deserializers
import com.fasterxml.jackson.databind.ser.BeanSerializerModifier
import com.fasterxml.jackson.databind.ser.Serializers
+import akka.annotation.InternalApi
+
/**
* INTERNAL API
*/
diff --git a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonObjectMapperProvider.scala b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonObjectMapperProvider.scala
index ad2cf03f0c..c011db3367 100644
--- a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonObjectMapperProvider.scala
+++ b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonObjectMapperProvider.scala
@@ -11,6 +11,27 @@ import scala.collection.immutable
import scala.compat.java8.OptionConverters._
import scala.util.Failure
import scala.util.Success
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect
+import com.fasterxml.jackson.annotation.JsonCreator
+import com.fasterxml.jackson.annotation.PropertyAccessor
+import com.fasterxml.jackson.core.JsonFactory
+import com.fasterxml.jackson.core.JsonFactoryBuilder
+import com.fasterxml.jackson.core.JsonGenerator
+import com.fasterxml.jackson.core.JsonParser
+import com.fasterxml.jackson.core.StreamReadFeature
+import com.fasterxml.jackson.core.StreamWriteFeature
+import com.fasterxml.jackson.core.json.JsonReadFeature
+import com.fasterxml.jackson.core.json.JsonWriteFeature
+import com.fasterxml.jackson.databind.DeserializationFeature
+import com.fasterxml.jackson.databind.MapperFeature
+import com.fasterxml.jackson.databind.Module
+import com.fasterxml.jackson.databind.ObjectMapper
+import com.fasterxml.jackson.databind.SerializationFeature
+import com.fasterxml.jackson.databind.json.JsonMapper
+import com.fasterxml.jackson.module.paramnames.ParameterNamesModule
+import com.typesafe.config.Config
+
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
import akka.actor.DynamicAccess
@@ -23,25 +44,6 @@ import akka.annotation.InternalStableApi
import akka.event.Logging
import akka.event.LoggingAdapter
import akka.util.unused
-import com.fasterxml.jackson.annotation.JsonAutoDetect
-import com.fasterxml.jackson.annotation.JsonCreator
-import com.fasterxml.jackson.annotation.PropertyAccessor
-import com.fasterxml.jackson.core.JsonFactory
-import com.fasterxml.jackson.core.JsonFactoryBuilder
-import com.fasterxml.jackson.databind.DeserializationFeature
-import com.fasterxml.jackson.databind.MapperFeature
-import com.fasterxml.jackson.databind.Module
-import com.fasterxml.jackson.databind.ObjectMapper
-import com.fasterxml.jackson.databind.SerializationFeature
-import com.fasterxml.jackson.module.paramnames.ParameterNamesModule
-import com.typesafe.config.Config
-import com.fasterxml.jackson.core.JsonParser
-import com.fasterxml.jackson.core.JsonGenerator
-import com.fasterxml.jackson.core.StreamReadFeature
-import com.fasterxml.jackson.core.StreamWriteFeature
-import com.fasterxml.jackson.core.json.JsonReadFeature
-import com.fasterxml.jackson.core.json.JsonWriteFeature
-import com.fasterxml.jackson.databind.json.JsonMapper
object JacksonObjectMapperProvider extends ExtensionId[JacksonObjectMapperProvider] with ExtensionIdProvider {
override def get(system: ActorSystem): JacksonObjectMapperProvider = super.get(system)
@@ -70,9 +72,15 @@ object JacksonObjectMapperProvider extends ExtensionId[JacksonObjectMapperProvid
config: Config,
baseJsonFactory: Option[JsonFactory]): JsonFactory = {
- val jsonFactoryBuilder = baseJsonFactory match {
- case Some(jsonFactory) => new JsonFactoryBuilder(jsonFactory)
- case None => new JsonFactoryBuilder()
+ val jsonFactory: JsonFactory = baseJsonFactory match {
+ case Some(factory) =>
+ // Issue #28918 not possible to use new JsonFactoryBuilder(jsonFactory) here.
+ // It doesn't preserve the formatParserFeatures and formatGeneratorFeatures in
+ // CBORFactor. Therefore we use JsonFactory and configure the features with mappedFeature
+ // instead of using JsonFactoryBuilder (new in Jackson 2.10.0).
+ factory
+ case None =>
+ new JsonFactoryBuilder().build()
}
val configuredStreamReadFeatures =
@@ -82,7 +90,7 @@ object JacksonObjectMapperProvider extends ExtensionId[JacksonObjectMapperProvid
val streamReadFeatures =
objectMapperFactory.overrideConfiguredStreamReadFeatures(bindingName, configuredStreamReadFeatures)
streamReadFeatures.foreach {
- case (feature, value) => jsonFactoryBuilder.configure(feature, value)
+ case (feature, value) => jsonFactory.configure(feature.mappedFeature, value)
}
val configuredStreamWriteFeatures =
@@ -92,7 +100,7 @@ object JacksonObjectMapperProvider extends ExtensionId[JacksonObjectMapperProvid
val streamWriteFeatures =
objectMapperFactory.overrideConfiguredStreamWriteFeatures(bindingName, configuredStreamWriteFeatures)
streamWriteFeatures.foreach {
- case (feature, value) => jsonFactoryBuilder.configure(feature, value)
+ case (feature, value) => jsonFactory.configure(feature.mappedFeature, value)
}
val configuredJsonReadFeatures =
@@ -102,7 +110,7 @@ object JacksonObjectMapperProvider extends ExtensionId[JacksonObjectMapperProvid
val jsonReadFeatures =
objectMapperFactory.overrideConfiguredJsonReadFeatures(bindingName, configuredJsonReadFeatures)
jsonReadFeatures.foreach {
- case (feature, value) => jsonFactoryBuilder.configure(feature, value)
+ case (feature, value) => jsonFactory.configure(feature.mappedFeature, value)
}
val configuredJsonWriteFeatures =
@@ -112,10 +120,10 @@ object JacksonObjectMapperProvider extends ExtensionId[JacksonObjectMapperProvid
val jsonWriteFeatures =
objectMapperFactory.overrideConfiguredJsonWriteFeatures(bindingName, configuredJsonWriteFeatures)
jsonWriteFeatures.foreach {
- case (feature, value) => jsonFactoryBuilder.configure(feature, value)
+ case (feature, value) => jsonFactory.configure(feature.mappedFeature, value)
}
- jsonFactoryBuilder.build()
+ jsonFactory
}
private def configureObjectMapperFeatures(
diff --git a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonSerializer.scala b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonSerializer.scala
index 72d4b76011..001adf0106 100644
--- a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonSerializer.scala
+++ b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/JacksonSerializer.scala
@@ -4,27 +4,25 @@
package akka.serialization.jackson
-import java.io.ByteArrayInputStream
-import java.io.ByteArrayOutputStream
-import java.io.NotSerializableException
-import java.util.zip.GZIPInputStream
-import java.util.zip.GZIPOutputStream
+import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, NotSerializableException }
+import java.nio.ByteBuffer
+import java.util.zip.{ GZIPInputStream, GZIPOutputStream }
import scala.annotation.tailrec
-import scala.util.Failure
-import scala.util.Success
+import scala.util.{ Failure, Success }
import scala.util.control.NonFatal
-import akka.actor.ExtendedActorSystem
-import akka.annotation.InternalApi
-import akka.event.LogMarker
-import akka.event.Logging
-import akka.serialization.BaseSerializer
-import akka.serialization.SerializationExtension
-import akka.serialization.SerializerWithStringManifest
-import akka.util.Helpers.toRootLowerCase
+
import com.fasterxml.jackson.databind.ObjectMapper
import com.fasterxml.jackson.databind.jsontype.impl.SubTypeValidator
import com.fasterxml.jackson.dataformat.cbor.CBORFactory
+import net.jpountz.lz4.LZ4Factory
+
+import akka.actor.ExtendedActorSystem
+import akka.annotation.InternalApi
+import akka.event.{ LogMarker, Logging }
+import akka.serialization.{ BaseSerializer, SerializationExtension, SerializerWithStringManifest }
+import akka.util.Helpers.toRootLowerCase
+import akka.util.OptionVal
/**
* INTERNAL API
@@ -84,6 +82,51 @@ import com.fasterxml.jackson.dataformat.cbor.CBORFactory
(bytes(0) == GZIPInputStream.GZIP_MAGIC.toByte) &&
(bytes(1) == (GZIPInputStream.GZIP_MAGIC >> 8).toByte)
}
+
+ final case class LZ4Meta(offset: Int, length: Int) {
+ import LZ4Meta._
+
+ def putInto(buffer: ByteBuffer): Unit = {
+ buffer.putInt(LZ4_MAGIC)
+ buffer.putInt(length)
+ }
+
+ def prependTo(bytes: Array[Byte]): Array[Byte] = {
+ val buffer = ByteBuffer.allocate(bytes.length + offset)
+ putInto(buffer)
+ buffer.put(bytes)
+ buffer.array()
+ }
+
+ }
+
+ object LZ4Meta {
+ val LZ4_MAGIC = 0x87d96df6 // The last 4 bytes of `printf akka | sha512sum`
+
+ def apply(bytes: Array[Byte]): LZ4Meta = {
+ LZ4Meta(8, bytes.length)
+ }
+
+ def get(buffer: ByteBuffer): OptionVal[LZ4Meta] = {
+ if (buffer.remaining() < 4) {
+ OptionVal.None
+ } else if (buffer.getInt() != LZ4_MAGIC) {
+ OptionVal.None
+ } else {
+ OptionVal.Some(LZ4Meta(8, buffer.getInt()))
+ }
+ }
+
+ def get(bytes: Array[Byte]): OptionVal[LZ4Meta] = {
+ get(ByteBuffer.wrap(bytes))
+ }
+
+ }
+
+ def isLZ4(bytes: Array[Byte]): Boolean = {
+ LZ4Meta.get(bytes).isDefined
+ }
+
}
/**
@@ -112,7 +155,7 @@ import com.fasterxml.jackson.dataformat.cbor.CBORFactory
sealed trait Algoritm
object Off extends Algoritm
final case class GZip(largerThan: Long) extends Algoritm
- // TODO add LZ4, issue #27066
+ final case class LZ4(largerThan: Long) extends Algoritm
}
/**
@@ -129,8 +172,7 @@ import com.fasterxml.jackson.dataformat.cbor.CBORFactory
val bindingName: String,
val objectMapper: ObjectMapper)
extends SerializerWithStringManifest {
- import JacksonSerializer.GadgetClassBlacklist
- import JacksonSerializer.isGZipped
+ import JacksonSerializer._
// TODO issue #27107: it should be possible to implement ByteBufferSerializer as well, using Jackson's
// ByteBufferBackedOutputStream/ByteBufferBackedInputStream
@@ -145,6 +187,9 @@ import com.fasterxml.jackson.dataformat.cbor.CBORFactory
case "gzip" =>
val compressLargerThan = conf.getBytes("compression.compress-larger-than")
Compression.GZip(compressLargerThan)
+ case "lz4" =>
+ val compressLargerThan = conf.getBytes("compression.compress-larger-than")
+ Compression.LZ4(compressLargerThan)
case other =>
throw new IllegalArgumentException(
s"Unknown compression algorithm [$other], possible values are " +
@@ -207,6 +252,10 @@ import com.fasterxml.jackson.dataformat.cbor.CBORFactory
// doesn't have to be volatile, doesn't matter if check is run more than once
private var serializationBindingsCheckedOk = false
+ private lazy val lz4Factory = LZ4Factory.fastestInstance()
+ private lazy val lz4Compressor = lz4Factory.fastCompressor()
+ private lazy val lz4Decompressor = lz4Factory.safeDecompressor()
+
override val identifier: Int = BaseSerializer.identifierFromConfig(bindingName, system)
override def manifest(obj: AnyRef): String = {
@@ -444,42 +493,47 @@ import com.fasterxml.jackson.dataformat.cbor.CBORFactory
def compress(bytes: Array[Byte]): Array[Byte] = {
compressionAlgorithm match {
- case Compression.Off => bytes
- case Compression.GZip(largerThan) =>
- if (bytes.length > largerThan) compressGzip(bytes) else bytes
+ case Compression.Off => bytes
+ case Compression.GZip(largerThan) if bytes.length <= largerThan => bytes
+ case Compression.GZip(_) =>
+ val bos = new ByteArrayOutputStream(BufferSize)
+ val zip = new GZIPOutputStream(bos)
+ try zip.write(bytes)
+ finally zip.close()
+ bos.toByteArray
+ case Compression.LZ4(largerThan) if bytes.length <= largerThan => bytes
+ case Compression.LZ4(_) => {
+ val meta = LZ4Meta(bytes)
+ val compressed = lz4Compressor.compress(bytes)
+ meta.prependTo(compressed)
+ }
}
}
- private def compressGzip(bytes: Array[Byte]): Array[Byte] = {
- val bos = new ByteArrayOutputStream(BufferSize)
- val zip = new GZIPOutputStream(bos)
- try zip.write(bytes)
- finally zip.close()
- bos.toByteArray
- }
-
def decompress(bytes: Array[Byte]): Array[Byte] = {
- if (isGZipped(bytes))
- decompressGzip(bytes)
- else
- bytes
- }
+ if (isGZipped(bytes)) {
+ val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
+ val out = new ByteArrayOutputStream()
+ val buffer = new Array[Byte](BufferSize)
- private def decompressGzip(bytes: Array[Byte]): Array[Byte] = {
- val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
- val out = new ByteArrayOutputStream()
- val buffer = new Array[Byte](BufferSize)
+ @tailrec def readChunk(): Unit = in.read(buffer) match {
+ case -1 => ()
+ case n =>
+ out.write(buffer, 0, n)
+ readChunk()
+ }
- @tailrec def readChunk(): Unit = in.read(buffer) match {
- case -1 => ()
- case n =>
- out.write(buffer, 0, n)
- readChunk()
+ try readChunk()
+ finally in.close()
+ out.toByteArray
+ } else {
+ LZ4Meta.get(bytes) match {
+ case OptionVal.None => bytes
+ case OptionVal.Some(meta) =>
+ val srcLen = bytes.length - meta.offset
+ lz4Decompressor.decompress(bytes, meta.offset, srcLen, meta.length)
+ }
}
-
- try readChunk()
- finally in.close()
- out.toByteArray
}
}
diff --git a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/StreamRefModule.scala b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/StreamRefModule.scala
index a2c896a2f6..41d776571c 100644
--- a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/StreamRefModule.scala
+++ b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/StreamRefModule.scala
@@ -6,10 +6,6 @@ package akka.serialization.jackson
// FIXME maybe move many things to `akka.serialization.jackson.internal` package?
-import akka.annotation.InternalApi
-import akka.stream.SinkRef
-import akka.stream.SourceRef
-import akka.stream.StreamRefResolver
import com.fasterxml.jackson.core.JsonGenerator
import com.fasterxml.jackson.core.JsonParser
import com.fasterxml.jackson.core.JsonTokenId
@@ -18,6 +14,11 @@ import com.fasterxml.jackson.databind.SerializerProvider
import com.fasterxml.jackson.databind.deser.std.StdScalarDeserializer
import com.fasterxml.jackson.databind.ser.std.StdScalarSerializer
+import akka.annotation.InternalApi
+import akka.stream.SinkRef
+import akka.stream.SourceRef
+import akka.stream.StreamRefResolver
+
/**
* INTERNAL API: Adds support for serializing and deserializing [[akka.stream.SourceRef]] and [[akka.stream.SinkRef]].
*/
diff --git a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/TypedActorRefModule.scala b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/TypedActorRefModule.scala
index 71a4a91779..ab3d1aad82 100644
--- a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/TypedActorRefModule.scala
+++ b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/TypedActorRefModule.scala
@@ -4,10 +4,6 @@
package akka.serialization.jackson
-import akka.actor.typed.ActorRef
-import akka.actor.typed.ActorRefResolver
-import akka.actor.typed.scaladsl.adapter._
-import akka.annotation.InternalApi
import com.fasterxml.jackson.core.JsonGenerator
import com.fasterxml.jackson.core.JsonParser
import com.fasterxml.jackson.core.JsonTokenId
@@ -16,6 +12,11 @@ import com.fasterxml.jackson.databind.SerializerProvider
import com.fasterxml.jackson.databind.deser.std.StdScalarDeserializer
import com.fasterxml.jackson.databind.ser.std.StdScalarSerializer
+import akka.actor.typed.ActorRef
+import akka.actor.typed.ActorRefResolver
+import akka.actor.typed.scaladsl.adapter._
+import akka.annotation.InternalApi
+
/**
* INTERNAL API: Adds support for serializing and deserializing [[akka.actor.typed.ActorRef]].
*/
diff --git a/akka-serialization-jackson/src/test/scala/akka/serialization/jackson/JacksonSerializerSpec.scala b/akka-serialization-jackson/src/test/scala/akka/serialization/jackson/JacksonSerializerSpec.scala
index c1010dbf0d..a75ce1f4b5 100644
--- a/akka-serialization-jackson/src/test/scala/akka/serialization/jackson/JacksonSerializerSpec.scala
+++ b/akka-serialization-jackson/src/test/scala/akka/serialization/jackson/JacksonSerializerSpec.scala
@@ -4,6 +4,7 @@
package akka.serialization.jackson
+import java.nio.charset.StandardCharsets
import java.time.Duration
import java.time.Instant
import java.time.LocalDateTime
@@ -15,8 +16,35 @@ import java.util.UUID
import java.util.logging.FileHandler
import scala.collection.immutable
-import scala.concurrent.duration.FiniteDuration
import scala.concurrent.duration._
+import scala.concurrent.duration.FiniteDuration
+
+import com.fasterxml.jackson.annotation.JsonSubTypes
+import com.fasterxml.jackson.annotation.JsonTypeInfo
+import com.fasterxml.jackson.core.JsonFactory
+import com.fasterxml.jackson.core.JsonGenerator
+import com.fasterxml.jackson.core.JsonParser
+import com.fasterxml.jackson.core.StreamReadFeature
+import com.fasterxml.jackson.core.StreamWriteFeature
+import com.fasterxml.jackson.core.`type`.TypeReference
+import com.fasterxml.jackson.databind.DeserializationFeature
+import com.fasterxml.jackson.databind.JsonNode
+import com.fasterxml.jackson.databind.MapperFeature
+import com.fasterxml.jackson.databind.Module
+import com.fasterxml.jackson.databind.ObjectMapper
+import com.fasterxml.jackson.databind.SerializationFeature
+import com.fasterxml.jackson.databind.exc.InvalidTypeIdException
+import com.fasterxml.jackson.databind.json.JsonMapper
+import com.fasterxml.jackson.databind.node.IntNode
+import com.fasterxml.jackson.databind.node.ObjectNode
+import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule
+import com.fasterxml.jackson.module.scala.JsonScalaEnumeration
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.Address
@@ -29,31 +57,6 @@ import akka.serialization.Serialization
import akka.serialization.SerializationExtension
import akka.testkit.TestActors
import akka.testkit.TestKit
-import com.fasterxml.jackson.annotation.JsonSubTypes
-import com.fasterxml.jackson.annotation.JsonTypeInfo
-import com.fasterxml.jackson.core.JsonFactory
-import com.fasterxml.jackson.databind.MapperFeature
-import com.fasterxml.jackson.databind.DeserializationFeature
-import com.fasterxml.jackson.databind.JsonNode
-import com.fasterxml.jackson.databind.Module
-import com.fasterxml.jackson.databind.ObjectMapper
-import com.fasterxml.jackson.databind.SerializationFeature
-import com.fasterxml.jackson.databind.exc.InvalidTypeIdException
-import com.fasterxml.jackson.databind.node.IntNode
-import com.fasterxml.jackson.databind.node.ObjectNode
-import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule
-import com.typesafe.config.ConfigFactory
-import org.scalatest.BeforeAndAfterAll
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpecLike
-import com.fasterxml.jackson.core.JsonParser
-import com.fasterxml.jackson.core.JsonGenerator
-import com.fasterxml.jackson.core.StreamReadFeature
-import com.fasterxml.jackson.core.StreamWriteFeature
-import com.fasterxml.jackson.core.`type`.TypeReference
-import com.fasterxml.jackson.databind.json.JsonMapper
-import com.fasterxml.jackson.module.scala.JsonScalaEnumeration
-import com.github.ghik.silencer.silent
object ScalaTestMessages {
trait TestMessage
@@ -505,6 +508,41 @@ class JacksonJsonSerializerSpec extends JacksonSerializerSpec("jackson-json") {
val bytes = serializeToBinary(msg)
JacksonSerializer.isGZipped(bytes) should ===(false)
}
+
+ "compress large payload with lz4" in withSystem("""
+ akka.serialization.jackson.jackson-json.compression {
+ algorithm = lz4
+ compress-larger-than = 32 KiB
+ }
+ """) { sys =>
+ val conf = JacksonObjectMapperProvider.configForBinding("jackson-json", sys.settings.config)
+ val compressLargerThan = conf.getBytes("compression.compress-larger-than")
+ def check(msg: AnyRef, compressed: Boolean): Unit = {
+ val bytes = serializeToBinary(msg, sys)
+ JacksonSerializer.isLZ4(bytes) should ===(compressed)
+ bytes.length should be < compressLargerThan.toInt
+ checkSerialization(msg, sys)
+ }
+ check(SimpleCommand("0" * (compressLargerThan + 1).toInt), true)
+ }
+
+ "not compress small payload with lz4" in withSystem("""
+ akka.serialization.jackson.jackson-json.compression {
+ algorithm = lz4
+ compress-larger-than = 32 KiB
+ }
+ """) { sys =>
+ val conf = JacksonObjectMapperProvider.configForBinding("jackson-json", sys.settings.config)
+ val compressLargerThan = conf.getBytes("compression.compress-larger-than")
+ def check(msg: AnyRef, compressed: Boolean): Unit = {
+ val bytes = serializeToBinary(msg, sys)
+ JacksonSerializer.isLZ4(bytes) should ===(compressed)
+ bytes.length should be < compressLargerThan.toInt
+ checkSerialization(msg, sys)
+ }
+ check(SimpleCommand("Bob"), false)
+ check(new SimpleCommandNotCaseClass("Bob"), false)
+ }
}
"JacksonJsonSerializer without type in manifest" should {
@@ -569,6 +607,17 @@ class JacksonJsonSerializerSpec extends JacksonSerializerSpec("jackson-json") {
""")(sys => checkSerialization(Elephant("Dumbo", 1), sys))
}
}
+
+ // issue #28918
+ "cbor compatibility for reading json" in {
+ val msg = SimpleCommand("abc")
+ val jsonSerializer = serializerFor(msg)
+ jsonSerializer.identifier should ===(31)
+ val manifest = jsonSerializer.manifest(msg)
+ val bytes = jsonSerializer.toBinary(msg)
+ val deserialized = serialization().deserialize(bytes, 32, manifest).get
+ deserialized should be(msg)
+ }
}
}
@@ -625,7 +674,21 @@ abstract class JacksonSerializerSpec(serializerName: String)
val serializer = serializerFor(obj, sys)
val manifest = serializer.manifest(obj)
val serializerId = serializer.identifier
- val blob = serializeToBinary(obj)
+ val blob = serializeToBinary(obj, sys)
+
+ // Issue #28918, check that CBOR format is used (not JSON).
+ if (blob.length > 0) {
+ serializer match {
+ case _: JacksonJsonSerializer =>
+ if (!JacksonSerializer.isGZipped(blob) && !JacksonSerializer.isLZ4(blob))
+ new String(blob.take(1), StandardCharsets.UTF_8) should ===("{")
+ case _: JacksonCborSerializer =>
+ new String(blob.take(1), StandardCharsets.UTF_8) should !==("{")
+ case _ =>
+ throw new IllegalArgumentException(s"Unexpected serializer $serializer")
+ }
+ }
+
val deserialized = deserializeFromBinary(blob, serializerId, manifest, sys)
deserialized should ===(obj)
}
diff --git a/akka-slf4j/src/main/scala/akka/event/slf4j/Slf4jLogger.scala b/akka-slf4j/src/main/scala/akka/event/slf4j/Slf4jLogger.scala
index d475239f48..2337e76726 100644
--- a/akka-slf4j/src/main/scala/akka/event/slf4j/Slf4jLogger.scala
+++ b/akka-slf4j/src/main/scala/akka/event/slf4j/Slf4jLogger.scala
@@ -5,11 +5,12 @@
package akka.event.slf4j
import org.slf4j.{ MDC, Marker, MarkerFactory, Logger => SLFLogger, LoggerFactory => SLFLoggerFactory }
-import akka.event.Logging._
+
import akka.actor._
-import akka.event.{ LogMarker, _ }
-import akka.util.{ unused, Helpers }
import akka.dispatch.RequiresMessageQueue
+import akka.event.{ LogMarker, _ }
+import akka.event.Logging._
+import akka.util.{ unused, Helpers }
/**
* Base trait for all classes that wants to be able use the SLF4J logging infrastructure.
diff --git a/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jLoggerSpec.scala b/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jLoggerSpec.scala
index 85ed4804af..7169076a27 100644
--- a/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jLoggerSpec.scala
+++ b/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jLoggerSpec.scala
@@ -4,18 +4,19 @@
package akka.event.slf4j
-import language.postfixOps
-import akka.testkit.AkkaSpec
-import akka.actor.{ Actor, DiagnosticActorLogging, Props }
-
-import scala.concurrent.duration._
-import akka.event.{ LogMarker, Logging }
-import ch.qos.logback.core.OutputStreamAppender
import java.io.ByteArrayOutputStream
+import scala.concurrent.duration._
+
+import ch.qos.logback.core.OutputStreamAppender
+import language.postfixOps
import org.scalatest.BeforeAndAfterEach
import org.slf4j.{ Marker, MarkerFactory }
+import akka.actor.{ Actor, DiagnosticActorLogging, Props }
+import akka.event.{ LogMarker, Logging }
+import akka.testkit.AkkaSpec
+
object Slf4jLoggerSpec {
// This test depends on logback configuration in src/test/resources/logback-test.xml
@@ -75,7 +76,7 @@ object Slf4jLoggerSpec {
class Slf4jLoggerSpec extends AkkaSpec(Slf4jLoggerSpec.config) with BeforeAndAfterEach {
import Slf4jLoggerSpec._
- val producer = system.actorOf(Props[LogProducer], name = "logProducer")
+ val producer = system.actorOf(Props[LogProducer](), name = "logProducer")
override def beforeEach(): Unit = {
output.reset()
diff --git a/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jLoggingFilterSpec.scala b/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jLoggingFilterSpec.scala
index 86fb8b7891..bd9e014cf3 100644
--- a/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jLoggingFilterSpec.scala
+++ b/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jLoggingFilterSpec.scala
@@ -4,19 +4,21 @@
package akka.event.slf4j
-import akka.testkit.AkkaSpec
-import akka.actor.{ Actor, ActorLogging, Props }
import scala.concurrent.duration._
-import akka.event.Logging
+
import org.scalatest.BeforeAndAfterEach
+
+import akka.actor.{ Actor, ActorLogging, Props }
import akka.actor.ActorRef
-import akka.event.Logging.InitializeLogger
-import akka.event.Logging.LoggerInitialized
-import akka.event.Logging.LogEvent
-import akka.testkit.TestProbe
-import akka.event.Logging.Warning
-import akka.event.Logging.Info
+import akka.event.Logging
import akka.event.Logging.Debug
+import akka.event.Logging.Info
+import akka.event.Logging.InitializeLogger
+import akka.event.Logging.LogEvent
+import akka.event.Logging.LoggerInitialized
+import akka.event.Logging.Warning
+import akka.testkit.AkkaSpec
+import akka.testkit.TestProbe
object Slf4jLoggingFilterSpec {
@@ -93,7 +95,7 @@ class Slf4jLoggingFilterSpec extends AkkaSpec(Slf4jLoggingFilterSpec.config) wit
val probe = TestProbe()
system.eventStream.publish(SetTarget(probe.ref))
probe.expectMsg("OK")
- val debugLevelProducer = system.actorOf(Props[DebugLevelProducer], name = "debugLevelProducer")
+ val debugLevelProducer = system.actorOf(Props[DebugLevelProducer](), name = "debugLevelProducer")
debugLevelProducer ! "test1"
probe.expectMsgType[Warning].message should be("test1")
probe.expectMsgType[Info].message should be("test1")
@@ -104,7 +106,7 @@ class Slf4jLoggingFilterSpec extends AkkaSpec(Slf4jLoggingFilterSpec.config) wit
val probe = TestProbe()
system.eventStream.publish(SetTarget(probe.ref))
probe.expectMsg("OK")
- val debugLevelProducer = system.actorOf(Props[WarningLevelProducer], name = "warningLevelProducer")
+ val debugLevelProducer = system.actorOf(Props[WarningLevelProducer](), name = "warningLevelProducer")
debugLevelProducer ! "test2"
probe.expectMsgType[Warning].message should be("test2")
probe.expectNoMessage(500.millis)
diff --git a/akka-stream-testkit/src/main/scala/akka/stream/testkit/StreamTestKit.scala b/akka-stream-testkit/src/main/scala/akka/stream/testkit/StreamTestKit.scala
index 327a504496..b1f72bed65 100644
--- a/akka-stream-testkit/src/main/scala/akka/stream/testkit/StreamTestKit.scala
+++ b/akka-stream-testkit/src/main/scala/akka/stream/testkit/StreamTestKit.scala
@@ -4,25 +4,25 @@
package akka.stream.testkit
-import akka.actor.{ ActorRef, ActorSystem, DeadLetterSuppression, NoSerializationVerificationNeeded }
-import akka.stream._
-import akka.stream.impl._
-import akka.testkit.{ TestActor, TestProbe }
-import org.reactivestreams.{ Publisher, Subscriber, Subscription }
+import java.io.PrintWriter
+import java.io.StringWriter
+import java.util.concurrent.CountDownLatch
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.duration._
-import java.io.StringWriter
-import java.io.PrintWriter
-import java.util.concurrent.CountDownLatch
+import scala.reflect.ClassTag
+import org.reactivestreams.{ Publisher, Subscriber, Subscription }
+
+import akka.actor.{ ActorRef, ActorSystem, DeadLetterSuppression, NoSerializationVerificationNeeded }
+import akka.stream._
+import akka.stream.impl._
+import akka.testkit.{ TestActor, TestProbe }
import akka.testkit.TestActor.AutoPilot
import akka.util.JavaDurationConverters
import akka.util.ccompat._
-import scala.reflect.ClassTag
-
/**
* Provides factory methods for various Publishers.
*/
@@ -35,7 +35,7 @@ object TestPublisher {
final case class CancelSubscription(subscription: Subscription, cause: Throwable) extends PublisherEvent
final case class RequestMore(subscription: Subscription, elements: Long) extends PublisherEvent
- final object SubscriptionDone extends NoSerializationVerificationNeeded
+ object SubscriptionDone extends NoSerializationVerificationNeeded
/**
* Publisher that signals complete to subscribers, after handing a void subscription.
diff --git a/akka-stream-testkit/src/main/scala/akka/stream/testkit/TestGraphStage.scala b/akka-stream-testkit/src/main/scala/akka/stream/testkit/TestGraphStage.scala
index fecc22a125..84167a33aa 100644
--- a/akka-stream-testkit/src/main/scala/akka/stream/testkit/TestGraphStage.scala
+++ b/akka-stream-testkit/src/main/scala/akka/stream/testkit/TestGraphStage.scala
@@ -4,14 +4,14 @@
package akka.stream.testkit
+import scala.util.control.NonFatal
+
import akka.actor.NoSerializationVerificationNeeded
+import akka.stream._
import akka.stream.scaladsl.{ Sink, Source }
import akka.stream.stage.{ GraphStageWithMaterializedValue, InHandler, OutHandler }
-import akka.stream._
import akka.testkit.TestProbe
-import scala.util.control.NonFatal
-
/**
* Messages emitted after the corresponding `stageUnderTest` methods has been invoked.
*/
diff --git a/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/StreamTestKit.scala b/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/StreamTestKit.scala
index 6f2736469d..b010c1b36c 100644
--- a/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/StreamTestKit.scala
+++ b/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/StreamTestKit.scala
@@ -6,6 +6,9 @@ package akka.stream.testkit.scaladsl
import java.util.concurrent.TimeUnit.MILLISECONDS
+import scala.concurrent.{ Await, ExecutionContext }
+import scala.concurrent.duration._
+
import akka.actor.{ ActorRef, ActorSystem }
import akka.annotation.InternalApi
import akka.stream._
@@ -13,9 +16,6 @@ import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor }
import akka.stream.snapshot._
import akka.testkit.TestProbe
-import scala.concurrent.duration._
-import scala.concurrent.{ Await, ExecutionContext }
-
object StreamTestKit {
/**
@@ -102,7 +102,7 @@ object StreamTestKit {
.append(logic.attributes.attributeList.mkString(", "))
.append("],\n")
}
- builder.setLength(builder.length - 2)
+ builder.setLength(builder.length() - 2)
shell match {
case running: RunningInterpreter =>
builder.append("\n ],\n connections: [\n")
@@ -119,7 +119,7 @@ object StreamTestKit {
.append(connection.state)
.append(")\n")
}
- builder.setLength(builder.length - 2)
+ builder.setLength(builder.length() - 2)
case _ =>
}
diff --git a/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSink.scala b/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSink.scala
index be9b8bd9a3..d212414a94 100644
--- a/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSink.scala
+++ b/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSink.scala
@@ -5,12 +5,12 @@
package akka.stream.testkit.scaladsl
import akka.actor.ActorSystem
-import akka.stream.Attributes.none
import akka.stream._
+import akka.stream.Attributes.none
import akka.stream.scaladsl._
-import akka.stream.testkit.TestSubscriber.Probe
import akka.stream.testkit._
import akka.stream.testkit.StreamTestKit.ProbeSink
+import akka.stream.testkit.TestSubscriber.Probe
/**
* Factory methods for test sinks.
diff --git a/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSource.scala b/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSource.scala
index 08b80501e7..6f8cb7d932 100644
--- a/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSource.scala
+++ b/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSource.scala
@@ -4,14 +4,13 @@
package akka.stream.testkit.scaladsl
+import akka.actor.ActorSystem
import akka.stream._
import akka.stream.Attributes.none
import akka.stream.scaladsl._
import akka.stream.testkit._
import akka.stream.testkit.StreamTestKit.ProbeSource
-import akka.actor.ActorSystem
-
/**
* Factory methods for test sources.
*/
diff --git a/akka-stream-testkit/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpecKit.scala b/akka-stream-testkit/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpecKit.scala
index a6d4840240..892aba5753 100644
--- a/akka-stream-testkit/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpecKit.scala
+++ b/akka-stream-testkit/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpecKit.scala
@@ -4,13 +4,19 @@
package akka.stream.impl.fusing
+import scala.collection.{ Map => SMap }
+import scala.concurrent.ExecutionContextExecutor
+import scala.concurrent.duration.FiniteDuration
+
+import com.github.ghik.silencer.silent
+
import akka.actor.ActorSystem
import akka.actor.Cancellable
import akka.actor.Props
import akka.annotation.InternalApi
import akka.event.Logging
-import akka.stream.Supervision.Decider
import akka.stream._
+import akka.stream.Supervision.Decider
import akka.stream.impl.fusing.GraphInterpreter.{
Connection,
DownstreamBoundaryStageLogic,
@@ -20,11 +26,6 @@ import akka.stream.impl.fusing.GraphInterpreter.{
import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler, _ }
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.Utils.TE
-import com.github.ghik.silencer.silent
-
-import scala.collection.{ Map => SMap }
-import scala.concurrent.ExecutionContextExecutor
-import scala.concurrent.duration.FiniteDuration
/**
* INTERNAL API
diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/BaseTwoStreamsSetup.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/BaseTwoStreamsSetup.scala
index c9fb603bc1..d2f43ab286 100644
--- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/BaseTwoStreamsSetup.scala
+++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/BaseTwoStreamsSetup.scala
@@ -4,13 +4,14 @@
package akka.stream.testkit
+import scala.collection.immutable
+import scala.util.control.NoStackTrace
+
+import org.reactivestreams.Publisher
+
import akka.stream.scaladsl._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.AkkaSpec
-import org.reactivestreams.Publisher
-
-import scala.collection.immutable
-import scala.util.control.NoStackTrace
abstract class BaseTwoStreamsSetup extends AkkaSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
@@ -25,7 +26,7 @@ abstract class BaseTwoStreamsSetup extends AkkaSpec("""
def failedPublisher[T]: Publisher[T] = TestPublisher.error[T](TestException)
- def completedPublisher[T]: Publisher[T] = TestPublisher.empty[T]
+ def completedPublisher[T]: Publisher[T] = TestPublisher.empty[T]()
def nonemptyPublisher[T](elems: immutable.Iterable[T]): Publisher[T] = Source(elems).runWith(Sink.asPublisher(false))
diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/ChainSetup.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/ChainSetup.scala
index 15cf1cb959..f9111af66a 100644
--- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/ChainSetup.scala
+++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/ChainSetup.scala
@@ -4,6 +4,9 @@
package akka.stream.testkit
+import com.github.ghik.silencer.silent
+import org.reactivestreams.Publisher
+
import akka.NotUsed
import akka.actor.ActorRefFactory
import akka.actor.ActorSystem
@@ -11,8 +14,6 @@ import akka.stream.ActorMaterializer
import akka.stream.ActorMaterializerSettings
import akka.stream.Materializer
import akka.stream.scaladsl._
-import com.github.ghik.silencer.silent
-import org.reactivestreams.Publisher
class ChainSetup[In, Out, M](
stream: Flow[In, In, NotUsed] => Flow[In, Out, M],
diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/ScriptedTest.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/ScriptedTest.scala
index b241daacb7..06fe748099 100644
--- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/ScriptedTest.scala
+++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/ScriptedTest.scala
@@ -5,26 +5,26 @@
package akka.stream.testkit
import java.util.concurrent.ThreadLocalRandom
+import java.util.concurrent.ThreadLocalRandom
+
+import scala.annotation.tailrec
+import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
+import org.reactivestreams.Publisher
+import org.scalatest.matchers.should.Matchers
import akka.NotUsed
import akka.actor.ActorSystem
import akka.stream.ActorMaterializerSettings
import akka.stream.Materializer
import akka.stream.SystemMaterializer
+import akka.stream.SystemMaterializer
import akka.stream.scaladsl.Flow
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
import akka.stream.testkit.TestPublisher._
import akka.stream.testkit.TestSubscriber._
-import org.reactivestreams.Publisher
-
-import scala.annotation.tailrec
-import scala.concurrent.duration._
-import java.util.concurrent.ThreadLocalRandom
-
-import akka.stream.SystemMaterializer
-import com.github.ghik.silencer.silent
-import org.scalatest.matchers.should.Matchers
trait ScriptedTest extends Matchers {
diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamSpec.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamSpec.scala
index 74cfc151f7..728c2eb27a 100644
--- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamSpec.scala
+++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamSpec.scala
@@ -4,28 +4,30 @@
package akka.stream.testkit
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
+import com.typesafe.config.{ Config, ConfigFactory }
+import org.scalatest.Failed
+
import akka.actor.{ ActorRef, ActorSystem }
import akka.stream.impl.StreamSupervisor
import akka.stream.snapshot.{ MaterializerState, StreamSnapshotImpl }
import akka.testkit.{ AkkaSpec, TestProbe }
-import com.typesafe.config.{ Config, ConfigFactory }
-import org.scalatest.Failed
-
-import scala.concurrent.Future
-import scala.concurrent.duration._
+import akka.testkit.TestKitUtils
abstract class StreamSpec(_system: ActorSystem) extends AkkaSpec(_system) {
def this(config: Config) =
this(
ActorSystem(
- AkkaSpec.testNameFromCallStack(classOf[StreamSpec]),
+ TestKitUtils.testNameFromCallStack(classOf[StreamSpec], "".r),
ConfigFactory.load(config.withFallback(AkkaSpec.testConf))))
def this(s: String) = this(ConfigFactory.parseString(s))
def this(configMap: Map[String, _]) = this(AkkaSpec.mapToConfig(configMap))
- def this() = this(ActorSystem(AkkaSpec.testNameFromCallStack(classOf[StreamSpec]), AkkaSpec.testConf))
+ def this() = this(ActorSystem(TestKitUtils.testNameFromCallStack(classOf[StreamSpec], "".r), AkkaSpec.testConf))
override def withFixture(test: NoArgTest) = {
super.withFixture(test) match {
diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestDefaultMailbox.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestDefaultMailbox.scala
index 9742071951..9405c6f9fd 100644
--- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestDefaultMailbox.scala
+++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestDefaultMailbox.scala
@@ -4,15 +4,16 @@
package akka.stream.testkit
-import akka.dispatch.ProducesMessageQueue
-import akka.dispatch.UnboundedMailbox
-import akka.dispatch.MessageQueue
import com.typesafe.config.Config
-import akka.actor.ActorSystem
-import akka.dispatch.MailboxType
+
+import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.ActorRefWithCell
-import akka.actor.Actor
+import akka.actor.ActorSystem
+import akka.dispatch.MailboxType
+import akka.dispatch.MessageQueue
+import akka.dispatch.ProducesMessageQueue
+import akka.dispatch.UnboundedMailbox
import akka.stream.impl.MaterializerGuardian
/**
@@ -29,7 +30,7 @@ private[akka] final case class StreamTestDefaultMailbox()
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = {
owner match {
case Some(r: ActorRefWithCell) =>
- val actorClass = r.underlying.props.actorClass
+ val actorClass = r.underlying.props.actorClass()
assert(
actorClass != classOf[Actor],
s"Don't use anonymous actor classes, actor class for $r was [${actorClass.getName}]")
diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestKitSpec.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestKitSpec.scala
index e46fc20837..c2b0cf6328 100644
--- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestKitSpec.scala
+++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestKitSpec.scala
@@ -4,6 +4,8 @@
package akka.stream.testkit
+import scala.concurrent.duration._
+
import akka.stream.scaladsl.Source
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.AkkaSpec
@@ -11,8 +13,6 @@ import akka.testkit.EventFilter
import akka.testkit.TestEvent.Mute
import akka.testkit.TestEvent.UnMute
-import scala.concurrent.duration._
-
class StreamTestKitSpec extends AkkaSpec {
val ex = new Exception("Boom!")
diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/TestPublisherSubscriberSpec.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/TestPublisherSubscriberSpec.scala
index 59ee9b741b..77568fbd0d 100644
--- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/TestPublisherSubscriberSpec.scala
+++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/TestPublisherSubscriberSpec.scala
@@ -4,13 +4,14 @@
package akka.stream.testkit
+import org.reactivestreams.Subscription
+
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
import akka.stream.testkit.TestPublisher._
import akka.stream.testkit.TestSubscriber._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.AkkaSpec
-import org.reactivestreams.Subscription
class TestPublisherSubscriberSpec extends AkkaSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/TwoStreamsSetup.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/TwoStreamsSetup.scala
index f82dd87626..b44f525f63 100644
--- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/TwoStreamsSetup.scala
+++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/TwoStreamsSetup.scala
@@ -4,9 +4,10 @@
package akka.stream.testkit
+import org.reactivestreams.Publisher
+
import akka.stream._
import akka.stream.scaladsl._
-import org.reactivestreams.Publisher
abstract class TwoStreamsSetup extends BaseTwoStreamsSetup {
diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/Utils.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/Utils.scala
index 44f885e8c5..f68444e8f8 100644
--- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/Utils.scala
+++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/Utils.scala
@@ -4,10 +4,12 @@
package akka.stream.testkit
+import scala.util.control.NoStackTrace
+
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ActorRef
import akka.actor.ActorRefWithCell
-import com.typesafe.config.ConfigFactory
-import scala.util.control.NoStackTrace
object Utils {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ActorSystemLifecycle.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ActorSystemLifecycle.scala
index 9252c0e273..bba6fdf605 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ActorSystemLifecycle.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ActorSystemLifecycle.scala
@@ -6,19 +6,20 @@ package akka.stream.tck
import java.util.concurrent.TimeoutException
+import scala.concurrent.Await
import scala.concurrent.duration._
-import akka.actor.ActorSystem
-import akka.actor.ActorSystemImpl
-import org.testng.annotations.AfterClass
-import akka.testkit.AkkaSpec
-import akka.event.Logging
-import akka.testkit.TestEvent
-import akka.testkit.EventFilter
+
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
+import org.testng.annotations.AfterClass
import org.testng.annotations.BeforeClass
-import scala.concurrent.Await
+import akka.actor.ActorSystem
+import akka.actor.ActorSystemImpl
+import akka.event.Logging
+import akka.testkit.AkkaSpec
+import akka.testkit.EventFilter
+import akka.testkit.TestEvent
trait ActorSystemLifecycle {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaIdentityProcessorVerification.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaIdentityProcessorVerification.scala
index bfb1a33169..1c1258e753 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaIdentityProcessorVerification.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaIdentityProcessorVerification.scala
@@ -4,16 +4,18 @@
package akka.stream.tck
-import java.util.concurrent.Executors
import java.util.concurrent.ExecutorService
+import java.util.concurrent.Executors
import java.util.concurrent.TimeUnit
-import akka.stream.testkit.TestPublisher
+
import org.reactivestreams.{ Processor, Publisher, Subscriber, Subscription }
import org.reactivestreams.tck.IdentityProcessorVerification
import org.reactivestreams.tck.TestEnvironment
import org.scalatestplus.testng.TestNGSuiteLike
import org.testng.annotations.AfterClass
+import akka.stream.testkit.TestPublisher
+
abstract class AkkaIdentityProcessorVerification[T](env: TestEnvironment, publisherShutdownTimeout: Long)
extends IdentityProcessorVerification[T](env, publisherShutdownTimeout)
with TestNGSuiteLike
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaPublisherVerification.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaPublisherVerification.scala
index d787f878da..01e994450e 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaPublisherVerification.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaPublisherVerification.scala
@@ -4,7 +4,8 @@
package akka.stream.tck
-import akka.stream.testkit.TestPublisher
+import scala.collection.immutable
+
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import org.reactivestreams.Publisher
@@ -12,7 +13,7 @@ import org.reactivestreams.tck.PublisherVerification
import org.reactivestreams.tck.TestEnvironment
import org.scalatestplus.testng.TestNGSuiteLike
-import scala.collection.immutable
+import akka.stream.testkit.TestPublisher
abstract class AkkaPublisherVerification[T](val env: TestEnvironment, publisherShutdownTimeout: Long)
extends PublisherVerification[T](env, publisherShutdownTimeout)
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaSubscriberVerification.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaSubscriberVerification.scala
index 31dadfa401..422ad457dc 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaSubscriberVerification.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaSubscriberVerification.scala
@@ -4,12 +4,13 @@
package akka.stream.tck
-import akka.actor.ActorSystem
import org.reactivestreams.tck.SubscriberBlackboxVerification
import org.reactivestreams.tck.SubscriberWhiteboxVerification
import org.reactivestreams.tck.TestEnvironment
import org.scalatestplus.testng.TestNGSuiteLike
+import akka.actor.ActorSystem
+
abstract class AkkaSubscriberBlackboxVerification[T](env: TestEnvironment)
extends SubscriberBlackboxVerification[T](env)
with TestNGSuiteLike
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/CancelledSinkSubscriberTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/CancelledSinkSubscriberTest.scala
index c850c79526..77b2817d5e 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/CancelledSinkSubscriberTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/CancelledSinkSubscriberTest.scala
@@ -4,10 +4,11 @@
package akka.stream.tck
-import akka.stream.scaladsl._
import org.reactivestreams.Subscriber
import org.testng.SkipException
+import akka.stream.scaladsl._
+
class CancelledSinkSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] {
override def createSubscriber(): Subscriber[Int] =
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ConcatTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ConcatTest.scala
index 75084ccdd2..fa954df087 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ConcatTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ConcatTest.scala
@@ -4,9 +4,10 @@
package akka.stream.tck
+import org.reactivestreams.Publisher
+
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
-import org.reactivestreams.Publisher
class ConcatTest extends AkkaPublisherVerification[Int] {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/EmptyPublisherTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/EmptyPublisherTest.scala
index 9c98f50ad2..1b13d95df1 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/EmptyPublisherTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/EmptyPublisherTest.scala
@@ -5,6 +5,7 @@
package akka.stream.tck
import org.reactivestreams.Publisher
+
import akka.stream.impl.EmptyPublisher
class EmptyPublisherTest extends AkkaPublisherVerification[Int] {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FanoutPublisherTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FanoutPublisherTest.scala
index 2b1d613a1c..80d33582d3 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FanoutPublisherTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FanoutPublisherTest.scala
@@ -5,9 +5,11 @@
package akka.stream.tck
import scala.collection.immutable
+
+import org.reactivestreams.Publisher
+
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
-import org.reactivestreams.Publisher
class FanoutPublisherTest extends AkkaPublisherVerification[Int] {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FilePublisherTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FilePublisherTest.scala
index b635d749f1..7d1c8b16b7 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FilePublisherTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FilePublisherTest.scala
@@ -5,15 +5,17 @@
package akka.stream.tck
import java.nio.file.Files
+
+import org.reactivestreams.Publisher
+import org.testng.annotations.{ AfterClass, BeforeClass }
+
import akka.actor.ActorSystem
import akka.event.Logging
import akka.stream.scaladsl.{ FileIO, Sink }
import akka.stream.testkit.Utils._
import akka.testkit.{ EventFilter, TestEvent }
-import akka.util.ByteString
-import org.reactivestreams.Publisher
-import org.testng.annotations.{ AfterClass, BeforeClass }
import akka.testkit.AkkaSpec
+import akka.util.ByteString
class FilePublisherTest extends AkkaPublisherVerification[ByteString] {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FlatMapConcatDoubleSubscriberTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FlatMapConcatDoubleSubscriberTest.scala
index cfd1f99f27..846f8aaedb 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FlatMapConcatDoubleSubscriberTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FlatMapConcatDoubleSubscriberTest.scala
@@ -4,11 +4,12 @@
package akka.stream.tck
-import akka.stream.scaladsl.{ Sink, Source }
+import scala.concurrent.{ Await, Promise }
+import scala.concurrent.duration._
+
import org.reactivestreams.{ Publisher, Subscriber }
-import scala.concurrent.duration._
-import scala.concurrent.{ Await, Promise }
+import akka.stream.scaladsl.{ Sink, Source }
class FlatMapConcatDoubleSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FlatMapPrefixTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FlatMapPrefixTest.scala
index a3d7caa0bd..ec005c050e 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FlatMapPrefixTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FlatMapPrefixTest.scala
@@ -4,9 +4,10 @@
package akka.stream.tck
-import akka.stream.scaladsl.{ Flow, Keep, Sink, Source }
import org.reactivestreams.Publisher
+import akka.stream.scaladsl.{ Flow, Keep, Sink, Source }
+
class FlatMapPrefixTest extends AkkaPublisherVerification[Int] {
override def createPublisher(elements: Long): Publisher[Int] = {
val publisher = Source(iterable(elements))
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FlattenTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FlattenTest.scala
index 45cad9e209..71e80b331a 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FlattenTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FlattenTest.scala
@@ -4,10 +4,11 @@
package akka.stream.tck
+import org.reactivestreams.Publisher
+
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
import akka.util.ConstantFun
-import org.reactivestreams.Publisher
class FlattenTest extends AkkaPublisherVerification[Int] {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FoldSinkSubscriberTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FoldSinkSubscriberTest.scala
index e0befa8ffb..ff65ba1798 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FoldSinkSubscriberTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FoldSinkSubscriberTest.scala
@@ -4,9 +4,10 @@
package akka.stream.tck
-import akka.stream.scaladsl._
import org.reactivestreams.Subscriber
+import akka.stream.scaladsl._
+
class FoldSinkSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] {
override def createSubscriber(): Subscriber[Int] =
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ForeachSinkSubscriberTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ForeachSinkSubscriberTest.scala
index 1f33e051cb..a4ec583f48 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ForeachSinkSubscriberTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ForeachSinkSubscriberTest.scala
@@ -4,9 +4,10 @@
package akka.stream.tck
-import akka.stream.scaladsl._
import org.reactivestreams.Subscriber
+import akka.stream.scaladsl._
+
class ForeachSinkSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] {
override def createSubscriber(): Subscriber[Int] =
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala
index 34948b4f6d..5653a95ff2 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala
@@ -4,9 +4,10 @@
package akka.stream.tck
+import org.reactivestreams.Processor
+
import akka.stream._
import akka.stream.scaladsl.Flow
-import org.reactivestreams.Processor
class FusableProcessorTest extends AkkaIdentityProcessorVerification[Int] {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FuturePublisherTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FuturePublisherTest.scala
index b3948d36a5..b49963ac93 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FuturePublisherTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FuturePublisherTest.scala
@@ -4,11 +4,12 @@
package akka.stream.tck
-import akka.stream.scaladsl.Sink
-import akka.stream.scaladsl.Source
+import scala.concurrent.Promise
+
import org.reactivestreams._
-import scala.concurrent.Promise
+import akka.stream.scaladsl.Sink
+import akka.stream.scaladsl.Source
class FuturePublisherTest extends AkkaPublisherVerification[Int] {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/GroupByTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/GroupByTest.scala
index 8e920424d8..e077292577 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/GroupByTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/GroupByTest.scala
@@ -7,10 +7,11 @@ package akka.stream.tck
import scala.concurrent.Await
import scala.concurrent.duration._
+import org.reactivestreams.Publisher
+
import akka.stream.impl.EmptyPublisher
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
-import org.reactivestreams.Publisher
class GroupByTest extends AkkaPublisherVerification[Int] {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/InputStreamSourceTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/InputStreamSourceTest.scala
index 50a00b19a3..928d04ec0f 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/InputStreamSourceTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/InputStreamSourceTest.scala
@@ -6,10 +6,11 @@ package akka.stream.tck
import java.io.InputStream
+import org.reactivestreams.Publisher
+
import akka.stream.ActorAttributes
import akka.stream.scaladsl.{ Sink, StreamConverters }
import akka.util.ByteString
-import org.reactivestreams.Publisher
class InputStreamSourceTest extends AkkaPublisherVerification[ByteString] {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/IterablePublisherTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/IterablePublisherTest.scala
index b85839f810..79915ce912 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/IterablePublisherTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/IterablePublisherTest.scala
@@ -4,9 +4,10 @@
package akka.stream.tck
+import org.reactivestreams._
+
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
-import org.reactivestreams._
class IterablePublisherTest extends AkkaPublisherVerification[Int] {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/MapTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/MapTest.scala
index c81b50cf78..60c4aa6dfb 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/MapTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/MapTest.scala
@@ -4,9 +4,10 @@
package akka.stream.tck
-import akka.stream.scaladsl.Flow
import org.reactivestreams.Processor
+import akka.stream.scaladsl.Flow
+
class MapTest extends AkkaIdentityProcessorVerification[Int] {
override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/MaybeSourceTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/MaybeSourceTest.scala
index 8c3cd6f270..7b466592c5 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/MaybeSourceTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/MaybeSourceTest.scala
@@ -5,6 +5,7 @@
package akka.stream.tck
import org.reactivestreams.Publisher
+
import akka.stream.scaladsl.{ Keep, Sink, Source }
class MaybeSourceTest extends AkkaPublisherVerification[Int] {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/PrefixAndTailTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/PrefixAndTailTest.scala
index c9639595a0..0bf31269e7 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/PrefixAndTailTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/PrefixAndTailTest.scala
@@ -6,9 +6,11 @@ package akka.stream.tck
import scala.concurrent.Await
import scala.concurrent.duration._
+
+import org.reactivestreams.Publisher
+
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
-import org.reactivestreams.Publisher
class PrefixAndTailTest extends AkkaPublisherVerification[Int] {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/SingleElementSourceTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/SingleElementSourceTest.scala
index 509a21c08d..a9527ac4ec 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/SingleElementSourceTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/SingleElementSourceTest.scala
@@ -4,11 +4,11 @@
package akka.stream.tck
+import org.reactivestreams.Publisher
+
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
-import org.reactivestreams.Publisher
-
class SingleElementSourceTest extends AkkaPublisherVerification[Int] {
def createPublisher(elements: Long): Publisher[Int] =
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/SinkholeSubscriberTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/SinkholeSubscriberTest.scala
index ebf90b0530..a049fbd5fc 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/SinkholeSubscriberTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/SinkholeSubscriberTest.scala
@@ -4,14 +4,17 @@
package akka.stream.tck
-import akka.Done
-import akka.stream.impl.SinkholeSubscriber
+import java.lang.{ Integer => JInt }
+
+import scala.concurrent.Promise
+
+import org.reactivestreams.{ Subscriber, Subscription }
import org.reactivestreams.tck.{ SubscriberWhiteboxVerification, TestEnvironment }
import org.reactivestreams.tck.SubscriberWhiteboxVerification.{ SubscriberPuppet, WhiteboxSubscriberProbe }
-import org.scalatestplus.testng.{ TestNGSuiteLike }
-import java.lang.{ Integer => JInt }
-import scala.concurrent.Promise
-import org.reactivestreams.{ Subscriber, Subscription }
+import org.scalatestplus.testng.TestNGSuiteLike
+
+import akka.Done
+import akka.stream.impl.SinkholeSubscriber
class SinkholeSubscriberTest extends SubscriberWhiteboxVerification[JInt](new TestEnvironment()) with TestNGSuiteLike {
override def createSubscriber(probe: WhiteboxSubscriberProbe[JInt]): Subscriber[JInt] = {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/SplitWhenTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/SplitWhenTest.scala
index fdf35cb116..2337de9788 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/SplitWhenTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/SplitWhenTest.scala
@@ -7,10 +7,11 @@ package akka.stream.tck
import scala.concurrent.Await
import scala.concurrent.duration._
+import org.reactivestreams.Publisher
+
import akka.stream.impl.EmptyPublisher
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
-import org.reactivestreams.Publisher
class SplitWhenTest extends AkkaPublisherVerification[Int] {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala
index afc4aca53b..66756a6612 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala
@@ -4,13 +4,14 @@
package akka.stream.tck
+import org.reactivestreams.Processor
+
import akka.stream.Attributes
import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
import akka.stream.scaladsl.Flow
import akka.stream.stage.GraphStageLogic
import akka.stream.stage.InHandler
import akka.stream.stage.OutHandler
-import org.reactivestreams.Processor
class TransformProcessorTest extends AkkaIdentityProcessorVerification[Int] {
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/VirtualPublisherTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/VirtualPublisherTest.scala
index 1377ec7320..abad1e1789 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/VirtualPublisherTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/VirtualPublisherTest.scala
@@ -4,9 +4,10 @@
package akka.stream.tck
+import org.reactivestreams.Processor
+
import akka.stream.impl.VirtualProcessor
import akka.stream.scaladsl.Flow
-import org.reactivestreams.Processor
class VirtualProcessorTest extends AkkaIdentityProcessorVerification[Int] {
diff --git a/akka-stream-tests/src/test/scala-jdk9-only/akka/stream/scaladsl/FlowPublisherSinkSpec.scala b/akka-stream-tests/src/test/scala-jdk9-only/akka/stream/scaladsl/FlowPublisherSinkSpec.scala
index 725e008c19..12728bb9cd 100644
--- a/akka-stream-tests/src/test/scala-jdk9-only/akka/stream/scaladsl/FlowPublisherSinkSpec.scala
+++ b/akka-stream-tests/src/test/scala-jdk9-only/akka/stream/scaladsl/FlowPublisherSinkSpec.scala
@@ -4,14 +4,13 @@
package akka.stream.scaladsl
-import akka.stream.testkit.StreamSpec
-import akka.stream.ClosedShape
-
-import akka.stream.testkit.Utils._
-import akka.stream.testkit.scaladsl.StreamTestKit._
+import scala.concurrent.Await
import scala.concurrent.duration._
-import scala.concurrent.Await
+import akka.stream.ClosedShape
+import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.Utils._
+import akka.stream.testkit.scaladsl.StreamTestKit._
class FlowPublisherSinkSpec extends StreamSpec {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala
index 56ac524626..aa377df3b0 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala
@@ -4,26 +4,27 @@
package akka.stream
-import akka.Done
-import akka.actor.ExtendedActorSystem
-import akka.actor.Extension
-import akka.actor.ExtensionId
-import akka.actor.ExtensionIdProvider
-import akka.actor.{ Actor, ActorSystem, PoisonPill, Props }
-import akka.stream.ActorMaterializerSpec.ActorWithMaterializer
-import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor }
-import akka.stream.scaladsl.{ Sink, Source }
-import akka.stream.testkit.{ StreamSpec, TestPublisher }
-import akka.testkit.TestKit
-import akka.testkit.{ ImplicitSender, TestProbe }
-import com.github.ghik.silencer.silent
-import com.typesafe.config.ConfigFactory
-
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.duration._
import scala.util.{ Failure, Try }
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
+import akka.Done
+import akka.actor.{ Actor, ActorSystem, PoisonPill, Props }
+import akka.actor.ExtendedActorSystem
+import akka.actor.Extension
+import akka.actor.ExtensionId
+import akka.actor.ExtensionIdProvider
+import akka.stream.ActorMaterializerSpec.ActorWithMaterializer
+import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor }
+import akka.stream.scaladsl.{ Sink, Source }
+import akka.stream.testkit.{ StreamSpec, TestPublisher }
+import akka.testkit.{ ImplicitSender, TestProbe }
+import akka.testkit.TestKit
+
object IndirectMaterializerCreation extends ExtensionId[IndirectMaterializerCreation] with ExtensionIdProvider {
def createExtension(system: ExtendedActorSystem): IndirectMaterializerCreation =
new IndirectMaterializerCreation(system)
@@ -34,7 +35,7 @@ object IndirectMaterializerCreation extends ExtensionId[IndirectMaterializerCrea
@silent
class IndirectMaterializerCreation(ex: ExtendedActorSystem) extends Extension {
// extension instantiation blocked on materializer (which has Await.result inside)
- implicit val mat = ActorMaterializer()(ex)
+ implicit val mat: ActorMaterializer = ActorMaterializer()(ex)
def futureThing(n: Int): Future[Int] = {
Source.single(n).runWith(Sink.head)
@@ -160,7 +161,7 @@ object ActorMaterializerSpec {
class ActorWithMaterializer(p: TestProbe) extends Actor {
private val settings: ActorMaterializerSettings =
ActorMaterializerSettings(context.system).withDispatcher("akka.test.stream-dispatcher")
- implicit val mat = ActorMaterializer(settings)(context)
+ implicit val mat: ActorMaterializer = ActorMaterializer(settings)(context)
Source
.repeat("hello")
diff --git a/akka-stream-tests/src/test/scala/akka/stream/DslConsistencySpec.scala b/akka-stream-tests/src/test/scala/akka/stream/DslConsistencySpec.scala
index 2a86584d1a..69584caad2 100755
--- a/akka-stream-tests/src/test/scala/akka/stream/DslConsistencySpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/DslConsistencySpec.scala
@@ -6,6 +6,7 @@ package akka.stream
import java.lang.reflect.Method
import java.lang.reflect.Modifier
+
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
diff --git a/akka-stream-tests/src/test/scala/akka/stream/StreamAttributeDocSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/StreamAttributeDocSpec.scala
index 964711bb53..30bd902445 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/StreamAttributeDocSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/StreamAttributeDocSpec.scala
@@ -4,6 +4,8 @@
package akka.stream
+import scala.concurrent.Future
+
import akka.Done
import akka.stream.scaladsl.Keep
import akka.stream.scaladsl.RunnableGraph
@@ -12,8 +14,6 @@ import akka.stream.scaladsl.Source
import akka.stream.scaladsl.TcpAttributes
import akka.stream.testkit.StreamSpec
-import scala.concurrent.Future
-
class StreamAttributeDocSpec extends StreamSpec("my-stream-dispatcher = \"akka.test.stream-dispatcher\"") {
"Setting attributes on the runnable stream" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/SystemMaterializerSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/SystemMaterializerSpec.scala
index f609f2c129..ef5e9aa02b 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/SystemMaterializerSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/SystemMaterializerSpec.scala
@@ -4,13 +4,14 @@
package akka.stream
+import scala.concurrent.Future
+
+import org.scalatest.concurrent.ScalaFutures
+
import akka.stream.scaladsl.Keep
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
import akka.stream.testkit.StreamSpec
-import org.scalatest.concurrent.ScalaFutures
-
-import scala.concurrent.Future
class SystemMaterializerSpec extends StreamSpec with ScalaFutures {
@@ -38,7 +39,7 @@ class SystemMaterializerEagerStartupSpec extends StreamSpec {
"The SystemMaterializer" must {
"be eagerly started on system startup" in {
- system.hasExtension(SystemMaterializer.lookup) should ===(true)
+ system.hasExtension(SystemMaterializer.lookup()) should ===(true)
}
}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/FixedBufferSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/FixedBufferSpec.scala
index a8d4c9df6f..0c2350ae39 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/impl/FixedBufferSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/impl/FixedBufferSpec.scala
@@ -4,11 +4,12 @@
package akka.stream.impl
+import com.github.ghik.silencer.silent
+
import akka.stream.ActorAttributes
import akka.stream.ActorAttributes.MaxFixedBufferSize
import akka.stream.ActorMaterializerSettings
import akka.stream.testkit.StreamSpec
-import com.github.ghik.silencer.silent
@silent("deprecated")
class FixedBufferSpec extends StreamSpec {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/GraphStageLogicSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/GraphStageLogicSpec.scala
index a1c12a46bf..228325a023 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/impl/GraphStageLogicSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/impl/GraphStageLogicSpec.scala
@@ -4,18 +4,19 @@
package akka.stream.impl
-import akka.NotUsed
-import akka.stream.stage.GraphStageLogic.{ EagerTerminateInput, EagerTerminateOutput }
-import akka.stream.testkit.StreamSpec
-import akka.stream._
-import akka.stream.scaladsl._
-import akka.stream.stage._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit.scaladsl.TestSink
-import akka.stream.impl.fusing._
+import scala.concurrent.duration.Duration
+
import org.scalatest.concurrent.ScalaFutures
-import scala.concurrent.duration.Duration
+import akka.NotUsed
+import akka.stream._
+import akka.stream.impl.fusing._
+import akka.stream.scaladsl._
+import akka.stream.stage._
+import akka.stream.stage.GraphStageLogic.{ EagerTerminateInput, EagerTerminateOutput }
+import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.stream.testkit.scaladsl.TestSink
class GraphStageLogicSpec extends StreamSpec with GraphInterpreterSpecKit with ScalaFutures {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/ResizableMultiReaderRingBufferSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/ResizableMultiReaderRingBufferSpec.scala
index 4780ec6c21..fc448ff22f 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/impl/ResizableMultiReaderRingBufferSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/impl/ResizableMultiReaderRingBufferSpec.scala
@@ -5,10 +5,12 @@
package akka.stream.impl
import scala.util.Random
-import akka.stream.impl.ResizableMultiReaderRingBuffer._
+
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.stream.impl.ResizableMultiReaderRingBuffer._
+
class ResizableMultiReaderRingBufferSpec extends AnyWordSpec with Matchers {
"A ResizableMultiReaderRingBuffer" should {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/TimeoutsSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/TimeoutsSpec.scala
index 1c55664039..87ddd384a9 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/impl/TimeoutsSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/impl/TimeoutsSpec.scala
@@ -6,6 +6,13 @@ package akka.stream.impl
import java.util.concurrent.TimeoutException
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.Done
import akka.stream._
import akka.stream.scaladsl._
@@ -15,12 +22,6 @@ import akka.stream.testkit.TestSubscriber
import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
-import scala.concurrent.Await
-import scala.concurrent.Future
-import scala.concurrent.duration._
-import org.scalatest.matchers.should.Matchers
-import org.scalatest.wordspec.AnyWordSpecLike
-
class TimeoutsSpec extends StreamSpec {
"InitialTimeout" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/TraversalTestUtils.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/TraversalTestUtils.scala
index 1ef57980eb..dfa26241fa 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/impl/TraversalTestUtils.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/impl/TraversalTestUtils.scala
@@ -5,8 +5,8 @@
package akka.stream.impl
import akka.NotUsed
-import akka.stream.impl.StreamLayout.AtomicModule
import akka.stream._
+import akka.stream.impl.StreamLayout.AtomicModule
object TraversalTestUtils {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ActorGraphInterpreterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ActorGraphInterpreterSpec.scala
index 0521b534db..446eb41ba5 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ActorGraphInterpreterSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ActorGraphInterpreterSpec.scala
@@ -6,6 +6,14 @@ package akka.stream.impl.fusing
import java.util.concurrent.CountDownLatch
+import scala.concurrent.Await
+import scala.concurrent.Promise
+import scala.concurrent.duration._
+
+import org.reactivestreams.Publisher
+import org.reactivestreams.Subscriber
+import org.reactivestreams.Subscription
+
import akka.Done
import akka.stream._
import akka.stream.impl.ReactiveStreamsCompliance.SpecViolation
@@ -22,13 +30,6 @@ import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.EventFilter
import akka.testkit.TestLatch
-import org.reactivestreams.Publisher
-import org.reactivestreams.Subscriber
-import org.reactivestreams.Subscription
-
-import scala.concurrent.Await
-import scala.concurrent.Promise
-import scala.concurrent.duration._
class ActorGraphInterpreterSpec extends StreamSpec {
"ActorGraphInterpreter" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/AsyncCallbackSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/AsyncCallbackSpec.scala
index 2ef97b28cf..be230f9f08 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/AsyncCallbackSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/AsyncCallbackSpec.scala
@@ -4,6 +4,11 @@
package akka.stream.impl.fusing
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.Promise
+import scala.language.reflectiveCalls
+
import akka.Done
import akka.actor.ActorRef
import akka.stream._
@@ -11,17 +16,12 @@ import akka.stream.scaladsl.Keep
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
import akka.stream.stage._
-import akka.stream.testkit.Utils.TE
import akka.stream.testkit.TestPublisher
import akka.stream.testkit.TestSubscriber
+import akka.stream.testkit.Utils.TE
import akka.testkit.AkkaSpec
import akka.testkit.TestProbe
-import scala.concurrent.Await
-import scala.concurrent.Future
-import scala.concurrent.Promise
-import scala.language.reflectiveCalls
-
class AsyncCallbackSpec extends AkkaSpec("""
akka.stream.materializer.debug.fuzzing-mode = off
""") {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ChasingEventsSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ChasingEventsSpec.scala
index 7880cdcb61..540753f697 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ChasingEventsSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ChasingEventsSpec.scala
@@ -11,9 +11,9 @@ import akka.stream.stage.GraphStage
import akka.stream.stage.GraphStageLogic
import akka.stream.stage.InHandler
import akka.stream.stage.OutHandler
-import akka.stream.testkit.Utils.TE
import akka.stream.testkit.TestPublisher
import akka.stream.testkit.TestSubscriber
+import akka.stream.testkit.Utils.TE
import akka.testkit.AkkaSpec
class ChasingEventsSpec extends AkkaSpec("""
diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpec.scala
index 1b275b692f..8760e27c8d 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpec.scala
@@ -15,7 +15,7 @@ class GraphInterpreterSpec extends StreamSpec with GraphInterpreterSpecKit {
// Reusable components
val identity = GraphStages.identity[Int]
val detach = detacher[Int]
- val zip = Zip[Int, String]
+ val zip = Zip[Int, String]()
val bcast = Broadcast[Int](2)
val merge = Merge[Int](2)
val balance = Balance[Int](2)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSpec.scala
index 581d0e5d60..f3f0b6191f 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSpec.scala
@@ -4,13 +4,14 @@
package akka.stream.impl.fusing
+import com.github.ghik.silencer.silent
+
+import akka.stream._
+import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
import akka.stream.stage._
import akka.stream.testkit.StreamSpec
import akka.testkit.EventFilter
-import akka.stream._
-import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
import akka.util.ConstantFun
-import com.github.ghik.silencer.silent
class InterpreterSpec extends StreamSpec with GraphInterpreterSpecKit {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSupervisionSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSupervisionSpec.scala
index 91fc061208..c5ba892fc1 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSupervisionSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSupervisionSpec.scala
@@ -4,10 +4,10 @@
package akka.stream.impl.fusing
-import akka.stream.testkit.StreamSpec
-
import scala.util.control.NoStackTrace
+
import akka.stream.Supervision
+import akka.stream.testkit.StreamSpec
class InterpreterSupervisionSpec extends StreamSpec with GraphInterpreterSpecKit {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/KeepGoingStageSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/KeepGoingStageSpec.scala
index 239cab0e06..012cae8c34 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/KeepGoingStageSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/KeepGoingStageSpec.scala
@@ -4,8 +4,16 @@
package akka.stream.impl.fusing
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.Promise
+import scala.concurrent.duration._
+
import akka.actor.ActorRef
import akka.actor.NoSerializationVerificationNeeded
+import akka.stream.Attributes
+import akka.stream.Inlet
+import akka.stream.SinkShape
import akka.stream.scaladsl.Keep
import akka.stream.scaladsl.Source
import akka.stream.stage.AsyncCallback
@@ -15,14 +23,6 @@ import akka.stream.stage.InHandler
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.Attributes
-import akka.stream.Inlet
-import akka.stream.SinkShape
-
-import scala.concurrent.duration._
-import scala.concurrent.Await
-import scala.concurrent.Future
-import scala.concurrent.Promise
class KeepGoingStageSpec extends StreamSpec {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/LifecycleInterpreterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/LifecycleInterpreterSpec.scala
index 59787868d9..4683b7ea68 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/LifecycleInterpreterSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/LifecycleInterpreterSpec.scala
@@ -4,14 +4,14 @@
package akka.stream.impl.fusing
+import scala.concurrent.duration._
+
import akka.stream.Attributes
import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
import akka.stream.stage._
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.Utils.TE
-import scala.concurrent.duration._
-
class LifecycleInterpreterSpec extends StreamSpec with GraphInterpreterSpecKit {
val boom = TE("Boom!")
diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/ByteStringParserSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/ByteStringParserSpec.scala
index 8c18ba8d9b..ea75420e40 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/io/ByteStringParserSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/io/ByteStringParserSpec.scala
@@ -4,6 +4,11 @@
package akka.stream.io
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import akka.stream.Attributes
+import akka.stream.ThrottleMode
import akka.stream.impl.io.ByteStringParser
import akka.stream.impl.io.ByteStringParser.ByteReader
import akka.stream.impl.io.ByteStringParser.ParseResult
@@ -14,13 +19,8 @@ import akka.stream.stage.GraphStageLogic
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.TestPublisher
import akka.stream.testkit.TestSubscriber
-import akka.stream.Attributes
-import akka.stream.ThrottleMode
import akka.util.ByteString
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
class ByteStringParserSpec extends StreamSpec {
"ByteStringParser" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/DeprecatedTlsSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/DeprecatedTlsSpec.scala
index d5c6fc13c4..dfa6da1907 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/io/DeprecatedTlsSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/io/DeprecatedTlsSpec.scala
@@ -8,6 +8,7 @@ import java.security.KeyStore
import java.security.SecureRandom
import java.security.cert.CertificateException
import java.util.concurrent.TimeoutException
+import javax.net.ssl._
import scala.collection.immutable
import scala.concurrent.Await
@@ -15,22 +16,22 @@ import scala.concurrent.Future
import scala.concurrent.duration._
import scala.util.Random
+import com.github.ghik.silencer.silent
+import com.typesafe.sslconfig.akka.AkkaSSLConfig
+
import akka.NotUsed
import akka.pattern.{ after => later }
-import akka.stream.TLSProtocol._
import akka.stream._
+import akka.stream.TLSProtocol._
import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
import akka.stream.scaladsl._
import akka.stream.stage._
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.testkit.WithLogCapturing
import akka.testkit.TestDuration
+import akka.testkit.WithLogCapturing
import akka.util.ByteString
import akka.util.JavaVersion
-import com.github.ghik.silencer.silent
-import com.typesafe.sslconfig.akka.AkkaSSLConfig
-import javax.net.ssl._
object DeprecatedTlsSpec {
@@ -98,8 +99,8 @@ object DeprecatedTlsSpec {
@silent("deprecated")
class DeprecatedTlsSpec extends StreamSpec(DeprecatedTlsSpec.configOverrides) with WithLogCapturing {
- import GraphDSL.Implicits._
import DeprecatedTlsSpec._
+ import GraphDSL.Implicits._
import system.dispatcher
val sslConfig: Option[AkkaSSLConfig] = None // no special settings to be applied here
diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/FileSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/FileSinkSpec.scala
index 1fb034e72a..721d4ecd50 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/io/FileSinkSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/io/FileSinkSpec.scala
@@ -4,32 +4,33 @@
package akka.stream.io
-import java.nio.file.StandardOpenOption.{ CREATE, WRITE }
import java.nio.file._
+import java.nio.file.StandardOpenOption.{ CREATE, WRITE }
+
+import scala.collection.mutable.ListBuffer
+import scala.concurrent.{ Await, Future }
+import scala.concurrent.duration._
+import scala.util.Success
+
+import com.github.ghik.silencer.silent
+import com.google.common.jimfs.{ Configuration, Jimfs }
+import org.scalatest.concurrent.ScalaFutures
import akka.dispatch.ExecutionContexts
import akka.stream._
import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor }
import akka.stream.impl.StreamSupervisor.Children
import akka.stream.scaladsl.{ FileIO, Keep, Sink, Source }
-import akka.stream.testkit.Utils._
import akka.stream.testkit._
+import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.util.ByteString
-import com.github.ghik.silencer.silent
-import com.google.common.jimfs.{ Configuration, Jimfs }
-import org.scalatest.concurrent.ScalaFutures
-
-import scala.collection.mutable.ListBuffer
-import scala.concurrent.duration._
-import scala.concurrent.{ Await, Future }
-import scala.util.Success
@silent
class FileSinkSpec extends StreamSpec(UnboundedMailboxConfig) with ScalaFutures {
val settings = ActorMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher")
- implicit val materializer = ActorMaterializer(settings)
+ implicit val materializer: ActorMaterializer = ActorMaterializer(settings)
val fs = Jimfs.newFileSystem("FileSinkSpec", Configuration.unix())
val TestLines = {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/FileSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/FileSourceSpec.scala
index 5b3006db58..773e33d31f 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/io/FileSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/io/FileSourceSpec.scala
@@ -8,24 +8,24 @@ import java.nio.charset.StandardCharsets.UTF_8
import java.nio.file.{ Files, NoSuchFileException }
import java.util.Random
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
+import com.google.common.jimfs.{ Configuration, Jimfs }
+
import akka.actor.ActorSystem
-import akka.stream.IOResult._
import akka.stream._
+import akka.stream.IOResult._
import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor }
import akka.stream.impl.StreamSupervisor.Children
import akka.stream.io.FileSourceSpec.Settings
import akka.stream.scaladsl.{ FileIO, Keep, Sink }
+import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit._
import akka.stream.testkit.scaladsl.TestSink
import akka.util.ByteString
-import com.google.common.jimfs.{ Configuration, Jimfs }
-
-import scala.concurrent.duration._
-import com.github.ghik.silencer.silent
-
-import scala.concurrent.Future
object FileSourceSpec {
final case class Settings(chunkSize: Int, readAhead: Int)
@@ -35,7 +35,7 @@ object FileSourceSpec {
class FileSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
val settings = ActorMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher")
- implicit val materializer = ActorMaterializer(settings)
+ implicit val materializer: ActorMaterializer = ActorMaterializer(settings)
val fs = Jimfs.newFileSystem("FileSourceSpec", Configuration.unix())
diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSinkSpec.scala
index 652c018565..3deeea3de7 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSinkSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSinkSpec.scala
@@ -9,8 +9,13 @@ import java.io.InputStream
import java.util.concurrent.ThreadLocalRandom
import java.util.concurrent.TimeoutException
-import akka.stream.Attributes.inputBuffer
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
import akka.stream._
+import akka.stream.Attributes.inputBuffer
import akka.stream.impl.PhasedFusingActorMaterializer
import akka.stream.impl.StreamSupervisor
import akka.stream.impl.StreamSupervisor.Children
@@ -18,18 +23,13 @@ import akka.stream.impl.io.InputStreamSinkStage
import akka.stream.scaladsl.Keep
import akka.stream.scaladsl.Source
import akka.stream.scaladsl.StreamConverters
-import akka.stream.testkit.Utils._
import akka.stream.testkit._
+import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSource
import akka.testkit.TestProbe
import akka.util.ByteString
-import scala.concurrent.Await
-import scala.concurrent.Future
-import scala.concurrent.duration._
-import scala.util.control.NoStackTrace
-
class InputStreamSinkSpec extends StreamSpec(UnboundedMailboxConfig) {
import system.dispatcher
@@ -263,10 +263,8 @@ class InputStreamSinkSpec extends StreamSpec(UnboundedMailboxConfig) {
"propagate error to InputStream" in {
val readTimeout = 3.seconds
- val (probe, inputStream) =
+ val (probe, inputStream: InputStream) =
TestSource.probe[ByteString].toMat(StreamConverters.asInputStream(readTimeout))(Keep.both).run()
-
- probe.sendNext(ByteString("one"))
val error = new RuntimeException("failure")
probe.sendError(error)
val buffer = Array.ofDim[Byte](5)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSourceSpec.scala
index d72a503eae..ccb27bec6c 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSourceSpec.scala
@@ -7,23 +7,24 @@ package akka.stream.io
import java.io.{ ByteArrayInputStream, InputStream }
import java.util.concurrent.CountDownLatch
+import scala.util.Success
+
+import com.github.ghik.silencer.silent
+
import akka.Done
+import akka.stream.{ AbruptStageTerminationException, ActorMaterializer, ActorMaterializerSettings, IOResult }
import akka.stream.scaladsl.{ Keep, Sink, StreamConverters }
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
-import akka.stream.{ AbruptStageTerminationException, ActorMaterializer, ActorMaterializerSettings, IOResult }
import akka.util.ByteString
-import scala.util.Success
-import com.github.ghik.silencer.silent
-
@silent
class InputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
val settings = ActorMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher")
- implicit val materializer = ActorMaterializer(settings)
+ implicit val materializer: ActorMaterializer = ActorMaterializer(settings)
private def inputStreamFor(bytes: Array[Byte]): InputStream =
new ByteArrayInputStream(bytes)
@@ -54,7 +55,7 @@ class InputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
override def close(): Unit = throw fail
})
.toMat(Sink.ignore)(Keep.left)
- .run
+ .run()
.failed
.futureValue
.getCause shouldEqual fail
@@ -67,7 +68,7 @@ class InputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
throw fail
})
.toMat(Sink.ignore)(Keep.left)
- .run
+ .run()
.failed
.futureValue
.getCause shouldEqual fail
@@ -78,7 +79,7 @@ class InputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
StreamConverters
.fromInputStream(() => () => throw fail)
.toMat(Sink.ignore)(Keep.left)
- .run
+ .run()
.failed
.futureValue
.getCause shouldEqual fail
diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSinkSpec.scala
index 19a4f7216b..683285c0b1 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSinkSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSinkSpec.scala
@@ -6,24 +6,25 @@ package akka.stream.io
import java.io.OutputStream
+import scala.util.Success
+
+import com.github.ghik.silencer.silent
+import org.scalatest.concurrent.ScalaFutures
+
import akka.Done
+import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, IOOperationIncompleteException }
import akka.stream.scaladsl.{ Source, StreamConverters }
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, IOOperationIncompleteException }
import akka.testkit.TestProbe
import akka.util.ByteString
-import org.scalatest.concurrent.ScalaFutures
-import com.github.ghik.silencer.silent
-
-import scala.util.Success
@silent
class OutputStreamSinkSpec extends StreamSpec(UnboundedMailboxConfig) with ScalaFutures {
val settings = ActorMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher")
- implicit val materializer = ActorMaterializer(settings)
+ implicit val materializer: ActorMaterializer = ActorMaterializer(settings)
"OutputStreamSink" must {
"write bytes to void OutputStream" in assertAllStagesStopped {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSourceSpec.scala
index 84a5a57436..06fe3ab2e0 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSourceSpec.scala
@@ -8,24 +8,24 @@ import java.io.IOException
import java.lang.management.ManagementFactory
import java.util.concurrent.TimeoutException
-import akka.stream.Attributes.inputBuffer
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.concurrent.duration.Duration.Zero
+import scala.util.Random
+
import akka.stream._
+import akka.stream.Attributes.inputBuffer
import akka.stream.impl.io.OutputStreamSourceStage
import akka.stream.scaladsl.Keep
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.StreamConverters
-import akka.stream.testkit.Utils._
import akka.stream.testkit._
+import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
import akka.util.ByteString
-import scala.concurrent.Await
-import scala.concurrent.Future
-import scala.concurrent.duration.Duration.Zero
-import scala.concurrent.duration._
-import scala.util.Random
-
class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
import system.dispatcher
@@ -57,7 +57,7 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
"OutputStreamSource" must {
"read bytes from OutputStream" in assertAllStagesStopped {
- val (outputStream, probe) = StreamConverters.asOutputStream().toMat(TestSink.probe[ByteString])(Keep.both).run
+ val (outputStream, probe) = StreamConverters.asOutputStream().toMat(TestSink.probe[ByteString])(Keep.both).run()
val s = probe.expectSubscription()
outputStream.write(bytesArray)
@@ -74,7 +74,7 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
StreamConverters
.asOutputStream()
.toMat(Sink.fold[ByteString, ByteString](ByteString.empty)(_ ++ _))(Keep.both)
- .run
+ .run()
outputStream.write(bytesArray)
outputStream.close()
result.futureValue should be(ByteString(bytesArray))
@@ -82,7 +82,7 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
}
"not block flushes when buffer is empty" in assertAllStagesStopped {
- val (outputStream, probe) = StreamConverters.asOutputStream().toMat(TestSink.probe[ByteString])(Keep.both).run
+ val (outputStream, probe) = StreamConverters.asOutputStream().toMat(TestSink.probe[ByteString])(Keep.both).run()
val s = probe.expectSubscription()
outputStream.write(bytesArray)
@@ -104,7 +104,7 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
.asOutputStream()
.toMat(TestSink.probe[ByteString])(Keep.both)
.withAttributes(Attributes.inputBuffer(16, 16))
- .run
+ .run()
val s = probe.expectSubscription()
(1 to 16).foreach { _ =>
@@ -126,7 +126,7 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
}
"throw error when write after stream is closed" in assertAllStagesStopped {
- val (outputStream, probe) = StreamConverters.asOutputStream().toMat(TestSink.probe[ByteString])(Keep.both).run
+ val (outputStream, probe) = StreamConverters.asOutputStream().toMat(TestSink.probe[ByteString])(Keep.both).run()
probe.expectSubscription()
outputStream.close()
@@ -135,7 +135,7 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
}
"throw IOException when writing to the stream after the subscriber has cancelled the reactive stream" in assertAllStagesStopped {
- val (outputStream, sink) = StreamConverters.asOutputStream().toMat(TestSink.probe[ByteString])(Keep.both).run
+ val (outputStream, sink) = StreamConverters.asOutputStream().toMat(TestSink.probe[ByteString])(Keep.both).run()
val s = sink.expectSubscription()
@@ -202,7 +202,7 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
.asOutputStream(timeout)
.addAttributes(Attributes.inputBuffer(bufSize, bufSize))
.toMat(TestSink.probe[ByteString])(Keep.both)
- .run
+ .run()
// fill the buffer up
(1 to (bufSize - 1)).foreach(outputStream.write)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala b/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala
index c1f9bec7e4..010aa52efc 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala
@@ -6,19 +6,19 @@ package akka.stream.io
import java.net.InetSocketAddress
+import scala.collection.immutable.Queue
+import scala.concurrent.duration._
+
import akka.actor._
-import akka.io.Tcp.ConnectionClosed
-import akka.io.Tcp.ResumeReading
import akka.io.IO
import akka.io.Tcp
+import akka.io.Tcp.ConnectionClosed
+import akka.io.Tcp.ResumeReading
import akka.stream.testkit._
import akka.testkit.SocketUtil.temporaryServerAddress
import akka.testkit.TestProbe
import akka.util.ByteString
-import scala.collection.immutable.Queue
-import scala.concurrent.duration._
-
object TcpHelper {
case class ClientWrite(bytes: ByteString) extends NoSerializationVerificationNeeded
case class ClientRead(count: Int, readTo: ActorRef) extends NoSerializationVerificationNeeded
diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala
index 3288c8523f..af5d203fcd 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala
@@ -8,6 +8,20 @@ import java.net._
import java.security.SecureRandom
import java.util.concurrent.atomic.AtomicInteger
+import scala.collection.immutable
+import scala.concurrent.Await
+import scala.concurrent.ExecutionContext
+import scala.concurrent.Future
+import scala.concurrent.Promise
+import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
+import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+import org.scalatest.concurrent.PatienceConfiguration
+import org.scalatest.concurrent.PatienceConfiguration.Timeout
+
import akka.Done
import akka.NotUsed
import akka.actor.Actor
@@ -23,10 +37,10 @@ import akka.io.SimpleDnsCache
import akka.io.Tcp._
import akka.io.dns.DnsProtocol
import akka.stream._
+import akka.stream.scaladsl._
import akka.stream.scaladsl.Flow
import akka.stream.scaladsl.Tcp.IncomingConnection
import akka.stream.scaladsl.Tcp.ServerBinding
-import akka.stream.scaladsl._
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.EventFilter
@@ -36,19 +50,6 @@ import akka.testkit.TestLatch
import akka.testkit.TestProbe
import akka.testkit.WithLogCapturing
import akka.util.ByteString
-import com.github.ghik.silencer.silent
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-import org.scalatest.concurrent.PatienceConfiguration
-import org.scalatest.concurrent.PatienceConfiguration.Timeout
-
-import scala.collection.immutable
-import scala.concurrent.Await
-import scala.concurrent.ExecutionContext
-import scala.concurrent.Future
-import scala.concurrent.Promise
-import scala.concurrent.duration._
-import com.github.ghik.silencer.silent
@silent("never used")
class NonResolvingDnsActor(cache: SimpleDnsCache, config: Config) extends Actor {
@@ -920,10 +921,11 @@ class TcpSpec extends StreamSpec("""
// #setting-up-ssl-engine
import java.security.KeyStore
- import javax.net.ssl.SSLEngine
- import javax.net.ssl.TrustManagerFactory
import javax.net.ssl.KeyManagerFactory
import javax.net.ssl.SSLContext
+ import javax.net.ssl.SSLEngine
+ import javax.net.ssl.TrustManagerFactory
+
import akka.stream.TLSRole
// initialize SSLContext once
@@ -1007,11 +1009,12 @@ class TcpSpec extends StreamSpec("""
def initSslMess() = {
// #setting-up-ssl-context
import java.security.KeyStore
+ import javax.net.ssl._
+
+ import com.typesafe.sslconfig.akka.AkkaSSLConfig
import akka.stream.TLSClientAuth
import akka.stream.TLSProtocol
- import com.typesafe.sslconfig.akka.AkkaSSLConfig
- import javax.net.ssl._
val sslConfig = AkkaSSLConfig()
diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/TlsSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/TlsSpec.scala
index 2902eff337..2c737b085c 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/io/TlsSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/io/TlsSpec.scala
@@ -8,26 +8,26 @@ import java.security.KeyStore
import java.security.SecureRandom
import java.security.cert.CertificateException
import java.util.concurrent.TimeoutException
+import javax.net.ssl._
-import akka.NotUsed
import scala.collection.immutable
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.duration._
import scala.util.Random
+import akka.NotUsed
import akka.pattern.{ after => later }
import akka.stream._
import akka.stream.TLSProtocol._
+import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
import akka.stream.scaladsl._
import akka.stream.stage._
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.TestDuration
-import akka.util.{ ByteString, JavaVersion }
-import javax.net.ssl._
-import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
import akka.testkit.WithLogCapturing
+import akka.util.{ ByteString, JavaVersion }
object TlsSpec {
@@ -97,11 +97,9 @@ object TlsSpec {
}
class TlsSpec extends StreamSpec(TlsSpec.configOverrides) with WithLogCapturing {
- import TlsSpec._
-
- import system.dispatcher
-
import GraphDSL.Implicits._
+ import TlsSpec._
+ import system.dispatcher
"SslTls" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/compression/CodecSpecSupport.scala b/akka-stream-tests/src/test/scala/akka/stream/io/compression/CodecSpecSupport.scala
index bce130c89b..2b4b3f5994 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/io/compression/CodecSpecSupport.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/io/compression/CodecSpecSupport.scala
@@ -4,13 +4,14 @@
package akka.stream.io.compression
-import akka.actor.ActorSystem
-import akka.testkit.TestKit
-import akka.util.ByteString
import org.scalatest.BeforeAndAfterAll
import org.scalatest.Suite
import org.scalatest.matchers.should.Matchers
+import akka.actor.ActorSystem
+import akka.testkit.TestKit
+import akka.util.ByteString
+
trait CodecSpecSupport extends Matchers with BeforeAndAfterAll { self: Suite =>
def readAs(string: String, charset: String = "UTF8") =
@@ -73,7 +74,7 @@ est Lorem ipsum dolor sit amet. Lorem ipsum dolor sit amet, consetetur sadipscin
"\r\n",
"\n")
- implicit val system = ActorSystem(getClass.getSimpleName)
+ implicit val system: ActorSystem = ActorSystem(getClass.getSimpleName)
override def afterAll() = TestKit.shutdownActorSystem(system)
}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/compression/CoderSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/compression/CoderSpec.scala
index 6c54b6ff05..dae319a6b5 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/io/compression/CoderSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/io/compression/CoderSpec.scala
@@ -8,18 +8,19 @@ import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, InputStream, Outpu
import java.util.concurrent.ThreadLocalRandom
import java.util.zip.DataFormatException
-import akka.NotUsed
-import akka.stream.impl.io.compression.Compressor
-import akka.stream.scaladsl.{ Compression, Flow, Sink, Source }
-import akka.util.ByteString
-import org.scalatest.Inspectors
-import org.scalatest.wordspec.AnyWordSpec
-
import scala.annotation.tailrec
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
+import org.scalatest.Inspectors
+import org.scalatest.wordspec.AnyWordSpec
+
+import akka.NotUsed
+import akka.stream.impl.io.compression.Compressor
+import akka.stream.scaladsl.{ Compression, Flow, Sink, Source }
+import akka.util.ByteString
+
abstract class CoderSpec(codecName: String) extends AnyWordSpec with CodecSpecSupport with Inspectors {
import CompressionTestingTools._
diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/compression/CompressionTestingTools.scala b/akka-stream-tests/src/test/scala/akka/stream/io/compression/CompressionTestingTools.scala
index 7d35e18c6b..5876b79ca9 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/io/compression/CompressionTestingTools.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/io/compression/CompressionTestingTools.scala
@@ -4,16 +4,16 @@
package akka.stream.io.compression
-import akka.stream.Materializer
-import akka.stream.scaladsl.Source
-import akka.util.ByteString
-import akka.util.ccompat._
-
import scala.annotation.tailrec
import scala.concurrent.{ Await, ExecutionContext, Future }
import scala.concurrent.duration.Duration
import scala.util.{ Failure, Success }
+import akka.stream.Materializer
+import akka.stream.scaladsl.Source
+import akka.util.ByteString
+import akka.util.ccompat._
+
// a few useful helpers copied over from akka-http
@ccompatUsedUntil213
object CompressionTestingTools {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSinkSpec.scala
index a92a307fb2..bf69d4349f 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSinkSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSinkSpec.scala
@@ -4,17 +4,17 @@
package akka.stream.scaladsl
+import scala.concurrent.Promise
+import scala.concurrent.duration._
+
import akka.actor.{ Actor, ActorRef, Props }
import akka.stream.Attributes.inputBuffer
import akka.stream.Materializer
import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl._
+import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.TestProbe
-import scala.concurrent.Promise
-import scala.concurrent.duration._
-
object ActorRefBackpressureSinkSpec {
val initMessage = "start"
val completeMessage = "done"
@@ -131,7 +131,7 @@ class ActorRefBackpressureSinkSpec extends StreamSpec {
val sink = Sink
.actorRefWithBackpressure(fw, initMessage, ackMessage, completeMessage, _ => failMessage)
.withAttributes(inputBuffer(bufferSize, bufferSize))
- val bufferFullProbe = Promise[akka.Done.type]
+ val bufferFullProbe = Promise[akka.Done.type]()
Source(1 to streamElementCount)
.alsoTo(Flow[Int].drop(bufferSize - 1).to(Sink.foreach(_ => bufferFullProbe.trySuccess(akka.Done))))
.to(sink)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSourceSpec.scala
index aa42160e2a..738b7ab88b 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSourceSpec.scala
@@ -4,16 +4,16 @@
package akka.stream.scaladsl
+import scala.concurrent.duration._
+
import akka.actor.Status
+import akka.stream.CompletionStrategy
+import akka.stream.testkit.StreamSpec
import akka.stream.testkit.Utils.TE
import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit.scaladsl.TestSink
-import akka.stream.CompletionStrategy
-import akka.stream.testkit.StreamSpec
import akka.testkit.TestProbe
-import scala.concurrent.duration._
-
private object ActorRefBackpressureSourceSpec {
case object AckMsg
}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSinkSpec.scala
index 363313e89e..5e08a69d13 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSinkSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSinkSpec.scala
@@ -4,14 +4,14 @@
package akka.stream.scaladsl
+import scala.util.control.NoStackTrace
+
import akka.actor.{ Actor, ActorRef, Props }
import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl._
+import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.TestProbe
-import scala.util.control.NoStackTrace
-
object ActorRefSinkSpec {
case class Fw(ref: ActorRef) extends Actor {
def receive = {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSourceSpec.scala
index 8b4eb9b5d4..4fad4641c1 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSourceSpec.scala
@@ -4,16 +4,17 @@
package akka.stream.scaladsl
+import scala.concurrent.duration._
+
+import org.reactivestreams.Publisher
+
import akka.Done
import akka.actor.{ ActorRef, PoisonPill, Status }
import akka.stream.{ OverflowStrategy, _ }
-import akka.stream.testkit.Utils._
import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl._
-import org.reactivestreams.Publisher
-
-import scala.concurrent.duration._
+import akka.stream.testkit.scaladsl.StreamTestKit._
class ActorRefSourceSpec extends StreamSpec {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/AttributesSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/AttributesSpec.scala
index a50f4d892a..65d4107889 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/AttributesSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/AttributesSpec.scala
@@ -6,17 +6,18 @@ package akka.stream.scaladsl
import java.util.concurrent.{ CompletionStage, TimeUnit }
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+
+import akka.{ Done, NotUsed }
import akka.actor.ActorSystem
import akka.dispatch.Dispatchers
-import akka.{ Done, NotUsed }
-import akka.stream.Attributes._
import akka.stream._
+import akka.stream.Attributes._
import akka.stream.javadsl
import akka.stream.stage._
import akka.stream.testkit._
import akka.testkit.TestKit
-import com.github.ghik.silencer.silent
-import com.typesafe.config.ConfigFactory
object AttributesSpec {
@@ -125,7 +126,7 @@ class AttributesSpec
val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16)
- implicit val materializer = ActorMaterializer(settings)
+ implicit val materializer: ActorMaterializer = ActorMaterializer(settings)
"an attributes instance" must {
@@ -459,7 +460,7 @@ class AttributesSpec
"make the attributes on Flow.fromGraph source behave the same as the stage itself" in {
val attributes: Attributes =
javadsl.Source
- .empty[Any]
+ .empty[Any]()
.viaMat(
javadsl.Flow
.fromGraph(new AttributesFlow(Attributes.name("original-name")))
@@ -481,7 +482,7 @@ class AttributesSpec
"make the attributes on Sink.fromGraph source behave the same as the stage itself" in {
val attributes: Attributes =
javadsl.Source
- .empty[Any]
+ .empty[Any]()
.toMat(
javadsl.Sink
.fromGraph(new AttributesSink(Attributes.name("original-name")))
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala
index 774486a7fd..8725572339 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala
@@ -4,16 +4,17 @@
package akka.stream.scaladsl
+import scala.collection.immutable
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
+
import akka.NotUsed
+import akka.stream._
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.util.ByteString
-import akka.stream._
-import com.github.ghik.silencer.silent
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
-import scala.collection.immutable
@silent // tests deprecated APIs
class BidiFlowSpec extends StreamSpec {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CancellationStrategySpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CancellationStrategySpec.scala
index d652853c29..2903cd6bcb 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CancellationStrategySpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CancellationStrategySpec.scala
@@ -4,6 +4,8 @@
package akka.stream.scaladsl
+import scala.concurrent.duration._
+
import akka.NotUsed
import akka.stream.Attributes
import akka.stream.Attributes.CancellationStrategy
@@ -26,10 +28,8 @@ import akka.stream.testkit.StreamSpec
import akka.stream.testkit.TestPublisher
import akka.stream.testkit.TestSubscriber
import akka.stream.testkit.Utils.TE
-import akka.testkit.WithLogCapturing
import akka.testkit._
-
-import scala.concurrent.duration._
+import akka.testkit.WithLogCapturing
class CancellationStrategySpec extends StreamSpec("""akka.loglevel = DEBUG
akka.loggers = ["akka.testkit.SilenceAllTestEventListener"]""") with WithLogCapturing {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CollectionSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CollectionSinkSpec.scala
index b8c23c1dde..03a248c09b 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CollectionSinkSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CollectionSinkSpec.scala
@@ -4,15 +4,15 @@
package akka.stream.scaladsl
+import scala.collection.immutable
+import scala.concurrent.Await
+import scala.concurrent.Future
+
import akka.stream.AbruptTerminationException
import akka.stream.Materializer
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.TestPublisher
-import scala.collection.immutable
-import scala.concurrent.Await
-import scala.concurrent.Future
-
class CollectionSinkSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CoupledTerminationFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CoupledTerminationFlowSpec.scala
index f11c33a8cb..4e65343b27 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CoupledTerminationFlowSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CoupledTerminationFlowSpec.scala
@@ -4,23 +4,24 @@
package akka.stream.scaladsl
-import akka.stream._
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.TestSource
-import akka.testkit.TestProbe
-import akka.Done
-import akka.NotUsed
-import org.reactivestreams.Publisher
-import org.reactivestreams.Subscriber
-import org.reactivestreams.Subscription
-import org.scalatest.Assertion
-
import scala.concurrent.Future
import scala.util.Failure
import scala.util.Success
import scala.util.Try
import scala.xml.Node
+import org.reactivestreams.Publisher
+import org.reactivestreams.Subscriber
+import org.reactivestreams.Subscription
+import org.scalatest.Assertion
+
+import akka.Done
+import akka.NotUsed
+import akka.stream._
+import akka.stream.testkit._
+import akka.stream.testkit.scaladsl.TestSource
+import akka.testkit.TestProbe
+
class CoupledTerminationFlowSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") with ScriptedTest {
@@ -130,7 +131,7 @@ class CoupledTerminationFlowSpec extends StreamSpec("""
val flow = Flow.fromSinkAndSourceCoupledMat(sink, source)(Keep.right)
- val (source1, source2) = TestSource.probe[Int].viaMat(flow)(Keep.both).toMat(Sink.ignore)(Keep.left).run
+ val (source1, source2) = TestSource.probe[Int].viaMat(flow)(Keep.both).toMat(Sink.ignore)(Keep.left).run()
source1.sendComplete()
source2.expectCancellation()
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FailedSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FailedSourceSpec.scala
index 4200114dcb..ed598e2fcf 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FailedSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FailedSourceSpec.scala
@@ -4,12 +4,12 @@
package akka.stream.scaladsl
+import scala.util.control.NoStackTrace
+
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.TestSubscriber
import akka.testkit.DefaultTimeout
-import scala.util.control.NoStackTrace
-
class FailedSourceSpec extends StreamSpec with DefaultTimeout {
"The Failed Source" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowAppendSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowAppendSpec.scala
index d21e5f4db4..125a9db2ca 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowAppendSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowAppendSpec.scala
@@ -4,11 +4,12 @@
package akka.stream.scaladsl
+import org.reactivestreams.Subscriber
+import org.scalatest.matchers.should.Matchers
+
import akka.actor.ActorSystem
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.TestSubscriber
-import org.reactivestreams.Subscriber
-import org.scalatest.matchers.should.Matchers
class FlowAppendSpec extends StreamSpec with River {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowAskSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowAskSpec.scala
index 468b0877bb..f4cd330de2 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowAskSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowAskSpec.scala
@@ -6,24 +6,24 @@ package akka.stream.scaladsl
import java.util.concurrent.ThreadLocalRandom
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.reflect.ClassTag
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.PoisonPill
import akka.actor.Props
+import akka.stream.ActorAttributes
import akka.stream.ActorAttributes.supervisionStrategy
+import akka.stream.Supervision
import akka.stream.Supervision.resumingDecider
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.ActorAttributes
-import akka.stream.Supervision
import akka.testkit.TestActors
import akka.testkit.TestProbe
-import scala.concurrent.duration._
-import scala.concurrent.Await
-import scala.concurrent.Future
-import scala.reflect.ClassTag
-
object FlowAskSpec {
case class Reply(payload: Int)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchSpec.scala
index 27269a305b..d1b1f05cc1 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchSpec.scala
@@ -6,12 +6,12 @@ package akka.stream.scaladsl
import java.util.concurrent.ThreadLocalRandom
-import akka.stream.OverflowStrategy
-import akka.stream.testkit._
-
import scala.concurrent.Await
import scala.concurrent.duration._
+import akka.stream.OverflowStrategy
+import akka.stream.testkit._
+
class FlowBatchSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
akka.stream.materializer.max-input-buffer-size = 2
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchWeightedSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchWeightedSpec.scala
index f701826d6d..a03aea17cc 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchWeightedSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchWeightedSpec.scala
@@ -4,10 +4,10 @@
package akka.stream.scaladsl
-import akka.stream.testkit._
-
import scala.concurrent.duration._
+import akka.stream.testkit._
+
class FlowBatchWeightedSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
akka.stream.materializer.max-input-buffer-size = 2
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBufferSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBufferSpec.scala
index b6558e2494..bfaa9653ca 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBufferSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBufferSpec.scala
@@ -4,16 +4,16 @@
package akka.stream.scaladsl
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit.scaladsl._
-import akka.stream.BufferOverflowException
-import akka.stream.OverflowStrategy
-
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.duration._
+import akka.stream.BufferOverflowException
+import akka.stream.OverflowStrategy
+import akka.stream.testkit._
+import akka.stream.testkit.scaladsl._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+
class FlowBufferSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 1
akka.stream.materializer.max-input-buffer-size = 1
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowCompileSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowCompileSpec.scala
index 7ee329e20e..a41a7a509a 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowCompileSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowCompileSpec.scala
@@ -4,13 +4,14 @@
package akka.stream.scaladsl
-import akka.NotUsed
-import akka.stream.testkit.StreamSpec
+import scala.collection.immutable.Seq
+import scala.concurrent.Future
+
import com.github.ghik.silencer.silent
import org.reactivestreams.Publisher
-import scala.collection.immutable.Seq
-import scala.concurrent.Future
+import akka.NotUsed
+import akka.stream.testkit.StreamSpec
@silent // unused vars are used in shouldNot compile tests
class FlowCompileSpec extends StreamSpec {
@@ -51,14 +52,14 @@ class FlowCompileSpec extends StreamSpec {
val appended: Sink[Int, _] = open.to(closedSink)
"appended.run()" shouldNot compile
"appended.to(Sink.head[Int])" shouldNot compile
- intSeq.to(appended).run
+ intSeq.to(appended).run()
}
"be appended to Source" in {
val open: Flow[Int, String, _] = Flow[Int].map(_.toString)
val closedSource: Source[Int, _] = strSeq.via(Flow[String].map(_.hashCode))
val closedSource2: Source[String, _] = closedSource.via(open)
"strSeq.to(closedSource2)" shouldNot compile
- closedSource2.to(Sink.asPublisher[String](false)).run
+ closedSource2.to(Sink.asPublisher[String](false)).run()
}
}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConcatAllSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConcatAllSpec.scala
index 3eae7ce879..5cedcc73fb 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConcatAllSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConcatAllSpec.scala
@@ -4,13 +4,13 @@
package akka.stream.scaladsl
+import scala.util.control.NoStackTrace
+
import akka.NotUsed
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.util.ConstantFun
-import scala.util.control.NoStackTrace
-
class FlowConcatAllSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
akka.stream.materializer.max-input-buffer-size = 2
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConcatSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConcatSpec.scala
index cade0be9e0..2e7134f8c8 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConcatSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConcatSpec.scala
@@ -4,14 +4,15 @@
package akka.stream.scaladsl
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit.scaladsl.TestSink
-import akka.stream.testkit.{ BaseTwoStreamsSetup, TestPublisher, TestSubscriber }
+import scala.concurrent.{ Await, Promise }
+import scala.concurrent.duration._
+
import org.reactivestreams.Publisher
-import scala.concurrent.duration._
-import scala.concurrent.{ Await, Promise }
import akka.NotUsed
+import akka.stream.testkit.{ BaseTwoStreamsSetup, TestPublisher, TestSubscriber }
+import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.stream.testkit.scaladsl.TestSink
class FlowConcatSpec extends BaseTwoStreamsSetup {
@@ -195,8 +196,8 @@ class FlowConcatSpec extends BaseTwoStreamsSetup {
"work in example" in {
//#concat
- import akka.stream.scaladsl.Source
import akka.stream.scaladsl.Sink
+ import akka.stream.scaladsl.Source
val sourceA = Source(List(1, 2, 3, 4))
val sourceB = Source(List(10, 20, 30, 40))
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConflateSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConflateSpec.scala
index 9c052310bd..266812e48e 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConflateSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConflateSpec.scala
@@ -6,18 +6,18 @@ package akka.stream.scaladsl
import java.util.concurrent.ThreadLocalRandom
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import akka.stream._
import akka.stream.ActorAttributes.supervisionStrategy
import akka.stream.Attributes.inputBuffer
import akka.stream.Supervision.restartingDecider
import akka.stream.Supervision.resumingDecider
-import akka.stream._
-import akka.stream.testkit.Utils.TE
import akka.stream.testkit._
+import akka.stream.testkit.Utils.TE
import akka.testkit.TestLatch
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
class FlowConflateSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
akka.stream.materializer.max-input-buffer-size = 2
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDelaySpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDelaySpec.scala
index f6c9bb8db0..a3f72b7612 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDelaySpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDelaySpec.scala
@@ -4,24 +4,25 @@
package akka.stream.scaladsl
-import akka.Done
-import akka.stream.Attributes._
-import akka.stream.OverflowStrategies.EmitEarly
-import akka.stream._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit.scaladsl.TestSink
-import akka.stream.testkit.StreamSpec
-import akka.stream.testkit.TestPublisher
-import akka.stream.testkit.TestSubscriber
-import akka.testkit.TimingTest
-import akka.testkit.TestDuration
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
import org.scalatest.concurrent.PatienceConfiguration
import org.scalatest.time.Milliseconds
import org.scalatest.time.Span
-import scala.concurrent.Await
-import scala.concurrent.duration._
-import scala.util.control.NoStackTrace
+import akka.Done
+import akka.stream._
+import akka.stream.Attributes._
+import akka.stream.OverflowStrategies.EmitEarly
+import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.TestPublisher
+import akka.stream.testkit.TestSubscriber
+import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.stream.testkit.scaladsl.TestSink
+import akka.testkit.TestDuration
+import akka.testkit.TimingTest
class FlowDelaySpec extends StreamSpec {
@@ -172,9 +173,10 @@ class FlowDelaySpec extends StreamSpec {
}
"properly delay according to buffer size" taggedAs TimingTest in {
- import akka.pattern.pipe
import system.dispatcher
+ import akka.pattern.pipe
+
// With a buffer size of 1, delays add up
Source(1 to 5)
.delay(500.millis, DelayOverflowStrategy.backpressure)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDetacherSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDetacherSpec.scala
index c1dc0c417b..48023e1458 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDetacherSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDetacherSpec.scala
@@ -4,13 +4,13 @@
package akka.stream.scaladsl
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
class FlowDetacherSpec extends StreamSpec {
"A Detacher" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDispatcherSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDispatcherSpec.scala
index 5a36b8263f..b4c5b8266d 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDispatcherSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDispatcherSpec.scala
@@ -4,11 +4,12 @@
package akka.stream.scaladsl
+import com.github.ghik.silencer.silent
+
import akka.stream.ActorMaterializer
import akka.stream.ActorMaterializerSettings
import akka.stream.testkit.StreamSpec
import akka.testkit.TestProbe
-import com.github.ghik.silencer.silent
@silent("deprecated")
class FlowDispatcherSpec extends StreamSpec(s"my-dispatcher = $${akka.test.stream-dispatcher}") {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWhileSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWhileSpec.scala
index 2bf76cf82e..5fa8e2149a 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWhileSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWhileSpec.scala
@@ -6,8 +6,8 @@ package akka.stream.scaladsl
import akka.stream.ActorAttributes._
import akka.stream.Supervision._
-import akka.stream.testkit.Utils._
import akka.stream.testkit._
+import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWithinSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWithinSpec.scala
index c951ace068..cffd47b7f7 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWithinSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWithinSpec.scala
@@ -4,10 +4,10 @@
package akka.stream.scaladsl
-import akka.stream.testkit._
-
import scala.concurrent.duration._
+import akka.stream.testkit._
+
class FlowDropWithinSpec extends StreamSpec {
"A DropWithin" must {
@@ -17,18 +17,18 @@ class FlowDropWithinSpec extends StreamSpec {
val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
Source.fromPublisher(p).dropWithin(1.second).to(Sink.fromSubscriber(c)).run()
- val pSub = p.expectSubscription
- val cSub = c.expectSubscription
+ val pSub = p.expectSubscription()
+ val cSub = c.expectSubscription()
cSub.request(100)
- val demand1 = pSub.expectRequest
+ val demand1 = pSub.expectRequest()
(1 to demand1.toInt).foreach { _ =>
pSub.sendNext(input.next())
}
- val demand2 = pSub.expectRequest
+ val demand2 = pSub.expectRequest()
(1 to demand2.toInt).foreach { _ =>
pSub.sendNext(input.next())
}
- val demand3 = pSub.expectRequest
+ val demand3 = pSub.expectRequest()
c.expectNoMessage(1500.millis)
(1 to demand3.toInt).foreach { _ =>
pSub.sendNext(input.next())
@@ -37,7 +37,7 @@ class FlowDropWithinSpec extends StreamSpec {
c.expectNext(n)
}
pSub.sendComplete()
- c.expectComplete
+ c.expectComplete()
c.expectNoMessage(200.millis)
}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExpandSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExpandSpec.scala
index ea1ec95052..e9e1523bb2 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExpandSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExpandSpec.scala
@@ -6,14 +6,14 @@ package akka.stream.scaladsl
import java.util.concurrent.ThreadLocalRandom
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
import akka.stream.ActorAttributes
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.scaladsl.TestSource
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
class FlowExpandSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
akka.stream.materializer.max-input-buffer-size = 2
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExtrapolateSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExtrapolateSpec.scala
index 63ecd8448e..69580739d9 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExtrapolateSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExtrapolateSpec.scala
@@ -6,13 +6,13 @@ package akka.stream.scaladsl
import java.util.concurrent.ThreadLocalRandom
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.scaladsl.TestSource
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
class FlowExtrapolateSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
akka.stream.materializer.max-input-buffer-size = 2
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFilterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFilterSpec.scala
index a5d4408499..bc1db98812 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFilterSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFilterSpec.scala
@@ -6,6 +6,9 @@ package akka.stream.scaladsl
import java.util.concurrent.ThreadLocalRandom.{ current => random }
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
import akka.stream.ActorAttributes._
import akka.stream.Attributes
import akka.stream.Supervision._
@@ -14,9 +17,6 @@ import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.scaladsl.TestSource
-import scala.concurrent.duration._
-import scala.util.control.NoStackTrace
-
class FlowFilterSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") with ScriptedTest {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlatMapPrefixSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlatMapPrefixSpec.scala
index 3941e1c45f..cbc39b20da 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlatMapPrefixSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlatMapPrefixSpec.scala
@@ -4,541 +4,588 @@
package akka.stream.scaladsl
-import akka.stream.testkit.Utils.TE
-import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped
-import akka.stream.testkit.{ StreamSpec, TestPublisher, TestSubscriber }
+import akka.{ Done, NotUsed }
import akka.stream.{
AbruptStageTerminationException,
AbruptTerminationException,
+ Attributes,
Materializer,
NeverMaterializedException,
SubscriptionWithCancelException
}
-import akka.{ Done, NotUsed }
+import akka.stream.testkit.{ StreamSpec, TestPublisher, TestSubscriber }
+import akka.stream.testkit.Utils.TE
+import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped
class FlowFlatMapPrefixSpec extends StreamSpec {
def src10(i: Int = 0) = Source(i until (i + 10))
- "A PrefixAndDownstream" must {
-
- "work in the simple identity case" in assertAllStagesStopped {
- src10()
- .flatMapPrefixMat(2) { _ =>
- Flow[Int]
- }(Keep.left)
- .runWith(Sink.seq[Int])
- .futureValue should ===(2 until 10)
- }
-
- "expose mat value in the simple identity case" in assertAllStagesStopped {
- val (prefixF, suffixF) = src10()
- .flatMapPrefixMat(2) { prefix =>
- Flow[Int].mapMaterializedValue(_ => prefix)
- }(Keep.right)
- .toMat(Sink.seq)(Keep.both)
- .run
-
- prefixF.futureValue should ===(0 until 2)
- suffixF.futureValue should ===(2 until 10)
- }
-
- "work when source is exactly the required prefix" in assertAllStagesStopped {
- val (prefixF, suffixF) = src10()
- .flatMapPrefixMat(10) { prefix =>
- Flow[Int].mapMaterializedValue(_ => prefix)
- }(Keep.right)
- .toMat(Sink.seq)(Keep.both)
- .run
-
- prefixF.futureValue should ===(0 until 10)
- suffixF.futureValue should be(empty)
- }
-
- "work when source has less than the required prefix" in assertAllStagesStopped {
- val (prefixF, suffixF) = src10()
- .flatMapPrefixMat(20) { prefix =>
- Flow[Int].mapMaterializedValue(_ => prefix)
- }(Keep.right)
- .toMat(Sink.seq)(Keep.both)
- .run
-
- prefixF.futureValue should ===(0 until 10)
- suffixF.futureValue should be(empty)
- }
-
- "simple identity case when downstream completes before consuming the entire stream" in assertAllStagesStopped {
- val (prefixF, suffixF) = Source(0 until 100)
- .flatMapPrefixMat(10) { prefix =>
- Flow[Int].mapMaterializedValue(_ => prefix)
- }(Keep.right)
- .take(10)
- .toMat(Sink.seq)(Keep.both)
- .run
-
- prefixF.futureValue should ===(0 until 10)
- suffixF.futureValue should ===(10 until 20)
- }
-
- "propagate failure to create the downstream flow" in assertAllStagesStopped {
- val suffixF = Source(0 until 100)
- .flatMapPrefixMat(10) { prefix =>
- throw TE(s"I hate mondays! (${prefix.size})")
- }(Keep.right)
- .to(Sink.ignore)
- .run
-
- val ex = suffixF.failed.futureValue
- ex.getCause should not be null
- ex.getCause should ===(TE("I hate mondays! (10)"))
- }
-
- "propagate flow failures" in assertAllStagesStopped {
- val (prefixF, suffixF) = Source(0 until 100)
- .flatMapPrefixMat(10) { prefix =>
- Flow[Int].mapMaterializedValue(_ => prefix).map {
- case 15 => throw TE("don't like 15 either!")
- case n => n
- }
- }(Keep.right)
- .toMat(Sink.ignore)(Keep.both)
- .run
- prefixF.futureValue should ===(0 until 10)
- val ex = suffixF.failed.futureValue
- ex should ===(TE("don't like 15 either!"))
- }
-
- "produce multiple elements per input" in assertAllStagesStopped {
- val (prefixF, suffixF) = src10()
- .flatMapPrefixMat(7) { prefix =>
- Flow[Int].mapMaterializedValue(_ => prefix).mapConcat(n => List.fill(n - 6)(n))
- }(Keep.right)
- .toMat(Sink.seq[Int])(Keep.both)
- .run()
-
- prefixF.futureValue should ===(0 until 7)
- suffixF.futureValue should ===(7 :: 8 :: 8 :: 9 :: 9 :: 9 :: Nil)
- }
-
- "succeed when upstream produces no elements" in assertAllStagesStopped {
- val (prefixF, suffixF) = Source
- .empty[Int]
- .flatMapPrefixMat(7) { prefix =>
- Flow[Int].mapMaterializedValue(_ => prefix).mapConcat(n => List.fill(n - 6)(n))
- }(Keep.right)
- .toMat(Sink.seq[Int])(Keep.both)
- .run()
-
- prefixF.futureValue should be(empty)
- suffixF.futureValue should be(empty)
- }
-
- "apply materialized flow's semantics when upstream produces no elements" in assertAllStagesStopped {
- val (prefixF, suffixF) = Source
- .empty[Int]
- .flatMapPrefixMat(7) { prefix =>
- Flow[Int].mapMaterializedValue(_ => prefix).mapConcat(n => List.fill(n - 6)(n)).prepend(Source(100 to 101))
- }(Keep.right)
- .toMat(Sink.seq[Int])(Keep.both)
- .run()
-
- prefixF.futureValue should be(empty)
- suffixF.futureValue should ===(100 :: 101 :: Nil)
- }
-
- "handles upstream completion" in assertAllStagesStopped {
- val publisher = TestPublisher.manualProbe[Int]()
- val subscriber = TestSubscriber.manualProbe[Int]()
-
- val matValue = Source
- .fromPublisher(publisher)
- .flatMapPrefixMat(2) { prefix =>
- Flow[Int].mapMaterializedValue(_ => prefix).prepend(Source(100 to 101))
- }(Keep.right)
- .to(Sink.fromSubscriber(subscriber))
- .run()
-
- matValue.value should be(empty)
-
- val upstream = publisher.expectSubscription()
- val downstream = subscriber.expectSubscription()
-
- downstream.request(1000)
-
- upstream.expectRequest()
- //completing publisher
- upstream.sendComplete()
-
- matValue.futureValue should ===(Nil)
-
- subscriber.expectNext(100)
-
- subscriber.expectNext(101).expectComplete()
-
- }
-
- "work when materialized flow produces no downstream elements" in assertAllStagesStopped {
- val (prefixF, suffixF) = Source(0 until 100)
- .flatMapPrefixMat(4) { prefix =>
- Flow[Int].mapMaterializedValue(_ => prefix).filter(_ => false)
- }(Keep.right)
- .toMat(Sink.seq)(Keep.both)
- .run
-
- prefixF.futureValue should ===(0 until 4)
- suffixF.futureValue should be(empty)
- }
-
- "work when materialized flow does not consume upstream" in assertAllStagesStopped {
- val (prefixF, suffixF) = Source(0 until 100)
- .map { i =>
- i should be <= 4
- i
- }
- .flatMapPrefixMat(4) { prefix =>
- Flow[Int].mapMaterializedValue(_ => prefix).take(0)
- }(Keep.right)
- .toMat(Sink.seq)(Keep.both)
- .run
-
- prefixF.futureValue should ===(0 until 4)
- suffixF.futureValue should be(empty)
- }
-
- "work when materialized flow cancels upstream but keep producing" in assertAllStagesStopped {
- val (prefixF, suffixF) = src10()
- .flatMapPrefixMat(4) { prefix =>
- Flow[Int].mapMaterializedValue(_ => prefix).take(0).concat(Source(11 to 12))
- }(Keep.right)
- .toMat(Sink.seq)(Keep.both)
- .run
-
- prefixF.futureValue should ===(0 until 4)
- suffixF.futureValue should ===(11 :: 12 :: Nil)
- }
-
- "propagate materialization failure (when application of 'f' succeeds)" in assertAllStagesStopped {
- val (prefixF, suffixF) = src10()
- .flatMapPrefixMat(4) { prefix =>
- Flow[Int].mapMaterializedValue(_ => throw TE(s"boom-bada-bang (${prefix.size})"))
- }(Keep.right)
- .toMat(Sink.seq)(Keep.both)
- .run
-
- prefixF.failed.futureValue should be(a[NeverMaterializedException])
- prefixF.failed.futureValue.getCause should ===(TE("boom-bada-bang (4)"))
- suffixF.failed.futureValue should ===(TE("boom-bada-bang (4)"))
- }
-
- "succeed when materialized flow completes downstream but keep consuming elements" in assertAllStagesStopped {
- val (prefixAndTailF, suffixF) = src10()
- .flatMapPrefixMat(4) { prefix =>
- Flow[Int]
- .mapMaterializedValue(_ => prefix)
- .viaMat {
- Flow.fromSinkAndSourceMat(Sink.seq[Int], Source.empty[Int])(Keep.left)
- }(Keep.both)
- }(Keep.right)
- .toMat(Sink.seq)(Keep.both)
- .run
-
- suffixF.futureValue should be(empty)
- val (prefix, suffix) = prefixAndTailF.futureValue
- prefix should ===(0 until 4)
- suffix.futureValue should ===(4 until 10)
- }
-
- "propagate downstream cancellation via the materialized flow" in assertAllStagesStopped {
- val publisher = TestPublisher.manualProbe[Int]()
- val subscriber = TestSubscriber.manualProbe[Int]()
-
- val ((srcWatchTermF, innerMatVal), sinkMatVal) = src10()
- .watchTermination()(Keep.right)
- .flatMapPrefixMat(2) { prefix =>
- prefix should ===(0 until 2)
- Flow.fromSinkAndSource(Sink.fromSubscriber(subscriber), Source.fromPublisher(publisher))
- }(Keep.both)
- .take(1)
- .toMat(Sink.seq)(Keep.both)
- .run()
-
- val subUpstream = publisher.expectSubscription()
- val subDownstream = subscriber.expectSubscription()
-
- // inner stream was materialized
- innerMatVal.futureValue should ===(NotUsed)
-
- subUpstream.expectRequest() should be >= (1L)
- subDownstream.request(1)
- subscriber.expectNext(2)
- subUpstream.sendNext(22)
- subUpstream.expectCancellation() // because take(1)
- // this should not automatically pass the cancellation upstream of nested flow
- srcWatchTermF.isCompleted should ===(false)
- sinkMatVal.futureValue should ===(Seq(22))
-
- // the nested flow then decides to cancel, which moves upstream
- subDownstream.cancel()
- srcWatchTermF.futureValue should ===(Done)
- }
-
- "early downstream cancellation is later handed out to materialized flow" in assertAllStagesStopped {
- val publisher = TestPublisher.manualProbe[Int]()
- val subscriber = TestSubscriber.manualProbe[Int]()
-
- val (srcWatchTermF, matFlowWatchTermFF) = Source
- .fromPublisher(publisher)
- .watchTermination()(Keep.right)
- .flatMapPrefixMat(3) { prefix =>
- prefix should ===(0 until 3)
- Flow[Int].watchTermination()(Keep.right)
- }(Keep.both)
- .to(Sink.fromSubscriber(subscriber))
- .run()
- val matFlowWatchTerm = matFlowWatchTermFF.flatten
-
- matFlowWatchTerm.value should be(empty)
- srcWatchTermF.value should be(empty)
-
- val subDownstream = subscriber.expectSubscription()
- val subUpstream = publisher.expectSubscription()
- subDownstream.request(1)
- subUpstream.expectRequest() should be >= (1L)
- subUpstream.sendNext(0)
- subUpstream.sendNext(1)
- subDownstream.cancel()
-
- //subflow not materialized yet, hence mat value (future) isn't ready yet
- matFlowWatchTerm.value should be(empty)
- srcWatchTermF.value should be(empty)
-
- //this one is sent AFTER downstream cancellation
- subUpstream.sendNext(2)
-
- subUpstream.expectCancellation()
-
- matFlowWatchTerm.futureValue should ===(Done)
- srcWatchTermF.futureValue should ===(Done)
-
- }
-
- "early downstream failure is deferred until prefix completion" in assertAllStagesStopped {
- val publisher = TestPublisher.manualProbe[Int]()
- val subscriber = TestSubscriber.manualProbe[Int]()
-
- val (srcWatchTermF, matFlowWatchTermFF) = Source
- .fromPublisher(publisher)
- .watchTermination()(Keep.right)
- .flatMapPrefixMat(3) { prefix =>
- prefix should ===(0 until 3)
- Flow[Int].watchTermination()(Keep.right)
- }(Keep.both)
- .to(Sink.fromSubscriber(subscriber))
- .run()
- val matFlowWatchTerm = matFlowWatchTermFF.flatten
-
- matFlowWatchTerm.value should be(empty)
- srcWatchTermF.value should be(empty)
-
- val subDownstream = subscriber.expectSubscription()
- val subUpstream = publisher.expectSubscription()
- subDownstream.request(1)
- subUpstream.expectRequest() should be >= (1L)
- subUpstream.sendNext(0)
- subUpstream.sendNext(1)
- subDownstream.asInstanceOf[SubscriptionWithCancelException].cancel(TE("that again?!"))
-
- matFlowWatchTerm.value should be(empty)
- srcWatchTermF.value should be(empty)
-
- subUpstream.sendNext(2)
-
- matFlowWatchTerm.failed.futureValue should ===(TE("that again?!"))
- srcWatchTermF.failed.futureValue should ===(TE("that again?!"))
-
- subUpstream.expectCancellation()
- }
-
- "downstream failure is propagated via the materialized flow" in assertAllStagesStopped {
- val publisher = TestPublisher.manualProbe[Int]()
- val subscriber = TestSubscriber.manualProbe[Int]()
-
- val ((srcWatchTermF, notUsedF), suffixF) = src10()
- .watchTermination()(Keep.right)
- .flatMapPrefixMat(2) { prefix =>
- prefix should ===(0 until 2)
- Flow.fromSinkAndSourceCoupled(Sink.fromSubscriber(subscriber), Source.fromPublisher(publisher))
- }(Keep.both)
- .map {
- case 2 => 2
- case 3 => throw TE("3!?!?!?")
- case i => fail(s"unexpected value $i")
- }
- .toMat(Sink.seq)(Keep.both)
- .run()
-
- notUsedF.value should be(empty)
- suffixF.value should be(empty)
- srcWatchTermF.value should be(empty)
-
- val subUpstream = publisher.expectSubscription()
- val subDownstream = subscriber.expectSubscription()
-
- notUsedF.futureValue should ===(NotUsed)
-
- subUpstream.expectRequest() should be >= (1L)
- subDownstream.request(1)
- subscriber.expectNext(2)
- subUpstream.sendNext(2)
- subDownstream.request(1)
- subscriber.expectNext(3)
- subUpstream.sendNext(3)
- subUpstream.expectCancellation() should ===(TE("3!?!?!?"))
- subscriber.expectError(TE("3!?!?!?"))
-
- suffixF.failed.futureValue should ===(TE("3!?!?!?"))
- srcWatchTermF.failed.futureValue should ===(TE("3!?!?!?"))
- }
-
- "complete mat value with failures on abrupt termination before materializing the flow" in assertAllStagesStopped {
- val mat = Materializer(system)
- val publisher = TestPublisher.manualProbe[Int]()
-
- val flow = Source
- .fromPublisher(publisher)
- .flatMapPrefixMat(2) { prefix =>
- fail(s"unexpected prefix (length = ${prefix.size})")
- Flow[Int]
- }(Keep.right)
- .toMat(Sink.ignore)(Keep.both)
-
- val (prefixF, doneF) = flow.run()(mat)
-
- publisher.expectSubscription()
- prefixF.value should be(empty)
- doneF.value should be(empty)
-
- mat.shutdown()
-
- prefixF.failed.futureValue match {
- case _: AbruptTerminationException =>
- case ex: NeverMaterializedException =>
- ex.getCause should not be null
- ex.getCause should be(a[AbruptTerminationException])
+ for {
+ att <- List(
+ Attributes.NestedMaterializationCancellationPolicy.EagerCancellation,
+ Attributes.NestedMaterializationCancellationPolicy.PropagateToNested)
+ delayDownstreanCancellation = att.propagateToNestedMaterialization
+ attributes = Attributes(att)
+ } {
+
+ s"A PrefixAndDownstream with $att" must {
+
+ "work in the simple identity case" in assertAllStagesStopped {
+ src10()
+ .flatMapPrefixMat(2) { _ =>
+ Flow[Int]
+ }(Keep.left)
+ .withAttributes(attributes)
+ .runWith(Sink.seq[Int])
+ .futureValue should ===(2 until 10)
}
- doneF.failed.futureValue should be(a[AbruptTerminationException])
- }
- "respond to abrupt termination after flow materialization" in assertAllStagesStopped {
- val mat = Materializer(system)
- val countFF = src10()
- .flatMapPrefixMat(2) { prefix =>
- prefix should ===(0 until 2)
- Flow[Int]
- .concat(Source.repeat(3))
- .fold(0L) {
- case (acc, _) => acc + 1
+ "expose mat value in the simple identity case" in assertAllStagesStopped {
+ val (prefixF, suffixF) = src10()
+ .flatMapPrefixMat(2) { prefix =>
+ Flow[Int].mapMaterializedValue(_ => prefix)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run
+
+ prefixF.futureValue should ===(0 until 2)
+ suffixF.futureValue should ===(2 until 10)
+ }
+
+ "work when source is exactly the required prefix" in assertAllStagesStopped {
+ val (prefixF, suffixF) = src10()
+ .flatMapPrefixMat(10) { prefix =>
+ Flow[Int].mapMaterializedValue(_ => prefix)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run
+
+ prefixF.futureValue should ===(0 until 10)
+ suffixF.futureValue should be(empty)
+ }
+
+ "work when source has less than the required prefix" in assertAllStagesStopped {
+ val (prefixF, suffixF) = src10()
+ .flatMapPrefixMat(20) { prefix =>
+ Flow[Int].mapMaterializedValue(_ => prefix)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run
+
+ prefixF.futureValue should ===(0 until 10)
+ suffixF.futureValue should be(empty)
+ }
+
+ "simple identity case when downstream completes before consuming the entire stream" in assertAllStagesStopped {
+ val (prefixF, suffixF) = Source(0 until 100)
+ .flatMapPrefixMat(10) { prefix =>
+ Flow[Int].mapMaterializedValue(_ => prefix)
+ }(Keep.right)
+ .take(10)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run
+
+ prefixF.futureValue should ===(0 until 10)
+ suffixF.futureValue should ===(10 until 20)
+ }
+
+ "propagate failure to create the downstream flow" in assertAllStagesStopped {
+ val suffixF = Source(0 until 100)
+ .flatMapPrefixMat(10) { prefix =>
+ throw TE(s"I hate mondays! (${prefix.size})")
+ }(Keep.right)
+ .to(Sink.ignore)
+ .withAttributes(attributes)
+ .run
+
+ val ex = suffixF.failed.futureValue
+ ex.getCause should not be null
+ ex.getCause should ===(TE("I hate mondays! (10)"))
+ }
+
+ "propagate flow failures" in assertAllStagesStopped {
+ val (prefixF, suffixF) = Source(0 until 100)
+ .flatMapPrefixMat(10) { prefix =>
+ Flow[Int].mapMaterializedValue(_ => prefix).map {
+ case 15 => throw TE("don't like 15 either!")
+ case n => n
}
- .alsoToMat(Sink.head)(Keep.right)
- }(Keep.right)
- .to(Sink.ignore)
- .run()(mat)
- val countF = countFF.futureValue
- //at this point we know the flow was materialized, now we can stop the materializer
- mat.shutdown()
- //expect the nested flow to be terminated abruptly.
- countF.failed.futureValue should be(a[AbruptStageTerminationException])
- }
-
- "behave like via when n = 0" in assertAllStagesStopped {
- val (prefixF, suffixF) = src10()
- .flatMapPrefixMat(0) { prefix =>
- prefix should be(empty)
- Flow[Int].mapMaterializedValue(_ => prefix)
- }(Keep.right)
- .toMat(Sink.seq)(Keep.both)
- .run()
-
- prefixF.futureValue should be(empty)
- suffixF.futureValue should ===(0 until 10)
- }
-
- "behave like via when n = 0 and upstream produces no elements" in assertAllStagesStopped {
- val (prefixF, suffixF) = Source
- .empty[Int]
- .flatMapPrefixMat(0) { prefix =>
- prefix should be(empty)
- Flow[Int].mapMaterializedValue(_ => prefix)
- }(Keep.right)
- .toMat(Sink.seq)(Keep.both)
- .run()
-
- prefixF.futureValue should be(empty)
- suffixF.futureValue should be(empty)
- }
-
- "propagate errors during flow's creation when n = 0" in assertAllStagesStopped {
- val (prefixF, suffixF) = src10()
- .flatMapPrefixMat(0) { prefix =>
- prefix should be(empty)
- throw TE("not this time my friend!")
- Flow[Int].mapMaterializedValue(_ => prefix)
- }(Keep.right)
- .toMat(Sink.seq)(Keep.both)
- .run()
-
- prefixF.failed.futureValue should be(a[NeverMaterializedException])
- prefixF.failed.futureValue.getCause === (TE("not this time my friend!"))
- suffixF.failed.futureValue should ===(TE("not this time my friend!"))
- }
-
- "propagate materialization failures when n = 0" in assertAllStagesStopped {
- val (prefixF, suffixF) = src10()
- .flatMapPrefixMat(0) { prefix =>
- prefix should be(empty)
- Flow[Int].mapMaterializedValue(_ => throw TE("Bang! no materialization this time"))
- }(Keep.right)
- .toMat(Sink.seq)(Keep.both)
- .run()
-
- prefixF.failed.futureValue should be(a[NeverMaterializedException])
- prefixF.failed.futureValue.getCause === (TE("Bang! no materialization this time"))
- suffixF.failed.futureValue should ===(TE("Bang! no materialization this time"))
- }
-
- "run a detached flow" in assertAllStagesStopped {
- val publisher = TestPublisher.manualProbe[Int]()
- val subscriber = TestSubscriber.manualProbe[String]()
-
- val detachedFlow = Flow.fromSinkAndSource(Sink.cancelled[Int], Source(List("a", "b", "c"))).via {
- Flow.fromSinkAndSource(Sink.fromSubscriber(subscriber), Source.empty[Int])
+ }(Keep.right)
+ .toMat(Sink.ignore)(Keep.both)
+ .withAttributes(attributes)
+ .run
+ prefixF.futureValue should ===(0 until 10)
+ val ex = suffixF.failed.futureValue
+ ex should ===(TE("don't like 15 either!"))
}
- val fHeadOpt = Source
- .fromPublisher(publisher)
- .flatMapPrefix(2) { prefix =>
- prefix should ===(0 until 2)
- detachedFlow
+
+ "produce multiple elements per input" in assertAllStagesStopped {
+ val (prefixF, suffixF) = src10()
+ .flatMapPrefixMat(7) { prefix =>
+ Flow[Int].mapMaterializedValue(_ => prefix).mapConcat(n => List.fill(n - 6)(n))
+ }(Keep.right)
+ .toMat(Sink.seq[Int])(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ prefixF.futureValue should ===(0 until 7)
+ suffixF.futureValue should ===(7 :: 8 :: 8 :: 9 :: 9 :: 9 :: Nil)
+ }
+
+ "succeed when upstream produces no elements" in assertAllStagesStopped {
+ val (prefixF, suffixF) = Source
+ .empty[Int]
+ .flatMapPrefixMat(7) { prefix =>
+ Flow[Int].mapMaterializedValue(_ => prefix).mapConcat(n => List.fill(n - 6)(n))
+ }(Keep.right)
+ .toMat(Sink.seq[Int])(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ prefixF.futureValue should be(empty)
+ suffixF.futureValue should be(empty)
+ }
+
+ "apply materialized flow's semantics when upstream produces no elements" in assertAllStagesStopped {
+ val (prefixF, suffixF) = Source
+ .empty[Int]
+ .flatMapPrefixMat(7) { prefix =>
+ Flow[Int].mapMaterializedValue(_ => prefix).mapConcat(n => List.fill(n - 6)(n)).prepend(Source(100 to 101))
+ }(Keep.right)
+ .toMat(Sink.seq[Int])(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ prefixF.futureValue should be(empty)
+ suffixF.futureValue should ===(100 :: 101 :: Nil)
+ }
+
+ "handles upstream completion" in assertAllStagesStopped {
+ val publisher = TestPublisher.manualProbe[Int]()
+ val subscriber = TestSubscriber.manualProbe[Int]()
+
+ val matValue = Source
+ .fromPublisher(publisher)
+ .flatMapPrefixMat(2) { prefix =>
+ Flow[Int].mapMaterializedValue(_ => prefix).prepend(Source(100 to 101))
+ }(Keep.right)
+ .to(Sink.fromSubscriber(subscriber))
+ .withAttributes(attributes)
+ .run()
+
+ matValue.value should be(empty)
+
+ val upstream = publisher.expectSubscription()
+ val downstream = subscriber.expectSubscription()
+
+ downstream.request(1000)
+
+ upstream.expectRequest()
+ //completing publisher
+ upstream.sendComplete()
+
+ matValue.futureValue should ===(Nil)
+
+ subscriber.expectNext(100)
+
+ subscriber.expectNext(101).expectComplete()
+
+ }
+
+ "work when materialized flow produces no downstream elements" in assertAllStagesStopped {
+ val (prefixF, suffixF) = Source(0 until 100)
+ .flatMapPrefixMat(4) { prefix =>
+ Flow[Int].mapMaterializedValue(_ => prefix).filter(_ => false)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run
+
+ prefixF.futureValue should ===(0 until 4)
+ suffixF.futureValue should be(empty)
+ }
+
+ "work when materialized flow does not consume upstream" in assertAllStagesStopped {
+ val (prefixF, suffixF) = Source(0 until 100)
+ .map { i =>
+ i should be <= 4
+ i
+ }
+ .flatMapPrefixMat(4) { prefix =>
+ Flow[Int].mapMaterializedValue(_ => prefix).take(0)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .withAttributes(attributes)
+ .run
+
+ prefixF.futureValue should ===(0 until 4)
+ suffixF.futureValue should be(empty)
+ }
+
+ "work when materialized flow cancels upstream but keep producing" in assertAllStagesStopped {
+ val (prefixF, suffixF) = src10()
+ .flatMapPrefixMat(4) { prefix =>
+ Flow[Int].mapMaterializedValue(_ => prefix).take(0).concat(Source(11 to 12))
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run
+
+ prefixF.futureValue should ===(0 until 4)
+ suffixF.futureValue should ===(11 :: 12 :: Nil)
+ }
+
+ "propagate materialization failure (when application of 'f' succeeds)" in assertAllStagesStopped {
+ val (prefixF, suffixF) = src10()
+ .flatMapPrefixMat(4) { prefix =>
+ Flow[Int].mapMaterializedValue(_ => throw TE(s"boom-bada-bang (${prefix.size})"))
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run
+
+ prefixF.failed.futureValue should be(a[NeverMaterializedException])
+ prefixF.failed.futureValue.getCause should ===(TE("boom-bada-bang (4)"))
+ suffixF.failed.futureValue should ===(TE("boom-bada-bang (4)"))
+ }
+
+ "succeed when materialized flow completes downstream but keep consuming elements" in assertAllStagesStopped {
+ val (prefixAndTailF, suffixF) = src10()
+ .flatMapPrefixMat(4) { prefix =>
+ Flow[Int]
+ .mapMaterializedValue(_ => prefix)
+ .viaMat {
+ Flow.fromSinkAndSourceMat(Sink.seq[Int], Source.empty[Int])(Keep.left)
+ }(Keep.both)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run
+
+ suffixF.futureValue should be(empty)
+ val (prefix, suffix) = prefixAndTailF.futureValue
+ prefix should ===(0 until 4)
+ suffix.futureValue should ===(4 until 10)
+ }
+
+ "propagate downstream cancellation via the materialized flow" in assertAllStagesStopped {
+ val publisher = TestPublisher.manualProbe[Int]()
+ val subscriber = TestSubscriber.manualProbe[Int]()
+
+ val ((srcWatchTermF, innerMatVal), sinkMatVal) = src10()
+ .watchTermination()(Keep.right)
+ .flatMapPrefixMat(2) { prefix =>
+ prefix should ===(0 until 2)
+ Flow.fromSinkAndSource(Sink.fromSubscriber(subscriber), Source.fromPublisher(publisher))
+ }(Keep.both)
+ .take(1)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ val subUpstream = publisher.expectSubscription()
+ val subDownstream = subscriber.expectSubscription()
+
+ // inner stream was materialized
+ innerMatVal.futureValue should ===(NotUsed)
+
+ subUpstream.expectRequest() should be >= (1L)
+ subDownstream.request(1)
+ subscriber.expectNext(2)
+ subUpstream.sendNext(22)
+ subUpstream.expectCancellation() // because take(1)
+ // this should not automatically pass the cancellation upstream of nested flow
+ srcWatchTermF.isCompleted should ===(false)
+ sinkMatVal.futureValue should ===(Seq(22))
+
+ // the nested flow then decides to cancel, which moves upstream
+ subDownstream.cancel()
+ srcWatchTermF.futureValue should ===(Done)
+ }
+
+ "early downstream cancellation is later handed out to materialized flow" in assertAllStagesStopped {
+ val publisher = TestPublisher.manualProbe[Int]()
+ val subscriber = TestSubscriber.manualProbe[Int]()
+
+ val (srcWatchTermF, matFlowWatchTermFF) = Source
+ .fromPublisher(publisher)
+ .watchTermination()(Keep.right)
+ .flatMapPrefixMat(3) { prefix =>
+ prefix should ===(0 until 3)
+ Flow[Int].watchTermination()(Keep.right)
+ }(Keep.both)
+ .to(Sink.fromSubscriber(subscriber))
+ .withAttributes(attributes)
+ .run()
+ val matFlowWatchTerm = matFlowWatchTermFF.flatten
+
+ matFlowWatchTerm.value should be(empty)
+ srcWatchTermF.value should be(empty)
+
+ val subDownstream = subscriber.expectSubscription()
+ val subUpstream = publisher.expectSubscription()
+ subDownstream.request(1)
+ subUpstream.expectRequest() should be >= (1L)
+ subUpstream.sendNext(0)
+ subUpstream.sendNext(1)
+ subDownstream.cancel()
+
+ //subflow not materialized yet, hence mat value (future) isn't ready yet
+ matFlowWatchTerm.value should be(empty)
+
+ if (delayDownstreanCancellation) {
+ srcWatchTermF.value should be(empty)
+ //this one is sent AFTER downstream cancellation
+ subUpstream.sendNext(2)
+
+ subUpstream.expectCancellation()
+
+ matFlowWatchTerm.futureValue should ===(Done)
+ srcWatchTermF.futureValue should ===(Done)
+ } else {
+ srcWatchTermF.futureValue should ===(Done)
+ matFlowWatchTerm.failed.futureValue should be(a[NeverMaterializedException])
}
- .runWith(Sink.headOption)
+ }
- subscriber.expectNoMessage()
- val subsc = publisher.expectSubscription()
- subsc.expectRequest() should be >= 2L
- subsc.sendNext(0)
- subscriber.expectNoMessage()
- subsc.sendNext(1)
- val sinkSubscription = subscriber.expectSubscription()
- //this indicates
- fHeadOpt.futureValue should be(empty)
+ "early downstream failure is deferred until prefix completion" in assertAllStagesStopped {
+ val publisher = TestPublisher.manualProbe[Int]()
+ val subscriber = TestSubscriber.manualProbe[Int]()
- //materializef flow immediately cancels upstream
- subsc.expectCancellation()
- //at this point both ends of the 'external' fow are closed
+ val (srcWatchTermF, matFlowWatchTermFF) = Source
+ .fromPublisher(publisher)
+ .watchTermination()(Keep.right)
+ .flatMapPrefixMat(3) { prefix =>
+ prefix should ===(0 until 3)
+ Flow[Int].watchTermination()(Keep.right)
+ }(Keep.both)
+ .to(Sink.fromSubscriber(subscriber))
+ .withAttributes(attributes)
+ .run()
+ val matFlowWatchTerm = matFlowWatchTermFF.flatten
- sinkSubscription.request(10)
- subscriber.expectNext("a", "b", "c")
- subscriber.expectComplete()
+ matFlowWatchTerm.value should be(empty)
+ srcWatchTermF.value should be(empty)
+
+ val subDownstream = subscriber.expectSubscription()
+ val subUpstream = publisher.expectSubscription()
+ subDownstream.request(1)
+ subUpstream.expectRequest() should be >= (1L)
+ subUpstream.sendNext(0)
+ subUpstream.sendNext(1)
+ subDownstream.asInstanceOf[SubscriptionWithCancelException].cancel(TE("that again?!"))
+
+ if (delayDownstreanCancellation) {
+ matFlowWatchTerm.value should be(empty)
+ srcWatchTermF.value should be(empty)
+
+ subUpstream.sendNext(2)
+
+ matFlowWatchTerm.failed.futureValue should ===(TE("that again?!"))
+ srcWatchTermF.failed.futureValue should ===(TE("that again?!"))
+
+ subUpstream.expectCancellation()
+ } else {
+ subUpstream.expectCancellation()
+ srcWatchTermF.failed.futureValue should ===(TE("that again?!"))
+ matFlowWatchTerm.failed.futureValue should be(a[NeverMaterializedException])
+ matFlowWatchTerm.failed.futureValue.getCause should ===(TE("that again?!"))
+ }
+ }
+
+ "downstream failure is propagated via the materialized flow" in assertAllStagesStopped {
+ val publisher = TestPublisher.manualProbe[Int]()
+ val subscriber = TestSubscriber.manualProbe[Int]()
+
+ val ((srcWatchTermF, notUsedF), suffixF) = src10()
+ .watchTermination()(Keep.right)
+ .flatMapPrefixMat(2) { prefix =>
+ prefix should ===(0 until 2)
+ Flow.fromSinkAndSourceCoupled(Sink.fromSubscriber(subscriber), Source.fromPublisher(publisher))
+ }(Keep.both)
+ .map {
+ case 2 => 2
+ case 3 => throw TE("3!?!?!?")
+ case i => fail(s"unexpected value $i")
+ }
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ notUsedF.value should be(empty)
+ suffixF.value should be(empty)
+ srcWatchTermF.value should be(empty)
+
+ val subUpstream = publisher.expectSubscription()
+ val subDownstream = subscriber.expectSubscription()
+
+ notUsedF.futureValue should ===(NotUsed)
+
+ subUpstream.expectRequest() should be >= (1L)
+ subDownstream.request(1)
+ subscriber.expectNext(2)
+ subUpstream.sendNext(2)
+ subDownstream.request(1)
+ subscriber.expectNext(3)
+ subUpstream.sendNext(3)
+ subUpstream.expectCancellation() should ===(TE("3!?!?!?"))
+ subscriber.expectError(TE("3!?!?!?"))
+
+ suffixF.failed.futureValue should ===(TE("3!?!?!?"))
+ srcWatchTermF.failed.futureValue should ===(TE("3!?!?!?"))
+ }
+
+ "complete mat value with failures on abrupt termination before materializing the flow" in assertAllStagesStopped {
+ val mat = Materializer(system)
+ val publisher = TestPublisher.manualProbe[Int]()
+
+ val flow = Source
+ .fromPublisher(publisher)
+ .flatMapPrefixMat(2) { prefix =>
+ fail(s"unexpected prefix (length = ${prefix.size})")
+ Flow[Int]
+ }(Keep.right)
+ .toMat(Sink.ignore)(Keep.both)
+ .withAttributes(attributes)
+
+ val (prefixF, doneF) = flow.run()(mat)
+
+ publisher.expectSubscription()
+ prefixF.value should be(empty)
+ doneF.value should be(empty)
+
+ mat.shutdown()
+
+ prefixF.failed.futureValue match {
+ case _: AbruptTerminationException =>
+ case ex: NeverMaterializedException =>
+ ex.getCause should not be null
+ ex.getCause should be(a[AbruptTerminationException])
+ }
+ doneF.failed.futureValue should be(a[AbruptTerminationException])
+ }
+
+ "respond to abrupt termination after flow materialization" in assertAllStagesStopped {
+ val mat = Materializer(system)
+ val countFF = src10()
+ .flatMapPrefixMat(2) { prefix =>
+ prefix should ===(0 until 2)
+ Flow[Int]
+ .concat(Source.repeat(3))
+ .fold(0L) {
+ case (acc, _) => acc + 1
+ }
+ .alsoToMat(Sink.head)(Keep.right)
+ }(Keep.right)
+ .to(Sink.ignore)
+ .withAttributes(attributes)
+ .run()(mat)
+ val countF = countFF.futureValue
+ //at this point we know the flow was materialized, now we can stop the materializer
+ mat.shutdown()
+ //expect the nested flow to be terminated abruptly.
+ countF.failed.futureValue should be(a[AbruptStageTerminationException])
+ }
+
+ "behave like via when n = 0" in assertAllStagesStopped {
+ val (prefixF, suffixF) = src10()
+ .flatMapPrefixMat(0) { prefix =>
+ prefix should be(empty)
+ Flow[Int].mapMaterializedValue(_ => prefix)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ prefixF.futureValue should be(empty)
+ suffixF.futureValue should ===(0 until 10)
+ }
+
+ "behave like via when n = 0 and upstream produces no elements" in assertAllStagesStopped {
+ val (prefixF, suffixF) = Source
+ .empty[Int]
+ .flatMapPrefixMat(0) { prefix =>
+ prefix should be(empty)
+ Flow[Int].mapMaterializedValue(_ => prefix)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ prefixF.futureValue should be(empty)
+ suffixF.futureValue should be(empty)
+ }
+
+ "propagate errors during flow's creation when n = 0" in assertAllStagesStopped {
+ val (prefixF, suffixF) = src10()
+ .flatMapPrefixMat(0) { prefix =>
+ prefix should be(empty)
+ throw TE("not this time my friend!")
+ Flow[Int].mapMaterializedValue(_ => prefix)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ prefixF.failed.futureValue should be(a[NeverMaterializedException])
+ prefixF.failed.futureValue.getCause === (TE("not this time my friend!"))
+ suffixF.failed.futureValue should ===(TE("not this time my friend!"))
+ }
+
+ "propagate materialization failures when n = 0" in assertAllStagesStopped {
+ val (prefixF, suffixF) = src10()
+ .flatMapPrefixMat(0) { prefix =>
+ prefix should be(empty)
+ Flow[Int].mapMaterializedValue(_ => throw TE("Bang! no materialization this time"))
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ prefixF.failed.futureValue should be(a[NeverMaterializedException])
+ prefixF.failed.futureValue.getCause === (TE("Bang! no materialization this time"))
+ suffixF.failed.futureValue should ===(TE("Bang! no materialization this time"))
+ }
+
+ "run a detached flow" in assertAllStagesStopped {
+ val publisher = TestPublisher.manualProbe[Int]()
+ val subscriber = TestSubscriber.manualProbe[String]()
+
+ val detachedFlow = Flow.fromSinkAndSource(Sink.cancelled[Int], Source(List("a", "b", "c"))).via {
+ Flow.fromSinkAndSource(Sink.fromSubscriber(subscriber), Source.empty[Int])
+ }
+ val fHeadOpt = Source
+ .fromPublisher(publisher)
+ .flatMapPrefix(2) { prefix =>
+ prefix should ===(0 until 2)
+ detachedFlow
+ }
+ .withAttributes(attributes)
+ .runWith(Sink.headOption)
+
+ subscriber.expectNoMessage()
+ val subsc = publisher.expectSubscription()
+ subsc.expectRequest() should be >= 2L
+ subsc.sendNext(0)
+ subscriber.expectNoMessage()
+ subsc.sendNext(1)
+ val sinkSubscription = subscriber.expectSubscription()
+ //this indicates
+ fHeadOpt.futureValue should be(empty)
+
+ //materialize flow immediately cancels upstream
+ subsc.expectCancellation()
+ //at this point both ends of the 'external' fow are closed
+
+ sinkSubscription.request(10)
+ subscriber.expectNext("a", "b", "c")
+ subscriber.expectComplete()
+ }
}
-
}
-
}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlattenMergeSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlattenMergeSpec.scala
index 610dd89aa0..5c6c9dbcdf 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlattenMergeSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlattenMergeSpec.scala
@@ -4,6 +4,11 @@
package akka.stream.scaladsl
+import scala.concurrent._
+import scala.concurrent.duration._
+
+import org.scalatest.exceptions.TestFailedException
+
import akka.NotUsed
import akka.stream._
import akka.stream.impl.TraversalBuilder
@@ -18,16 +23,12 @@ import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.TestLatch
import akka.util.OptionVal
-import org.scalatest.exceptions.TestFailedException
-
-import scala.concurrent._
-import scala.concurrent.duration._
class FlowFlattenMergeSpec extends StreamSpec {
import system.dispatcher
def src10(i: Int) = Source(i until (i + 10))
- def blocked = Source.future(Promise[Int].future)
+ def blocked = Source.future(Promise[Int]().future)
val toSeq = Flow[Int].grouped(1000).toMat(Sink.head)(Keep.right)
val toSet = toSeq.mapMaterializedValue(_.map(_.toSet))
@@ -107,7 +108,7 @@ class FlowFlattenMergeSpec extends StreamSpec {
"cancel substreams when failing from main stream" in assertAllStagesStopped {
val p1, p2 = TestPublisher.probe[Int]()
val ex = new Exception("buh")
- val p = Promise[Source[Int, NotUsed]]
+ val p = Promise[Source[Int, NotUsed]]()
(Source(List(Source.fromPublisher(p1), Source.fromPublisher(p2))) ++ Source.future(p.future))
.flatMapMerge(5, identity)
.runWith(Sink.head)
@@ -121,7 +122,7 @@ class FlowFlattenMergeSpec extends StreamSpec {
"cancel substreams when failing from substream" in assertAllStagesStopped {
val p1, p2 = TestPublisher.probe[Int]()
val ex = new Exception("buh")
- val p = Promise[Int]
+ val p = Promise[Int]()
Source(List(Source.fromPublisher(p1), Source.fromPublisher(p2), Source.future(p.future)))
.flatMapMerge(5, identity)
.runWith(Sink.head)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFoldAsyncSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFoldAsyncSpec.scala
index 390caa5018..54048bfc53 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFoldAsyncSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFoldAsyncSpec.scala
@@ -4,25 +4,27 @@
package akka.stream.scaladsl
+import scala.concurrent.Await
+import scala.concurrent.ExecutionContextExecutor
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
+import org.scalatest.concurrent.PatienceConfiguration.Timeout
+
import akka.NotUsed
import akka.stream.ActorAttributes.supervisionStrategy
import akka.stream.Supervision.restartingDecider
import akka.stream.Supervision.resumingDecider
import akka.stream.impl.ReactiveStreamsCompliance
-import akka.stream.testkit.Utils._
import akka.stream.testkit._
+import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.LongRunningTest
-import org.scalatest.concurrent.PatienceConfiguration.Timeout
-
-import scala.concurrent.duration._
-import scala.concurrent.Await
-import scala.concurrent.Future
-import scala.util.control.NoStackTrace
class FlowFoldAsyncSpec extends StreamSpec {
- implicit def ec = system.dispatcher
+ implicit def ec: ExecutionContextExecutor = system.dispatcher
val timeout = Timeout(3.seconds)
"A FoldAsync" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFoldSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFoldSpec.scala
index 9d0f233e41..dcb91a2652 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFoldSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFoldSpec.scala
@@ -4,15 +4,15 @@
package akka.stream.scaladsl
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
import akka.NotUsed
+import akka.stream.ActorAttributes
+import akka.stream.Supervision
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.ActorAttributes
-import akka.stream.Supervision
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
class FlowFoldSpec extends StreamSpec {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowForeachSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowForeachSpec.scala
index 053f4f7995..d9b1af9eca 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowForeachSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowForeachSpec.scala
@@ -4,14 +4,14 @@
package akka.stream.scaladsl
-import akka.stream.testkit.Utils._
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
+import akka.stream.testkit._
+import akka.stream.testkit.Utils._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+
class FlowForeachSpec extends StreamSpec {
import system.dispatcher
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFromFutureSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFromFutureSpec.scala
index 5dc712eafb..6c4c0154e8 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFromFutureSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFromFutureSpec.scala
@@ -4,15 +4,16 @@
package akka.stream.scaladsl
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import com.github.ghik.silencer.silent
-
-import scala.concurrent.duration._
import scala.concurrent.Future
import scala.concurrent.Promise
+import scala.concurrent.duration._
import scala.util.control.NoStackTrace
+import com.github.ghik.silencer.silent
+
+import akka.stream.testkit._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+
@silent("deprecated") // testing deprecated API
class FlowFromFutureSpec extends StreamSpec {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFutureFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFutureFlowSpec.scala
new file mode 100644
index 0000000000..f3abc71c98
--- /dev/null
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFutureFlowSpec.scala
@@ -0,0 +1,529 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.stream.scaladsl
+
+import akka.NotUsed
+import akka.stream.SubscriptionWithCancelException.NonFailureCancellation
+import akka.stream.{ AbruptStageTerminationException, Attributes, Materializer, NeverMaterializedException }
+import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.Utils.TE
+import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped
+
+import scala.concurrent.{ Future, Promise }
+
+class FlowFutureFlowSpec extends StreamSpec {
+ def src10(i: Int = 0) = Source(i until (i + 10))
+ def src10WithFailure(i: Int = 0)(failOn: Int) = src10(i).map {
+ case `failOn` => throw TE(s"fail on $failOn")
+ case x => x
+ }
+
+ //this stage's behaviour in case of an 'early' downstream cancellation is governed by an attribute
+ //so we run all tests cases using both modes of the attributes.
+ //please notice most of the cases don't exhibit any difference in behaviour between the two modes
+ for {
+ att <- List(
+ Attributes.NestedMaterializationCancellationPolicy.EagerCancellation,
+ Attributes.NestedMaterializationCancellationPolicy.PropagateToNested)
+ delayDownstreanCancellation = att.propagateToNestedMaterialization
+ attributes = Attributes(att)
+ } {
+
+ s"a futureFlow with $att" must {
+ "work in the simple case with a completed future" in assertAllStagesStopped {
+ val (fNotUsed, fSeq) = src10()
+ .viaMat {
+ Flow.futureFlow {
+ Future.successful(Flow[Int])
+ }
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.futureValue should be(NotUsed)
+ fSeq.futureValue should equal(0 until 10)
+ }
+
+ "work in the simple case with a late future" in assertAllStagesStopped {
+ val prFlow = Promise[Flow[Int, Int, NotUsed]]
+ val (fNotUsed, fSeq) = src10()
+ .viaMat {
+ Flow.futureFlow(prFlow.future)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.value should be(empty)
+ fSeq.value should be(empty)
+
+ prFlow.success(Flow[Int])
+
+ fNotUsed.futureValue should be(NotUsed)
+ fSeq.futureValue should equal(0 until 10)
+ }
+
+ "fail properly when future is a completed failed future" in assertAllStagesStopped {
+ val (fNotUsed, fSeq) = src10()
+ .viaMat {
+ Flow.futureFlow {
+ Future.failed[Flow[Int, Int, NotUsed]](TE("damn!"))
+ }
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.failed.futureValue should be(a[NeverMaterializedException])
+ fNotUsed.failed.futureValue.getCause should equal(TE("damn!"))
+
+ fSeq.failed.futureValue should equal(TE("damn!"))
+
+ }
+
+ "fail properly when future is late completed failed future" in assertAllStagesStopped {
+ val prFlow = Promise[Flow[Int, Int, NotUsed]]
+ val (fNotUsed, fSeq) = src10()
+ .viaMat {
+ Flow.futureFlow(prFlow.future)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.value should be(empty)
+ fSeq.value should be(empty)
+
+ prFlow.failure(TE("damn!"))
+
+ fNotUsed.failed.futureValue should be(a[NeverMaterializedException])
+ fNotUsed.failed.futureValue.getCause should equal(TE("damn!"))
+
+ fSeq.failed.futureValue should equal(TE("damn!"))
+
+ }
+
+ "handle upstream failure when future is pre-completed" in assertAllStagesStopped {
+ val (fNotUsed, fSeq) = src10WithFailure()(5)
+ .viaMat {
+ Flow.futureFlow {
+ Future.successful {
+ Flow[Int].recover {
+ case TE("fail on 5") => 99
+ }
+ }
+ }
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.futureValue should be(NotUsed)
+ fSeq.futureValue should equal(List(0, 1, 2, 3, 4, 99))
+ }
+
+ "handle upstream failure when future is late-completed" in assertAllStagesStopped {
+ val prFlow = Promise[Flow[Int, Int, NotUsed]]
+ val (fNotUsed, fSeq) = src10WithFailure()(5)
+ .viaMat {
+ Flow.futureFlow(prFlow.future)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.value should be(empty)
+ fSeq.value should be(empty)
+
+ prFlow.success {
+ Flow[Int].recover {
+ case TE("fail on 5") => 99
+ }
+ }
+
+ fNotUsed.futureValue should be(NotUsed)
+ fSeq.futureValue should equal(List(0, 1, 2, 3, 4, 99))
+ }
+
+ "propagate upstream failure when future is pre-completed" in assertAllStagesStopped {
+ val (fNotUsed, fSeq) = src10WithFailure()(5)
+ .viaMat {
+ Flow.futureFlow {
+ Future.successful {
+ Flow[Int]
+ }
+ }
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.futureValue should be(NotUsed)
+ fSeq.failed.futureValue should equal(TE("fail on 5"))
+ }
+
+ "propagate upstream failure when future is late-completed" in assertAllStagesStopped {
+ val prFlow = Promise[Flow[Int, Int, NotUsed]]
+ val (fNotUsed, fSeq) = src10WithFailure()(5)
+ .viaMat {
+ Flow.futureFlow(prFlow.future)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.value should be(empty)
+ fSeq.value should be(empty)
+
+ prFlow.success {
+ Flow[Int]
+ }
+
+ fNotUsed.futureValue should be(NotUsed)
+ fSeq.failed.futureValue should equal(TE("fail on 5"))
+ }
+
+ "handle early upstream error when flow future is pre-completed" in assertAllStagesStopped {
+ val (fNotUsed, fSeq) = Source
+ .failed(TE("not today my friend"))
+ .viaMat {
+ Flow.futureFlow {
+ Future.successful {
+ Flow[Int]
+ .recover {
+ case TE("not today my friend") => 99
+ }
+ .concat(src10())
+ }
+ }
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.futureValue should be(NotUsed)
+ fSeq.futureValue should equal(99 +: (0 until 10))
+
+ }
+
+ "handle early upstream error when flow future is late-completed" in assertAllStagesStopped {
+ val prFlow = Promise[Flow[Int, Int, NotUsed]]
+ val (fNotUsed, fSeq) = Source
+ .failed(TE("not today my friend"))
+ .viaMat {
+ Flow.futureFlow(prFlow.future)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.value should be(empty)
+ fSeq.value should be(empty)
+
+ prFlow.success {
+ Flow[Int]
+ .recover {
+ case TE("not today my friend") => 99
+ }
+ .concat(src10())
+ }
+
+ fNotUsed.futureValue should be(NotUsed)
+ fSeq.futureValue should equal(99 +: (0 until 10))
+
+ }
+
+ "handle closed downstream when flow future is pre completed" in assertAllStagesStopped {
+ val (fSeq1, fSeq2) = src10()
+ .viaMat {
+ Flow.futureFlow {
+ Future.successful {
+ Flow[Int].alsoToMat(Sink.seq)(Keep.right)
+ }
+ }
+ }(Keep.right)
+ .mapMaterializedValue(_.flatten)
+ .take(0)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fSeq1.futureValue should be(empty)
+ fSeq2.futureValue should be(empty)
+
+ }
+
+ "handle closed downstream when flow future is late completed" in assertAllStagesStopped {
+ val prFlow = Promise[Flow[Int, Int, Future[collection.immutable.Seq[Int]]]]
+ val (fSeq1, fSeq2) = src10()
+ .viaMat {
+ Flow.futureFlow(prFlow.future)
+ }(Keep.right)
+ .mapMaterializedValue(_.flatten)
+ .take(0)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ if (delayDownstreanCancellation) {
+ fSeq1.value should be(empty)
+ fSeq2.value should be(empty)
+
+ prFlow.success {
+ Flow[Int].alsoToMat(Sink.seq)(Keep.right)
+ }
+
+ fSeq1.futureValue should be(empty)
+ fSeq2.futureValue should be(empty)
+ } else {
+ fSeq1.failed.futureValue should be(a[NeverMaterializedException])
+ fSeq1.failed.futureValue.getCause should be(a[NonFailureCancellation])
+ fSeq2.futureValue should be(empty)
+ }
+ }
+
+ "handle early downstream failure when flow future is pre-completed" in assertAllStagesStopped {
+ val (fSeq1, fSeq2) = src10()
+ .viaMat {
+ Flow.futureFlow {
+ Future.successful {
+ Flow[Int].alsoToMat(Sink.seq)(Keep.right)
+ }
+ }
+ }(Keep.right)
+ .mapMaterializedValue(_.flatten)
+ .prepend(Source.failed(TE("damn!")))
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fSeq1.failed.futureValue should equal(TE("damn!"))
+ fSeq2.failed.futureValue should equal(TE("damn!"))
+ }
+
+ "handle early downstream failure when flow future is late completed" in assertAllStagesStopped {
+ val prFlow = Promise[Flow[Int, Int, Future[collection.immutable.Seq[Int]]]]
+ val (fSeq1, fSeq2) = src10()
+ .viaMat {
+ Flow.futureFlow(prFlow.future)
+ }(Keep.right)
+ .mapMaterializedValue(_.flatten)
+ .prepend(Source.failed(TE("damn!")))
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ if (delayDownstreanCancellation) {
+ fSeq2.failed.futureValue should equal(TE("damn!"))
+ fSeq1.value should be(empty)
+
+ prFlow.success {
+ Flow[Int].alsoToMat(Sink.seq)(Keep.right)
+ }
+
+ fSeq1.failed.futureValue should equal(TE("damn!"))
+ } else {
+ fSeq1.failed.futureValue should be(a[NeverMaterializedException])
+ fSeq1.failed.futureValue.getCause should equal(TE("damn!"))
+ fSeq2.failed.futureValue should equal(TE("damn!"))
+ }
+ }
+
+ "handle early upstream completion when flow future is pre-completed" in assertAllStagesStopped {
+ val (fNotUsed, fSeq) = Source
+ .empty[Int]
+ .viaMat {
+ Flow.futureFlow {
+ Future.successful {
+ Flow[Int].orElse(Source.single(99))
+ }
+ }
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.futureValue should be(NotUsed)
+ fSeq.futureValue should equal(99 :: Nil)
+ }
+
+ "handle early upstream completion when flow future is late-completed" in assertAllStagesStopped {
+ val prFlow = Promise[Flow[Int, Int, NotUsed]]
+ val (fNotUsed, fSeq) = Source
+ .empty[Int]
+ .viaMat {
+ Flow.futureFlow(prFlow.future)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.value should be(empty)
+ fSeq.value should be(empty)
+
+ prFlow.success {
+ Flow[Int].orElse(Source.single(99))
+ }
+
+ fNotUsed.futureValue should be(NotUsed)
+ fSeq.futureValue should equal(99 :: Nil)
+ }
+
+ "fails properly on materialization failure with a completed future" in assertAllStagesStopped {
+ val (fNotUsed, fSeq) = src10()
+ .viaMat {
+ Flow.futureFlow {
+ Future.successful(Flow[Int].mapMaterializedValue[NotUsed](_ => throw TE("BBOM!")))
+ }
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.failed.futureValue should be(a[NeverMaterializedException])
+ fNotUsed.failed.futureValue.getCause should equal(TE("BBOM!"))
+ fSeq.failed.futureValue should equal(TE("BBOM!"))
+ }
+
+ "fails properly on materialization failure with a late future" in assertAllStagesStopped {
+ val prFlow = Promise[Flow[Int, Int, NotUsed]]
+ val (fNotUsed, fSeq) = src10()
+ .viaMat {
+ Flow.futureFlow(prFlow.future)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.value should be(empty)
+ fSeq.value should be(empty)
+
+ prFlow.success(Flow[Int].mapMaterializedValue[NotUsed](_ => throw TE("BBOM!")))
+
+ fNotUsed.failed.futureValue should be(a[NeverMaterializedException])
+ fNotUsed.failed.futureValue.getCause should equal(TE("BBOM!"))
+ fSeq.failed.futureValue should equal(TE("BBOM!"))
+ }
+
+ "propagate flow failures with a completed future" in assertAllStagesStopped {
+ val (fNotUsed, fSeq) = src10()
+ .viaMat {
+ Flow.futureFlow {
+ Future.successful {
+ Flow[Int].map {
+ case 5 => throw TE("fail on 5")
+ case x => x
+ }
+ }
+ }
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.futureValue should be(NotUsed)
+ fSeq.failed.futureValue should equal(TE("fail on 5"))
+ }
+
+ "propagate flow failures with a late future" in assertAllStagesStopped {
+ val prFlow = Promise[Flow[Int, Int, NotUsed]]
+ val (fNotUsed, fSeq) = src10()
+ .viaMat {
+ Flow.futureFlow(prFlow.future)
+ }(Keep.right)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fNotUsed.value should be(empty)
+ fSeq.value should be(empty)
+
+ prFlow.success {
+ Flow[Int].map {
+ case 5 => throw TE("fail on 5")
+ case x => x
+ }
+ }
+
+ fNotUsed.futureValue should be(NotUsed)
+ fSeq.failed.futureValue should equal(TE("fail on 5"))
+ }
+
+ "allow flow to handle downstream completion with a completed future" in assertAllStagesStopped {
+ val (fSeq1, fSeq2) = src10()
+ .viaMat {
+ Flow.futureFlow {
+ Future.successful {
+ Flow.fromSinkAndSourceMat(Sink.seq[Int], src10(10))(Keep.left)
+ }
+ }
+ }(Keep.right)
+ .take(5)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fSeq1.flatten.futureValue should be(0 until 10)
+ fSeq2.futureValue should equal(10 until 15)
+ }
+
+ "allow flow to handle downstream completion with a late future" in assertAllStagesStopped {
+ val pr = Promise[Flow[Int, Int, Future[Seq[Int]]]]
+ val (fSeq1, fSeq2) = src10()
+ .viaMat {
+ Flow.futureFlow(pr.future)
+ }(Keep.right)
+ .take(5)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()
+
+ fSeq1.value should be(empty)
+ fSeq2.value should be(empty)
+
+ pr.success {
+ Flow.fromSinkAndSourceMat(Sink.seq[Int], src10(10))(Keep.left)
+ }
+
+ fSeq1.flatten.futureValue should be(0 until 10)
+ fSeq2.futureValue should equal(10 until 15)
+ }
+
+ "abrupt termination before future completion" in assertAllStagesStopped {
+ val mat = Materializer(system)
+ val prFlow = Promise[Flow[Int, Int, Future[collection.immutable.Seq[Int]]]]
+ val (fSeq1, fSeq2) = src10()
+ .viaMat {
+ Flow.futureFlow(prFlow.future)
+ }(Keep.right)
+ .take(5)
+ .toMat(Sink.seq)(Keep.both)
+ .withAttributes(attributes)
+ .run()(mat)
+
+ fSeq1.value should be(empty)
+ fSeq2.value should be(empty)
+
+ mat.shutdown()
+
+ fSeq1.failed.futureValue should be(a[AbruptStageTerminationException])
+ fSeq2.failed.futureValue should be(a[AbruptStageTerminationException])
+ }
+ }
+ }
+
+ "NestedMaterializationCancellationPolicy" must {
+ "default to false" in assertAllStagesStopped {
+ val fl = Flow.fromMaterializer {
+ case (_, attributes) =>
+ val att = attributes.mandatoryAttribute[Attributes.NestedMaterializationCancellationPolicy]
+ att.propagateToNestedMaterialization should be(false)
+ Flow[Any]
+ }
+ Source.empty.via(fl).runWith(Sink.headOption).futureValue should be(empty)
+ }
+ }
+}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala
index 8778dc3181..de5621a710 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala
@@ -7,29 +7,30 @@ package akka.stream.scaladsl
import java.util.concurrent.ThreadLocalRandom
import java.util.concurrent.atomic.AtomicInteger
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.concurrent.Await
+import scala.concurrent.Promise
+import scala.concurrent.duration._
+
+import org.reactivestreams.Publisher
+import org.scalatest.concurrent.PatienceConfiguration.Timeout
+
+import akka.Done
+import akka.NotUsed
import akka.actor.ActorSystem
+import akka.stream._
import akka.stream.Attributes._
import akka.stream.Supervision.resumingDecider
-import akka.stream._
import akka.stream.impl.SinkModule
import akka.stream.impl.fusing.GroupBy
-import akka.stream.testkit.Utils._
import akka.stream.testkit._
+import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.scaladsl.TestSource
import akka.testkit.TestLatch
import akka.util.ByteString
-import akka.Done
-import akka.NotUsed
-import org.reactivestreams.Publisher
-import org.scalatest.concurrent.PatienceConfiguration.Timeout
-
-import scala.annotation.tailrec
-import scala.collection.mutable
-import scala.concurrent.duration._
-import scala.concurrent.Await
-import scala.concurrent.Promise
object FlowGroupBySpec {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedSpec.scala
index 7ee01fbfb1..77246c3c41 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedSpec.scala
@@ -6,11 +6,11 @@ package akka.stream.scaladsl
import java.util.concurrent.ThreadLocalRandom.{ current => random }
+import scala.collection.immutable
+
import akka.stream.testkit.ScriptedTest
import akka.stream.testkit.StreamSpec
-import scala.collection.immutable
-
class FlowGroupedSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") with ScriptedTest {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedWithinSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedWithinSpec.scala
index fd8f0f2962..77fc9db39f 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedWithinSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedWithinSpec.scala
@@ -6,15 +6,15 @@ package akka.stream.scaladsl
import java.util.concurrent.ThreadLocalRandom.{ current => random }
+import scala.collection.immutable
+import scala.concurrent.duration._
+
import akka.stream.ThrottleMode
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.TimingTest
import akka.util.ConstantFun
-import scala.collection.immutable
-import scala.concurrent.duration._
-
class FlowGroupedWithinSpec extends StreamSpec with ScriptedTest {
"A GroupedWithin" must {
@@ -24,18 +24,18 @@ class FlowGroupedWithinSpec extends StreamSpec with ScriptedTest {
val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.manualProbe[immutable.Seq[Int]]()
Source.fromPublisher(p).groupedWithin(1000, 1.second).to(Sink.fromSubscriber(c)).run()
- val pSub = p.expectSubscription
- val cSub = c.expectSubscription
+ val pSub = p.expectSubscription()
+ val cSub = c.expectSubscription()
cSub.request(100)
- val demand1 = pSub.expectRequest.toInt
+ val demand1 = pSub.expectRequest().toInt
(1 to demand1).foreach { _ =>
pSub.sendNext(input.next())
}
- val demand2 = pSub.expectRequest.toInt
+ val demand2 = pSub.expectRequest().toInt
(1 to demand2).foreach { _ =>
pSub.sendNext(input.next())
}
- val demand3 = pSub.expectRequest.toInt
+ val demand3 = pSub.expectRequest().toInt
c.expectNext((1 to (demand1 + demand2).toInt).toVector)
(1 to demand3).foreach { _ =>
pSub.sendNext(input.next())
@@ -43,22 +43,22 @@ class FlowGroupedWithinSpec extends StreamSpec with ScriptedTest {
c.expectNoMessage(300.millis)
c.expectNext(((demand1 + demand2 + 1).toInt to (demand1 + demand2 + demand3).toInt).toVector)
c.expectNoMessage(300.millis)
- pSub.expectRequest
+ pSub.expectRequest()
val last = input.next()
pSub.sendNext(last)
pSub.sendComplete()
c.expectNext(List(last))
- c.expectComplete
+ c.expectComplete()
c.expectNoMessage(200.millis)
}
"deliver buffered elements onComplete before the timeout" taggedAs TimingTest in {
val c = TestSubscriber.manualProbe[immutable.Seq[Int]]()
Source(1 to 3).groupedWithin(1000, 10.second).to(Sink.fromSubscriber(c)).run()
- val cSub = c.expectSubscription
+ val cSub = c.expectSubscription()
cSub.request(100)
c.expectNext((1 to 3).toList)
- c.expectComplete
+ c.expectComplete()
c.expectNoMessage(200.millis)
}
@@ -67,15 +67,15 @@ class FlowGroupedWithinSpec extends StreamSpec with ScriptedTest {
val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.manualProbe[immutable.Seq[Int]]()
Source.fromPublisher(p).groupedWithin(1000, 1.second).to(Sink.fromSubscriber(c)).run()
- val pSub = p.expectSubscription
- val cSub = c.expectSubscription
+ val pSub = p.expectSubscription()
+ val cSub = c.expectSubscription()
cSub.request(1)
- val demand1 = pSub.expectRequest.toInt
+ val demand1 = pSub.expectRequest().toInt
(1 to demand1).foreach { _ =>
pSub.sendNext(input.next())
}
c.expectNext((1 to demand1).toVector)
- val demand2 = pSub.expectRequest.toInt
+ val demand2 = pSub.expectRequest().toInt
(1 to demand2).foreach { _ =>
pSub.sendNext(input.next())
}
@@ -83,7 +83,7 @@ class FlowGroupedWithinSpec extends StreamSpec with ScriptedTest {
cSub.request(1)
c.expectNext(((demand1 + 1) to (demand1 + demand2)).toVector)
pSub.sendComplete()
- c.expectComplete
+ c.expectComplete()
c.expectNoMessage(100.millis)
}
@@ -91,10 +91,10 @@ class FlowGroupedWithinSpec extends StreamSpec with ScriptedTest {
val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.manualProbe[immutable.Seq[Int]]()
Source.fromPublisher(p).groupedWithin(1000, 500.millis).to(Sink.fromSubscriber(c)).run()
- val pSub = p.expectSubscription
- val cSub = c.expectSubscription
+ val pSub = p.expectSubscription()
+ val cSub = c.expectSubscription()
cSub.request(2)
- pSub.expectRequest
+ pSub.expectRequest()
c.expectNoMessage(600.millis)
pSub.sendNext(1)
pSub.sendNext(2)
@@ -104,19 +104,19 @@ class FlowGroupedWithinSpec extends StreamSpec with ScriptedTest {
cSub.request(3)
c.expectNoMessage(600.millis)
pSub.sendComplete()
- c.expectComplete
+ c.expectComplete()
}
"not emit empty group when finished while not being pushed" taggedAs TimingTest in {
val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.manualProbe[immutable.Seq[Int]]()
Source.fromPublisher(p).groupedWithin(1000, 50.millis).to(Sink.fromSubscriber(c)).run()
- val pSub = p.expectSubscription
- val cSub = c.expectSubscription
+ val pSub = p.expectSubscription()
+ val cSub = c.expectSubscription()
cSub.request(1)
- pSub.expectRequest
- pSub.sendComplete
- c.expectComplete
+ pSub.expectRequest()
+ pSub.sendComplete()
+ c.expectComplete()
}
"reset time window when max elements reached" taggedAs TimingTest in {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIdleInjectSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIdleInjectSpec.scala
index 7036dbc4c0..663c6f835d 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIdleInjectSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIdleInjectSpec.scala
@@ -4,13 +4,13 @@
package akka.stream.scaladsl
-import akka.stream.testkit.scaladsl.StreamTestKit._
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.TestPublisher
import akka.stream.testkit.TestSubscriber
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
+import akka.stream.testkit.scaladsl.StreamTestKit._
class FlowIdleInjectSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInitialDelaySpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInitialDelaySpec.scala
index 3118a8565b..b310d758f9 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInitialDelaySpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInitialDelaySpec.scala
@@ -6,13 +6,13 @@ package akka.stream.scaladsl
import java.util.concurrent.TimeoutException
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit.StreamSpec
-import akka.stream.testkit.TestSubscriber
-
import scala.concurrent.Await
import scala.concurrent.duration._
+import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.TestSubscriber
+import akka.stream.testkit.scaladsl.StreamTestKit._
+
class FlowInitialDelaySpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInterleaveSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInterleaveSpec.scala
index e12eafd628..c602efad5c 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInterleaveSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInterleaveSpec.scala
@@ -4,10 +4,11 @@
package akka.stream.scaladsl
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit._
import org.reactivestreams.Publisher
+import akka.stream.testkit._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+
class FlowInterleaveSpec extends BaseTwoStreamsSetup {
override type Outputs = Int
@@ -228,7 +229,7 @@ class FlowInterleaveSpec extends BaseTwoStreamsSetup {
.asSubscriber[Int]
.interleaveMat(Source.asSubscriber[Int], 2)((_, _))
.toMat(Sink.fromSubscriber(down))(Keep.left)
- .run
+ .run()
val downstream = down.expectSubscription()
downstream.cancel()
@@ -240,8 +241,8 @@ class FlowInterleaveSpec extends BaseTwoStreamsSetup {
"work in example" in {
//#interleave
- import akka.stream.scaladsl.Source
import akka.stream.scaladsl.Sink
+ import akka.stream.scaladsl.Source
val sourceA = Source(List(1, 2, 3, 4))
val sourceB = Source(List(10, 20, 30, 40))
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIntersperseSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIntersperseSpec.scala
index cd22a9425f..1b3227882f 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIntersperseSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIntersperseSpec.scala
@@ -4,12 +4,12 @@
package akka.stream.scaladsl
+import scala.concurrent.duration._
+
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.scaladsl.TestSource
-import scala.concurrent.duration._
-
class FlowIntersperseSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") {
@@ -55,14 +55,14 @@ class FlowIntersperseSpec extends StreamSpec("""
}
"complete the stage when the Source has been completed" in {
- val (p1, p2) = TestSource.probe[String].intersperse(",").toMat(TestSink.probe[String])(Keep.both).run
+ val (p1, p2) = TestSource.probe[String].intersperse(",").toMat(TestSink.probe[String])(Keep.both).run()
p2.request(10)
p1.sendNext("a").sendNext("b").sendComplete()
p2.expectNext("a").expectNext(",").expectNext("b").expectComplete()
}
"complete the stage when the Sink has been cancelled" in {
- val (p1, p2) = TestSource.probe[String].intersperse(",").toMat(TestSink.probe[String])(Keep.both).run
+ val (p1, p2) = TestSource.probe[String].intersperse(",").toMat(TestSink.probe[String])(Keep.both).run()
p2.request(10)
p1.sendNext("a").sendNext("b")
p2.expectNext("a").expectNext(",").cancel()
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIteratorSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIteratorSpec.scala
index 8fdd801dc2..c1ff6cbe30 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIteratorSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIteratorSpec.scala
@@ -4,13 +4,13 @@
package akka.stream.scaladsl
+import scala.collection.immutable
+import scala.concurrent.duration._
+
import akka.NotUsed
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
-import scala.collection.immutable
-import scala.concurrent.duration._
-
class FlowIteratorSpec extends AbstractFlowIteratorSpec {
override def testName = "A Flow based on an iterator producing function"
override def createSource(elements: Int): Source[Int, NotUsed] =
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala
index 1a06683879..186d4cf14b 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala
@@ -4,20 +4,21 @@
package akka.stream.scaladsl
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit.scaladsl._
-import akka.stream.FlowShape
-import akka.stream.OverflowStrategy
+import scala.collection.immutable
+
import org.scalatest.time._
-import scala.collection.immutable
+import akka.stream.FlowShape
+import akka.stream.OverflowStrategy
+import akka.stream.testkit._
+import akka.stream.testkit.scaladsl._
+import akka.stream.testkit.scaladsl.StreamTestKit._
class FlowJoinSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") {
- implicit val defaultPatience =
+ implicit val defaultPatience: PatienceConfig =
PatienceConfig(timeout = Span(2, Seconds), interval = Span(200, Millis))
"A Flow using join" must {
@@ -92,7 +93,7 @@ class FlowJoinSpec extends StreamSpec("""
val flow = Flow.fromGraph(GraphDSL.create(TestSink.probe[(String, String)]) { implicit b => sink =>
import GraphDSL.Implicits._
- val zip = b.add(Zip[String, String])
+ val zip = b.add(Zip[String, String]())
val broadcast = b.add(Broadcast[(String, String)](2))
source ~> zip.in0
zip.out ~> broadcast.in
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowKillSwitchSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowKillSwitchSpec.scala
index ab6fd0be5d..ecb7e4536e 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowKillSwitchSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowKillSwitchSpec.scala
@@ -4,16 +4,16 @@
package akka.stream.scaladsl
+import scala.concurrent.duration._
+
import akka.Done
+import akka.stream.ClosedShape
+import akka.stream.KillSwitches
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.Utils.TE
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.scaladsl.TestSource
-import akka.stream.ClosedShape
-import akka.stream.KillSwitches
-
-import scala.concurrent.duration._
class FlowKillSwitchSpec extends StreamSpec {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitSpec.scala
index fc3e8535de..35303a737e 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitSpec.scala
@@ -4,11 +4,11 @@
package akka.stream.scaladsl
+import scala.concurrent.Await
+
import akka.stream.StreamLimitReachedException
import akka.stream.testkit.StreamSpec
-import scala.concurrent.Await
-
class FlowLimitSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitWeightedSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitWeightedSpec.scala
index ca797ffc02..03dfe40a55 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitWeightedSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitWeightedSpec.scala
@@ -4,12 +4,12 @@
package akka.stream.scaladsl
+import scala.concurrent.Await
+
import akka.stream.StreamLimitReachedException
import akka.stream.testkit.StreamSpec
import akka.util.unused
-import scala.concurrent.Await
-
class FlowLimitWeightedSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala
index 2eaafbb2ca..79d61e05ec 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala
@@ -4,19 +4,19 @@
package akka.stream.scaladsl
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
import akka.NotUsed
import akka.event.{ DummyClassForStringSources, Logging }
+import akka.stream._
import akka.stream.ActorAttributes._
import akka.stream.Attributes.LogLevels
import akka.stream.Supervision._
import akka.stream.testkit.{ ScriptedTest, StreamSpec }
-import akka.stream._
import akka.testkit.TestProbe
-import scala.concurrent.duration._
-import scala.concurrent.Await
-import scala.util.control.NoStackTrace
-
class FlowLogSpec extends StreamSpec("""
akka.loglevel = DEBUG # test verifies logging
""") with ScriptedTest {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogWithMarkerSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogWithMarkerSpec.scala
index 2218ec7495..38eadb821c 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogWithMarkerSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogWithMarkerSpec.scala
@@ -4,19 +4,19 @@
package akka.stream.scaladsl
-import akka.NotUsed
-import akka.event.{ DummyClassForStringSources, LogMarker, Logging }
-import akka.stream.ActorAttributes._
-import akka.stream.Attributes.LogLevels
-import akka.stream.Supervision._
-import akka.stream._
-import akka.stream.testkit.{ ScriptedTest, StreamSpec }
-import akka.testkit.TestProbe
-
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
+import akka.NotUsed
+import akka.event.{ DummyClassForStringSources, LogMarker, Logging }
+import akka.stream._
+import akka.stream.ActorAttributes._
+import akka.stream.Attributes.LogLevels
+import akka.stream.Supervision._
+import akka.stream.testkit.{ ScriptedTest, StreamSpec }
+import akka.testkit.TestProbe
+
class FlowLogWithMarkerSpec extends StreamSpec("""
akka.loglevel = DEBUG # test verifies logging
""") with ScriptedTest {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncSpec.scala
index 7ca062ea17..1ced15f919 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncSpec.scala
@@ -4,29 +4,30 @@
package akka.stream.scaladsl
-import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.LinkedBlockingQueue
import java.util.concurrent.ThreadLocalRandom
-
-import akka.stream.ActorAttributes.supervisionStrategy
-import akka.stream.Supervision.resumingDecider
-import akka.stream.testkit.Utils._
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit.scaladsl.TestSink
-import akka.stream.ActorAttributes
-import akka.stream.Supervision
-import akka.testkit.TestLatch
-import akka.testkit.TestProbe
-import org.scalatest.concurrent.PatienceConfiguration.Timeout
+import java.util.concurrent.atomic.AtomicInteger
import scala.annotation.tailrec
-import scala.concurrent.duration._
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.Promise
+import scala.concurrent.duration._
import scala.util.control.NoStackTrace
+import org.scalatest.concurrent.PatienceConfiguration.Timeout
+
+import akka.stream.ActorAttributes
+import akka.stream.ActorAttributes.supervisionStrategy
+import akka.stream.Supervision
+import akka.stream.Supervision.resumingDecider
+import akka.stream.testkit._
+import akka.stream.testkit.Utils._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.stream.testkit.scaladsl.TestSink
+import akka.testkit.TestLatch
+import akka.testkit.TestProbe
+
class FlowMapAsyncSpec extends StreamSpec {
"A Flow with mapAsync" must {
@@ -457,7 +458,7 @@ class FlowMapAsyncSpec extends StreamSpec {
def deferred(): Future[Int] = {
if (counter.incrementAndGet() > parallelism) Future.failed(new Exception("parallelism exceeded"))
else {
- val p = Promise[Int]
+ val p = Promise[Int]()
queue.offer(p -> System.nanoTime())
p.future
}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncUnorderedSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncUnorderedSpec.scala
index 7280800e3c..5009bd1384 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncUnorderedSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncUnorderedSpec.scala
@@ -7,15 +7,6 @@ package akka.stream.scaladsl
import java.util.concurrent.LinkedBlockingQueue
import java.util.concurrent.atomic.AtomicInteger
-import akka.stream.ActorAttributes.supervisionStrategy
-import akka.stream.Supervision.resumingDecider
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit.scaladsl._
-import akka.testkit.TestLatch
-import akka.testkit.TestProbe
-import org.scalatest.concurrent.PatienceConfiguration.Timeout
-
import scala.annotation.tailrec
import scala.concurrent.Await
import scala.concurrent.Future
@@ -23,6 +14,16 @@ import scala.concurrent.Promise
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
+import org.scalatest.concurrent.PatienceConfiguration.Timeout
+
+import akka.stream.ActorAttributes.supervisionStrategy
+import akka.stream.Supervision.resumingDecider
+import akka.stream.testkit._
+import akka.stream.testkit.scaladsl._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.testkit.TestLatch
+import akka.testkit.TestProbe
+
class FlowMapAsyncUnorderedSpec extends StreamSpec {
"A Flow with mapAsyncUnordered" must {
@@ -137,7 +138,7 @@ class FlowMapAsyncUnorderedSpec extends StreamSpec {
.run()
val sub = c.expectSubscription()
sub.request(10)
- c.expectError.getMessage should be("err1")
+ c.expectError().getMessage should be("err1")
latch.countDown()
}
@@ -180,7 +181,7 @@ class FlowMapAsyncUnorderedSpec extends StreamSpec {
.run()
val sub = c.expectSubscription()
sub.request(10)
- c.expectError.getMessage should be("err2")
+ c.expectError().getMessage should be("err2")
latch.countDown()
}
@@ -339,7 +340,7 @@ class FlowMapAsyncUnorderedSpec extends StreamSpec {
def deferred(): Future[Int] = {
if (counter.incrementAndGet() > parallelism) Future.failed(new Exception("parallelism exceeded"))
else {
- val p = Promise[Int]
+ val p = Promise[Int]()
queue.offer(p -> System.nanoTime())
p.future
}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapConcatSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapConcatSpec.scala
index a18e7af2e6..ef3755b830 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapConcatSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapConcatSpec.scala
@@ -4,13 +4,13 @@
package akka.stream.scaladsl
+import scala.util.control.NoStackTrace
+
+import akka.stream.ActorAttributes
+import akka.stream.Supervision
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
-import akka.stream.ActorAttributes
-import akka.stream.Supervision
-
-import scala.util.control.NoStackTrace
class FlowMapConcatSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
@@ -30,7 +30,7 @@ class FlowMapConcatSpec extends StreamSpec("""
}
"map and concat grouping with slow downstream" in assertAllStagesStopped {
- val s = TestSubscriber.manualProbe[Int]
+ val s = TestSubscriber.manualProbe[Int]()
val input = (1 to 20).grouped(5).toList
Source(input).mapConcat(identity).map(x => { Thread.sleep(10); x }).runWith(Sink.fromSubscriber(s))
val sub = s.expectSubscription()
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapErrorSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapErrorSpec.scala
index 34458997e0..2cd370316c 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapErrorSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapErrorSpec.scala
@@ -4,12 +4,12 @@
package akka.stream.scaladsl
+import scala.util.control.NoStackTrace
+
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
-import scala.util.control.NoStackTrace
-
class FlowMapErrorSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 1
akka.stream.materializer.max-input-buffer-size = 1
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMergeSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMergeSpec.scala
index 1534c06423..ed7274658c 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMergeSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMergeSpec.scala
@@ -4,10 +4,11 @@
package akka.stream.scaladsl
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit._
import org.reactivestreams.Publisher
+import akka.stream.testkit._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+
class FlowMergeSpec extends BaseTwoStreamsSetup {
override type Outputs = Int
@@ -90,7 +91,7 @@ class FlowMergeSpec extends BaseTwoStreamsSetup {
.asSubscriber[Int]
.mergeMat(Source.asSubscriber[Int])((_, _))
.toMat(Sink.fromSubscriber(down))(Keep.left)
- .run
+ .run()
val downstream = down.expectSubscription()
downstream.cancel()
@@ -106,7 +107,11 @@ class FlowMergeSpec extends BaseTwoStreamsSetup {
val up2 = TestPublisher.probe[Int]()
val down = TestSubscriber.probe[Int]()
- Source.fromPublisher(up1).merge(Source.fromPublisher(up2), eagerComplete = true).to(Sink.fromSubscriber(down)).run
+ Source
+ .fromPublisher(up1)
+ .merge(Source.fromPublisher(up2), eagerComplete = true)
+ .to(Sink.fromSubscriber(down))
+ .run()
up1.ensureSubscription()
up2.ensureSubscription()
@@ -126,8 +131,8 @@ class FlowMergeSpec extends BaseTwoStreamsSetup {
"works in number example for merge sorted" in {
//#merge-sorted
- import akka.stream.scaladsl.Source
import akka.stream.scaladsl.Sink
+ import akka.stream.scaladsl.Source
val sourceA = Source(List(1, 3, 5, 7))
val sourceB = Source(List(2, 4, 6, 8))
@@ -144,8 +149,8 @@ class FlowMergeSpec extends BaseTwoStreamsSetup {
"works in number example for merge" in {
//#merge
- import akka.stream.scaladsl.Source
import akka.stream.scaladsl.Sink
+ import akka.stream.scaladsl.Source
val sourceA = Source(List(1, 2, 3, 4))
val sourceB = Source(List(10, 20, 30, 40))
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMonitorSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMonitorSpec.scala
index 217ff3f21f..60bf859083 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMonitorSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMonitorSpec.scala
@@ -4,6 +4,8 @@
package akka.stream.scaladsl
+import scala.concurrent.duration._
+
import akka.stream.FlowMonitorState
import akka.stream.FlowMonitorState._
import akka.stream.Materializer
@@ -11,8 +13,6 @@ import akka.stream.testkit.StreamSpec
import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.scaladsl.TestSource
-import scala.concurrent.duration._
-
class FlowMonitorSpec extends StreamSpec {
"A FlowMonitor" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOnCompleteSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOnCompleteSpec.scala
index 8162350b1c..cce41e1b95 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOnCompleteSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOnCompleteSpec.scala
@@ -4,17 +4,17 @@
package akka.stream.scaladsl
+import scala.concurrent.duration._
+import scala.util.Failure
+import scala.util.Success
+import scala.util.control.NoStackTrace
+
import akka.Done
import akka.stream.Materializer
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.TestProbe
-import scala.concurrent.duration._
-import scala.util.Failure
-import scala.util.Success
-import scala.util.control.NoStackTrace
-
class FlowOnCompleteSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") with ScriptedTest {
@@ -25,7 +25,7 @@ class FlowOnCompleteSpec extends StreamSpec("""
val onCompleteProbe = TestProbe()
val p = TestPublisher.manualProbe[Int]()
Source.fromPublisher(p).to(Sink.onComplete[Int](onCompleteProbe.ref ! _)).run()
- val proc = p.expectSubscription
+ val proc = p.expectSubscription()
proc.expectRequest()
proc.sendNext(42)
onCompleteProbe.expectNoMessage(100.millis)
@@ -37,7 +37,7 @@ class FlowOnCompleteSpec extends StreamSpec("""
val onCompleteProbe = TestProbe()
val p = TestPublisher.manualProbe[Int]()
Source.fromPublisher(p).to(Sink.onComplete[Int](onCompleteProbe.ref ! _)).run()
- val proc = p.expectSubscription
+ val proc = p.expectSubscription()
proc.expectRequest()
val ex = new RuntimeException("ex") with NoStackTrace
proc.sendError(ex)
@@ -49,7 +49,7 @@ class FlowOnCompleteSpec extends StreamSpec("""
val onCompleteProbe = TestProbe()
val p = TestPublisher.manualProbe[Int]()
Source.fromPublisher(p).to(Sink.onComplete[Int](onCompleteProbe.ref ! _)).run()
- val proc = p.expectSubscription
+ val proc = p.expectSubscription()
proc.expectRequest()
proc.sendComplete()
onCompleteProbe.expectMsg(Success(Done))
@@ -71,7 +71,7 @@ class FlowOnCompleteSpec extends StreamSpec("""
}
.runWith(foreachSink)
future.onComplete { onCompleteProbe.ref ! _ }
- val proc = p.expectSubscription
+ val proc = p.expectSubscription()
proc.expectRequest()
proc.sendNext(42)
proc.sendComplete()
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOrElseSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOrElseSpec.scala
index c805a92ba3..42f313e846 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOrElseSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOrElseSpec.scala
@@ -4,14 +4,14 @@
package akka.stream.scaladsl
-import akka.stream.testkit.Utils.TE
-import akka.stream.testkit.TestPublisher
-import akka.stream.testkit.TestSubscriber
-import akka.testkit.AkkaSpec
-
import scala.collection.immutable.Seq
import scala.concurrent.duration._
+import akka.stream.testkit.TestPublisher
+import akka.stream.testkit.TestSubscriber
+import akka.stream.testkit.Utils.TE
+import akka.testkit.AkkaSpec
+
class FlowOrElseSpec extends AkkaSpec {
"An OrElse flow" should {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala
index 3bc76cf449..f634bb2d10 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala
@@ -4,16 +4,17 @@
package akka.stream.scaladsl
-import akka.stream._
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import com.github.ghik.silencer.silent
-
import scala.collection.immutable
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
+import com.github.ghik.silencer.silent
+
+import akka.stream._
+import akka.stream.testkit._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+
class FlowPrefixAndTailSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
akka.stream.materializer.max-input-buffer-size = 2
@@ -30,7 +31,7 @@ class FlowPrefixAndTailSpec extends StreamSpec("""
val fut = Source.empty.prefixAndTail(10).runWith(futureSink)
val (prefix, tailFlow) = Await.result(fut, 3.seconds)
prefix should be(Nil)
- val tailSubscriber = TestSubscriber.manualProbe[Int]
+ val tailSubscriber = TestSubscriber.manualProbe[Int]()
tailFlow.to(Sink.fromSubscriber(tailSubscriber)).run()
tailSubscriber.expectSubscriptionAndComplete()
}
@@ -40,7 +41,7 @@ class FlowPrefixAndTailSpec extends StreamSpec("""
val fut = Source(List(1, 2, 3)).prefixAndTail(10).runWith(futureSink)
val (prefix, tailFlow) = Await.result(fut, 3.seconds)
prefix should be(List(1, 2, 3))
- val tailSubscriber = TestSubscriber.manualProbe[Int]
+ val tailSubscriber = TestSubscriber.manualProbe[Int]()
tailFlow.to(Sink.fromSubscriber(tailSubscriber)).run()
tailSubscriber.expectSubscriptionAndComplete()
}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverSpec.scala
index 3bf67dafaa..739da4e40b 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverSpec.scala
@@ -4,13 +4,13 @@
package akka.stream.scaladsl
+import scala.util.control.NoStackTrace
+
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.EventFilter
-import scala.util.control.NoStackTrace
-
class FlowRecoverSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 1
akka.stream.materializer.max-input-buffer-size = 1
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverWithSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverWithSpec.scala
index efcd5266f7..632d9c5b59 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverWithSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverWithSpec.scala
@@ -4,22 +4,23 @@
package akka.stream.scaladsl
-import akka.stream.stage.{ GraphStage, GraphStageLogic }
-import akka.stream.testkit.StreamSpec
-import akka.stream.testkit.scaladsl.TestSink
-import akka.stream._
-import akka.stream.testkit.Utils._
-import akka.stream.testkit.scaladsl.StreamTestKit._
+import scala.util.control.NoStackTrace
+
import com.github.ghik.silencer.silent
-import scala.util.control.NoStackTrace
+import akka.stream._
+import akka.stream.stage.{ GraphStage, GraphStageLogic }
+import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.Utils._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.stream.testkit.scaladsl.TestSink
@silent // tests deprecated APIs
class FlowRecoverWithSpec extends StreamSpec {
val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 1, maxSize = 1)
- implicit val materializer = ActorMaterializer(settings)
+ implicit val materializer: ActorMaterializer = ActorMaterializer(settings)
val ex = new RuntimeException("ex") with NoStackTrace
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowReduceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowReduceSpec.scala
index 6ca3db24db..a8af5afe6e 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowReduceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowReduceSpec.scala
@@ -4,14 +4,14 @@
package akka.stream.scaladsl
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import akka.stream.ActorAttributes
+import akka.stream.Supervision
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.ActorAttributes
-import akka.stream.Supervision
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
class FlowReduceSpec extends StreamSpec {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanAsyncSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanAsyncSpec.scala
index 9f4ddd2cc9..9850268d45 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanAsyncSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanAsyncSpec.scala
@@ -4,25 +4,27 @@
package akka.stream.scaladsl
-import akka.pattern
-import akka.stream.impl.ReactiveStreamsCompliance
-import akka.stream.testkit.TestSubscriber.Probe
-import akka.stream.testkit.Utils.TE
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl._
-import akka.stream.ActorAttributes
-import akka.stream.Supervision
-
import scala.collection.immutable
-import scala.concurrent.duration._
+import scala.concurrent.ExecutionContextExecutor
import scala.concurrent.Future
import scala.concurrent.Promise
+import scala.concurrent.duration._
import scala.util.Failure
+
import org.scalatest.matchers.should.Matchers
+import akka.pattern
+import akka.stream.ActorAttributes
+import akka.stream.Supervision
+import akka.stream.impl.ReactiveStreamsCompliance
+import akka.stream.testkit._
+import akka.stream.testkit.TestSubscriber.Probe
+import akka.stream.testkit.Utils.TE
+import akka.stream.testkit.scaladsl._
+
class FlowScanAsyncSpec extends StreamSpec with Matchers {
- implicit val executionContext = system.dispatcher
+ implicit val executionContext: ExecutionContextExecutor = system.dispatcher
"A ScanAsync" must {
@@ -127,7 +129,7 @@ class FlowScanAsyncSpec extends StreamSpec with Matchers {
}
"skip error values and handle stage completion after future get resolved" in {
- val promises = Promise[Int].success(1) :: Promise[Int] :: Nil
+ val promises = Promise[Int]().success(1) :: Promise[Int]() :: Nil
val (pub, sub) = whenEventualFuture(promises, 0, decider = Supervision.restartingDecider)
pub.sendNext(0)
sub.expectNext(0, 1)
@@ -138,7 +140,7 @@ class FlowScanAsyncSpec extends StreamSpec with Matchers {
}
"skip error values and handle stage completion before future get resolved" in {
- val promises = Promise[Int].success(1) :: Promise[Int] :: Nil
+ val promises = Promise[Int]().success(1) :: Promise[Int]() :: Nil
val (pub, sub) = whenEventualFuture(promises, 0, decider = Supervision.restartingDecider)
pub.sendNext(0)
sub.expectNext(0, 1)
@@ -161,7 +163,7 @@ class FlowScanAsyncSpec extends StreamSpec with Matchers {
}
"skip error values and handle stage completion after future get resolved" in {
- val promises = Promise[Int].success(1) :: Promise[Int] :: Nil
+ val promises = Promise[Int]().success(1) :: Promise[Int]() :: Nil
val (pub, sub) = whenEventualFuture(promises, 0, decider = Supervision.resumingDecider)
pub.sendNext(0)
sub.expectNext(0, 1)
@@ -172,7 +174,7 @@ class FlowScanAsyncSpec extends StreamSpec with Matchers {
}
"skip error values and handle stage completion before future get resolved" in {
- val promises = Promise[Int].success(1) :: Promise[Int] :: Nil
+ val promises = Promise[Int]().success(1) :: Promise[Int]() :: Nil
val (pub, sub) = whenEventualFuture(promises, 0, decider = Supervision.resumingDecider)
pub.sendNext(0)
sub.expectNext(0, 1)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala
index e139027d1b..249c2c87e6 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala
@@ -6,17 +6,17 @@ package akka.stream.scaladsl
import java.util.concurrent.ThreadLocalRandom.{ current => random }
+import scala.collection.immutable
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
import akka.NotUsed
+import akka.stream.ActorAttributes
+import akka.stream.Supervision
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
-import akka.stream.ActorAttributes
-import akka.stream.Supervision
-
-import scala.collection.immutable
-import scala.concurrent.Await
-import scala.concurrent.duration._
class FlowScanSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSlidingSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSlidingSpec.scala
index 6f57edd33b..0ee7942055 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSlidingSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSlidingSpec.scala
@@ -4,20 +4,21 @@
package akka.stream.scaladsl
-import akka.stream.testkit.scaladsl.StreamTestKit._
+import com.github.ghik.silencer.silent
+import org.scalacheck.Gen
+import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks
+
+import akka.pattern.pipe
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings }
import akka.stream.testkit._
-import org.scalacheck.Gen
-import akka.pattern.pipe
-import com.github.ghik.silencer.silent
-import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks
+import akka.stream.testkit.scaladsl.StreamTestKit._
@silent
class FlowSlidingSpec extends StreamSpec with ScalaCheckPropertyChecks {
import system.dispatcher
val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16)
- implicit val materializer = ActorMaterializer(settings)
+ implicit val materializer: ActorMaterializer = ActorMaterializer(settings)
"Sliding" must {
import org.scalacheck.Shrink.shrinkAny
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala
index d87e5a3cc6..8eb54e90e3 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala
@@ -6,22 +6,23 @@ package akka.stream.scaladsl
import java.util.concurrent.ThreadLocalRandom
-import akka.NotUsed
-import akka.actor._
-import akka.stream.impl._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit._
-import akka.stream._
-import akka.testkit.TestDuration
-import com.github.ghik.silencer.silent
-import com.typesafe.config.ConfigFactory
-import org.reactivestreams.{ Publisher, Subscriber }
-
import scala.collection.immutable
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
+import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+import org.reactivestreams.{ Publisher, Subscriber }
+
+import akka.NotUsed
+import akka.actor._
+import akka.stream._
+import akka.stream.impl._
+import akka.stream.testkit._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.testkit.TestDuration
+
object FlowSpec {
class Fruit extends Serializable
class Apple extends Fruit
@@ -40,7 +41,7 @@ class FlowSpec extends StreamSpec(ConfigFactory.parseString("akka.actor.debug.re
val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 2)
- implicit val materializer = ActorMaterializer(settings)
+ implicit val materializer: ActorMaterializer = ActorMaterializer(settings)
val identity: Flow[Any, Any, NotUsed] => Flow[Any, Any, NotUsed] = in => in.map(e => e)
val identity2: Flow[Any, Any, NotUsed] => Flow[Any, Any, NotUsed] = in => identity(in)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala
index d7f6306ff3..9ff96901e2 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala
@@ -4,20 +4,21 @@
package akka.stream.scaladsl
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import org.reactivestreams.Publisher
+
import akka.NotUsed
+import akka.stream._
import akka.stream.StreamSubscriptionTimeoutTerminationMode
import akka.stream.Supervision.resumingDecider
-import akka.stream._
import akka.stream.impl.SubscriptionTimeoutException
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.TestPublisher
import akka.stream.testkit.TestSubscriber
import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
-import org.reactivestreams.Publisher
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
object FlowSplitAfterSpec {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala
index a2d1767a3d..0be8c6798c 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala
@@ -4,22 +4,23 @@
package akka.stream.scaladsl
-import akka.Done
-import akka.NotUsed
-import akka.stream.Supervision.resumingDecider
-import akka.stream._
-import akka.stream.impl.SubscriptionTimeoutException
-import akka.stream.impl.fusing.Split
-import akka.stream.testkit.Utils._
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit.scaladsl.TestSink
-import org.reactivestreams.Publisher
-
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.duration._
+import org.reactivestreams.Publisher
+
+import akka.Done
+import akka.NotUsed
+import akka.stream._
+import akka.stream.Supervision.resumingDecider
+import akka.stream.impl.SubscriptionTimeoutException
+import akka.stream.impl.fusing.Split
+import akka.stream.testkit._
+import akka.stream.testkit.Utils._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.stream.testkit.scaladsl.TestSink
+
class FlowSplitWhenSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
akka.stream.materializer.max-input-buffer-size = 2
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowStatefulMapConcatSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowStatefulMapConcatSpec.scala
index 3597677467..82af09dade 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowStatefulMapConcatSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowStatefulMapConcatSpec.scala
@@ -4,12 +4,12 @@
package akka.stream.scaladsl
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.TestSink
+import scala.util.control.NoStackTrace
+
import akka.stream.ActorAttributes
import akka.stream.Supervision
-
-import scala.util.control.NoStackTrace
+import akka.stream.testkit._
+import akka.stream.testkit.scaladsl.TestSink
class FlowStatefulMapConcatSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSupervisionSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSupervisionSpec.scala
index 2a380114ce..9f181c881d 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSupervisionSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSupervisionSpec.scala
@@ -4,17 +4,17 @@
package akka.stream.scaladsl
+import scala.collection.immutable
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
import akka.NotUsed
import akka.stream.ActorAttributes
import akka.stream.Supervision
import akka.stream.impl.ReactiveStreamsCompliance
import akka.stream.testkit._
-import scala.collection.immutable
-import scala.concurrent.Await
-import scala.concurrent.duration._
-import scala.util.control.NoStackTrace
-
class FlowSupervisionSpec extends StreamSpec {
import ActorAttributes.supervisionStrategy
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeSpec.scala
index 1a15b304fd..e15c7a1fe2 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeSpec.scala
@@ -6,14 +6,14 @@ package akka.stream.scaladsl
import java.util.concurrent.ThreadLocalRandom.{ current => random }
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
import akka.stream.impl.ActorSubscriberMessage.OnComplete
import akka.stream.impl.ActorSubscriberMessage.OnNext
import akka.stream.impl.RequestMore
import akka.stream.testkit._
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
class FlowTakeSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") with ScriptedTest {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWhileSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWhileSpec.scala
index aa3b342bf5..175135e2f8 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWhileSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWhileSpec.scala
@@ -4,14 +4,14 @@
package akka.stream.scaladsl
+import scala.util.control.NoStackTrace
+
import akka.stream.ActorAttributes._
import akka.stream.Supervision._
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
-import scala.util.control.NoStackTrace
-
class FlowTakeWhileSpec extends StreamSpec {
"A TakeWhile" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWithinSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWithinSpec.scala
index 616076471d..e7d3a16fd0 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWithinSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWithinSpec.scala
@@ -4,11 +4,11 @@
package akka.stream.scaladsl
+import scala.concurrent.duration._
+
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
-import scala.concurrent.duration._
-
class FlowTakeWithinSpec extends StreamSpec {
"A TakeWithin" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowThrottleSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowThrottleSpec.scala
index 72d43cf2dc..c67a3b061a 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowThrottleSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowThrottleSpec.scala
@@ -7,19 +7,19 @@ package akka.stream.scaladsl
import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.atomic.AtomicLong
-import akka.stream.ThrottleMode.{ Enforcing, Shaping }
-import akka.stream._
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.TestSink
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.util.ByteString
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.Random
import scala.util.control.NoStackTrace
import akka.Done
+import akka.stream._
+import akka.stream.ThrottleMode.{ Enforcing, Shaping }
+import akka.stream.testkit._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.TimingTest
+import akka.util.ByteString
class FlowThrottleSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchSpec.scala
index 462072eb0e..0faf0bfc5d 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchSpec.scala
@@ -4,6 +4,9 @@
package akka.stream.scaladsl
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
import akka.actor.Actor
import akka.actor.PoisonPill
import akka.actor.Props
@@ -11,9 +14,6 @@ import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.TestActors
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
object FlowWatchSpec {
case class Reply(payload: Int)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchTerminationSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchTerminationSpec.scala
index ed83a34239..813f5b42e0 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchTerminationSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchTerminationSpec.scala
@@ -4,6 +4,9 @@
package akka.stream.scaladsl
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
import akka.Done
import akka.pattern.pipe
import akka.stream._
@@ -12,9 +15,6 @@ import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.scaladsl.TestSource
-import scala.concurrent.duration._
-import scala.util.control.NoStackTrace
-
class FlowWatchTerminationSpec extends StreamSpec {
"A WatchTermination" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWireTapSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWireTapSpec.scala
index 5000ecdf95..ba48600b4d 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWireTapSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWireTapSpec.scala
@@ -4,13 +4,13 @@
package akka.stream.scaladsl
-import akka.Done
-import akka.stream.testkit.Utils._
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-
import scala.util.control.NoStackTrace
+import akka.Done
+import akka.stream.testkit._
+import akka.stream.testkit.Utils._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+
class FlowWireTapSpec extends StreamSpec("akka.stream.materializer.debug.fuzzing-mode = off") {
import system.dispatcher
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWithContextLogSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWithContextLogSpec.scala
index 915c739554..ba272160ff 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWithContextLogSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWithContextLogSpec.scala
@@ -5,8 +5,8 @@
package akka.stream.scaladsl
import akka.event.Logging
-import akka.stream.Attributes.LogLevels
import akka.stream._
+import akka.stream.Attributes.LogLevels
import akka.stream.testkit.ScriptedTest
import akka.stream.testkit.StreamSpec
import akka.testkit.TestProbe
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWithContextSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWithContextSpec.scala
index eb8ee7638c..753f4d5b73 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWithContextSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWithContextSpec.scala
@@ -4,11 +4,11 @@
package akka.stream.scaladsl
+import scala.util.control.NoStackTrace
+
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.scaladsl.TestSink
-import scala.util.control.NoStackTrace
-
class FlowWithContextSpec extends StreamSpec {
"A FlowWithContext" must {
@@ -40,7 +40,7 @@ class FlowWithContextSpec extends StreamSpec {
.asSourceWithContext(_.offset)
.viaMat(mapMaterializedValueFlow)(Keep.both)
.toMat(TestSink.probe[(Message, Long)])(Keep.both)
- .run
+ .run()
matValue shouldBe (42 -> materializedValue)
probe.request(1).expectNext(((Message("a", 1L), 1L))).expectComplete()
}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipAllSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipAllSpec.scala
index e38ce37c8f..3cc471f016 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipAllSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipAllSpec.scala
@@ -4,9 +4,10 @@
package akka.stream.scaladsl
+import org.reactivestreams.Publisher
+
import akka.stream.testkit.{ BaseTwoStreamsSetup, TestSubscriber }
import akka.stream.testkit.scaladsl.StreamTestKit._
-import org.reactivestreams.Publisher
class FlowZipAllSpec extends BaseTwoStreamsSetup {
override type Outputs = (Int, Int)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FramingSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FramingSpec.scala
index dec8e6772d..1a8937873c 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FramingSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FramingSpec.scala
@@ -7,10 +7,15 @@ package akka.stream.scaladsl
import java.nio.ByteOrder
import java.util.concurrent.ThreadLocalRandom
+import scala.collection.immutable
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.util.Random
+
import akka.stream._
import akka.stream.scaladsl.Framing.FramingException
-import akka.stream.stage.GraphStage
import akka.stream.stage._
+import akka.stream.stage.GraphStage
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.TestPublisher
import akka.stream.testkit.TestSubscriber
@@ -18,11 +23,6 @@ import akka.util.ByteString
import akka.util.ByteStringBuilder
import akka.util.unused
-import scala.collection.immutable
-import scala.concurrent.Future
-import scala.concurrent.duration._
-import scala.util.Random
-
class FramingSpec extends StreamSpec {
class Rechunker extends GraphStage[FlowShape[ByteString, ByteString]] {
@@ -385,7 +385,7 @@ class FramingSpec extends StreamSpec {
def computeFrameSize(@unused arr: Array[Byte], @unused l: Int): Int = 8
- val bs = ByteString.newBuilder.putInt(0xFF010203).putInt(0x04050607).result
+ val bs = ByteString.newBuilder.putInt(0xFF010203).putInt(0x04050607).result()
val res =
Source
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBackedFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBackedFlowSpec.scala
index c423ef93c0..0199e053dc 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBackedFlowSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBackedFlowSpec.scala
@@ -4,12 +4,13 @@
package akka.stream.scaladsl
+import com.github.ghik.silencer.silent
+import org.reactivestreams.Subscriber
+
+import akka.stream._
import akka.stream.ActorMaterializer
import akka.stream.ActorMaterializerSettings
import akka.stream.testkit._
-import akka.stream._
-import com.github.ghik.silencer.silent
-import org.reactivestreams.Subscriber
object GraphFlowSpec {
val source1 = Source(0 to 3)
@@ -44,7 +45,7 @@ class GraphFlowSpec extends StreamSpec {
val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16)
- implicit val materializer = ActorMaterializer(settings)
+ implicit val materializer: ActorMaterializer = ActorMaterializer(settings)
def validateProbe(probe: TestSubscriber.ManualProbe[Int], requests: Int, result: Set[Int]): Unit = {
val subscription = probe.expectSubscription()
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala
index 32074205d1..aa4098b98e 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala
@@ -4,15 +4,15 @@
package akka.stream.scaladsl
-import akka.stream._
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit.scaladsl._
-
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.duration._
+import akka.stream._
+import akka.stream.testkit._
+import akka.stream.testkit.scaladsl._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+
class GraphBalanceSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala
index 02cd1cf752..bda68c9302 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala
@@ -4,16 +4,16 @@
package akka.stream.scaladsl
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
import akka.stream._
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.scaladsl.TestSource
-import scala.concurrent.duration._
-import scala.concurrent.Await
-import scala.concurrent.Future
-
class GraphBroadcastSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphConcatSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphConcatSpec.scala
index 3bcaff91d9..2fc9e100dc 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphConcatSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphConcatSpec.scala
@@ -4,7 +4,7 @@
package akka.stream.scaladsl
-import scala.concurrent.{ Promise }
+import scala.concurrent.Promise
import akka.stream._
import akka.stream.testkit._
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphDSLCompileSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphDSLCompileSpec.scala
index e4a5de307a..32f5ad1630 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphDSLCompileSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphDSLCompileSpec.scala
@@ -4,13 +4,14 @@
package akka.stream.scaladsl
-import akka.NotUsed
-import akka.stream.impl.fusing.GraphStages
-import akka.stream._
-import akka.stream.testkit._
-import akka.stream.stage._
import com.github.ghik.silencer.silent
+import akka.NotUsed
+import akka.stream._
+import akka.stream.impl.fusing.GraphStages
+import akka.stream.stage._
+import akka.stream.testkit._
+
object GraphDSLCompileSpec {
class Fruit
class Apple extends Fruit
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala
index 5117337842..145c581db7 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala
@@ -6,16 +6,16 @@ package akka.stream.scaladsl
import java.util.concurrent.atomic.AtomicInteger
-import akka.Done
-import akka.NotUsed
-import akka.stream._
-import akka.stream.testkit._
-
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.Promise
import scala.concurrent.duration._
+import akka.Done
+import akka.NotUsed
+import akka.stream._
+import akka.stream.testkit._
+
class GraphMatValueSpec extends StreamSpec {
import GraphDSL.Implicits._
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeLatestSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeLatestSpec.scala
index 49fe82df16..3bbdf79246 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeLatestSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeLatestSpec.scala
@@ -4,14 +4,14 @@
package akka.stream.scaladsl
-import akka.stream._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.TestSource
-
import scala.concurrent.Await
import scala.concurrent.duration._
+import akka.stream._
+import akka.stream.testkit._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.stream.testkit.scaladsl.TestSource
+
class GraphMergeLatestSpec extends TwoStreamsSetup {
import GraphDSL.Implicits._
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergePreferredSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergePreferredSpec.scala
index e3e9f6414d..882ebf4528 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergePreferredSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergePreferredSpec.scala
@@ -4,13 +4,14 @@
package akka.stream.scaladsl
-import akka.stream.testkit.TwoStreamsSetup
-import akka.stream._
-import com.github.ghik.silencer.silent
-
import scala.concurrent.Await
import scala.concurrent.duration._
+import com.github.ghik.silencer.silent
+
+import akka.stream._
+import akka.stream.testkit.TwoStreamsSetup
+
@silent // stream usage
class GraphMergePreferredSpec extends TwoStreamsSetup {
import GraphDSL.Implicits._
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergePrioritizedSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergePrioritizedSpec.scala
index a0f1b23f69..5c80b063b6 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergePrioritizedSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergePrioritizedSpec.scala
@@ -4,11 +4,12 @@
package akka.stream.scaladsl
+import scala.concurrent.duration._
+
import akka.NotUsed
-import akka.stream.testkit.TestSubscriber.ManualProbe
import akka.stream.{ ClosedShape, Inlet, Outlet }
import akka.stream.testkit.{ TestSubscriber, TwoStreamsSetup }
-import scala.concurrent.duration._
+import akka.stream.testkit.TestSubscriber.ManualProbe
class GraphMergePrioritizedSpec extends TwoStreamsSetup {
import GraphDSL.Implicits._
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSortedSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSortedSpec.scala
index 89504aa9fc..8527e97e9a 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSortedSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSortedSpec.scala
@@ -4,13 +4,14 @@
package akka.stream.scaladsl
-import akka.stream._
-import akka.stream.testkit.TwoStreamsSetup
import com.github.ghik.silencer.silent
import org.scalacheck.Gen
import org.scalacheck.Shrink
import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks
+import akka.stream._
+import akka.stream.testkit.TwoStreamsSetup
+
@silent // tests deprecated apis
class GraphMergeSortedSpec extends TwoStreamsSetup with ScalaCheckPropertyChecks {
@@ -24,7 +25,8 @@ class GraphMergeSortedSpec extends TwoStreamsSetup with ScalaCheckPropertyChecks
override def out: Outlet[Outputs] = merge.out
}
- implicit def noShrink[T] = Shrink[T](_ => Stream.empty) // do not shrink failures, it only destroys evidence
+ implicit def noShrink[T]: Shrink[T] =
+ Shrink[T](_ => Stream.empty) // do not shrink failures, it only destroys evidence
"MergeSorted" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSpec.scala
index 5233247118..bd1bb8f4fd 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSpec.scala
@@ -4,11 +4,10 @@
package akka.stream.scaladsl
-import akka.stream._
-
import scala.concurrent.Await
import scala.concurrent.duration._
+import akka.stream._
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala
index cb347e1958..c89b16ecf7 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala
@@ -4,15 +4,15 @@
package akka.stream.scaladsl
+import scala.collection.immutable
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
import akka.NotUsed
import akka.stream._
import akka.stream.testkit._
-import scala.collection.immutable
-import scala.concurrent.duration._
-import scala.concurrent.Await
-import scala.concurrent.Future
-
object GraphOpsIntegrationSpec {
import GraphDSL.Implicits._
@@ -42,7 +42,6 @@ object GraphOpsIntegrationSpec {
class GraphOpsIntegrationSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") {
- import GraphDSL.Implicits._
import akka.stream.scaladsl.GraphOpsIntegrationSpec._
"GraphDSLs" must {
@@ -50,6 +49,7 @@ class GraphOpsIntegrationSpec extends StreamSpec("""
"support broadcast - merge layouts" in {
val resultFuture = RunnableGraph
.fromGraph(GraphDSL.create(Sink.head[Seq[Int]]) { implicit b => (sink) =>
+ import GraphDSL.Implicits._
val bcast = b.add(Broadcast[Int](2))
val merge = b.add(Merge[Int](2))
@@ -68,6 +68,7 @@ class GraphOpsIntegrationSpec extends StreamSpec("""
val elements = 0 to 10
val out = RunnableGraph
.fromGraph(GraphDSL.create(Sink.head[Seq[Int]]) { implicit b => (sink) =>
+ import GraphDSL.Implicits._
val balance = b.add(Balance[Int](5))
val merge = b.add(Merge[Int](5))
@@ -89,6 +90,7 @@ class GraphOpsIntegrationSpec extends StreamSpec("""
val (resultFuture2, resultFuture9, resultFuture10) = RunnableGraph
.fromGraph(GraphDSL.create(seqSink, seqSink, seqSink)(Tuple3.apply) { implicit b => (sink2, sink9, sink10) =>
+ import GraphDSL.Implicits._
val b3 = b.add(Broadcast[Int](2))
val b7 = b.add(Broadcast[Int](2))
val b11 = b.add(Broadcast[Int](3))
@@ -137,6 +139,7 @@ class GraphOpsIntegrationSpec extends StreamSpec("""
val resultFuture = RunnableGraph
.fromGraph(GraphDSL.create(Sink.head[Seq[Int]]) { implicit b => (sink) =>
+ import GraphDSL.Implicits._
val bcast = b.add(Broadcast[Int](2))
val merge = b.add(Merge[Int](2))
@@ -153,10 +156,11 @@ class GraphOpsIntegrationSpec extends StreamSpec("""
"be able to run plain flow" in {
val p = Source(List(1, 2, 3)).runWith(Sink.asPublisher(false))
- val s = TestSubscriber.manualProbe[Int]
+ val s = TestSubscriber.manualProbe[Int]()
val flow = Flow[Int].map(_ * 2)
RunnableGraph
.fromGraph(GraphDSL.create() { implicit builder =>
+ import GraphDSL.Implicits._
Source.fromPublisher(p) ~> flow ~> Sink.fromSubscriber(s)
ClosedShape
})
@@ -175,6 +179,7 @@ class GraphOpsIntegrationSpec extends StreamSpec("""
val f: Future[Seq[Int]] = RunnableGraph
.fromGraph(GraphDSL.create(shuffler, shuffler, shuffler, Sink.head[Seq[Int]])((_, _, _, fut) => fut) {
implicit b => (s1, s2, s3, sink) =>
+ import GraphDSL.Implicits._
val merge = b.add(Merge[Int](2))
Source(List(1, 2, 3)) ~> s1.in1
@@ -209,6 +214,7 @@ class GraphOpsIntegrationSpec extends StreamSpec("""
val g: RunnableGraph[Seq[Future[String]]] = RunnableGraph.fromGraph(GraphDSL.create(sinks) {
implicit b => sinkList =>
+ import GraphDSL.Implicits._
val broadcast = b.add(Broadcast[String](sinkList.size))
Source(List("ax", "bx", "cx")) ~> broadcast
@@ -235,6 +241,7 @@ class GraphOpsIntegrationSpec extends StreamSpec("""
val g: RunnableGraph[Seq[Future[immutable.Seq[Int]]]] = RunnableGraph.fromGraph(GraphDSL.create(sinks) {
implicit b => sinkList =>
+ import GraphDSL.Implicits._
val broadcast = b.add(Broadcast[Int](sinkList.size))
Source(List(1, 2, 3)) ~> broadcast
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala
index 05148bc30d..b90b79d1dc 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala
@@ -4,13 +4,13 @@
package akka.stream.scaladsl
-import akka.stream.testkit.StreamSpec
-import akka.stream.ClosedShape
-import akka.stream.FlowShape
-
import scala.concurrent.Await
import scala.concurrent.duration._
+import akka.stream.ClosedShape
+import akka.stream.FlowShape
+import akka.stream.testkit.StreamSpec
+
class GraphPartialSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartitionSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartitionSpec.scala
index 9488c6913e..79583cb819 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartitionSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartitionSpec.scala
@@ -4,17 +4,17 @@
package akka.stream.scaladsl
-import akka.stream.ActorAttributes
-import akka.stream.Supervision
-import akka.stream.testkit.Utils.TE
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.ClosedShape
-import akka.stream.OverflowStrategy
-
import scala.concurrent.Await
import scala.concurrent.duration._
+import akka.stream.ActorAttributes
+import akka.stream.ClosedShape
+import akka.stream.OverflowStrategy
+import akka.stream.Supervision
+import akka.stream.testkit._
+import akka.stream.testkit.Utils.TE
+import akka.stream.testkit.scaladsl.StreamTestKit._
+
class GraphPartitionSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala
index c96cd310a9..baade9267b 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala
@@ -4,6 +4,9 @@
package akka.stream.scaladsl
+import scala.concurrent.Promise
+import scala.concurrent.duration._
+
import akka.actor.ActorRef
import akka.stream.Attributes
import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
@@ -11,14 +14,11 @@ import akka.stream.stage.AsyncCallback
import akka.stream.stage.InHandler
import akka.stream.stage.OutHandler
import akka.stream.stage.TimerGraphStageLogic
-import akka.stream.testkit.Utils._
import akka.stream.testkit._
+import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.TestDuration
-import scala.concurrent.Promise
-import scala.concurrent.duration._
-
object GraphStageTimersSpec {
case object TestSingleTimer
case object TestSingleTimerResubmit
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala
index 884ca4c99a..39be569aa6 100755
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala
@@ -4,12 +4,12 @@
package akka.stream.scaladsl
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
+import scala.concurrent.duration._
+
import akka.stream.ClosedShape
import akka.stream.OverflowStrategy
-
-import scala.concurrent.duration._
+import akka.stream.testkit._
+import akka.stream.testkit.scaladsl.StreamTestKit._
class GraphUnzipSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala
index de2b5d2418..feed781481 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala
@@ -4,21 +4,22 @@
package akka.stream.scaladsl
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
+import org.reactivestreams.Publisher
+
+import akka.Done
+import akka.NotUsed
import akka.stream._
+import akka.stream.testkit._
import akka.stream.testkit.TestSubscriber.Probe
import akka.stream.testkit.Utils.TE
-import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.EventFilter
import akka.testkit.TestProbe
import akka.util.unused
-import akka.Done
-import akka.NotUsed
-import org.reactivestreams.Publisher
-
-import scala.concurrent.Future
-import scala.concurrent.duration._
-import scala.util.control.NoStackTrace
class GraphUnzipWithSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
@@ -81,7 +82,7 @@ class GraphUnzipWithSpec extends StreamSpec("""
"UnzipWith" must {
"work with immediately completed publisher" in assertAllStagesStopped {
- val subscribers = setup(TestPublisher.empty[Int])
+ val subscribers = setup(TestPublisher.empty[Int]())
validateSubscriptionAndComplete(subscribers)
}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipLatestSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipLatestSpec.scala
index d87412f297..c115ba7dd7 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipLatestSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipLatestSpec.scala
@@ -4,20 +4,21 @@
package akka.stream.scaladsl
-import akka.stream.ClosedShape
-import akka.stream.testkit.TestPublisher.Probe
-import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped
-import akka.stream.testkit.scaladsl.TestSink
-import akka.stream.testkit.scaladsl.TestSource
-import akka.stream.testkit.StreamSpec
-import akka.stream.testkit.TestPublisher
-import akka.stream.testkit.TestSubscriber
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
import org.scalacheck.Gen
import org.scalatest.concurrent.ScalaFutures
import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks
-import scala.concurrent.duration._
-import scala.language.postfixOps
+import akka.stream.ClosedShape
+import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.TestPublisher
+import akka.stream.testkit.TestPublisher.Probe
+import akka.stream.testkit.TestSubscriber
+import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped
+import akka.stream.testkit.scaladsl.TestSink
+import akka.stream.testkit.scaladsl.TestSource
object GraphZipLatestSpec {
val someString = "someString"
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipLatestWithSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipLatestWithSpec.scala
index fbfe300f27..a92bc4fd37 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipLatestWithSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipLatestWithSpec.scala
@@ -4,13 +4,14 @@
package akka.stream.scaladsl
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+import org.reactivestreams.Publisher
+
import akka.stream._
import akka.stream.testkit._
import akka.testkit.EventFilter
-import org.reactivestreams.Publisher
-
-import scala.concurrent.duration._
-import scala.language.postfixOps
class GraphZipLatestWithSpec extends TwoStreamsSetup {
import GraphDSL.Implicits._
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipNSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipNSpec.scala
index 79e7ba4da8..c39edaad89 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipNSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipNSpec.scala
@@ -4,12 +4,12 @@
package akka.stream.scaladsl
+import scala.collection.immutable
+import scala.concurrent.duration._
+
+import akka.stream._
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream._
-
-import scala.concurrent.duration._
-import scala.collection.immutable
class GraphZipNSpec extends TwoStreamsSetup {
import GraphDSL.Implicits._
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipSpec.scala
index aa82af7f17..9c9ed08b13 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipSpec.scala
@@ -4,13 +4,13 @@
package akka.stream.scaladsl
-import akka.stream.testkit._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream._
-
import scala.concurrent.Await
import scala.concurrent.duration._
+import akka.stream._
+import akka.stream.testkit._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+
class GraphZipSpec extends TwoStreamsSetup {
import GraphDSL.Implicits._
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithNSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithNSpec.scala
index a2dbd3be4c..7d2b222004 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithNSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithNSpec.scala
@@ -4,11 +4,12 @@
package akka.stream.scaladsl
-import akka.stream.testkit._
-import scala.concurrent.duration._
-import akka.stream._
-import akka.testkit.EventFilter
import scala.collection.immutable
+import scala.concurrent.duration._
+
+import akka.stream._
+import akka.stream.testkit._
+import akka.testkit.EventFilter
class GraphZipWithNSpec extends TwoStreamsSetup {
import GraphDSL.Implicits._
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithSpec.scala
index a78082db2f..f0a017a7e5 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithSpec.scala
@@ -4,9 +4,10 @@
package akka.stream.scaladsl
-import akka.stream.testkit._
import scala.concurrent.duration._
+
import akka.stream._
+import akka.stream.testkit._
import akka.testkit.EventFilter
class GraphZipWithSpec extends TwoStreamsSetup {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HeadSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HeadSinkSpec.scala
index 66f379cbe4..925d980a7c 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HeadSinkSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HeadSinkSpec.scala
@@ -4,15 +4,15 @@
package akka.stream.scaladsl
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
import akka.stream.AbruptTerminationException
import akka.stream.Materializer
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
-import scala.concurrent.Await
-import scala.concurrent.Future
-import scala.concurrent.duration._
-
class HeadSinkSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") with ScriptedTest {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HubSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HubSpec.scala
index 33789df10e..e739025530 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HubSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HubSpec.scala
@@ -4,21 +4,21 @@
package akka.stream.scaladsl
+import scala.collection.immutable
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
+import akka.stream.KillSwitches
+import akka.stream.ThrottleMode
+import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.TestPublisher
+import akka.stream.testkit.TestSubscriber
import akka.stream.testkit.Utils.TE
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.scaladsl.TestSource
-import akka.stream.testkit.StreamSpec
-import akka.stream.testkit.TestPublisher
-import akka.stream.testkit.TestSubscriber
-import akka.stream.KillSwitches
-import akka.stream.ThrottleMode
import akka.testkit.EventFilter
-import scala.collection.immutable
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
class HubSpec extends StreamSpec {
"MergeHub" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/JsonFramingSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/JsonFramingSpec.scala
index df91326d09..6e2aced1eb 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/JsonFramingSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/JsonFramingSpec.scala
@@ -4,18 +4,18 @@
package akka.stream.scaladsl
-import akka.stream.impl.JsonObjectParser
-import akka.stream.scaladsl.Framing.FramingException
-import akka.stream.testkit.scaladsl.TestSink
-import akka.stream.testkit.TestPublisher
-import akka.stream.testkit.TestSubscriber
-import akka.testkit.AkkaSpec
-import akka.util.ByteString
-
import scala.collection.immutable.Seq
import scala.concurrent.Await
import scala.concurrent.duration._
+import akka.stream.impl.JsonObjectParser
+import akka.stream.scaladsl.Framing.FramingException
+import akka.stream.testkit.TestPublisher
+import akka.stream.testkit.TestSubscriber
+import akka.stream.testkit.scaladsl.TestSink
+import akka.testkit.AkkaSpec
+import akka.util.ByteString
+
class JsonFramingSpec extends AkkaSpec {
"collecting multiple json" should {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LastSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LastSinkSpec.scala
index a614a0a8c3..82936f2943 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LastSinkSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LastSinkSpec.scala
@@ -4,16 +4,17 @@
package akka.stream.scaladsl
+import scala.concurrent.Await
+import scala.concurrent.ExecutionContextExecutor
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
-import scala.concurrent.duration._
-import scala.concurrent.Await
-import scala.concurrent.Future
-
class LastSinkSpec extends StreamSpec with ScriptedTest {
- implicit val ec = system.dispatcher
+ implicit val ec: ExecutionContextExecutor = system.dispatcher
"A Flow with Sink.last" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazilyAsyncSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazilyAsyncSpec.scala
index 2435baeeac..757e6605be 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazilyAsyncSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazilyAsyncSpec.scala
@@ -6,15 +6,16 @@ package akka.stream.scaladsl
import java.util.concurrent.atomic.AtomicBoolean
-import akka.Done
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit.StreamSpec
-import akka.stream.testkit.TestSubscriber
-import akka.testkit.DefaultTimeout
+import scala.concurrent.Future
+
import com.github.ghik.silencer.silent
import org.scalatest.concurrent.ScalaFutures
-import scala.concurrent.Future
+import akka.Done
+import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.TestSubscriber
+import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.testkit.DefaultTimeout
@silent("deprecated") // tests deprecated methods
class LazilyAsyncSpec extends StreamSpec with DefaultTimeout with ScalaFutures {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazyFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazyFlowSpec.scala
index cfb98626f5..e807f54ab1 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazyFlowSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazyFlowSpec.scala
@@ -4,7 +4,12 @@
package akka.stream.scaladsl
-import akka.NotUsed
+import scala.collection.immutable
+import scala.concurrent.Future
+import scala.concurrent.Promise
+import scala.concurrent.duration._
+import com.github.ghik.silencer.silent
+import akka.{ Done, NotUsed }
import akka.stream.AbruptStageTerminationException
import akka.stream.Materializer
import akka.stream.NeverMaterializedException
@@ -15,12 +20,6 @@ import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.scaladsl.TestSource
import akka.testkit.TestProbe
-import com.github.ghik.silencer.silent
-
-import scala.collection.immutable
-import scala.concurrent.Future
-import scala.concurrent.Promise
-import scala.concurrent.duration._
@silent("deprecated") // tests deprecated API as well
class LazyFlowSpec extends StreamSpec("""
@@ -126,7 +125,8 @@ class LazyFlowSpec extends StreamSpec("""
val deferredMatVal = result._1
val list = result._2
list.failed.futureValue shouldBe a[TE]
- deferredMatVal.failed.futureValue shouldBe a[TE]
+ deferredMatVal.failed.futureValue shouldBe a[NeverMaterializedException]
+ deferredMatVal.failed.futureValue.getCause shouldBe a[TE]
}
"fail the flow when the future is initially failed" in assertAllStagesStopped {
@@ -139,7 +139,8 @@ class LazyFlowSpec extends StreamSpec("""
val deferredMatVal = result._1
val list = result._2
list.failed.futureValue shouldBe a[TE]
- deferredMatVal.failed.futureValue shouldBe a[TE]
+ deferredMatVal.failed.futureValue shouldBe a[NeverMaterializedException]
+ deferredMatVal.failed.futureValue.getCause shouldBe a[TE]
}
"fail the flow when the future is failed after the fact" in assertAllStagesStopped {
@@ -155,7 +156,28 @@ class LazyFlowSpec extends StreamSpec("""
promise.failure(TE("later-no-flow-for-you"))
list.failed.futureValue shouldBe a[TE]
- deferredMatVal.failed.futureValue shouldBe a[TE]
+ deferredMatVal.failed.futureValue shouldBe a[NeverMaterializedException]
+ deferredMatVal.failed.futureValue.getCause shouldBe a[TE]
+ }
+
+ "work for a single element when the future is completed after the fact" in assertAllStagesStopped {
+ import system.dispatcher
+ val flowPromise = Promise[Flow[Int, String, NotUsed]]()
+ val firstElementArrived = Promise[Done]()
+
+ val result: Future[immutable.Seq[String]] =
+ Source(List(1))
+ .via(Flow.lazyFutureFlow { () =>
+ firstElementArrived.success(Done)
+ flowPromise.future
+ })
+ .runWith(Sink.seq)
+
+ firstElementArrived.future.map { _ =>
+ flowPromise.success(Flow[Int].map(_.toString))
+ }
+
+ result.futureValue shouldBe List("1")
}
"fail the flow when the future materialization fails" in assertAllStagesStopped {
@@ -169,7 +191,9 @@ class LazyFlowSpec extends StreamSpec("""
val deferredMatVal = result._1
val list = result._2
list.failed.futureValue shouldBe a[TE]
- deferredMatVal.failed.futureValue shouldBe a[TE]
+ //futureFlow's behaviour in case of mat failure (follows flatMapPrefix)
+ deferredMatVal.failed.futureValue shouldBe a[NeverMaterializedException]
+ deferredMatVal.failed.futureValue.getCause shouldEqual TE("mat-failed")
}
"fail the flow when there was elements but the inner flow failed" in assertAllStagesStopped {
@@ -244,12 +268,12 @@ class LazyFlowSpec extends StreamSpec("""
"complete when there was no elements in the stream" in assertAllStagesStopped {
def flowMaker() = flowF
- val probe = Source.empty.via(Flow.lazyInitAsync(() => flowMaker)).runWith(TestSink.probe[Int])
+ val probe = Source.empty.via(Flow.lazyInitAsync(() => flowMaker())).runWith(TestSink.probe[Int])
probe.request(1).expectComplete()
}
"complete normally when upstream completes BEFORE the stage has switched to the inner flow" in assertAllStagesStopped {
- val promise = Promise[Flow[Int, Int, NotUsed]]
+ val promise = Promise[Flow[Int, Int, NotUsed]]()
val (pub, sub) = TestSource
.probe[Int]
.viaMat(Flow.lazyInitAsync(() => promise.future))(Keep.left)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySinkSpec.scala
index 25d236d02a..70bd3fd1d2 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySinkSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySinkSpec.scala
@@ -6,6 +6,14 @@ package akka.stream.scaladsl
import java.util.concurrent.TimeoutException
+import scala.collection.immutable
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.Promise
+import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
+
import akka.NotUsed
import akka.stream._
import akka.stream.stage.GraphStage
@@ -16,13 +24,6 @@ import akka.stream.testkit.TestSubscriber.Probe
import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
-import com.github.ghik.silencer.silent
-
-import scala.collection.immutable
-import scala.concurrent.Await
-import scala.concurrent.Future
-import scala.concurrent.Promise
-import scala.concurrent.duration._
@silent("deprecated")
class LazySinkSpec extends StreamSpec("""
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySourceSpec.scala
index bd0ffb43fe..feb23b249b 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySourceSpec.scala
@@ -6,24 +6,25 @@ package akka.stream.scaladsl
import java.util.concurrent.atomic.AtomicBoolean
-import akka.stream._
-import akka.stream.stage.GraphStage
-import akka.stream.stage.GraphStageLogic
-import akka.stream.testkit.Utils.TE
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit.StreamSpec
-import akka.stream.testkit.TestPublisher
-import akka.stream.testkit.TestSubscriber
-import akka.testkit.DefaultTimeout
-import akka.testkit.TestProbe
-import akka.Done
-import akka.NotUsed
-import org.scalatest.concurrent.ScalaFutures
-
import scala.collection.immutable.Seq
import scala.concurrent.Future
import scala.concurrent.Promise
+import org.scalatest.concurrent.ScalaFutures
+
+import akka.Done
+import akka.NotUsed
+import akka.stream._
+import akka.stream.stage.GraphStage
+import akka.stream.stage.GraphStageLogic
+import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.TestPublisher
+import akka.stream.testkit.TestSubscriber
+import akka.stream.testkit.Utils.TE
+import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.testkit.DefaultTimeout
+import akka.testkit.TestProbe
+
class LazySourceSpec extends StreamSpec with DefaultTimeout with ScalaFutures {
import system.dispatcher
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/MaybeSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/MaybeSourceSpec.scala
index 93d73de1cc..10ce335cbd 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/MaybeSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/MaybeSourceSpec.scala
@@ -4,16 +4,16 @@
package akka.stream.scaladsl
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
import akka.stream.{ AbruptStageTerminationException, KillSwitches, Materializer }
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.TestSubscriber
-import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.Utils.TE
+import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.DefaultTimeout
-import scala.concurrent.duration._
-import scala.util.control.NoStackTrace
-
class MaybeSourceSpec extends StreamSpec with DefaultTimeout {
"The Maybe Source" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/NeverSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/NeverSourceSpec.scala
new file mode 100644
index 0000000000..102a44db54
--- /dev/null
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/NeverSourceSpec.scala
@@ -0,0 +1,33 @@
+/*
+ * Copyright (C) 2014-2020 Lightbend Inc.
+ */
+
+package akka.stream.scaladsl
+
+import scala.concurrent.duration._
+
+import akka.stream.testkit.{ StreamSpec, TestSubscriber }
+import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.testkit.DefaultTimeout
+
+class NeverSourceSpec extends StreamSpec with DefaultTimeout {
+
+ "The Never Source" must {
+
+ "never completes" in assertAllStagesStopped {
+ val neverSource = Source.never[Int]
+ val pubSink = Sink.asPublisher[Int](false)
+
+ val neverPub = neverSource.toMat(pubSink)(Keep.right).run()
+
+ val c = TestSubscriber.manualProbe[Int]()
+ neverPub.subscribe(c)
+ val subs = c.expectSubscription()
+
+ subs.request(1)
+ c.expectNoMessage(300.millis)
+
+ subs.cancel()
+ }
+ }
+}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/PublisherSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/PublisherSinkSpec.scala
index 8fac7afa05..5abb4dba74 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/PublisherSinkSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/PublisherSinkSpec.scala
@@ -4,13 +4,13 @@
package akka.stream.scaladsl
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
import akka.stream.ClosedShape
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.scaladsl.StreamTestKit._
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
class PublisherSinkSpec extends StreamSpec {
"A PublisherSink" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSinkSpec.scala
index cfa5fc352e..b01736a648 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSinkSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSinkSpec.scala
@@ -4,6 +4,12 @@
package akka.stream.scaladsl
+import scala.concurrent.Await
+import scala.concurrent.ExecutionContextExecutor
+import scala.concurrent.Promise
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
import akka.pattern.pipe
import akka.stream.AbruptTerminationException
import akka.stream.Attributes.inputBuffer
@@ -12,13 +18,9 @@ import akka.stream.StreamDetachedException
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSource
-import scala.concurrent.Await
-import scala.concurrent.Promise
-import scala.concurrent.duration._
-import scala.util.control.NoStackTrace
class QueueSinkSpec extends StreamSpec {
- implicit val ec = system.dispatcher
+ implicit val ec: ExecutionContextExecutor = system.dispatcher
val ex = new RuntimeException("ex") with NoStackTrace
@@ -169,7 +171,7 @@ class QueueSinkSpec extends StreamSpec {
val bufferSize = 16
val streamElementCount = bufferSize + 4
val sink = Sink.queue[Int]().withAttributes(inputBuffer(bufferSize, bufferSize))
- val bufferFullProbe = Promise[akka.Done.type]
+ val bufferFullProbe = Promise[akka.Done.type]()
val queue = Source(1 to streamElementCount)
.alsoTo(Flow[Int].drop(bufferSize - 1).to(Sink.foreach(_ => bufferFullProbe.trySuccess(akka.Done))))
.toMat(sink)(Keep.right)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSourceSpec.scala
index 6541c7a042..6107f8e5a7 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSourceSpec.scala
@@ -4,6 +4,11 @@
package akka.stream.scaladsl
+import scala.concurrent._
+import scala.concurrent.duration._
+
+import org.scalatest.time.Span
+
import akka.Done
import akka.actor.Status
import akka.pattern.pipe
@@ -16,17 +21,13 @@ import akka.stream.testkit.TestSubscriber
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.TestProbe
-import org.scalatest.time.Span
-
-import scala.concurrent._
-import scala.concurrent.duration._
class QueueSourceSpec extends StreamSpec {
- implicit val ec = system.dispatcher
+ implicit val ec: ExecutionContextExecutor = system.dispatcher
val pause = 300.millis
// more frequent checks than defaults from AkkaSpec
- implicit val testPatience =
+ implicit val testPatience: PatienceConfig =
PatienceConfig(testKitSettings.DefaultTimeout.duration, Span(5, org.scalatest.time.Millis))
def assertSuccess(f: Future[QueueOfferResult]): Unit = {
@@ -38,7 +39,7 @@ class QueueSourceSpec extends StreamSpec {
"emit received messages to the stream" in {
val s = TestSubscriber.manualProbe[Int]()
val queue = Source.queue(10, OverflowStrategy.fail).to(Sink.fromSubscriber(s)).run()
- val sub = s.expectSubscription
+ val sub = s.expectSubscription()
for (i <- 1 to 3) {
sub.request(1)
assertSuccess(queue.offer(i))
@@ -85,7 +86,7 @@ class QueueSourceSpec extends StreamSpec {
"buffer when needed" in {
val s = TestSubscriber.manualProbe[Int]()
val queue = Source.queue(100, OverflowStrategy.dropHead).to(Sink.fromSubscriber(s)).run()
- val sub = s.expectSubscription
+ val sub = s.expectSubscription()
for (n <- 1 to 20) assertSuccess(queue.offer(n))
sub.request(10)
for (n <- 1 to 10) assertSuccess(queue.offer(n))
@@ -101,7 +102,7 @@ class QueueSourceSpec extends StreamSpec {
"not fail when 0 buffer space and demand is signalled" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]()
val queue = Source.queue(0, OverflowStrategy.dropHead).to(Sink.fromSubscriber(s)).run()
- val sub = s.expectSubscription
+ val sub = s.expectSubscription()
sub.request(1)
assertSuccess(queue.offer(1))
@@ -112,7 +113,7 @@ class QueueSourceSpec extends StreamSpec {
"wait for demand when buffer is 0" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]()
val queue = Source.queue(0, OverflowStrategy.dropHead).to(Sink.fromSubscriber(s)).run()
- val sub = s.expectSubscription
+ val sub = s.expectSubscription()
queue.offer(1).pipeTo(testActor)
expectNoMessage(pause)
sub.request(1)
@@ -124,7 +125,7 @@ class QueueSourceSpec extends StreamSpec {
"finish offer and complete futures when stream completed" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]()
val queue = Source.queue(0, OverflowStrategy.dropHead).to(Sink.fromSubscriber(s)).run()
- val sub = s.expectSubscription
+ val sub = s.expectSubscription()
queue.watchCompletion.pipeTo(testActor)
queue.offer(1).pipeTo(testActor)
@@ -144,7 +145,7 @@ class QueueSourceSpec extends StreamSpec {
"fail stream on buffer overflow in fail mode" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]()
val queue = Source.queue(1, OverflowStrategy.fail).to(Sink.fromSubscriber(s)).run()
- s.expectSubscription
+ s.expectSubscription()
queue.offer(1)
queue.offer(2)
@@ -156,7 +157,7 @@ class QueueSourceSpec extends StreamSpec {
val probe = TestProbe()
val queue =
TestSourceStage(new QueueSource[Int](1, OverflowStrategy.dropHead, 1), probe).to(Sink.fromSubscriber(s)).run()
- val sub = s.expectSubscription
+ val sub = s.expectSubscription()
sub.request(1)
probe.expectMsg(GraphStageMessages.Pull)
@@ -225,7 +226,7 @@ class QueueSourceSpec extends StreamSpec {
"return false when element was not added to buffer" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]()
val queue = Source.queue(1, OverflowStrategy.dropNew).to(Sink.fromSubscriber(s)).run()
- val sub = s.expectSubscription
+ val sub = s.expectSubscription()
queue.offer(1)
queue.offer(2).pipeTo(testActor)
@@ -239,7 +240,7 @@ class QueueSourceSpec extends StreamSpec {
"wait when buffer is full and backpressure is on" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]()
val queue = Source.queue(1, OverflowStrategy.backpressure).to(Sink.fromSubscriber(s)).run()
- val sub = s.expectSubscription
+ val sub = s.expectSubscription()
assertSuccess(queue.offer(1))
queue.offer(2).pipeTo(testActor)
@@ -258,7 +259,7 @@ class QueueSourceSpec extends StreamSpec {
"fail offer future when stream is completed" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]()
val queue = Source.queue(1, OverflowStrategy.dropNew).to(Sink.fromSubscriber(s)).run()
- val sub = s.expectSubscription
+ val sub = s.expectSubscription()
queue.watchCompletion().pipeTo(testActor)
sub.cancel()
expectMsg(Done)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RestartSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RestartSpec.scala
index 27a5e80545..0ee74dd84c 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RestartSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RestartSpec.scala
@@ -6,26 +6,26 @@ package akka.stream.scaladsl
import java.util.concurrent.atomic.AtomicInteger
-import akka.stream.scaladsl.RestartWithBackoffFlow.Delay
-import akka.stream.testkit.Utils.TE
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit.scaladsl.TestSink
-import akka.stream.testkit.scaladsl.TestSource
-import akka.stream.testkit.StreamSpec
-import akka.stream.testkit.TestPublisher
-import akka.stream.testkit.TestSubscriber
-import akka.stream.Attributes
-import akka.stream.OverflowStrategy
-import akka.testkit.DefaultTimeout
-import akka.testkit.TestDuration
-import akka.Done
-import akka.NotUsed
-
import scala.concurrent.Promise
import scala.concurrent.duration._
import scala.util.Failure
import scala.util.Success
+import akka.Done
+import akka.NotUsed
+import akka.stream.Attributes
+import akka.stream.OverflowStrategy
+import akka.stream.scaladsl.RestartWithBackoffFlow.Delay
+import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.TestPublisher
+import akka.stream.testkit.TestSubscriber
+import akka.stream.testkit.Utils.TE
+import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.stream.testkit.scaladsl.TestSink
+import akka.stream.testkit.scaladsl.TestSource
+import akka.testkit.DefaultTimeout
+import akka.testkit.TestDuration
+
class RestartSpec extends StreamSpec(Map("akka.test.single-expect-default" -> "10s")) with DefaultTimeout {
import system.dispatcher
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RetryFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RetryFlowSpec.scala
index 683f44bd4f..b04733509a 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RetryFlowSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RetryFlowSpec.scala
@@ -4,15 +4,16 @@
package akka.stream.scaladsl
-import akka.NotUsed
-import akka.stream.OverflowStrategy
-import akka.stream.testkit.scaladsl.{ TestSink, TestSource }
-import akka.stream.testkit.{ StreamSpec, TestPublisher, TestSubscriber }
-import org.scalatest.matchers.{ MatchResult, Matcher }
-
import scala.concurrent.duration._
import scala.util.{ Failure, Success, Try }
+import org.scalatest.matchers.{ MatchResult, Matcher }
+
+import akka.NotUsed
+import akka.stream.OverflowStrategy
+import akka.stream.testkit.{ StreamSpec, TestPublisher, TestSubscriber }
+import akka.stream.testkit.scaladsl.{ TestSink, TestSource }
+
class RetryFlowSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 1
akka.stream.materializer.max-input-buffer-size = 1
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ReverseArrowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ReverseArrowSpec.scala
index 0d1bc71e80..c99dc59ffc 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ReverseArrowSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ReverseArrowSpec.scala
@@ -4,12 +4,12 @@
package akka.stream.scaladsl
-import akka.stream._
-import akka.stream.testkit._
-
import scala.concurrent.Await
import scala.concurrent.duration._
+import akka.stream._
+import akka.stream.testkit._
+
class ReverseArrowSpec extends StreamSpec {
import GraphDSL.Implicits._
@@ -41,7 +41,7 @@ class ReverseArrowSpec extends StreamSpec {
}
"work from Sink" in {
- val sub = TestSubscriber.manualProbe[Int]
+ val sub = TestSubscriber.manualProbe[Int]()
RunnableGraph
.fromGraph(GraphDSL.create() { implicit b =>
Sink.fromSubscriber(sub) <~ source
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RunnableGraphSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RunnableGraphSpec.scala
index 684136593b..feb2fb0ea3 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RunnableGraphSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RunnableGraphSpec.scala
@@ -28,7 +28,7 @@ class RunnableGraphSpec extends StreamSpec {
}
"allow conversion from java to scala" in {
- val runnable: RunnableGraph[NotUsed] = javadsl.Source.empty.to(javadsl.Sink.ignore).asScala
+ val runnable: RunnableGraph[NotUsed] = javadsl.Source.empty().to(javadsl.Sink.ignore()).asScala
runnable.run() shouldBe NotUsed
}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SeqSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SeqSinkSpec.scala
index 641bb0b5e4..1f66a7e5e4 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SeqSinkSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SeqSinkSpec.scala
@@ -4,15 +4,15 @@
package akka.stream.scaladsl
+import scala.collection.immutable
+import scala.concurrent.Await
+import scala.concurrent.Future
+
import akka.stream.AbruptTerminationException
import akka.stream.Materializer
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.TestPublisher
-import scala.collection.immutable
-import scala.concurrent.Await
-import scala.concurrent.Future
-
class SeqSinkSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SetupSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SetupSpec.scala
index 02b28abdbb..1d0e14c1c8 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SetupSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SetupSpec.scala
@@ -4,9 +4,10 @@
package akka.stream.scaladsl
+import com.github.ghik.silencer.silent
+
import akka.NotUsed
import akka.stream.testkit.StreamSpec
-import com.github.ghik.silencer.silent
@silent("deprecated")
class SetupSpec extends StreamSpec {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkAsJavaStreamSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkAsJavaStreamSpec.scala
index 5c4d3950eb..b996b497f4 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkAsJavaStreamSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkAsJavaStreamSpec.scala
@@ -10,8 +10,8 @@ import akka.stream._
import akka.stream.impl.PhasedFusingActorMaterializer
import akka.stream.impl.StreamSupervisor
import akka.stream.impl.StreamSupervisor.Children
-import akka.stream.testkit.Utils._
import akka.stream.testkit._
+import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSource
import akka.util.ByteString
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachAsyncSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachAsyncSpec.scala
index f67aa78d04..5461956965 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachAsyncSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachAsyncSpec.scala
@@ -8,6 +8,13 @@ import java.util.concurrent.CountDownLatch
import java.util.concurrent.Executors
import java.util.concurrent.TimeUnit
+import scala.concurrent.Await
+import scala.concurrent.ExecutionContext
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.language.postfixOps
+import scala.util.control.NoStackTrace
+
import akka.Done
import akka.stream.ActorAttributes.supervisionStrategy
import akka.stream.Supervision.resumingDecider
@@ -17,13 +24,6 @@ import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.TestLatch
import akka.testkit.TestProbe
-import scala.concurrent.duration._
-import scala.concurrent.Await
-import scala.concurrent.ExecutionContext
-import scala.concurrent.Future
-import scala.language.postfixOps
-import scala.util.control.NoStackTrace
-
class SinkForeachAsyncSpec extends StreamSpec {
"A foreachAsync" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachParallelSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachParallelSpec.scala
index d550e9d4c0..6b39bbbe04 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachParallelSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachParallelSpec.scala
@@ -7,17 +7,18 @@ package akka.stream.scaladsl
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
+import com.github.ghik.silencer.silent
+
import akka.stream.ActorAttributes._
import akka.stream.Supervision._
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.TestLatch
import akka.testkit.TestProbe
-import com.github.ghik.silencer.silent
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
-import scala.util.control.NoStackTrace
@silent // tests deprecated APIs
class SinkForeachParallelSpec extends StreamSpec {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala
index d51b77be55..b5a8b1f92b 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala
@@ -4,17 +4,18 @@
package akka.stream.scaladsl
+import scala.concurrent.{ Await, Future }
+import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
+import org.reactivestreams.Publisher
+import org.scalatest.concurrent.ScalaFutures
+
import akka.Done
import akka.stream._
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.DefaultTimeout
-import com.github.ghik.silencer.silent
-import org.reactivestreams.Publisher
-import org.scalatest.concurrent.ScalaFutures
-
-import scala.concurrent.{ Await, Future }
-import scala.concurrent.duration._
class SinkSpec extends StreamSpec with DefaultTimeout with ScalaFutures {
@@ -22,7 +23,7 @@ class SinkSpec extends StreamSpec with DefaultTimeout with ScalaFutures {
"A Sink" must {
"be composable without importing modules" in {
- val probes = Array.fill(3)(TestSubscriber.manualProbe[Int])
+ val probes = Array.fill(3)(TestSubscriber.manualProbe[Int]())
val sink = Sink.fromGraph(GraphDSL.create() { implicit b =>
val bcast = b.add(Broadcast[Int](3))
for (i <- 0 to 2) bcast.out(i).filter(_ == i) ~> Sink.fromSubscriber(probes(i))
@@ -39,7 +40,7 @@ class SinkSpec extends StreamSpec with DefaultTimeout with ScalaFutures {
}
"be composable with importing 1 module" in {
- val probes = Array.fill(3)(TestSubscriber.manualProbe[Int])
+ val probes = Array.fill(3)(TestSubscriber.manualProbe[Int]())
val sink = Sink.fromGraph(GraphDSL.create(Sink.fromSubscriber(probes(0))) { implicit b => s0 =>
val bcast = b.add(Broadcast[Int](3))
bcast.out(0) ~> Flow[Int].filter(_ == 0) ~> s0.in
@@ -57,7 +58,7 @@ class SinkSpec extends StreamSpec with DefaultTimeout with ScalaFutures {
}
"be composable with importing 2 modules" in {
- val probes = Array.fill(3)(TestSubscriber.manualProbe[Int])
+ val probes = Array.fill(3)(TestSubscriber.manualProbe[Int]())
val sink =
Sink.fromGraph(GraphDSL.create(Sink.fromSubscriber(probes(0)), Sink.fromSubscriber(probes(1)))(List(_, _)) {
implicit b => (s0, s1) =>
@@ -78,7 +79,7 @@ class SinkSpec extends StreamSpec with DefaultTimeout with ScalaFutures {
}
"be composable with importing 3 modules" in {
- val probes = Array.fill(3)(TestSubscriber.manualProbe[Int])
+ val probes = Array.fill(3)(TestSubscriber.manualProbe[Int]())
val sink = Sink.fromGraph(
GraphDSL.create(Sink.fromSubscriber(probes(0)), Sink.fromSubscriber(probes(1)), Sink.fromSubscriber(probes(2)))(
List(_, _, _)) { implicit b => (s0, s1, s2) =>
@@ -120,7 +121,7 @@ class SinkSpec extends StreamSpec with DefaultTimeout with ScalaFutures {
}
probes.foreach { p =>
p.expectNextN(List(1, 2))
- p.expectComplete
+ p.expectComplete()
}
}
@@ -144,7 +145,7 @@ class SinkSpec extends StreamSpec with DefaultTimeout with ScalaFutures {
}
probes.foreach { p =>
p.expectNextN(List(1, 2))
- p.expectComplete
+ p.expectComplete()
}
}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala
index 6d2364cf1f..ca6e17c147 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala
@@ -26,7 +26,7 @@ import scala.collection.immutable
@silent // tests assigning to typed val
class SourceSpec extends StreamSpec with DefaultTimeout {
- implicit val config = PatienceConfig(timeout = Span(timeout.duration.toMillis, Millis))
+ implicit val config: PatienceConfig = PatienceConfig(timeout = Span(timeout.duration.toMillis, Millis))
"Single Source" must {
@@ -86,7 +86,7 @@ class SourceSpec extends StreamSpec with DefaultTimeout {
"merge from many inputs" in {
val probes = immutable.Seq.fill(5)(TestPublisher.manualProbe[Int]())
val source = Source.asSubscriber[Int]
- val out = TestSubscriber.manualProbe[Int]
+ val out = TestSubscriber.manualProbe[Int]()
val s = Source
.fromGraph(GraphDSL.create(source, source, source, source, source)(immutable.Seq(_, _, _, _, _)) {
@@ -122,7 +122,7 @@ class SourceSpec extends StreamSpec with DefaultTimeout {
"combine from many inputs with simplified API" in {
val probes = immutable.Seq.fill(3)(TestPublisher.manualProbe[Int]())
val source = for (i <- 0 to 2) yield Source.fromPublisher(probes(i))
- val out = TestSubscriber.manualProbe[Int]
+ val out = TestSubscriber.manualProbe[Int]()
Source.combine(source(0), source(1), source(2))(Merge(_)).to(Sink.fromSubscriber(out)).run()
val sub = out.expectSubscription()
@@ -143,7 +143,7 @@ class SourceSpec extends StreamSpec with DefaultTimeout {
"combine from two inputs with simplified API" in {
val probes = immutable.Seq.fill(2)(TestPublisher.manualProbe[Int]())
val source = Source.fromPublisher(probes(0)) :: Source.fromPublisher(probes(1)) :: Nil
- val out = TestSubscriber.manualProbe[Int]
+ val out = TestSubscriber.manualProbe[Int]()
Source.combine(source(0), source(1))(Merge(_)).to(Sink.fromSubscriber(out)).run()
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceWithContextSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceWithContextSpec.scala
index ca2d161a83..64145a6163 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceWithContextSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceWithContextSpec.scala
@@ -4,11 +4,11 @@
package akka.stream.scaladsl
+import scala.util.control.NoStackTrace
+
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.scaladsl.TestSink
-import scala.util.control.NoStackTrace
-
case class Message(data: String, offset: Long)
class SourceWithContextSpec extends StreamSpec {
@@ -20,7 +20,7 @@ class SourceWithContextSpec extends StreamSpec {
Source(Vector(msg))
.asSourceWithContext(_.offset)
.toMat(TestSink.probe[(Message, Long)])(Keep.right)
- .run
+ .run()
.request(1)
.expectNext((msg, 1L))
.expectComplete()
@@ -57,7 +57,7 @@ class SourceWithContextSpec extends StreamSpec {
.filter(_ != "b")
.filterNot(_ == "d")
.toMat(TestSink.probe[(String, Long)])(Keep.right)
- .run
+ .run()
.request(2)
.expectNext(("a", 1L))
.expectNext(("c", 4L))
@@ -100,7 +100,7 @@ class SourceWithContextSpec extends StreamSpec {
}
.grouped(2)
.toMat(TestSink.probe[(Seq[String], Seq[Long])])(Keep.right)
- .run
+ .run()
.request(2)
.expectNext((Seq("a-1", "a-2"), Seq(1L, 1L)), (Seq("a-3", "a-4"), Seq(1L, 1L)))
.expectComplete()
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StageActorRefSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StageActorRefSpec.scala
index f420314d1b..2ef72c1200 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StageActorRefSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StageActorRefSpec.scala
@@ -4,6 +4,10 @@
package akka.stream.scaladsl
+import scala.concurrent.Future
+import scala.concurrent.Promise
+import scala.concurrent.duration._
+
import akka.actor.ActorRef
import akka.actor.Kill
import akka.actor.NoSerializationVerificationNeeded
@@ -19,13 +23,9 @@ import akka.testkit.ImplicitSender
import akka.testkit.TestEvent
import akka.testkit.TestProbe
-import scala.concurrent.duration._
-import scala.concurrent.Future
-import scala.concurrent.Promise
-
class StageActorRefSpec extends StreamSpec with ImplicitSender {
import StageActorRefSpec._
- import ControlProtocol._
+ import StageActorRefSpec.ControlProtocol._
def sumStage(probe: ActorRef) = SumTestStage(probe)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StreamConvertersSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StreamConvertersSpec.scala
index 72d0ccbcce..bdd367f1ac 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StreamConvertersSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StreamConvertersSpec.scala
@@ -9,26 +9,26 @@ import java.util.function.BiConsumer
import java.util.function.BinaryOperator
import java.util.function.Supplier
import java.util.function.ToIntFunction
-import java.util.stream.Collector.Characteristics
import java.util.stream.BaseStream
import java.util.stream.Collector
+import java.util.stream.Collector.Characteristics
import java.util.stream.Collectors
-import akka.stream.ActorAttributes
-import akka.stream.testkit.StreamSpec
-import akka.stream.testkit.Utils.TE
-import akka.util.ByteString
-
-import akka.testkit.DefaultTimeout
-import org.scalatest.time.Millis
-import org.scalatest.time.Span
-
import scala.concurrent.Await
import scala.concurrent.duration._
+import org.scalatest.time.Millis
+import org.scalatest.time.Span
+
+import akka.stream.ActorAttributes
+import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.Utils.TE
+import akka.testkit.DefaultTimeout
+import akka.util.ByteString
+
class StreamConvertersSpec extends StreamSpec with DefaultTimeout {
- implicit val config = PatienceConfig(timeout = Span(timeout.duration.toMillis, Millis))
+ implicit val config: PatienceConfig = PatienceConfig(timeout = Span(timeout.duration.toMillis, Millis))
"Java Stream source" must {
import java.util.stream.IntStream
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StreamRefsSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StreamRefsSpec.scala
index 791f8a601d..cf3620147e 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StreamRefsSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StreamRefsSpec.scala
@@ -4,6 +4,14 @@
package akka.stream.scaladsl
+import scala.collection.immutable
+import scala.concurrent.{ Await, Future }
+import scala.concurrent.Promise
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
+import com.typesafe.config._
+
import akka.{ Done, NotUsed }
import akka.actor.{ Actor, ActorIdentity, ActorLogging, ActorRef, ActorSystem, ActorSystemImpl, Identify, Props }
import akka.actor.Status.Failure
@@ -15,13 +23,6 @@ import akka.stream.testkit.Utils.TE
import akka.stream.testkit.scaladsl._
import akka.testkit.{ AkkaSpec, TestKit, TestProbe }
import akka.util.ByteString
-import com.typesafe.config._
-
-import scala.collection.immutable
-import scala.concurrent.Promise
-import scala.concurrent.{ Await, Future }
-import scala.concurrent.duration._
-import scala.util.control.NoStackTrace
object StreamRefsSpec {
@@ -33,8 +34,8 @@ object StreamRefsSpec {
class DataSourceActor() extends Actor with ActorLogging {
- import context.system
import context.dispatcher
+ import context.system
def receive = {
case "give" =>
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubscriberSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubscriberSourceSpec.scala
index 2cc641a9a3..d9d706f72e 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubscriberSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubscriberSourceSpec.scala
@@ -4,11 +4,11 @@
package akka.stream.scaladsl
-import akka.stream.testkit.StreamSpec
-
import scala.concurrent.Await
import scala.concurrent.duration._
+import akka.stream.testkit.StreamSpec
+
class SubscriberSourceSpec extends StreamSpec {
"A SubscriberSource" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubstreamSubscriptionTimeoutSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubstreamSubscriptionTimeoutSpec.scala
index 6e6d267f72..ab131ebc83 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubstreamSubscriptionTimeoutSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubstreamSubscriptionTimeoutSpec.scala
@@ -4,12 +4,12 @@
package akka.stream.scaladsl
+import scala.concurrent.Await
+
import akka.stream.impl.SubscriptionTimeoutException
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
-import scala.concurrent.Await
-
class SubstreamSubscriptionTimeoutSpec extends StreamSpec("""
akka.stream.materializer {
initial-input-buffer-size = 2
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TakeLastSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TakeLastSinkSpec.scala
index a77208d201..fd326800de 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TakeLastSinkSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TakeLastSinkSpec.scala
@@ -4,19 +4,20 @@
package akka.stream.scaladsl
-import akka.stream.testkit.{ StreamSpec, TestPublisher }
-import akka.stream.{ AbruptTerminationException, ActorMaterializer, ActorMaterializerSettings }
-import com.github.ghik.silencer.silent
-
import scala.collection.immutable
import scala.concurrent.{ Await, Future }
+import com.github.ghik.silencer.silent
+
+import akka.stream.{ AbruptTerminationException, ActorMaterializer, ActorMaterializerSettings }
+import akka.stream.testkit.{ StreamSpec, TestPublisher }
+
@silent
class TakeLastSinkSpec extends StreamSpec {
val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16)
- implicit val mat = ActorMaterializer(settings)
+ implicit val mat: ActorMaterializer = ActorMaterializer(settings)
"Sink.takeLast" must {
"return the last 3 elements" in {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TickSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TickSourceSpec.scala
index e232d32964..190c565f07 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TickSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TickSourceSpec.scala
@@ -4,13 +4,13 @@
package akka.stream.scaladsl
+import scala.concurrent.duration._
+
import akka.stream.ClosedShape
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.TimingTest
-import scala.concurrent.duration._
-
class TickSourceSpec extends StreamSpec {
"A Flow based on tick publisher" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceAsyncSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceAsyncSourceSpec.scala
index f81b94d7ae..375b6cd756 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceAsyncSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceAsyncSourceSpec.scala
@@ -6,25 +6,25 @@ package akka.stream.scaladsl
import java.util.concurrent.atomic.AtomicInteger
-import akka.Done
-import akka.stream.ActorAttributes
-import akka.stream.Materializer
-import akka.stream.Supervision
-import akka.stream.impl.StreamSupervisor.Children
-import akka.stream.impl.PhasedFusingActorMaterializer
-import akka.stream.impl.StreamSupervisor
-import akka.stream.testkit.Utils._
-import akka.stream.testkit.scaladsl.StreamTestKit._
-import akka.stream.testkit.StreamSpec
-import akka.stream.testkit.TestSubscriber
-import akka.testkit.TestLatch
-import akka.testkit.TestProbe
-
-import scala.concurrent.duration._
import scala.concurrent.Await
import scala.concurrent.ExecutionContext
import scala.concurrent.Future
import scala.concurrent.Promise
+import scala.concurrent.duration._
+
+import akka.Done
+import akka.stream.ActorAttributes
+import akka.stream.Materializer
+import akka.stream.Supervision
+import akka.stream.impl.PhasedFusingActorMaterializer
+import akka.stream.impl.StreamSupervisor
+import akka.stream.impl.StreamSupervisor.Children
+import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.TestSubscriber
+import akka.stream.testkit.Utils._
+import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.testkit.TestLatch
+import akka.testkit.TestProbe
object UnfoldResourceAsyncSourceSpec {
@@ -80,7 +80,7 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
val probe = TestSubscriber.probe[Int]()
Source
- .unfoldResourceAsync[Int, ResourceDummy[Int]](resource.create _, _.read, _.close)
+ .unfoldResourceAsync[Int, ResourceDummy[Int]](resource.create _, _.read, _.close())
.runWith(Sink.fromSubscriber(probe))
probe.request(1)
@@ -106,7 +106,7 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
val resource = new ResourceDummy[Int](1 :: Nil, firstReadFuture = firstRead.future)
Source
- .unfoldResourceAsync[Int, ResourceDummy[Int]](resource.create _, _.read, _.close)
+ .unfoldResourceAsync[Int, ResourceDummy[Int]](resource.create _, _.read, _.close())
.runWith(Sink.fromSubscriber(probe))
probe.request(1L)
@@ -216,7 +216,7 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
if (!failed) {
failed = true
throw TE("read-error")
- } else if (reader.hasNext) Future.successful(Some(reader.next))
+ } else if (reader.hasNext) Future.successful(Some(reader.next()))
else Future.successful(None),
_ => Future.successful(Done))
.withAttributes(ActorAttributes.supervisionStrategy(Supervision.restartingDecider))
@@ -241,7 +241,7 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
if (!failed) {
failed = true
Future.failed(TE("read-error"))
- } else if (reader.hasNext) Future.successful(Some(reader.next))
+ } else if (reader.hasNext) Future.successful(Some(reader.next()))
else Future.successful(None),
_ => Future.successful(Done))
.withAttributes(ActorAttributes.supervisionStrategy(Supervision.restartingDecider))
@@ -319,7 +319,7 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
Source
.unfoldResourceAsync[String, Unit](
- () => Promise[Unit].future, // never complete
+ () => Promise[Unit]().future, // never complete
_ => ???,
_ => ???)
.runWith(Sink.ignore)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceSourceSpec.scala
index 35eb3918d2..1340738954 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceSourceSpec.scala
@@ -9,6 +9,11 @@ import java.nio.charset.StandardCharsets
import java.nio.file.Files
import java.util.concurrent.atomic.AtomicInteger
+import scala.concurrent.duration._
+
+import com.google.common.jimfs.Configuration
+import com.google.common.jimfs.Jimfs
+
import akka.stream.ActorAttributes
import akka.stream.ActorAttributes._
import akka.stream.Supervision._
@@ -23,10 +28,6 @@ import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.EventFilter
import akka.util.ByteString
-import com.google.common.jimfs.Configuration
-import com.google.common.jimfs.Jimfs
-
-import scala.concurrent.duration._
class UnfoldResourceSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/WithContextUsageSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/WithContextUsageSpec.scala
index 84ba83d3e3..de4314f3cf 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/WithContextUsageSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/WithContextUsageSpec.scala
@@ -4,13 +4,13 @@
package akka.stream.scaladsl
+import scala.collection.immutable
+
import akka.NotUsed
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.TestSubscriber.Probe
import akka.stream.testkit.scaladsl.TestSink
-import scala.collection.immutable
-
class WithContextUsageSpec extends StreamSpec {
"Context propagation used for committing offsets" must {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/snapshot/MaterializerStateSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/snapshot/MaterializerStateSpec.scala
index 8ab91b2c35..77d1fb1d26 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/snapshot/MaterializerStateSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/snapshot/MaterializerStateSpec.scala
@@ -4,19 +4,15 @@
package akka.stream.snapshot
+import scala.concurrent.Promise
+import java.net.InetSocketAddress
+
import akka.stream.FlowShape
import akka.stream.Materializer
-import akka.stream.scaladsl.Flow
-import akka.stream.scaladsl.GraphDSL
-import akka.stream.scaladsl.Keep
-import akka.stream.scaladsl.Merge
-import akka.stream.scaladsl.Partition
-import akka.stream.scaladsl.Sink
-import akka.stream.scaladsl.Source
+import akka.stream.scaladsl.{ Flow, GraphDSL, Keep, Merge, Partition, Sink, Source, Tcp }
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.scaladsl.TestSink
-
-import scala.concurrent.Promise
+import javax.net.ssl.SSLContext
class MaterializerStateSpec extends StreamSpec {
@@ -53,6 +49,20 @@ class MaterializerStateSpec extends StreamSpec {
promise.success(1)
}
+ "snapshot a running stream that includes a TLSActor" in {
+ Source.never
+ .via(Tcp().outgoingConnectionWithTls(InetSocketAddress.createUnresolved("akka.io", 443), () => {
+ val engine = SSLContext.getDefault.createSSLEngine("akka.io", 443)
+ engine.setUseClientMode(true)
+ engine
+ }))
+ .runWith(Sink.seq)
+
+ val snapshots = MaterializerState.streamSnapshots(system).futureValue
+ snapshots.size should be(2)
+ snapshots.toString should include("TLS-")
+ }
+
"snapshot a stream that has a stopped stage" in {
implicit val mat = Materializer(system)
try {
diff --git a/akka-stream-typed/src/main/scala/akka/stream/typed/javadsl/ActorFlow.scala b/akka-stream-typed/src/main/scala/akka/stream/typed/javadsl/ActorFlow.scala
index 9bf18757e3..200f339ef3 100644
--- a/akka-stream-typed/src/main/scala/akka/stream/typed/javadsl/ActorFlow.scala
+++ b/akka-stream-typed/src/main/scala/akka/stream/typed/javadsl/ActorFlow.scala
@@ -6,13 +6,13 @@ package akka.stream.typed.javadsl
import java.util.function.BiFunction
+import scala.concurrent.duration._
+
import akka.NotUsed
import akka.actor.typed.ActorRef
import akka.stream.javadsl.Flow
import akka.util.JavaDurationConverters
-import scala.concurrent.duration._
-
/**
* Collection of Flows aimed at integrating with typed Actors.
*/
diff --git a/akka-stream-typed/src/main/scala/akka/stream/typed/javadsl/ActorSink.scala b/akka-stream-typed/src/main/scala/akka/stream/typed/javadsl/ActorSink.scala
index 31eb825660..ab26ff4add 100644
--- a/akka-stream-typed/src/main/scala/akka/stream/typed/javadsl/ActorSink.scala
+++ b/akka-stream-typed/src/main/scala/akka/stream/typed/javadsl/ActorSink.scala
@@ -4,8 +4,8 @@
package akka.stream.typed.javadsl
-import akka.actor.typed._
import akka.NotUsed
+import akka.actor.typed._
import akka.stream.javadsl._
import akka.stream.typed
diff --git a/akka-stream-typed/src/main/scala/akka/stream/typed/javadsl/ActorSource.scala b/akka-stream-typed/src/main/scala/akka/stream/typed/javadsl/ActorSource.scala
index 1eff7bb3e5..b5925cd0cb 100644
--- a/akka-stream-typed/src/main/scala/akka/stream/typed/javadsl/ActorSource.scala
+++ b/akka-stream-typed/src/main/scala/akka/stream/typed/javadsl/ActorSource.scala
@@ -8,8 +8,8 @@ import java.util.function.Predicate
import akka.actor.typed._
import akka.japi.JavaPartialFunction
-import akka.stream.javadsl._
import akka.stream.{ CompletionStrategy, OverflowStrategy }
+import akka.stream.javadsl._
/**
* Collection of Sources aimed at integrating with typed Actors.
diff --git a/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorFlow.scala b/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorFlow.scala
index b40e0f77fa..4a1efffc2c 100644
--- a/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorFlow.scala
+++ b/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorFlow.scala
@@ -4,6 +4,9 @@
package akka.stream.typed.scaladsl
+import scala.annotation.implicitNotFound
+import scala.concurrent.Future
+
import akka.NotUsed
import akka.actor.typed.ActorRef
import akka.pattern.AskTimeoutException
@@ -11,9 +14,6 @@ import akka.stream._
import akka.stream.scaladsl._
import akka.util.Timeout
-import scala.annotation.implicitNotFound
-import scala.concurrent.Future
-
/**
* Collection of Flows aimed at integrating with typed Actors.
*/
diff --git a/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorSink.scala b/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorSink.scala
index 9fc870fc3f..8e371bc376 100644
--- a/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorSink.scala
+++ b/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorSink.scala
@@ -4,9 +4,9 @@
package akka.stream.typed.scaladsl
+import akka.NotUsed
import akka.actor.typed._
import akka.stream.scaladsl._
-import akka.NotUsed
/**
* Collection of Sinks aimed at integrating with typed Actors.
diff --git a/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorSource.scala b/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorSource.scala
index c1318d537c..60df2faa95 100644
--- a/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorSource.scala
+++ b/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorSource.scala
@@ -5,8 +5,8 @@
package akka.stream.typed.scaladsl
import akka.actor.typed._
-import akka.stream.scaladsl._
import akka.stream.{ CompletionStrategy, OverflowStrategy }
+import akka.stream.scaladsl._
/**
* Collection of Sources aimed at integrating with typed Actors.
diff --git a/akka-stream-typed/src/test/java/akka/stream/typed/javadsl/ActorFlowCompileTest.java b/akka-stream-typed/src/test/java/akka/stream/typed/javadsl/ActorFlowCompileTest.java
deleted file mode 100644
index 2dd6561901..0000000000
--- a/akka-stream-typed/src/test/java/akka/stream/typed/javadsl/ActorFlowCompileTest.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Copyright (C) 2018-2020 Lightbend Inc.
- */
-
-package akka.stream.typed.javadsl;
-
-import akka.actor.typed.ActorRef;
-import akka.actor.typed.ActorSystem;
-import akka.stream.ActorMaterializer;
-import akka.stream.javadsl.Sink;
-import akka.stream.javadsl.Source;
-
-import java.time.Duration;
-import java.time.temporal.ChronoUnit;
-
-public class ActorFlowCompileTest {
-
- interface Protocol {}
-
- class Init implements Protocol {}
-
- class Msg implements Protocol {}
-
- class Complete implements Protocol {}
-
- class Failure implements Protocol {
- public Exception ex;
- }
-
- {
- final ActorSystem system = null;
- }
-
- static
- // #ask-actor
- class AskMe {
- final String payload;
- final ActorRef replyTo;
-
- AskMe(String payload, ActorRef replyTo) {
- this.payload = payload;
- this.replyTo = replyTo;
- }
- }
-
- // #ask-actor
-
- {
- final ActorRef ref = null;
-
- // #ask
- Duration timeout = Duration.of(1, ChronoUnit.SECONDS);
-
- Source.repeat("hello").via(ActorFlow.ask(ref, timeout, AskMe::new)).to(Sink.ignore());
-
- Source.repeat("hello")
- .via(
- ActorFlow.ask(
- ref, timeout, (msg, replyTo) -> new AskMe(msg, replyTo)))
- .to(Sink.ignore());
- // #ask
- }
-}
diff --git a/akka-stream-typed/src/test/java/docs/javadsl/ActorFlowCompileTest.java b/akka-stream-typed/src/test/java/docs/javadsl/ActorFlowCompileTest.java
new file mode 100644
index 0000000000..625585abfa
--- /dev/null
+++ b/akka-stream-typed/src/test/java/docs/javadsl/ActorFlowCompileTest.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright (C) 2018-2020 Lightbend Inc.
+ */
+
+package docs.javadsl;
+
+import akka.NotUsed;
+// #ask-actor
+import akka.actor.typed.ActorRef;
+import akka.actor.typed.ActorSystem;
+import akka.stream.javadsl.Flow;
+import akka.stream.javadsl.Sink;
+import akka.stream.javadsl.Source;
+import akka.stream.typed.javadsl.ActorFlow;
+
+// #ask-actor
+import java.time.Duration;
+
+public class ActorFlowCompileTest {
+
+ final ActorSystem system = null;
+
+ static
+ // #ask-actor
+ class Asking {
+ final String payload;
+ final ActorRef replyTo;
+
+ public Asking(String payload, ActorRef replyTo) {
+ this.payload = payload;
+ this.replyTo = replyTo;
+ }
+ }
+
+ // #ask-actor
+ static
+ // #ask-actor
+ class Reply {
+ public final String msg;
+
+ public Reply(String msg) {
+ this.msg = msg;
+ }
+ }
+
+ // #ask-actor
+
+ {
+ // #ask
+ final ActorRef actorRef = // ???
+ // #ask
+ null;
+
+ // #ask
+ Duration timeout = Duration.ofSeconds(1);
+
+ // method reference notation
+ Flow askFlow = ActorFlow.ask(actorRef, timeout, Asking::new);
+
+ // explicit creation of the sent message
+ Flow askFlowExplicit =
+ ActorFlow.ask(actorRef, timeout, (msg, replyTo) -> new Asking(msg, replyTo));
+
+ Source.repeat("hello").via(askFlow).map(reply -> reply.msg).runWith(Sink.seq(), system);
+ // #ask
+ }
+}
diff --git a/akka-stream-typed/src/test/scala/akka/stream/typed/scaladsl/ActorSourceSinkSpec.scala b/akka-stream-typed/src/test/scala/akka/stream/typed/scaladsl/ActorSourceSinkSpec.scala
index 367c4340ea..0ae5b85d75 100644
--- a/akka-stream-typed/src/test/scala/akka/stream/typed/scaladsl/ActorSourceSinkSpec.scala
+++ b/akka-stream-typed/src/test/scala/akka/stream/typed/scaladsl/ActorSourceSinkSpec.scala
@@ -4,14 +4,15 @@
package akka.stream.typed.scaladsl
+import org.scalatest.wordspec.AnyWordSpecLike
+
+import akka.actor.testkit.typed.scaladsl._
import akka.actor.typed.ActorRef
import akka.actor.typed.scaladsl.Behaviors
import akka.stream.{ CompletionStrategy, OverflowStrategy }
import akka.stream.scaladsl.Keep
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
-import akka.actor.testkit.typed.scaladsl._
-import org.scalatest.wordspec.AnyWordSpecLike
object ActorSourceSinkSpec {
diff --git a/akka-stream-typed/src/test/scala/akka/stream/typed/scaladsl/MaterializerForTypedSpec.scala b/akka-stream-typed/src/test/scala/akka/stream/typed/scaladsl/MaterializerForTypedSpec.scala
index b62f2d6a9c..090b42f5db 100644
--- a/akka-stream-typed/src/test/scala/akka/stream/typed/scaladsl/MaterializerForTypedSpec.scala
+++ b/akka-stream-typed/src/test/scala/akka/stream/typed/scaladsl/MaterializerForTypedSpec.scala
@@ -4,6 +4,11 @@
package akka.stream.typed.scaladsl
+import scala.concurrent.Future
+import scala.util.Success
+
+import org.scalatest.wordspec.AnyWordSpecLike
+
import akka.Done
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.typed.scaladsl.Behaviors
@@ -12,10 +17,6 @@ import akka.stream.Materializer
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
-import scala.concurrent.Future
-import scala.util.Success
-import org.scalatest.wordspec.AnyWordSpecLike
-
class MaterializerForTypedSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike {
"Materialization in typed" should {
diff --git a/akka-stream-typed/src/test/scala/akka/stream/typed/scaladsl/ActorFlowSpec.scala b/akka-stream-typed/src/test/scala/docs/scaladsl/ActorFlowSpec.scala
similarity index 76%
rename from akka-stream-typed/src/test/scala/akka/stream/typed/scaladsl/ActorFlowSpec.scala
rename to akka-stream-typed/src/test/scala/docs/scaladsl/ActorFlowSpec.scala
index 11232ba4dd..d55184193e 100644
--- a/akka-stream-typed/src/test/scala/akka/stream/typed/scaladsl/ActorFlowSpec.scala
+++ b/akka-stream-typed/src/test/scala/docs/scaladsl/ActorFlowSpec.scala
@@ -2,26 +2,30 @@
* Copyright (C) 2018-2020 Lightbend Inc.
*/
-package akka.stream.typed.scaladsl
+package docs.scaladsl
+import akka.NotUsed
//#imports
-import akka.stream.scaladsl._
+import akka.stream.scaladsl.{ Flow, Sink, Source }
+import akka.stream.typed.scaladsl.ActorFlow
import akka.actor.typed.ActorRef
import akka.actor.typed.scaladsl.Behaviors
-import scala.concurrent.duration._
+//#imports
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
+import akka.stream.testkit.TestSubscriber
import org.scalatest.wordspec.AnyWordSpecLike
-//#imports
-import akka.stream.testkit.TestSubscriber
-
import scala.collection.immutable
+import scala.concurrent.duration._
import scala.concurrent.{ Await, Future }
object ActorFlowSpec {
+ //#ask-actor
final case class Asking(s: String, replyTo: ActorRef[Reply])
- final case class Reply(s: String)
+ final case class Reply(msg: String)
+
+ //#ask-actor
}
class ActorFlowSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike {
@@ -59,14 +63,21 @@ class ActorFlowSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike {
//#ask-actor
//#ask
- val in: Future[immutable.Seq[Reply]] =
- Source(1 to 50)
- .map(_.toString)
- .via(ActorFlow.ask(ref)((el, replyTo: ActorRef[Reply]) => Asking(el, replyTo)))
- .runWith(Sink.seq)
- //#ask
+ implicit val timeout: akka.util.Timeout = 1.second
- in.futureValue shouldEqual List.tabulate(51)(i => Reply(s"$i!!!")).drop(1)
+ val askFlow: Flow[String, Reply, NotUsed] =
+ ActorFlow.ask(ref)(Asking.apply)
+
+ // explicit creation of the sent message
+ val askFlowExplicit: Flow[String, Reply, NotUsed] =
+ ActorFlow.ask(ref)(makeMessage = (el, replyTo: ActorRef[Reply]) => Asking(el, replyTo))
+
+ val in: Future[immutable.Seq[String]] =
+ Source(1 to 50).map(_.toString).via(askFlow).map(_.msg).runWith(Sink.seq)
+ //#ask
+ askFlowExplicit.map(identity)
+
+ in.futureValue shouldEqual List.tabulate(51)(i => s"$i!!!").drop(1)
}
"signal ask timeout failure" in {
diff --git a/akka-stream/src/main/java/akka/stream/StreamRefMessages.java b/akka-stream/src/main/java/akka/stream/StreamRefMessages.java
index dd272df005..dcea5b873e 100644
--- a/akka-stream/src/main/java/akka/stream/StreamRefMessages.java
+++ b/akka-stream/src/main/java/akka/stream/StreamRefMessages.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2019-2020 Lightbend Inc.
+ * Copyright (C) 2020 Lightbend Inc.
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
diff --git a/akka-stream/src/main/mima-filters/2.6.5.backwards.excludes/28729-future-flow.backwards.excludes b/akka-stream/src/main/mima-filters/2.6.5.backwards.excludes/28729-future-flow.backwards.excludes
new file mode 100644
index 0000000000..3a3654c452
--- /dev/null
+++ b/akka-stream/src/main/mima-filters/2.6.5.backwards.excludes/28729-future-flow.backwards.excludes
@@ -0,0 +1,2 @@
+# Changes to internals
+ProblemFilters.exclude[MissingClassProblem]("akka.stream.impl.fusing.LazyFlow")
diff --git a/akka-stream/src/main/mima-filters/2.6.5.backwards.excludes/change-StreamSnapshotImpl-toString.backwards.excludes b/akka-stream/src/main/mima-filters/2.6.5.backwards.excludes/change-StreamSnapshotImpl-toString.backwards.excludes
new file mode 100644
index 0000000000..5f4c2016b0
--- /dev/null
+++ b/akka-stream/src/main/mima-filters/2.6.5.backwards.excludes/change-StreamSnapshotImpl-toString.backwards.excludes
@@ -0,0 +1 @@
+ProblemFilters.exclude[MissingTypesProblem]("akka.stream.snapshot.StreamSnapshotImpl")
diff --git a/akka-stream/src/main/resources/reference.conf b/akka-stream/src/main/resources/reference.conf
index 8cf220c6c8..dd982714a1 100644
--- a/akka-stream/src/main/resources/reference.conf
+++ b/akka-stream/src/main/resources/reference.conf
@@ -3,7 +3,7 @@
#####################################
# eager creation of the system wide materializer
-akka.library-extensions += "akka.stream.SystemMaterializer"
+akka.library-extensions += "akka.stream.SystemMaterializer$"
akka {
stream {
diff --git a/akka-stream/src/main/scala-jdk-9/akka/stream/impl/JavaFlowAndRsConverters.scala b/akka-stream/src/main/scala-jdk-9/akka/stream/impl/JavaFlowAndRsConverters.scala
index 7fe03f2201..b4e20c0f5b 100644
--- a/akka-stream/src/main/scala-jdk-9/akka/stream/impl/JavaFlowAndRsConverters.scala
+++ b/akka-stream/src/main/scala-jdk-9/akka/stream/impl/JavaFlowAndRsConverters.scala
@@ -6,9 +6,10 @@ package akka.stream.impl
import java.util.concurrent.Flow
-import akka.annotation.InternalApi
-import org.{ reactivestreams => rs }
import JavaFlowAndRsConverters.Implicits._
+import org.{reactivestreams => rs}
+
+import akka.annotation.InternalApi
/**
* INTERNAL API: Provides converters between Reactive Streams (reactive-streams.org) and their Java 9+ counter-parts,
diff --git a/akka-stream/src/main/scala-jdk-9/akka/stream/scaladsl/JavaFlowSupport.scala b/akka-stream/src/main/scala-jdk-9/akka/stream/scaladsl/JavaFlowSupport.scala
index 01f0dad606..d4b08ce757 100644
--- a/akka-stream/src/main/scala-jdk-9/akka/stream/scaladsl/JavaFlowSupport.scala
+++ b/akka-stream/src/main/scala-jdk-9/akka/stream/scaladsl/JavaFlowSupport.scala
@@ -4,14 +4,14 @@
package akka.stream.scaladsl
-import java.util.{ concurrent => juc }
+import java.util.{concurrent => juc}
+
+import scala.annotation.unchecked.uncheckedVariance
import akka.NotUsed
import akka.stream.impl.JavaFlowAndRsConverters
import akka.stream.scaladsl
-import scala.annotation.unchecked.uncheckedVariance
-
/**
* For use only with `JDK 9+`.
*
diff --git a/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala b/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala
index d603f8ac72..9b8aceb4c7 100644
--- a/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala
+++ b/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala
@@ -6,6 +6,13 @@ package akka.stream
import java.util.concurrent.TimeUnit
+import scala.concurrent.duration._
+import scala.util.control.NoStackTrace
+
+import com.github.ghik.silencer.silent
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
+
import akka.actor.ActorContext
import akka.actor.ActorRef
import akka.actor.ActorRefFactory
@@ -18,12 +25,6 @@ import akka.japi.function
import akka.stream.impl._
import akka.stream.stage.GraphStageLogic
import akka.util.Helpers.toRootLowerCase
-import com.github.ghik.silencer.silent
-import com.typesafe.config.Config
-import com.typesafe.config.ConfigFactory
-
-import scala.concurrent.duration._
-import scala.util.control.NoStackTrace
object ActorMaterializer {
@@ -744,6 +745,7 @@ final class ActorMaterializerSettings @InternalApi private (
// for stream refs and io live with the respective stages
Attributes.InputBuffer(initialInputBufferSize, maxInputBufferSize) ::
Attributes.CancellationStrategy.Default :: // FIXME: make configurable, see https://github.com/akka/akka/issues/28000
+ Attributes.NestedMaterializationCancellationPolicy.Default ::
ActorAttributes.Dispatcher(dispatcher) ::
ActorAttributes.SupervisionStrategy(supervisionDecider) ::
ActorAttributes.DebugLogging(debugLogging) ::
diff --git a/akka-stream/src/main/scala/akka/stream/Attributes.scala b/akka-stream/src/main/scala/akka/stream/Attributes.scala
index 3f8a3832a8..72f09ea988 100644
--- a/akka-stream/src/main/scala/akka/stream/Attributes.scala
+++ b/akka-stream/src/main/scala/akka/stream/Attributes.scala
@@ -4,26 +4,23 @@
package akka.stream
-import java.util.Optional
-
-import akka.event.Logging
-
-import scala.annotation.tailrec
-import scala.reflect.{ classTag, ClassTag }
-import akka.japi.function
import java.net.URLEncoder
import java.time.Duration
+import java.util.Optional
+
+import scala.annotation.tailrec
+import scala.compat.java8.OptionConverters._
+import scala.concurrent.duration.FiniteDuration
+import scala.reflect.{ classTag, ClassTag }
import akka.annotation.ApiMayChange
import akka.annotation.DoNotInherit
import akka.annotation.InternalApi
+import akka.event.Logging
+import akka.japi.function
import akka.stream.impl.TraversalBuilder
-import akka.util.JavaDurationConverters._
-
-import scala.compat.java8.OptionConverters._
import akka.util.{ ByteString, OptionVal }
-
-import scala.concurrent.duration.FiniteDuration
+import akka.util.JavaDurationConverters._
/**
* Holds attributes which can be used to alter [[akka.stream.scaladsl.Flow]] / [[akka.stream.javadsl.Flow]]
@@ -310,7 +307,7 @@ object Attributes {
final case class LogLevels(onElement: Logging.LogLevel, onFinish: Logging.LogLevel, onFailure: Logging.LogLevel)
extends Attribute
- final case object AsyncBoundary extends Attribute
+ case object AsyncBoundary extends Attribute
/**
* Cancellation strategies provide a way to configure the behavior of a stage when `cancelStage` is called.
@@ -442,6 +439,78 @@ object Attributes {
strategy: CancellationStrategy.Strategy): CancellationStrategy.Strategy =
CancellationStrategy.AfterDelay(delay, strategy)
+ /**
+ * Nested materialization cancellation strategy provides a way to configure the cancellation behavior of stages that materialize a nested flow.
+ *
+ * When cancelled before materializing their nested flows, these stages can either immediately cancel (default behaviour) without materializing the nested flow
+ * or wait for the nested flow to materialize and then propagate the cancellation signal through it.
+ *
+ * This applies to [[akka.stream.scaladsl.FlowOps.flatMapPrefix]], [[akka.stream.scaladsl.Flow.futureFlow]] (and derivations such as [[akka.stream.scaladsl.Flow.lazyFutureFlow]]).
+ * These operators either delay the nested flow's materialization or wait for a future to complete before doing so,
+ * in this period of time they may receive a downstream cancellation signal. When this happens these operators will behave according to
+ * this [[Attribute]]: when set to true they will 'stash' the signal and later deliver it to the materialized nested flow
+ * , otherwise these stages will immediately cancel without materializing the nested flow.
+ */
+ @ApiMayChange
+ class NestedMaterializationCancellationPolicy private[NestedMaterializationCancellationPolicy] (
+ val propagateToNestedMaterialization: Boolean)
+ extends MandatoryAttribute
+
+ @ApiMayChange
+ object NestedMaterializationCancellationPolicy {
+
+ /**
+ * A [[NestedMaterializationCancellationPolicy]] that configures graph stages
+ * delaying nested flow materialization to cancel immediately when downstream cancels before
+ * nested flow materialization.
+ * This applies to [[akka.stream.scaladsl.FlowOps.flatMapPrefix]], [[akka.stream.scaladsl.Flow.futureFlow]] and derived operators.
+ */
+ val EagerCancellation = new NestedMaterializationCancellationPolicy(false)
+
+ /**
+ * A [[NestedMaterializationCancellationPolicy]] that configures graph stages
+ * delaying nested flow materialization to delay cancellation when downstream cancels before
+ * nested flow materialization. Once the nested flow is materialized it will be cancelled immediately.
+ * This applies to [[akka.stream.scaladsl.FlowOps.flatMapPrefix]], [[akka.stream.scaladsl.Flow.futureFlow]] and derived operators.
+ */
+ val PropagateToNested = new NestedMaterializationCancellationPolicy(true)
+
+ /**
+ * Default [[NestedMaterializationCancellationPolicy]],
+ * please see [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.EagerCancellation()]] for details.
+ */
+ val Default = EagerCancellation
+ }
+
+ /**
+ * JAVA API
+ * A [[NestedMaterializationCancellationPolicy]] that configures graph stages
+ * delaying nested flow materialization to cancel immediately when downstream cancels before
+ * nested flow materialization.
+ * This applies to [[akka.stream.scaladsl.FlowOps.flatMapPrefix]], [[akka.stream.scaladsl.Flow.futureFlow]] and derived operators.
+ */
+ @ApiMayChange
+ def nestedMaterializationCancellationPolicyEagerCancellation(): NestedMaterializationCancellationPolicy =
+ NestedMaterializationCancellationPolicy.EagerCancellation
+
+ /**
+ * JAVA API
+ * A [[NestedMaterializationCancellationPolicy]] that configures graph stages
+ * delaying nested flow materialization to delay cancellation when downstream cancels before
+ * nested flow materialization. Once the nested flow is materialized it will be cancelled immediately.
+ * This applies to [[akka.stream.scaladsl.FlowOps.flatMapPrefix]], [[akka.stream.scaladsl.Flow.futureFlow]] and derived operators.
+ */
+ @ApiMayChange
+ def nestedMaterializationCancellationPolicyPropagateToNested(): NestedMaterializationCancellationPolicy =
+ NestedMaterializationCancellationPolicy.PropagateToNested
+
+ /**
+ * Default [[NestedMaterializationCancellationPolicy]],
+ * please see [[akka.stream.Attributes#nestedMaterializationCancellationPolicyEagerCancellation()]] for details.
+ */
+ def nestedMaterializationCancellationPolicyDefault(): NestedMaterializationCancellationPolicy =
+ NestedMaterializationCancellationPolicy.Default
+
object LogLevels {
/** Use to disable logging on certain operations when configuring [[Attributes#logLevels]] */
diff --git a/akka-stream/src/main/scala/akka/stream/FanInShape.scala b/akka-stream/src/main/scala/akka/stream/FanInShape.scala
index 588ab4cfe1..4a84c79e5c 100644
--- a/akka-stream/src/main/scala/akka/stream/FanInShape.scala
+++ b/akka-stream/src/main/scala/akka/stream/FanInShape.scala
@@ -4,8 +4,8 @@
package akka.stream
-import scala.collection.immutable
import scala.annotation.unchecked.uncheckedVariance
+import scala.collection.immutable
object FanInShape {
sealed trait Init[O] {
diff --git a/akka-stream/src/main/scala/akka/stream/FanOutShape.scala b/akka-stream/src/main/scala/akka/stream/FanOutShape.scala
index 253cb3db6d..76ef3c2265 100644
--- a/akka-stream/src/main/scala/akka/stream/FanOutShape.scala
+++ b/akka-stream/src/main/scala/akka/stream/FanOutShape.scala
@@ -4,8 +4,8 @@
package akka.stream
-import scala.collection.immutable
import scala.annotation.unchecked.uncheckedVariance
+import scala.collection.immutable
object FanOutShape {
sealed trait Init[I] {
diff --git a/akka-stream/src/main/scala/akka/stream/Graph.scala b/akka-stream/src/main/scala/akka/stream/Graph.scala
index a7b1c40f81..63ce1ceb1a 100644
--- a/akka-stream/src/main/scala/akka/stream/Graph.scala
+++ b/akka-stream/src/main/scala/akka/stream/Graph.scala
@@ -4,12 +4,12 @@
package akka.stream
+import scala.annotation.unchecked.uncheckedVariance
+
import akka.annotation.InternalApi
import akka.stream.impl.TraversalBuilder
import akka.stream.scaladsl.GenericGraph
-import scala.annotation.unchecked.uncheckedVariance
-
/**
* Not intended to be directly extended by user classes
*
diff --git a/akka-stream/src/main/scala/akka/stream/IOResult.scala b/akka-stream/src/main/scala/akka/stream/IOResult.scala
index ced4c65382..be51e45b74 100644
--- a/akka-stream/src/main/scala/akka/stream/IOResult.scala
+++ b/akka-stream/src/main/scala/akka/stream/IOResult.scala
@@ -4,11 +4,12 @@
package akka.stream
-import akka.Done
+import scala.util.{ Failure, Success, Try }
+import scala.util.control.NoStackTrace
+
import com.github.ghik.silencer.silent
-import scala.util.control.NoStackTrace
-import scala.util.{ Failure, Success, Try }
+import akka.Done
/**
* Holds a result of an IO operation.
diff --git a/akka-stream/src/main/scala/akka/stream/KillSwitch.scala b/akka-stream/src/main/scala/akka/stream/KillSwitch.scala
index 9ae737576f..a5e6c83d72 100644
--- a/akka-stream/src/main/scala/akka/stream/KillSwitch.scala
+++ b/akka-stream/src/main/scala/akka/stream/KillSwitch.scala
@@ -4,14 +4,14 @@
package akka.stream
-import akka.{ Done, NotUsed }
-import akka.stream.stage._
+import java.util.concurrent.atomic.AtomicReference
-import scala.concurrent.{ Future, Promise }
import scala.collection.concurrent.TrieMap
+import scala.concurrent.{ Future, Promise }
import scala.util.{ Failure, Success, Try }
-import java.util.concurrent.atomic.AtomicReference
+import akka.{ Done, NotUsed }
+import akka.stream.stage._
/**
* Creates shared or single kill switches which can be used to control completion of graphs from the outside.
@@ -78,7 +78,7 @@ object KillSwitches {
override def toString: String = "UniqueKillSwitchFlow"
override def createLogicAndMaterializedValue(attr: Attributes) = {
- val promise = Promise[Done]
+ val promise = Promise[Done]()
val switch = new UniqueKillSwitch(promise)
val logic = new KillableGraphStageLogic(promise.future, shape) with InHandler with OutHandler {
@@ -104,7 +104,7 @@ object KillSwitches {
override def toString: String = "UniqueKillSwitchBidi"
override def createLogicAndMaterializedValue(attr: Attributes) = {
- val promise = Promise[Done]
+ val promise = Promise[Done]()
val switch = new UniqueKillSwitch(promise)
val logic = new KillableGraphStageLogic(promise.future, shape) {
@@ -159,7 +159,7 @@ trait KillSwitch {
private[stream] final class TerminationSignal {
final class Listener private[TerminationSignal] {
- private[TerminationSignal] val promise = Promise[Done]
+ private[TerminationSignal] val promise = Promise[Done]()
def future: Future[Done] = promise.future
def unregister(): Unit = removeListener(this)
}
diff --git a/akka-stream/src/main/scala/akka/stream/Materializer.scala b/akka-stream/src/main/scala/akka/stream/Materializer.scala
index e8142e160d..46b9ca1631 100644
--- a/akka-stream/src/main/scala/akka/stream/Materializer.scala
+++ b/akka-stream/src/main/scala/akka/stream/Materializer.scala
@@ -4,6 +4,11 @@
package akka.stream
+import scala.concurrent.ExecutionContextExecutor
+import scala.concurrent.duration.FiniteDuration
+
+import com.github.ghik.silencer.silent
+
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.Cancellable
@@ -13,10 +18,6 @@ import akka.actor.Props
import akka.annotation.DoNotInherit
import akka.annotation.InternalApi
import akka.event.LoggingAdapter
-import com.github.ghik.silencer.silent
-
-import scala.concurrent.ExecutionContextExecutor
-import scala.concurrent.duration.FiniteDuration
/**
* The Materializer is the component responsible for turning a stream blueprint into a running stream.
diff --git a/akka-stream/src/main/scala/akka/stream/OverflowStrategy.scala b/akka-stream/src/main/scala/akka/stream/OverflowStrategy.scala
index 4567491a79..0277032da8 100644
--- a/akka-stream/src/main/scala/akka/stream/OverflowStrategy.scala
+++ b/akka-stream/src/main/scala/akka/stream/OverflowStrategy.scala
@@ -5,6 +5,7 @@
package akka.stream
import OverflowStrategies._
+
import akka.annotation.{ DoNotInherit, InternalApi }
import akka.event.Logging
import akka.event.Logging.LogLevel
diff --git a/akka-stream/src/main/scala/akka/stream/QueueOfferResult.scala b/akka-stream/src/main/scala/akka/stream/QueueOfferResult.scala
index 95ac081734..d4913f2db9 100644
--- a/akka-stream/src/main/scala/akka/stream/QueueOfferResult.scala
+++ b/akka-stream/src/main/scala/akka/stream/QueueOfferResult.scala
@@ -14,7 +14,7 @@ object QueueOfferResult {
/**
* Type is used to indicate that stream is successfully enqueued an element
*/
- final case object Enqueued extends QueueOfferResult
+ case object Enqueued extends QueueOfferResult
/**
* Java API: The `Enqueued` singleton instance
@@ -24,7 +24,7 @@ object QueueOfferResult {
/**
* Type is used to indicate that stream is dropped an element
*/
- final case object Dropped extends QueueOfferResult
+ case object Dropped extends QueueOfferResult
/**
* Java API: The `Dropped` singleton instance
diff --git a/akka-stream/src/main/scala/akka/stream/Shape.scala b/akka-stream/src/main/scala/akka/stream/Shape.scala
index 8a4fa3fdea..9f62895e75 100644
--- a/akka-stream/src/main/scala/akka/stream/Shape.scala
+++ b/akka-stream/src/main/scala/akka/stream/Shape.scala
@@ -4,11 +4,12 @@
package akka.stream
-import akka.util.Collections.EmptyImmutableSeq
-import scala.collection.immutable
-import akka.util.ccompat.JavaConverters._
import scala.annotation.unchecked.uncheckedVariance
+import scala.collection.immutable
+
import akka.annotation.InternalApi
+import akka.util.Collections.EmptyImmutableSeq
+import akka.util.ccompat.JavaConverters._
/**
* An input port of a StreamLayout.Module. This type logically belongs
diff --git a/akka-stream/src/main/scala/akka/stream/SslTlsOptions.scala b/akka-stream/src/main/scala/akka/stream/SslTlsOptions.scala
index e0c3b1bf08..9961721efa 100644
--- a/akka-stream/src/main/scala/akka/stream/SslTlsOptions.scala
+++ b/akka-stream/src/main/scala/akka/stream/SslTlsOptions.scala
@@ -6,11 +6,11 @@ package akka.stream
import javax.net.ssl._
-import akka.util.ByteString
-
import scala.annotation.varargs
import scala.collection.immutable
+import akka.util.ByteString
+
/**
* Many protocols are asymmetric and distinguish between the client and the
* server, where the latter listens passively for messages and the former
diff --git a/akka-stream/src/main/scala/akka/stream/StreamRefSettings.scala b/akka-stream/src/main/scala/akka/stream/StreamRefSettings.scala
index 7bf96032e7..76f44ad5f0 100644
--- a/akka-stream/src/main/scala/akka/stream/StreamRefSettings.scala
+++ b/akka-stream/src/main/scala/akka/stream/StreamRefSettings.scala
@@ -6,13 +6,14 @@ package akka.stream
import java.util.concurrent.TimeUnit
-import akka.actor.ActorSystem
-import akka.annotation.DoNotInherit
-import akka.stream.impl.streamref.StreamRefSettingsImpl
+import scala.concurrent.duration._
+
import com.github.ghik.silencer.silent
import com.typesafe.config.Config
-import scala.concurrent.duration._
+import akka.actor.ActorSystem
+import akka.annotation.DoNotInherit
+import akka.stream.impl.streamref.StreamRefSettingsImpl
@silent("deprecated")
object StreamRefSettings {
diff --git a/akka-stream/src/main/scala/akka/stream/StreamRefs.scala b/akka-stream/src/main/scala/akka/stream/StreamRefs.scala
index 13b22d936a..2a780bb896 100644
--- a/akka-stream/src/main/scala/akka/stream/StreamRefs.scala
+++ b/akka-stream/src/main/scala/akka/stream/StreamRefs.scala
@@ -4,6 +4,8 @@
package akka.stream
+import scala.language.implicitConversions
+
import akka.NotUsed
import akka.actor.ActorRef
import akka.actor.ActorSystem
@@ -15,8 +17,6 @@ import akka.annotation.DoNotInherit
import akka.stream.impl.streamref.StreamRefResolverImpl
import akka.stream.scaladsl.{ Sink, Source }
-import scala.language.implicitConversions
-
/**
* See full documentation on [[SinkRef]].
*/
diff --git a/akka-stream/src/main/scala/akka/stream/SubscriptionWithCancelException.scala b/akka-stream/src/main/scala/akka/stream/SubscriptionWithCancelException.scala
index b05ced9a37..b7aacdff59 100644
--- a/akka-stream/src/main/scala/akka/stream/SubscriptionWithCancelException.scala
+++ b/akka-stream/src/main/scala/akka/stream/SubscriptionWithCancelException.scala
@@ -4,10 +4,11 @@
package akka.stream
-import akka.annotation.DoNotInherit
+import scala.util.control.NoStackTrace
+
import org.reactivestreams.Subscription
-import scala.util.control.NoStackTrace
+import akka.annotation.DoNotInherit
/**
* Extension of Subscription that allows to pass a cause when a subscription is cancelled.
diff --git a/akka-stream/src/main/scala/akka/stream/SubstreamCancelStrategy.scala b/akka-stream/src/main/scala/akka/stream/SubstreamCancelStrategy.scala
index c0587a5be4..701a66b083 100644
--- a/akka-stream/src/main/scala/akka/stream/SubstreamCancelStrategy.scala
+++ b/akka-stream/src/main/scala/akka/stream/SubstreamCancelStrategy.scala
@@ -16,12 +16,12 @@ private[akka] object SubstreamCancelStrategies {
/**
* INTERNAL API
*/
- private[akka] final case object Propagate extends SubstreamCancelStrategy
+ private[akka] case object Propagate extends SubstreamCancelStrategy
/**
* INTERNAL API
*/
- private[akka] final case object Drain extends SubstreamCancelStrategy
+ private[akka] case object Drain extends SubstreamCancelStrategy
}
object SubstreamCancelStrategy {
diff --git a/akka-stream/src/main/scala/akka/stream/SystemMaterializer.scala b/akka-stream/src/main/scala/akka/stream/SystemMaterializer.scala
index 2361432d7d..27b521bdf6 100644
--- a/akka-stream/src/main/scala/akka/stream/SystemMaterializer.scala
+++ b/akka-stream/src/main/scala/akka/stream/SystemMaterializer.scala
@@ -4,6 +4,11 @@
package akka.stream
+import scala.concurrent.Await
+import scala.concurrent.Promise
+
+import com.github.ghik.silencer.silent
+
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
import akka.actor.Deploy
@@ -13,14 +18,10 @@ import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
import akka.annotation.InternalApi
import akka.dispatch.Dispatchers
-import akka.stream.impl.MaterializerGuardian
-
-import scala.concurrent.Await
-import scala.concurrent.Promise
-import akka.util.JavaDurationConverters._
import akka.pattern.ask
+import akka.stream.impl.MaterializerGuardian
+import akka.util.JavaDurationConverters._
import akka.util.Timeout
-import com.github.ghik.silencer.silent
/**
* The system materializer is a default materializer to use for most cases running streams, it is a single instance
diff --git a/akka-stream/src/main/scala/akka/stream/UniformFanInShape.scala b/akka-stream/src/main/scala/akka/stream/UniformFanInShape.scala
index 585401fb2f..9ffac112d4 100644
--- a/akka-stream/src/main/scala/akka/stream/UniformFanInShape.scala
+++ b/akka-stream/src/main/scala/akka/stream/UniformFanInShape.scala
@@ -4,8 +4,8 @@
package akka.stream
-import scala.collection.immutable
import scala.annotation.unchecked.uncheckedVariance
+import scala.collection.immutable
object UniformFanInShape {
def apply[I, O](outlet: Outlet[O], inlets: Inlet[I]*): UniformFanInShape[I, O] =
diff --git a/akka-stream/src/main/scala/akka/stream/UniformFanOutShape.scala b/akka-stream/src/main/scala/akka/stream/UniformFanOutShape.scala
index 6658eb0db0..ae6fe73a8a 100644
--- a/akka-stream/src/main/scala/akka/stream/UniformFanOutShape.scala
+++ b/akka-stream/src/main/scala/akka/stream/UniformFanOutShape.scala
@@ -4,8 +4,8 @@
package akka.stream
-import scala.collection.immutable
import scala.annotation.unchecked.uncheckedVariance
+import scala.collection.immutable
object UniformFanOutShape {
def apply[I, O](inlet: Inlet[I], outlets: Outlet[O]*): UniformFanOutShape[I, O] =
diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorMaterializerImpl.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorMaterializerImpl.scala
index 2cc79371d9..482254c0ba 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/ActorMaterializerImpl.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/ActorMaterializerImpl.scala
@@ -6,6 +6,13 @@ package akka.stream.impl
import java.util.concurrent.atomic.AtomicBoolean
+import scala.collection.immutable
+import scala.concurrent.ExecutionContextExecutor
+import scala.concurrent.Future
+import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
+
import akka.actor._
import akka.annotation.DoNotInherit
import akka.annotation.InternalApi
@@ -20,12 +27,6 @@ import akka.stream.impl.fusing.GraphInterpreterShell
import akka.stream.snapshot.StreamSnapshot
import akka.util.OptionVal
import akka.util.Timeout
-import com.github.ghik.silencer.silent
-
-import scala.collection.immutable
-import scala.concurrent.duration._
-import scala.concurrent.ExecutionContextExecutor
-import scala.concurrent.Future
/**
* ExtendedActorMaterializer used by subtypes which delegates in-island wiring to [[akka.stream.impl.PhaseIsland]]s
@@ -225,10 +226,10 @@ private[akka] class SubFusingActorMaterializerImpl(
*/
@InternalApi private[akka] class StreamSupervisor(haveShutDown: AtomicBoolean) extends Actor {
import akka.stream.impl.StreamSupervisor._
- implicit val ec = context.dispatcher
+ implicit val ec: ExecutionContextExecutor = context.dispatcher
override def supervisorStrategy: SupervisorStrategy = SupervisorStrategy.stoppingStrategy
- def receive = {
+ def receive: Receive = {
case Materialize(props, name) =>
val impl = context.actorOf(props, name)
sender() ! impl
@@ -243,7 +244,6 @@ private[akka] class SubFusingActorMaterializerImpl(
}
def takeSnapshotsOfChildren(): Future[immutable.Seq[StreamSnapshot]] = {
- implicit val scheduler = context.system.scheduler
// Arbitrary timeout but should always be quick, the failure scenario is that
// the child/stream stopped, and we do retry below
implicit val timeout: Timeout = 1.second
@@ -255,7 +255,7 @@ private[akka] class SubFusingActorMaterializerImpl(
// If the timeout hits it is likely because one of the streams stopped between looking at the list
// of children and asking it for a snapshot. We retry the entire snapshot in that case
- retry(() => takeSnapshot(), 3, Duration.Zero)
+ retry(() => takeSnapshot(), 3)
}
override def postStop(): Unit = haveShutDown.set(true)
diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorProcessor.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorProcessor.scala
index cd3782bc6f..d548570e43 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/ActorProcessor.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/ActorProcessor.scala
@@ -4,13 +4,14 @@
package akka.stream.impl
+import org.reactivestreams.{ Processor, Subscriber, Subscription }
+
import akka.actor._
import akka.annotation.InternalApi
-import akka.stream.{ AbruptTerminationException, Attributes }
-import akka.stream.impl.ActorSubscriberMessage.{ OnComplete, OnError, OnNext, OnSubscribe }
-import org.reactivestreams.{ Processor, Subscriber, Subscription }
import akka.event.Logging
+import akka.stream.{ AbruptTerminationException, Attributes }
import akka.stream.ActorAttributes
+import akka.stream.impl.ActorSubscriberMessage.{ OnComplete, OnError, OnNext, OnSubscribe }
import akka.util.unused
/**
diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorPublisher.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorPublisher.scala
index 3d211826da..168a207aa8 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/ActorPublisher.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/ActorPublisher.scala
@@ -10,11 +10,12 @@ import scala.annotation.tailrec
import scala.collection.immutable
import scala.util.control.NoStackTrace
-import akka.actor.{ Actor, ActorRef, Terminated }
-import akka.annotation.InternalApi
import org.reactivestreams.{ Publisher, Subscriber }
import org.reactivestreams.Subscription
+import akka.actor.{ Actor, ActorRef, Terminated }
+import akka.annotation.InternalApi
+
/**
* INTERNAL API
*/
diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorRefBackpressureSinkStage.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorRefBackpressureSinkStage.scala
index ce5f460ba4..aea6e87c82 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/ActorRefBackpressureSinkStage.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/ActorRefBackpressureSinkStage.scala
@@ -8,9 +8,9 @@ import java.util
import akka.actor._
import akka.annotation.InternalApi
-import akka.stream.impl.Stages.DefaultAttributes
import akka.stream._
import akka.stream.Attributes.InputBuffer
+import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.stage._
/**
diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorRefSinkStage.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorRefSinkStage.scala
index 1227928412..6ab22c2993 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/ActorRefSinkStage.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/ActorRefSinkStage.scala
@@ -6,8 +6,8 @@ package akka.stream.impl
import akka.actor.{ ActorRef, Terminated }
import akka.annotation.InternalApi
-import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.{ AbruptStageTerminationException, Attributes, Inlet, SinkShape }
+import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, StageLogging }
/**
diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorRefSource.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorRefSource.scala
index 1d26120300..fcd8a7daa0 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/ActorRefSource.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/ActorRefSource.scala
@@ -6,8 +6,8 @@ package akka.stream.impl
import akka.actor.{ ActorRef, PoisonPill }
import akka.annotation.InternalApi
-import akka.stream.OverflowStrategies._
import akka.stream._
+import akka.stream.OverflowStrategies._
import akka.stream.stage._
import akka.util.OptionVal
diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorSubscriberMessage.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorSubscriberMessage.scala
index c198fbfe9a..60dbcaa937 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/ActorSubscriberMessage.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/ActorSubscriberMessage.scala
@@ -4,10 +4,11 @@
package akka.stream.impl
+import org.reactivestreams.Subscription
+
import akka.actor.DeadLetterSuppression
import akka.actor.NoSerializationVerificationNeeded
import akka.annotation.InternalApi
-import org.reactivestreams.Subscription
/**
* INTERNAL API
diff --git a/akka-stream/src/main/scala/akka/stream/impl/CompletedPublishers.scala b/akka-stream/src/main/scala/akka/stream/impl/CompletedPublishers.scala
index 3082f348d7..e2c7606790 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/CompletedPublishers.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/CompletedPublishers.scala
@@ -4,9 +4,10 @@
package akka.stream.impl
-import akka.annotation.InternalApi
import org.reactivestreams.{ Publisher, Subscriber, Subscription }
+import akka.annotation.InternalApi
+
/**
* INTERNAL API
*/
diff --git a/akka-stream/src/main/scala/akka/stream/impl/ConstantFun.scala b/akka-stream/src/main/scala/akka/stream/impl/ConstantFun.scala
index d7e32c8bc0..d758ef09be 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/ConstantFun.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/ConstantFun.scala
@@ -5,8 +5,8 @@
package akka.stream.impl
import akka.annotation.InternalApi
-import akka.japi.function.{ Function => JFun, Function2 => JFun2 }
import akka.japi.{ Pair => JPair }
+import akka.japi.function.{ Function => JFun, Function2 => JFun2 }
/**
* INTERNAL API
diff --git a/akka-stream/src/main/scala/akka/stream/impl/EmptySource.scala b/akka-stream/src/main/scala/akka/stream/impl/EmptySource.scala
index 1cfde5bf99..a40ec938d0 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/EmptySource.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/EmptySource.scala
@@ -12,7 +12,7 @@ import akka.stream.stage._
/**
* INTERNAL API
*/
-@InternalApi private[akka] final object EmptySource extends GraphStage[SourceShape[Nothing]] {
+@InternalApi private[akka] object EmptySource extends GraphStage[SourceShape[Nothing]] {
val out = Outlet[Nothing]("EmptySource.out")
override val shape = SourceShape(out)
diff --git a/akka-stream/src/main/scala/akka/stream/impl/FanIn.scala b/akka-stream/src/main/scala/akka/stream/impl/FanIn.scala
index b8ec41b199..920aad53dd 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/FanIn.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/FanIn.scala
@@ -4,13 +4,14 @@
package akka.stream.impl
+import org.reactivestreams.{ Subscriber, Subscription }
+
import akka.actor._
import akka.annotation.{ DoNotInherit, InternalApi }
+import akka.stream.AbruptTerminationException
import akka.stream.ActorAttributes
import akka.stream.Attributes
-import akka.stream.AbruptTerminationException
import akka.util.unused
-import org.reactivestreams.{ Subscriber, Subscription }
/**
* INTERNAL API
diff --git a/akka-stream/src/main/scala/akka/stream/impl/FanOut.scala b/akka-stream/src/main/scala/akka/stream/impl/FanOut.scala
index 0430e6e6e4..b3d0d846ab 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/FanOut.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/FanOut.scala
@@ -4,15 +4,16 @@
package akka.stream.impl
-import akka.stream.AbruptTerminationException
-
import scala.collection.immutable
+
+import org.reactivestreams.Subscription
+
import akka.actor._
import akka.annotation.{ DoNotInherit, InternalApi }
+import akka.stream.AbruptTerminationException
import akka.stream.ActorAttributes
import akka.stream.Attributes
import akka.util.unused
-import org.reactivestreams.Subscription
/**
* INTERNAL API
diff --git a/akka-stream/src/main/scala/akka/stream/impl/FanoutProcessor.scala b/akka-stream/src/main/scala/akka/stream/impl/FanoutProcessor.scala
index a0a034f5bc..39e9529611 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/FanoutProcessor.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/FanoutProcessor.scala
@@ -4,6 +4,8 @@
package akka.stream.impl
+import org.reactivestreams.Subscriber
+
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Deploy
@@ -12,7 +14,7 @@ import akka.annotation.InternalApi
import akka.stream.ActorAttributes.StreamSubscriptionTimeout
import akka.stream.Attributes
import akka.stream.StreamSubscriptionTimeoutTerminationMode
-import org.reactivestreams.Subscriber
+import akka.util.OptionVal
/**
* INTERNAL API
@@ -120,14 +122,11 @@ import org.reactivestreams.Subscriber
*/
@InternalApi private[akka] class FanoutProcessorImpl(attributes: Attributes) extends ActorProcessorImpl(attributes) {
- val timeoutMode = {
- val StreamSubscriptionTimeout(timeout, mode) = attributes.mandatoryAttribute[StreamSubscriptionTimeout]
- if (mode != StreamSubscriptionTimeoutTerminationMode.noop) {
- import context.dispatcher
- context.system.scheduler.scheduleOnce(timeout, self, ActorProcessorImpl.SubscriptionTimeout)
- }
- mode
- }
+ val StreamSubscriptionTimeout(timeout, timeoutMode) = attributes.mandatoryAttribute[StreamSubscriptionTimeout]
+ val timeoutTimer = if (timeoutMode != StreamSubscriptionTimeoutTerminationMode.noop) {
+ import context.dispatcher
+ OptionVal.Some(context.system.scheduler.scheduleOnce(timeout, self, ActorProcessorImpl.SubscriptionTimeout))
+ } else OptionVal.None
override val primaryOutputs: FanoutOutputs = {
val inputBuffer = attributes.mandatoryAttribute[Attributes.InputBuffer]
@@ -145,6 +144,14 @@ import org.reactivestreams.Subscriber
primaryOutputs.complete()
}
+ override def postStop(): Unit = {
+ super.postStop()
+ timeoutTimer match {
+ case OptionVal.Some(timer) => timer.cancel()
+ case _ =>
+ }
+ }
+
def afterFlush(): Unit = context.stop(self)
initialPhase(1, running)
diff --git a/akka-stream/src/main/scala/akka/stream/impl/JavaStreamSource.scala b/akka-stream/src/main/scala/akka/stream/impl/JavaStreamSource.scala
index 37ec2604e7..52cedecaa4 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/JavaStreamSource.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/JavaStreamSource.scala
@@ -4,9 +4,9 @@
package akka.stream.impl
+import akka.annotation.InternalApi
import akka.stream._
import akka.stream.stage.{ GraphStage, GraphStageLogic, OutHandler }
-import akka.annotation.InternalApi
/** INTERNAL API */
@InternalApi private[stream] final class JavaStreamSource[T, S <: java.util.stream.BaseStream[T, S]](
diff --git a/akka-stream/src/main/scala/akka/stream/impl/JsonObjectParser.scala b/akka-stream/src/main/scala/akka/stream/impl/JsonObjectParser.scala
index 7de77599d3..dd3a9486bc 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/JsonObjectParser.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/JsonObjectParser.scala
@@ -4,12 +4,12 @@
package akka.stream.impl
+import scala.annotation.switch
+
import akka.annotation.InternalApi
import akka.stream.scaladsl.Framing.FramingException
import akka.util.ByteString
-import scala.annotation.switch
-
/**
* INTERNAL API: Use [[akka.stream.scaladsl.JsonFraming]] instead.
*/
diff --git a/akka-stream/src/main/scala/akka/stream/impl/LazySource.scala b/akka-stream/src/main/scala/akka/stream/impl/LazySource.scala
index 2da7982d6a..586befd6a3 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/LazySource.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/LazySource.scala
@@ -4,15 +4,15 @@
package akka.stream.impl
+import scala.concurrent.{ Future, Promise }
+import scala.util.control.NonFatal
+
import akka.annotation.InternalApi
import akka.stream._
import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.scaladsl.{ Keep, Source }
import akka.stream.stage._
-import scala.concurrent.{ Future, Promise }
-import scala.util.control.NonFatal
-
/**
* INTERNAL API
*/
diff --git a/akka-stream/src/main/scala/akka/stream/impl/MaterializerGuardian.scala b/akka-stream/src/main/scala/akka/stream/impl/MaterializerGuardian.scala
index 0559ef2ab6..4bbfd918e6 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/MaterializerGuardian.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/MaterializerGuardian.scala
@@ -4,14 +4,15 @@
package akka.stream.impl
+import scala.concurrent.Promise
+
+import com.github.ghik.silencer.silent
+
import akka.actor.Actor
import akka.actor.Props
import akka.annotation.InternalApi
import akka.stream.ActorMaterializerSettings
import akka.stream.Materializer
-import com.github.ghik.silencer.silent
-
-import scala.concurrent.Promise
/**
* INTERNAL API
@@ -22,7 +23,7 @@ import scala.concurrent.Promise
@InternalApi
private[akka] object MaterializerGuardian {
- final case object StartMaterializer
+ case object StartMaterializer
final case class MaterializerStarted(materializer: Materializer)
// this is available to keep backwards compatibility with ActorMaterializer and should
diff --git a/akka-stream/src/main/scala/akka/stream/impl/MaybeSource.scala b/akka-stream/src/main/scala/akka/stream/impl/MaybeSource.scala
index b2d532b94e..98fdae89b5 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/MaybeSource.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/MaybeSource.scala
@@ -4,16 +4,16 @@
package akka.stream.impl
-import akka.annotation.InternalApi
-import akka.dispatch.ExecutionContexts
-import akka.stream.impl.Stages.DefaultAttributes
-import akka.stream.stage.{ GraphStageLogic, GraphStageWithMaterializedValue, OutHandler }
-import akka.stream._
-import akka.util.OptionVal
-
import scala.concurrent.Promise
import scala.util.Try
+import akka.annotation.InternalApi
+import akka.dispatch.ExecutionContexts
+import akka.stream._
+import akka.stream.impl.Stages.DefaultAttributes
+import akka.stream.stage.{ GraphStageLogic, GraphStageWithMaterializedValue, OutHandler }
+import akka.util.OptionVal
+
/**
* INTERNAL API
*/
diff --git a/akka-stream/src/main/scala/akka/stream/impl/Modules.scala b/akka-stream/src/main/scala/akka/stream/impl/Modules.scala
index ebe2dbb625..fbc1695645 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/Modules.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/Modules.scala
@@ -4,14 +4,15 @@
package akka.stream.impl
-import akka.NotUsed
-import akka.annotation.{ DoNotInherit, InternalApi }
-import akka.stream._
-import akka.stream.impl.StreamLayout.AtomicModule
+import scala.annotation.unchecked.uncheckedVariance
+
import org.reactivestreams._
-import scala.annotation.unchecked.uncheckedVariance
+import akka.NotUsed
+import akka.annotation.{ DoNotInherit, InternalApi }
import akka.event.Logging
+import akka.stream._
+import akka.stream.impl.StreamLayout.AtomicModule
/**
* INTERNAL API
diff --git a/akka-stream/src/main/scala/akka/stream/impl/PhasedFusingActorMaterializer.scala b/akka-stream/src/main/scala/akka/stream/impl/PhasedFusingActorMaterializer.scala
index 36fb7d20d1..9fea322785 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/PhasedFusingActorMaterializer.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/PhasedFusingActorMaterializer.scala
@@ -7,6 +7,16 @@ package akka.stream.impl
import java.util
import java.util.concurrent.atomic.AtomicBoolean
+import scala.collection.immutable.Map
+import scala.concurrent.ExecutionContextExecutor
+import scala.concurrent.ExecutionContextExecutor
+import scala.concurrent.duration.FiniteDuration
+
+import com.github.ghik.silencer.silent
+import org.reactivestreams.Processor
+import org.reactivestreams.Publisher
+import org.reactivestreams.Subscriber
+
import akka.NotUsed
import akka.actor.ActorContext
import akka.actor.ActorRef
@@ -21,30 +31,21 @@ import akka.annotation.InternalStableApi
import akka.dispatch.Dispatchers
import akka.event.Logging
import akka.event.LoggingAdapter
-import akka.stream.Attributes.InputBuffer
import akka.stream._
+import akka.stream.Attributes.InputBuffer
import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.impl.StreamLayout.AtomicModule
+import akka.stream.impl.fusing._
import akka.stream.impl.fusing.ActorGraphInterpreter.ActorOutputBoundary
import akka.stream.impl.fusing.ActorGraphInterpreter.BatchingActorInputBoundary
import akka.stream.impl.fusing.GraphInterpreter.Connection
-import akka.stream.impl.fusing._
import akka.stream.impl.io.TLSActor
import akka.stream.impl.io.TlsModule
import akka.stream.stage.GraphStageLogic
import akka.stream.stage.InHandler
import akka.stream.stage.OutHandler
import akka.util.OptionVal
-import org.reactivestreams.Processor
-import org.reactivestreams.Publisher
-import org.reactivestreams.Subscriber
-
-import scala.collection.immutable.Map
-import scala.concurrent.ExecutionContextExecutor
-import scala.concurrent.duration.FiniteDuration
-import scala.concurrent.ExecutionContextExecutor
import akka.util.OptionVal
-import com.github.ghik.silencer.silent
/**
* INTERNAL API
@@ -966,7 +967,7 @@ private final case class SavedIslandData(
val props =
TLSActor.props(maxInputBuffer, tls.createSSLEngine, tls.verifySession, tls.closing).withDispatcher(dispatcher)
- tlsActor = materializer.actorOf(props, islandName)
+ tlsActor = materializer.actorOf(props, "TLS-for-" + islandName)
def factory(id: Int) = new ActorPublisher[Any](tlsActor) {
override val wakeUpMsg = FanOut.SubstreamSubscribePending(id)
}
diff --git a/akka-stream/src/main/scala/akka/stream/impl/QueueSource.scala b/akka-stream/src/main/scala/akka/stream/impl/QueueSource.scala
index 6747ebea25..8606a314a0 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/QueueSource.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/QueueSource.scala
@@ -4,13 +4,14 @@
package akka.stream.impl
+import scala.concurrent.{ Future, Promise }
+
import akka.Done
import akka.annotation.InternalApi
-import akka.stream.OverflowStrategies._
import akka.stream._
-import akka.stream.stage._
+import akka.stream.OverflowStrategies._
import akka.stream.scaladsl.SourceQueueWithComplete
-import scala.concurrent.{ Future, Promise }
+import akka.stream.stage._
/**
* INTERNAL API
@@ -40,7 +41,7 @@ import scala.concurrent.{ Future, Promise }
override val shape: SourceShape[T] = SourceShape.of(out)
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
- val completion = Promise[Done]
+ val completion = Promise[Done]()
val name = inheritedAttributes.nameOrDefault(getClass.toString)
val stageLogic = new GraphStageLogic(shape) with OutHandler with SourceQueueWithComplete[T] with StageLogging {
@@ -209,7 +210,7 @@ import scala.concurrent.{ Future, Promise }
override def watchCompletion() = completion.future
override def offer(element: T): Future[QueueOfferResult] = {
- val p = Promise[QueueOfferResult]
+ val p = Promise[QueueOfferResult]()
callback
.invokeWithFeedback(Offer(element, p))
.onComplete {
diff --git a/akka-stream/src/main/scala/akka/stream/impl/ReactiveStreamsCompliance.scala b/akka-stream/src/main/scala/akka/stream/impl/ReactiveStreamsCompliance.scala
index 81033acd27..1bfe32f631 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/ReactiveStreamsCompliance.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/ReactiveStreamsCompliance.scala
@@ -4,12 +4,13 @@
package akka.stream.impl
+import scala.util.control.NonFatal
+
+import org.reactivestreams.{ Subscriber, Subscription }
+
import akka.annotation.InternalApi
import akka.stream.SubscriptionWithCancelException
-import scala.util.control.NonFatal
-import org.reactivestreams.{ Subscriber, Subscription }
-
/**
* INTERNAL API
*/
diff --git a/akka-stream/src/main/scala/akka/stream/impl/ResizableMultiReaderRingBuffer.scala b/akka-stream/src/main/scala/akka/stream/impl/ResizableMultiReaderRingBuffer.scala
index 68563948e9..4587b3bf38 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/ResizableMultiReaderRingBuffer.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/ResizableMultiReaderRingBuffer.scala
@@ -6,7 +6,9 @@ package akka.stream.impl
import scala.annotation.tailrec
import scala.util.control.NoStackTrace
+
import ResizableMultiReaderRingBuffer._
+
import akka.annotation.InternalApi
/**
diff --git a/akka-stream/src/main/scala/akka/stream/impl/RetryFlowCoordinator.scala b/akka-stream/src/main/scala/akka/stream/impl/RetryFlowCoordinator.scala
index 6fee0c8267..3f915a7144 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/RetryFlowCoordinator.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/RetryFlowCoordinator.scala
@@ -4,15 +4,15 @@
package akka.stream.impl
+import scala.concurrent.duration._
+
import akka.annotation.InternalApi
import akka.pattern.BackoffSupervisor
+import akka.stream.{ Attributes, BidiShape, Inlet, Outlet }
import akka.stream.SubscriptionWithCancelException.NonFailureCancellation
import akka.stream.stage._
-import akka.stream.{ Attributes, BidiShape, Inlet, Outlet }
import akka.util.OptionVal
-import scala.concurrent.duration._
-
/**
* INTERNAL API.
*
diff --git a/akka-stream/src/main/scala/akka/stream/impl/SetupStage.scala b/akka-stream/src/main/scala/akka/stream/impl/SetupStage.scala
index 71e6a8f052..dd3330b65b 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/SetupStage.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/SetupStage.scala
@@ -4,6 +4,10 @@
package akka.stream.impl
+import scala.concurrent.Future
+import scala.concurrent.Promise
+import scala.util.control.NonFatal
+
import akka.annotation.InternalApi
import akka.stream._
import akka.stream.scaladsl.Flow
@@ -15,10 +19,6 @@ import akka.stream.stage.GraphStageWithMaterializedValue
import akka.stream.stage.InHandler
import akka.stream.stage.OutHandler
-import scala.concurrent.Future
-import scala.concurrent.Promise
-import scala.util.control.NonFatal
-
/** Internal Api */
@InternalApi private[stream] final class SetupSinkStage[T, M](factory: (Materializer, Attributes) => Sink[T, M])
extends GraphStageWithMaterializedValue[SinkShape[T], Future[M]] {
@@ -27,7 +27,7 @@ import scala.util.control.NonFatal
override val shape = SinkShape(in)
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[M]) = {
- val matPromise = Promise[M]
+ val matPromise = Promise[M]()
(createStageLogic(matPromise), matPromise.future)
}
@@ -63,7 +63,7 @@ import scala.util.control.NonFatal
override val shape = FlowShape(in, out)
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[M]) = {
- val matPromise = Promise[M]
+ val matPromise = Promise[M]()
(createStageLogic(matPromise), matPromise.future)
}
@@ -106,7 +106,7 @@ import scala.util.control.NonFatal
override val shape = SourceShape(out)
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[M]) = {
- val matPromise = Promise[M]
+ val matPromise = Promise[M]()
(createStageLogic(matPromise), matPromise.future)
}
diff --git a/akka-stream/src/main/scala/akka/stream/impl/SinkholeSubscriber.scala b/akka-stream/src/main/scala/akka/stream/impl/SinkholeSubscriber.scala
index 4aac9ebd81..45f7d41ea6 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/SinkholeSubscriber.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/SinkholeSubscriber.scala
@@ -4,12 +4,13 @@
package akka.stream.impl
+import scala.concurrent.Promise
+
+import org.reactivestreams.{ Subscriber, Subscription }
+
import akka.Done
import akka.annotation.InternalApi
-import scala.concurrent.Promise
-import org.reactivestreams.{ Subscriber, Subscription }
-
/**
* INTERNAL API
*/
diff --git a/akka-stream/src/main/scala/akka/stream/impl/Sinks.scala b/akka-stream/src/main/scala/akka/stream/impl/Sinks.scala
index 158d6ac40a..3e42f9d872 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/Sinks.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/Sinks.scala
@@ -6,26 +6,6 @@ package akka.stream.impl
import java.util.function.BinaryOperator
-import akka.NotUsed
-import akka.annotation.DoNotInherit
-import akka.annotation.InternalApi
-import akka.dispatch.ExecutionContexts
-import akka.event.Logging
-import akka.stream.ActorAttributes.StreamSubscriptionTimeout
-import akka.stream.Attributes.InputBuffer
-import akka.stream._
-import akka.stream.impl.QueueSink.Output
-import akka.stream.impl.QueueSink.Pull
-import akka.stream.impl.Stages.DefaultAttributes
-import akka.stream.impl.StreamLayout.AtomicModule
-import akka.stream.scaladsl.Sink
-import akka.stream.scaladsl.SinkQueueWithCancel
-import akka.stream.scaladsl.Source
-import akka.stream.stage._
-import akka.util.ccompat._
-import org.reactivestreams.Publisher
-import org.reactivestreams.Subscriber
-
import scala.annotation.unchecked.uncheckedVariance
import scala.collection.immutable
import scala.collection.mutable
@@ -36,6 +16,27 @@ import scala.util.Success
import scala.util.Try
import scala.util.control.NonFatal
+import org.reactivestreams.Publisher
+import org.reactivestreams.Subscriber
+
+import akka.NotUsed
+import akka.annotation.DoNotInherit
+import akka.annotation.InternalApi
+import akka.dispatch.ExecutionContexts
+import akka.event.Logging
+import akka.stream._
+import akka.stream.ActorAttributes.StreamSubscriptionTimeout
+import akka.stream.Attributes.InputBuffer
+import akka.stream.impl.QueueSink.Output
+import akka.stream.impl.QueueSink.Pull
+import akka.stream.impl.Stages.DefaultAttributes
+import akka.stream.impl.StreamLayout.AtomicModule
+import akka.stream.scaladsl.Sink
+import akka.stream.scaladsl.SinkQueueWithCancel
+import akka.stream.scaladsl.Source
+import akka.stream.stage._
+import akka.util.ccompat._
+
/**
* INTERNAL API
*/
@@ -391,7 +392,7 @@ import scala.util.control.NonFatal
// SinkQueueWithCancel impl
override def pull(): Future[Option[T]] = {
- val p = Promise[Option[T]]
+ val p = Promise[Option[T]]()
callback
.invokeWithFeedback(Pull(p))
.failed
diff --git a/akka-stream/src/main/scala/akka/stream/impl/Stages.scala b/akka-stream/src/main/scala/akka/stream/impl/Stages.scala
index 0aa4719579..1cbfc50e23 100755
--- a/akka-stream/src/main/scala/akka/stream/impl/Stages.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/Stages.scala
@@ -5,8 +5,8 @@
package akka.stream.impl
import akka.annotation.InternalApi
-import akka.stream.Attributes._
import akka.stream._
+import akka.stream.Attributes._
/**
* INTERNAL API
@@ -104,6 +104,7 @@ import akka.stream._
val singleSource = name("singleSource")
val emptySource = name("emptySource")
val maybeSource = name("MaybeSource")
+ val neverSource = name("neverSource")
val failedSource = name("failedSource")
val concatSource = name("concatSource")
val concatMatSource = name("concatMatSource")
diff --git a/akka-stream/src/main/scala/akka/stream/impl/StreamLayout.scala b/akka-stream/src/main/scala/akka/stream/impl/StreamLayout.scala
index 539be980c6..faaad6c797 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/StreamLayout.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/StreamLayout.scala
@@ -6,17 +6,18 @@ package akka.stream.impl
import java.util.concurrent.atomic.AtomicReference
-import akka.annotation.InternalApi
-import akka.stream._
-import akka.stream.impl.Stages.DefaultAttributes
-import akka.util.OptionVal
+import scala.annotation.tailrec
+import scala.util.control.NonFatal
+
import org.reactivestreams.Processor
import org.reactivestreams.Publisher
import org.reactivestreams.Subscriber
import org.reactivestreams.Subscription
-import scala.annotation.tailrec
-import scala.util.control.NonFatal
+import akka.annotation.InternalApi
+import akka.stream._
+import akka.stream.impl.Stages.DefaultAttributes
+import akka.util.OptionVal
/**
* INTERNAL API
diff --git a/akka-stream/src/main/scala/akka/stream/impl/StreamSubscriptionTimeout.scala b/akka-stream/src/main/scala/akka/stream/impl/StreamSubscriptionTimeout.scala
index b62da2fd4a..c4361cff32 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/StreamSubscriptionTimeout.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/StreamSubscriptionTimeout.scala
@@ -4,15 +4,16 @@
package akka.stream.impl
-import akka.actor._
-import akka.annotation.InternalApi
-import akka.stream.StreamSubscriptionTimeoutTerminationMode.{ CancelTermination, NoopTermination, WarnTermination }
-import akka.stream.StreamSubscriptionTimeoutSettings
+import scala.concurrent.duration.FiniteDuration
+import scala.util.control.NoStackTrace
+
import com.github.ghik.silencer.silent
import org.reactivestreams._
-import scala.concurrent.duration.FiniteDuration
-import scala.util.control.NoStackTrace
+import akka.actor._
+import akka.annotation.InternalApi
+import akka.stream.StreamSubscriptionTimeoutSettings
+import akka.stream.StreamSubscriptionTimeoutTerminationMode.{ CancelTermination, NoopTermination, WarnTermination }
/**
* INTERNAL API
diff --git a/akka-stream/src/main/scala/akka/stream/impl/SubscriberManagement.scala b/akka-stream/src/main/scala/akka/stream/impl/SubscriberManagement.scala
index c3dfda54d8..f45ed42485 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/SubscriberManagement.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/SubscriberManagement.scala
@@ -5,8 +5,9 @@
package akka.stream.impl
import scala.annotation.tailrec
-import org.reactivestreams.{ Subscriber, Subscription }
+
import SubscriberManagement.ShutDown
+import org.reactivestreams.{ Subscriber, Subscription }
/**
* INTERNAL API
diff --git a/akka-stream/src/main/scala/akka/stream/impl/Throttle.scala b/akka-stream/src/main/scala/akka/stream/impl/Throttle.scala
index f438f98e91..1e98dae943 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/Throttle.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/Throttle.scala
@@ -4,15 +4,15 @@
package akka.stream.impl
+import scala.concurrent.duration.{ FiniteDuration, _ }
+
import akka.annotation.InternalApi
+import akka.stream._
import akka.stream.ThrottleMode.{ Enforcing, Shaping }
import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
import akka.stream.stage._
-import akka.stream._
import akka.util.NanoTimeTokenBucket
-import scala.concurrent.duration.{ FiniteDuration, _ }
-
/**
* INTERNAL API
*/
diff --git a/akka-stream/src/main/scala/akka/stream/impl/Timers.scala b/akka-stream/src/main/scala/akka/stream/impl/Timers.scala
index 22cb1c9da0..917fbfcb5a 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/Timers.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/Timers.scala
@@ -6,14 +6,14 @@ package akka.stream.impl
import java.util.concurrent.{ TimeUnit, TimeoutException }
+import scala.concurrent.duration.{ Duration, FiniteDuration }
+
import akka.annotation.InternalApi
import akka.stream._
import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
import akka.stream.stage._
-import scala.concurrent.duration.{ Duration, FiniteDuration }
-
/**
* INTERNAL API
*
diff --git a/akka-stream/src/main/scala/akka/stream/impl/Transfer.scala b/akka-stream/src/main/scala/akka/stream/impl/Transfer.scala
index fa34bc8861..f8e71eacfb 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/Transfer.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/Transfer.scala
@@ -5,6 +5,7 @@
package akka.stream.impl
import scala.util.control.NonFatal
+
import akka.actor.Actor
import akka.annotation.InternalApi
diff --git a/akka-stream/src/main/scala/akka/stream/impl/TraversalBuilder.scala b/akka-stream/src/main/scala/akka/stream/impl/TraversalBuilder.scala
index 7f9575d0d7..49bb4b7d59 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/TraversalBuilder.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/TraversalBuilder.scala
@@ -4,17 +4,17 @@
package akka.stream.impl
+import scala.collection.immutable.Map.Map1
+import scala.language.existentials
+
import akka.annotation.{ DoNotInherit, InternalApi }
import akka.stream._
import akka.stream.impl.StreamLayout.AtomicModule
import akka.stream.impl.TraversalBuilder.{ AnyFunction1, AnyFunction2 }
-import akka.stream.scaladsl.Keep
-import akka.util.OptionVal
-
-import scala.language.existentials
-import scala.collection.immutable.Map.Map1
import akka.stream.impl.fusing.GraphStageModule
import akka.stream.impl.fusing.GraphStages.SingleSource
+import akka.stream.scaladsl.Keep
+import akka.util.OptionVal
import akka.util.unused
/**
@@ -180,7 +180,7 @@ import akka.util.unused
/**
* INTERNAL API
*/
-@InternalApi private[akka] final case object PopAttributes extends Traversal
+@InternalApi private[akka] case object PopAttributes extends Traversal
/**
* INTERNAL API
@@ -190,7 +190,7 @@ import akka.util.unused
/**
* INTERNAL API
*/
-@InternalApi private[akka] final case object ExitIsland extends Traversal
+@InternalApi private[akka] case object ExitIsland extends Traversal
/**
* INTERNAL API
diff --git a/akka-stream/src/main/scala/akka/stream/impl/Unfold.scala b/akka-stream/src/main/scala/akka/stream/impl/Unfold.scala
index 896be90077..6b97314137 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/Unfold.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/Unfold.scala
@@ -4,14 +4,14 @@
package akka.stream.impl
-import akka.annotation.InternalApi
-import akka.stream.impl.Stages.DefaultAttributes
-import akka.stream.stage.{ GraphStage, GraphStageLogic, OutHandler }
-import akka.stream._
-
import scala.concurrent.Future
import scala.util.{ Failure, Success, Try }
+import akka.annotation.InternalApi
+import akka.stream._
+import akka.stream.impl.Stages.DefaultAttributes
+import akka.stream.stage.{ GraphStage, GraphStageLogic, OutHandler }
+
/**
* INTERNAL API
*/
diff --git a/akka-stream/src/main/scala/akka/stream/impl/UnfoldResourceSource.scala b/akka-stream/src/main/scala/akka/stream/impl/UnfoldResourceSource.scala
index 2a7b466446..7d66e41460 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/UnfoldResourceSource.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/UnfoldResourceSource.scala
@@ -4,15 +4,15 @@
package akka.stream.impl
-import akka.annotation.InternalApi
-import akka.stream.ActorAttributes.SupervisionStrategy
-import akka.stream._
-import akka.stream.impl.Stages.DefaultAttributes
-import akka.stream.stage._
-
import scala.annotation.tailrec
import scala.util.control.NonFatal
+import akka.annotation.InternalApi
+import akka.stream._
+import akka.stream.ActorAttributes.SupervisionStrategy
+import akka.stream.impl.Stages.DefaultAttributes
+import akka.stream.stage._
+
/**
* INTERNAL API
*/
diff --git a/akka-stream/src/main/scala/akka/stream/impl/UnfoldResourceSourceAsync.scala b/akka-stream/src/main/scala/akka/stream/impl/UnfoldResourceSourceAsync.scala
index 3feb338de6..9b0fce64c3 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/UnfoldResourceSourceAsync.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/UnfoldResourceSourceAsync.scala
@@ -4,18 +4,18 @@
package akka.stream.impl
-import akka.Done
-import akka.annotation.InternalApi
-import akka.dispatch.ExecutionContexts.parasitic
-import akka.stream.ActorAttributes.SupervisionStrategy
-import akka.stream._
-import akka.stream.impl.Stages.DefaultAttributes
-import akka.stream.stage._
-
import scala.concurrent.Future
import scala.util.{ Failure, Success, Try }
import scala.util.control.NonFatal
+import akka.Done
+import akka.annotation.InternalApi
+import akka.dispatch.ExecutionContexts.parasitic
+import akka.stream._
+import akka.stream.ActorAttributes.SupervisionStrategy
+import akka.stream.impl.Stages.DefaultAttributes
+import akka.stream.stage._
+
/**
* INTERNAL API
*/
diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/ActorGraphInterpreter.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/ActorGraphInterpreter.scala
index 102330ee75..5d83ca3e94 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/fusing/ActorGraphInterpreter.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/ActorGraphInterpreter.scala
@@ -8,31 +8,32 @@ import java.util
import java.util.concurrent.TimeoutException
import java.util.concurrent.atomic.AtomicReference
+import scala.annotation.tailrec
+import scala.collection.immutable
+import scala.concurrent.Promise
+import scala.util.control.NonFatal
+
+import org.reactivestreams.Publisher
+import org.reactivestreams.Subscriber
+import org.reactivestreams.Subscription
+
import akka.Done
import akka.actor._
import akka.annotation.InternalApi
import akka.annotation.InternalStableApi
import akka.event.Logging
import akka.stream._
+import akka.stream.impl._
import akka.stream.impl.ReactiveStreamsCompliance._
+import akka.stream.impl.SubFusingActorMaterializerImpl
import akka.stream.impl.fusing.GraphInterpreter.Connection
import akka.stream.impl.fusing.GraphInterpreter.DownstreamBoundaryStageLogic
import akka.stream.impl.fusing.GraphInterpreter.UpstreamBoundaryStageLogic
-import akka.stream.impl.SubFusingActorMaterializerImpl
-import akka.stream.impl._
import akka.stream.snapshot._
import akka.stream.stage.GraphStageLogic
import akka.stream.stage.InHandler
import akka.stream.stage.OutHandler
import akka.util.OptionVal
-import org.reactivestreams.Publisher
-import org.reactivestreams.Subscriber
-import org.reactivestreams.Subscription
-
-import scala.annotation.tailrec
-import scala.collection.immutable
-import scala.concurrent.Promise
-import scala.util.control.NonFatal
/**
* INTERNAL API
diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/FlatMapPrefix.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/FlatMapPrefix.scala
index bf04363bba..80f005093c 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/fusing/FlatMapPrefix.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/FlatMapPrefix.scala
@@ -4,17 +4,17 @@
package akka.stream.impl.fusing
-import akka.annotation.InternalApi
-import akka.stream.scaladsl.{ Flow, Keep, Source }
-import akka.stream.stage.{ GraphStageLogic, GraphStageWithMaterializedValue, InHandler, OutHandler }
-import akka.stream._
-import akka.stream.impl.Stages.DefaultAttributes
-import akka.util.OptionVal
-
import scala.collection.immutable
import scala.concurrent.{ Future, Promise }
import scala.util.control.NonFatal
+import akka.annotation.InternalApi
+import akka.stream._
+import akka.stream.impl.Stages.DefaultAttributes
+import akka.stream.scaladsl.{ Flow, Keep, Source }
+import akka.stream.stage.{ GraphStageLogic, GraphStageWithMaterializedValue, InHandler, OutHandler }
+import akka.util.OptionVal
+
@InternalApi private[akka] final class FlatMapPrefix[In, Out, M](n: Int, f: immutable.Seq[In] => Flow[In, Out, M])
extends GraphStageWithMaterializedValue[FlowShape[In, Out], Future[M]] {
@@ -27,6 +27,10 @@ import scala.util.control.NonFatal
override def initialAttributes: Attributes = DefaultAttributes.flatMapPrefix
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[M]) = {
+ val propagateToNestedMaterialization =
+ inheritedAttributes
+ .mandatoryAttribute[Attributes.NestedMaterializationCancellationPolicy]
+ .propagateToNestedMaterialization
val matPromise = Promise[M]
val logic = new GraphStageLogic(shape) with InHandler with OutHandler {
val accumulated = collection.mutable.Buffer.empty[In]
@@ -90,7 +94,10 @@ import scala.util.control.NonFatal
override def onDownstreamFinish(cause: Throwable): Unit = {
subSink match {
- case OptionVal.None => downstreamCause = OptionVal.Some(cause)
+ case OptionVal.None if propagateToNestedMaterialization => downstreamCause = OptionVal.Some(cause)
+ case OptionVal.None =>
+ matPromise.failure(new NeverMaterializedException(cause))
+ cancelStage(cause)
case OptionVal.Some(s) => s.cancel(cause)
}
}
diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/FutureFlow.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/FutureFlow.scala
new file mode 100644
index 0000000000..8dfd00b1b7
--- /dev/null
+++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/FutureFlow.scala
@@ -0,0 +1,147 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.stream.impl.fusing
+
+import akka.annotation.InternalApi
+import akka.dispatch.ExecutionContexts
+import akka.stream.{ AbruptStageTerminationException, Attributes, FlowShape, Inlet, NeverMaterializedException, Outlet }
+import akka.stream.scaladsl.{ Flow, Keep, Source }
+import akka.stream.stage.{ GraphStageLogic, GraphStageWithMaterializedValue, InHandler, OutHandler }
+import akka.util.OptionVal
+
+import scala.concurrent.{ Future, Promise }
+import scala.util.{ Failure, Success, Try }
+
+@InternalApi private[akka] final class FutureFlow[In, Out, M](futureFlow: Future[Flow[In, Out, M]])
+ extends GraphStageWithMaterializedValue[FlowShape[In, Out], Future[M]] {
+ val in = Inlet[In](s"${this}.in")
+ val out = Outlet[Out](s"${this}.out")
+ override val shape: FlowShape[In, Out] = FlowShape(in, out)
+
+ override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[M]) = {
+ val propagateToNestedMaterialization =
+ inheritedAttributes
+ .mandatoryAttribute[Attributes.NestedMaterializationCancellationPolicy]
+ .propagateToNestedMaterialization
+ val innerMatValue = Promise[M]
+ val logic = new GraphStageLogic(shape) {
+
+ //seems like we must set handlers BEFORE preStart
+ setHandlers(in, out, Initializing)
+
+ override def preStart(): Unit = {
+ futureFlow.value match {
+ case Some(tryFlow) =>
+ Initializing.onFuture(tryFlow)
+ case None =>
+ val cb = getAsyncCallback(Initializing.onFuture)
+ futureFlow.onComplete(cb.invoke)(ExecutionContexts.parasitic)
+ //in case both ports are closed before future completion
+ setKeepGoing(true)
+ }
+ }
+
+ override def postStop(): Unit = {
+ if (!innerMatValue.isCompleted) {
+ innerMatValue.failure(new AbruptStageTerminationException(this))
+ }
+ }
+
+ object Initializing extends InHandler with OutHandler {
+ // we don't expect a push since we bever pull upstream during initialization
+ override def onPush(): Unit = throw new IllegalStateException("unexpected push during initialization")
+
+ var upstreamFailure = OptionVal.none[Throwable]
+
+ override def onUpstreamFailure(ex: Throwable): Unit = {
+ upstreamFailure = OptionVal.Some(ex)
+ }
+
+ //will later be propagated to the materialized flow (by examining isClosed(in))
+ override def onUpstreamFinish(): Unit = {}
+
+ //will later be propagated to the materialized flow (by examining isAvailable(out))
+ override def onPull(): Unit = {}
+
+ var downstreamCause = OptionVal.none[Throwable]
+
+ override def onDownstreamFinish(cause: Throwable): Unit =
+ if (propagateToNestedMaterialization) {
+ downstreamCause = OptionVal.Some(cause)
+ } else {
+ innerMatValue.failure(new NeverMaterializedException(cause))
+ cancelStage(cause)
+ }
+
+ def onFuture(futureRes: Try[Flow[In, Out, M]]) = futureRes match {
+ case Failure(exception) =>
+ setKeepGoing(false)
+ innerMatValue.failure(new NeverMaterializedException(exception))
+ failStage(exception)
+ case Success(flow) =>
+ //materialize flow, connect inlet and outlet, feed with potential events and set handlers
+ connect(flow)
+ setKeepGoing(false)
+ }
+
+ def connect(flow: Flow[In, Out, M]): Unit = {
+ val subSource = new SubSourceOutlet[In](s"${FutureFlow.this}.subIn")
+ val subSink = new SubSinkInlet[Out](s"${FutureFlow.this}.subOut")
+
+ subSource.setHandler {
+ new OutHandler {
+ override def onPull(): Unit = if (!isClosed(in)) tryPull(in)
+
+ override def onDownstreamFinish(cause: Throwable): Unit = if (!isClosed(in)) cancel(in, cause)
+ }
+ }
+ subSink.setHandler {
+ new InHandler {
+ override def onPush(): Unit = push(out, subSink.grab())
+
+ override def onUpstreamFinish(): Unit = complete(out)
+
+ override def onUpstreamFailure(ex: Throwable): Unit = fail(out, ex)
+ }
+ }
+ Try {
+ Source.fromGraph(subSource.source).viaMat(flow)(Keep.right).to(subSink.sink).run()(subFusingMaterializer)
+ } match {
+ case Success(matVal) =>
+ innerMatValue.success(matVal)
+ upstreamFailure match {
+ case OptionVal.Some(ex) =>
+ subSource.fail(ex)
+ case OptionVal.None =>
+ if (isClosed(in))
+ subSource.complete()
+ }
+ downstreamCause match {
+ case OptionVal.Some(cause) =>
+ subSink.cancel(cause)
+ case OptionVal.None =>
+ if (isAvailable(out)) subSink.pull()
+ }
+ setHandlers(in, out, new InHandler with OutHandler {
+ override def onPull(): Unit = subSink.pull()
+
+ override def onDownstreamFinish(cause: Throwable): Unit = subSink.cancel(cause)
+
+ override def onPush(): Unit = subSource.push(grab(in))
+
+ override def onUpstreamFinish(): Unit = subSource.complete()
+
+ override def onUpstreamFailure(ex: Throwable): Unit = subSource.fail(ex)
+ })
+ case Failure(ex) =>
+ innerMatValue.failure(new NeverMaterializedException(ex))
+ failStage(ex)
+ }
+ }
+ }
+ }
+ (logic, innerMatValue.future)
+ }
+}
diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphInterpreter.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphInterpreter.scala
index 2e9b8f96bb..cfff2a5ca3 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphInterpreter.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphInterpreter.scala
@@ -4,19 +4,19 @@
package akka.stream.impl.fusing
-import akka.actor.ActorRef
-import akka.event.LoggingAdapter
-import akka.stream.stage._
-import akka.stream._
import java.util.concurrent.ThreadLocalRandom
-import akka.Done
-import akka.annotation.{ InternalApi, InternalStableApi }
-
import scala.concurrent.Promise
import scala.util.control.NonFatal
+
+import akka.Done
+import akka.actor.ActorRef
+import akka.annotation.{ InternalApi, InternalStableApi }
+import akka.event.LoggingAdapter
+import akka.stream._
import akka.stream.Attributes.LogLevels
import akka.stream.snapshot._
+import akka.stream.stage._
/**
* INTERNAL API
diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphStages.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphStages.scala
index 47ceb941a2..dd8b8cd5f1 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphStages.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphStages.scala
@@ -6,23 +6,23 @@ package akka.stream.impl.fusing
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicReference }
+import scala.annotation.unchecked.uncheckedVariance
+import scala.concurrent.{ Future, Promise }
+import scala.concurrent.duration.FiniteDuration
+import scala.util.Try
+
import akka.Done
import akka.actor.Cancellable
import akka.annotation.InternalApi
import akka.dispatch.ExecutionContexts
import akka.event.Logging
+import akka.stream.{ Shape, _ }
import akka.stream.FlowMonitorState._
+import akka.stream.impl.{ LinearTraversalBuilder, ReactiveStreamsCompliance }
import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.impl.StreamLayout._
-import akka.stream.impl.{ LinearTraversalBuilder, ReactiveStreamsCompliance }
import akka.stream.scaladsl._
import akka.stream.stage._
-import akka.stream.{ Shape, _ }
-
-import scala.annotation.unchecked.uncheckedVariance
-import scala.util.Try
-import scala.concurrent.duration.FiniteDuration
-import scala.concurrent.{ Future, Promise }
/**
* INTERNAL API
diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala
index 934ed08262..7339cd67b7 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala
@@ -6,33 +6,32 @@ package akka.stream.impl.fusing
import java.util.concurrent.TimeUnit.NANOSECONDS
-import akka.actor.{ ActorRef, Terminated }
-import akka.annotation.{ DoNotInherit, InternalApi }
-import akka.dispatch.ExecutionContexts
-import akka.event.Logging.LogLevel
-import akka.event.{ LogMarker, LogSource, Logging, LoggingAdapter, MarkerLoggingAdapter }
-import akka.stream.Attributes.{ InputBuffer, LogLevels }
-import akka.stream.OverflowStrategies._
-import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
-import akka.stream.impl.{ ReactiveStreamsCompliance, Buffer => BufferImpl }
-import akka.stream.scaladsl.{ DelayStrategy, Flow, Keep, Source }
-import akka.stream.stage._
-import akka.stream.{ Supervision, _ }
-
import scala.annotation.tailrec
import scala.collection.immutable
import scala.collection.immutable.VectorBuilder
-import scala.concurrent.{ Future, Promise }
-import scala.util.control.{ NoStackTrace, NonFatal }
-import scala.util.{ Failure, Success, Try }
-import akka.stream.ActorAttributes.SupervisionStrategy
-
+import scala.concurrent.Future
import scala.concurrent.duration.{ FiniteDuration, _ }
+import scala.util.{ Failure, Success, Try }
+import scala.util.control.{ NoStackTrace, NonFatal }
import scala.util.control.Exception.Catcher
+
+import com.github.ghik.silencer.silent
+
+import akka.actor.{ ActorRef, Terminated }
+import akka.annotation.{ DoNotInherit, InternalApi }
+import akka.event.{ LogMarker, LogSource, Logging, LoggingAdapter, MarkerLoggingAdapter }
+import akka.event.Logging.LogLevel
+import akka.stream.{ Supervision, _ }
+import akka.stream.ActorAttributes.SupervisionStrategy
+import akka.stream.Attributes.{ InputBuffer, LogLevels }
+import akka.stream.OverflowStrategies._
+import akka.stream.impl.{ ReactiveStreamsCompliance, Buffer => BufferImpl }
import akka.stream.impl.Stages.DefaultAttributes
+import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
+import akka.stream.scaladsl.{ DelayStrategy, Source }
+import akka.stream.stage._
import akka.util.OptionVal
import akka.util.unused
-import com.github.ghik.silencer.silent
/**
* INTERNAL API
@@ -2224,199 +2223,3 @@ private[stream] object Collect {
override def toString = "StatefulMapConcat"
}
-
-/**
- * INTERNAL API
- */
-@InternalApi private[akka] final class LazyFlow[I, O, M](flowFactory: I => Future[Flow[I, O, M]])
- extends GraphStageWithMaterializedValue[FlowShape[I, O], Future[M]] {
-
- // FIXME: when removing the deprecated I => Flow factories we can remove that complication from this stage
-
- val in = Inlet[I]("LazyFlow.in")
- val out = Outlet[O]("LazyFlow.out")
-
- override def initialAttributes = DefaultAttributes.lazyFlow
-
- override val shape: FlowShape[I, O] = FlowShape.of(in, out)
-
- override def toString: String = "LazyFlow"
-
- override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[M]) = {
- val matPromise = Promise[M]()
- val stageLogic = new GraphStageLogic(shape) with InHandler with OutHandler {
- var switching = false
-
- //
- // implementation of handler methods in initial state
- //
- private def onFlowFutureComplete(firstElement: I)(result: Try[Flow[I, O, M]]) = result match {
- case Success(flow) =>
- // check if the stage is still in need for the lazy flow
- // (there could have been an onUpstreamFailure or onDownstreamFinish in the meantime that has completed the promise)
- if (!matPromise.isCompleted) {
- try {
- val mat = switchTo(flow, firstElement)
- matPromise.success(mat)
- } catch {
- case NonFatal(e) =>
- matPromise.failure(e)
- failStage(e)
- }
- }
- case Failure(e) =>
- matPromise.failure(e)
- failStage(e)
- }
-
- override def onPush(): Unit =
- try {
- val element = grab(in)
- switching = true
- val futureFlow = flowFactory(element)
-
- // optimization avoid extra scheduling if already completed
- futureFlow.value match {
- case Some(completed) =>
- onFlowFutureComplete(element)(completed)
- case None =>
- val cb = getAsyncCallback[Try[Flow[I, O, M]]](onFlowFutureComplete(element))
- futureFlow.onComplete(cb.invoke)(ExecutionContexts.parasitic)
- }
- } catch {
- case NonFatal(e) =>
- matPromise.failure(e)
- failStage(e)
- }
-
- override def onUpstreamFinish(): Unit = {
- if (!matPromise.isCompleted)
- matPromise.tryFailure(new NeverMaterializedException)
- // ignore onUpstreamFinish while the stage is switching but setKeepGoing
- if (switching) {
- setKeepGoing(true)
- } else {
- super.onUpstreamFinish()
- }
- }
-
- override def onUpstreamFailure(ex: Throwable): Unit = {
- super.onUpstreamFailure(ex)
- if (!matPromise.isCompleted)
- matPromise.tryFailure(new NeverMaterializedException(ex))
- }
-
- override def onPull(): Unit = {
- pull(in)
- }
-
- override def postStop(): Unit = {
- if (!matPromise.isCompleted)
- matPromise.tryFailure(new AbruptStageTerminationException(this))
- }
-
- setHandler(in, this)
- setHandler(out, this)
-
- private def switchTo(flow: Flow[I, O, M], firstElement: I): M = {
-
- //
- // ports are wired in the following way:
- //
- // in ~> subOutlet ~> lazyFlow ~> subInlet ~> out
- //
-
- val subInlet = new SubSinkInlet[O]("LazyFlowSubSink")
- val subOutlet = new SubSourceOutlet[I]("LazyFlowSubSource")
-
- val matVal = Source
- .fromGraph(subOutlet.source)
- .prepend(Source.single(firstElement))
- .viaMat(flow)(Keep.right)
- .toMat(subInlet.sink)(Keep.left)
- .run()(interpreter.subFusingMaterializer)
-
- // The lazily materialized flow may be constructed from a sink and a source. Therefore termination
- // signals (completion, cancellation, and errors) are not guaranteed to pass through the flow. This
- // means that this stage must not be completed as soon as one side of the flow is finished.
- //
- // Invariant: isClosed(out) == subInlet.isClosed after each event because termination signals (i.e.
- // completion, cancellation, and failure) between these two ports are always forwarded.
- //
- // However, isClosed(in) and subOutlet.isClosed may be different. This happens if upstream completes before
- // the cached element was pushed.
- def maybeCompleteStage(): Unit = {
- if (isClosed(in) && subOutlet.isClosed && isClosed(out)) {
- completeStage()
- }
- }
-
- // The stage must not be shut down automatically; it is completed when maybeCompleteStage decides
- setKeepGoing(true)
-
- setHandler(
- in,
- new InHandler {
- override def onPush(): Unit = {
- subOutlet.push(grab(in))
- }
- override def onUpstreamFinish(): Unit = {
- subOutlet.complete()
- maybeCompleteStage()
- }
- override def onUpstreamFailure(ex: Throwable): Unit = {
- // propagate exception irrespective if the cached element has been pushed or not
- subOutlet.fail(ex)
- maybeCompleteStage()
- }
- })
-
- setHandler(out, new OutHandler {
- override def onPull(): Unit = {
- subInlet.pull()
- }
- override def onDownstreamFinish(cause: Throwable): Unit = {
- subInlet.cancel(cause)
- maybeCompleteStage()
- }
- })
-
- subOutlet.setHandler(new OutHandler {
- override def onPull(): Unit = {
- pull(in)
- }
- override def onDownstreamFinish(cause: Throwable): Unit = {
- if (!isClosed(in)) {
- cancel(in, cause)
- }
- maybeCompleteStage()
- }
- })
-
- subInlet.setHandler(new InHandler {
- override def onPush(): Unit = {
- push(out, subInlet.grab())
- }
- override def onUpstreamFinish(): Unit = {
- complete(out)
- maybeCompleteStage()
- }
- override def onUpstreamFailure(ex: Throwable): Unit = {
- fail(out, ex)
- maybeCompleteStage()
- }
- })
-
- if (isClosed(out)) {
- // downstream may have been canceled while the stage was switching
- subInlet.cancel()
- } else {
- subInlet.pull()
- }
-
- matVal
- }
- }
- (stageLogic, matPromise.future)
- }
-}
diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/StreamOfStreams.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/StreamOfStreams.scala
index 49460d197a..c90a2f9a6d 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/fusing/StreamOfStreams.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/StreamOfStreams.scala
@@ -7,27 +7,27 @@ package akka.stream.impl.fusing
import java.util.Collections
import java.util.concurrent.atomic.AtomicReference
-import akka.NotUsed
-import akka.annotation.InternalApi
-import akka.stream.ActorAttributes.StreamSubscriptionTimeout
-import akka.stream.ActorAttributes.SupervisionStrategy
-import akka.stream._
-import akka.stream.impl.Stages.DefaultAttributes
-import akka.stream.impl.fusing.GraphStages.SingleSource
-import akka.stream.impl.ActorSubscriberMessage
-import akka.stream.impl.SubscriptionTimeoutException
-import akka.stream.impl.TraversalBuilder
-import akka.stream.impl.{ Buffer => BufferImpl }
-import akka.stream.scaladsl._
-import akka.stream.stage._
-import akka.util.OptionVal
-import akka.util.ccompat.JavaConverters._
-
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.duration.FiniteDuration
import scala.util.control.NonFatal
+import akka.NotUsed
+import akka.annotation.InternalApi
+import akka.stream._
+import akka.stream.ActorAttributes.StreamSubscriptionTimeout
+import akka.stream.ActorAttributes.SupervisionStrategy
+import akka.stream.impl.{ Buffer => BufferImpl }
+import akka.stream.impl.ActorSubscriberMessage
+import akka.stream.impl.Stages.DefaultAttributes
+import akka.stream.impl.SubscriptionTimeoutException
+import akka.stream.impl.TraversalBuilder
+import akka.stream.impl.fusing.GraphStages.SingleSource
+import akka.stream.scaladsl._
+import akka.stream.stage._
+import akka.util.OptionVal
+import akka.util.ccompat.JavaConverters._
+
/**
* INTERNAL API
*/
@@ -221,7 +221,7 @@ import scala.util.control.NonFatal
override def onUpstreamFinish(): Unit = {
if (!prefixComplete) {
// This handles the unpulled out case as well
- emit(out, (builder.result, Source.empty), () => completeStage())
+ emit(out, (builder.result(), Source.empty), () => completeStage())
} else {
if (!tailSource.isClosed) tailSource.complete()
completeStage()
@@ -413,7 +413,7 @@ import scala.util.control.NonFatal
override def onPull(): Unit = {
cancelTimer(key)
- if (firstPush) {
+ if (firstPush()) {
firstPushCounter -= 1
push(firstElement)
firstElement = null.asInstanceOf[T]
diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/ByteStringParser.scala b/akka-stream/src/main/scala/akka/stream/impl/io/ByteStringParser.scala
index bdd8c380c6..0709c4e0ec 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/io/ByteStringParser.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/io/ByteStringParser.scala
@@ -4,14 +4,14 @@
package akka.stream.impl.io
+import scala.annotation.tailrec
+import scala.util.control.{ NoStackTrace, NonFatal }
+
import akka.annotation.InternalApi
import akka.stream._
import akka.stream.stage._
import akka.util.ByteString
-import scala.annotation.tailrec
-import scala.util.control.{ NoStackTrace, NonFatal }
-
/**
* INTERNAL API
*/
diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/FileOutputStage.scala b/akka-stream/src/main/scala/akka/stream/impl/io/FileOutputStage.scala
index 2f9a76d88f..06b4bb2624 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/io/FileOutputStage.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/io/FileOutputStage.scala
@@ -8,8 +8,11 @@ import java.nio.channels.FileChannel
import java.nio.file.{ OpenOption, Path }
import scala.collection.immutable
+import scala.concurrent.{ Future, Promise }
+import scala.util.Success
+import scala.util.control.NonFatal
+
import akka.annotation.InternalApi
-import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.{
AbruptStageTerminationException,
Attributes,
@@ -18,14 +21,11 @@ import akka.stream.{
Inlet,
SinkShape
}
+import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.stage.{ GraphStageLogic, GraphStageWithMaterializedValue, InHandler }
import akka.util.ByteString
import akka.util.ccompat.JavaConverters._
-import scala.concurrent.{ Future, Promise }
-import scala.util.Success
-import scala.util.control.NonFatal
-
/**
* INTERNAL API
*/
@@ -38,7 +38,7 @@ private[akka] final class FileOutputStage(path: Path, startPosition: Long, openO
override def initialAttributes: Attributes = DefaultAttributes.fileSink
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[IOResult]) = {
- val mat = Promise[IOResult]
+ val mat = Promise[IOResult]()
val logic = new GraphStageLogic(shape) with InHandler {
private var chan: FileChannel = _
private var bytesWritten: Long = 0
diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/IOSources.scala b/akka-stream/src/main/scala/akka/stream/impl/io/IOSources.scala
index c34ae00913..5e8a2cea82 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/io/IOSources.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/io/IOSources.scala
@@ -8,16 +8,16 @@ import java.nio.ByteBuffer
import java.nio.channels.{ CompletionHandler, FileChannel }
import java.nio.file.{ Files, NoSuchFileException, Path, StandardOpenOption }
-import akka.Done
-import akka.stream.Attributes.InputBuffer
-import akka.stream.stage._
-import akka.stream.{ IOResult, _ }
-import akka.util.ByteString
-
import scala.annotation.tailrec
import scala.concurrent.{ Future, Promise }
-import scala.util.control.NonFatal
import scala.util.{ Failure, Success, Try }
+import scala.util.control.NonFatal
+
+import akka.Done
+import akka.stream.{ IOResult, _ }
+import akka.stream.Attributes.InputBuffer
+import akka.stream.stage._
+import akka.util.ByteString
/**
* INTERNAL API
diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/InputStreamSinkStage.scala b/akka-stream/src/main/scala/akka/stream/impl/io/InputStreamSinkStage.scala
index 211b18a92a..3df85f973c 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/io/InputStreamSinkStage.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/io/InputStreamSinkStage.scala
@@ -8,17 +8,17 @@ import java.io.{ IOException, InputStream }
import java.util.concurrent.{ BlockingQueue, LinkedBlockingDeque, TimeUnit }
import java.util.concurrent.atomic.AtomicBoolean
+import scala.annotation.tailrec
+import scala.concurrent.duration.FiniteDuration
+
import akka.annotation.InternalApi
+import akka.stream.{ AbruptStageTerminationException, Attributes, Inlet, SinkShape }
import akka.stream.Attributes.InputBuffer
import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.impl.io.InputStreamSinkStage._
import akka.stream.stage._
-import akka.stream.{ AbruptStageTerminationException, Attributes, Inlet, SinkShape }
import akka.util.ByteString
-import scala.annotation.tailrec
-import scala.concurrent.duration.FiniteDuration
-
private[stream] object InputStreamSinkStage {
sealed trait AdapterToStageMessage
diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/InputStreamSource.scala b/akka-stream/src/main/scala/akka/stream/impl/io/InputStreamSource.scala
index 18a2f5ebe4..9bcc91efe6 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/io/InputStreamSource.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/io/InputStreamSource.scala
@@ -6,8 +6,10 @@ package akka.stream.impl.io
import java.io.InputStream
+import scala.concurrent.{ Future, Promise }
+import scala.util.control.NonFatal
+
import akka.annotation.InternalApi
-import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.{
AbruptStageTerminationException,
Attributes,
@@ -17,12 +19,10 @@ import akka.stream.{
SourceShape,
SubscriptionWithCancelException
}
+import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.stage.{ GraphStageLogic, GraphStageLogicWithLogging, GraphStageWithMaterializedValue, OutHandler }
import akka.util.ByteString
-import scala.concurrent.{ Future, Promise }
-import scala.util.control.NonFatal
-
/**
* INTERNAL API
*/
@@ -39,7 +39,7 @@ private[akka] final class InputStreamSource(factory: () => InputStream, chunkSiz
override protected def initialAttributes: Attributes = DefaultAttributes.inputStreamSource
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[IOResult]) = {
- val mat = Promise[IOResult]
+ val mat = Promise[IOResult]()
val logic = new GraphStageLogicWithLogging(shape) with OutHandler {
private val buffer = new Array[Byte](chunkSize)
private var readBytesTotal = 0L
diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/OutputStreamGraphStage.scala b/akka-stream/src/main/scala/akka/stream/impl/io/OutputStreamGraphStage.scala
index b0f322ecb0..a1a1c2ca73 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/io/OutputStreamGraphStage.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/io/OutputStreamGraphStage.scala
@@ -6,15 +6,15 @@ package akka.stream.impl.io
import java.io.OutputStream
-import akka.annotation.InternalApi
-import akka.stream.impl.Stages.DefaultAttributes
-import akka.stream.stage.{ GraphStageLogic, GraphStageLogicWithLogging, GraphStageWithMaterializedValue, InHandler }
-import akka.stream.{ Attributes, IOOperationIncompleteException, IOResult, Inlet, SinkShape }
-import akka.util.ByteString
-
import scala.concurrent.{ Future, Promise }
import scala.util.control.NonFatal
+import akka.annotation.InternalApi
+import akka.stream.{ Attributes, IOOperationIncompleteException, IOResult, Inlet, SinkShape }
+import akka.stream.impl.Stages.DefaultAttributes
+import akka.stream.stage.{ GraphStageLogic, GraphStageLogicWithLogging, GraphStageWithMaterializedValue, InHandler }
+import akka.util.ByteString
+
/**
* INTERNAL API
*/
@@ -29,7 +29,7 @@ private[akka] final class OutputStreamGraphStage(factory: () => OutputStream, au
override protected def initialAttributes: Attributes = DefaultAttributes.outputStreamSink
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[IOResult]) = {
- val mat = Promise[IOResult]
+ val mat = Promise[IOResult]()
val logic = new GraphStageLogicWithLogging(shape) with InHandler {
var outputStream: OutputStream = _
var bytesWritten: Long = 0L
diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/OutputStreamSourceStage.scala b/akka-stream/src/main/scala/akka/stream/impl/io/OutputStreamSourceStage.scala
index b1295c2e73..747561dd11 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/io/OutputStreamSourceStage.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/io/OutputStreamSourceStage.scala
@@ -7,17 +7,17 @@ package akka.stream.impl.io
import java.io.{ IOException, OutputStream }
import java.util.concurrent.{ Semaphore, TimeUnit }
+import scala.concurrent.Await
+import scala.concurrent.duration.FiniteDuration
+import scala.util.control.NonFatal
+
+import akka.stream.{ Attributes, Outlet, SourceShape }
import akka.stream.Attributes.InputBuffer
import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.impl.io.OutputStreamSourceStage._
import akka.stream.stage._
-import akka.stream.{ Attributes, Outlet, SourceShape }
import akka.util.ByteString
-import scala.concurrent.Await
-import scala.concurrent.duration.FiniteDuration
-import scala.util.control.NonFatal
-
private[stream] object OutputStreamSourceStage {
sealed trait AdapterToStageMessage
case class Send(data: ByteString) extends AdapterToStageMessage
diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/TLSActor.scala b/akka-stream/src/main/scala/akka/stream/impl/io/TLSActor.scala
index 5252eb9cf6..dd5a9ce955 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/io/TLSActor.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/io/TLSActor.scala
@@ -5,25 +5,27 @@
package akka.stream.impl.io
import java.nio.ByteBuffer
+
+import javax.net.ssl._
import javax.net.ssl.SSLEngineResult.HandshakeStatus
import javax.net.ssl.SSLEngineResult.HandshakeStatus._
import javax.net.ssl.SSLEngineResult.Status._
-import javax.net.ssl._
+
+import scala.annotation.tailrec
+import scala.util.{ Failure, Success, Try }
+import scala.util.control.NonFatal
import akka.actor._
import akka.annotation.InternalApi
import akka.stream._
+import akka.stream.TLSProtocol._
+import akka.stream.impl._
import akka.stream.impl.FanIn.InputBunch
import akka.stream.impl.FanOut.OutputBunch
-import akka.stream.impl._
+import akka.stream.impl.fusing.ActorGraphInterpreter
+import akka.stream.snapshot.StreamSnapshotImpl
import akka.util.ByteString
-import scala.annotation.tailrec
-import akka.stream.TLSProtocol._
-
-import scala.util.control.NonFatal
-import scala.util.{ Failure, Success, Try }
-
/**
* INTERNAL API.
*/
@@ -442,7 +444,10 @@ import scala.util.{ Failure, Success, Try }
}
}
- override def receive = inputBunch.subreceive.orElse[Any, Unit](outputBunch.subreceive)
+ override def receive = inputBunch.subreceive.orElse[Any, Unit](outputBunch.subreceive).orElse {
+ case ActorGraphInterpreter.Snapshot =>
+ sender() ! StreamSnapshotImpl(self.path, Seq.empty, Seq.empty)
+ }
initialPhase(2, bidirectional)
diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/TcpStages.scala b/akka-stream/src/main/scala/akka/stream/impl/io/TcpStages.scala
index d69d1510ac..97fd534198 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/io/TcpStages.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/io/TcpStages.scala
@@ -8,6 +8,12 @@ import java.net.InetSocketAddress
import java.util.concurrent.TimeoutException
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicLong }
+import scala.collection.immutable
+import scala.concurrent.{ Future, Promise }
+import scala.concurrent.duration.{ Duration, FiniteDuration }
+
+import com.github.ghik.silencer.silent
+
import akka.{ Done, NotUsed }
import akka.actor.{ ActorRef, Terminated }
import akka.annotation.InternalApi
@@ -18,16 +24,11 @@ import akka.io.Tcp._
import akka.stream._
import akka.stream.impl.ReactiveStreamsCompliance
import akka.stream.impl.fusing.GraphStages.detacher
+import akka.stream.scaladsl.{ BidiFlow, Flow, TcpIdleTimeoutException, Tcp => StreamTcp }
import akka.stream.scaladsl.Tcp.{ OutgoingConnection, ServerBinding }
import akka.stream.scaladsl.TcpAttributes
-import akka.stream.scaladsl.{ BidiFlow, Flow, TcpIdleTimeoutException, Tcp => StreamTcp }
import akka.stream.stage._
import akka.util.ByteString
-import com.github.ghik.silencer.silent
-
-import scala.collection.immutable
-import scala.concurrent.duration.{ Duration, FiniteDuration }
-import scala.concurrent.{ Future, Promise }
/**
* INTERNAL API
@@ -53,7 +54,7 @@ import scala.concurrent.{ Future, Promise }
// TODO: Timeout on bind
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes, eagerMaterialzer: Materializer) = {
- val bindingPromise = Promise[ServerBinding]
+ val bindingPromise = Promise[ServerBinding]()
val logic = new TimerGraphStageLogic(shape) with StageLogging {
implicit def self: ActorRef = stageActor.ref
@@ -521,7 +522,7 @@ private[stream] object ConnectionSourceStage {
case _ => None
}
- val localAddressPromise = Promise[InetSocketAddress]
+ val localAddressPromise = Promise[InetSocketAddress]()
val logic = new TcpStreamLogic(
shape,
Outbound(
diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/TlsModule.scala b/akka-stream/src/main/scala/akka/stream/impl/io/TlsModule.scala
index b3686f3ded..5f1961dfb7 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/io/TlsModule.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/io/TlsModule.scala
@@ -6,17 +6,17 @@ package akka.stream.impl.io
import javax.net.ssl.{ SSLEngine, SSLSession }
+import scala.util.Try
+
import akka.NotUsed
import akka.actor.ActorSystem
import akka.annotation.InternalApi
import akka.stream._
-import akka.stream.impl.StreamLayout.AtomicModule
import akka.stream.TLSProtocol._
import akka.stream.impl.{ TlsModuleIslandTag, TraversalBuilder }
+import akka.stream.impl.StreamLayout.AtomicModule
import akka.util.ByteString
-import scala.util.Try
-
/**
* INTERNAL API.
*/
diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/compression/DeflateCompressor.scala b/akka-stream/src/main/scala/akka/stream/impl/io/compression/DeflateCompressor.scala
index d234b345ab..46757b7c5f 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/io/compression/DeflateCompressor.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/io/compression/DeflateCompressor.scala
@@ -6,11 +6,11 @@ package akka.stream.impl.io.compression
import java.util.zip.Deflater
+import scala.annotation.tailrec
+
import akka.annotation.InternalApi
import akka.util.{ ByteString, ByteStringBuilder }
-import scala.annotation.tailrec
-
/** INTERNAL API */
@InternalApi private[akka] class DeflateCompressor(level: Int = Deflater.BEST_COMPRESSION, nowrap: Boolean = false)
extends Compressor {
diff --git a/akka-stream/src/main/scala/akka/stream/impl/streamref/SinkRefImpl.scala b/akka-stream/src/main/scala/akka/stream/impl/streamref/SinkRefImpl.scala
index 45fa08d359..5e79888820 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/streamref/SinkRefImpl.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/streamref/SinkRefImpl.scala
@@ -4,6 +4,10 @@
package akka.stream.impl.streamref
+import scala.util.{ Failure, Success, Try }
+
+import com.github.ghik.silencer.silent
+
import akka.Done
import akka.NotUsed
import akka.actor.{ ActorRef, Terminated }
@@ -13,9 +17,6 @@ import akka.stream._
import akka.stream.scaladsl.Sink
import akka.stream.stage._
import akka.util.{ OptionVal, PrettyDuration }
-import com.github.ghik.silencer.silent
-
-import scala.util.{ Failure, Success, Try }
/** INTERNAL API: Implementation class, not intended to be touched directly by end-users */
@InternalApi
diff --git a/akka-stream/src/main/scala/akka/stream/impl/streamref/SourceRefImpl.scala b/akka-stream/src/main/scala/akka/stream/impl/streamref/SourceRefImpl.scala
index a6cb2ab795..896173f69b 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/streamref/SourceRefImpl.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/streamref/SourceRefImpl.scala
@@ -4,6 +4,8 @@
package akka.stream.impl.streamref
+import com.github.ghik.silencer.silent
+
import akka.NotUsed
import akka.actor.{ ActorRef, Terminated }
import akka.annotation.InternalApi
@@ -13,7 +15,6 @@ import akka.stream.impl.FixedSizeBuffer
import akka.stream.scaladsl.Source
import akka.stream.stage._
import akka.util.{ OptionVal, PrettyDuration }
-import com.github.ghik.silencer.silent
/** INTERNAL API: Implementation class, not intended to be touched directly by end-users */
@InternalApi
diff --git a/akka-stream/src/main/scala/akka/stream/impl/streamref/StreamRefSettingsImpl.scala b/akka-stream/src/main/scala/akka/stream/impl/streamref/StreamRefSettingsImpl.scala
index bf1f88ef83..85bec19d60 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/streamref/StreamRefSettingsImpl.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/streamref/StreamRefSettingsImpl.scala
@@ -4,12 +4,12 @@
package akka.stream.impl.streamref
+import scala.concurrent.duration.FiniteDuration
+
import akka.annotation.InternalApi
import akka.event.Logging
import akka.stream.StreamRefSettings
-import scala.concurrent.duration.FiniteDuration
-
/** INTERNAL API */
@InternalApi
private[akka] final case class StreamRefSettingsImpl private (
diff --git a/akka-stream/src/main/scala/akka/stream/impl/streamref/StreamRefsMaster.scala b/akka-stream/src/main/scala/akka/stream/impl/streamref/StreamRefsMaster.scala
index ef0abf61f2..52272ef98b 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/streamref/StreamRefsMaster.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/streamref/StreamRefsMaster.scala
@@ -4,8 +4,8 @@
package akka.stream.impl.streamref
-import akka.actor.ClassicActorSystemProvider
import akka.actor.{ ActorSystem, ExtendedActorSystem, Extension, ExtensionId, ExtensionIdProvider }
+import akka.actor.ClassicActorSystemProvider
import akka.annotation.InternalApi
import akka.stream.impl.SeqActorName
diff --git a/akka-stream/src/main/scala/akka/stream/impl/streamref/StreamRefsProtocol.scala b/akka-stream/src/main/scala/akka/stream/impl/streamref/StreamRefsProtocol.scala
index 2a89a21caa..6cba371323 100644
--- a/akka-stream/src/main/scala/akka/stream/impl/streamref/StreamRefsProtocol.scala
+++ b/akka-stream/src/main/scala/akka/stream/impl/streamref/StreamRefsProtocol.scala
@@ -72,6 +72,6 @@ private[akka] object StreamRefsProtocol {
* Ack that failure or completion has been seen and the remote side can stop
*/
@InternalApi
- private[akka] final case object Ack extends StreamRefsProtocol with DeadLetterSuppression
+ private[akka] case object Ack extends StreamRefsProtocol with DeadLetterSuppression
}
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala b/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala
index 7bafd72a19..c94af574ce 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala
@@ -4,12 +4,13 @@
package akka.stream.javadsl
+import scala.concurrent.duration.FiniteDuration
+
+import com.github.ghik.silencer.silent
+
import akka.NotUsed
import akka.japi.function
import akka.stream._
-import com.github.ghik.silencer.silent
-
-import scala.concurrent.duration.FiniteDuration
object BidiFlow {
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/DelayStrategy.scala b/akka-stream/src/main/scala/akka/stream/javadsl/DelayStrategy.scala
index d27c5cd501..dbbe90f210 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/DelayStrategy.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/DelayStrategy.scala
@@ -4,12 +4,12 @@
package akka.stream.javadsl
+import scala.concurrent.duration.FiniteDuration
+
import akka.annotation.InternalApi
import akka.stream.scaladsl
import akka.util.JavaDurationConverters.JavaDurationOps
-import scala.concurrent.duration.FiniteDuration
-
/**
* Allows to manage delay and can be stateful to compute delay for any sequence of elements,
* all elements go through nextDelay() updating state and returning delay for each element
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/FileIO.scala b/akka-stream/src/main/scala/akka/stream/javadsl/FileIO.scala
index 7618eb2eb0..9e96109924 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/FileIO.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/FileIO.scala
@@ -11,7 +11,6 @@ import java.util.concurrent.CompletionStage
import akka.stream.{ javadsl, scaladsl, IOResult }
import akka.util.ByteString
-
import akka.util.ccompat.JavaConverters._
/**
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala
index 12ab4fc8b2..0f7e7d4611 100755
--- a/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala
@@ -4,13 +4,22 @@
package akka.stream.javadsl
+import java.util.Comparator
+import java.util.Optional
import java.util.concurrent.CompletionStage
import java.util.function.BiFunction
import java.util.function.Supplier
-import java.util.Comparator
-import java.util.Optional
-import java.util.concurrent.CompletableFuture
+import scala.annotation.unchecked.uncheckedVariance
+import scala.compat.java8.FutureConverters._
+import scala.concurrent.duration.FiniteDuration
+import scala.reflect.ClassTag
+
+import com.github.ghik.silencer.silent
+import org.reactivestreams.Processor
+
+import akka.Done
+import akka.NotUsed
import akka.actor.ActorRef
import akka.actor.ClassicActorSystemProvider
import akka.dispatch.ExecutionContexts
@@ -18,22 +27,12 @@ import akka.event.{ LogMarker, LoggingAdapter, MarkerLoggingAdapter }
import akka.japi.Pair
import akka.japi.Util
import akka.japi.function
-import akka.stream._
-import akka.stream.impl.fusing.LazyFlow
-import akka.util.JavaDurationConverters._
-import akka.util.unused
-import akka.util.ConstantFun
-import akka.util.Timeout
-import akka.Done
-import akka.NotUsed
import akka.japi.function.Creator
-import com.github.ghik.silencer.silent
-import org.reactivestreams.Processor
-
-import scala.annotation.unchecked.uncheckedVariance
-import scala.compat.java8.FutureConverters._
-import scala.concurrent.duration.FiniteDuration
-import scala.reflect.ClassTag
+import akka.stream._
+import akka.util.ConstantFun
+import akka.util.JavaDurationConverters._
+import akka.util.Timeout
+import akka.util.unused
object Flow {
@@ -262,9 +261,9 @@ object Flow {
flowFactory: function.Function[I, CompletionStage[Flow[I, O, M]]],
fallback: function.Creator[M]): Flow[I, O, M] = {
import scala.compat.java8.FutureConverters._
- val sflow = scaladsl.Flow
- .fromGraph(new LazyFlow[I, O, M](t => flowFactory.apply(t).toScala.map(_.asScala)(ExecutionContexts.parasitic)))
- .mapMaterializedValue(_ => fallback.create())
+ val sflow = scaladsl.Flow.lazyInit(
+ (flowFactory.apply(_)).andThen(_.toScala.map(_.asScala)(ExecutionContexts.parasitic)),
+ fallback.create _)
new Flow(sflow)
}
@@ -303,8 +302,12 @@ object Flow {
* The materialized completion stage value is completed with the materialized value of the future flow or failed with a
* [[NeverMaterializedException]] if upstream fails or downstream cancels before the completion stage has completed.
*/
- def completionStageFlow[I, O, M](flow: CompletionStage[Flow[I, O, M]]): Flow[I, O, CompletionStage[M]] =
- lazyCompletionStageFlow(() => flow)
+ def completionStageFlow[I, O, M](flow: CompletionStage[Flow[I, O, M]]): Flow[I, O, CompletionStage[M]] = {
+ import scala.compat.java8.FutureConverters._
+ val sflow =
+ scaladsl.Flow.futureFlow(flow.toScala.map(_.asScala)(ExecutionContexts.parasitic)).mapMaterializedValue(_.toJava)
+ new javadsl.Flow(sflow)
+ }
/**
* Defers invoking the `create` function to create a future flow until there is downstream demand and passing
@@ -321,8 +324,15 @@ object Flow {
*
* '''Cancels when''' downstream cancels
*/
- def lazyFlow[I, O, M](create: Creator[Flow[I, O, M]]): Flow[I, O, CompletionStage[M]] =
- lazyCompletionStageFlow(() => CompletableFuture.completedFuture(create.create()))
+ def lazyFlow[I, O, M](create: Creator[Flow[I, O, M]]): Flow[I, O, CompletionStage[M]] = {
+ import scala.compat.java8.FutureConverters._
+ val sflow = scaladsl.Flow
+ .lazyFlow { () =>
+ create.create().asScala
+ }
+ .mapMaterializedValue(_.toJava)
+ new javadsl.Flow(sflow)
+ }
/**
* Defers invoking the `create` function to create a future flow until there downstream demand has caused upstream
@@ -364,7 +374,9 @@ object Flow {
}
-/** Create a `Flow` which can process elements of type `T`. */
+/**
+ * A `Flow` is a set of stream processing steps that has one open input and one open output.
+ */
final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Graph[FlowShape[In, Out], Mat] {
import akka.util.ccompat.JavaConverters._
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/FlowWithContext.scala b/akka-stream/src/main/scala/akka/stream/javadsl/FlowWithContext.scala
index 22e331f989..4267165d1b 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/FlowWithContext.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/FlowWithContext.scala
@@ -4,17 +4,17 @@
package akka.stream.javadsl
-import akka.japi.{ function, Pair, Util }
-import akka.stream._
-import akka.event.{ LogMarker, LoggingAdapter, MarkerLoggingAdapter }
-import akka.util.ConstantFun
-
-import scala.annotation.unchecked.uncheckedVariance
-import akka.util.ccompat.JavaConverters._
import java.util.concurrent.CompletionStage
+import scala.annotation.unchecked.uncheckedVariance
import scala.compat.java8.FutureConverters._
+import akka.event.{ LogMarker, LoggingAdapter, MarkerLoggingAdapter }
+import akka.japi.{ function, Pair, Util }
+import akka.stream._
+import akka.util.ConstantFun
+import akka.util.ccompat.JavaConverters._
+
object FlowWithContext {
def create[In, Ctx](): FlowWithContext[In, Ctx, In, Ctx, akka.NotUsed] =
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Graph.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Graph.scala
index 174990b86a..a45990f27b 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/Graph.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/Graph.scala
@@ -6,14 +6,14 @@ package akka.stream.javadsl
import java.util
-import akka.NotUsed
-import akka.stream._
-import akka.japi.{ function, Pair }
-import akka.util.ConstantFun
-
import scala.annotation.unchecked.uncheckedVariance
-import akka.util.ccompat.JavaConverters._
+
+import akka.NotUsed
+import akka.japi.{ function, Pair }
+import akka.stream._
import akka.stream.scaladsl.GenericGraph
+import akka.util.ConstantFun
+import akka.util.ccompat.JavaConverters._
import akka.util.unused
/**
@@ -380,8 +380,8 @@ object Balance {
* '''Cancels when''' downstream cancels
*/
object Zip {
- import akka.japi.function.Function2
import akka.japi.Pair
+ import akka.japi.function.Function2
/**
* Create a new `Zip` operator with the specified input types and zipping-function
@@ -409,8 +409,8 @@ object Zip {
* '''Cancels when''' downstream cancels
*/
object ZipLatest {
- import akka.japi.function.Function2
import akka.japi.Pair
+ import akka.japi.function.Function2
/**
* Create a new `ZipLatest` operator with the specified input types and zipping-function
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Hub.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Hub.scala
index f5b840fa0f..4bd0cb3100 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/Hub.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/Hub.scala
@@ -4,9 +4,9 @@
package akka.stream.javadsl
-import akka.NotUsed
import java.util.function.{ BiFunction, Supplier, ToLongBiFunction }
+import akka.NotUsed
import akka.annotation.DoNotInherit
import akka.util.unused
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Keep.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Keep.scala
index c08bc891ae..31f3ba833a 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/Keep.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/Keep.scala
@@ -5,8 +5,8 @@
package akka.stream.javadsl
import akka.NotUsed
-import akka.japi.function
import akka.japi.Pair
+import akka.japi.function
object Keep {
private val _left = new function.Function2[Any, Any, Any] with ((Any, Any) => Any) { def apply(l: Any, r: Any) = l }
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/MergeLatest.scala b/akka-stream/src/main/scala/akka/stream/javadsl/MergeLatest.scala
index 94a853cc06..c4debf0f67 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/MergeLatest.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/MergeLatest.scala
@@ -4,9 +4,8 @@
package akka.stream.javadsl
-import akka.stream.stage.GraphStage
import akka.stream.{ scaladsl, UniformFanInShape }
-
+import akka.stream.stage.GraphStage
import akka.util.ccompat.JavaConverters._
/**
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Queue.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Queue.scala
index 1bc96ba2e6..a99812cdf4 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/Queue.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/Queue.scala
@@ -7,14 +7,14 @@ package akka.stream.javadsl
import java.util.Optional
import java.util.concurrent.CompletionStage
-import akka.Done
-import akka.dispatch.ExecutionContexts
-import akka.stream.QueueOfferResult
-
import scala.compat.java8.FutureConverters._
import scala.compat.java8.OptionConverters._
import scala.concurrent.Future
+import akka.Done
+import akka.dispatch.ExecutionContexts
+import akka.stream.QueueOfferResult
+
/**
* This trait allows to have a queue as a data source for some stream.
*/
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/RestartFlow.scala b/akka-stream/src/main/scala/akka/stream/javadsl/RestartFlow.scala
index 0b5442ab64..0dfc51ff62 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/RestartFlow.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/RestartFlow.scala
@@ -4,11 +4,12 @@
package akka.stream.javadsl
-import akka.NotUsed
-import akka.japi.function.Creator
+import scala.concurrent.duration.FiniteDuration
+
import com.github.ghik.silencer.silent
-import scala.concurrent.duration.FiniteDuration
+import akka.NotUsed
+import akka.japi.function.Creator
/**
* A RestartFlow wraps a [[Flow]] that gets restarted when it completes or fails.
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/RestartSink.scala b/akka-stream/src/main/scala/akka/stream/javadsl/RestartSink.scala
index 305c79a81b..56bf831939 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/RestartSink.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/RestartSink.scala
@@ -4,11 +4,12 @@
package akka.stream.javadsl
-import akka.NotUsed
-import akka.japi.function.Creator
+import scala.concurrent.duration.FiniteDuration
+
import com.github.ghik.silencer.silent
-import scala.concurrent.duration.FiniteDuration
+import akka.NotUsed
+import akka.japi.function.Creator
/**
* A RestartSink wraps a [[Sink]] that gets restarted when it completes or fails.
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/RestartSource.scala b/akka-stream/src/main/scala/akka/stream/javadsl/RestartSource.scala
index cd97a09ba4..fc533c4b59 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/RestartSource.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/RestartSource.scala
@@ -4,11 +4,12 @@
package akka.stream.javadsl
-import akka.NotUsed
-import akka.japi.function.Creator
+import scala.concurrent.duration.FiniteDuration
+
import com.github.ghik.silencer.silent
-import scala.concurrent.duration.FiniteDuration
+import akka.NotUsed
+import akka.japi.function.Creator
/**
* A RestartSource wraps a [[Source]] that gets restarted when it completes or fails.
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/RetryFlow.scala b/akka-stream/src/main/scala/akka/stream/javadsl/RetryFlow.scala
index d88f6b99d2..09df716c65 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/RetryFlow.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/RetryFlow.scala
@@ -6,13 +6,13 @@ package akka.stream.javadsl
import java.util.Optional
+import scala.compat.java8.OptionConverters._
+
import akka.annotation.ApiMayChange
import akka.japi.Pair
import akka.stream.scaladsl
import akka.util.JavaDurationConverters._
-import scala.compat.java8.OptionConverters._
-
object RetryFlow {
/**
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala
index 9e4638e8e5..85ef30685a 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala
@@ -9,15 +9,6 @@ import java.util.concurrent.CompletableFuture
import java.util.concurrent.CompletionStage
import java.util.function.BiFunction
-import akka.actor.{ ActorRef, ClassicActorSystemProvider, Status }
-import akka.dispatch.ExecutionContexts
-import akka._
-import akka.japi.function
-import akka.japi.function.Creator
-import akka.stream.impl.LinearTraversalBuilder
-import akka.stream.{ javadsl, scaladsl, _ }
-import org.reactivestreams.{ Publisher, Subscriber }
-
import scala.annotation.unchecked.uncheckedVariance
import scala.collection.immutable
import scala.compat.java8.FutureConverters._
@@ -25,6 +16,16 @@ import scala.compat.java8.OptionConverters._
import scala.concurrent.ExecutionContext
import scala.util.Try
+import org.reactivestreams.{ Publisher, Subscriber }
+
+import akka._
+import akka.actor.{ ActorRef, ClassicActorSystemProvider, Status }
+import akka.dispatch.ExecutionContexts
+import akka.japi.function
+import akka.japi.function.Creator
+import akka.stream.{ javadsl, scaladsl, _ }
+import akka.stream.impl.LinearTraversalBuilder
+
/** Java API */
object Sink {
@@ -424,7 +425,7 @@ object Sink {
* case the materialized future value is failed with a [[akka.stream.NeverMaterializedException]].
*/
def lazySink[T, M](create: Creator[Sink[T, M]]): Sink[T, CompletionStage[M]] =
- lazyCompletionStageSink(() => CompletableFuture.completedFuture(create.create))
+ lazyCompletionStageSink(() => CompletableFuture.completedFuture(create.create()))
/**
* Defers invoking the `create` function to create a future sink until there is a first element passed from upstream.
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala
index 9d60e0ad22..770f33fd5b 100755
--- a/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala
@@ -9,28 +9,29 @@ import java.util.Optional
import java.util.concurrent.{ CompletableFuture, CompletionStage }
import java.util.function.{ BiFunction, Supplier }
-import akka.actor.{ ActorRef, Cancellable, ClassicActorSystemProvider }
-import akka.dispatch.ExecutionContexts
-import akka.event.{ LogMarker, LoggingAdapter, MarkerLoggingAdapter }
-import akka.japi.function.Creator
-import akka.japi.{ function, JavaPartialFunction, Pair, Util }
-import akka.stream._
-import akka.stream.impl.LinearTraversalBuilder
-import akka.util.JavaDurationConverters._
-import akka.util.ccompat.JavaConverters._
-import akka.util.{ unused, _ }
-import akka.{ Done, NotUsed }
-import com.github.ghik.silencer.silent
-import org.reactivestreams.{ Publisher, Subscriber }
-
import scala.annotation.unchecked.uncheckedVariance
import scala.collection.immutable
import scala.compat.java8.FutureConverters._
import scala.compat.java8.OptionConverters._
-import scala.concurrent.duration.FiniteDuration
import scala.concurrent.{ Future, Promise }
+import scala.concurrent.duration.FiniteDuration
import scala.reflect.ClassTag
+import com.github.ghik.silencer.silent
+import org.reactivestreams.{ Publisher, Subscriber }
+
+import akka.{ Done, NotUsed }
+import akka.actor.{ ActorRef, Cancellable, ClassicActorSystemProvider }
+import akka.dispatch.ExecutionContexts
+import akka.event.{ LogMarker, LoggingAdapter, MarkerLoggingAdapter }
+import akka.japi.{ function, JavaPartialFunction, Pair, Util }
+import akka.japi.function.Creator
+import akka.stream._
+import akka.stream.impl.LinearTraversalBuilder
+import akka.util.{ unused, _ }
+import akka.util.JavaDurationConverters._
+import akka.util.ccompat.JavaConverters._
+
/** Java API */
object Source {
private[this] val _empty = new Source[Any, NotUsed](scaladsl.Source.empty)
@@ -99,6 +100,17 @@ object Source {
def fromIterator[O](f: function.Creator[java.util.Iterator[O]]): javadsl.Source[O, NotUsed] =
new Source(scaladsl.Source.fromIterator(() => f.create().asScala))
+ /**
+ * Creates a source that wraps a Java 8 ``Stream``. ``Source`` uses a stream iterator to get all its
+ * elements and send them downstream on demand.
+ *
+ * You can use [[Source.async]] to create asynchronous boundaries between synchronous java stream
+ * and the rest of flow.
+ */
+ def fromJavaStream[O, S <: java.util.stream.BaseStream[O, S]](
+ stream: function.Creator[java.util.stream.BaseStream[O, S]]): javadsl.Source[O, NotUsed] =
+ StreamConverters.fromJavaStream(stream)
+
/**
* Helper to create 'cycled' [[Source]] from iterator provider.
* Example usage:
@@ -292,6 +304,13 @@ object Source {
def future[T](futureElement: Future[T]): Source[T, NotUsed] =
scaladsl.Source.future(futureElement).asJava
+ /**
+ * Never emits any elements, never completes and never fails.
+ * This stream could be useful in tests.
+ */
+ def never[T]: Source[T, NotUsed] =
+ scaladsl.Source.never.asJava
+
/**
* Emits a single value when the given `CompletionStage` is successfully completed and then completes the stream.
* If the `CompletionStage` is completed with a failure the stream is failed.
@@ -1626,7 +1645,7 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
def zipMat[T, M, M2](
that: Graph[SourceShape[T], M],
matF: function.Function2[Mat, M, M2]): javadsl.Source[Out @uncheckedVariance Pair T, M2] =
- this.viaMat(Flow.create[Out].zipMat(that, Keep.right[NotUsed, M]), matF)
+ this.viaMat(Flow.create[Out]().zipMat(that, Keep.right[NotUsed, M]), matF)
/**
* Combine the elements of current flow and the given [[Source]] into a stream of tuples.
@@ -1689,7 +1708,7 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
def zipLatestMat[T, M, M2](
that: Graph[SourceShape[T], M],
matF: function.Function2[Mat, M, M2]): javadsl.Source[Out @uncheckedVariance Pair T, M2] =
- this.viaMat(Flow.create[Out].zipLatestMat(that, Keep.right[NotUsed, M]), matF)
+ this.viaMat(Flow.create[Out]().zipLatestMat(that, Keep.right[NotUsed, M]), matF)
/**
* Put together the elements of current [[Source]] and the given one
@@ -4311,6 +4330,9 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
def logWithMarker(name: String, marker: function.Function[Out, LogMarker]): javadsl.Source[Out, Mat] =
this.logWithMarker(name, marker, ConstantFun.javaIdentityFunction[Out], null)
+ /**
+ * Transform this source whose element is ``e`` into a source producing tuple ``(e, f(e))``
+ **/
def asSourceWithContext[Ctx](extractContext: function.Function[Out, Ctx]): SourceWithContext[Out, Ctx, Mat] =
new scaladsl.SourceWithContext(this.asScala.map(x => (x, extractContext.apply(x)))).asJava
}
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/SourceWithContext.scala b/akka-stream/src/main/scala/akka/stream/javadsl/SourceWithContext.scala
index 58a434a7d2..4544482283 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/SourceWithContext.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/SourceWithContext.scala
@@ -6,6 +6,9 @@ package akka.stream.javadsl
import java.util.concurrent.CompletionStage
+import scala.annotation.unchecked.uncheckedVariance
+import scala.compat.java8.FutureConverters._
+
import akka.actor.ClassicActorSystemProvider
import akka.event.{ LogMarker, LoggingAdapter, MarkerLoggingAdapter }
import akka.japi.Pair
@@ -15,9 +18,6 @@ import akka.stream._
import akka.util.ConstantFun
import akka.util.ccompat.JavaConverters._
-import scala.annotation.unchecked.uncheckedVariance
-import scala.compat.java8.FutureConverters._
-
object SourceWithContext {
/**
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/StreamConverters.scala b/akka-stream/src/main/scala/akka/stream/javadsl/StreamConverters.scala
index 06285322b5..5707e19a56 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/StreamConverters.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/StreamConverters.scala
@@ -5,19 +5,19 @@
package akka.stream.javadsl
import java.io.{ InputStream, OutputStream }
+import java.util.concurrent.CompletionStage
import java.util.stream.Collector
+import scala.concurrent.duration.FiniteDuration
+
+import com.github.ghik.silencer.silent
+
+import akka.NotUsed
import akka.japi.function
import akka.stream.{ javadsl, scaladsl }
import akka.stream.IOResult
import akka.util.ByteString
-import scala.concurrent.duration.FiniteDuration
-import java.util.concurrent.CompletionStage
-
-import akka.NotUsed
-import com.github.ghik.silencer.silent
-
/**
* Converters for interacting with the blocking `java.io` streams APIs and Java 8 Streams
*/
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/SubFlow.scala b/akka-stream/src/main/scala/akka/stream/javadsl/SubFlow.scala
index d2848cbc77..aa7562fbda 100755
--- a/akka-stream/src/main/scala/akka/stream/javadsl/SubFlow.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/SubFlow.scala
@@ -4,6 +4,17 @@
package akka.stream.javadsl
+import java.util.Comparator
+import java.util.concurrent.CompletionStage
+import java.util.function.Supplier
+
+import scala.annotation.unchecked.uncheckedVariance
+import scala.compat.java8.FutureConverters._
+import scala.concurrent.duration.FiniteDuration
+import scala.reflect.ClassTag
+
+import com.github.ghik.silencer.silent
+
import akka.NotUsed
import akka.event.{ LogMarker, LoggingAdapter, MarkerLoggingAdapter }
import akka.japi.{ function, Pair, Util }
@@ -12,18 +23,6 @@ import akka.util.ConstantFun
import akka.util.JavaDurationConverters._
import akka.util.ccompat.JavaConverters._
-import scala.annotation.unchecked.uncheckedVariance
-import scala.concurrent.duration.FiniteDuration
-import java.util.Comparator
-import java.util.function.Supplier
-
-import scala.compat.java8.FutureConverters._
-import java.util.concurrent.CompletionStage
-
-import com.github.ghik.silencer.silent
-
-import scala.reflect.ClassTag
-
object SubFlow {
/**
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/SubSource.scala b/akka-stream/src/main/scala/akka/stream/javadsl/SubSource.scala
index 217c1bea0f..e55670c992 100755
--- a/akka-stream/src/main/scala/akka/stream/javadsl/SubSource.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/SubSource.scala
@@ -4,6 +4,17 @@
package akka.stream.javadsl
+import java.util.Comparator
+import java.util.concurrent.CompletionStage
+import java.util.function.Supplier
+
+import scala.annotation.unchecked.uncheckedVariance
+import scala.compat.java8.FutureConverters._
+import scala.concurrent.duration.FiniteDuration
+import scala.reflect.ClassTag
+
+import com.github.ghik.silencer.silent
+
import akka.NotUsed
import akka.event.{ LogMarker, LoggingAdapter, MarkerLoggingAdapter }
import akka.japi.{ function, Pair, Util }
@@ -12,17 +23,6 @@ import akka.util.ConstantFun
import akka.util.JavaDurationConverters._
import akka.util.ccompat.JavaConverters._
-import scala.annotation.unchecked.uncheckedVariance
-import scala.concurrent.duration.FiniteDuration
-import java.util.Comparator
-import java.util.concurrent.CompletionStage
-import java.util.function.Supplier
-
-import com.github.ghik.silencer.silent
-
-import scala.compat.java8.FutureConverters._
-import scala.reflect.ClassTag
-
/**
* * Upcast a stream of elements to a stream of supertypes of that element. Useful in combination with
* fan-in operators where you do not want to pay the cost of casting each element in a `map`.
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/TLS.scala b/akka-stream/src/main/scala/akka/stream/javadsl/TLS.scala
index 3b03ee5d7a..2b456fbb92 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/TLS.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/TLS.scala
@@ -8,14 +8,15 @@ import java.util.Optional
import java.util.function.{ Consumer, Supplier }
import javax.net.ssl.{ SSLContext, SSLEngine, SSLSession }
+import scala.compat.java8.OptionConverters
+import scala.util.Try
+
+import com.typesafe.sslconfig.akka.AkkaSSLConfig
+
import akka.{ japi, NotUsed }
import akka.stream._
import akka.stream.TLSProtocol._
import akka.util.ByteString
-import com.typesafe.sslconfig.akka.AkkaSSLConfig
-
-import scala.compat.java8.OptionConverters
-import scala.util.Try
/**
* Stream cipher support based upon JSSE.
diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Tcp.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Tcp.scala
index f720e1d2e2..f17b12202f 100644
--- a/akka-stream/src/main/scala/akka/stream/javadsl/Tcp.scala
+++ b/akka-stream/src/main/scala/akka/stream/javadsl/Tcp.scala
@@ -5,40 +5,39 @@
package akka.stream.javadsl
import java.lang.{ Iterable => JIterable }
-import java.util.Optional
-import java.util.function.{ Function => JFunction }
-
-import akka.{ Done, NotUsed }
-import scala.concurrent.duration._
import java.net.InetSocketAddress
-
-import akka.actor.ActorSystem
-import akka.actor.ExtendedActorSystem
-import akka.actor.ExtensionId
-import akka.actor.ExtensionIdProvider
-import akka.stream.Materializer
-import akka.stream.scaladsl
-import akka.util.ByteString
-import akka.japi.Util.immutableSeq
-import akka.io.Inet.SocketOption
-import scala.compat.java8.OptionConverters._
-import scala.compat.java8.FutureConverters._
+import java.util.Optional
import java.util.concurrent.CompletionStage
+import java.util.function.{ Function => JFunction }
import java.util.function.Supplier
+import javax.net.ssl.SSLContext
+import javax.net.ssl.SSLEngine
+import javax.net.ssl.SSLSession
+import scala.compat.java8.FutureConverters._
+import scala.compat.java8.OptionConverters._
+import scala.concurrent.duration._
import scala.util.Failure
import scala.util.Success
+import com.github.ghik.silencer.silent
+
+import akka.{ Done, NotUsed }
+import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
-import javax.net.ssl.SSLContext
+import akka.actor.ExtendedActorSystem
+import akka.actor.ExtensionId
+import akka.actor.ExtensionIdProvider
import akka.annotation.InternalApi
+import akka.io.Inet.SocketOption
+import akka.japi.Util.immutableSeq
+import akka.stream.Materializer
import akka.stream.SystemMaterializer
import akka.stream.TLSClosing
import akka.stream.TLSProtocol.NegotiateNewSession
+import akka.stream.scaladsl
+import akka.util.ByteString
import akka.util.JavaDurationConverters._
-import com.github.ghik.silencer.silent
-import javax.net.ssl.SSLEngine
-import javax.net.ssl.SSLSession
object Tcp extends ExtensionId[Tcp] with ExtensionIdProvider {
@@ -139,6 +138,7 @@ object Tcp extends ExtensionId[Tcp] with ExtensionIdProvider {
class Tcp(system: ExtendedActorSystem) extends akka.actor.Extension {
import Tcp._
+
import akka.dispatch.ExecutionContexts.parasitic
private lazy val delegate: scaladsl.Tcp = scaladsl.Tcp(system)
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala
index 56e0a6d993..9b653d3bca 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala
@@ -4,12 +4,12 @@
package akka.stream.scaladsl
+import scala.concurrent.duration.FiniteDuration
+
import akka.NotUsed
import akka.stream.{ BidiShape, _ }
import akka.stream.impl.{ LinearTraversalBuilder, Timers, TraversalBuilder }
-import scala.concurrent.duration.FiniteDuration
-
final class BidiFlow[-I1, +O1, -I2, +O2, +Mat](
override val traversalBuilder: TraversalBuilder,
override val shape: BidiShape[I1, O1, I2, O2])
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/FileIO.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/FileIO.scala
index f0d84d3abc..8f072b2417 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/FileIO.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/FileIO.scala
@@ -8,13 +8,13 @@ import java.io.File
import java.nio.file.{ OpenOption, Path }
import java.nio.file.StandardOpenOption._
+import scala.concurrent.Future
+
+import akka.stream.IOResult
import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.impl.io._
-import akka.stream.IOResult
import akka.util.ByteString
-import scala.concurrent.Future
-
/**
* Factories to create sinks and sources from files
*/
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala
index c3d79c9e34..9ffefd69d2 100755
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala
@@ -4,9 +4,21 @@
package akka.stream.scaladsl
+import scala.annotation.implicitNotFound
+import scala.annotation.unchecked.uncheckedVariance
+import scala.collection.immutable
+import scala.concurrent.Future
+import scala.concurrent.duration.FiniteDuration
+import scala.reflect.ClassTag
+
+import org.reactivestreams.{ Processor, Publisher, Subscriber, Subscription }
+
+import akka.Done
+import akka.NotUsed
+import akka.actor.ActorRef
+import akka.annotation.DoNotInherit
import akka.event.{ LogMarker, LoggingAdapter, MarkerLoggingAdapter }
import akka.stream._
-import akka.Done
import akka.stream.impl.{
fusing,
LinearTraversalBuilder,
@@ -18,21 +30,9 @@ import akka.stream.impl.{
TraversalBuilder
}
import akka.stream.impl.fusing._
+import akka.stream.impl.fusing.FlattenMerge
import akka.stream.stage._
import akka.util.{ ConstantFun, Timeout }
-import org.reactivestreams.{ Processor, Publisher, Subscriber, Subscription }
-
-import scala.annotation.unchecked.uncheckedVariance
-import scala.collection.immutable
-import scala.concurrent.Future
-import scala.concurrent.duration.FiniteDuration
-import akka.stream.impl.fusing.FlattenMerge
-import akka.NotUsed
-import akka.actor.ActorRef
-import akka.annotation.DoNotInherit
-
-import scala.annotation.implicitNotFound
-import scala.reflect.ClassTag
/**
* A `Flow` is a set of stream processing steps that has one open input and one open output.
@@ -578,13 +578,22 @@ object Flow {
*
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
*
- * '''Cancels when''' downstream cancels
+ * '''Cancels when''' downstream cancels (see below)
+ *
+ * The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
+ * This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
+ * this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
*/
@deprecated(
"Use 'Flow.futureFlow' in combination with prefixAndTail(1) instead, see `futureFlow` operator docs for details",
"2.6.0")
def lazyInit[I, O, M](flowFactory: I => Future[Flow[I, O, M]], fallback: () => M): Flow[I, O, M] =
- Flow.fromGraph(new LazyFlow[I, O, M](flowFactory)).mapMaterializedValue(_ => fallback())
+ Flow[I]
+ .flatMapPrefix(1) {
+ case Seq(a) => futureFlow(flowFactory(a)).mapMaterializedValue(_ => NotUsed)
+ case Nil => Flow[I].asInstanceOf[Flow[I, O, NotUsed]]
+ }
+ .mapMaterializedValue(_ => fallback())
/**
* Creates a real `Flow` upon receiving the first element. Internal `Flow` will not be created
@@ -600,13 +609,17 @@ object Flow {
*
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
*
- * '''Cancels when''' downstream cancels
+ * '''Cancels when''' downstream cancels (see below)
+ *
+ * The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
+ * This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
+ * this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
*/
@deprecated("Use 'Flow.lazyFutureFlow' instead", "2.6.0")
def lazyInitAsync[I, O, M](flowFactory: () => Future[Flow[I, O, M]]): Flow[I, O, Future[Option[M]]] =
- Flow.fromGraph(new LazyFlow[I, O, M](_ => flowFactory())).mapMaterializedValue { v =>
+ Flow.lazyFutureFlow(flowFactory).mapMaterializedValue {
implicit val ec = akka.dispatch.ExecutionContexts.parasitic
- v.map[Option[M]](Some.apply _).recover { case _: NeverMaterializedException => None }
+ _.map(Some.apply).recover { case _: NeverMaterializedException => None }
}
/**
@@ -615,9 +628,13 @@ object Flow {
*
* The materialized future value is completed with the materialized value of the future flow or failed with a
* [[NeverMaterializedException]] if upstream fails or downstream cancels before the future has completed.
+ *
+ * The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
+ * This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
+ * this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
*/
def futureFlow[I, O, M](flow: Future[Flow[I, O, M]]): Flow[I, O, Future[M]] =
- lazyFutureFlow(() => flow)
+ Flow.fromGraph(new FutureFlow(flow))
/**
* Defers invoking the `create` function to create a future flow until there is downstream demand and passing
@@ -638,7 +655,11 @@ object Flow {
*
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
*
- * '''Cancels when''' downstream cancels
+ * '''Cancels when''' downstream cancels (see below)
+ *
+ * The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
+ * This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
+ * this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
*/
def lazyFlow[I, O, M](create: () => Flow[I, O, M]): Flow[I, O, Future[M]] =
lazyFutureFlow(() => Future.successful(create()))
@@ -662,10 +683,27 @@ object Flow {
*
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
*
- * '''Cancels when''' downstream cancels
+ * '''Cancels when''' downstream cancels (see below)
+ *
+ * The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
+ * This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
+ * this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
*/
def lazyFutureFlow[I, O, M](create: () => Future[Flow[I, O, M]]): Flow[I, O, Future[M]] =
- Flow.fromGraph(new LazyFlow(_ => create()))
+ Flow[I]
+ .flatMapPrefixMat(1) {
+ case Seq(a) =>
+ val f: Flow[I, O, Future[M]] =
+ futureFlow(create()
+ .map(Flow[I].prepend(Source.single(a)).viaMat(_)(Keep.right))(akka.dispatch.ExecutionContexts.parasitic))
+ f
+ case Nil =>
+ val f: Flow[I, O, Future[M]] = Flow[I]
+ .asInstanceOf[Flow[I, O, NotUsed]]
+ .mapMaterializedValue(_ => Future.failed[M](new NeverMaterializedException()))
+ f
+ }(Keep.right)
+ .mapMaterializedValue(_.flatten)
}
@@ -743,9 +781,10 @@ final case class RunnableGraph[+Mat](override val traversalBuilder: TraversalBui
*/
@DoNotInherit
trait FlowOps[+Out, +Mat] {
- import akka.stream.impl.Stages._
import GraphDSL.Implicits._
+ import akka.stream.impl.Stages._
+
type Repr[+O] <: FlowOps[O, Mat] {
type Repr[+OO] = FlowOps.this.Repr[OO]
type Closed = FlowOps.this.Closed
@@ -1944,7 +1983,9 @@ trait FlowOps[+Out, +Mat] {
* the resulting flow will be materialized and signalled for upstream completion, it can then complete or continue to emit elements at its own discretion.
*
* '''Cancels when''' the materialized flow cancels.
- * Notice that when downstream cancels prior to prefix completion, the cancellation cause is stashed until prefix completion (or upstream completion) and then handed to the materialized flow.
+ * When downstream cancels before materialization of the nested flow, the operator's default behaviour is to cancel immediately,
+ * this behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy]] attribute on the flow.
+ * When this attribute is configured to true, downstream cancellation is delayed until the nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
*
* @param n the number of elements to accumulate before materializing the downstream flow.
* @param f a function that produces the downstream flow based on the upstream's prefix.
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/FlowWithContext.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/FlowWithContext.scala
index 395e564ecf..6fb63381b0 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/FlowWithContext.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/FlowWithContext.scala
@@ -4,9 +4,9 @@
package akka.stream.scaladsl
-import akka.NotUsed
-
import scala.annotation.unchecked.uncheckedVariance
+
+import akka.NotUsed
import akka.japi.Pair
import akka.stream._
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/FlowWithContextOps.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/FlowWithContextOps.scala
index 343904d5e6..00eadebe80 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/FlowWithContextOps.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/FlowWithContextOps.scala
@@ -4,14 +4,15 @@
package akka.stream.scaladsl
+import scala.annotation.unchecked.uncheckedVariance
import scala.collection.immutable
import scala.concurrent.Future
-import scala.annotation.unchecked.uncheckedVariance
+
import akka.NotUsed
import akka.dispatch.ExecutionContexts
+import akka.event.{ LogMarker, LoggingAdapter, MarkerLoggingAdapter }
import akka.stream._
import akka.util.ConstantFun
-import akka.event.{ LogMarker, LoggingAdapter, MarkerLoggingAdapter }
/**
* Shared stream operations for [[FlowWithContext]] and [[SourceWithContext]] that automatically propagate a context
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Framing.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Framing.scala
index fe5dde2cb4..ac9f7ab6ad 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/Framing.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Framing.scala
@@ -6,16 +6,16 @@ package akka.stream.scaladsl
import java.nio.ByteOrder
+import scala.annotation.tailrec
+import scala.reflect.ClassTag
+
import akka.NotUsed
+import akka.stream.{ Attributes, FlowShape, Inlet, Outlet }
import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
import akka.stream.stage._
-import akka.stream.{ Attributes, FlowShape, Inlet, Outlet }
import akka.util.{ ByteIterator, ByteString, OptionVal }
-import scala.annotation.tailrec
-import scala.reflect.ClassTag
-
object Framing {
/**
@@ -371,9 +371,8 @@ object Framing {
extends GraphStage[FlowShape[ByteString, ByteString]] {
//for the sake of binary compatibility
- def this(lengthFieldLength: Int, lengthFieldOffset: Int, maximumFrameLength: Int, byteOrder: ByteOrder) {
+ def this(lengthFieldLength: Int, lengthFieldOffset: Int, maximumFrameLength: Int, byteOrder: ByteOrder) =
this(lengthFieldLength, lengthFieldOffset, maximumFrameLength, byteOrder, None)
- }
private val minimumChunkSize = lengthFieldOffset + lengthFieldLength
private val intDecoder = byteOrder match {
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala
index 76f35f00fa..232db8a9f4 100755
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala
@@ -6,22 +6,22 @@ package akka.stream.scaladsl
import java.util.SplittableRandom
-import akka.NotUsed
-import akka.annotation.InternalApi
-import akka.stream._
-import akka.stream.impl.Stages.DefaultAttributes
-import akka.stream.impl._
-import akka.stream.impl.fusing.GraphStages
-import akka.stream.scaladsl.Partition.PartitionOutOfBoundsException
-import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler }
-import akka.util.ConstantFun
-
import scala.annotation.tailrec
import scala.annotation.unchecked.uncheckedVariance
import scala.collection.{ immutable, mutable }
import scala.concurrent.Promise
import scala.util.control.{ NoStackTrace, NonFatal }
+
+import akka.NotUsed
+import akka.annotation.InternalApi
+import akka.stream._
import akka.stream.ActorAttributes.SupervisionStrategy
+import akka.stream.impl._
+import akka.stream.impl.Stages.DefaultAttributes
+import akka.stream.impl.fusing.GraphStages
+import akka.stream.scaladsl.Partition.PartitionOutOfBoundsException
+import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler }
+import akka.util.ConstantFun
/**
* INTERNAL API
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Hub.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Hub.scala
index 60cd9670ad..55689134d0 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/Hub.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Hub.scala
@@ -5,26 +5,25 @@
package akka.stream.scaladsl
import java.util
-import java.util.concurrent.atomic.{ AtomicLong, AtomicReference }
-
-import akka.NotUsed
-import akka.dispatch.AbstractNodeQueue
-import akka.stream._
-import akka.stream.stage._
-import scala.annotation.tailrec
-import scala.concurrent.{ Future, Promise }
-import scala.util.{ Failure, Success, Try }
import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.{ AtomicLong, AtomicReference }
import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.atomic.AtomicReferenceArray
+import scala.annotation.tailrec
import scala.collection.immutable
-import scala.collection.mutable.LongMap
import scala.collection.immutable.Queue
+import scala.collection.mutable.LongMap
+import scala.concurrent.{ Future, Promise }
+import scala.util.{ Failure, Success, Try }
-import akka.annotation.InternalApi
+import akka.NotUsed
import akka.annotation.DoNotInherit
+import akka.annotation.InternalApi
+import akka.dispatch.AbstractNodeQueue
+import akka.stream._
import akka.stream.Attributes.LogLevels
+import akka.stream.stage._
/**
* A MergeHub is a special streaming hub that is able to collect streamed elements from a dynamic set of
@@ -1012,8 +1011,8 @@ object PartitionHub {
startAfterNrOfConsumers: Int,
bufferSize: Int)
extends GraphStageWithMaterializedValue[SinkShape[T], Source[T, NotUsed]] {
- import PartitionHub.Internal._
import PartitionHub.ConsumerInfo
+ import PartitionHub.Internal._
val in: Inlet[T] = Inlet("PartitionHub.in")
override val shape: SinkShape[T] = SinkShape(in)
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/JsonFraming.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/JsonFraming.scala
index c9614c2306..8449264632 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/JsonFraming.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/JsonFraming.scala
@@ -4,6 +4,8 @@
package akka.stream.scaladsl
+import scala.util.control.NonFatal
+
import akka.NotUsed
import akka.stream.Attributes
import akka.stream.impl.JsonObjectParser
@@ -11,8 +13,6 @@ import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
import akka.stream.stage.{ GraphStageLogic, InHandler, OutHandler }
import akka.util.ByteString
-import scala.util.control.NonFatal
-
/** Provides JSON framing operators that can separate valid JSON objects from incoming [[ByteString]] objects. */
object JsonFraming {
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/MergeLatest.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/MergeLatest.scala
index cd2d16ad82..61de2f6d80 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/MergeLatest.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/MergeLatest.scala
@@ -4,11 +4,11 @@
package akka.stream.scaladsl
-import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler }
-import akka.stream.{ Attributes, Inlet, Outlet, UniformFanInShape }
-
import scala.collection.immutable
+import akka.stream.{ Attributes, Inlet, Outlet, UniformFanInShape }
+import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler }
+
/**
* MergeLatest joins elements from N input streams into stream of lists of size N.
* i-th element in list is the latest emitted element from i-th input stream.
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Queue.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Queue.scala
index 7902877bfc..5962ddb036 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/Queue.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Queue.scala
@@ -7,14 +7,14 @@ package akka.stream.scaladsl
import java.util.Optional
import java.util.concurrent.CompletionStage
-import scala.concurrent.Future
-import akka.Done
-import akka.stream.QueueOfferResult
-
import scala.compat.java8.FutureConverters._
import scala.compat.java8.OptionConverters._
+import scala.concurrent.Future
+
+import akka.Done
import akka.annotation.InternalApi
import akka.dispatch.ExecutionContexts
+import akka.stream.QueueOfferResult
/**
* This trait allows to have a queue as a data source for some stream.
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/RestartFlow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/RestartFlow.scala
index b7900487d7..931a2819d2 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/RestartFlow.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/RestartFlow.scala
@@ -4,17 +4,17 @@
package akka.stream.scaladsl
+import scala.concurrent.duration._
+
import akka.NotUsed
import akka.event.Logging
import akka.pattern.BackoffSupervisor
-import akka.stream.Attributes.Attribute
import akka.stream._
+import akka.stream.Attributes.Attribute
+import akka.stream.Attributes.LogLevels
import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
import akka.stream.scaladsl.RestartWithBackoffFlow.Delay
import akka.stream.stage._
-
-import scala.concurrent.duration._
-import akka.stream.Attributes.LogLevels
import akka.util.OptionVal
/**
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/RestartSink.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/RestartSink.scala
index e341163ecb..8c48dea817 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/RestartSink.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/RestartSink.scala
@@ -4,12 +4,12 @@
package akka.stream.scaladsl
+import scala.concurrent.duration.FiniteDuration
+
import akka.NotUsed
import akka.stream.{ Attributes, Inlet, SinkShape }
import akka.stream.stage.{ GraphStage, InHandler }
-import scala.concurrent.duration.FiniteDuration
-
/**
* A RestartSink wraps a [[Sink]] that gets restarted when it completes or fails.
*
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/RestartSource.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/RestartSource.scala
index fa3bcbb6c0..46c063940e 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/RestartSource.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/RestartSource.scala
@@ -4,12 +4,12 @@
package akka.stream.scaladsl
+import scala.concurrent.duration.FiniteDuration
+
import akka.NotUsed
import akka.stream.{ Attributes, Outlet, SourceShape }
import akka.stream.stage.{ GraphStage, OutHandler }
-import scala.concurrent.duration.FiniteDuration
-
/**
* A RestartSource wraps a [[Source]] that gets restarted when it completes or fails.
*
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/RetryFlow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/RetryFlow.scala
index f64b67b74f..90afccf82b 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/RetryFlow.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/RetryFlow.scala
@@ -4,11 +4,11 @@
package akka.stream.scaladsl
+import scala.concurrent.duration._
+
import akka.annotation.ApiMayChange
import akka.stream.impl.RetryFlowCoordinator
-import scala.concurrent.duration._
-
object RetryFlow {
/**
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala
index 713cb6deb3..61ce286cc1 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala
@@ -4,22 +4,6 @@
package akka.stream.scaladsl
-import akka.actor.ActorRef
-import akka.actor.Status
-import akka.annotation.InternalApi
-import akka.dispatch.ExecutionContexts
-import akka.stream.impl.Stages.DefaultAttributes
-import akka.stream.impl._
-import akka.stream.impl.fusing.GraphStages
-import akka.stream.stage._
-import akka.stream.javadsl
-import akka.stream._
-import akka.util.ccompat._
-import akka.Done
-import akka.NotUsed
-import org.reactivestreams.Publisher
-import org.reactivestreams.Subscriber
-
import scala.annotation.tailrec
import scala.annotation.unchecked.uncheckedVariance
import scala.collection.immutable
@@ -29,6 +13,23 @@ import scala.util.Failure
import scala.util.Success
import scala.util.Try
+import org.reactivestreams.Publisher
+import org.reactivestreams.Subscriber
+
+import akka.Done
+import akka.NotUsed
+import akka.actor.ActorRef
+import akka.actor.Status
+import akka.annotation.InternalApi
+import akka.dispatch.ExecutionContexts
+import akka.stream._
+import akka.stream.impl._
+import akka.stream.impl.Stages.DefaultAttributes
+import akka.stream.impl.fusing.GraphStages
+import akka.stream.javadsl
+import akka.stream.stage._
+import akka.util.ccompat._
+
/**
* A `Sink` is a set of stream processing steps that has one open input.
* Can be used as a `Subscriber`
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala
index 8477f6aa16..8270764fc3 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala
@@ -6,25 +6,25 @@ package akka.stream.scaladsl
import java.util.concurrent.CompletionStage
-import akka.actor.{ ActorRef, Cancellable }
-import akka.annotation.InternalApi
-import akka.stream.impl.Stages.DefaultAttributes
-import akka.stream.impl.fusing.GraphStages
-import akka.stream.impl.fusing.GraphStages._
-import akka.stream.impl.{ PublisherSource, _ }
-import akka.stream.{ Outlet, SourceShape, _ }
-import akka.util.ConstantFun
-import akka.{ Done, NotUsed }
-import org.reactivestreams.{ Publisher, Subscriber }
-
import scala.annotation.tailrec
import scala.annotation.unchecked.uncheckedVariance
import scala.collection.immutable
-import scala.concurrent.duration.FiniteDuration
-import scala.concurrent.{ Future, Promise }
-import akka.stream.stage.GraphStageWithMaterializedValue
-
import scala.compat.java8.FutureConverters._
+import scala.concurrent.{ Future, Promise }
+import scala.concurrent.duration.FiniteDuration
+
+import org.reactivestreams.{ Publisher, Subscriber }
+
+import akka.{ Done, NotUsed }
+import akka.actor.{ ActorRef, Cancellable }
+import akka.annotation.InternalApi
+import akka.stream.{ Outlet, SourceShape, _ }
+import akka.stream.impl.{ PublisherSource, _ }
+import akka.stream.impl.Stages.DefaultAttributes
+import akka.stream.impl.fusing.GraphStages
+import akka.stream.impl.fusing.GraphStages._
+import akka.stream.stage.GraphStageWithMaterializedValue
+import akka.util.ConstantFun
/**
* A `Source` is a set of stream processing steps that has one open output. It can comprise
@@ -246,6 +246,9 @@ final class Source[+Out, +Mat](
combineRest(2, rest.iterator)
})
+ /**
+ * Transform this source whose element is ``e`` into a source producing tuple ``(e, f(e))``
+ **/
def asSourceWithContext[Ctx](f: Out => Ctx): SourceWithContext[Out, Ctx, Mat] =
new SourceWithContext(this.map(e => (e, f(e))))
}
@@ -282,6 +285,17 @@ object Source {
override def toString: String = "() => Iterator"
})
+ /**
+ * Creates a source that wraps a Java 8 ``Stream``. ``Source`` uses a stream iterator to get all its
+ * elements and send them downstream on demand.
+ *
+ * You can use [[Source.async]] to create asynchronous boundaries between synchronous Java ``Stream``
+ * and the rest of flow.
+ */
+ def fromJavaStream[T, S <: java.util.stream.BaseStream[T, S]](
+ stream: () => java.util.stream.BaseStream[T, S]): Source[T, NotUsed] =
+ StreamConverters.fromJavaStream(stream);
+
/**
* Creates [[Source]] that will continually produce given elements in specified order.
*
@@ -498,6 +512,14 @@ object Source {
def future[T](futureElement: Future[T]): Source[T, NotUsed] =
fromGraph(new FutureSource[T](futureElement))
+ /**
+ * Never emits any elements, never completes and never fails.
+ * This stream could be useful in tests.
+ */
+ def never[T]: Source[T, NotUsed] = _never
+ private[this] val _never: Source[Nothing, NotUsed] =
+ future(Future.never).withAttributes(DefaultAttributes.neverSource)
+
/**
* Emits a single value when the given `CompletionStage` is successfully completed and then completes the stream.
* If the `CompletionStage` is completed with a failure the stream is failed.
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/StreamConverters.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/StreamConverters.scala
index 6fc893c080..983416855c 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/StreamConverters.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/StreamConverters.scala
@@ -8,17 +8,17 @@ import java.io.{ InputStream, OutputStream }
import java.util.Spliterators
import java.util.stream.{ Collector, StreamSupport }
+import scala.concurrent.{ Await, Future }
+import scala.concurrent.duration._
+import scala.concurrent.duration.Duration._
+
+import akka.NotUsed
import akka.stream.{ Attributes, IOResult, SinkShape }
import akka.stream.impl._
import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.impl.io.{ InputStreamSinkStage, InputStreamSource, OutputStreamGraphStage, OutputStreamSourceStage }
import akka.util.ByteString
-import scala.concurrent.duration.Duration._
-import scala.concurrent.{ Await, Future }
-import scala.concurrent.duration._
-import akka.NotUsed
-
/**
* Converters for interacting with the blocking `java.io` streams APIs and Java 8 Streams
*/
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/SubFlow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/SubFlow.scala
index 2c66c94ef3..4cae7ac6e8 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/SubFlow.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/SubFlow.scala
@@ -4,9 +4,10 @@
package akka.stream.scaladsl
-import akka.stream._
import scala.annotation.unchecked.uncheckedVariance
+import akka.stream._
+
/**
* A “stream of streams” sub-flow of data elements, e.g. produced by `groupBy`.
* SubFlows cannot contribute to the super-flow’s materialized value since they
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/TLS.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/TLS.scala
index 0af2ab80cb..381618fa35 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/TLS.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/TLS.scala
@@ -5,18 +5,19 @@
package akka.stream.scaladsl
import java.util.Collections
-
import javax.net.ssl.{ SNIHostName, SSLContext, SSLEngine, SSLSession }
-import akka.stream.impl.io.{ TlsModule, TlsUtils }
+import javax.net.ssl.SSLParameters
+
+import scala.util.{ Failure, Success, Try }
+
+import com.typesafe.sslconfig.akka.AkkaSSLConfig
+
import akka.NotUsed
import akka.actor.ActorSystem
import akka.stream._
import akka.stream.TLSProtocol._
+import akka.stream.impl.io.{ TlsModule, TlsUtils }
import akka.util.ByteString
-import com.typesafe.sslconfig.akka.AkkaSSLConfig
-import scala.util.{ Failure, Success, Try }
-
-import javax.net.ssl.SSLParameters
/**
* Stream cipher support based upon JSSE.
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Tcp.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Tcp.scala
index 245ad3e236..94721bfdd2 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/Tcp.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Tcp.scala
@@ -6,28 +6,10 @@ package akka.stream.scaladsl
import java.net.InetSocketAddress
import java.util.concurrent.TimeoutException
-
-import akka.actor._
-import akka.annotation.InternalApi
-import akka.io.Inet.SocketOption
-import akka.io.IO
-import akka.io.{ Tcp => IoTcp }
-import akka.stream.Attributes.Attribute
-import akka.stream.TLSProtocol.NegotiateNewSession
-import akka.stream._
-import akka.stream.impl.fusing.GraphStages.detacher
-import akka.stream.impl.io.ConnectionSourceStage
-import akka.stream.impl.io.OutgoingConnectionStage
-import akka.stream.impl.io.TcpIdleTimeout
-import akka.util.ByteString
-import akka.util.unused
-import akka.util.JavaDurationConverters._
-import akka.Done
-import akka.NotUsed
-import com.github.ghik.silencer.silent
import javax.net.ssl.SSLContext
import javax.net.ssl.SSLEngine
import javax.net.ssl.SSLSession
+
import scala.collection.immutable
import scala.concurrent.Future
import scala.concurrent.duration._
@@ -37,6 +19,26 @@ import scala.util.Success
import scala.util.Try
import scala.util.control.NoStackTrace
+import com.github.ghik.silencer.silent
+
+import akka.Done
+import akka.NotUsed
+import akka.actor._
+import akka.annotation.InternalApi
+import akka.io.{ Tcp => IoTcp }
+import akka.io.IO
+import akka.io.Inet.SocketOption
+import akka.stream._
+import akka.stream.Attributes.Attribute
+import akka.stream.TLSProtocol.NegotiateNewSession
+import akka.stream.impl.fusing.GraphStages.detacher
+import akka.stream.impl.io.ConnectionSourceStage
+import akka.stream.impl.io.OutgoingConnectionStage
+import akka.stream.impl.io.TcpIdleTimeout
+import akka.util.ByteString
+import akka.util.JavaDurationConverters._
+import akka.util.unused
+
object Tcp extends ExtensionId[Tcp] with ExtensionIdProvider {
/**
diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/package.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/package.scala
index b9384fa7d2..3ed4c6d2a5 100644
--- a/akka-stream/src/main/scala/akka/stream/scaladsl/package.scala
+++ b/akka-stream/src/main/scala/akka/stream/scaladsl/package.scala
@@ -5,8 +5,9 @@
package akka.stream
import java.util.concurrent.CompletionStage
-import scala.concurrent.Future
+
import scala.compat.java8.FutureConverters
+import scala.concurrent.Future
/**
* Scala API: The flow DSL allows the formulation of stream transformations based on some
diff --git a/akka-stream/src/main/scala/akka/stream/serialization/StreamRefSerializer.scala b/akka-stream/src/main/scala/akka/stream/serialization/StreamRefSerializer.scala
index 5e21921451..a5b36331d6 100644
--- a/akka-stream/src/main/scala/akka/stream/serialization/StreamRefSerializer.scala
+++ b/akka-stream/src/main/scala/akka/stream/serialization/StreamRefSerializer.scala
@@ -4,9 +4,9 @@
package akka.stream.serialization
-import akka.protobufv3.internal.ByteString
import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi
+import akka.protobufv3.internal.ByteString
import akka.serialization._
import akka.stream.StreamRefMessages
import akka.stream.impl.streamref._
diff --git a/akka-stream/src/main/scala/akka/stream/snapshot/MaterializerState.scala b/akka-stream/src/main/scala/akka/stream/snapshot/MaterializerState.scala
index 872b84ce1d..1166c5dd31 100644
--- a/akka-stream/src/main/scala/akka/stream/snapshot/MaterializerState.scala
+++ b/akka-stream/src/main/scala/akka/stream/snapshot/MaterializerState.scala
@@ -4,20 +4,20 @@
package akka.stream.snapshot
-import akka.actor.ActorSystem
-import akka.actor.{ ActorPath, ActorRef }
-import akka.annotation.{ ApiMayChange, DoNotInherit, InternalApi }
-import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor }
-import akka.pattern.ask
-import akka.stream.SystemMaterializer
-import akka.stream.impl.fusing.ActorGraphInterpreter
-import akka.stream.{ Attributes, Materializer }
-import akka.util.Timeout
-
import scala.collection.immutable
import scala.concurrent.{ ExecutionContext, Future }
import scala.concurrent.duration._
+import akka.actor.{ ActorPath, ActorRef }
+import akka.actor.ActorSystem
+import akka.annotation.{ ApiMayChange, DoNotInherit, InternalApi }
+import akka.pattern.ask
+import akka.stream.{ Attributes, Materializer }
+import akka.stream.SystemMaterializer
+import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor }
+import akka.stream.impl.fusing.ActorGraphInterpreter
+import akka.util.Timeout
+
/**
* Debug utility to dump the running streams of a materializers in a structure describing the graph layout
* and "waits-on" relationships.
@@ -170,8 +170,9 @@ final private[akka] case class StreamSnapshotImpl(
self: ActorPath,
activeInterpreters: Seq[RunningInterpreter],
newShells: Seq[UninitializedInterpreter])
- extends StreamSnapshot
- with HideImpl
+ extends StreamSnapshot {
+ override def toString: String = s"StreamSnapshot($self, $activeInterpreters, $newShells)"
+}
/**
* INTERNAL API
diff --git a/akka-stream/src/main/scala/akka/stream/stage/GraphStage.scala b/akka-stream/src/main/scala/akka/stream/stage/GraphStage.scala
index 353b396c15..ce8da0c079 100644
--- a/akka-stream/src/main/scala/akka/stream/stage/GraphStage.scala
+++ b/akka-stream/src/main/scala/akka/stream/stage/GraphStage.scala
@@ -6,23 +6,24 @@ package akka.stream.stage
import java.util.concurrent.atomic.AtomicReference
+import scala.annotation.tailrec
+import scala.collection.{ immutable, mutable }
+import scala.concurrent.{ Future, Promise }
+import scala.concurrent.duration.FiniteDuration
+
+import com.github.ghik.silencer.silent
+
+import akka.{ Done, NotUsed }
import akka.actor._
import akka.annotation.InternalApi
import akka.japi.function.{ Effect, Procedure }
import akka.stream._
+import akka.stream.impl.{ ReactiveStreamsCompliance, TraversalBuilder }
import akka.stream.impl.ActorSubscriberMessage
import akka.stream.impl.fusing.{ GraphInterpreter, GraphStageModule, SubSink, SubSource }
-import akka.stream.impl.{ ReactiveStreamsCompliance, TraversalBuilder }
import akka.stream.scaladsl.GenericGraphWithChangedAttributes
import akka.util.OptionVal
import akka.util.unused
-import akka.{ Done, NotUsed }
-
-import scala.annotation.tailrec
-import scala.collection.{ immutable, mutable }
-import scala.concurrent.duration.FiniteDuration
-import scala.concurrent.{ Future, Promise }
-import com.github.ghik.silencer.silent
/**
* Scala API: A GraphStage represents a reusable graph stream processing operator.
diff --git a/akka-stream/src/main/scala/com/typesafe/sslconfig/akka/AkkaSSLConfig.scala b/akka-stream/src/main/scala/com/typesafe/sslconfig/akka/AkkaSSLConfig.scala
index df410fa7c2..3aad9c62d6 100644
--- a/akka-stream/src/main/scala/com/typesafe/sslconfig/akka/AkkaSSLConfig.scala
+++ b/akka-stream/src/main/scala/com/typesafe/sslconfig/akka/AkkaSSLConfig.scala
@@ -7,15 +7,16 @@ package com.typesafe.sslconfig.akka
import java.security.KeyStore
import java.security.cert.CertPathValidatorException
import java.util.Collections
-
import javax.net.ssl._
-import akka.actor._
-import akka.annotation.InternalApi
-import akka.event.Logging
+
import com.typesafe.sslconfig.akka.util.AkkaLoggerFactory
import com.typesafe.sslconfig.ssl._
import com.typesafe.sslconfig.util.LoggerFactory
+import akka.actor._
+import akka.annotation.InternalApi
+import akka.event.Logging
+
@deprecated("Use Tcp and TLS with SSLEngine parameters instead. Setup the SSLEngine with needed parameters.", "2.6.0")
object AkkaSSLConfig extends ExtensionId[AkkaSSLConfig] with ExtensionIdProvider {
diff --git a/akka-stream/src/main/scala/com/typesafe/sslconfig/akka/util/AkkaLoggerBridge.scala b/akka-stream/src/main/scala/com/typesafe/sslconfig/akka/util/AkkaLoggerBridge.scala
index 08c22a1f9e..91f474159c 100644
--- a/akka-stream/src/main/scala/com/typesafe/sslconfig/akka/util/AkkaLoggerBridge.scala
+++ b/akka-stream/src/main/scala/com/typesafe/sslconfig/akka/util/AkkaLoggerBridge.scala
@@ -4,10 +4,11 @@
package com.typesafe.sslconfig.akka.util
+import com.typesafe.sslconfig.util.{ LoggerFactory, NoDepsLogger }
+
import akka.actor.ActorSystem
import akka.event.{ DummyClassForStringSources, EventStream }
import akka.event.Logging._
-import com.typesafe.sslconfig.util.{ LoggerFactory, NoDepsLogger }
final class AkkaLoggerFactory(system: ActorSystem) extends LoggerFactory {
override def apply(clazz: Class[_]): NoDepsLogger = new AkkaLoggerBridge(system.eventStream, clazz)
@@ -17,7 +18,7 @@ final class AkkaLoggerFactory(system: ActorSystem) extends LoggerFactory {
}
class AkkaLoggerBridge(bus: EventStream, logSource: String, logClass: Class[_]) extends NoDepsLogger {
- def this(bus: EventStream, clazz: Class[_]) { this(bus, clazz.getCanonicalName, clazz) }
+ def this(bus: EventStream, clazz: Class[_]) = this(bus, clazz.getCanonicalName, clazz)
override def isDebugEnabled: Boolean = true
diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala
index 41b90be549..fc1d36a36e 100644
--- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala
@@ -4,12 +4,18 @@
package akka.testkit
+import java.lang.ref.WeakReference
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.locks.ReentrantLock
+
+import scala.annotation.tailrec
+import scala.concurrent.duration._
+import scala.concurrent.duration.Duration
+import scala.util.control.NonFatal
+
+import com.typesafe.config.Config
import language.postfixOps
-import java.lang.ref.WeakReference
-import java.util.concurrent.locks.ReentrantLock
-import scala.annotation.tailrec
-import com.typesafe.config.Config
import akka.actor.{
ActorCell,
ActorInitializationException,
@@ -31,11 +37,7 @@ import akka.dispatch.{
TaskInvocation
}
import akka.dispatch.sysmsg.{ Resume, Suspend, SystemMessage }
-import scala.concurrent.duration._
import akka.util.Switch
-import scala.concurrent.duration.Duration
-import scala.util.control.NonFatal
-import java.util.concurrent.TimeUnit
/*
* Locking rules:
@@ -77,7 +79,7 @@ private[testkit] class CallingThreadDispatcherQueues extends Extension {
val nv = v.filter(_.get ne null)
if (nv.isEmpty) m else m += (k -> nv)
}
- .result
+ .result()
}
protected[akka] def registerQueue(mbox: CallingThreadMailbox, q: MessageQueue): Unit = synchronized {
@@ -236,7 +238,7 @@ class CallingThreadDispatcher(_configurator: MessageDispatcherConfigurator) exte
}
}
- protected[akka] override def executeTask(invocation: TaskInvocation): Unit = { invocation.run }
+ protected[akka] override def executeTask(invocation: TaskInvocation): Unit = { invocation.run() }
/*
* This method must be called with this thread's queue.
@@ -276,7 +278,7 @@ class CallingThreadDispatcher(_configurator: MessageDispatcherConfigurator) exte
}
if (handle ne null) {
try {
- if (Mailbox.debug) println(mbox.actor.self + " processing message " + handle)
+ if (Mailbox.debug) println("" + mbox.actor.self + " processing message " + handle)
mbox.actor.invoke(handle)
intex = checkThreadInterruption(intex)
true
diff --git a/akka-testkit/src/main/scala/akka/testkit/ExplicitlyTriggeredScheduler.scala b/akka-testkit/src/main/scala/akka/testkit/ExplicitlyTriggeredScheduler.scala
index 9a022d3830..99d7bfa2f6 100644
--- a/akka-testkit/src/main/scala/akka/testkit/ExplicitlyTriggeredScheduler.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/ExplicitlyTriggeredScheduler.scala
@@ -4,21 +4,22 @@
package akka.testkit
-import java.util.concurrent.ThreadFactory
import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.ThreadFactory
import java.util.concurrent.atomic.AtomicLong
import scala.annotation.tailrec
-import akka.util.ccompat.JavaConverters._
-
import scala.concurrent.ExecutionContext
import scala.concurrent.duration.{ Duration, FiniteDuration }
import scala.util.Try
+
+import com.typesafe.config.Config
+
import akka.actor.Cancellable
import akka.actor.Scheduler
import akka.event.LoggingAdapter
+import akka.util.ccompat.JavaConverters._
import akka.util.unused
-import com.typesafe.config.Config
/**
* For testing: scheduler that does not look at the clock, but must be
diff --git a/akka-testkit/src/main/scala/akka/testkit/SocketUtil.scala b/akka-testkit/src/main/scala/akka/testkit/SocketUtil.scala
index d3a2a97906..be5af84012 100644
--- a/akka-testkit/src/main/scala/akka/testkit/SocketUtil.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/SocketUtil.scala
@@ -4,12 +4,12 @@
package akka.testkit
-import scala.collection.immutable
-import scala.util.Random
import java.net.{ DatagramSocket, InetSocketAddress, NetworkInterface, StandardProtocolFamily }
import java.nio.channels.DatagramChannel
import java.nio.channels.ServerSocketChannel
+import scala.collection.immutable
+import scala.util.Random
import scala.util.control.NonFatal
/**
@@ -30,9 +30,9 @@ object SocketUtil {
}
sealed trait Protocol
- final case object Tcp extends Protocol
- final case object Udp extends Protocol
- final case object Both extends Protocol
+ case object Tcp extends Protocol
+ case object Udp extends Protocol
+ case object Both extends Protocol
/** @return A port on 'localhost' that is currently available */
def temporaryLocalPort(udp: Boolean = false): Int = temporaryServerAddress("localhost", udp).getPort
diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala
index 52a1e21f4a..0bab6a9dbd 100644
--- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala
@@ -4,16 +4,17 @@
package akka.testkit
-import akka.actor._
import java.util.concurrent.atomic.AtomicLong
-import akka.dispatch._
-
import scala.concurrent.Await
import scala.reflect.ClassTag
-import akka.pattern.ask
+
import com.github.ghik.silencer.silent
+import akka.actor._
+import akka.dispatch._
+import akka.pattern.ask
+
/**
* This special ActorRef is exclusively for use during unit testing in a single-threaded environment. Therefore, it
* overrides the dispatcher to CallingThreadDispatcher and sets the receiveTimeout to None. Otherwise,
diff --git a/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala b/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala
index c9541a3b19..7c704e1f03 100644
--- a/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala
@@ -4,11 +4,13 @@
package akka.testkit
-import scala.concurrent.duration.Duration
import java.util.concurrent.{ CyclicBarrier, TimeUnit, TimeoutException }
-import akka.actor.ActorSystem
+
+import scala.concurrent.duration.Duration
import scala.concurrent.duration.FiniteDuration
+import akka.actor.ActorSystem
+
class TestBarrierTimeoutException(message: String) extends RuntimeException(message)
/**
diff --git a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala
index 07c53f59f1..978da1cad0 100644
--- a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala
@@ -4,20 +4,20 @@
package akka.testkit
-import scala.util.matching.Regex
+import java.lang.{ Iterable => JIterable }
+
import scala.collection.immutable
import scala.concurrent.duration.Duration
import scala.reflect.ClassTag
+import scala.util.matching.Regex
import akka.actor.{ ActorSystem, DeadLetter, UnhandledMessage }
-import akka.dispatch.sysmsg.{ SystemMessage, Terminate }
-import akka.event.Logging.{ Debug, Error, Info, InitializeLogger, LogEvent, LoggerInitialized, Warning }
-import akka.event.Logging
-import akka.actor.NoSerializationVerificationNeeded
-import akka.japi.Util.immutableSeq
-import java.lang.{ Iterable => JIterable }
-
import akka.actor.Dropped
+import akka.actor.NoSerializationVerificationNeeded
+import akka.dispatch.sysmsg.{ SystemMessage, Terminate }
+import akka.event.Logging
+import akka.event.Logging.{ Debug, Error, Info, InitializeLogger, LogEvent, LoggerInitialized, Warning }
+import akka.japi.Util.immutableSeq
import akka.util.BoxedType
import akka.util.ccompat._
diff --git a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala
index b7f8a46475..080035663f 100644
--- a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala
@@ -4,10 +4,11 @@
package akka.testkit
-import akka.actor._
import scala.concurrent.duration.FiniteDuration
import scala.reflect.ClassTag
+import akka.actor._
+
/**
* This is a specialized form of the TestActorRef with support for querying and
* setting the state of a FSM. Use a LoggingFSM with this class if you also
diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala
index 566485c3fa..f4266ee317 100644
--- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala
@@ -5,23 +5,25 @@
package akka.testkit
import java.util.concurrent._
-import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicInteger
-import scala.language.postfixOps
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.Await
import scala.concurrent.duration._
+import scala.language.postfixOps
import scala.reflect.ClassTag
import scala.util.control.NonFatal
+
+import com.github.ghik.silencer.silent
+
import akka.actor._
-import akka.util.{ BoxedType, Timeout }
-import akka.actor.IllegalActorStateException
import akka.actor.DeadLetter
+import akka.actor.IllegalActorStateException
import akka.actor.Terminated
import akka.annotation.InternalApi
-import com.github.ghik.silencer.silent
+import akka.util.{ BoxedType, Timeout }
object TestActor {
type Ignore = Option[PartialFunction[Any, Boolean]]
@@ -964,7 +966,7 @@ trait TestKitBase {
* @since 1.1
*/
@silent // 'early initializers' are deprecated on 2.13 and will be replaced with trait parameters on 2.14. https://github.com/akka/akka/issues/26753
-class TestKit(_system: ActorSystem) extends { implicit val system = _system } with TestKitBase
+class TestKit(_system: ActorSystem) extends { implicit val system: ActorSystem = _system } with TestKitBase
object TestKit {
@@ -1071,7 +1073,7 @@ object TestProbe {
}
trait ImplicitSender { this: TestKitBase =>
- implicit def self = testActor
+ implicit def self: ActorRef = testActor
}
trait DefaultTimeout { this: TestKitBase =>
diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala b/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala
index 109d3c7b50..2a45470471 100644
--- a/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala
@@ -4,13 +4,14 @@
package akka.testkit
-import akka.actor.ClassicActorSystemProvider
-import com.typesafe.config.Config
-import akka.util.Timeout
-import akka.actor.{ ActorSystem, ExtendedActorSystem, Extension, ExtensionId }
-
import scala.concurrent.duration.FiniteDuration
+import com.typesafe.config.Config
+
+import akka.actor.{ ActorSystem, ExtendedActorSystem, Extension, ExtensionId }
+import akka.actor.ClassicActorSystemProvider
+import akka.util.Timeout
+
object TestKitExtension extends ExtensionId[TestKitSettings] {
override def get(system: ActorSystem): TestKitSettings = super.get(system)
override def get(system: ClassicActorSystemProvider): TestKitSettings = super.get(system)
diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKitUtils.scala b/akka-testkit/src/main/scala/akka/testkit/TestKitUtils.scala
new file mode 100644
index 0000000000..f8149dfc56
--- /dev/null
+++ b/akka-testkit/src/main/scala/akka/testkit/TestKitUtils.scala
@@ -0,0 +1,62 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka.testkit
+
+import java.lang.reflect.Modifier
+
+import scala.util.matching.Regex
+
+import akka.annotation.InternalApi
+
+/**
+ * INTERNAL API
+ */
+@InternalApi
+private[akka] object TestKitUtils {
+
+ def testNameFromCallStack(classToStartFrom: Class[_], testKitRegex: Regex): String = {
+
+ def isAbstractClass(className: String): Boolean = {
+ try {
+ Modifier.isAbstract(Class.forName(className).getModifiers)
+ } catch {
+ case _: Throwable => false // yes catch everything, best effort check
+ }
+ }
+
+ val startFrom = classToStartFrom.getName
+ val filteredStack = Thread.currentThread.getStackTrace.iterator
+ .map(_.getClassName)
+ // drop until we find the first occurrence of classToStartFrom
+ .dropWhile(!_.startsWith(startFrom))
+ // then continue to the next entry after classToStartFrom that makes sense
+ .dropWhile {
+ case `startFrom` => true
+ case str if str.startsWith(startFrom + "$") => true // lambdas inside startFrom etc
+ case testKitRegex() => true // testkit internals
+ case str if isAbstractClass(str) => true
+ case _ => false
+ }
+
+ if (filteredStack.isEmpty)
+ throw new IllegalArgumentException(s"Couldn't find [${classToStartFrom.getName}] in call stack")
+
+ // sanitize for actor system name
+ scrubActorSystemName(filteredStack.next())
+ }
+
+ /**
+ * Sanitize the `name` to be used as valid actor system name by
+ * replacing invalid characters. `name` may for example be a fully qualified
+ * class name and then the short class name will be used.
+ */
+ def scrubActorSystemName(name: String): String = {
+ name
+ .replaceFirst("""^.*\.""", "") // drop package name
+ .replaceAll("""\$\$?\w+""", "") // drop scala anonymous functions/classes
+ .replaceAll("[^a-zA-Z_0-9]", "_")
+ .replaceAll("""MultiJvmNode\d+""", "") // drop MultiJvm suffix
+ }
+}
diff --git a/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala b/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala
index f6d31a0989..a0d79d8cc4 100644
--- a/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala
@@ -4,12 +4,14 @@
package akka.testkit
-import scala.concurrent.duration.Duration
-import akka.actor.ActorSystem
-import scala.concurrent.{ Awaitable, CanAwait }
import java.util.concurrent.{ CountDownLatch, TimeUnit, TimeoutException }
+
+import scala.concurrent.{ Awaitable, CanAwait }
+import scala.concurrent.duration.Duration
import scala.concurrent.duration.FiniteDuration
+import akka.actor.ActorSystem
+
/**
* A count down latch wrapper for use in testing.
* It always uses a timeout when waiting and timeouts are specified as durations.
diff --git a/akka-testkit/src/main/scala/akka/testkit/javadsl/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/javadsl/TestKit.scala
index 31b6328f1d..5b3d16520a 100644
--- a/akka-testkit/src/main/scala/akka/testkit/javadsl/TestKit.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/javadsl/TestKit.scala
@@ -4,19 +4,19 @@
package akka.testkit.javadsl
-import java.util.function.{ Supplier, Function => JFunction }
import java.util.{ List => JList }
+import java.util.function.{ Supplier, Function => JFunction }
+
+import scala.annotation.varargs
+import scala.concurrent.duration._
+
+import com.github.ghik.silencer.silent
import akka.actor._
import akka.annotation.InternalApi
import akka.testkit.{ TestActor, TestDuration, TestProbe }
import akka.util.JavaDurationConverters._
-
-import scala.annotation.varargs
import akka.util.ccompat.JavaConverters._
-import com.github.ghik.silencer.silent
-
-import scala.concurrent.duration._
/**
* Java API: Test kit for testing actors. Inheriting from this class enables
diff --git a/akka-testkit/src/main/scala/akka/testkit/package.scala b/akka-testkit/src/main/scala/akka/testkit/package.scala
index f751b111f8..c3b934771a 100644
--- a/akka-testkit/src/main/scala/akka/testkit/package.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/package.scala
@@ -4,12 +4,14 @@
package akka
-import akka.actor.ActorSystem
-import akka.util.ccompat._
+import java.util.concurrent.TimeUnit.MILLISECONDS
+
+import scala.collection.immutable
import scala.concurrent.duration.{ Duration, FiniteDuration }
import scala.reflect.ClassTag
-import scala.collection.immutable
-import java.util.concurrent.TimeUnit.MILLISECONDS
+
+import akka.actor.ActorSystem
+import akka.util.ccompat._
package object testkit {
@ccompatUsedUntil213
diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala
index 6610e726db..498f26f91c 100644
--- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala
@@ -4,13 +4,10 @@
package akka.testkit
-import java.lang.reflect.Modifier
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.language.postfixOps
-import akka.actor.ActorSystem
-import akka.dispatch.Dispatchers
-import akka.event.Logging
-import akka.event.LoggingAdapter
-import akka.testkit.TestEvent._
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import org.scalactic.CanEqual
@@ -22,9 +19,11 @@ import org.scalatest.time.Millis
import org.scalatest.time.Span
import org.scalatest.wordspec.AnyWordSpecLike
-import scala.concurrent.Future
-import scala.concurrent.duration._
-import scala.language.postfixOps
+import akka.actor.ActorSystem
+import akka.dispatch.Dispatchers
+import akka.event.Logging
+import akka.event.LoggingAdapter
+import akka.testkit.TestEvent._
object AkkaSpec {
val testConf: Config = ConfigFactory.parseString("""
@@ -43,54 +42,13 @@ object AkkaSpec {
}
}
}
- """)
+ """)
def mapToConfig(map: Map[String, Any]): Config = {
import akka.util.ccompat.JavaConverters._
ConfigFactory.parseMap(map.asJava)
}
- def testNameFromCallStack(classToStartFrom: Class[_]): String = {
-
- def isAbstractClass(className: String): Boolean = {
- try {
- Modifier.isAbstract(Class.forName(className).getModifiers)
- } catch {
- case _: Throwable => false // yes catch everything, best effort check
- }
- }
-
- val startFrom = classToStartFrom.getName
- val filteredStack = Thread.currentThread.getStackTrace.iterator
- .map(_.getClassName)
- // drop until we find the first occurrence of classToStartFrom
- .dropWhile(!_.startsWith(startFrom))
- // then continue to the next entry after classToStartFrom that makes sense
- .dropWhile {
- case `startFrom` => true
- case str if str.startsWith(startFrom + "$") => true // lambdas inside startFrom etc
- case str if isAbstractClass(str) => true
- case _ => false
- }
-
- if (filteredStack.isEmpty)
- throw new IllegalArgumentException(s"Couldn't find [${classToStartFrom.getName}] in call stack")
-
- // sanitize for actor system name
- scrubActorSystemName(filteredStack.next())
- }
-
- /**
- * Sanitize the `name` to be used as valid actor system name by
- * replacing invalid characters. `name` may for example be a fully qualified
- * class name and then the short class name will be used.
- */
- def scrubActorSystemName(name: String): String = {
- name
- .replaceFirst("""^.*\.""", "") // drop package name
- .replaceAll("""\$\$?\w+""", "") // drop scala anonymous functions/classes
- .replaceAll("[^a-zA-Z_0-9]", "_")
- }
}
abstract class AkkaSpec(_system: ActorSystem)
@@ -102,19 +60,19 @@ abstract class AkkaSpec(_system: ActorSystem)
with TypeCheckedTripleEquals
with ScalaFutures {
- implicit val patience = PatienceConfig(testKitSettings.DefaultTimeout.duration, Span(100, Millis))
+ implicit val patience: PatienceConfig = PatienceConfig(testKitSettings.DefaultTimeout.duration, Span(100, Millis))
def this(config: Config) =
this(
ActorSystem(
- AkkaSpec.testNameFromCallStack(classOf[AkkaSpec]),
+ TestKitUtils.testNameFromCallStack(classOf[AkkaSpec], "".r),
ConfigFactory.load(config.withFallback(AkkaSpec.testConf))))
def this(s: String) = this(ConfigFactory.parseString(s))
def this(configMap: Map[String, _]) = this(AkkaSpec.mapToConfig(configMap))
- def this() = this(ActorSystem(AkkaSpec.testNameFromCallStack(classOf[AkkaSpec]), AkkaSpec.testConf))
+ def this() = this(ActorSystem(TestKitUtils.testNameFromCallStack(classOf[AkkaSpec], "".r), AkkaSpec.testConf))
val log: LoggingAdapter = Logging(system, this.getClass)
diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpecSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpecSpec.scala
index e5bc37f12b..9b32535ab0 100644
--- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpecSpec.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpecSpec.scala
@@ -4,18 +4,19 @@
package akka.testkit
-import language.postfixOps
-import akka.actor._
-import com.typesafe.config.ConfigFactory
-
import scala.concurrent.Await
import scala.concurrent.duration._
-import akka.actor.DeadLetter
-import akka.pattern.ask
+
import com.github.ghik.silencer.silent
+import com.typesafe.config.ConfigFactory
+import language.postfixOps
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor._
+import akka.actor.DeadLetter
+import akka.pattern.ask
+
@silent
class AkkaSpecSpec extends AnyWordSpec with Matchers {
diff --git a/akka-testkit/src/test/scala/akka/testkit/Coroner.scala b/akka-testkit/src/test/scala/akka/testkit/Coroner.scala
index 8423639e2d..df159fd32a 100644
--- a/akka-testkit/src/test/scala/akka/testkit/Coroner.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/Coroner.scala
@@ -8,6 +8,7 @@ import java.io.PrintStream
import java.lang.management.{ ManagementFactory, ThreadInfo }
import java.util.Date
import java.util.concurrent.{ CountDownLatch, TimeoutException }
+
import scala.concurrent.{ Await, Awaitable, CanAwait, Promise }
import scala.concurrent.duration._
import scala.util.control.NonFatal
@@ -39,7 +40,7 @@ object Coroner {
}
private class WatchHandleImpl(startAndStopDuration: FiniteDuration) extends WatchHandle {
- val cancelPromise = Promise[Boolean]
+ val cancelPromise = Promise[Boolean]()
val startedLatch = new CountDownLatch(1)
val finishedLatch = new CountDownLatch(1)
diff --git a/akka-testkit/src/test/scala/akka/testkit/CoronerSpec.scala b/akka-testkit/src/test/scala/akka/testkit/CoronerSpec.scala
index da8d466465..ad43b9cb14 100644
--- a/akka-testkit/src/test/scala/akka/testkit/CoronerSpec.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/CoronerSpec.scala
@@ -8,8 +8,10 @@ import java.io._
import java.lang.management.ManagementFactory
import java.util.concurrent.Semaphore
import java.util.concurrent.locks.ReentrantLock
-import scala.concurrent.duration._
+
import scala.concurrent.Await
+import scala.concurrent.duration._
+
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
diff --git a/akka-testkit/src/test/scala/akka/testkit/DefaultTimeoutSpec.scala b/akka-testkit/src/test/scala/akka/testkit/DefaultTimeoutSpec.scala
index 88894c153a..20bb50f0f3 100644
--- a/akka-testkit/src/test/scala/akka/testkit/DefaultTimeoutSpec.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/DefaultTimeoutSpec.scala
@@ -5,15 +5,16 @@
package akka.testkit
import org.scalatest.BeforeAndAfterAll
-import akka.actor.ActorSystem
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.ActorSystem
+
class DefaultTimeoutSpec extends AnyWordSpec with Matchers with BeforeAndAfterAll with TestKitBase with DefaultTimeout {
- implicit lazy val system = ActorSystem("AkkaCustomSpec")
+ implicit lazy val system: ActorSystem = ActorSystem("AkkaCustomSpec")
- override def afterAll = system.terminate
+ override def afterAll = system.terminate()
"A spec with DefaultTimeout" should {
"use timeout from settings" in {
diff --git a/akka-testkit/src/test/scala/akka/testkit/ImplicitSenderSpec.scala b/akka-testkit/src/test/scala/akka/testkit/ImplicitSenderSpec.scala
index 5abbbe449e..b81721d363 100644
--- a/akka-testkit/src/test/scala/akka/testkit/ImplicitSenderSpec.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/ImplicitSenderSpec.scala
@@ -4,16 +4,17 @@
package akka.testkit
-import akka.actor.ActorSystem
import org.scalatest.BeforeAndAfterAll
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
+import akka.actor.ActorSystem
+
class ImplicitSenderSpec extends AnyWordSpec with Matchers with BeforeAndAfterAll with TestKitBase with ImplicitSender {
- implicit lazy val system = ActorSystem("AkkaCustomSpec")
+ implicit lazy val system: ActorSystem = ActorSystem("AkkaCustomSpec")
- override def afterAll = system.terminate
+ override def afterAll = system.terminate()
"An ImplicitSender" should {
"have testActor as its self" in {
diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala
index aa6c8e6296..0844021d19 100644
--- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala
@@ -4,14 +4,16 @@
package akka.testkit
-import language.{ postfixOps }
-import org.scalatest.{ BeforeAndAfterEach }
-import akka.actor._
-import akka.event.Logging.Warning
import scala.concurrent.{ Await, Promise }
import scala.concurrent.duration._
-import akka.pattern.ask
+
+import language.postfixOps
+import org.scalatest.BeforeAndAfterEach
+
+import akka.actor._
import akka.dispatch.Dispatcher
+import akka.event.Logging.Warning
+import akka.pattern.ask
/**
* Test whether TestActorRef behaves as an ActorRef should, besides its own spec.
@@ -42,11 +44,11 @@ object TestActorRefSpec {
def receiveT = {
case "complexRequest" => {
replyTo = sender()
- val worker = TestActorRef(Props[WorkerActor])
+ val worker = TestActorRef(Props[WorkerActor]())
worker ! "work"
}
case "complexRequest2" =>
- val worker = TestActorRef(Props[WorkerActor])
+ val worker = TestActorRef(Props[WorkerActor]())
worker ! sender()
case "workDone" => replyTo ! "complexReply"
case "simpleRequest" => sender() ! "simpleReply"
@@ -143,7 +145,7 @@ class TestActorRefSpec extends AkkaSpec("disp1.type=Dispatcher") with BeforeAndA
}
"support reply via sender()" in {
- val serverRef = TestActorRef(Props[ReplyActor])
+ val serverRef = TestActorRef(Props[ReplyActor]())
val clientRef = TestActorRef(Props(classOf[SenderActor], serverRef))
counter = 4
@@ -169,7 +171,7 @@ class TestActorRefSpec extends AkkaSpec("disp1.type=Dispatcher") with BeforeAndA
"stop when sent a poison pill" in {
EventFilter[ActorKilledException]().intercept {
- val a = TestActorRef(Props[WorkerActor])
+ val a = TestActorRef(Props[WorkerActor]())
system.actorOf(Props(new Actor {
context.watch(a)
def receive = {
@@ -251,7 +253,7 @@ class TestActorRefSpec extends AkkaSpec("disp1.type=Dispatcher") with BeforeAndA
}
"allow override of dispatcher" in {
- val a = TestActorRef(Props[WorkerActor].withDispatcher("disp1"))
+ val a = TestActorRef(Props[WorkerActor]().withDispatcher("disp1"))
a.underlying.dispatcher.getClass should ===(classOf[Dispatcher])
}
@@ -323,24 +325,24 @@ class TestActorRefSpec extends AkkaSpec("disp1.type=Dispatcher") with BeforeAndA
}
"allow creation of a TestActorRef with a default supervisor with Props" in {
- val ref = TestActorRef[WorkerActor](Props[WorkerActor])
+ val ref = TestActorRef[WorkerActor](Props[WorkerActor]())
ref.underlyingActor.supervisor should be(system.asInstanceOf[ActorSystemImpl].guardian)
}
"allow creation of a TestActorRef with a default supervisor and specified name with Props" in {
- val ref = TestActorRef[WorkerActor](Props[WorkerActor], "specificPropsActor")
+ val ref = TestActorRef[WorkerActor](Props[WorkerActor](), "specificPropsActor")
ref.underlyingActor.name should be("specificPropsActor")
}
"allow creation of a TestActorRef with a specified supervisor with Props" in {
val parent = TestActorRef[ReplyActor]
- val ref = TestActorRef[WorkerActor](Props[WorkerActor], parent)
+ val ref = TestActorRef[WorkerActor](Props[WorkerActor](), parent)
ref.underlyingActor.supervisor should be(parent)
}
"allow creation of a TestActorRef with a specified supervisor and specified name with Props" in {
val parent = TestActorRef[ReplyActor]
- val ref = TestActorRef[WorkerActor](Props[WorkerActor], parent, "specificSupervisedPropsActor")
+ val ref = TestActorRef[WorkerActor](Props[WorkerActor](), parent, "specificSupervisedPropsActor")
ref.underlyingActor.name should be("specificSupervisedPropsActor")
ref.underlyingActor.supervisor should be(parent)
}
diff --git a/akka-testkit/src/test/scala/akka/testkit/TestFSMRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestFSMRefSpec.scala
index cd8eb69f00..f4b967f0fa 100644
--- a/akka-testkit/src/test/scala/akka/testkit/TestFSMRefSpec.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/TestFSMRefSpec.scala
@@ -4,10 +4,11 @@
package akka.testkit
+import scala.concurrent.duration._
+
import language.postfixOps
import akka.actor._
-import scala.concurrent.duration._
class TestFSMRefSpec extends AkkaSpec {
@@ -45,7 +46,7 @@ class TestFSMRefSpec extends AkkaSpec {
val fsm = TestFSMRef(new Actor with FSM[Int, Null] {
startWith(1, null)
when(1) {
- case _ => stay
+ case _ => stay()
}
}, "test-fsm-ref-2")
fsm.isTimerActive("test") should ===(false)
@@ -65,7 +66,7 @@ class TestFSMRefSpec extends AkkaSpec {
class TestFSMActor extends Actor with FSM[Int, Null] {
startWith(1, null)
when(1) {
- case _ => stay
+ case _ => stay()
}
val supervisor = context.parent
val name = context.self.path.name
diff --git a/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala
index 1e152fd19c..37b96b091c 100644
--- a/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala
@@ -4,18 +4,18 @@
package akka.testkit
-import language.postfixOps
-import akka.actor._
+import java.util.concurrent.atomic.AtomicInteger
import scala.concurrent.Await
import scala.concurrent.duration._
-import akka.pattern.ask
-
import scala.util.Try
-import java.util.concurrent.atomic.AtomicInteger
+import language.postfixOps
import org.scalatest.concurrent.Eventually
+import akka.actor._
+import akka.pattern.ask
+
class TestProbeSpec extends AkkaSpec with DefaultTimeout with Eventually {
"A TestProbe" must {
diff --git a/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala
index 24f50a915c..f3e319c03d 100644
--- a/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala
@@ -5,6 +5,7 @@
package akka.testkit
import scala.concurrent.duration._
+
import org.scalatest.exceptions.TestFailedException
class TestTimeSpec extends AkkaSpec(Map("akka.test.timefactor" -> 2.0)) {
diff --git a/akka-testkit/src/test/scala/akka/testkit/WithLogCapturing.scala b/akka-testkit/src/test/scala/akka/testkit/WithLogCapturing.scala
index 48496a8515..984fff4d3f 100644
--- a/akka-testkit/src/test/scala/akka/testkit/WithLogCapturing.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/WithLogCapturing.scala
@@ -6,10 +6,11 @@ package akka.testkit
import java.io.{ OutputStream, PrintStream }
+import org.scalatest.{ Outcome, SuiteMixin, TestSuite }
+
import akka.actor.ActorSystem
import akka.event.Logging
import akka.event.Logging._
-import org.scalatest.{ Outcome, SuiteMixin, TestSuite }
/**
* Mixin this trait to a test to make log lines appear only when the test failed.
diff --git a/akka-testkit/src/test/scala/akka/testkit/metrics/FileDescriptorMetricSet.scala b/akka-testkit/src/test/scala/akka/testkit/metrics/FileDescriptorMetricSet.scala
index e4760de308..194341358d 100644
--- a/akka-testkit/src/test/scala/akka/testkit/metrics/FileDescriptorMetricSet.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/metrics/FileDescriptorMetricSet.scala
@@ -4,13 +4,15 @@
package akka.testkit.metrics
-import java.util
-import akka.util.ccompat.JavaConverters._
import java.lang.management.{ ManagementFactory, OperatingSystemMXBean }
+import java.util
+
import com.codahale.metrics.{ Gauge, Metric, MetricSet }
import com.codahale.metrics.MetricRegistry._
import com.codahale.metrics.jvm.FileDescriptorRatioGauge
+import akka.util.ccompat.JavaConverters._
+
/**
* MetricSet exposing number of open and maximum file descriptors used by the JVM process.
*/
diff --git a/akka-testkit/src/test/scala/akka/testkit/metrics/KnownOpsInTimespanTimer.scala b/akka-testkit/src/test/scala/akka/testkit/metrics/KnownOpsInTimespanTimer.scala
index 36fd031354..96e3da995c 100644
--- a/akka-testkit/src/test/scala/akka/testkit/metrics/KnownOpsInTimespanTimer.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/metrics/KnownOpsInTimespanTimer.scala
@@ -4,9 +4,10 @@
package akka.testkit.metrics
-import com.codahale.metrics._
-import java.util.concurrent.atomic.AtomicLong
import java.util.concurrent.TimeUnit._
+import java.util.concurrent.atomic.AtomicLong
+
+import com.codahale.metrics._
/**
* Specialized "one-shot" Timer.
diff --git a/akka-testkit/src/test/scala/akka/testkit/metrics/MetricsKit.scala b/akka-testkit/src/test/scala/akka/testkit/metrics/MetricsKit.scala
index ebee113f9a..9049d97304 100644
--- a/akka-testkit/src/test/scala/akka/testkit/metrics/MetricsKit.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/metrics/MetricsKit.scala
@@ -4,17 +4,19 @@
package akka.testkit.metrics
-import com.codahale.metrics._
-
-import java.util.concurrent.TimeUnit
-import scala.concurrent.duration._
-import com.typesafe.config.Config
import java.util
-import scala.util.matching.Regex
+import java.util.concurrent.TimeUnit
+
import scala.collection.mutable
-import akka.testkit.metrics.reporter.AkkaConsoleReporter
-import org.scalatest.Notifying
+import scala.concurrent.duration._
import scala.reflect.ClassTag
+import scala.util.matching.Regex
+
+import com.codahale.metrics._
+import com.typesafe.config.Config
+import org.scalatest.Notifying
+
+import akka.testkit.metrics.reporter.AkkaConsoleReporter
/**
* Allows to easily measure performance / memory / file descriptor use in tests.
@@ -29,6 +31,7 @@ private[akka] trait MetricsKit extends MetricsKitOps {
this: Notifying =>
import MetricsKit._
+
import akka.util.ccompat.JavaConverters._
private var reporters: List[ScheduledReporter] = Nil
diff --git a/akka-testkit/src/test/scala/akka/testkit/metrics/MetricsKitOps.scala b/akka-testkit/src/test/scala/akka/testkit/metrics/MetricsKitOps.scala
index b7795f252f..f1a247e242 100644
--- a/akka-testkit/src/test/scala/akka/testkit/metrics/MetricsKitOps.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/metrics/MetricsKitOps.scala
@@ -4,8 +4,9 @@
package akka.testkit.metrics
-import com.codahale.metrics._
import java.util
+
+import com.codahale.metrics._
import com.codahale.metrics.jvm
import com.codahale.metrics.jvm.MemoryUsageGaugeSet
diff --git a/akka-testkit/src/test/scala/akka/testkit/metrics/MetricsKitSpec.scala b/akka-testkit/src/test/scala/akka/testkit/metrics/MetricsKitSpec.scala
index 9b11bff17e..20ae2b81e7 100644
--- a/akka-testkit/src/test/scala/akka/testkit/metrics/MetricsKitSpec.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/metrics/MetricsKitSpec.scala
@@ -4,8 +4,8 @@
package akka.testkit.metrics
-import org.scalatest._
import com.typesafe.config.ConfigFactory
+import org.scalatest._
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
diff --git a/akka-testkit/src/test/scala/akka/testkit/metrics/reporter/AkkaConsoleReporter.scala b/akka-testkit/src/test/scala/akka/testkit/metrics/reporter/AkkaConsoleReporter.scala
index b8b93a7066..72c7946ef5 100644
--- a/akka-testkit/src/test/scala/akka/testkit/metrics/reporter/AkkaConsoleReporter.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/metrics/reporter/AkkaConsoleReporter.scala
@@ -7,7 +7,9 @@ package akka.testkit.metrics.reporter
import java.io.PrintStream
import java.util
import java.util.concurrent.TimeUnit
+
import com.codahale.metrics._
+
import akka.testkit.metrics._
/**
@@ -115,8 +117,9 @@ class AkkaConsoleReporter(registry: AkkaMetricRegistry, verbose: Boolean, output
private def printKnownOpsInTimespanCounter(counter: KnownOpsInTimespanTimer): Unit = {
import concurrent.duration._
+
import akka.util.PrettyDuration._
- output.print(" ops = %d%n".format(counter.getCount))
+ output.print(" ops = %d%n".format(counter.getCount()))
output.print(" time = %s%n".format(counter.elapsedTime.nanos.pretty))
output.print(" ops/s = %2.2f%n".format(counter.opsPerSecond))
output.print(" avg = %s%n".format(counter.avgDuration.nanos.pretty))
@@ -140,7 +143,7 @@ class AkkaConsoleReporter(registry: AkkaMetricRegistry, verbose: Boolean, output
}
private def printAveragingGauge(gauge: AveragingGauge): Unit = {
- output.print(" avg = %2.2f%n".format(gauge.getValue))
+ output.print(" avg = %2.2f%n".format(gauge.getValue()))
}
private def printWithBanner(s: String, c: Char): Unit = {
diff --git a/build.sbt b/build.sbt
index 59d2f657ac..df345dbc1f 100644
--- a/build.sbt
+++ b/build.sbt
@@ -1,4 +1,4 @@
-import akka.{ AutomaticModuleName, CopyrightHeaderForBuild, Paradox, ParadoxSupport, ScalafixIgnoreFilePlugin }
+import akka.{ AutomaticModuleName, CopyrightHeaderForBuild, Paradox, ScalafixIgnoreFilePlugin }
enablePlugins(
UnidocRoot,
@@ -11,10 +11,15 @@ enablePlugins(
disablePlugins(MimaPlugin)
addCommandAlias(
name = "fixall",
- value = ";scalafixEnable;compile:scalafix;test:scalafix;multi-jvm:scalafix;test:compile;reload")
+ value =
+ ";scalafixEnable;compile:scalafix;test:scalafix;multi-jvm:scalafix;scalafmtAll;test:compile;multi-jvm:compile;reload")
+
+addCommandAlias(
+ name = "sortImports",
+ value = ";scalafixEnable;compile:scalafix SortImports;test:scalafix SortImports;scalafmtAll")
import akka.AkkaBuild._
-import akka.{ AkkaBuild, Dependencies, GitHub, OSGi, Protobuf, SigarLoader, VersionGenerator }
+import akka.{ AkkaBuild, Dependencies, OSGi, Protobuf, SigarLoader, VersionGenerator }
import com.typesafe.sbt.SbtMultiJvm.MultiJvmKeys.MultiJvm
import com.typesafe.tools.mima.plugin.MimaPlugin
import sbt.Keys.{ initialCommands, parallelExecution }
@@ -63,6 +68,7 @@ lazy val aggregatedProjects: Seq[ProjectReference] = List[ProjectReference](
persistenceTestkit,
protobuf,
protobufV3,
+ pki,
remote,
remoteTests,
slf4j,
@@ -75,6 +81,7 @@ lazy val aggregatedProjects: Seq[ProjectReference] = List[ProjectReference](
lazy val root = Project(id = "akka", base = file("."))
.aggregate(aggregatedProjects: _*)
+ .enablePlugins(PublishRsyncPlugin)
.settings(rootSettings: _*)
.settings(unidocRootIgnoreProjects := Seq(remoteTests, benchJmh, protobuf, protobufV3, akkaScalaNightly, docs))
.settings(unmanagedSources in (Compile, headerCreate) := (baseDirectory.value / "project").**("*.scala").get)
@@ -112,7 +119,12 @@ lazy val benchJmh = akkaModule("akka-bench-jmh")
.disablePlugins(MimaPlugin, WhiteSourcePlugin, ValidatePullRequest, CopyrightHeaderInPr)
lazy val cluster = akkaModule("akka-cluster")
- .dependsOn(remote, remoteTests % "test->test", testkit % "test->test", jackson % "test->test")
+ .dependsOn(
+ remote,
+ coordination % "compile->compile;test->test",
+ remoteTests % "test->test",
+ testkit % "test->test",
+ jackson % "test->test")
.settings(Dependencies.cluster)
.settings(AutomaticModuleName.settings("akka.cluster"))
.settings(OSGi.cluster)
@@ -154,7 +166,10 @@ lazy val clusterSharding = akkaModule("akka-cluster-sharding")
.enablePlugins(MultiNode, ScaladocNoVerificationOfDiagrams)
lazy val clusterTools = akkaModule("akka-cluster-tools")
- .dependsOn(cluster % "compile->compile;test->test;multi-jvm->multi-jvm", coordination, jackson % "test->test")
+ .dependsOn(
+ cluster % "compile->compile;test->test;multi-jvm->multi-jvm",
+ coordination % "compile->compile;test->test",
+ jackson % "test->test")
.settings(Dependencies.clusterTools)
.settings(AutomaticModuleName.settings("akka.cluster.tools"))
.settings(OSGi.clusterTools)
@@ -199,10 +214,10 @@ lazy val docs = akkaModule("akka-docs")
persistenceTestkit % "compile->compile;test->test")
.settings(Dependencies.docs)
.settings(Paradox.settings)
- .settings(ParadoxSupport.paradoxWithCustomDirectives)
+ .settings(javacOptions += "-parameters") // for Jackson
.enablePlugins(
AkkaParadoxPlugin,
- DeployRsync,
+ PublishRsyncPlugin,
NoPublish,
ParadoxBrowse,
ScaladocNoVerificationOfDiagrams,
@@ -274,6 +289,7 @@ lazy val persistenceTestkit = akkaModule("akka-persistence-testkit")
.dependsOn(
persistenceTyped % "compile->compile;provided->provided;test->test",
testkit % "compile->compile;test->test",
+ actorTestkitTyped,
persistenceTck % "test")
.settings(Dependencies.persistenceTestKit)
.settings(AutomaticModuleName.settings("akka.persistence.testkit"))
@@ -290,11 +306,11 @@ lazy val protobufV3 = akkaModule("akka-protobuf-v3")
.enablePlugins(ScaladocNoVerificationOfDiagrams)
.disablePlugins(MimaPlugin)
.settings(
- libraryDependencies += Dependencies.Compile.protobufRuntime,
+ libraryDependencies += Dependencies.Compile.Provided.protobufRuntime,
assemblyShadeRules in assembly := Seq(
ShadeRule
.rename("com.google.protobuf.**" -> "akka.protobufv3.internal.@1")
- .inLibrary(Dependencies.Compile.protobufRuntime)),
+ .inLibrary(Dependencies.Compile.Provided.protobufRuntime)),
assemblyOption in assembly := (assemblyOption in assembly).value.copy(includeScala = false, includeBin = false),
autoScalaLibrary := false, // do not include scala dependency in pom
exportJars := true, // in dependent projects, use assembled and shaded jar
@@ -307,6 +323,14 @@ lazy val protobufV3 = akkaModule("akka-protobuf-v3")
test in assembly := {}, // assembly runs tests for unknown reason which introduces another cyclic dependency to packageBin via exportedJars
description := "Akka Protobuf V3 is a shaded version of the protobuf runtime. Original POM: https://github.com/protocolbuffers/protobuf/blob/v3.9.0/java/pom.xml")
+lazy val pki =
+ akkaModule("akka-pki")
+ .dependsOn(actor) // this dependency only exists for "@ApiMayChange"
+ .settings(Dependencies.pki)
+ .settings(AutomaticModuleName.settings("akka.pki"))
+ // The akka-pki artifact was added in Akka 2.6.2, no MiMa checks yet.
+ .disablePlugins(MimaPlugin)
+
lazy val remote =
akkaModule("akka-remote")
.dependsOn(
@@ -443,8 +467,10 @@ lazy val clusterShardingTyped = akkaModule("akka-cluster-sharding-typed")
actorTestkitTyped % "test->test",
actorTypedTests % "test->test",
persistenceTyped % "test->test",
+ persistenceTestkit % "test->test",
remote % "compile->CompileJdk9;test->test",
remoteTests % "test->test",
+ remoteTests % "test->test;multi-jvm->multi-jvm",
jackson % "test->test")
.settings(javacOptions += "-parameters") // for Jackson
.settings(AutomaticModuleName.settings("akka.cluster.sharding.typed"))
@@ -482,7 +508,7 @@ lazy val discovery = akkaModule("akka-discovery")
.settings(OSGi.discovery)
lazy val coordination = akkaModule("akka-coordination")
- .dependsOn(actor, testkit % "test->test", actorTests % "test->test", cluster % "test->test")
+ .dependsOn(actor, testkit % "test->test", actorTests % "test->test")
.settings(Dependencies.coordination)
.settings(AutomaticModuleName.settings("akka.coordination"))
.settings(OSGi.coordination)
diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala
index a75474f272..9a6dd92100 100644
--- a/project/AkkaBuild.scala
+++ b/project/AkkaBuild.scala
@@ -30,7 +30,6 @@ object AkkaBuild {
Dependencies.Versions)
lazy val rootSettings = Def.settings(
- Release.settings,
UnidocRoot.akkaSettings,
Protobuf.settings,
parallelExecution in GlobalScope := System.getProperty("akka.parallelExecution", parallelExecutionByDefault.toString).toBoolean,
@@ -118,16 +117,6 @@ object AkkaBuild {
crossVersion := CrossVersion.binary,
- // Adds a `src/main/scala-2.13+` source directory for Scala 2.13 and newer
- // and a `src/main/scala-2.13-` source directory for Scala version older than 2.13
- unmanagedSourceDirectories in Compile += {
- val sourceDir = (sourceDirectory in Compile).value
- CrossVersion.partialVersion(scalaVersion.value) match {
- case Some((2, n)) if n >= 13 => sourceDir / "scala-2.13+"
- case _ => sourceDir / "scala-2.13-"
- }
- },
-
ivyLoggingLevel in ThisBuild := UpdateLogging.Quiet,
licenses := Seq(("Apache-2.0", url("https://www.apache.org/licenses/LICENSE-2.0.html"))),
@@ -153,7 +142,7 @@ object AkkaBuild {
|implicit def _system = system
|def startSystem(remoting: Boolean = false) { system = ActorSystem("repl", if(remoting) remoteConfig else config); println("don’t forget to system.terminate()!") }
|implicit def ec = system.dispatcher
- |implicit val timeout = Timeout(5 seconds)
+ |implicit val timeout: Timeout = Timeout(5 seconds)
|""".stripMargin,
/**
diff --git a/project/AkkaDisciplinePlugin.scala b/project/AkkaDisciplinePlugin.scala
index 1a15fbcace..0c42b3a8d7 100644
--- a/project/AkkaDisciplinePlugin.scala
+++ b/project/AkkaDisciplinePlugin.scala
@@ -8,12 +8,10 @@ import sbt._
import Keys.{ scalacOptions, _ }
import sbt.plugins.JvmPlugin
-object AkkaDisciplinePlugin extends AutoPlugin with ScalafixSupport {
-
- import scalafix.sbt.ScalafixPlugin
+object AkkaDisciplinePlugin extends AutoPlugin {
override def trigger: PluginTrigger = allRequirements
- override def requires: Plugins = JvmPlugin && ScalafixPlugin
+ override def requires: Plugins = JvmPlugin
override lazy val projectSettings = disciplineSettings
// allow toggling for pocs/exploration of ideas without discpline
diff --git a/project/Dependencies.scala b/project/Dependencies.scala
index f0e99d8740..6c7aa30173 100644
--- a/project/Dependencies.scala
+++ b/project/Dependencies.scala
@@ -6,6 +6,7 @@ package akka
import sbt._
import Keys._
+import scala.language.implicitConversions
object Dependencies {
import DependencyHelpers._
@@ -15,13 +16,13 @@ object Dependencies {
val junitVersion = "4.13"
val slf4jVersion = "1.7.30"
// check agrona version when updating this
- val aeronVersion = "1.26.0"
+ val aeronVersion = "1.27.0"
// needs to be inline with the aeron version, check
// https://github.com/real-logic/aeron/blob/1.x.y/build.gradle
- val agronaVersion = "1.4.0"
+ val agronaVersion = "1.4.1"
val nettyVersion = "3.10.6.Final"
- val jacksonVersion = "2.10.3"
- val protobufJavaVersion = "3.10.0"
+ val jacksonVersion = "2.10.4"
+ val protobufJavaVersion = "3.11.4"
val logbackVersion = "1.2.3"
val scala212Version = "2.12.11"
@@ -68,7 +69,7 @@ object Dependencies {
val reactiveStreams = "org.reactivestreams" % "reactive-streams" % reactiveStreamsVersion // CC0
// ssl-config
- val sslConfigCore = Def.setting { "com.typesafe" %% "ssl-config-core" % sslConfigVersion } // ApacheV2
+ val sslConfigCore = "com.typesafe" %% "ssl-config-core" % sslConfigVersion // ApacheV2
val lmdb = "org.lmdbjava" % "lmdbjava" % "0.7.0" // ApacheV2, OpenLDAP Public License
@@ -82,6 +83,8 @@ object Dependencies {
// Added explicitly for when artery tcp is used
val agrona = "org.agrona" % "agrona" % agronaVersion // ApacheV2
+ val asnOne = "com.hierynomus" % "asn-one" % "0.4.0" // ApacheV2
+
val jacksonCore = "com.fasterxml.jackson.core" % "jackson-core" % jacksonVersion // ApacheV2
val jacksonAnnotations = "com.fasterxml.jackson.core" % "jackson-annotations" % jacksonVersion // ApacheV2
val jacksonDatabind = "com.fasterxml.jackson.core" % "jackson-databind" % jacksonVersion // ApacheV2
@@ -90,8 +93,7 @@ object Dependencies {
val jacksonScala = "com.fasterxml.jackson.module" %% "jackson-module-scala" % jacksonVersion // ApacheV2
val jacksonParameterNames = "com.fasterxml.jackson.module" % "jackson-module-parameter-names" % jacksonVersion // ApacheV2
val jacksonCbor = "com.fasterxml.jackson.dataformat" % "jackson-dataformat-cbor" % jacksonVersion // ApacheV2
-
- val protobufRuntime = "com.google.protobuf" % "protobuf-java" % protobufJavaVersion
+ val lz4Java = "org.lz4" % "lz4-java" % "1.7.1" // ApacheV2
val logback = "ch.qos.logback" % "logback-classic" % logbackVersion // EPL 1.0
@@ -129,8 +131,8 @@ object Dependencies {
val dockerClient = "com.spotify" % "docker-client" % "8.16.0" % "test" // ApacheV2
// metrics, measurements, perf testing
- val metrics = "io.dropwizard.metrics" % "metrics-core" % "4.1.5" % "test" // ApacheV2
- val metricsJvm = "io.dropwizard.metrics" % "metrics-jvm" % "4.1.5" % "test" // ApacheV2
+ val metrics = "io.dropwizard.metrics" % "metrics-core" % "4.1.9" % "test" // ApacheV2
+ val metricsJvm = "io.dropwizard.metrics" % "metrics-jvm" % "4.1.9" % "test" // ApacheV2
val latencyUtils = "org.latencyutils" % "LatencyUtils" % "2.0.3" % "test" // Free BSD
val hdrHistogram = "org.hdrhistogram" % "HdrHistogram" % "2.1.12" % "test" // CC0
val metricsAll = Seq(metrics, metricsJvm, latencyUtils, hdrHistogram)
@@ -161,6 +163,8 @@ object Dependencies {
val logback = Compile.logback % "optional;provided;test" // EPL 1.0
+ val protobufRuntime = "com.google.protobuf" % "protobuf-java" % protobufJavaVersion % "optional;provided"
+
}
}
@@ -194,6 +198,13 @@ object Dependencies {
val actorTestkitTyped = l ++= Seq(Provided.logback, Provided.junit, Provided.scalatest, Test.scalatestJUnit)
+ val pki = l ++=
+ Seq(
+ asnOne,
+ // pull up slf4j version from the one provided transitively in asnOne to fix unidoc
+ Compile.slf4jApi % "provided",
+ Test.scalatest)
+
val remoteDependencies = Seq(netty, aeronDriver, aeronClient)
val remoteOptionalDependencies = remoteDependencies.map(_ % "optional")
@@ -242,7 +253,7 @@ object Dependencies {
Provided.levelDB,
Provided.levelDBNative)
- val persistenceTestKit = l ++= Seq(Test.scalatest)
+ val persistenceTestKit = l ++= Seq(Test.scalatest, Test.logback)
val persistenceShared = l ++= Seq(Provided.levelDB, Provided.levelDBNative)
@@ -255,6 +266,7 @@ object Dependencies {
jacksonJsr310,
jacksonParameterNames,
jacksonCbor,
+ lz4Java,
Test.junit,
Test.scalatest)
@@ -274,7 +286,7 @@ object Dependencies {
// akka stream
- lazy val stream = l ++= Seq[sbt.ModuleID](reactiveStreams, sslConfigCore.value, Test.scalatest)
+ lazy val stream = l ++= Seq[sbt.ModuleID](reactiveStreams, sslConfigCore, Test.scalatest)
lazy val streamTestkit = l ++= Seq(Test.scalatest, Test.scalacheck, Test.junit)
diff --git a/project/Doc.scala b/project/Doc.scala
index 1880fa6b1f..004b1f4fe4 100644
--- a/project/Doc.scala
+++ b/project/Doc.scala
@@ -12,6 +12,7 @@ import sbtunidoc.GenJavadocPlugin.autoImport._
import sbt.Keys._
import sbt.File
import scala.annotation.tailrec
+import com.lightbend.sbt.publishrsync.PublishRsyncPlugin.autoImport.publishRsyncArtifacts
import sbt.ScopeFilter.ProjectFilter
@@ -132,10 +133,18 @@ object UnidocRoot extends AutoPlugin {
.getOrElse(sbtunidoc.ScalaUnidocPlugin)
val akkaSettings = UnidocRoot.CliOptions.genjavadocEnabled
- .ifTrue(Seq(javacOptions in (JavaUnidoc, unidoc) := {
- if (JdkOptions.isJdk8) Seq("-Xdoclint:none")
- else Seq("-Xdoclint:none", "--ignore-source-errors", "--no-module-directories")
- }))
+ .ifTrue(Seq(
+ javacOptions in (JavaUnidoc, unidoc) := {
+ if (JdkOptions.isJdk8) Seq("-Xdoclint:none")
+ else Seq("-Xdoclint:none", "--ignore-source-errors", "--no-module-directories")
+ },
+ publishRsyncArtifacts ++= {
+ val releaseVersion = if (isSnapshot.value) "snapshot" else version.value
+ (Compile / unidoc).value match {
+ case Seq(japi, api) =>
+ Seq((japi -> s"www/japi/akka/$releaseVersion"), (api -> s"www/api/akka/$releaseVersion"))
+ }
+ }))
.getOrElse(Nil)
override lazy val projectSettings = {
@@ -177,6 +186,6 @@ object BootstrapGenjavadoc extends AutoPlugin {
override lazy val projectSettings = UnidocRoot.CliOptions.genjavadocEnabled
.ifTrue(Seq(
unidocGenjavadocVersion := "0.16",
- scalacOptions in Compile ++= Seq("-P:genjavadoc:fabricateParams=true", "-P:genjavadoc:suppressSynthetic=false")))
+ scalacOptions in Compile ++= Seq("-P:genjavadoc:fabricateParams=false", "-P:genjavadoc:suppressSynthetic=false")))
.getOrElse(Nil)
}
diff --git a/project/JavaFormatter.scala b/project/JavaFormatter.scala
index 55ea7dedbd..bc7825ae2d 100644
--- a/project/JavaFormatter.scala
+++ b/project/JavaFormatter.scala
@@ -4,7 +4,7 @@
import akka.ProjectFileIgnoreSupport
import com.lightbend.sbt.JavaFormatterPlugin
-import sbt.{AutoPlugin, PluginTrigger, Plugins}
+import sbt.{ AutoPlugin, PluginTrigger, Plugins }
object JavaFormatter extends AutoPlugin {
@@ -15,6 +15,8 @@ object JavaFormatter extends AutoPlugin {
private val ignoreConfigFileName: String = ".sbt-java-formatter.conf"
private val descriptor: String = "sbt-java-formatter"
+ private val formatOnCompile = !sys.props.contains("akka.no.discipline")
+
import JavaFormatterPlugin.autoImport._
import sbt.Keys._
import sbt._
@@ -23,9 +25,10 @@ object JavaFormatter extends AutoPlugin {
override def projectSettings: Seq[Def.Setting[_]] = Seq(
//below is for sbt java formatter
(excludeFilter in javafmt) := {
- val ignoreSupport = new ProjectFileIgnoreSupport((baseDirectory in ThisBuild).value / ignoreConfigFileName, descriptor)
+ val ignoreSupport =
+ new ProjectFileIgnoreSupport((baseDirectory in ThisBuild).value / ignoreConfigFileName, descriptor)
val simpleFileFilter = new SimpleFileFilter(file => ignoreSupport.isIgnoredByFileOrPackages(file))
simpleFileFilter || (excludeFilter in javafmt).value
- }
- )
+ },
+ javafmtOnCompile := formatOnCompile)
}
diff --git a/project/MiMa.scala b/project/MiMa.scala
index fb19c4d2b9..3035b8c816 100644
--- a/project/MiMa.scala
+++ b/project/MiMa.scala
@@ -12,8 +12,8 @@ import com.typesafe.tools.mima.plugin.MimaPlugin.autoImport._
object MiMa extends AutoPlugin {
- private val latestPatchOf25 = 29
- private val latestPatchOf26 = 4
+ private val latestPatchOf25 = 31
+ private val latestPatchOf26 = 5
override def requires = MimaPlugin
override def trigger = allRequirements
@@ -40,7 +40,9 @@ object MiMa extends AutoPlugin {
// https://github.com/scala/bug/issues/11207 produced many more
// static methods than expected. These are hard to filter out, so
// we exclude it here and rely on the checks for 2.5.17 and 2.5.19.
- expandVersions(2, 5, ((firstPatchOf25 to latestPatchOf25).toSet - 18).toList)
+ // Additionally, 2.5.30 had some problems related to
+ // https://github.com/akka/akka/issues/28807
+ expandVersions(2, 5, ((firstPatchOf25 to latestPatchOf25).toSet - 18 - 30).toList)
} else {
Nil
}
diff --git a/project/Paradox.scala b/project/Paradox.scala
index 790ed3e466..2f6b2b4690 100644
--- a/project/Paradox.scala
+++ b/project/Paradox.scala
@@ -7,6 +7,7 @@ package akka
import com.lightbend.paradox.sbt.ParadoxPlugin
import com.lightbend.paradox.sbt.ParadoxPlugin.autoImport._
import com.lightbend.paradox.apidoc.ApidocPlugin
+import com.lightbend.sbt.publishrsync.PublishRsyncPlugin.autoImport._
import sbt.Keys._
import sbt._
@@ -32,7 +33,7 @@ object Paradox {
"javadoc.akka.http.base_url" -> "https://doc.akka.io/japi/akka-http/current",
"javadoc.akka.http.link_style" -> "frames",
"scala.version" -> scalaVersion.value,
- "scala.binary_version" -> scalaBinaryVersion.value,
+ "scala.binary.version" -> scalaBinaryVersion.value,
"akka.version" -> version.value,
"scalatest.version" -> Dependencies.scalaTestVersion,
"sigar_loader.version" -> "1.6.6-rev002",
@@ -76,6 +77,8 @@ object Paradox {
name in (Compile, paradox) := "Akka",
resolvers += Resolver.jcenterRepo,
ApidocPlugin.autoImport.apidocRootPackage := "akka",
- DeployRsync.autoImport.deployRsyncArtifact := List(
- (Compile / paradox).value -> s"www/docs/akka/${version.value}"))
+ publishRsyncArtifacts += {
+ val releaseVersion = if (isSnapshot.value) "snapshot" else version.value
+ ((Compile / paradox).value -> s"www/docs/akka/$releaseVersion")
+ })
}
diff --git a/project/ParadoxSupport.scala b/project/ParadoxSupport.scala
deleted file mode 100644
index 4d8289534c..0000000000
--- a/project/ParadoxSupport.scala
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Copyright (C) 2016-2020 Lightbend Inc.
- */
-
-package akka
-
-import java.io.{File, FileNotFoundException}
-
-import com.lightbend.paradox.markdown._
-import com.lightbend.paradox.sbt.ParadoxPlugin.autoImport._
-import org.pegdown.Printer
-import org.pegdown.ast._
-import sbt.Keys._
-import sbt._
-
-import scala.annotation.tailrec
-import scala.io.{Codec, Source}
-import scala.collection.JavaConverters._
-
-object ParadoxSupport {
- val paradoxWithCustomDirectives = Seq(
- paradoxDirectives ++= Def.taskDyn {
- val log = streams.value.log
- val classpath = (fullClasspath in Compile).value.files.map(_.toURI.toURL).toArray
- val classloader = new java.net.URLClassLoader(classpath, this.getClass().getClassLoader())
- import _root_.io.github.classgraph.ClassGraph
- lazy val scanner = new ClassGraph()
- .whitelistPackages("akka")
- .addClassLoader(classloader)
- .scan()
- val allClasses = scanner.getAllClasses.getNames.asScala.toVector
- Def.task { Seq(
- { context: Writer.Context =>
- new SignatureDirective(context.location.tree.label, context.properties, context)
- },
- )}
- }.value
- )
-
- class SignatureDirective(page: Page, variables: Map[String, String], ctx: Writer.Context) extends LeafBlockDirective("signature") {
- def render(node: DirectiveNode, visitor: Visitor, printer: Printer): Unit =
- try {
- val labels = node.attributes.values("identifier").asScala.map(_.toLowerCase())
- val source = node.source match {
- case direct: DirectiveNode.Source.Direct => direct.value
- case _ => sys.error("Source references are not supported")
- }
- val file =
- if (source startsWith "/") {
- // snip.build.base_dir defined by Paradox
- val base = new File(PropertyUrl("snip.build.base_dir", variables.get).base.trim)
- new File(base, source)
- } else new File(page.file.getParentFile, source)
-
- val Signature = """\s*((def|val|type) (\w+)(?=[:(\[]).*)(\s+\=.*)""".r // stupid approximation to match a signature
-
- val text =
- getDefs(file).collect {
- case line@Signature(signature, kind, l, definition) if labels contains l.replaceAll("Mat$", "").toLowerCase() =>
- //println(s"Found label '$l' with sig '$full' in line $line")
- if (kind == "type") signature + definition
- else signature
- }.mkString("\n")
-
- if (text.trim.isEmpty) {
- throw new IllegalArgumentException(
- s"Did not find any signatures with one of those names [${labels.mkString(", ")}] in $source " +
- s"(was referenced from [${page.path}])")
- } else {
- val lang = Option(node.attributes.value("type")).getOrElse(Snippet.language(file))
- new VerbatimNode(text, lang).accept(visitor)
- }
- } catch {
- case e: FileNotFoundException =>
- ctx.error(s"Unknown snippet [${e.getMessage}]", node)
- }
- }
-
- def getDefs(file: File): Seq[String] = {
- val Indented = "(\\s*)(.*)".r
-
- @tailrec
- def rec(lines: Iterator[String], currentDef: Option[String], defIndent: Integer, soFar: Seq[String]): Seq[String] = {
- if (!lines.hasNext) soFar ++ currentDef
- else lines.next() match {
- case Indented(indent, line) =>
- if (line.startsWith("def")) rec(lines, Some(line), indent.length, soFar ++ currentDef)
- else if (indent.length == defIndent + 4) rec(lines, currentDef.map(_ ++ line), defIndent, soFar)
- else rec(lines, None, 0, soFar ++ currentDef)
- }
- }
- rec(Source.fromFile(file)(Codec.UTF8).getLines, None, 0, Nil)
- }
-}
diff --git a/project/ProjectFileIgnoreSupport.scala b/project/ProjectFileIgnoreSupport.scala
index 5be747f960..f36f9dfa83 100644
--- a/project/ProjectFileIgnoreSupport.scala
+++ b/project/ProjectFileIgnoreSupport.scala
@@ -7,14 +7,23 @@ package akka
import java.io.File
import com.typesafe.config.ConfigFactory
-import sbt.AutoPlugin
-import sbt.Def
-import sbt.file
import sbt.internal.sbtscalafix.Compat
class ProjectFileIgnoreSupport(ignoreConfigFile: File, descriptor: String) {
private val stdoutLogger = Compat.ConsoleLogger(System.out)
+ private val javaSourceDirectories = Set(
+ "java",
+ Jdk9.JAVA_SOURCE_DIRECTORY,
+ Jdk9.JAVA_TEST_SOURCE_DIRECTORY
+ )
+
+ private val scalaSourceDirectories = Set(
+ "scala",
+ Jdk9.SCALA_SOURCE_DIRECTORY,
+ Jdk9.SCALA_TEST_SOURCE_DIRECTORY
+ )
+
private lazy val ignoreConfig = {
require(ignoreConfigFile.exists(), s"Expected ignore configuration for $descriptor at ${ignoreConfigFile.getAbsolutePath} but was missing")
ConfigFactory.parseFile(ignoreConfigFile)
@@ -55,7 +64,7 @@ class ProjectFileIgnoreSupport(ignoreConfigFile: File, descriptor: String) {
case Some(packageName) =>
val ignored = packageName.startsWith(pkg)
if (ignored) {
- stdoutLogger.debug(s"$descriptor ignored file with pkg:$pkg file:[${file.toPath}] ")
+ stdoutLogger.debug(s"$descriptor ignored file with pkg:$pkg for package:$packageName file:[${file.toPath}] ")
}
ignored
case None => false
@@ -65,22 +74,23 @@ class ProjectFileIgnoreSupport(ignoreConfigFile: File, descriptor: String) {
}
private def getPackageName(fileName: String): Option[String] = {
- def getPackageName0(fileType: String): String = {
+ def getPackageName0(sourceDirectories:Set[String]): String = {
import java.io.{File => JFile}
- fileName.split(JFile.separatorChar)
- .dropWhile(part => part != fileType)
+ val packageName = fileName.split(JFile.separatorChar)
+ .dropWhile(part => !sourceDirectories(part))
.drop(1)
.dropRight(1)
.mkString(".")
+ packageName
}
fileName.split('.').lastOption match {
case Some(fileType) =>
fileType match {
case "java" =>
- Option(getPackageName0("java"))
+ Option(getPackageName0(javaSourceDirectories))
case "scala" =>
- Option(getPackageName0("scala"))
+ Option(getPackageName0(scalaSourceDirectories))
case _ => None
}
case None => None
diff --git a/project/Protobuf.scala b/project/Protobuf.scala
index 77cde53ac3..af75c3af1d 100644
--- a/project/Protobuf.scala
+++ b/project/Protobuf.scala
@@ -32,7 +32,7 @@ object Protobuf {
Compile / unmanagedJars ++= Seq(
baseDirectory.value / ".." / "akka-protobuf-v3" / "target" / s"scala-${scalaBinaryVersion.value}" / s"akka-protobuf-v3-assembly-${version.value}.jar"),
protoc := "protoc",
- protocVersion := "3.10.0",
+ protocVersion := "3.11.4",
generate := {
val sourceDirs = paths.value
val targetDirs = outputPaths.value
diff --git a/project/Publish.scala b/project/Publish.scala
index 7e4b8c93bf..ab9037ca65 100644
--- a/project/Publish.scala
+++ b/project/Publish.scala
@@ -8,6 +8,7 @@ import sbt._
import sbt.Keys._
import java.io.File
import sbtwhitesource.WhiteSourcePlugin.autoImport.whitesourceIgnore
+import com.lightbend.sbt.publishrsync.PublishRsyncPlugin.autoImport.publishRsyncHost
object Publish extends AutoPlugin {
@@ -18,6 +19,7 @@ object Publish extends AutoPlugin {
override lazy val projectSettings = Seq(
pomExtra := akkaPomExtra,
publishTo := Some(akkaPublishTo.value),
+ publishRsyncHost := "akkarepo@gustav.akka.io",
credentials ++= akkaCredentials,
organizationName := "Lightbend Inc.",
organizationHomepage := Some(url("https://www.lightbend.com")),
@@ -33,7 +35,7 @@ object Publish extends AutoPlugin {
akka-contributors
Akka Contributors
- akka-dev@googlegroups.com
+ akka.official@gmail.com
https://github.com/akka/akka/graphs/contributors
@@ -41,7 +43,8 @@ object Publish extends AutoPlugin {
private def akkaPublishTo = Def.setting {
val key = new java.io.File(
- Option(System.getProperty("akka.gustav.key")).getOrElse(System.getProperty("user.home") + "/.ssh/id_rsa_gustav.pem"))
+ Option(System.getProperty("akka.gustav.key"))
+ .getOrElse(System.getProperty("user.home") + "/.ssh/id_rsa_gustav.pem"))
if (isSnapshot.value)
Resolver.sftp("Akka snapshots", "gustav.akka.io", "/home/akkarepo/www/snapshots").as("akkarepo", key)
else
@@ -61,26 +64,3 @@ object NoPublish extends AutoPlugin {
override def projectSettings =
Seq(skip in publish := true, sources in (Compile, doc) := Seq.empty, whitesourceIgnore := true)
}
-
-object DeployRsync extends AutoPlugin {
- import scala.sys.process._
- import sbt.complete.DefaultParsers._
-
- override def requires = plugins.JvmPlugin
-
- trait Keys {
- val deployRsyncArtifact = taskKey[Seq[(File, String)]]("File or directory and a path to deploy to")
- val deployRsync = inputKey[Unit]("Deploy using SCP")
- }
-
- object autoImport extends Keys
- import autoImport._
-
- override def projectSettings =
- Seq(deployRsync := {
- val (_, host) = (Space ~ StringBasic).parsed
- deployRsyncArtifact.value.foreach {
- case (from, to) => s"rsync -rvz $from/ $host:$to" !
- }
- })
-}
diff --git a/project/Release.scala b/project/Release.scala
deleted file mode 100644
index 2facd041f0..0000000000
--- a/project/Release.scala
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Copyright (C) 2016-2020 Lightbend Inc.
- */
-
-package akka
-
-import sbt._
-import sbt.Keys._
-import java.io.File
-import com.jsuereth.sbtpgp.PgpKeys.publishSigned
-import sbtunidoc.BaseUnidocPlugin.autoImport.unidoc
-import com.lightbend.paradox.sbt.ParadoxKeys
-
-object Release extends ParadoxKeys {
- val releaseDirectory = SettingKey[File]("release-directory")
-
- lazy val settings: Seq[Setting[_]] = commandSettings ++ Seq(releaseDirectory := target.value / "release")
-
- lazy val commandSettings = Seq(commands ++= Seq(buildReleaseCommand, buildDocsCommand))
-
- def buildReleaseCommand = Command.command("buildRelease") { state =>
- val extracted = Project.extract(state)
- val release = extracted.get(releaseDirectory)
- val releaseVersion = extracted.get(version)
- val projectRef = extracted.get(thisProjectRef)
- val repo = extracted.get(Publish.defaultPublishTo)
- val state1 = extracted.runAggregated(publishSigned in projectRef, state)
-
- IO.delete(release)
- IO.createDirectory(release)
- IO.copyDirectory(repo, release / "releases")
- state1
- }
-
- def buildDocsCommand = Command.command("buildDocs") { state =>
- if (!sys.props.contains("akka.genjavadoc.enabled"))
- throw new RuntimeException(
- "Make sure you start sbt with \"-Dakka.genjavadoc.enabled=true\" otherwise no japi will be generated")
- val extracted = Project.extract(state)
- // we want to build the api-docs and docs with the current "default" version of scala
- val scalaV = extracted.get(scalaVersion)
- val expectedScalaV = extracted.get(crossScalaVersions).head
- if (scalaV != expectedScalaV)
- throw new RuntimeException(s"The docs should be built with Scala $expectedScalaV (was $scalaV)")
- val release = extracted.get(releaseDirectory)
- val releaseVersion = extracted.get(version)
- val projectRef = extracted.get(thisProjectRef)
-
- val (state2, Seq(japi, api)) = extracted.runTask(unidoc in Compile, state)
- val (state3, docs) = extracted.runTask(paradox in ProjectRef(projectRef.build, "akka-docs") in Compile, state2)
-
- IO.delete(release / "api")
- IO.delete(release / "japi")
- IO.delete(release / "docsapi")
- IO.createDirectory(release)
- IO.copyDirectory(api, release / "api" / "akka" / releaseVersion)
- IO.copyDirectory(japi, release / "japi" / "akka" / releaseVersion)
- IO.copyDirectory(docs, release / "docs" / "akka" / releaseVersion)
-
- state3
- }
-
-}
diff --git a/project/ScalaFixExtraRulesPlugin.scala b/project/ScalaFixExtraRulesPlugin.scala
new file mode 100644
index 0000000000..601071f6dd
--- /dev/null
+++ b/project/ScalaFixExtraRulesPlugin.scala
@@ -0,0 +1,20 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka
+
+import sbt.{AutoPlugin, PluginTrigger, Plugins, ScalafixSupport}
+import scalafix.sbt.ScalafixPlugin
+
+object ScalaFixExtraRulesPlugin extends AutoPlugin with ScalafixSupport{
+ override def trigger: PluginTrigger = allRequirements
+
+ override def requires: Plugins = ScalafixPlugin
+
+ import sbt._
+ import scalafix.sbt.ScalafixPlugin.autoImport.scalafixDependencies
+ override def projectSettings: Seq[Def.Setting[_]] = super.projectSettings ++ {
+ scalafixDependencies in ThisBuild += "com.nequissimus" %% "sort-imports" % "0.5.0"
+ }
+}
diff --git a/project/ScalaFixForJdk9Plugin.scala b/project/ScalaFixForJdk9Plugin.scala
new file mode 100644
index 0000000000..06f1fb5386
--- /dev/null
+++ b/project/ScalaFixForJdk9Plugin.scala
@@ -0,0 +1,31 @@
+/*
+ * Copyright (C) 2020 Lightbend Inc.
+ */
+
+package akka
+
+import sbt.{AutoPlugin, PluginTrigger, Plugins, ScalafixSupport}
+import scalafix.sbt.ScalafixPlugin
+object ScalaFixForJdk9Plugin extends AutoPlugin with ScalafixSupport {
+ override def trigger: PluginTrigger = allRequirements
+ import Jdk9._
+ override def requires: Plugins = Jdk9
+
+ import ScalafixPlugin.autoImport.scalafixConfigSettings
+ import sbt._
+
+ lazy val scalafixIgnoredSetting: Seq[Setting[_]] = Seq(
+ ignore(TestJdk9)
+ )
+
+ override def projectSettings: Seq[Def.Setting[_]] =
+ Seq(CompileJdk9, TestJdk9).flatMap(c => inConfig(c)(scalafixConfigSettings(c))) ++
+ scalafixIgnoredSetting ++ Seq(
+ updateProjectCommands(
+ alias = "fixall",
+ value = ";scalafixEnable;compile:scalafix;test:scalafix;multi-jvm:scalafix;scalafmtAll;test:compile;multi-jvm:compile;reload"),
+ updateProjectCommands(
+ alias = "sortImports",
+ value = ";scalafixEnable;compile:scalafix SortImports;test:scalafix SortImports;CompileJdk9:scalafix SortImports;TestJdk9:scalafix SortImports;scalafmtAll")
+ )
+}
diff --git a/project/ScalafixForMultiNodePlugin.scala b/project/ScalafixForMultiNodePlugin.scala
index 2f590ef992..1fc02619e2 100644
--- a/project/ScalafixForMultiNodePlugin.scala
+++ b/project/ScalafixForMultiNodePlugin.scala
@@ -23,6 +23,10 @@ object ScalafixForMultiNodePlugin extends AutoPlugin with ScalafixSupport {
Seq(MultiJvm).flatMap(c => inConfig(c)(scalafixConfigSettings(c))) ++
scalafixIgnoredSetting ++ Seq(
updateProjectCommands(
- alias = "fix",
- value = ";scalafixEnable;compile:scalafix;test:scalafix;multi-jvm:scalafix;test:compile;reload"))
+ alias = "fixall",
+ value = ";scalafixEnable;compile:scalafix;test:scalafix;multi-jvm:scalafix;scalafmtAll"),
+ updateProjectCommands(
+ alias = "sortImports",
+ value = ";scalafixEnable;compile:scalafix SortImports;test:scalafix SortImports;multi-jvm:scalafix SortImports;scalafmtAll")
+ )
}
diff --git a/project/StreamOperatorsIndexGenerator.scala b/project/StreamOperatorsIndexGenerator.scala
index 760fc4ac6e..380d20c525 100644
--- a/project/StreamOperatorsIndexGenerator.scala
+++ b/project/StreamOperatorsIndexGenerator.scala
@@ -70,8 +70,7 @@ object StreamOperatorsIndexGenerator extends AutoPlugin {
"alsoToGraph",
"orElseGraph",
"divertToGraph",
- "zipWithGraph",
- )
+ "zipWithGraph")
// FIXME document these methods as well
val pendingTestCases = Map(
@@ -98,14 +97,8 @@ object StreamOperatorsIndexGenerator extends AutoPlugin {
"fromGraph",
"actorSubscriber",
"foldAsync",
- "newOnCompleteStage",
- ),
- "Compression" -> Seq(
- "inflate",
- "gunzip",
- )
-
- )
+ "newOnCompleteStage"),
+ "Compression" -> Seq("inflate", "gunzip"))
val ignore =
Set("equals", "hashCode", "notify", "notifyAll", "wait", "toString", "getClass") ++
@@ -230,7 +223,10 @@ object StreamOperatorsIndexGenerator extends AutoPlugin {
"# Operators\n\n" +
tables +
"\n\n@@@ index\n\n" +
- groupedDefs.map { case (_, method, md) => s"* [${methodToShow(method)}]($md)" }.mkString("\n") + "\n\n@@@\n"
+ groupedDefs
+ .sortBy { case (_, method, _) => method.toLowerCase }
+ .map { case (_, method, md) => s"* [$method]($md)" }
+ .mkString("\n") + "\n\n@@@\n"
if (!file.exists || IO.read(file) != content) IO.write(file, content)
Seq(file)
diff --git a/project/build.properties b/project/build.properties
index a919a9b5f4..797e7ccfdb 100644
--- a/project/build.properties
+++ b/project/build.properties
@@ -1 +1 @@
-sbt.version=1.3.8
+sbt.version=1.3.10
diff --git a/project/plugins.sbt b/project/plugins.sbt
index 02f0caa6c4..727bf1272c 100644
--- a/project/plugins.sbt
+++ b/project/plugins.sbt
@@ -8,9 +8,9 @@ libraryDependencies += Defaults.sbtPluginExtra(
addSbtPlugin("com.typesafe.sbt" % "sbt-multi-jvm" % "0.4.0")
//#sbt-multi-jvm
-addSbtPlugin("com.lightbend.sbt" % "sbt-java-formatter" % "0.5.0")
-addSbtPlugin("org.scalameta" % "sbt-scalafmt" % "2.0.6")
-addSbtPlugin("ch.epfl.scala" % "sbt-scalafix" % "0.9.11")
+addSbtPlugin("com.lightbend.sbt" % "sbt-java-formatter" % "0.5.1")
+addSbtPlugin("org.scalameta" % "sbt-scalafmt" % "2.3.4")
+addSbtPlugin("ch.epfl.scala" % "sbt-scalafix" % "0.9.15")
// sbt-osgi 0.9.5 is available but breaks including jdk9-only classes
addSbtPlugin("com.typesafe.sbt" % "sbt-osgi" % "0.9.4")
addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.7.0")
@@ -19,9 +19,10 @@ addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.4.3")
addSbtPlugin("com.thoughtworks.sbt-api-mappings" % "sbt-api-mappings" % "3.0.0")
addSbtPlugin("pl.project13.scala" % "sbt-jmh" % "0.3.7")
addSbtPlugin("io.spray" % "sbt-boilerplate" % "0.6.1")
-addSbtPlugin("com.lightbend.akka" % "sbt-paradox-akka" % "0.31")
+addSbtPlugin("com.lightbend.akka" % "sbt-paradox-akka" % "0.33")
addSbtPlugin("com.lightbend" % "sbt-whitesource" % "0.1.18")
-addSbtPlugin("de.heikoseeberger" % "sbt-header" % "5.4.0") // for maintenance of copyright file header
+addSbtPlugin("de.heikoseeberger" % "sbt-header" % "5.6.0")
addSbtPlugin("com.hpe.sbt" % "sbt-pull-request-validator" % "1.0.0")
-addSbtPlugin("net.bzzt" % "sbt-reproducible-builds" % "0.23")
+addSbtPlugin("net.bzzt" % "sbt-reproducible-builds" % "0.24")
addSbtPlugin("com.dwijnand" % "sbt-dynver" % "4.0.0")
+addSbtPlugin("com.lightbend.sbt" % "sbt-publish-rsync" % "0.2")
diff --git a/project/project-info.conf b/project/project-info.conf
index ef6cf88c1e..a6d8a82332 100644
--- a/project/project-info.conf
+++ b/project/project-info.conf
@@ -1,7 +1,7 @@
project-info {
version: "current"
scaladoc: "https://doc.akka.io/api/akka/"${project-info.version}"/akka/"
- javadoc: "https://doc.akka.io/japi/akka/"${project-info.version}"/index.html"
+ javadoc: "https://doc.akka.io/japi/akka/"${project-info.version}"/akka/"
shared-info {
jdk-versions: ["Adopt OpenJDK 8", "Adopt OpenJDK 11"]
// snapshots: { }
@@ -41,7 +41,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/actor/package-summary.html"
+ url: ${project-info.javadoc}"actor/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -67,7 +67,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/actor/testkit/typed/package-summary.html"
+ url: ${project-info.javadoc}"actor/testkit/typed/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -93,7 +93,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/actor/typed/package-summary.html"
+ url: ${project-info.javadoc}"actor/typed/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -114,7 +114,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/cluster/package-summary.html"
+ url: ${project-info.javadoc}"cluster/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -135,7 +135,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/cluster/metrics/package-summary.html"
+ url: ${project-info.javadoc}"cluster/metrics/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -156,7 +156,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/cluster/sharding/package-summary.html"
+ url: ${project-info.javadoc}"cluster/sharding/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -182,7 +182,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/cluster/sharding/typed/package-summary.html"
+ url: ${project-info.javadoc}"cluster/sharding/typed/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -203,7 +203,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/cluster/tools/package-summary.html"
+ url: ${project-info.javadoc}"cluster/tools/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -229,7 +229,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/cluster/typed/package-summary.html"
+ url: ${project-info.javadoc}"cluster/typed/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -250,7 +250,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/coordination/package-summary.html"
+ url: ${project-info.javadoc}"coordination/lease/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -271,7 +271,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/discovery/package-summary.html"
+ url: ${project-info.javadoc}"discovery/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -292,7 +292,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/discovery/package-summary.html"
+ url: ${project-info.javadoc}"discovery/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -313,7 +313,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/remote/testkit/package-summary.html"
+ url: ${project-info.javadoc}"remote/testkit/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -334,7 +334,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/osgi/package-summary.html"
+ url: ${project-info.javadoc}"osgi/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -355,7 +355,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/persistence/package-summary.html"
+ url: ${project-info.javadoc}"persistence/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -376,7 +376,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/persistence/query/package-summary.html"
+ url: ${project-info.javadoc}"persistence/query/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -402,7 +402,28 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/persistence/typed/package-summary.html"
+ url: ${project-info.javadoc}"persistence/typed/package-summary.html"
+ text: "API (Javadoc)"
+ }
+ ]
+ }
+ akka-persistence-testkit: ${project-info.shared-info} {
+ title: "Akka Persistence Testkit"
+ jpms-name: "akka.persistence.testkit"
+ levels: [
+ {
+ readiness: Incubating
+ since: "2020-04-30"
+ since-version: "2.6.5"
+ }
+ ]
+ api-docs: [
+ {
+ url: ${project-info.scaladoc}"persistence/testkit/scaladsl/index.html"
+ text: "API (Scaladoc)"
+ }
+ {
+ url: ${project-info.javadoc}"persistence/testkit/javadsl/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -429,7 +450,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/remote/package-summary.html"
+ url: ${project-info.javadoc}"remote/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -450,7 +471,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/event/slf4j/package-summary.html"
+ url: ${project-info.javadoc}"event/slf4j/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -471,7 +492,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/stream/package-summary.html"
+ url: ${project-info.javadoc}"stream/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -492,7 +513,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/stream/testkit/package-summary.html"
+ url: ${project-info.javadoc}"stream/testkit/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -518,7 +539,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/stream/package-summary.html"
+ url: ${project-info.javadoc}"stream/package-summary.html"
text: "API (Javadoc)"
}
]
@@ -539,7 +560,7 @@ project-info {
text: "API (Scaladoc)"
}
{
- url: ${project-info.javadoc}"?akka/testkit/package-summary.html"
+ url: ${project-info.javadoc}"testkit/package-summary.html"
text: "API (Javadoc)"
}
]
diff --git a/project/scripts/release b/project/scripts/release
index a5babb8363..a0316f6a33 100755
--- a/project/scripts/release
+++ b/project/scripts/release
@@ -67,7 +67,7 @@
#
# 3.3) Also make it available for publishing snapshots.
# From the command line:
-# shell> cp ~/.ssh/id_rsa.pub ~/.ssh/id_rsa_gustav.pem
+# shell> cp ~/.ssh/id_rsa ~/.ssh/id_rsa_gustav.pem
# shell> ssh-keygen -p -f ~/.ssh/id_rsa_gustav.pem -m pem
#
# 4) Have access to github.com/akka/akka. This should be a given.
@@ -89,7 +89,6 @@ declare -r default_server="akkarepo@gustav.akka.io"
declare -r default_path="www"
# settings
-declare -r release_dir="target/release"
declare release_server=${default_server}
declare release_path=${default_path}
@@ -279,14 +278,9 @@ else
fi
# Release artifacts
-try sbt $RELEASE_OPT +buildRelease
+try sbt $RELEASE_OPT +publishSigned
-echolog "Successfully created released artifacts"
-
-# Build the docs
-try sbt $RELEASE_OPT buildDocs
-
-echolog "Successfully created docs"
+echolog "Successfully released artifacts"
try sbt $RELEASE_OPT whitesourceCheckPolicies
@@ -325,19 +319,14 @@ else
important git push origin --tags
fi
-# push the docs to the server
-echolog "Pushing ${release_dir} docs to ${publish_path} ..."
+echolog "Building docs and pushing to the server..."
if [ $dry_run ]; then
echodry "Not actually pushing to server. Commands:"
- echodry " sbt $RELEASE_OPT deployRsync"
- echodry " rsync -rlpvz --chmod=Dg+ws,Fg+w --exclude ${release_dir}/downloads --exclude ${release_dir}/docs ${release_dir}/ ${publish_path}/"
+ echodry " sbt $RELEASE_OPT publishRsync"
else
important ssh ${release_server} "cd ${release_path}/docs/akka; git add .; git commit -m 'before publishing version $version'; true"
# using Scala 2.13 here to avoid the infamous problem with missing AskSupport in classpath
- important sbt -Dakka.build.scalaVersion=2.13.0 $RELEASE_OPT "deployRsync ${release_server}"
- important rsync -rlpvz --chmod=Dg+ws,Fg+w --exclude downloads --exclude docs ${release_dir}/ ${publish_path}/
- #important ssh ${release_server} cp -v ${release_path}/docs/akka/${version}/_static/warnOldDocs.js ${release_path}/docs/akka
- #important ssh ${release_server} ln -snvf ../../warnOldDocs.js ${release_path}/docs/akka/${version}/_static/warnOldDocs.js
+ important sbt -Dakka.build.scalaVersion=2.13.0 $RELEASE_OPT publishRsync
important ssh ${release_server} "cd ${release_path}/docs/akka; git add .; git commit -m 'publish version $version'"
fi