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:
Jonas Bonér 2011-09-20 21:44:50 +02:00
parent 7bc698f864
commit 978cbe4437
22 changed files with 1024 additions and 1012 deletions

View file

@ -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 {

View file

@ -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;

View file

@ -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 {

View file

@ -1,3 +1,7 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.routing;
/**

View file

@ -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 }

View file

@ -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

View file

@ -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._

View file

@ -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

View file

@ -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&#233;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)

View file

@ -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
*******************************************/

View file

@ -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 }

View file

@ -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

View file

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

View file

@ -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._

View file

@ -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._

View file

@ -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 =

View file

@ -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)
}
}

View file

@ -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

View file

@ -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.

View file

@ -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 }