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 org.junit.Test;
|
||||
import akka.actor.Actors;
|
||||
import akka.cluster.RemoteSupport;
|
||||
import akka.remote.RemoteSupport;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
public class JavaAPI {
|
||||
|
|
|
|||
|
|
@ -1,7 +1,11 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.actor;
|
||||
|
||||
import akka.japi.Creator;
|
||||
import akka.cluster.RemoteSupport;
|
||||
import akka.remote.RemoteSupport;
|
||||
|
||||
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;
|
||||
|
||||
/**
|
||||
* Java API for Akka EventHandler
|
||||
*/
|
||||
|
||||
public class JavaEventHandler {
|
||||
|
||||
|
||||
|
|
|
|||
|
|
@ -1,3 +1,7 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.routing;
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import akka.routing._
|
|||
import Config._
|
||||
import akka.util.{ ReflectiveAccess, Duration }
|
||||
import ReflectiveAccess._
|
||||
import akka.cluster.RemoteSupport
|
||||
import akka.remote.RemoteSupport
|
||||
import akka.cluster.ClusterNode
|
||||
import akka.japi.{ Creator, Procedure }
|
||||
import akka.serialization.{ Serializer, Serialization }
|
||||
|
|
|
|||
|
|
@ -2,7 +2,7 @@
|
|||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
package akka.remote
|
||||
|
||||
import akka.actor.Actor
|
||||
import akka.event.EventHandler
|
||||
|
|
@ -2,7 +2,7 @@
|
|||
* Copyright (C) 2009-2010 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
package akka.remote
|
||||
|
||||
import akka.japi.Creator
|
||||
import akka.actor._
|
||||
|
|
@ -265,23 +265,24 @@ trait BasicRouter extends Router {
|
|||
def route(message: Any)(implicit sender: Option[ActorRef]) = message match {
|
||||
case Routing.Broadcast(message) ⇒
|
||||
//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 {
|
||||
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 {
|
||||
case e: Exception ⇒
|
||||
connections.remove(actor)
|
||||
connections.remove(connection)
|
||||
throw e
|
||||
})
|
||||
}
|
||||
}
|
||||
case _ ⇒
|
||||
//it no broadcast message, we are going to select an actor from the connections and send the message to him.
|
||||
next match {
|
||||
case Some(actor) ⇒
|
||||
case Some(connection) ⇒
|
||||
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 {
|
||||
case e: Exception ⇒
|
||||
connections.remove(actor)
|
||||
connections.remove(connection)
|
||||
throw e
|
||||
}
|
||||
case None ⇒
|
||||
|
|
@ -295,13 +296,13 @@ trait BasicRouter extends Router {
|
|||
case _ ⇒
|
||||
//it no broadcast message, we are going to select an actor from the connections and send the message to him.
|
||||
next match {
|
||||
case Some(actor) ⇒
|
||||
case Some(connection) ⇒
|
||||
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
|
||||
actor.?(message, timeout)(sender).asInstanceOf[Future[T]]
|
||||
connection.?(message, timeout)(sender).asInstanceOf[Future[T]]
|
||||
} catch {
|
||||
case e: Exception ⇒
|
||||
connections.remove(actor)
|
||||
connections.remove(connection)
|
||||
throw e
|
||||
}
|
||||
case None ⇒
|
||||
|
|
|
|||
|
|
@ -9,7 +9,8 @@ import DeploymentConfig.ReplicationScheme
|
|||
import akka.dispatch.MessageInvocation
|
||||
import akka.config.{ Config, ModuleNotAvailableException }
|
||||
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 java.net.InetSocketAddress
|
||||
|
|
@ -137,7 +138,7 @@ object ReflectiveAccess {
|
|||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
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)
|
||||
|
||||
|
|
@ -145,13 +146,14 @@ object ReflectiveAccess {
|
|||
|
||||
def ensureEnabled() = {
|
||||
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)
|
||||
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 Left(exception) ⇒
|
||||
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>
|
||||
*/
|
||||
|
||||
option java_package = "akka.cluster";
|
||||
option java_package = "akka.remote";
|
||||
option optimize_for = SPEED;
|
||||
|
||||
/******************************************
|
||||
Compile with:
|
||||
cd ./akka-cluster/src/main/protocol
|
||||
cd ./akka-remote/src/main/protocol
|
||||
protoc RemoteProtocol.proto --java_out ../java
|
||||
*******************************************/
|
||||
|
||||
|
|
|
|||
|
|
@ -2,7 +2,7 @@
|
|||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
package akka.remote
|
||||
|
||||
import akka.actor.{ Actor, BootableActorLoaderService }
|
||||
import akka.util.{ ReflectiveAccess, Bootable }
|
||||
|
|
@ -2,9 +2,9 @@
|
|||
* 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 com.google.protobuf.ByteString
|
||||
|
|
@ -2,7 +2,7 @@
|
|||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
package akka.remote
|
||||
|
||||
import akka.dispatch.PinnedDispatcher
|
||||
|
||||
|
|
@ -2,7 +2,7 @@
|
|||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
package akka.remote
|
||||
|
||||
import akka.actor._
|
||||
import DeploymentConfig._
|
||||
|
|
@ -2,7 +2,7 @@
|
|||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
package akka.remote
|
||||
|
||||
import akka.util.Duration
|
||||
import akka.config.Config._
|
||||
|
|
@ -2,7 +2,7 @@
|
|||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
package akka.remote
|
||||
|
||||
import akka.actor._
|
||||
import Actor._
|
||||
|
|
@ -41,12 +41,12 @@ object Remote extends RemoteService {
|
|||
// FIXME configure computeGridDispatcher to what?
|
||||
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()),
|
||||
Remote.remoteAddress,
|
||||
systemService = true)
|
||||
|
||||
private[cluster] lazy val remoteDaemonSupervisor = Supervisor(
|
||||
private[remote] lazy val remoteDaemonSupervisor = Supervisor(
|
||||
SupervisorConfig(
|
||||
OneForOneStrategy(List(classOf[Exception]), Int.MaxValue, Int.MaxValue), // is infinite restart what we want?
|
||||
Supervise(
|
||||
|
|
@ -54,7 +54,7 @@ object Remote extends RemoteService {
|
|||
Permanent)
|
||||
:: Nil))
|
||||
|
||||
private[cluster] lazy val remoteClientLifeCycleHandler = actorOf(Props(new Actor {
|
||||
private[remote] lazy val remoteClientLifeCycleHandler = actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case RemoteClientError(cause, client, address) ⇒ client.shutdownClientModule()
|
||||
case RemoteClientDisconnected(client, address) ⇒ client.shutdownClientModule()
|
||||
|
|
@ -63,7 +63,7 @@ object Remote extends RemoteService {
|
|||
}), "akka.cluster.RemoteClientLifeCycleListener")
|
||||
|
||||
lazy val server: RemoteSupport = {
|
||||
val remote = new akka.cluster.netty.NettyRemoteSupport
|
||||
val remote = new akka.remote.netty.NettyRemoteSupport
|
||||
remote.start(hostname, port)
|
||||
remote.register(Remote.remoteAddress, remoteDaemon)
|
||||
remote.addListener(NetworkEventStream.channel)
|
||||
|
|
@ -73,6 +73,8 @@ object Remote extends RemoteService {
|
|||
|
||||
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 uuidToUuidProtocol(uuid: UUID): UuidProtocol =
|
||||
|
|
@ -2,12 +2,12 @@
|
|||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
package akka.remote
|
||||
|
||||
import akka.actor._
|
||||
import Actor._
|
||||
import akka.cluster._
|
||||
import akka.routing._
|
||||
import akka.dispatch.PinnedDispatcher
|
||||
import akka.event.EventHandler
|
||||
import akka.util.{ ListenerManagement, Duration }
|
||||
|
||||
|
|
@ -18,7 +18,6 @@ import scala.annotation.tailrec
|
|||
import java.net.InetSocketAddress
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import System.{ currentTimeMillis ⇒ newTimestamp }
|
||||
import akka.dispatch.PinnedDispatcher
|
||||
|
||||
/**
|
||||
* 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)
|
||||
}
|
||||
}
|
||||
|
|
@ -2,14 +2,13 @@
|
|||
* 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.cluster.{ MessageSerializer, RemoteClientSettings, RemoteServerSettings }
|
||||
import akka.cluster.RemoteProtocol._
|
||||
import akka.serialization.RemoteActorSerialization
|
||||
import akka.serialization.RemoteActorSerialization._
|
||||
import akka.cluster._
|
||||
import RemoteActorSerialization._
|
||||
import akka.remote._
|
||||
import RemoteProtocol._
|
||||
import akka.actor.{
|
||||
PoisonPill,
|
||||
Actor,
|
||||
|
|
@ -22,7 +21,7 @@ import akka.actor.{
|
|||
LifeCycleMessage,
|
||||
Address
|
||||
}
|
||||
import akka.actor.Actor._
|
||||
import Actor._
|
||||
import akka.config.Config
|
||||
import Config._
|
||||
import akka.util._
|
||||
|
|
@ -172,9 +171,9 @@ abstract class RemoteClient private[akka] (
|
|||
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
|
||||
|
||||
|
|
@ -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
|
||||
val nrOfMessages = pendingRequests.size
|
||||
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
|
||||
private var bootstrap: ClientBootstrap = _
|
||||
@volatile
|
||||
private[cluster] var connection: ChannelFuture = _
|
||||
private[remote] var connection: ChannelFuture = _
|
||||
@volatile
|
||||
private[cluster] var openChannels: DefaultChannelGroup = _
|
||||
private[remote] var openChannels: DefaultChannelGroup = _
|
||||
@volatile
|
||||
private var timer: HashedWheelTimer = _
|
||||
@volatile
|
||||
|
|
@ -10,19 +10,18 @@ import akka.actor._
|
|||
import DeploymentConfig._
|
||||
import akka.dispatch.MessageInvocation
|
||||
import akka.util.{ ReflectiveAccess, Duration }
|
||||
import akka.cluster.{ RemoteClientSettings, MessageSerializer }
|
||||
import akka.cluster.RemoteProtocol
|
||||
import akka.event.EventHandler
|
||||
import akka.remote.{ RemoteProtocol, RemoteClientSettings, MessageSerializer }
|
||||
import RemoteProtocol._
|
||||
|
||||
import scala.collection.immutable.Stack
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.{ LinkedList, Collections }
|
||||
|
||||
import com.google.protobuf.ByteString
|
||||
|
||||
import com.eaio.uuid.UUID
|
||||
import akka.event.EventHandler
|
||||
import java.util.{ LinkedList, Collections }
|
||||
|
||||
/**
|
||||
* Module for local actor serialization.
|
||||
|
|
|
|||
|
|
@ -2,7 +2,7 @@
|
|||
* 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.matchers.MustMatchers
|
||||
|
|
@ -10,7 +10,7 @@ import org.scalatest.junit.JUnitRunner
|
|||
|
||||
import org.junit.runner.RunWith
|
||||
|
||||
import akka.cluster.netty.NettyRemoteSupport
|
||||
import akka.remote.netty.NettyRemoteSupport
|
||||
import akka.actor.{ Actor, ActorRegistry }
|
||||
|
||||
import java.util.concurrent.{ TimeUnit, CountDownLatch }
|
||||
|
|
@ -457,7 +457,7 @@ object Dependency {
|
|||
val jsr250 = "javax.annotation" % "jsr250-api" % "1.0" // CDDL v1
|
||||
val jsr311 = "javax.ws.rs" % "jsr311-api" % "1.1" // CDDL v1
|
||||
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 netty = "org.jboss.netty" % "netty" % V.Netty // ApacheV2
|
||||
val osgi = "org.osgi" % "org.osgi.core" % "4.2.0" // ApacheV2
|
||||
|
|
@ -476,7 +476,7 @@ object Dependency {
|
|||
// 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 jetty = "org.eclipse.jetty" % "jetty-server" % V.Jetty % "provided" // Eclipse license
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue