Rename sbt akka modules
Co-authored-by: Sean Glover <sean@seanglover.com>
This commit is contained in:
parent
b92b749946
commit
24c03cde19
2930 changed files with 1466 additions and 1462 deletions
File diff suppressed because it is too large
Load diff
|
|
@ -0,0 +1,70 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
syntax = "proto2";
|
||||
|
||||
option java_package = "org.apache.pekko.remote.testconductor";
|
||||
option optimize_for = SPEED;
|
||||
|
||||
message Wrapper {
|
||||
optional Hello hello = 1;
|
||||
optional EnterBarrier barrier = 2;
|
||||
optional InjectFailure failure = 3;
|
||||
optional string done = 4;
|
||||
optional AddressRequest addr = 5;
|
||||
}
|
||||
|
||||
message Hello {
|
||||
required string name = 1;
|
||||
required Address address = 2;
|
||||
}
|
||||
|
||||
enum BarrierOp {
|
||||
Enter = 1;
|
||||
Fail = 2;
|
||||
Succeeded = 3;
|
||||
Failed = 4;
|
||||
}
|
||||
|
||||
message EnterBarrier {
|
||||
required string name = 1;
|
||||
required BarrierOp op = 2;
|
||||
optional int64 timeout = 3;
|
||||
}
|
||||
|
||||
message AddressRequest {
|
||||
required string node = 1;
|
||||
optional Address addr = 2;
|
||||
}
|
||||
|
||||
message Address {
|
||||
required string protocol = 1;
|
||||
required string system = 2;
|
||||
required string host = 3;
|
||||
required int32 port = 4;
|
||||
}
|
||||
|
||||
enum FailType {
|
||||
Throttle = 1;
|
||||
Disconnect = 2;
|
||||
Abort = 3;
|
||||
Exit = 4;
|
||||
Shutdown = 5;
|
||||
ShutdownAbrupt = 6;
|
||||
}
|
||||
|
||||
enum Direction {
|
||||
Send = 1;
|
||||
Receive = 2;
|
||||
Both = 3;
|
||||
}
|
||||
|
||||
message InjectFailure {
|
||||
required FailType failure = 1;
|
||||
optional Direction direction = 2;
|
||||
optional Address address = 3;
|
||||
optional float rateMBit = 6;
|
||||
optional int32 exitValue = 7;
|
||||
}
|
||||
|
||||
65
multi-node-testkit/src/main/resources/reference.conf
Normal file
65
multi-node-testkit/src/main/resources/reference.conf
Normal file
|
|
@ -0,0 +1,65 @@
|
|||
#############################################
|
||||
# Pekko Remote Testing Reference Config File #
|
||||
#############################################
|
||||
|
||||
# This is the reference config file that contains all the default settings.
|
||||
# Make your edits/overrides in your application.conf.
|
||||
|
||||
pekko {
|
||||
testconductor {
|
||||
|
||||
# Timeout for joining a barrier: this is the maximum time any participants
|
||||
# waits for everybody else to join a named barrier.
|
||||
barrier-timeout = 30s
|
||||
|
||||
# Timeout for interrogation of TestConductor’s Controller actor
|
||||
query-timeout = 10s
|
||||
|
||||
# Threshold for packet size in time unit above which the failure injector will
|
||||
# split the packet and deliver in smaller portions; do not give value smaller
|
||||
# than HashedWheelTimer resolution (would not make sense)
|
||||
packet-split-threshold = 100ms
|
||||
|
||||
# amount of time for the ClientFSM to wait for the connection to the conductor
|
||||
# to be successful
|
||||
connect-timeout = 20s
|
||||
|
||||
# Number of connect attempts to be made to the conductor controller
|
||||
client-reconnects = 30
|
||||
|
||||
# minimum time interval which is to be inserted between reconnect attempts
|
||||
reconnect-backoff = 1s
|
||||
|
||||
netty {
|
||||
# (I&O) Used to configure the number of I/O worker threads on server sockets
|
||||
server-socket-worker-pool {
|
||||
# Min number of threads to cap factor-based number to
|
||||
pool-size-min = 1
|
||||
|
||||
# The pool size factor is used to determine thread pool size
|
||||
# using the following formula: ceil(available processors * factor).
|
||||
# Resulting size is then bounded by the pool-size-min and
|
||||
# pool-size-max values.
|
||||
pool-size-factor = 1.0
|
||||
|
||||
# Max number of threads to cap factor-based number to
|
||||
pool-size-max = 2
|
||||
}
|
||||
|
||||
# (I&O) Used to configure the number of I/O worker threads on client sockets
|
||||
client-socket-worker-pool {
|
||||
# Min number of threads to cap factor-based number to
|
||||
pool-size-min = 1
|
||||
|
||||
# The pool size factor is used to determine thread pool size
|
||||
# using the following formula: ceil(available processors * factor).
|
||||
# Resulting size is then bounded by the pool-size-min and
|
||||
# pool-size-max values.
|
||||
pool-size-factor = 1.0
|
||||
|
||||
# Max number of threads to cap factor-based number to
|
||||
pool-size-max = 2
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,675 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.remote.testconductor
|
||||
|
||||
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 org.apache.pekko
|
||||
import pekko.PekkoException
|
||||
import pekko.ConfigurationException
|
||||
import pekko.actor.{
|
||||
Actor,
|
||||
ActorRef,
|
||||
Address,
|
||||
DeadLetterSuppression,
|
||||
Deploy,
|
||||
LoggingFSM,
|
||||
NoSerializationVerificationNeeded,
|
||||
OneForOneStrategy,
|
||||
Props,
|
||||
Status,
|
||||
SupervisorStrategy
|
||||
}
|
||||
import pekko.event.{ Logging, LoggingAdapter }
|
||||
import pekko.event.LoggingReceive
|
||||
import pekko.pattern.ask
|
||||
import pekko.remote.transport.ThrottlerTransportAdapter.Direction
|
||||
import pekko.util.Timeout
|
||||
|
||||
/**
|
||||
* The conductor is the one orchestrating the test: it governs the
|
||||
* [[pekko.remote.testconductor.Controller]]’s port to which all
|
||||
* [[pekko.remote.testconductor.Player]]s connect, it issues commands to their
|
||||
* `org.apache.pekko.remote.testconductor.NetworkFailureInjector` and provides support
|
||||
* for barriers using the [[pekko.remote.testconductor.BarrierCoordinator]].
|
||||
* All of this is bundled inside the [[pekko.remote.testconductor.TestConductorExt]]
|
||||
* extension.
|
||||
*/
|
||||
trait Conductor { this: TestConductorExt =>
|
||||
|
||||
import Controller._
|
||||
|
||||
private var _controller: ActorRef = _
|
||||
private def controller: ActorRef = _controller match {
|
||||
case null => throw new IllegalStateException("TestConductorServer was not started")
|
||||
case x => x
|
||||
}
|
||||
|
||||
/**
|
||||
* Start the [[pekko.remote.testconductor.Controller]], which in turn will
|
||||
* bind to a TCP port as specified in the `pekko.testconductor.port` config
|
||||
* property, where 0 denotes automatic allocation. Since the latter is
|
||||
* actually preferred, a `Future[Int]` is returned which will be completed
|
||||
* with the port number actually chosen, so that this can then be communicated
|
||||
* to the players for their proper start-up.
|
||||
*
|
||||
* This method also invokes [[pekko.remote.testconductor.Player]].startClient,
|
||||
* since it is expected that the conductor participates in barriers for
|
||||
* overall coordination. The returned Future will only be completed once the
|
||||
* client’s start-up finishes, which in fact waits for all other players to
|
||||
* connect.
|
||||
*
|
||||
* @param participants gives the number of participants which shall connect
|
||||
* before any of their startClient() operations complete.
|
||||
*/
|
||||
def startController(
|
||||
participants: Int,
|
||||
name: RoleName,
|
||||
controllerPort: InetSocketAddress): Future[InetSocketAddress] = {
|
||||
if (_controller ne null) throw new RuntimeException("TestConductorServer was already started")
|
||||
_controller = system.systemActorOf(Props(classOf[Controller], participants, controllerPort), "controller")
|
||||
import Settings.BarrierTimeout
|
||||
import system.dispatcher
|
||||
(controller ? GetSockAddr).mapTo[InetSocketAddress].flatMap {
|
||||
case sockAddr: InetSocketAddress => startClient(name, sockAddr).map(_ => sockAddr)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain the port to which the controller’s socket is actually bound. This
|
||||
* will deviate from the configuration in `pekko.testconductor.port` in case
|
||||
* that was given as zero.
|
||||
*/
|
||||
def sockAddr: Future[InetSocketAddress] = {
|
||||
import Settings.QueryTimeout
|
||||
(controller ? GetSockAddr).mapTo(classTag[InetSocketAddress])
|
||||
}
|
||||
|
||||
/**
|
||||
* Make the remoting pipeline on the node throttle data sent to or received
|
||||
* from the given remote peer. Throttling works by delaying packet submission
|
||||
* within the netty pipeline until the packet would have been completely sent
|
||||
* according to the given rate, the previous packet completion and the current
|
||||
* packet length. In case of large packets they are split up if the calculated
|
||||
* send pause would exceed `pekko.testconductor.packet-split-threshold`
|
||||
* (roughly). All of this uses the system’s scheduler, which is not
|
||||
* terribly precise and will execute tasks later than they are schedule (even
|
||||
* on average), but that is countered by using the actual execution time for
|
||||
* determining how much to send, leading to the correct output rate, but with
|
||||
* increased latency.
|
||||
*
|
||||
* ====Note====
|
||||
* To use this feature you must activate the failure injector and throttler
|
||||
* transport adapters by specifying `testTransport(on = true)` in your MultiNodeConfig.
|
||||
*
|
||||
* @param node is the symbolic name of the node which is to be affected
|
||||
* @param target is the symbolic name of the other node to which connectivity shall be throttled
|
||||
* @param direction can be either `Direction.Send`, `Direction.Receive` or `Direction.Both`
|
||||
* @param rateMBit is the maximum data rate in MBit
|
||||
*/
|
||||
def throttle(node: RoleName, target: RoleName, direction: Direction, rateMBit: Double): Future[Done] = {
|
||||
import Settings.QueryTimeout
|
||||
requireTestConductorTranport()
|
||||
(controller ? Throttle(node, target, direction, rateMBit.toFloat)).mapTo(classTag[Done])
|
||||
}
|
||||
|
||||
/**
|
||||
* Switch the Netty pipeline of the remote support into blackhole mode for
|
||||
* sending and/or receiving: it will just drop all messages right before
|
||||
* submitting them to the Socket or right after receiving them from the
|
||||
* Socket.
|
||||
*
|
||||
* ====Note====
|
||||
* To use this feature you must activate the failure injector and throttler
|
||||
* transport adapters by specifying `testTransport(on = true)` in your MultiNodeConfig.
|
||||
*
|
||||
* @param node is the symbolic name of the node which is to be affected
|
||||
* @param target is the symbolic name of the other node to which connectivity shall be impeded
|
||||
* @param direction can be either `Direction.Send`, `Direction.Receive` or `Direction.Both`
|
||||
*/
|
||||
def blackhole(node: RoleName, target: RoleName, direction: Direction): Future[Done] =
|
||||
throttle(node, target, direction, 0f)
|
||||
|
||||
private def requireTestConductorTranport(): Unit = {
|
||||
if (transport.provider.remoteSettings.Artery.Enabled) {
|
||||
if (!transport.provider.remoteSettings.Artery.Advanced.TestMode)
|
||||
throw new ConfigurationException(
|
||||
"To use this feature you must activate the test mode " +
|
||||
"by specifying `testTransport(on = true)` in your MultiNodeConfig.")
|
||||
} else {
|
||||
if (!transport.defaultAddress.protocol.contains(".trttl.gremlin."))
|
||||
throw new ConfigurationException(
|
||||
"To use this feature you must activate the failure injector adapters " +
|
||||
"(trttl, gremlin) by specifying `testTransport(on = true)` in your MultiNodeConfig.")
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Switch the Netty pipeline of the remote support into pass through mode for
|
||||
* sending and/or receiving.
|
||||
*
|
||||
* ====Note====
|
||||
* To use this feature you must activate the failure injector and throttler
|
||||
* transport adapters by specifying `testTransport(on = true)` in your MultiNodeConfig.
|
||||
*
|
||||
* @param node is the symbolic name of the node which is to be affected
|
||||
* @param target is the symbolic name of the other node to which connectivity shall be impeded
|
||||
* @param direction can be either `Direction.Send`, `Direction.Receive` or `Direction.Both`
|
||||
*/
|
||||
def passThrough(node: RoleName, target: RoleName, direction: Direction): Future[Done] =
|
||||
throttle(node, target, direction, -1f)
|
||||
|
||||
/**
|
||||
* Tell the remote support to shutdown the connection to the given remote
|
||||
* peer. It works regardless of whether the recipient was initiator or
|
||||
* responder.
|
||||
*
|
||||
* @param node is the symbolic name of the node which is to be affected
|
||||
* @param target is the symbolic name of the other node to which connectivity shall be impeded
|
||||
*/
|
||||
def disconnect(node: RoleName, target: RoleName): Future[Done] = {
|
||||
import Settings.QueryTimeout
|
||||
(controller ? Disconnect(node, target, false)).mapTo(classTag[Done])
|
||||
}
|
||||
|
||||
/**
|
||||
* Tell the remote support to TCP_RESET the connection to the given remote
|
||||
* peer. It works regardless of whether the recipient was initiator or
|
||||
* responder.
|
||||
*
|
||||
* @param node is the symbolic name of the node which is to be affected
|
||||
* @param target is the symbolic name of the other node to which connectivity shall be impeded
|
||||
*/
|
||||
def abort(node: RoleName, target: RoleName): Future[Done] = {
|
||||
import Settings.QueryTimeout
|
||||
(controller ? Disconnect(node, target, true)).mapTo(classTag[Done])
|
||||
}
|
||||
|
||||
/**
|
||||
* Tell the remote node to shut itself down using System.exit with the given
|
||||
* exitValue. The node will also be removed, so that the remaining nodes may still
|
||||
* pass subsequent barriers.
|
||||
*
|
||||
* @param node is the symbolic name of the node which is to be affected
|
||||
* @param exitValue is the return code which shall be given to System.exit
|
||||
*/
|
||||
def exit(node: RoleName, exitValue: Int): Future[Done] = {
|
||||
import Settings.QueryTimeout
|
||||
import system.dispatcher
|
||||
// the recover is needed to handle ClientDisconnectedException exception,
|
||||
// which is normal during shutdown
|
||||
(controller ? Terminate(node, Right(exitValue))).mapTo(classTag[Done]).recover {
|
||||
case _: ClientDisconnectedException => Done
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tell the actor system at the remote node to shut itself down. The node will also be
|
||||
* removed, so that the remaining nodes may still pass subsequent barriers.
|
||||
*
|
||||
* @param node is the symbolic name of the node which is to be affected
|
||||
*/
|
||||
def shutdown(node: RoleName): Future[Done] = shutdown(node, abort = false)
|
||||
|
||||
/**
|
||||
* Tell the actor system at the remote node to shut itself down without
|
||||
* awaiting termination of remote-deployed children. The node will also be
|
||||
* removed, so that the remaining nodes may still pass subsequent barriers.
|
||||
*
|
||||
* @param node is the symbolic name of the node which is to be affected
|
||||
*/
|
||||
def shutdown(node: RoleName, abort: Boolean): Future[Done] = {
|
||||
import Settings.QueryTimeout
|
||||
import system.dispatcher
|
||||
// the recover is needed to handle ClientDisconnectedException exception,
|
||||
// which is normal during shutdown
|
||||
(controller ? Terminate(node, Left(abort))).mapTo(classTag[Done]).recover {
|
||||
case _: ClientDisconnectedException => Done
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain the list of remote host names currently registered.
|
||||
*/
|
||||
def getNodes: Future[Iterable[RoleName]] = {
|
||||
import Settings.QueryTimeout
|
||||
(controller ? GetNodes).mapTo(classTag[Iterable[RoleName]])
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove a remote host from the list, so that the remaining nodes may still
|
||||
* pass subsequent barriers. This must be done before the client connection
|
||||
* breaks down in order to affect an “orderly” removal (i.e. without failing
|
||||
* present and future barriers).
|
||||
*
|
||||
* @param node is the symbolic name of the node which is to be removed
|
||||
*/
|
||||
def removeNode(node: RoleName): Future[Done] = {
|
||||
import Settings.QueryTimeout
|
||||
(controller ? Remove(node)).mapTo(classTag[Done])
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* This handler is installed at the end of the controller’s netty pipeline. Its only
|
||||
* purpose is to dispatch incoming messages to the right ServerFSM actor. There is
|
||||
* one shared instance of this class for all connections accepted by one Controller.
|
||||
*
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] class ConductorHandler(_createTimeout: Timeout, controller: ActorRef, log: LoggingAdapter)
|
||||
extends SimpleChannelUpstreamHandler {
|
||||
|
||||
implicit val createTimeout: Timeout = _createTimeout
|
||||
val clients = new ConcurrentHashMap[Channel, ActorRef]()
|
||||
|
||||
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val channel = event.getChannel
|
||||
log.debug("connection from {}", getAddrString(channel))
|
||||
val fsm: ActorRef =
|
||||
Await.result((controller ? Controller.CreateServerFSM(channel)).mapTo(classTag[ActorRef]), Duration.Inf)
|
||||
clients.put(channel, fsm)
|
||||
}
|
||||
|
||||
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val channel = event.getChannel
|
||||
log.debug("disconnect from {}", getAddrString(channel))
|
||||
val fsm = clients.get(channel)
|
||||
fsm ! Controller.ClientDisconnected
|
||||
clients.remove(channel)
|
||||
}
|
||||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = {
|
||||
val channel = event.getChannel
|
||||
log.debug("message from {}: {}", getAddrString(channel), event.getMessage)
|
||||
event.getMessage match {
|
||||
case msg: NetworkOp =>
|
||||
clients.get(channel) ! msg
|
||||
case msg =>
|
||||
log.info("client {} sent garbage '{}', disconnecting", getAddrString(channel), msg)
|
||||
channel.close()
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] object ServerFSM {
|
||||
sealed trait State
|
||||
case object Initial extends State
|
||||
case object Ready extends State
|
||||
}
|
||||
|
||||
/**
|
||||
* The server part of each client connection is represented by a ServerFSM.
|
||||
* The Initial state handles reception of the new client’s
|
||||
* [[pekko.remote.testconductor.Hello]] message (which is needed for all subsequent
|
||||
* node name translations).
|
||||
*
|
||||
* In the Ready state, messages from the client are forwarded to the controller
|
||||
* and `Send` requests are sent, but the latter is
|
||||
* treated specially: all client operations are to be confirmed by a
|
||||
* [[pekko.remote.testconductor.Done]] message, and there can be only one such
|
||||
* request outstanding at a given time (i.e. a Send fails if the previous has
|
||||
* not yet been acknowledged).
|
||||
*
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] class ServerFSM(val controller: ActorRef, val channel: Channel)
|
||||
extends Actor
|
||||
with LoggingFSM[ServerFSM.State, Option[ActorRef]] {
|
||||
import Controller._
|
||||
import ServerFSM._
|
||||
|
||||
var roleName: RoleName = null
|
||||
|
||||
startWith(Initial, None)
|
||||
|
||||
whenUnhandled {
|
||||
case Event(ClientDisconnected, Some(s)) =>
|
||||
s ! Status.Failure(new ClientDisconnectedException("client disconnected in state " + stateName + ": " + channel))
|
||||
stop()
|
||||
case Event(ClientDisconnected, None) => stop()
|
||||
}
|
||||
|
||||
onTermination {
|
||||
case _ =>
|
||||
controller ! ClientDisconnected(roleName)
|
||||
channel.close()
|
||||
}
|
||||
|
||||
when(Initial, stateTimeout = 10 seconds) {
|
||||
case Event(Hello(name, address), _) =>
|
||||
roleName = RoleName(name)
|
||||
controller ! NodeInfo(roleName, address, self)
|
||||
goto(Ready)
|
||||
case Event(x: NetworkOp, _) =>
|
||||
log.warning("client {} sent no Hello in first message (instead {}), disconnecting", getAddrString(channel), x)
|
||||
channel.close()
|
||||
stop()
|
||||
case Event(ToClient(msg), _) =>
|
||||
log.warning("cannot send {} in state Initial", msg)
|
||||
stay()
|
||||
case Event(StateTimeout, _) =>
|
||||
log.info("closing channel to {} because of Hello timeout", getAddrString(channel))
|
||||
channel.close()
|
||||
stop()
|
||||
}
|
||||
|
||||
when(Ready) {
|
||||
case Event(d: Done, Some(s)) =>
|
||||
s ! d
|
||||
stay().using(None)
|
||||
case Event(op: ServerOp, _) =>
|
||||
controller ! op
|
||||
stay()
|
||||
case Event(msg: NetworkOp, _) =>
|
||||
log.warning("client {} sent unsupported message {}", getAddrString(channel), msg)
|
||||
stop()
|
||||
case Event(ToClient(msg: UnconfirmedClientOp), _) =>
|
||||
channel.write(msg)
|
||||
stay()
|
||||
case Event(ToClient(msg), None) =>
|
||||
channel.write(msg)
|
||||
stay().using(Some(sender()))
|
||||
case Event(ToClient(msg), _) =>
|
||||
log.warning("cannot send {} while waiting for previous ACK", msg)
|
||||
stay()
|
||||
}
|
||||
|
||||
initialize()
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] object Controller {
|
||||
final case class ClientDisconnected(name: RoleName) extends DeadLetterSuppression
|
||||
class ClientDisconnectedException(msg: String) extends PekkoException(msg) with NoStackTrace
|
||||
case object GetNodes
|
||||
case object GetSockAddr
|
||||
final case class CreateServerFSM(channel: Channel) extends NoSerializationVerificationNeeded
|
||||
|
||||
final case class NodeInfo(name: RoleName, addr: Address, fsm: ActorRef)
|
||||
}
|
||||
|
||||
/**
|
||||
* This controls test execution by managing barriers (delegated to
|
||||
* [[pekko.remote.testconductor.BarrierCoordinator]], its child) and allowing
|
||||
* network and other failures to be injected at the test nodes.
|
||||
*
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] class Controller(private var initialParticipants: Int, controllerPort: InetSocketAddress) extends Actor {
|
||||
import BarrierCoordinator._
|
||||
import Controller._
|
||||
|
||||
val settings = TestConductor().Settings
|
||||
val connection = RemoteConnection(
|
||||
Server,
|
||||
controllerPort,
|
||||
settings.ServerSocketWorkerPoolSize,
|
||||
new ConductorHandler(settings.QueryTimeout, self, Logging(context.system, classOf[ConductorHandler])))
|
||||
|
||||
/*
|
||||
* Supervision of the BarrierCoordinator means to catch all his bad emotions
|
||||
* and sometimes console him (BarrierEmpty, BarrierTimeout), sometimes tell
|
||||
* him to hate the world (WrongBarrier, DuplicateNode, ClientLost). The latter shall help
|
||||
* terminate broken tests as quickly as possible (i.e. without awaiting
|
||||
* BarrierTimeouts in the players).
|
||||
*/
|
||||
override def supervisorStrategy = OneForOneStrategy() {
|
||||
case BarrierTimeout(data) => failBarrier(data)
|
||||
case FailedBarrier(data) => failBarrier(data)
|
||||
case BarrierEmpty(_, _) => SupervisorStrategy.Resume
|
||||
case WrongBarrier(name, client, data) => { client ! ToClient(BarrierResult(name, false)); failBarrier(data) }
|
||||
case ClientLost(data, _) => failBarrier(data)
|
||||
case DuplicateNode(data, _) => failBarrier(data)
|
||||
}
|
||||
|
||||
def failBarrier(data: Data): SupervisorStrategy.Directive = {
|
||||
for (c <- data.arrived) c ! ToClient(BarrierResult(data.barrier, false))
|
||||
SupervisorStrategy.Restart
|
||||
}
|
||||
|
||||
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)
|
||||
var addrInterest = Map[RoleName, Set[ActorRef]]()
|
||||
val generation = Iterator.from(1)
|
||||
|
||||
override def receive = LoggingReceive {
|
||||
case CreateServerFSM(channel) =>
|
||||
val (ip, port) = channel.getRemoteAddress match {
|
||||
case s: InetSocketAddress => (s.getAddress.getHostAddress, s.getPort)
|
||||
case _ => throw new RuntimeException() // compiler exhaustiveness check pleaser
|
||||
}
|
||||
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)
|
||||
if (nodes contains name) {
|
||||
if (initialParticipants > 0) {
|
||||
for (NodeInfo(_, _, client) <- nodes.values) client ! ToClient(BarrierResult("initial startup", false))
|
||||
initialParticipants = 0
|
||||
}
|
||||
fsm ! ToClient(BarrierResult("initial startup", false))
|
||||
} else {
|
||||
nodes += name -> c
|
||||
if (initialParticipants <= 0) fsm ! ToClient(Done)
|
||||
else if (nodes.size == initialParticipants) {
|
||||
for (NodeInfo(_, _, client) <- nodes.values) client ! ToClient(Done)
|
||||
initialParticipants = 0
|
||||
}
|
||||
if (addrInterest contains name) {
|
||||
addrInterest(name).foreach(_ ! ToClient(AddressReply(name, address)))
|
||||
addrInterest -= name
|
||||
}
|
||||
}
|
||||
case c @ ClientDisconnected(name) =>
|
||||
nodes -= name
|
||||
barrier.forward(c)
|
||||
case op: ServerOp =>
|
||||
op match {
|
||||
case _: EnterBarrier => barrier.forward(op)
|
||||
case _: FailBarrier => barrier.forward(op)
|
||||
case GetAddress(node) =>
|
||||
if (nodes contains node) sender() ! ToClient(AddressReply(node, nodes(node).addr))
|
||||
else addrInterest += node -> ((addrInterest.get(node).getOrElse(Set())) + sender())
|
||||
case _: Done => // FIXME what should happen?
|
||||
}
|
||||
case op: CommandOp =>
|
||||
op match {
|
||||
case Throttle(node, target, direction, rateMBit) =>
|
||||
val t = nodes(target)
|
||||
nodes(node).fsm.forward(ToClient(ThrottleMsg(t.addr, direction, rateMBit)))
|
||||
case Disconnect(node, target, abort) =>
|
||||
val t = nodes(target)
|
||||
nodes(node).fsm.forward(ToClient(DisconnectMsg(t.addr, abort)))
|
||||
case Terminate(node, shutdownOrExit) =>
|
||||
barrier ! BarrierCoordinator.RemoveClient(node)
|
||||
nodes(node).fsm.forward(ToClient(TerminateMsg(shutdownOrExit)))
|
||||
nodes -= node
|
||||
case Remove(node) =>
|
||||
barrier ! BarrierCoordinator.RemoveClient(node)
|
||||
}
|
||||
case GetNodes => sender() ! nodes.keys
|
||||
case GetSockAddr => sender() ! connection.getLocalAddress
|
||||
}
|
||||
|
||||
override def postStop(): Unit = {
|
||||
RemoteConnection.shutdown(connection)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] object BarrierCoordinator {
|
||||
sealed trait State
|
||||
case object Idle extends State
|
||||
case object Waiting extends State
|
||||
|
||||
final case class RemoveClient(name: RoleName)
|
||||
|
||||
final case class Data(clients: Set[Controller.NodeInfo], barrier: String, arrived: List[ActorRef], deadline: Deadline)
|
||||
|
||||
trait Printer { this: Product with Throwable with NoStackTrace =>
|
||||
override def toString = productPrefix + productIterator.mkString("(", ", ", ")")
|
||||
}
|
||||
|
||||
final case class BarrierTimeout(data: Data)
|
||||
extends RuntimeException("timeout while waiting for barrier '" + data.barrier + "'")
|
||||
with NoStackTrace
|
||||
with Printer
|
||||
final case class FailedBarrier(data: Data)
|
||||
extends RuntimeException("failing barrier '" + data.barrier + "'")
|
||||
with NoStackTrace
|
||||
with Printer
|
||||
final case class DuplicateNode(data: Data, node: Controller.NodeInfo)
|
||||
extends RuntimeException(node.toString)
|
||||
with NoStackTrace
|
||||
with Printer
|
||||
final case class WrongBarrier(barrier: String, client: ActorRef, data: Data)
|
||||
extends RuntimeException(
|
||||
data.clients.find(_.fsm == client).map(_.name.toString).getOrElse(client.toString) +
|
||||
" tried to enter '" + barrier + "' while we were waiting for '" + data.barrier + "'")
|
||||
with NoStackTrace
|
||||
with Printer
|
||||
final case class BarrierEmpty(data: Data, msg: String) extends RuntimeException(msg) with NoStackTrace with Printer
|
||||
final case class ClientLost(data: Data, client: RoleName)
|
||||
extends RuntimeException("unannounced disconnect of " + client)
|
||||
with NoStackTrace
|
||||
with Printer
|
||||
}
|
||||
|
||||
/**
|
||||
* This barrier coordinator gets informed of players connecting (NodeInfo),
|
||||
* players being deliberately removed (RemoveClient) or failing (ClientDisconnected)
|
||||
* by the controller. It also receives EnterBarrier requests, where upon the first
|
||||
* one received the name of the current barrier is set and all other known clients
|
||||
* are expected to join the barrier, whereupon all of the will be sent the successful
|
||||
* EnterBarrier return message. In case of planned removals, this may just happen
|
||||
* earlier, in case of failures the current barrier (and all subsequent ones) will
|
||||
* be failed by sending BarrierFailed responses.
|
||||
*
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] class BarrierCoordinator
|
||||
extends Actor
|
||||
with LoggingFSM[BarrierCoordinator.State, BarrierCoordinator.Data] {
|
||||
import BarrierCoordinator._
|
||||
import Controller._
|
||||
|
||||
// this shall be set to true if all subsequent barriers shall fail
|
||||
var failed = false
|
||||
|
||||
override def preRestart(reason: Throwable, message: Option[Any]): Unit = {}
|
||||
override def postRestart(reason: Throwable): Unit = { failed = true }
|
||||
|
||||
// TODO what happens with the other waiting players in case of a test failure?
|
||||
|
||||
startWith(Idle, Data(Set(), "", Nil, null))
|
||||
|
||||
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))
|
||||
case Event(ClientDisconnected(name), d @ Data(clients, _, arrived, _)) =>
|
||||
if (arrived.isEmpty)
|
||||
stay().using(d.copy(clients = clients.filterNot(_.name == name)))
|
||||
else {
|
||||
clients.find(_.name == name) match {
|
||||
case None => stay()
|
||||
case Some(c) => throw ClientLost(d.copy(clients = clients - c, arrived = arrived.filterNot(_ == c.fsm)), name)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
when(Idle) {
|
||||
case Event(EnterBarrier(name, timeout), d @ Data(clients, _, _, _)) =>
|
||||
if (failed)
|
||||
stay().replying(ToClient(BarrierResult(name, false)))
|
||||
else if (clients.map(_.fsm) == Set(sender()))
|
||||
stay().replying(ToClient(BarrierResult(name, true)))
|
||||
else if (clients.find(_.fsm == sender()).isEmpty)
|
||||
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)))
|
||||
}
|
||||
|
||||
onTransition {
|
||||
case Idle -> Waiting => startSingleTimer("Timeout", StateTimeout, nextStateData.deadline.timeLeft)
|
||||
case Waiting -> Idle => cancelTimer("Timeout")
|
||||
}
|
||||
|
||||
when(Waiting) {
|
||||
case Event(EnterBarrier(name, timeout), d @ Data(clients, barrier, arrived, deadline)) =>
|
||||
if (name != barrier) throw WrongBarrier(name, sender(), d)
|
||||
val together = if (clients.exists(_.fsm == sender())) sender() :: arrived else arrived
|
||||
val enterDeadline = getDeadline(timeout)
|
||||
// we only allow the deadlines to get shorter
|
||||
if (enterDeadline.timeLeft < deadline.timeLeft) {
|
||||
startSingleTimer("Timeout", StateTimeout, enterDeadline.timeLeft)
|
||||
handleBarrier(d.copy(arrived = together, deadline = enterDeadline))
|
||||
} else
|
||||
handleBarrier(d.copy(arrived = together))
|
||||
case Event(RemoveClient(name), d @ Data(clients, _, arrived, _)) =>
|
||||
clients.find(_.name == name) match {
|
||||
case None => stay()
|
||||
case Some(client) =>
|
||||
handleBarrier(d.copy(clients = clients - client, arrived = arrived.filterNot(_ == client.fsm)))
|
||||
}
|
||||
case Event(FailBarrier(name), d @ Data(_, barrier, _, _)) =>
|
||||
if (name != barrier) throw WrongBarrier(name, sender(), d)
|
||||
throw FailedBarrier(d)
|
||||
case Event(StateTimeout, d) =>
|
||||
throw BarrierTimeout(d)
|
||||
}
|
||||
|
||||
initialize()
|
||||
|
||||
def handleBarrier(data: Data): State = {
|
||||
log.debug("handleBarrier({})", data)
|
||||
if (data.arrived.isEmpty) {
|
||||
goto(Idle).using(data.copy(barrier = ""))
|
||||
} else if ((data.clients.map(_.fsm) -- data.arrived).isEmpty) {
|
||||
data.arrived.foreach(_ ! ToClient(BarrierResult(data.barrier, true)))
|
||||
goto(Idle).using(data.copy(barrier = "", arrived = Nil))
|
||||
} else {
|
||||
stay().using(data)
|
||||
}
|
||||
}
|
||||
|
||||
def getDeadline(timeout: Option[FiniteDuration]): Deadline = {
|
||||
Deadline.now + timeout.getOrElse(TestConductor().Settings.BarrierTimeout.duration)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,179 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.remote.testconductor
|
||||
|
||||
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 org.apache.pekko
|
||||
import pekko.actor.Address
|
||||
import pekko.remote.testconductor.{ TestConductorProtocol => TCP }
|
||||
import pekko.remote.testconductor.TestConductorProtocol.BarrierOp
|
||||
import pekko.remote.transport.ThrottlerTransportAdapter.Direction
|
||||
|
||||
final case class RoleName(name: String)
|
||||
|
||||
private[pekko] final case class ToClient(msg: ClientOp with NetworkOp)
|
||||
private[pekko] final case class ToServer(msg: ServerOp with NetworkOp)
|
||||
|
||||
private[pekko] sealed trait ClientOp // messages sent to from Conductor to Player
|
||||
private[pekko] sealed trait ServerOp // messages sent to from Player to Conductor
|
||||
private[pekko] sealed trait CommandOp // messages sent from TestConductorExt to Conductor
|
||||
private[pekko] sealed trait NetworkOp // messages sent over the wire
|
||||
private[pekko] sealed trait UnconfirmedClientOp extends ClientOp // unconfirmed messages going to the Player
|
||||
private[pekko] sealed trait ConfirmedClientOp extends ClientOp
|
||||
|
||||
/**
|
||||
* First message of connection sets names straight.
|
||||
*/
|
||||
private[pekko] final case class Hello(name: String, addr: Address) extends NetworkOp
|
||||
|
||||
private[pekko] final case class EnterBarrier(name: String, timeout: Option[FiniteDuration])
|
||||
extends ServerOp
|
||||
with NetworkOp
|
||||
private[pekko] final case class FailBarrier(name: String) extends ServerOp with NetworkOp
|
||||
private[pekko] final case class BarrierResult(name: String, success: Boolean) extends UnconfirmedClientOp with NetworkOp
|
||||
|
||||
private[pekko] final case class Throttle(node: RoleName, target: RoleName, direction: Direction, rateMBit: Float)
|
||||
extends CommandOp
|
||||
private[pekko] final case class ThrottleMsg(target: Address, direction: Direction, rateMBit: Float)
|
||||
extends ConfirmedClientOp
|
||||
with NetworkOp
|
||||
|
||||
private[pekko] final case class Disconnect(node: RoleName, target: RoleName, abort: Boolean) extends CommandOp
|
||||
private[pekko] final case class DisconnectMsg(target: Address, abort: Boolean) extends ConfirmedClientOp with NetworkOp
|
||||
|
||||
private[pekko] final case class Terminate(node: RoleName, shutdownOrExit: Either[Boolean, Int]) extends CommandOp
|
||||
private[pekko] final case class TerminateMsg(shutdownOrExit: Either[Boolean, Int])
|
||||
extends ConfirmedClientOp
|
||||
with NetworkOp
|
||||
|
||||
private[pekko] final case class GetAddress(node: RoleName) extends ServerOp with NetworkOp
|
||||
private[pekko] final case class AddressReply(node: RoleName, addr: Address) extends UnconfirmedClientOp with NetworkOp
|
||||
|
||||
private[pekko] abstract class Done extends ServerOp with UnconfirmedClientOp with NetworkOp
|
||||
private[pekko] case object Done extends Done {
|
||||
def getInstance: Done = this
|
||||
}
|
||||
|
||||
private[pekko] final case class Remove(node: RoleName) extends CommandOp
|
||||
|
||||
private[pekko] class MsgEncoder extends OneToOneEncoder {
|
||||
|
||||
implicit def address2proto(addr: Address): TCP.Address =
|
||||
TCP.Address.newBuilder
|
||||
.setProtocol(addr.protocol)
|
||||
.setSystem(addr.system)
|
||||
.setHost(addr.host.get)
|
||||
.setPort(addr.port.get)
|
||||
.build
|
||||
|
||||
implicit def direction2proto(dir: Direction): TCP.Direction = dir match {
|
||||
case Direction.Send => TCP.Direction.Send
|
||||
case Direction.Receive => TCP.Direction.Receive
|
||||
case Direction.Both => TCP.Direction.Both
|
||||
}
|
||||
|
||||
def encode(ctx: ChannelHandlerContext, ch: Channel, msg: AnyRef): AnyRef = msg match {
|
||||
case x: NetworkOp =>
|
||||
val w = TCP.Wrapper.newBuilder
|
||||
x match {
|
||||
case Hello(name, address) =>
|
||||
w.setHello(TCP.Hello.newBuilder.setName(name).setAddress(address))
|
||||
case EnterBarrier(name, timeout) =>
|
||||
val barrier = TCP.EnterBarrier.newBuilder.setName(name)
|
||||
timeout.foreach(t => barrier.setTimeout(t.toNanos))
|
||||
barrier.setOp(BarrierOp.Enter)
|
||||
w.setBarrier(barrier)
|
||||
case BarrierResult(name, success) =>
|
||||
val res = if (success) BarrierOp.Succeeded else BarrierOp.Failed
|
||||
w.setBarrier(TCP.EnterBarrier.newBuilder.setName(name).setOp(res))
|
||||
case FailBarrier(name) =>
|
||||
w.setBarrier(TCP.EnterBarrier.newBuilder.setName(name).setOp(BarrierOp.Fail))
|
||||
case ThrottleMsg(target, dir, rate) =>
|
||||
w.setFailure(
|
||||
TCP.InjectFailure.newBuilder
|
||||
.setAddress(target)
|
||||
.setFailure(TCP.FailType.Throttle)
|
||||
.setDirection(dir)
|
||||
.setRateMBit(rate))
|
||||
case DisconnectMsg(target, abort) =>
|
||||
w.setFailure(
|
||||
TCP.InjectFailure.newBuilder
|
||||
.setAddress(target)
|
||||
.setFailure(if (abort) TCP.FailType.Abort else TCP.FailType.Disconnect))
|
||||
case TerminateMsg(Right(exitValue)) =>
|
||||
w.setFailure(TCP.InjectFailure.newBuilder.setFailure(TCP.FailType.Exit).setExitValue(exitValue))
|
||||
case TerminateMsg(Left(false)) =>
|
||||
w.setFailure(TCP.InjectFailure.newBuilder.setFailure(TCP.FailType.Shutdown))
|
||||
case TerminateMsg(Left(true)) =>
|
||||
w.setFailure(TCP.InjectFailure.newBuilder.setFailure(TCP.FailType.ShutdownAbrupt))
|
||||
case GetAddress(node) =>
|
||||
w.setAddr(TCP.AddressRequest.newBuilder.setNode(node.name))
|
||||
case AddressReply(node, address) =>
|
||||
w.setAddr(TCP.AddressRequest.newBuilder.setNode(node.name).setAddr(address))
|
||||
case _: Done =>
|
||||
w.setDone("")
|
||||
}
|
||||
w.build
|
||||
case _ => throw new IllegalArgumentException("wrong message " + msg)
|
||||
}
|
||||
}
|
||||
|
||||
private[pekko] class MsgDecoder extends OneToOneDecoder {
|
||||
|
||||
implicit def address2scala(addr: TCP.Address): Address =
|
||||
Address(addr.getProtocol, addr.getSystem, addr.getHost, addr.getPort)
|
||||
|
||||
implicit def direction2scala(dir: TCP.Direction): Direction = dir match {
|
||||
case TCP.Direction.Send => Direction.Send
|
||||
case TCP.Direction.Receive => Direction.Receive
|
||||
case TCP.Direction.Both => Direction.Both
|
||||
}
|
||||
|
||||
def decode(ctx: ChannelHandlerContext, ch: Channel, msg: AnyRef): AnyRef = msg match {
|
||||
case w: TCP.Wrapper if w.getAllFields.size == 1 =>
|
||||
if (w.hasHello) {
|
||||
val h = w.getHello
|
||||
Hello(h.getName, h.getAddress)
|
||||
} else if (w.hasBarrier) {
|
||||
val barrier = w.getBarrier
|
||||
barrier.getOp match {
|
||||
case BarrierOp.Succeeded => BarrierResult(barrier.getName, true)
|
||||
case BarrierOp.Failed => BarrierResult(barrier.getName, false)
|
||||
case BarrierOp.Fail => FailBarrier(barrier.getName)
|
||||
case BarrierOp.Enter =>
|
||||
EnterBarrier(
|
||||
barrier.getName,
|
||||
if (barrier.hasTimeout) Option(Duration.fromNanos(barrier.getTimeout)) else None)
|
||||
}
|
||||
} else if (w.hasFailure) {
|
||||
val f = w.getFailure
|
||||
import TCP.{ FailType => FT }
|
||||
f.getFailure match {
|
||||
case FT.Throttle => ThrottleMsg(f.getAddress, f.getDirection, f.getRateMBit)
|
||||
case FT.Abort => DisconnectMsg(f.getAddress, true)
|
||||
case FT.Disconnect => DisconnectMsg(f.getAddress, false)
|
||||
case FT.Exit => TerminateMsg(Right(f.getExitValue))
|
||||
case FT.Shutdown => TerminateMsg(Left(false))
|
||||
case FT.ShutdownAbrupt => TerminateMsg(Left(true))
|
||||
}
|
||||
} else if (w.hasAddr) {
|
||||
val a = w.getAddr
|
||||
if (a.hasAddr) AddressReply(RoleName(a.getNode), a.getAddr)
|
||||
else GetAddress(RoleName(a.getNode))
|
||||
} else if (w.hasDone) {
|
||||
Done
|
||||
} else {
|
||||
throw new IllegalArgumentException("unknown message " + msg)
|
||||
}
|
||||
case _ => throw new IllegalArgumentException("wrong message " + msg)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,91 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.remote.testconductor
|
||||
|
||||
import com.typesafe.config.Config
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.actor.{ ActorContext, ActorSystem, ExtendedActorSystem, Extension, ExtensionId, ExtensionIdProvider }
|
||||
import pekko.actor.ClassicActorSystemProvider
|
||||
import pekko.dispatch.ThreadPoolConfig
|
||||
import pekko.remote.RemoteActorRefProvider
|
||||
import pekko.util.Timeout
|
||||
|
||||
/**
|
||||
* Access to the [[pekko.remote.testconductor.TestConductorExt]] extension:
|
||||
*
|
||||
* {{{
|
||||
* val tc = TestConductor(system)
|
||||
* tc.startController(numPlayers)
|
||||
* // OR
|
||||
* tc.startClient(conductorPort)
|
||||
* }}}
|
||||
*/
|
||||
object TestConductor extends ExtensionId[TestConductorExt] with ExtensionIdProvider {
|
||||
|
||||
override def lookup = TestConductor
|
||||
|
||||
override def createExtension(system: ExtendedActorSystem): TestConductorExt = new TestConductorExt(system)
|
||||
|
||||
/**
|
||||
* Java API: retrieve the TestConductor extension for the given system.
|
||||
*/
|
||||
override def get(system: ActorSystem): TestConductorExt = super.get(system)
|
||||
override def get(system: ClassicActorSystemProvider): TestConductorExt = super.get(system)
|
||||
|
||||
def apply()(implicit ctx: ActorContext): TestConductorExt = apply(ctx.system)
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* This binds together the [[pekko.remote.testconductor.Conductor]] and
|
||||
* [[pekko.remote.testconductor.Player]] roles inside an Akka
|
||||
* [[pekko.actor.Extension]]. Please follow the aforementioned links for
|
||||
* more information.
|
||||
*
|
||||
* ====Note====
|
||||
* This extension requires the `pekko.actor.provider`
|
||||
* to be a [[pekko.remote.RemoteActorRefProvider]].
|
||||
*
|
||||
* To use ``blackhole``, ``passThrough``, and ``throttle`` you must activate the
|
||||
* failure injector and throttler transport adapters by specifying `testTransport(on = true)`
|
||||
* in your MultiNodeConfig.
|
||||
*/
|
||||
class TestConductorExt(val system: ExtendedActorSystem) extends Extension with Conductor with Player {
|
||||
|
||||
object Settings {
|
||||
val config = system.settings.config.getConfig("pekko.testconductor")
|
||||
import org.apache.pekko.util.Helpers.ConfigOps
|
||||
|
||||
val ConnectTimeout = config.getMillisDuration("connect-timeout")
|
||||
val ClientReconnects = config.getInt("client-reconnects")
|
||||
val ReconnectBackoff = config.getMillisDuration("reconnect-backoff")
|
||||
|
||||
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 =
|
||||
ThreadPoolConfig.scaledPoolSize(
|
||||
config.getInt("pool-size-min"),
|
||||
config.getDouble("pool-size-factor"),
|
||||
config.getInt("pool-size-max"))
|
||||
|
||||
val ServerSocketWorkerPoolSize = computeWPS(config.getConfig("netty.server-socket-worker-pool"))
|
||||
|
||||
val ClientSocketWorkerPoolSize = computeWPS(config.getConfig("netty.client-socket-worker-pool"))
|
||||
}
|
||||
|
||||
/**
|
||||
* Remote transport used by the actor ref provider.
|
||||
*/
|
||||
val transport = system.provider.asInstanceOf[RemoteActorRefProvider].transport
|
||||
|
||||
/**
|
||||
* Transport address of this Netty-like remote transport.
|
||||
*/
|
||||
val address = transport.defaultAddress
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,391 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.remote.testconductor
|
||||
|
||||
import java.net.{ ConnectException, InetSocketAddress }
|
||||
import java.util.concurrent.TimeoutException
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.{ Await, ExecutionContext, Future }
|
||||
import scala.concurrent.duration._
|
||||
import scala.reflect.classTag
|
||||
import scala.util.control.NoStackTrace
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
import org.jboss.netty.channel.{
|
||||
Channel,
|
||||
ChannelHandlerContext,
|
||||
ChannelStateEvent,
|
||||
ExceptionEvent,
|
||||
MessageEvent,
|
||||
SimpleChannelUpstreamHandler,
|
||||
WriteCompletionEvent
|
||||
}
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.actor._
|
||||
import pekko.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
|
||||
import pekko.event.{ Logging, LoggingAdapter }
|
||||
import pekko.pattern.{ ask, AskTimeoutException }
|
||||
import pekko.remote.testconductor.RemoteConnection.getAddrString
|
||||
import pekko.remote.transport.ThrottlerTransportAdapter.{ Blackhole, SetThrottle, TokenBucket, Unthrottled }
|
||||
import pekko.util.Timeout
|
||||
import pekko.util.ccompat._
|
||||
|
||||
@ccompatUsedUntil213
|
||||
object Player {
|
||||
|
||||
final class Waiter extends Actor with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
|
||||
|
||||
import ClientFSM._
|
||||
import FSM._
|
||||
|
||||
var waiting: ActorRef = _
|
||||
|
||||
def receive = {
|
||||
case fsm: ActorRef =>
|
||||
waiting = sender(); fsm ! SubscribeTransitionCallBack(self)
|
||||
case Transition(_, f: ClientFSM.State, t: ClientFSM.State) if f == Connecting && t == AwaitDone => // step 1, not there yet // // SI-5900 workaround
|
||||
case Transition(_, f: ClientFSM.State, t: ClientFSM.State)
|
||||
if f == AwaitDone && t == Connected => // SI-5900 workaround
|
||||
waiting ! Done; context.stop(self)
|
||||
case t: Transition[_] =>
|
||||
waiting ! Status.Failure(new RuntimeException("unexpected transition: " + t)); context.stop(self)
|
||||
case CurrentState(_, s: ClientFSM.State) if s == Connected => // SI-5900 workaround
|
||||
waiting ! Done; context.stop(self)
|
||||
case _: CurrentState[_] =>
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
def waiterProps = Props[Waiter]()
|
||||
}
|
||||
|
||||
/**
|
||||
* The Player is the client component of the
|
||||
* [[pekko.remote.testconductor.TestConductorExt]] extension. It registers with
|
||||
* the [[pekko.remote.testconductor.Conductor]]’s [[pekko.remote.testconductor.Controller]]
|
||||
* in order to participate in barriers and enable network failure injection.
|
||||
*/
|
||||
trait Player { this: TestConductorExt =>
|
||||
|
||||
private var _client: ActorRef = _
|
||||
private def client = _client match {
|
||||
case null =>
|
||||
throw new IllegalStateException("TestConductor client not yet started")
|
||||
case _ if system.whenTerminated.isCompleted =>
|
||||
throw new IllegalStateException(
|
||||
"TestConductor unavailable because system is terminated; you need to startNewSystem() before this point")
|
||||
case x => x
|
||||
}
|
||||
|
||||
/**
|
||||
* Connect to the conductor on the given port (the host is taken from setting
|
||||
* `pekko.testconductor.host`). The connection is made asynchronously, but you
|
||||
* should await completion of the returned Future because that implies that
|
||||
* all expected participants of this test have successfully connected (i.e.
|
||||
* this is a first barrier in itself). The number of expected participants is
|
||||
* set in [[pekko.remote.testconductor.Conductor]]`.startController()`.
|
||||
*/
|
||||
def startClient(name: RoleName, controllerAddr: InetSocketAddress): Future[Done] = {
|
||||
import Settings.BarrierTimeout
|
||||
|
||||
if (_client ne null) throw new IllegalStateException("TestConductorClient already started")
|
||||
_client = system.systemActorOf(Props(classOf[ClientFSM], name, controllerAddr), "TestConductorClient")
|
||||
val a = system.systemActorOf(Player.waiterProps, "TestConductorWaiter")
|
||||
(a ? client).mapTo(classTag[Done])
|
||||
}
|
||||
|
||||
/**
|
||||
* Enter the named barriers, one after the other, in the order given. Will
|
||||
* throw an exception in case of timeouts or other errors.
|
||||
*/
|
||||
def enter(name: String*): Unit = enter(Settings.BarrierTimeout, name.to(immutable.Seq))
|
||||
|
||||
/**
|
||||
* Enter the named barriers, one after the other, in the order given. Will
|
||||
* throw an exception in case of timeouts or other errors.
|
||||
*/
|
||||
def enter(timeout: Timeout, name: immutable.Seq[String]): Unit = {
|
||||
system.log.debug("entering barriers " + name.mkString("(", ", ", ")"))
|
||||
val stop = Deadline.now + timeout.duration
|
||||
name.foreach { b =>
|
||||
val barrierTimeout = stop.timeLeft
|
||||
if (barrierTimeout < Duration.Zero) {
|
||||
client ! ToServer(FailBarrier(b))
|
||||
throw new TimeoutException("Server timed out while waiting for barrier " + b)
|
||||
}
|
||||
try {
|
||||
implicit val timeout = Timeout(barrierTimeout + Settings.QueryTimeout.duration)
|
||||
Await.result(client ? ToServer(EnterBarrier(b, Option(barrierTimeout))), Duration.Inf)
|
||||
} catch {
|
||||
case _: AskTimeoutException =>
|
||||
client ! ToServer(FailBarrier(b))
|
||||
// Why don't TimeoutException have a constructor that takes a cause?
|
||||
throw new TimeoutException("Client timed out while waiting for barrier " + b);
|
||||
}
|
||||
system.log.debug("passed barrier {}", b)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Query remote transport address of named node.
|
||||
*/
|
||||
def getAddressFor(name: RoleName): Future[Address] = {
|
||||
import Settings.QueryTimeout
|
||||
(client ? ToServer(GetAddress(name))).mapTo(classTag[Address])
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] object ClientFSM {
|
||||
sealed trait State
|
||||
case object Connecting extends State
|
||||
case object AwaitDone extends State
|
||||
case object Connected extends State
|
||||
case object Failed extends State
|
||||
|
||||
final case class Data(channel: Option[Channel], runningOp: Option[(String, ActorRef)])
|
||||
|
||||
final case class Connected(channel: Channel) extends NoSerializationVerificationNeeded
|
||||
final case class ConnectionFailure(msg: String) extends RuntimeException(msg) with NoStackTrace
|
||||
case object Disconnected
|
||||
}
|
||||
|
||||
/**
|
||||
* This is the controlling entity on the [[pekko.remote.testconductor.Player]]
|
||||
* side: in a first step it registers itself with a symbolic name and its remote
|
||||
* address at the [[pekko.remote.testconductor.Controller]], then waits for the
|
||||
* `Done` message which signals that all other expected test participants have
|
||||
* done the same. After that, it will pass barrier requests to and from the
|
||||
* coordinator and react to the [[pekko.remote.testconductor.Conductor]]’s
|
||||
* requests for failure injection.
|
||||
*
|
||||
* Note that you can't perform requests concurrently, e.g. enter barrier
|
||||
* from one thread and ask for node address from another thread.
|
||||
*
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress)
|
||||
extends Actor
|
||||
with LoggingFSM[ClientFSM.State, ClientFSM.Data]
|
||||
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
|
||||
import ClientFSM._
|
||||
|
||||
val settings = TestConductor().Settings
|
||||
|
||||
val handler = new PlayerHandler(
|
||||
controllerAddr,
|
||||
settings.ClientReconnects,
|
||||
settings.ReconnectBackoff,
|
||||
settings.ClientSocketWorkerPoolSize,
|
||||
self,
|
||||
Logging(context.system, classOf[PlayerHandler]),
|
||||
context.system.scheduler)(context.dispatcher)
|
||||
|
||||
startWith(Connecting, Data(None, None))
|
||||
|
||||
when(Connecting, stateTimeout = settings.ConnectTimeout) {
|
||||
case Event(_: ClientOp, _) =>
|
||||
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))
|
||||
case Event(e: ConnectionFailure, _) =>
|
||||
log.error(e, "ConnectionFailure")
|
||||
goto(Failed)
|
||||
case Event(StateTimeout, _) =>
|
||||
log.error("Failed to connect to test conductor within {} ms.", settings.ConnectTimeout.toMillis)
|
||||
goto(Failed)
|
||||
}
|
||||
|
||||
when(AwaitDone, stateTimeout = settings.BarrierTimeout.duration) {
|
||||
case Event(Done, _) =>
|
||||
log.debug("received Done: starting test")
|
||||
goto(Connected)
|
||||
case Event(msg: NetworkOp, _) =>
|
||||
log.error("received {} instead of Done", msg)
|
||||
goto(Failed)
|
||||
case Event(_: ServerOp, _) =>
|
||||
stay().replying(Status.Failure(new IllegalStateException("not connected yet")))
|
||||
case Event(StateTimeout, _) =>
|
||||
log.error("connect timeout to TestConductor")
|
||||
goto(Failed)
|
||||
}
|
||||
|
||||
when(Connected) {
|
||||
case Event(Disconnected, _) =>
|
||||
log.info("disconnected from TestConductor")
|
||||
throw new ConnectionFailure("disconnect")
|
||||
case Event(ToServer(_: Done), Data(Some(channel), _)) =>
|
||||
channel.write(Done)
|
||||
stay()
|
||||
case Event(ToServer(msg), d @ Data(Some(channel), None)) =>
|
||||
channel.write(msg)
|
||||
val token = msg match {
|
||||
case EnterBarrier(barrier, _) => Some(barrier -> sender())
|
||||
case GetAddress(node) => Some(node.name -> sender())
|
||||
case _ => None
|
||||
}
|
||||
stay().using(d.copy(runningOp = token))
|
||||
case Event(ToServer(op), Data(_, Some((token, _)))) =>
|
||||
log.error("cannot write {} while waiting for {}", op, token)
|
||||
stay()
|
||||
case Event(op: ClientOp, d @ Data(Some(channel @ _), runningOp)) =>
|
||||
op match {
|
||||
case BarrierResult(b, success) =>
|
||||
runningOp match {
|
||||
case Some((barrier, requester)) =>
|
||||
val response =
|
||||
if (b != barrier)
|
||||
Status.Failure(new RuntimeException("wrong barrier " + b + " received while waiting for " + barrier))
|
||||
else if (!success) Status.Failure(new RuntimeException("barrier failed: " + b))
|
||||
else b
|
||||
requester ! response
|
||||
case None =>
|
||||
log.warning("did not expect {}", op)
|
||||
}
|
||||
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))
|
||||
case t: ThrottleMsg =>
|
||||
import context.dispatcher // FIXME is this the right EC for the future below?
|
||||
val mode =
|
||||
if (t.rateMBit < 0.0f) Unthrottled
|
||||
else if (t.rateMBit == 0.0f) Blackhole
|
||||
// Conversion needed as the TokenBucket measures in octets: 125000 Octets/s = 1Mbit/s
|
||||
// FIXME: Initial capacity should be carefully chosen
|
||||
else
|
||||
TokenBucket(
|
||||
capacity = 1000,
|
||||
tokensPerSecond = t.rateMBit * 125000.0,
|
||||
nanoTimeOfLastSend = 0,
|
||||
availableTokens = 0)
|
||||
|
||||
val cmdFuture = TestConductor().transport.managementCommand(SetThrottle(t.target, t.direction, mode))
|
||||
|
||||
cmdFuture.foreach {
|
||||
case true => self ! ToServer(Done)
|
||||
case _ =>
|
||||
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()
|
||||
case _: DisconnectMsg =>
|
||||
// FIXME: Currently ignoring, needs support from Remoting
|
||||
stay()
|
||||
case TerminateMsg(Left(false)) =>
|
||||
context.system.terminate()
|
||||
stop()
|
||||
case TerminateMsg(Left(true)) =>
|
||||
context.system.asInstanceOf[ActorSystemImpl].abort()
|
||||
stop()
|
||||
case TerminateMsg(Right(exitValue)) =>
|
||||
System.exit(exitValue)
|
||||
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")))
|
||||
case Event(msg: NetworkOp, _) =>
|
||||
log.warning("ignoring network message {} while Failed", msg)
|
||||
stay()
|
||||
}
|
||||
|
||||
onTermination {
|
||||
case StopEvent(_, _, Data(Some(channel), _)) =>
|
||||
try {
|
||||
channel.close()
|
||||
} catch {
|
||||
case NonFatal(ex) =>
|
||||
// silence this one to not make tests look like they failed, it's not really critical
|
||||
log.debug(s"Failed closing channel with ${ex.getClass.getName} ${ex.getMessage}")
|
||||
}
|
||||
}
|
||||
|
||||
initialize()
|
||||
}
|
||||
|
||||
/**
|
||||
* This handler only forwards messages received from the conductor to the [[pekko.remote.testconductor.ClientFSM]].
|
||||
*
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] class PlayerHandler(
|
||||
server: InetSocketAddress,
|
||||
private var reconnects: Int,
|
||||
backoff: FiniteDuration,
|
||||
poolSize: Int,
|
||||
fsm: ActorRef,
|
||||
log: LoggingAdapter,
|
||||
scheduler: Scheduler)(implicit executor: ExecutionContext)
|
||||
extends SimpleChannelUpstreamHandler {
|
||||
|
||||
import ClientFSM._
|
||||
|
||||
reconnect()
|
||||
|
||||
var nextAttempt: Deadline = _
|
||||
|
||||
override def channelOpen(ctx: ChannelHandlerContext, event: ChannelStateEvent) =
|
||||
log.debug("channel {} open", event.getChannel)
|
||||
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) =
|
||||
log.debug("channel {} closed", event.getChannel)
|
||||
override def channelBound(ctx: ChannelHandlerContext, event: ChannelStateEvent) =
|
||||
log.debug("channel {} bound", event.getChannel)
|
||||
override def channelUnbound(ctx: ChannelHandlerContext, event: ChannelStateEvent) =
|
||||
log.debug("channel {} unbound", event.getChannel)
|
||||
override def writeComplete(ctx: ChannelHandlerContext, event: WriteCompletionEvent) =
|
||||
log.debug("channel {} written {}", event.getChannel, event.getWrittenAmount)
|
||||
|
||||
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
|
||||
log.debug("channel {} exception {}", event.getChannel, event.getCause)
|
||||
event.getCause match {
|
||||
case _: ConnectException if reconnects > 0 =>
|
||||
reconnects -= 1
|
||||
scheduler.scheduleOnce(nextAttempt.timeLeft)(reconnect())
|
||||
case e => fsm ! ConnectionFailure(e.getMessage)
|
||||
}
|
||||
}
|
||||
|
||||
private def reconnect(): Unit = {
|
||||
nextAttempt = Deadline.now + backoff
|
||||
RemoteConnection(Client, server, poolSize, this)
|
||||
}
|
||||
|
||||
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val ch = event.getChannel
|
||||
log.debug("connected to {}", getAddrString(ch))
|
||||
fsm ! Connected(ch)
|
||||
}
|
||||
|
||||
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val channel = event.getChannel
|
||||
log.debug("disconnected from {}", getAddrString(channel))
|
||||
fsm ! PoisonPill
|
||||
executor.execute(new Runnable { def run = RemoteConnection.shutdown(channel) }) // Must be shutdown outside of the Netty IO pool
|
||||
}
|
||||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = {
|
||||
val channel = event.getChannel
|
||||
log.debug("message from {}: {}", getAddrString(channel), event.getMessage)
|
||||
event.getMessage match {
|
||||
case msg: NetworkOp =>
|
||||
fsm ! msg
|
||||
case msg =>
|
||||
log.info("server {} sent garbage '{}', disconnecting", getAddrString(channel), msg)
|
||||
channel.close()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,128 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.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,
|
||||
ChannelPipelineFactory,
|
||||
ChannelUpstreamHandler,
|
||||
DefaultChannelPipeline
|
||||
}
|
||||
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 org.apache.pekko
|
||||
import pekko.event.Logging
|
||||
import pekko.protobufv3.internal.Message
|
||||
import pekko.util.Helpers
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] class ProtobufEncoder extends OneToOneEncoder {
|
||||
override def encode(ctx: ChannelHandlerContext, ch: Channel, msg: AnyRef): AnyRef =
|
||||
msg match {
|
||||
case m: Message =>
|
||||
val bytes = m.toByteArray()
|
||||
ctx.getChannel.getConfig.getBufferFactory.getBuffer(bytes, 0, bytes.length)
|
||||
case other => other
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] class ProtobufDecoder(prototype: Message) extends OneToOneDecoder {
|
||||
override def decode(ctx: ChannelHandlerContext, ch: Channel, obj: AnyRef): AnyRef =
|
||||
obj match {
|
||||
case buf: ChannelBuffer =>
|
||||
val len = buf.readableBytes()
|
||||
val bytes = new Array[Byte](len)
|
||||
buf.getBytes(buf.readerIndex, bytes, 0, len)
|
||||
prototype.getParserForType.parseFrom(bytes)
|
||||
case other => other
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] class TestConductorPipelineFactory(handler: ChannelUpstreamHandler) extends ChannelPipelineFactory {
|
||||
def getPipeline: ChannelPipeline = {
|
||||
val encap = List(new LengthFieldPrepender(4), new LengthFieldBasedFrameDecoder(10000, 0, 4, 0, 4))
|
||||
val proto = List(new ProtobufEncoder, new ProtobufDecoder(TestConductorProtocol.Wrapper.getDefaultInstance))
|
||||
val msg = List(new MsgEncoder, new MsgDecoder)
|
||||
(encap ::: proto ::: msg ::: handler :: Nil).foldLeft(new DefaultChannelPipeline) { (pipe, handler) =>
|
||||
pipe.addLast(Logging.simpleName(handler.getClass), handler); pipe
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] sealed trait Role
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] case object Client extends Role
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] case object Server extends Role
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[pekko] object RemoteConnection {
|
||||
def apply(role: Role, sockaddr: InetSocketAddress, poolSize: Int, handler: ChannelUpstreamHandler): Channel = {
|
||||
role match {
|
||||
case Client =>
|
||||
val socketfactory =
|
||||
new NioClientSocketChannelFactory(Executors.newCachedThreadPool, Executors.newCachedThreadPool, poolSize)
|
||||
val bootstrap = new ClientBootstrap(socketfactory)
|
||||
bootstrap.setPipelineFactory(new TestConductorPipelineFactory(handler))
|
||||
bootstrap.setOption("tcpNoDelay", true)
|
||||
bootstrap.connect(sockaddr).getChannel
|
||||
case Server =>
|
||||
val socketfactory =
|
||||
new NioServerSocketChannelFactory(Executors.newCachedThreadPool, Executors.newCachedThreadPool, poolSize)
|
||||
val bootstrap = new ServerBootstrap(socketfactory)
|
||||
bootstrap.setPipelineFactory(new TestConductorPipelineFactory(handler))
|
||||
bootstrap.setOption("reuseAddress", !Helpers.isWindows)
|
||||
bootstrap.setOption("child.tcpNoDelay", true)
|
||||
bootstrap.bind(sockaddr)
|
||||
}
|
||||
}
|
||||
|
||||
def getAddrString(channel: Channel) = channel.getRemoteAddress match {
|
||||
case i: InetSocketAddress => i.toString
|
||||
case _ => "[unknown]"
|
||||
}
|
||||
|
||||
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
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,579 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.remote.testkit
|
||||
|
||||
import java.net.{ InetAddress, InetSocketAddress }
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.{ Await, Awaitable }
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.control.NonFatal
|
||||
import com.typesafe.config.{ Config, ConfigFactory, ConfigObject }
|
||||
|
||||
import language.implicitConversions
|
||||
import org.jboss.netty.channel.ChannelException
|
||||
import org.apache.pekko
|
||||
import pekko.actor._
|
||||
import pekko.actor.RootActorPath
|
||||
import pekko.event.{ Logging, LoggingAdapter }
|
||||
import pekko.remote.RemoteTransportException
|
||||
import pekko.remote.testconductor.{ TestConductor, TestConductorExt }
|
||||
import pekko.remote.testconductor.RoleName
|
||||
import pekko.testkit._
|
||||
import pekko.testkit.TestEvent._
|
||||
import pekko.testkit.TestKit
|
||||
import pekko.util.Timeout
|
||||
import pekko.util.ccompat._
|
||||
|
||||
/**
|
||||
* Configure the role names and participants of the test, including configuration settings.
|
||||
*/
|
||||
@ccompatUsedUntil213
|
||||
abstract class MultiNodeConfig {
|
||||
|
||||
private var _commonConf: Option[Config] = None
|
||||
private var _nodeConf = Map[RoleName, Config]()
|
||||
private var _roles = Vector[RoleName]()
|
||||
private var _deployments = Map[RoleName, immutable.Seq[String]]()
|
||||
private var _allDeploy = Vector[String]()
|
||||
private var _testTransport = false
|
||||
|
||||
/**
|
||||
* Register a common base config for all test participants, if so desired.
|
||||
*/
|
||||
def commonConfig(config: Config): Unit = _commonConf = Some(config)
|
||||
|
||||
/**
|
||||
* Register a config override for a specific participant.
|
||||
*/
|
||||
def nodeConfig(roles: RoleName*)(configs: Config*): Unit = {
|
||||
val c = configs.reduceLeft(_.withFallback(_))
|
||||
_nodeConf ++= roles.map { _ -> c }
|
||||
}
|
||||
|
||||
/**
|
||||
* Include for verbose debug logging
|
||||
* @param on when `true` debug Config is returned, otherwise config with info logging
|
||||
*/
|
||||
def debugConfig(on: Boolean): Config =
|
||||
if (on)
|
||||
ConfigFactory.parseString("""
|
||||
pekko.loglevel = DEBUG
|
||||
pekko.remote {
|
||||
log-received-messages = on
|
||||
log-sent-messages = on
|
||||
}
|
||||
pekko.remote.artery {
|
||||
log-received-messages = on
|
||||
log-sent-messages = on
|
||||
}
|
||||
pekko.actor.debug {
|
||||
receive = on
|
||||
fsm = on
|
||||
}
|
||||
pekko.remote.log-remote-lifecycle-events = on
|
||||
""")
|
||||
else
|
||||
ConfigFactory.empty
|
||||
|
||||
/**
|
||||
* Construct a RoleName and return it, to be used as an identifier in the
|
||||
* test. Registration of a role name creates a role which then needs to be
|
||||
* filled.
|
||||
*/
|
||||
def role(name: String): RoleName = {
|
||||
if (_roles.exists(_.name == name)) throw new IllegalArgumentException("non-unique role name " + name)
|
||||
val r = RoleName(name)
|
||||
_roles :+= r
|
||||
r
|
||||
}
|
||||
|
||||
def deployOn(role: RoleName, deployment: String): Unit =
|
||||
_deployments += role -> ((_deployments.get(role).getOrElse(Vector())) :+ deployment)
|
||||
|
||||
def deployOnAll(deployment: String): Unit = _allDeploy :+= deployment
|
||||
|
||||
/**
|
||||
* To be able to use `blackhole`, `passThrough`, and `throttle` you must
|
||||
* activate the failure injector and throttler transport adapters by
|
||||
* specifying `testTransport(on = true)` in your MultiNodeConfig.
|
||||
*/
|
||||
def testTransport(on: Boolean): Unit = _testTransport = on
|
||||
|
||||
private[testkit] lazy val myself: RoleName = {
|
||||
require(_roles.size > MultiNodeSpec.selfIndex, "not enough roles declared for this test")
|
||||
_roles(MultiNodeSpec.selfIndex)
|
||||
}
|
||||
|
||||
private[pekko] def config: Config = {
|
||||
val transportConfig =
|
||||
if (_testTransport) ConfigFactory.parseString("""
|
||||
pekko.remote.classic.netty.tcp.applied-adapters = [trttl, gremlin]
|
||||
pekko.remote.artery.advanced.test-mode = on
|
||||
""")
|
||||
else ConfigFactory.empty
|
||||
|
||||
val configs = _nodeConf
|
||||
.get(myself)
|
||||
.toList ::: _commonConf.toList ::: transportConfig :: MultiNodeSpec.nodeConfig :: MultiNodeSpec.baseConfig :: Nil
|
||||
configs.reduceLeft(_.withFallback(_))
|
||||
}
|
||||
|
||||
private[testkit] def deployments(node: RoleName): immutable.Seq[String] =
|
||||
(_deployments.get(node).getOrElse(Nil)) ++ _allDeploy
|
||||
|
||||
private[testkit] def roles: immutable.Seq[RoleName] = _roles
|
||||
|
||||
}
|
||||
|
||||
object MultiNodeSpec {
|
||||
|
||||
/**
|
||||
* Number of nodes node taking part in this test.
|
||||
*
|
||||
* {{{
|
||||
* -Dmultinode.max-nodes=4
|
||||
* }}}
|
||||
*/
|
||||
val maxNodes: Int = Option(Integer.getInteger("multinode.max-nodes"))
|
||||
.getOrElse(throw new IllegalStateException("need system property multinode.max-nodes to be set"))
|
||||
|
||||
require(maxNodes > 0, "multinode.max-nodes must be greater than 0")
|
||||
|
||||
/**
|
||||
* Name (or IP address; must be resolvable using InetAddress.getByName)
|
||||
* of the host this node is running on.
|
||||
*
|
||||
* {{{
|
||||
* -Dmultinode.host=host.example.com
|
||||
* }}}
|
||||
*
|
||||
* InetAddress.getLocalHost.getHostAddress is used if empty or "localhost"
|
||||
* is defined as system property "multinode.host".
|
||||
*/
|
||||
val selfName: String = Option(System.getProperty("multinode.host")) match {
|
||||
case None => throw new IllegalStateException("need system property multinode.host to be set")
|
||||
case Some("") => InetAddress.getLocalHost.getHostAddress
|
||||
case Some(host) => host
|
||||
}
|
||||
|
||||
require(selfName != "", "multinode.host must not be empty")
|
||||
|
||||
/**
|
||||
* TCP Port number to be used when running tests on TCP. 0 means a random port.
|
||||
*
|
||||
* {{{
|
||||
* -Dmultinode.port=0
|
||||
* }}}
|
||||
*/
|
||||
val tcpPort: Int = Integer.getInteger("multinode.port", 0)
|
||||
|
||||
require(tcpPort >= 0 && tcpPort < 65535, "multinode.port is out of bounds: " + tcpPort)
|
||||
|
||||
/**
|
||||
* UDP Port number to be used when running tests on UDP. 0 means a random port.
|
||||
*
|
||||
* {{{
|
||||
* -Dmultinode.udp.port=0
|
||||
* }}}
|
||||
*/
|
||||
val udpPort: Option[Int] =
|
||||
Option(System.getProperty("multinode.udp.port")).map { _ =>
|
||||
Integer.getInteger("multinode.udp.port", 0)
|
||||
}
|
||||
|
||||
require(udpPort.getOrElse(1) >= 0 && udpPort.getOrElse(1) < 65535, "multinode.udp.port is out of bounds: " + udpPort)
|
||||
|
||||
/**
|
||||
* Port number of this node.
|
||||
*
|
||||
* This is defined in function of property `multinode.protocol`.
|
||||
* If set to 'udp', udpPort will be used. If unset or any other value, it will default to tcpPort.
|
||||
*/
|
||||
val selfPort: Int =
|
||||
System.getProperty("multinode.protocol") match {
|
||||
case "udp" => udpPort.getOrElse(0)
|
||||
case _ => tcpPort
|
||||
}
|
||||
|
||||
/**
|
||||
* Name (or IP address; must be resolvable using InetAddress.getByName)
|
||||
* of the host that the server node is running on.
|
||||
*
|
||||
* {{{
|
||||
* -Dmultinode.server-host=server.example.com
|
||||
* }}}
|
||||
*/
|
||||
val serverName: String = Option(System.getProperty("multinode.server-host"))
|
||||
.getOrElse(throw new IllegalStateException("need system property multinode.server-host to be set"))
|
||||
|
||||
require(serverName != "", "multinode.server-host must not be empty")
|
||||
|
||||
/**
|
||||
* Port number of the node that's running the server system. Defaults to 4711.
|
||||
*
|
||||
* {{{
|
||||
* -Dmultinode.server-port=4711
|
||||
* }}}
|
||||
*/
|
||||
val serverPort: Int = Integer.getInteger("multinode.server-port", 4711)
|
||||
|
||||
require(serverPort > 0 && serverPort < 65535, "multinode.server-port is out of bounds: " + serverPort)
|
||||
|
||||
/**
|
||||
* Index of this node in the roles sequence. The TestConductor
|
||||
* is started in “controller” mode on selfIndex 0, i.e. there you can inject
|
||||
* failures and shutdown other nodes etc.
|
||||
*
|
||||
* {{{
|
||||
* -Dmultinode.index=0
|
||||
* }}}
|
||||
*/
|
||||
val selfIndex = Option(Integer.getInteger("multinode.index"))
|
||||
.getOrElse(throw new IllegalStateException("need system property multinode.index to be set"))
|
||||
|
||||
require(selfIndex >= 0 && selfIndex < maxNodes, "multinode.index is out of bounds: " + selfIndex)
|
||||
|
||||
private[testkit] val nodeConfig = mapToConfig(
|
||||
Map(
|
||||
"pekko.actor.provider" -> "remote",
|
||||
"pekko.remote.artery.canonical.hostname" -> selfName,
|
||||
"pekko.remote.classic.netty.tcp.hostname" -> selfName,
|
||||
"pekko.remote.classic.netty.tcp.port" -> tcpPort,
|
||||
"pekko.remote.artery.canonical.port" -> selfPort))
|
||||
|
||||
private[testkit] val baseConfig: Config =
|
||||
ConfigFactory.parseString("""
|
||||
pekko {
|
||||
loggers = ["org.apache.pekko.testkit.TestEventListener"]
|
||||
loglevel = "WARNING"
|
||||
stdout-loglevel = "WARNING"
|
||||
coordinated-shutdown.terminate-actor-system = off
|
||||
coordinated-shutdown.run-by-actor-system-terminate = off
|
||||
coordinated-shutdown.run-by-jvm-shutdown-hook = off
|
||||
actor {
|
||||
default-dispatcher {
|
||||
executor = "fork-join-executor"
|
||||
fork-join-executor {
|
||||
parallelism-min = 8
|
||||
parallelism-factor = 2.0
|
||||
parallelism-max = 8
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
""")
|
||||
|
||||
private def mapToConfig(map: Map[String, Any]): Config = {
|
||||
import pekko.util.ccompat.JavaConverters._
|
||||
ConfigFactory.parseMap(map.asJava)
|
||||
}
|
||||
|
||||
// Multi node tests on kubernetes require fixed ports to be mapped and exposed
|
||||
// This method change the port bindings to avoid conflicts
|
||||
// Please note that with the current setup only port 5000 and 5001 (or 6000 and 6001 when using UDP)
|
||||
// are exposed in kubernetes
|
||||
def configureNextPortIfFixed(config: Config): Config = {
|
||||
val arteryPortConfig = getNextPortString("pekko.remote.artery.canonical.port", config)
|
||||
val nettyPortConfig = getNextPortString("pekko.remote.classic.netty.tcp.port", config)
|
||||
ConfigFactory.parseString(s"""{
|
||||
$arteryPortConfig
|
||||
$nettyPortConfig
|
||||
}""").withFallback(config)
|
||||
}
|
||||
|
||||
private def getNextPortString(key: String, config: Config): String = {
|
||||
val port = config.getInt(key)
|
||||
if (port != 0)
|
||||
s"""$key = ${port + 1}"""
|
||||
else ""
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Note: To be able to run tests with everything ignored or excluded by tags
|
||||
* you must not use `testconductor`, or helper methods that use `testconductor`,
|
||||
* from the constructor of your test class. Otherwise the controller node might
|
||||
* be shutdown before other nodes have completed and you will see errors like:
|
||||
* `AskTimeoutException: sending to terminated ref breaks promises`. Using lazy
|
||||
* val is fine.
|
||||
*/
|
||||
abstract class MultiNodeSpec(
|
||||
val myself: RoleName,
|
||||
_system: ActorSystem,
|
||||
_roles: immutable.Seq[RoleName],
|
||||
deployments: RoleName => Seq[String])
|
||||
extends TestKit(_system)
|
||||
with MultiNodeSpecCallbacks {
|
||||
|
||||
import MultiNodeSpec._
|
||||
|
||||
/**
|
||||
* Constructor for using arbitrary logic to create the actor system used in
|
||||
* the multi node spec (the `Config` passed to the creator must be used in
|
||||
* the created actor system for the multi node tests to work)
|
||||
*/
|
||||
def this(config: MultiNodeConfig, actorSystemCreator: Config => ActorSystem) =
|
||||
this(config.myself, actorSystemCreator(ConfigFactory.load(config.config)), config.roles, config.deployments)
|
||||
|
||||
def this(config: MultiNodeConfig) =
|
||||
this(config, {
|
||||
val name = TestKitUtils.testNameFromCallStack(classOf[MultiNodeSpec], "".r)
|
||||
config =>
|
||||
try {
|
||||
ActorSystem(name, config)
|
||||
} catch {
|
||||
// Retry creating the system once as when using port = 0 two systems may try and use the same one.
|
||||
// RTE is for aeron, CE for netty
|
||||
case _: RemoteTransportException => ActorSystem(name, config)
|
||||
case _: ChannelException => ActorSystem(name, config)
|
||||
}
|
||||
})
|
||||
|
||||
val log: LoggingAdapter = Logging(system, this)(_.getClass.getName)
|
||||
|
||||
/**
|
||||
* Enrich `.await()` onto all Awaitables, using remaining duration from the innermost
|
||||
* enclosing `within` block or QueryTimeout.
|
||||
*/
|
||||
implicit def awaitHelper[T](w: Awaitable[T]): AwaitHelper[T] = new AwaitHelper(w)
|
||||
class AwaitHelper[T](w: Awaitable[T]) {
|
||||
def await: T = Await.result(w, remainingOr(testConductor.Settings.QueryTimeout.duration))
|
||||
}
|
||||
|
||||
final override def multiNodeSpecBeforeAll(): Unit = {
|
||||
atStartup()
|
||||
}
|
||||
|
||||
final override def multiNodeSpecAfterAll(): Unit = {
|
||||
// wait for all nodes to remove themselves before we shut the conductor down
|
||||
if (selfIndex == 0) {
|
||||
testConductor.removeNode(myself)
|
||||
within(testConductor.Settings.BarrierTimeout.duration) {
|
||||
awaitCond({
|
||||
// Await.result(testConductor.getNodes, remaining).filterNot(_ == myself).isEmpty
|
||||
testConductor.getNodes.await.forall(_ == myself)
|
||||
}, message = s"Nodes not shutdown: ${testConductor.getNodes.await}")
|
||||
}
|
||||
}
|
||||
shutdown(system, duration = shutdownTimeout)
|
||||
afterTermination()
|
||||
}
|
||||
|
||||
def shutdownTimeout: FiniteDuration = 15.seconds.dilated
|
||||
|
||||
/**
|
||||
* Override this and return `true` to assert that the
|
||||
* shutdown of the `ActorSystem` was done properly.
|
||||
*/
|
||||
def verifySystemShutdown: Boolean = false
|
||||
|
||||
/*
|
||||
* Test Class Interface
|
||||
*/
|
||||
|
||||
/**
|
||||
* Override this method to do something when the whole test is starting up.
|
||||
*/
|
||||
protected def atStartup(): Unit = ()
|
||||
|
||||
/**
|
||||
* Override this method to do something when the whole test is terminating.
|
||||
*/
|
||||
protected def afterTermination(): Unit = ()
|
||||
|
||||
/**
|
||||
* All registered roles
|
||||
*/
|
||||
def roles: immutable.Seq[RoleName] = _roles
|
||||
|
||||
/**
|
||||
* TO BE DEFINED BY USER: Defines the number of participants required for starting the test. This
|
||||
* might not be equals to the number of nodes available to the test.
|
||||
*
|
||||
* Must be a `def`:
|
||||
* {{{
|
||||
* def initialParticipants = 5
|
||||
* }}}
|
||||
*/
|
||||
def initialParticipants: Int
|
||||
require(
|
||||
initialParticipants > 0,
|
||||
"initialParticipants must be a 'def' or early initializer, and it must be greater zero")
|
||||
require(initialParticipants <= maxNodes, "not enough nodes to run this test")
|
||||
|
||||
/**
|
||||
* Access to the barriers, failure injection, etc. The extension will have
|
||||
* been started either in Conductor or Player mode when the constructor of
|
||||
* MultiNodeSpec finishes, i.e. do not call the start*() methods yourself!
|
||||
*/
|
||||
var testConductor: TestConductorExt = null
|
||||
|
||||
/**
|
||||
* Execute the given block of code only on the given nodes (names according
|
||||
* to the `roleMap`).
|
||||
*/
|
||||
def runOn(nodes: RoleName*)(thunk: => Unit): Unit = {
|
||||
if (isNode(nodes: _*)) {
|
||||
thunk
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify that the running node matches one of the given nodes
|
||||
*/
|
||||
def isNode(nodes: RoleName*): Boolean = nodes contains myself
|
||||
|
||||
/**
|
||||
* Enter the named barriers in the order given. Use the remaining duration from
|
||||
* the innermost enclosing `within` block or the default `BarrierTimeout`.
|
||||
*/
|
||||
def enterBarrier(name: String*): Unit =
|
||||
testConductor.enter(
|
||||
Timeout.durationToTimeout(remainingOr(testConductor.Settings.BarrierTimeout.duration)),
|
||||
name.to(immutable.Seq))
|
||||
|
||||
/**
|
||||
* Enter the named barriers in the order given. Use the remaining duration from
|
||||
* the innermost enclosing `within` block or the passed `max` timeout.
|
||||
*
|
||||
* Note that the `max` timeout is scaled using Duration.dilated,
|
||||
* which uses the configuration entry "pekko.test.timefactor".
|
||||
*/
|
||||
def enterBarrier(max: FiniteDuration, name: String*): Unit =
|
||||
testConductor.enter(Timeout.durationToTimeout(remainingOr(max.dilated)), name.to(immutable.Seq))
|
||||
|
||||
/**
|
||||
* Query the controller for the transport address of the given node (by role name) and
|
||||
* return that as an ActorPath for easy composition:
|
||||
*
|
||||
* {{{
|
||||
* val serviceA = system.actorSelection(node("master") / "user" / "serviceA")
|
||||
* }}}
|
||||
*/
|
||||
def node(role: RoleName): ActorPath = RootActorPath(testConductor.getAddressFor(role).await)
|
||||
|
||||
def muteDeadLetters(messageClasses: Class[_]*)(sys: ActorSystem = system): Unit =
|
||||
if (!sys.log.isDebugEnabled) {
|
||||
def mute(clazz: Class[_]): Unit =
|
||||
sys.eventStream.publish(Mute(DeadLettersFilter(clazz)(occurrences = Int.MaxValue)))
|
||||
if (messageClasses.isEmpty) mute(classOf[AnyRef])
|
||||
else messageClasses.foreach(mute)
|
||||
}
|
||||
|
||||
/*
|
||||
* Implementation (i.e. wait for start etc.)
|
||||
*/
|
||||
|
||||
private val controllerAddr = new InetSocketAddress(serverName, serverPort)
|
||||
|
||||
protected def attachConductor(tc: TestConductorExt): Unit = {
|
||||
val timeout = tc.Settings.BarrierTimeout.duration
|
||||
val startFuture =
|
||||
if (selfIndex == 0) tc.startController(initialParticipants, myself, controllerAddr)
|
||||
else tc.startClient(myself, controllerAddr)
|
||||
try Await.result(startFuture, timeout)
|
||||
catch {
|
||||
case NonFatal(x) => throw new RuntimeException("failure while attaching new conductor", x)
|
||||
}
|
||||
testConductor = tc
|
||||
}
|
||||
|
||||
attachConductor(TestConductor(system))
|
||||
|
||||
// now add deployments, if so desired
|
||||
|
||||
// Cannot be final because of https://github.com/scala/bug/issues/4440
|
||||
private case class Replacement(tag: String, role: RoleName) {
|
||||
lazy val addr = node(role).address.toString
|
||||
}
|
||||
|
||||
private val replacements = roles.map(r => Replacement("@" + r.name + "@", r))
|
||||
|
||||
protected def injectDeployments(sys: ActorSystem, role: RoleName): Unit = {
|
||||
val deployer = sys.asInstanceOf[ExtendedActorSystem].provider.deployer
|
||||
deployments(role).foreach { str =>
|
||||
val deployString = replacements.foldLeft(str) {
|
||||
case (base, r @ Replacement(tag, _)) =>
|
||||
base.indexOf(tag) match {
|
||||
case -1 => base
|
||||
case _ =>
|
||||
val replaceWith =
|
||||
try r.addr
|
||||
catch {
|
||||
case NonFatal(e) =>
|
||||
// might happen if all test cases are ignored (excluded) and
|
||||
// controller node is finished/exited before r.addr is run
|
||||
// on the other nodes
|
||||
val unresolved = "akka://unresolved-replacement-" + r.role.name
|
||||
log.warning(unresolved + " due to: " + e.getMessage)
|
||||
unresolved
|
||||
}
|
||||
base.replace(tag, replaceWith)
|
||||
}
|
||||
}
|
||||
import pekko.util.ccompat.JavaConverters._
|
||||
ConfigFactory.parseString(deployString).root.asScala.foreach {
|
||||
case (key, value: ConfigObject) => deployer.parseConfig(key, value.toConfig).foreach(deployer.deploy)
|
||||
case (key, x) =>
|
||||
throw new IllegalArgumentException(s"key $key must map to deployment section, not simple value $x")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
injectDeployments(system, myself)
|
||||
|
||||
protected val myAddress = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
|
||||
|
||||
// useful to see which jvm is running which role, used by LogRoleReplace utility
|
||||
log.info("Role [{}] started with address [{}]", myself.name, myAddress)
|
||||
|
||||
/**
|
||||
* This method starts a new ActorSystem with the same configuration as the
|
||||
* previous one on the current node, including deployments. It also creates
|
||||
* a new TestConductor client and registers itself with the conductor so
|
||||
* that it is possible to use barriers etc. normally after this method has
|
||||
* been called.
|
||||
*
|
||||
* NOTICE: you MUST start a new system before trying to enter a barrier or
|
||||
* otherwise using the TestConductor after having terminated this node’s
|
||||
* system.
|
||||
*/
|
||||
protected def startNewSystem(): ActorSystem = {
|
||||
val config = ConfigFactory
|
||||
.parseString(s"pekko.remote.classic.netty.tcp{port=${myAddress.port.get}\nhostname=${myAddress.host.get}}")
|
||||
.withFallback(system.settings.config)
|
||||
val sys = ActorSystem(system.name, config)
|
||||
injectDeployments(sys, myself)
|
||||
attachConductor(TestConductor(sys))
|
||||
sys
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Use this to hook MultiNodeSpec into your test framework lifecycle, either by having your test extend MultiNodeSpec
|
||||
* and call these methods or by creating a trait that calls them and then mixing that trait with your test together
|
||||
* with MultiNodeSpec.
|
||||
*
|
||||
* Example trait for MultiNodeSpec with ScalaTest
|
||||
*
|
||||
* {{{
|
||||
* trait STMultiNodeSpec extends MultiNodeSpecCallbacks with AnyWordSpecLike with Matchers with BeforeAndAfterAll {
|
||||
* override def beforeAll() = multiNodeSpecBeforeAll()
|
||||
* override def afterAll() = multiNodeSpecAfterAll()
|
||||
* }
|
||||
* }}}
|
||||
*/
|
||||
trait MultiNodeSpecCallbacks {
|
||||
|
||||
/**
|
||||
* Call this before the start of the test run. NOT before every test case.
|
||||
*/
|
||||
def multiNodeSpecBeforeAll(): Unit
|
||||
|
||||
/**
|
||||
* Call this after the all test cases have run. NOT after every test case.
|
||||
*/
|
||||
def multiNodeSpecAfterAll(): Unit
|
||||
}
|
||||
|
|
@ -0,0 +1,58 @@
|
|||
/*
|
||||
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.remote.testkit
|
||||
|
||||
import java.io.File
|
||||
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.remote.testconductor.RoleName
|
||||
|
||||
/**
|
||||
* INTERNAL API: Support trait allowing trivially recording perf metrics from [[MultiNodeSpec]]s
|
||||
*/
|
||||
private[pekko] trait PerfFlamesSupport { self: MultiNodeSpec =>
|
||||
|
||||
/**
|
||||
* Runs `perf-java-flames` script on given node (JVM process).
|
||||
* Refer to https://github.com/jrudolph/perf-map-agent for options and manual.
|
||||
*
|
||||
* Options are currently to be passed in via `export PERF_MAP_OPTIONS` etc.
|
||||
*/
|
||||
def runPerfFlames(nodes: RoleName*)(delay: FiniteDuration): Unit = {
|
||||
if (isPerfJavaFlamesAvailable && isNode(nodes: _*)) {
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
|
||||
val afterDelay = pekko.pattern.after(delay, system.scheduler)(Future.successful("GO!"))
|
||||
afterDelay.onComplete { _ =>
|
||||
import java.lang.management._
|
||||
val name = ManagementFactory.getRuntimeMXBean.getName
|
||||
val pid = name.substring(0, name.indexOf('@')).toInt
|
||||
|
||||
val perfCommand = s"$perfJavaFlamesPath $pid"
|
||||
println(s"[perf @ $myself($pid)][OUT]: " + perfCommand)
|
||||
|
||||
import scala.sys.process._
|
||||
perfCommand.run(new ProcessLogger {
|
||||
override def buffer[T](f: => T): T = f
|
||||
override def out(s: => String): Unit = println(s"[perf @ $myself($pid)][OUT] " + s)
|
||||
override def err(s: => String): Unit = println(s"[perf @ $myself($pid)][ERR] " + s)
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def perfJavaFlamesPath: String =
|
||||
"/home/ubuntu/perf-java-flames"
|
||||
|
||||
def isPerfJavaFlamesAvailable: Boolean = {
|
||||
val isIt = new File(perfJavaFlamesPath).exists()
|
||||
if (!isIt) println(s"WARN: perf-java-flames not available under [$perfJavaFlamesPath]! Skipping perf profiling.")
|
||||
isIt
|
||||
}
|
||||
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue