= str - #15618 - Renames Flow and Duct 'tee' operation to 'broadcast'
This commit is contained in:
parent
2ccf028a94
commit
6c7dc10a50
11 changed files with 21 additions and 21 deletions
|
|
@ -42,7 +42,7 @@ private[http] class HttpClientPipeline(effectiveSettings: ClientConnectionSettin
|
||||||
|
|
||||||
val requestSubscriber =
|
val requestSubscriber =
|
||||||
Duct[(HttpRequest, Any)]
|
Duct[(HttpRequest, Any)]
|
||||||
.tee(contextBypassSubscriber)
|
.broadcast(contextBypassSubscriber)
|
||||||
.map(requestMethodByPass)
|
.map(requestMethodByPass)
|
||||||
.transform(responseRendererFactory.newRenderer)
|
.transform(responseRendererFactory.newRenderer)
|
||||||
.flatten(FlattenStrategy.concat)
|
.flatten(FlattenStrategy.concat)
|
||||||
|
|
|
||||||
|
|
@ -44,7 +44,7 @@ private[http] class HttpServerPipeline(settings: ServerSettings,
|
||||||
.transform(rootParser.copyWith(warnOnIllegalHeader))
|
.transform(rootParser.copyWith(warnOnIllegalHeader))
|
||||||
.splitWhen(_.isInstanceOf[MessageStart])
|
.splitWhen(_.isInstanceOf[MessageStart])
|
||||||
.headAndTail(materializer)
|
.headAndTail(materializer)
|
||||||
.tee(applicationBypassSubscriber)
|
.broadcast(applicationBypassSubscriber)
|
||||||
.collect {
|
.collect {
|
||||||
case (RequestStart(method, uri, protocol, headers, createEntity, _), entityParts) ⇒
|
case (RequestStart(method, uri, protocol, headers, createEntity, _), entityParts) ⇒
|
||||||
val effectiveUri = HttpRequest.effectiveUri(uri, headers, securedConnection = false, settings.defaultHostHeader)
|
val effectiveUri = HttpRequest.effectiveUri(uri, headers, securedConnection = false, settings.defaultHostHeader)
|
||||||
|
|
|
||||||
|
|
@ -51,8 +51,8 @@ private[akka] object Ast {
|
||||||
case class Concat(next: Publisher[Any]) extends AstNode {
|
case class Concat(next: Publisher[Any]) extends AstNode {
|
||||||
override def name = "concat"
|
override def name = "concat"
|
||||||
}
|
}
|
||||||
case class Tee(other: Subscriber[Any]) extends AstNode {
|
case class Broadcast(other: Subscriber[Any]) extends AstNode {
|
||||||
override def name = "tee"
|
override def name = "broadcast"
|
||||||
}
|
}
|
||||||
case class PrefixAndTail(n: Int) extends AstNode {
|
case class PrefixAndTail(n: Int) extends AstNode {
|
||||||
override def name = "prefixAndTail"
|
override def name = "prefixAndTail"
|
||||||
|
|
|
||||||
|
|
@ -26,7 +26,7 @@ private[akka] object ActorProcessor {
|
||||||
case m: Merge ⇒ Props(new MergeImpl(settings, m.other))
|
case m: Merge ⇒ Props(new MergeImpl(settings, m.other))
|
||||||
case z: Zip ⇒ Props(new ZipImpl(settings, z.other))
|
case z: Zip ⇒ Props(new ZipImpl(settings, z.other))
|
||||||
case c: Concat ⇒ Props(new ConcatImpl(settings, c.next))
|
case c: Concat ⇒ Props(new ConcatImpl(settings, c.next))
|
||||||
case t: Tee ⇒ Props(new TeeImpl(settings, t.other))
|
case b: Broadcast ⇒ Props(new BroadcastImpl(settings, b.other))
|
||||||
case cf: Conflate ⇒ Props(new ConflateImpl(settings, cf.seed, cf.aggregate))
|
case cf: Conflate ⇒ Props(new ConflateImpl(settings, cf.seed, cf.aggregate))
|
||||||
case ex: Expand ⇒ Props(new ExpandImpl(settings, ex.seed, ex.extrapolate))
|
case ex: Expand ⇒ Props(new ExpandImpl(settings, ex.seed, ex.extrapolate))
|
||||||
case bf: Buffer ⇒ Props(new BufferImpl(settings, bf.size, bf.overflowStrategy))
|
case bf: Buffer ⇒ Props(new BufferImpl(settings, bf.size, bf.overflowStrategy))
|
||||||
|
|
|
||||||
|
|
@ -313,7 +313,7 @@ private[akka] trait Builder[Out] {
|
||||||
|
|
||||||
def groupBy[K](f: (Out) ⇒ K): Thing[(K, Publisher[Out])] = andThen(GroupBy(f.asInstanceOf[Any ⇒ Any]))
|
def groupBy[K](f: (Out) ⇒ K): Thing[(K, Publisher[Out])] = andThen(GroupBy(f.asInstanceOf[Any ⇒ Any]))
|
||||||
|
|
||||||
def tee(other: Subscriber[_ >: Out]): Thing[Out] = andThen(Tee(other.asInstanceOf[Subscriber[Any]]))
|
def broadcast(other: Subscriber[_ >: Out]): Thing[Out] = andThen(Broadcast(other.asInstanceOf[Subscriber[Any]]))
|
||||||
|
|
||||||
def conflate[S](seed: Out ⇒ S, aggregate: (S, Out) ⇒ S): Thing[S] =
|
def conflate[S](seed: Out ⇒ S, aggregate: (S, Out) ⇒ S): Thing[S] =
|
||||||
andThen(Conflate(seed.asInstanceOf[Any ⇒ Any], aggregate.asInstanceOf[(Any, Any) ⇒ Any]))
|
andThen(Conflate(seed.asInstanceOf[Any ⇒ Any], aggregate.asInstanceOf[(Any, Any) ⇒ Any]))
|
||||||
|
|
|
||||||
|
|
@ -9,7 +9,7 @@ import org.reactivestreams.{ Subscriber, Subscription, Publisher }
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] class TeeImpl(_settings: MaterializerSettings, other: Subscriber[Any])
|
private[akka] class BroadcastImpl(_settings: MaterializerSettings, other: Subscriber[Any])
|
||||||
extends ActorProcessorImpl(_settings) {
|
extends ActorProcessorImpl(_settings) {
|
||||||
|
|
||||||
override val primaryOutputs = new FanoutOutputs(settings.maxFanOutBufferSize, settings.initialFanOutBufferSize, self, pump = this) {
|
override val primaryOutputs = new FanoutOutputs(settings.maxFanOutBufferSize, settings.initialFanOutBufferSize, self, pump = this) {
|
||||||
|
|
|
||||||
|
|
@ -226,7 +226,7 @@ abstract class Duct[In, Out] {
|
||||||
* not shutdown until the subscriptions for `other` and at least
|
* not shutdown until the subscriptions for `other` and at least
|
||||||
* one downstream subscriber have been established.
|
* one downstream subscriber have been established.
|
||||||
*/
|
*/
|
||||||
def tee(other: Subscriber[_ >: Out]): Duct[In, Out]
|
def broadcast(other: Subscriber[_ >: Out]): Duct[In, Out]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Transforms a stream of streams into a contiguous stream of elements using the provided flattening strategy.
|
* Transforms a stream of streams into a contiguous stream of elements using the provided flattening strategy.
|
||||||
|
|
@ -393,8 +393,8 @@ private[akka] class DuctAdapter[In, T](delegate: SDuct[In, T]) extends Duct[In,
|
||||||
override def concat[U >: T](next: Publisher[U]): Duct[In, U] =
|
override def concat[U >: T](next: Publisher[U]): Duct[In, U] =
|
||||||
new DuctAdapter(delegate.concat(next))
|
new DuctAdapter(delegate.concat(next))
|
||||||
|
|
||||||
override def tee(other: Subscriber[_ >: T]): Duct[In, T] =
|
override def broadcast(other: Subscriber[_ >: T]): Duct[In, T] =
|
||||||
new DuctAdapter(delegate.tee(other))
|
new DuctAdapter(delegate.broadcast(other))
|
||||||
|
|
||||||
override def buffer(size: Int, overflowStrategy: OverflowStrategy): Duct[In, T] =
|
override def buffer(size: Int, overflowStrategy: OverflowStrategy): Duct[In, T] =
|
||||||
new DuctAdapter(delegate.buffer(size, overflowStrategy))
|
new DuctAdapter(delegate.buffer(size, overflowStrategy))
|
||||||
|
|
|
||||||
|
|
@ -287,7 +287,7 @@ abstract class Flow[T] {
|
||||||
* not shutdown until the subscriptions for `other` and at least
|
* not shutdown until the subscriptions for `other` and at least
|
||||||
* one downstream subscriber have been established.
|
* one downstream subscriber have been established.
|
||||||
*/
|
*/
|
||||||
def tee(other: Subscriber[_ >: T]): Flow[T]
|
def broadcast(other: Subscriber[_ >: T]): Flow[T]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Append the operations of a [[Duct]] to this flow.
|
* Append the operations of a [[Duct]] to this flow.
|
||||||
|
|
@ -460,8 +460,8 @@ private[akka] class FlowAdapter[T](delegate: SFlow[T]) extends Flow[T] {
|
||||||
override def concat[U >: T](next: Publisher[U]): Flow[U] =
|
override def concat[U >: T](next: Publisher[U]): Flow[U] =
|
||||||
new FlowAdapter(delegate.concat(next))
|
new FlowAdapter(delegate.concat(next))
|
||||||
|
|
||||||
override def tee(other: Subscriber[_ >: T]): Flow[T] =
|
override def broadcast(other: Subscriber[_ >: T]): Flow[T] =
|
||||||
new FlowAdapter(delegate.tee(other))
|
new FlowAdapter(delegate.broadcast(other))
|
||||||
|
|
||||||
override def flatten[U](strategy: FlattenStrategy[T, U]): Flow[U] =
|
override def flatten[U](strategy: FlattenStrategy[T, U]): Flow[U] =
|
||||||
new FlowAdapter(delegate.flatten(strategy))
|
new FlowAdapter(delegate.flatten(strategy))
|
||||||
|
|
|
||||||
|
|
@ -215,7 +215,7 @@ trait Duct[In, +Out] {
|
||||||
* not shutdown until the subscriptions for `other` and at least
|
* not shutdown until the subscriptions for `other` and at least
|
||||||
* one downstream subscriber have been established.
|
* one downstream subscriber have been established.
|
||||||
*/
|
*/
|
||||||
def tee(other: Subscriber[_ >: Out]): Duct[In, Out]
|
def broadcast(other: Subscriber[_ >: Out]): Duct[In, Out]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Transforms a stream of streams into a contiguous stream of elements using the provided flattening strategy.
|
* Transforms a stream of streams into a contiguous stream of elements using the provided flattening strategy.
|
||||||
|
|
|
||||||
|
|
@ -283,7 +283,7 @@ trait Flow[+T] {
|
||||||
* not shutdown until the subscriptions for `other` and at least
|
* not shutdown until the subscriptions for `other` and at least
|
||||||
* one downstream subscriber have been established.
|
* one downstream subscriber have been established.
|
||||||
*/
|
*/
|
||||||
def tee(other: Subscriber[_ >: T]): Flow[T]
|
def broadcast(other: Subscriber[_ >: T]): Flow[T]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Transforms a stream of streams into a contiguous stream of elements using the provided flattening strategy.
|
* Transforms a stream of streams into a contiguous stream of elements using the provided flattening strategy.
|
||||||
|
|
|
||||||
|
|
@ -9,7 +9,7 @@ import akka.stream.testkit.AkkaSpec
|
||||||
import akka.stream.testkit.StreamTestKit
|
import akka.stream.testkit.StreamTestKit
|
||||||
|
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowTeeSpec extends AkkaSpec {
|
class FlowBroadcastSpec extends AkkaSpec {
|
||||||
|
|
||||||
val materializer = FlowMaterializer(MaterializerSettings(
|
val materializer = FlowMaterializer(MaterializerSettings(
|
||||||
initialInputBufferSize = 2,
|
initialInputBufferSize = 2,
|
||||||
|
|
@ -18,13 +18,13 @@ class FlowTeeSpec extends AkkaSpec {
|
||||||
maxFanOutBufferSize = 16,
|
maxFanOutBufferSize = 16,
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
dispatcher = "akka.test.stream-dispatcher"))
|
||||||
|
|
||||||
"A Tee" must {
|
"A broadcast" must {
|
||||||
|
|
||||||
"tee to other subscriber" in {
|
"broadcast to other subscriber" in {
|
||||||
val c1 = StreamTestKit.SubscriberProbe[Int]()
|
val c1 = StreamTestKit.SubscriberProbe[Int]()
|
||||||
val c2 = StreamTestKit.SubscriberProbe[Int]()
|
val c2 = StreamTestKit.SubscriberProbe[Int]()
|
||||||
val p = Flow(List(1, 2, 3)).
|
val p = Flow(List(1, 2, 3)).
|
||||||
tee(c2).
|
broadcast(c2).
|
||||||
toPublisher(materializer)
|
toPublisher(materializer)
|
||||||
p.subscribe(c1)
|
p.subscribe(c1)
|
||||||
val sub1 = c1.expectSubscription()
|
val sub1 = c1.expectSubscription()
|
||||||
|
|
@ -49,7 +49,7 @@ class FlowTeeSpec extends AkkaSpec {
|
||||||
val c1 = StreamTestKit.SubscriberProbe[Int]()
|
val c1 = StreamTestKit.SubscriberProbe[Int]()
|
||||||
val c2 = StreamTestKit.SubscriberProbe[Int]()
|
val c2 = StreamTestKit.SubscriberProbe[Int]()
|
||||||
val p = Flow(List(1, 2, 3)).
|
val p = Flow(List(1, 2, 3)).
|
||||||
tee(c2).
|
broadcast(c2).
|
||||||
toPublisher(materializer)
|
toPublisher(materializer)
|
||||||
p.subscribe(c1)
|
p.subscribe(c1)
|
||||||
val sub1 = c1.expectSubscription()
|
val sub1 = c1.expectSubscription()
|
||||||
|
|
@ -66,7 +66,7 @@ class FlowTeeSpec extends AkkaSpec {
|
||||||
val c1 = StreamTestKit.SubscriberProbe[Int]()
|
val c1 = StreamTestKit.SubscriberProbe[Int]()
|
||||||
val c2 = StreamTestKit.SubscriberProbe[Int]()
|
val c2 = StreamTestKit.SubscriberProbe[Int]()
|
||||||
val p = Flow(List(1, 2, 3)).
|
val p = Flow(List(1, 2, 3)).
|
||||||
tee(c1).
|
broadcast(c1).
|
||||||
toPublisher(materializer)
|
toPublisher(materializer)
|
||||||
p.subscribe(c2)
|
p.subscribe(c2)
|
||||||
val sub1 = c1.expectSubscription()
|
val sub1 = c1.expectSubscription()
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue