+cdd #16799 Add Distributed Data module

Previously know as [patriknw/akka-data-replication](https://github.com/patriknw/akka-data-replication),
which was originally inspired by [jboner/akka-crdt](https://github.com/jboner/akka-crdt).

The functionality is very similar to akka-data-replication 0.11.

Here is a list of the most important changes:

* The package name changed to `akka.cluster.ddata`
* The extension was renamed to `DistributedData`
* The keys changed from strings to classes with unique identifiers and type information of the data values,
  e.g. `ORSetKey[Int]("set2")`
* The optional read consistency parameter was removed from the `Update` message. If you need to read from
  other replicas before performing the update you have to first send a `Get` message and then continue with
  the ``Update`` when the ``GetSuccess`` is received.
* `BigInt` is used in `GCounter` and `PNCounter` instead of `Long`
* Improvements of java api
* Better documentation
This commit is contained in:
Patrik Nordwall 2015-05-17 12:28:47 +02:00
parent bf28260cd0
commit cbe5dd2cf5
69 changed files with 40036 additions and 3 deletions

View file

@ -0,0 +1,137 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata.sample
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorSystem
import akka.actor.Props
import akka.cluster.Cluster
import akka.cluster.ddata.DistributedData
import akka.cluster.ddata.ORSet
import com.typesafe.config.ConfigFactory
import akka.cluster.ddata.Replicator
import akka.cluster.ddata.ORSetKey
object LotsOfDataBot {
def main(args: Array[String]): Unit = {
if (args.isEmpty)
startup(Seq("2551", "2552", "0"))
else
startup(args)
}
def startup(ports: Seq[String]): Unit = {
ports.foreach { port
// Override the configuration of the port
val config = ConfigFactory.parseString("akka.remote.netty.tcp.port=" + port).
withFallback(ConfigFactory.load(
ConfigFactory.parseString("""
passive = off
max-entries = 100000
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote {
netty.tcp {
hostname = "127.0.0.1"
port = 0
}
}
akka.cluster {
seed-nodes = [
"akka.tcp://ClusterSystem@127.0.0.1:2551",
"akka.tcp://ClusterSystem@127.0.0.1:2552"]
auto-down-unreachable-after = 10s
}
akka.cluster.distributed-data.use-offheap-memory = off
akka.remote.log-frame-size-exceeding = 10000b
""")))
// Create an Akka system
val system = ActorSystem("ClusterSystem", config)
// Create an actor that handles cluster domain events
system.actorOf(Props[LotsOfDataBot], name = "dataBot")
}
}
private case object Tick
}
class LotsOfDataBot extends Actor with ActorLogging {
import LotsOfDataBot._
import Replicator._
val replicator = DistributedData(context.system).replicator
implicit val cluster = Cluster(context.system)
import context.dispatcher
val isPassive = context.system.settings.config.getBoolean("passive")
var tickTask =
if (isPassive)
context.system.scheduler.schedule(1.seconds, 1.seconds, self, Tick)
else
context.system.scheduler.schedule(20.millis, 20.millis, self, Tick)
val startTime = System.nanoTime()
var count = 1L
val maxEntries = context.system.settings.config.getInt("max-entries")
def receive = if (isPassive) passive else active
def active: Receive = {
case Tick
val loop = if (count >= maxEntries) 1 else 100
for (_ 1 to loop) {
count += 1
if (count % 10000 == 0)
log.info("Reached {} entries", count)
if (count == maxEntries) {
log.info("Reached {} entries", count)
tickTask.cancel()
tickTask = context.system.scheduler.schedule(1.seconds, 1.seconds, self, Tick)
}
val key = ORSetKey[String]((count % maxEntries).toString)
if (count <= 100)
replicator ! Subscribe(key, self)
val s = ThreadLocalRandom.current().nextInt(97, 123).toChar.toString
if (count <= maxEntries || ThreadLocalRandom.current().nextBoolean()) {
// add
replicator ! Update(key, ORSet(), WriteLocal)(_ + s)
} else {
// remove
replicator ! Update(key, ORSet(), WriteLocal)(_ - s)
}
}
case _: UpdateResponse[_] // ignore
case c @ Changed(ORSetKey(id))
val ORSet(elements) = c.dataValue
log.info("Current elements: {} -> {}", id, elements)
}
def passive: Receive = {
case Tick
if (!tickTask.isCancelled)
replicator ! GetKeyIds
case GetKeyIdsResult(keys)
if (keys.size >= maxEntries) {
tickTask.cancel()
val duration = (System.nanoTime() - startTime).nanos.toMillis
log.info("It took {} ms to replicate {} entries", duration, keys.size)
}
case c @ Changed(ORSetKey(id))
val ORSet(elements) = c.dataValue
log.info("Current elements: {} -> {}", id, elements)
}
override def postStop(): Unit = tickTask.cancel()
}