fixed compilation problems in akka-cluster

This commit is contained in:
Jonas Bonér 2011-04-28 20:12:37 +02:00
parent fb008632cb
commit 2b1332e220
27 changed files with 490 additions and 2936 deletions

View file

@ -0,0 +1,48 @@
/**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.util
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object Convert {
def intToBytes(value: Int): Array[Byte] = {
val bytes = Array.fill[Byte](4)(0)
bytes(0) = (value >>> 24).asInstanceOf[Byte]
bytes(1) = (value >>> 16).asInstanceOf[Byte]
bytes(2) = (value >>> 8).asInstanceOf[Byte]
bytes(3) = value.asInstanceOf[Byte]
bytes
}
def bytesToInt(bytes: Array[Byte], offset: Int): Int = {
(0 until 4).foldLeft(0)((value, index) => value + ((bytes(index + offset) & 0x000000FF) << ((4 - 1 - index) * 8)))
}
def longToBytes(value: Long): Array[Byte] = {
val writeBuffer = Array.fill[Byte](8)(0)
writeBuffer(0) = (value >>> 56).asInstanceOf[Byte]
writeBuffer(1) = (value >>> 48).asInstanceOf[Byte]
writeBuffer(2) = (value >>> 40).asInstanceOf[Byte]
writeBuffer(3) = (value >>> 32).asInstanceOf[Byte]
writeBuffer(4) = (value >>> 24).asInstanceOf[Byte]
writeBuffer(5) = (value >>> 16).asInstanceOf[Byte]
writeBuffer(6) = (value >>> 8).asInstanceOf[Byte]
writeBuffer(7) = (value >>> 0).asInstanceOf[Byte]
writeBuffer
}
def bytesToLong(buf: Array[Byte]): Long = {
((buf(0) & 0xFFL) << 56) |
((buf(1) & 0xFFL) << 48) |
((buf(2) & 0xFFL) << 40) |
((buf(3) & 0xFFL) << 32) |
((buf(4) & 0xFFL) << 24) |
((buf(5) & 0xFFL) << 16) |
((buf(6) & 0xFFL) << 8) |
((buf(7) & 0xFFL) << 0)
}
}

View file

@ -4,6 +4,8 @@
package akka.util package akka.util
import akka.event.EventHandler
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
@ -24,6 +26,30 @@ object Helpers {
(0 until 4).foldLeft(0)((value, index) => value + ((bytes(index + offset) & 0x000000FF) << ((4 - 1 - index) * 8))) (0 until 4).foldLeft(0)((value, index) => value + ((bytes(index + offset) & 0x000000FF) << ((4 - 1 - index) * 8)))
} }
def flatten[T: ClassManifest](array: Array[Any]) = array.flatMap {
case arr: Array[T] => arr
case elem: T => Array(elem)
}
def ignore[E : Manifest](body: => Unit): Unit = {
try {
body
}
catch {
case e if manifest[E].erasure.isAssignableFrom(e.getClass) => ()
}
}
def withPrintStackTraceOnError(body: => Unit) = {
try {
body
} catch {
case e: Throwable =>
EventHandler.error(e, this, "")
throw e
}
}
/** /**
* Convenience helper to cast the given Option of Any to an Option of the given type. Will throw a ClassCastException * Convenience helper to cast the given Option of Any to an Option of the given type. Will throw a ClassCastException
* if the actual type is not assignable from the given one. * if the actual type is not assignable from the given one.

View file

@ -0,0 +1,37 @@
/**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.util
import akka.event.EventHandler
import java.lang.management.ManagementFactory
import javax.management.{ObjectInstance, ObjectName, InstanceAlreadyExistsException, InstanceNotFoundException}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object JMX {
private val mbeanServer = ManagementFactory.getPlatformMBeanServer
def nameFor(hostname: String, service: String, bean: String): ObjectName =
new ObjectName("akka.%s:type=%s,name=%s".format(hostname, service, bean.replace(":", "_")))
def register(name: ObjectName, mbean: AnyRef): Option[ObjectInstance] = try {
Some(mbeanServer.registerMBean(mbean, name))
} catch {
case e: InstanceAlreadyExistsException =>
Some(mbeanServer.getObjectInstance(name))
case e: Exception =>
EventHandler.error(e, this, "Error when registering mbean [%s]".format(mbean))
None
}
def unregister(mbean: ObjectName) = try {
mbeanServer.unregisterMBean(mbean)
} catch {
case e: InstanceNotFoundException => {}
case e: Exception => EventHandler.error(e, this, "Error while unregistering mbean [%s]".format(mbean))
}
}

View file

@ -1,4 +1,4 @@
package akka.cloud.cluster; package akka.cluster;
/** /**
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one

View file

@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package akka.cloud.zookeeper; package akka.cluster.zookeeper;
import java.util.List; import java.util.List;
import java.util.NoSuchElementException; import java.util.NoSuchElementException;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package akka.cloud.zookeeper; package akka.cluster.zookeeper;
import java.io.Serializable; import java.io.Serializable;
import java.util.List; import java.util.List;

View file

@ -2,12 +2,12 @@
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
option java_package = "akka.cloud.cluster"; option java_package = "akka.cluster";
option optimize_for = SPEED; option optimize_for = SPEED;
/****************************************** /******************************************
Compile with: Compile with:
cd ./akka-cloud-cluster/src/main/protocol cd ./akka-cluster/src/main/protocol
protoc ClusterProtocol.proto --java_out ../java protoc ClusterProtocol.proto --java_out ../java
*******************************************/ *******************************************/
@ -17,7 +17,7 @@ option optimize_for = SPEED;
message RemoteDaemonMessageProtocol { message RemoteDaemonMessageProtocol {
required RemoteDaemonMessageType messageType = 1; required RemoteDaemonMessageType messageType = 1;
optional UuidProtocol actorUuid = 2; optional UuidProtocol actorUuid = 2;
optional string actorId = 3; optional string actorAddress = 3;
optional string actorClassName = 4; optional string actorClassName = 4;
optional bytes payload = 5; optional bytes payload = 5;
} }
@ -46,8 +46,8 @@ enum RemoteDaemonMessageType {
* Defines the durable mailbox message. * Defines the durable mailbox message.
*/ */
message DurableMailboxMessageProtocol { message DurableMailboxMessageProtocol {
required string ownerActorId = 1; required string ownerActorAddress= 1;
optional string senderActorId = 2; optional string senderActorAddress = 2;
optional UuidProtocol futureUuid = 3; optional UuidProtocol futureUuid = 3;
required bytes message = 4; required bytes message = 4;
} }

View file

@ -1,49 +0,0 @@
/**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/
option java_package = "akka.cloud.cluster";
option optimize_for = SPEED;
/******************************************
Compile with:
cd ./akka-cloud-cluster/src/main/protocol
protoc MonitoringProtocol.proto --java_out ../java
*/
message GenericLoggingEvent {
required string context = 1;
required string message = 2;
required uint64 time = 3;
}
message AuditEvent {
required string name = 1;
required string context = 2;
required string message = 3;
required uint64 time = 4;
}
message CounterAddEvent {
required string name = 1;
required uint64 delta = 2;
}
message CounterSetEvent {
required string name = 1;
required uint32 value = 2;
}
message CounterResetEvent {
required string name = 1;
}
message AverageAddEvent {
required string name = 1;
required uint64 value = 2;
}
message AverageResetEvent {
required string name = 1;
}

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package akka.cloud.cluster package akka.cluster
import org.apache.bookkeeper.proto.BookieServer import org.apache.bookkeeper.proto.BookieServer

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package akka.cloud.cluster package akka.cluster
import org.apache.zookeeper._ import org.apache.zookeeper._
import org.apache.zookeeper.Watcher.Event._ import org.apache.zookeeper.Watcher.Event._
@ -26,6 +26,7 @@ import ClusterProtocol._
import RemoteDaemonMessageType._ import RemoteDaemonMessageType._
import akka.util._ import akka.util._
import Helpers._
import akka.actor._ import akka.actor._
import akka.actor.Actor._ import akka.actor.Actor._
import akka.event.EventHandler import akka.event.EventHandler
@ -36,10 +37,8 @@ import akka.serialization.{Format, Serializer}
import akka.serialization.Compression.LZF import akka.serialization.Compression.LZF
import akka.AkkaException import akka.AkkaException
import akka.cloud.common.JMX //import akka.cloud.monitoring.Monitoring
import akka.cloud.common.Util._ import akka.cluster.zookeeper._
import akka.cloud.monitoring.Monitoring
import akka.cloud.zookeeper._
import com.eaio.uuid.UUID import com.eaio.uuid.UUID
@ -113,14 +112,14 @@ final case class NodeAddress(
} }
case class ActorAddress( case class ActorAddress(
actorUuid: UUID = null, uuid: UUID = null,
actorId: String = Cluster.EMPTY_STRING, address: String = Cluster.EMPTY_STRING,
actorClassName: String = Cluster.EMPTY_STRING) className: String = Cluster.EMPTY_STRING)
object ActorAddress { object ActorAddress {
def forUuid(actorUuid: UUID) = ActorAddress(actorUuid, Cluster.EMPTY_STRING, Cluster.EMPTY_STRING) def forUuid(uuid: UUID) = ActorAddress(uuid, Cluster.EMPTY_STRING, Cluster.EMPTY_STRING)
def forId(actorId: String) = ActorAddress(null, actorId, Cluster.EMPTY_STRING) def forAddress(address: String) = ActorAddress(null, address, Cluster.EMPTY_STRING)
def forClassName(actorClassName: String) = ActorAddress(null, actorClassName, Cluster.EMPTY_STRING) def forClassName(className: String) = ActorAddress(null, className, Cluster.EMPTY_STRING)
} }
/** /**
@ -133,12 +132,12 @@ object Cluster {
val UUID_PREFIX = "uuid:".intern val UUID_PREFIX = "uuid:".intern
// config options // config options
val zooKeeperServers = config.getString("akka.cloud.cluster.zookeeper-server-addresses", "localhost:2181") val zooKeeperServers = config.getString("akka.cluster.zookeeper-server-addresses", "localhost:2181")
val remoteServerPort = config.getInt("akka.cloud.cluster.remote-server-port", 2552) val remoteServerPort = config.getInt("akka.cluster.remote-server-port", 2552)
val sessionTimeout = Duration(config.getInt("akka.cloud.cluster.session-timeout", 60), TIME_UNIT).toMillis.toInt val sessionTimeout = Duration(config.getInt("akka.cluster.session-timeout", 60), TIME_UNIT).toMillis.toInt
val connectionTimeout = Duration(config.getInt("akka.cloud.cluster.connection-timeout", 60), TIME_UNIT).toMillis.toInt val connectionTimeout = Duration(config.getInt("akka.cluster.connection-timeout", 60), TIME_UNIT).toMillis.toInt
val maxTimeToWaitUntilConnected = Duration(config.getInt("akka.cloud.cluster.max-time-to-wait-until-connected", 30), TIME_UNIT).toMillis.toInt val maxTimeToWaitUntilConnected = Duration(config.getInt("akka.cluster.max-time-to-wait-until-connected", 30), TIME_UNIT).toMillis.toInt
val shouldCompressData = config.getBool("akka.cloud.cluster.use-compression", false) val shouldCompressData = config.getBool("akka.cluster.use-compression", false)
val enableJMX = config.getBool("akka.enable-jmx", true) val enableJMX = config.getBool("akka.enable-jmx", true)
/** /**
@ -409,14 +408,14 @@ class ClusterNode private[akka] (
case RemoteClientDisconnected(client, address) => client.shutdownClientModule case RemoteClientDisconnected(client, address) => client.shutdownClientModule
case _ => //ignore other case _ => //ignore other
} }
}).start }, "akka.cluster.remoteClientLifeCycleListener").start
val remoteDaemon = actorOf(new RemoteClusterDaemon(this)).start val remoteDaemon = actorOf(new RemoteClusterDaemon(this), RemoteClusterDaemon.ADDRESS).start
val remoteService: RemoteSupport = { val remoteService: RemoteSupport = {
val remote = new akka.remote.netty.NettyRemoteSupport val remote = new akka.remote.netty.NettyRemoteSupport
remote.start(nodeAddress.hostname, nodeAddress.port) remote.start(nodeAddress.hostname, nodeAddress.port)
remote.register(RemoteClusterDaemon.ID, remoteDaemon) remote.register(RemoteClusterDaemon.ADDRESS, remoteDaemon)
remote.addListener(remoteClientLifeCycleListener) remote.addListener(remoteClientLifeCycleListener)
remote remote
} }
@ -530,7 +529,7 @@ class ClusterNode private[akka] (
remoteDaemon.stop remoteDaemon.stop
// for monitoring remote listener // for monitoring remote listener
registry.actors.filter(remoteService.hasListener).foreach(_.stop) registry.local.actors.filter(remoteService.hasListener).foreach(_.stop)
replicaConnections.clear replicaConnections.clear
updateNodes(_ - nodeAddress) updateNodes(_ - nodeAddress)
@ -689,11 +688,11 @@ class ClusterNode private[akka] (
case e: ZkNodeExistsException => zkClient.writeData(actorRegistryFormatNodePathFor(uuid), format) case e: ZkNodeExistsException => zkClient.writeData(actorRegistryFormatNodePathFor(uuid), format)
} }
// create UUID -> ID registry // create UUID -> ADDRESS registry
try { try {
zkClient.createPersistent(actorRegistryActorIdNodePathFor(uuid), actorRef.id) zkClient.createPersistent(actorRegistryactorAddressNodePathFor(uuid), actorRef.address)
} catch { } catch {
case e: ZkNodeExistsException => zkClient.writeData(actorRegistryActorIdNodePathFor(uuid), actorRef.id) case e: ZkNodeExistsException => zkClient.writeData(actorRegistryactorAddressNodePathFor(uuid), actorRef.address)
} }
// create UUID -> class name registry // create UUID -> class name registry
@ -710,8 +709,8 @@ class ClusterNode private[akka] (
ignore[ZkNodeExistsException]( zkClient.createPersistent(actorLocationsNodePathFor(uuid)) ) ignore[ZkNodeExistsException]( zkClient.createPersistent(actorLocationsNodePathFor(uuid)) )
// create ID -> UUIDs registry // create ID -> UUIDs registry
ignore[ZkNodeExistsException]( zkClient.createPersistent(actorIdToUuidsNodePathFor(actorRef.id)) ) ignore[ZkNodeExistsException]( zkClient.createPersistent(actorAddressToUuidsNodePathFor(actorRef.address)) )
ignore[ZkNodeExistsException]( zkClient.createPersistent("%s/%s".format(actorIdToUuidsNodePathFor(actorRef.id), uuid)) ) ignore[ZkNodeExistsException]( zkClient.createPersistent("%s/%s".format(actorAddressToUuidsNodePathFor(actorRef.address), uuid)) )
// create class name -> UUIDs registry // create class name -> UUIDs registry
ignore[ZkNodeExistsException]( zkClient.createPersistent(actorClassNameToUuidsNodePathFor(actorRef.actorClassName)) ) ignore[ZkNodeExistsException]( zkClient.createPersistent(actorClassNameToUuidsNodePathFor(actorRef.actorClassName)) )
@ -735,46 +734,46 @@ class ClusterNode private[akka] (
* clusterNode remove classOf[MyActor] * clusterNode remove classOf[MyActor]
* </pre> * </pre>
*/ */
def remove[T <: Actor](actorClass: Class[T]): ClusterNode = remove(ActorAddress(actorClassName = actorClass.getName)) def remove[T <: Actor](actorClass: Class[T]): ClusterNode = remove(ActorAddress(className = actorClass.getName))
/** /**
* Removes actor with UUID from the cluster. * Removes actor with UUID from the cluster.
*/ */
def remove(actorAddress: ActorAddress): ClusterNode = { def remove(actorAddress: ActorAddress): ClusterNode = {
def removeByUuid(actorUuid: UUID) = { def removeByUuid(uuid: UUID) = {
releaseActorOnAllNodes(actorUuid) releaseActorOnAllNodes(uuid)
locallyCheckedOutActors.remove(actorUuid) locallyCheckedOutActors.remove(uuid)
// warning: ordering matters here // warning: ordering matters here
ignore[ZkNoNodeException](zkClient.deleteRecursive(actorIdToUuidsNodePathFor(actorIdForUuid(actorUuid)))) // remove ID to UUID mapping ignore[ZkNoNodeException](zkClient.deleteRecursive(actorAddressToUuidsNodePathFor(actorAddressForUuid(uuid)))) // remove ID to UUID mapping
ignore[ZkNoNodeException](zkClient.deleteRecursive(actorClassNameToUuidsNodePathFor(actorClassNameForUuid(actorUuid)))) // remove class name to UUID mapping ignore[ZkNoNodeException](zkClient.deleteRecursive(actorClassNameToUuidsNodePathFor(actorClassNameForUuid(uuid)))) // remove class name to UUID mapping
ignore[ZkNoNodeException](zkClient.deleteRecursive(actorAtAddressNodePathFor(nodeAddress.nodeName, actorUuid))) ignore[ZkNoNodeException](zkClient.deleteRecursive(actorAtAddressNodePathFor(nodeAddress.nodeName, uuid)))
ignore[ZkNoNodeException](zkClient.deleteRecursive(actorRegistryNodePathFor(actorUuid))) ignore[ZkNoNodeException](zkClient.deleteRecursive(actorRegistryNodePathFor(uuid)))
ignore[ZkNoNodeException](zkClient.deleteRecursive(actorLocationsNodePathFor(actorUuid))) ignore[ZkNoNodeException](zkClient.deleteRecursive(actorLocationsNodePathFor(uuid)))
} }
isConnected ifOn { isConnected ifOn {
// remove by UUID // remove by UUID
if (actorAddress.actorUuid ne null) { if (actorAddress.uuid ne null) {
EventHandler.debug(this, EventHandler.debug(this,
"Removing actor with UUID [%s] from cluster".format(actorAddress.actorUuid)) "Removing actor with UUID [%s] from cluster".format(actorAddress.uuid))
removeByUuid(actorAddress.actorUuid) removeByUuid(actorAddress.uuid)
// remove by ID // remove by ID
} else if (actorAddress.actorId != EMPTY_STRING) { } else if (actorAddress.address != EMPTY_STRING) {
EventHandler.debug(this, EventHandler.debug(this,
"Removing actor(s) with ID [%s] from cluster".format(actorAddress.actorId)) "Removing actor(s) with ID [%s] from cluster".format(actorAddress.address))
uuidsForActorId(actorAddress.actorId) foreach (uuid => removeByUuid(uuid)) uuidsForActorAddress(actorAddress.address) foreach (uuid => removeByUuid(uuid))
// remove by class name // remove by class name
} else if (actorAddress.actorClassName != EMPTY_STRING) { } else if (actorAddress.className != EMPTY_STRING) {
EventHandler.debug(this, EventHandler.debug(this,
"Removing actor(s) with class name [%s] from cluster".format(actorAddress.actorClassName)) "Removing actor(s) with class name [%s] from cluster".format(actorAddress.className))
uuidsForActorClassName(actorAddress.actorClassName) foreach (uuid => removeByUuid(uuid)) uuidsForActorClassName(actorAddress.className) foreach (uuid => removeByUuid(uuid))
} else throw new IllegalArgumentException( } else throw new IllegalArgumentException(
"You need to pass in at least one of 'actorUuid' or 'actorId' or 'actorClassName' to 'ClusterNode.remove(..)'") "You need to pass in at least one of 'uuid' or 'actorAddress' or 'className' to 'ClusterNode.remove(..)'")
} }
this this
} }
@ -919,48 +918,48 @@ class ClusterNode private[akka] (
def registerClusterActorRefForAddress(actorRef: ClusterActorRef, addresses: Array[(UUID, InetSocketAddress)]) = def registerClusterActorRefForAddress(actorRef: ClusterActorRef, addresses: Array[(UUID, InetSocketAddress)]) =
addresses foreach { case (_, address) => clusterActorRefs.put(address, actorRef) } addresses foreach { case (_, address) => clusterActorRefs.put(address, actorRef) }
def refByUuid(actorUuid: UUID): ActorRef = { def refByUuid(uuid: UUID): ActorRef = {
val actorClassName = actorClassNameForUuid(actorUuid) val className = actorClassNameForUuid(uuid)
val actor = Router newRouter ( val actor = Router newRouter (
router, addresses, router, addresses,
uuidToString(actorUuid), actorClassName, uuidToString(uuid), className,
Cluster.lookupLocalhostName, Cluster.remoteServerPort, // set it to local hostname:port Cluster.lookupLocalhostName, Cluster.remoteServerPort, // set it to local hostname:port
Actor.TIMEOUT, actorType) Actor.TIMEOUT, actorType)
registerClusterActorRefForAddress(actor, addresses) registerClusterActorRefForAddress(actor, addresses)
actor actor
} }
def refById(actorId: String): ActorRef = { def refById(actorAddress: String): ActorRef = {
val uuids = uuidsForActorId(actorId) val uuids = uuidsForActorAddress(actorAddress)
val actorClassName = uuids.map(uuid => actorClassNameForUuid(uuid)).head val className = uuids.map(uuid => actorClassNameForUuid(uuid)).head
if (actorClassName eq null) throw new IllegalStateException( if (className eq null) throw new IllegalStateException(
"Actor class name for actor with UUID [" + uuids.head + "] could not be retrieved") "Actor class name for actor with UUID [" + uuids.head + "] could not be retrieved")
val actor = Router newRouter ( val actor = Router newRouter (
router, addresses, router, addresses,
actorId, actorClassName, actorAddress, className,
Cluster.lookupLocalhostName, Cluster.remoteServerPort, // set it to local hostname:port Cluster.lookupLocalhostName, Cluster.remoteServerPort, // set it to local hostname:port
Actor.TIMEOUT, actorType) Actor.TIMEOUT, actorType)
registerClusterActorRefForAddress(actor, addresses) registerClusterActorRefForAddress(actor, addresses)
actor actor
} }
def refByClassName(actorClassName: String): ActorRef = { def refByClassName(className: String): ActorRef = {
val actor = Router newRouter ( val actor = Router newRouter (
router, addresses, router, addresses,
actorClassName, actorClassName, className, className,
Cluster.lookupLocalhostName, Cluster.remoteServerPort, // set it to local hostname:port Cluster.lookupLocalhostName, Cluster.remoteServerPort, // set it to local hostname:port
Actor.TIMEOUT, actorType) Actor.TIMEOUT, actorType)
registerClusterActorRefForAddress(actor, addresses) registerClusterActorRefForAddress(actor, addresses)
actor actor
} }
val actorUuid = actorAddress.actorUuid val uuid = actorAddress.uuid
val actorId = actorAddress.actorId val address = actorAddress.address
val actorClassName = actorAddress.actorClassName val className = actorAddress.className
if ((actorUuid ne null) && actorId == EMPTY_STRING && actorClassName == EMPTY_STRING) refByUuid(actorUuid) if ((uuid ne null) && address == EMPTY_STRING && className == EMPTY_STRING) refByUuid(uuid)
else if (actorId != EMPTY_STRING && (actorUuid eq null) && actorClassName == EMPTY_STRING) refById(actorId) else if (address != EMPTY_STRING && (uuid eq null) && className == EMPTY_STRING) refById(address)
else if (actorClassName != EMPTY_STRING && (actorUuid eq null) && actorId == EMPTY_STRING) refByClassName(actorClassName) else if (className != EMPTY_STRING && (uuid eq null) && address == EMPTY_STRING) refByClassName(className)
else throw new IllegalArgumentException("You need to pass in either 'actorUuid' or 'actorId' or 'actorClassName' and only one of them") else throw new IllegalArgumentException("You need to pass in either 'uuid' or 'actorAddress' or 'className' and only one of them")
} else throw new ClusterException("Not connected to cluster") } else throw new ClusterException("Not connected to cluster")
/** /**
@ -990,7 +989,7 @@ class ClusterNode private[akka] (
/** /**
* Returns the IDs of all actors checked out on this node. * Returns the IDs of all actors checked out on this node.
*/ */
def idsForActorsInUse: Array[String] = actorIdsForUuids(uuidsForActorsInUse) def idsForActorsInUse: Array[String] = actorAddresssForUuids(uuidsForActorsInUse)
/** /**
* Returns the class names of all actors checked out on this node. * Returns the class names of all actors checked out on this node.
@ -1007,7 +1006,7 @@ class ClusterNode private[akka] (
/** /**
* Returns the IDs of all actors registered in this cluster. * Returns the IDs of all actors registered in this cluster.
*/ */
def idsForClusteredActors: Array[String] = actorIdsForUuids(uuidsForClusteredActors) def idsForClusteredActors: Array[String] = actorAddresssForUuids(uuidsForClusteredActors)
/** /**
* Returns the class names of all actors registered in this cluster. * Returns the class names of all actors registered in this cluster.
@ -1017,15 +1016,15 @@ class ClusterNode private[akka] (
/** /**
* Returns the actor id for the actor with a specific UUID. * Returns the actor id for the actor with a specific UUID.
*/ */
def actorIdForUuid(uuid: UUID): String = if (isConnected.isOn) { def actorAddressForUuid(uuid: UUID): String = if (isConnected.isOn) {
try { zkClient.readData(actorRegistryActorIdNodePathFor(uuid)).asInstanceOf[String] } try { zkClient.readData(actorRegistryactorAddressNodePathFor(uuid)).asInstanceOf[String] }
catch { case e: ZkNoNodeException => "" } catch { case e: ZkNoNodeException => "" }
} else "" } else ""
/** /**
* Returns the actor ids for all the actors with a specific UUID. * Returns the actor ids for all the actors with a specific UUID.
*/ */
def actorIdsForUuids(uuids: Array[UUID]): Array[String] = uuids map (actorIdForUuid(_)) filter (_ != "") def actorAddresssForUuids(uuids: Array[UUID]): Array[String] = uuids map (actorAddressForUuid(_)) filter (_ != "")
/** /**
* Returns the actor class name for the actor with a specific UUID. * Returns the actor class name for the actor with a specific UUID.
@ -1043,16 +1042,16 @@ class ClusterNode private[akka] (
/** /**
* Returns the actor UUIDs for actor ID. * Returns the actor UUIDs for actor ID.
*/ */
def uuidsForActorId(actorId: String): Array[UUID] = if (isConnected.isOn) { def uuidsForActorAddress(actorAddress: String): Array[UUID] = if (isConnected.isOn) {
try { zkClient.getChildren(actorIdToUuidsNodePathFor(actorId)).toList.map(new UUID(_)).toArray.asInstanceOf[Array[UUID]] } try { zkClient.getChildren(actorAddressToUuidsNodePathFor(actorAddress)).toList.map(new UUID(_)).toArray.asInstanceOf[Array[UUID]] }
catch { case e: ZkNoNodeException => Array[UUID]() } catch { case e: ZkNoNodeException => Array[UUID]() }
} else Array.empty[UUID] } else Array.empty[UUID]
/** /**
* Returns the actor UUIDs for actor class name. * Returns the actor UUIDs for actor class name.
*/ */
def uuidsForActorClassName(actorClassName: String): Array[UUID] = if (isConnected.isOn) { def uuidsForActorClassName(className: String): Array[UUID] = if (isConnected.isOn) {
try { zkClient.getChildren(actorClassNameToUuidsNodePathFor(actorClassName)).toList.map(new UUID(_)).toArray.asInstanceOf[Array[UUID]] } try { zkClient.getChildren(actorClassNameToUuidsNodePathFor(className)).toList.map(new UUID(_)).toArray.asInstanceOf[Array[UUID]] }
catch { case e: ZkNoNodeException => Array[UUID]() } catch { case e: ZkNoNodeException => Array[UUID]() }
} else Array.empty[UUID] } else Array.empty[UUID]
@ -1103,7 +1102,7 @@ class ClusterNode private[akka] (
val uuids = val uuids =
try { zkClient.getChildren(actorsAtAddressNodePathFor(nodeName)).toList.map(new UUID(_)).toArray.asInstanceOf[Array[UUID]] } try { zkClient.getChildren(actorsAtAddressNodePathFor(nodeName)).toList.map(new UUID(_)).toArray.asInstanceOf[Array[UUID]] }
catch { case e: ZkNoNodeException => Array[UUID]() } catch { case e: ZkNoNodeException => Array[UUID]() }
actorIdsForUuids(uuids) actorAddresssForUuids(uuids)
} else Array.empty[String] } else Array.empty[String]
/** /**
@ -1270,23 +1269,23 @@ class ClusterNode private[akka] (
private[cluster] def configurationNodePathFor(key: String) = "%s/%s".format(CONFIGURATION_NODE, key) private[cluster] def configurationNodePathFor(key: String) = "%s/%s".format(CONFIGURATION_NODE, key)
private[cluster] def actorIdToUuidsNodePathFor(actorId: String) = "%s/%s".format(ACTOR_ID_TO_UUIDS_NODE, actorId.replace('.', '_')) private[cluster] def actorAddressToUuidsNodePathFor(actorAddress: String) = "%s/%s".format(ACTOR_ID_TO_UUIDS_NODE, actorAddress.replace('.', '_'))
private[cluster] def actorClassNameToUuidsNodePathFor(actorClassName: String) = "%s/%s".format(ACTOR_CLASS_TO_UUIDS_NODE, actorClassName) private[cluster] def actorClassNameToUuidsNodePathFor(className: String) = "%s/%s".format(ACTOR_CLASS_TO_UUIDS_NODE, className)
private[cluster] def actorLocationsNodePathFor(actorUuid: UUID) = "%s/%s".format(ACTOR_LOCATIONS_NODE, actorUuid) private[cluster] def actorLocationsNodePathFor(uuid: UUID) = "%s/%s".format(ACTOR_LOCATIONS_NODE, uuid)
private[cluster] def actorLocationsNodePathFor(actorUuid: UUID, node: NodeAddress) = private[cluster] def actorLocationsNodePathFor(uuid: UUID, node: NodeAddress) =
"%s/%s/%s".format(ACTOR_LOCATIONS_NODE, actorUuid, node.nodeName) "%s/%s/%s".format(ACTOR_LOCATIONS_NODE, uuid, node.nodeName)
private[cluster] def actorsAtAddressNodePathFor(node: String) = "%s/%s".format(ACTORS_AT_ADDRESS_NODE, node) private[cluster] def actorsAtAddressNodePathFor(node: String) = "%s/%s".format(ACTORS_AT_ADDRESS_NODE, node)
private[cluster] def actorAtAddressNodePathFor(node: String, uuid: UUID) = "%s/%s/%s".format(ACTORS_AT_ADDRESS_NODE, node, uuid) private[cluster] def actorAtAddressNodePathFor(node: String, uuid: UUID) = "%s/%s/%s".format(ACTORS_AT_ADDRESS_NODE, node, uuid)
private[cluster] def actorRegistryNodePathFor(actorUuid: UUID) = "%s/%s".format(ACTOR_REGISTRY_NODE, actorUuid) private[cluster] def actorRegistryNodePathFor(uuid: UUID) = "%s/%s".format(ACTOR_REGISTRY_NODE, uuid)
private[cluster] def actorRegistryFormatNodePathFor(actorUuid: UUID) = "%s/%s".format(actorRegistryNodePathFor(actorUuid), "format") private[cluster] def actorRegistryFormatNodePathFor(uuid: UUID) = "%s/%s".format(actorRegistryNodePathFor(uuid), "format")
private[cluster] def actorRegistryActorIdNodePathFor(actorUuid: UUID) = "%s/%s".format(actorRegistryNodePathFor(actorUuid), "id") private[cluster] def actorRegistryactorAddressNodePathFor(uuid: UUID) = "%s/%s".format(actorRegistryNodePathFor(uuid), "id")
private[cluster] def actorRegistryActorClassNameNodePathFor(actorUuid: UUID) = "%s/%s".format(actorRegistryNodePathFor(actorUuid), "class") private[cluster] def actorRegistryActorClassNameNodePathFor(uuid: UUID) = "%s/%s".format(actorRegistryNodePathFor(uuid), "class")
private[cluster] def actorRegistryAddressNodePathFor(actorUuid: UUID): String = "%s/%s".format(actorRegistryNodePathFor(actorUuid), "address") private[cluster] def actorRegistryAddressNodePathFor(uuid: UUID): String = "%s/%s".format(actorRegistryNodePathFor(uuid), "address")
private[cluster] def actorRegistryAddressNodePathFor(actorUuid: UUID, address: InetSocketAddress): String = private[cluster] def actorRegistryAddressNodePathFor(uuid: UUID, address: InetSocketAddress): String =
"%s/%s:%s".format(actorRegistryAddressNodePathFor(actorUuid), address.getHostName, address.getPort) "%s/%s:%s".format(actorRegistryAddressNodePathFor(uuid), address.getHostName, address.getPort)
private[cluster] def initializeNode = { private[cluster] def initializeNode = {
EventHandler.info(this, "Initializing cluster node [%s]".format(nodeAddress)) EventHandler.info(this, "Initializing cluster node [%s]".format(nodeAddress))
@ -1311,13 +1310,13 @@ class ClusterNode private[akka] (
} }
private def actorUuidsForActorAddress(actorAddress: ActorAddress): Array[UUID] = { private def actorUuidsForActorAddress(actorAddress: ActorAddress): Array[UUID] = {
val actorUuid = actorAddress.actorUuid val uuid = actorAddress.uuid
val actorId = actorAddress.actorId val address = actorAddress.address
val actorClassName = actorAddress.actorClassName val className = actorAddress.className
if ((actorUuid ne null) && actorId == EMPTY_STRING && actorClassName == EMPTY_STRING) Array(actorUuid) if ((uuid ne null) && address == EMPTY_STRING && className == EMPTY_STRING) Array(uuid)
else if (actorId != EMPTY_STRING && (actorUuid eq null) && actorClassName == EMPTY_STRING) uuidsForActorId(actorId) else if (address != EMPTY_STRING && (uuid eq null) && className == EMPTY_STRING) uuidsForActorAddress(address)
else if (actorClassName != EMPTY_STRING && (actorUuid eq null) && actorId == EMPTY_STRING) uuidsForActorClassName(actorClassName) else if (className != EMPTY_STRING && (uuid eq null) && address == EMPTY_STRING) uuidsForActorClassName(className)
else throw new IllegalArgumentException("You need to pass in either 'actorUuid' or 'actorId' or 'actorClassName' and only one of them") else throw new IllegalArgumentException("You need to pass in either 'uuid' or 'actorAddress' or 'className' and only one of them")
} filter (_ ne null) } filter (_ ne null)
/** /**
@ -1355,7 +1354,7 @@ class ClusterNode private[akka] (
membershipNodes foreach { node => membershipNodes foreach { node =>
if (!replicaConnections.contains(node)) { if (!replicaConnections.contains(node)) {
val address = addressForNode(node) val address = addressForNode(node)
val clusterDaemon = Actor.remote.actorFor(RemoteClusterDaemon.ID, address.getHostName, address.getPort) val clusterDaemon = Actor.remote.actorFor(RemoteClusterDaemon.ADDRESS, address.getHostName, address.getPort)
replicaConnections.put(node, (address, clusterDaemon)) replicaConnections.put(node, (address, clusterDaemon))
} }
} }
@ -1404,12 +1403,12 @@ class ClusterNode private[akka] (
"Migrating actors from failed node [%s] to node [%s]: Actor UUIDs [%s]" "Migrating actors from failed node [%s] to node [%s]: Actor UUIDs [%s]"
.format(failedNodeName, nodeAddress.nodeName, actorUuidsForFailedNode)) .format(failedNodeName, nodeAddress.nodeName, actorUuidsForFailedNode))
actorUuidsForFailedNode.foreach { actorUuid => actorUuidsForFailedNode.foreach { uuid =>
EventHandler.debug(this, EventHandler.debug(this,
"Cluster node [%s] has failed, migrating actor with UUID [%s] to [%s]" "Cluster node [%s] has failed, migrating actor with UUID [%s] to [%s]"
.format(failedNodeName, actorUuid, nodeAddress.nodeName)) .format(failedNodeName, uuid, nodeAddress.nodeName))
val actorAddress = ActorAddress(actorUuid = stringToUuid(actorUuid)) val actorAddress = ActorAddress(uuid = stringToUuid(uuid))
migrateWithoutCheckingThatActorResidesOnItsHomeNode( // since the ephemeral node is already gone, so can't check migrateWithoutCheckingThatActorResidesOnItsHomeNode( // since the ephemeral node is already gone, so can't check
NodeAddress(nodeAddress.clusterName, failedNodeName), nodeAddress, actorAddress) NodeAddress(nodeAddress.clusterName, failedNodeName), nodeAddress, actorAddress)
@ -1421,7 +1420,7 @@ class ClusterNode private[akka] (
homeAddress.setAccessible(true) homeAddress.setAccessible(true)
homeAddress.set(actor, Some(remoteServerAddress)) homeAddress.set(actor, Some(remoteServerAddress))
remoteService.register(actorUuid, actor) remoteService.register(uuid, actor)
} }
} }
@ -1448,7 +1447,7 @@ class ClusterNode private[akka] (
from: NodeAddress, to: NodeAddress, actorAddress: ActorAddress) { from: NodeAddress, to: NodeAddress, actorAddress: ActorAddress) {
actorUuidsForActorAddress(actorAddress) map { uuid => actorUuidsForActorAddress(actorAddress) map { uuid =>
val actorAddress = ActorAddress(actorUuid = uuid) val actorAddress = ActorAddress(uuid = uuid)
if (!isInUseOnNode(actorAddress, to)) { if (!isInUseOnNode(actorAddress, to)) {
release(actorAddress) release(actorAddress)
@ -1560,7 +1559,9 @@ class ClusterNode private[akka] (
} }
JMX.register(clusterJmxObjectName, clusterMBean) JMX.register(clusterJmxObjectName, clusterMBean)
Monitoring.registerLocalMBean(clusterJmxObjectName, clusterMBean)
// FIXME need monitoring to lookup the cluster MBean dynamically
// Monitoring.registerLocalMBean(clusterJmxObjectName, clusterMBean)
} }
} }
@ -1639,7 +1640,7 @@ trait ErrorHandler {
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
object RemoteClusterDaemon { object RemoteClusterDaemon {
val ID = "akka:cloud:cluster:daemon" val ADDRESS = "akka:cloud:cluster:daemon"
// FIXME configure functionServerDispatcher to what? // FIXME configure functionServerDispatcher to what?
val functionServerDispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("akka:cloud:cluster:function:server").build val functionServerDispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("akka:cloud:cluster:function:server").build
@ -1652,7 +1653,6 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
import RemoteClusterDaemon._ import RemoteClusterDaemon._
import Cluster._ import Cluster._
self.id = ID
self.dispatcher = Dispatchers.newThreadBasedDispatcher(self) self.dispatcher = Dispatchers.newThreadBasedDispatcher(self)
def receive: Receive = { def receive: Receive = {
@ -1663,28 +1663,28 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
case USE => case USE =>
if (message.hasActorUuid) { if (message.hasActorUuid) {
val uuid = uuidProtocolToUuid(message.getActorUuid) val uuid = uuidProtocolToUuid(message.getActorUuid)
val address = ActorAddress(actorUuid = uuid) val address = ActorAddress(uuid = uuid)
implicit val format: Format[Actor] = cluster formatForActor address implicit val format: Format[Actor] = cluster formatForActor address
val actors = cluster use address val actors = cluster use address
} else if (message.hasActorId) { } else if (message.hasActorAddress) {
val id = message.getActorId val id = message.getActorAddress
val address = ActorAddress(actorId = id) val address = ActorAddress(address = id)
implicit val format: Format[Actor] = cluster formatForActor address implicit val format: Format[Actor] = cluster formatForActor address
val actors = cluster use address val actors = cluster use address
} else if (message.hasActorClassName) { } else if (message.hasActorClassName) {
val actorClassName = message.getActorClassName val actorClassName = message.getActorClassName
val address = ActorAddress(actorClassName = actorClassName) val address = ActorAddress(className = actorClassName)
implicit val format: Format[Actor] = cluster formatForActor address implicit val format: Format[Actor] = cluster formatForActor address
val actors = cluster use address val actors = cluster use address
} else EventHandler.warning(this, } else EventHandler.warning(this,
"None of 'actorUuid', 'actorId' or 'actorClassName' is specified, ignoring remote cluster daemon command [%s]".format(message)) "None of 'uuid', 'actorAddress' or 'className' is specified, ignoring remote cluster daemon command [%s]".format(message))
case RELEASE => case RELEASE =>
if (message.hasActorUuid) { cluster release ActorAddress(actorUuid = uuidProtocolToUuid(message.getActorUuid)) } if (message.hasActorUuid) { cluster release ActorAddress(uuid = uuidProtocolToUuid(message.getActorUuid)) }
else if (message.hasActorId) { cluster release ActorAddress(actorId = message.getActorId) } else if (message.hasActorAddress) { cluster release ActorAddress(address = message.getActorAddress) }
else if (message.hasActorClassName) { cluster release ActorAddress(actorClassName = message.getActorClassName) } else if (message.hasActorClassName) { cluster release ActorAddress(className = message.getActorClassName) }
else EventHandler.warning(this, else EventHandler.warning(this,
"None of 'actorUuid', 'actorId' or 'actorClassName' is specified, ignoring remote cluster daemon command [%s]".format(message)) "None of 'uuid', 'actorAddress' or 'className' is specified, ignoring remote cluster daemon command [%s]".format(message))
case START => cluster.start case START => cluster.start

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package akka.cloud.cluster package akka.cluster
import Cluster._ import Cluster._
@ -22,16 +22,13 @@ class ClusterActorRef private[akka] (
actorAddresses: Array[Tuple2[UUID, InetSocketAddress]], actorAddresses: Array[Tuple2[UUID, InetSocketAddress]],
val serviceId: String, val serviceId: String,
actorClassName: String, actorClassName: String,
hostname: String,
port: Int,
timeout: Long, timeout: Long,
actorType: ActorType, actorType: ActorType,
val replicationStrategy: ReplicationStrategy) val replicationStrategy: ReplicationStrategy)
extends RemoteActorRef(serviceId, actorClassName, hostname, port, timeout, None, actorType) { extends RemoteActorRef(serviceId, actorClassName, timeout, None, actorType) {
this: ClusterActorRef with Router.Router => this: ClusterActorRef with Router.Router =>
EventHandler.debug(this, "Creating a ClusterActorRef [%s] for Actor [%s] on [%s:%s]" EventHandler.debug(this, "Creating a ClusterActorRef [%s] for Actor [%s]".format(serviceId, actorClassName))
.format(serviceId, actorClassName, hostname, port))
private[akka] val addresses = new AtomicReference[Map[InetSocketAddress, ActorRef]]( private[akka] val addresses = new AtomicReference[Map[InetSocketAddress, ActorRef]](
createConnections(actorAddresses, actorClassName)) createConnections(actorAddresses, actorClassName))
@ -72,7 +69,6 @@ class ClusterActorRef private[akka] (
private def createRemoteActorRef(uuid: UUID, address: InetSocketAddress) = { private def createRemoteActorRef(uuid: UUID, address: InetSocketAddress) = {
RemoteActorRef( RemoteActorRef(
UUID_PREFIX + uuidToString(uuid), actorClassName, // clustered refs are always registered and looked up by UUID UUID_PREFIX + uuidToString(uuid), actorClassName, // clustered refs are always registered and looked up by UUID
address.getHostName, address.getPort,
Actor.TIMEOUT, None, actorType) Actor.TIMEOUT, None, actorType)
} }
} }

View file

@ -6,7 +6,7 @@
** |/ ** ** |/ **
\* */ \* */
package akka.cloud.cluster package akka.cluster
/** An implementation of Austin Appleby's MurmurHash 3.0 algorithm /** An implementation of Austin Appleby's MurmurHash 3.0 algorithm
* (32 bit version); reference: http://code.google.com/p/smhasher * (32 bit version); reference: http://code.google.com/p/smhasher

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package akka.cloud.cluster package akka.cluster
import Cluster._ import Cluster._
@ -38,18 +38,15 @@ object Router {
routerType match { routerType match {
case Direct => new ClusterActorRef( case Direct => new ClusterActorRef(
addresses, serviceId, actorClassName, addresses, serviceId, actorClassName, timeout,
hostname, port, timeout,
actorType, replicationStrategy) with Direct actorType, replicationStrategy) with Direct
case Random => new ClusterActorRef( case Random => new ClusterActorRef(
addresses, serviceId, actorClassName, addresses, serviceId, actorClassName, timeout,
hostname, port, timeout,
actorType, replicationStrategy) with Random actorType, replicationStrategy) with Random
case RoundRobin => new ClusterActorRef( case RoundRobin => new ClusterActorRef(
addresses, serviceId, actorClassName, addresses, serviceId, actorClassName, timeout,
hostname, port, timeout,
actorType, replicationStrategy) with RoundRobin actorType, replicationStrategy) with RoundRobin
} }
} }

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package akka.cloud.cluster package akka.cluster
import Cluster._ import Cluster._
@ -34,11 +34,10 @@ object ReplicationStrategy {
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class ReplicatedActorRef private[akka] (actorRef: ActorRef) extends ActorRef with ScalaActorRef { class ReplicatedActorRef private[akka] (actorRef: ActorRef, val address: String) extends ActorRef with ScalaActorRef {
private lazy val txLog = { private lazy val txLog = {
EventHandler.debug(this, "Creating a ReplicatedActorRef for Actor [%s] on [%s]" EventHandler.debug(this, "Creating a ReplicatedActorRef for Actor [%s]".format(actorClassName))
.format(actorClassName, homeAddress))
TransactionLog.newLogFor(uuid.toString) TransactionLog.newLogFor(uuid.toString)
} }
@ -62,18 +61,13 @@ class ReplicatedActorRef private[akka] (actorRef: ActorRef) extends ActorRef wit
override def getFaultHandler(): FaultHandlingStrategy = actorRef.getFaultHandler() override def getFaultHandler(): FaultHandlingStrategy = actorRef.getFaultHandler()
override def setLifeCycle(lifeCycle: LifeCycle): Unit = actorRef.setLifeCycle(lifeCycle) override def setLifeCycle(lifeCycle: LifeCycle): Unit = actorRef.setLifeCycle(lifeCycle)
override def getLifeCycle(): LifeCycle = actorRef.getLifeCycle override def getLifeCycle(): LifeCycle = actorRef.getLifeCycle
def homeAddress: Option[InetSocketAddress] = actorRef.homeAddress
def actorClass: Class[_ <: Actor] = actorRef.actorClass def actorClass: Class[_ <: Actor] = actorRef.actorClass
def actorClassName: String = actorRef.actorClassName def actorClassName: String = actorRef.actorClassName
def dispatcher_=(md: MessageDispatcher): Unit = actorRef.dispatcher_=(md) def dispatcher_=(md: MessageDispatcher): Unit = actorRef.dispatcher_=(md)
def dispatcher: MessageDispatcher = actorRef.dispatcher def dispatcher: MessageDispatcher = actorRef.dispatcher
def link(actorRef: ActorRef): Unit = actorRef.link(actorRef) def link(actorRef: ActorRef): Unit = actorRef.link(actorRef)
def unlink(actorRef: ActorRef): Unit = actorRef.unlink(actorRef) def unlink(actorRef: ActorRef): Unit = actorRef.unlink(actorRef)
def startLink(actorRef: ActorRef): Unit = actorRef.startLink(actorRef) def startLink(actorRef: ActorRef): ActorRef = actorRef.startLink(actorRef)
def spawn(clazz: Class[_ <: Actor]): ActorRef = actorRef.spawn(clazz)
def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long): ActorRef = actorRef.spawnRemote(clazz, hostname, port, timeout)
def spawnLink(clazz: Class[_ <: Actor]): ActorRef = actorRef.spawnLink(clazz)
def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long): ActorRef = actorRef.spawnLinkRemote(clazz, hostname, port, timeout)
def supervisor: Option[ActorRef] = actorRef.supervisor def supervisor: Option[ActorRef] = actorRef.supervisor
def linkedActors: JMap[Uuid, ActorRef] = actorRef.linkedActors def linkedActors: JMap[Uuid, ActorRef] = actorRef.linkedActors
protected[akka] def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit = actorRef.postMessageToMailbox(message, senderOption) protected[akka] def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit = actorRef.postMessageToMailbox(message, senderOption)
@ -89,5 +83,4 @@ class ReplicatedActorRef private[akka] (actorRef: ActorRef) extends ActorRef wit
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = actorRef.handleTrapExit(dead, reason) protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = actorRef.handleTrapExit(dead, reason)
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = actorRef.restart(reason, maxNrOfRetries, withinTimeRange) protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = actorRef.restart(reason, maxNrOfRetries, withinTimeRange)
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = actorRef.restartLinkedActors(reason, maxNrOfRetries, withinTimeRange) protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = actorRef.restartLinkedActors(reason, maxNrOfRetries, withinTimeRange)
protected[akka] def registerSupervisorAsRemoteActor: Option[Uuid] = actorRef.registerSupervisorAsRemoteActor
} }

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package akka.cloud.cluster package akka.cluster
import org.apache.bookkeeper.client.{BookKeeper, LedgerHandle, LedgerEntry, BKException, AsyncCallback} import org.apache.bookkeeper.client.{BookKeeper, LedgerHandle, LedgerEntry, BKException, AsyncCallback}
import org.apache.zookeeper.CreateMode import org.apache.zookeeper.CreateMode
@ -15,7 +15,7 @@ import akka.event.EventHandler
import akka.dispatch.{DefaultCompletableFuture, CompletableFuture} import akka.dispatch.{DefaultCompletableFuture, CompletableFuture}
import akka.AkkaException import akka.AkkaException
import akka.cloud.zookeeper._ import akka.cluster.zookeeper._
import java.util.Enumeration import java.util.Enumeration

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package akka.cloud.zookeeper package akka.cluster.zookeeper
import org.I0Itec.zkclient._ import org.I0Itec.zkclient._
import org.I0Itec.zkclient.serialize._ import org.I0Itec.zkclient.serialize._

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package akka.cloud.zookeeper package akka.cluster.zookeeper
import org.I0Itec.zkclient._ import org.I0Itec.zkclient._
import org.apache.commons.io.FileUtils import org.apache.commons.io.FileUtils

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package akka.cloud.zookeeper package akka.cluster.zookeeper
import akka.util.Duration import akka.util.Duration
import akka.util.duration._ import akka.util.duration._

View file

@ -2,13 +2,13 @@
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package akka.cloud.cluster package akka.cluster
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
import akka.cloud.zookeeper._ import akka.zookeeper._
import org.I0Itec.zkclient._ import org.I0Itec.zkclient._
object MultiNodeTest { object MultiNodeTest {

View file

@ -1,4 +1,4 @@
package akka.cloud.cluster package akka.cluster
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
@ -9,8 +9,7 @@ import org.I0Itec.zkclient._
import akka.actor._ import akka.actor._
import akka.actor.Actor._ import akka.actor.Actor._
import akka.serialization.{Serializer, SerializerBasedActorFormat} import akka.serialization.{Serializer, SerializerBasedActorFormat}
import akka.util.Helpers._
import akka.cloud.common.Util._
import java.util.concurrent.{ CyclicBarrier, TimeUnit } import java.util.concurrent.{ CyclicBarrier, TimeUnit }

View file

@ -4,7 +4,7 @@
package sample.cluster package sample.cluster
import akka.cloud.cluster._ import akka.cluster._
import akka.dispatch.Futures import akka.dispatch.Futures
/** /**

View file

@ -4,7 +4,7 @@
package sample.cluster package sample.cluster
import akka.cloud.cluster._ import akka.cluster._
import akka.actor._ import akka.actor._
import akka.actor.Actor._ import akka.actor.Actor._

View file

@ -4,8 +4,7 @@
package example.cluster package example.cluster
import akka.cloud.cluster._ import akka.cluster._
import akka.cloud.monitoring._
import akka.actor._ import akka.actor._
import akka.serialization.{Serializer, SerializerBasedActorFormat} import akka.serialization.{Serializer, SerializerBasedActorFormat}
@ -183,11 +182,11 @@ object PingPongMultiJvmNode1 {
pause("shutdown", "Ready to shutdown") pause("shutdown", "Ready to shutdown")
println("Stopping everything ...") println("Stopping everything ...")
Monitoring.stopLocalDaemons //Monitoring.stopLocalDaemons
MonitoringServer.stop //MonitoringServer.stop
Actor.remote.shutdown Actor.remote.shutdown
Actor.registry.shutdownAll Actor.registry.local.shutdownAll
node.stop node.stop
@ -231,7 +230,7 @@ class PongNode(number: Int) {
// clean up and stop // clean up and stop
Actor.remote.shutdown Actor.remote.shutdown
Actor.registry.shutdownAll Actor.registry.local.shutdownAll
node.stop node.stop
} }

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package akka.cloud.cluster package akka.cluster
import org.apache.bookkeeper.client.{BookKeeper, BKException} import org.apache.bookkeeper.client.{BookKeeper, BKException}
import BKException._ import BKException._

View file

@ -197,9 +197,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val akka_stm = project("akka-stm", "akka-stm", new AkkaStmProject(_), akka_actor) lazy val akka_stm = project("akka-stm", "akka-stm", new AkkaStmProject(_), akka_actor)
lazy val akka_typed_actor = project("akka-typed-actor", "akka-typed-actor", new AkkaTypedActorProject(_), akka_stm, akka_actor_tests) lazy val akka_typed_actor = project("akka-typed-actor", "akka-typed-actor", new AkkaTypedActorProject(_), akka_stm, akka_actor_tests)
// lazy val akka_remote = project("akka-remote", "akka-remote", new AkkaRemoteProject(_), akka_typed_actor) lazy val akka_remote = project("akka-remote", "akka-remote", new AkkaRemoteProject(_), akka_typed_actor)
// lazy val akka_zookeeper = project("akka-zookeeper", "akka-zookeeper", new AkkaZookeeperProject(_), akka_remote) lazy val akka_cluster = project("akka-cluster", "akka-cluster", new AkkaClusterProject(_), akka_remote)
// lazy val akka_cluster = project("akka-cluster", "akka-cluster", new AkkaClusterProject(_), akka_zookeeper)
lazy val akka_http = project("akka-http", "akka-http", new AkkaHttpProject(_), akka_actor) lazy val akka_http = project("akka-http", "akka-http", new AkkaHttpProject(_), akka_actor)
lazy val akka_slf4j = project("akka-slf4j", "akka-slf4j", new AkkaSlf4jProject(_), akka_actor) lazy val akka_slf4j = project("akka-slf4j", "akka-slf4j", new AkkaSlf4jProject(_), akka_actor)
@ -348,24 +347,17 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
} }
} }
// -------------------------------------------------------------------------------------------------------------------
// akka-zookeeper sub project
// -------------------------------------------------------------------------------------------------------------------
class AkkaZookeeperProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) {
val log4j = Dependencies.log4j
val zookeeper = Dependencies.zookeeper
val zookeeperLock = Dependencies.zookeeperLock
val zkClient = Dependencies.zkClient
val commons_io = Dependencies.commons_io
}
// ------------------------------------------------------------------------------------------------------------------- // -------------------------------------------------------------------------------------------------------------------
// akka-cluster sub project // akka-cluster sub project
// ------------------------------------------------------------------------------------------------------------------- // -------------------------------------------------------------------------------------------------------------------
class AkkaClusterProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) with MultiJvmTests { class AkkaClusterProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) with MultiJvmTests {
val bookkeeper = Dependencies.bookkeeper val bookkeeper = Dependencies.bookkeeper
val zookeeper = Dependencies.zookeeper
val zookeeperLock = Dependencies.zookeeperLock
val zkClient = Dependencies.zkClient
val commons_io = Dependencies.commons_io
val log4j = Dependencies.log4j
// test dependencies // test dependencies
@ -437,10 +429,10 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
new AkkaSampleAntsProject(_), akka_stm) new AkkaSampleAntsProject(_), akka_stm)
lazy val akka_sample_fsm = project("akka-sample-fsm", "akka-sample-fsm", lazy val akka_sample_fsm = project("akka-sample-fsm", "akka-sample-fsm",
new AkkaSampleFSMProject(_), akka_actor) new AkkaSampleFSMProject(_), akka_actor)
// lazy val akka_sample_remote = project("akka-sample-remote", "akka-sample-remote", lazy val akka_sample_remote = project("akka-sample-remote", "akka-sample-remote",
// new AkkaSampleRemoteProject(_), akka_remote) new AkkaSampleRemoteProject(_), akka_remote)
// lazy val akka_sample_chat = project("akka-sample-chat", "akka-sample-chat", lazy val akka_sample_chat = project("akka-sample-chat", "akka-sample-chat",
// new AkkaSampleChatProject(_), akka_remote) new AkkaSampleChatProject(_), akka_remote)
lazy val akka_sample_osgi = project("akka-sample-osgi", "akka-sample-osgi", lazy val akka_sample_osgi = project("akka-sample-osgi", "akka-sample-osgi",
new AkkaSampleOsgiProject(_), akka_actor) new AkkaSampleOsgiProject(_), akka_actor)