Remove docs for ActorPub/Sub and write persistence query docs as stages (#26844)
* use separate db columns * Use io dispatcher for sample stage
This commit is contained in:
parent
2bbf13f707
commit
c65bf2d276
22 changed files with 272 additions and 2194 deletions
|
|
@ -1,101 +0,0 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package docs.persistence.query
|
||||
|
||||
import akka.actor.Props
|
||||
import akka.persistence.PersistentRepr
|
||||
import akka.persistence.query.{ EventEnvelope, Sequence }
|
||||
import akka.serialization.SerializationExtension
|
||||
import akka.stream.actor.ActorPublisher
|
||||
import akka.stream.actor.ActorPublisherMessage.{ Cancel, Request }
|
||||
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
|
||||
object MyEventsByTagPublisher {
|
||||
def props(tag: String, offset: Long, refreshInterval: FiniteDuration): Props =
|
||||
Props(new MyEventsByTagPublisher(tag, offset, refreshInterval))
|
||||
}
|
||||
|
||||
//#events-by-tag-publisher
|
||||
class MyEventsByTagPublisher(tag: String, offset: Long, refreshInterval: FiniteDuration)
|
||||
extends ActorPublisher[EventEnvelope] {
|
||||
|
||||
private case object Continue
|
||||
|
||||
private val connection: java.sql.Connection = ???
|
||||
|
||||
private val Limit = 1000
|
||||
private var currentOffset = offset
|
||||
var buf = Vector.empty[EventEnvelope]
|
||||
|
||||
import context.dispatcher
|
||||
val continueTask = context.system.scheduler.schedule(refreshInterval, refreshInterval, self, Continue)
|
||||
|
||||
override def postStop(): Unit = {
|
||||
continueTask.cancel()
|
||||
}
|
||||
|
||||
def receive = {
|
||||
case _: Request | Continue =>
|
||||
query()
|
||||
deliverBuf()
|
||||
|
||||
case Cancel =>
|
||||
context.stop(self)
|
||||
}
|
||||
|
||||
object Select {
|
||||
private def statement() = connection.prepareStatement("""
|
||||
SELECT id, persistent_repr FROM journal
|
||||
WHERE tag = ? AND id > ?
|
||||
ORDER BY id LIMIT ?
|
||||
""")
|
||||
|
||||
def run(tag: String, from: Long, limit: Int): Vector[(Long, Array[Byte])] = {
|
||||
val s = statement()
|
||||
try {
|
||||
s.setString(1, tag)
|
||||
s.setLong(2, from)
|
||||
s.setLong(3, limit)
|
||||
val rs = s.executeQuery()
|
||||
|
||||
val b = Vector.newBuilder[(Long, Array[Byte])]
|
||||
while (rs.next()) b += (rs.getLong(1) -> rs.getBytes(2))
|
||||
b.result()
|
||||
} finally s.close()
|
||||
}
|
||||
}
|
||||
|
||||
def query(): Unit =
|
||||
if (buf.isEmpty) {
|
||||
try {
|
||||
val result = Select.run(tag, currentOffset, Limit)
|
||||
currentOffset = if (result.nonEmpty) result.last._1 else currentOffset
|
||||
val serialization = SerializationExtension(context.system)
|
||||
|
||||
buf = result.map {
|
||||
case (id, bytes) =>
|
||||
val p = serialization.deserialize(bytes, classOf[PersistentRepr]).get
|
||||
EventEnvelope(offset = Sequence(id), p.persistenceId, p.sequenceNr, p.payload)
|
||||
}
|
||||
} catch {
|
||||
case e: Exception =>
|
||||
onErrorThenStop(e)
|
||||
}
|
||||
}
|
||||
|
||||
final def deliverBuf(): Unit =
|
||||
if (totalDemand > 0 && buf.nonEmpty) {
|
||||
if (totalDemand <= Int.MaxValue) {
|
||||
val (use, keep) = buf.splitAt(totalDemand.toInt)
|
||||
buf = keep
|
||||
use.foreach(onNext)
|
||||
} else {
|
||||
buf.foreach(onNext)
|
||||
buf = Vector.empty
|
||||
}
|
||||
}
|
||||
}
|
||||
//#events-by-tag-publisher
|
||||
|
|
@ -0,0 +1,110 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package docs.persistence.query
|
||||
|
||||
import akka.persistence.query.{ EventEnvelope, Offset }
|
||||
import akka.serialization.SerializationExtension
|
||||
import akka.stream.{ ActorAttributes, ActorMaterializer, Attributes, Outlet, SourceShape }
|
||||
import akka.stream.stage.{ GraphStage, GraphStageLogic, OutHandler, TimerGraphStageLogic }
|
||||
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
//#events-by-tag-publisher
|
||||
class MyEventsByTagSource(tag: String, offset: Long, refreshInterval: FiniteDuration)
|
||||
extends GraphStage[SourceShape[EventEnvelope]] {
|
||||
|
||||
private case object Continue
|
||||
val out: Outlet[EventEnvelope] = Outlet("MyEventByTagSource.out")
|
||||
override def shape: SourceShape[EventEnvelope] = SourceShape(out)
|
||||
|
||||
override protected def initialAttributes: Attributes = Attributes(ActorAttributes.IODispatcher)
|
||||
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
|
||||
new TimerGraphStageLogic(shape) with OutHandler {
|
||||
lazy val system = materializer match {
|
||||
case a: ActorMaterializer => a.system
|
||||
case _ =>
|
||||
throw new IllegalStateException("EventsByTagStage requires ActorMaterializer")
|
||||
}
|
||||
private val Limit = 1000
|
||||
private val connection: java.sql.Connection = ???
|
||||
private var currentOffset = offset
|
||||
private var buf = Vector.empty[EventEnvelope]
|
||||
private val serialization = SerializationExtension(system)
|
||||
|
||||
override def preStart(): Unit = {
|
||||
schedulePeriodically(Continue, refreshInterval)
|
||||
}
|
||||
|
||||
override def onPull(): Unit = {
|
||||
query()
|
||||
tryPush()
|
||||
}
|
||||
|
||||
override def onDownstreamFinish(): Unit = {
|
||||
// close connection if responsible for doing so
|
||||
}
|
||||
|
||||
private def query(): Unit = {
|
||||
if (buf.isEmpty) {
|
||||
try {
|
||||
buf = Select.run(tag, currentOffset, Limit)
|
||||
} catch {
|
||||
case NonFatal(e) =>
|
||||
failStage(e)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def tryPush(): Unit = {
|
||||
if (buf.nonEmpty && isAvailable(out)) {
|
||||
push(out, buf.head)
|
||||
buf = buf.tail
|
||||
}
|
||||
}
|
||||
|
||||
override protected def onTimer(timerKey: Any): Unit = timerKey match {
|
||||
case Continue =>
|
||||
query()
|
||||
tryPush()
|
||||
}
|
||||
|
||||
object Select {
|
||||
private def statement() =
|
||||
connection.prepareStatement("""
|
||||
SELECT id, persistence_id, seq_nr, serializer_id, serializer_manifest, payload
|
||||
FROM journal WHERE tag = ? AND id > ?
|
||||
ORDER BY id LIMIT ?
|
||||
""")
|
||||
|
||||
def run(tag: String, from: Long, limit: Int): Vector[EventEnvelope] = {
|
||||
val s = statement()
|
||||
try {
|
||||
s.setString(1, tag)
|
||||
s.setLong(2, from)
|
||||
s.setLong(3, limit)
|
||||
val rs = s.executeQuery()
|
||||
|
||||
val b = Vector.newBuilder[EventEnvelope]
|
||||
while (rs.next()) {
|
||||
val deserialized = serialization
|
||||
.deserialize(rs.getBytes("payload"), rs.getInt("serializer_id"), rs.getString("serializer_manifest"))
|
||||
.get
|
||||
currentOffset = rs.getLong("id")
|
||||
b += EventEnvelope(
|
||||
Offset.sequence(currentOffset),
|
||||
rs.getString("persistence_id"),
|
||||
rs.getLong("seq_nr"),
|
||||
deserialized)
|
||||
}
|
||||
b.result()
|
||||
} finally s.close()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
//#events-by-tag-publisher
|
||||
|
|
@ -66,11 +66,10 @@ object PersistenceQueryDocSpec {
|
|||
*/
|
||||
override def eventsByTag(tag: String, offset: Offset): Source[EventEnvelope, NotUsed] = offset match {
|
||||
case Sequence(offsetValue) =>
|
||||
val props = MyEventsByTagPublisher.props(tag, offsetValue, refreshInterval)
|
||||
Source.actorPublisher[EventEnvelope](props).mapMaterializedValue(_ => NotUsed)
|
||||
Source.fromGraph(new MyEventsByTagSource(tag, offsetValue, refreshInterval))
|
||||
case NoOffset => eventsByTag(tag, Sequence(0L)) //recursive
|
||||
case _ =>
|
||||
throw new IllegalArgumentException("LevelDB does not support " + offset.getClass.getName + " offsets")
|
||||
throw new IllegalArgumentException("MyJournal does not support " + offset.getClass.getName + " offsets")
|
||||
}
|
||||
|
||||
override def eventsByPersistenceId(
|
||||
|
|
|
|||
|
|
@ -1,99 +0,0 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package docs.stream
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import akka.actor.Props
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.actor.ActorPublisher
|
||||
import akka.stream.scaladsl.{ Flow, Sink, Source }
|
||||
import akka.testkit.AkkaSpec
|
||||
|
||||
object ActorPublisherDocSpec {
|
||||
|
||||
//#job-manager
|
||||
object JobManager {
|
||||
def props: Props = Props[JobManager]
|
||||
|
||||
final case class Job(payload: String)
|
||||
case object JobAccepted
|
||||
case object JobDenied
|
||||
}
|
||||
|
||||
class JobManager extends ActorPublisher[JobManager.Job] {
|
||||
import akka.stream.actor.ActorPublisherMessage._
|
||||
import JobManager._
|
||||
|
||||
val MaxBufferSize = 100
|
||||
var buf = Vector.empty[Job]
|
||||
|
||||
def receive = {
|
||||
case job: Job if buf.size == MaxBufferSize =>
|
||||
sender() ! JobDenied
|
||||
case job: Job =>
|
||||
sender() ! JobAccepted
|
||||
if (buf.isEmpty && totalDemand > 0)
|
||||
onNext(job)
|
||||
else {
|
||||
buf :+= job
|
||||
deliverBuf()
|
||||
}
|
||||
case Request(_) =>
|
||||
deliverBuf()
|
||||
case Cancel =>
|
||||
context.stop(self)
|
||||
}
|
||||
|
||||
@tailrec final def deliverBuf(): Unit =
|
||||
if (totalDemand > 0) {
|
||||
/*
|
||||
* totalDemand is a Long and could be larger than
|
||||
* what buf.splitAt can accept
|
||||
*/
|
||||
if (totalDemand <= Int.MaxValue) {
|
||||
val (use, keep) = buf.splitAt(totalDemand.toInt)
|
||||
buf = keep
|
||||
use.foreach(onNext)
|
||||
} else {
|
||||
val (use, keep) = buf.splitAt(Int.MaxValue)
|
||||
buf = keep
|
||||
use.foreach(onNext)
|
||||
deliverBuf()
|
||||
}
|
||||
}
|
||||
}
|
||||
//#job-manager
|
||||
}
|
||||
|
||||
class ActorPublisherDocSpec extends AkkaSpec {
|
||||
import ActorPublisherDocSpec._
|
||||
|
||||
implicit val materializer = ActorMaterializer()
|
||||
|
||||
"illustrate usage of ActorPublisher" in {
|
||||
def println(s: String): Unit =
|
||||
testActor ! s
|
||||
|
||||
//#actor-publisher-usage
|
||||
val jobManagerSource = Source.actorPublisher[JobManager.Job](JobManager.props)
|
||||
val ref = Flow[JobManager.Job]
|
||||
.map(_.payload.toUpperCase)
|
||||
.map { elem =>
|
||||
println(elem); elem
|
||||
}
|
||||
.to(Sink.ignore)
|
||||
.runWith(jobManagerSource)
|
||||
|
||||
ref ! JobManager.Job("a")
|
||||
ref ! JobManager.Job("b")
|
||||
ref ! JobManager.Job("c")
|
||||
//#actor-publisher-usage
|
||||
|
||||
expectMsg("A")
|
||||
expectMsg("B")
|
||||
expectMsg("C")
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,99 +0,0 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package docs.stream
|
||||
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Props
|
||||
import akka.routing.ActorRefRoutee
|
||||
import akka.routing.RoundRobinRoutingLogic
|
||||
import akka.routing.Router
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.actor.ActorSubscriber
|
||||
import akka.stream.actor.ActorSubscriberMessage
|
||||
import akka.stream.actor.MaxInFlightRequestStrategy
|
||||
import akka.stream.scaladsl.Sink
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.testkit.AkkaSpec
|
||||
import scala.concurrent.duration._
|
||||
|
||||
object ActorSubscriberDocSpec {
|
||||
//#worker-pool
|
||||
object WorkerPool {
|
||||
case class Msg(id: Int, replyTo: ActorRef)
|
||||
case class Work(id: Int)
|
||||
case class Reply(id: Int)
|
||||
case class Done(id: Int)
|
||||
|
||||
def props: Props = Props(new WorkerPool)
|
||||
}
|
||||
|
||||
class WorkerPool extends ActorSubscriber {
|
||||
import WorkerPool._
|
||||
import ActorSubscriberMessage._
|
||||
|
||||
val MaxQueueSize = 10
|
||||
var queue = Map.empty[Int, ActorRef]
|
||||
|
||||
val router = {
|
||||
val routees = Vector.fill(3) {
|
||||
ActorRefRoutee(context.actorOf(Props[Worker]))
|
||||
}
|
||||
Router(RoundRobinRoutingLogic(), routees)
|
||||
}
|
||||
|
||||
override val requestStrategy = new MaxInFlightRequestStrategy(max = MaxQueueSize) {
|
||||
override def inFlightInternally: Int = queue.size
|
||||
}
|
||||
|
||||
def receive = {
|
||||
case OnNext(Msg(id, replyTo)) =>
|
||||
queue += (id -> replyTo)
|
||||
assert(queue.size <= MaxQueueSize, s"queued too many: ${queue.size}")
|
||||
router.route(Work(id), self)
|
||||
case Reply(id) =>
|
||||
queue(id) ! Done(id)
|
||||
queue -= id
|
||||
if (canceled && queue.isEmpty) {
|
||||
context.stop(self)
|
||||
}
|
||||
case OnComplete =>
|
||||
if (queue.isEmpty) {
|
||||
context.stop(self)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class Worker extends Actor {
|
||||
import WorkerPool._
|
||||
def receive = {
|
||||
case Work(id) =>
|
||||
// ...
|
||||
sender() ! Reply(id)
|
||||
}
|
||||
}
|
||||
//#worker-pool
|
||||
|
||||
}
|
||||
|
||||
class ActorSubscriberDocSpec extends AkkaSpec {
|
||||
import ActorSubscriberDocSpec._
|
||||
|
||||
implicit val materializer = ActorMaterializer()
|
||||
|
||||
"illustrate usage of ActorSubscriber" in {
|
||||
val replyTo = testActor
|
||||
|
||||
//#actor-subscriber-usage
|
||||
val N = 117
|
||||
val worker = Source(1 to N).map(WorkerPool.Msg(_, replyTo)).runWith(Sink.actorSubscriber(WorkerPool.props))
|
||||
//#actor-subscriber-usage
|
||||
|
||||
watch(worker)
|
||||
receiveN(N).toSet should be((1 to N).map(WorkerPool.Done).toSet)
|
||||
expectTerminated(worker, 10.seconds)
|
||||
}
|
||||
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue