!str #16902: Unify stream internal representation

also =str #16912: Fix StreamTcpSpec flakiness
This commit is contained in:
Endre Sándor Varga 2015-01-28 14:19:50 +01:00
parent cac9c9f2fb
commit 8d77fa8b29
230 changed files with 7814 additions and 9596 deletions

View file

@ -21,9 +21,10 @@ class HttpServerExampleSpec
implicit val system = ActorSystem() implicit val system = ActorSystem()
implicit val materializer = ActorFlowMaterializer() implicit val materializer = ActorFlowMaterializer()
val serverBinding = Http(system).bind(interface = "localhost", port = 8080) val serverSource = Http(system).bind(interface = "localhost", port = 8080)
serverBinding.connections.runForeach { connection => // foreach materializes the source serverSource.runForeach { connection => // foreach materializes the source
println("Accepted new connection from " + connection.remoteAddress) println("Accepted new connection from " + connection.remoteAddress)
// ... and then actually handle the connection
} }
//#bind-example //#bind-example
} }
@ -35,11 +36,11 @@ class HttpServerExampleSpec
implicit val system = ActorSystem() implicit val system = ActorSystem()
implicit val materializer = ActorFlowMaterializer() implicit val materializer = ActorFlowMaterializer()
val serverBinding = Http(system).bind(interface = "localhost", port = 8080) val serverSource = Http(system).bind(interface = "localhost", port = 8080)
//#full-server-example //#full-server-example
import akka.http.model.HttpMethods._ import akka.http.model.HttpMethods._
import akka.stream.scaladsl.Flow import akka.stream.scaladsl.{ Flow, Sink }
val requestHandler: HttpRequest => HttpResponse = { val requestHandler: HttpRequest => HttpResponse = {
case HttpRequest(GET, Uri.Path("/"), _, _, _) => case HttpRequest(GET, Uri.Path("/"), _, _, _) =>
@ -52,13 +53,13 @@ class HttpServerExampleSpec
case _: HttpRequest => HttpResponse(404, entity = "Unknown resource!") case _: HttpRequest => HttpResponse(404, entity = "Unknown resource!")
} }
serverBinding.connections runForeach { connection => val bindingFuture = serverSource.to(Sink.foreach { connection =>
println("Accepted new connection from " + connection.remoteAddress) println("Accepted new connection from " + connection.remoteAddress)
connection handleWithSyncHandler requestHandler connection handleWithSyncHandler requestHandler
// this is equivalent to // this is equivalent to
// connection handleWith { Flow[HttpRequest] map requestHandler } // connection handleWith { Flow[HttpRequest] map requestHandler }
} }).run()
//#full-server-example //#full-server-example
} }
} }

View file

@ -7,8 +7,7 @@ import scala.annotation.tailrec
import akka.actor.Props import akka.actor.Props
import akka.stream.ActorFlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.actor.ActorPublisher import akka.stream.actor.ActorPublisher
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.{ Flow, Sink, Source }
import akka.stream.scaladsl.Source
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
object ActorPublisherDocSpec { object ActorPublisherDocSpec {
@ -78,13 +77,12 @@ class ActorPublisherDocSpec extends AkkaSpec {
//#actor-publisher-usage //#actor-publisher-usage
val jobManagerSource = Source[JobManager.Job](JobManager.props) val jobManagerSource = Source[JobManager.Job](JobManager.props)
val materializedMap = jobManagerSource val ref = Flow[JobManager.Job]
.map(_.payload.toUpperCase) .map(_.payload.toUpperCase)
.map { elem => println(elem); elem } .map { elem => println(elem); elem }
.to(Sink.ignore) .to(Sink.ignore)
.run() .runWith(jobManagerSource)
val ref = materializedMap.get(jobManagerSource)
ref ! JobManager.Job("a") ref ! JobManager.Job("a")
ref ! JobManager.Job("b") ref ! JobManager.Job("b")
ref ! JobManager.Job("c") ref ! JobManager.Job("c")

View file

@ -3,37 +3,43 @@
*/ */
package docs.stream package docs.stream
import akka.stream.ActorFlowMaterializer import akka.stream._
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import scala.collection.immutable.IndexedSeq
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
object FlexiDocSpec {
//#fleximerge-zip-states
//#fleximerge-zip-readall
import akka.stream.FanInShape._
class ZipPorts[A, B](_init: Init[(A, B)] = Name("Zip"))
extends FanInShape[(A, B)](_init) {
val left = newInlet[A]("left")
val right = newInlet[B]("right")
protected override def construct(i: Init[(A, B)]) = new ZipPorts(i)
}
//#fleximerge-zip-readall
//#fleximerge-zip-states
}
class FlexiDocSpec extends AkkaSpec { class FlexiDocSpec extends AkkaSpec {
import FlexiDocSpec._
implicit val ec = system.dispatcher implicit val ec = system.dispatcher
implicit val mat = ActorFlowMaterializer() implicit val mat = ActorFlowMaterializer()
"implement zip using readall" in { "implement zip using readall" in {
//#fleximerge-zip-readall //#fleximerge-zip-readall
class Zip[A, B] extends FlexiMerge[(A, B)] { class Zip[A, B] extends FlexiMerge[(A, B), ZipPorts[A, B]](
new ZipPorts, OperationAttributes.name("Zip1State")) {
import FlexiMerge._ import FlexiMerge._
val left = createInputPort[A]() override def createMergeLogic(p: PortT) = new MergeLogic[(A, B)] {
val right = createInputPort[B]() override def initialState =
State(ReadAll(p.left, p.right)) { (ctx, _, inputs) =>
def createMergeLogic = new MergeLogic[(A, B)] { val a = inputs(p.left)
override def inputHandles(inputCount: Int) = { val b = inputs(p.right)
require(inputCount == 2, s"Zip must have two connected inputs, was $inputCount")
Vector(left, right)
}
override def initialState: State[_] =
State[ReadAllInputs](ReadAll(left, right)) { (ctx, _, inputs) =>
val a: A = inputs(left)
val b: B = inputs(right)
ctx.emit((a, b)) ctx.emit((a, b))
SameState SameState
} }
@ -44,49 +50,40 @@ class FlexiDocSpec extends AkkaSpec {
//#fleximerge-zip-readall //#fleximerge-zip-readall
//format: OFF //format: OFF
val res =
//#fleximerge-zip-connecting //#fleximerge-zip-connecting
val head = Sink.head[(Int, String)] FlowGraph.closed(Sink.head[(Int, String)]) { implicit b =>
//#fleximerge-zip-connecting o =>
import FlowGraph.Implicits._
val map = val zip = b.add(new Zip[Int, String])
//#fleximerge-zip-connecting
FlowGraph { implicit b =>
import FlowGraphImplicits._
val zip = Zip[Int, String]
Source.single(1) ~> zip.left Source.single(1) ~> zip.left
Source.single("A") ~> zip.right Source.single("1") ~> zip.right
zip.out ~> head zip.out ~> o.inlet
} }
//#fleximerge-zip-connecting //#fleximerge-zip-connecting
.run() .run()
//format: ON //format: ON
Await.result(map.get(head), remaining) should equal((1, "A")) Await.result(res, 300.millis) should equal((1, "1"))
} }
"implement zip using two states" in { "implement zip using two states" in {
//#fleximerge-zip-states //#fleximerge-zip-states
class Zip[A, B] extends FlexiMerge[(A, B)] { class Zip[A, B] extends FlexiMerge[(A, B), ZipPorts[A, B]](
new ZipPorts, OperationAttributes.name("Zip2State")) {
import FlexiMerge._ import FlexiMerge._
val left = createInputPort[A]()
val right = createInputPort[B]()
def createMergeLogic = new MergeLogic[(A, B)] { override def createMergeLogic(p: PortT) = new MergeLogic[(A, B)] {
var lastInA: A = _ var lastInA: A = _
override def inputHandles(inputCount: Int) = { val readA: State[A] = State[A](Read(p.left)) { (ctx, input, element) =>
require(inputCount == 2, s"Zip must have two connected inputs, was $inputCount")
Vector(left, right)
}
val readA: State[A] = State[A](Read(left)) { (ctx, input, element) =>
lastInA = element lastInA = element
readB readB
} }
val readB: State[B] = State[B](Read(right)) { (ctx, input, element) => val readB: State[B] = State[B](Read(p.right)) { (ctx, input, element) =>
ctx.emit((lastInA, element)) ctx.emit((lastInA, element))
readA readA
} }
@ -98,37 +95,37 @@ class FlexiDocSpec extends AkkaSpec {
} }
//#fleximerge-zip-states //#fleximerge-zip-states
val head = Sink.head[(Int, String)] val res = FlowGraph.closed(Sink.head[(Int, String)]) { implicit b =>
val map = FlowGraph { implicit b => o =>
import akka.stream.scaladsl.FlowGraphImplicits._ import FlowGraph.Implicits._
val zip = new Zip[Int, String] val zip = b.add(new Zip[Int, String])
Source(1 to 2) ~> zip.left Source(1 to 2) ~> zip.left
Source(List("A", "B")) ~> zip.right Source((1 to 2).map(_.toString)) ~> zip.right
zip.out ~> head zip.out ~> o.inlet
}.run() }.run()
Await.result(map.get(head), remaining) should equal((1, "A")) Await.result(res, 300.millis) should equal((1, "1"))
} }
"fleximerge completion handling" in { "fleximerge completion handling" in {
import FanInShape._
//#fleximerge-completion //#fleximerge-completion
class ImportantWithBackups[A] extends FlexiMerge[A] { class ImportantWithBackupShape[A](_init: Init[A] = Name("Zip"))
extends FanInShape[A](_init) {
val important = newInlet[A]("important")
val replica1 = newInlet[A]("replica1")
val replica2 = newInlet[A]("replica2")
protected override def construct(i: Init[A]) =
new ImportantWithBackupShape(i)
}
class ImportantWithBackups[A] extends FlexiMerge[A, ImportantWithBackupShape[A]](
new ImportantWithBackupShape, OperationAttributes.name("ImportantWithBackups")) {
import FlexiMerge._ import FlexiMerge._
val important = createInputPort[A]() override def createMergeLogic(p: PortT) = new MergeLogic[A] {
val replica1 = createInputPort[A]() import p.important
val replica2 = createInputPort[A]()
def createMergeLogic = new MergeLogic[A] {
val inputs = Vector(important, replica1, replica2)
override def inputHandles(inputCount: Int) = {
require(inputCount == 3, s"Must connect 3 inputs, connected only $inputCount")
inputs
}
override def initialCompletionHandling = override def initialCompletionHandling =
CompletionHandling( CompletionHandling(
onUpstreamFinish = (ctx, input) => input match { onUpstreamFinish = (ctx, input) => input match {
@ -159,7 +156,8 @@ class FlexiDocSpec extends AkkaSpec {
SameState SameState
}) })
override def initialState = State[A](ReadAny(inputs)) { override def initialState =
State[A](ReadAny(p.important, p.replica1, p.replica2)) {
(ctx, input, element) => (ctx, input, element) =>
ctx.emit(element) ctx.emit(element)
SameState SameState
@ -168,9 +166,9 @@ class FlexiDocSpec extends AkkaSpec {
} }
//#fleximerge-completion //#fleximerge-completion
FlowGraph { implicit b => FlowGraph.closed() { implicit b =>
import FlowGraphImplicits._ import FlowGraph.Implicits._
val importantWithBackups = new ImportantWithBackups[Int] val importantWithBackups = b.add(new ImportantWithBackups[Int])
Source.single(1) ~> importantWithBackups.important Source.single(1) ~> importantWithBackups.important
Source.single(2) ~> importantWithBackups.replica1 Source.single(2) ~> importantWithBackups.replica1
Source.failed[Int](new Exception("Boom!") with NoStackTrace) ~> importantWithBackups.replica2 Source.failed[Int](new Exception("Boom!") with NoStackTrace) ~> importantWithBackups.replica2
@ -179,22 +177,22 @@ class FlexiDocSpec extends AkkaSpec {
} }
"flexi preferring merge" in { "flexi preferring merge" in {
import FanInShape._
//#flexi-preferring-merge //#flexi-preferring-merge
class PreferringMerge extends FlexiMerge[Int] { class PreferringMergeShape[A](_init: Init[A] = Name("PreferringMerge"))
extends FanInShape[A](_init) {
val preferred = newInlet[A]("preferred")
val secondary1 = newInlet[A]("secondary1")
val secondary2 = newInlet[A]("secondary2")
protected override def construct(i: Init[A]) = new PreferringMergeShape(i)
}
class PreferringMerge extends FlexiMerge[Int, PreferringMergeShape[Int]](
new PreferringMergeShape, OperationAttributes.name("ImportantWithBackups")) {
import akka.stream.scaladsl.FlexiMerge._ import akka.stream.scaladsl.FlexiMerge._
val preferred = createInputPort[Int]() override def createMergeLogic(p: PortT) = new MergeLogic[Int] {
val secondary1 = createInputPort[Int]()
val secondary2 = createInputPort[Int]()
def createMergeLogic = new MergeLogic[Int] {
override def inputHandles(inputCount: Int) = {
require(inputCount == 3, s"PreferringMerge must have 3 connected inputs, was $inputCount")
Vector(preferred, secondary1, secondary2)
}
override def initialState = override def initialState =
State[Int](ReadPreferred(preferred)(secondary1, secondary2)) { State[Int](ReadPreferred(p.preferred, p.secondary1, p.secondary2)) {
(ctx, input, element) => (ctx, input, element) =>
ctx.emit(element) ctx.emit(element)
SameState SameState
@ -204,59 +202,26 @@ class FlexiDocSpec extends AkkaSpec {
//#flexi-preferring-merge //#flexi-preferring-merge
} }
"flexi read conditions" in {
class X extends FlexiMerge[Int] {
import FlexiMerge._
override def createMergeLogic(): MergeLogic[Int] = new MergeLogic[Int] {
//#read-conditions
val first = createInputPort[Int]()
val second = createInputPort[Int]()
val third = createInputPort[Int]()
//#read-conditions
//#read-conditions
val onlyFirst = Read(first)
val firstOrThird = ReadAny(first, third)
val firstAndSecond = ReadAll(first, second)
val firstAndThird = ReadAll(first, third)
val mostlyFirst = ReadPreferred(first)(second, third)
//#read-conditions
override def inputHandles(inputCount: Int): IndexedSeq[InputHandle] = Vector()
override def initialState: State[_] = State[ReadAllInputs](firstAndSecond) {
(ctx, input, inputs) =>
val in1: Int = inputs(first)
SameState
}
}
}
}
"flexi route" in { "flexi route" in {
//#flexiroute-unzip //#flexiroute-unzip
class Unzip[A, B] extends FlexiRoute[(A, B)] { import FanOutShape._
import FlexiRoute._ class UnzipShape[A, B](_init: Init[(A, B)] = Name[(A, B)]("Unzip"))
val outA = createOutputPort[A]() extends FanOutShape[(A, B)](_init) {
val outB = createOutputPort[B]() val outA = newOutlet[A]("outA")
val outB = newOutlet[B]("outB")
override def createRouteLogic() = new RouteLogic[(A, B)] { protected override def construct(i: Init[(A, B)]) = new UnzipShape(i)
override def outputHandles(outputCount: Int) = {
require(outputCount == 2, s"Unzip must have two connected outputs, was $outputCount")
Vector(outA, outB)
} }
class Unzip[A, B] extends FlexiRoute[(A, B), UnzipShape[A, B]](
new UnzipShape, OperationAttributes.name("Unzip")) {
import FlexiRoute._
override def initialState = State[Any](DemandFromAll(outA, outB)) { override def createRouteLogic(p: PortT) = new RouteLogic[(A, B)] {
override def initialState =
State[Any](DemandFromAll(p.outA, p.outB)) {
(ctx, _, element) => (ctx, _, element) =>
val (a, b) = element val (a, b) = element
ctx.emit(outA, a) ctx.emit(p.outA)(a)
ctx.emit(outB, b) ctx.emit(p.outB)(b)
SameState SameState
} }
@ -267,20 +232,20 @@ class FlexiDocSpec extends AkkaSpec {
} }
"flexi route completion handling" in { "flexi route completion handling" in {
import FanOutShape._
//#flexiroute-completion //#flexiroute-completion
class ImportantRoute[A] extends FlexiRoute[A] { class ImportantRouteShape[A](_init: Init[A] = Name[A]("ImportantRoute")) extends FanOutShape[A](_init) {
import FlexiRoute._ val important = newOutlet[A]("important")
val important = createOutputPort[A]() val additional1 = newOutlet[A]("additional1")
val additional1 = createOutputPort[A]() val additional2 = newOutlet[A]("additional2")
val additional2 = createOutputPort[A]() protected override def construct(i: Init[A]) = new ImportantRouteShape(i)
override def createRouteLogic() = new RouteLogic[A] {
val outputs = Vector(important, additional1, additional2)
override def outputHandles(outputCount: Int) = {
require(outputCount == 3, s"Must have three connected outputs, was $outputCount")
outputs
} }
class ImportantRoute[A] extends FlexiRoute[A, ImportantRouteShape[A]](
new ImportantRouteShape, OperationAttributes.name("ImportantRoute")) {
import FlexiRoute._
override def createRouteLogic(p: PortT) = new RouteLogic[A] {
import p.important
private val select = (p.important | p.additional1 | p.additional2)
override def initialCompletionHandling = override def initialCompletionHandling =
CompletionHandling( CompletionHandling(
@ -297,18 +262,19 @@ class FlexiDocSpec extends AkkaSpec {
SameState SameState
}) })
override def initialState = State[A](DemandFromAny(outputs)) { override def initialState =
State(DemandFromAny(p.important, p.additional1, p.additional2)) {
(ctx, output, element) => (ctx, output, element) =>
ctx.emit(output, element) ctx.emit(select(output))(element)
SameState SameState
} }
} }
} }
//#flexiroute-completion //#flexiroute-completion
FlowGraph { implicit b => FlowGraph.closed() { implicit b =>
import FlowGraphImplicits._ import FlowGraph.Implicits._
val route = new ImportantRoute[Int] val route = b.add(new ImportantRoute[Int])
Source.single(1) ~> route.in Source.single(1) ~> route.in
route.important ~> Sink.ignore route.important ~> Sink.ignore
route.additional1 ~> Sink.ignore route.additional1 ~> Sink.ignore

View file

@ -36,13 +36,10 @@ class FlowDocSpec extends AkkaSpec {
val sink = Sink.fold[Int, Int](0)(_ + _) val sink = Sink.fold[Int, Int](0)(_ + _)
// connect the Source to the Sink, obtaining a RunnableFlow // connect the Source to the Sink, obtaining a RunnableFlow
val runnable: RunnableFlow = source.to(sink) val runnable: RunnableFlow[Future[Int]] = source.toMat(sink)(Keep.right)
// materialize the flow // materialize the flow and get the value of the FoldSink
val materialized: MaterializedMap = runnable.run() val sum: Future[Int] = runnable.run()
// get the materialized value of the FoldSink
val sum: Future[Int] = materialized.get(sink)
//#materialization-in-steps //#materialization-in-steps
} }
@ -61,17 +58,20 @@ class FlowDocSpec extends AkkaSpec {
//#stream-reuse //#stream-reuse
// connect the Source to the Sink, obtaining a RunnableFlow // connect the Source to the Sink, obtaining a RunnableFlow
val sink = Sink.fold[Int, Int](0)(_ + _) val sink = Sink.fold[Int, Int](0)(_ + _)
val runnable: RunnableFlow = Source(1 to 10).to(sink) val runnable: RunnableFlow[Future[Int]] =
Source(1 to 10).toMat(sink)(Keep.right)
// get the materialized value of the FoldSink // get the materialized value of the FoldSink
val sum1: Future[Int] = runnable.run().get(sink) val sum1: Future[Int] = runnable.run()
val sum2: Future[Int] = runnable.run().get(sink) val sum2: Future[Int] = runnable.run()
// sum1 and sum2 are different Futures! // sum1 and sum2 are different Futures!
//#stream-reuse //#stream-reuse
} }
"compound source cannot be used as key" in { "compound source cannot be used as key" in {
// FIXME #16902 This example is now turned around
// The WRONG case has been switched
//#compound-source-is-not-keyed-runWith //#compound-source-is-not-keyed-runWith
import scala.concurrent.duration._ import scala.concurrent.duration._
case object Tick case object Tick
@ -82,14 +82,14 @@ class FlowDocSpec extends AkkaSpec {
timerCancel.cancel() timerCancel.cancel()
val timerMap = timer.map(tick => "tick") val timerMap = timer.map(tick => "tick")
val _ = Sink.ignore.runWith(timerMap) // WRONG: returned type is not the timers Cancellable! // materialize the flow and retrieve the timers Cancellable
val timerCancellable = Sink.ignore.runWith(timerMap)
timerCancellable.cancel()
//#compound-source-is-not-keyed-runWith //#compound-source-is-not-keyed-runWith
//#compound-source-is-not-keyed-run //#compound-source-is-not-keyed-run
// retain the materialized map, in order to retrieve the timer's Cancellable val timerCancellable2 = timerMap.to(Sink.ignore).run()
val materialized = timerMap.to(Sink.ignore).run() timerCancellable2.cancel()
val timerCancellable = materialized.get(timer)
timerCancellable.cancel()
//#compound-source-is-not-keyed-run //#compound-source-is-not-keyed-run
} }
@ -133,7 +133,7 @@ class FlowDocSpec extends AkkaSpec {
source.to(Sink.foreach(println(_))) source.to(Sink.foreach(println(_)))
// Starting from a Sink // Starting from a Sink
val sink: Sink[Int] = Flow[Int].map(_ * 2).to(Sink.foreach(println(_))) val sink: Sink[Int, Unit] = Flow[Int].map(_ * 2).to(Sink.foreach(println(_)))
Source(1 to 6).to(sink) Source(1 to 6).to(sink)
//#flow-connecting //#flow-connecting

View file

@ -75,7 +75,7 @@ class FlowErrorDocSpec extends AkkaSpec {
else acc + elem else acc + elem
} }
} }
val result = source.grouped(1000).runWith(Sink.head) val result = source.grouped(1000).runWith(Sink.head())
// the negative element cause the scan stage to be restarted, // the negative element cause the scan stage to be restarted,
// i.e. start from 0 again // i.e. start from 0 again
// result here will be a Future completed with Success(Vector(0, 1, 0, 5, 12)) // result here will be a Future completed with Success(Vector(0, 1, 0, 5, 12))

View file

@ -7,8 +7,6 @@ import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Broadcast import akka.stream.scaladsl.Broadcast
import akka.stream.scaladsl.Flow import akka.stream.scaladsl.Flow
import akka.stream.scaladsl.FlowGraph import akka.stream.scaladsl.FlowGraph
import akka.stream.scaladsl.FlowGraphImplicits
import akka.stream.scaladsl.MaterializedMap
import akka.stream.scaladsl.Merge import akka.stream.scaladsl.Merge
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
@ -27,44 +25,46 @@ class FlowGraphDocSpec extends AkkaSpec {
"build simple graph" in { "build simple graph" in {
//format: OFF //format: OFF
//#simple-flow-graph //#simple-flow-graph
val g = FlowGraph { implicit b => val g = FlowGraph.closed() { implicit b =>
import FlowGraphImplicits._ import FlowGraph.Implicits._
val in = Source(1 to 10) val in = Source(1 to 10)
val out = Sink.ignore val out = Sink.ignore
val bcast = Broadcast[Int] val bcast = b.add(Broadcast[Int](2))
val merge = Merge[Int] val merge = b.add(Merge[Int](2))
val f1, f2, f3, f4 = Flow[Int].map(_ + 10) val f1, f2, f3, f4 = Flow[Int].map(_ + 10)
in ~> f1 ~> bcast ~> f2 ~> merge ~> f3 ~> out in ~> f1 ~> bcast.in
bcast ~> f4 ~> merge bcast.out(0) ~> f2 ~> merge.in(0)
bcast.out(1) ~> f4 ~> merge.in(1)
merge.out ~> f3 ~> out
} }
//#simple-flow-graph //#simple-flow-graph
//format: ON //format: ON
//#simple-graph-run //#simple-graph-run
val map: MaterializedMap = g.run() g.run()
//#simple-graph-run //#simple-graph-run
} }
"build simple graph without implicits" in { "build simple graph without implicits" in {
//#simple-flow-graph-no-implicits //#simple-flow-graph-no-implicits
val g = FlowGraph { b => val g = FlowGraph.closed() { b =>
val in = Source(1 to 10) val in = Source(1 to 10)
val out = Sink.ignore val out = Sink.ignore
val broadcast = Broadcast[Int] val broadcast = b.add(Broadcast[Int](2))
val merge = Merge[Int] val merge = b.add(Merge[Int](2))
val f1 = Flow[Int].map(_ + 10) val f1 = Flow[Int].map(_ + 10)
val f3 = Flow[Int].map(_.toString) val f3 = Flow[Int].map(_.toString)
val f2 = Flow[Int].map(_ + 20) val f2 = Flow[Int].map(_ + 20)
b.addEdge(in, broadcast) b.addEdge(b.add(in), broadcast.in)
.addEdge(broadcast, f1, merge) b.addEdge(broadcast.out(0), f1, merge.in(0))
.addEdge(broadcast, f2, merge) b.addEdge(broadcast.out(1), f2, merge.in(1))
.addEdge(merge, f3, out) b.addEdge(merge.out, f3, b.add(out))
} }
//#simple-flow-graph-no-implicits //#simple-flow-graph-no-implicits
@ -74,19 +74,19 @@ class FlowGraphDocSpec extends AkkaSpec {
"flow connection errors" in { "flow connection errors" in {
intercept[IllegalArgumentException] { intercept[IllegalArgumentException] {
//#simple-graph //#simple-graph
FlowGraph { implicit b => FlowGraph.closed() { implicit b =>
import FlowGraphImplicits._ import FlowGraph.Implicits._
val source1 = Source(1 to 10) val source1 = Source(1 to 10)
val source2 = Source(1 to 10) val source2 = Source(1 to 10)
val zip = Zip[Int, Int] val zip = b.add(Zip[Int, Int]())
source1 ~> zip.left source1 ~> zip.in0
source2 ~> zip.right source2 ~> zip.in1
// unconnected zip.out (!) => "must have at least 1 outgoing edge" // unconnected zip.out (!) => "must have at least 1 outgoing edge"
} }
//#simple-graph //#simple-graph
}.getMessage should include("must have at least 1 outgoing edge") }.getMessage should include("unconnected ports: Zip.out")
} }
"reusing a flow in a graph" in { "reusing a flow in a graph" in {
@ -101,19 +101,20 @@ class FlowGraphDocSpec extends AkkaSpec {
// format: OFF // format: OFF
val g = val g =
//#flow-graph-reusing-a-flow //#flow-graph-reusing-a-flow
FlowGraph { implicit b => FlowGraph.closed(topHeadSink, bottomHeadSink)((_, _)) { implicit b =>
import FlowGraphImplicits._ (topHS, bottomHS) =>
val broadcast = Broadcast[Int] import FlowGraph.Implicits._
Source.single(1) ~> broadcast val broadcast = b.add(Broadcast[Int](2))
Source.single(1) ~> broadcast.in
broadcast ~> sharedDoubler ~> topHeadSink broadcast.out(0) ~> sharedDoubler ~> topHS.inlet
broadcast ~> sharedDoubler ~> bottomHeadSink broadcast.out(1) ~> sharedDoubler ~> bottomHS.inlet
} }
//#flow-graph-reusing-a-flow //#flow-graph-reusing-a-flow
// format: ON // format: ON
val map = g.run() val (topFuture, bottomFuture) = g.run()
Await.result(map.get(topHeadSink), 300.millis) shouldEqual 2 Await.result(topFuture, 300.millis) shouldEqual 2
Await.result(map.get(bottomHeadSink), 300.millis) shouldEqual 2 Await.result(bottomFuture, 300.millis) shouldEqual 2
} }
} }

View file

@ -1,13 +1,13 @@
package docs.stream package docs.stream
import akka.stream.ActorFlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.{ RunnableFlow, Sink, Source, Flow } import akka.stream.scaladsl.{ RunnableFlow, Sink, Source, Flow, Keep }
import akka.stream.stage.PushPullStage import akka.stream.stage.PushPullStage
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import org.scalatest.concurrent.{ ScalaFutures, Futures } import org.scalatest.concurrent.{ ScalaFutures, Futures }
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.Await import scala.concurrent.{ Future, Await }
import scala.concurrent.duration._ import scala.concurrent.duration._
class FlowStagesSpec extends AkkaSpec with ScalaFutures { class FlowStagesSpec extends AkkaSpec with ScalaFutures {
@ -75,17 +75,17 @@ class FlowStagesSpec extends AkkaSpec with ScalaFutures {
//#one-to-many //#one-to-many
val keyedSink = Sink.head[immutable.Seq[Int]] val keyedSink = Sink.head[immutable.Seq[Int]]
val sink = Flow[Int].grouped(10).to(keyedSink) val sink = Flow[Int].grouped(10).toMat(keyedSink)(Keep.right)
//#stage-chain //#stage-chain
val runnable: RunnableFlow = Source(1 to 10) val resultFuture = Source(1 to 10)
.transform(() => new Filter(_ % 2 == 0)) .transform(() => new Filter(_ % 2 == 0))
.transform(() => new Duplicator()) .transform(() => new Duplicator())
.transform(() => new Map(_ / 2)) .transform(() => new Map(_ / 2))
.to(sink) .runWith(sink)
//#stage-chain //#stage-chain
Await.result(runnable.run().get(keyedSink), 3.seconds) should be(Seq(1, 1, 2, 2, 3, 3, 4, 4, 5, 5)) Await.result(resultFuture, 3.seconds) should be(Seq(1, 1, 2, 2, 3, 3, 4, 4, 5, 5))
} }

View file

@ -13,89 +13,90 @@ class GraphCyclesSpec extends AkkaSpec {
"include a deadlocked cycle" in { "include a deadlocked cycle" in {
// format: OFF
//#deadlocked //#deadlocked
// WARNING! The graph below deadlocks! // WARNING! The graph below deadlocks!
FlowGraph { implicit b => FlowGraph.closed() { implicit b =>
import FlowGraphImplicits._ import FlowGraph.Implicits._
b.allowCycles()
val merge = Merge[Int] val merge = b.add(Merge[Int](2))
val bcast = Broadcast[Int] val bcast = b.add(Broadcast[Int](2))
source ~> merge ~> Flow[Int].map { (s) => println(s); s } ~> bcast ~> Sink.ignore source ~> merge ~> Flow[Int].map { s => println(s); s } ~> bcast ~> Sink.ignore()
bcast ~> merge merge <~ bcast
} }
//#deadlocked //#deadlocked
// format: ON
} }
"include an unfair cycle" in { "include an unfair cycle" in {
// format: OFF
//#unfair //#unfair
// WARNING! The graph below stops consuming from "source" after a few steps // WARNING! The graph below stops consuming from "source" after a few steps
FlowGraph { implicit b => FlowGraph.closed() { implicit b =>
import FlowGraphImplicits._ import FlowGraph.Implicits._
b.allowCycles()
val merge = MergePreferred[Int] val merge = b.add(MergePreferred[Int](1))
val bcast = Broadcast[Int] val bcast = b.add(Broadcast[Int](2))
source ~> merge ~> Flow[Int].map { (s) => println(s); s } ~> bcast ~> Sink.ignore source ~> merge ~> Flow[Int].map { s => println(s); s } ~> bcast ~> Sink.ignore()
bcast ~> merge.preferred merge.preferred <~ bcast
} }
//#unfair //#unfair
// format: ON
} }
"include a dropping cycle" in { "include a dropping cycle" in {
// format: OFF
//#dropping //#dropping
FlowGraph { implicit b => FlowGraph.closed() { implicit b =>
import FlowGraphImplicits._ import FlowGraph.Implicits._
b.allowCycles()
val merge = Merge[Int] val merge = b.add(Merge[Int](2))
val bcast = Broadcast[Int] val bcast = b.add(Broadcast[Int](2))
source ~> merge ~> Flow[Int].map { (s) => println(s); s } ~> bcast ~> Sink.ignore source ~> merge ~> Flow[Int].map { s => println(s); s } ~> bcast ~> Sink.ignore()
bcast ~> Flow[Int].buffer(10, OverflowStrategy.dropHead) ~> merge merge <~ Flow[Int].buffer(10, OverflowStrategy.dropHead) <~ bcast
} }
//#dropping //#dropping
// format: ON
} }
"include a dead zipping cycle" in { "include a dead zipping cycle" in {
// format: OFF
//#zipping-dead //#zipping-dead
// WARNING! The graph below never processes any elements // WARNING! The graph below never processes any elements
FlowGraph { implicit b => FlowGraph.closed() { implicit b =>
import FlowGraphImplicits._ import FlowGraph.Implicits._
b.allowCycles()
val zip = ZipWith[Int, Int, Int]((left, right) => right) val zip = b.add(ZipWith[Int, Int, Int]((left, right) => right))
val bcast = Broadcast[Int] val bcast = b.add(Broadcast[Int](2))
source ~> zip.left ~> Flow[Int].map { (s) => println(s); s } ~> bcast ~> Sink.ignore source ~> zip.in0
bcast ~> zip.right zip.out.map { s => println(s); s } ~> bcast ~> Sink.ignore()
zip.in1 <~ bcast
} }
//#zipping-dead //#zipping-dead
// format: ON
} }
"include a live zipping cycle" in { "include a live zipping cycle" in {
// format: OFF
//#zipping-live //#zipping-live
FlowGraph { implicit b => FlowGraph.closed() { implicit b =>
import FlowGraphImplicits._ import FlowGraph.Implicits._
b.allowCycles()
val zip = ZipWith[Int, Int, Int]((left, right) => left) val zip = b.add(ZipWith((left: Int, right: Int) => left))
val bcast = Broadcast[Int] val bcast = b.add(Broadcast[Int](2))
val concat = Concat[Int] val concat = b.add(Concat[Int]())
source ~> zip.left ~> Flow[Int].map { (s) => println(s); s } ~> bcast ~> Sink.ignore
bcast ~> concat.second ~> zip.right
Source.single(0) ~> concat.first
source ~> zip.in0
zip.out.map { s => println(s); s } ~> bcast ~> Sink.ignore()
zip.in1 <~ concat <~ bcast
concat <~ Source.single(0)
} }
//#zipping-live //#zipping-live
// format: ON
} }
} }

View file

@ -133,21 +133,21 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
val emailServer = new EmailServer(probe.ref) val emailServer = new EmailServer(probe.ref)
//#tweet-authors //#tweet-authors
val authors: Source[Author] = val authors: Source[Author, Unit] =
tweets tweets
.filter(_.hashtags.contains(akka)) .filter(_.hashtags.contains(akka))
.map(_.author) .map(_.author)
//#tweet-authors //#tweet-authors
//#email-addresses-mapAsync //#email-addresses-mapAsync
val emailAddresses: Source[String] = val emailAddresses: Source[String, Unit] =
authors authors
.mapAsync(author => addressSystem.lookupEmail(author.handle)) .mapAsync(author => addressSystem.lookupEmail(author.handle))
.collect { case Some(emailAddress) => emailAddress } .collect { case Some(emailAddress) => emailAddress }
//#email-addresses-mapAsync //#email-addresses-mapAsync
//#send-emails //#send-emails
val sendEmails: RunnableFlow = val sendEmails: RunnableFlow[Unit] =
emailAddresses emailAddresses
.mapAsync { address => .mapAsync { address =>
emailServer.send( emailServer.send(
@ -169,14 +169,14 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
"lookup email with mapAsync and supervision" in { "lookup email with mapAsync and supervision" in {
val addressSystem = new AddressSystem2 val addressSystem = new AddressSystem2
val authors: Source[Author] = val authors: Source[Author, Unit] =
tweets.filter(_.hashtags.contains(akka)).map(_.author) tweets.filter(_.hashtags.contains(akka)).map(_.author)
//#email-addresses-mapAsync-supervision //#email-addresses-mapAsync-supervision
import OperationAttributes.supervisionStrategy import OperationAttributes.supervisionStrategy
import Supervision.resumingDecider import Supervision.resumingDecider
val emailAddresses: Source[String] = val emailAddresses: Source[String, Unit] =
authors.section(supervisionStrategy(resumingDecider)) { authors.section(supervisionStrategy(resumingDecider)) {
_.mapAsync(author => addressSystem.lookupEmail(author.handle)) _.mapAsync(author => addressSystem.lookupEmail(author.handle))
} }
@ -189,15 +189,15 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
val emailServer = new EmailServer(probe.ref) val emailServer = new EmailServer(probe.ref)
//#external-service-mapAsyncUnordered //#external-service-mapAsyncUnordered
val authors: Source[Author] = val authors: Source[Author, Unit] =
tweets.filter(_.hashtags.contains(akka)).map(_.author) tweets.filter(_.hashtags.contains(akka)).map(_.author)
val emailAddresses: Source[String] = val emailAddresses: Source[String, Unit] =
authors authors
.mapAsyncUnordered(author => addressSystem.lookupEmail(author.handle)) .mapAsyncUnordered(author => addressSystem.lookupEmail(author.handle))
.collect { case Some(emailAddress) => emailAddress } .collect { case Some(emailAddress) => emailAddress }
val sendEmails: RunnableFlow = val sendEmails: RunnableFlow[Unit] =
emailAddresses emailAddresses
.mapAsyncUnordered { address => .mapAsyncUnordered { address =>
emailServer.send( emailServer.send(
@ -232,7 +232,7 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
//#blocking-mapAsync //#blocking-mapAsync
val blockingExecutionContext = system.dispatchers.lookup("blocking-dispatcher") val blockingExecutionContext = system.dispatchers.lookup("blocking-dispatcher")
val sendTextMessages: RunnableFlow = val sendTextMessages: RunnableFlow[Unit] =
phoneNumbers phoneNumbers
.mapAsync { phoneNo => .mapAsync { phoneNo =>
Future { Future {
@ -267,7 +267,7 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
.collect { case Some(phoneNo) => phoneNo } .collect { case Some(phoneNo) => phoneNo }
//#blocking-map //#blocking-map
val sendTextMessages: RunnableFlow = val sendTextMessages: RunnableFlow[Unit] =
phoneNumbers phoneNumbers
.section(OperationAttributes.dispatcher("blocking-dispatcher")) { .section(OperationAttributes.dispatcher("blocking-dispatcher")) {
_.map { phoneNo => _.map { phoneNo =>
@ -294,10 +294,10 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
val database = system.actorOf(Props(classOf[DatabaseService], probe.ref), "db") val database = system.actorOf(Props(classOf[DatabaseService], probe.ref), "db")
//#save-tweets //#save-tweets
val akkaTweets: Source[Tweet] = tweets.filter(_.hashtags.contains(akka)) val akkaTweets: Source[Tweet, Unit] = tweets.filter(_.hashtags.contains(akka))
implicit val timeout = Timeout(3.seconds) implicit val timeout = Timeout(3.seconds)
val saveTweets: RunnableFlow = val saveTweets: RunnableFlow[Unit] =
akkaTweets akkaTweets
.mapAsync(tweet => database ? Save(tweet)) .mapAsync(tweet => database ? Save(tweet))
.to(Sink.ignore) .to(Sink.ignore)

View file

@ -43,7 +43,7 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
val impl = new Fixture { val impl = new Fixture {
override def tweets: Publisher[Tweet] = override def tweets: Publisher[Tweet] =
TwitterStreamQuickstartDocSpec.tweets.runWith(Sink.publisher) TwitterStreamQuickstartDocSpec.tweets.runWith(Sink.publisher())
override def storage = SubscriberProbe[Author] override def storage = SubscriberProbe[Author]
@ -95,7 +95,7 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
//#source-publisher //#source-publisher
val authorPublisher: Publisher[Author] = val authorPublisher: Publisher[Author] =
Source(tweets).via(authors).runWith(Sink.publisher) Source(tweets).via(authors).runWith(Sink.publisher())
authorPublisher.subscribe(storage) authorPublisher.subscribe(storage)
//#source-publisher //#source-publisher

View file

@ -43,15 +43,15 @@ class StreamBuffersRateSpec extends AkkaSpec {
import scala.concurrent.duration._ import scala.concurrent.duration._
case class Tick() case class Tick()
FlowGraph { implicit b => FlowGraph.closed() { implicit b =>
import FlowGraphImplicits._ import FlowGraph.Implicits._
val zipper = ZipWith[Tick, Int, Int]((tick, count) => count) val zipper = b.add(ZipWith[Tick, Int, Int]((tick, count) => count))
Source(initialDelay = 3.second, interval = 3.second, Tick()) ~> zipper.in0
Source(initialDelay = 1.second, interval = 1.second, "message!") Source(initialDelay = 1.second, interval = 1.second, "message!")
.conflate(seed = (_) => 1)((count, _) => count + 1) ~> zipper.right .conflate(seed = (_) => 1)((count, _) => count + 1) ~> zipper.in1
Source(initialDelay = 3.second, interval = 3.second, Tick()) ~> zipper.left
zipper.out ~> Sink.foreach(println) zipper.out ~> Sink.foreach(println)
} }
@ -60,10 +60,10 @@ class StreamBuffersRateSpec extends AkkaSpec {
"explcit buffers" in { "explcit buffers" in {
trait Job trait Job
def inboundJobsConnector(): Source[Job] = Source.empty() def inboundJobsConnector(): Source[Job, Unit] = Source.empty()
//#explicit-buffers-backpressure //#explicit-buffers-backpressure
// Getting a stream of jobs from an imaginary external system as a Source // Getting a stream of jobs from an imaginary external system as a Source
val jobs: Source[Job] = inboundJobsConnector() val jobs: Source[Job, Unit] = inboundJobsConnector()
jobs.buffer(1000, OverflowStrategy.backpressure) jobs.buffer(1000, OverflowStrategy.backpressure)
//#explicit-buffers-backpressure //#explicit-buffers-backpressure

View file

@ -3,20 +3,11 @@
*/ */
package docs.stream package docs.stream
import akka.stream.ActorFlowMaterializer import akka.stream.scaladsl._
import akka.stream.scaladsl.Broadcast import akka.stream._
import akka.stream.scaladsl.Flow
import akka.stream.scaladsl.FlowGraph
import akka.stream.scaladsl.FlowGraphImplicits
import akka.stream.scaladsl.PartialFlowGraph
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
import akka.stream.scaladsl.UndefinedSink
import akka.stream.scaladsl.UndefinedSource
import akka.stream.scaladsl.Zip
import akka.stream.scaladsl.ZipWith
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import scala.collection.immutable
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.Future import scala.concurrent.Future
import scala.concurrent.duration._ import scala.concurrent.duration._
@ -28,83 +19,55 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
implicit val mat = ActorFlowMaterializer() implicit val mat = ActorFlowMaterializer()
"build with open ports" in { "build with open ports" in {
// format: OFF
//#simple-partial-flow-graph //#simple-partial-flow-graph
// defined outside as they will be used by different FlowGraphs val pickMaxOfThree = FlowGraph.partial() { implicit b =>
// 1) first by the PartialFlowGraph to mark its open input and output ports import FlowGraph.Implicits._
// 2) then by the assembling FlowGraph which will attach real sinks and sources to them
val in1 = UndefinedSource[Int]
val in2 = UndefinedSource[Int]
val in3 = UndefinedSource[Int]
val out = UndefinedSink[Int]
val pickMaxOfThree: PartialFlowGraph = PartialFlowGraph { implicit b => val zip1 = b.add(ZipWith[Int, Int, Int](math.max _))
import FlowGraphImplicits._ val zip2 = b.add(ZipWith[Int, Int, Int](math.max _))
zip1.out ~> zip2.in0
val zip1 = ZipWith[Int, Int, Int](math.max _) UniformFanInShape(zip2.out, zip1.in0, zip1.in1, zip2.in1)
val zip2 = ZipWith[Int, Int, Int](math.max _)
in1 ~> zip1.left
in2 ~> zip1.right
zip1.out ~> zip2.left
in3 ~> zip2.right
zip2.out ~> out
} }
//#simple-partial-flow-graph
// format: ON
//#simple-partial-flow-graph
val resultSink = Sink.head[Int] val resultSink = Sink.head[Int]
val g = FlowGraph { b => val g = FlowGraph.closed(resultSink) { implicit b =>
// import the partial flow graph explicitly sink =>
b.importPartialFlowGraph(pickMaxOfThree) import FlowGraph.Implicits._
b.attachSource(in1, Source.single(1)) // importing the partial graph will return its shape (inlets & outlets)
b.attachSource(in2, Source.single(2)) val pm3 = b.add(pickMaxOfThree)
b.attachSource(in3, Source.single(3))
b.attachSink(out, resultSink) Source.single(1) ~> pm3.in(0)
Source.single(2) ~> pm3.in(1)
Source.single(3) ~> pm3.in(2)
pm3.out ~> sink.inlet
} }
val materialized = g.run() val max: Future[Int] = g.run()
val max: Future[Int] = materialized.get(resultSink)
Await.result(max, 300.millis) should equal(3) Await.result(max, 300.millis) should equal(3)
//#simple-partial-flow-graph //#simple-partial-flow-graph
val g2 =
//#simple-partial-flow-graph-import-shorthand
FlowGraph(pickMaxOfThree) { b =>
b.attachSource(in1, Source.single(1))
b.attachSource(in2, Source.single(2))
b.attachSource(in3, Source.single(3))
b.attachSink(out, resultSink)
}
//#simple-partial-flow-graph-import-shorthand
val materialized2 = g.run()
val max2: Future[Int] = materialized2.get(resultSink)
Await.result(max2, 300.millis) should equal(3)
} }
"build source from partial flow graph" in { "build source from partial flow graph" in {
//#source-from-partial-flow-graph //#source-from-partial-flow-graph
val pairs: Source[(Int, Int)] = Source() { implicit b => val pairs = Source() { implicit b =>
import FlowGraphImplicits._ import FlowGraph.Implicits._
// prepare graph elements // prepare graph elements
val undefinedSink = UndefinedSink[(Int, Int)] val zip = b.add(Zip[Int, Int]())
val zip = Zip[Int, Int]
def ints = Source(() => Iterator.from(1)) def ints = Source(() => Iterator.from(1))
// connect the graph // connect the graph
ints ~> Flow[Int].filter(_ % 2 != 0) ~> zip.left ints ~> Flow[Int].filter(_ % 2 != 0) ~> zip.in0
ints ~> Flow[Int].filter(_ % 2 == 0) ~> zip.right ints ~> Flow[Int].filter(_ % 2 == 0) ~> zip.in1
zip.out ~> undefinedSink
// expose undefined sink // expose port
undefinedSink zip.out
} }
val firstPair: Future[(Int, Int)] = pairs.runWith(Sink.head) val firstPair: Future[(Int, Int)] = pairs.runWith(Sink.head())
//#source-from-partial-flow-graph //#source-from-partial-flow-graph
Await.result(firstPair, 300.millis) should equal(1 -> 2) Await.result(firstPair, 300.millis) should equal(1 -> 2)
} }
@ -112,23 +75,18 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
"build flow from partial flow graph" in { "build flow from partial flow graph" in {
//#flow-from-partial-flow-graph //#flow-from-partial-flow-graph
val pairUpWithToString = Flow() { implicit b => val pairUpWithToString = Flow() { implicit b =>
import FlowGraphImplicits._ import FlowGraph.Implicits._
// prepare graph elements // prepare graph elements
val undefinedSource = UndefinedSource[Int] val broadcast = b.add(Broadcast[Int](2))
val undefinedSink = UndefinedSink[(Int, String)] val zip = b.add(Zip[Int, String]())
val broadcast = Broadcast[Int]
val zip = Zip[Int, String]
// connect the graph // connect the graph
undefinedSource ~> broadcast broadcast.out(0) ~> Flow[Int].map(identity) ~> zip.in0
broadcast ~> Flow[Int].map(identity) ~> zip.left broadcast.out(1) ~> Flow[Int].map(_.toString) ~> zip.in1
broadcast ~> Flow[Int].map(_.toString) ~> zip.right
zip.out ~> undefinedSink
// expose undefined ports // expose ports
(undefinedSource, undefinedSink) (broadcast.in, zip.out)
} }
//#flow-from-partial-flow-graph //#flow-from-partial-flow-graph
@ -136,7 +94,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
// format: OFF // format: OFF
val (_, matSink: Future[(Int, String)]) = val (_, matSink: Future[(Int, String)]) =
//#flow-from-partial-flow-graph //#flow-from-partial-flow-graph
pairUpWithToString.runWith(Source(List(1)), Sink.head) pairUpWithToString.runWith(Source(List(1)), Sink.head())
//#flow-from-partial-flow-graph //#flow-from-partial-flow-graph
// format: ON // format: ON

View file

@ -6,20 +6,16 @@ package docs.stream
import java.net.InetSocketAddress import java.net.InetSocketAddress
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer import akka.stream._
import akka.stream.scaladsl.Concat import akka.stream.scaladsl._
import akka.stream.scaladsl.Flow
import akka.stream.scaladsl.FlowGraphImplicits
import akka.stream.scaladsl.Source
import akka.stream.scaladsl.StreamTcp
import akka.stream.scaladsl.StreamTcp._
import akka.stream.scaladsl.UndefinedSink
import akka.stream.scaladsl.UndefinedSource
import akka.stream.stage.{ PushStage, Directive, Context } import akka.stream.stage.{ PushStage, Directive, Context }
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.testkit.TestProbe import akka.testkit.TestProbe
import akka.util.ByteString import akka.util.ByteString
import cookbook.RecipeParseLines import cookbook.RecipeParseLines
import StreamTcp._
import scala.concurrent.Future
class StreamTcpDocSpec extends AkkaSpec { class StreamTcpDocSpec extends AkkaSpec {
@ -34,11 +30,9 @@ class StreamTcpDocSpec extends AkkaSpec {
"simple server connection" ignore { "simple server connection" ignore {
//#echo-server-simple-bind //#echo-server-simple-bind
val localhost = new InetSocketAddress("127.0.0.1", 8888) val localhost = new InetSocketAddress("127.0.0.1", 8888)
val binding = StreamTcp().bind(localhost)
//#echo-server-simple-bind
//#echo-server-simple-handle //#echo-server-simple-handle
val connections: Source[IncomingConnection] = binding.connections val connections: Source[IncomingConnection, Future[ServerBinding]] = StreamTcp().bind(localhost)
//#echo-server-simple-bind
connections runForeach { connection => connections runForeach { connection =>
println(s"New connection from: ${connection.remoteAddress}") println(s"New connection from: ${connection.remoteAddress}")
@ -53,19 +47,35 @@ class StreamTcpDocSpec extends AkkaSpec {
//#echo-server-simple-handle //#echo-server-simple-handle
} }
"actually working client-server CLI app" in { "simple repl client" ignore {
val sys: ActorSystem = ???
//#repl-client
val connection: Flow[ByteString, ByteString, Future[OutgoingConnection]] = StreamTcp().outgoingConnection(localhost)
val repl = Flow[ByteString]
.transform(() => RecipeParseLines.parseLines("\n", maximumLineBytes = 256))
.map(text => println("Server: " + text))
.map(_ => readLine("> "))
.map {
case "q" =>
sys.shutdown(); ByteString("BYE")
case text => ByteString(s"$text")
}
connection.join(repl)
//#repl-client
}
"initial server banner echo server" ignore {
val connections = StreamTcp().bind(localhost)
val serverProbe = TestProbe() val serverProbe = TestProbe()
val binding = StreamTcp().bind(localhost)
//#welcome-banner-chat-server //#welcome-banner-chat-server
binding.connections runForeach { connection => connections runForeach { connection =>
val serverLogic = Flow() { implicit b => val serverLogic = Flow() { implicit b =>
import FlowGraphImplicits._ import FlowGraph.Implicits._
// to be filled in by StreamTCP
val in = UndefinedSource[ByteString]
val out = UndefinedSink[ByteString]
// server logic, parses incoming commands // server logic, parses incoming commands
val commandParser = new PushStage[String, String] { val commandParser = new PushStage[String, String] {
@ -81,23 +91,22 @@ class StreamTcpDocSpec extends AkkaSpec {
val welcomeMsg = s"Welcome to: $localAddress, you are: $remoteAddress!\n" val welcomeMsg = s"Welcome to: $localAddress, you are: $remoteAddress!\n"
val welcome = Source.single(ByteString(welcomeMsg)) val welcome = Source.single(ByteString(welcomeMsg))
val echo = Flow[ByteString] val echo = b.add(Flow[ByteString]
.transform(() => RecipeParseLines.parseLines("\n", maximumLineBytes = 256)) .transform(() => RecipeParseLines.parseLines("\n", maximumLineBytes = 256))
//#welcome-banner-chat-server //#welcome-banner-chat-server
.map { command serverProbe.ref ! command; command } .map { command serverProbe.ref ! command; command }
//#welcome-banner-chat-server //#welcome-banner-chat-server
.transform(() commandParser) .transform(() commandParser)
.map(_ + "\n") .map(_ + "\n")
.map(ByteString(_)) .map(ByteString(_)))
val concat = Concat[ByteString] val concat = b.add(Concat[ByteString]())
// first we emit the welcome message, // first we emit the welcome message,
welcome ~> concat.first welcome ~> concat.in(0)
// then we continue using the echo-logic Flow // then we continue using the echo-logic Flow
in ~> echo ~> concat.second echo.outlet ~> concat.in(1)
concat.out ~> out (echo.inlet, concat.out)
(in, out)
} }
connection.handleWith(serverLogic) connection.handleWith(serverLogic)
@ -114,7 +123,7 @@ class StreamTcpDocSpec extends AkkaSpec {
} }
//#repl-client //#repl-client
val connection: OutgoingConnection = StreamTcp().outgoingConnection(localhost) val connection = StreamTcp().outgoingConnection(localhost)
val replParser = new PushStage[String, ByteString] { val replParser = new PushStage[String, ByteString] {
override def onPush(elem: String, ctx: Context[ByteString]): Directive = { override def onPush(elem: String, ctx: Context[ByteString]): Directive = {
@ -131,7 +140,7 @@ class StreamTcpDocSpec extends AkkaSpec {
.map(_ => readLine("> ")) .map(_ => readLine("> "))
.transform(() replParser) .transform(() replParser)
connection.handleWith(repl) connection.join(repl)
//#repl-client //#repl-client
serverProbe.expectMsg("Hello world") serverProbe.expectMsg("Hello world")

View file

@ -8,14 +8,7 @@ package docs.stream
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.OverflowStrategy import akka.stream.OverflowStrategy
import akka.stream.scaladsl.Broadcast import akka.stream.scaladsl._
import akka.stream.scaladsl.Flow
import akka.stream.scaladsl.FlowGraph
import akka.stream.scaladsl.FlowGraphImplicits
import akka.stream.scaladsl.MaterializedMap
import akka.stream.scaladsl.RunnableFlow
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.Future import scala.concurrent.Future
@ -57,9 +50,12 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
implicit val executionContext = system.dispatcher implicit val executionContext = system.dispatcher
// Disable println
def println(s: Any): Unit = ()
trait Example0 { trait Example0 {
//#tweet-source //#tweet-source
val tweets: Source[Tweet] val tweets: Source[Tweet, Unit]
//#tweet-source //#tweet-source
} }
@ -74,7 +70,7 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
"filter and map" in { "filter and map" in {
//#authors-filter-map //#authors-filter-map
val authors: Source[Author] = val authors: Source[Author, Unit] =
tweets tweets
.filter(_.hashtags.contains(akka)) .filter(_.hashtags.contains(akka))
.map(_.author) .map(_.author)
@ -82,7 +78,7 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
trait Example3 { trait Example3 {
//#authors-collect //#authors-collect
val authors: Source[Author] = val authors: Source[Author, Unit] =
tweets.collect { case t if t.hashtags.contains(akka) => t.author } tweets.collect { case t if t.hashtags.contains(akka) => t.author }
//#authors-collect //#authors-collect
} }
@ -98,29 +94,30 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
"mapConcat hashtags" in { "mapConcat hashtags" in {
//#hashtags-mapConcat //#hashtags-mapConcat
val hashtags: Source[Hashtag] = tweets.mapConcat(_.hashtags.toList) val hashtags: Source[Hashtag, Unit] = tweets.mapConcat(_.hashtags.toList)
//#hashtags-mapConcat //#hashtags-mapConcat
} }
trait HiddenDefinitions { trait HiddenDefinitions {
//#flow-graph-broadcast //#flow-graph-broadcast
val writeAuthors: Sink[Author] = ??? val writeAuthors: Sink[Author, Unit] = ???
val writeHashtags: Sink[Hashtag] = ??? val writeHashtags: Sink[Hashtag, Unit] = ???
//#flow-graph-broadcast //#flow-graph-broadcast
} }
"simple broadcast" in { "simple broadcast" in {
val writeAuthors: Sink[Author] = Sink.ignore val writeAuthors: Sink[Author, Unit] = Sink.ignore
val writeHashtags: Sink[Hashtag] = Sink.ignore val writeHashtags: Sink[Hashtag, Unit] = Sink.ignore
// format: OFF // format: OFF
//#flow-graph-broadcast //#flow-graph-broadcast
val g = FlowGraph { implicit builder => val g = FlowGraph.closed() { implicit b =>
import FlowGraphImplicits._ import FlowGraph.Implicits._
val b = Broadcast[Tweet] val bcast = b.add(Broadcast[Tweet](2))
tweets ~> b ~> Flow[Tweet].map(_.author) ~> writeAuthors tweets ~> bcast.in
b ~> Flow[Tweet].mapConcat(_.hashtags.toList) ~> writeHashtags bcast.out(0) ~> Flow[Tweet].map(_.author) ~> writeAuthors
bcast.out(1) ~> Flow[Tweet].mapConcat(_.hashtags.toList) ~> writeHashtags
} }
g.run() g.run()
//#flow-graph-broadcast //#flow-graph-broadcast
@ -160,10 +157,9 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
//#tweets-fold-count //#tweets-fold-count
val sumSink = Sink.fold[Int, Int](0)(_ + _) val sumSink = Sink.fold[Int, Int](0)(_ + _)
val counter: RunnableFlow = tweets.map(t => 1).to(sumSink) val counter: RunnableFlow[Future[Int]] = tweets.map(t => 1).toMat(sumSink)(Keep.right)
val map: MaterializedMap = counter.run()
val sum: Future[Int] = map.get(sumSink) val sum: Future[Int] = counter.run()
sum.foreach(c => println(s"Total tweets processed: $c")) sum.foreach(c => println(s"Total tweets processed: $c"))
//#tweets-fold-count //#tweets-fold-count
@ -180,26 +176,20 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
//#tweets-runnable-flow-materialized-twice //#tweets-runnable-flow-materialized-twice
val sumSink = Sink.fold[Int, Int](0)(_ + _) val sumSink = Sink.fold[Int, Int](0)(_ + _)
val counterRunnableFlow: RunnableFlow = val counterRunnableFlow: RunnableFlow[Future[Int]] =
tweetsInMinuteFromNow tweetsInMinuteFromNow
.filter(_.hashtags contains akka) .filter(_.hashtags contains akka)
.map(t => 1) .map(t => 1)
.to(sumSink) .toMat(sumSink)(Keep.right)
// materialize the stream once in the morning // materialize the stream once in the morning
val morningMaterialized = counterRunnableFlow.run() val morningTweetsCount: Future[Int] = counterRunnableFlow.run()
// and once in the evening, reusing the // and once in the evening, reusing the flow
val eveningMaterialized = counterRunnableFlow.run() val eveningTweetsCount: Future[Int] = counterRunnableFlow.run()
// the sumSink materialized two different futures
// we use it as key to get the materialized value out of the materialized map
val morningTweetsCount: Future[Int] = morningMaterialized.get(sumSink)
val eveningTweetsCount: Future[Int] = eveningMaterialized.get(sumSink)
//#tweets-runnable-flow-materialized-twice //#tweets-runnable-flow-materialized-twice
val map: MaterializedMap = counterRunnableFlow.run() val sum: Future[Int] = counterRunnableFlow.run()
val sum: Future[Int] = map.get(sumSink)
sum.map { c => println(s"Total tweets processed: $c") } sum.map { c => println(s"Total tweets processed: $c") }
} }

View file

@ -41,7 +41,7 @@ class RecipeByteStrings extends RecipeSpec {
val chunksStream = rawBytes.transform(() => new Chunker(ChunkLimit)) val chunksStream = rawBytes.transform(() => new Chunker(ChunkLimit))
//#bytestring-chunker //#bytestring-chunker
val chunksFuture = chunksStream.grouped(10).runWith(Sink.head) val chunksFuture = chunksStream.grouped(10).runWith(Sink.head())
val chunks = Await.result(chunksFuture, 3.seconds) val chunks = Await.result(chunksFuture, 3.seconds)
@ -70,11 +70,11 @@ class RecipeByteStrings extends RecipeSpec {
val bytes1 = Source(List(ByteString(1, 2), ByteString(3), ByteString(4, 5, 6), ByteString(7, 8, 9))) val bytes1 = Source(List(ByteString(1, 2), ByteString(3), ByteString(4, 5, 6), ByteString(7, 8, 9)))
val bytes2 = Source(List(ByteString(1, 2), ByteString(3), ByteString(4, 5, 6), ByteString(7, 8, 9, 10))) val bytes2 = Source(List(ByteString(1, 2), ByteString(3), ByteString(4, 5, 6), ByteString(7, 8, 9, 10)))
Await.result(bytes1.via(limiter).grouped(10).runWith(Sink.head), 3.seconds) Await.result(bytes1.via(limiter).grouped(10).runWith(Sink.head()), 3.seconds)
.fold(ByteString())(_ ++ _) should be(ByteString(1, 2, 3, 4, 5, 6, 7, 8, 9)) .fold(ByteString())(_ ++ _) should be(ByteString(1, 2, 3, 4, 5, 6, 7, 8, 9))
an[IllegalStateException] must be thrownBy { an[IllegalStateException] must be thrownBy {
Await.result(bytes2.via(limiter).grouped(10).runWith(Sink.head), 3.seconds) Await.result(bytes2.via(limiter).grouped(10).runWith(Sink.head()), 3.seconds)
} }
} }
@ -83,10 +83,10 @@ class RecipeByteStrings extends RecipeSpec {
val data = Source(List(ByteString(1, 2), ByteString(3), ByteString(4, 5, 6), ByteString(7, 8, 9))) val data = Source(List(ByteString(1, 2), ByteString(3), ByteString(4, 5, 6), ByteString(7, 8, 9)))
//#compacting-bytestrings //#compacting-bytestrings
val compacted: Source[ByteString] = data.map(_.compact) val compacted: Source[ByteString, Unit] = data.map(_.compact)
//#compacting-bytestrings //#compacting-bytestrings
Await.result(compacted.grouped(10).runWith(Sink.head), 3.seconds).forall(_.isCompact) should be(true) Await.result(compacted.grouped(10).runWith(Sink.head()), 3.seconds).forall(_.isCompact) should be(true)
} }
} }

View file

@ -41,10 +41,10 @@ class RecipeDigest extends RecipeSpec {
} }
} }
val digest: Source[ByteString] = data.transform(() => digestCalculator("SHA-256")) val digest: Source[ByteString, Unit] = data.transform(() => digestCalculator("SHA-256"))
//#calculating-digest //#calculating-digest
Await.result(digest.runWith(Sink.head), 3.seconds) should be( Await.result(digest.runWith(Sink.head()), 3.seconds) should be(
ByteString( ByteString(
0x24, 0x8d, 0x6a, 0x61, 0x24, 0x8d, 0x6a, 0x61,
0xd2, 0x06, 0x38, 0xb8, 0xd2, 0x06, 0x38, 0xb8,

View file

@ -16,30 +16,26 @@ class RecipeDroppyBroadcast extends RecipeSpec {
val sub1 = SubscriberProbe[Int]() val sub1 = SubscriberProbe[Int]()
val sub2 = SubscriberProbe[Int]() val sub2 = SubscriberProbe[Int]()
val futureSink = Sink.head[Seq[Int]]
val mySink1 = Sink(sub1) val mySink1 = Sink(sub1)
val mySink2 = Sink(sub2) val mySink2 = Sink(sub2)
val futureSink = Sink.head[Seq[Int]] val mySink3 = Flow[Int].grouped(200).toMat(futureSink)(Keep.right)
val mySink3 = Flow[Int].grouped(200).to(futureSink)
//#droppy-bcast //#droppy-bcast
// Makes a sink drop elements if too slow val graph = FlowGraph.closed(mySink1, mySink2, mySink3)((_, _, _)) { implicit b =>
def droppySink[T](sink: Sink[T], bufferSize: Int): Sink[T] = { (sink1, sink2, sink3) =>
Flow[T].buffer(bufferSize, OverflowStrategy.dropHead).to(sink) import FlowGraph.Implicits._
}
import FlowGraphImplicits._
val graph = FlowGraph { implicit builder =>
val bcast = Broadcast[Int]
val bcast = b.add(Broadcast[Int](3))
myElements ~> bcast myElements ~> bcast
bcast ~> droppySink(mySink1, 10) bcast.buffer(10, OverflowStrategy.dropHead) ~> sink1
bcast ~> droppySink(mySink2, 10) bcast.buffer(10, OverflowStrategy.dropHead) ~> sink2
bcast ~> droppySink(mySink3, 10) bcast.buffer(10, OverflowStrategy.dropHead) ~> sink3
} }
//#droppy-bcast //#droppy-bcast
Await.result(graph.run().get(futureSink), 3.seconds).sum should be(5050) Await.result(graph.run()._3, 3.seconds).sum should be(5050)
sub1.expectSubscription().request(10) sub1.expectSubscription().request(10)
sub2.expectSubscription().request(10) sub2.expectSubscription().request(10)

View file

@ -15,11 +15,11 @@ class RecipeFlattenSeq extends RecipeSpec {
val someDataSource = Source(List(List("1"), List("2"), List("3", "4", "5"), List("6", "7"))) val someDataSource = Source(List(List("1"), List("2"), List("3", "4", "5"), List("6", "7")))
//#flattening-seqs //#flattening-seqs
val myData: Source[List[Message]] = someDataSource val myData: Source[List[Message], Unit] = someDataSource
val flattened: Source[Message] = myData.mapConcat(identity) val flattened: Source[Message, Unit] = myData.mapConcat(identity)
//#flattening-seqs //#flattening-seqs
Await.result(flattened.grouped(8).runWith(Sink.head), 3.seconds) should be(List("1", "2", "3", "4", "5", "6", "7")) Await.result(flattened.grouped(8).runWith(Sink.head()), 3.seconds) should be(List("1", "2", "3", "4", "5", "6", "7"))
} }

View file

@ -76,7 +76,7 @@ class RecipeGlobalRateLimit extends RecipeSpec {
"work" in { "work" in {
//#global-limiter-flow //#global-limiter-flow
def limitGlobal[T](limiter: ActorRef, maxAllowedWait: FiniteDuration): Flow[T, T] = { def limitGlobal[T](limiter: ActorRef, maxAllowedWait: FiniteDuration): Flow[T, T, Unit] = {
import akka.pattern.ask import akka.pattern.ask
import akka.util.Timeout import akka.util.Timeout
Flow[T].mapAsync { (element: T) => Flow[T].mapAsync { (element: T) =>
@ -97,9 +97,9 @@ class RecipeGlobalRateLimit extends RecipeSpec {
val probe = SubscriberProbe[String]() val probe = SubscriberProbe[String]()
FlowGraph { implicit b => FlowGraph.closed() { implicit b =>
import FlowGraphImplicits._ import FlowGraph.Implicits._
val merge = Merge[String] val merge = b.add(Merge[String](2))
source1 ~> merge ~> Sink(probe) source1 ~> merge ~> Sink(probe)
source2 ~> merge source2 ~> merge
}.run() }.run()

View file

@ -23,17 +23,16 @@ class RecipeKeepAlive extends RecipeSpec {
val sink = Sink(sub) val sink = Sink(sub)
//#inject-keepalive //#inject-keepalive
val keepAliveStream: Source[ByteString] = ticks val keepAliveStream: Source[ByteString, Unit] = ticks
.conflate(seed = (tick) => keepaliveMessage)((msg, newTick) => msg) .conflate(seed = (tick) => keepaliveMessage)((msg, newTick) => msg)
import FlowGraphImplicits._ val graph = FlowGraph.closed() { implicit builder =>
val graph = FlowGraph { implicit builder => import FlowGraph.Implicits._
val unfairMerge = MergePreferred[ByteString] val unfairMerge = builder.add(MergePreferred[ByteString](1))
dataStream ~> unfairMerge.preferred // If data is available then no keepalive is injected dataStream ~> unfairMerge.preferred
keepAliveStream ~> unfairMerge // If data is available then no keepalive is injected
keepAliveStream ~> unfairMerge ~> sink
unfairMerge ~> sink
} }
//#inject-keepalive //#inject-keepalive

View file

@ -18,11 +18,11 @@ class RecipeManualTrigger extends RecipeSpec {
val sink = Sink(sub) val sink = Sink(sub)
//#manually-triggered-stream //#manually-triggered-stream
import FlowGraphImplicits._ val graph = FlowGraph.closed() { implicit builder =>
val graph = FlowGraph { implicit builder => import FlowGraph.Implicits._
val zip = Zip[Message, Trigger] val zip = builder.add(Zip[Message, Trigger]())
elements ~> zip.left elements ~> zip.in0
triggerSource ~> zip.right triggerSource ~> zip.in1
zip.out ~> Flow[(Message, Trigger)].map { case (msg, trigger) => msg } ~> sink zip.out ~> Flow[(Message, Trigger)].map { case (msg, trigger) => msg } ~> sink
} }
//#manually-triggered-stream //#manually-triggered-stream
@ -57,13 +57,12 @@ class RecipeManualTrigger extends RecipeSpec {
val sink = Sink(sub) val sink = Sink(sub)
//#manually-triggered-stream-zipwith //#manually-triggered-stream-zipwith
import FlowGraphImplicits._ val graph = FlowGraph.closed() { implicit builder =>
val graph = FlowGraph { implicit builder => import FlowGraph.Implicits._
val zip = ZipWith[Message, Trigger, Message]( val zip = builder.add(ZipWith((msg: Message, trigger: Trigger) => msg))
(msg: Message, trigger: Trigger) => msg)
elements ~> zip.left elements ~> zip.in0
triggerSource ~> zip.right triggerSource ~> zip.in1
zip.out ~> sink zip.out ~> sink
} }
//#manually-triggered-stream-zipwith //#manually-triggered-stream-zipwith

View file

@ -20,7 +20,7 @@ class RecipeMissedTicks extends RecipeSpec {
//#missed-ticks //#missed-ticks
// tickStream is a Source[Tick] // tickStream is a Source[Tick]
val missedTicks: Source[Int] = val missedTicks: Source[Int, Unit] =
tickStream.conflate(seed = (_) => 0)( tickStream.conflate(seed = (_) => 0)(
(missedTicks, tick) => missedTicks + 1) (missedTicks, tick) => missedTicks + 1)
//#missed-ticks //#missed-ticks

View file

@ -15,7 +15,7 @@ class RecipeMultiGroupBy extends RecipeSpec {
case class Topic(name: String) case class Topic(name: String)
val elems = Source(List("1: a", "1: b", "all: c", "all: d", "1: e")) val elems = Source(List("1: a", "1: b", "all: c", "all: d", "1: e"))
val topicMapper: (Message) => immutable.Seq[Topic] = { msg => val topicMapper = { msg: Message =>
if (msg.startsWith("1")) List(Topic("1")) if (msg.startsWith("1")) List(Topic("1"))
else List(Topic("1"), Topic("2")) else List(Topic("1"), Topic("2"))
} }
@ -28,14 +28,14 @@ class RecipeMultiGroupBy extends RecipeSpec {
} }
//#multi-groupby //#multi-groupby
val messageAndTopic: Source[(Message, Topic)] = elems.mapConcat { msg: Message => val messageAndTopic: Source[(Message, Topic), Unit] = elems.mapConcat { msg: Message =>
val topicsForMessage = topicMapper(msg) val topicsForMessage = topicMapper(msg)
// Create a (Msg, Topic) pair for each of the topics // Create a (Msg, Topic) pair for each of the topics
// the message belongs to // the message belongs to
topicsForMessage.map(msg -> _) topicsForMessage.map(msg -> _)
} }
val multiGroups: Source[(Topic, Source[String])] = messageAndTopic.groupBy(_._2).map { val multiGroups: Source[(Topic, Source[String, Unit]), Unit] = messageAndTopic.groupBy(_._2).map {
case (topic, topicStream) => case (topic, topicStream) =>
// chopping of the topic from the (Message, Topic) pairs // chopping of the topic from the (Message, Topic) pairs
(topic, topicStream.map(_._1)) (topic, topicStream.map(_._1))
@ -43,8 +43,8 @@ class RecipeMultiGroupBy extends RecipeSpec {
//#multi-groupby //#multi-groupby
val result = multiGroups.map { val result = multiGroups.map {
case (topic, topicMessages) => topicMessages.grouped(10).map(topic.name + _.mkString("[", ", ", "]")).runWith(Sink.head) case (topic, topicMessages) => topicMessages.grouped(10).map(topic.name + _.mkString("[", ", ", "]")).runWith(Sink.head())
}.mapAsync(identity).grouped(10).runWith(Sink.head) }.mapAsync(identity).grouped(10).runWith(Sink.head())
Await.result(result, 3.seconds).toSet should be(Set( Await.result(result, 3.seconds).toSet should be(Set(
"1[1: a, 1: b, all: c, all: d, 1: e]", "1[1: a, 1: b, all: c, all: d, 1: e]",

View file

@ -24,7 +24,7 @@ class RecipeParseLines extends RecipeSpec {
val linesStream = rawData.transform(() => parseLines("\r\n", 100)) val linesStream = rawData.transform(() => parseLines("\r\n", 100))
Await.result(linesStream.grouped(10).runWith(Sink.head), 3.seconds) should be(List( Await.result(linesStream.grouped(10).runWith(Sink.head()), 3.seconds) should be(List(
"Hello World\r!", "Hello World\r!",
"Hello Akka!", "Hello Akka!",
"Hello Streams!", "Hello Streams!",

View file

@ -18,10 +18,10 @@ class RecipeReduceByKey extends RecipeSpec {
//#word-count //#word-count
// split the words into separate streams first // split the words into separate streams first
val wordStreams: Source[(String, Source[String])] = words.groupBy(identity) val wordStreams: Source[(String, Source[String, Unit]), Unit] = words.groupBy(identity)
// add counting logic to the streams // add counting logic to the streams
val countedWords: Source[Future[(String, Int)]] = wordStreams.map { val countedWords: Source[Future[(String, Int)], Unit] = wordStreams.map {
case (word, wordStream) => case (word, wordStream) =>
wordStream.runFold((word, 0)) { wordStream.runFold((word, 0)) {
case ((w, count), _) => (w, count + 1) case ((w, count), _) => (w, count + 1)
@ -29,13 +29,13 @@ class RecipeReduceByKey extends RecipeSpec {
} }
// get a stream of word counts // get a stream of word counts
val counts: Source[(String, Int)] = val counts: Source[(String, Int), Unit] =
countedWords countedWords
.buffer(MaximumDistinctWords, OverflowStrategy.fail) .buffer(MaximumDistinctWords, OverflowStrategy.fail)
.mapAsync(identity) .mapAsync(identity)
//#word-count //#word-count
Await.result(counts.grouped(10).runWith(Sink.head), 3.seconds).toSet should be(Set( Await.result(counts.grouped(10).runWith(Sink.head()), 3.seconds).toSet should be(Set(
("hello", 2), ("hello", 2),
("world", 1), ("world", 1),
("and", 1), ("and", 1),
@ -52,7 +52,7 @@ class RecipeReduceByKey extends RecipeSpec {
def reduceByKey[In, K, Out]( def reduceByKey[In, K, Out](
maximumGroupSize: Int, maximumGroupSize: Int,
groupKey: (In) => K, groupKey: (In) => K,
foldZero: (K) => Out)(fold: (Out, In) => Out): Flow[In, (K, Out)] = { foldZero: (K) => Out)(fold: (Out, In) => Out): Flow[In, (K, Out), Unit] = {
val groupStreams = Flow[In].groupBy(groupKey) val groupStreams = Flow[In].groupBy(groupKey)
val reducedValues = groupStreams.map { val reducedValues = groupStreams.map {
@ -72,7 +72,7 @@ class RecipeReduceByKey extends RecipeSpec {
//#reduce-by-key-general //#reduce-by-key-general
Await.result(wordCounts.grouped(10).runWith(Sink.head), 3.seconds).toSet should be(Set( Await.result(wordCounts.grouped(10).runWith(Sink.head()), 3.seconds).toSet should be(Set(
("hello", 2), ("hello", 2),
("world", 1), ("world", 1),
("and", 1), ("and", 1),

View file

@ -13,7 +13,7 @@ class RecipeSimpleDrop extends RecipeSpec {
"work" in { "work" in {
//#simple-drop //#simple-drop
val droppyStream: Flow[Message, Message] = val droppyStream: Flow[Message, Message, Unit] =
Flow[Message].conflate(seed = identity)((lastMessage, newMessage) => newMessage) Flow[Message].conflate(seed = identity)((lastMessage, newMessage) => newMessage)
//#simple-drop //#simple-drop

View file

@ -16,7 +16,7 @@ class RecipeToStrict extends RecipeSpec {
//#draining-to-seq //#draining-to-seq
val strict: Future[immutable.Seq[Message]] = val strict: Future[immutable.Seq[Message]] =
myData.grouped(MaxAllowedSeqSize).runWith(Sink.head) myData.grouped(MaxAllowedSeqSize).runWith(Sink.head())
//#draining-to-seq //#draining-to-seq
Await.result(strict, 3.seconds) should be(List("1", "2", "3")) Await.result(strict, 3.seconds) should be(List("1", "2", "3"))

View file

@ -17,18 +17,12 @@ class RecipeWorkerPool extends RecipeSpec {
val worker = Flow[String].map(_ + " done") val worker = Flow[String].map(_ + " done")
//#worker-pool //#worker-pool
def balancer[In, Out](worker: Flow[In, Out], workerCount: Int): Flow[In, Out] = { def balancer[In, Out](worker: Flow[In, Out, Unit], workerCount: Int): Flow[In, Out, Unit] = {
import FlowGraphImplicits._ import FlowGraph.Implicits._
Flow[In, Out]() { implicit graphBuilder => Flow() { implicit b =>
val jobsIn = UndefinedSource[In] val balancer = b.add(Balance[In](workerCount, waitForAllDownstreams = true))
val resultsOut = UndefinedSink[Out] val merge = b.add(Merge[Out](workerCount))
val balancer = Balance[In](waitForAllDownstreams = true)
val merge = Merge[Out]
jobsIn ~> balancer // Jobs are fed into the balancer
merge ~> resultsOut // the merged results are sent out
for (_ <- 1 to workerCount) { for (_ <- 1 to workerCount) {
// for each worker, add an edge from the balancer to the worker, then wire // for each worker, add an edge from the balancer to the worker, then wire
@ -36,14 +30,14 @@ class RecipeWorkerPool extends RecipeSpec {
balancer ~> worker ~> merge balancer ~> worker ~> merge
} }
(jobsIn, resultsOut) (balancer.in, merge.out)
} }
} }
val processedJobs: Source[Result] = myJobs.via(balancer(worker, 3)) val processedJobs: Source[Result, Unit] = myJobs.via(balancer(worker, 3))
//#worker-pool //#worker-pool
Await.result(processedJobs.grouped(10).runWith(Sink.head), 3.seconds).toSet should be(Set( Await.result(processedJobs.grouped(10).runWith(Sink.head()), 3.seconds).toSet should be(Set(
"1 done", "2 done", "3 done", "4 done", "5 done")) "1 done", "2 done", "3 done", "4 done", "5 done"))
} }

View file

@ -42,19 +42,19 @@ public final class HttpEntities {
return HttpEntity$.MODULE$.apply((akka.http.model.ContentType) contentType, file); return HttpEntity$.MODULE$.apply((akka.http.model.ContentType) contentType, file);
} }
public static HttpEntityDefault create(ContentType contentType, long contentLength, Source<ByteString> data) { public static HttpEntityDefault create(ContentType contentType, long contentLength, Source<ByteString, scala.runtime.BoxedUnit> data) {
return new akka.http.model.HttpEntity.Default((akka.http.model.ContentType) contentType, contentLength, data); return new akka.http.model.HttpEntity.Default((akka.http.model.ContentType) contentType, contentLength, data);
} }
public static HttpEntityCloseDelimited createCloseDelimited(ContentType contentType, Source<ByteString> data) { public static HttpEntityCloseDelimited createCloseDelimited(ContentType contentType, Source<ByteString, scala.runtime.BoxedUnit> data) {
return new akka.http.model.HttpEntity.CloseDelimited((akka.http.model.ContentType) contentType, data); return new akka.http.model.HttpEntity.CloseDelimited((akka.http.model.ContentType) contentType, data);
} }
public static HttpEntityIndefiniteLength createIndefiniteLength(ContentType contentType, Source<ByteString> data) { public static HttpEntityIndefiniteLength createIndefiniteLength(ContentType contentType, Source<ByteString, scala.runtime.BoxedUnit> data) {
return new akka.http.model.HttpEntity.IndefiniteLength((akka.http.model.ContentType) contentType, data); return new akka.http.model.HttpEntity.IndefiniteLength((akka.http.model.ContentType) contentType, data);
} }
public static HttpEntityChunked createChunked(ContentType contentType, Source<ByteString> data) { public static HttpEntityChunked createChunked(ContentType contentType, Source<ByteString, scala.runtime.BoxedUnit> data) {
return akka.http.model.HttpEntity.Chunked$.MODULE$.fromData( return akka.http.model.HttpEntity.Chunked$.MODULE$.fromData(
(akka.http.model.ContentType) contentType, (akka.http.model.ContentType) contentType,
data); data);

View file

@ -73,5 +73,5 @@ public interface HttpEntity {
/** /**
* Returns a stream of data bytes this entity consists of. * Returns a stream of data bytes this entity consists of.
*/ */
public abstract Source<ByteString> getDataBytes(); public abstract Source<ByteString, scala.Unit> getDataBytes();
} }

View file

@ -11,5 +11,5 @@ import akka.stream.scaladsl.Source;
* stream of {@link ChunkStreamPart}. * stream of {@link ChunkStreamPart}.
*/ */
public abstract class HttpEntityChunked implements RequestEntity, ResponseEntity { public abstract class HttpEntityChunked implements RequestEntity, ResponseEntity {
public abstract Source<ChunkStreamPart> getChunks(); public abstract Source<ChunkStreamPart, scala.Unit> getChunks();
} }

View file

@ -13,5 +13,5 @@ import akka.stream.scaladsl.Source;
* available for Http responses. * available for Http responses.
*/ */
public abstract class HttpEntityCloseDelimited implements ResponseEntity { public abstract class HttpEntityCloseDelimited implements ResponseEntity {
public abstract Source<ByteString> data(); public abstract Source<ByteString, scala.Unit> data();
} }

View file

@ -12,5 +12,5 @@ import akka.stream.scaladsl.Source;
*/ */
public abstract class HttpEntityDefault implements BodyPartEntity, RequestEntity, ResponseEntity { public abstract class HttpEntityDefault implements BodyPartEntity, RequestEntity, ResponseEntity {
public abstract long contentLength(); public abstract long contentLength();
public abstract Source<ByteString> data(); public abstract Source<ByteString, scala.Unit> data();
} }

View file

@ -11,5 +11,5 @@ import akka.stream.scaladsl.Source;
* Represents an entity without a predetermined content-length to use in a BodyParts. * Represents an entity without a predetermined content-length to use in a BodyParts.
*/ */
public abstract class HttpEntityIndefiniteLength implements BodyPartEntity { public abstract class HttpEntityIndefiniteLength implements BodyPartEntity {
public abstract Source<ByteString> data(); public abstract Source<ByteString, scala.Unit> data();
} }

View file

@ -27,12 +27,12 @@ public abstract class Util {
@SuppressWarnings("unchecked") // no support for covariance of Publisher in Java @SuppressWarnings("unchecked") // no support for covariance of Publisher in Java
// needed to provide covariant conversions that the Java interfaces don't provide automatically. // needed to provide covariant conversions that the Java interfaces don't provide automatically.
// The alternative would be having to cast around everywhere instead of doing it here in a central place. // The alternative would be having to cast around everywhere instead of doing it here in a central place.
public static <U, T extends U> Source<U> convertPublisher(Source<T> p) { public static <U, T extends U> Source<U, scala.Unit> convertPublisher(Source<T, scala.Unit> p) {
return (Source<U>)(Source) p; return (Source<U, scala.Unit>)(Source) p;
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public static <T, U extends T> Source<U> upcastSource(Source<T> p) { public static <T, U extends T> Source<U, scala.Unit> upcastSource(Source<T, scala.Unit> p) {
return (Source<U>)(Source) p; return (Source<U, scala.Unit>)(Source) p;
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public static scala.collection.immutable.Map<String, String> convertMapToScala(Map<String, String> map) { public static scala.collection.immutable.Map<String, String> convertMapToScala(Map<String, String> map) {

View file

@ -5,13 +5,15 @@
package akka.http package akka.http
import java.net.InetSocketAddress import java.net.InetSocketAddress
import akka.http.engine.server.HttpServer.HttpServerPorts
import akka.stream.Graph
import com.typesafe.config.Config import com.typesafe.config.Config
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.Future import scala.concurrent.Future
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.util.ByteString import akka.util.ByteString
import akka.io.Inet import akka.io.Inet
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.http.engine.client.{ HttpClient, ClientConnectionSettings } import akka.http.engine.client.{ HttpClient, ClientConnectionSettings }
import akka.http.engine.server.{ HttpServer, ServerSettings } import akka.http.engine.server.{ HttpServer, ServerSettings }
@ -27,32 +29,93 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
def bind(interface: String, port: Int = 80, backlog: Int = 100, def bind(interface: String, port: Int = 80, backlog: Int = 100,
options: immutable.Traversable[Inet.SocketOption] = Nil, options: immutable.Traversable[Inet.SocketOption] = Nil,
settings: Option[ServerSettings] = None, settings: Option[ServerSettings] = None,
log: LoggingAdapter = system.log): ServerBinding = { log: LoggingAdapter = system.log)(implicit fm: ActorFlowMaterializer): Source[IncomingConnection, Future[ServerBinding]] = {
val endpoint = new InetSocketAddress(interface, port) val endpoint = new InetSocketAddress(interface, port)
val effectiveSettings = ServerSettings(settings) val effectiveSettings = ServerSettings(settings)
val tcpBinding = StreamTcp().bind(endpoint, backlog, options, effectiveSettings.timeouts.idleTimeout)
new ServerBinding { val connections: Source[StreamTcp.IncomingConnection, Future[StreamTcp.ServerBinding]] = StreamTcp().bind(endpoint, backlog, options, effectiveSettings.timeouts.idleTimeout)
def localAddress(mm: MaterializedMap): Future[InetSocketAddress] = tcpBinding.localAddress(mm) val serverBlueprint: Graph[HttpServerPorts, Unit] = HttpServer.serverBlueprint(effectiveSettings, log)
val connections = tcpBinding.connections map { tcpConn
new IncomingConnection { connections.map { conn
def localAddress = tcpConn.localAddress val flow = Flow(conn.flow, serverBlueprint)(Keep.right) { implicit b
def remoteAddress = tcpConn.remoteAddress (tcp, http)
def handleWith(handler: Flow[HttpRequest, HttpResponse])(implicit fm: FlowMaterializer) = import FlowGraph.Implicits._
tcpConn.handleWith(HttpServer.serverFlowToTransport(handler, effectiveSettings, log)) tcp.outlet ~> http.bytesIn
http.bytesOut ~> tcp.inlet
(http.httpResponses, http.httpRequests)
} }
IncomingConnection(conn.localAddress, conn.remoteAddress, flow)
}.mapMaterialized { tcpBindingFuture
import system.dispatcher
tcpBindingFuture.map { tcpBinding ServerBinding(tcpBinding.localAddress)(() tcpBinding.unbind()) }
} }
def unbind(mm: MaterializedMap): Future[Unit] = tcpBinding.unbind(mm)
} }
/**
* Materializes the `connections` [[Source]] and handles all connections with the given flow.
*
* Note that there is no backpressure being applied to the `connections` [[Source]], i.e. all
* connections are being accepted at maximum rate, which, depending on the applications, might
* present a DoS risk!
*/
def bindAndstartHandlingWith(handler: Flow[HttpRequest, HttpResponse, _],
interface: String, port: Int = 80, backlog: Int = 100,
options: immutable.Traversable[Inet.SocketOption] = Nil,
settings: Option[ServerSettings] = None,
log: LoggingAdapter = system.log)(implicit fm: ActorFlowMaterializer): Future[ServerBinding] = {
bind(interface, port, backlog, options, settings, log).toMat(Sink.foreach { conn
conn.flow.join(handler)
})(Keep.left).run()
} }
/**
* Materializes the `connections` [[Source]] and handles all connections with the given flow.
*
* Note that there is no backpressure being applied to the `connections` [[Source]], i.e. all
* connections are being accepted at maximum rate, which, depending on the applications, might
* present a DoS risk!
*/
def bindAndStartHandlingWithSyncHandler(handler: HttpRequest HttpResponse,
interface: String, port: Int = 80, backlog: Int = 100,
options: immutable.Traversable[Inet.SocketOption] = Nil,
settings: Option[ServerSettings] = None,
log: LoggingAdapter = system.log)(implicit fm: ActorFlowMaterializer): Future[ServerBinding] =
bindAndstartHandlingWith(Flow[HttpRequest].map(handler), interface, port, backlog, options, settings, log)
/**
* Materializes the `connections` [[Source]] and handles all connections with the given flow.
*
* Note that there is no backpressure being applied to the `connections` [[Source]], i.e. all
* connections are being accepted at maximum rate, which, depending on the applications, might
* present a DoS risk!
*/
def startHandlingWithAsyncHandler(handler: HttpRequest Future[HttpResponse],
interface: String, port: Int = 80, backlog: Int = 100,
options: immutable.Traversable[Inet.SocketOption] = Nil,
settings: Option[ServerSettings] = None,
log: LoggingAdapter = system.log)(implicit fm: ActorFlowMaterializer): Future[ServerBinding] =
bindAndstartHandlingWith(Flow[HttpRequest].mapAsync(handler), interface, port, backlog, options, settings, log)
/** /**
* Transforms a given HTTP-level server [[Flow]] into a lower-level TCP transport flow. * Transforms a given HTTP-level server [[Flow]] into a lower-level TCP transport flow.
*/ */
def serverFlowToTransport(serverFlow: Flow[HttpRequest, HttpResponse], def serverFlowToTransport[Mat](serverFlow: Flow[HttpRequest, HttpResponse, Mat],
settings: Option[ServerSettings] = None, settings: Option[ServerSettings] = None,
log: LoggingAdapter = system.log)(implicit mat: FlowMaterializer): Flow[ByteString, ByteString] = { log: LoggingAdapter = system.log)(implicit mat: ActorFlowMaterializer): Flow[ByteString, ByteString, Mat] = {
val effectiveSettings = ServerSettings(settings) val effectiveSettings = ServerSettings(settings)
HttpServer.serverFlowToTransport(serverFlow, effectiveSettings, log) val serverBlueprint: Graph[HttpServerPorts, Unit] = HttpServer.serverBlueprint(effectiveSettings, log)
Flow(serverBlueprint, serverFlow)(Keep.right) { implicit b
(server, user)
import FlowGraph.Implicits._
server.httpRequests ~> user.inlet
user.outlet ~> server.httpResponses
(server.bytesIn, server.bytesOut)
}
} }
/** /**
@ -62,27 +125,46 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
localAddress: Option[InetSocketAddress] = None, localAddress: Option[InetSocketAddress] = None,
options: immutable.Traversable[Inet.SocketOption] = Nil, options: immutable.Traversable[Inet.SocketOption] = Nil,
settings: Option[ClientConnectionSettings] = None, settings: Option[ClientConnectionSettings] = None,
log: LoggingAdapter = system.log): OutgoingConnection = { log: LoggingAdapter = system.log): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] = {
val effectiveSettings = ClientConnectionSettings(settings) val effectiveSettings = ClientConnectionSettings(settings)
val remoteAddr = new InetSocketAddress(host, port) val remoteAddr = new InetSocketAddress(host, port)
val transportFlow = StreamTcp().outgoingConnection(remoteAddr, localAddress, val transportFlow = StreamTcp().outgoingConnection(remoteAddr, localAddress,
options, effectiveSettings.connectingTimeout, effectiveSettings.idleTimeout) options, effectiveSettings.connectingTimeout, effectiveSettings.idleTimeout)
new OutgoingConnection { val clientBluePrint = HttpClient.clientBlueprint(remoteAddr, effectiveSettings, log)
def remoteAddress = remoteAddr
def localAddress(mm: MaterializedMap) = transportFlow.localAddress(mm) Flow(transportFlow, clientBluePrint)(Keep.left) { implicit b
val flow = HttpClient.transportToConnectionClientFlow(transportFlow.flow, remoteAddr, effectiveSettings, log) (tcp, client)
import FlowGraph.Implicits._
tcp.outlet ~> client.bytesIn
client.bytesOut ~> tcp.inlet
(client.httpRequests, client.httpResponses)
}.mapMaterialized { tcpConnFuture
import system.dispatcher
tcpConnFuture.map { tcpConn OutgoingConnection(tcpConn.localAddress, tcpConn.remoteAddress) }
} }
} }
/** /**
* Transforms the given low-level TCP client transport [[Flow]] into a higher-level HTTP client flow. * Transforms the given low-level TCP client transport [[Flow]] into a higher-level HTTP client flow.
*/ */
def transportToConnectionClientFlow(transport: Flow[ByteString, ByteString], def transportToConnectionClientFlow[Mat](transport: Flow[ByteString, ByteString, Mat],
remoteAddress: InetSocketAddress, // TODO: removed after #16168 is cleared remoteAddress: InetSocketAddress, // TODO: removed after #16168 is cleared
settings: Option[ClientConnectionSettings] = None, settings: Option[ClientConnectionSettings] = None,
log: LoggingAdapter = system.log): Flow[HttpRequest, HttpResponse] = { log: LoggingAdapter = system.log): Flow[HttpRequest, HttpResponse, Mat] = {
val effectiveSettings = ClientConnectionSettings(settings) val effectiveSettings = ClientConnectionSettings(settings)
HttpClient.transportToConnectionClientFlow(transport, remoteAddress, effectiveSettings, log) val clientBlueprint = HttpClient.clientBlueprint(remoteAddress, effectiveSettings, log)
Flow(clientBlueprint, transport)(Keep.right) { implicit b
(client, tcp)
import FlowGraph.Implicits._
client.bytesOut ~> tcp.inlet
tcp.outlet ~> client.bytesIn
(client.httpRequests, client.httpResponses)
}
} }
} }
@ -90,119 +172,57 @@ object Http extends ExtensionId[HttpExt] with ExtensionIdProvider {
/** /**
* Represents a prospective HTTP server binding. * Represents a prospective HTTP server binding.
*
* @param localAddress The local address of the endpoint bound by the materialization of the `connections` [[Source]]
*
*/ */
sealed trait ServerBinding { case class ServerBinding(localAddress: InetSocketAddress)(private val unbindAction: () Future[Unit]) {
/**
* The local address of the endpoint bound by the materialization of the `connections` [[Source]]
* whose [[MaterializedMap]] is passed as parameter.
*/
def localAddress(materializedMap: MaterializedMap): Future[InetSocketAddress]
/**
* The stream of accepted incoming connections.
* Can be materialized several times but only one subscription can be "live" at one time, i.e.
* subsequent materializations will reject subscriptions with an [[StreamTcp.BindFailedException]] if the previous
* materialization still has an uncancelled subscription.
* Cancelling the subscription to a materialization of this source will cause the listening port to be unbound.
*/
def connections: Source[IncomingConnection]
/** /**
* Asynchronously triggers the unbinding of the port that was bound by the materialization of the `connections` * Asynchronously triggers the unbinding of the port that was bound by the materialization of the `connections`
* [[Source]] whose [[MaterializedMap]] is passed as parameter. * [[Source]]
* *
* The produced [[Future]] is fulfilled when the unbinding has been completed. * The produced [[Future]] is fulfilled when the unbinding has been completed.
*/ */
def unbind(materializedMap: MaterializedMap): Future[Unit] def unbind(): Future[Unit] = unbindAction()
/**
* Materializes the `connections` [[Source]] and handles all connections with the given flow.
*
* Note that there is no backpressure being applied to the `connections` [[Source]], i.e. all
* connections are being accepted at maximum rate, which, depending on the applications, might
* present a DoS risk!
*/
def startHandlingWith(handler: Flow[HttpRequest, HttpResponse])(implicit fm: FlowMaterializer): MaterializedMap =
connections.to(ForeachSink(_ handleWith handler)).run()
/**
* Materializes the `connections` [[Source]] and handles all connections with the given flow.
*
* Note that there is no backpressure being applied to the `connections` [[Source]], i.e. all
* connections are being accepted at maximum rate, which, depending on the applications, might
* present a DoS risk!
*/
def startHandlingWithSyncHandler(handler: HttpRequest HttpResponse)(implicit fm: FlowMaterializer): MaterializedMap =
startHandlingWith(Flow[HttpRequest].map(handler))
/**
* Materializes the `connections` [[Source]] and handles all connections with the given flow.
*
* Note that there is no backpressure being applied to the `connections` [[Source]], i.e. all
* connections are being accepted at maximum rate, which, depending on the applications, might
* present a DoS risk!
*/
def startHandlingWithAsyncHandler(handler: HttpRequest Future[HttpResponse])(implicit fm: FlowMaterializer): MaterializedMap =
startHandlingWith(Flow[HttpRequest].mapAsync(handler))
} }
/** /**
* Represents one accepted incoming HTTP connection. * Represents one accepted incoming HTTP connection.
*/ */
sealed trait IncomingConnection { case class IncomingConnection(
/** localAddress: InetSocketAddress,
* The local address this connection is bound to. remoteAddress: InetSocketAddress,
*/ flow: Flow[HttpResponse, HttpRequest, Unit]) {
def localAddress: InetSocketAddress
/**
* The remote address this connection is bound to.
*/
def remoteAddress: InetSocketAddress
/** /**
* Handles the connection with the given flow, which is materialized exactly once * Handles the connection with the given flow, which is materialized exactly once
* and the respective [[MaterializedMap]] returned. * and the respective [[MaterializedMap]] returned.
*/ */
def handleWith(handler: Flow[HttpRequest, HttpResponse])(implicit fm: FlowMaterializer): MaterializedMap def handleWith[Mat](handler: Flow[HttpRequest, HttpResponse, Mat])(implicit fm: ActorFlowMaterializer): Mat =
flow.join(handler).mapMaterialized(_._2).run()
/** /**
* Handles the connection with the given handler function. * Handles the connection with the given handler function.
* Returns the [[MaterializedMap]] of the underlying flow materialization. * Returns the [[MaterializedMap]] of the underlying flow materialization.
*/ */
def handleWithSyncHandler(handler: HttpRequest HttpResponse)(implicit fm: FlowMaterializer): MaterializedMap = def handleWithSyncHandler(handler: HttpRequest HttpResponse)(implicit fm: ActorFlowMaterializer): Unit =
handleWith(Flow[HttpRequest].map(handler)) handleWith(Flow[HttpRequest].map(handler))
/** /**
* Handles the connection with the given handler function. * Handles the connection with the given handler function.
* Returns the [[MaterializedMap]] of the underlying flow materialization. * Returns the [[MaterializedMap]] of the underlying flow materialization.
*/ */
def handleWithAsyncHandler(handler: HttpRequest Future[HttpResponse])(implicit fm: FlowMaterializer): MaterializedMap = def handleWithAsyncHandler(handler: HttpRequest Future[HttpResponse])(implicit fm: ActorFlowMaterializer): Unit =
handleWith(Flow[HttpRequest].mapAsync(handler)) handleWith(Flow[HttpRequest].mapAsync(handler))
} }
/** /**
* Represents a prospective outgoing HTTP connection. * Represents a prospective outgoing HTTP connection.
*/ */
sealed trait OutgoingConnection { case class OutgoingConnection(localAddress: InetSocketAddress, remoteAddress: InetSocketAddress) {
/**
* The remote address this connection is or will be bound to.
*/
def remoteAddress: InetSocketAddress
/**
* The local address of the endpoint bound by the materialization of the connection materialization
* whose [[MaterializedMap]] is passed as parameter.
*/
def localAddress(mMap: MaterializedMap): Future[InetSocketAddress]
/**
* A flow representing the HTTP server on a single HTTP connection.
* This flow can be materialized several times, every materialization will open a new connection to the `remoteAddress`.
* If the connection cannot be established the materialized stream will immediately be terminated
* with a [[akka.stream.StreamTcpException]].
*/
def flow: Flow[HttpRequest, HttpResponse]
} }
//////////////////// EXTENSION SETUP /////////////////// //////////////////// EXTENSION SETUP ///////////////////

View file

@ -5,12 +5,14 @@
package akka.http.engine.client package akka.http.engine.client
import java.net.InetSocketAddress import java.net.InetSocketAddress
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.immutable.Seq
import scala.collection.mutable.ListBuffer import scala.collection.mutable.ListBuffer
import akka.stream.stage._ import akka.stream.stage._
import akka.util.ByteString import akka.util.ByteString
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.stream.FlattenStrategy import akka.stream._
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.scaladsl.OperationAttributes._ import akka.stream.scaladsl.OperationAttributes._
import akka.http.model.{ IllegalResponseException, HttpMethod, HttpRequest, HttpResponse } import akka.http.model.{ IllegalResponseException, HttpMethod, HttpRequest, HttpResponse }
@ -23,10 +25,35 @@ import akka.http.util._
*/ */
private[http] object HttpClient { private[http] object HttpClient {
def transportToConnectionClientFlow(transport: Flow[ByteString, ByteString], case class HttpClientPorts(
remoteAddress: InetSocketAddress, bytesIn: Inlet[ByteString],
bytesOut: Outlet[ByteString],
httpRequests: Inlet[HttpRequest],
httpResponses: Outlet[HttpResponse]) extends Shape {
override val inlets: Seq[Inlet[_]] = bytesIn :: httpRequests :: Nil
override val outlets: Seq[Outlet[_]] = bytesOut :: httpResponses :: Nil
override def deepCopy(): Shape = HttpClientPorts(
new Inlet(bytesIn.toString),
new Outlet(bytesOut.toString),
new Inlet(httpResponses.toString),
new Outlet(httpRequests.toString))
override def copyFromPorts(inlets: Seq[Inlet[_]], outlets: Seq[Outlet[_]]): Shape = {
val bIn :: htpIn :: Nil = inlets
val bOut :: htpOut :: Nil = outlets
HttpClientPorts(
bIn.asInstanceOf[Inlet[ByteString]],
bOut.asInstanceOf[Outlet[ByteString]],
htpIn.asInstanceOf[Inlet[HttpRequest]],
htpOut.asInstanceOf[Outlet[HttpResponse]])
}
}
def clientBlueprint(remoteAddress: InetSocketAddress,
settings: ClientConnectionSettings, settings: ClientConnectionSettings,
log: LoggingAdapter): Flow[HttpRequest, HttpResponse] = { log: LoggingAdapter): Graph[HttpClientPorts, Unit] = {
import settings._ import settings._
// the initial header parser we initially use for every connection, // the initial header parser we initially use for every connection,
@ -57,24 +84,11 @@ private[http] object HttpClient {
+------------+ +------------+
*/ */
val requestIn = UndefinedSource[HttpRequest] val requestRendering: Flow[HttpRequest, ByteString, Unit] = Flow[HttpRequest]
val responseOut = UndefinedSink[HttpResponse]
val methodBypassFanout = Broadcast[HttpRequest]
val responseParsingMerge = new ResponseParsingMerge(rootParser)
val terminationFanout = Broadcast[HttpResponse]
val terminationMerge = new TerminationMerge
val requestRendering = Flow[HttpRequest]
.map(RequestRenderingContext(_, remoteAddress)) .map(RequestRenderingContext(_, remoteAddress))
.section(name("renderer"))(_.transform(() requestRendererFactory.newRenderer)) .section(name("renderer"))(_.transform(() requestRendererFactory.newRenderer))
.flatten(FlattenStrategy.concat) .flatten(FlattenStrategy.concat)
val transportFlow = Flow[ByteString]
.section(name("errorLogger"))(_.transform(() errorLogger(log, "Outgoing request stream error")))
.via(transport)
val methodBypass = Flow[HttpRequest].map(_.method) val methodBypass = Flow[HttpRequest].map(_.method)
import ParserOutput._ import ParserOutput._
@ -89,34 +103,42 @@ private[http] object HttpClient {
case (MessageStartError(_, info), _) throw IllegalResponseException(info) case (MessageStartError(_, info), _) throw IllegalResponseException(info)
} }
import FlowGraphImplicits._ FlowGraph.partial() { implicit b
import FlowGraph.Implicits._
val methodBypassFanout = b.add(Broadcast[HttpRequest](2))
val responseParsingMerge = b.add(new ResponseParsingMerge(rootParser))
Flow() { implicit b val terminationFanout = b.add(Broadcast[HttpResponse](2))
requestIn ~> methodBypassFanout ~> terminationMerge.requestInput ~> requestRendering ~> transportFlow ~> val terminationMerge = b.add(new TerminationMerge)
responseParsingMerge.dataInput ~> responsePrep ~> terminationFanout ~> responseOut
methodBypassFanout ~> methodBypass ~> responseParsingMerge.methodBypassInput
terminationFanout ~> terminationMerge.terminationBackchannelInput
b.allowCycles() val bytesOut = (terminationMerge.out ~>
requestRendering.section(name("errorLogger"))(_.transform(() errorLogger(log, "Outgoing request stream error")))).outlet
requestIn -> responseOut val bytesIn = responseParsingMerge.in0
methodBypassFanout.out(0) ~> terminationMerge.in0
methodBypassFanout.out(1) ~> methodBypass ~> responseParsingMerge.in1
responseParsingMerge.out ~> responsePrep ~> terminationFanout.in
terminationFanout.out(0) ~> terminationMerge.in1
HttpClientPorts(bytesIn, bytesOut, methodBypassFanout.in, terminationFanout.out(1))
} }
} }
// a simple merge stage that simply forwards its first input and ignores its second input // a simple merge stage that simply forwards its first input and ignores its second input
// (the terminationBackchannelInput), but applies a special completion handling // (the terminationBackchannelInput), but applies a special completion handling
class TerminationMerge extends FlexiMerge[HttpRequest] { class TerminationMerge
extends FlexiMerge[HttpRequest, FanInShape2[HttpRequest, HttpResponse, HttpRequest]](new FanInShape2("TerminationMerge"), OperationAttributes.name("TerminationMerge")) {
import FlexiMerge._ import FlexiMerge._
val requestInput = createInputPort[HttpRequest]()
val terminationBackchannelInput = createInputPort[HttpResponse]()
def createMergeLogic() = new MergeLogic[HttpRequest] { def createMergeLogic(p: PortT) = new MergeLogic[HttpRequest] {
override def inputHandles(inputCount: Int) = {
require(inputCount == 2, s"TerminationMerge must have 2 connected inputs, was $inputCount")
Vector(requestInput, terminationBackchannelInput)
}
override def initialState = State[Any](ReadAny(requestInput, terminationBackchannelInput)) { val requestInput = p.in0
val terminationBackchannelInput = p.in1
override def initialState = State[Any](ReadAny(p)) {
case (ctx, _, request: HttpRequest) { ctx.emit(request); SameState } case (ctx, _, request: HttpRequest) { ctx.emit(request); SameState }
case _ SameState // simply drop all responses, we are only interested in the completion of the response input case _ SameState // simply drop all responses, we are only interested in the completion of the response input
} }
@ -140,22 +162,17 @@ private[http] object HttpClient {
* 2. Read from the dataInput until exactly one response has been fully received * 2. Read from the dataInput until exactly one response has been fully received
* 3. Go back to 1. * 3. Go back to 1.
*/ */
class ResponseParsingMerge(rootParser: HttpResponseParser) extends FlexiMerge[List[ResponseOutput]] { class ResponseParsingMerge(rootParser: HttpResponseParser)
extends FlexiMerge[List[ResponseOutput], FanInShape2[ByteString, HttpMethod, List[ResponseOutput]]](new FanInShape2("ResponseParsingMerge"), OperationAttributes.name("ResponsePersingMerge")) {
import FlexiMerge._ import FlexiMerge._
val dataInput = createInputPort[ByteString]()
val methodBypassInput = createInputPort[HttpMethod]()
def createMergeLogic() = new MergeLogic[List[ResponseOutput]] { def createMergeLogic(p: PortT) = new MergeLogic[List[ResponseOutput]] {
val dataInput = p.in0
val methodBypassInput = p.in1
// each connection uses a single (private) response parser instance for all its responses // each connection uses a single (private) response parser instance for all its responses
// which builds a cache of all header instances seen on that connection // which builds a cache of all header instances seen on that connection
val parser = rootParser.createShallowCopy() val parser = rootParser.createShallowCopy()
var methodBypassCompleted = false var methodBypassCompleted = false
override def inputHandles(inputCount: Int) = {
require(inputCount == 2, s"ResponseParsingMerge must have 2 connected inputs, was $inputCount")
Vector(dataInput, methodBypassInput)
}
private val stay = (ctx: MergeLogicContext) SameState private val stay = (ctx: MergeLogicContext) SameState
private val gotoResponseReading = (ctx: MergeLogicContext) { private val gotoResponseReading = (ctx: MergeLogicContext) {
ctx.changeCompletionHandling(responseReadingCompletionHandling) ctx.changeCompletionHandling(responseReadingCompletionHandling)

View file

@ -254,7 +254,7 @@ private[http] object BodyPartParser {
val boundaryCharNoSpace = CharPredicate.Digit ++ CharPredicate.Alpha ++ "'()+_,-./:=?" val boundaryCharNoSpace = CharPredicate.Digit ++ CharPredicate.Alpha ++ "'()+_,-./:=?"
sealed trait Output sealed trait Output
final case class BodyPartStart(headers: List[HttpHeader], createEntity: Source[Output] BodyPartEntity) extends Output final case class BodyPartStart(headers: List[HttpHeader], createEntity: Source[Output, Unit] BodyPartEntity) extends Output
final case class EntityPart(data: ByteString) extends Output final case class EntityPart(data: ByteString) extends Output
final case class ParseError(info: ErrorInfo) extends Output final case class ParseError(info: ErrorInfo) extends Output

View file

@ -305,7 +305,7 @@ private[http] abstract class HttpMessageParser[Output >: MessageOutput <: Parser
def defaultEntity(cth: Option[`Content-Type`], def defaultEntity(cth: Option[`Content-Type`],
contentLength: Long, contentLength: Long,
transformData: Source[ByteString] Source[ByteString] = identityFunc)(entityParts: Source[_ <: ParserOutput]): UniversalEntity = { transformData: Source[ByteString, Unit] Source[ByteString, Unit] = identityFunc)(entityParts: Source[_ <: ParserOutput, Unit]): UniversalEntity = {
val data = entityParts.collect { val data = entityParts.collect {
case EntityPart(bytes) bytes case EntityPart(bytes) bytes
case EntityStreamError(info) throw EntityStreamException(info) case EntityStreamError(info) throw EntityStreamException(info)
@ -314,7 +314,7 @@ private[http] abstract class HttpMessageParser[Output >: MessageOutput <: Parser
} }
def chunkedEntity(cth: Option[`Content-Type`], def chunkedEntity(cth: Option[`Content-Type`],
transformChunks: Source[HttpEntity.ChunkStreamPart] Source[HttpEntity.ChunkStreamPart] = identityFunc)(entityChunks: Source[_ <: ParserOutput]): RequestEntity = { transformChunks: Source[HttpEntity.ChunkStreamPart, Unit] Source[HttpEntity.ChunkStreamPart, Unit] = identityFunc)(entityChunks: Source[_ <: ParserOutput, Unit]): RequestEntity = {
val chunks = entityChunks.collect { val chunks = entityChunks.collect {
case EntityChunk(chunk) chunk case EntityChunk(chunk) chunk
case EntityStreamError(info) throw EntityStreamException(info) case EntityStreamError(info) throw EntityStreamException(info)

View file

@ -118,7 +118,7 @@ private[http] class HttpRequestParser(_settings: ParserSettings,
clh: Option[`Content-Length`], cth: Option[`Content-Type`], teh: Option[`Transfer-Encoding`], clh: Option[`Content-Length`], cth: Option[`Content-Type`], teh: Option[`Transfer-Encoding`],
expect100continue: Boolean, hostHeaderPresent: Boolean, closeAfterResponseCompletion: Boolean): StateResult = expect100continue: Boolean, hostHeaderPresent: Boolean, closeAfterResponseCompletion: Boolean): StateResult =
if (hostHeaderPresent || protocol == HttpProtocols.`HTTP/1.0`) { if (hostHeaderPresent || protocol == HttpProtocols.`HTTP/1.0`) {
def emitRequestStart(createEntity: Source[RequestOutput] RequestEntity, def emitRequestStart(createEntity: Source[RequestOutput, Unit] RequestEntity,
headers: List[HttpHeader] = headers) = { headers: List[HttpHeader] = headers) = {
val allHeaders = val allHeaders =
if (rawRequestUriHeader) `Raw-Request-URI`(new String(uriBytes, HttpCharsets.`US-ASCII`.nioCharset)) :: headers if (rawRequestUriHeader) `Raw-Request-URI`(new String(uriBytes, HttpCharsets.`US-ASCII`.nioCharset)) :: headers
@ -126,7 +126,7 @@ private[http] class HttpRequestParser(_settings: ParserSettings,
emit(RequestStart(method, uri, protocol, allHeaders, createEntity, expect100continue, closeAfterResponseCompletion)) emit(RequestStart(method, uri, protocol, allHeaders, createEntity, expect100continue, closeAfterResponseCompletion))
} }
def expect100continueHandling[T]: Source[T] Source[T] = def expect100continueHandling[T]: Source[T, Unit] Source[T, Unit] =
if (expect100continue) { if (expect100continue) {
_.section(name("expect100continueTrigger"))(_.transform(() new PushPullStage[T, T] { _.section(name("expect100continueTrigger"))(_.transform(() new PushPullStage[T, T] {
private var oneHundredContinueSent = false private var oneHundredContinueSent = false

View file

@ -78,7 +78,7 @@ private[http] class HttpResponseParser(_settings: ParserSettings, _headerParser:
def parseEntity(headers: List[HttpHeader], protocol: HttpProtocol, input: ByteString, bodyStart: Int, def parseEntity(headers: List[HttpHeader], protocol: HttpProtocol, input: ByteString, bodyStart: Int,
clh: Option[`Content-Length`], cth: Option[`Content-Type`], teh: Option[`Transfer-Encoding`], clh: Option[`Content-Length`], cth: Option[`Content-Type`], teh: Option[`Transfer-Encoding`],
expect100continue: Boolean, hostHeaderPresent: Boolean, closeAfterResponseCompletion: Boolean): StateResult = { expect100continue: Boolean, hostHeaderPresent: Boolean, closeAfterResponseCompletion: Boolean): StateResult = {
def emitResponseStart(createEntity: Source[ResponseOutput] ResponseEntity, def emitResponseStart(createEntity: Source[ResponseOutput, Unit] ResponseEntity,
headers: List[HttpHeader] = headers) = headers: List[HttpHeader] = headers) =
emit(ResponseStart(statusCode, protocol, headers, createEntity, closeAfterResponseCompletion)) emit(ResponseStart(statusCode, protocol, headers, createEntity, closeAfterResponseCompletion))
def finishEmptyResponse() = { def finishEmptyResponse() = {

View file

@ -28,7 +28,7 @@ private[http] object ParserOutput {
uri: Uri, uri: Uri,
protocol: HttpProtocol, protocol: HttpProtocol,
headers: List[HttpHeader], headers: List[HttpHeader],
createEntity: Source[RequestOutput] RequestEntity, createEntity: Source[RequestOutput, Unit] RequestEntity,
expect100ContinueResponsePending: Boolean, expect100ContinueResponsePending: Boolean,
closeAfterResponseCompletion: Boolean) extends MessageStart with RequestOutput closeAfterResponseCompletion: Boolean) extends MessageStart with RequestOutput
@ -36,7 +36,7 @@ private[http] object ParserOutput {
statusCode: StatusCode, statusCode: StatusCode,
protocol: HttpProtocol, protocol: HttpProtocol,
headers: List[HttpHeader], headers: List[HttpHeader],
createEntity: Source[ResponseOutput] ResponseEntity, createEntity: Source[ResponseOutput, Unit] ResponseEntity,
closeAfterResponseCompletion: Boolean) extends MessageStart with ResponseOutput closeAfterResponseCompletion: Boolean) extends MessageStart with ResponseOutput
case object MessageEnd extends MessageOutput case object MessageEnd extends MessageOutput

View file

@ -24,19 +24,19 @@ private[http] object BodyPartRenderer {
def streamed(boundary: String, def streamed(boundary: String,
nioCharset: Charset, nioCharset: Charset,
partHeadersSizeHint: Int, partHeadersSizeHint: Int,
log: LoggingAdapter): PushPullStage[Multipart.BodyPart, Source[ChunkStreamPart]] = log: LoggingAdapter): PushPullStage[Multipart.BodyPart, Source[ChunkStreamPart, Unit]] =
new PushPullStage[Multipart.BodyPart, Source[ChunkStreamPart]] { new PushPullStage[Multipart.BodyPart, Source[ChunkStreamPart, Unit]] {
var firstBoundaryRendered = false var firstBoundaryRendered = false
override def onPush(bodyPart: Multipart.BodyPart, ctx: Context[Source[ChunkStreamPart]]): Directive = { override def onPush(bodyPart: Multipart.BodyPart, ctx: Context[Source[ChunkStreamPart, Unit]]): Directive = {
val r = new CustomCharsetByteStringRendering(nioCharset, partHeadersSizeHint) val r = new CustomCharsetByteStringRendering(nioCharset, partHeadersSizeHint)
def bodyPartChunks(data: Source[ByteString]): Source[ChunkStreamPart] = { def bodyPartChunks(data: Source[ByteString, Unit]): Source[ChunkStreamPart, Unit] = {
val entityChunks = data.map[ChunkStreamPart](Chunk(_)) val entityChunks = data.map[ChunkStreamPart](Chunk(_))
chunkStream(r.get) ++ entityChunks (chunkStream(r.get) ++ entityChunks).mapMaterialized((_) ())
} }
def completePartRendering(): Source[ChunkStreamPart] = def completePartRendering(): Source[ChunkStreamPart, Unit] =
bodyPart.entity match { bodyPart.entity match {
case x if x.isKnownEmpty chunkStream(r.get) case x if x.isKnownEmpty chunkStream(r.get)
case Strict(_, data) chunkStream((r ~~ data).get) case Strict(_, data) chunkStream((r ~~ data).get)
@ -51,7 +51,7 @@ private[http] object BodyPartRenderer {
ctx.push(completePartRendering()) ctx.push(completePartRendering())
} }
override def onPull(ctx: Context[Source[ChunkStreamPart]]): Directive = { override def onPull(ctx: Context[Source[ChunkStreamPart, Unit]]): Directive = {
val finishing = ctx.isFinishing val finishing = ctx.isFinishing
if (finishing && firstBoundaryRendered) { if (finishing && firstBoundaryRendered) {
val r = new ByteStringRendering(boundary.length + 4) val r = new ByteStringRendering(boundary.length + 4)
@ -63,9 +63,9 @@ private[http] object BodyPartRenderer {
ctx.pull() ctx.pull()
} }
override def onUpstreamFinish(ctx: Context[Source[ChunkStreamPart]]): TerminationDirective = ctx.absorbTermination() override def onUpstreamFinish(ctx: Context[Source[ChunkStreamPart, Unit]]): TerminationDirective = ctx.absorbTermination()
private def chunkStream(byteString: ByteString): Source[ChunkStreamPart] = private def chunkStream(byteString: ByteString): Source[ChunkStreamPart, Unit] =
Source.single(Chunk(byteString)) Source.single(Chunk(byteString))
} }

View file

@ -25,9 +25,9 @@ private[http] class HttpRequestRendererFactory(userAgentHeader: Option[headers.`
def newRenderer: HttpRequestRenderer = new HttpRequestRenderer def newRenderer: HttpRequestRenderer = new HttpRequestRenderer
final class HttpRequestRenderer extends PushStage[RequestRenderingContext, Source[ByteString]] { final class HttpRequestRenderer extends PushStage[RequestRenderingContext, Source[ByteString, Unit]] {
override def onPush(ctx: RequestRenderingContext, opCtx: Context[Source[ByteString]]): Directive = { override def onPush(ctx: RequestRenderingContext, opCtx: Context[Source[ByteString, Unit]]): Directive = {
val r = new ByteStringRendering(requestHeaderSizeHint) val r = new ByteStringRendering(requestHeaderSizeHint)
import ctx.request._ import ctx.request._
@ -102,7 +102,7 @@ private[http] class HttpRequestRendererFactory(userAgentHeader: Option[headers.`
def renderContentLength(contentLength: Long) = def renderContentLength(contentLength: Long) =
if (method.isEntityAccepted) r ~~ `Content-Length` ~~ contentLength ~~ CrLf else r if (method.isEntityAccepted) r ~~ `Content-Length` ~~ contentLength ~~ CrLf else r
def completeRequestRendering(): Source[ByteString] = def completeRequestRendering(): Source[ByteString, Unit] =
entity match { entity match {
case x if x.isKnownEmpty case x if x.isKnownEmpty
renderContentLength(0) ~~ CrLf renderContentLength(0) ~~ CrLf

View file

@ -51,14 +51,14 @@ private[http] class HttpResponseRendererFactory(serverHeader: Option[headers.Ser
def newRenderer: HttpResponseRenderer = new HttpResponseRenderer def newRenderer: HttpResponseRenderer = new HttpResponseRenderer
final class HttpResponseRenderer extends PushStage[ResponseRenderingContext, Source[ByteString]] { final class HttpResponseRenderer extends PushStage[ResponseRenderingContext, Source[ByteString, Unit]] {
private[this] var close = false // signals whether the connection is to be closed after the current response private[this] var close = false // signals whether the connection is to be closed after the current response
// need this for testing // need this for testing
private[http] def isComplete = close private[http] def isComplete = close
override def onPush(ctx: ResponseRenderingContext, opCtx: Context[Source[ByteString]]): Directive = { override def onPush(ctx: ResponseRenderingContext, opCtx: Context[Source[ByteString, Unit]]): Directive = {
val r = new ByteStringRendering(responseHeaderSizeHint) val r = new ByteStringRendering(responseHeaderSizeHint)
import ctx.response._ import ctx.response._
@ -140,10 +140,10 @@ private[http] class HttpResponseRendererFactory(serverHeader: Option[headers.Ser
def renderContentLengthHeader(contentLength: Long) = def renderContentLengthHeader(contentLength: Long) =
if (status.allowsEntity) r ~~ `Content-Length` ~~ contentLength ~~ CrLf else r if (status.allowsEntity) r ~~ `Content-Length` ~~ contentLength ~~ CrLf else r
def byteStrings(entityBytes: Source[ByteString]): Source[ByteString] = def byteStrings(entityBytes: Source[ByteString, Unit]): Source[ByteString, Unit] =
renderByteStrings(r, entityBytes, skipEntity = noEntity) renderByteStrings(r, entityBytes, skipEntity = noEntity)
def completeResponseRendering(entity: ResponseEntity): Source[ByteString] = def completeResponseRendering(entity: ResponseEntity): Source[ByteString, Unit] =
entity match { entity match {
case HttpEntity.Strict(_, data) case HttpEntity.Strict(_, data)
renderHeaders(headers.toList) renderHeaders(headers.toList)

View file

@ -5,7 +5,6 @@
package akka.http.engine.rendering package akka.http.engine.rendering
import akka.parboiled2.CharUtils import akka.parboiled2.CharUtils
import akka.stream.ActorFlowMaterializer
import akka.util.ByteString import akka.util.ByteString
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.stream.scaladsl._ import akka.stream.scaladsl._
@ -30,25 +29,24 @@ private object RenderSupport {
val defaultLastChunkBytes: ByteString = renderChunk(HttpEntity.LastChunk) val defaultLastChunkBytes: ByteString = renderChunk(HttpEntity.LastChunk)
// This hooks into the materialization to cancel the not needed second source. This helper class def CancelSecond[T](first: Source[T, _], second: Source[T, _]): Source[T, Unit] = {
// allows us to not take a FlowMaterializer but delegate the cancellation to the point when the whole stream Source(first) { implicit b
// materializes frst
private case class CancelSecond[T](first: Source[T], second: Source[T]) extends SimpleActorFlowSource[T] { import FlowGraph.Implicits._
override def attach(flowSubscriber: Subscriber[T], materializer: ActorFlowMaterializer, flowName: String): Unit = { second ~> Sink.cancelled
first.to(Sink(flowSubscriber)).run()(materializer) frst.outlet
second.to(Sink.cancelled).run()(materializer) }.mapMaterialized((_) ())
}
} }
def renderEntityContentType(r: Rendering, entity: HttpEntity) = def renderEntityContentType(r: Rendering, entity: HttpEntity) =
if (entity.contentType != ContentTypes.NoContentType) r ~~ headers.`Content-Type` ~~ entity.contentType ~~ CrLf if (entity.contentType != ContentTypes.NoContentType) r ~~ headers.`Content-Type` ~~ entity.contentType ~~ CrLf
else r else r
def renderByteStrings(r: ByteStringRendering, entityBytes: Source[ByteString], def renderByteStrings(r: ByteStringRendering, entityBytes: Source[ByteString, Unit],
skipEntity: Boolean = false): Source[ByteString] = { skipEntity: Boolean = false): Source[ByteString, Unit] = {
val messageStart = Source.single(r.get) val messageStart = Source.single(r.get)
val messageBytes = val messageBytes =
if (!skipEntity) messageStart ++ entityBytes if (!skipEntity) (messageStart ++ entityBytes).mapMaterialized((_) ())
else CancelSecond(messageStart, entityBytes) else CancelSecond(messageStart, entityBytes)
messageBytes messageBytes
} }

View file

@ -4,13 +4,15 @@
package akka.http.engine.server package akka.http.engine.server
import akka.stream.scaladsl.OperationAttributes._
import akka.stream.scaladsl._
import akka.stream._
import scala.collection.immutable
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.actor.{ ActorRef, Props } import akka.actor.{ ActorRef, Props }
import akka.util.ByteString import akka.util.ByteString
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.stream.scaladsl.OperationAttributes._
import akka.stream.FlattenStrategy
import akka.stream.scaladsl._
import akka.stream.stage.PushPullStage import akka.stream.stage.PushPullStage
import akka.http.engine.parsing.{ HttpHeaderParser, HttpRequestParser } import akka.http.engine.parsing.{ HttpHeaderParser, HttpRequestParser }
import akka.http.engine.rendering.{ ResponseRenderingContext, HttpResponseRendererFactory } import akka.http.engine.rendering.{ ResponseRenderingContext, HttpResponseRendererFactory }
@ -18,17 +20,37 @@ import akka.http.engine.parsing.ParserOutput._
import akka.http.engine.TokenSourceActor import akka.http.engine.TokenSourceActor
import akka.http.model._ import akka.http.model._
import akka.http.util._ import akka.http.util._
import akka.stream.FlowMaterializer
import akka.stream.OverflowStrategy
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[http] object HttpServer { private[http] object HttpServer {
def serverFlowToTransport(serverFlow: Flow[HttpRequest, HttpResponse], case class HttpServerPorts(
settings: ServerSettings, bytesIn: Inlet[ByteString],
log: LoggingAdapter)(implicit mat: FlowMaterializer): Flow[ByteString, ByteString] = { bytesOut: Outlet[ByteString],
httpResponses: Inlet[HttpResponse],
httpRequests: Outlet[HttpRequest]) extends Shape {
override def inlets: immutable.Seq[Inlet[_]] = bytesIn :: httpResponses :: Nil
override def outlets: immutable.Seq[Outlet[_]] = bytesOut :: httpRequests :: Nil
override def deepCopy() = HttpServerPorts(
new Inlet(bytesIn.toString),
new Outlet(bytesOut.toString),
new Inlet(httpRequests.toString),
new Outlet(httpResponses.toString))
override def copyFromPorts(inlets: immutable.Seq[Inlet[_]], outlets: immutable.Seq[Outlet[_]]): Shape = {
require(inlets.size == 2, s"proposed inlets [${inlets.mkString(", ")}] do not fit BidiShape")
require(outlets.size == 2, s"proposed outlets [${outlets.mkString(", ")}] do not fit BidiShape")
HttpServerPorts(inlets(0).asInstanceOf[Inlet[ByteString]], outlets(0).asInstanceOf[Outlet[ByteString]],
inlets(1).asInstanceOf[Inlet[HttpResponse]], outlets(1).asInstanceOf[Outlet[HttpRequest]])
}
}
def serverBlueprint(settings: ServerSettings,
log: LoggingAdapter)(implicit mat: ActorFlowMaterializer): Graph[HttpServerPorts, Unit] = {
// the initial header parser we initially use for every connection, // the initial header parser we initially use for every connection,
// will not be mutated, all "shared copy" parsers copy on first-write into the header cache // will not be mutated, all "shared copy" parsers copy on first-write into the header cache
@ -50,10 +72,7 @@ private[http] object HttpServer {
} }
} }
val bypassFanout = Broadcast[RequestOutput](OperationAttributes.name("bypassFanout")) val requestParsingFlow = Flow[ByteString].section(name("rootParser"))(_.transform(()
val bypassMerge = new BypassMerge(settings, log)
val requestParsing = Flow[ByteString].section(name("rootParser"))(_.transform(()
// each connection uses a single (private) request parser instance for all its requests // each connection uses a single (private) request parser instance for all its requests
// which builds a cache of all header instances seen on that connection // which builds a cache of all header instances seen on that connection
rootParser.createShallowCopy(() oneHundredContinueRef).stage)) rootParser.createShallowCopy(() oneHundredContinueRef).stage))
@ -67,7 +86,7 @@ private[http] object HttpServer {
val effectiveUri = HttpRequest.effectiveUri(uri, headers, securedConnection = false, settings.defaultHostHeader) val effectiveUri = HttpRequest.effectiveUri(uri, headers, securedConnection = false, settings.defaultHostHeader)
val effectiveMethod = if (method == HttpMethods.HEAD && settings.transparentHeadRequests) HttpMethods.GET else method val effectiveMethod = if (method == HttpMethods.HEAD && settings.transparentHeadRequests) HttpMethods.GET else method
HttpRequest(effectiveMethod, effectiveUri, headers, createEntity(entityParts), protocol) HttpRequest(effectiveMethod, effectiveUri, headers, createEntity(entityParts), protocol)
case (_, src) src.runWith(BlackholeSink) case (_, src) src.runWith(Sink.ignore)
}.collect { }.collect {
case r: HttpRequest r case r: HttpRequest r
}.buffer(1, OverflowStrategy.backpressure) }.buffer(1, OverflowStrategy.backpressure)
@ -89,39 +108,44 @@ private[http] object HttpServer {
.flatten(FlattenStrategy.concat) .flatten(FlattenStrategy.concat)
.section(name("errorLogger"))(_.transform(() errorLogger(log, "Outgoing response stream error"))) .section(name("errorLogger"))(_.transform(() errorLogger(log, "Outgoing response stream error")))
val transportIn = UndefinedSource[ByteString] FlowGraph.partial(requestParsingFlow, rendererPipeline)(Keep.right) { implicit b
val transportOut = UndefinedSink[ByteString] (requestParsing, renderer)
import FlowGraph.Implicits._
import FlowGraphImplicits._ val bypassFanout = b.add(Broadcast[RequestOutput](2, OperationAttributes.name("bypassFanout")))
val bypassMerge = b.add(new BypassMerge(settings, log))
val bypassInput = bypassMerge.in0
val bypassOneHundredContinueInput = bypassMerge.in1
val bypassApplicationInput = bypassMerge.in2
Flow() { implicit b requestParsing.outlet ~> bypassFanout.in
//FIXME: the graph is unnecessary after fixing #15957 bypassMerge.out ~> renderer.inlet
transportIn ~> requestParsing ~> bypassFanout ~> requestPreparation ~> serverFlow ~> bypassMerge.applicationInput ~> rendererPipeline ~> transportOut val requestsIn = (bypassFanout.out(0) ~> requestPreparation).outlet
bypassFanout ~> bypass ~> bypassMerge.bypassInput
oneHundredContinueSource ~> bypassMerge.oneHundredContinueInput
b.allowCycles() bypassFanout.out(1) ~> bypass ~> bypassInput
oneHundredContinueSource ~> bypassOneHundredContinueInput
transportIn -> transportOut HttpServerPorts(
requestParsing.inlet,
renderer.outlet,
bypassApplicationInput,
requestsIn)
} }
} }
class BypassMerge(settings: ServerSettings, log: LoggingAdapter) class BypassMerge(settings: ServerSettings, log: LoggingAdapter)
extends FlexiMerge[ResponseRenderingContext](OperationAttributes.name("BypassMerge")) { extends FlexiMerge[ResponseRenderingContext, FanInShape3[RequestOutput, OneHundredContinue.type, HttpResponse, ResponseRenderingContext]](new FanInShape3("BypassMerge"), OperationAttributes.name("BypassMerge")) {
import FlexiMerge._ import FlexiMerge._
val bypassInput = createInputPort[RequestOutput]()
val oneHundredContinueInput = createInputPort[OneHundredContinue.type]()
val applicationInput = createInputPort[HttpResponse]()
def createMergeLogic() = new MergeLogic[ResponseRenderingContext] { def createMergeLogic(p: PortT) = new MergeLogic[ResponseRenderingContext] {
var requestStart: RequestStart = _ var requestStart: RequestStart = _
override def inputHandles(inputCount: Int) = { val bypassInput: Inlet[RequestOutput] = p.in0
require(inputCount == 3, s"BypassMerge must have 3 connected inputs, was $inputCount") val oneHundredContinueInput: Inlet[OneHundredContinue.type] = p.in1
Vector(bypassInput, oneHundredContinueInput, applicationInput) val applicationInput: Inlet[HttpResponse] = p.in2
}
override val initialState: State[Any] = State[Any](Read(bypassInput)) { override val initialState: State[RequestOutput] = State[RequestOutput](Read(bypassInput)) {
case (ctx, _, requestStart: RequestStart) case (ctx, _, requestStart: RequestStart)
this.requestStart = requestStart this.requestStart = requestStart
ctx.changeCompletionHandling(waitingForApplicationResponseCompletionHandling) ctx.changeCompletionHandling(waitingForApplicationResponseCompletionHandling)
@ -133,7 +157,7 @@ private[http] object HttpServer {
override val initialCompletionHandling = eagerClose override val initialCompletionHandling = eagerClose
val waitingForApplicationResponse = val waitingForApplicationResponse =
State[Any](ReadAny(oneHundredContinueInput, applicationInput)) { State[Any](ReadAny(oneHundredContinueInput.asInstanceOf[Inlet[Any]] :: applicationInput.asInstanceOf[Inlet[Any]] :: Nil)) {
case (ctx, _, response: HttpResponse) case (ctx, _, response: HttpResponse)
// see the comment on [[OneHundredContinue]] for an explanation of the closing logic here (and more) // see the comment on [[OneHundredContinue]] for an explanation of the closing logic here (and more)
val close = requestStart.closeAfterResponseCompletion || requestStart.expect100ContinueResponsePending val close = requestStart.closeAfterResponseCompletion || requestStart.expect100ContinueResponsePending

View file

@ -13,7 +13,7 @@ import scala.collection.immutable
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.util.ByteString import akka.util.ByteString
import akka.stream.scaladsl.OperationAttributes._ import akka.stream.scaladsl.OperationAttributes._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.TimerTransformer import akka.stream.TimerTransformer
import akka.http.util._ import akka.http.util._
@ -38,13 +38,13 @@ sealed trait HttpEntity extends japi.HttpEntity {
/** /**
* A stream of the data of this entity. * A stream of the data of this entity.
*/ */
def dataBytes: Source[ByteString] def dataBytes: Source[ByteString, Unit]
/** /**
* Collects all possible parts and returns a potentially future Strict entity for easier processing. * Collects all possible parts and returns a potentially future Strict entity for easier processing.
* The Future is failed with an TimeoutException if the stream isn't completed after the given timeout. * The Future is failed with an TimeoutException if the stream isn't completed after the given timeout.
*/ */
def toStrict(timeout: FiniteDuration)(implicit fm: FlowMaterializer): Future[HttpEntity.Strict] = { def toStrict(timeout: FiniteDuration)(implicit fm: ActorFlowMaterializer): Future[HttpEntity.Strict] = {
def transformer() = def transformer() =
new TimerTransformer[ByteString, HttpEntity.Strict] { new TimerTransformer[ByteString, HttpEntity.Strict] {
var bytes = ByteString.newBuilder var bytes = ByteString.newBuilder
@ -64,7 +64,7 @@ sealed trait HttpEntity extends japi.HttpEntity {
} }
// TODO timerTransform is meant to be replaced / rewritten, it's currently private[akka]; See https://github.com/akka/akka/issues/16393 // TODO timerTransform is meant to be replaced / rewritten, it's currently private[akka]; See https://github.com/akka/akka/issues/16393
dataBytes.section(name("toStrict"))(_.timerTransform(transformer)).runWith(Sink.head) dataBytes.section(name("toStrict"))(_.timerTransform(transformer)).runWith(Sink.head())
} }
/** /**
@ -75,7 +75,7 @@ sealed trait HttpEntity extends japi.HttpEntity {
* This method may only throw an exception if the `transformer` function throws an exception while creating the transformer. * This method may only throw an exception if the `transformer` function throws an exception while creating the transformer.
* Any other errors are reported through the new entity data stream. * Any other errors are reported through the new entity data stream.
*/ */
def transformDataBytes(transformer: Flow[ByteString, ByteString]): HttpEntity def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): HttpEntity
/** /**
* Creates a copy of this HttpEntity with the `contentType` overridden with the given one. * Creates a copy of this HttpEntity with the `contentType` overridden with the given one.
@ -83,7 +83,7 @@ sealed trait HttpEntity extends japi.HttpEntity {
def withContentType(contentType: ContentType): HttpEntity def withContentType(contentType: ContentType): HttpEntity
/** Java API */ /** Java API */
def getDataBytes: Source[ByteString] = dataBytes def getDataBytes: Source[ByteString, Unit] = dataBytes
// default implementations, should be overridden // default implementations, should be overridden
def isCloseDelimited: Boolean = false def isCloseDelimited: Boolean = false
@ -100,13 +100,13 @@ sealed trait BodyPartEntity extends HttpEntity with japi.BodyPartEntity {
sealed trait RequestEntity extends HttpEntity with japi.RequestEntity with ResponseEntity { sealed trait RequestEntity extends HttpEntity with japi.RequestEntity with ResponseEntity {
def withContentType(contentType: ContentType): RequestEntity def withContentType(contentType: ContentType): RequestEntity
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): RequestEntity override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): RequestEntity
} }
/* An entity that can be used for responses */ /* An entity that can be used for responses */
sealed trait ResponseEntity extends HttpEntity with japi.ResponseEntity { sealed trait ResponseEntity extends HttpEntity with japi.ResponseEntity {
def withContentType(contentType: ContentType): ResponseEntity def withContentType(contentType: ContentType): ResponseEntity
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): ResponseEntity override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): ResponseEntity
} }
/* An entity that can be used for requests, responses, and body parts */ /* An entity that can be used for requests, responses, and body parts */
sealed trait UniversalEntity extends japi.UniversalEntity with MessageEntity with BodyPartEntity { sealed trait UniversalEntity extends japi.UniversalEntity with MessageEntity with BodyPartEntity {
@ -117,7 +117,7 @@ sealed trait UniversalEntity extends japi.UniversalEntity with MessageEntity wit
* Transforms this' entities data bytes with a transformer that will produce exactly the number of bytes given as * Transforms this' entities data bytes with a transformer that will produce exactly the number of bytes given as
* ``newContentLength``. * ``newContentLength``.
*/ */
def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString]): UniversalEntity def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString, _]): UniversalEntity
} }
object HttpEntity { object HttpEntity {
@ -130,7 +130,7 @@ object HttpEntity {
if (bytes.length == 0) empty(contentType) else apply(contentType, ByteString(bytes)) if (bytes.length == 0) empty(contentType) else apply(contentType, ByteString(bytes))
def apply(contentType: ContentType, data: ByteString): Strict = def apply(contentType: ContentType, data: ByteString): Strict =
if (data.isEmpty) empty(contentType) else Strict(contentType, data) if (data.isEmpty) empty(contentType) else Strict(contentType, data)
def apply(contentType: ContentType, contentLength: Long, data: Source[ByteString]): UniversalEntity = def apply(contentType: ContentType, contentLength: Long, data: Source[ByteString, Unit]): UniversalEntity =
if (contentLength == 0) empty(contentType) else Default(contentType, contentLength, data) if (contentLength == 0) empty(contentType) else Default(contentType, contentLength, data)
def apply(contentType: ContentType, file: File): UniversalEntity = { def apply(contentType: ContentType, file: File): UniversalEntity = {
@ -148,50 +148,26 @@ object HttpEntity {
// TODO: re-establish serializability // TODO: re-establish serializability
// TODO: equal/hashcode ? // TODO: equal/hashcode ?
object Strict {
// FIXME configurable?
private val MaxByteSize = 1L * 1024 * 1024 * 1024
private val MaxElements = 1000
}
/** /**
* The model for the entity of a "regular" unchunked HTTP message with known, fixed data. * The model for the entity of a "regular" unchunked HTTP message with known, fixed data.
*/ */
final case class Strict(contentType: ContentType, data: ByteString) final case class Strict(contentType: ContentType, data: ByteString)
extends japi.HttpEntityStrict with UniversalEntity { extends japi.HttpEntityStrict with UniversalEntity {
import Strict._
def contentLength: Long = data.length def contentLength: Long = data.length
def isKnownEmpty: Boolean = data.isEmpty def isKnownEmpty: Boolean = data.isEmpty
def dataBytes: Source[ByteString] = Source(data :: Nil) def dataBytes: Source[ByteString, Unit] = Source(data :: Nil)
override def toStrict(timeout: FiniteDuration)(implicit fm: FlowMaterializer) = override def toStrict(timeout: FiniteDuration)(implicit fm: ActorFlowMaterializer) =
FastFuture.successful(this) FastFuture.successful(this)
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): MessageEntity = override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): MessageEntity =
StreamUtils.runStrict(data, transformer, MaxByteSize, MaxElements) match {
case Success(Some(newData))
copy(data = newData)
case Success(None)
Chunked.fromData(contentType, Source.single(data).via(transformer)) Chunked.fromData(contentType, Source.single(data).via(transformer))
case Failure(ex)
Chunked(contentType, Source.failed(ex))
}
override def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString]): UniversalEntity = override def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString, _]): UniversalEntity =
StreamUtils.runStrict(data, transformer, MaxByteSize, MaxElements) match {
case Success(Some(newData))
if (newData.length.toLong != newContentLength)
throw new IllegalStateException(s"Transformer didn't produce as much bytes (${newData.length}:'${newData.utf8String}') as claimed ($newContentLength)")
copy(data = newData)
case Success(None)
Default(contentType, newContentLength, Source.single(data).via(transformer)) Default(contentType, newContentLength, Source.single(data).via(transformer))
case Failure(ex)
Default(contentType, newContentLength, Source.failed(ex))
}
def withContentType(contentType: ContentType): Strict = def withContentType(contentType: ContentType): Strict =
if (contentType == this.contentType) this else copy(contentType = contentType) if (contentType == this.contentType) this else copy(contentType = contentType)
@ -204,20 +180,20 @@ object HttpEntity {
*/ */
final case class Default(contentType: ContentType, final case class Default(contentType: ContentType,
contentLength: Long, contentLength: Long,
data: Source[ByteString]) data: Source[ByteString, Unit])
extends japi.HttpEntityDefault with UniversalEntity { extends japi.HttpEntityDefault with UniversalEntity {
require(contentLength > 0, "contentLength must be positive (use `HttpEntity.empty(contentType)` for empty entities)") require(contentLength > 0, "contentLength must be positive (use `HttpEntity.empty(contentType)` for empty entities)")
def isKnownEmpty = false def isKnownEmpty = false
override def isDefault: Boolean = true override def isDefault: Boolean = true
def dataBytes: Source[ByteString] = data def dataBytes: Source[ByteString, Unit] = data
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): Chunked = override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): Chunked =
Chunked.fromData(contentType, data.via(transformer)) Chunked.fromData(contentType, data.viaMat(transformer)(Keep.left))
override def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString]): UniversalEntity = override def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString, _]): UniversalEntity =
Default(contentType, newContentLength, data.via(transformer)) Default(contentType, newContentLength, data.viaMat(transformer)(Keep.left))
def withContentType(contentType: ContentType): Default = def withContentType(contentType: ContentType): Default =
if (contentType == this.contentType) this else copy(contentType = contentType) if (contentType == this.contentType) this else copy(contentType = contentType)
@ -232,11 +208,11 @@ object HttpEntity {
*/ */
private[http] sealed trait WithoutKnownLength extends HttpEntity { private[http] sealed trait WithoutKnownLength extends HttpEntity {
def contentType: ContentType def contentType: ContentType
def data: Source[ByteString] def data: Source[ByteString, Unit]
def isKnownEmpty = data eq Source.empty def isKnownEmpty = data eq Source.empty
def dataBytes: Source[ByteString] = data def dataBytes: Source[ByteString, Unit] = data
} }
/** /**
@ -244,7 +220,7 @@ object HttpEntity {
* The content-length of such responses is unknown at the time the response headers have been received. * The content-length of such responses is unknown at the time the response headers have been received.
* Note that this type of HttpEntity can only be used for HttpResponses. * Note that this type of HttpEntity can only be used for HttpResponses.
*/ */
final case class CloseDelimited(contentType: ContentType, data: Source[ByteString]) final case class CloseDelimited(contentType: ContentType, data: Source[ByteString, Unit])
extends japi.HttpEntityCloseDelimited with ResponseEntity with WithoutKnownLength { extends japi.HttpEntityCloseDelimited with ResponseEntity with WithoutKnownLength {
type Self = CloseDelimited type Self = CloseDelimited
@ -252,8 +228,8 @@ object HttpEntity {
def withContentType(contentType: ContentType): CloseDelimited = def withContentType(contentType: ContentType): CloseDelimited =
if (contentType == this.contentType) this else copy(contentType = contentType) if (contentType == this.contentType) this else copy(contentType = contentType)
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): CloseDelimited = override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): CloseDelimited =
HttpEntity.CloseDelimited(contentType, data.via(transformer)) HttpEntity.CloseDelimited(contentType, data.viaMat(transformer)(Keep.left))
override def productPrefix = "HttpEntity.CloseDelimited" override def productPrefix = "HttpEntity.CloseDelimited"
} }
@ -262,15 +238,15 @@ object HttpEntity {
* The model for the entity of a BodyPart with an indefinite length. * The model for the entity of a BodyPart with an indefinite length.
* Note that this type of HttpEntity can only be used for BodyParts. * Note that this type of HttpEntity can only be used for BodyParts.
*/ */
final case class IndefiniteLength(contentType: ContentType, data: Source[ByteString]) final case class IndefiniteLength(contentType: ContentType, data: Source[ByteString, Unit])
extends japi.HttpEntityIndefiniteLength with BodyPartEntity with WithoutKnownLength { extends japi.HttpEntityIndefiniteLength with BodyPartEntity with WithoutKnownLength {
override def isIndefiniteLength: Boolean = true override def isIndefiniteLength: Boolean = true
def withContentType(contentType: ContentType): IndefiniteLength = def withContentType(contentType: ContentType): IndefiniteLength =
if (contentType == this.contentType) this else copy(contentType = contentType) if (contentType == this.contentType) this else copy(contentType = contentType)
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): IndefiniteLength = override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): IndefiniteLength =
HttpEntity.IndefiniteLength(contentType, data.via(transformer)) HttpEntity.IndefiniteLength(contentType, data.viaMat(transformer)(Keep.left))
override def productPrefix = "HttpEntity.IndefiniteLength" override def productPrefix = "HttpEntity.IndefiniteLength"
} }
@ -278,23 +254,23 @@ object HttpEntity {
/** /**
* The model for the entity of a chunked HTTP message (with `Transfer-Encoding: chunked`). * The model for the entity of a chunked HTTP message (with `Transfer-Encoding: chunked`).
*/ */
final case class Chunked(contentType: ContentType, chunks: Source[ChunkStreamPart]) final case class Chunked(contentType: ContentType, chunks: Source[ChunkStreamPart, Unit])
extends japi.HttpEntityChunked with MessageEntity { extends japi.HttpEntityChunked with MessageEntity {
def isKnownEmpty = chunks eq Source.empty def isKnownEmpty = chunks eq Source.empty
override def isChunked: Boolean = true override def isChunked: Boolean = true
def dataBytes: Source[ByteString] = def dataBytes: Source[ByteString, Unit] =
chunks.map(_.data).filter(_.nonEmpty) chunks.map(_.data).filter(_.nonEmpty)
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): Chunked = { override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): Chunked = {
val newData = val newData =
chunks.map { chunks.map {
case Chunk(data, "") data case Chunk(data, "") data
case LastChunk("", Nil) ByteString.empty case LastChunk("", Nil) ByteString.empty
case _ case _
throw new IllegalArgumentException("Chunked.transformDataBytes not allowed for chunks with metadata") throw new IllegalArgumentException("Chunked.transformDataBytes not allowed for chunks with metadata")
}.via(transformer) }.viaMat(transformer)(Keep.left)
Chunked.fromData(contentType, newData) Chunked.fromData(contentType, newData)
} }
@ -305,14 +281,14 @@ object HttpEntity {
override def productPrefix = "HttpEntity.Chunked" override def productPrefix = "HttpEntity.Chunked"
/** Java API */ /** Java API */
def getChunks: Source[japi.ChunkStreamPart] = chunks.asInstanceOf[Source[japi.ChunkStreamPart]] def getChunks: Source[japi.ChunkStreamPart, Unit] = chunks.asInstanceOf[Source[japi.ChunkStreamPart, Unit]]
} }
object Chunked { object Chunked {
/** /**
* Returns a ``Chunked`` entity where one Chunk is produced for every non-empty ByteString of the given * Returns a ``Chunked`` entity where one Chunk is produced for every non-empty ByteString of the given
* ``Publisher[ByteString]``. * ``Publisher[ByteString]``.
*/ */
def fromData(contentType: ContentType, chunks: Source[ByteString]): Chunked = def fromData(contentType: ContentType, chunks: Source[ByteString, Unit]): Chunked =
Chunked(contentType, chunks.collect[ChunkStreamPart] { Chunked(contentType, chunks.collect[ChunkStreamPart] {
case b: ByteString if b.nonEmpty Chunk(b) case b: ByteString if b.nonEmpty Chunk(b)
}) })

View file

@ -11,7 +11,7 @@ import scala.concurrent.duration.FiniteDuration
import scala.concurrent.{ Future, ExecutionContext } import scala.concurrent.{ Future, ExecutionContext }
import scala.collection.immutable import scala.collection.immutable
import scala.reflect.{ classTag, ClassTag } import scala.reflect.{ classTag, ClassTag }
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.util.ByteString import akka.util.ByteString
import akka.http.util._ import akka.http.util._
import headers._ import headers._
@ -51,7 +51,7 @@ sealed trait HttpMessage extends japi.HttpMessage {
def withEntity(entity: MessageEntity): Self def withEntity(entity: MessageEntity): Self
/** Returns a sharable and serializable copy of this message with a strict entity. */ /** Returns a sharable and serializable copy of this message with a strict entity. */
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Self] = def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[Self] =
entity.toStrict(timeout).fast.map(this.withEntity) entity.toStrict(timeout).fast.map(this.withEntity)
/** Returns a copy of this message with the entity and headers set to the given ones. */ /** Returns a copy of this message with the entity and headers set to the given ones. */

View file

@ -9,7 +9,7 @@ import scala.concurrent.duration.FiniteDuration
import scala.concurrent.{ Future, ExecutionContext } import scala.concurrent.{ Future, ExecutionContext }
import scala.collection.immutable import scala.collection.immutable
import scala.util.{ Failure, Success, Try } import scala.util.{ Failure, Success, Try }
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.http.util.FastFuture import akka.http.util.FastFuture
import akka.http.model.headers._ import akka.http.model.headers._
@ -17,14 +17,14 @@ import FastFuture._
trait Multipart { trait Multipart {
def mediaType: MultipartMediaType def mediaType: MultipartMediaType
def parts: Source[Multipart.BodyPart] def parts: Source[Multipart.BodyPart, Unit]
/** /**
* Converts this content into its strict counterpart. * Converts this content into its strict counterpart.
* The given ``timeout`` denotes the max time that an individual part must be read in. * The given ``timeout`` denotes the max time that an individual part must be read in.
* The Future is failed with an TimeoutException if one part isn't read completely after the given timeout. * The Future is failed with an TimeoutException if one part isn't read completely after the given timeout.
*/ */
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Multipart.Strict] def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[Multipart.Strict]
} }
object Multipart { object Multipart {
@ -47,7 +47,7 @@ object Multipart {
def dispositionType: Option[ContentDispositionType] = def dispositionType: Option[ContentDispositionType] =
contentDispositionHeader.map(_.dispositionType) contentDispositionHeader.map(_.dispositionType)
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[BodyPart.Strict] def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[BodyPart.Strict]
} }
object BodyPart { object BodyPart {
@ -56,7 +56,7 @@ object Multipart {
} }
} }
private def strictify[BP <: Multipart.BodyPart, BPS <: Multipart.BodyPart.Strict](parts: Source[BP])(f: BP Future[BPS])(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Vector[BPS]] = private def strictify[BP <: Multipart.BodyPart, BPS <: Multipart.BodyPart.Strict](parts: Source[BP, Unit])(f: BP Future[BPS])(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[Vector[BPS]] =
// TODO: move to Vector `:+` when https://issues.scala-lang.org/browse/SI-8930 is fixed // TODO: move to Vector `:+` when https://issues.scala-lang.org/browse/SI-8930 is fixed
parts.runFold(new VectorBuilder[Future[BPS]]) { parts.runFold(new VectorBuilder[Future[BPS]]) {
case (builder, part) builder += f(part) case (builder, part) builder += f(part)
@ -69,28 +69,28 @@ object Multipart {
*/ */
sealed abstract class General extends Multipart { sealed abstract class General extends Multipart {
def mediaType: MultipartMediaType def mediaType: MultipartMediaType
def parts: Source[General.BodyPart] def parts: Source[General.BodyPart, Unit]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[General.Strict] = def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[General.Strict] =
strictify(parts)(_.toStrict(timeout)).fast.map(General.Strict(mediaType, _)) strictify(parts)(_.toStrict(timeout)).fast.map(General.Strict(mediaType, _))
} }
object General { object General {
def apply(mediaType: MultipartMediaType, parts: BodyPart.Strict*): Strict = Strict(mediaType, parts.toVector) def apply(mediaType: MultipartMediaType, parts: BodyPart.Strict*): Strict = Strict(mediaType, parts.toVector)
def apply(_mediaType: MultipartMediaType, _parts: Source[BodyPart]): General = def apply(_mediaType: MultipartMediaType, _parts: Source[BodyPart, Unit]): General =
new General { new General {
def mediaType = _mediaType def mediaType = _mediaType
def parts = _parts def parts = _parts
override def toString = s"General($mediaType, $parts)" override def toString = s"General($mediaType, $parts)"
} }
def unapply(value: General): Option[(MultipartMediaType, Source[BodyPart])] = Some(value.mediaType -> value.parts) def unapply(value: General): Option[(MultipartMediaType, Source[BodyPart, Unit])] = Some(value.mediaType -> value.parts)
/** /**
* Strict [[General]]. * Strict [[General]].
*/ */
case class Strict(mediaType: MultipartMediaType, strictParts: immutable.Seq[BodyPart.Strict]) extends General with Multipart.Strict { case class Strict(mediaType: MultipartMediaType, strictParts: immutable.Seq[BodyPart.Strict]) extends General with Multipart.Strict {
def parts: Source[BodyPart.Strict] = Source(strictParts) def parts: Source[BodyPart.Strict, Unit] = Source(strictParts)
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) = override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer) =
FastFuture.successful(this) FastFuture.successful(this)
override def productPrefix = "General.Strict" override def productPrefix = "General.Strict"
} }
@ -99,7 +99,7 @@ object Multipart {
* Body part of the [[General]] model. * Body part of the [[General]] model.
*/ */
sealed abstract class BodyPart extends Multipart.BodyPart { sealed abstract class BodyPart extends Multipart.BodyPart {
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[BodyPart.Strict] = def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[BodyPart.Strict] =
entity.toStrict(timeout).map(BodyPart.Strict(_, headers)) entity.toStrict(timeout).map(BodyPart.Strict(_, headers))
def toFormDataBodyPart: Try[FormData.BodyPart] def toFormDataBodyPart: Try[FormData.BodyPart]
def toByteRangesBodyPart: Try[ByteRanges.BodyPart] def toByteRangesBodyPart: Try[ByteRanges.BodyPart]
@ -133,7 +133,7 @@ object Multipart {
* Strict [[General.BodyPart]]. * Strict [[General.BodyPart]].
*/ */
case class Strict(entity: HttpEntity.Strict, headers: immutable.Seq[HttpHeader] = Nil) extends BodyPart with Multipart.BodyPart.Strict { case class Strict(entity: HttpEntity.Strict, headers: immutable.Seq[HttpHeader] = Nil) extends BodyPart with Multipart.BodyPart.Strict {
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Strict] = override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[Strict] =
FastFuture.successful(this) FastFuture.successful(this)
override def toFormDataBodyPart: Try[FormData.BodyPart.Strict] = tryCreateFormDataBodyPart(FormData.BodyPart.Strict(_, entity, _, _)) override def toFormDataBodyPart: Try[FormData.BodyPart.Strict] = tryCreateFormDataBodyPart(FormData.BodyPart.Strict(_, entity, _, _))
override def toByteRangesBodyPart: Try[ByteRanges.BodyPart.Strict] = tryCreateByteRangesBodyPart(ByteRanges.BodyPart.Strict(_, entity, _, _)) override def toByteRangesBodyPart: Try[ByteRanges.BodyPart.Strict] = tryCreateByteRangesBodyPart(ByteRanges.BodyPart.Strict(_, entity, _, _))
@ -148,8 +148,8 @@ object Multipart {
*/ */
sealed abstract class FormData extends Multipart { sealed abstract class FormData extends Multipart {
def mediaType = MediaTypes.`multipart/form-data` def mediaType = MediaTypes.`multipart/form-data`
def parts: Source[FormData.BodyPart] def parts: Source[FormData.BodyPart, Unit]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[FormData.Strict] = def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[FormData.Strict] =
strictify(parts)(_.toStrict(timeout)).fast.map(FormData.Strict(_)) strictify(parts)(_.toStrict(timeout)).fast.map(FormData.Strict(_))
} }
object FormData { object FormData {
@ -159,7 +159,7 @@ object Multipart {
fields.map { case (name, entity) BodyPart.Strict(name, entity) }(collection.breakOut) fields.map { case (name, entity) BodyPart.Strict(name, entity) }(collection.breakOut)
} }
def apply(_parts: Source[BodyPart]): FormData = new FormData { def apply(_parts: Source[BodyPart, Unit]): FormData = new FormData {
def parts = _parts def parts = _parts
override def toString = s"FormData($parts)" override def toString = s"FormData($parts)"
} }
@ -168,8 +168,8 @@ object Multipart {
* Strict [[FormData]]. * Strict [[FormData]].
*/ */
case class Strict(strictParts: immutable.Seq[BodyPart.Strict]) extends FormData with Multipart.Strict { case class Strict(strictParts: immutable.Seq[BodyPart.Strict]) extends FormData with Multipart.Strict {
def parts: Source[BodyPart.Strict] = Source(strictParts) def parts: Source[BodyPart.Strict, Unit] = Source(strictParts)
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) = override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer) =
FastFuture.successful(this) FastFuture.successful(this)
override def productPrefix = "FormData.Strict" override def productPrefix = "FormData.Strict"
} }
@ -186,7 +186,7 @@ object Multipart {
override def dispositionParams = additionalDispositionParams.updated("name", name) override def dispositionParams = additionalDispositionParams.updated("name", name)
override def dispositionType = Some(ContentDispositionTypes.`form-data`) override def dispositionType = Some(ContentDispositionTypes.`form-data`)
def filename: Option[String] = additionalDispositionParams.get("filename") def filename: Option[String] = additionalDispositionParams.get("filename")
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[BodyPart.Strict] = def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[BodyPart.Strict] =
entity.toStrict(timeout).map(BodyPart.Strict(name, _, additionalDispositionParams, additionalHeaders)) entity.toStrict(timeout).map(BodyPart.Strict(name, _, additionalDispositionParams, additionalHeaders))
} }
object BodyPart { object BodyPart {
@ -210,7 +210,7 @@ object Multipart {
case class Strict(name: String, entity: HttpEntity.Strict, case class Strict(name: String, entity: HttpEntity.Strict,
additionalDispositionParams: Map[String, String] = Map.empty, additionalDispositionParams: Map[String, String] = Map.empty,
additionalHeaders: immutable.Seq[HttpHeader] = Nil) extends BodyPart with Multipart.BodyPart.Strict { additionalHeaders: immutable.Seq[HttpHeader] = Nil) extends BodyPart with Multipart.BodyPart.Strict {
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Strict] = override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[Strict] =
FastFuture.successful(this) FastFuture.successful(this)
override def productPrefix = "FormData.BodyPart.Strict" override def productPrefix = "FormData.BodyPart.Strict"
} }
@ -223,14 +223,14 @@ object Multipart {
*/ */
sealed abstract class ByteRanges extends Multipart { sealed abstract class ByteRanges extends Multipart {
def mediaType = MediaTypes.`multipart/byteranges` def mediaType = MediaTypes.`multipart/byteranges`
def parts: Source[ByteRanges.BodyPart] def parts: Source[ByteRanges.BodyPart, Unit]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[ByteRanges.Strict] = def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[ByteRanges.Strict] =
strictify(parts)(_.toStrict(timeout)).fast.map(ByteRanges.Strict(_)) strictify(parts)(_.toStrict(timeout)).fast.map(ByteRanges.Strict(_))
} }
object ByteRanges { object ByteRanges {
def apply(parts: BodyPart.Strict*): Strict = Strict(parts.toVector) def apply(parts: BodyPart.Strict*): Strict = Strict(parts.toVector)
def apply(_parts: Source[BodyPart]): ByteRanges = def apply(_parts: Source[BodyPart, Unit]): ByteRanges =
new ByteRanges { new ByteRanges {
def parts = _parts def parts = _parts
override def toString = s"ByteRanges($parts)" override def toString = s"ByteRanges($parts)"
@ -240,8 +240,8 @@ object Multipart {
* Strict [[ByteRanges]]. * Strict [[ByteRanges]].
*/ */
case class Strict(strictParts: immutable.Seq[BodyPart.Strict]) extends ByteRanges with Multipart.Strict { case class Strict(strictParts: immutable.Seq[BodyPart.Strict]) extends ByteRanges with Multipart.Strict {
def parts: Source[BodyPart.Strict] = Source(strictParts) def parts: Source[BodyPart.Strict, Unit] = Source(strictParts)
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) = override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer) =
FastFuture.successful(this) FastFuture.successful(this)
override def productPrefix = "ByteRanges.Strict" override def productPrefix = "ByteRanges.Strict"
} }
@ -255,7 +255,7 @@ object Multipart {
def additionalHeaders: immutable.Seq[HttpHeader] def additionalHeaders: immutable.Seq[HttpHeader]
override def headers = contentRangeHeader +: additionalHeaders override def headers = contentRangeHeader +: additionalHeaders
def contentRangeHeader = `Content-Range`(rangeUnit, contentRange) def contentRangeHeader = `Content-Range`(rangeUnit, contentRange)
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[BodyPart.Strict] = def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[BodyPart.Strict] =
entity.toStrict(timeout).map(BodyPart.Strict(contentRange, _, rangeUnit, additionalHeaders)) entity.toStrict(timeout).map(BodyPart.Strict(contentRange, _, rangeUnit, additionalHeaders))
} }
object BodyPart { object BodyPart {
@ -277,7 +277,7 @@ object Multipart {
*/ */
case class Strict(contentRange: ContentRange, entity: HttpEntity.Strict, rangeUnit: RangeUnit = RangeUnits.Bytes, case class Strict(contentRange: ContentRange, entity: HttpEntity.Strict, rangeUnit: RangeUnit = RangeUnits.Bytes,
additionalHeaders: immutable.Seq[HttpHeader] = Nil) extends BodyPart with Multipart.BodyPart.Strict { additionalHeaders: immutable.Seq[HttpHeader] = Nil) extends BodyPart with Multipart.BodyPart.Strict {
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Strict] = override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[Strict] =
FastFuture.successful(this) FastFuture.successful(this)
override def productPrefix = "ByteRanges.BodyPart.Strict" override def productPrefix = "ByteRanges.BodyPart.Strict"
} }

View file

@ -14,15 +14,11 @@ import scala.concurrent.{ ExecutionContext, Future }
import scala.util.Try import scala.util.Try
import akka.actor.Props import akka.actor.Props
import akka.http.model.RequestEntity import akka.http.model.RequestEntity
import akka.stream.ActorFlowMaterializer import akka.stream.{ ActorFlowMaterializerSettings, ActorFlowMaterializer, impl }
import akka.stream.FlowMaterializer
import akka.stream.impl.Ast.AstNode
import akka.stream.impl.Ast.StageFactory
import akka.stream.impl.fusing.IteratorInterpreter import akka.stream.impl.fusing.IteratorInterpreter
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.scaladsl.OperationAttributes._ import akka.stream.scaladsl.OperationAttributes._
import akka.stream.stage._ import akka.stream.stage._
import akka.stream.impl
import akka.util.ByteString import akka.util.ByteString
import org.reactivestreams.{ Subscriber, Publisher } import org.reactivestreams.{ Subscriber, Publisher }
@ -51,7 +47,7 @@ private[http] object StreamUtils {
def failedPublisher[T](ex: Throwable): Publisher[T] = def failedPublisher[T](ex: Throwable): Publisher[T] =
impl.ErrorPublisher(ex, "failed").asInstanceOf[Publisher[T]] impl.ErrorPublisher(ex, "failed").asInstanceOf[Publisher[T]]
def mapErrorTransformer(f: Throwable Throwable): Flow[ByteString, ByteString] = { def mapErrorTransformer(f: Throwable Throwable): Flow[ByteString, ByteString, Unit] = {
val transformer = new PushStage[ByteString, ByteString] { val transformer = new PushStage[ByteString, ByteString] {
override def onPush(element: ByteString, ctx: Context[ByteString]): Directive = override def onPush(element: ByteString, ctx: Context[ByteString]): Directive =
ctx.push(element) ctx.push(element)
@ -63,11 +59,12 @@ private[http] object StreamUtils {
Flow[ByteString].section(name("transformError"))(_.transform(() transformer)) Flow[ByteString].section(name("transformError"))(_.transform(() transformer))
} }
def sliceBytesTransformer(start: Long, length: Long): Flow[ByteString, ByteString] = { def sliceBytesTransformer(start: Long, length: Long): Flow[ByteString, ByteString, Unit] = {
val transformer = new StatefulStage[ByteString, ByteString] { val transformer = new StatefulStage[ByteString, ByteString] {
def skipping = new State { def skipping = new State {
var toSkip = start var toSkip = start
override def onPush(element: ByteString, ctx: Context[ByteString]): Directive = override def onPush(element: ByteString, ctx: Context[ByteString]): Directive =
if (element.length < toSkip) { if (element.length < toSkip) {
// keep skipping // keep skipping
@ -79,8 +76,10 @@ private[http] object StreamUtils {
current.onPush(element.drop(toSkip.toInt), ctx) current.onPush(element.drop(toSkip.toInt), ctx)
} }
} }
def taking(initiallyRemaining: Long) = new State { def taking(initiallyRemaining: Long) = new State {
var remaining: Long = initiallyRemaining var remaining: Long = initiallyRemaining
override def onPush(element: ByteString, ctx: Context[ByteString]): Directive = { override def onPush(element: ByteString, ctx: Context[ByteString]): Directive = {
val data = element.take(math.min(remaining, Int.MaxValue).toInt) val data = element.take(math.min(remaining, Int.MaxValue).toInt)
remaining -= data.size remaining -= data.size
@ -94,9 +93,10 @@ private[http] object StreamUtils {
Flow[ByteString].section(name("sliceBytes"))(_.transform(() transformer)) Flow[ByteString].section(name("sliceBytes"))(_.transform(() transformer))
} }
def limitByteChunksStage(maxBytesPerChunk: Int): Stage[ByteString, ByteString] = def limitByteChunksStage(maxBytesPerChunk: Int): PushPullStage[ByteString, ByteString] =
new StatefulStage[ByteString, ByteString] { new StatefulStage[ByteString, ByteString] {
def initial = WaitingForData def initial = WaitingForData
case object WaitingForData extends State { case object WaitingForData extends State {
def onPush(elem: ByteString, ctx: Context[ByteString]): Directive = def onPush(elem: ByteString, ctx: Context[ByteString]): Directive =
if (elem.size <= maxBytesPerChunk) ctx.push(elem) if (elem.size <= maxBytesPerChunk) ctx.push(elem)
@ -105,6 +105,7 @@ private[http] object StreamUtils {
ctx.push(elem.take(maxBytesPerChunk)) ctx.push(elem.take(maxBytesPerChunk))
} }
} }
case class DeliveringData(remaining: ByteString) extends State { case class DeliveringData(remaining: ByteString) extends State {
def onPush(elem: ByteString, ctx: Context[ByteString]): Directive = def onPush(elem: ByteString, ctx: Context[ByteString]): Directive =
throw new IllegalStateException("Not expecting data") throw new IllegalStateException("Not expecting data")
@ -133,23 +134,21 @@ private[http] object StreamUtils {
* Applies a sequence of transformers on one source and returns a sequence of sources with the result. The input source * Applies a sequence of transformers on one source and returns a sequence of sources with the result. The input source
* will only be traversed once. * will only be traversed once.
*/ */
def transformMultiple(input: Source[ByteString], transformers: immutable.Seq[Flow[ByteString, ByteString]])(implicit materializer: FlowMaterializer): immutable.Seq[Source[ByteString]] = def transformMultiple(input: Source[ByteString, Unit], transformers: immutable.Seq[Flow[ByteString, ByteString, _]])(implicit materializer: ActorFlowMaterializer): immutable.Seq[Source[ByteString, Unit]] =
transformers match { transformers match {
case Nil Nil case Nil Nil
case Seq(one) Vector(input.via(one)) case Seq(one) Vector(input.via(one))
case multiple case multiple
val results = Vector.fill(multiple.size)(Sink.publisher[ByteString]) val (fanoutSub, fanoutPub) = Source.subscriber[ByteString]().toMat(Sink.fanoutPublisher(16, 16))(Keep.both).run()
val mat = val sources = transformers.map { flow
FlowGraph { implicit b // Doubly wrap to ensure that subscription to the running publisher happens before the final sources
import FlowGraphImplicits._ // are exposed, so there is no race
Source(Source(fanoutPub).via(flow).runWith(Sink.publisher()))
val broadcast = Broadcast[ByteString](OperationAttributes.name("transformMultipleInputBroadcast"))
input ~> broadcast
(multiple, results).zipped.foreach { (trans, sink)
broadcast ~> trans ~> sink
} }
}.run() // The fanout publisher must be wired to the original source after all fanout subscribers have been subscribed
results.map(s Source(mat.get(s))) input.runWith(Sink(fanoutSub))
sources
} }
def mapEntityError(f: Throwable Throwable): RequestEntity RequestEntity = def mapEntityError(f: Throwable Throwable): RequestEntity RequestEntity =
@ -160,13 +159,20 @@ private[http] object StreamUtils {
* *
* FIXME: should be provided by akka-stream, see #15588 * FIXME: should be provided by akka-stream, see #15588
*/ */
def fromInputStreamSource(inputStream: InputStream, defaultChunkSize: Int = 65536): Source[ByteString] = { def fromInputStreamSource(inputStream: InputStream,
fileIODispatcher: String,
defaultChunkSize: Int = 65536): Source[ByteString, Unit] = {
import akka.stream.impl._ import akka.stream.impl._
def props(materializer: ActorFlowMaterializer): Props = { val onlyOnceFlag = new AtomicBoolean(false)
val iterator = new Iterator[ByteString] { val iterator = new Iterator[ByteString] {
var finished = false var finished = false
if (onlyOnceFlag.get() || !onlyOnceFlag.compareAndSet(false, true))
throw new IllegalStateException("One time source can only be instantiated once")
def hasNext: Boolean = !finished def hasNext: Boolean = !finished
def next(): ByteString = def next(): ByteString =
if (!finished) { if (!finished) {
val buffer = new Array[Byte](defaultChunkSize) val buffer = new Array[Byte](defaultChunkSize)
@ -179,99 +185,29 @@ private[http] object StreamUtils {
} else ByteString.empty } else ByteString.empty
} }
IteratorPublisher.props(iterator, materializer.settings).withDispatcher(materializer.settings.fileIODispatcher) Source(() iterator).withAttributes(OperationAttributes.dispatcher(fileIODispatcher))
}
new AtomicBoolean(false) with SimpleActorFlowSource[ByteString] {
override def attach(flowSubscriber: Subscriber[ByteString], materializer: ActorFlowMaterializer, flowName: String): Unit =
create(materializer, flowName)._1.subscribe(flowSubscriber)
override def isActive: Boolean = true
override def create(materializer: ActorFlowMaterializer, flowName: String): (Publisher[ByteString], Unit) =
if (!getAndSet(true)) {
val ref = materializer.actorOf(props(materializer), name = s"$flowName-0-InputStream-source")
val publisher = ActorPublisher[ByteString](ref)
ref ! ExposedPublisher(publisher.asInstanceOf[impl.ActorPublisher[Any]])
(publisher, ())
} else (ErrorPublisher(new IllegalStateException("One time source can only be instantiated once"), "failed").asInstanceOf[Publisher[ByteString]], ())
}
} }
/** /**
* Returns a source that can only be used once for testing purposes. * Returns a source that can only be used once for testing purposes.
*/ */
def oneTimeSource[T](other: Source[T]): Source[T] = { def oneTimeSource[T, Mat](other: Source[T, Mat]): Source[T, Mat] = {
import akka.stream.impl._ val onlyOnceFlag = new AtomicBoolean(false)
val original = other.asInstanceOf[ActorFlowSource[T]] other.map { elem
new AtomicBoolean(false) with SimpleActorFlowSource[T] { if (onlyOnceFlag.get() || !onlyOnceFlag.compareAndSet(false, true))
override def attach(flowSubscriber: Subscriber[T], materializer: ActorFlowMaterializer, flowName: String): Unit = throw new IllegalStateException("One time source can only be instantiated once")
create(materializer, flowName)._1.subscribe(flowSubscriber) elem
override def isActive: Boolean = true
override def create(materializer: ActorFlowMaterializer, flowName: String): (Publisher[T], Unit) =
if (!getAndSet(true)) (original.create(materializer, flowName)._1, ())
else (ErrorPublisher(new IllegalStateException("One time source can only be instantiated once"), "failed").asInstanceOf[Publisher[T]], ())
} }
} }
def runStrict(sourceData: ByteString, transformer: Flow[ByteString, ByteString], maxByteSize: Long, maxElements: Int): Try[Option[ByteString]] =
runStrict(Iterator.single(sourceData), transformer, maxByteSize, maxElements)
def runStrict(sourceData: Iterator[ByteString], transformer: Flow[ByteString, ByteString], maxByteSize: Long, maxElements: Int): Try[Option[ByteString]] =
Try {
transformer match {
// FIXME #16382 right now the flow can't use keys, should that be allowed?
case Pipe(ops, keys, _) if keys.isEmpty
if (ops.isEmpty)
Some(sourceData.foldLeft(ByteString.empty)(_ ++ _))
else {
@tailrec def tryBuild(remaining: List[AstNode], acc: List[PushPullStage[ByteString, ByteString]]): List[PushPullStage[ByteString, ByteString]] =
remaining match {
case Nil acc.reverse
case StageFactory(mkStage, _) :: tail
mkStage() match {
case d: PushPullStage[ByteString, ByteString]
tryBuild(tail, d :: acc)
case _ Nil
}
case _ Nil
}
val strictOps = tryBuild(ops, Nil)
if (strictOps.isEmpty)
None
else {
val iter: Iterator[ByteString] = new IteratorInterpreter(sourceData, strictOps).iterator
var byteSize = 0L
var result = ByteString.empty
var i = 0
// note that iter.next() will throw exception if the stream fails, caught by the enclosing Try
while (iter.hasNext) {
i += 1
if (i > maxElements)
throw new IllegalArgumentException(s"Too many elements produced by byte transformation, $i was greater than max allowed $maxElements elements")
val elem = iter.next()
byteSize += elem.size
if (byteSize > maxByteSize)
throw new IllegalArgumentException(s"Too large data result, $byteSize bytes was greater than max allowed $maxByteSize bytes")
result ++= elem
}
Some(result)
}
}
case _ None
}
}
} }
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[http] class EnhancedByteStringSource(val byteStringStream: Source[ByteString]) extends AnyVal { private[http] class EnhancedByteStringSource[Mat](val byteStringStream: Source[ByteString, Mat]) extends AnyVal {
def join(implicit materializer: FlowMaterializer): Future[ByteString] = def join(implicit materializer: ActorFlowMaterializer): Future[ByteString] =
byteStringStream.runFold(ByteString.empty)(_ ++ _) byteStringStream.runFold(ByteString.empty)(_ ++ _)
def utf8String(implicit materializer: FlowMaterializer, ec: ExecutionContext): Future[String] = def utf8String(implicit materializer: ActorFlowMaterializer, ec: ExecutionContext): Future[String] =
join.map(_.utf8String) join.map(_.utf8String)
} }

View file

@ -9,7 +9,7 @@ import language.higherKinds
import scala.collection.immutable import scala.collection.immutable
import java.nio.charset.Charset import java.nio.charset.Charset
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.stream.{ FlowMaterializer, FlattenStrategy } import akka.stream.{ ActorFlowMaterializer, FlattenStrategy }
import akka.stream.scaladsl.{ Flow, Source } import akka.stream.scaladsl.{ Flow, Source }
import akka.stream.stage._ import akka.stream.stage._
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
@ -40,22 +40,22 @@ package object util {
private[http] implicit def enhanceRegex(regex: Regex): EnhancedRegex = new EnhancedRegex(regex) private[http] implicit def enhanceRegex(regex: Regex): EnhancedRegex = new EnhancedRegex(regex)
private[http] implicit def enhanceByteStrings(byteStrings: TraversableOnce[ByteString]): EnhancedByteStringTraversableOnce = private[http] implicit def enhanceByteStrings(byteStrings: TraversableOnce[ByteString]): EnhancedByteStringTraversableOnce =
new EnhancedByteStringTraversableOnce(byteStrings) new EnhancedByteStringTraversableOnce(byteStrings)
private[http] implicit def enhanceByteStrings(byteStrings: Source[ByteString]): EnhancedByteStringSource = private[http] implicit def enhanceByteStrings[Mat](byteStrings: Source[ByteString, Mat]): EnhancedByteStringSource[Mat] =
new EnhancedByteStringSource(byteStrings) new EnhancedByteStringSource(byteStrings)
private[http] implicit class SourceWithHeadAndTail[T](val underlying: Source[Source[T]]) extends AnyVal { private[http] implicit class SourceWithHeadAndTail[T, Mat](val underlying: Source[Source[T, Unit], Mat]) extends AnyVal {
def headAndTail: Source[(T, Source[T])] = def headAndTail: Source[(T, Source[T, Unit]), Mat] =
underlying.map { _.prefixAndTail(1).map { case (prefix, tail) (prefix.head, tail) } } underlying.map { _.prefixAndTail(1).map { case (prefix, tail) (prefix.head, tail) } }
.flatten(FlattenStrategy.concat) .flatten(FlattenStrategy.concat)
} }
private[http] implicit class FlowWithHeadAndTail[In, Out](val underlying: Flow[In, Source[Out]]) extends AnyVal { private[http] implicit class FlowWithHeadAndTail[In, Out, Mat](val underlying: Flow[In, Source[Out, Unit], Mat]) extends AnyVal {
def headAndTail: Flow[In, (Out, Source[Out])] = def headAndTail: Flow[In, (Out, Source[Out, Unit]), Mat] =
underlying.map { _.prefixAndTail(1).map { case (prefix, tail) (prefix.head, tail) } } underlying.map { _.prefixAndTail(1).map { case (prefix, tail) (prefix.head, tail) } }
.flatten(FlattenStrategy.concat) .flatten(FlattenStrategy.concat)
} }
def printEvent[T](marker: String): Flow[T, T] = def printEvent[T](marker: String): Flow[T, T, Unit] =
Flow[T].transform(() new PushStage[T, T] { Flow[T].transform(() new PushStage[T, T] {
override def onPush(element: T, ctx: Context[T]): Directive = { override def onPush(element: T, ctx: Context[T]): Directive = {
println(s"$marker: $element") println(s"$marker: $element")

View file

@ -6,18 +6,6 @@ package akka.http.model.japi;
import static akka.pattern.Patterns.ask; import static akka.pattern.Patterns.ask;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.dispatch.Foreach;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.stream.FlowMaterializer;
import scala.concurrent.Future;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
public abstract class JavaTestServer { public abstract class JavaTestServer {
// FIXME Java Http API // FIXME Java Http API

View file

@ -6,6 +6,7 @@ package akka.http
import java.io.{ BufferedReader, BufferedWriter, InputStreamReader, OutputStreamWriter } import java.io.{ BufferedReader, BufferedWriter, InputStreamReader, OutputStreamWriter }
import java.net.Socket import java.net.Socket
import akka.stream.impl.{ PublisherSink, SubscriberSource }
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.concurrent.Await import scala.concurrent.Await
@ -39,12 +40,10 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
"properly bind a server" in { "properly bind a server" in {
val (hostname, port) = temporaryServerHostnameAndPort() val (hostname, port) = temporaryServerHostnameAndPort()
val binding = Http().bind(hostname, port)
val probe = StreamTestKit.SubscriberProbe[Http.IncomingConnection]() val probe = StreamTestKit.SubscriberProbe[Http.IncomingConnection]()
val mm = binding.connections.to(Sink(probe)).run() val binding = Http().bind(hostname, port).toMat(Sink(probe))(Keep.left).run()
val sub = probe.expectSubscription() val sub = probe.expectSubscription() // if we get it we are bound
// if the future finishes successfully, we are bound val address = Await.result(binding, 1.second).localAddress
val address = Await.result(binding.localAddress(mm), 1.second)
sub.cancel() sub.cancel()
} }
@ -52,40 +51,30 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
val (hostname, port) = temporaryServerHostnameAndPort() val (hostname, port) = temporaryServerHostnameAndPort()
val binding = Http().bind(hostname, port) val binding = Http().bind(hostname, port)
val probe1 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]() val probe1 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]()
val mm1 = binding.connections.to(Sink(probe1)).run() // Bind succeeded, we have a local address
val b1 = Await.result(binding.to(Sink(probe1)).run(), 3.seconds)
probe1.expectSubscription() probe1.expectSubscription()
// Bind succeeded, we have a local address
Await.result(binding.localAddress(mm1), 1.second)
val probe2 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]() val probe2 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]()
val mm2 = binding.connections.to(Sink(probe2)).run() an[BindFailedException] shouldBe thrownBy { Await.result(binding.to(Sink(probe2)).run(), 3.seconds) }
probe2.expectErrorOrSubscriptionFollowedByError() probe2.expectErrorOrSubscriptionFollowedByError()
val probe3 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]() val probe3 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]()
val mm3 = binding.connections.to(Sink(probe3)).run() an[BindFailedException] shouldBe thrownBy { Await.result(binding.to(Sink(probe3)).run(), 3.seconds) }
probe3.expectErrorOrSubscriptionFollowedByError() probe3.expectErrorOrSubscriptionFollowedByError()
an[BindFailedException] shouldBe thrownBy { Await.result(binding.localAddress(mm2), 1.second) }
an[BindFailedException] shouldBe thrownBy { Await.result(binding.localAddress(mm3), 1.second) }
// The unbind should NOT fail even though the bind failed.
Await.result(binding.unbind(mm2), 1.second)
Await.result(binding.unbind(mm3), 1.second)
// Now unbind the first // Now unbind the first
Await.result(binding.unbind(mm1), 1.second) Await.result(b1.unbind(), 1.second)
probe1.expectComplete() probe1.expectComplete()
if (!akka.util.Helpers.isWindows) { if (!akka.util.Helpers.isWindows) {
val probe4 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]() val probe4 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]()
val mm4 = binding.connections.to(Sink(probe4)).run() // Bind succeeded, we have a local address
val b2 = Await.result(binding.to(Sink(probe4)).run(), 3.seconds)
probe4.expectSubscription() probe4.expectSubscription()
// Bind succeeded, we have a local address
Await.result(binding.localAddress(mm4), 1.second)
// clean up // clean up
Await.result(binding.unbind(mm4), 1.second) Await.result(b2.unbind(), 1.second)
} }
} }
@ -134,7 +123,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
private val HttpRequest(POST, uri, List(Accept(Seq(MediaRanges.`*/*`)), Host(_, _), `User-Agent`(_)), private val HttpRequest(POST, uri, List(Accept(Seq(MediaRanges.`*/*`)), Host(_, _), `User-Agent`(_)),
Chunked(`chunkedContentType`, chunkStream), HttpProtocols.`HTTP/1.1`) = serverIn.expectNext() Chunked(`chunkedContentType`, chunkStream), HttpProtocols.`HTTP/1.1`) = serverIn.expectNext()
uri shouldEqual Uri(s"http://$hostname:$port/chunked") uri shouldEqual Uri(s"http://$hostname:$port/chunked")
Await.result(chunkStream.grouped(4).runWith(Sink.head), 100.millis) shouldEqual chunks Await.result(chunkStream.grouped(4).runWith(Sink.head()), 100.millis) shouldEqual chunks
val serverOutSub = serverOut.expectSubscription() val serverOutSub = serverOut.expectSubscription()
serverOutSub.expectRequest() serverOutSub.expectRequest()
@ -144,7 +133,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
clientInSub.request(1) clientInSub.request(1)
val HttpResponse(StatusCodes.PartialContent, List(Age(42), Server(_), Date(_)), val HttpResponse(StatusCodes.PartialContent, List(Age(42), Server(_), Date(_)),
Chunked(`chunkedContentType`, chunkStream2), HttpProtocols.`HTTP/1.1`) = clientIn.expectNext() Chunked(`chunkedContentType`, chunkStream2), HttpProtocols.`HTTP/1.1`) = clientIn.expectNext()
Await.result(chunkStream2.grouped(1000).runWith(Sink.head), 100.millis) shouldEqual chunks Await.result(chunkStream2.grouped(1000).runWith(Sink.head()), 100.millis) shouldEqual chunks
clientOutSub.sendComplete() clientOutSub.sendComplete()
serverInSub.request(1) // work-around for #16552 serverInSub.request(1) // work-around for #16552
@ -194,7 +183,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
val settings = configOverrides.toOption.map(ServerSettings.apply) val settings = configOverrides.toOption.map(ServerSettings.apply)
val binding = Http().bind(hostname, port, settings = settings) val binding = Http().bind(hostname, port, settings = settings)
val probe = StreamTestKit.SubscriberProbe[Http.IncomingConnection] val probe = StreamTestKit.SubscriberProbe[Http.IncomingConnection]
binding.connections.runWith(Sink(probe)) binding.runWith(Sink(probe))
probe probe
} }
val connSourceSub = connSource.expectSubscription() val connSourceSub = connSource.expectSubscription()
@ -202,23 +191,35 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
def openNewClientConnection(settings: Option[ClientConnectionSettings] = None): (PublisherProbe[HttpRequest], SubscriberProbe[HttpResponse]) = { def openNewClientConnection(settings: Option[ClientConnectionSettings] = None): (PublisherProbe[HttpRequest], SubscriberProbe[HttpResponse]) = {
val requestPublisherProbe = StreamTestKit.PublisherProbe[HttpRequest]() val requestPublisherProbe = StreamTestKit.PublisherProbe[HttpRequest]()
val responseSubscriberProbe = StreamTestKit.SubscriberProbe[HttpResponse]() val responseSubscriberProbe = StreamTestKit.SubscriberProbe[HttpResponse]()
val connection = Http().outgoingConnection(hostname, port, settings = settings)
val connectionFuture = Source(requestPublisherProbe)
.viaMat(Http().outgoingConnection(hostname, port, settings = settings))(Keep.right)
.to(Sink(responseSubscriberProbe)).run()
val connection = Await.result(connectionFuture, 3.seconds)
connection.remoteAddress.getHostName shouldEqual hostname connection.remoteAddress.getHostName shouldEqual hostname
connection.remoteAddress.getPort shouldEqual port connection.remoteAddress.getPort shouldEqual port
Source(requestPublisherProbe).via(connection.flow).runWith(Sink(responseSubscriberProbe))
requestPublisherProbe -> responseSubscriberProbe requestPublisherProbe -> responseSubscriberProbe
} }
def acceptConnection(): (SubscriberProbe[HttpRequest], PublisherProbe[HttpResponse]) = { def acceptConnection(): (SubscriberProbe[HttpRequest], PublisherProbe[HttpResponse]) = {
connSourceSub.request(1) connSourceSub.request(1)
val incomingConnection = connSource.expectNext() val incomingConnection = connSource.expectNext()
val sink = PublisherSink[HttpRequest]() val sink = Sink.publisher[HttpRequest]
val source = SubscriberSource[HttpResponse]() val source = Source.subscriber[HttpResponse]
val mm = incomingConnection.handleWith(Flow(sink, source))
val handler = Flow(sink, source)(Keep.both) { implicit b
(snk, src)
(snk.inlet, src.outlet)
}
val (pub, sub) = incomingConnection.handleWith(handler)
val requestSubscriberProbe = StreamTestKit.SubscriberProbe[HttpRequest]() val requestSubscriberProbe = StreamTestKit.SubscriberProbe[HttpRequest]()
val responsePublisherProbe = StreamTestKit.PublisherProbe[HttpResponse]() val responsePublisherProbe = StreamTestKit.PublisherProbe[HttpResponse]()
mm.get(sink).subscribe(requestSubscriberProbe)
responsePublisherProbe.subscribe(mm.get(source)) pub.subscribe(requestSubscriberProbe)
responsePublisherProbe.subscribe(sub)
requestSubscriberProbe -> responsePublisherProbe requestSubscriberProbe -> responsePublisherProbe
} }

View file

@ -8,7 +8,7 @@ import com.typesafe.config.{ Config, ConfigFactory }
import scala.util.{ Failure, Success } import scala.util.{ Failure, Success }
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.{ Sink, Source } import akka.stream.scaladsl.{ Keep, Sink, Source }
import akka.http.model._ import akka.http.model._
object TestClient extends App { object TestClient extends App {
@ -25,7 +25,7 @@ object TestClient extends App {
println(s"Fetching HTTP server version of host `$host` ...") println(s"Fetching HTTP server version of host `$host` ...")
val connection = Http().outgoingConnection(host) val connection = Http().outgoingConnection(host)
val result = Source.single(HttpRequest()).via(connection.flow).runWith(Sink.head) val result = Source.single(HttpRequest()).via(connection).runWith(Sink.head())
result.map(_.header[headers.Server]) onComplete { result.map(_.header[headers.Server]) onComplete {
case Success(res) println(s"$host is running ${res mkString ", "}") case Success(res) println(s"$host is running ${res mkString ", "}")

View file

@ -19,14 +19,12 @@ object TestServer extends App {
implicit val system = ActorSystem("ServerTest", testConf) implicit val system = ActorSystem("ServerTest", testConf)
implicit val fm = ActorFlowMaterializer() implicit val fm = ActorFlowMaterializer()
val binding = Http().bind(interface = "localhost", port = 8080) val binding = Http().bindAndStartHandlingWithSyncHandler({
binding startHandlingWithSyncHandler {
case HttpRequest(GET, Uri.Path("/"), _, _, _) index case HttpRequest(GET, Uri.Path("/"), _, _, _) index
case HttpRequest(GET, Uri.Path("/ping"), _, _, _) HttpResponse(entity = "PONG!") case HttpRequest(GET, Uri.Path("/ping"), _, _, _) HttpResponse(entity = "PONG!")
case HttpRequest(GET, Uri.Path("/crash"), _, _, _) sys.error("BOOM!") case HttpRequest(GET, Uri.Path("/crash"), _, _, _) sys.error("BOOM!")
case _: HttpRequest HttpResponse(404, entity = "Unknown resource!") case _: HttpRequest HttpResponse(404, entity = "Unknown resource!")
} }, interface = "localhost", port = 8080)
println(s"Server online at http://localhost:8080") println(s"Server online at http://localhost:8080")
println("Press RETURN to stop...") println("Press RETURN to stop...")

View file

@ -358,9 +358,16 @@ class HttpClientSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
val (netOut, netIn) = { val (netOut, netIn) = {
val netOut = StreamTestKit.SubscriberProbe[ByteString] val netOut = StreamTestKit.SubscriberProbe[ByteString]
val netIn = StreamTestKit.PublisherProbe[ByteString] val netIn = StreamTestKit.PublisherProbe[ByteString]
val clientFlow = HttpClient.transportToConnectionClientFlow(
Flow(Sink(netOut), Source(netIn)), remoteAddress, settings, NoLogging) FlowGraph.closed(HttpClient.clientBlueprint(remoteAddress, settings, NoLogging)) { implicit b
Source(requests).via(clientFlow).runWith(Sink(responses)) client
import FlowGraph.Implicits._
Source(netIn) ~> client.bytesIn
client.bytesOut ~> Sink(netOut)
Source(requests) ~> client.httpRequests
client.httpResponses ~> Sink(responses)
}.run()
netOut -> netIn netOut -> netIn
} }

View file

@ -233,7 +233,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
val parser = newParser val parser = newParser
val result = multiParse(newParser)(Seq(prep(start + manyChunks))) val result = multiParse(newParser)(Seq(prep(start + manyChunks)))
val HttpEntity.Chunked(_, chunks) = result.head.right.get.req.entity val HttpEntity.Chunked(_, chunks) = result.head.right.get.req.entity
val strictChunks = chunks.grouped(100000).runWith(Sink.head).awaitResult(awaitAtMost) val strictChunks = chunks.grouped(100000).runWith(Sink.head()).awaitResult(awaitAtMost)
strictChunks.size shouldEqual numChunks strictChunks.size shouldEqual numChunks
} }
} }
@ -462,7 +462,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
} }
.flatten(FlattenStrategy.concat) .flatten(FlattenStrategy.concat)
.map(strictEqualify) .map(strictEqualify)
.grouped(100000).runWith(Sink.head) .grouped(100000).runWith(Sink.head())
.awaitResult(awaitAtMost) .awaitResult(awaitAtMost)
protected def parserSettings: ParserSettings = ParserSettings(system) protected def parserSettings: ParserSettings = ParserSettings(system)
@ -474,12 +474,12 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
case _ entity.toStrict(awaitAtMost) case _ entity.toStrict(awaitAtMost)
} }
private def compactEntityChunks(data: Source[ChunkStreamPart]): Future[Seq[ChunkStreamPart]] = private def compactEntityChunks(data: Source[ChunkStreamPart, Unit]): Future[Seq[ChunkStreamPart]] =
data.grouped(100000).runWith(Sink.head) data.grouped(100000).runWith(Sink.head())
.fast.recover { case _: NoSuchElementException Nil } .fast.recover { case _: NoSuchElementException Nil }
def prep(response: String) = response.stripMarginWithNewline("\r\n") def prep(response: String) = response.stripMarginWithNewline("\r\n")
} }
def source[T](elems: T*): Source[T] = Source(elems.toList) def source[T](elems: T*): Source[T, Unit] = Source(elems.toList)
} }

View file

@ -279,7 +279,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
} }
.flatten(FlattenStrategy.concat) .flatten(FlattenStrategy.concat)
.map(strictEqualify) .map(strictEqualify)
.grouped(100000).runWith(Sink.head) .grouped(100000).runWith(Sink.head())
Await.result(future, 500.millis) Await.result(future, 500.millis)
} }
@ -297,13 +297,13 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
case _ entity.toStrict(250.millis) case _ entity.toStrict(250.millis)
} }
private def compactEntityChunks(data: Source[ChunkStreamPart]): Future[Source[ChunkStreamPart]] = private def compactEntityChunks(data: Source[ChunkStreamPart, Unit]): Future[Source[ChunkStreamPart, Unit]] =
data.grouped(100000).runWith(Sink.head) data.grouped(100000).runWith(Sink.head())
.fast.map(source(_: _*)) .fast.map(source(_: _*))
.fast.recover { case _: NoSuchElementException source() } .fast.recover { case _: NoSuchElementException source() }
def prep(response: String) = response.stripMarginWithNewline("\r\n") def prep(response: String) = response.stripMarginWithNewline("\r\n")
def source[T](elems: T*): Source[T] = Source(elems.toList) def source[T](elems: T*): Source[T, Unit] = Source(elems.toList)
} }
} }

View file

@ -21,6 +21,7 @@ import akka.stream.ActorFlowMaterializer
import akka.stream.impl.SynchronousIterablePublisher import akka.stream.impl.SynchronousIterablePublisher
import HttpEntity._ import HttpEntity._
import HttpMethods._ import HttpMethods._
import akka.util.ByteString
class RequestRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll { class RequestRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
val testConf: Config = ConfigFactory.parseString(""" val testConf: Config = ConfigFactory.parseString("""
@ -255,8 +256,8 @@ class RequestRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll
val renderer = newRenderer val renderer = newRenderer
val byteStringSource = Await.result(Source.single(RequestRenderingContext(request, serverAddress)). val byteStringSource = Await.result(Source.single(RequestRenderingContext(request, serverAddress)).
section(name("renderer"))(_.transform(() renderer)). section(name("renderer"))(_.transform(() renderer)).
runWith(Sink.head), 1.second) runWith(Sink.head()), 1.second)
val future = byteStringSource.grouped(1000).runWith(Sink.head).map(_.reduceLeft(_ ++ _).utf8String) val future = byteStringSource.grouped(1000).runWith(Sink.head()).map(_.reduceLeft(_ ++ _).utf8String)
Await.result(future, 250.millis) Await.result(future, 250.millis)
} }
} }

View file

@ -413,8 +413,8 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll
val renderer = newRenderer val renderer = newRenderer
val byteStringSource = Await.result(Source.single(ctx). val byteStringSource = Await.result(Source.single(ctx).
section(name("renderer"))(_.transform(() renderer)). section(name("renderer"))(_.transform(() renderer)).
runWith(Sink.head), 1.second) runWith(Sink.head()), 1.second)
val future = byteStringSource.grouped(1000).runWith(Sink.head).map(_.reduceLeft(_ ++ _).utf8String) val future = byteStringSource.grouped(1000).runWith(Sink.head()).map(_.reduceLeft(_ ++ _).utf8String)
Await.result(future, 250.millis) -> renderer.isComplete Await.result(future, 250.millis) -> renderer.isComplete
} }

View file

@ -659,8 +659,16 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
val (netIn, netOut) = { val (netIn, netOut) = {
val netIn = StreamTestKit.PublisherProbe[ByteString] val netIn = StreamTestKit.PublisherProbe[ByteString]
val netOut = StreamTestKit.SubscriberProbe[ByteString] val netOut = StreamTestKit.SubscriberProbe[ByteString]
val transportFlow = HttpServer.serverFlowToTransport(Flow(Sink(requests), Source(responses)), settings, NoLogging)
Source(netIn).via(transportFlow).runWith(Sink(netOut)) FlowGraph.closed(HttpServer.serverBlueprint(settings, NoLogging)) { implicit b
server
import FlowGraph.Implicits._
Source(netIn) ~> server.bytesIn
server.bytesOut ~> Sink(netOut)
server.httpRequests ~> Sink(requests)
Source(responses) ~> server.httpResponses
}.run()
netIn -> netOut netIn -> netOut
} }

View file

@ -107,7 +107,7 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
def collectBytesTo(bytes: ByteString*): Matcher[HttpEntity] = def collectBytesTo(bytes: ByteString*): Matcher[HttpEntity] =
equal(bytes.toVector).matcher[Seq[ByteString]].compose { entity equal(bytes.toVector).matcher[Seq[ByteString]].compose { entity
val future = entity.dataBytes.grouped(1000).runWith(Sink.head) val future = entity.dataBytes.grouped(1000).runWith(Sink.head())
Await.result(future, 250.millis) Await.result(future, 250.millis)
} }
@ -120,7 +120,7 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
Await.result(transformed.toStrict(250.millis), 250.millis) Await.result(transformed.toStrict(250.millis), 250.millis)
} }
def duplicateBytesTransformer(): Flow[ByteString, ByteString] = def duplicateBytesTransformer(): Flow[ByteString, ByteString, Unit] =
Flow[ByteString].transform(() StreamUtils.byteStringTransformer(doubleChars, () trailer)) Flow[ByteString].transform(() StreamUtils.byteStringTransformer(doubleChars, () trailer))
def trailer: ByteString = ByteString("--dup") def trailer: ByteString = ByteString("--dup")

View file

@ -11,18 +11,18 @@ import scala.concurrent.{ ExecutionContext, Await }
import akka.http.marshalling._ import akka.http.marshalling._
import akka.http.model.HttpEntity import akka.http.model.HttpEntity
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import scala.util.Try import scala.util.Try
trait MarshallingTestUtils { trait MarshallingTestUtils {
def marshal[T: ToEntityMarshaller](value: T)(implicit ec: ExecutionContext, mat: FlowMaterializer): HttpEntity.Strict = def marshal[T: ToEntityMarshaller](value: T)(implicit ec: ExecutionContext, mat: ActorFlowMaterializer): HttpEntity.Strict =
Await.result(Marshal(value).to[HttpEntity].flatMap(_.toStrict(1.second)), 1.second) Await.result(Marshal(value).to[HttpEntity].flatMap(_.toStrict(1.second)), 1.second)
def unmarshalValue[T: FromEntityUnmarshaller](entity: HttpEntity)(implicit ec: ExecutionContext, mat: FlowMaterializer): T = def unmarshalValue[T: FromEntityUnmarshaller](entity: HttpEntity)(implicit ec: ExecutionContext, mat: ActorFlowMaterializer): T =
unmarshal(entity).get unmarshal(entity).get
def unmarshal[T: FromEntityUnmarshaller](entity: HttpEntity)(implicit ec: ExecutionContext, mat: FlowMaterializer): Try[T] = { def unmarshal[T: FromEntityUnmarshaller](entity: HttpEntity)(implicit ec: ExecutionContext, mat: ActorFlowMaterializer): Try[T] = {
val fut = Unmarshal(entity).to[T] val fut = Unmarshal(entity).to[T]
Await.ready(fut, 1.second) Await.ready(fut, 1.second)
fut.value.get fut.value.get

View file

@ -12,7 +12,6 @@ import scala.util.DynamicVariable
import scala.reflect.ClassTag import scala.reflect.ClassTag
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.FlowMaterializer
import akka.http.client.RequestBuilding import akka.http.client.RequestBuilding
import akka.http.util.FastFuture import akka.http.util.FastFuture
import akka.http.server._ import akka.http.server._

View file

@ -9,7 +9,7 @@ import scala.collection.immutable
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext
import akka.http.util._ import akka.http.util._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.http.model.HttpEntity.ChunkStreamPart import akka.http.model.HttpEntity.ChunkStreamPart
import akka.http.server._ import akka.http.server._
@ -22,7 +22,7 @@ trait RouteTestResultComponent {
/** /**
* A receptacle for the response or rejections created by a route. * A receptacle for the response or rejections created by a route.
*/ */
class RouteTestResult(timeout: FiniteDuration)(implicit fm: FlowMaterializer) { class RouteTestResult(timeout: FiniteDuration)(implicit fm: ActorFlowMaterializer) {
private[this] var result: Option[Either[immutable.Seq[Rejection], HttpResponse]] = None private[this] var result: Option[Either[immutable.Seq[Rejection], HttpResponse]] = None
private[this] val latch = new CountDownLatch(1) private[this] val latch = new CountDownLatch(1)
@ -95,7 +95,7 @@ trait RouteTestResultComponent {
private def failNeitherCompletedNorRejected(): Nothing = private def failNeitherCompletedNorRejected(): Nothing =
failTest("Request was neither completed nor rejected within " + timeout) failTest("Request was neither completed nor rejected within " + timeout)
private def awaitAllElements[T](data: Source[T]): immutable.Seq[T] = private def awaitAllElements[T](data: Source[T, _]): immutable.Seq[T] =
data.grouped(100000).runWith(Sink.head).awaitResult(timeout) data.grouped(100000).runWith(Sink.head()).awaitResult(timeout)
} }
} }

View file

@ -6,7 +6,7 @@ package akka.http.testkit
import akka.http.model.HttpEntity import akka.http.model.HttpEntity
import akka.http.unmarshalling.FromEntityUnmarshaller import akka.http.unmarshalling.FromEntityUnmarshaller
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import org.scalatest.Suite import org.scalatest.Suite
import org.scalatest.matchers.Matcher import org.scalatest.matchers.Matcher
@ -22,10 +22,10 @@ trait ScalatestUtils extends MarshallingTestUtils {
def haveFailedWith(t: Throwable): Matcher[Future[_]] = def haveFailedWith(t: Throwable): Matcher[Future[_]] =
equal(t).matcher[Throwable] compose (x Await.result(x.failed, 1.second)) equal(t).matcher[Throwable] compose (x Await.result(x.failed, 1.second))
def unmarshalToValue[T: FromEntityUnmarshaller](value: T)(implicit ec: ExecutionContext, mat: FlowMaterializer): Matcher[HttpEntity] = def unmarshalToValue[T: FromEntityUnmarshaller](value: T)(implicit ec: ExecutionContext, mat: ActorFlowMaterializer): Matcher[HttpEntity] =
equal(value).matcher[T] compose (unmarshalValue(_)) equal(value).matcher[T] compose (unmarshalValue(_))
def unmarshalTo[T: FromEntityUnmarshaller](value: Try[T])(implicit ec: ExecutionContext, mat: FlowMaterializer): Matcher[HttpEntity] = def unmarshalTo[T: FromEntityUnmarshaller](value: Try[T])(implicit ec: ExecutionContext, mat: ActorFlowMaterializer): Matcher[HttpEntity] =
equal(value).matcher[Try[T]] compose (unmarshal(_)) equal(value).matcher[Try[T]] compose (unmarshal(_))
} }

View file

@ -19,6 +19,8 @@ import akka.http.model.HttpMethods._
import akka.http.model.{ HttpEntity, HttpRequest } import akka.http.model.{ HttpEntity, HttpRequest }
import akka.stream.scaladsl.{ Sink, Source } import akka.stream.scaladsl.{ Sink, Source }
import akka.util.ByteString import akka.util.ByteString
import scala.concurrent.Await
import scala.concurrent.ExecutionContext.Implicits.global
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
@ -53,15 +55,17 @@ abstract class CoderSpec extends WordSpec with CodecSpecSupport with Inspectors
} }
"properly round-trip encode/decode an HttpRequest" in { "properly round-trip encode/decode an HttpRequest" in {
val request = HttpRequest(POST, entity = HttpEntity(largeText)) val request = HttpRequest(POST, entity = HttpEntity(largeText))
Coder.decode(Coder.encode(request)) should equal(request) Coder.decode(Coder.encode(request)).toStrict(1.second).awaitResult(1.second) should equal(request)
} }
if (corruptInputCheck) { if (corruptInputCheck) {
"throw an error on corrupt input" in { "throw an error on corrupt input" in {
a[DataFormatException] should be thrownBy { (the[RuntimeException] thrownBy {
ourDecode(corruptContent) ourDecode(corruptContent)
}).getCause should be(a[DataFormatException])
} }
} }
}
"not throw an error if a subsequent block is corrupt" in { "not throw an error if a subsequent block is corrupt" in {
pending // FIXME: should we read as long as possible and only then report an error, that seems somewhat arbitrary pending // FIXME: should we read as long as possible and only then report an error, that seems somewhat arbitrary
ourDecode(Seq(encode("Hello,"), encode(" dear "), corruptContent).join) should readAs("Hello, dear ") ourDecode(Seq(encode("Hello,"), encode(" dear "), corruptContent).join) should readAs("Hello, dear ")
@ -75,7 +79,7 @@ abstract class CoderSpec extends WordSpec with CodecSpecSupport with Inspectors
val chunks = largeTextBytes.grouped(512).toVector val chunks = largeTextBytes.grouped(512).toVector
val comp = Coder.newCompressor val comp = Coder.newCompressor
val compressedChunks = chunks.map { chunk comp.compressAndFlush(chunk) } :+ comp.finish() val compressedChunks = chunks.map { chunk comp.compressAndFlush(chunk) } :+ comp.finish()
val uncompressed = Coder.decodeFromIterator(compressedChunks.iterator) val uncompressed = decodeFromIterator(() compressedChunks.iterator)
uncompressed should readAs(largeText) uncompressed should readAs(largeText)
} }
@ -107,7 +111,7 @@ abstract class CoderSpec extends WordSpec with CodecSpecSupport with Inspectors
val resultBs = val resultBs =
Source.single(compressed) Source.single(compressed)
.via(Coder.withMaxBytesPerChunk(limit).decoderFlow) .via(Coder.withMaxBytesPerChunk(limit).decoderFlow)
.grouped(4200).runWith(Sink.head) .grouped(4200).runWith(Sink.head())
.awaitResult(1.second) .awaitResult(1.second)
forAll(resultBs) { bs forAll(resultBs) { bs
@ -119,7 +123,7 @@ abstract class CoderSpec extends WordSpec with CodecSpecSupport with Inspectors
def encode(s: String) = ourEncode(ByteString(s, "UTF8")) def encode(s: String) = ourEncode(ByteString(s, "UTF8"))
def ourEncode(bytes: ByteString): ByteString = Coder.encode(bytes) def ourEncode(bytes: ByteString): ByteString = Coder.encode(bytes)
def ourDecode(bytes: ByteString): ByteString = Coder.decode(bytes) def ourDecode(bytes: ByteString): ByteString = Coder.decode(bytes).awaitResult(1.second)
lazy val corruptContent = { lazy val corruptContent = {
val content = encode(largeText).toArray val content = encode(largeText).toArray
@ -150,6 +154,9 @@ abstract class CoderSpec extends WordSpec with CodecSpecSupport with Inspectors
ByteString(output.toByteArray) ByteString(output.toByteArray)
} }
def decodeChunks(input: Source[ByteString]): ByteString = def decodeChunks(input: Source[ByteString, _]): ByteString =
input.via(Coder.decoderFlow).join.awaitResult(3.seconds) input.via(Coder.decoderFlow).join.awaitResult(3.seconds)
def decodeFromIterator(iterator: () Iterator[ByteString]): ByteString =
Await.result(Source(iterator).via(Coder.decoderFlow).join, 3.seconds)
} }

View file

@ -10,6 +10,8 @@ import org.scalatest.WordSpec
import akka.http.model._ import akka.http.model._
import headers._ import headers._
import HttpMethods.POST import HttpMethods.POST
import akka.http.util._
import scala.concurrent.duration._
class DecoderSpec extends WordSpec with CodecSpecSupport { class DecoderSpec extends WordSpec with CodecSpecSupport {
@ -22,12 +24,12 @@ class DecoderSpec extends WordSpec with CodecSpecSupport {
val request = HttpRequest(POST, entity = HttpEntity(smallText), headers = List(`Content-Encoding`(DummyDecoder.encoding))) val request = HttpRequest(POST, entity = HttpEntity(smallText), headers = List(`Content-Encoding`(DummyDecoder.encoding)))
val decoded = DummyDecoder.decode(request) val decoded = DummyDecoder.decode(request)
decoded.headers shouldEqual Nil decoded.headers shouldEqual Nil
decoded.entity shouldEqual HttpEntity(dummyDecompress(smallText)) decoded.entity.toStrict(1.second).awaitResult(1.second) shouldEqual HttpEntity(dummyDecompress(smallText))
} }
} }
def dummyDecompress(s: String): String = dummyDecompress(ByteString(s, "UTF8")).decodeString("UTF8") def dummyDecompress(s: String): String = dummyDecompress(ByteString(s, "UTF8")).decodeString("UTF8")
def dummyDecompress(bytes: ByteString): ByteString = DummyDecoder.decode(bytes) def dummyDecompress(bytes: ByteString): ByteString = DummyDecoder.decode(bytes).awaitResult(1.second)
case object DummyDecoder extends StreamDecoder { case object DummyDecoder extends StreamDecoder {
val encoding = HttpEncodings.compress val encoding = HttpEncodings.compress

View file

@ -20,9 +20,9 @@ class DeflateSpec extends CoderSpec {
override def extraTests(): Unit = { override def extraTests(): Unit = {
"throw early if header is corrupt" in { "throw early if header is corrupt" in {
a[DataFormatException] should be thrownBy { (the[RuntimeException] thrownBy {
ourDecode(ByteString(0, 1, 2, 3, 4)) ourDecode(ByteString(0, 1, 2, 3, 4))
} }).getCause should be(a[DataFormatException])
} }
} }
} }

View file

@ -9,6 +9,8 @@ import org.scalatest.WordSpec
import akka.http.model._ import akka.http.model._
import headers._ import headers._
import HttpMethods.POST import HttpMethods.POST
import scala.concurrent.duration._
import akka.http.util._
class EncoderSpec extends WordSpec with CodecSpecSupport { class EncoderSpec extends WordSpec with CodecSpecSupport {
@ -21,7 +23,7 @@ class EncoderSpec extends WordSpec with CodecSpecSupport {
val request = HttpRequest(POST, entity = HttpEntity(smallText)) val request = HttpRequest(POST, entity = HttpEntity(smallText))
val encoded = DummyEncoder.encode(request) val encoded = DummyEncoder.encode(request)
encoded.headers shouldEqual List(`Content-Encoding`(DummyEncoder.encoding)) encoded.headers shouldEqual List(`Content-Encoding`(DummyEncoder.encoding))
encoded.entity shouldEqual HttpEntity(dummyCompress(smallText)) encoded.entity.toStrict(1.second).awaitResult(1.second) shouldEqual HttpEntity(dummyCompress(smallText))
} }
} }

View file

@ -22,18 +22,20 @@ class GzipSpec extends CoderSpec {
override def extraTests(): Unit = { override def extraTests(): Unit = {
"decode concatenated compressions" in { "decode concatenated compressions" in {
pending // FIXME: unbreak
ourDecode(Seq(encode("Hello, "), encode("dear "), encode("User!")).join) should readAs("Hello, dear User!") ourDecode(Seq(encode("Hello, "), encode("dear "), encode("User!")).join) should readAs("Hello, dear User!")
} }
"provide a better compression ratio than the standard Gzip/Gunzip streams" in { "provide a better compression ratio than the standard Gzip/Gunzip streams" in {
ourEncode(largeTextBytes).length should be < streamEncode(largeTextBytes).length ourEncode(largeTextBytes).length should be < streamEncode(largeTextBytes).length
} }
"throw an error on truncated input" in { "throw an error on truncated input" in {
pending // FIXME: unbreak
val ex = the[ZipException] thrownBy ourDecode(streamEncode(smallTextBytes).dropRight(5)) val ex = the[ZipException] thrownBy ourDecode(streamEncode(smallTextBytes).dropRight(5))
ex.getMessage should equal("Truncated GZIP stream") ex.getMessage should equal("Truncated GZIP stream")
} }
"throw early if header is corrupt" in { "throw early if header is corrupt" in {
val ex = the[ZipException] thrownBy ourDecode(ByteString(0, 1, 2, 3, 4)) val cause = (the[RuntimeException] thrownBy ourDecode(ByteString(0, 1, 2, 3, 4))).getCause
ex.getMessage should equal("Not in GZIP format") cause should (be(a[ZipException]) and have message "Not in GZIP format")
} }
} }
} }

View file

@ -28,9 +28,7 @@ object TestServer extends App {
case _ false case _ false
} }
val binding = Http().bind(interface = "localhost", port = 8080) val bindingFuture = Http().bindAndstartHandlingWith({
val materializedMap = binding startHandlingWith {
get { get {
path("") { path("") {
complete(index) complete(index)
@ -47,11 +45,12 @@ object TestServer extends App {
complete(sys.error("BOOM!")) complete(sys.error("BOOM!"))
} }
} }
} }, interface = "localhost", port = 8080)
println(s"Server online at http://localhost:8080/\nPress RETURN to stop...") println(s"Server online at http://localhost:8080/\nPress RETURN to stop...")
Console.readLine() Console.readLine()
binding.unbind(materializedMap).onComplete(_ system.shutdown())
bindingFuture.flatMap(_.unbind()).onComplete(_ system.shutdown())
lazy val index = lazy val index =
<html> <html>

View file

@ -18,6 +18,8 @@ import HttpEncodings._
import MediaTypes._ import MediaTypes._
import StatusCodes._ import StatusCodes._
import scala.concurrent.duration._
class CodingDirectivesSpec extends RoutingSpec { class CodingDirectivesSpec extends RoutingSpec {
val echoRequestContent: Route = { ctx ctx.complete(ctx.request.entity.dataBytes.utf8String) } val echoRequestContent: Route = { ctx ctx.complete(ctx.request.entity.dataBytes.utf8String) }
@ -119,13 +121,13 @@ class CodingDirectivesSpec extends RoutingSpec {
encodeResponseWith(Gzip) { yeah } encodeResponseWith(Gzip) { yeah }
} ~> check { } ~> check {
response should haveContentEncoding(gzip) response should haveContentEncoding(gzip)
responseEntity shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahGzipped) strictify(responseEntity) shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahGzipped)
} }
} }
"encode the response content with GZIP if the request has no Accept-Encoding header" in { "encode the response content with GZIP if the request has no Accept-Encoding header" in {
Post() ~> { Post() ~> {
encodeResponseWith(Gzip) { yeah } encodeResponseWith(Gzip) { yeah }
} ~> check { responseEntity shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahGzipped) } } ~> check { strictify(responseEntity) shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahGzipped) }
} }
"reject the request if the client does not accept GZIP encoding" in { "reject the request if the client does not accept GZIP encoding" in {
Post() ~> `Accept-Encoding`(identity) ~> { Post() ~> `Accept-Encoding`(identity) ~> {
@ -163,7 +165,7 @@ class CodingDirectivesSpec extends RoutingSpec {
response should haveContentEncoding(gzip) response should haveContentEncoding(gzip)
chunks.size shouldEqual (11 + 1) // 11 regular + the last one chunks.size shouldEqual (11 + 1) // 11 regular + the last one
val bytes = chunks.foldLeft(ByteString.empty)(_ ++ _.data) val bytes = chunks.foldLeft(ByteString.empty)(_ ++ _.data)
Gzip.decode(bytes) should readAs(text) Gzip.decode(bytes).awaitResult(1.second) should readAs(text)
} }
} }
} }
@ -212,7 +214,7 @@ class CodingDirectivesSpec extends RoutingSpec {
encodeGzipOrIdentity { yeah } encodeGzipOrIdentity { yeah }
} ~> check { } ~> check {
response should haveContentEncoding(gzip) response should haveContentEncoding(gzip)
responseEntity shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahGzipped) strictify(responseEntity) shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahGzipped)
} }
} }
"produce a non-encoded response if the request has an `Accept-Encoding: identity` header" in { "produce a non-encoded response if the request has an `Accept-Encoding: identity` header" in {
@ -247,7 +249,7 @@ class CodingDirectivesSpec extends RoutingSpec {
encodeResponse { yeah } encodeResponse { yeah }
} ~> check { } ~> check {
response should haveContentEncoding(gzip) response should haveContentEncoding(gzip)
responseEntity shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahGzipped) strictify(responseEntity) shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahGzipped)
} }
} }
"produce a Deflate encoded response if the request has an `Accept-Encoding: deflate` header" in { "produce a Deflate encoded response if the request has an `Accept-Encoding: deflate` header" in {
@ -255,7 +257,7 @@ class CodingDirectivesSpec extends RoutingSpec {
encodeResponse { yeah } encodeResponse { yeah }
} ~> check { } ~> check {
response should haveContentEncoding(deflate) response should haveContentEncoding(deflate)
responseEntity shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahDeflated) strictify(responseEntity) shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahDeflated)
} }
} }
} }
@ -266,7 +268,7 @@ class CodingDirectivesSpec extends RoutingSpec {
encodeResponseWith(Gzip) { yeah } encodeResponseWith(Gzip) { yeah }
} ~> check { } ~> check {
response should haveContentEncoding(gzip) response should haveContentEncoding(gzip)
responseEntity shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahGzipped) strictify(responseEntity) shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahGzipped)
} }
} }
"produce a response encoded with one of the specified Encoders if the request has a matching Accept-Encoding header" in { "produce a response encoded with one of the specified Encoders if the request has a matching Accept-Encoding header" in {
@ -274,7 +276,7 @@ class CodingDirectivesSpec extends RoutingSpec {
encodeResponseWith(Gzip, Deflate) { yeah } encodeResponseWith(Gzip, Deflate) { yeah }
} ~> check { } ~> check {
response should haveContentEncoding(deflate) response should haveContentEncoding(deflate)
responseEntity shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahDeflated) strictify(responseEntity) shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahDeflated)
} }
} }
"produce a response encoded with the first of the specified Encoders if the request has no Accept-Encoding header" in { "produce a response encoded with the first of the specified Encoders if the request has no Accept-Encoding header" in {
@ -282,7 +284,7 @@ class CodingDirectivesSpec extends RoutingSpec {
encodeResponseWith(Gzip, Deflate) { yeah } encodeResponseWith(Gzip, Deflate) { yeah }
} ~> check { } ~> check {
response should haveContentEncoding(gzip) response should haveContentEncoding(gzip)
responseEntity shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahGzipped) strictify(responseEntity) shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahGzipped)
} }
} }
"produce a response with no encoding if the request has an empty Accept-Encoding header" in { "produce a response with no encoding if the request has an empty Accept-Encoding header" in {
@ -298,7 +300,7 @@ class CodingDirectivesSpec extends RoutingSpec {
encodeResponseWith(NoCoding, Deflate, Gzip) { yeah } encodeResponseWith(NoCoding, Deflate, Gzip) { yeah }
} ~> check { } ~> check {
response should haveContentEncoding(gzip) response should haveContentEncoding(gzip)
responseEntity shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahGzipped) strictify(responseEntity) shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), yeahGzipped)
} }
} }
"reject the request if it has an Accept-Encoding header with an encoding that doesn't match" in { "reject the request if it has an Accept-Encoding header with an encoding that doesn't match" in {
@ -372,7 +374,7 @@ class CodingDirectivesSpec extends RoutingSpec {
decodeEncode { echoRequestContent } decodeEncode { echoRequestContent }
} ~> check { } ~> check {
response should haveNoContentEncoding response should haveNoContentEncoding
responseEntity shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), "Hello") strictify(responseEntity) shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), "Hello")
} }
} }
"decode a GZIP encoded request and produce a Deflate encoded response if the request has an `Accept-Encoding: deflate` header" in { "decode a GZIP encoded request and produce a Deflate encoded response if the request has an `Accept-Encoding: deflate` header" in {
@ -380,7 +382,7 @@ class CodingDirectivesSpec extends RoutingSpec {
decodeEncode { echoRequestContent } decodeEncode { echoRequestContent }
} ~> check { } ~> check {
response should haveContentEncoding(deflate) response should haveContentEncoding(deflate)
responseEntity shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), helloDeflated) strictify(responseEntity) shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), helloDeflated)
} }
} }
"decode an unencoded request and produce a GZIP encoded response if the request has an `Accept-Encoding: gzip` header" in { "decode an unencoded request and produce a GZIP encoded response if the request has an `Accept-Encoding: gzip` header" in {
@ -388,7 +390,7 @@ class CodingDirectivesSpec extends RoutingSpec {
decodeEncode { echoRequestContent } decodeEncode { echoRequestContent }
} ~> check { } ~> check {
response should haveContentEncoding(gzip) response should haveContentEncoding(gzip)
responseEntity shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), helloGzipped) strictify(responseEntity) shouldEqual HttpEntity(ContentType(`text/plain`, `UTF-8`), helloGzipped)
} }
} }
} }
@ -406,4 +408,6 @@ class CodingDirectivesSpec extends RoutingSpec {
be(Some(`Content-Encoding`(encoding))) compose { (_: HttpResponse).header[`Content-Encoding`] } be(Some(`Content-Encoding`(encoding))) compose { (_: HttpResponse).header[`Content-Encoding`] }
def readAs(string: String, charset: String = "UTF8") = be(string) compose { (_: ByteString).decodeString(charset) } def readAs(string: String, charset: String = "UTF8") = be(string) compose { (_: ByteString).decodeString(charset) }
def strictify(entity: HttpEntity) = entity.toStrict(1.second).awaitResult(1.second)
} }

View file

@ -82,7 +82,7 @@ class FileAndResourceDirectivesSpec extends RoutingSpec with Inspectors with Ins
header[`Content-Range`] shouldEqual None header[`Content-Range`] shouldEqual None
mediaType.withParams(Map.empty) shouldEqual `multipart/byteranges` mediaType.withParams(Map.empty) shouldEqual `multipart/byteranges`
val parts = responseAs[Multipart.ByteRanges].toStrict(100.millis).awaitResult(100.millis).strictParts val parts = responseAs[Multipart.ByteRanges].toStrict(1.second).awaitResult(3.seconds).strictParts
parts.size shouldEqual 2 parts.size shouldEqual 2
parts(0).entity.data.utf8String shouldEqual "BCDEFGHIJK" parts(0).entity.data.utf8String shouldEqual "BCDEFGHIJK"
parts(1).entity.data.utf8String shouldEqual "QRSTUVWXYZ" parts(1).entity.data.utf8String shouldEqual "QRSTUVWXYZ"

View file

@ -99,7 +99,7 @@ class RangeDirectivesSpec extends RoutingSpec with Inspectors with Inside {
wrs { complete("Some random and not super short entity.") } wrs { complete("Some random and not super short entity.") }
} ~> check { } ~> check {
header[`Content-Range`] should be(None) header[`Content-Range`] should be(None)
val parts = Await.result(responseAs[Multipart.ByteRanges].parts.grouped(1000).runWith(Sink.head), 1.second) val parts = Await.result(responseAs[Multipart.ByteRanges].parts.grouped(1000).runWith(Sink.head()), 1.second)
parts.size shouldEqual 2 parts.size shouldEqual 2
inside(parts(0)) { inside(parts(0)) {
case Multipart.ByteRanges.BodyPart(range, entity, unit, headers) case Multipart.ByteRanges.BodyPart(range, entity, unit, headers)
@ -124,7 +124,7 @@ class RangeDirectivesSpec extends RoutingSpec with Inspectors with Inside {
wrs { complete(HttpEntity.Default(MediaTypes.`text/plain`, content.length, entityData())) } wrs { complete(HttpEntity.Default(MediaTypes.`text/plain`, content.length, entityData())) }
} ~> check { } ~> check {
header[`Content-Range`] should be(None) header[`Content-Range`] should be(None)
val parts = Await.result(responseAs[Multipart.ByteRanges].parts.grouped(1000).runWith(Sink.head), 1.second) val parts = Await.result(responseAs[Multipart.ByteRanges].parts.grouped(1000).runWith(Sink.head()), 1.second)
parts.size shouldEqual 2 parts.size shouldEqual 2
} }
} }

View file

@ -213,7 +213,7 @@ class UnmarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll wi
def haveParts[T <: Multipart](parts: Multipart.BodyPart*): Matcher[Future[T]] = def haveParts[T <: Multipart](parts: Multipart.BodyPart*): Matcher[Future[T]] =
equal(parts).matcher[Seq[Multipart.BodyPart]] compose { x equal(parts).matcher[Seq[Multipart.BodyPart]] compose { x
Await.result(x Await.result(x
.fast.flatMap(x x.parts.grouped(100).runWith(Sink.head)) .fast.flatMap(x x.parts.grouped(100).runWith(Sink.head()))
.fast.recover { case _: NoSuchElementException Nil }, 1.second) .fast.recover { case _: NoSuchElementException Nil }, 1.second)
} }
} }

View file

@ -36,4 +36,8 @@ akka.http.routing {
# The maximum number of bytes per ByteString a decoding directive will produce # The maximum number of bytes per ByteString a decoding directive will produce
# for an entity data stream. # for an entity data stream.
decode-max-bytes-per-chunk = 1m decode-max-bytes-per-chunk = 1m
# Fully qualified config path which holds the dispatcher configuration
# to be used by FlowMaterialiser when creating Actors for IO operations.
file-io-dispatcher = ${akka.io.tcp.file-io-dispatcher}
} }

View file

@ -10,17 +10,17 @@ import akka.stream.scaladsl.Flow
/** An abstraction to transform data bytes of HttpMessages or HttpEntities */ /** An abstraction to transform data bytes of HttpMessages or HttpEntities */
sealed trait DataMapper[T] { sealed trait DataMapper[T] {
def transformDataBytes(t: T, transformer: Flow[ByteString, ByteString]): T def transformDataBytes(t: T, transformer: Flow[ByteString, ByteString, _]): T
} }
object DataMapper { object DataMapper {
implicit val mapRequestEntity: DataMapper[RequestEntity] = implicit val mapRequestEntity: DataMapper[RequestEntity] =
new DataMapper[RequestEntity] { new DataMapper[RequestEntity] {
def transformDataBytes(t: RequestEntity, transformer: Flow[ByteString, ByteString]): RequestEntity = def transformDataBytes(t: RequestEntity, transformer: Flow[ByteString, ByteString, _]): RequestEntity =
t.transformDataBytes(transformer) t.transformDataBytes(transformer)
} }
implicit val mapResponseEntity: DataMapper[ResponseEntity] = implicit val mapResponseEntity: DataMapper[ResponseEntity] =
new DataMapper[ResponseEntity] { new DataMapper[ResponseEntity] {
def transformDataBytes(t: ResponseEntity, transformer: Flow[ByteString, ByteString]): ResponseEntity = def transformDataBytes(t: ResponseEntity, transformer: Flow[ByteString, ByteString, _]): ResponseEntity =
t.transformDataBytes(transformer) t.transformDataBytes(transformer)
} }
@ -29,7 +29,7 @@ object DataMapper {
def mapMessage[T, E](entityMapper: DataMapper[E])(mapEntity: (T, E E) T): DataMapper[T] = def mapMessage[T, E](entityMapper: DataMapper[E])(mapEntity: (T, E E) T): DataMapper[T] =
new DataMapper[T] { new DataMapper[T] {
def transformDataBytes(t: T, transformer: Flow[ByteString, ByteString]): T = def transformDataBytes(t: T, transformer: Flow[ByteString, ByteString, _]): T =
mapEntity(t, entityMapper.transformDataBytes(_, transformer)) mapEntity(t, entityMapper.transformDataBytes(_, transformer))
} }
} }

View file

@ -6,10 +6,13 @@ package akka.http.coding
import akka.http.model._ import akka.http.model._
import akka.http.util.StreamUtils import akka.http.util.StreamUtils
import akka.stream.ActorFlowMaterializer
import akka.stream.stage.Stage import akka.stream.stage.Stage
import akka.util.ByteString import akka.util.ByteString
import headers.HttpEncoding import headers.HttpEncoding
import akka.stream.scaladsl.Flow import akka.stream.scaladsl.{ Sink, Source, Flow }
import scala.concurrent.Future
trait Decoder { trait Decoder {
def encoding: HttpEncoding def encoding: HttpEncoding
@ -24,8 +27,9 @@ trait Decoder {
def maxBytesPerChunk: Int def maxBytesPerChunk: Int
def withMaxBytesPerChunk(maxBytesPerChunk: Int): Decoder def withMaxBytesPerChunk(maxBytesPerChunk: Int): Decoder
def decoderFlow: Flow[ByteString, ByteString] def decoderFlow: Flow[ByteString, ByteString, Unit]
def decode(input: ByteString): ByteString def decode(input: ByteString)(implicit mat: ActorFlowMaterializer): Future[ByteString] =
Source.single(input).via(decoderFlow).runWith(Sink.head())
} }
object Decoder { object Decoder {
val MaxBytesPerChunkDefault: Int = 65536 val MaxBytesPerChunkDefault: Int = 65536
@ -45,12 +49,7 @@ trait StreamDecoder extends Decoder { outer ⇒
outer.newDecompressorStage(maxBytesPerChunk) outer.newDecompressorStage(maxBytesPerChunk)
} }
def decoderFlow: Flow[ByteString, ByteString] = def decoderFlow: Flow[ByteString, ByteString, Unit] =
Flow[ByteString].transform(newDecompressorStage(maxBytesPerChunk)) Flow[ByteString].transform(newDecompressorStage(maxBytesPerChunk))
def decode(input: ByteString): ByteString = decodeWithLimits(input)
def decodeWithLimits(input: ByteString, maxBytesSize: Int = Int.MaxValue, maxIterations: Int = 1000): ByteString =
StreamUtils.runStrict(input, decoderFlow, maxBytesSize, maxIterations).get.get
def decodeFromIterator(input: Iterator[ByteString], maxBytesSize: Int = Int.MaxValue, maxIterations: Int = 1000): ByteString =
StreamUtils.runStrict(input, decoderFlow, maxBytesSize, maxIterations).get.get
} }

View file

@ -8,7 +8,7 @@ import scala.annotation.implicitNotFound
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.{ ExecutionContext, Future } import scala.concurrent.{ ExecutionContext, Future }
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.http.util.FastFuture import akka.http.util.FastFuture
import akka.http.unmarshalling._ import akka.http.unmarshalling._
import akka.http.model._ import akka.http.model._
@ -87,7 +87,7 @@ object StrictForm {
implicit def unmarshaller(implicit formDataUM: FromEntityUnmarshaller[FormData], implicit def unmarshaller(implicit formDataUM: FromEntityUnmarshaller[FormData],
multipartUM: FromEntityUnmarshaller[Multipart.FormData], multipartUM: FromEntityUnmarshaller[Multipart.FormData],
ec: ExecutionContext, fm: FlowMaterializer): FromEntityUnmarshaller[StrictForm] = { ec: ExecutionContext, fm: ActorFlowMaterializer): FromEntityUnmarshaller[StrictForm] = {
def tryUnmarshalToQueryForm(entity: HttpEntity): Future[StrictForm] = def tryUnmarshalToQueryForm(entity: HttpEntity): Future[StrictForm] =
for (formData formDataUM(entity).fast) yield { for (formData formDataUM(entity).fast) yield {

View file

@ -4,7 +4,7 @@
package akka.http.server package akka.http.server
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import scala.concurrent.{ Future, ExecutionContext } import scala.concurrent.{ Future, ExecutionContext }
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
@ -31,7 +31,7 @@ trait RequestContext {
/** /**
* The default FlowMaterializer. * The default FlowMaterializer.
*/ */
implicit def flowMaterializer: FlowMaterializer implicit def flowMaterializer: ActorFlowMaterializer
/** /**
* The default LoggingAdapter to be used for logging messages related to this request. * The default LoggingAdapter to be used for logging messages related to this request.
@ -48,7 +48,7 @@ trait RequestContext {
*/ */
def reconfigure( def reconfigure(
executionContext: ExecutionContext = executionContext, executionContext: ExecutionContext = executionContext,
flowMaterializer: FlowMaterializer = flowMaterializer, flowMaterializer: ActorFlowMaterializer = flowMaterializer,
log: LoggingAdapter = log, log: LoggingAdapter = log,
settings: RoutingSettings = settings): RequestContext settings: RoutingSettings = settings): RequestContext
@ -82,7 +82,7 @@ trait RequestContext {
/** /**
* Returns a copy of this context with the new HttpRequest. * Returns a copy of this context with the new HttpRequest.
*/ */
def withFlowMaterializer(materializer: FlowMaterializer): RequestContext def withFlowMaterializer(materializer: ActorFlowMaterializer): RequestContext
/** /**
* Returns a copy of this context with the new LoggingAdapter. * Returns a copy of this context with the new LoggingAdapter.

View file

@ -4,7 +4,7 @@
package akka.http.server package akka.http.server
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import scala.concurrent.{ Future, ExecutionContext } import scala.concurrent.{ Future, ExecutionContext }
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
@ -20,14 +20,14 @@ private[http] class RequestContextImpl(
val request: HttpRequest, val request: HttpRequest,
val unmatchedPath: Uri.Path, val unmatchedPath: Uri.Path,
val executionContext: ExecutionContext, val executionContext: ExecutionContext,
val flowMaterializer: FlowMaterializer, val flowMaterializer: ActorFlowMaterializer,
val log: LoggingAdapter, val log: LoggingAdapter,
val settings: RoutingSettings) extends RequestContext { val settings: RoutingSettings) extends RequestContext {
def this(request: HttpRequest, log: LoggingAdapter, settings: RoutingSettings)(implicit ec: ExecutionContext, materializer: FlowMaterializer) = def this(request: HttpRequest, log: LoggingAdapter, settings: RoutingSettings)(implicit ec: ExecutionContext, materializer: ActorFlowMaterializer) =
this(request, request.uri.path, ec, materializer, log, settings) this(request, request.uri.path, ec, materializer, log, settings)
def reconfigure(executionContext: ExecutionContext, flowMaterializer: FlowMaterializer, log: LoggingAdapter, settings: RoutingSettings): RequestContext = def reconfigure(executionContext: ExecutionContext, flowMaterializer: ActorFlowMaterializer, log: LoggingAdapter, settings: RoutingSettings): RequestContext =
copy(executionContext = executionContext, flowMaterializer = flowMaterializer, log = log, settings = settings) copy(executionContext = executionContext, flowMaterializer = flowMaterializer, log = log, settings = settings)
override def complete(trm: ToResponseMarshallable): Future[RouteResult] = override def complete(trm: ToResponseMarshallable): Future[RouteResult] =
@ -51,7 +51,7 @@ private[http] class RequestContextImpl(
override def withExecutionContext(executionContext: ExecutionContext): RequestContext = override def withExecutionContext(executionContext: ExecutionContext): RequestContext =
if (executionContext != this.executionContext) copy(executionContext = executionContext) else this if (executionContext != this.executionContext) copy(executionContext = executionContext) else this
override def withFlowMaterializer(flowMaterializer: FlowMaterializer): RequestContext = override def withFlowMaterializer(flowMaterializer: ActorFlowMaterializer): RequestContext =
if (flowMaterializer != this.flowMaterializer) copy(flowMaterializer = flowMaterializer) else this if (flowMaterializer != this.flowMaterializer) copy(flowMaterializer = flowMaterializer) else this
override def withLog(log: LoggingAdapter): RequestContext = override def withLog(log: LoggingAdapter): RequestContext =
@ -85,7 +85,7 @@ private[http] class RequestContextImpl(
private def copy(request: HttpRequest = request, private def copy(request: HttpRequest = request,
unmatchedPath: Uri.Path = unmatchedPath, unmatchedPath: Uri.Path = unmatchedPath,
executionContext: ExecutionContext = executionContext, executionContext: ExecutionContext = executionContext,
flowMaterializer: FlowMaterializer = flowMaterializer, flowMaterializer: ActorFlowMaterializer = flowMaterializer,
log: LoggingAdapter = log, log: LoggingAdapter = log,
settings: RoutingSettings = settings) = settings: RoutingSettings = settings) =
new RequestContextImpl(request, unmatchedPath, executionContext, flowMaterializer, log, settings) new RequestContextImpl(request, unmatchedPath, executionContext, flowMaterializer, log, settings)

View file

@ -38,7 +38,7 @@ object Route {
/** /**
* Turns a `Route` into an server flow. * Turns a `Route` into an server flow.
*/ */
def handlerFlow(route: Route)(implicit setup: RoutingSetup): Flow[HttpRequest, HttpResponse] = def handlerFlow(route: Route)(implicit setup: RoutingSetup): Flow[HttpRequest, HttpResponse, Unit] =
Flow[HttpRequest].mapAsync(asyncHandler(route)) Flow[HttpRequest].mapAsync(asyncHandler(route))
/** /**

View file

@ -20,6 +20,6 @@ object RouteResult {
final case class Complete(response: HttpResponse) extends RouteResult final case class Complete(response: HttpResponse) extends RouteResult
final case class Rejected(rejections: immutable.Seq[Rejection]) extends RouteResult final case class Rejected(rejections: immutable.Seq[Rejection]) extends RouteResult
implicit def route2HandlerFlow(route: Route)(implicit setup: RoutingSetup): Flow[HttpRequest, HttpResponse] = implicit def route2HandlerFlow(route: Route)(implicit setup: RoutingSetup): Flow[HttpRequest, HttpResponse, Unit] =
Route.handlerFlow(route) Route.handlerFlow(route)
} }

View file

@ -14,7 +14,8 @@ case class RoutingSettings(
renderVanityFooter: Boolean, renderVanityFooter: Boolean,
rangeCountLimit: Int, rangeCountLimit: Int,
rangeCoalescingThreshold: Long, rangeCoalescingThreshold: Long,
decodeMaxBytesPerChunk: Int) decodeMaxBytesPerChunk: Int,
fileIODispatcher: String)
object RoutingSettings extends SettingsCompanion[RoutingSettings]("akka.http.routing") { object RoutingSettings extends SettingsCompanion[RoutingSettings]("akka.http.routing") {
def fromSubConfig(c: Config) = apply( def fromSubConfig(c: Config) = apply(
@ -23,7 +24,8 @@ object RoutingSettings extends SettingsCompanion[RoutingSettings]("akka.http.rou
c getBoolean "render-vanity-footer", c getBoolean "render-vanity-footer",
c getInt "range-count-limit", c getInt "range-count-limit",
c getBytes "range-coalescing-threshold", c getBytes "range-coalescing-threshold",
c getIntBytes "decode-max-bytes-per-chunk") c getIntBytes "decode-max-bytes-per-chunk",
c getString "file-io-dispatcher")
implicit def default(implicit refFactory: ActorRefFactory) = implicit def default(implicit refFactory: ActorRefFactory) =
apply(actorSystem) apply(actorSystem)

View file

@ -7,7 +7,7 @@ package akka.http.server
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.actor.{ ActorSystem, ActorContext } import akka.actor.{ ActorSystem, ActorContext }
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.http.Http import akka.http.Http
import akka.http.model.HttpRequest import akka.http.model.HttpRequest
@ -34,12 +34,12 @@ class RoutingSetup(
val exceptionHandler: ExceptionHandler, val exceptionHandler: ExceptionHandler,
val rejectionHandler: RejectionHandler, val rejectionHandler: RejectionHandler,
val executionContext: ExecutionContext, val executionContext: ExecutionContext,
val flowMaterializer: FlowMaterializer, val flowMaterializer: ActorFlowMaterializer,
val routingLog: RoutingLog) { val routingLog: RoutingLog) {
// enable `import setup._` to properly bring implicits in scope // enable `import setup._` to properly bring implicits in scope
implicit def executor: ExecutionContext = executionContext implicit def executor: ExecutionContext = executionContext
implicit def materializer: FlowMaterializer = flowMaterializer implicit def materializer: ActorFlowMaterializer = flowMaterializer
} }
object RoutingSetup { object RoutingSetup {
@ -47,7 +47,7 @@ object RoutingSetup {
exceptionHandler: ExceptionHandler = null, exceptionHandler: ExceptionHandler = null,
rejectionHandler: RejectionHandler = null, rejectionHandler: RejectionHandler = null,
executionContext: ExecutionContext, executionContext: ExecutionContext,
flowMaterializer: FlowMaterializer, flowMaterializer: ActorFlowMaterializer,
routingLog: RoutingLog): RoutingSetup = routingLog: RoutingLog): RoutingSetup =
new RoutingSetup( new RoutingSetup(
routingSettings, routingSettings,

View file

@ -6,7 +6,7 @@ package akka.http.server
package directives package directives
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import scala.concurrent.{ Future, ExecutionContext } import scala.concurrent.{ Future, ExecutionContext }
import scala.collection.immutable import scala.collection.immutable
@ -144,13 +144,13 @@ trait BasicDirectives {
/** /**
* Runs its inner route with the given alternative [[FlowMaterializer]]. * Runs its inner route with the given alternative [[FlowMaterializer]].
*/ */
def withFlowMaterializer(materializer: FlowMaterializer): Directive0 = def withFlowMaterializer(materializer: ActorFlowMaterializer): Directive0 =
mapRequestContext(_ withFlowMaterializer materializer) mapRequestContext(_ withFlowMaterializer materializer)
/** /**
* Extracts the [[ExecutionContext]] from the [[RequestContext]]. * Extracts the [[ExecutionContext]] from the [[RequestContext]].
*/ */
def extractFlowMaterializer: Directive1[FlowMaterializer] = BasicDirectives._extractFlowMaterializer def extractFlowMaterializer: Directive1[ActorFlowMaterializer] = BasicDirectives._extractFlowMaterializer
/** /**
* Runs its inner route with the given alternative [[LoggingAdapter]]. * Runs its inner route with the given alternative [[LoggingAdapter]].
@ -193,7 +193,7 @@ object BasicDirectives extends BasicDirectives {
private val _extractRequest: Directive1[HttpRequest] = extract(_.request) private val _extractRequest: Directive1[HttpRequest] = extract(_.request)
private val _extractUri: Directive1[Uri] = extract(_.request.uri) private val _extractUri: Directive1[Uri] = extract(_.request.uri)
private val _extractExecutionContext: Directive1[ExecutionContext] = extract(_.executionContext) private val _extractExecutionContext: Directive1[ExecutionContext] = extract(_.executionContext)
private val _extractFlowMaterializer: Directive1[FlowMaterializer] = extract(_.flowMaterializer) private val _extractFlowMaterializer: Directive1[ActorFlowMaterializer] = extract(_.flowMaterializer)
private val _extractLog: Directive1[LoggingAdapter] = extract(_.log) private val _extractLog: Directive1[LoggingAdapter] = extract(_.log)
private val _extractSettings: Directive1[RoutingSettings] = extract(_.settings) private val _extractSettings: Directive1[RoutingSettings] = extract(_.settings)
private val _extractRequestContext: Directive1[RequestContext] = extract(akka.http.util.identityFunc) private val _extractRequestContext: Directive1[RequestContext] = extract(akka.http.util.identityFunc)

View file

@ -52,9 +52,11 @@ trait FileAndResourceDirectives {
get { get {
if (file.isFile && file.canRead) if (file.isFile && file.canRead)
conditionalFor(file.length, file.lastModified).apply { conditionalFor(file.length, file.lastModified).apply {
withRangeSupport { withRangeSupport { ctx
extractExecutionContext { implicit ec import ctx.executionContext
complete(HttpEntity.Default(contentType, file.length, StreamUtils.fromInputStreamSource(new FileInputStream(file)))) ctx.complete {
HttpEntity.Default(contentType, file.length,
StreamUtils.fromInputStreamSource(new FileInputStream(file), ctx.settings.fileIODispatcher))
} }
} }
} }
@ -100,11 +102,11 @@ trait FileAndResourceDirectives {
} finally conn.getInputStream.close() } finally conn.getInputStream.close()
} }
conditionalFor(length, lastModified).apply { conditionalFor(length, lastModified).apply {
withRangeSupport { withRangeSupport { ctx
extractExecutionContext { implicit ec import ctx.executionContext
complete { ctx.complete {
HttpEntity.Default(contentType, length, StreamUtils.fromInputStreamSource(url.openStream())) HttpEntity.Default(contentType, length,
} StreamUtils.fromInputStreamSource(url.openStream(), ctx.settings.fileIODispatcher))
} }
} }
} }

View file

@ -54,7 +54,7 @@ trait MultipartUnmarshallers {
def multipartUnmarshaller[T <: Multipart, BP <: Multipart.BodyPart, BPS <: Multipart.BodyPart.Strict](mediaRange: MediaRange, def multipartUnmarshaller[T <: Multipart, BP <: Multipart.BodyPart, BPS <: Multipart.BodyPart.Strict](mediaRange: MediaRange,
defaultContentType: ContentType, defaultContentType: ContentType,
createBodyPart: (BodyPartEntity, List[HttpHeader]) BP, createBodyPart: (BodyPartEntity, List[HttpHeader]) BP,
createStreamed: (MultipartMediaType, Source[BP]) T, createStreamed: (MultipartMediaType, Source[BP, Unit]) T,
createStrictBodyPart: (HttpEntity.Strict, List[HttpHeader]) BPS, createStrictBodyPart: (HttpEntity.Strict, List[HttpHeader]) BPS,
createStrict: (MultipartMediaType, immutable.Seq[BPS]) T)(implicit ec: ExecutionContext, log: LoggingAdapter = NoLogging): FromEntityUnmarshaller[T] = createStrict: (MultipartMediaType, immutable.Seq[BPS]) T)(implicit ec: ExecutionContext, log: LoggingAdapter = NoLogging): FromEntityUnmarshaller[T] =
Unmarshaller { entity Unmarshaller { entity

View file

@ -5,24 +5,24 @@
package akka.http.unmarshalling package akka.http.unmarshalling
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.util.ByteString import akka.util.ByteString
import akka.http.util.FastFuture import akka.http.util.FastFuture
import akka.http.model._ import akka.http.model._
trait PredefinedFromEntityUnmarshallers extends MultipartUnmarshallers { trait PredefinedFromEntityUnmarshallers extends MultipartUnmarshallers {
implicit def byteStringUnmarshaller(implicit fm: FlowMaterializer): FromEntityUnmarshaller[ByteString] = implicit def byteStringUnmarshaller(implicit fm: ActorFlowMaterializer): FromEntityUnmarshaller[ByteString] =
Unmarshaller { Unmarshaller {
case HttpEntity.Strict(_, data) FastFuture.successful(data) case HttpEntity.Strict(_, data) FastFuture.successful(data)
case entity entity.dataBytes.runFold(ByteString.empty)(_ ++ _) case entity entity.dataBytes.runFold(ByteString.empty)(_ ++ _)
} }
implicit def byteArrayUnmarshaller(implicit fm: FlowMaterializer, implicit def byteArrayUnmarshaller(implicit fm: ActorFlowMaterializer,
ec: ExecutionContext): FromEntityUnmarshaller[Array[Byte]] = ec: ExecutionContext): FromEntityUnmarshaller[Array[Byte]] =
byteStringUnmarshaller.map(_.toArray[Byte]) byteStringUnmarshaller.map(_.toArray[Byte])
implicit def charArrayUnmarshaller(implicit fm: FlowMaterializer, implicit def charArrayUnmarshaller(implicit fm: ActorFlowMaterializer,
ec: ExecutionContext): FromEntityUnmarshaller[Array[Char]] = ec: ExecutionContext): FromEntityUnmarshaller[Array[Char]] =
byteStringUnmarshaller(fm) mapWithInput { (entity, bytes) byteStringUnmarshaller(fm) mapWithInput { (entity, bytes)
val charBuffer = entity.contentType.charset.nioCharset.decode(bytes.asByteBuffer) val charBuffer = entity.contentType.charset.nioCharset.decode(bytes.asByteBuffer)
@ -31,17 +31,17 @@ trait PredefinedFromEntityUnmarshallers extends MultipartUnmarshallers {
array array
} }
implicit def stringUnmarshaller(implicit fm: FlowMaterializer, implicit def stringUnmarshaller(implicit fm: ActorFlowMaterializer,
ec: ExecutionContext): FromEntityUnmarshaller[String] = ec: ExecutionContext): FromEntityUnmarshaller[String] =
byteStringUnmarshaller(fm) mapWithInput { (entity, bytes) byteStringUnmarshaller(fm) mapWithInput { (entity, bytes)
// FIXME: add `ByteString::decodeString(java.nio.Charset): String` overload!!! // FIXME: add `ByteString::decodeString(java.nio.Charset): String` overload!!!
bytes.decodeString(entity.contentType.charset.nioCharset.name) // ouch!!! bytes.decodeString(entity.contentType.charset.nioCharset.name) // ouch!!!
} }
implicit def defaultUrlEncodedFormDataUnmarshaller(implicit fm: FlowMaterializer, implicit def defaultUrlEncodedFormDataUnmarshaller(implicit fm: ActorFlowMaterializer,
ec: ExecutionContext): FromEntityUnmarshaller[FormData] = ec: ExecutionContext): FromEntityUnmarshaller[FormData] =
urlEncodedFormDataUnmarshaller(MediaTypes.`application/x-www-form-urlencoded`) urlEncodedFormDataUnmarshaller(MediaTypes.`application/x-www-form-urlencoded`)
def urlEncodedFormDataUnmarshaller(ranges: ContentTypeRange*)(implicit fm: FlowMaterializer, def urlEncodedFormDataUnmarshaller(ranges: ContentTypeRange*)(implicit fm: ActorFlowMaterializer,
ec: ExecutionContext): FromEntityUnmarshaller[FormData] = ec: ExecutionContext): FromEntityUnmarshaller[FormData] =
stringUnmarshaller.forContentTypes(ranges: _*).mapWithInput { (entity, string) stringUnmarshaller.forContentTypes(ranges: _*).mapWithInput { (entity, string)
try { try {

View file

@ -8,13 +8,12 @@ import akka.event.Logging
import scala.collection.{ mutable, immutable } import scala.collection.{ mutable, immutable }
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.{ Flow, Sink, Source }
import akka.stream.scaladsl.Source
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit import akka.stream.testkit.StreamTestKit
import akka.testkit.EventFilter import akka.testkit.EventFilter
import akka.testkit.TestEvent import akka.testkit.TestEvent
import org.reactivestreams.Publisher import org.reactivestreams.{ Subscriber, Subscription, Processor, Publisher }
import org.reactivestreams.tck.IdentityProcessorVerification import org.reactivestreams.tck.IdentityProcessorVerification
import org.reactivestreams.tck.TestEnvironment import org.reactivestreams.tck.TestEnvironment
import org.scalatest.testng.TestNGSuiteLike import org.scalatest.testng.TestNGSuiteLike
@ -45,7 +44,19 @@ abstract class AkkaIdentityProcessorVerification[T](val system: ActorSystem, env
if (elements == Long.MaxValue) 1 to Int.MaxValue if (elements == Long.MaxValue) 1 to Int.MaxValue
else 0 until elements.toInt else 0 until elements.toInt
Source(iterable).runWith(Sink.publisher) Source(iterable).runWith(Sink.publisher())
}
def processorFromFlow[T](flow: Flow[T, T, _])(implicit mat: ActorFlowMaterializer): Processor[T, T] = {
val (sub: Subscriber[T], pub: Publisher[T]) = flow.runWith(Source.subscriber[T](), Sink.publisher[T]())
new Processor[T, T] {
override def onSubscribe(s: Subscription): Unit = sub.onSubscribe(s)
override def onError(t: Throwable): Unit = sub.onError(t)
override def onComplete(): Unit = sub.onComplete()
override def onNext(t: T): Unit = sub.onNext(t)
override def subscribe(s: Subscriber[_ >: T]): Unit = pub.subscribe(s)
}
} }
/** By default Akka Publishers do not support Fanout! */ /** By default Akka Publishers do not support Fanout! */

View file

@ -64,7 +64,7 @@ trait AkkaSubscriberVerificationLike {
if (elements == Long.MaxValue) 1 to Int.MaxValue if (elements == Long.MaxValue) 1 to Int.MaxValue
else 0 until elements.toInt else 0 until elements.toInt
Source(iterable).runWith(Sink.publisher) Source(iterable).runWith(Sink.publisher())
} }
@AfterClass @AfterClass

View file

@ -4,14 +4,11 @@
package akka.stream.tck package akka.stream.tck
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import akka.stream.impl.{ Ast, ActorFlowMaterializerImpl }
import akka.stream.scaladsl.MaterializedMap import akka.stream.impl.Stages.Identity
import akka.stream.scaladsl.OperationAttributes._ import akka.stream.scaladsl.{ OperationAttributes, Flow }
import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings } import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings }
import org.reactivestreams.{ Publisher, Processor } import org.reactivestreams.{ Processor, Publisher }
import akka.stream.impl.fusing.Map
import scala.concurrent.Promise
import akka.stream.Supervision
class FusableProcessorTest extends AkkaIdentityProcessorVerification[Int] { class FusableProcessorTest extends AkkaIdentityProcessorVerification[Int] {
@ -23,12 +20,9 @@ class FusableProcessorTest extends AkkaIdentityProcessorVerification[Int] {
implicit val materializer = ActorFlowMaterializer(settings)(system) implicit val materializer = ActorFlowMaterializer(settings)(system)
val flowName = getClass.getSimpleName + "-" + processorCounter.incrementAndGet() processorFromFlow(
// withAttributes "wraps" the underlying identity and protects it from automatic removal
val (processor, _ns) = materializer.asInstanceOf[ActorFlowMaterializerImpl].processorForNode( Flow[Int].andThen(Identity()).withAttributes(OperationAttributes.name("identity")))
Ast.Fused(List(Map[Int, Int](identity, Supervision.stoppingDecider)), name("identity")), flowName, 1)
processor.asInstanceOf[Processor[Int, Int]]
} }
override def createHelperPublisher(elements: Long): Publisher[Int] = { override def createHelperPublisher(elements: Long): Publisher[Int] = {

View file

@ -13,7 +13,7 @@ class FuturePublisherTest extends AkkaPublisherVerification[Int] {
def createPublisher(elements: Long): Publisher[Int] = { def createPublisher(elements: Long): Publisher[Int] = {
val p = Promise[Int]() val p = Promise[Int]()
val pub = Source(p.future).runWith(Sink.publisher) val pub = Source(p.future).runWith(Sink.publisher())
p.success(0) p.success(0)
pub pub
} }

View file

@ -3,6 +3,7 @@
*/ */
package akka.stream.tck package akka.stream.tck
import akka.stream.impl.HeadSink
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import org.reactivestreams.Subscriber import org.reactivestreams.Subscriber

View file

@ -17,6 +17,11 @@ class IterablePublisherTest extends AkkaPublisherVerification[Int] {
else else
0 until elements.toInt 0 until elements.toInt
Source(iterable).runWith(Sink.publisher) Source(iterable).runWith(Sink.publisher())
}
override def spec317_mustSignalOnErrorWhenPendingAboveLongMaxValue(): Unit = {
// FIXME: This test needs RC3
notVerified()
} }
} }

View file

@ -19,7 +19,7 @@ class SyncIterablePublisherTest extends AkkaPublisherVerification[Int] {
else else
0 until elements.toInt 0 until elements.toInt
Source(SynchronousIterablePublisher(iterable, "synchronous-iterable-publisher")).runWith(Sink.publisher) Source(SynchronousIterablePublisher(iterable, "synchronous-iterable-publisher")).runWith(Sink.publisher())
} }
override def spec317_mustSignalOnErrorWhenPendingAboveLongMaxValue() = notVerified("RS TCK 1.0.0.M3 does not handle sync publishers well") override def spec317_mustSignalOnErrorWhenPendingAboveLongMaxValue() = notVerified("RS TCK 1.0.0.M3 does not handle sync publishers well")

View file

@ -3,19 +3,15 @@
*/ */
package akka.stream.tck package akka.stream.tck
import akka.stream.scaladsl.OperationAttributes._
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.impl.ActorFlowMaterializerImpl
import akka.stream.impl.Ast
import akka.stream.ActorFlowMaterializer
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import akka.stream.scaladsl.MaterializedMap
import org.reactivestreams.Processor
import org.reactivestreams.Publisher
import akka.stream.stage.PushStage
import akka.stream.stage.Context
import scala.concurrent.Promise import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings }
import akka.stream.impl.ActorFlowMaterializerImpl
import akka.stream.impl.Stages.Identity
import akka.stream.scaladsl.Flow
import akka.stream.scaladsl.OperationAttributes._
import akka.stream.stage.{ Context, PushStage }
import org.reactivestreams.{ Processor, Publisher }
class TransformProcessorTest extends AkkaIdentityProcessorVerification[Int] { class TransformProcessorTest extends AkkaIdentityProcessorVerification[Int] {
@ -27,17 +23,12 @@ class TransformProcessorTest extends AkkaIdentityProcessorVerification[Int] {
implicit val materializer = ActorFlowMaterializer(settings)(system) implicit val materializer = ActorFlowMaterializer(settings)(system)
val flowName = getClass.getSimpleName + "-" + processorCounter.incrementAndGet()
val mkStage = () val mkStage = ()
new PushStage[Any, Any] { new PushStage[Int, Int] {
override def onPush(in: Any, ctx: Context[Any]) = ctx.push(in) override def onPush(in: Int, ctx: Context[Int]) = ctx.push(in)
} }
val (processor, _) = materializer.asInstanceOf[ActorFlowMaterializerImpl].processorForNode( processorFromFlow(Flow[Int].transform(mkStage))
Ast.StageFactory(mkStage, name("transform")), flowName, 1)
processor.asInstanceOf[Processor[Int, Int]]
} }
override def createHelperPublisher(elements: Long): Publisher[Int] = { override def createHelperPublisher(elements: Long): Publisher[Int] = {

View file

@ -7,20 +7,23 @@ import org.reactivestreams.Publisher
import akka.stream.ActorFlowMaterializer import akka.stream.ActorFlowMaterializer
class ChainSetup[In, Out]( class ChainSetup[In, Out](
stream: Flow[In, In] Flow[In, Out], stream: Flow[In, In, _] Flow[In, Out, _],
val settings: ActorFlowMaterializerSettings, val settings: ActorFlowMaterializerSettings,
materializer: ActorFlowMaterializer, materializer: ActorFlowMaterializer,
toPublisher: (Source[Out], ActorFlowMaterializer) Publisher[Out])(implicit val system: ActorSystem) { toPublisher: (Source[Out, _], ActorFlowMaterializer) Publisher[Out])(implicit val system: ActorSystem) {
def this(stream: Flow[In, In] Flow[In, Out], settings: ActorFlowMaterializerSettings, toPublisher: (Source[Out], ActorFlowMaterializer) Publisher[Out])(implicit system: ActorSystem) = def this(stream: Flow[In, In, _] Flow[In, Out, _], settings: ActorFlowMaterializerSettings, toPublisher: (Source[Out, _], ActorFlowMaterializer) Publisher[Out])(implicit system: ActorSystem) =
this(stream, settings, ActorFlowMaterializer(settings)(system), toPublisher)(system) this(stream, settings, ActorFlowMaterializer(settings)(system), toPublisher)(system)
def this(stream: Flow[In, In] Flow[In, Out], settings: ActorFlowMaterializerSettings, materializerCreator: (ActorFlowMaterializerSettings, ActorRefFactory) ActorFlowMaterializer, toPublisher: (Source[Out], ActorFlowMaterializer) Publisher[Out])(implicit system: ActorSystem) = def this(stream: Flow[In, In, _] Flow[In, Out, _],
settings: ActorFlowMaterializerSettings,
materializerCreator: (ActorFlowMaterializerSettings, ActorRefFactory) ActorFlowMaterializer,
toPublisher: (Source[Out, _], ActorFlowMaterializer) Publisher[Out])(implicit system: ActorSystem) =
this(stream, settings, materializerCreator(settings, system), toPublisher)(system) this(stream, settings, materializerCreator(settings, system), toPublisher)(system)
val upstream = StreamTestKit.PublisherProbe[In]() val upstream = StreamTestKit.PublisherProbe[In]()
val downstream = StreamTestKit.SubscriberProbe[Out]() val downstream = StreamTestKit.SubscriberProbe[Out]()
private val s = Source(upstream).via(stream(Flow[In])) private val s = Source(upstream).via(stream(Flow[In].map(x x).withAttributes(OperationAttributes.name("buh"))))
val publisher = toPublisher(s, materializer) val publisher = toPublisher(s, materializer)
val upstreamSubscription = upstream.expectSubscription() val upstreamSubscription = upstream.expectSubscription()
publisher.subscribe(downstream) publisher.subscribe(downstream)

Some files were not shown because too many files have changed in this diff Show more