Fixed bug in RemoteClient reconnect, now works flawlessly + Added option to declaratively configure an Actor to be remote
This commit is contained in:
parent
89178aec23
commit
90f7e0ea4e
5 changed files with 24 additions and 13 deletions
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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,
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue