1. Added configuration option for 'preferred-nodes' for a clustered actor. The replica set is now tried to be satisfied by the nodes in the list of preferred nodes, if that is not possible, it is randomly selected among the rest.
2. Added test for it. 3. Fixed wrong Java fault-tolerance docs 4. Fixed race condition in maintenance of connections to new nodes Signed-off-by: Jonas Bonér <jonas@jonasboner.com>
This commit is contained in:
parent
e28db64def
commit
f2dd6bddb3
23 changed files with 269 additions and 299 deletions
|
|
@ -31,7 +31,7 @@ import Helpers._
|
|||
import akka.actor._
|
||||
import Actor._
|
||||
import Status._
|
||||
import DeploymentConfig.{ ReplicationScheme, ReplicationStrategy, Transient, WriteThrough, WriteBehind }
|
||||
import DeploymentConfig._
|
||||
|
||||
import akka.event.EventHandler
|
||||
import akka.dispatch.{ Dispatchers, Future }
|
||||
|
|
@ -54,7 +54,6 @@ import com.eaio.uuid.UUID
|
|||
import com.google.protobuf.ByteString
|
||||
|
||||
// FIXME add watch for each node that when the entry for the node is removed then the node shuts itself down
|
||||
// FIXME Provisioning data in ZK (file names etc) and files in S3 and on disk
|
||||
|
||||
/**
|
||||
* JMX MBean for the cluster service.
|
||||
|
|
@ -131,11 +130,20 @@ object Cluster {
|
|||
val shouldCompressData = config.getBool("akka.cluster.use-compression", false)
|
||||
val enableJMX = config.getBool("akka.enable-jmx", true)
|
||||
val remoteDaemonAckTimeout = Duration(config.getInt("akka.cluster.remote-daemon-ack-timeout", 30), TIME_UNIT).toMillis.toInt
|
||||
val excludeRefNodeInReplicaSet = config.getBool("akka.cluster.exclude-ref-node-in-replica-set", true)
|
||||
val includeRefNodeInReplicaSet = config.getBool("akka.cluster.include-ref-node-in-replica-set", true)
|
||||
|
||||
@volatile
|
||||
private var properties = Map.empty[String, String]
|
||||
|
||||
/**
|
||||
* Use to override JVM options such as <code>-Dakka.cluster.nodename=node1</code> etc.
|
||||
* Currently supported options are:
|
||||
* <pre>
|
||||
* Cluster setProperty ("akka.cluster.nodename", "node1")
|
||||
* Cluster setProperty ("akka.cluster.hostname", "darkstar.lan")
|
||||
* Cluster setProperty ("akka.cluster.port", "1234")
|
||||
* </pre>
|
||||
*/
|
||||
def setProperty(property: (String, String)) {
|
||||
properties = properties + property
|
||||
}
|
||||
|
|
@ -155,7 +163,7 @@ object Cluster {
|
|||
case None ⇒ Config.remoteServerPort
|
||||
}
|
||||
|
||||
val defaultSerializer = new SerializableSerializer
|
||||
val defaultZooKeeperSerializer = new SerializableSerializer
|
||||
|
||||
private val _zkServer = new AtomicReference[Option[ZkServer]](None)
|
||||
|
||||
|
|
@ -169,7 +177,7 @@ object Cluster {
|
|||
*/
|
||||
val node = {
|
||||
if (nodeAddress eq null) throw new IllegalArgumentException("NodeAddress can't be null")
|
||||
new DefaultClusterNode(nodeAddress, hostname, port, zooKeeperServers, defaultSerializer)
|
||||
new DefaultClusterNode(nodeAddress, hostname, port, zooKeeperServers, defaultZooKeeperSerializer)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -230,7 +238,7 @@ object Cluster {
|
|||
/**
|
||||
* Creates a new AkkaZkClient.
|
||||
*/
|
||||
def newZkClient(): AkkaZkClient = new AkkaZkClient(zooKeeperServers, sessionTimeout, connectionTimeout, defaultSerializer)
|
||||
def newZkClient(): AkkaZkClient = new AkkaZkClient(zooKeeperServers, sessionTimeout, connectionTimeout, defaultZooKeeperSerializer)
|
||||
|
||||
def createQueue(rootPath: String, blocking: Boolean = true) = new ZooKeeperQueue(node.zkClient, rootPath, blocking)
|
||||
|
||||
|
|
@ -364,7 +372,8 @@ class DefaultClusterNode private[akka] (
|
|||
|
||||
private[akka] val nodeConnections: ConcurrentMap[String, Tuple2[InetSocketAddress, ActorRef]] = {
|
||||
val conns = new ConcurrentHashMap[String, Tuple2[InetSocketAddress, ActorRef]]
|
||||
conns.put(nodeAddress.nodeName, (remoteServerAddress, remoteDaemon)) // add the remote connection to 'this' node as well, but as a 'local' actor
|
||||
if (includeRefNodeInReplicaSet)
|
||||
conns.put(nodeAddress.nodeName, (remoteServerAddress, remoteDaemon)) // add the remote connection to 'this' node as well, but as a 'local' actor
|
||||
conns
|
||||
}
|
||||
|
||||
|
|
@ -406,14 +415,14 @@ class DefaultClusterNode private[akka] (
|
|||
// =======================================
|
||||
|
||||
def start(): ClusterNode = {
|
||||
isConnected switchOn {
|
||||
if (isConnected.compareAndSet(false, true)) {
|
||||
initializeNode()
|
||||
}
|
||||
this
|
||||
}
|
||||
|
||||
def shutdown() {
|
||||
isConnected switchOff {
|
||||
if (isConnected.compareAndSet(true, false)) {
|
||||
ignore[ZkNoNodeException](zkClient.deleteRecursive(membershipNodePath))
|
||||
|
||||
locallyCachedMembershipNodes.clear()
|
||||
|
|
@ -633,9 +642,7 @@ class DefaultClusterNode private[akka] (
|
|||
replicationFactor: Int,
|
||||
replicationScheme: ReplicationScheme,
|
||||
serializeMailbox: Boolean,
|
||||
serializer: Serializer): ClusterNode = if (isConnected.isOn) {
|
||||
|
||||
import akka.serialization.ActorSerialization._
|
||||
serializer: Serializer): ClusterNode = if (isConnected.get) {
|
||||
|
||||
val serializerClassName = serializer.getClass.getName
|
||||
|
||||
|
|
@ -654,7 +661,7 @@ class DefaultClusterNode private[akka] (
|
|||
|
||||
// create ADDRESS -> Array[Byte] for actor registry
|
||||
try {
|
||||
zkClient.writeData(actorAddressRegistryPath, actorFactoryBytes) // FIXME store actor factory bytes in Data Grid not ZooKeeper
|
||||
zkClient.writeData(actorAddressRegistryPath, actorFactoryBytes)
|
||||
} catch {
|
||||
case e: ZkNoNodeException ⇒ // if not stored yet, store the actor
|
||||
zkClient.retryUntilConnected(new Callable[Either[String, Exception]]() {
|
||||
|
|
@ -684,13 +691,7 @@ class DefaultClusterNode private[akka] (
|
|||
ignore[ZkNodeExistsException](zkClient.createPersistent(actorAddressToUuidsPathFor(actorAddress)))
|
||||
}
|
||||
|
||||
import RemoteClusterDaemon._
|
||||
val command = RemoteDaemonMessageProtocol.newBuilder
|
||||
.setMessageType(USE)
|
||||
.setActorAddress(actorAddress)
|
||||
.build
|
||||
|
||||
nodeConnectionsForReplicationFactor(replicationFactor) foreach { connection ⇒ sendCommandToNode(connection, command, async = false) }
|
||||
useActorOnNodes(nodesForReplicationFactor(replicationFactor, Some(actorAddress)).toArray, actorAddress)
|
||||
|
||||
this
|
||||
} else throw new ClusterException("Not connected to cluster")
|
||||
|
|
@ -717,7 +718,7 @@ class DefaultClusterNode private[akka] (
|
|||
/**
|
||||
* Is the actor with uuid clustered or not?
|
||||
*/
|
||||
def isClustered(actorAddress: String): Boolean = if (isConnected.isOn) {
|
||||
def isClustered(actorAddress: String): Boolean = if (isConnected.get) {
|
||||
zkClient.exists(actorAddressRegistryPathFor(actorAddress))
|
||||
} else false
|
||||
|
||||
|
|
@ -729,7 +730,7 @@ class DefaultClusterNode private[akka] (
|
|||
/**
|
||||
* Is the actor with uuid in use or not?
|
||||
*/
|
||||
def isInUseOnNode(actorAddress: String, node: NodeAddress): Boolean = if (isConnected.isOn) {
|
||||
def isInUseOnNode(actorAddress: String, node: NodeAddress): Boolean = if (isConnected.get) {
|
||||
zkClient.exists(actorAddressToNodesPathFor(actorAddress, node.nodeName))
|
||||
} else false
|
||||
|
||||
|
|
@ -743,13 +744,11 @@ class DefaultClusterNode private[akka] (
|
|||
* Checks out an actor for use on this node, e.g. checked out as a 'LocalActorRef' but it makes it available
|
||||
* for remote access through lookup by its UUID.
|
||||
*/
|
||||
def use[T <: Actor](actorAddress: String, serializer: Serializer): Option[ActorRef] = if (isConnected.isOn) {
|
||||
def use[T <: Actor](actorAddress: String, serializer: Serializer): Option[ActorRef] = if (isConnected.get) {
|
||||
val nodeName = nodeAddress.nodeName
|
||||
|
||||
ignore[ZkNodeExistsException](zkClient.createEphemeral(actorAddressToNodesPathFor(actorAddress, nodeName)))
|
||||
|
||||
// FIXME should not grab bytes from ZK but load the class and instantiate it with newInstance
|
||||
|
||||
val actorFactoryPath = actorAddressRegistryPathFor(actorAddress)
|
||||
zkClient.retryUntilConnected(new Callable[Either[Exception, () ⇒ ActorRef]]() {
|
||||
def call: Either[Exception, () ⇒ ActorRef] = {
|
||||
|
|
@ -824,41 +823,40 @@ class DefaultClusterNode private[akka] (
|
|||
} else None
|
||||
|
||||
/**
|
||||
* Using (checking out) actor on all nodes in the cluster.
|
||||
* Using (checking out) actor on a specific set of nodes.
|
||||
*/
|
||||
def useActorOnAllNodes(actorAddress: String) {
|
||||
isConnected ifOn {
|
||||
EventHandler.debug(this,
|
||||
"Using (checking out) actor with address [%s] on all nodes in cluster".format(actorAddress))
|
||||
def useActorOnNodes(nodes: Array[String], actorAddress: String) {
|
||||
EventHandler.debug(this,
|
||||
"Sending command to nodes [%s] for checking out actor [%s]".format(nodes.mkString(", "), actorAddress))
|
||||
|
||||
if (isConnected.get) {
|
||||
|
||||
val command = RemoteDaemonMessageProtocol.newBuilder
|
||||
.setMessageType(USE)
|
||||
.setActorAddress(actorAddress)
|
||||
.build
|
||||
|
||||
nodeConnections.get(node) foreach {
|
||||
case (_, connection) ⇒ sendCommandToNode(connection, command, async = false)
|
||||
nodes foreach { node ⇒
|
||||
nodeConnections.get(node) foreach {
|
||||
case (_, connection) ⇒
|
||||
sendCommandToNode(connection, command, async = false)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Using (checking out) actor on all nodes in the cluster.
|
||||
*/
|
||||
def useActorOnAllNodes(actorAddress: String) {
|
||||
useActorOnNodes(membershipNodes, actorAddress)
|
||||
}
|
||||
|
||||
/**
|
||||
* Using (checking out) actor on a specific node.
|
||||
*/
|
||||
def useActorOnNode(node: String, actorAddress: String) {
|
||||
EventHandler.debug(this,
|
||||
"Sending command to node [%s] for checking out actor [%s]".format(node, actorAddress))
|
||||
|
||||
isConnected ifOn {
|
||||
nodeConnections.get(node) foreach {
|
||||
case (_, connection) ⇒
|
||||
val command = RemoteDaemonMessageProtocol.newBuilder
|
||||
.setMessageType(USE)
|
||||
.setActorAddress(actorAddress)
|
||||
.build
|
||||
sendCommandToNode(connection, command, async = false)
|
||||
}
|
||||
}
|
||||
useActorOnNodes(Array(node), actorAddress)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -875,7 +873,7 @@ class DefaultClusterNode private[akka] (
|
|||
|
||||
// FIXME 'Cluster.release' needs to notify all existing ClusterActorRef's that are using the instance that it is no longer available. Then what to do? Should we even remove this method?
|
||||
|
||||
isConnected ifOn {
|
||||
if (isConnected.get) {
|
||||
ignore[ZkNoNodeException](zkClient.delete(actorAddressToNodesPathFor(actorAddress, nodeAddress.nodeName)))
|
||||
|
||||
uuidsForActorAddress(actorAddress) foreach { uuid ⇒
|
||||
|
|
@ -892,13 +890,13 @@ class DefaultClusterNode private[akka] (
|
|||
* Releases (checking in) all actors with a specific address on all nodes in the cluster where the actor is in 'use'.
|
||||
*/
|
||||
private[akka] def releaseActorOnAllNodes(actorAddress: String) {
|
||||
isConnected ifOn {
|
||||
if (isConnected.get) {
|
||||
EventHandler.debug(this,
|
||||
"Releasing (checking in) all actors with address [%s] on all nodes in cluster".format(actorAddress))
|
||||
|
||||
val command = RemoteDaemonMessageProtocol.newBuilder
|
||||
.setMessageType(RELEASE)
|
||||
.setActorAddress(actorAddress) // FIXME rename to actorAddress in protobuf protocol
|
||||
.setActorAddress(actorAddress)
|
||||
.build
|
||||
|
||||
nodesForActorsInUseWithAddress(actorAddress) foreach { node ⇒
|
||||
|
|
@ -912,7 +910,7 @@ class DefaultClusterNode private[akka] (
|
|||
/**
|
||||
* Creates an ActorRef with a Router to a set of clustered actors.
|
||||
*/
|
||||
def ref(actorAddress: String, router: RouterType): ActorRef = if (isConnected.isOn) {
|
||||
def ref(actorAddress: String, router: RouterType): ActorRef = if (isConnected.get) {
|
||||
val addresses = addressesForActor(actorAddress)
|
||||
EventHandler.debug(this,
|
||||
"Checking out cluster actor ref with address [%s] and router [%s] on [%s] connected to [\n\t%s]"
|
||||
|
|
@ -936,7 +934,7 @@ class DefaultClusterNode private[akka] (
|
|||
*/
|
||||
def migrate(
|
||||
from: NodeAddress, to: NodeAddress, actorAddress: String) {
|
||||
isConnected ifOn {
|
||||
if (isConnected.get) {
|
||||
if (from eq null) throw new IllegalArgumentException("NodeAddress 'from' can not be 'null'")
|
||||
if (to eq null) throw new IllegalArgumentException("NodeAddress 'to' can not be 'null'")
|
||||
if (isInUseOnNode(actorAddress, from)) {
|
||||
|
|
@ -960,7 +958,7 @@ class DefaultClusterNode private[akka] (
|
|||
/**
|
||||
* Returns the UUIDs of all actors registered in this cluster.
|
||||
*/
|
||||
private[akka] def uuidsForClusteredActors: Array[UUID] = if (isConnected.isOn) {
|
||||
private[akka] def uuidsForClusteredActors: Array[UUID] = if (isConnected.get) {
|
||||
zkClient.getChildren(ACTOR_UUID_REGISTRY_PATH).toList.map(new UUID(_)).toArray.asInstanceOf[Array[UUID]]
|
||||
} else Array.empty[UUID]
|
||||
|
||||
|
|
@ -972,7 +970,7 @@ class DefaultClusterNode private[akka] (
|
|||
/**
|
||||
* Returns the actor id for the actor with a specific UUID.
|
||||
*/
|
||||
private[akka] def actorAddressForUuid(uuid: UUID): Option[String] = if (isConnected.isOn) {
|
||||
private[akka] def actorAddressForUuid(uuid: UUID): Option[String] = if (isConnected.get) {
|
||||
try {
|
||||
Some(zkClient.readData(actorUuidRegistryAddressPathFor(uuid)).asInstanceOf[String])
|
||||
} catch {
|
||||
|
|
@ -989,7 +987,7 @@ class DefaultClusterNode private[akka] (
|
|||
/**
|
||||
* Returns the actor UUIDs for actor ID.
|
||||
*/
|
||||
private[akka] def uuidsForActorAddress(actorAddress: String): Array[UUID] = if (isConnected.isOn) {
|
||||
private[akka] def uuidsForActorAddress(actorAddress: String): Array[UUID] = if (isConnected.get) {
|
||||
try {
|
||||
zkClient.getChildren(actorAddressToUuidsPathFor(actorAddress)).toList.toArray map {
|
||||
case c: CharSequence ⇒ new UUID(c)
|
||||
|
|
@ -1002,7 +1000,7 @@ class DefaultClusterNode private[akka] (
|
|||
/**
|
||||
* Returns the node names of all actors in use with UUID.
|
||||
*/
|
||||
private[akka] def nodesForActorsInUseWithAddress(actorAddress: String): Array[String] = if (isConnected.isOn) {
|
||||
private[akka] def nodesForActorsInUseWithAddress(actorAddress: String): Array[String] = if (isConnected.get) {
|
||||
try {
|
||||
zkClient.getChildren(actorAddressToNodesPathFor(actorAddress)).toList.toArray.asInstanceOf[Array[String]]
|
||||
} catch {
|
||||
|
|
@ -1013,7 +1011,7 @@ class DefaultClusterNode private[akka] (
|
|||
/**
|
||||
* Returns the UUIDs of all actors in use registered on a specific node.
|
||||
*/
|
||||
private[akka] def uuidsForActorsInUseOnNode(nodeName: String): Array[UUID] = if (isConnected.isOn) {
|
||||
private[akka] def uuidsForActorsInUseOnNode(nodeName: String): Array[UUID] = if (isConnected.get) {
|
||||
try {
|
||||
zkClient.getChildren(nodeToUuidsPathFor(nodeName)).toList.toArray map {
|
||||
case c: CharSequence ⇒ new UUID(c)
|
||||
|
|
@ -1026,7 +1024,7 @@ class DefaultClusterNode private[akka] (
|
|||
/**
|
||||
* Returns the addresses of all actors in use registered on a specific node.
|
||||
*/
|
||||
def addressesForActorsInUseOnNode(nodeName: String): Array[String] = if (isConnected.isOn) {
|
||||
def addressesForActorsInUseOnNode(nodeName: String): Array[String] = if (isConnected.get) {
|
||||
val uuids =
|
||||
try {
|
||||
zkClient.getChildren(nodeToUuidsPathFor(nodeName)).toList.toArray map {
|
||||
|
|
@ -1042,8 +1040,6 @@ class DefaultClusterNode private[akka] (
|
|||
* Returns Serializer for actor with specific address.
|
||||
*/
|
||||
def serializerForActor(actorAddress: String): Serializer = {
|
||||
// FIXME should only be 1 single class name per actor address - FIX IT
|
||||
|
||||
val serializerClassName =
|
||||
try {
|
||||
zkClient.readData(actorAddressRegistrySerializerPathFor(actorAddress), new Stat).asInstanceOf[String]
|
||||
|
|
@ -1276,34 +1272,68 @@ class DefaultClusterNode private[akka] (
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns a random set with replica connections of size 'replicationFactor'.
|
||||
* Default replicationFactor is 0, which returns the empty set.
|
||||
* Returns a random set with node names of size 'replicationFactor'.
|
||||
* Default replicationFactor is 0, which returns the empty Vector.
|
||||
*/
|
||||
private def nodeConnectionsForReplicationFactor(replicationFactor: Int = 0): Set[ActorRef] = {
|
||||
var replicas = HashSet.empty[ActorRef]
|
||||
if (replicationFactor < 1) return replicas
|
||||
private def nodesForReplicationFactor(replicationFactor: Int = 0, actorAddress: Option[String] = None): Vector[String] = {
|
||||
var replicaNames = Vector.empty[String]
|
||||
val nrOfClusterNodes = nodeConnections.size
|
||||
|
||||
val numberOfReplicas = nodeConnections.size
|
||||
val nodeConnectionsAsArray = nodeConnections.toList map {
|
||||
case (node, (address, actorRef)) ⇒ actorRef
|
||||
} // the ActorRefs
|
||||
if (replicationFactor < 1) return replicaNames
|
||||
if (nrOfClusterNodes < replicationFactor) throw new IllegalArgumentException(
|
||||
"Replication factor [" + replicationFactor +
|
||||
"] is greater than the number of available nodeNames [" + nrOfClusterNodes + "]")
|
||||
|
||||
if (numberOfReplicas < replicationFactor) {
|
||||
throw new IllegalArgumentException(
|
||||
"Replication factor [" + replicationFactor +
|
||||
"] is greater than the number of available nodes [" + numberOfReplicas + "]")
|
||||
} else if (numberOfReplicas == replicationFactor) {
|
||||
replicas = replicas ++ nodeConnectionsAsArray
|
||||
} else {
|
||||
val random = new java.util.Random(System.currentTimeMillis)
|
||||
while (replicas.size < replicationFactor) {
|
||||
val index = random.nextInt(numberOfReplicas)
|
||||
replicas = replicas + nodeConnectionsAsArray(index)
|
||||
val preferredNodes =
|
||||
if (actorAddress.isDefined) { // use 'preferred-nodes' in deployment config for the actor
|
||||
Deployer.deploymentFor(actorAddress.get) match {
|
||||
case Deploy(_, _, _, Clustered(nodes, _, _)) ⇒
|
||||
nodes map (node ⇒ Deployer.nodeNameFor(node)) take replicationFactor
|
||||
case _ ⇒
|
||||
throw new ClusterException("Actor [" + actorAddress.get + "] is not configured as clustered")
|
||||
}
|
||||
} else Vector.empty[String]
|
||||
|
||||
for {
|
||||
nodeName ← preferredNodes
|
||||
key ← nodeConnections.keys
|
||||
if key == nodeName
|
||||
} replicaNames = replicaNames :+ nodeName
|
||||
|
||||
val nrOfCurrentReplicaNames = replicaNames.size
|
||||
|
||||
val replicaSet =
|
||||
if (nrOfCurrentReplicaNames > replicationFactor) throw new IllegalStateException("Replica set is larger than replication factor")
|
||||
else if (nrOfCurrentReplicaNames == replicationFactor) replicaNames
|
||||
else {
|
||||
val random = new java.util.Random(System.currentTimeMillis)
|
||||
while (replicaNames.size < replicationFactor) {
|
||||
replicaNames = replicaNames :+ membershipNodes(random.nextInt(nrOfClusterNodes))
|
||||
}
|
||||
replicaNames
|
||||
}
|
||||
}
|
||||
replicas
|
||||
|
||||
EventHandler.debug(this,
|
||||
"Picked out replica set [%s] for actor [%s]".format(replicaSet.mkString(", "), actorAddress))
|
||||
|
||||
replicaSet
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a random set with replica connections of size 'replicationFactor'.
|
||||
* Default replicationFactor is 0, which returns the empty Vector.
|
||||
*/
|
||||
private def nodeConnectionsForReplicationFactor(replicationFactor: Int = 0, actorAddress: Option[String] = None): Vector[ActorRef] = {
|
||||
for {
|
||||
node ← nodesForReplicationFactor(replicationFactor, actorAddress)
|
||||
connectionOption ← nodeConnections.get(node)
|
||||
connection ← connectionOption
|
||||
actorRef ← connection._2
|
||||
} yield actorRef
|
||||
}
|
||||
|
||||
private val connectToAllNewlyArrivedMembershipNodesInClusterLock = new AtomicBoolean(false)
|
||||
|
||||
/**
|
||||
* Update the list of connections to other nodes in the cluster.
|
||||
*
|
||||
|
|
@ -1311,7 +1341,7 @@ class DefaultClusterNode private[akka] (
|
|||
*/
|
||||
private[cluster] def connectToAllNewlyArrivedMembershipNodesInCluster(
|
||||
newlyConnectedMembershipNodes: Traversable[String],
|
||||
newlyDisconnectedMembershipNodes: Traversable[String]): Map[String, InetSocketAddress] = { // to prevent race in startup (fetchMembershipNodes vs MembershipChildListener)
|
||||
newlyDisconnectedMembershipNodes: Traversable[String]): Map[String, InetSocketAddress] = {
|
||||
|
||||
// cache the disconnected connections in a map, needed for fail-over of these connections later
|
||||
var disconnectedConnections = Map.empty[String, InetSocketAddress]
|
||||
|
|
@ -1319,25 +1349,29 @@ class DefaultClusterNode private[akka] (
|
|||
disconnectedConnections += (node -> (nodeConnections(node) match { case (address, _) ⇒ address }))
|
||||
}
|
||||
|
||||
// remove connections to failed nodes
|
||||
newlyDisconnectedMembershipNodes foreach (nodeConnections.remove(_))
|
||||
if (connectToAllNewlyArrivedMembershipNodesInClusterLock.compareAndSet(false, true)) {
|
||||
try {
|
||||
// remove connections to failed nodes
|
||||
newlyDisconnectedMembershipNodes foreach (nodeConnections.remove(_))
|
||||
|
||||
// add connections newly arrived nodes
|
||||
newlyConnectedMembershipNodes foreach { node ⇒
|
||||
if (!nodeConnections.contains(node)) { // only connect to each replica once
|
||||
// add connections newly arrived nodes
|
||||
newlyConnectedMembershipNodes foreach { node ⇒
|
||||
if (!nodeConnections.contains(node)) { // only connect to each replica once
|
||||
|
||||
val addressOption = remoteSocketAddressForNode(node)
|
||||
if (addressOption.isDefined) {
|
||||
val address = addressOption.get
|
||||
remoteSocketAddressForNode(node) foreach { address ⇒
|
||||
EventHandler.debug(this,
|
||||
"Setting up connection to node with nodename [%s] and address [%s]".format(node, address))
|
||||
|
||||
EventHandler.debug(this,
|
||||
"Setting up connection to node with nodename [%s] and address [%s]".format(node, address))
|
||||
|
||||
val clusterDaemon = Actor.remote.actorFor(RemoteClusterDaemon.ADDRESS, address.getHostName, address.getPort).start()
|
||||
nodeConnections.put(node, (address, clusterDaemon))
|
||||
val clusterDaemon = Actor.remote.actorFor(RemoteClusterDaemon.ADDRESS, address.getHostName, address.getPort).start()
|
||||
nodeConnections.put(node, (address, clusterDaemon))
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
connectToAllNewlyArrivedMembershipNodesInClusterLock.set(false)
|
||||
}
|
||||
}
|
||||
|
||||
disconnectedConnections
|
||||
}
|
||||
|
||||
|
|
@ -1530,7 +1564,7 @@ class DefaultClusterNode private[akka] (
|
|||
|
||||
override def resign(): Unit = self.resign()
|
||||
|
||||
override def isConnected = self.isConnected.isOn
|
||||
override def isConnected = self.isConnected.get
|
||||
|
||||
override def getRemoteServerHostname = self.hostname
|
||||
|
||||
|
|
@ -1661,7 +1695,7 @@ object RemoteClusterDaemon {
|
|||
val ADDRESS = "akka-cluster-daemon".intern
|
||||
|
||||
// FIXME configure computeGridDispatcher to what?
|
||||
val computeGridDispatcher = Dispatchers.newDispatcher("akka:cloud:cluster:compute-grid").build
|
||||
val computeGridDispatcher = Dispatchers.newDispatcher("akka:compute-grid").build
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue