1. Implemented replication through transaction log, e.g. logging all messages and replaying them after actor migration
2. Added first replication test (out of many) 3. Improved ScalaDoc 4. Enhanced the remote protocol with replication info Signed-off-by: Jonas Bonér <jonas@jonasboner.com>
This commit is contained in:
parent
6117e599d6
commit
0b1ee758f5
36 changed files with 893 additions and 550 deletions
|
|
@ -18,7 +18,6 @@ class DeployerSpec extends WordSpec with MustMatchers {
|
|||
Deploy(
|
||||
"service-ping",
|
||||
LeastCPU,
|
||||
"akka.serialization.Format$Default$",
|
||||
Clustered(
|
||||
Vector(Node("node1")),
|
||||
Replicate(3),
|
||||
|
|
|
|||
|
|
@ -119,7 +119,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
|
|||
for (i ← 1 to 500) d ! i
|
||||
|
||||
try {
|
||||
latch.await(10 seconds)
|
||||
latch.await(20 seconds)
|
||||
} finally {
|
||||
// because t1 is much slower and thus has a bigger mailbox all the time
|
||||
t1Count.get must be < (t2Count.get)
|
||||
|
|
|
|||
|
|
@ -326,7 +326,7 @@ object Actor extends ListenerManagement {
|
|||
* </pre>
|
||||
*/
|
||||
def actorOf[T <: Actor](creator: ⇒ T, address: String): ActorRef = {
|
||||
createActor(address, () ⇒ new LocalActorRef(() ⇒ creator, address, Transient))
|
||||
createActor(address, () ⇒ new LocalActorRef(() ⇒ creator, address))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -349,7 +349,7 @@ object Actor extends ListenerManagement {
|
|||
* JAVA API
|
||||
*/
|
||||
def actorOf[T <: Actor](creator: Creator[T], address: String): ActorRef = {
|
||||
createActor(address, () ⇒ new LocalActorRef(() ⇒ creator.create, address, Transient))
|
||||
createActor(address, () ⇒ new LocalActorRef(() ⇒ creator.create, address))
|
||||
}
|
||||
|
||||
def localActorOf[T <: Actor: Manifest]: ActorRef = {
|
||||
|
|
@ -369,11 +369,11 @@ object Actor extends ListenerManagement {
|
|||
}
|
||||
|
||||
def localActorOf[T <: Actor](factory: ⇒ T): ActorRef = {
|
||||
new LocalActorRef(() ⇒ factory, new UUID().toString, Transient)
|
||||
new LocalActorRef(() ⇒ factory, new UUID().toString)
|
||||
}
|
||||
|
||||
def localActorOf[T <: Actor](factory: ⇒ T, address: String): ActorRef = {
|
||||
new LocalActorRef(() ⇒ factory, address, Transient)
|
||||
new LocalActorRef(() ⇒ factory, address)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -410,12 +410,12 @@ object Actor extends ListenerManagement {
|
|||
private[akka] def createActor(address: String, actorFactory: () ⇒ ActorRef): ActorRef = {
|
||||
Address.validate(address)
|
||||
registry.actorFor(address) match { // check if the actor for the address is already in the registry
|
||||
case Some(actorRef) ⇒ actorRef // it is -> return it
|
||||
case None ⇒ // it is not -> create it
|
||||
case Some(actorRef) ⇒ actorRef // it is -> return it
|
||||
case None ⇒ // it is not -> create it
|
||||
try {
|
||||
Deployer.deploymentFor(address) match {
|
||||
case Deploy(_, router, _, Local) ⇒ actorFactory() // create a local actor
|
||||
case deploy ⇒ newClusterActorRef(actorFactory, address, deploy)
|
||||
case Deploy(_, router, Local) ⇒ actorFactory() // create a local actor
|
||||
case deploy ⇒ newClusterActorRef(actorFactory, address, deploy)
|
||||
}
|
||||
} catch {
|
||||
case e: DeploymentException ⇒
|
||||
|
|
@ -438,17 +438,17 @@ object Actor extends ListenerManagement {
|
|||
|
||||
throw new ActorInitializationException(
|
||||
"Could not instantiate Actor of " + clazz +
|
||||
"\nMake sure Actor is NOT defined inside a class/trait," +
|
||||
"\nif so put it outside the class/trait, f.e. in a companion object," +
|
||||
"\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'.", cause)
|
||||
"\nMake sure Actor is NOT defined inside a class/trait," +
|
||||
"\nif so put it outside the class/trait, f.e. in a companion object," +
|
||||
"\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'.", cause)
|
||||
}
|
||||
}, address, Transient)
|
||||
}, address)
|
||||
}
|
||||
|
||||
private def newClusterActorRef(factory: () ⇒ ActorRef, address: String, deploy: Deploy): ActorRef = {
|
||||
deploy match {
|
||||
case Deploy(
|
||||
configAdress, router, serializerClassName,
|
||||
configAdress, router,
|
||||
Clustered(
|
||||
preferredHomeNodes,
|
||||
replicas,
|
||||
|
|
@ -461,14 +461,11 @@ object Actor extends ListenerManagement {
|
|||
if (!Actor.remote.isRunning) throw new IllegalStateException(
|
||||
"Remote server is not running")
|
||||
|
||||
val isHomeNode = preferredHomeNodes exists (home ⇒ DeploymentConfig.isHomeNode(home))
|
||||
val isHomeNode = DeploymentConfig.isHomeNode(preferredHomeNodes)
|
||||
val nrOfReplicas = DeploymentConfig.replicaValueFor(replicas)
|
||||
|
||||
def serializerErrorDueTo(reason: String) =
|
||||
throw new akka.config.ConfigurationException(
|
||||
"Could not create Serializer object [" + serializerClassName +
|
||||
"] for serialization of actor [" + address +
|
||||
"] since " + reason)
|
||||
def serializerErrorDueTo(reason: String) = throw new akka.config.ConfigurationException(
|
||||
"Could not create Serializer for actor [" + address + "] due to: " + reason)
|
||||
|
||||
val serializer: Serializer =
|
||||
Serialization.serializerFor(this.getClass).fold(x ⇒ serializerErrorDueTo(x.toString), s ⇒ s)
|
||||
|
|
@ -487,13 +484,16 @@ object Actor extends ListenerManagement {
|
|||
storeActorAndGetClusterRef(Transient, serializer)
|
||||
|
||||
case replication: Replication ⇒
|
||||
if (DeploymentConfig.routerTypeFor(router) != akka.routing.RouterType.Direct) throw new ConfigurationException(
|
||||
"Can't replicate an actor [" + address + "] configured with another router than \"direct\" - found [" + router + "]")
|
||||
|
||||
if (isHomeNode) { // stateful actor's home node
|
||||
cluster
|
||||
.use(address, serializer)
|
||||
.getOrElse(throw new ConfigurationException(
|
||||
"Could not check out actor [" + address + "] from cluster registry as a \"local\" actor"))
|
||||
|
||||
} else {
|
||||
// FIXME later manage different 'storage' (data grid) as well
|
||||
storeActorAndGetClusterRef(replication, serializer)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -9,10 +9,10 @@ import akka.dispatch._
|
|||
import akka.config._
|
||||
import akka.config.Supervision._
|
||||
import akka.util._
|
||||
import akka.serialization.{ Format, Serializer }
|
||||
import akka.serialization.{ Format, Serializer, Serialization }
|
||||
import ReflectiveAccess._
|
||||
import ClusterModule._
|
||||
import DeploymentConfig.{ ReplicationScheme, Replication, Transient, WriteThrough, WriteBehind }
|
||||
import DeploymentConfig.{ TransactionLog ⇒ TransactionLogConfig, _ }
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
|
|
@ -416,10 +416,7 @@ trait ActorRef extends ActorRefShared with ForwardableChannel with java.lang.Com
|
|||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class LocalActorRef private[akka] (
|
||||
private[this] val actorFactory: () ⇒ Actor,
|
||||
val address: String,
|
||||
replicationScheme: ReplicationScheme)
|
||||
class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, val address: String)
|
||||
extends ActorRef with ScalaActorRef {
|
||||
|
||||
protected[akka] val guard = new ReentrantGuard
|
||||
|
|
@ -447,49 +444,38 @@ class LocalActorRef private[akka] (
|
|||
|
||||
protected[akka] val actorInstance = guard.withGuard { new AtomicReference[Actor](newActor) }
|
||||
|
||||
private val isReplicated: Boolean = replicationScheme match {
|
||||
case _: Transient | Transient ⇒ false
|
||||
case _ ⇒ true
|
||||
}
|
||||
|
||||
def serializerErrorDueTo(reason: String) =
|
||||
throw new akka.config.ConfigurationException(
|
||||
"Could not create Serializer object [" + this.getClass.getName +
|
||||
"]")
|
||||
|
||||
private val serializer: Serializer =
|
||||
akka.serialization.Serialization.serializerFor(this.getClass).fold(x ⇒ serializerErrorDueTo(x.toString), s ⇒ s)
|
||||
Serialization.serializerFor(this.getClass).fold(x ⇒ serializerErrorDueTo(x.toString), s ⇒ s)
|
||||
|
||||
private lazy val replicationScheme: ReplicationScheme =
|
||||
DeploymentConfig.replicationSchemeFor(Deployer.deploymentFor(address)).getOrElse(Transient)
|
||||
|
||||
private lazy val isReplicated: Boolean = DeploymentConfig.isReplicated(replicationScheme)
|
||||
|
||||
private lazy val isWriteBehindReplication: Boolean = DeploymentConfig.isWriteBehindReplication(replicationScheme)
|
||||
|
||||
private lazy val replicationStorage: Either[TransactionLog, AnyRef] = {
|
||||
replicationScheme match {
|
||||
case _: Transient | Transient ⇒
|
||||
throw new IllegalStateException("Can not replicate 'transient' actor [" + toString + "]")
|
||||
if (DeploymentConfig.isReplicatedWithTransactionLog(replicationScheme)) {
|
||||
EventHandler.debug(this,
|
||||
"Creating a transaction log for Actor [%s] with replication strategy [%s]"
|
||||
.format(address, replicationScheme))
|
||||
|
||||
case Replication(storage, strategy) ⇒
|
||||
val isWriteBehind = strategy match {
|
||||
case _: WriteBehind | WriteBehind ⇒ true
|
||||
case _: WriteThrough | WriteThrough ⇒ false
|
||||
}
|
||||
Left(transactionLog.newLogFor(_uuid.toString, isWriteBehindReplication, replicationScheme))
|
||||
|
||||
storage match {
|
||||
case _: DeploymentConfig.TransactionLog | DeploymentConfig.TransactionLog ⇒
|
||||
EventHandler.debug(this,
|
||||
"Creating a transaction log for Actor [%s] with replication strategy [%s]"
|
||||
.format(address, replicationScheme))
|
||||
// Left(transactionLog.newLogFor(_uuid.toString, isWriteBehind, replicationScheme, serializer))
|
||||
// to fix null
|
||||
Left(transactionLog.newLogFor(_uuid.toString, isWriteBehind, replicationScheme, null))
|
||||
} else if (DeploymentConfig.isReplicatedWithDataGrid(replicationScheme)) {
|
||||
throw new ConfigurationException("Replication storage type \"data-grid\" is not yet supported")
|
||||
|
||||
case _: DeploymentConfig.DataGrid | DeploymentConfig.DataGrid ⇒
|
||||
throw new ConfigurationException("Replication storage type \"data-grid\" is not yet supported")
|
||||
|
||||
case unknown ⇒
|
||||
throw new ConfigurationException("Unknown replication storage type [" + unknown + "]")
|
||||
}
|
||||
} else {
|
||||
throw new ConfigurationException("Unknown replication storage type [" + replicationScheme + "]")
|
||||
}
|
||||
}
|
||||
|
||||
//If it was started inside "newActor", initialize it
|
||||
// If it was started inside "newActor", initialize it
|
||||
if (isRunning) initializeActorInstance
|
||||
|
||||
// used only for deserialization
|
||||
|
|
@ -501,10 +487,9 @@ class LocalActorRef private[akka] (
|
|||
__lifeCycle: LifeCycle,
|
||||
__supervisor: Option[ActorRef],
|
||||
__hotswap: Stack[PartialFunction[Any, Unit]],
|
||||
__factory: () ⇒ Actor,
|
||||
__replicationStrategy: ReplicationScheme) = {
|
||||
__factory: () ⇒ Actor) = {
|
||||
|
||||
this(__factory, __address, __replicationStrategy)
|
||||
this(__factory, __address)
|
||||
|
||||
_uuid = __uuid
|
||||
timeout = __timeout
|
||||
|
|
|
|||
|
|
@ -16,172 +16,6 @@ import akka.util.ReflectiveAccess._
|
|||
import akka.serialization._
|
||||
import akka.AkkaException
|
||||
|
||||
/**
|
||||
* Module holding the programmatic deployment configuration classes.
|
||||
* Defines the deployment specification.
|
||||
* Most values have defaults and can be left out.
|
||||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
object DeploymentConfig {
|
||||
|
||||
// --------------------------------
|
||||
// --- Deploy
|
||||
// --------------------------------
|
||||
case class Deploy(
|
||||
address: String,
|
||||
routing: Routing = Direct,
|
||||
format: String = Serializer.defaultSerializerName,
|
||||
scope: Scope = Local)
|
||||
|
||||
// --------------------------------
|
||||
// --- Routing
|
||||
// --------------------------------
|
||||
sealed trait Routing
|
||||
case class CustomRouter(router: AnyRef) extends Routing
|
||||
|
||||
// For Java API
|
||||
case class Direct() extends Routing
|
||||
case class RoundRobin() extends Routing
|
||||
case class Random() extends Routing
|
||||
case class LeastCPU() extends Routing
|
||||
case class LeastRAM() extends Routing
|
||||
case class LeastMessages() extends Routing
|
||||
|
||||
// For Scala API
|
||||
case object Direct extends Routing
|
||||
case object RoundRobin extends Routing
|
||||
case object Random extends Routing
|
||||
case object LeastCPU extends Routing
|
||||
case object LeastRAM extends Routing
|
||||
case object LeastMessages extends Routing
|
||||
|
||||
// --------------------------------
|
||||
// --- Scope
|
||||
// --------------------------------
|
||||
sealed trait Scope
|
||||
case class Clustered(
|
||||
preferredNodes: Iterable[Home] = Vector(Host("localhost")),
|
||||
replicas: Replicas = NoReplicas,
|
||||
replication: ReplicationScheme = Transient) extends Scope
|
||||
|
||||
// For Java API
|
||||
case class Local() extends Scope
|
||||
|
||||
// For Scala API
|
||||
case object Local extends Scope
|
||||
|
||||
// --------------------------------
|
||||
// --- Home
|
||||
// --------------------------------
|
||||
sealed trait Home
|
||||
case class Host(hostName: String) extends Home
|
||||
case class Node(nodeName: String) extends Home
|
||||
case class IP(ipAddress: String) extends Home
|
||||
|
||||
// --------------------------------
|
||||
// --- Replicas
|
||||
// --------------------------------
|
||||
sealed trait Replicas
|
||||
case class Replicate(factor: Int) extends Replicas {
|
||||
if (factor < 1) throw new IllegalArgumentException("Replicas factor can not be negative or zero")
|
||||
}
|
||||
|
||||
// For Java API
|
||||
case class AutoReplicate() extends Replicas
|
||||
case class NoReplicas() extends Replicas
|
||||
|
||||
// For Scala API
|
||||
case object AutoReplicate extends Replicas
|
||||
case object NoReplicas extends Replicas
|
||||
|
||||
// --------------------------------
|
||||
// --- Replication
|
||||
// --------------------------------
|
||||
sealed trait ReplicationScheme
|
||||
|
||||
// For Java API
|
||||
case class Transient() extends ReplicationScheme
|
||||
|
||||
// For Scala API
|
||||
case object Transient extends ReplicationScheme
|
||||
case class Replication(
|
||||
storage: ReplicationStorage,
|
||||
strategy: ReplicationStrategy) extends ReplicationScheme
|
||||
|
||||
// --------------------------------
|
||||
// --- ReplicationStorage
|
||||
// --------------------------------
|
||||
sealed trait ReplicationStorage
|
||||
|
||||
// For Java API
|
||||
case class TransactionLog() extends ReplicationStorage
|
||||
case class DataGrid() extends ReplicationStorage
|
||||
|
||||
// For Scala API
|
||||
case object TransactionLog extends ReplicationStorage
|
||||
case object DataGrid extends ReplicationStorage
|
||||
|
||||
// --------------------------------
|
||||
// --- ReplicationStrategy
|
||||
// --------------------------------
|
||||
sealed trait ReplicationStrategy
|
||||
|
||||
// For Java API
|
||||
case class WriteBehind() extends ReplicationStrategy
|
||||
case class WriteThrough() extends ReplicationStrategy
|
||||
|
||||
// For Scala API
|
||||
case object WriteBehind extends ReplicationStrategy
|
||||
case object WriteThrough extends ReplicationStrategy
|
||||
|
||||
// --------------------------------
|
||||
// --- Helper methods for parsing
|
||||
// --------------------------------
|
||||
|
||||
def nodeNameFor(home: Home): String = {
|
||||
home match {
|
||||
case Node(nodename) ⇒ nodename
|
||||
case Host("localhost") ⇒ Config.nodename
|
||||
case IP("0.0.0.0") ⇒ Config.nodename
|
||||
case IP("127.0.0.1") ⇒ Config.nodename
|
||||
case Host(hostname) ⇒ throw new UnsupportedOperationException("Specifying preferred node name by 'hostname' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]")
|
||||
case IP(address) ⇒ throw new UnsupportedOperationException("Specifying preferred node name by 'IP address' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]")
|
||||
}
|
||||
}
|
||||
|
||||
def isHomeNode(home: Home): Boolean = nodeNameFor(home) == Config.nodename
|
||||
|
||||
def replicaValueFor(replicas: Replicas): Int = replicas match {
|
||||
case Replicate(replicas) ⇒ replicas
|
||||
case AutoReplicate ⇒ -1
|
||||
case AutoReplicate() ⇒ -1
|
||||
case NoReplicas ⇒ 0
|
||||
case NoReplicas() ⇒ 0
|
||||
}
|
||||
|
||||
def routerTypeFor(routing: Routing): RouterType = routing match {
|
||||
case Direct ⇒ RouterType.Direct
|
||||
case Direct() ⇒ RouterType.Direct
|
||||
case RoundRobin ⇒ RouterType.RoundRobin
|
||||
case RoundRobin() ⇒ RouterType.RoundRobin
|
||||
case Random ⇒ RouterType.Random
|
||||
case Random() ⇒ RouterType.Random
|
||||
case LeastCPU ⇒ RouterType.LeastCPU
|
||||
case LeastCPU() ⇒ RouterType.LeastCPU
|
||||
case LeastRAM ⇒ RouterType.LeastRAM
|
||||
case LeastRAM() ⇒ RouterType.LeastRAM
|
||||
case LeastMessages ⇒ RouterType.LeastMessages
|
||||
case LeastMessages() ⇒ RouterType.LeastMessages
|
||||
case c: CustomRouter ⇒ throw new UnsupportedOperationException("Unknown Router [" + c + "]")
|
||||
}
|
||||
|
||||
def isReplicationAsync(strategy: ReplicationStrategy): Boolean = strategy match {
|
||||
case _: WriteBehind | WriteBehind ⇒ true
|
||||
case _: WriteThrough | WriteThrough ⇒ false
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Deployer maps actor deployments to actor addresses.
|
||||
*
|
||||
|
|
@ -230,8 +64,8 @@ object Deployer {
|
|||
}
|
||||
|
||||
def isLocal(deployment: Deploy): Boolean = deployment match {
|
||||
case Deploy(_, _, _, Local) ⇒ true
|
||||
case _ ⇒ false
|
||||
case Deploy(_, _, Local) ⇒ true
|
||||
case _ ⇒ false
|
||||
}
|
||||
|
||||
def isClustered(deployment: Deploy): Boolean = isLocal(deployment)
|
||||
|
|
@ -306,7 +140,7 @@ object Deployer {
|
|||
// --------------------------------
|
||||
val addressPath = "akka.actor.deployment." + address
|
||||
Config.config.getSection(addressPath) match {
|
||||
case None ⇒ Some(Deploy(address, Direct, Serializer.defaultSerializerName, Local))
|
||||
case None ⇒ Some(Deploy(address, Direct, Local))
|
||||
case Some(addressConfig) ⇒
|
||||
|
||||
// --------------------------------
|
||||
|
|
@ -330,17 +164,12 @@ object Deployer {
|
|||
CustomRouter(customRouter)
|
||||
}
|
||||
|
||||
// --------------------------------
|
||||
// akka.actor.deployment.<address>.format
|
||||
// --------------------------------
|
||||
val format = addressConfig.getString("format", Serializer.defaultSerializerName)
|
||||
|
||||
// --------------------------------
|
||||
// akka.actor.deployment.<address>.clustered
|
||||
// --------------------------------
|
||||
addressConfig.getSection("clustered") match {
|
||||
case None ⇒
|
||||
Some(Deploy(address, router, Serializer.defaultSerializerName, Local)) // deploy locally
|
||||
Some(Deploy(address, router, Local)) // deploy locally
|
||||
|
||||
case Some(clusteredConfig) ⇒
|
||||
|
||||
|
|
@ -349,7 +178,7 @@ object Deployer {
|
|||
// --------------------------------
|
||||
|
||||
val preferredNodes = clusteredConfig.getList("preferred-nodes") match {
|
||||
case Nil ⇒ Vector(Host("localhost"))
|
||||
case Nil ⇒ Nil
|
||||
case homes ⇒
|
||||
def raiseHomeConfigError() = throw new ConfigurationException(
|
||||
"Config option [" + addressPath +
|
||||
|
|
@ -375,19 +204,24 @@ object Deployer {
|
|||
// --------------------------------
|
||||
// akka.actor.deployment.<address>.clustered.replicas
|
||||
// --------------------------------
|
||||
val replicas = clusteredConfig.getAny("replicas", "0") match {
|
||||
case "auto" ⇒ AutoReplicate
|
||||
case "0" ⇒ NoReplicas
|
||||
case nrOfReplicas: String ⇒
|
||||
try {
|
||||
Replicate(nrOfReplicas.toInt)
|
||||
} catch {
|
||||
case e: NumberFormatException ⇒
|
||||
throw new ConfigurationException(
|
||||
"Config option [" + addressPath +
|
||||
".clustered.replicas] needs to be either [\"auto\"] or [0-N] - was [" +
|
||||
nrOfReplicas + "]")
|
||||
val replicas = {
|
||||
if (router == Direct) Replicate(1)
|
||||
else {
|
||||
clusteredConfig.getAny("replicas", "0") match {
|
||||
case "auto" ⇒ AutoReplicate
|
||||
case "0" ⇒ NoReplicas
|
||||
case nrOfReplicas: String ⇒
|
||||
try {
|
||||
Replicate(nrOfReplicas.toInt)
|
||||
} catch {
|
||||
case e: NumberFormatException ⇒
|
||||
throw new ConfigurationException(
|
||||
"Config option [" + addressPath +
|
||||
".clustered.replicas] needs to be either [\"auto\"] or [0-N] - was [" +
|
||||
nrOfReplicas + "]")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// --------------------------------
|
||||
|
|
@ -395,7 +229,7 @@ object Deployer {
|
|||
// --------------------------------
|
||||
clusteredConfig.getSection("replication") match {
|
||||
case None ⇒
|
||||
Some(Deploy(address, router, format, Clustered(preferredNodes, replicas, Transient)))
|
||||
Some(Deploy(address, router, Clustered(preferredNodes, replicas, Transient)))
|
||||
|
||||
case Some(replicationConfig) ⇒
|
||||
val storage = replicationConfig.getString("storage", "transaction-log") match {
|
||||
|
|
@ -414,7 +248,7 @@ object Deployer {
|
|||
".clustered.replication.strategy] needs to be either [\"write-through\"] or [\"write-behind\"] - was [" +
|
||||
unknown + "]")
|
||||
}
|
||||
Some(Deploy(address, router, format, Clustered(preferredNodes, replicas, Replication(storage, strategy))))
|
||||
Some(Deploy(address, router, Clustered(preferredNodes, replicas, Replication(storage, strategy))))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
217
akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala
Normal file
217
akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala
Normal file
|
|
@ -0,0 +1,217 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
|
||||
*/
|
||||
|
||||
package akka.actor
|
||||
|
||||
import akka.config.Config
|
||||
import akka.routing.RouterType
|
||||
import akka.serialization.Serializer
|
||||
|
||||
/**
|
||||
* Module holding the programmatic deployment configuration classes.
|
||||
* Defines the deployment specification.
|
||||
* Most values have defaults and can be left out.
|
||||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
object DeploymentConfig {
|
||||
|
||||
// --------------------------------
|
||||
// --- Deploy
|
||||
// --------------------------------
|
||||
case class Deploy(
|
||||
address: String,
|
||||
routing: Routing = Direct,
|
||||
scope: Scope = Local)
|
||||
|
||||
// --------------------------------
|
||||
// --- Routing
|
||||
// --------------------------------
|
||||
sealed trait Routing
|
||||
case class CustomRouter(router: AnyRef) extends Routing
|
||||
|
||||
// For Java API
|
||||
case class Direct() extends Routing
|
||||
case class RoundRobin() extends Routing
|
||||
case class Random() extends Routing
|
||||
case class LeastCPU() extends Routing
|
||||
case class LeastRAM() extends Routing
|
||||
case class LeastMessages() extends Routing
|
||||
|
||||
// For Scala API
|
||||
case object Direct extends Routing
|
||||
case object RoundRobin extends Routing
|
||||
case object Random extends Routing
|
||||
case object LeastCPU extends Routing
|
||||
case object LeastRAM extends Routing
|
||||
case object LeastMessages extends Routing
|
||||
|
||||
// --------------------------------
|
||||
// --- Scope
|
||||
// --------------------------------
|
||||
sealed trait Scope
|
||||
case class Clustered(
|
||||
preferredNodes: Iterable[Home] = Vector(Host("localhost")),
|
||||
replicas: Replicas = NoReplicas,
|
||||
replication: ReplicationScheme = Transient) extends Scope
|
||||
|
||||
// For Java API
|
||||
case class Local() extends Scope
|
||||
|
||||
// For Scala API
|
||||
case object Local extends Scope
|
||||
|
||||
// --------------------------------
|
||||
// --- Home
|
||||
// --------------------------------
|
||||
sealed trait Home
|
||||
case class Host(hostName: String) extends Home
|
||||
case class Node(nodeName: String) extends Home
|
||||
case class IP(ipAddress: String) extends Home
|
||||
|
||||
// --------------------------------
|
||||
// --- Replicas
|
||||
// --------------------------------
|
||||
sealed trait Replicas
|
||||
case class Replicate(factor: Int) extends Replicas {
|
||||
if (factor < 1) throw new IllegalArgumentException("Replicas factor can not be negative or zero")
|
||||
}
|
||||
|
||||
// For Java API
|
||||
case class AutoReplicate() extends Replicas
|
||||
case class NoReplicas() extends Replicas
|
||||
|
||||
// For Scala API
|
||||
case object AutoReplicate extends Replicas
|
||||
case object NoReplicas extends Replicas
|
||||
|
||||
// --------------------------------
|
||||
// --- Replication
|
||||
// --------------------------------
|
||||
sealed trait ReplicationScheme
|
||||
|
||||
// For Java API
|
||||
case class Transient() extends ReplicationScheme
|
||||
|
||||
// For Scala API
|
||||
case object Transient extends ReplicationScheme
|
||||
case class Replication(
|
||||
storage: ReplicationStorage,
|
||||
strategy: ReplicationStrategy) extends ReplicationScheme
|
||||
|
||||
// --------------------------------
|
||||
// --- ReplicationStorage
|
||||
// --------------------------------
|
||||
sealed trait ReplicationStorage
|
||||
|
||||
// For Java API
|
||||
case class TransactionLog() extends ReplicationStorage
|
||||
case class DataGrid() extends ReplicationStorage
|
||||
|
||||
// For Scala API
|
||||
case object TransactionLog extends ReplicationStorage
|
||||
case object DataGrid extends ReplicationStorage
|
||||
|
||||
// --------------------------------
|
||||
// --- ReplicationStrategy
|
||||
// --------------------------------
|
||||
sealed trait ReplicationStrategy
|
||||
|
||||
// For Java API
|
||||
case class WriteBehind() extends ReplicationStrategy
|
||||
case class WriteThrough() extends ReplicationStrategy
|
||||
|
||||
// For Scala API
|
||||
case object WriteBehind extends ReplicationStrategy
|
||||
case object WriteThrough extends ReplicationStrategy
|
||||
|
||||
// --------------------------------
|
||||
// --- Helper methods for parsing
|
||||
// --------------------------------
|
||||
|
||||
def nodeNameFor(home: Home): String = home match {
|
||||
case Node(nodename) ⇒ nodename
|
||||
case Host("localhost") ⇒ Config.nodename
|
||||
case IP("0.0.0.0") ⇒ Config.nodename
|
||||
case IP("127.0.0.1") ⇒ Config.nodename
|
||||
case Host(hostname) ⇒ throw new UnsupportedOperationException("Specifying preferred node name by 'hostname' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]")
|
||||
case IP(address) ⇒ throw new UnsupportedOperationException("Specifying preferred node name by 'IP address' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]")
|
||||
}
|
||||
|
||||
def isHomeNode(homes: Iterable[Home]): Boolean = homes exists (home ⇒ nodeNameFor(home) == Config.nodename)
|
||||
|
||||
def replicaValueFor(replicas: Replicas): Int = replicas match {
|
||||
case Replicate(replicas) ⇒ replicas
|
||||
case AutoReplicate ⇒ -1
|
||||
case AutoReplicate() ⇒ -1
|
||||
case NoReplicas ⇒ 0
|
||||
case NoReplicas() ⇒ 0
|
||||
}
|
||||
|
||||
def routerTypeFor(routing: Routing): RouterType = routing match {
|
||||
case Direct ⇒ RouterType.Direct
|
||||
case Direct() ⇒ RouterType.Direct
|
||||
case RoundRobin ⇒ RouterType.RoundRobin
|
||||
case RoundRobin() ⇒ RouterType.RoundRobin
|
||||
case Random ⇒ RouterType.Random
|
||||
case Random() ⇒ RouterType.Random
|
||||
case LeastCPU ⇒ RouterType.LeastCPU
|
||||
case LeastCPU() ⇒ RouterType.LeastCPU
|
||||
case LeastRAM ⇒ RouterType.LeastRAM
|
||||
case LeastRAM() ⇒ RouterType.LeastRAM
|
||||
case LeastMessages ⇒ RouterType.LeastMessages
|
||||
case LeastMessages() ⇒ RouterType.LeastMessages
|
||||
case c: CustomRouter ⇒ throw new UnsupportedOperationException("Unknown Router [" + c + "]")
|
||||
}
|
||||
|
||||
def replicationSchemeFor(deployment: Deploy): Option[ReplicationScheme] = deployment match {
|
||||
case Deploy(_, _, Clustered(_, _, replicationScheme)) ⇒ Some(replicationScheme)
|
||||
case _ ⇒ None
|
||||
}
|
||||
|
||||
def isReplicated(deployment: Deploy): Boolean = replicationSchemeFor(deployment) match {
|
||||
case Some(replicationScheme) ⇒ isReplicated(replicationScheme)
|
||||
case _ ⇒ false
|
||||
}
|
||||
|
||||
def isReplicated(replicationScheme: ReplicationScheme): Boolean =
|
||||
isReplicatedWithTransactionLog(replicationScheme) ||
|
||||
isReplicatedWithDataGrid(replicationScheme)
|
||||
|
||||
def isWriteBehindReplication(replicationScheme: ReplicationScheme): Boolean = replicationScheme match {
|
||||
case _: Transient | Transient ⇒ false
|
||||
case Replication(_, strategy) ⇒
|
||||
strategy match {
|
||||
case _: WriteBehind | WriteBehind ⇒ true
|
||||
case _: WriteThrough | WriteThrough ⇒ false
|
||||
}
|
||||
}
|
||||
|
||||
def isWriteThroughReplication(replicationScheme: ReplicationScheme): Boolean = replicationScheme match {
|
||||
case _: Transient | Transient ⇒ false
|
||||
case Replication(_, strategy) ⇒
|
||||
strategy match {
|
||||
case _: WriteBehind | WriteBehind ⇒ true
|
||||
case _: WriteThrough | WriteThrough ⇒ false
|
||||
}
|
||||
}
|
||||
|
||||
def isReplicatedWithTransactionLog(replicationScheme: ReplicationScheme): Boolean = replicationScheme match {
|
||||
case _: Transient | Transient ⇒ false
|
||||
case Replication(storage, _) ⇒
|
||||
storage match {
|
||||
case _: TransactionLog | TransactionLog ⇒ true
|
||||
case _: DataGrid | DataGrid ⇒ throw new UnsupportedOperationException("ReplicationStorage 'DataGrid' is no supported yet")
|
||||
}
|
||||
}
|
||||
|
||||
def isReplicatedWithDataGrid(replicationScheme: ReplicationScheme): Boolean = replicationScheme match {
|
||||
case _: Transient | Transient ⇒ false
|
||||
case Replication(storage, _) ⇒
|
||||
storage match {
|
||||
case _: TransactionLog | TransactionLog ⇒ false
|
||||
case _: DataGrid | DataGrid ⇒ throw new UnsupportedOperationException("ReplicationStorage 'DataGrid' is no supported yet")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -327,17 +327,17 @@ trait ClusterNode {
|
|||
/**
|
||||
* Using (checking out) actor on a specific set of nodes.
|
||||
*/
|
||||
def useActorOnNodes(nodes: Array[String], actorAddress: String)
|
||||
def useActorOnNodes(nodes: Array[String], actorAddress: String, replicateFromUuid: Option[UUID])
|
||||
|
||||
/**
|
||||
* Using (checking out) actor on all nodes in the cluster.
|
||||
*/
|
||||
def useActorOnAllNodes(actorAddress: String)
|
||||
def useActorOnAllNodes(actorAddress: String, replicateFromUuid: Option[UUID])
|
||||
|
||||
/**
|
||||
* Using (checking out) actor on a specific node.
|
||||
*/
|
||||
def useActorOnNode(node: String, actorAddress: String)
|
||||
def useActorOnNode(node: String, actorAddress: String, replicateFromUuid: Option[UUID])
|
||||
|
||||
/**
|
||||
* Checks in an actor after done using it on this node.
|
||||
|
|
@ -354,16 +354,6 @@ trait ClusterNode {
|
|||
*/
|
||||
def ref(actorAddress: String, router: RouterType): ActorRef
|
||||
|
||||
/**
|
||||
* Migrate the actor from 'this' node to node 'to'.
|
||||
*/
|
||||
def migrate(to: NodeAddress, actorAddress: String)
|
||||
|
||||
/**
|
||||
* Migrate the actor from node 'from' to node 'to'.
|
||||
*/
|
||||
def migrate(from: NodeAddress, to: NodeAddress, actorAddress: String)
|
||||
|
||||
/**
|
||||
* Returns the addresses of all actors checked out on this node.
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -112,14 +112,12 @@ object ReflectiveAccess {
|
|||
def newLogFor(
|
||||
id: String,
|
||||
isAsync: Boolean,
|
||||
replicationScheme: ReplicationScheme,
|
||||
format: Serializer): TransactionLog
|
||||
replicationScheme: ReplicationScheme): TransactionLog
|
||||
|
||||
def logFor(
|
||||
id: String,
|
||||
isAsync: Boolean,
|
||||
replicationScheme: ReplicationScheme,
|
||||
format: Serializer): TransactionLog
|
||||
replicationScheme: ReplicationScheme): TransactionLog
|
||||
|
||||
def shutdown()
|
||||
}
|
||||
|
|
@ -131,6 +129,7 @@ object ReflectiveAccess {
|
|||
def entries: Vector[Array[Byte]]
|
||||
def entriesFromLatestSnapshot: Tuple2[Array[Byte], Vector[Array[Byte]]]
|
||||
def entriesInRange(from: Long, to: Long): Vector[Array[Byte]]
|
||||
def latestSnapshotAndSubsequentEntries: (Array[Byte], Vector[Array[Byte]])
|
||||
def latestEntryId: Long
|
||||
def latestSnapshotId: Long
|
||||
def delete()
|
||||
|
|
|
|||
|
|
@ -132,6 +132,11 @@ public final class ClusterProtocol {
|
|||
// optional bytes payload = 5;
|
||||
boolean hasPayload();
|
||||
com.google.protobuf.ByteString getPayload();
|
||||
|
||||
// optional .UuidProtocol replicateActorFromUuid = 6;
|
||||
boolean hasReplicateActorFromUuid();
|
||||
akka.cluster.ClusterProtocol.UuidProtocol getReplicateActorFromUuid();
|
||||
akka.cluster.ClusterProtocol.UuidProtocolOrBuilder getReplicateActorFromUuidOrBuilder();
|
||||
}
|
||||
public static final class RemoteDaemonMessageProtocol extends
|
||||
com.google.protobuf.GeneratedMessage
|
||||
|
|
@ -227,11 +232,25 @@ public final class ClusterProtocol {
|
|||
return payload_;
|
||||
}
|
||||
|
||||
// optional .UuidProtocol replicateActorFromUuid = 6;
|
||||
public static final int REPLICATEACTORFROMUUID_FIELD_NUMBER = 6;
|
||||
private akka.cluster.ClusterProtocol.UuidProtocol replicateActorFromUuid_;
|
||||
public boolean hasReplicateActorFromUuid() {
|
||||
return ((bitField0_ & 0x00000010) == 0x00000010);
|
||||
}
|
||||
public akka.cluster.ClusterProtocol.UuidProtocol getReplicateActorFromUuid() {
|
||||
return replicateActorFromUuid_;
|
||||
}
|
||||
public akka.cluster.ClusterProtocol.UuidProtocolOrBuilder getReplicateActorFromUuidOrBuilder() {
|
||||
return replicateActorFromUuid_;
|
||||
}
|
||||
|
||||
private void initFields() {
|
||||
messageType_ = akka.cluster.ClusterProtocol.RemoteDaemonMessageType.START;
|
||||
actorUuid_ = akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance();
|
||||
actorAddress_ = "";
|
||||
payload_ = com.google.protobuf.ByteString.EMPTY;
|
||||
replicateActorFromUuid_ = akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance();
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
|
|
@ -248,6 +267,12 @@ public final class ClusterProtocol {
|
|||
return false;
|
||||
}
|
||||
}
|
||||
if (hasReplicateActorFromUuid()) {
|
||||
if (!getReplicateActorFromUuid().isInitialized()) {
|
||||
memoizedIsInitialized = 0;
|
||||
return false;
|
||||
}
|
||||
}
|
||||
memoizedIsInitialized = 1;
|
||||
return true;
|
||||
}
|
||||
|
|
@ -267,6 +292,9 @@ public final class ClusterProtocol {
|
|||
if (((bitField0_ & 0x00000008) == 0x00000008)) {
|
||||
output.writeBytes(5, payload_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000010) == 0x00000010)) {
|
||||
output.writeMessage(6, replicateActorFromUuid_);
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
||||
|
|
@ -292,6 +320,10 @@ public final class ClusterProtocol {
|
|||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeBytesSize(5, payload_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000010) == 0x00000010)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeMessageSize(6, replicateActorFromUuid_);
|
||||
}
|
||||
size += getUnknownFields().getSerializedSize();
|
||||
memoizedSerializedSize = size;
|
||||
return size;
|
||||
|
|
@ -402,13 +434,14 @@ public final class ClusterProtocol {
|
|||
maybeForceBuilderInitialization();
|
||||
}
|
||||
|
||||
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
||||
private Builder(BuilderParent parent) {
|
||||
super(parent);
|
||||
maybeForceBuilderInitialization();
|
||||
}
|
||||
private void maybeForceBuilderInitialization() {
|
||||
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
|
||||
getActorUuidFieldBuilder();
|
||||
getReplicateActorFromUuidFieldBuilder();
|
||||
}
|
||||
}
|
||||
private static Builder create() {
|
||||
|
|
@ -429,6 +462,12 @@ public final class ClusterProtocol {
|
|||
bitField0_ = (bitField0_ & ~0x00000004);
|
||||
payload_ = com.google.protobuf.ByteString.EMPTY;
|
||||
bitField0_ = (bitField0_ & ~0x00000008);
|
||||
if (replicateActorFromUuidBuilder_ == null) {
|
||||
replicateActorFromUuid_ = akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance();
|
||||
} else {
|
||||
replicateActorFromUuidBuilder_.clear();
|
||||
}
|
||||
bitField0_ = (bitField0_ & ~0x00000010);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
|
@ -487,6 +526,14 @@ public final class ClusterProtocol {
|
|||
to_bitField0_ |= 0x00000008;
|
||||
}
|
||||
result.payload_ = payload_;
|
||||
if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
|
||||
to_bitField0_ |= 0x00000010;
|
||||
}
|
||||
if (replicateActorFromUuidBuilder_ == null) {
|
||||
result.replicateActorFromUuid_ = replicateActorFromUuid_;
|
||||
} else {
|
||||
result.replicateActorFromUuid_ = replicateActorFromUuidBuilder_.build();
|
||||
}
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
|
|
@ -515,6 +562,9 @@ public final class ClusterProtocol {
|
|||
if (other.hasPayload()) {
|
||||
setPayload(other.getPayload());
|
||||
}
|
||||
if (other.hasReplicateActorFromUuid()) {
|
||||
mergeReplicateActorFromUuid(other.getReplicateActorFromUuid());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
|
@ -530,6 +580,12 @@ public final class ClusterProtocol {
|
|||
return false;
|
||||
}
|
||||
}
|
||||
if (hasReplicateActorFromUuid()) {
|
||||
if (!getReplicateActorFromUuid().isInitialized()) {
|
||||
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
|
@ -586,6 +642,15 @@ public final class ClusterProtocol {
|
|||
payload_ = input.readBytes();
|
||||
break;
|
||||
}
|
||||
case 50: {
|
||||
akka.cluster.ClusterProtocol.UuidProtocol.Builder subBuilder = akka.cluster.ClusterProtocol.UuidProtocol.newBuilder();
|
||||
if (hasReplicateActorFromUuid()) {
|
||||
subBuilder.mergeFrom(getReplicateActorFromUuid());
|
||||
}
|
||||
input.readMessage(subBuilder, extensionRegistry);
|
||||
setReplicateActorFromUuid(subBuilder.buildPartial());
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -766,6 +831,96 @@ public final class ClusterProtocol {
|
|||
return this;
|
||||
}
|
||||
|
||||
// optional .UuidProtocol replicateActorFromUuid = 6;
|
||||
private akka.cluster.ClusterProtocol.UuidProtocol replicateActorFromUuid_ = akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance();
|
||||
private com.google.protobuf.SingleFieldBuilder<
|
||||
akka.cluster.ClusterProtocol.UuidProtocol, akka.cluster.ClusterProtocol.UuidProtocol.Builder, akka.cluster.ClusterProtocol.UuidProtocolOrBuilder> replicateActorFromUuidBuilder_;
|
||||
public boolean hasReplicateActorFromUuid() {
|
||||
return ((bitField0_ & 0x00000010) == 0x00000010);
|
||||
}
|
||||
public akka.cluster.ClusterProtocol.UuidProtocol getReplicateActorFromUuid() {
|
||||
if (replicateActorFromUuidBuilder_ == null) {
|
||||
return replicateActorFromUuid_;
|
||||
} else {
|
||||
return replicateActorFromUuidBuilder_.getMessage();
|
||||
}
|
||||
}
|
||||
public Builder setReplicateActorFromUuid(akka.cluster.ClusterProtocol.UuidProtocol value) {
|
||||
if (replicateActorFromUuidBuilder_ == null) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
replicateActorFromUuid_ = value;
|
||||
onChanged();
|
||||
} else {
|
||||
replicateActorFromUuidBuilder_.setMessage(value);
|
||||
}
|
||||
bitField0_ |= 0x00000010;
|
||||
return this;
|
||||
}
|
||||
public Builder setReplicateActorFromUuid(
|
||||
akka.cluster.ClusterProtocol.UuidProtocol.Builder builderForValue) {
|
||||
if (replicateActorFromUuidBuilder_ == null) {
|
||||
replicateActorFromUuid_ = builderForValue.build();
|
||||
onChanged();
|
||||
} else {
|
||||
replicateActorFromUuidBuilder_.setMessage(builderForValue.build());
|
||||
}
|
||||
bitField0_ |= 0x00000010;
|
||||
return this;
|
||||
}
|
||||
public Builder mergeReplicateActorFromUuid(akka.cluster.ClusterProtocol.UuidProtocol value) {
|
||||
if (replicateActorFromUuidBuilder_ == null) {
|
||||
if (((bitField0_ & 0x00000010) == 0x00000010) &&
|
||||
replicateActorFromUuid_ != akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance()) {
|
||||
replicateActorFromUuid_ =
|
||||
akka.cluster.ClusterProtocol.UuidProtocol.newBuilder(replicateActorFromUuid_).mergeFrom(value).buildPartial();
|
||||
} else {
|
||||
replicateActorFromUuid_ = value;
|
||||
}
|
||||
onChanged();
|
||||
} else {
|
||||
replicateActorFromUuidBuilder_.mergeFrom(value);
|
||||
}
|
||||
bitField0_ |= 0x00000010;
|
||||
return this;
|
||||
}
|
||||
public Builder clearReplicateActorFromUuid() {
|
||||
if (replicateActorFromUuidBuilder_ == null) {
|
||||
replicateActorFromUuid_ = akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance();
|
||||
onChanged();
|
||||
} else {
|
||||
replicateActorFromUuidBuilder_.clear();
|
||||
}
|
||||
bitField0_ = (bitField0_ & ~0x00000010);
|
||||
return this;
|
||||
}
|
||||
public akka.cluster.ClusterProtocol.UuidProtocol.Builder getReplicateActorFromUuidBuilder() {
|
||||
bitField0_ |= 0x00000010;
|
||||
onChanged();
|
||||
return getReplicateActorFromUuidFieldBuilder().getBuilder();
|
||||
}
|
||||
public akka.cluster.ClusterProtocol.UuidProtocolOrBuilder getReplicateActorFromUuidOrBuilder() {
|
||||
if (replicateActorFromUuidBuilder_ != null) {
|
||||
return replicateActorFromUuidBuilder_.getMessageOrBuilder();
|
||||
} else {
|
||||
return replicateActorFromUuid_;
|
||||
}
|
||||
}
|
||||
private com.google.protobuf.SingleFieldBuilder<
|
||||
akka.cluster.ClusterProtocol.UuidProtocol, akka.cluster.ClusterProtocol.UuidProtocol.Builder, akka.cluster.ClusterProtocol.UuidProtocolOrBuilder>
|
||||
getReplicateActorFromUuidFieldBuilder() {
|
||||
if (replicateActorFromUuidBuilder_ == null) {
|
||||
replicateActorFromUuidBuilder_ = new com.google.protobuf.SingleFieldBuilder<
|
||||
akka.cluster.ClusterProtocol.UuidProtocol, akka.cluster.ClusterProtocol.UuidProtocol.Builder, akka.cluster.ClusterProtocol.UuidProtocolOrBuilder>(
|
||||
replicateActorFromUuid_,
|
||||
getParentForChildren(),
|
||||
isClean());
|
||||
replicateActorFromUuid_ = null;
|
||||
}
|
||||
return replicateActorFromUuidBuilder_;
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(builder_scope:RemoteDaemonMessageProtocol)
|
||||
}
|
||||
|
||||
|
|
@ -1092,7 +1247,7 @@ public final class ClusterProtocol {
|
|||
maybeForceBuilderInitialization();
|
||||
}
|
||||
|
||||
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
||||
private Builder(BuilderParent parent) {
|
||||
super(parent);
|
||||
maybeForceBuilderInitialization();
|
||||
}
|
||||
|
|
@ -1694,7 +1849,7 @@ public final class ClusterProtocol {
|
|||
maybeForceBuilderInitialization();
|
||||
}
|
||||
|
||||
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
||||
private Builder(BuilderParent parent) {
|
||||
super(parent);
|
||||
maybeForceBuilderInitialization();
|
||||
}
|
||||
|
|
@ -1912,23 +2067,24 @@ public final class ClusterProtocol {
|
|||
descriptor;
|
||||
static {
|
||||
java.lang.String[] descriptorData = {
|
||||
"\n\025ClusterProtocol.proto\"\225\001\n\033RemoteDaemon" +
|
||||
"\n\025ClusterProtocol.proto\"\304\001\n\033RemoteDaemon" +
|
||||
"MessageProtocol\022-\n\013messageType\030\001 \002(\0162\030.R" +
|
||||
"emoteDaemonMessageType\022 \n\tactorUuid\030\002 \001(" +
|
||||
"\0132\r.UuidProtocol\022\024\n\014actorAddress\030\003 \001(\t\022\017" +
|
||||
"\n\007payload\030\005 \001(\014\"\212\001\n\035DurableMailboxMessag" +
|
||||
"eProtocol\022\031\n\021ownerActorAddress\030\001 \002(\t\022\032\n\022" +
|
||||
"senderActorAddress\030\002 \001(\t\022!\n\nfutureUuid\030\003" +
|
||||
" \001(\0132\r.UuidProtocol\022\017\n\007message\030\004 \002(\014\")\n\014" +
|
||||
"UuidProtocol\022\014\n\004high\030\001 \002(\004\022\013\n\003low\030\002 \002(\004*" +
|
||||
"\232\002\n\027RemoteDaemonMessageType\022\t\n\005START\020\001\022\010",
|
||||
"\n\004STOP\020\002\022\007\n\003USE\020\003\022\013\n\007RELEASE\020\004\022\022\n\016MAKE_A" +
|
||||
"VAILABLE\020\005\022\024\n\020MAKE_UNAVAILABLE\020\006\022\016\n\nDISC" +
|
||||
"ONNECT\020\007\022\r\n\tRECONNECT\020\010\022\n\n\006RESIGN\020\t\022\031\n\025F" +
|
||||
"AIL_OVER_CONNECTIONS\020\n\022\026\n\022FUNCTION_FUN0_" +
|
||||
"UNIT\020\013\022\025\n\021FUNCTION_FUN0_ANY\020\014\022\032\n\026FUNCTIO" +
|
||||
"N_FUN1_ARG_UNIT\020\r\022\031\n\025FUNCTION_FUN1_ARG_A" +
|
||||
"NY\020\016B\020\n\014akka.clusterH\001"
|
||||
"\n\007payload\030\005 \001(\014\022-\n\026replicateActorFromUui" +
|
||||
"d\030\006 \001(\0132\r.UuidProtocol\"\212\001\n\035DurableMailbo" +
|
||||
"xMessageProtocol\022\031\n\021ownerActorAddress\030\001 " +
|
||||
"\002(\t\022\032\n\022senderActorAddress\030\002 \001(\t\022!\n\nfutur" +
|
||||
"eUuid\030\003 \001(\0132\r.UuidProtocol\022\017\n\007message\030\004 " +
|
||||
"\002(\014\")\n\014UuidProtocol\022\014\n\004high\030\001 \002(\004\022\013\n\003low",
|
||||
"\030\002 \002(\004*\232\002\n\027RemoteDaemonMessageType\022\t\n\005ST" +
|
||||
"ART\020\001\022\010\n\004STOP\020\002\022\007\n\003USE\020\003\022\013\n\007RELEASE\020\004\022\022\n" +
|
||||
"\016MAKE_AVAILABLE\020\005\022\024\n\020MAKE_UNAVAILABLE\020\006\022" +
|
||||
"\016\n\nDISCONNECT\020\007\022\r\n\tRECONNECT\020\010\022\n\n\006RESIGN" +
|
||||
"\020\t\022\031\n\025FAIL_OVER_CONNECTIONS\020\n\022\026\n\022FUNCTIO" +
|
||||
"N_FUN0_UNIT\020\013\022\025\n\021FUNCTION_FUN0_ANY\020\014\022\032\n\026" +
|
||||
"FUNCTION_FUN1_ARG_UNIT\020\r\022\031\n\025FUNCTION_FUN" +
|
||||
"1_ARG_ANY\020\016B\020\n\014akka.clusterH\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
|
@ -1940,7 +2096,7 @@ public final class ClusterProtocol {
|
|||
internal_static_RemoteDaemonMessageProtocol_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_RemoteDaemonMessageProtocol_descriptor,
|
||||
new java.lang.String[] { "MessageType", "ActorUuid", "ActorAddress", "Payload", },
|
||||
new java.lang.String[] { "MessageType", "ActorUuid", "ActorAddress", "Payload", "ReplicateActorFromUuid", },
|
||||
akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol.class,
|
||||
akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol.Builder.class);
|
||||
internal_static_DurableMailboxMessageProtocol_descriptor =
|
||||
|
|
|
|||
|
|
@ -19,6 +19,7 @@ message RemoteDaemonMessageProtocol {
|
|||
optional UuidProtocol actorUuid = 2;
|
||||
optional string actorAddress = 3;
|
||||
optional bytes payload = 5;
|
||||
optional UuidProtocol replicateActorFromUuid = 6;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -113,7 +113,7 @@ trait ClusterNodeMBean {
|
|||
}
|
||||
|
||||
/**
|
||||
* Module for the ClusterNode. Also holds global state such as configuration data etc.
|
||||
* Module for the Cluster. Also holds global state such as configuration data etc.
|
||||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
|
|
@ -131,6 +131,10 @@ object Cluster {
|
|||
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 includeRefNodeInReplicaSet = config.getBool("akka.cluster.include-ref-node-in-replica-set", true)
|
||||
val clusterDirectory = config.getString("akka.cluster.log-directory", "_akka_cluster")
|
||||
|
||||
val clusterDataDirectory = clusterDirectory + "/data"
|
||||
val clusterLogDirectory = clusterDirectory + "/log"
|
||||
|
||||
@volatile
|
||||
private var properties = Map.empty[String, String]
|
||||
|
|
@ -189,19 +193,19 @@ object Cluster {
|
|||
* Starts up a local ZooKeeper server. Should only be used for testing purposes.
|
||||
*/
|
||||
def startLocalCluster(): ZkServer =
|
||||
startLocalCluster("_akka_cluster/data", "_akka_cluster/log", 2181, 5000)
|
||||
startLocalCluster(clusterDataDirectory, clusterLogDirectory, 2181, 5000)
|
||||
|
||||
/**
|
||||
* Starts up a local ZooKeeper server. Should only be used for testing purposes.
|
||||
*/
|
||||
def startLocalCluster(port: Int, tickTime: Int): ZkServer =
|
||||
startLocalCluster("_akka_cluster/data", "_akka_cluster/log", port, tickTime)
|
||||
startLocalCluster(clusterDataDirectory, clusterLogDirectory, port, tickTime)
|
||||
|
||||
/**
|
||||
* Starts up a local ZooKeeper server. Should only be used for testing purposes.
|
||||
*/
|
||||
def startLocalCluster(tickTime: Int): ZkServer =
|
||||
startLocalCluster("_akka_cluster/data", "_akka_cluster/log", 2181, tickTime)
|
||||
startLocalCluster(clusterDataDirectory, clusterLogDirectory, 2181, tickTime)
|
||||
|
||||
/**
|
||||
* Starts up a local ZooKeeper server. Should only be used for testing purposes.
|
||||
|
|
@ -322,7 +326,7 @@ class DefaultClusterNode private[akka] (
|
|||
}
|
||||
}, "akka.cluster.RemoteClientLifeCycleListener").start()
|
||||
|
||||
private[cluster] lazy val remoteDaemon = localActorOf(new RemoteClusterDaemon(this), RemoteClusterDaemon.ADDRESS).start()
|
||||
private[cluster] lazy val remoteDaemon = localActorOf(new RemoteClusterDaemon(this), RemoteClusterDaemon.Address).start()
|
||||
|
||||
private[cluster] lazy val remoteDaemonSupervisor = Supervisor(
|
||||
SupervisorConfig(
|
||||
|
|
@ -335,7 +339,7 @@ class DefaultClusterNode private[akka] (
|
|||
lazy val remoteService: RemoteSupport = {
|
||||
val remote = new akka.remote.netty.NettyRemoteSupport
|
||||
remote.start(hostname, port)
|
||||
remote.register(RemoteClusterDaemon.ADDRESS, remoteDaemon)
|
||||
remote.register(RemoteClusterDaemon.Address, remoteDaemon)
|
||||
remote.addListener(remoteClientLifeCycleListener)
|
||||
remote
|
||||
}
|
||||
|
|
@ -676,21 +680,21 @@ class DefaultClusterNode private[akka] (
|
|||
case Left(path) ⇒ path
|
||||
case Right(exception) ⇒ actorAddressRegistryPath
|
||||
}
|
||||
|
||||
// create ADDRESS -> SERIALIZER CLASS NAME mapping
|
||||
try {
|
||||
zkClient.createPersistent(actorAddressRegistrySerializerPathFor(actorAddress), serializerClassName)
|
||||
} catch {
|
||||
case e: ZkNodeExistsException ⇒ zkClient.writeData(actorAddressRegistrySerializerPathFor(actorAddress), serializerClassName)
|
||||
}
|
||||
|
||||
// create ADDRESS -> NODE mapping
|
||||
ignore[ZkNodeExistsException](zkClient.createPersistent(actorAddressToNodesPathFor(actorAddress)))
|
||||
|
||||
// create ADDRESS -> UUIDs mapping
|
||||
ignore[ZkNodeExistsException](zkClient.createPersistent(actorAddressToUuidsPathFor(actorAddress)))
|
||||
}
|
||||
|
||||
// create ADDRESS -> SERIALIZER CLASS NAME mapping
|
||||
try {
|
||||
zkClient.createPersistent(actorAddressRegistrySerializerPathFor(actorAddress), serializerClassName)
|
||||
} catch {
|
||||
case e: ZkNodeExistsException ⇒ zkClient.writeData(actorAddressRegistrySerializerPathFor(actorAddress), serializerClassName)
|
||||
}
|
||||
|
||||
// create ADDRESS -> NODE mapping
|
||||
ignore[ZkNodeExistsException](zkClient.createPersistent(actorAddressToNodesPathFor(actorAddress)))
|
||||
|
||||
// create ADDRESS -> UUIDs mapping
|
||||
ignore[ZkNodeExistsException](zkClient.createPersistent(actorAddressToUuidsPathFor(actorAddress)))
|
||||
|
||||
useActorOnNodes(nodesForReplicationFactor(replicationFactor, Some(actorAddress)).toArray, actorAddress)
|
||||
|
||||
this
|
||||
|
|
@ -825,16 +829,20 @@ class DefaultClusterNode private[akka] (
|
|||
/**
|
||||
* Using (checking out) actor on a specific set of nodes.
|
||||
*/
|
||||
def useActorOnNodes(nodes: Array[String], actorAddress: String) {
|
||||
def useActorOnNodes(nodes: Array[String], actorAddress: String, replicateFromUuid: Option[UUID] = None) {
|
||||
EventHandler.debug(this,
|
||||
"Sending command to nodes [%s] for checking out actor [%s]".format(nodes.mkString(", "), actorAddress))
|
||||
|
||||
if (isConnected.get) {
|
||||
|
||||
val command = RemoteDaemonMessageProtocol.newBuilder
|
||||
val builder = RemoteDaemonMessageProtocol.newBuilder
|
||||
.setMessageType(USE)
|
||||
.setActorAddress(actorAddress)
|
||||
.build
|
||||
|
||||
// set the UUID to replicated from - if available
|
||||
replicateFromUuid foreach (uuid ⇒ builder.setReplicateActorFromUuid(uuidToUuidProtocol(uuid)))
|
||||
|
||||
val command = builder.build
|
||||
|
||||
nodes foreach { node ⇒
|
||||
nodeConnections.get(node) foreach {
|
||||
|
|
@ -848,15 +856,15 @@ class DefaultClusterNode private[akka] (
|
|||
/**
|
||||
* Using (checking out) actor on all nodes in the cluster.
|
||||
*/
|
||||
def useActorOnAllNodes(actorAddress: String) {
|
||||
useActorOnNodes(membershipNodes, actorAddress)
|
||||
def useActorOnAllNodes(actorAddress: String, replicateFromUuid: Option[UUID] = None) {
|
||||
useActorOnNodes(membershipNodes, actorAddress, replicateFromUuid)
|
||||
}
|
||||
|
||||
/**
|
||||
* Using (checking out) actor on a specific node.
|
||||
*/
|
||||
def useActorOnNode(node: String, actorAddress: String) {
|
||||
useActorOnNodes(Array(node), actorAddress)
|
||||
def useActorOnNode(node: String, actorAddress: String, replicateFromUuid: Option[UUID] = None) {
|
||||
useActorOnNodes(Array(node), actorAddress, replicateFromUuid)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -922,29 +930,6 @@ class DefaultClusterNode private[akka] (
|
|||
|
||||
} else throw new ClusterException("Not connected to cluster")
|
||||
|
||||
/**
|
||||
* Migrate the actor from 'this' node to node 'to'.
|
||||
*/
|
||||
def migrate(to: NodeAddress, actorAddress: String) {
|
||||
migrate(nodeAddress, to, actorAddress)
|
||||
}
|
||||
|
||||
/**
|
||||
* Migrate the actor from node 'from' to node 'to'.
|
||||
*/
|
||||
def migrate(
|
||||
from: NodeAddress, to: NodeAddress, actorAddress: String) {
|
||||
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)) {
|
||||
migrateWithoutCheckingThatActorResidesOnItsHomeNode(from, to, actorAddress)
|
||||
} else {
|
||||
throw new ClusterException("Can't move actor from node [" + from + "] since it does not exist on this node")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the UUIDs of all actors checked out on this node.
|
||||
*/
|
||||
|
|
@ -1285,7 +1270,7 @@ class DefaultClusterNode private[akka] (
|
|||
val preferredNodes =
|
||||
if (actorAddress.isDefined) { // use 'preferred-nodes' in deployment config for the actor
|
||||
Deployer.deploymentFor(actorAddress.get) match {
|
||||
case Deploy(_, _, _, Clustered(nodes, _, _)) ⇒
|
||||
case Deploy(_, _, Clustered(nodes, _, _)) ⇒
|
||||
nodes map (node ⇒ DeploymentConfig.nodeNameFor(node)) take replicationFactor
|
||||
case _ ⇒
|
||||
throw new ClusterException("Actor [" + actorAddress.get + "] is not configured as clustered")
|
||||
|
|
@ -1360,7 +1345,7 @@ class DefaultClusterNode private[akka] (
|
|||
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()
|
||||
val clusterDaemon = Actor.remote.actorFor(RemoteClusterDaemon.Address, address.getHostName, address.getPort).start()
|
||||
nodeConnections.put(node, (address, clusterDaemon))
|
||||
}
|
||||
}
|
||||
|
|
@ -1457,7 +1442,16 @@ class DefaultClusterNode private[akka] (
|
|||
nodeAddress
|
||||
}
|
||||
|
||||
migrateWithoutCheckingThatActorResidesOnItsHomeNode(failedNodeAddress, migrateToNodeAddress, actorAddress) // since the ephemeral node is already gone, so can't check
|
||||
// if actor is replicated => pass along the UUID for the actor to replicate from (replay transaction log etc.)
|
||||
val replicateFromUuid =
|
||||
if (isReplicated(actorAddress)) Some(uuid)
|
||||
else None
|
||||
|
||||
migrateWithoutCheckingThatActorResidesOnItsHomeNode(
|
||||
failedNodeAddress,
|
||||
migrateToNodeAddress,
|
||||
actorAddress,
|
||||
replicateFromUuid)
|
||||
}
|
||||
|
||||
// notify all available nodes that they should fail-over all connections from 'from' to 'to'
|
||||
|
|
@ -1486,7 +1480,7 @@ class DefaultClusterNode private[akka] (
|
|||
* Used when the ephemeral "home" node is already gone, so we can't check if it is available.
|
||||
*/
|
||||
private def migrateWithoutCheckingThatActorResidesOnItsHomeNode(
|
||||
from: NodeAddress, to: NodeAddress, actorAddress: String) {
|
||||
from: NodeAddress, to: NodeAddress, actorAddress: String, replicateFromUuid: Option[UUID]) {
|
||||
|
||||
EventHandler.debug(this, "Migrating actor [%s] from node [%s] to node [%s]".format(actorAddress, from, to))
|
||||
if (!isInUseOnNode(actorAddress, to)) {
|
||||
|
|
@ -1502,7 +1496,7 @@ class DefaultClusterNode private[akka] (
|
|||
//ignore[ZkNoNodeException](zkClient.delete(nodeToUuidsPathFor(from.nodeName, uuid)))
|
||||
|
||||
// 'use' (check out) actor on the remote 'to' node
|
||||
useActorOnNode(to.nodeName, actorAddress)
|
||||
useActorOnNode(to.nodeName, actorAddress, replicateFromUuid)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -1542,6 +1536,8 @@ class DefaultClusterNode private[akka] (
|
|||
connectToAllNewlyArrivedMembershipNodesInCluster(membershipNodes, Nil)
|
||||
}
|
||||
|
||||
private def isReplicated(actorAddress: String): Boolean = DeploymentConfig.isReplicated(Deployer.deploymentFor(actorAddress))
|
||||
|
||||
private def createMBean = {
|
||||
val clusterMBean = new StandardMBean(classOf[ClusterNodeMBean]) with ClusterNodeMBean {
|
||||
|
||||
|
|
@ -1672,7 +1668,7 @@ class StateListener(self: ClusterNode) extends IZkStateListener {
|
|||
trait ErrorHandler {
|
||||
def withErrorHandler[T](body: ⇒ T) = {
|
||||
try {
|
||||
ignore[ZkInterruptedException](body)
|
||||
ignore[ZkInterruptedException](body) // FIXME Is it good to ignore ZkInterruptedException? If not, how should we handle it?
|
||||
} catch {
|
||||
case e: Throwable ⇒
|
||||
EventHandler.error(e, this, e.toString)
|
||||
|
|
@ -1685,13 +1681,15 @@ trait ErrorHandler {
|
|||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
object RemoteClusterDaemon {
|
||||
val ADDRESS = "akka-cluster-daemon".intern
|
||||
val Address = "akka-cluster-daemon".intern
|
||||
|
||||
// FIXME configure computeGridDispatcher to what?
|
||||
val computeGridDispatcher = Dispatchers.newDispatcher("akka:compute-grid").build
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal "daemon" actor for cluster internal communication.
|
||||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
|
||||
|
|
@ -1720,12 +1718,51 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
|
|||
cluster.serializerForActor(actorAddress) foreach { serializer ⇒
|
||||
cluster.use(actorAddress, serializer) foreach { actor ⇒
|
||||
cluster.remoteService.register(actorAddress, actor)
|
||||
|
||||
if (message.hasReplicateActorFromUuid) {
|
||||
// replication is used - fetch the messages and replay them
|
||||
import akka.remote.protocol.RemoteProtocol._
|
||||
import akka.remote.MessageSerializer
|
||||
|
||||
val replicateFromUuid = uuidProtocolToUuid(message.getReplicateActorFromUuid)
|
||||
val deployment = Deployer.deploymentFor(actorAddress)
|
||||
val replicationScheme = DeploymentConfig.replicationSchemeFor(deployment).getOrElse(
|
||||
throw new IllegalStateException(
|
||||
"Actor [" + actorAddress + "] should have been configured as a replicated actor but could not find its ReplicationScheme"))
|
||||
val isWriteBehind = DeploymentConfig.isWriteBehindReplication(replicationScheme)
|
||||
|
||||
try {
|
||||
// get the transaction log for the actor UUID
|
||||
val txLog = TransactionLog.logFor(replicateFromUuid.toString, isWriteBehind, replicationScheme)
|
||||
|
||||
// deserialize all messages
|
||||
val entriesAsBytes = txLog.entries
|
||||
// val (snapshotAsBytes, entriesAsBytes) = txLog.latestSnapshotAndSubsequentEntries // FIXME should work equally good if not a snapshot has been taken yet. => return all entries
|
||||
|
||||
val messages: Vector[AnyRef] = entriesAsBytes map { bytes ⇒
|
||||
val messageBytes =
|
||||
if (Cluster.shouldCompressData) LZF.uncompress(bytes)
|
||||
else bytes
|
||||
MessageSerializer.deserialize(MessageProtocol.parseFrom(messageBytes), None)
|
||||
}
|
||||
|
||||
// replay all messages
|
||||
EventHandler.info(this, "Replaying [%s] messages to actor [%s]".format(messages.size, actorAddress))
|
||||
|
||||
messages foreach { message ⇒
|
||||
EventHandler.debug(this, "Replaying message [%s] to actor [%s]".format(message, actorAddress))
|
||||
actor ! message // FIXME how to handle '?' messages???
|
||||
}
|
||||
} catch {
|
||||
case e: Throwable ⇒
|
||||
EventHandler.error(e, this, e.toString)
|
||||
throw e
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
EventHandler.error(this,
|
||||
"Actor 'address' is not defined, ignoring remote cluster daemon command [%s]"
|
||||
.format(message))
|
||||
EventHandler.error(this, "Actor 'address' is not defined, ignoring remote cluster daemon command [%s]".format(message))
|
||||
}
|
||||
self.reply(Success)
|
||||
|
||||
|
|
|
|||
|
|
@ -167,7 +167,7 @@ object ClusterDeployer {
|
|||
ensureRunning {
|
||||
LocalDeployer.deploy(deployment)
|
||||
deployment match {
|
||||
case Deploy(_, _, _, Local) ⇒ {} // local deployment, do nothing here
|
||||
case Deploy(_, _, Local) ⇒ {} // local deployment, do nothing here
|
||||
case _ ⇒ // cluster deployment
|
||||
val path = deploymentAddressPath.format(deployment.address)
|
||||
try {
|
||||
|
|
|
|||
|
|
@ -18,7 +18,6 @@ import DeploymentConfig.{ ReplicationScheme, ReplicationStrategy, Transient, Wri
|
|||
import akka.event.EventHandler
|
||||
import akka.dispatch.{ DefaultPromise, Promise, MessageInvocation }
|
||||
import akka.remote.MessageSerializer
|
||||
import akka.serialization.ActorSerialization._
|
||||
import akka.cluster.zookeeper._
|
||||
import akka.serialization.{ Serializer, Compression }
|
||||
import Compression.LZF
|
||||
|
|
@ -33,15 +32,11 @@ import java.util.concurrent.atomic.AtomicLong
|
|||
// FIXME delete tx log after migration of actor has been made and create a new one
|
||||
|
||||
/**
|
||||
* TODO: Improved documentation,
|
||||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class ReplicationException(message: String) extends AkkaException(message)
|
||||
|
||||
/**
|
||||
* TODO: Improved documentation.
|
||||
*
|
||||
* TODO: Explain something about threadsafety.
|
||||
*
|
||||
* A TransactionLog makes chunks of data durable.
|
||||
|
|
@ -52,8 +47,7 @@ class TransactionLog private (
|
|||
ledger: LedgerHandle,
|
||||
val id: String,
|
||||
val isAsync: Boolean,
|
||||
replicationScheme: ReplicationScheme,
|
||||
format: Serializer) {
|
||||
replicationScheme: ReplicationScheme) {
|
||||
|
||||
import TransactionLog._
|
||||
|
||||
|
|
@ -65,7 +59,7 @@ class TransactionLog private (
|
|||
private val isOpen = new Switch(true)
|
||||
|
||||
/**
|
||||
* TODO document method
|
||||
* Record an Actor message invocation.
|
||||
*/
|
||||
def recordEntry(messageHandle: MessageInvocation, actorRef: ActorRef) {
|
||||
if (nrOfEntries.incrementAndGet % snapshotFrequency == 0) {
|
||||
|
|
@ -79,7 +73,7 @@ class TransactionLog private (
|
|||
}
|
||||
|
||||
/**
|
||||
* TODO document method
|
||||
* Record an entry.
|
||||
*/
|
||||
def recordEntry(entry: Array[Byte]) {
|
||||
if (isOpen.isOn) {
|
||||
|
|
@ -96,8 +90,7 @@ class TransactionLog private (
|
|||
entryId: Long,
|
||||
ctx: AnyRef) {
|
||||
handleReturnCode(returnCode)
|
||||
EventHandler.debug(this,
|
||||
"Writing entry [%s] to log [%s]".format(entryId, logId))
|
||||
EventHandler.debug(this, "Writing entry [%s] to log [%s]".format(entryId, logId))
|
||||
}
|
||||
},
|
||||
null)
|
||||
|
|
@ -113,7 +106,7 @@ class TransactionLog private (
|
|||
}
|
||||
|
||||
/**
|
||||
* TODO document method
|
||||
* Record a snapshot.
|
||||
*/
|
||||
def recordSnapshot(snapshot: Array[Byte]) {
|
||||
if (isOpen.isOn) {
|
||||
|
|
@ -145,14 +138,14 @@ class TransactionLog private (
|
|||
}
|
||||
|
||||
/**
|
||||
* TODO document method
|
||||
* Get all the entries for this transaction log.
|
||||
*/
|
||||
def entries: Vector[Array[Byte]] = entriesInRange(0, ledger.getLastAddConfirmed)
|
||||
|
||||
/**
|
||||
* TODO document method
|
||||
* Get the latest snapshot and all subsequent entries from this snapshot.
|
||||
*/
|
||||
def toByteArraysLatestSnapshot: (Array[Byte], Vector[Array[Byte]]) = {
|
||||
def latestSnapshotAndSubsequentEntries: (Array[Byte], Vector[Array[Byte]]) = {
|
||||
val snapshotId = latestSnapshotId
|
||||
EventHandler.debug(this,
|
||||
"Reading entries from snapshot id [%s] for log [%s]".format(snapshotId, logId))
|
||||
|
|
@ -160,7 +153,7 @@ class TransactionLog private (
|
|||
}
|
||||
|
||||
/**
|
||||
* TODO document method
|
||||
* Get a range of entries from 'from' to 'to' for this transaction log.
|
||||
*/
|
||||
def entriesInRange(from: Long, to: Long): Vector[Array[Byte]] = if (isOpen.isOn) {
|
||||
try {
|
||||
|
|
@ -197,12 +190,12 @@ class TransactionLog private (
|
|||
} else transactionClosedError
|
||||
|
||||
/**
|
||||
* TODO document method
|
||||
* Get the last entry written to this transaction log.
|
||||
*/
|
||||
def latestEntryId: Long = ledger.getLastAddConfirmed
|
||||
|
||||
/**
|
||||
* TODO document method
|
||||
* Get the id for the last snapshot written to this transaction log.
|
||||
*/
|
||||
def latestSnapshotId: Long = {
|
||||
try {
|
||||
|
|
@ -219,7 +212,7 @@ class TransactionLog private (
|
|||
}
|
||||
|
||||
/**
|
||||
* TODO document method
|
||||
* Delete all entries for this transaction log.
|
||||
*/
|
||||
def delete() {
|
||||
if (isOpen.isOn) {
|
||||
|
|
@ -244,7 +237,7 @@ class TransactionLog private (
|
|||
}
|
||||
|
||||
/**
|
||||
* TODO document method
|
||||
* Close this transaction log.
|
||||
*/
|
||||
def close() {
|
||||
if (isOpen.switchOff) {
|
||||
|
|
@ -371,9 +364,8 @@ object TransactionLog {
|
|||
ledger: LedgerHandle,
|
||||
id: String,
|
||||
isAsync: Boolean,
|
||||
replicationScheme: ReplicationScheme,
|
||||
format: Serializer) =
|
||||
new TransactionLog(ledger, id, isAsync, replicationScheme, format)
|
||||
replicationScheme: ReplicationScheme) =
|
||||
new TransactionLog(ledger, id, isAsync, replicationScheme)
|
||||
|
||||
/**
|
||||
* Shuts down the transaction log.
|
||||
|
|
@ -392,13 +384,12 @@ object TransactionLog {
|
|||
}
|
||||
|
||||
/**
|
||||
* TODO document method
|
||||
* Creates a new transaction log for the 'id' specified.
|
||||
*/
|
||||
def newLogFor(
|
||||
id: String,
|
||||
isAsync: Boolean,
|
||||
replicationScheme: ReplicationScheme,
|
||||
format: Serializer): TransactionLog = {
|
||||
replicationScheme: ReplicationScheme): TransactionLog = {
|
||||
|
||||
val txLogPath = transactionLogNode + "/" + id
|
||||
|
||||
|
|
@ -443,17 +434,16 @@ object TransactionLog {
|
|||
}
|
||||
|
||||
EventHandler.info(this, "Created new transaction log [%s] for UUID [%s]".format(logId, id))
|
||||
TransactionLog(ledger, id, isAsync, replicationScheme, format)
|
||||
TransactionLog(ledger, id, isAsync, replicationScheme)
|
||||
}
|
||||
|
||||
/**
|
||||
* TODO document method
|
||||
* Fetches an existing transaction log for the 'id' specified.
|
||||
*/
|
||||
def logFor(
|
||||
id: String,
|
||||
isAsync: Boolean,
|
||||
replicationScheme: ReplicationScheme,
|
||||
format: Serializer): TransactionLog = {
|
||||
replicationScheme: ReplicationScheme): TransactionLog = {
|
||||
|
||||
val txLogPath = transactionLogNode + "/" + id
|
||||
|
||||
|
|
@ -493,7 +483,7 @@ object TransactionLog {
|
|||
case e ⇒ handleError(e)
|
||||
}
|
||||
|
||||
TransactionLog(ledger, id, isAsync, replicationScheme, format)
|
||||
TransactionLog(ledger, id, isAsync, replicationScheme)
|
||||
}
|
||||
|
||||
private[akka] def await[T](future: Promise[T]): T = {
|
||||
|
|
|
|||
|
|
@ -147,28 +147,27 @@ object ActorSerialization {
|
|||
if (protocol.hasSupervisor) Some(RemoteActorSerialization.fromProtobufToRemoteActorRef(protocol.getSupervisor, loader))
|
||||
else None
|
||||
|
||||
import ReplicationStorageType._
|
||||
import ReplicationStrategyType._
|
||||
|
||||
val replicationScheme =
|
||||
if (protocol.hasReplicationStorage) {
|
||||
protocol.getReplicationStorage match {
|
||||
case TRANSIENT ⇒ Transient
|
||||
case store ⇒
|
||||
val storage = store match {
|
||||
case TRANSACTION_LOG ⇒ TransactionLog
|
||||
case DATA_GRID ⇒ DataGrid
|
||||
}
|
||||
val strategy = if (protocol.hasReplicationStrategy) {
|
||||
protocol.getReplicationStrategy match {
|
||||
case WRITE_THROUGH ⇒ WriteThrough
|
||||
case WRITE_BEHIND ⇒ WriteBehind
|
||||
}
|
||||
} else throw new IllegalActorStateException(
|
||||
"Expected replication strategy for replication storage [" + storage + "]")
|
||||
Replication(storage, strategy)
|
||||
}
|
||||
} else Transient
|
||||
// import ReplicationStorageType._
|
||||
// import ReplicationStrategyType._
|
||||
// val replicationScheme =
|
||||
// if (protocol.hasReplicationStorage) {
|
||||
// protocol.getReplicationStorage match {
|
||||
// case TRANSIENT ⇒ Transient
|
||||
// case store ⇒
|
||||
// val storage = store match {
|
||||
// case TRANSACTION_LOG ⇒ TransactionLog
|
||||
// case DATA_GRID ⇒ DataGrid
|
||||
// }
|
||||
// val strategy = if (protocol.hasReplicationStrategy) {
|
||||
// protocol.getReplicationStrategy match {
|
||||
// case WRITE_THROUGH ⇒ WriteThrough
|
||||
// case WRITE_BEHIND ⇒ WriteBehind
|
||||
// }
|
||||
// } else throw new IllegalActorStateException(
|
||||
// "Expected replication strategy for replication storage [" + storage + "]")
|
||||
// Replication(storage, strategy)
|
||||
// }
|
||||
// } else Transient
|
||||
|
||||
val hotswap =
|
||||
try {
|
||||
|
|
@ -205,8 +204,7 @@ object ActorSerialization {
|
|||
lifeCycle,
|
||||
supervisor,
|
||||
hotswap,
|
||||
factory,
|
||||
replicationScheme)
|
||||
factory)
|
||||
|
||||
val messages = protocol.getMessagesList.toArray.toList.asInstanceOf[List[RemoteMessageProtocol]]
|
||||
messages.foreach(message ⇒ ar ! MessageSerializer.deserialize(message.getMessage, Some(classLoader)))
|
||||
|
|
|
|||
|
|
@ -32,31 +32,31 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
"A Transaction Log" should {
|
||||
"be able to record entries - synchronous" in {
|
||||
val uuid = (new UUID).toString
|
||||
val txlog = TransactionLog.newLogFor(uuid, false, null, JavaSerializer)
|
||||
val txlog = TransactionLog.newLogFor(uuid, false, null)
|
||||
val entry = "hello".getBytes("UTF-8")
|
||||
txlog.recordEntry(entry)
|
||||
}
|
||||
|
||||
"be able to record and delete entries - synchronous" in {
|
||||
val uuid = (new UUID).toString
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, false, null, JavaSerializer)
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, false, null)
|
||||
val entry = "hello".getBytes("UTF-8")
|
||||
txlog1.recordEntry(entry)
|
||||
txlog1.recordEntry(entry)
|
||||
txlog1.delete
|
||||
txlog1.close
|
||||
intercept[BKNoSuchLedgerExistsException](TransactionLog.logFor(uuid, false, null, JavaSerializer))
|
||||
intercept[BKNoSuchLedgerExistsException](TransactionLog.logFor(uuid, false, null))
|
||||
}
|
||||
|
||||
"be able to record entries and read entries with 'entriesInRange' - synchronous" in {
|
||||
val uuid = (new UUID).toString
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, false, null, JavaSerializer)
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, false, null)
|
||||
val entry = "hello".getBytes("UTF-8")
|
||||
txlog1.recordEntry(entry)
|
||||
txlog1.recordEntry(entry)
|
||||
txlog1.close
|
||||
|
||||
val txlog2 = TransactionLog.logFor(uuid, false, null, JavaSerializer)
|
||||
val txlog2 = TransactionLog.logFor(uuid, false, null)
|
||||
val entries = txlog2.entriesInRange(0, 1).map(bytes ⇒ new String(bytes, "UTF-8"))
|
||||
entries.size must equal(2)
|
||||
entries(0) must equal("hello")
|
||||
|
|
@ -66,15 +66,15 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
|
||||
"be able to record entries and read entries with 'entries' - synchronous" in {
|
||||
val uuid = (new UUID).toString
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, false, null, JavaSerializer)
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, false, null)
|
||||
val entry = "hello".getBytes("UTF-8")
|
||||
txlog1.recordEntry(entry)
|
||||
txlog1.recordEntry(entry)
|
||||
txlog1.recordEntry(entry)
|
||||
txlog1.recordEntry(entry)
|
||||
txlog1.close
|
||||
// txlog1.close // should work without txlog.close
|
||||
|
||||
val txlog2 = TransactionLog.logFor(uuid, false, null, JavaSerializer)
|
||||
val txlog2 = TransactionLog.logFor(uuid, false, null)
|
||||
val entries = txlog2.entries.map(bytes ⇒ new String(bytes, "UTF-8"))
|
||||
entries.size must equal(4)
|
||||
entries(0) must equal("hello")
|
||||
|
|
@ -86,7 +86,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
|
||||
"be able to record a snapshot - synchronous" in {
|
||||
val uuid = (new UUID).toString
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, false, null, JavaSerializer)
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, false, null)
|
||||
val snapshot = "snapshot".getBytes("UTF-8")
|
||||
txlog1.recordSnapshot(snapshot)
|
||||
txlog1.close
|
||||
|
|
@ -94,7 +94,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
|
||||
"be able to record and read a snapshot and following entries - synchronous" in {
|
||||
val uuid = (new UUID).toString
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, false, null, JavaSerializer)
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, false, null)
|
||||
val snapshot = "snapshot".getBytes("UTF-8")
|
||||
txlog1.recordSnapshot(snapshot)
|
||||
|
||||
|
|
@ -105,8 +105,8 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
txlog1.recordEntry(entry)
|
||||
txlog1.close
|
||||
|
||||
val txlog2 = TransactionLog.logFor(uuid, false, null, JavaSerializer)
|
||||
val (snapshotAsBytes, entriesAsBytes) = txlog2.toByteArraysLatestSnapshot
|
||||
val txlog2 = TransactionLog.logFor(uuid, false, null)
|
||||
val (snapshotAsBytes, entriesAsBytes) = txlog2.latestSnapshotAndSubsequentEntries
|
||||
new String(snapshotAsBytes, "UTF-8") must equal("snapshot")
|
||||
|
||||
val entries = entriesAsBytes.map(bytes ⇒ new String(bytes, "UTF-8"))
|
||||
|
|
@ -120,7 +120,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
|
||||
"be able to record entries then a snapshot then more entries - and then read from the snapshot and the following entries - synchronous" in {
|
||||
val uuid = (new UUID).toString
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, false, null, JavaSerializer)
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, false, null)
|
||||
|
||||
val entry = "hello".getBytes("UTF-8")
|
||||
txlog1.recordEntry(entry)
|
||||
|
|
@ -134,8 +134,8 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
txlog1.recordEntry(entry)
|
||||
txlog1.close
|
||||
|
||||
val txlog2 = TransactionLog.logFor(uuid, false, null, JavaSerializer)
|
||||
val (snapshotAsBytes, entriesAsBytes) = txlog2.toByteArraysLatestSnapshot
|
||||
val txlog2 = TransactionLog.logFor(uuid, false, null)
|
||||
val (snapshotAsBytes, entriesAsBytes) = txlog2.latestSnapshotAndSubsequentEntries
|
||||
new String(snapshotAsBytes, "UTF-8") must equal("snapshot")
|
||||
|
||||
val entries = entriesAsBytes.map(bytes ⇒ new String(bytes, "UTF-8"))
|
||||
|
|
@ -149,7 +149,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
"A Transaction Log" should {
|
||||
"be able to record entries - asynchronous" in {
|
||||
val uuid = (new UUID).toString
|
||||
val txlog = TransactionLog.newLogFor(uuid, true, null, JavaSerializer)
|
||||
val txlog = TransactionLog.newLogFor(uuid, true, null)
|
||||
val entry = "hello".getBytes("UTF-8")
|
||||
txlog.recordEntry(entry)
|
||||
Thread.sleep(200)
|
||||
|
|
@ -158,7 +158,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
|
||||
"be able to record and delete entries - asynchronous" in {
|
||||
val uuid = (new UUID).toString
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, true, null, JavaSerializer)
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, true, null)
|
||||
Thread.sleep(200)
|
||||
val entry = "hello".getBytes("UTF-8")
|
||||
txlog1.recordEntry(entry)
|
||||
|
|
@ -167,11 +167,11 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
Thread.sleep(200)
|
||||
txlog1.delete
|
||||
Thread.sleep(200)
|
||||
intercept[BKNoSuchLedgerExistsException](TransactionLog.logFor(uuid, true, null, JavaSerializer))
|
||||
intercept[BKNoSuchLedgerExistsException](TransactionLog.logFor(uuid, true, null))
|
||||
}
|
||||
"be able to record entries and read entries with 'entriesInRange' - asynchronous" in {
|
||||
val uuid = (new UUID).toString
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, true, null, JavaSerializer)
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, true, null)
|
||||
Thread.sleep(200)
|
||||
val entry = "hello".getBytes("UTF-8")
|
||||
txlog1.recordEntry(entry)
|
||||
|
|
@ -180,7 +180,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
Thread.sleep(200)
|
||||
txlog1.close
|
||||
|
||||
val txlog2 = TransactionLog.logFor(uuid, true, null, JavaSerializer)
|
||||
val txlog2 = TransactionLog.logFor(uuid, true, null)
|
||||
Thread.sleep(200)
|
||||
val entries = txlog2.entriesInRange(0, 1).map(bytes ⇒ new String(bytes, "UTF-8"))
|
||||
Thread.sleep(200)
|
||||
|
|
@ -193,7 +193,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
|
||||
"be able to record entries and read entries with 'entries' - asynchronous" in {
|
||||
val uuid = (new UUID).toString
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, true, null, JavaSerializer)
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, true, null)
|
||||
Thread.sleep(200)
|
||||
val entry = "hello".getBytes("UTF-8")
|
||||
txlog1.recordEntry(entry)
|
||||
|
|
@ -206,7 +206,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
Thread.sleep(200)
|
||||
txlog1.close
|
||||
|
||||
val txlog2 = TransactionLog.logFor(uuid, true, null, JavaSerializer)
|
||||
val txlog2 = TransactionLog.logFor(uuid, true, null)
|
||||
val entries = txlog2.entries.map(bytes ⇒ new String(bytes, "UTF-8"))
|
||||
Thread.sleep(200)
|
||||
entries.size must equal(4)
|
||||
|
|
@ -220,7 +220,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
|
||||
"be able to record a snapshot - asynchronous" in {
|
||||
val uuid = (new UUID).toString
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, true, null, JavaSerializer)
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, true, null)
|
||||
Thread.sleep(200)
|
||||
val snapshot = "snapshot".getBytes("UTF-8")
|
||||
txlog1.recordSnapshot(snapshot)
|
||||
|
|
@ -230,7 +230,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
|
||||
"be able to record and read a snapshot and following entries - asynchronous" in {
|
||||
val uuid = (new UUID).toString
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, true, null, JavaSerializer)
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, true, null)
|
||||
Thread.sleep(200)
|
||||
val snapshot = "snapshot".getBytes("UTF-8")
|
||||
txlog1.recordSnapshot(snapshot)
|
||||
|
|
@ -247,9 +247,9 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
Thread.sleep(200)
|
||||
txlog1.close
|
||||
|
||||
val txlog2 = TransactionLog.logFor(uuid, true, null, JavaSerializer)
|
||||
val txlog2 = TransactionLog.logFor(uuid, true, null)
|
||||
Thread.sleep(200)
|
||||
val (snapshotAsBytes, entriesAsBytes) = txlog2.toByteArraysLatestSnapshot
|
||||
val (snapshotAsBytes, entriesAsBytes) = txlog2.latestSnapshotAndSubsequentEntries
|
||||
Thread.sleep(200)
|
||||
new String(snapshotAsBytes, "UTF-8") must equal("snapshot")
|
||||
|
||||
|
|
@ -266,7 +266,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
|
||||
"be able to record entries then a snapshot then more entries - and then read from the snapshot and the following entries - asynchronous" in {
|
||||
val uuid = (new UUID).toString
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, true, null, JavaSerializer)
|
||||
val txlog1 = TransactionLog.newLogFor(uuid, true, null)
|
||||
Thread.sleep(200)
|
||||
|
||||
val entry = "hello".getBytes("UTF-8")
|
||||
|
|
@ -286,9 +286,9 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA
|
|||
Thread.sleep(200)
|
||||
txlog1.close
|
||||
|
||||
val txlog2 = TransactionLog.logFor(uuid, true, null, JavaSerializer)
|
||||
val txlog2 = TransactionLog.logFor(uuid, true, null)
|
||||
Thread.sleep(200)
|
||||
val (snapshotAsBytes, entriesAsBytes) = txlog2.toByteArraysLatestSnapshot
|
||||
val (snapshotAsBytes, entriesAsBytes) = txlog2.latestSnapshotAndSubsequentEntries
|
||||
Thread.sleep(200)
|
||||
new String(snapshotAsBytes, "UTF-8") must equal("snapshot")
|
||||
val entries = entriesAsBytes.map(bytes ⇒ new String(bytes, "UTF-8"))
|
||||
|
|
|
|||
|
|
@ -1,110 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
|
||||
*/
|
||||
|
||||
package akka.cluster.api.migration.explicit
|
||||
|
||||
import org.scalatest.WordSpec
|
||||
import org.scalatest.matchers.MustMatchers
|
||||
import org.scalatest.BeforeAndAfterAll
|
||||
|
||||
import akka.actor._
|
||||
import Actor._
|
||||
import akka.cluster._
|
||||
import ChangeListener._
|
||||
import Cluster._
|
||||
import akka.config.Config
|
||||
import akka.serialization.Serialization
|
||||
|
||||
import java.util.concurrent._
|
||||
|
||||
object MigrationExplicitMultiJvmSpec {
|
||||
var NrOfNodes = 2
|
||||
|
||||
class HelloWorld extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "Hello" ⇒
|
||||
self.reply("World from node [" + Config.nodename + "]")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class MigrationExplicitMultiJvmNode1 extends MasterClusterTestNode {
|
||||
import MigrationExplicitMultiJvmSpec._
|
||||
|
||||
val testNodes = NrOfNodes
|
||||
|
||||
"A cluster" must {
|
||||
|
||||
"be able to migrate an actor from one node to another" in {
|
||||
|
||||
barrier("start-node-1", NrOfNodes) {
|
||||
node.start()
|
||||
}
|
||||
|
||||
barrier("start-node-2", NrOfNodes) {
|
||||
}
|
||||
|
||||
barrier("store-1-in-node-1", NrOfNodes) {
|
||||
val serializer = Serialization.serializerFor(classOf[HelloWorld]).fold(x ⇒ fail("No serializer found"), s ⇒ s)
|
||||
node.store("hello-world", classOf[HelloWorld], serializer)
|
||||
}
|
||||
|
||||
barrier("use-1-in-node-2", NrOfNodes) {
|
||||
}
|
||||
|
||||
barrier("migrate-from-node2-to-node1", NrOfNodes) {
|
||||
}
|
||||
|
||||
barrier("check-actor-is-moved-to-node1", NrOfNodes) {
|
||||
node.isInUseOnNode("hello-world") must be(true)
|
||||
|
||||
val actorRef = Actor.registry.local.actorFor("hello-world").getOrElse(fail("Actor should have been in the local actor registry"))
|
||||
actorRef.address must be("hello-world")
|
||||
(actorRef ? "Hello").as[String].get must be("World from node [node1]")
|
||||
}
|
||||
|
||||
node.shutdown()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class MigrationExplicitMultiJvmNode2 extends ClusterTestNode {
|
||||
import MigrationExplicitMultiJvmSpec._
|
||||
|
||||
"A cluster" must {
|
||||
|
||||
"be able to migrate an actor from one node to another" in {
|
||||
|
||||
barrier("start-node-1", NrOfNodes) {
|
||||
}
|
||||
|
||||
barrier("start-node-2", NrOfNodes) {
|
||||
node.start()
|
||||
}
|
||||
|
||||
barrier("store-1-in-node-1", NrOfNodes) {
|
||||
}
|
||||
|
||||
barrier("use-1-in-node-2", NrOfNodes) {
|
||||
val actorOrOption = node.use("hello-world")
|
||||
if (actorOrOption.isEmpty) fail("Actor could not be retrieved")
|
||||
|
||||
val actorRef = actorOrOption.get
|
||||
actorRef.address must be("hello-world")
|
||||
|
||||
(actorRef ? "Hello").as[String].get must be("World from node [node2]")
|
||||
}
|
||||
|
||||
barrier("migrate-from-node2-to-node1", NrOfNodes) {
|
||||
node.migrate(NodeAddress(node.nodeAddress.clusterName, "node1"), "hello-world")
|
||||
Thread.sleep(2000)
|
||||
}
|
||||
|
||||
barrier("check-actor-is-moved-to-node1", NrOfNodes) {
|
||||
}
|
||||
|
||||
node.shutdown()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -2,7 +2,7 @@
|
|||
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
|
||||
*/
|
||||
|
||||
package akka.cluster.api.migration.automatic
|
||||
package akka.cluster.migration.automatic
|
||||
|
||||
import org.scalatest.WordSpec
|
||||
import org.scalatest.matchers.MustMatchers
|
||||
|
|
@ -0,0 +1,111 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
|
||||
*
|
||||
*
|
||||
* package akka.cluster.migration.explicit
|
||||
*
|
||||
* import org.scalatest.WordSpec
|
||||
* import org.scalatest.matchers.MustMatchers
|
||||
* import org.scalatest.BeforeAndAfterAll
|
||||
*
|
||||
* import akka.actor._
|
||||
* import Actor._
|
||||
* import akka.cluster._
|
||||
* import ChangeListener._
|
||||
* import Cluster._
|
||||
* import akka.config.Config
|
||||
* import akka.serialization.Serialization
|
||||
*
|
||||
* import java.util.concurrent._
|
||||
*
|
||||
* object MigrationExplicitMultiJvmSpec {
|
||||
* var NrOfNodes = 2
|
||||
*
|
||||
* class HelloWorld extends Actor with Serializable {
|
||||
* def receive = {
|
||||
* case "Hello" ⇒
|
||||
* self.reply("World from node [" + Config.nodename + "]")
|
||||
* }
|
||||
* }
|
||||
* }
|
||||
*
|
||||
* class MigrationExplicitMultiJvmNode1 extends MasterClusterTestNode {
|
||||
* import MigrationExplicitMultiJvmSpec._
|
||||
*
|
||||
* val testNodes = NrOfNodes
|
||||
*
|
||||
* "A cluster" must {
|
||||
*
|
||||
* "be able to migrate an actor from one node to another" in {
|
||||
*
|
||||
* barrier("start-node-1", NrOfNodes) {
|
||||
* node.start()
|
||||
* }
|
||||
*
|
||||
* barrier("start-node-2", NrOfNodes) {
|
||||
* }
|
||||
*
|
||||
* barrier("store-1-in-node-1", NrOfNodes) {
|
||||
* val serializer = Serialization.serializerFor(classOf[HelloWorld]).fold(x ⇒ fail("No serializer found"), s ⇒ s)
|
||||
* node.store("hello-world", classOf[HelloWorld], serializer)
|
||||
* }
|
||||
*
|
||||
* barrier("use-1-in-node-2", NrOfNodes) {
|
||||
* }
|
||||
*
|
||||
* barrier("migrate-from-node2-to-node1", NrOfNodes) {
|
||||
* }
|
||||
*
|
||||
* barrier("check-actor-is-moved-to-node1", NrOfNodes) {
|
||||
* node.isInUseOnNode("hello-world") must be(true)
|
||||
*
|
||||
* val actorRef = Actor.registry.local.actorFor("hello-world").getOrElse(fail("Actor should have been in the local actor registry"))
|
||||
* actorRef.address must be("hello-world")
|
||||
* (actorRef ? "Hello").as[String].get must be("World from node [node1]")
|
||||
* }
|
||||
*
|
||||
* node.shutdown()
|
||||
* }
|
||||
* }
|
||||
* }
|
||||
*
|
||||
* class MigrationExplicitMultiJvmNode2 extends ClusterTestNode {
|
||||
* import MigrationExplicitMultiJvmSpec._
|
||||
*
|
||||
* "A cluster" must {
|
||||
*
|
||||
* "be able to migrate an actor from one node to another" in {
|
||||
*
|
||||
* barrier("start-node-1", NrOfNodes) {
|
||||
* }
|
||||
*
|
||||
* barrier("start-node-2", NrOfNodes) {
|
||||
* node.start()
|
||||
* }
|
||||
*
|
||||
* barrier("store-1-in-node-1", NrOfNodes) {
|
||||
* }
|
||||
*
|
||||
* barrier("use-1-in-node-2", NrOfNodes) {
|
||||
* val actorOrOption = node.use("hello-world")
|
||||
* if (actorOrOption.isEmpty) fail("Actor could not be retrieved")
|
||||
*
|
||||
* val actorRef = actorOrOption.get
|
||||
* actorRef.address must be("hello-world")
|
||||
*
|
||||
* (actorRef ? "Hello").as[String].get must be("World from node [node2]")
|
||||
* }
|
||||
*
|
||||
* barrier("migrate-from-node2-to-node1", NrOfNodes) {
|
||||
* node.migrate(NodeAddress(node.nodeAddress.clusterName, "node1"), "hello-world")
|
||||
* Thread.sleep(2000)
|
||||
* }
|
||||
*
|
||||
* barrier("check-actor-is-moved-to-node1", NrOfNodes) {
|
||||
* }
|
||||
*
|
||||
* node.shutdown()
|
||||
* }
|
||||
* }
|
||||
* }
|
||||
*/
|
||||
|
|
@ -0,0 +1,8 @@
|
|||
akka.enabled-modules = ["cluster"]
|
||||
akka.event-handler-level = "DEBUG"
|
||||
akka.actor.deployment.hello-world.router = "direct"
|
||||
akka.actor.deployment.hello-world.clustered.replicas = 1
|
||||
|
||||
akka.actor.deployment.hello-world.clustered.replication.storage = "transaction-log"
|
||||
akka.actor.deployment.hello-world.clustered.replication.strategy = "write-through"
|
||||
akka.cluster.replication.snapshot-frequency = 1000
|
||||
|
|
@ -0,0 +1 @@
|
|||
-Dakka.cluster.nodename=node1 -Dakka.cluster.port=9991
|
||||
|
|
@ -0,0 +1,7 @@
|
|||
akka.enabled-modules = ["cluster"]
|
||||
akka.event-handler-level = "DEBUG"
|
||||
akka.actor.deployment.hello-world.router = "direct"
|
||||
akka.actor.deployment.hello-world.clustered.replicas = 1
|
||||
akka.actor.deployment.hello-world.clustered.replication.storage = "transaction-log"
|
||||
akka.actor.deployment.hello-world.clustered.replication.strategy = "write-through"
|
||||
akka.cluster.replication.snapshot-frequency = 1000
|
||||
|
|
@ -0,0 +1 @@
|
|||
-Dakka.cluster.nodename=node2 -Dakka.cluster.port=9992
|
||||
|
|
@ -0,0 +1,118 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
|
||||
*/
|
||||
|
||||
package akka.cluster.replication.transactionlog.writethrough.nosnapshot
|
||||
|
||||
import akka.actor._
|
||||
import akka.cluster._
|
||||
import Cluster._
|
||||
import akka.config.Config
|
||||
|
||||
object ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmSpec {
|
||||
var NrOfNodes = 2
|
||||
|
||||
sealed trait TransactionLogMessage extends Serializable
|
||||
case class Count(nr: Int) extends TransactionLogMessage
|
||||
case class Log(full: String) extends TransactionLogMessage
|
||||
case object GetLog extends TransactionLogMessage
|
||||
|
||||
class HelloWorld extends Actor with Serializable {
|
||||
var log = ""
|
||||
def receive = {
|
||||
case Count(nr) ⇒
|
||||
log += nr.toString
|
||||
self.reply("World from node [" + Config.nodename + "]")
|
||||
case GetLog ⇒
|
||||
self.reply(Log(log))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode1 extends ClusterTestNode {
|
||||
import ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmSpec._
|
||||
|
||||
"A cluster" must {
|
||||
|
||||
"be able to replicate an actor with a transaction log and replay transaction log after actor migration" in {
|
||||
|
||||
barrier("start-node1", NrOfNodes) {
|
||||
node.start()
|
||||
}
|
||||
|
||||
barrier("create-actor-on-node1", NrOfNodes) {
|
||||
val actorRef = Actor.actorOf[HelloWorld]("hello-world").start()
|
||||
node.isInUseOnNode("hello-world") must be(true)
|
||||
actorRef.address must be("hello-world")
|
||||
var counter = 0
|
||||
(actorRef ? Count(counter)).as[String].get must be("World from node [node1]")
|
||||
counter += 1
|
||||
(actorRef ? Count(counter)).as[String].get must be("World from node [node1]")
|
||||
counter += 1
|
||||
(actorRef ? Count(counter)).as[String].get must be("World from node [node1]")
|
||||
counter += 1
|
||||
(actorRef ? Count(counter)).as[String].get must be("World from node [node1]")
|
||||
counter += 1
|
||||
(actorRef ? Count(counter)).as[String].get must be("World from node [node1]")
|
||||
counter += 1
|
||||
(actorRef ? Count(counter)).as[String].get must be("World from node [node1]")
|
||||
counter += 1
|
||||
(actorRef ? Count(counter)).as[String].get must be("World from node [node1]")
|
||||
counter += 1
|
||||
(actorRef ? Count(counter)).as[String].get must be("World from node [node1]")
|
||||
counter += 1
|
||||
(actorRef ? Count(counter)).as[String].get must be("World from node [node1]")
|
||||
counter += 1
|
||||
(actorRef ? Count(counter)).as[String].get must be("World from node [node1]")
|
||||
}
|
||||
|
||||
barrier("start-node2", NrOfNodes) {
|
||||
}
|
||||
|
||||
node.shutdown()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode2 extends MasterClusterTestNode {
|
||||
import ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmSpec._
|
||||
|
||||
val testNodes = NrOfNodes
|
||||
|
||||
"A cluster" must {
|
||||
|
||||
"be able to replicate an actor with a transaction log and replay transaction log after actor migration" in {
|
||||
|
||||
barrier("start-node1", NrOfNodes) {
|
||||
}
|
||||
|
||||
barrier("create-actor-on-node1", NrOfNodes) {
|
||||
}
|
||||
|
||||
barrier("start-node2", NrOfNodes) {
|
||||
node.start()
|
||||
}
|
||||
|
||||
Thread.sleep(5000) // wait for fail-over from node1 to node2
|
||||
|
||||
barrier("check-fail-over-to-node2", NrOfNodes - 1) {
|
||||
// both remaining nodes should now have the replica
|
||||
node.isInUseOnNode("hello-world") must be(true)
|
||||
val actorRef = Actor.registry.local.actorFor("hello-world").getOrElse(fail("Actor should have been in the local actor registry"))
|
||||
actorRef.address must be("hello-world")
|
||||
(actorRef ? GetLog).as[Log].get must be(Log("0123456789"))
|
||||
}
|
||||
|
||||
node.shutdown()
|
||||
}
|
||||
}
|
||||
|
||||
override def onReady() {
|
||||
LocalBookKeeperEnsemble.start()
|
||||
}
|
||||
|
||||
override def onShutdown() {
|
||||
TransactionLog.shutdown()
|
||||
LocalBookKeeperEnsemble.shutdown()
|
||||
}
|
||||
}
|
||||
|
|
@ -1,5 +1,5 @@
|
|||
akka.enabled-modules = ["cluster"]
|
||||
akka.event-handler-level = "DEBUG"
|
||||
akka.actor.deployment.service-hello.router = "round-robin"
|
||||
akka.actor.deployment.service-hello.clustered.preferred-nodes = ["host:node1"]
|
||||
akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node1"]
|
||||
akka.actor.deployment.service-hello.clustered.replicas = 1
|
||||
|
|
@ -18,8 +18,7 @@ import com.eaio.uuid.UUID
|
|||
* @author Roland Kuhn
|
||||
* @since 1.1
|
||||
*/
|
||||
class TestActorRef[T <: Actor](factory: () ⇒ T, address: String)
|
||||
extends LocalActorRef(factory, address, DeploymentConfig.Transient) {
|
||||
class TestActorRef[T <: Actor](factory: () ⇒ T, address: String) extends LocalActorRef(factory, address) {
|
||||
|
||||
dispatcher = CallingThreadDispatcher.global
|
||||
receiveTimeout = None
|
||||
|
|
|
|||
|
|
@ -40,13 +40,12 @@ akka {
|
|||
|
||||
service-ping { # stateless actor with replication factor 3 and round-robin load-balancer
|
||||
|
||||
format = "akka.serialization.Format$Default$" # serializer for messages and actor instance
|
||||
|
||||
router = "least-cpu" # routing (load-balance) scheme to use
|
||||
# available: "direct", "round-robin", "random",
|
||||
# "least-cpu", "least-ram", "least-messages"
|
||||
# or: fully qualified class name of the router class
|
||||
# default is "direct";
|
||||
# if 'replication' is used then the only available router is "direct"
|
||||
|
||||
clustered { # makes the actor available in the cluster registry
|
||||
# default (if omitted) is local non-clustered actor
|
||||
|
|
@ -56,16 +55,17 @@ akka {
|
|||
# available: "host:<hostname>", "ip:<ip address>" and "node:<node name>"
|
||||
# default is "host:localhost"
|
||||
|
||||
replicas = 3 # number of actor replicas in the cluster
|
||||
# available: positivoe integer (0-N) or the string "auto" for auto-scaling
|
||||
replicas = 3 # number of actor instances in the cluster
|
||||
# available: positive integer (0-N) or the string "auto" for auto-scaling
|
||||
# if "auto" is used then 'home' has no meaning
|
||||
# default is '0', meaning no replicas;
|
||||
# if the "direct" router is used then this configuration element is ignored
|
||||
# if the "direct" router is used then this element is ignored (always '1')
|
||||
|
||||
replication { # use replication or not?
|
||||
replication { # use replication or not? only makes sense for a stateful actor
|
||||
|
||||
# FIXME should we have this config option here? If so, implement it all through.
|
||||
serialize-mailbox = on # should the actor mailbox be part of the serialized snapshot?
|
||||
serialize-mailbox = off # should the actor mailbox be part of the serialized snapshot?
|
||||
# default is 'off'
|
||||
|
||||
storage = "transaction-log" # storage model for replication
|
||||
# available: "transaction-log" and "data-grid"
|
||||
|
|
@ -189,6 +189,8 @@ akka {
|
|||
secure-cookie = "" # Generate your own with '$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh'
|
||||
# or using 'akka.util.Crypt.generateSecureCookie'
|
||||
|
||||
log-directory = "_akka_cluster" # Where ZooKeeper should store the logs and data files
|
||||
|
||||
replication {
|
||||
digest-type = "MAC" # Options: CRC32 (cheap & unsafe), MAC (expensive & secure using password)
|
||||
password = "secret" # FIXME: store open in file?
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue