!str #18692 javadsl.FlowGraph.Builder.add()
* also make factories more consistent by only offering FlowGraph.create() * also remove secondary (edge-based) FlowGraph.Builder DSL * also improve naming for conversions from Graph to Source/Flow/BidiFlow/Sink
This commit is contained in:
parent
0f99a42df9
commit
f29d7affbd
120 changed files with 1535 additions and 1897 deletions
|
|
@ -76,7 +76,7 @@ private[http] object OutgoingConnectionBlueprint {
|
|||
case (MessageStartError(_, info), _) ⇒ throw IllegalResponseException(info)
|
||||
}
|
||||
|
||||
val core = BidiFlow() { implicit b ⇒
|
||||
val core = BidiFlow.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val methodBypassFanout = b.add(Broadcast[HttpRequest](2, eagerCancel = true))
|
||||
val responseParsingMerge = b.add(new ResponseParsingMerge(rootParser))
|
||||
|
|
@ -103,7 +103,7 @@ private[http] object OutgoingConnectionBlueprint {
|
|||
wrapTls.outlet,
|
||||
unwrapTls.inlet,
|
||||
terminationFanout.out(1))
|
||||
}
|
||||
})
|
||||
|
||||
One2OneBidiFlow[HttpRequest, HttpResponse](-1) atop core
|
||||
}
|
||||
|
|
|
|||
|
|
@ -65,7 +65,7 @@ private object PoolConductor {
|
|||
|
||||
*/
|
||||
def apply(slotCount: Int, maxRetries: Int, pipeliningLimit: Int, log: LoggingAdapter): Graph[Ports, Any] =
|
||||
FlowGraph.partial() { implicit b ⇒
|
||||
FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val retryMerge = b.add(MergePreferred[RequestContext](1, eagerClose = true))
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import scala.concurrent.{ Promise, Future }
|
|||
import scala.util.Try
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.actor._
|
||||
import akka.stream.Materializer
|
||||
import akka.stream.{ FlowShape, Materializer }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.http.scaladsl.model._
|
||||
import akka.http.scaladsl.Http
|
||||
|
|
@ -70,7 +70,7 @@ private object PoolFlow {
|
|||
def apply(connectionFlow: Flow[HttpRequest, HttpResponse, Future[Http.OutgoingConnection]],
|
||||
remoteAddress: InetSocketAddress, settings: ConnectionPoolSettings, log: LoggingAdapter)(
|
||||
implicit system: ActorSystem, fm: Materializer): Flow[RequestContext, ResponseContext, Unit] =
|
||||
Flow() { implicit b ⇒
|
||||
Flow.fromGraph(FlowGraph.create[FlowShape[RequestContext, ResponseContext]]() { implicit b ⇒
|
||||
import settings._
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
|
|
@ -87,6 +87,6 @@ private object PoolFlow {
|
|||
slot.out0 ~> responseMerge.in(ix)
|
||||
slot.out1 ~> slotEventMerge.in(ix)
|
||||
}
|
||||
(conductor.requestIn, responseMerge.out)
|
||||
}
|
||||
FlowShape(conductor.requestIn, responseMerge.out)
|
||||
})
|
||||
}
|
||||
|
|
|
|||
|
|
@ -53,7 +53,7 @@ private object PoolSlot {
|
|||
remoteAddress: InetSocketAddress, // TODO: remove after #16168 is cleared
|
||||
settings: ConnectionPoolSettings)(implicit system: ActorSystem,
|
||||
fm: Materializer): Graph[FanOutShape2[RequestContext, ResponseContext, RawSlotEvent], Any] =
|
||||
FlowGraph.partial() { implicit b ⇒
|
||||
FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val slotProcessor = b.add {
|
||||
|
|
|
|||
|
|
@ -5,6 +5,7 @@
|
|||
package akka.http.impl.engine.rendering
|
||||
|
||||
import akka.parboiled2.CharUtils
|
||||
import akka.stream.SourceShape
|
||||
import akka.util.ByteString
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.stream.scaladsl._
|
||||
|
|
@ -31,12 +32,12 @@ private object RenderSupport {
|
|||
val defaultLastChunkBytes: ByteString = renderChunk(HttpEntity.LastChunk)
|
||||
|
||||
def CancelSecond[T, Mat](first: Source[T, Mat], second: Source[T, Any]): Source[T, Mat] = {
|
||||
Source(first) { implicit b ⇒
|
||||
Source.fromGraph(FlowGraph.create(first) { implicit b ⇒
|
||||
frst ⇒
|
||||
import FlowGraph.Implicits._
|
||||
second ~> Sink.cancelled
|
||||
frst.outlet
|
||||
}
|
||||
SourceShape(frst.outlet)
|
||||
})
|
||||
}
|
||||
|
||||
def renderEntityContentType(r: Rendering, entity: HttpEntity) =
|
||||
|
|
|
|||
|
|
@ -116,7 +116,7 @@ private[http] object HttpServerBluePrint {
|
|||
.flatten(FlattenStrategy.concat)
|
||||
.via(Flow[ResponseRenderingOutput].transform(() ⇒ errorLogger(log, "Outgoing response stream error")).named("errorLogger"))
|
||||
|
||||
BidiFlow(requestParsingFlow, rendererPipeline, oneHundredContinueSource)((_, _, _) ⇒ ()) { implicit b ⇒
|
||||
BidiFlow.fromGraph(FlowGraph.create(requestParsingFlow, rendererPipeline, oneHundredContinueSource)((_, _, _) ⇒ ()) { implicit b ⇒
|
||||
(requestParsing, renderer, oneHundreds) ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
|
|
@ -172,7 +172,7 @@ private[http] object HttpServerBluePrint {
|
|||
wrapTls.outlet,
|
||||
unwrapTls.inlet,
|
||||
requestsIn)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
class BypassMerge(settings: ServerSettings, log: LoggingAdapter)
|
||||
|
|
@ -319,7 +319,7 @@ private[http] object HttpServerBluePrint {
|
|||
val sink = StreamUtils.oneTimePublisherSink[FrameEvent](sinkCell, "frameHandler.in")
|
||||
val source = StreamUtils.oneTimeSubscriberSource[FrameEvent](sourceCell, "frameHandler.out")
|
||||
|
||||
val flow = Websocket.framing.join(Flow.wrap(sink, source)(Keep.none))
|
||||
val flow = Websocket.framing.join(Flow.fromSinkAndSourceMat(sink, source)(Keep.none))
|
||||
|
||||
new WebsocketSetup {
|
||||
def websocketFlow: Flow[ByteString, ByteString, Any] = flow
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@ import akka.stream.stage.{ SyncDirective, Context, StatefulStage }
|
|||
*/
|
||||
private[http] object Masking {
|
||||
def apply(serverSide: Boolean, maskRandom: () ⇒ Random): BidiFlow[ /* net in */ FrameEvent, /* app out */ FrameEvent, /* app in */ FrameEvent, /* net out */ FrameEvent, Unit] =
|
||||
BidiFlow.wrap(unmaskIf(serverSide), maskIf(!serverSide, maskRandom))(Keep.none)
|
||||
BidiFlow.fromFlowsMat(unmaskIf(serverSide), maskIf(!serverSide, maskRandom))(Keep.none)
|
||||
|
||||
def maskIf(condition: Boolean, maskRandom: () ⇒ Random): Flow[FrameEvent, FrameEvent, Unit] =
|
||||
if (condition) Flow[FrameEvent].transform(() ⇒ new Masking(maskRandom())) // new random per materialization
|
||||
|
|
|
|||
|
|
@ -39,7 +39,7 @@ private[http] object Websocket {
|
|||
|
||||
/** The lowest layer that implements the binary protocol */
|
||||
def framing: BidiFlow[ByteString, FrameEvent, FrameEvent, ByteString, Unit] =
|
||||
BidiFlow.wrap(
|
||||
BidiFlow.fromFlowsMat(
|
||||
Flow[ByteString].transform(() ⇒ new FrameEventParser),
|
||||
Flow[FrameEvent].transform(() ⇒ new FrameEventRenderer))(Keep.none)
|
||||
.named("ws-framing")
|
||||
|
|
@ -56,7 +56,7 @@ private[http] object Websocket {
|
|||
def frameHandling(serverSide: Boolean = true,
|
||||
closeTimeout: FiniteDuration,
|
||||
log: LoggingAdapter): BidiFlow[FrameEvent, FrameHandler.Output, FrameOutHandler.Input, FrameStart, Unit] =
|
||||
BidiFlow.wrap(
|
||||
BidiFlow.fromFlowsMat(
|
||||
FrameHandler.create(server = serverSide),
|
||||
FrameOutHandler.create(serverSide, closeTimeout, log))(Keep.none)
|
||||
.named("ws-frame-handling")
|
||||
|
|
@ -121,7 +121,7 @@ private[http] object Websocket {
|
|||
MessageToFrameRenderer.create(serverSide)
|
||||
.named("ws-render-messages")
|
||||
|
||||
BidiFlow() { implicit b ⇒
|
||||
BidiFlow.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val split = b.add(BypassRouter)
|
||||
|
|
@ -146,7 +146,7 @@ private[http] object Websocket {
|
|||
messagePreparation.outlet,
|
||||
messageRendering.inlet,
|
||||
merge.out)
|
||||
}.named("ws-message-api")
|
||||
}.named("ws-message-api"))
|
||||
}
|
||||
|
||||
private object BypassRouter extends GraphStage[FanOutShape2[Output, BypassEvent, MessagePart]] {
|
||||
|
|
|
|||
|
|
@ -109,7 +109,7 @@ object WebsocketClientBlueprint {
|
|||
}
|
||||
}
|
||||
|
||||
BidiFlow() { implicit b ⇒
|
||||
BidiFlow.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val networkIn = b.add(Flow[ByteString].transform(() ⇒ new UpgradeStage))
|
||||
|
|
@ -126,11 +126,11 @@ object WebsocketClientBlueprint {
|
|||
networkIn.outlet,
|
||||
wsIn.inlet,
|
||||
httpRequestBytesAndThenWSBytes.out)
|
||||
} mapMaterializedValue (_ ⇒ result.future)
|
||||
}) mapMaterializedValue (_ ⇒ result.future)
|
||||
}
|
||||
|
||||
def simpleTls: BidiFlow[SslTlsInbound, ByteString, ByteString, SendBytes, Unit] =
|
||||
BidiFlow.wrap(
|
||||
BidiFlow.fromFlowsMat(
|
||||
Flow[SslTlsInbound].collect { case SessionBytes(_, bytes) ⇒ bytes },
|
||||
Flow[ByteString].map(SendBytes))(Keep.none)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -113,7 +113,7 @@ private[http] object JavaMapping {
|
|||
def toScala(javaObject: javadsl.Flow[JIn, JOut, M]): S =
|
||||
scaladsl.Flow[SIn].map(inMapping.toJava(_)).viaMat(javaObject)(scaladsl.Keep.right).map(outMapping.toScala(_))
|
||||
def toJava(scalaObject: scaladsl.Flow[SIn, SOut, M]): J =
|
||||
javadsl.Flow.wrap {
|
||||
javadsl.Flow.fromGraph {
|
||||
scaladsl.Flow[JIn].map(inMapping.toScala(_)).viaMat(scalaObject)(scaladsl.Keep.right).map(outMapping.toJava(_))
|
||||
}
|
||||
}
|
||||
|
|
@ -123,7 +123,7 @@ private[http] object JavaMapping {
|
|||
def javaToScalaAdapterFlow[J, S](implicit mapping: JavaMapping[J, S]): scaladsl.Flow[J, S, Unit] =
|
||||
scaladsl.Flow[J].map(mapping.toScala(_))
|
||||
def adapterBidiFlow[JIn, SIn, SOut, JOut](implicit inMapping: JavaMapping[JIn, SIn], outMapping: JavaMapping[JOut, SOut]): scaladsl.BidiFlow[JIn, SIn, SOut, JOut, Unit] =
|
||||
scaladsl.BidiFlow.wrap(javaToScalaAdapterFlow(inMapping), scalaToJavaAdapterFlow(outMapping))(scaladsl.Keep.none)
|
||||
scaladsl.BidiFlow.fromFlowsMat(javaToScalaAdapterFlow(inMapping), scalaToJavaAdapterFlow(outMapping))(scaladsl.Keep.none)
|
||||
|
||||
implicit def pairMapping[J1, J2, S1, S2](implicit _1Mapping: JavaMapping[J1, S1], _2Mapping: JavaMapping[J2, S2]): JavaMapping[Pair[J1, J2], (S1, S2)] =
|
||||
new JavaMapping[Pair[J1, J2], (S1, S2)] {
|
||||
|
|
|
|||
|
|
@ -86,7 +86,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
* [[ServerBinding]].
|
||||
*/
|
||||
def bind(interface: String, port: Int, materializer: Materializer): Source[IncomingConnection, Future[ServerBinding]] =
|
||||
Source.adapt(delegate.bind(interface, port)(materializer)
|
||||
new Source(delegate.bind(interface, port)(materializer)
|
||||
.map(new IncomingConnection(_))
|
||||
.mapMaterializedValue(_.map(new ServerBinding(_))(ec)))
|
||||
|
||||
|
|
@ -105,7 +105,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
httpsContext: Option[HttpsContext],
|
||||
log: LoggingAdapter,
|
||||
materializer: Materializer): Source[IncomingConnection, Future[ServerBinding]] =
|
||||
Source.adapt(delegate.bind(interface, port, settings, httpsContext, log)(materializer)
|
||||
new Source(delegate.bind(interface, port, settings, httpsContext, log)(materializer)
|
||||
.map(new IncomingConnection(_))
|
||||
.mapMaterializedValue(_.map(new ServerBinding(_))(ec)))
|
||||
|
||||
|
|
@ -238,7 +238,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
* Every materialization of the produced flow will attempt to establish a new outgoing connection.
|
||||
*/
|
||||
def outgoingConnection(host: String, port: Int): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
|
||||
Flow.wrap {
|
||||
Flow.fromGraph {
|
||||
akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala)
|
||||
.viaMat(delegate.outgoingConnection(host, port))(Keep.right)
|
||||
.mapMaterializedValue(_.map(new OutgoingConnection(_))(ec))
|
||||
|
|
@ -248,7 +248,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
* Same as [[outgoingConnection]] but with HTTPS encryption.
|
||||
*/
|
||||
def outgoingConnectionTls(host: String, port: Int): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
|
||||
Flow.wrap {
|
||||
Flow.fromGraph {
|
||||
akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala)
|
||||
.viaMat(delegate.outgoingConnectionTls(host, port))(Keep.right)
|
||||
.mapMaterializedValue(_.map(new OutgoingConnection(_))(ec))
|
||||
|
|
@ -262,7 +262,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
localAddress: Option[InetSocketAddress],
|
||||
settings: ClientConnectionSettings,
|
||||
log: LoggingAdapter): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
|
||||
Flow.wrap {
|
||||
Flow.fromGraph {
|
||||
akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala)
|
||||
.viaMat(delegate.outgoingConnection(host, port, localAddress.asScala, settings, log))(Keep.right)
|
||||
.mapMaterializedValue(_.map(new OutgoingConnection(_))(ec))
|
||||
|
|
@ -279,7 +279,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
settings: ClientConnectionSettings,
|
||||
httpsContext: Option[HttpsContext],
|
||||
log: LoggingAdapter): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
|
||||
Flow.wrap {
|
||||
Flow.fromGraph {
|
||||
akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala)
|
||||
.viaMat(delegate.outgoingConnectionTls(host, port, localAddress.asScala, settings,
|
||||
httpsContext.map(_.asInstanceOf[akka.http.scaladsl.HttpsContext]), log))(Keep.right)
|
||||
|
|
@ -629,7 +629,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
.atopMat(wsLayer)((_, s) ⇒ adaptWsUpgradeResponse(s)))
|
||||
|
||||
private def adaptWsFlow(wsLayer: stream.scaladsl.Flow[sm.ws.Message, sm.ws.Message, Future[scaladsl.model.ws.WebsocketUpgradeResponse]]): Flow[Message, Message, Future[WebsocketUpgradeResponse]] =
|
||||
Flow.adapt(JavaMapping.adapterBidiFlow[Message, sm.ws.Message, sm.ws.Message, Message].joinMat(wsLayer)(Keep.right).mapMaterializedValue(adaptWsUpgradeResponse _))
|
||||
Flow.fromGraph(JavaMapping.adapterBidiFlow[Message, sm.ws.Message, sm.ws.Message, Message].joinMat(wsLayer)(Keep.right).mapMaterializedValue(adaptWsUpgradeResponse _))
|
||||
|
||||
private def adaptWsFlow[Mat](javaFlow: Flow[Message, Message, Mat]): stream.scaladsl.Flow[scaladsl.model.ws.Message, scaladsl.model.ws.Message, Mat] =
|
||||
stream.scaladsl.Flow[scaladsl.model.ws.Message]
|
||||
|
|
|
|||
|
|
@ -31,7 +31,7 @@ class IncomingConnection private[http] (delegate: akka.http.scaladsl.Http.Incomi
|
|||
*
|
||||
* Use `Flow.join` or one of the handleXXX methods to consume handle requests on this connection.
|
||||
*/
|
||||
def flow: Flow[HttpResponse, HttpRequest, Unit] = Flow.adapt(delegate.flow).asInstanceOf[Flow[HttpResponse, HttpRequest, Unit]]
|
||||
def flow: Flow[HttpResponse, HttpRequest, Unit] = Flow.fromGraph(delegate.flow).asInstanceOf[Flow[HttpResponse, HttpRequest, Unit]]
|
||||
|
||||
/**
|
||||
* Handles the connection with the given flow, which is materialized exactly once
|
||||
|
|
|
|||
|
|
@ -38,18 +38,18 @@ object WebsocketUpgradeResponse {
|
|||
import akka.http.impl.util.JavaMapping.Implicits._
|
||||
def adapt(scalaResponse: scaladsl.model.ws.WebsocketUpgradeResponse): WebsocketUpgradeResponse =
|
||||
scalaResponse match {
|
||||
case ValidUpgrade(response, chosen) ⇒
|
||||
case ValidUpgrade(resp, chosen) ⇒
|
||||
new WebsocketUpgradeResponse {
|
||||
def isValid: Boolean = true
|
||||
def response: HttpResponse = scalaResponse.response
|
||||
def response: HttpResponse = resp
|
||||
def chosenSubprotocol: Option[String] = chosen.asJava
|
||||
def invalidationReason: String =
|
||||
throw new UnsupportedOperationException("invalidationReason must not be called for valid response")
|
||||
}
|
||||
case InvalidUpgradeResponse(response, cause) ⇒
|
||||
case InvalidUpgradeResponse(resp, cause) ⇒
|
||||
new WebsocketUpgradeResponse {
|
||||
def isValid: Boolean = false
|
||||
def response: HttpResponse = scalaResponse.response
|
||||
def response: HttpResponse = resp
|
||||
def chosenSubprotocol: Option[String] = throw new UnsupportedOperationException("chosenSubprotocol must not be called for valid response")
|
||||
def invalidationReason: String = cause
|
||||
}
|
||||
|
|
|
|||
|
|
@ -75,7 +75,7 @@ sealed trait HttpEntity extends jm.HttpEntity {
|
|||
def withContentType(contentType: ContentType): HttpEntity
|
||||
|
||||
/** Java API */
|
||||
def getDataBytes: stream.javadsl.Source[ByteString, _] = stream.javadsl.Source.adapt(dataBytes)
|
||||
def getDataBytes: stream.javadsl.Source[ByteString, _] = stream.javadsl.Source.fromGraph(dataBytes)
|
||||
|
||||
/** Java API */
|
||||
def getContentLengthOption: japi.Option[JLong] =
|
||||
|
|
@ -296,7 +296,7 @@ object HttpEntity {
|
|||
override def productPrefix = "HttpEntity.Chunked"
|
||||
|
||||
/** Java API */
|
||||
def getChunks: stream.javadsl.Source[jm.ChunkStreamPart, Any] = stream.javadsl.Source.adapt(chunks)
|
||||
def getChunks: stream.javadsl.Source[jm.ChunkStreamPart, Any] = stream.javadsl.Source.fromGraph(chunks)
|
||||
}
|
||||
object Chunked {
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -52,7 +52,7 @@ trait UpgradeToWebsocket extends jm.ws.UpgradeToWebsocket {
|
|||
def handleMessagesWithSinkSource(inSink: Sink[Message, Any],
|
||||
outSource: Source[Message, Any],
|
||||
subprotocol: Option[String] = None): HttpResponse =
|
||||
handleMessages(Flow.wrap(inSink, outSource)((_, _) ⇒ ()), subprotocol)
|
||||
handleMessages(Flow.fromSinkAndSource(inSink, outSource), subprotocol)
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
|
|
@ -88,5 +88,5 @@ trait UpgradeToWebsocket extends jm.ws.UpgradeToWebsocket {
|
|||
handleMessages(createScalaFlow(inSink, outSource), subprotocol = Some(subprotocol))
|
||||
|
||||
private[this] def createScalaFlow(inSink: stream.javadsl.Sink[jm.ws.Message, _], outSource: stream.javadsl.Source[jm.ws.Message, _]): Flow[Message, Message, Any] =
|
||||
JavaMapping.toScala(Flow.wrap(inSink.asScala, outSource.asScala)((_, _) ⇒ ()).asJava)
|
||||
JavaMapping.toScala(Flow.fromSinkAndSourceMat(inSink.asScala, outSource.asScala)((_, _) ⇒ ()).asJava)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -63,7 +63,7 @@ public class WSEchoTestClientApp {
|
|||
.toMat(Sink.<List<String>>head(), Keep.<BoxedUnit, Future<List<String>>>right());
|
||||
|
||||
Flow<Message, Message, Future<List<String>>> echoClient =
|
||||
Flow.wrap(echoSink, echoSource, Keep.<Future<List<String>>, BoxedUnit>left());
|
||||
Flow.fromSinkAndSourceMat(echoSink, echoSource, Keep.<Future<List<String>>, BoxedUnit>left());
|
||||
|
||||
Future<List<String>> result =
|
||||
Http.get(system).singleWebsocketRequest(
|
||||
|
|
|
|||
|
|
@ -298,12 +298,10 @@ class ConnectionPoolSpec extends AkkaSpec("""
|
|||
val incomingConnectionCounter = new AtomicInteger
|
||||
val incomingConnections = TestSubscriber.manualProbe[Http.IncomingConnection]
|
||||
val incomingConnectionsSub = {
|
||||
val rawBytesInjection = BidiFlow() { b ⇒
|
||||
val top = b.add(Flow[SslTlsOutbound].collect[ByteString] { case SendBytes(x) ⇒ mapServerSideOutboundRawBytes(x) }
|
||||
.transform(StreamUtils.recover { case NoErrorComplete ⇒ ByteString.empty }))
|
||||
val bottom = b.add(Flow[ByteString].map(SessionBytes(null, _)))
|
||||
BidiShape(top.inlet, top.outlet, bottom.inlet, bottom.outlet)
|
||||
}
|
||||
val rawBytesInjection = BidiFlow.fromFlows(
|
||||
Flow[SslTlsOutbound].collect[ByteString] { case SendBytes(x) ⇒ mapServerSideOutboundRawBytes(x) }
|
||||
.transform(StreamUtils.recover { case NoErrorComplete ⇒ ByteString.empty }),
|
||||
Flow[ByteString].map(SessionBytes(null, _)))
|
||||
val sink = if (autoAccept) Sink.foreach[Http.IncomingConnection](handleConnection) else Sink(incomingConnections)
|
||||
// TODO getHostString in Java7
|
||||
Tcp().bind(serverEndpoint.getHostName, serverEndpoint.getPort, idleTimeout = serverSettings.timeouts.idleTimeout)
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.http.impl.engine.client
|
|||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.{ FlowShape, ActorMaterializer }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.http.scaladsl.{ Http, TestUtils }
|
||||
|
|
@ -44,7 +44,7 @@ class HighLevelOutgoingConnectionSpec extends AkkaSpec {
|
|||
val connFlow = Http().outgoingConnection(serverHostName, serverPort)
|
||||
|
||||
val C = 4
|
||||
val doubleConnection = Flow() { implicit b ⇒
|
||||
val doubleConnection = Flow.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val bcast = b.add(Broadcast[HttpRequest](C))
|
||||
|
|
@ -52,8 +52,8 @@ class HighLevelOutgoingConnectionSpec extends AkkaSpec {
|
|||
|
||||
for (i ← 0 until C)
|
||||
bcast.out(i) ~> connFlow ~> merge.in(i)
|
||||
(bcast.in, merge.out)
|
||||
}
|
||||
FlowShape(bcast.in, merge.out)
|
||||
})
|
||||
|
||||
val N = 100
|
||||
val result = Source(() ⇒ Iterator.from(1))
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import akka.stream.io.{ SessionBytes, SslTlsOutbound, SendBytes }
|
|||
import org.scalatest.Inside
|
||||
import akka.util.ByteString
|
||||
import akka.event.NoLogging
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.{ ClosedShape, ActorMaterializer }
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.scaladsl._
|
||||
import akka.http.scaladsl.model.HttpEntity._
|
||||
|
|
@ -359,14 +359,15 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
val netOut = TestSubscriber.manualProbe[ByteString]
|
||||
val netIn = TestPublisher.manualProbe[ByteString]()
|
||||
|
||||
FlowGraph.closed(OutgoingConnectionBlueprint(Host("example.com"), settings, NoLogging)) { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create(OutgoingConnectionBlueprint(Host("example.com"), settings, NoLogging)) { implicit b ⇒
|
||||
client ⇒
|
||||
import FlowGraph.Implicits._
|
||||
Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> client.in2
|
||||
client.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) ⇒ x } ~> Sink(netOut)
|
||||
Source(requests) ~> client.in1
|
||||
client.out2 ~> Sink(responses)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
netOut -> netIn
|
||||
}
|
||||
|
|
|
|||
|
|
@ -15,8 +15,8 @@ import akka.actor.ActorSystem
|
|||
import akka.event.NoLogging
|
||||
import akka.util.ByteString
|
||||
|
||||
import akka.stream.Materializer
|
||||
import akka.stream.scaladsl.{ Flow, Sink, Source, FlowGraph }
|
||||
import akka.stream.{ ClosedShape, Materializer }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit.{ TestPublisher, TestSubscriber }
|
||||
|
||||
import akka.http.impl.util._
|
||||
|
|
@ -39,14 +39,15 @@ abstract class HttpServerTestSetupBase {
|
|||
val netIn = TestPublisher.probe[ByteString]()
|
||||
val netOut = ByteStringSinkProbe()
|
||||
|
||||
FlowGraph.closed(HttpServerBluePrint(settings, remoteAddress = remoteAddress, log = NoLogging)) { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create(HttpServerBluePrint(settings, remoteAddress = remoteAddress, log = NoLogging)) { implicit b ⇒
|
||||
server ⇒
|
||||
import FlowGraph.Implicits._
|
||||
Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> server.in2
|
||||
server.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) ⇒ x } ~> netOut.sink
|
||||
server.out2 ~> Sink(requests)
|
||||
Source(responses) ~> server.in1
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
netIn -> netOut
|
||||
}
|
||||
|
|
|
|||
|
|
@ -50,7 +50,7 @@ object EchoTestClientApp extends App {
|
|||
.grouped(10000)
|
||||
.toMat(Sink.head)(Keep.right)
|
||||
|
||||
def echoClient = Flow.wrap(sink, source)(Keep.left)
|
||||
def echoClient = Flow.fromSinkAndSourceMat(sink, source)(Keep.left)
|
||||
|
||||
val (upgrade, res) = Http().singleWebsocketRequest("wss://echo.websocket.org", echoClient)
|
||||
res onComplete {
|
||||
|
|
|
|||
|
|
@ -817,8 +817,8 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
val messageOut = TestPublisher.probe[Message]()
|
||||
|
||||
val messageHandler: Flow[Message, Message, Unit] =
|
||||
Flow.wrap {
|
||||
FlowGraph.partial() { implicit b ⇒
|
||||
Flow.fromGraph {
|
||||
FlowGraph.create() { implicit b ⇒
|
||||
val in = b.add(Sink(messageIn))
|
||||
val out = b.add(Source(messageOut))
|
||||
|
||||
|
|
|
|||
|
|
@ -199,7 +199,7 @@ object WSClientAutobahnTest extends App {
|
|||
*/
|
||||
def runToSingleText(uri: Uri): Future[String] = {
|
||||
val sink = Sink.head[Message]
|
||||
runWs(uri, Flow.wrap(sink, Source.lazyEmpty[Message])(Keep.left)).flatMap {
|
||||
runWs(uri, Flow.fromSinkAndSourceMat(sink, Source.maybe[Message])(Keep.left)).flatMap {
|
||||
case tm: TextMessage ⇒ tm.textStream.runWith(Sink.fold("")(_ + _))
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -7,6 +7,7 @@ package akka.http.impl.engine.ws
|
|||
import java.util.Random
|
||||
|
||||
import akka.http.scaladsl.model.ws.{ InvalidUpgradeResponse, WebsocketUpgradeResponse }
|
||||
import akka.stream.ClosedShape
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
|
|
@ -119,7 +120,7 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
|
|||
|
||||
"don't send out frames before handshake was finished successfully" in new TestSetup {
|
||||
def clientImplementation: Flow[Message, Message, Unit] =
|
||||
Flow.wrap(Sink.ignore, Source.single(TextMessage("fast message")))(Keep.none)
|
||||
Flow.fromSinkAndSourceMat(Sink.ignore, Source.single(TextMessage("fast message")))(Keep.none)
|
||||
|
||||
expectWireData(UpgradeRequestBytes)
|
||||
expectNoWireData()
|
||||
|
|
@ -311,13 +312,14 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
|
|||
val netIn = TestPublisher.probe[ByteString]()
|
||||
|
||||
val graph =
|
||||
FlowGraph.closed(clientLayer) { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create(clientLayer) { implicit b ⇒
|
||||
client ⇒
|
||||
import FlowGraph.Implicits._
|
||||
Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> client.in2
|
||||
client.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) ⇒ x } ~> netOut.sink
|
||||
client.out2 ~> clientImplementation ~> client.in1
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
|
||||
val response = graph.run()
|
||||
|
||||
|
|
@ -365,6 +367,6 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
|
|||
lazy val messagesIn = TestSubscriber.probe[Message]()
|
||||
|
||||
override def clientImplementation: Flow[Message, Message, Unit] =
|
||||
Flow.wrap(Sink(messagesIn), Source(messagesOut))(Keep.none)
|
||||
Flow.fromSinkAndSourceMat(Sink(messagesIn), Source(messagesOut))(Keep.none)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -38,7 +38,7 @@ class WebsocketServerSpec extends FreeSpec with Matchers with WithMaterializerSp
|
|||
|
||||
val source =
|
||||
Source(List(1, 2, 3, 4, 5)).map(num ⇒ TextMessage.Strict(s"Message $num"))
|
||||
val handler = Flow.wrap(Sink.ignore, source)(Keep.none)
|
||||
val handler = Flow.fromSinkAndSourceMat(Sink.ignore, source)(Keep.none)
|
||||
val response = upgrade.get.handleMessages(handler)
|
||||
responses.sendNext(response)
|
||||
|
||||
|
|
|
|||
|
|
@ -330,10 +330,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
val sink = Sink.publisher[HttpRequest]
|
||||
val source = Source.subscriber[HttpResponse]
|
||||
|
||||
val handler = Flow(sink, source)(Keep.both) { implicit b ⇒
|
||||
(snk, src) ⇒
|
||||
(snk.inlet, src.outlet)
|
||||
}
|
||||
val handler = Flow.fromSinkAndSourceMat(sink, source)(Keep.both)
|
||||
|
||||
val (pub, sub) = incomingConnection.handleWith(handler)
|
||||
val requestSubscriberProbe = TestSubscriber.manualProbe[HttpRequest]()
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue