!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:
Viktor Klang 2015-10-21 22:45:39 +02:00 committed by Roland Kuhn
parent 0f99a42df9
commit f29d7affbd
120 changed files with 1535 additions and 1897 deletions

View file

@ -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
}

View file

@ -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))

View file

@ -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)
})
}

View file

@ -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 {

View file

@ -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) =

View file

@ -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

View file

@ -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

View file

@ -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]] {

View file

@ -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)
}

View file

@ -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)] {

View file

@ -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]

View file

@ -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

View file

@ -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
}

View file

@ -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 {
/**

View file

@ -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)
}

View file

@ -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(

View file

@ -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)

View file

@ -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))

View file

@ -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
}

View file

@ -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
}

View file

@ -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 {

View file

@ -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))

View file

@ -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("")(_ + _))
}
}

View file

@ -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)
}
}

View file

@ -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)

View file

@ -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]()