2016-05-04 13:31:08 +02:00
|
|
|
/**
|
|
|
|
|
* Copyright (C) 2016 Lightbend Inc. <http://www.lightbend.com>
|
|
|
|
|
*/
|
|
|
|
|
package akka.remote.artery
|
|
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
import java.util.concurrent.ThreadLocalRandom
|
|
|
|
|
|
2016-05-04 13:31:08 +02:00
|
|
|
import scala.concurrent.Await
|
|
|
|
|
import scala.concurrent.duration._
|
|
|
|
|
|
|
|
|
|
import akka.NotUsed
|
|
|
|
|
import akka.actor.ActorIdentity
|
|
|
|
|
import akka.actor.ActorSystem
|
|
|
|
|
import akka.actor.ExtendedActorSystem
|
|
|
|
|
import akka.actor.Identify
|
|
|
|
|
import akka.actor.InternalActorRef
|
|
|
|
|
import akka.actor.PoisonPill
|
|
|
|
|
import akka.actor.RootActorPath
|
2016-05-09 07:31:41 +02:00
|
|
|
import akka.remote.AddressUidExtension
|
2016-05-04 13:31:08 +02:00
|
|
|
import akka.remote.EndpointManager.Send
|
|
|
|
|
import akka.remote.RemoteActorRef
|
2016-05-09 07:31:41 +02:00
|
|
|
import akka.remote.UniqueAddress
|
2016-05-04 13:31:08 +02:00
|
|
|
import akka.remote.artery.SystemMessageDelivery._
|
|
|
|
|
import akka.stream.ActorMaterializer
|
|
|
|
|
import akka.stream.ActorMaterializerSettings
|
|
|
|
|
import akka.stream.ThrottleMode
|
|
|
|
|
import akka.stream.scaladsl.Flow
|
|
|
|
|
import akka.stream.scaladsl.Sink
|
|
|
|
|
import akka.stream.scaladsl.Source
|
|
|
|
|
import akka.stream.testkit.scaladsl.TestSink
|
|
|
|
|
import akka.testkit.AkkaSpec
|
|
|
|
|
import akka.testkit.ImplicitSender
|
|
|
|
|
import akka.testkit.SocketUtil
|
|
|
|
|
import akka.testkit.TestActors
|
|
|
|
|
import akka.testkit.TestProbe
|
|
|
|
|
import com.typesafe.config.ConfigFactory
|
|
|
|
|
|
|
|
|
|
object SystemMessageDeliverySpec {
|
|
|
|
|
|
|
|
|
|
val Seq(portA, portB) = SocketUtil.temporaryServerAddresses(2, "localhost", udp = true).map(_.getPort)
|
|
|
|
|
|
|
|
|
|
val commonConfig = ConfigFactory.parseString(s"""
|
|
|
|
|
akka {
|
|
|
|
|
actor.provider = "akka.remote.RemoteActorRefProvider"
|
|
|
|
|
remote.artery.enabled = on
|
|
|
|
|
remote.artery.hostname = localhost
|
|
|
|
|
remote.artery.port = $portA
|
|
|
|
|
}
|
|
|
|
|
akka.actor.serialize-creators = off
|
|
|
|
|
akka.actor.serialize-messages = off
|
|
|
|
|
""")
|
|
|
|
|
|
|
|
|
|
val configB = ConfigFactory.parseString(s"akka.remote.artery.port = $portB")
|
|
|
|
|
.withFallback(commonConfig)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commonConfig) with ImplicitSender {
|
|
|
|
|
import SystemMessageDeliverySpec._
|
|
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
val addressA = UniqueAddress(
|
|
|
|
|
system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress,
|
|
|
|
|
AddressUidExtension(system).addressUid)
|
2016-05-04 13:31:08 +02:00
|
|
|
val systemB = ActorSystem("systemB", configB)
|
2016-05-09 07:31:41 +02:00
|
|
|
val addressB = UniqueAddress(
|
|
|
|
|
systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress,
|
|
|
|
|
AddressUidExtension(systemB).addressUid)
|
|
|
|
|
val rootB = RootActorPath(addressB.address)
|
2016-05-04 13:31:08 +02:00
|
|
|
val matSettings = ActorMaterializerSettings(system).withFuzzing(true)
|
|
|
|
|
implicit val mat = ActorMaterializer(matSettings)(system)
|
|
|
|
|
|
|
|
|
|
override def afterTermination(): Unit = shutdown(systemB)
|
|
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
private def send(sendCount: Int, resendInterval: FiniteDuration, outboundContext: OutboundContext): Source[Send, NotUsed] = {
|
2016-05-04 13:31:08 +02:00
|
|
|
val remoteRef = null.asInstanceOf[RemoteActorRef] // not used
|
|
|
|
|
Source(1 to sendCount)
|
|
|
|
|
.map(n ⇒ Send("msg-" + n, None, remoteRef, None))
|
2016-05-13 15:34:37 +02:00
|
|
|
.via(new SystemMessageDelivery(outboundContext, resendInterval, maxBufferSize = 1000))
|
2016-05-04 13:31:08 +02:00
|
|
|
}
|
|
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
private def inbound(inboundContext: InboundContext): Flow[Send, InboundEnvelope, NotUsed] = {
|
2016-05-04 13:31:08 +02:00
|
|
|
val recipient = null.asInstanceOf[InternalActorRef] // not used
|
|
|
|
|
Flow[Send]
|
|
|
|
|
.map {
|
|
|
|
|
case Send(sysEnv: SystemMessageEnvelope, _, _, _) ⇒
|
2016-05-13 08:06:13 +02:00
|
|
|
InboundEnvelope(recipient, addressB.address, sysEnv, None, addressA)
|
2016-05-04 13:31:08 +02:00
|
|
|
}
|
|
|
|
|
.async
|
2016-05-09 07:31:41 +02:00
|
|
|
.via(new SystemMessageAcker(inboundContext))
|
2016-05-04 13:31:08 +02:00
|
|
|
}
|
|
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
private def drop(dropSeqNumbers: Vector[Long]): Flow[Send, Send, NotUsed] = {
|
2016-05-04 13:31:08 +02:00
|
|
|
Flow[Send]
|
|
|
|
|
.statefulMapConcat(() ⇒ {
|
|
|
|
|
var dropping = dropSeqNumbers
|
|
|
|
|
|
|
|
|
|
{
|
|
|
|
|
case s @ Send(SystemMessageEnvelope(_, seqNo, _), _, _, _) ⇒
|
|
|
|
|
val i = dropping.indexOf(seqNo)
|
|
|
|
|
if (i >= 0) {
|
|
|
|
|
dropping = dropping.updated(i, -1L)
|
|
|
|
|
Nil
|
|
|
|
|
} else
|
|
|
|
|
List(s)
|
|
|
|
|
}
|
|
|
|
|
})
|
|
|
|
|
}
|
|
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
private def randomDrop[T](dropRate: Double): Flow[T, T, NotUsed] = Flow[T].mapConcat { elem ⇒
|
2016-05-04 13:31:08 +02:00
|
|
|
if (ThreadLocalRandom.current().nextDouble() < dropRate) Nil
|
|
|
|
|
else List(elem)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"System messages" must {
|
|
|
|
|
|
|
|
|
|
"be delivered with real actors" in {
|
|
|
|
|
val actorOnSystemB = systemB.actorOf(TestActors.echoActorProps, "echo")
|
|
|
|
|
|
|
|
|
|
val remoteRef = {
|
|
|
|
|
system.actorSelection(rootB / "user" / "echo") ! Identify(None)
|
|
|
|
|
expectMsgType[ActorIdentity].ref.get
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
watch(remoteRef)
|
|
|
|
|
remoteRef ! PoisonPill
|
|
|
|
|
expectTerminated(remoteRef)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"be resent when some in the middle are lost" in {
|
2016-05-09 07:31:41 +02:00
|
|
|
val replyProbe = TestProbe()
|
2016-05-12 08:56:28 +02:00
|
|
|
val controlSubject = new TestControlMessageSubject
|
|
|
|
|
val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, controlSubject)
|
|
|
|
|
val inboundContextA = new TestInboundContext(addressB, controlSubject)
|
2016-05-09 07:31:41 +02:00
|
|
|
val outboundContextA = inboundContextA.association(addressB.address)
|
|
|
|
|
|
|
|
|
|
val sink = send(sendCount = 5, resendInterval = 60.seconds, outboundContextA)
|
|
|
|
|
.via(drop(dropSeqNumbers = Vector(3L, 4L)))
|
|
|
|
|
.via(inbound(inboundContextB))
|
|
|
|
|
.map(_.message.asInstanceOf[String])
|
|
|
|
|
.runWith(TestSink.probe)
|
2016-05-04 13:31:08 +02:00
|
|
|
|
|
|
|
|
sink.request(100)
|
|
|
|
|
sink.expectNext("msg-1")
|
|
|
|
|
sink.expectNext("msg-2")
|
2016-05-09 07:31:41 +02:00
|
|
|
replyProbe.expectMsg(Ack(1L, addressB))
|
|
|
|
|
replyProbe.expectMsg(Ack(2L, addressB))
|
2016-05-04 13:31:08 +02:00
|
|
|
// 3 and 4 was dropped
|
2016-05-09 07:31:41 +02:00
|
|
|
replyProbe.expectMsg(Nack(2L, addressB))
|
2016-05-04 13:31:08 +02:00
|
|
|
sink.expectNoMsg(100.millis) // 3 was dropped
|
2016-05-09 07:31:41 +02:00
|
|
|
inboundContextB.deliverLastReply()
|
2016-05-04 13:31:08 +02:00
|
|
|
// resending 3, 4, 5
|
|
|
|
|
sink.expectNext("msg-3")
|
2016-05-09 07:31:41 +02:00
|
|
|
replyProbe.expectMsg(Ack(3L, addressB))
|
2016-05-04 13:31:08 +02:00
|
|
|
sink.expectNext("msg-4")
|
2016-05-09 07:31:41 +02:00
|
|
|
replyProbe.expectMsg(Ack(4L, addressB))
|
2016-05-04 13:31:08 +02:00
|
|
|
sink.expectNext("msg-5")
|
2016-05-09 07:31:41 +02:00
|
|
|
replyProbe.expectMsg(Ack(5L, addressB))
|
|
|
|
|
replyProbe.expectNoMsg(100.millis)
|
|
|
|
|
inboundContextB.deliverLastReply()
|
2016-05-04 13:31:08 +02:00
|
|
|
sink.expectComplete()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"be resent when first is lost" in {
|
2016-05-09 07:31:41 +02:00
|
|
|
val replyProbe = TestProbe()
|
2016-05-12 08:56:28 +02:00
|
|
|
val controlSubject = new TestControlMessageSubject
|
|
|
|
|
val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, controlSubject)
|
|
|
|
|
val inboundContextA = new TestInboundContext(addressB, controlSubject)
|
2016-05-09 07:31:41 +02:00
|
|
|
val outboundContextA = inboundContextA.association(addressB.address)
|
|
|
|
|
|
|
|
|
|
val sink = send(sendCount = 3, resendInterval = 60.seconds, outboundContextA)
|
|
|
|
|
.via(drop(dropSeqNumbers = Vector(1L)))
|
|
|
|
|
.via(inbound(inboundContextB))
|
|
|
|
|
.map(_.message.asInstanceOf[String])
|
|
|
|
|
.runWith(TestSink.probe)
|
2016-05-04 13:31:08 +02:00
|
|
|
|
|
|
|
|
sink.request(100)
|
2016-05-09 07:31:41 +02:00
|
|
|
replyProbe.expectMsg(Nack(0L, addressB)) // from receiving 2
|
|
|
|
|
replyProbe.expectMsg(Nack(0L, addressB)) // from receiving 3
|
2016-05-04 13:31:08 +02:00
|
|
|
sink.expectNoMsg(100.millis) // 1 was dropped
|
2016-05-09 07:31:41 +02:00
|
|
|
inboundContextB.deliverLastReply() // it's ok to not delivery all nacks
|
2016-05-04 13:31:08 +02:00
|
|
|
// resending 1, 2, 3
|
|
|
|
|
sink.expectNext("msg-1")
|
2016-05-09 07:31:41 +02:00
|
|
|
replyProbe.expectMsg(Ack(1L, addressB))
|
2016-05-04 13:31:08 +02:00
|
|
|
sink.expectNext("msg-2")
|
2016-05-09 07:31:41 +02:00
|
|
|
replyProbe.expectMsg(Ack(2L, addressB))
|
2016-05-04 13:31:08 +02:00
|
|
|
sink.expectNext("msg-3")
|
2016-05-09 07:31:41 +02:00
|
|
|
replyProbe.expectMsg(Ack(3L, addressB))
|
|
|
|
|
inboundContextB.deliverLastReply()
|
2016-05-04 13:31:08 +02:00
|
|
|
sink.expectComplete()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"be resent when last is lost" in {
|
2016-05-09 07:31:41 +02:00
|
|
|
val replyProbe = TestProbe()
|
2016-05-12 08:56:28 +02:00
|
|
|
val controlSubject = new TestControlMessageSubject
|
|
|
|
|
val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, controlSubject)
|
|
|
|
|
val inboundContextA = new TestInboundContext(addressB, controlSubject)
|
2016-05-09 07:31:41 +02:00
|
|
|
val outboundContextA = inboundContextA.association(addressB.address)
|
|
|
|
|
|
2016-05-17 14:05:50 +02:00
|
|
|
val sink = send(sendCount = 3, resendInterval = 2.seconds, outboundContextA)
|
2016-05-09 07:31:41 +02:00
|
|
|
.via(drop(dropSeqNumbers = Vector(3L)))
|
|
|
|
|
.via(inbound(inboundContextB))
|
|
|
|
|
.map(_.message.asInstanceOf[String])
|
|
|
|
|
.runWith(TestSink.probe)
|
2016-05-04 13:31:08 +02:00
|
|
|
|
|
|
|
|
sink.request(100)
|
|
|
|
|
sink.expectNext("msg-1")
|
2016-05-09 07:31:41 +02:00
|
|
|
replyProbe.expectMsg(Ack(1L, addressB))
|
|
|
|
|
inboundContextB.deliverLastReply()
|
2016-05-04 13:31:08 +02:00
|
|
|
sink.expectNext("msg-2")
|
2016-05-09 07:31:41 +02:00
|
|
|
replyProbe.expectMsg(Ack(2L, addressB))
|
|
|
|
|
inboundContextB.deliverLastReply()
|
2016-05-04 13:31:08 +02:00
|
|
|
sink.expectNoMsg(200.millis) // 3 was dropped
|
|
|
|
|
// resending 3 due to timeout
|
|
|
|
|
sink.expectNext("msg-3")
|
2016-05-17 14:05:50 +02:00
|
|
|
replyProbe.expectMsg(4.seconds, Ack(3L, addressB))
|
|
|
|
|
// continue resending
|
|
|
|
|
replyProbe.expectMsg(4.seconds, Ack(3L, addressB))
|
2016-05-09 07:31:41 +02:00
|
|
|
inboundContextB.deliverLastReply()
|
2016-05-17 14:05:50 +02:00
|
|
|
replyProbe.expectNoMsg(2200.millis)
|
2016-05-04 13:31:08 +02:00
|
|
|
sink.expectComplete()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"deliver all during stress and random dropping" in {
|
|
|
|
|
val N = 10000
|
|
|
|
|
val dropRate = 0.1
|
2016-05-12 08:56:28 +02:00
|
|
|
val controlSubject = new TestControlMessageSubject
|
|
|
|
|
val inboundContextB = new TestInboundContext(addressB, controlSubject, replyDropRate = dropRate)
|
|
|
|
|
val inboundContextA = new TestInboundContext(addressB, controlSubject)
|
2016-05-09 07:31:41 +02:00
|
|
|
val outboundContextA = inboundContextA.association(addressB.address)
|
2016-05-04 13:31:08 +02:00
|
|
|
|
|
|
|
|
val output =
|
2016-05-09 07:31:41 +02:00
|
|
|
send(N, 1.second, outboundContextA)
|
2016-05-04 13:31:08 +02:00
|
|
|
.via(randomDrop(dropRate))
|
2016-05-09 07:31:41 +02:00
|
|
|
.via(inbound(inboundContextB))
|
2016-05-04 13:31:08 +02:00
|
|
|
.map(_.message.asInstanceOf[String])
|
|
|
|
|
.runWith(Sink.seq)
|
|
|
|
|
|
|
|
|
|
Await.result(output, 20.seconds) should ===((1 to N).map("msg-" + _).toVector)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"deliver all during throttling and random dropping" in {
|
|
|
|
|
val N = 500
|
|
|
|
|
val dropRate = 0.1
|
2016-05-12 08:56:28 +02:00
|
|
|
val controlSubject = new TestControlMessageSubject
|
|
|
|
|
val inboundContextB = new TestInboundContext(addressB, controlSubject, replyDropRate = dropRate)
|
|
|
|
|
val inboundContextA = new TestInboundContext(addressB, controlSubject)
|
2016-05-09 07:31:41 +02:00
|
|
|
val outboundContextA = inboundContextA.association(addressB.address)
|
2016-05-04 13:31:08 +02:00
|
|
|
|
|
|
|
|
val output =
|
2016-05-09 07:31:41 +02:00
|
|
|
send(N, 1.second, outboundContextA)
|
2016-05-04 13:31:08 +02:00
|
|
|
.throttle(200, 1.second, 10, ThrottleMode.shaping)
|
|
|
|
|
.via(randomDrop(dropRate))
|
2016-05-09 07:31:41 +02:00
|
|
|
.via(inbound(inboundContextB))
|
2016-05-04 13:31:08 +02:00
|
|
|
.map(_.message.asInstanceOf[String])
|
|
|
|
|
.runWith(Sink.seq)
|
|
|
|
|
|
|
|
|
|
Await.result(output, 20.seconds) should ===((1 to N).map("msg-" + _).toVector)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|