Moved all 'akka.remote' to 'akka.cluster', no more 'remote' package.

Signed-off-by: Jonas Bonér <jonas@jonasboner.com>
This commit is contained in:
Jonas Bonér 2011-08-29 11:44:33 +02:00
parent 43bbc199b2
commit 66f339e202
34 changed files with 1068 additions and 1033 deletions

View file

@ -3,7 +3,7 @@ package akka.actor;
import akka.japi.Creator; import akka.japi.Creator;
import org.junit.Test; import org.junit.Test;
import akka.actor.Actors; import akka.actor.Actors;
import akka.remoteinterface.RemoteSupport; import akka.cluster.RemoteSupport;
import static org.junit.Assert.*; import static org.junit.Assert.*;
public class JavaAPI { public class JavaAPI {

View file

@ -23,7 +23,7 @@ class ClusterSpec extends WordSpec with MustMatchers {
getBool("akka.cluster.include-ref-node-in-replica-set") must equal(Some(true)) getBool("akka.cluster.include-ref-node-in-replica-set") must equal(Some(true))
getString("akka.cluster.compression-scheme") must equal(Some("")) getString("akka.cluster.compression-scheme") must equal(Some(""))
getInt("akka.cluster.zlib-compression-level") must equal(Some(6)) getInt("akka.cluster.zlib-compression-level") must equal(Some(6))
getString("akka.cluster.layer") must equal(Some("akka.remote.netty.NettyRemoteSupport")) getString("akka.cluster.layer") must equal(Some("akka.cluster.netty.NettyRemoteSupport"))
getString("akka.cluster.secure-cookie") must equal(Some("")) getString("akka.cluster.secure-cookie") must equal(Some(""))
getString("akka.cluster.log-directory") must equal(Some("_akka_cluster")) getString("akka.cluster.log-directory") must equal(Some("_akka_cluster"))
@ -38,7 +38,7 @@ class ClusterSpec extends WordSpec with MustMatchers {
//akka.cluster.server //akka.cluster.server
getInt("akka.cluster.server.port") must equal(Some(2552)) getInt("akka.cluster.server.port") must equal(Some(2552))
getInt("akka.cluster.server.message-frame-size") must equal(Some(1048576)) getInt("akka.cluster.server.message-frame-size") must equal(Some(1048576))
getInt("akka.cluster.server.connection-timeout") must equal(Some(100)) getInt("akka.cluster.server.connection-timeout") must equal(Some(120))
getBool("akka.cluster.server.require-cookie") must equal(Some(false)) getBool("akka.cluster.server.require-cookie") must equal(Some(false))
getBool("akka.cluster.server.untrusted-mode") must equal(Some(false)) getBool("akka.cluster.server.untrusted-mode") must equal(Some(false))
getInt("akka.cluster.server.backlog") must equal(Some(4096)) getInt("akka.cluster.server.backlog") must equal(Some(4096))
@ -51,7 +51,7 @@ class ClusterSpec extends WordSpec with MustMatchers {
getBool("akka.cluster.client.buffering.retry-message-send-on-failure") must equal(Some(true)) getBool("akka.cluster.client.buffering.retry-message-send-on-failure") must equal(Some(true))
getInt("akka.cluster.client.buffering.capacity") must equal(Some(-1)) getInt("akka.cluster.client.buffering.capacity") must equal(Some(-1))
getInt("akka.cluster.client.reconnect-delay") must equal(Some(5)) getInt("akka.cluster.client.reconnect-delay") must equal(Some(5))
getInt("akka.cluster.client.read-timeout") must equal(Some(10)) getInt("akka.cluster.client.read-timeout") must equal(Some(3600))
getInt("akka.cluster.client.reap-futures-delay") must equal(Some(5)) getInt("akka.cluster.client.reap-futures-delay") must equal(Some(5))
getInt("akka.cluster.client.reconnection-time-window") must equal(Some(600)) getInt("akka.cluster.client.reconnection-time-window") must equal(Some(600))
} }

View file

@ -39,7 +39,7 @@ class ConfigSpec extends WordSpec with MustMatchers {
getInt("akka.actor.throughput") must equal(Some(5)) getInt("akka.actor.throughput") must equal(Some(5))
getInt("akka.actor.throughput-deadline-time") must equal(Some(-1)) getInt("akka.actor.throughput-deadline-time") must equal(Some(-1))
getString("akka.cluster.layer") must equal(Some("akka.remote.netty.NettyRemoteSupport")) getString("akka.cluster.layer") must equal(Some("akka.cluster.netty.NettyRemoteSupport"))
getInt("akka.cluster.server.port") must equal(Some(2552)) getInt("akka.cluster.server.port") must equal(Some(2552))
} }
} }

View file

@ -1,7 +1,7 @@
package akka.actor; package akka.actor;
import akka.japi.Creator; import akka.japi.Creator;
import akka.remoteinterface.RemoteSupport; import akka.cluster.RemoteSupport;
import com.eaio.uuid.UUID; import com.eaio.uuid.UUID;

View file

@ -11,7 +11,7 @@ import akka.config._
import Config._ import Config._
import akka.util.{ ReflectiveAccess, Duration } import akka.util.{ ReflectiveAccess, Duration }
import ReflectiveAccess._ import ReflectiveAccess._
import akka.remoteinterface.RemoteSupport import akka.cluster.RemoteSupport
import akka.japi.{ Creator, Procedure } import akka.japi.{ Creator, Procedure }
import akka.AkkaException import akka.AkkaException
import akka.serialization.{ Serializer, Serialization } import akka.serialization.{ Serializer, Serialization }

View file

@ -44,7 +44,7 @@ private[akka] object ActorRefInternals {
* ActorRef configuration object, this is threadsafe and fully sharable * ActorRef configuration object, this is threadsafe and fully sharable
* *
* Props() returns default configuration * Props() returns default configuration
* TODO document me * FIXME document me
*/ */
object Props { object Props {
object Default { object Default {
@ -91,6 +91,7 @@ case class Props(creator: () ⇒ Actor = Props.Default.creator,
faultHandler = Props.Default.faultHandler, faultHandler = Props.Default.faultHandler,
supervisor = Props.Default.supervisor, supervisor = Props.Default.supervisor,
localOnly = Props.Default.localOnly) localOnly = Props.Default.localOnly)
/** /**
* Returns a new Props with the specified creator set * Returns a new Props with the specified creator set
* Scala API * Scala API

View file

@ -4,7 +4,7 @@
package akka.cluster package akka.cluster
import akka.remoteinterface.RemoteSupport import akka.cluster.RemoteSupport
import akka.serialization.Serializer import akka.serialization.Serializer
import akka.actor._ import akka.actor._
import DeploymentConfig._ import DeploymentConfig._
@ -426,9 +426,9 @@ trait ClusterNode {
// =============== PRIVATE METHODS =============== // =============== PRIVATE METHODS ===============
// FIXME considering moving all these private[cluster] methods to a separate trait to get them out of the user's view // FIXME BAD BAD BAD - considering moving all these private[cluster] methods to a separate trait to get them out of the user's view
private[cluster] def remoteClientLifeCycleListener: ActorRef private[cluster] def remoteClientLifeCycleHandler: ActorRef
private[cluster] def remoteDaemon: ActorRef private[cluster] def remoteDaemon: ActorRef

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.remoteinterface package akka.cluster
import akka.actor.Actor import akka.actor.Actor
import akka.event.EventHandler import akka.event.EventHandler

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2010 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.remoteinterface package akka.cluster
import akka.japi.Creator import akka.japi.Creator
import akka.actor._ import akka.actor._
@ -88,10 +88,18 @@ trait RemoteModule {
*/ */
} }
/**
* Remote life-cycle events.
*/
sealed trait RemoteLifeCycleEvent
/** /**
* Life-cycle events for RemoteClient. * Life-cycle events for RemoteClient.
*/ */
sealed trait RemoteClientLifeCycleEvent trait RemoteClientLifeCycleEvent extends RemoteLifeCycleEvent {
def remoteAddress: InetSocketAddress
}
case class RemoteClientError( case class RemoteClientError(
@BeanProperty cause: Throwable, @BeanProperty cause: Throwable,
@BeanProperty client: RemoteClientModule, @BeanProperty client: RemoteClientModule,
@ -117,7 +125,8 @@ case class RemoteClientWriteFailed(
/** /**
* Life-cycle events for RemoteServer. * Life-cycle events for RemoteServer.
*/ */
sealed trait RemoteServerLifeCycleEvent trait RemoteServerLifeCycleEvent extends RemoteLifeCycleEvent
case class RemoteServerStarted( case class RemoteServerStarted(
@BeanProperty val server: RemoteServerModule) extends RemoteServerLifeCycleEvent @BeanProperty val server: RemoteServerModule) extends RemoteServerLifeCycleEvent
case class RemoteServerShutdown( case class RemoteServerShutdown(

View file

@ -133,13 +133,13 @@ trait RouterConnections {
* reasons this can happen is that multiple thread could at the 'same' moment discover for the same ActorRef that * reasons this can happen is that multiple thread could at the 'same' moment discover for the same ActorRef that
* not working. * not working.
* *
* It could be that even after a signalDeadActor has been called for a specific ActorRef, that the ActorRef * It could be that even after a remove has been called for a specific ActorRef, that the ActorRef
* is still being used. A good behaving Router will eventually discard this reference, but no guarantees are * is still being used. A good behaving Router will eventually discard this reference, but no guarantees are
* made how long this takes place. * made how long this takes place.
* *
* @param ref the dead * @param ref the dead
*/ */
def signalDeadActor(deadRef: ActorRef): Unit def remove(deadRef: ActorRef): Unit
} }
object Routing { object Routing {
@ -250,7 +250,7 @@ class RoutedActorRef(val address: String, val router: Router, val connectionIter
} }
@tailrec @tailrec
final def signalDeadActor(ref: ActorRef) = { final def remove(ref: ActorRef) = {
val oldState = state.get() val oldState = state.get()
//remote the ref from the connections. //remote the ref from the connections.
@ -261,7 +261,7 @@ class RoutedActorRef(val address: String, val router: Router, val connectionIter
val newState = new State(oldState.version + 1, newList) val newState = new State(oldState.version + 1, newList)
//if we are not able to update the state, we just try again. //if we are not able to update the state, we just try again.
if (!state.compareAndSet(oldState, newState)) signalDeadActor(ref) if (!state.compareAndSet(oldState, newState)) remove(ref)
} }
} }
@ -293,7 +293,7 @@ trait BasicRouter extends Router {
actor.!(message)(sender) actor.!(message)(sender)
} catch { } catch {
case e: Exception case e: Exception
connections.signalDeadActor(actor) connections.remove(actor)
throw e throw e
}) })
case _ case _
@ -304,7 +304,7 @@ trait BasicRouter extends Router {
actor.!(message)(sender) actor.!(message)(sender)
} catch { } catch {
case e: Exception case e: Exception
connections.signalDeadActor(actor) connections.remove(actor)
throw e throw e
} }
case None case None
@ -323,7 +323,7 @@ trait BasicRouter extends Router {
actor.?(message, timeout)(sender).asInstanceOf[Future[T]] actor.?(message, timeout)(sender).asInstanceOf[Future[T]]
} catch { } catch {
case e: Exception case e: Exception
connections.signalDeadActor(actor) connections.remove(actor)
throw e throw e
} }
case None case None
@ -505,7 +505,7 @@ trait ScatterGatherRouter extends BasicRouter with Serializable {
Some(actor.?(message, timeout)(sender).asInstanceOf[Future[S]]) Some(actor.?(message, timeout)(sender).asInstanceOf[Future[S]])
} catch { } catch {
case e: Exception case e: Exception
connections.signalDeadActor(actor) connections.remove(actor)
None None
} }
} }

View file

@ -6,7 +6,7 @@ package akka.util
import akka.dispatch.MessageInvocation import akka.dispatch.MessageInvocation
import akka.config.{ Config, ModuleNotAvailableException } import akka.config.{ Config, ModuleNotAvailableException }
import akka.remoteinterface.RemoteSupport import akka.cluster.RemoteSupport
import akka.actor._ import akka.actor._
import DeploymentConfig.ReplicationScheme import DeploymentConfig.ReplicationScheme
import akka.event.EventHandler import akka.event.EventHandler
@ -122,7 +122,7 @@ object ReflectiveAccess {
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
object RemoteModule { object RemoteModule {
val TRANSPORT = Config.config.getString("akka.remote.layer", "akka.remote.netty.NettyRemoteSupport") val TRANSPORT = Config.config.getString("akka.cluster.layer", "akka.cluster.netty.NettyRemoteSupport")
val configDefaultAddress = new InetSocketAddress(Config.hostname, Config.remoteServerPort) val configDefaultAddress = new InetSocketAddress(Config.hostname, Config.remoteServerPort)

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/ */
option java_package = "akka.remote.protocol"; option java_package = "akka.cluster.protocol";
option optimize_for = SPEED; option optimize_for = SPEED;
/****************************************** /******************************************

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.remote package akka.cluster
import akka.actor.{ Actor, BootableActorLoaderService } import akka.actor.{ Actor, BootableActorLoaderService }
import akka.util.{ ReflectiveAccess, Bootable } import akka.util.{ ReflectiveAccess, Bootable }

View file

@ -30,7 +30,8 @@ import DeploymentConfig._
import akka.event.EventHandler import akka.event.EventHandler
import akka.dispatch.{ Dispatchers, Future } import akka.dispatch.{ Dispatchers, Future }
import akka.remoteinterface._ import akka.cluster._
import akka.cluster._
import akka.routing.RouterType import akka.routing.RouterType
import akka.config.{ Config, Supervision } import akka.config.{ Config, Supervision }
@ -277,7 +278,7 @@ class DefaultClusterNode private[akka] (
// private val connectToAllNewlyArrivedMembershipNodesInClusterLock = new AtomicBoolean(false) // private val connectToAllNewlyArrivedMembershipNodesInClusterLock = new AtomicBoolean(false)
private[cluster] lazy val remoteClientLifeCycleListener = localActorOf(new Actor { private[cluster] lazy val remoteClientLifeCycleHandler = localActorOf(new Actor {
def receive = { def receive = {
case RemoteClientError(cause, client, address) client.shutdownClientModule() case RemoteClientError(cause, client, address) client.shutdownClientModule()
case RemoteClientDisconnected(client, address) client.shutdownClientModule() case RemoteClientDisconnected(client, address) client.shutdownClientModule()
@ -296,10 +297,11 @@ class DefaultClusterNode private[akka] (
:: Nil)) :: Nil))
lazy val remoteService: RemoteSupport = { lazy val remoteService: RemoteSupport = {
val remote = new akka.remote.netty.NettyRemoteSupport val remote = new akka.cluster.netty.NettyRemoteSupport
remote.start(hostname, port) remote.start(hostname, port)
remote.register(RemoteClusterDaemon.Address, remoteDaemon) remote.register(RemoteClusterDaemon.Address, remoteDaemon)
remote.addListener(remoteClientLifeCycleListener) remote.addListener(FailureDetector.registry)
remote.addListener(remoteClientLifeCycleHandler)
remote remote
} }
@ -421,7 +423,8 @@ class DefaultClusterNode private[akka] (
remoteService.shutdown() // shutdown server remoteService.shutdown() // shutdown server
remoteClientLifeCycleListener.stop() FailureDetector.registry.stop()
remoteClientLifeCycleHandler.stop()
remoteDaemon.stop() remoteDaemon.stop()
// for monitoring remote listener // for monitoring remote listener
@ -1756,8 +1759,8 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
def handleUse(message: ClusterProtocol.RemoteDaemonMessageProtocol) { def handleUse(message: ClusterProtocol.RemoteDaemonMessageProtocol) {
def deserializeMessages(entriesAsBytes: Vector[Array[Byte]]): Vector[AnyRef] = { def deserializeMessages(entriesAsBytes: Vector[Array[Byte]]): Vector[AnyRef] = {
import akka.remote.protocol.RemoteProtocol._ import akka.cluster.protocol.RemoteProtocol._
import akka.remote.MessageSerializer import akka.cluster.MessageSerializer
entriesAsBytes map { bytes entriesAsBytes map { bytes
val messageBytes = val messageBytes =

View file

@ -10,6 +10,7 @@ import ReflectiveAccess._
import akka.dispatch.Future import akka.dispatch.Future
import akka.routing._ import akka.routing._
import RouterType._ import RouterType._
import akka.cluster._
import java.net.InetSocketAddress import java.net.InetSocketAddress
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
@ -45,6 +46,10 @@ object ClusterActorRef {
* Finds the cluster actor reference that has a specific address. * Finds the cluster actor reference that has a specific address.
*/ */
def actorFor(address: String): Option[ActorRef] = Actor.registry.local.actorFor(Address.clusterActorRefPrefix + address) def actorFor(address: String): Option[ActorRef] = Actor.registry.local.actorFor(Address.clusterActorRefPrefix + address)
private[cluster] def createRemoteActorRef(actorAddress: String, inetSocketAddress: InetSocketAddress) = {
RemoteActorRef(inetSocketAddress, actorAddress, Actor.TIMEOUT, None)
}
} }
/** /**
@ -53,12 +58,15 @@ object ClusterActorRef {
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class ClusterActorRef private[akka] (inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]], class ClusterActorRef private[akka] (
inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]],
_address: String, _address: String,
_timeout: Long, _timeout: Long,
val router: Router) val router: Router)
extends UnsupportedActorRef { extends UnsupportedActorRef {
import ClusterActorRef._
ClusterModule.ensureEnabled() ClusterModule.ensureEnabled()
// val address = Address.clusterActorRefPrefix + _address // val address = Address.clusterActorRefPrefix + _address
@ -92,10 +100,6 @@ class ClusterActorRef private[akka] (inetSocketAddresses: Array[Tuple2[UUID, Ine
router.route[Any](message, timeout.duration.toMillis)(sender) router.route[Any](message, timeout.duration.toMillis)(sender)
} }
private def createRemoteActorRef(actorAddress: String, inetSocketAddress: InetSocketAddress) = {
RemoteActorRef(inetSocketAddress, actorAddress, Actor.TIMEOUT, None)
}
private[akka] def failOver(from: InetSocketAddress, to: InetSocketAddress): Unit = { private[akka] def failOver(from: InetSocketAddress, to: InetSocketAddress): Unit = {
connections.failOver(from, to) connections.failOver(from, to)
} }
@ -120,8 +124,10 @@ class ClusterActorRef private[akka] (inetSocketAddresses: Array[Tuple2[UUID, Ine
} }
} }
} }
}
class ClusterActorRefConnections() extends RouterConnections { class ClusterActorRefConnections() extends RouterConnections {
import ClusterActorRef._
private val state = new AtomicReference[State]() private val state = new AtomicReference[State]()
@ -132,6 +138,8 @@ class ClusterActorRef private[akka] (inetSocketAddresses: Array[Tuple2[UUID, Ine
def version: Long = state.get().version def version: Long = state.get().version
def connections: Map[InetSocketAddress, ActorRef] = state.get.connections
def versionedIterator = { def versionedIterator = {
val s = state.get val s = state.get
(s.version, s.connections.values) (s.version, s.connections.values)
@ -145,7 +153,7 @@ class ClusterActorRef private[akka] (inetSocketAddresses: Array[Tuple2[UUID, Ine
@tailrec @tailrec
final def failOver(from: InetSocketAddress, to: InetSocketAddress): Unit = { final def failOver(from: InetSocketAddress, to: InetSocketAddress): Unit = {
EventHandler.debug(this, "ClusterActorRef [%s] failover from [%s] to [%s]".format(address, from, to)) EventHandler.debug(this, "ClusterActorRef failover from [%s] to [%s]".format(from, to))
val oldState = state.get val oldState = state.get
var change = false var change = false
@ -167,8 +175,8 @@ class ClusterActorRef private[akka] (inetSocketAddresses: Array[Tuple2[UUID, Ine
} }
@tailrec @tailrec
final def signalDeadActor(deadRef: ActorRef) = { final def remove(deadRef: ActorRef) = {
EventHandler.debug(this, "ClusterActorRef [%s] signalDeadActor [%s]".format(uuid, deadRef.uuid)) EventHandler.debug(this, "ClusterActorRef remove [%s]".format(deadRef.uuid))
val oldState = state.get() val oldState = state.get()
@ -185,10 +193,9 @@ class ClusterActorRef private[akka] (inetSocketAddresses: Array[Tuple2[UUID, Ine
val newState = new State(oldState.version + 1, newConnections) val newState = new State(oldState.version + 1, newConnections)
//if we are not able to update the state, we just try again. //if we are not able to update the state, we just try again.
if (!state.compareAndSet(oldState, newState)) signalDeadActor(deadRef) if (!state.compareAndSet(oldState, newState)) remove(deadRef)
} }
} }
case class State(val version: Long, val connections: Map[InetSocketAddress, ActorRef]) case class State(val version: Long, val connections: Map[InetSocketAddress, ActorRef])
} }
}

View file

@ -2,9 +2,9 @@
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.remote package akka.cluster
import akka.remote.protocol.RemoteProtocol._ import akka.cluster.protocol.RemoteProtocol._
import akka.serialization.Serialization import akka.serialization.Serialization
import com.google.protobuf.ByteString import com.google.protobuf.ByteString

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.remote package akka.cluster
import akka.util.Duration import akka.util.Duration
import akka.config.Config._ import akka.config.Config._

View file

@ -17,7 +17,7 @@ import akka.actor._
import DeploymentConfig.ReplicationScheme import DeploymentConfig.ReplicationScheme
import akka.event.EventHandler import akka.event.EventHandler
import akka.dispatch.{ DefaultPromise, Promise, MessageInvocation } import akka.dispatch.{ DefaultPromise, Promise, MessageInvocation }
import akka.remote.MessageSerializer import akka.cluster.MessageSerializer
import akka.cluster.zookeeper._ import akka.cluster.zookeeper._
import akka.serialization.ActorSerialization._ import akka.serialization.ActorSerialization._
import akka.serialization.Compression.LZF import akka.serialization.Compression.LZF

View file

@ -2,14 +2,14 @@
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.remote.netty package akka.cluster.netty
import akka.dispatch.{ ActorPromise, DefaultPromise, Promise } import akka.dispatch.{ ActorPromise, DefaultPromise, Promise }
import akka.remote.{ MessageSerializer, RemoteClientSettings, RemoteServerSettings } import akka.cluster.{ MessageSerializer, RemoteClientSettings, RemoteServerSettings }
import akka.remote.protocol.RemoteProtocol._ import akka.cluster.protocol.RemoteProtocol._
import akka.serialization.RemoteActorSerialization import akka.serialization.RemoteActorSerialization
import akka.serialization.RemoteActorSerialization._ import akka.serialization.RemoteActorSerialization._
import akka.remoteinterface._ import akka.cluster._
import akka.actor.{ import akka.actor.{
PoisonPill, PoisonPill,
Actor, Actor,
@ -80,7 +80,7 @@ trait NettyRemoteClientModule extends RemoteClientModule {
isOneWay: Boolean, isOneWay: Boolean,
actorRef: ActorRef, actorRef: ActorRef,
loader: Option[ClassLoader]): Option[Promise[T]] = loader: Option[ClassLoader]): Option[Promise[T]] =
withClientFor(remoteAddress, loader) { client => withClientFor(remoteAddress, loader) { client
client.send[T](message, senderOption, senderFuture, remoteAddress, timeout, isOneWay, actorRef) client.send[T](message, senderOption, senderFuture, remoteAddress, timeout, isOneWay, actorRef)
} }
@ -159,7 +159,7 @@ abstract class RemoteClient private[akka] (
val module: NettyRemoteClientModule, val module: NettyRemoteClientModule,
val remoteAddress: InetSocketAddress) { val remoteAddress: InetSocketAddress) {
val useTransactionLog = config.getBool("akka.cluster.client.buffering.retry-message-send-on-failure", true) val useTransactionLog = config.getBool("akka.cluster.client.buffering.retry-message-send-on-failure", false)
val transactionLogCapacity = config.getInt("akka.cluster.client.buffering.capacity", -1) val transactionLogCapacity = config.getInt("akka.cluster.client.buffering.capacity", -1)
val name = this.getClass.getSimpleName + "@" + val name = this.getClass.getSimpleName + "@" +
@ -172,9 +172,9 @@ abstract class RemoteClient private[akka] (
else new LinkedBlockingQueue[(Boolean, Uuid, RemoteMessageProtocol)](transactionLogCapacity) else new LinkedBlockingQueue[(Boolean, Uuid, RemoteMessageProtocol)](transactionLogCapacity)
} }
private[remote] val runSwitch = new Switch() private[cluster] val runSwitch = new Switch()
private[remote] def isRunning = runSwitch.isOn private[cluster] def isRunning = runSwitch.isOn
protected def notifyListeners(msg: Any): Unit protected def notifyListeners(msg: Any): Unit
@ -207,10 +207,11 @@ abstract class RemoteClient private[akka] (
remoteAddress: InetSocketAddress, remoteAddress: InetSocketAddress,
timeout: Long, timeout: Long,
isOneWay: Boolean, isOneWay: Boolean,
actorRef: ActorRef): Option[Promise[T]] = actorRef: ActorRef): Option[Promise[T]] = {
send(createRemoteMessageProtocolBuilder( val messageProtocol = createRemoteMessageProtocolBuilder(
Some(actorRef), Left(actorRef.uuid), actorRef.address, timeout, Right(message), isOneWay, senderOption).build, Some(actorRef), Left(actorRef.uuid), actorRef.address, timeout, Right(message), isOneWay, senderOption).build
senderFuture) send(messageProtocol, senderFuture)
}
/** /**
* Sends the message across the wire * Sends the message across the wire
@ -222,25 +223,26 @@ abstract class RemoteClient private[akka] (
if (isRunning) { if (isRunning) {
EventHandler.debug(this, "Sending to connection [%s] message [\n%s]".format(remoteAddress, request)) EventHandler.debug(this, "Sending to connection [%s] message [\n%s]".format(remoteAddress, request))
// tell
if (request.getOneWay) { if (request.getOneWay) {
try { try {
val future = currentChannel.write(RemoteEncoder.encode(request)) val future = currentChannel.write(RemoteEncoder.encode(request))
future.awaitUninterruptibly() future.awaitUninterruptibly()
if (!future.isCancelled && !future.isSuccess) { if (!future.isCancelled && !future.isSuccess) {
notifyListeners(RemoteClientWriteFailed(request, future.getCause, module, remoteAddress)) notifyListeners(RemoteClientWriteFailed(request, future.getCause, module, remoteAddress))
throw future.getCause
} }
} catch { } catch {
case e: Throwable case e: Exception
// add the request to the tx log after a failing send
notifyListeners(RemoteClientError(e, module, remoteAddress)) notifyListeners(RemoteClientError(e, module, remoteAddress))
if (useTransactionLog) {
if (!pendingRequests.offer((true, null, request))) if (useTransactionLog && !pendingRequests.offer((true, null, request))) { // Add the request to the tx log after a failing send
pendingRequests.clear()
throw new RemoteClientMessageBufferException("Buffer limit [" + transactionLogCapacity + "] reached") throw new RemoteClientMessageBufferException("Buffer limit [" + transactionLogCapacity + "] reached")
} else throw e }
} }
None None
// ask
} else { } else {
val futureResult = val futureResult =
if (senderFuture.isDefined) senderFuture.get if (senderFuture.isDefined) senderFuture.get
@ -250,9 +252,10 @@ abstract class RemoteClient private[akka] (
futures.put(futureUuid, futureResult) // Add future prematurely, remove it if write fails futures.put(futureUuid, futureResult) // Add future prematurely, remove it if write fails
def handleRequestReplyError(future: ChannelFuture) = { def handleRequestReplyError(future: ChannelFuture) = {
if (useTransactionLog) { if (useTransactionLog && !pendingRequests.offer((false, futureUuid, request))) { // Add the request to the tx log after a failing send
if (!pendingRequests.offer((false, futureUuid, request))) // Add the request to the tx log after a failing send pendingRequests.clear()
throw new RemoteClientMessageBufferException("Buffer limit [" + transactionLogCapacity + "] reached") throw new RemoteClientMessageBufferException("Buffer limit [" + transactionLogCapacity + "] reached")
} else { } else {
val f = futures.remove(futureUuid) // Clean up future val f = futures.remove(futureUuid) // Clean up future
if (f ne null) f.completeWithException(future.getCause) if (f ne null) f.completeWithException(future.getCause)
@ -264,53 +267,57 @@ abstract class RemoteClient private[akka] (
// try to send the original one // try to send the original one
future = currentChannel.write(RemoteEncoder.encode(request)) future = currentChannel.write(RemoteEncoder.encode(request))
future.awaitUninterruptibly() future.awaitUninterruptibly()
if (future.isCancelled) futures.remove(futureUuid) // Clean up future
else if (!future.isSuccess) { if (future.isCancelled || !future.isSuccess) {
notifyListeners(RemoteClientWriteFailed(request, future.getCause, module, remoteAddress)) notifyListeners(RemoteClientWriteFailed(request, future.getCause, module, remoteAddress))
handleRequestReplyError(future) handleRequestReplyError(future)
} }
} catch { } catch {
case e: Exception case e: Exception
notifyListeners(RemoteClientWriteFailed(request, e, module, remoteAddress)) notifyListeners(RemoteClientWriteFailed(request, e, module, remoteAddress))
handleRequestReplyError(future) handleRequestReplyError(future)
throw e
} }
Some(futureResult) Some(futureResult)
} }
} else { } else {
val exception = new RemoteClientException("RemoteModule client is not running, make sure you have invoked 'RemoteClient.connect' before using it.", module, remoteAddress) val exception = new RemoteClientException("RemoteModule client is not running, make sure you have invoked 'RemoteClient.connect()' before using it.", module, remoteAddress)
notifyListeners(RemoteClientError(exception, module, remoteAddress)) notifyListeners(RemoteClientError(exception, module, remoteAddress))
throw exception throw exception
} }
} }
private[remote] def sendPendingRequests() = pendingRequests synchronized { private[cluster] def sendPendingRequests() = pendingRequests synchronized {
// ensure only one thread at a time can flush the log // ensure only one thread at a time can flush the log
val nrOfMessages = pendingRequests.size val nrOfMessages = pendingRequests.size
if (nrOfMessages > 0) EventHandler.info(this, "Resending [%s] previously failed messages after remote client reconnect" format nrOfMessages) if (nrOfMessages > 0) EventHandler.info(this, "Resending [%s] previously failed messages after remote client reconnect" format nrOfMessages)
var pendingRequest = pendingRequests.peek var pendingRequest = pendingRequests.peek
while (pendingRequest ne null) { while (pendingRequest ne null) {
val (isOneWay, futureUuid, message) = pendingRequest val (isOneWay, futureUuid, message) = pendingRequest
if (isOneWay) { if (isOneWay) {
// tell // tell
val future = currentChannel.write(RemoteEncoder.encode(message)) val future = currentChannel.write(RemoteEncoder.encode(message))
future.awaitUninterruptibly() future.awaitUninterruptibly()
if (!future.isCancelled && !future.isSuccess) {
if (future.isCancelled && !future.isSuccess) {
notifyListeners(RemoteClientWriteFailed(message, future.getCause, module, remoteAddress)) notifyListeners(RemoteClientWriteFailed(message, future.getCause, module, remoteAddress))
throw future.getCause
} }
} else { } else {
// sendRequestReply // ask
val future = currentChannel.write(RemoteEncoder.encode(message)) val future = currentChannel.write(RemoteEncoder.encode(message))
future.awaitUninterruptibly() future.awaitUninterruptibly()
if (future.isCancelled) futures.remove(futureUuid) // Clean up future
else if (!future.isSuccess) { if (future.isCancelled || !future.isSuccess) {
val f = futures.remove(futureUuid) // Clean up future val f = futures.remove(futureUuid) // Clean up future
if (f ne null) f.completeWithException(future.getCause) if (f ne null) f.completeWithException(future.getCause)
notifyListeners(RemoteClientWriteFailed(message, future.getCause, module, remoteAddress)) notifyListeners(RemoteClientWriteFailed(message, future.getCause, module, remoteAddress))
} }
} }
pendingRequests.remove(pendingRequest) pendingRequests.remove(pendingRequest)
pendingRequest = pendingRequests.peek // try to grab next message pendingRequest = pendingRequests.peek // try to grab next message
} }
@ -332,9 +339,9 @@ class ActiveRemoteClient private[akka] (
@volatile @volatile
private var bootstrap: ClientBootstrap = _ private var bootstrap: ClientBootstrap = _
@volatile @volatile
private[remote] var connection: ChannelFuture = _ private[cluster] var connection: ChannelFuture = _
@volatile @volatile
private[remote] var openChannels: DefaultChannelGroup = _ private[cluster] var openChannels: DefaultChannelGroup = _
@volatile @volatile
private var timer: HashedWheelTimer = _ private var timer: HashedWheelTimer = _
@volatile @volatile
@ -344,7 +351,40 @@ class ActiveRemoteClient private[akka] (
def currentChannel = connection.getChannel def currentChannel = connection.getChannel
/**
* Connect to remote server.
*/
def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = { def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = {
def sendSecureCookie(connection: ChannelFuture) {
val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT)
if (SECURE_COOKIE.nonEmpty) handshake.setCookie(SECURE_COOKIE.get)
connection.getChannel.write(RemoteEncoder.encode(handshake.build))
}
def closeChannel(connection: ChannelFuture) = {
val channel = connection.getChannel
openChannels.remove(channel)
channel.close
}
def attemptReconnect(): Boolean = {
EventHandler.debug(this, "Remote client reconnecting to [%s]".format(remoteAddress))
val connection = bootstrap.connect(remoteAddress)
openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails.
if (!connection.isSuccess) {
notifyListeners(RemoteClientError(connection.getCause, module, remoteAddress))
EventHandler.error(connection.getCause, this, "Reconnection to [%s] has failed".format(remoteAddress))
false
} else {
sendSecureCookie(connection)
true
}
}
runSwitch switchOn { runSwitch switchOn {
openChannels = new DefaultDisposableChannelGroup(classOf[RemoteClient].getName) openChannels = new DefaultDisposableChannelGroup(classOf[RemoteClient].getName)
timer = new HashedWheelTimer timer = new HashedWheelTimer
@ -356,29 +396,18 @@ class ActiveRemoteClient private[akka] (
EventHandler.debug(this, "Starting remote client connection to [%s]".format(remoteAddress)) EventHandler.debug(this, "Starting remote client connection to [%s]".format(remoteAddress))
// Wait until the connection attempt succeeds or fails.
try {
connection = bootstrap.connect(remoteAddress) connection = bootstrap.connect(remoteAddress)
} catch {
case e: Exception
EventHandler.error(e, this, "Remote client failed to connect to [%s]".format(remoteAddress))
throw e
}
openChannels.add(connection.awaitUninterruptibly.getChannel) val channel = connection.awaitUninterruptibly.getChannel
openChannels.add(channel)
if (!connection.isSuccess) { if (!connection.isSuccess) {
notifyListeners(RemoteClientError(connection.getCause, module, remoteAddress)) notifyListeners(RemoteClientError(connection.getCause, module, remoteAddress))
EventHandler.error(connection.getCause, this, "Remote client connection to [%s] has failed".format(remoteAddress)) EventHandler.error(connection.getCause, this, "Remote client connection to [%s] has failed".format(remoteAddress))
false false
} else {
//Send cookie
val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT)
if (SECURE_COOKIE.nonEmpty)
handshake.setCookie(SECURE_COOKIE.get)
connection.getChannel.write(RemoteEncoder.encode(handshake.build)) } else {
sendSecureCookie(connection)
//Add a task that does GCing of expired Futures //Add a task that does GCing of expired Futures
timer.newTimeout(new TimerTask() { timer.newTimeout(new TimerTask() {
@ -399,26 +428,11 @@ class ActiveRemoteClient private[akka] (
} match { } match {
case true true case true true
case false if reconnectIfAlreadyConnected case false if reconnectIfAlreadyConnected
closeChannel(connection)
EventHandler.debug(this, "Remote client reconnecting to [%s]".format(remoteAddress)) EventHandler.debug(this, "Remote client reconnecting to [%s]".format(remoteAddress))
attemptReconnect()
openChannels.remove(connection.getChannel)
connection.getChannel.close
connection = bootstrap.connect(remoteAddress)
openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails.
if (!connection.isSuccess) {
notifyListeners(RemoteClientError(connection.getCause, module, remoteAddress))
EventHandler.error(connection.getCause, this, "Reconnection to [%s] has failed".format(remoteAddress))
false
} else {
//Send cookie
val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT)
if (SECURE_COOKIE.nonEmpty)
handshake.setCookie(SECURE_COOKIE.get)
connection.getChannel.write(RemoteEncoder.encode(handshake.build))
true
}
case false false case false false
} }
} }
@ -511,8 +525,7 @@ class ActiveRemoteClientHandler(
case arp: AkkaRemoteProtocol if arp.hasMessage case arp: AkkaRemoteProtocol if arp.hasMessage
val reply = arp.getMessage val reply = arp.getMessage
val replyUuid = uuidFrom(reply.getActorInfo.getUuid.getHigh, reply.getActorInfo.getUuid.getLow) val replyUuid = uuidFrom(reply.getActorInfo.getUuid.getHigh, reply.getActorInfo.getUuid.getLow)
EventHandler.debug(this, "Remote client received RemoteMessageProtocol[\n%s]".format(reply)) EventHandler.debug(this, "Remote client received RemoteMessageProtocol[\n%s]\nTrying to map back to future: %s".format(reply, replyUuid))
EventHandler.debug(this, "Trying to map back to future: %s".format(replyUuid))
futures.remove(replyUuid).asInstanceOf[Promise[Any]] match { futures.remove(replyUuid).asInstanceOf[Promise[Any]] match {
case null case null
@ -531,10 +544,9 @@ class ActiveRemoteClientHandler(
throw new RemoteClientException("Unknown message received in remote client handler: " + other, client.module, client.remoteAddress) throw new RemoteClientException("Unknown message received in remote client handler: " + other, client.module, client.remoteAddress)
} }
} catch { } catch {
case e: Throwable case e: Exception
EventHandler.error(e, this, e.getMessage) EventHandler.error(e, this, e.getMessage)
client.notifyListeners(RemoteClientError(e, client.module, client.remoteAddress)) client.notifyListeners(RemoteClientError(e, client.module, client.remoteAddress))
throw e
} }
} }
@ -549,7 +561,7 @@ class ActiveRemoteClientHandler(
} }
}, RemoteClientSettings.RECONNECT_DELAY.toMillis, TimeUnit.MILLISECONDS) }, RemoteClientSettings.RECONNECT_DELAY.toMillis, TimeUnit.MILLISECONDS)
} else spawn { } else spawn {
client.module.shutdownClientConnection(remoteAddress) client.module.shutdownClientConnection(remoteAddress) // spawn in another thread
} }
} }
@ -560,10 +572,9 @@ class ActiveRemoteClientHandler(
EventHandler.debug(this, "Remote client connected to [%s]".format(ctx.getChannel.getRemoteAddress)) EventHandler.debug(this, "Remote client connected to [%s]".format(ctx.getChannel.getRemoteAddress))
client.resetReconnectionTimeWindow client.resetReconnectionTimeWindow
} catch { } catch {
case e: Throwable case e: Exception
EventHandler.error(e, this, e.getMessage) EventHandler.error(e, this, e.getMessage)
client.notifyListeners(RemoteClientError(e, client.module, client.remoteAddress)) client.notifyListeners(RemoteClientError(e, client.module, client.remoteAddress))
throw e
} }
} }
@ -573,20 +584,21 @@ class ActiveRemoteClientHandler(
} }
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
if (event.getCause ne null) val cause = event.getCause
EventHandler.error(event.getCause, this, "Unexpected exception from downstream in remote client: %s".format(event)) if (cause ne null) {
else EventHandler.error(event.getCause, this, "Unexpected exception [%s] from downstream in remote client [%s]".format(event.getCause, event))
EventHandler.error(this, "Unexpected exception from downstream in remote client: %s".format(event))
event.getCause match { cause match {
case e: ReadTimeoutException case e: ReadTimeoutException
spawn { spawn {
client.module.shutdownClientConnection(remoteAddress) client.module.shutdownClientConnection(remoteAddress) // spawn in another thread
} }
case e: Exception case e: Exception
client.notifyListeners(RemoteClientError(e, client.module, client.remoteAddress)) client.notifyListeners(RemoteClientError(e, client.module, client.remoteAddress))
event.getChannel.close //FIXME Is this the correct behavior? event.getChannel.close //FIXME Is this the correct behavior?
} }
} else EventHandler.error(this, "Unexpected exception from downstream in remote client [%s]".format(event))
} }
private def parseException(reply: RemoteMessageProtocol, loader: Option[ClassLoader]): Throwable = { private def parseException(reply: RemoteMessageProtocol, loader: Option[ClassLoader]): Throwable = {
@ -872,7 +884,7 @@ class RemoteServerAuthenticationHandler(secureCookie: Option[String]) extends Si
"The remote client [" + ctx.getChannel.getRemoteAddress + "] secure cookie is not the same as remote server secure cookie") "The remote client [" + ctx.getChannel.getRemoteAddress + "] secure cookie is not the same as remote server secure cookie")
} }
case _ case _
throw new SecurityException("The remote client [" + ctx.getChannel.getRemoteAddress + "] is not Authorized!") throw new SecurityException("The remote client [" + ctx.getChannel.getRemoteAddress + "] is not authorized!")
} }
} }
} }
@ -957,9 +969,12 @@ class RemoteServerHandler(
event.getMessage match { event.getMessage match {
case null case null
throw new IllegalActorStateException("Message in remote MessageEvent is null: " + event) throw new IllegalActorStateException("Message in remote MessageEvent is null: " + event)
case remote: AkkaRemoteProtocol if remote.hasMessage case remote: AkkaRemoteProtocol if remote.hasMessage
handleRemoteMessageProtocol(remote.getMessage, event.getChannel) handleRemoteMessageProtocol(remote.getMessage, event.getChannel)
//case remote: AkkaRemoteProtocol if remote.hasInstruction => RemoteServer cannot receive control messages (yet) //case remote: AkkaRemoteProtocol if remote.hasInstruction => RemoteServer cannot receive control messages (yet)
case _ //ignore case _ //ignore
} }
} }
@ -1065,8 +1080,8 @@ class RemoteServerHandler(
val actorRef = Actor.createActor(address, () createSessionActor(actorInfo, channel)) val actorRef = Actor.createActor(address, () createSessionActor(actorInfo, channel))
if (actorRef eq null) throw new IllegalActorStateException( if (actorRef eq null) throw new IllegalActorStateException("Could not find a remote actor with address [" + address + "] or uuid [" + uuid + "]")
"Could not find a remote actor with address [" + address + "] or uuid [" + uuid + "]")
actorRef actorRef
} }

View file

@ -10,8 +10,8 @@ import akka.actor._
import DeploymentConfig._ import DeploymentConfig._
import akka.dispatch.MessageInvocation import akka.dispatch.MessageInvocation
import akka.util.ReflectiveAccess import akka.util.ReflectiveAccess
import akka.remote.{ RemoteClientSettings, MessageSerializer } import akka.cluster.{ RemoteClientSettings, MessageSerializer }
import akka.remote.protocol.RemoteProtocol import akka.cluster.protocol.RemoteProtocol
import RemoteProtocol._ import RemoteProtocol._
import scala.collection.immutable.Stack import scala.collection.immutable.Stack

View file

@ -503,7 +503,7 @@ So a simple listener actor can look like this:
.. code-block:: java .. code-block:: java
import akka.actor.UntypedActor; import akka.actor.UntypedActor;
import akka.remoteinterface.*; import akka.cluster.*;
class Listener extends UntypedActor { class Listener extends UntypedActor {
@ -561,7 +561,7 @@ So a simple listener actor can look like this:
.. code-block:: java .. code-block:: java
import akka.actor.UntypedActor; import akka.actor.UntypedActor;
import akka.remoteinterface.*; import akka.cluster.*;
class Listener extends UntypedActor { class Listener extends UntypedActor {

View file

@ -18,7 +18,7 @@ If you deploy Akka in a JEE container, don't forget to create an Akka initializa
package com.my //<--- your own package package com.my //<--- your own package
import akka.util.AkkaLoader import akka.util.AkkaLoader
import akka.remote.BootableRemoteActorService import akka.cluster.BootableRemoteActorService
import akka.actor.BootableActorLoaderService import akka.actor.BootableActorLoaderService
import javax.servlet.{ServletContextListener, ServletContextEvent} import javax.servlet.{ServletContextListener, ServletContextEvent}
@ -63,7 +63,7 @@ If you want to use akka-camel or any other modules that have their own "Bootable
.. code-block:: scala .. code-block:: scala
package com.my //<--- your own package package com.my //<--- your own package
import akka.remote.BootableRemoteActorService import akka.cluster.BootableRemoteActorService
import akka.actor.BootableActorLoaderService import akka.actor.BootableActorLoaderService
import akka.camel.CamelService import akka.camel.CamelService
import javax.servlet.{ServletContextListener, ServletContextEvent} import javax.servlet.{ServletContextListener, ServletContextEvent}

View file

@ -580,7 +580,7 @@ So a simple listener actor can look like this:
import akka.actor.Actor import akka.actor.Actor
import akka.actor.Actor._ import akka.actor.Actor._
import akka.remoteinterface._ import akka.cluster._
val listener = actorOf(new Actor { val listener = actorOf(new Actor {
def receive = { def receive = {
@ -638,7 +638,7 @@ So a simple listener actor can look like this:
import akka.actor.Actor import akka.actor.Actor
import akka.actor.Actor._ import akka.actor.Actor._
import akka.remoteinterface._ import akka.cluster._
val listener = actorOf(new Actor { val listener = actorOf(new Actor {
def receive = { def receive = {

View file

@ -8,8 +8,8 @@ import MailboxProtocol._
import akka.actor.{ Actor, ActorRef, NullChannel } import akka.actor.{ Actor, ActorRef, NullChannel }
import akka.dispatch._ import akka.dispatch._
import akka.event.EventHandler import akka.event.EventHandler
import akka.remote.MessageSerializer import akka.cluster.MessageSerializer
import akka.remote.protocol.RemoteProtocol.MessageProtocol import akka.cluster.protocol.RemoteProtocol.MessageProtocol
import akka.AkkaException import akka.AkkaException
/** /**

View file

@ -8,8 +8,8 @@ import akka.config.Config.config
import akka.dispatch._ import akka.dispatch._
import akka.event.EventHandler import akka.event.EventHandler
import akka.AkkaException import akka.AkkaException
import akka.remote.MessageSerializer import akka.cluster.MessageSerializer
import akka.remote.protocol.RemoteProtocol.MessageProtocol import akka.cluster.protocol.RemoteProtocol.MessageProtocol
import MailboxProtocol._ import MailboxProtocol._

View file

@ -6,7 +6,7 @@ package akka.http
import akka.config.Config import akka.config.Config
import akka.util.{ Bootable, AkkaLoader } import akka.util.{ Bootable, AkkaLoader }
import akka.remote.BootableRemoteActorService import akka.cluster.BootableRemoteActorService
import akka.actor.BootableActorLoaderService import akka.actor.BootableActorLoaderService
class DefaultAkkaLoader extends AkkaLoader { class DefaultAkkaLoader extends AkkaLoader {

View file

@ -6,7 +6,7 @@ package akka.kernel
import akka.http.EmbeddedAppServer import akka.http.EmbeddedAppServer
import akka.util.AkkaLoader import akka.util.AkkaLoader
import akka.remote.BootableRemoteActorService import akka.cluster.BootableRemoteActorService
import akka.actor.BootableActorLoaderService import akka.actor.BootableActorLoaderService
import akka.camel.CamelService import akka.camel.CamelService

View file

@ -4,7 +4,7 @@
package akka.servlet package akka.servlet
import akka.remote.BootableRemoteActorService import akka.cluster.BootableRemoteActorService
import akka.actor.BootableActorLoaderService import akka.actor.BootableActorLoaderService
import akka.config.Config import akka.config.Config
import akka.util.{ Bootable, AkkaLoader } import akka.util.{ Bootable, AkkaLoader }

View file

@ -5,8 +5,8 @@ import org.scalatest.{ GivenWhenThen, BeforeAndAfterAll, FeatureSpec }
import akka.actor.Actor._ import akka.actor.Actor._
import akka.actor._ import akka.actor._
import akka.camel._ import akka.camel._
//import akka.remote.netty.NettyRemoteSupport //import akka.cluster.netty.NettyRemoteSupport
//import akka.remoteinterface.RemoteServerModule //import akka.cluster.RemoteServerModule
/** /**
* @author Martin Krasser * @author Martin Krasser

View file

@ -131,7 +131,7 @@ akka {
compression-scheme = "zlib" # Options: "zlib" (lzf to come), leave out for no compression compression-scheme = "zlib" # Options: "zlib" (lzf to come), leave out for no compression
zlib-compression-level = 6 # Options: 0-9 (1 being fastest and 9 being the most compressed), default is 6 zlib-compression-level = 6 # Options: 0-9 (1 being fastest and 9 being the most compressed), default is 6
layer = "akka.remote.netty.NettyRemoteSupport" layer = "akka.cluster.netty.NettyRemoteSupport"
server { server {
hostname = "localhost" # The hostname or IP that clients should connect to hostname = "localhost" # The hostname or IP that clients should connect to

View file

@ -15,7 +15,7 @@ import org.springframework.context.support.ClassPathXmlApplicationContext
import org.springframework.core.io.{ ClassPathResource, Resource } import org.springframework.core.io.{ ClassPathResource, Resource }
import org.scalatest.{ BeforeAndAfterAll, FeatureSpec } import org.scalatest.{ BeforeAndAfterAll, FeatureSpec }
import java.util.concurrent.CountDownLatch import java.util.concurrent.CountDownLatch
import akka.remote.netty.NettyRemoteSupport import akka.cluster.netty.NettyRemoteSupport
import akka.actor._ import akka.actor._
import akka.actor.Actor._ import akka.actor.Actor._

View file

@ -9,7 +9,7 @@ import org.scalatest.matchers.ShouldMatchers
import org.scalatest.junit.JUnitRunner import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith import org.junit.runner.RunWith
import org.springframework.context.support.ClassPathXmlApplicationContext import org.springframework.context.support.ClassPathXmlApplicationContext
import akka.remote.netty.NettyRemoteSupport import akka.cluster.netty.NettyRemoteSupport
import org.scalatest.{ BeforeAndAfterAll, FeatureSpec } import org.scalatest.{ BeforeAndAfterAll, FeatureSpec }
import java.util.concurrent.CountDownLatch import java.util.concurrent.CountDownLatch

View file

@ -195,7 +195,7 @@ akka {
zlib-compression-level = 6 # Options: 0-9 (1 being fastest and 9 being the most compressed), default is 6 zlib-compression-level = 6 # Options: 0-9 (1 being fastest and 9 being the most compressed), default is 6
# FIXME rename to transport # FIXME rename to transport
layer = "akka.remote.netty.NettyRemoteSupport" layer = "akka.cluster.netty.NettyRemoteSupport"
secure-cookie = "" # Generate your own with '$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh' secure-cookie = "" # Generate your own with '$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh'
# or using 'akka.util.Crypt.generateSecureCookie' # or using 'akka.util.Crypt.generateSecureCookie'
@ -226,7 +226,7 @@ akka {
client { client {
buffering { buffering {
retry-message-send-on-failure = on retry-message-send-on-failure = false # Should message buffering on remote client error be used (buffer flushed on successful reconnect)
capacity = -1 # If negative (or zero) then an unbounded mailbox is used (default) capacity = -1 # If negative (or zero) then an unbounded mailbox is used (default)
# If positive then a bounded mailbox is used and the capacity is set using the property # If positive then a bounded mailbox is used and the capacity is set using the property
} }