Change the package name of all classes in remote module to 'akka.remote'.
Signed-off-by: Jonas Bonér <jonas@jonasboner.com>
This commit is contained in:
parent
7bc698f864
commit
978cbe4437
22 changed files with 1024 additions and 1012 deletions
|
|
@ -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.cluster.RemoteSupport;
|
import akka.remote.RemoteSupport;
|
||||||
import static org.junit.Assert.*;
|
import static org.junit.Assert.*;
|
||||||
|
|
||||||
public class JavaAPI {
|
public class JavaAPI {
|
||||||
|
|
|
||||||
|
|
@ -1,7 +1,11 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||||
|
*/
|
||||||
|
|
||||||
package akka.actor;
|
package akka.actor;
|
||||||
|
|
||||||
import akka.japi.Creator;
|
import akka.japi.Creator;
|
||||||
import akka.cluster.RemoteSupport;
|
import akka.remote.RemoteSupport;
|
||||||
|
|
||||||
import com.eaio.uuid.UUID;
|
import com.eaio.uuid.UUID;
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -1,12 +1,14 @@
|
||||||
package akka.event;
|
/**
|
||||||
|
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||||
|
*/
|
||||||
|
|
||||||
|
package akka.event;
|
||||||
|
|
||||||
import akka.actor.ActorRef;
|
import akka.actor.ActorRef;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API for Akka EventHandler
|
* Java API for Akka EventHandler
|
||||||
*/
|
*/
|
||||||
|
|
||||||
public class JavaEventHandler {
|
public class JavaEventHandler {
|
||||||
|
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -1,3 +1,7 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||||
|
*/
|
||||||
|
|
||||||
package akka.routing;
|
package akka.routing;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -11,7 +11,7 @@ import akka.routing._
|
||||||
import Config._
|
import Config._
|
||||||
import akka.util.{ ReflectiveAccess, Duration }
|
import akka.util.{ ReflectiveAccess, Duration }
|
||||||
import ReflectiveAccess._
|
import ReflectiveAccess._
|
||||||
import akka.cluster.RemoteSupport
|
import akka.remote.RemoteSupport
|
||||||
import akka.cluster.ClusterNode
|
import akka.cluster.ClusterNode
|
||||||
import akka.japi.{ Creator, Procedure }
|
import akka.japi.{ Creator, Procedure }
|
||||||
import akka.serialization.{ Serializer, Serialization }
|
import akka.serialization.{ Serializer, Serialization }
|
||||||
|
|
|
||||||
|
|
@ -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.cluster
|
package akka.remote
|
||||||
|
|
||||||
import akka.actor.Actor
|
import akka.actor.Actor
|
||||||
import akka.event.EventHandler
|
import akka.event.EventHandler
|
||||||
|
|
@ -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.cluster
|
package akka.remote
|
||||||
|
|
||||||
import akka.japi.Creator
|
import akka.japi.Creator
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
|
|
@ -265,23 +265,24 @@ trait BasicRouter extends Router {
|
||||||
def route(message: Any)(implicit sender: Option[ActorRef]) = message match {
|
def route(message: Any)(implicit sender: Option[ActorRef]) = message match {
|
||||||
case Routing.Broadcast(message) ⇒
|
case Routing.Broadcast(message) ⇒
|
||||||
//it is a broadcast message, we are going to send to message to all connections.
|
//it is a broadcast message, we are going to send to message to all connections.
|
||||||
connections.versionedIterable.iterable.foreach(actor ⇒
|
connections.versionedIterable.iterable foreach { connection ⇒
|
||||||
try {
|
try {
|
||||||
actor.!(message)(sender) // we use original sender, so this is essentially a 'forward'
|
connection.!(message)(sender) // we use original sender, so this is essentially a 'forward'
|
||||||
} catch {
|
} catch {
|
||||||
case e: Exception ⇒
|
case e: Exception ⇒
|
||||||
connections.remove(actor)
|
connections.remove(connection)
|
||||||
throw e
|
throw e
|
||||||
})
|
}
|
||||||
|
}
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
//it no broadcast message, we are going to select an actor from the connections and send the message to him.
|
//it no broadcast message, we are going to select an actor from the connections and send the message to him.
|
||||||
next match {
|
next match {
|
||||||
case Some(actor) ⇒
|
case Some(connection) ⇒
|
||||||
try {
|
try {
|
||||||
actor.!(message)(sender) // we use original sender, so this is essentially a 'forward'
|
connection.!(message)(sender) // we use original sender, so this is essentially a 'forward'
|
||||||
} catch {
|
} catch {
|
||||||
case e: Exception ⇒
|
case e: Exception ⇒
|
||||||
connections.remove(actor)
|
connections.remove(connection)
|
||||||
throw e
|
throw e
|
||||||
}
|
}
|
||||||
case None ⇒
|
case None ⇒
|
||||||
|
|
@ -295,13 +296,13 @@ trait BasicRouter extends Router {
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
//it no broadcast message, we are going to select an actor from the connections and send the message to him.
|
//it no broadcast message, we are going to select an actor from the connections and send the message to him.
|
||||||
next match {
|
next match {
|
||||||
case Some(actor) ⇒
|
case Some(connection) ⇒
|
||||||
try {
|
try {
|
||||||
// FIXME is this not wrong? it will not pass on and use the original Future but create a new one. Should reuse 'channel: UntypedChannel' in the AbstractRoutedActorRef
|
// FIXME is this not wrong? it will not pass on and use the original Future but create a new one. Should reuse 'channel: UntypedChannel' in the AbstractRoutedActorRef
|
||||||
actor.?(message, timeout)(sender).asInstanceOf[Future[T]]
|
connection.?(message, timeout)(sender).asInstanceOf[Future[T]]
|
||||||
} catch {
|
} catch {
|
||||||
case e: Exception ⇒
|
case e: Exception ⇒
|
||||||
connections.remove(actor)
|
connections.remove(connection)
|
||||||
throw e
|
throw e
|
||||||
}
|
}
|
||||||
case None ⇒
|
case None ⇒
|
||||||
|
|
|
||||||
|
|
@ -9,7 +9,8 @@ import DeploymentConfig.ReplicationScheme
|
||||||
import akka.dispatch.MessageInvocation
|
import akka.dispatch.MessageInvocation
|
||||||
import akka.config.{ Config, ModuleNotAvailableException }
|
import akka.config.{ Config, ModuleNotAvailableException }
|
||||||
import akka.event.EventHandler
|
import akka.event.EventHandler
|
||||||
import akka.cluster.{ RemoteSupport, ClusterNode, RemoteService }
|
import akka.cluster.ClusterNode
|
||||||
|
import akka.remote.{ RemoteSupport, RemoteService }
|
||||||
import akka.routing.{ RoutedProps, Router }
|
import akka.routing.{ RoutedProps, Router }
|
||||||
|
|
||||||
import java.net.InetSocketAddress
|
import java.net.InetSocketAddress
|
||||||
|
|
@ -137,7 +138,7 @@ object ReflectiveAccess {
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
object RemoteModule {
|
object RemoteModule {
|
||||||
val TRANSPORT = Config.config.getString("akka.remote.layer", "akka.cluster.netty.NettyRemoteSupport")
|
val TRANSPORT = Config.config.getString("akka.remote.layer", "akka.remote.netty.NettyRemoteSupport")
|
||||||
|
|
||||||
val configDefaultAddress = new InetSocketAddress(Config.hostname, Config.remoteServerPort)
|
val configDefaultAddress = new InetSocketAddress(Config.hostname, Config.remoteServerPort)
|
||||||
|
|
||||||
|
|
@ -145,13 +146,14 @@ object ReflectiveAccess {
|
||||||
|
|
||||||
def ensureEnabled() = {
|
def ensureEnabled() = {
|
||||||
if (!isEnabled) {
|
if (!isEnabled) {
|
||||||
val e = new ModuleNotAvailableException("Can't load the remoting module, make sure that akka-remote.jar is on the classpath")
|
val e = new ModuleNotAvailableException(
|
||||||
|
"Can't load the remote module, make sure it is enabled in the config ('akka.enabled-modules = [\"remote\"])' and that akka-remote.jar is on the classpath")
|
||||||
EventHandler.debug(this, e.toString)
|
EventHandler.debug(this, e.toString)
|
||||||
throw e
|
throw e
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
lazy val remoteInstance: Option[RemoteService] = getObjectFor("akka.cluster.Remote$") match {
|
lazy val remoteInstance: Option[RemoteService] = getObjectFor("akka.remote.Remote$") match {
|
||||||
case Right(value) ⇒ Some(value)
|
case Right(value) ⇒ Some(value)
|
||||||
case Left(exception) ⇒
|
case Left(exception) ⇒
|
||||||
EventHandler.debug(this, exception.toString)
|
EventHandler.debug(this, exception.toString)
|
||||||
|
|
|
||||||
File diff suppressed because it is too large
Load diff
|
|
@ -2,12 +2,12 @@
|
||||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||||
*/
|
*/
|
||||||
|
|
||||||
option java_package = "akka.cluster";
|
option java_package = "akka.remote";
|
||||||
option optimize_for = SPEED;
|
option optimize_for = SPEED;
|
||||||
|
|
||||||
/******************************************
|
/******************************************
|
||||||
Compile with:
|
Compile with:
|
||||||
cd ./akka-cluster/src/main/protocol
|
cd ./akka-remote/src/main/protocol
|
||||||
protoc RemoteProtocol.proto --java_out ../java
|
protoc RemoteProtocol.proto --java_out ../java
|
||||||
*******************************************/
|
*******************************************/
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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.cluster
|
package akka.remote
|
||||||
|
|
||||||
import akka.actor.{ Actor, BootableActorLoaderService }
|
import akka.actor.{ Actor, BootableActorLoaderService }
|
||||||
import akka.util.{ ReflectiveAccess, Bootable }
|
import akka.util.{ ReflectiveAccess, Bootable }
|
||||||
|
|
@ -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.cluster
|
package akka.remote
|
||||||
|
|
||||||
import akka.cluster.RemoteProtocol._
|
import akka.remote.RemoteProtocol._
|
||||||
import akka.serialization.Serialization
|
import akka.serialization.Serialization
|
||||||
|
|
||||||
import com.google.protobuf.ByteString
|
import com.google.protobuf.ByteString
|
||||||
|
|
@ -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.cluster
|
package akka.remote
|
||||||
|
|
||||||
import akka.dispatch.PinnedDispatcher
|
import akka.dispatch.PinnedDispatcher
|
||||||
|
|
||||||
|
|
@ -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.cluster
|
package akka.remote
|
||||||
|
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import DeploymentConfig._
|
import DeploymentConfig._
|
||||||
|
|
@ -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.cluster
|
package akka.remote
|
||||||
|
|
||||||
import akka.util.Duration
|
import akka.util.Duration
|
||||||
import akka.config.Config._
|
import akka.config.Config._
|
||||||
|
|
@ -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.cluster
|
package akka.remote
|
||||||
|
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import Actor._
|
import Actor._
|
||||||
|
|
@ -41,12 +41,12 @@ object Remote extends RemoteService {
|
||||||
// FIXME configure computeGridDispatcher to what?
|
// FIXME configure computeGridDispatcher to what?
|
||||||
val computeGridDispatcher = Dispatchers.newDispatcher("akka:compute-grid").build
|
val computeGridDispatcher = Dispatchers.newDispatcher("akka:compute-grid").build
|
||||||
|
|
||||||
private[cluster] lazy val remoteDaemon = new LocalActorRef(
|
private[remote] lazy val remoteDaemon = new LocalActorRef(
|
||||||
Props(new RemoteDaemon).copy(dispatcher = new PinnedDispatcher()),
|
Props(new RemoteDaemon).copy(dispatcher = new PinnedDispatcher()),
|
||||||
Remote.remoteAddress,
|
Remote.remoteAddress,
|
||||||
systemService = true)
|
systemService = true)
|
||||||
|
|
||||||
private[cluster] lazy val remoteDaemonSupervisor = Supervisor(
|
private[remote] lazy val remoteDaemonSupervisor = Supervisor(
|
||||||
SupervisorConfig(
|
SupervisorConfig(
|
||||||
OneForOneStrategy(List(classOf[Exception]), Int.MaxValue, Int.MaxValue), // is infinite restart what we want?
|
OneForOneStrategy(List(classOf[Exception]), Int.MaxValue, Int.MaxValue), // is infinite restart what we want?
|
||||||
Supervise(
|
Supervise(
|
||||||
|
|
@ -54,7 +54,7 @@ object Remote extends RemoteService {
|
||||||
Permanent)
|
Permanent)
|
||||||
:: Nil))
|
:: Nil))
|
||||||
|
|
||||||
private[cluster] lazy val remoteClientLifeCycleHandler = actorOf(Props(new Actor {
|
private[remote] lazy val remoteClientLifeCycleHandler = actorOf(Props(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()
|
||||||
|
|
@ -63,7 +63,7 @@ object Remote extends RemoteService {
|
||||||
}), "akka.cluster.RemoteClientLifeCycleListener")
|
}), "akka.cluster.RemoteClientLifeCycleListener")
|
||||||
|
|
||||||
lazy val server: RemoteSupport = {
|
lazy val server: RemoteSupport = {
|
||||||
val remote = new akka.cluster.netty.NettyRemoteSupport
|
val remote = new akka.remote.netty.NettyRemoteSupport
|
||||||
remote.start(hostname, port)
|
remote.start(hostname, port)
|
||||||
remote.register(Remote.remoteAddress, remoteDaemon)
|
remote.register(Remote.remoteAddress, remoteDaemon)
|
||||||
remote.addListener(NetworkEventStream.channel)
|
remote.addListener(NetworkEventStream.channel)
|
||||||
|
|
@ -73,6 +73,8 @@ object Remote extends RemoteService {
|
||||||
|
|
||||||
lazy val address = server.address
|
lazy val address = server.address
|
||||||
|
|
||||||
|
def start() { EventHandler.info(this, "Starting remote server on [%s]".format(address)) }
|
||||||
|
|
||||||
def uuidProtocolToUuid(uuid: UuidProtocol): UUID = new UUID(uuid.getHigh, uuid.getLow)
|
def uuidProtocolToUuid(uuid: UuidProtocol): UUID = new UUID(uuid.getHigh, uuid.getLow)
|
||||||
|
|
||||||
def uuidToUuidProtocol(uuid: UUID): UuidProtocol =
|
def uuidToUuidProtocol(uuid: UUID): UuidProtocol =
|
||||||
|
|
@ -2,12 +2,12 @@
|
||||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package akka.cluster
|
package akka.remote
|
||||||
|
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import Actor._
|
import Actor._
|
||||||
import akka.cluster._
|
|
||||||
import akka.routing._
|
import akka.routing._
|
||||||
|
import akka.dispatch.PinnedDispatcher
|
||||||
import akka.event.EventHandler
|
import akka.event.EventHandler
|
||||||
import akka.util.{ ListenerManagement, Duration }
|
import akka.util.{ ListenerManagement, Duration }
|
||||||
|
|
||||||
|
|
@ -18,7 +18,6 @@ import scala.annotation.tailrec
|
||||||
import java.net.InetSocketAddress
|
import java.net.InetSocketAddress
|
||||||
import java.util.concurrent.atomic.AtomicReference
|
import java.util.concurrent.atomic.AtomicReference
|
||||||
import System.{ currentTimeMillis ⇒ newTimestamp }
|
import System.{ currentTimeMillis ⇒ newTimestamp }
|
||||||
import akka.dispatch.PinnedDispatcher
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Base class for remote failure detection management.
|
* Base class for remote failure detection management.
|
||||||
|
|
@ -170,7 +169,7 @@ abstract class RemoteFailureDetectorBase(initialConnections: Map[InetSocketAddre
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private[cluster] def newConnection(actorAddress: String, inetSocketAddress: InetSocketAddress) = {
|
private[remote] def newConnection(actorAddress: String, inetSocketAddress: InetSocketAddress) = {
|
||||||
RemoteActorRef(inetSocketAddress, actorAddress, Actor.TIMEOUT, None)
|
RemoteActorRef(inetSocketAddress, actorAddress, Actor.TIMEOUT, None)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -2,14 +2,13 @@
|
||||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package akka.cluster.netty
|
package akka.remote.netty
|
||||||
|
|
||||||
import akka.dispatch.{ ActorPromise, DefaultPromise, Promise }
|
import akka.dispatch.{ ActorPromise, DefaultPromise, Promise }
|
||||||
import akka.cluster.{ MessageSerializer, RemoteClientSettings, RemoteServerSettings }
|
|
||||||
import akka.cluster.RemoteProtocol._
|
|
||||||
import akka.serialization.RemoteActorSerialization
|
import akka.serialization.RemoteActorSerialization
|
||||||
import akka.serialization.RemoteActorSerialization._
|
import RemoteActorSerialization._
|
||||||
import akka.cluster._
|
import akka.remote._
|
||||||
|
import RemoteProtocol._
|
||||||
import akka.actor.{
|
import akka.actor.{
|
||||||
PoisonPill,
|
PoisonPill,
|
||||||
Actor,
|
Actor,
|
||||||
|
|
@ -22,7 +21,7 @@ import akka.actor.{
|
||||||
LifeCycleMessage,
|
LifeCycleMessage,
|
||||||
Address
|
Address
|
||||||
}
|
}
|
||||||
import akka.actor.Actor._
|
import Actor._
|
||||||
import akka.config.Config
|
import akka.config.Config
|
||||||
import Config._
|
import Config._
|
||||||
import akka.util._
|
import akka.util._
|
||||||
|
|
@ -172,9 +171,9 @@ abstract class RemoteClient private[akka] (
|
||||||
else new LinkedBlockingQueue[(Boolean, Uuid, RemoteMessageProtocol)](transactionLogCapacity)
|
else new LinkedBlockingQueue[(Boolean, Uuid, RemoteMessageProtocol)](transactionLogCapacity)
|
||||||
}
|
}
|
||||||
|
|
||||||
private[cluster] val runSwitch = new Switch()
|
private[remote] val runSwitch = new Switch()
|
||||||
|
|
||||||
private[cluster] def isRunning = runSwitch.isOn
|
private[remote] def isRunning = runSwitch.isOn
|
||||||
|
|
||||||
protected def notifyListeners(msg: ⇒ Any): Unit
|
protected def notifyListeners(msg: ⇒ Any): Unit
|
||||||
|
|
||||||
|
|
@ -288,7 +287,7 @@ abstract class RemoteClient private[akka] (
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private[cluster] def sendPendingRequests() = pendingRequests synchronized {
|
private[remote] 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)
|
||||||
|
|
@ -339,9 +338,9 @@ class ActiveRemoteClient private[akka] (
|
||||||
@volatile
|
@volatile
|
||||||
private var bootstrap: ClientBootstrap = _
|
private var bootstrap: ClientBootstrap = _
|
||||||
@volatile
|
@volatile
|
||||||
private[cluster] var connection: ChannelFuture = _
|
private[remote] var connection: ChannelFuture = _
|
||||||
@volatile
|
@volatile
|
||||||
private[cluster] var openChannels: DefaultChannelGroup = _
|
private[remote] var openChannels: DefaultChannelGroup = _
|
||||||
@volatile
|
@volatile
|
||||||
private var timer: HashedWheelTimer = _
|
private var timer: HashedWheelTimer = _
|
||||||
@volatile
|
@volatile
|
||||||
|
|
@ -10,19 +10,18 @@ import akka.actor._
|
||||||
import DeploymentConfig._
|
import DeploymentConfig._
|
||||||
import akka.dispatch.MessageInvocation
|
import akka.dispatch.MessageInvocation
|
||||||
import akka.util.{ ReflectiveAccess, Duration }
|
import akka.util.{ ReflectiveAccess, Duration }
|
||||||
import akka.cluster.{ RemoteClientSettings, MessageSerializer }
|
import akka.event.EventHandler
|
||||||
import akka.cluster.RemoteProtocol
|
import akka.remote.{ RemoteProtocol, RemoteClientSettings, MessageSerializer }
|
||||||
import RemoteProtocol._
|
import RemoteProtocol._
|
||||||
|
|
||||||
import scala.collection.immutable.Stack
|
import scala.collection.immutable.Stack
|
||||||
|
|
||||||
import java.net.InetSocketAddress
|
import java.net.InetSocketAddress
|
||||||
|
import java.util.{ LinkedList, Collections }
|
||||||
|
|
||||||
import com.google.protobuf.ByteString
|
import com.google.protobuf.ByteString
|
||||||
|
|
||||||
import com.eaio.uuid.UUID
|
import com.eaio.uuid.UUID
|
||||||
import akka.event.EventHandler
|
|
||||||
import java.util.{ LinkedList, Collections }
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Module for local actor serialization.
|
* Module for local actor serialization.
|
||||||
|
|
|
||||||
|
|
@ -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.cluster
|
package akka.remote
|
||||||
|
|
||||||
import org.scalatest.{ Spec, WordSpec, BeforeAndAfterAll, BeforeAndAfterEach }
|
import org.scalatest.{ Spec, WordSpec, BeforeAndAfterAll, BeforeAndAfterEach }
|
||||||
import org.scalatest.matchers.MustMatchers
|
import org.scalatest.matchers.MustMatchers
|
||||||
|
|
@ -10,7 +10,7 @@ import org.scalatest.junit.JUnitRunner
|
||||||
|
|
||||||
import org.junit.runner.RunWith
|
import org.junit.runner.RunWith
|
||||||
|
|
||||||
import akka.cluster.netty.NettyRemoteSupport
|
import akka.remote.netty.NettyRemoteSupport
|
||||||
import akka.actor.{ Actor, ActorRegistry }
|
import akka.actor.{ Actor, ActorRegistry }
|
||||||
|
|
||||||
import java.util.concurrent.{ TimeUnit, CountDownLatch }
|
import java.util.concurrent.{ TimeUnit, CountDownLatch }
|
||||||
|
|
@ -457,7 +457,7 @@ object Dependency {
|
||||||
val jsr250 = "javax.annotation" % "jsr250-api" % "1.0" // CDDL v1
|
val jsr250 = "javax.annotation" % "jsr250-api" % "1.0" // CDDL v1
|
||||||
val jsr311 = "javax.ws.rs" % "jsr311-api" % "1.1" // CDDL v1
|
val jsr311 = "javax.ws.rs" % "jsr311-api" % "1.1" // CDDL v1
|
||||||
val log4j = "log4j" % "log4j" % "1.2.15" // ApacheV2
|
val log4j = "log4j" % "log4j" % "1.2.15" // ApacheV2
|
||||||
val mongoAsync = "com.mongodb.async" % "mongo-driver_2.9.0-1" % "0.2.7" //ApacheV2
|
val mongoAsync = "com.mongodb.async" % "mongo-driver_2.9.0-1" % "0.2.7" // ApacheV2
|
||||||
val multiverse = "org.multiverse" % "multiverse-alpha" % V.Multiverse // ApacheV2
|
val multiverse = "org.multiverse" % "multiverse-alpha" % V.Multiverse // ApacheV2
|
||||||
val netty = "org.jboss.netty" % "netty" % V.Netty // ApacheV2
|
val netty = "org.jboss.netty" % "netty" % V.Netty // ApacheV2
|
||||||
val osgi = "org.osgi" % "org.osgi.core" % "4.2.0" // ApacheV2
|
val osgi = "org.osgi" % "org.osgi.core" % "4.2.0" // ApacheV2
|
||||||
|
|
@ -476,9 +476,9 @@ object Dependency {
|
||||||
// Provided
|
// Provided
|
||||||
|
|
||||||
object Provided {
|
object Provided {
|
||||||
val javaxServlet = "org.apache.geronimo.specs" % "geronimo-servlet_3.0_spec" % "1.0" % "provided" //CDDL v1
|
val javaxServlet = "org.apache.geronimo.specs" % "geronimo-servlet_3.0_spec" % "1.0" % "provided" // CDDL v1
|
||||||
val jerseyServer = "com.sun.jersey" % "jersey-server" % V.Jersey % "provided" // CDDL v1
|
val jerseyServer = "com.sun.jersey" % "jersey-server" % V.Jersey % "provided" // CDDL v1
|
||||||
val jetty = "org.eclipse.jetty" % "jetty-server" % V.Jetty % "provided" // Eclipse license
|
val jetty = "org.eclipse.jetty" % "jetty-server" % V.Jetty % "provided" // Eclipse license
|
||||||
}
|
}
|
||||||
|
|
||||||
// Runtime
|
// Runtime
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue