Documentation improvements

* Re enabling Java tests in akka-docs (they were not run before)
* Fixed bug #19764
* #19735 Rewrote every sample using the deprecated PushPullStage and friends
  using GraphStage
* Pruned old unused graph images
* Added missing graffle file for new graph images
This commit is contained in:
Johan Andrén 2016-02-11 16:39:25 +01:00
parent 8f3c5aa17f
commit 737991c01c
103 changed files with 1136 additions and 4749 deletions

View file

@ -11,7 +11,6 @@ import akka.http.scaladsl.Http.ServerBinding
import akka.http.scaladsl.model._
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.{ Flow, Sink }
import akka.stream.stage.{ Context, PushStage }
import akka.testkit.TestActors
import org.scalatest.{ Matchers, WordSpec }
import scala.language.postfixOps
@ -107,17 +106,9 @@ class HttpServerExampleSpec extends WordSpec with Matchers {
val failureMonitor: ActorRef = system.actorOf(MyExampleMonitoringActor.props)
val reactToTopLevelFailures = Flow[IncomingConnection]
.transform { () =>
new PushStage[IncomingConnection, IncomingConnection] {
override def onPush(elem: IncomingConnection, ctx: Context[IncomingConnection]) =
ctx.push(elem)
override def onUpstreamFailure(cause: Throwable, ctx: Context[IncomingConnection]) = {
failureMonitor ! cause
super.onUpstreamFailure(cause, ctx)
}
}
}
.watchTermination()((_, termination) => termination.onFailure {
case cause => failureMonitor ! cause
})
serverSource
.via(reactToTopLevelFailures)
@ -134,16 +125,10 @@ class HttpServerExampleSpec extends WordSpec with Matchers {
val serverSource = Http().bind(host, port)
val reactToConnectionFailure = Flow[HttpRequest]
.transform { () =>
new PushStage[HttpRequest, HttpRequest] {
override def onPush(elem: HttpRequest, ctx: Context[HttpRequest]) =
ctx.push(elem)
override def onUpstreamFailure(cause: Throwable, ctx: Context[HttpRequest]) = {
// handle the failure somehow
super.onUpstreamFailure(cause, ctx)
}
}
.recover[HttpRequest] {
case ex =>
// handle the failure somehow
throw ex
}
val httpEcho = Flow[HttpRequest]

View file

@ -66,52 +66,68 @@ object BidiFlowDocSpec {
ByteString.newBuilder.putInt(len).append(bytes).result()
}
class FrameParser extends PushPullStage[ByteString, ByteString] {
// this holds the received but not yet parsed bytes
var stash = ByteString.empty
// this holds the current message length or -1 if at a boundary
var needed = -1
class FrameParser extends GraphStage[FlowShape[ByteString, ByteString]] {
override def onPush(bytes: ByteString, ctx: Context[ByteString]) = {
stash ++= bytes
run(ctx)
}
override def onPull(ctx: Context[ByteString]) = run(ctx)
override def onUpstreamFinish(ctx: Context[ByteString]) =
if (stash.isEmpty) ctx.finish()
else ctx.absorbTermination() // we still have bytes to emit
val in = Inlet[ByteString]("FrameParser.in")
val out = Outlet[ByteString]("FrameParser.out")
override val shape = FlowShape.of(in, out)
private def run(ctx: Context[ByteString]): SyncDirective =
if (needed == -1) {
// are we at a boundary? then figure out next length
if (stash.length < 4) pullOrFinish(ctx)
else {
needed = stash.iterator.getInt
stash = stash.drop(4)
run(ctx) // cycle back to possibly already emit the next chunk
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
// this holds the received but not yet parsed bytes
var stash = ByteString.empty
// this holds the current message length or -1 if at a boundary
var needed = -1
setHandler(out, new OutHandler {
override def onPull(): Unit = {
if (isClosed(in)) run()
else pull(in)
}
})
setHandler(in, new InHandler {
override def onPush(): Unit = {
val bytes = grab(in)
stash = stash ++ bytes
run()
}
} else if (stash.length < needed) {
// we are in the middle of a message, need more bytes
pullOrFinish(ctx)
} else {
// we have enough to emit at least one message, so do it
val emit = stash.take(needed)
stash = stash.drop(needed)
needed = -1
ctx.push(emit)
}
/*
* After having called absorbTermination() we cannot pull any more, so if we need
* more data we will just have to give up.
*/
private def pullOrFinish(ctx: Context[ByteString]) =
if (ctx.isFinishing) ctx.finish()
else ctx.pull()
override def onUpstreamFinish(): Unit = {
if (stash.isEmpty) completeStage()
// wait with completion and let run() complete when the
// rest of the stash has been sent downstream
}
})
private def run(): Unit = {
if (needed == -1) {
// are we at a boundary? then figure out next length
if (stash.length < 4) {
if (isClosed(in)) completeStage()
else pull(in)
} else {
needed = stash.iterator.getInt
stash = stash.drop(4)
run() // cycle back to possibly already emit the next chunk
}
} else if (stash.length < needed) {
// we are in the middle of a message, need more bytes,
// or have to stop if input closed
if (isClosed(in)) completeStage()
else pull(in)
} else {
// we have enough to emit at least one message, so do it
val emit = stash.take(needed)
stash = stash.drop(needed)
needed = -1
push(out, emit)
}
}
}
}
val outbound = b.add(Flow[ByteString].map(addLengthHeader))
val inbound = b.add(Flow[ByteString].transform(() => new FrameParser))
val inbound = b.add(Flow[ByteString].via(new FrameParser))
BidiShape.fromFlows(outbound, inbound)
})
//#framing

View file

@ -1,192 +0,0 @@
package docs.stream
import akka.stream._
import akka.stream.scaladsl.{ Sink, Source, Flow, Keep }
import akka.stream.testkit.AkkaSpec
import org.scalatest.concurrent.ScalaFutures
import scala.collection.immutable
import scala.concurrent.Await
import scala.concurrent.duration._
class FlowStagesSpec extends AkkaSpec with ScalaFutures {
//#import-stage
import akka.stream.stage._
//#import-stage
implicit val materializer = ActorMaterializer()
"stages demo" must {
"demonstrate various PushPullStages" in {
//#one-to-one
class Map[A, B](f: A => B) extends PushPullStage[A, B] {
override def onPush(elem: A, ctx: Context[B]): SyncDirective =
ctx.push(f(elem))
override def onPull(ctx: Context[B]): SyncDirective =
ctx.pull()
}
//#one-to-one
//#many-to-one
class Filter[A](p: A => Boolean) extends PushPullStage[A, A] {
override def onPush(elem: A, ctx: Context[A]): SyncDirective =
if (p(elem)) ctx.push(elem)
else ctx.pull()
override def onPull(ctx: Context[A]): SyncDirective =
ctx.pull()
}
//#many-to-one
//#one-to-many
class Duplicator[A]() extends PushPullStage[A, A] {
private var lastElem: A = _
private var oneLeft = false
override def onPush(elem: A, ctx: Context[A]): SyncDirective = {
lastElem = elem
oneLeft = true
ctx.push(elem)
}
override def onPull(ctx: Context[A]): SyncDirective =
if (!ctx.isFinishing) {
// the main pulling logic is below as it is demonstrated on the illustration
if (oneLeft) {
oneLeft = false
ctx.push(lastElem)
} else
ctx.pull()
} else {
// If we need to emit a final element after the upstream
// finished
if (oneLeft) ctx.pushAndFinish(lastElem)
else ctx.finish()
}
override def onUpstreamFinish(ctx: Context[A]): TerminationDirective =
ctx.absorbTermination()
}
//#one-to-many
val keyedSink = Sink.head[immutable.Seq[Int]]
val sink = Flow[Int].grouped(10).toMat(keyedSink)(Keep.right)
//#stage-chain
val resultFuture = Source(1 to 10)
.transform(() => new Filter(_ % 2 == 0))
.transform(() => new Duplicator())
.transform(() => new Map(_ / 2))
.runWith(sink)
//#stage-chain
Await.result(resultFuture, 3.seconds) should be(Seq(1, 1, 2, 2, 3, 3, 4, 4, 5, 5))
}
"demonstrate various PushStages" in {
import akka.stream.stage._
//#pushstage
class Map[A, B](f: A => B) extends PushStage[A, B] {
override def onPush(elem: A, ctx: Context[B]): SyncDirective =
ctx.push(f(elem))
}
class Filter[A](p: A => Boolean) extends PushStage[A, A] {
override def onPush(elem: A, ctx: Context[A]): SyncDirective =
if (p(elem)) ctx.push(elem)
else ctx.pull()
}
//#pushstage
}
"demonstrate GraphStage" in {
//#doubler-stateful
class Duplicator[A] extends GraphStage[FlowShape[A, A]] {
val in = Inlet[A]("Duplicator.in")
val out = Outlet[A]("Duplicator.out")
val shape: FlowShape[A, A] = FlowShape(in, out)
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
new GraphStageLogic(shape) {
setHandler(in, new InHandler {
override def onPush(): Unit = {
val elem = grab(in)
emitMultiple(out, List(elem, elem))
}
})
setHandler(out, new OutHandler {
override def onPull(): Unit = pull(in)
})
}
}
//#doubler-stateful
val duplicator = Flow.fromGraph(new Duplicator[Int])
val fold = Source(1 to 2).via(duplicator).runFold("")(_ + _)
whenReady(fold) { s
s should be("1122")
}
}
"demonstrate DetachedStage" in {
//#detached
class Buffer2[T]() extends DetachedStage[T, T] {
private var buf = Vector.empty[T]
private var capacity = 2
private def isFull = capacity == 0
private def isEmpty = capacity == 2
private def dequeue(): T = {
capacity += 1
val next = buf.head
buf = buf.tail
next
}
private def enqueue(elem: T) = {
capacity -= 1
buf = buf :+ elem
}
override def onPull(ctx: DetachedContext[T]): DownstreamDirective = {
if (isEmpty) {
if (ctx.isFinishing) ctx.finish() // No more elements will arrive
else ctx.holdDownstream() // waiting until new elements
} else {
val next = dequeue()
if (ctx.isHoldingUpstream) ctx.pushAndPull(next) // release upstream
else ctx.push(next)
}
}
override def onPush(elem: T, ctx: DetachedContext[T]): UpstreamDirective = {
enqueue(elem)
if (isFull) ctx.holdUpstream() // Queue is now full, wait until new empty slot
else {
if (ctx.isHoldingDownstream) ctx.pushAndPull(dequeue()) // Release downstream
else ctx.pull()
}
}
override def onUpstreamFinish(ctx: DetachedContext[T]): TerminationDirective = {
if (!isEmpty) ctx.absorbTermination() // still need to flush from buffer
else ctx.finish() // already empty, finishing
}
}
//#detached
}
}
}

View file

@ -9,6 +9,7 @@ import akka.stream.stage._
import akka.stream._
import akka.stream.testkit.{ TestPublisher, TestSubscriber, AkkaSpec }
import akka.testkit.TestLatch
import scala.collection.mutable
import scala.concurrent.{ Promise, Await, Future }
@ -271,6 +272,7 @@ class GraphStageDocSpec extends AkkaSpec {
"Demonstrate an asynchronous side channel" in {
import system.dispatcher
//#async-side-channel
// will close upstream when the future completes
class KillSwitch[A](switch: Future[Unit]) extends GraphStage[FlowShape[A, A]] {
@ -301,20 +303,31 @@ class GraphStageDocSpec extends AkkaSpec {
//#async-side-channel
// tests:
val switch = Promise[Unit]()
val duplicator = Flow.fromGraph(new KillSwitch[Int](switch.future))
// TODO this is probably racey, is there a way to make sure it happens after?
val valueAfterKill = switch.future.flatMap(_ => Future(4))
val in = TestPublisher.probe[Int]()
val out = TestSubscriber.probe[Int]()
val result =
Source(Vector(1, 2, 3)).concat(Source.fromFuture(valueAfterKill))
.via(duplicator)
.runFold(Seq.empty[Int])((elem, acc) => elem :+ acc)
Source.fromPublisher(in)
.via(duplicator)
.to(Sink.fromSubscriber(out))
.withAttributes(Attributes.inputBuffer(1, 1))
.run()
val sub = in.expectSubscription()
out.request(1)
sub.expectRequest()
sub.sendNext(1)
out.expectNext(1)
switch.success(Unit)
Await.result(result, 3.seconds) should ===(Seq(1, 2, 3))
out.expectComplete()
}
"Demonstrate a graph stage with a timer" in {

View file

@ -1,6 +1,7 @@
package docs.stream.cookbook
import akka.NotUsed
import akka.stream.{ Attributes, Outlet, Inlet, FlowShape }
import akka.stream.scaladsl.{ Flow, Sink, Source }
import akka.util.ByteString
@ -18,34 +19,49 @@ class RecipeByteStrings extends RecipeSpec {
//#bytestring-chunker
import akka.stream.stage._
class Chunker(val chunkSize: Int) extends PushPullStage[ByteString, ByteString] {
private var buffer = ByteString.empty
class Chunker(val chunkSize: Int) extends GraphStage[FlowShape[ByteString, ByteString]] {
val in = Inlet[ByteString]("Chunker.in")
val out = Outlet[ByteString]("Chunker.out")
override val shape = FlowShape.of(in, out)
override def onPush(elem: ByteString, ctx: Context[ByteString]): SyncDirective = {
buffer ++= elem
emitChunkOrPull(ctx)
}
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
private var buffer = ByteString.empty
override def onPull(ctx: Context[ByteString]): SyncDirective = emitChunkOrPull(ctx)
setHandler(out, new OutHandler {
override def onPull(): Unit = {
if (isClosed(in)) emitChunk()
else pull(in)
}
})
setHandler(in, new InHandler {
override def onPush(): Unit = {
val elem = grab(in)
buffer ++= elem
emitChunk()
}
override def onUpstreamFinish(ctx: Context[ByteString]): TerminationDirective =
if (buffer.nonEmpty) ctx.absorbTermination()
else ctx.finish()
override def onUpstreamFinish(): Unit = {
if (buffer.isEmpty) completeStage()
// elements left in buffer, keep accepting downstream pulls
// and push from buffer until buffer is emitted
}
})
private def emitChunkOrPull(ctx: Context[ByteString]): SyncDirective = {
if (buffer.isEmpty) {
if (ctx.isFinishing) ctx.finish()
else ctx.pull()
} else {
val (emit, nextBuffer) = buffer.splitAt(chunkSize)
buffer = nextBuffer
ctx.push(emit)
private def emitChunk(): Unit = {
if (buffer.isEmpty) {
if (isClosed(in)) completeStage()
else pull(in)
} else {
val (chunk, nextBuffer) = buffer.splitAt(chunkSize)
buffer = nextBuffer
push(out, chunk)
}
}
}
}
}
val chunksStream = rawBytes.transform(() => new Chunker(ChunkLimit))
val chunksStream = rawBytes.via(new Chunker(ChunkLimit))
//#bytestring-chunker
val chunksFuture = chunksStream.limit(10).runWith(Sink.seq)
@ -61,17 +77,31 @@ class RecipeByteStrings extends RecipeSpec {
//#bytes-limiter
import akka.stream.stage._
class ByteLimiter(val maximumBytes: Long) extends PushStage[ByteString, ByteString] {
private var count = 0
class ByteLimiter(val maximumBytes: Long) extends GraphStage[FlowShape[ByteString, ByteString]] {
val in = Inlet[ByteString]("ByteLimiter.in")
val out = Outlet[ByteString]("ByteLimiter.out")
override val shape = FlowShape.of(in, out)
override def onPush(chunk: ByteString, ctx: Context[ByteString]): SyncDirective = {
count += chunk.size
if (count > maximumBytes) ctx.fail(new IllegalStateException("Too much bytes"))
else ctx.push(chunk)
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
private var count = 0
setHandlers(in, out, new InHandler with OutHandler {
override def onPull(): Unit = {
pull(in)
}
override def onPush(): Unit = {
val chunk = grab(in)
count += chunk.size
if (count > maximumBytes) failStage(new IllegalStateException("Too much bytes"))
else push(out, chunk)
}
})
}
}
val limiter = Flow[ByteString].transform(() => new ByteLimiter(SizeLimit))
val limiter = Flow[ByteString].via(new ByteLimiter(SizeLimit))
//#bytes-limiter
val bytes1 = Source(List(ByteString(1, 2), ByteString(3), ByteString(4, 5, 6), ByteString(7, 8, 9)))

View file

@ -3,6 +3,7 @@ package docs.stream.cookbook
import java.security.MessageDigest
import akka.NotUsed
import akka.stream.{ Attributes, Outlet, Inlet, FlowShape }
import akka.stream.scaladsl.{ Sink, Source }
import akka.util.ByteString
@ -21,28 +22,36 @@ class RecipeDigest extends RecipeSpec {
//#calculating-digest
import akka.stream.stage._
def digestCalculator(algorithm: String) = new PushPullStage[ByteString, ByteString] {
val digest = MessageDigest.getInstance(algorithm)
class DigestCalculator(algorithm: String) extends GraphStage[FlowShape[ByteString, ByteString]] {
val in = Inlet[ByteString]("DigestCalculator.in")
val out = Outlet[ByteString]("DigestCalculator.out")
override val shape = FlowShape.of(in, out)
override def onPush(chunk: ByteString, ctx: Context[ByteString]): SyncDirective = {
digest.update(chunk.toArray)
ctx.pull()
}
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
val digest = MessageDigest.getInstance(algorithm)
override def onPull(ctx: Context[ByteString]): SyncDirective = {
if (ctx.isFinishing) ctx.pushAndFinish(ByteString(digest.digest()))
else ctx.pull()
}
setHandler(out, new OutHandler {
override def onPull(): Trigger = {
pull(in)
}
})
setHandler(in, new InHandler {
override def onPush(): Trigger = {
val chunk = grab(in)
digest.update(chunk.toArray)
pull(in)
}
override def onUpstreamFinish(): Unit = {
emit(out, ByteString(digest.digest()))
completeStage()
}
})
override def onUpstreamFinish(ctx: Context[ByteString]): TerminationDirective = {
// If the stream is finished, we need to emit the last element in the onPull block.
// It is not allowed to directly emit elements from a termination block
// (onUpstreamFinish or onUpstreamFailure)
ctx.absorbTermination()
}
}
val digest: Source[ByteString, NotUsed] = data.transform(() => digestCalculator("SHA-256"))
val digest: Source[ByteString, NotUsed] = data.via(new DigestCalculator("SHA-256"))
//#calculating-digest
Await.result(digest.runWith(Sink.head), 3.seconds) should be(

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook
import akka.stream.Attributes
import akka.stream.scaladsl.{ Sink, Source }
import akka.stream.testkit._
@ -7,40 +8,68 @@ import scala.concurrent.duration._
object HoldOps {
//#hold-version-1
import akka.stream._
import akka.stream.stage._
class HoldWithInitial[T](initial: T) extends DetachedStage[T, T] {
private var currentValue: T = initial
final class HoldWithInitial[T](initial: T) extends GraphStage[FlowShape[T, T]] {
val in = Inlet[T]("HoldWithInitial.in")
val out = Outlet[T]("HoldWithInitial.out")
override def onPush(elem: T, ctx: DetachedContext[T]): UpstreamDirective = {
currentValue = elem
ctx.pull()
}
override val shape = FlowShape.of(in, out)
override def onPull(ctx: DetachedContext[T]): DownstreamDirective = {
ctx.push(currentValue)
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
private var currentValue: T = initial
setHandlers(in, out, new InHandler with OutHandler {
override def onPush(): Unit = {
currentValue = grab(in)
pull(in)
}
override def onPull(): Unit = {
push(out, currentValue)
}
})
override def preStart(): Unit = {
pull(in)
}
}
}
//#hold-version-1
//#hold-version-2
import akka.stream._
import akka.stream.stage._
class HoldWithWait[T] extends DetachedStage[T, T] {
private var currentValue: T = _
private var waitingFirstValue = true
final class HoldWithWait[T] extends GraphStage[FlowShape[T, T]] {
val in = Inlet[T]("HoldWithWait.in")
val out = Outlet[T]("HoldWithWait.out")
override def onPush(elem: T, ctx: DetachedContext[T]): UpstreamDirective = {
currentValue = elem
waitingFirstValue = false
if (ctx.isHoldingDownstream) ctx.pushAndPull(currentValue)
else ctx.pull()
override val shape = FlowShape.of(in, out)
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
private var currentValue: T = _
private var waitingFirstValue = true
setHandlers(in, out, new InHandler with OutHandler {
override def onPush(): Unit = {
currentValue = grab(in)
if (waitingFirstValue) {
waitingFirstValue = false
if (isAvailable(out)) push(out, currentValue)
}
pull(in)
}
override def onPull(): Unit = {
if (!waitingFirstValue) push(out, currentValue)
}
})
override def preStart(): Unit = {
pull(in)
}
}
override def onPull(ctx: DetachedContext[T]): DownstreamDirective = {
if (waitingFirstValue) ctx.holdDownstream()
else ctx.push(currentValue)
}
}
//#hold-version-2
}
@ -57,7 +86,9 @@ class RecipeHold extends RecipeSpec {
val source = Source.fromPublisher(pub)
val sink = Sink.fromSubscriber(sub)
source.transform(() => new HoldWithInitial(0)).to(sink).run()
source.via(new HoldWithInitial(0)).to(sink)
.withAttributes(Attributes.inputBuffer(1, 1))
.run()
val subscription = sub.expectSubscription()
sub.expectNoMsg(100.millis)
@ -87,7 +118,7 @@ class RecipeHold extends RecipeSpec {
val source = Source.fromPublisher(pub)
val sink = Sink.fromSubscriber(sub)
source.transform(() => new HoldWithWait).to(sink).run()
source.via(new HoldWithWait).to(sink).run()
val subscription = sub.expectSubscription()
sub.expectNoMsg(100.millis)

View file

@ -8,7 +8,6 @@ import java.util.concurrent.atomic.AtomicReference
import akka.stream._
import akka.stream.scaladsl.Tcp._
import akka.stream.scaladsl._
import akka.stream.stage.{ Context, PushStage, SyncDirective }
import akka.stream.testkit.AkkaSpec
import akka.testkit.TestProbe
import akka.util.ByteString
@ -70,46 +69,29 @@ class StreamTcpDocSpec extends AkkaSpec {
import akka.stream.io.Framing
//#welcome-banner-chat-server
connections runForeach { connection =>
connections.runForeach { connection =>
val serverLogic = Flow.fromGraph(GraphDSL.create() { implicit b =>
import GraphDSL.Implicits._
// server logic, parses incoming commands
val commandParser = Flow[String].takeWhile(_ != "BYE").map(_ + "!")
// server logic, parses incoming commands
val commandParser = new PushStage[String, String] {
override def onPush(elem: String, ctx: Context[String]): SyncDirective = {
elem match {
case "BYE" ctx.finish()
case _ ctx.push(elem + "!")
}
}
}
import connection._
val welcomeMsg = s"Welcome to: $localAddress, you are: $remoteAddress!"
val welcome = Source.single(welcomeMsg)
import connection._
val welcomeMsg = s"Welcome to: $localAddress, you are: $remoteAddress!\n"
val welcome = Source.single(ByteString(welcomeMsg))
val echo = b.add(Flow[ByteString]
.via(Framing.delimiter(
ByteString("\n"),
maximumFrameLength = 256,
allowTruncation = true))
.map(_.utf8String)
//#welcome-banner-chat-server
.map { command serverProbe.ref ! command; command }
//#welcome-banner-chat-server
.transform(() commandParser)
.map(_ + "\n")
.map(ByteString(_)))
val concat = b.add(Concat[ByteString]())
// first we emit the welcome message,
welcome ~> concat.in(0)
// then we continue using the echo-logic Flow
echo.outlet ~> concat.in(1)
FlowShape(echo.in, concat.out)
})
val serverLogic = Flow[ByteString]
.via(Framing.delimiter(
ByteString("\n"),
maximumFrameLength = 256,
allowTruncation = true))
.map(_.utf8String)
//#welcome-banner-chat-server
.map { command serverProbe.ref ! command; command }
//#welcome-banner-chat-server
.via(commandParser)
// merge in the initial banner after parser
.merge(welcome)
.map(_ + "\n")
.map(ByteString(_))
connection.handleWith(serverLogic)
}
@ -135,14 +117,10 @@ class StreamTcpDocSpec extends AkkaSpec {
val connection = Tcp().outgoingConnection(localhost)
//#repl-client
val replParser = new PushStage[String, ByteString] {
override def onPush(elem: String, ctx: Context[ByteString]): SyncDirective = {
elem match {
case "q" ctx.pushAndFinish(ByteString("BYE\n"))
case _ ctx.push(ByteString(s"$elem\n"))
}
}
}
val replParser =
Flow[String].takeWhile(_ != "q")
.concat(Source.single("BYE"))
.map(elem => ByteString(s"$elem\n"))
val repl = Flow[ByteString]
.via(Framing.delimiter(
@ -152,7 +130,7 @@ class StreamTcpDocSpec extends AkkaSpec {
.map(_.utf8String)
.map(text => println("Server: " + text))
.map(_ => readLine("> "))
.transform(() replParser)
.via(replParser)
connection.join(repl).run()
}