Fixed bug in RemoteClient reconnect, now works flawlessly + Added option to declaratively configure an Actor to be remote

This commit is contained in:
Jonas Bonér 2009-12-29 14:24:48 +01:00
parent 89178aec23
commit 90f7e0ea4e
5 changed files with 24 additions and 13 deletions

View file

@ -112,7 +112,7 @@ sealed class Supervisor private[akka] (handler: FaultHandlingStrategy, trapExcep
case SupervisorConfig(_, servers) =>
servers.map(server =>
server match {
case Supervise(actor, lifeCycle) =>
case Supervise(actor, lifeCycle, remoteAddress) =>
actors.put(actor.getClass.getName, actor)
actor.lifeCycle = Some(lifeCycle)
startLink(actor)

View file

@ -24,7 +24,15 @@ object ScalaConfig {
abstract class Scope extends ConfigElement
case class SupervisorConfig(restartStrategy: RestartStrategy, worker: List[Server]) extends Server
case class Supervise(actor: Actor, lifeCycle: LifeCycle) extends Server
class Supervise(val actor: Actor, val lifeCycle: LifeCycle, _remoteAddress: RemoteAddress) extends Server {
val remoteAddress: Option[RemoteAddress] = if (_remoteAddress eq null) None else Some(_remoteAddress)
}
object Supervise {
def apply(actor: Actor, lifeCycle: LifeCycle, remoteAddress: RemoteAddress) = new Supervise(actor, lifeCycle, remoteAddress)
def apply(actor: Actor, lifeCycle: LifeCycle) = new Supervise(actor, lifeCycle, null)
def unapply(supervise: Supervise) = Some((supervise.actor, supervise.lifeCycle, supervise.remoteAddress))
}
case class RestartStrategy(
scheme: FailOverScheme,

View file

@ -21,7 +21,7 @@ import org.jboss.netty.handler.codec.protobuf.{ProtobufDecoder, ProtobufEncoder}
import org.jboss.netty.handler.timeout.ReadTimeoutHandler
import org.jboss.netty.util.{TimerTask, Timeout, HashedWheelTimer}
import java.net.InetSocketAddress
import java.net.{SocketAddress, InetSocketAddress}
import java.util.concurrent.{TimeUnit, Executors, ConcurrentMap, ConcurrentHashMap}
import java.util.concurrent.atomic.AtomicLong
@ -84,8 +84,9 @@ class RemoteClient(hostname: String, port: Int) extends Logging {
private val bootstrap = new ClientBootstrap(channelFactory)
private val timer = new HashedWheelTimer
private val remoteAddress = new InetSocketAddress(hostname, port)
bootstrap.setPipelineFactory(new RemoteClientPipelineFactory(name, futures, supervisors, bootstrap, timer))
bootstrap.setPipelineFactory(new RemoteClientPipelineFactory(name, futures, supervisors, bootstrap, remoteAddress, timer))
bootstrap.setOption("tcpNoDelay", true)
bootstrap.setOption("keepAlive", true)
@ -93,7 +94,7 @@ class RemoteClient(hostname: String, port: Int) extends Logging {
def connect = synchronized {
if (!isRunning) {
connection = bootstrap.connect(new InetSocketAddress(hostname, port))
connection = bootstrap.connect(remoteAddress)
log.info("Starting remote client connection to [%s:%s]", hostname, port)
// Wait until the connection attempt succeeds or fails.
@ -146,6 +147,7 @@ class RemoteClientPipelineFactory(name: String,
futures: ConcurrentMap[Long, CompletableFutureResult],
supervisors: ConcurrentMap[String, Actor],
bootstrap: ClientBootstrap,
remoteAddress: SocketAddress,
timer: HashedWheelTimer) extends ChannelPipelineFactory {
def getPipeline: ChannelPipeline = {
val pipeline = Channels.pipeline()
@ -164,7 +166,7 @@ class RemoteClientPipelineFactory(name: String,
}
pipeline.addLast("frameEncoder", new LengthFieldPrepender(4))
pipeline.addLast("protobufEncoder", new ProtobufEncoder())
pipeline.addLast("handler", new RemoteClientHandler(name, futures, supervisors, bootstrap, timer))
pipeline.addLast("handler", new RemoteClientHandler(name, futures, supervisors, bootstrap, remoteAddress, timer))
pipeline
}
}
@ -177,6 +179,7 @@ class RemoteClientHandler(val name: String,
val futures: ConcurrentMap[Long, CompletableFutureResult],
val supervisors: ConcurrentMap[String, Actor],
val bootstrap: ClientBootstrap,
val remoteAddress: SocketAddress,
val timer: HashedWheelTimer)
extends SimpleChannelUpstreamHandler with Logging {
import Actor.Sender.Self
@ -221,8 +224,8 @@ class RemoteClientHandler(val name: String,
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
timer.newTimeout(new TimerTask() {
def run(timeout: Timeout) = {
log.debug("Remote client reconnecting to [%s]", ctx.getChannel.getRemoteAddress)
bootstrap.connect
log.debug("Remote client reconnecting to [%s]", remoteAddress)
bootstrap.connect(remoteAddress)
}
}, RemoteClient.RECONNECT_DELAY, TimeUnit.MILLISECONDS)
}

View file

@ -170,7 +170,8 @@ class Boot {
RestartStrategy(OneForOne, 3, 100, List(classOf[Exception])),
Supervise(
ChatService,
LifeCycle(Permanent))
LifeCycle(Permanent),
RemoteAddress("localhost", 9999))
:: Nil))
factory.newInstance.start
}

View file

@ -47,8 +47,7 @@ class SimpleService extends Transactor {
* Or browse to the URL from a web browser.
*/
@Path("/persistentliftcount")
class PersistentSimpleService extends Actor {
makeTransactionRequired
class PersistentSimpleService extends Transactor {
case object Tick
private val KEY = "COUNTER"