2011-09-14 16:09:17 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
|
|
|
|
|
*/
|
|
|
|
|
|
package akka.remote.testconductor
|
|
|
|
|
|
|
2012-05-02 21:56:26 +02:00
|
|
|
|
import akka.actor.{ Actor, ActorRef, ActorSystem, LoggingFSM, Props }
|
2011-09-14 16:09:17 +02:00
|
|
|
|
import RemoteConnection.getAddrString
|
|
|
|
|
|
import akka.util.duration._
|
|
|
|
|
|
import org.jboss.netty.channel.{ Channel, SimpleChannelUpstreamHandler, ChannelHandlerContext, ChannelStateEvent, MessageEvent }
|
|
|
|
|
|
import com.eaio.uuid.UUID
|
2012-05-02 21:56:26 +02:00
|
|
|
|
import com.typesafe.config.ConfigFactory
|
|
|
|
|
|
import akka.util.Timeout
|
|
|
|
|
|
import akka.util.Duration
|
|
|
|
|
|
import java.util.concurrent.TimeUnit.MILLISECONDS
|
2012-05-04 22:33:08 +02:00
|
|
|
|
import akka.pattern.{ ask, pipe }
|
2012-05-02 21:56:26 +02:00
|
|
|
|
import akka.dispatch.Await
|
|
|
|
|
|
import scala.util.control.NoStackTrace
|
|
|
|
|
|
import akka.actor.Status
|
|
|
|
|
|
import akka.event.LoggingAdapter
|
|
|
|
|
|
import akka.actor.PoisonPill
|
|
|
|
|
|
import akka.event.Logging
|
2012-05-04 22:33:08 +02:00
|
|
|
|
import akka.dispatch.Future
|
2011-09-14 16:09:17 +02:00
|
|
|
|
|
2012-05-10 21:08:06 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* The Player is the client component of the
|
|
|
|
|
|
* [[akka.remote.testconductor.TestConductorExt]] extension. It registers with
|
|
|
|
|
|
* the [[akka.remote.testconductor.Conductor]]’s [[akka.remote.testconductor.Controller]]
|
|
|
|
|
|
* in order to participate in barriers and enable network failure injection.
|
|
|
|
|
|
*/
|
|
|
|
|
|
trait Player { this: TestConductorExt ⇒
|
2011-09-14 16:09:17 +02:00
|
|
|
|
|
2012-05-03 20:48:27 +02:00
|
|
|
|
private var _client: ActorRef = _
|
|
|
|
|
|
private def client = _client match {
|
|
|
|
|
|
case null ⇒ throw new IllegalStateException("TestConductor client not yet started")
|
|
|
|
|
|
case x ⇒ x
|
|
|
|
|
|
}
|
2012-05-02 21:56:26 +02:00
|
|
|
|
|
2012-05-10 21:08:06 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* Connect to the conductor on the given port (the host is taken from setting
|
|
|
|
|
|
* `akka.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 [[akka.remote.testconductor.Conductor]]`.startController()`.
|
|
|
|
|
|
*/
|
2012-05-04 22:33:08 +02:00
|
|
|
|
def startClient(port: Int): Future[Done] = {
|
2012-05-03 20:48:27 +02:00
|
|
|
|
import ClientFSM._
|
|
|
|
|
|
import akka.actor.FSM._
|
|
|
|
|
|
import Settings.BarrierTimeout
|
|
|
|
|
|
|
|
|
|
|
|
if (_client ne null) throw new IllegalStateException("TestConductorClient already started")
|
|
|
|
|
|
_client = system.actorOf(Props(new ClientFSM(port)), "TestConductorClient")
|
|
|
|
|
|
val a = system.actorOf(Props(new Actor {
|
|
|
|
|
|
var waiting: ActorRef = _
|
|
|
|
|
|
def receive = {
|
|
|
|
|
|
case fsm: ActorRef ⇒ waiting = sender; fsm ! SubscribeTransitionCallBack(self)
|
2012-05-07 08:04:15 +02:00
|
|
|
|
case Transition(_, Connecting, AwaitDone) ⇒ // step 1, not there yet
|
|
|
|
|
|
case Transition(_, AwaitDone, Connected) ⇒ waiting ! Done
|
2012-05-03 20:48:27 +02:00
|
|
|
|
case t: Transition[_] ⇒ waiting ! Status.Failure(new RuntimeException("unexpected transition: " + t))
|
2012-05-04 22:33:08 +02:00
|
|
|
|
case CurrentState(_, Connected) ⇒ waiting ! Done
|
2012-05-03 20:48:27 +02:00
|
|
|
|
case _: CurrentState[_] ⇒
|
|
|
|
|
|
}
|
|
|
|
|
|
}))
|
2012-05-02 21:56:26 +02:00
|
|
|
|
|
2012-05-04 22:33:08 +02:00
|
|
|
|
a ? client mapTo
|
2012-05-02 21:56:26 +02:00
|
|
|
|
}
|
|
|
|
|
|
|
2012-05-10 21:08:06 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* 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*) {
|
2012-05-02 21:56:26 +02:00
|
|
|
|
system.log.debug("entering barriers " + name.mkString("(", ", ", ")"))
|
2011-09-14 16:09:17 +02:00
|
|
|
|
name foreach { b ⇒
|
2012-05-02 21:56:26 +02:00
|
|
|
|
import Settings.BarrierTimeout
|
2012-05-04 22:33:08 +02:00
|
|
|
|
Await.result(client ? Send(EnterBarrier(b)), Duration.Inf)
|
2012-05-02 21:56:26 +02:00
|
|
|
|
system.log.debug("passed barrier {}", b)
|
2011-09-14 16:09:17 +02:00
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
object ClientFSM {
|
|
|
|
|
|
sealed trait State
|
|
|
|
|
|
case object Connecting extends State
|
2012-05-07 08:04:15 +02:00
|
|
|
|
case object AwaitDone extends State
|
2011-09-14 16:09:17 +02:00
|
|
|
|
case object Connected extends State
|
|
|
|
|
|
|
2012-05-03 20:48:27 +02:00
|
|
|
|
case class Data(channel: Channel, barrier: Option[(String, ActorRef)])
|
2011-09-14 16:09:17 +02:00
|
|
|
|
|
|
|
|
|
|
class ConnectionFailure(msg: String) extends RuntimeException(msg) with NoStackTrace
|
|
|
|
|
|
case object Disconnected
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2012-05-10 21:08:06 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* This is the controlling entity on the [[akka.remote.testconductor.Player]]
|
|
|
|
|
|
* side: in a first step it registers itself with a symbolic name and its remote
|
|
|
|
|
|
* address at the [[akka.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 [[akka.remote.testconductor.Conductor]]’s
|
|
|
|
|
|
* requests for failure injection.
|
|
|
|
|
|
*/
|
2012-05-03 20:48:27 +02:00
|
|
|
|
class ClientFSM(port: Int) extends Actor with LoggingFSM[ClientFSM.State, ClientFSM.Data] {
|
2011-09-14 16:09:17 +02:00
|
|
|
|
import ClientFSM._
|
|
|
|
|
|
|
2012-05-03 20:48:27 +02:00
|
|
|
|
val settings = TestConductor().Settings
|
2012-05-02 21:56:26 +02:00
|
|
|
|
|
|
|
|
|
|
val handler = new PlayerHandler(self, Logging(context.system, "PlayerHandler"))
|
2011-09-14 16:09:17 +02:00
|
|
|
|
|
2012-05-03 20:48:27 +02:00
|
|
|
|
startWith(Connecting, Data(RemoteConnection(Client, settings.host, port, handler), None))
|
2011-09-14 16:09:17 +02:00
|
|
|
|
|
|
|
|
|
|
when(Connecting, stateTimeout = 10 seconds) {
|
2012-05-03 20:48:27 +02:00
|
|
|
|
case Event(msg: ClientOp, _) ⇒
|
|
|
|
|
|
stay replying Status.Failure(new IllegalStateException("not connected yet"))
|
|
|
|
|
|
case Event(Connected, d @ Data(channel, _)) ⇒
|
2012-05-04 22:33:08 +02:00
|
|
|
|
channel.write(Hello(settings.name, TestConductor().address))
|
2012-05-07 08:04:15 +02:00
|
|
|
|
goto(AwaitDone)
|
2011-09-14 16:09:17 +02:00
|
|
|
|
case Event(_: ConnectionFailure, _) ⇒
|
|
|
|
|
|
// System.exit(1)
|
|
|
|
|
|
stop
|
|
|
|
|
|
case Event(StateTimeout, _) ⇒
|
2012-05-02 21:56:26 +02:00
|
|
|
|
log.error("connect timeout to TestConductor")
|
2011-09-14 16:09:17 +02:00
|
|
|
|
// System.exit(1)
|
|
|
|
|
|
stop
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2012-05-07 08:04:15 +02:00
|
|
|
|
when(AwaitDone, stateTimeout = settings.BarrierTimeout.duration) {
|
|
|
|
|
|
case Event(Done, _) ⇒
|
|
|
|
|
|
log.debug("received Done: starting test")
|
|
|
|
|
|
goto(Connected)
|
|
|
|
|
|
case Event(msg: ClientOp, _) ⇒
|
|
|
|
|
|
stay replying Status.Failure(new IllegalStateException("not connected yet"))
|
|
|
|
|
|
case Event(StateTimeout, _) ⇒
|
|
|
|
|
|
log.error("connect timeout to TestConductor")
|
|
|
|
|
|
// System.exit(1)
|
|
|
|
|
|
stop
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2011-09-14 16:09:17 +02:00
|
|
|
|
when(Connected) {
|
|
|
|
|
|
case Event(Disconnected, _) ⇒
|
2012-05-02 21:56:26 +02:00
|
|
|
|
log.info("disconnected from TestConductor")
|
2011-09-14 16:09:17 +02:00
|
|
|
|
throw new ConnectionFailure("disconnect")
|
2012-05-04 22:33:08 +02:00
|
|
|
|
case Event(Send(msg: EnterBarrier), Data(channel, None)) ⇒
|
|
|
|
|
|
channel.write(msg)
|
2012-05-03 20:48:27 +02:00
|
|
|
|
stay using Data(channel, Some(msg.name, sender))
|
2012-05-04 22:33:08 +02:00
|
|
|
|
case Event(Send(d: Done), Data(channel, _)) ⇒
|
|
|
|
|
|
channel.write(d)
|
|
|
|
|
|
stay
|
|
|
|
|
|
case Event(Send(x), _) ⇒
|
|
|
|
|
|
log.warning("cannot send message {}", x)
|
|
|
|
|
|
stay
|
|
|
|
|
|
case Event(EnterBarrier(b), Data(channel, Some((barrier, sender)))) ⇒
|
|
|
|
|
|
if (b != barrier) {
|
|
|
|
|
|
sender ! Status.Failure(new RuntimeException("wrong barrier " + b + " received while waiting for " + barrier))
|
|
|
|
|
|
} else {
|
|
|
|
|
|
sender ! b
|
2011-09-14 16:09:17 +02:00
|
|
|
|
}
|
2012-05-03 20:48:27 +02:00
|
|
|
|
stay using Data(channel, None)
|
2012-05-08 11:08:43 +02:00
|
|
|
|
case Event(BarrierFailed(b), Data(channel, Some((_, sender)))) ⇒
|
|
|
|
|
|
sender ! Status.Failure(new RuntimeException("barrier failed: " + b))
|
|
|
|
|
|
stay using Data(channel, None)
|
2012-05-04 22:33:08 +02:00
|
|
|
|
case Event(ThrottleMsg(target, dir, rate), _) ⇒
|
|
|
|
|
|
import settings.QueryTimeout
|
|
|
|
|
|
import context.dispatcher
|
|
|
|
|
|
TestConductor().failureInjectors.get(target.copy(system = "")) match {
|
|
|
|
|
|
case null ⇒ log.warning("cannot throttle unknown address {}", target)
|
|
|
|
|
|
case inj ⇒
|
|
|
|
|
|
Future.sequence(inj.refs(dir) map (_ ? NetworkFailureInjector.SetRate(rate))) map (_ ⇒ Send(Done)) pipeTo self
|
|
|
|
|
|
}
|
|
|
|
|
|
stay
|
|
|
|
|
|
case Event(DisconnectMsg(target, abort), _) ⇒
|
|
|
|
|
|
import settings.QueryTimeout
|
|
|
|
|
|
TestConductor().failureInjectors.get(target.copy(system = "")) match {
|
|
|
|
|
|
case null ⇒ log.warning("cannot disconnect unknown address {}", target)
|
|
|
|
|
|
case inj ⇒ inj.sender ? NetworkFailureInjector.Disconnect(abort) map (_ ⇒ Send(Done)) pipeTo self
|
|
|
|
|
|
}
|
|
|
|
|
|
stay
|
|
|
|
|
|
case Event(TerminateMsg(exit), _) ⇒
|
|
|
|
|
|
System.exit(exit)
|
|
|
|
|
|
stay // needed because Java doesn’t have Nothing
|
2011-09-14 16:09:17 +02:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
onTermination {
|
|
|
|
|
|
case StopEvent(_, _, Data(channel, _)) ⇒
|
|
|
|
|
|
channel.close()
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2012-05-03 20:48:27 +02:00
|
|
|
|
initialize
|
|
|
|
|
|
|
2011-09-14 16:09:17 +02:00
|
|
|
|
}
|
|
|
|
|
|
|
2012-05-10 21:08:06 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* This handler only forwards messages received from the conductor to the [[akka.remote.testconductor.ClientFSM]].
|
|
|
|
|
|
*/
|
2012-05-02 21:56:26 +02:00
|
|
|
|
class PlayerHandler(fsm: ActorRef, log: LoggingAdapter) extends SimpleChannelUpstreamHandler {
|
2011-09-14 16:09:17 +02:00
|
|
|
|
|
|
|
|
|
|
import ClientFSM._
|
|
|
|
|
|
|
|
|
|
|
|
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
|
|
|
|
|
val channel = event.getChannel
|
2012-05-02 21:56:26 +02:00
|
|
|
|
log.debug("connected to {}", getAddrString(channel))
|
2011-09-14 16:09:17 +02:00
|
|
|
|
fsm ! Connected
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
|
|
|
|
|
val channel = event.getChannel
|
2012-05-02 21:56:26 +02:00
|
|
|
|
log.debug("disconnected from {}", getAddrString(channel))
|
|
|
|
|
|
fsm ! PoisonPill
|
2011-09-14 16:09:17 +02:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = {
|
|
|
|
|
|
val channel = event.getChannel
|
2012-05-02 21:56:26 +02:00
|
|
|
|
log.debug("message from {}: {}", getAddrString(channel), event.getMessage)
|
2011-09-14 16:09:17 +02:00
|
|
|
|
event.getMessage match {
|
2012-05-04 22:33:08 +02:00
|
|
|
|
case msg: NetworkOp ⇒
|
2011-09-14 16:09:17 +02:00
|
|
|
|
fsm ! msg
|
|
|
|
|
|
case msg ⇒
|
2012-05-02 21:56:26 +02:00
|
|
|
|
log.info("server {} sent garbage '{}', disconnecting", getAddrString(channel), msg)
|
2011-09-14 16:09:17 +02:00
|
|
|
|
channel.close()
|
|
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
|