2016-01-13 16:25:24 +01:00
|
|
|
/**
|
|
|
|
|
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
|
|
|
|
|
*/
|
|
|
|
|
package docs.stream
|
2015-11-02 17:23:39 +01:00
|
|
|
|
2016-01-13 16:25:24 +01:00
|
|
|
import java.io.File
|
2015-11-14 22:42:22 +01:00
|
|
|
|
2016-01-13 16:25:24 +01:00
|
|
|
import _root_.akka.http.scaladsl.model.Uri
|
|
|
|
|
import _root_.akka.stream._
|
|
|
|
|
import _root_.akka.stream.scaladsl._
|
2016-01-14 00:35:47 +01:00
|
|
|
import _root_.akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler }
|
|
|
|
|
import _root_.akka.stream.testkit.{ AkkaSpec, TestPublisher, TestSubscriber }
|
2015-11-02 17:23:39 +01:00
|
|
|
|
|
|
|
|
import scala.concurrent.duration._
|
2016-01-14 00:35:47 +01:00
|
|
|
import scala.concurrent.{ ExecutionContext, Future, Promise }
|
|
|
|
|
import scala.util.{ Failure, Random, Success, Try }
|
2015-11-02 17:23:39 +01:00
|
|
|
|
2015-11-03 14:46:17 +01:00
|
|
|
class MigrationsScala extends AkkaSpec {
|
2015-11-02 17:23:39 +01:00
|
|
|
|
|
|
|
|
"Examples in migration guide" must {
|
|
|
|
|
"compile" in {
|
|
|
|
|
val flow1 = Flow[Int]
|
|
|
|
|
val flow2 = Flow[Int]
|
|
|
|
|
|
|
|
|
|
def inlet: Inlet[Int] = ???
|
|
|
|
|
def outlet: Outlet[Int] = ???
|
|
|
|
|
|
|
|
|
|
def inlet1: Inlet[Int] = ???
|
|
|
|
|
def outlet1: Outlet[Int] = ???
|
|
|
|
|
def inlet2: Inlet[Int] = ???
|
|
|
|
|
def outlet2: Outlet[Int] = ???
|
|
|
|
|
|
|
|
|
|
lazy val dontExecuteMe = {
|
|
|
|
|
//#flow-wrap
|
|
|
|
|
val graphSource: Graph[SourceShape[Int], Unit] = ???
|
|
|
|
|
val source: Source[Int, Unit] = Source.fromGraph(graphSource)
|
|
|
|
|
|
|
|
|
|
val graphSink: Graph[SinkShape[Int], Unit] = ???
|
|
|
|
|
val sink: Sink[Int, Unit] = Sink.fromGraph(graphSink)
|
|
|
|
|
|
|
|
|
|
val graphFlow: Graph[FlowShape[Int, Int], Unit] = ???
|
|
|
|
|
val flow: Flow[Int, Int, Unit] = Flow.fromGraph(graphFlow)
|
|
|
|
|
|
|
|
|
|
Flow.fromSinkAndSource(Sink.head[Int], Source.single(0))
|
|
|
|
|
//#flow-wrap
|
|
|
|
|
|
|
|
|
|
//#bidiflow-wrap
|
|
|
|
|
val bidiGraph: Graph[BidiShape[Int, Int, Int, Int], Unit] = ???
|
|
|
|
|
val bidi: BidiFlow[Int, Int, Int, Int, Unit] = BidiFlow.fromGraph(bidiGraph)
|
|
|
|
|
|
|
|
|
|
BidiFlow.fromFlows(flow1, flow2)
|
|
|
|
|
|
|
|
|
|
BidiFlow.fromFunctions((x: Int) => x + 1, (y: Int) => y * 3)
|
|
|
|
|
//#bidiflow-wrap
|
|
|
|
|
|
|
|
|
|
//#graph-create
|
2015-11-30 15:45:37 +01:00
|
|
|
// Replaces GraphDSL.closed()
|
|
|
|
|
GraphDSL.create() { builder =>
|
2015-11-02 17:23:39 +01:00
|
|
|
//...
|
|
|
|
|
ClosedShape
|
|
|
|
|
}
|
|
|
|
|
|
2015-11-30 15:45:37 +01:00
|
|
|
// Replaces GraphDSL.partial()
|
|
|
|
|
GraphDSL.create() { builder =>
|
2015-11-02 17:23:39 +01:00
|
|
|
//...
|
|
|
|
|
FlowShape(inlet, outlet)
|
|
|
|
|
}
|
|
|
|
|
//#graph-create
|
|
|
|
|
|
|
|
|
|
//#graph-create-2
|
|
|
|
|
Source.fromGraph(
|
2015-11-30 15:45:37 +01:00
|
|
|
GraphDSL.create() { builder =>
|
2015-11-02 17:23:39 +01:00
|
|
|
//...
|
|
|
|
|
SourceShape(outlet)
|
|
|
|
|
})
|
|
|
|
|
|
|
|
|
|
Sink.fromGraph(
|
2015-11-30 15:45:37 +01:00
|
|
|
GraphDSL.create() { builder =>
|
2015-11-02 17:23:39 +01:00
|
|
|
//...
|
|
|
|
|
SinkShape(inlet)
|
|
|
|
|
})
|
|
|
|
|
|
|
|
|
|
Flow.fromGraph(
|
2015-11-30 15:45:37 +01:00
|
|
|
GraphDSL.create() { builder =>
|
2015-11-02 17:23:39 +01:00
|
|
|
//...
|
|
|
|
|
FlowShape(inlet, outlet)
|
|
|
|
|
})
|
|
|
|
|
|
|
|
|
|
BidiFlow.fromGraph(
|
2015-11-30 15:45:37 +01:00
|
|
|
GraphDSL.create() { builder =>
|
2015-11-02 17:23:39 +01:00
|
|
|
//...
|
|
|
|
|
BidiShape(inlet1, outlet1, inlet2, outlet2)
|
|
|
|
|
})
|
|
|
|
|
//#graph-create-2
|
|
|
|
|
|
|
|
|
|
//#graph-edges
|
|
|
|
|
RunnableGraph.fromGraph(
|
2015-11-30 15:45:37 +01:00
|
|
|
GraphDSL.create() { implicit builder =>
|
|
|
|
|
import GraphDSL.Implicits._
|
2015-11-02 17:23:39 +01:00
|
|
|
outlet ~> inlet
|
|
|
|
|
outlet ~> flow ~> inlet
|
|
|
|
|
//...
|
|
|
|
|
ClosedShape
|
|
|
|
|
})
|
|
|
|
|
//#graph-edges
|
|
|
|
|
|
|
|
|
|
val promise = Promise[Unit]()
|
|
|
|
|
|
|
|
|
|
//#source-creators
|
|
|
|
|
val src: Source[Int, Promise[Option[Int]]] = Source.maybe[Int]
|
|
|
|
|
//...
|
|
|
|
|
// This finishes the stream without emitting anything, just like Source.lazyEmpty did
|
|
|
|
|
promise.trySuccess(Some(()))
|
|
|
|
|
|
2015-11-04 11:43:11 +02:00
|
|
|
val ticks = Source.tick(1.second, 3.seconds, "tick")
|
2015-12-17 11:48:30 +02:00
|
|
|
|
|
|
|
|
val pubSource = Source.fromPublisher(TestPublisher.manualProbe[Int]())
|
|
|
|
|
|
|
|
|
|
val itSource = Source.fromIterator(() => Iterator.continually(Random.nextGaussian))
|
|
|
|
|
|
|
|
|
|
val futSource = Source.fromFuture(Future.successful(42))
|
|
|
|
|
|
|
|
|
|
val subSource = Source.asSubscriber
|
2015-11-02 17:23:39 +01:00
|
|
|
//#source-creators
|
|
|
|
|
|
2015-12-17 11:48:30 +02:00
|
|
|
//#sink-creators
|
|
|
|
|
val subSink = Sink.fromSubscriber(TestSubscriber.manualProbe[Int]())
|
|
|
|
|
//#sink-creators
|
|
|
|
|
|
|
|
|
|
//#sink-as-publisher
|
|
|
|
|
val pubSink = Sink.asPublisher(fanout = false)
|
|
|
|
|
|
|
|
|
|
val pubSinkFanout = Sink.asPublisher(fanout = true)
|
|
|
|
|
//#sink-as-publisher
|
|
|
|
|
|
2015-11-03 14:46:17 +01:00
|
|
|
//#flatMapConcat
|
|
|
|
|
Flow[Source[Int, Any]].flatMapConcat(identity)
|
|
|
|
|
//#flatMapConcat
|
2015-11-02 17:23:39 +01:00
|
|
|
|
2015-11-25 19:58:48 +01:00
|
|
|
//#group-flatten
|
|
|
|
|
Flow[Int]
|
|
|
|
|
.groupBy(2, _ % 2) // the first parameter sets max number of substreams
|
|
|
|
|
.map(_ + 3)
|
|
|
|
|
.concatSubstreams
|
|
|
|
|
//#group-flatten
|
|
|
|
|
|
|
|
|
|
val MaxDistinctWords = 1000
|
|
|
|
|
//#group-fold
|
|
|
|
|
Flow[String]
|
|
|
|
|
.groupBy(MaxDistinctWords, identity)
|
|
|
|
|
.fold(("", 0))((pair, word) => (word, pair._2 + 1))
|
|
|
|
|
.mergeSubstreams
|
|
|
|
|
//#group-fold
|
|
|
|
|
|
2015-11-02 17:23:39 +01:00
|
|
|
//#port-async
|
|
|
|
|
class MapAsyncOne[In, Out](f: In ⇒ Future[Out])(implicit ec: ExecutionContext)
|
|
|
|
|
extends GraphStage[FlowShape[In, Out]] {
|
|
|
|
|
val in: Inlet[In] = Inlet("MapAsyncOne.in")
|
|
|
|
|
val out: Outlet[Out] = Outlet("MapAsyncOne.out")
|
|
|
|
|
override val shape: FlowShape[In, Out] = FlowShape(in, out)
|
|
|
|
|
|
|
|
|
|
// The actual logic is encapsulated in a GraphStageLogic now
|
|
|
|
|
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
|
|
|
|
|
new GraphStageLogic(shape) {
|
|
|
|
|
|
|
|
|
|
// All of the state *must* be encapsulated in the GraphStageLogic,
|
|
|
|
|
// not in the GraphStage
|
|
|
|
|
private var elemInFlight: Out = _
|
|
|
|
|
|
|
|
|
|
val callback = getAsyncCallback(onAsyncInput)
|
|
|
|
|
var holdingUpstream = false
|
|
|
|
|
|
|
|
|
|
// All upstream related events now are handled in an InHandler instance
|
|
|
|
|
setHandler(in, new InHandler {
|
|
|
|
|
// No context or element parameter for onPush
|
|
|
|
|
override def onPush(): Unit = {
|
|
|
|
|
// The element is not passed as an argument but needs to be dequeued explicitly
|
|
|
|
|
val elem = grab(in)
|
|
|
|
|
val future = f(elem)
|
|
|
|
|
future.onComplete(callback.invoke)
|
|
|
|
|
// ctx.holdUpstream is no longer needed, but we need to track the state
|
|
|
|
|
holdingUpstream = true
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// No context parameter
|
|
|
|
|
override def onUpstreamFinish(): Unit = {
|
|
|
|
|
if (holdingUpstream) absorbTermination()
|
|
|
|
|
else completeStage() // ctx.finish turns into completeStage()
|
|
|
|
|
}
|
|
|
|
|
})
|
|
|
|
|
|
|
|
|
|
setHandler(out, new OutHandler {
|
|
|
|
|
override def onPull(): Unit = {
|
|
|
|
|
if (elemInFlight != null) {
|
|
|
|
|
val e = elemInFlight
|
|
|
|
|
elemInFlight = null.asInstanceOf[Out]
|
|
|
|
|
pushIt(e)
|
|
|
|
|
} // holdDownstream is no longer needed
|
|
|
|
|
}
|
|
|
|
|
})
|
|
|
|
|
|
|
|
|
|
// absorbTermination turns into the code below.
|
|
|
|
|
// This emulates the behavior of the AsyncStage stage.
|
|
|
|
|
private def absorbTermination(): Unit =
|
2015-12-14 14:52:06 +01:00
|
|
|
if (isAvailable(shape.out)) getHandler(out).onPull()
|
2015-11-02 17:23:39 +01:00
|
|
|
|
|
|
|
|
// The line below emulates the behavior of the AsyncStage holdingDownstream
|
|
|
|
|
private def holdingDownstream(): Boolean =
|
|
|
|
|
!(isClosed(in) || hasBeenPulled(in))
|
|
|
|
|
|
|
|
|
|
// Any method can be used as a callback, we chose the previous name for
|
|
|
|
|
// easier comparison with the original code
|
|
|
|
|
private def onAsyncInput(input: Try[Out]) =
|
|
|
|
|
input match {
|
|
|
|
|
case Failure(ex) ⇒ failStage(ex)
|
|
|
|
|
case Success(e) if holdingDownstream() ⇒ pushIt(e)
|
|
|
|
|
case Success(e) ⇒
|
|
|
|
|
elemInFlight = e
|
|
|
|
|
// ctx.ignore is no longer needed
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def pushIt(elem: Out): Unit = {
|
|
|
|
|
// ctx.isFinishing turns into isClosed(in)
|
|
|
|
|
if (isClosed(in)) {
|
|
|
|
|
// pushAndFinish is now two actions
|
|
|
|
|
push(out, elem)
|
|
|
|
|
completeStage()
|
|
|
|
|
} else {
|
|
|
|
|
// pushAndPull is now two actions
|
|
|
|
|
push(out, elem)
|
|
|
|
|
pull(in)
|
|
|
|
|
holdingUpstream = false
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
//#port-async
|
2015-11-04 11:03:21 +01:00
|
|
|
|
|
|
|
|
val uri: Uri = ???
|
|
|
|
|
//#raw-query
|
|
|
|
|
val queryPart: Option[String] = uri.rawQueryString
|
|
|
|
|
//#raw-query
|
|
|
|
|
|
|
|
|
|
//#query-param
|
|
|
|
|
val param: Option[String] = uri.query().get("a")
|
|
|
|
|
//#query-param
|
2015-11-14 22:42:22 +01:00
|
|
|
|
|
|
|
|
//#file-source-sink
|
2015-12-08 18:47:58 +01:00
|
|
|
val fileSrc = FileIO.fromFile(new File("."))
|
2015-11-14 22:42:22 +01:00
|
|
|
|
2015-12-08 18:47:58 +01:00
|
|
|
val otherFileSrc = FileIO.fromFile(new File("."), 1024)
|
2015-11-14 22:42:22 +01:00
|
|
|
|
2015-12-08 18:47:58 +01:00
|
|
|
val someFileSink = FileIO.toFile(new File("."))
|
2015-11-14 22:42:22 +01:00
|
|
|
//#file-source-sink
|
2015-11-17 13:17:30 +01:00
|
|
|
|
|
|
|
|
class SomeInputStream extends java.io.InputStream { override def read(): Int = 0 }
|
|
|
|
|
class SomeOutputStream extends java.io.OutputStream { override def write(b: Int): Unit = () }
|
|
|
|
|
|
|
|
|
|
//#input-output-stream-source-sink
|
2015-12-08 18:47:58 +01:00
|
|
|
val inputStreamSrc = StreamConverters.fromInputStream(() => new SomeInputStream())
|
2015-11-17 13:17:30 +01:00
|
|
|
|
2015-12-08 18:47:58 +01:00
|
|
|
val otherInputStreamSrc = StreamConverters.fromInputStream(() => new SomeInputStream())
|
2015-11-17 13:17:30 +01:00
|
|
|
|
2015-12-08 18:47:58 +01:00
|
|
|
val someOutputStreamSink = StreamConverters.fromOutputStream(() => new SomeOutputStream())
|
2015-11-17 13:17:30 +01:00
|
|
|
//#input-output-stream-source-sink
|
|
|
|
|
|
|
|
|
|
//#output-input-stream-source-sink
|
|
|
|
|
val timeout: FiniteDuration = 0.seconds
|
|
|
|
|
|
2015-12-08 18:47:58 +01:00
|
|
|
val outputStreamSrc = StreamConverters.asOutputStream()
|
2015-11-17 13:17:30 +01:00
|
|
|
|
2015-12-08 18:47:58 +01:00
|
|
|
val otherOutputStreamSrc = StreamConverters.asOutputStream(timeout)
|
2015-11-17 13:17:30 +01:00
|
|
|
|
2015-12-08 18:47:58 +01:00
|
|
|
val someInputStreamSink = StreamConverters.asInputStream()
|
2015-11-17 13:17:30 +01:00
|
|
|
|
2015-12-08 18:47:58 +01:00
|
|
|
val someOtherInputStreamSink = StreamConverters.asInputStream(timeout)
|
2015-11-17 13:17:30 +01:00
|
|
|
//#output-input-stream-source-sink
|
2015-11-02 17:23:39 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|