Rename OperationAttributes to Attributes
This commit is contained in:
parent
e6f2db127c
commit
dc7269e620
92 changed files with 458 additions and 462 deletions
|
|
@ -9,7 +9,7 @@ import akka.stream.testkit.AkkaSpec
|
|||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.control.NoStackTrace
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.Attributes
|
||||
|
||||
object FlexiDocSpec {
|
||||
//#fleximerge-zip-states
|
||||
|
|
@ -34,7 +34,7 @@ class FlexiDocSpec extends AkkaSpec {
|
|||
"implement zip using readall" in {
|
||||
//#fleximerge-zip-readall
|
||||
class Zip[A, B] extends FlexiMerge[(A, B), ZipPorts[A, B]](
|
||||
new ZipPorts, OperationAttributes.name("Zip1State")) {
|
||||
new ZipPorts, Attributes.name("Zip1State")) {
|
||||
import FlexiMerge._
|
||||
override def createMergeLogic(p: PortT) = new MergeLogic[(A, B)] {
|
||||
override def initialState =
|
||||
|
|
@ -73,7 +73,7 @@ class FlexiDocSpec extends AkkaSpec {
|
|||
"implement zip using two states" in {
|
||||
//#fleximerge-zip-states
|
||||
class Zip[A, B] extends FlexiMerge[(A, B), ZipPorts[A, B]](
|
||||
new ZipPorts, OperationAttributes.name("Zip2State")) {
|
||||
new ZipPorts, Attributes.name("Zip2State")) {
|
||||
import FlexiMerge._
|
||||
|
||||
override def createMergeLogic(p: PortT) = new MergeLogic[(A, B)] {
|
||||
|
|
@ -122,7 +122,7 @@ class FlexiDocSpec extends AkkaSpec {
|
|||
new ImportantWithBackupShape(i)
|
||||
}
|
||||
class ImportantWithBackups[A] extends FlexiMerge[A, ImportantWithBackupShape[A]](
|
||||
new ImportantWithBackupShape, OperationAttributes.name("ImportantWithBackups")) {
|
||||
new ImportantWithBackupShape, Attributes.name("ImportantWithBackups")) {
|
||||
import FlexiMerge._
|
||||
|
||||
override def createMergeLogic(p: PortT) = new MergeLogic[A] {
|
||||
|
|
@ -192,7 +192,7 @@ class FlexiDocSpec extends AkkaSpec {
|
|||
//#flexi-preferring-merge
|
||||
|
||||
class PreferringMerge extends FlexiMerge[Int, PreferringMergeShape[Int]](
|
||||
new PreferringMergeShape, OperationAttributes.name("ImportantWithBackups")) {
|
||||
new PreferringMergeShape, Attributes.name("ImportantWithBackups")) {
|
||||
import akka.stream.scaladsl.FlexiMerge._
|
||||
|
||||
override def createMergeLogic(p: PortT) = new MergeLogic[Int] {
|
||||
|
|
@ -217,7 +217,7 @@ class FlexiDocSpec extends AkkaSpec {
|
|||
protected override def construct(i: Init[(A, B)]) = new UnzipShape(i)
|
||||
}
|
||||
class Unzip[A, B] extends FlexiRoute[(A, B), UnzipShape[A, B]](
|
||||
new UnzipShape, OperationAttributes.name("Unzip")) {
|
||||
new UnzipShape, Attributes.name("Unzip")) {
|
||||
import FlexiRoute._
|
||||
|
||||
override def createRouteLogic(p: PortT) = new RouteLogic[(A, B)] {
|
||||
|
|
@ -246,7 +246,7 @@ class FlexiDocSpec extends AkkaSpec {
|
|||
protected override def construct(i: Init[A]) = new ImportantRouteShape(i)
|
||||
}
|
||||
class ImportantRoute[A] extends FlexiRoute[A, ImportantRouteShape[A]](
|
||||
new ImportantRouteShape, OperationAttributes.name("ImportantRoute")) {
|
||||
new ImportantRouteShape, Attributes.name("ImportantRoute")) {
|
||||
import FlexiRoute._
|
||||
override def createRouteLogic(p: PortT) = new RouteLogic[A] {
|
||||
import p.important
|
||||
|
|
|
|||
|
|
@ -9,8 +9,8 @@ import akka.stream.ActorFlowMaterializerSettings
|
|||
import akka.stream.Supervision
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.ActorOperationAttributes
|
||||
import akka.stream.Attributes
|
||||
import akka.stream.ActorAttributes
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class FlowErrorDocSpec extends AkkaSpec {
|
||||
|
|
@ -55,7 +55,7 @@ class FlowErrorDocSpec extends AkkaSpec {
|
|||
}
|
||||
val flow = Flow[Int]
|
||||
.filter(100 / _ < 50).map(elem => 100 / (5 - elem))
|
||||
.withAttributes(ActorOperationAttributes.supervisionStrategy(decider))
|
||||
.withAttributes(ActorAttributes.supervisionStrategy(decider))
|
||||
val source = Source(0 to 5).via(flow)
|
||||
|
||||
val result = source.runWith(Sink.fold(0)(_ + _))
|
||||
|
|
@ -78,7 +78,7 @@ class FlowErrorDocSpec extends AkkaSpec {
|
|||
if (elem < 0) throw new IllegalArgumentException("negative not allowed")
|
||||
else acc + elem
|
||||
}
|
||||
.withAttributes(ActorOperationAttributes.supervisionStrategy(decider))
|
||||
.withAttributes(ActorAttributes.supervisionStrategy(decider))
|
||||
val source = Source(List(1, 3, -1, 5, 7)).via(flow)
|
||||
val result = source.grouped(1000).runWith(Sink.head)
|
||||
// the negative element cause the scan stage to be restarted,
|
||||
|
|
|
|||
|
|
@ -10,7 +10,7 @@ import akka.stream.testkit.AkkaSpec
|
|||
import scala.collection.immutable
|
||||
import scala.concurrent.{ Future, Await }
|
||||
import scala.concurrent.duration._
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.Attributes
|
||||
|
||||
class FlowGraphDocSpec extends AkkaSpec {
|
||||
|
||||
|
|
|
|||
|
|
@ -15,8 +15,8 @@ import akka.actor.Actor
|
|||
import akka.actor.Props
|
||||
import akka.pattern.ask
|
||||
import akka.util.Timeout
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.ActorOperationAttributes
|
||||
import akka.stream.Attributes
|
||||
import akka.stream.ActorAttributes
|
||||
import scala.concurrent.ExecutionContext
|
||||
import akka.stream.ActorFlowMaterializerSettings
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
|
@ -172,7 +172,7 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
|
|||
tweets.filter(_.hashtags.contains(akka)).map(_.author)
|
||||
|
||||
//#email-addresses-mapAsync-supervision
|
||||
import ActorOperationAttributes.supervisionStrategy
|
||||
import ActorAttributes.supervisionStrategy
|
||||
import Supervision.resumingDecider
|
||||
|
||||
val emailAddresses: Source[String, Unit] =
|
||||
|
|
@ -270,7 +270,7 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
|
|||
.map { phoneNo =>
|
||||
smsServer.send(TextMessage(to = phoneNo, body = "I like your tweet"))
|
||||
}
|
||||
.withAttributes(ActorOperationAttributes.dispatcher("blocking-dispatcher"))
|
||||
.withAttributes(ActorAttributes.dispatcher("blocking-dispatcher"))
|
||||
val sendTextMessages: RunnableFlow[Unit] =
|
||||
phoneNumbers.via(send).to(Sink.ignore)
|
||||
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@ package docs.stream
|
|||
import akka.stream.{ OverflowStrategy, ActorFlowMaterializerSettings, ActorFlowMaterializer }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.Attributes
|
||||
|
||||
class StreamBuffersRateSpec extends AkkaSpec {
|
||||
implicit val mat = ActorFlowMaterializer()
|
||||
|
|
@ -30,7 +30,7 @@ class StreamBuffersRateSpec extends AkkaSpec {
|
|||
|
||||
//#section-buffer
|
||||
val section = Flow[Int].map(_ * 2)
|
||||
.withAttributes(OperationAttributes.inputBuffer(initial = 1, max = 1))
|
||||
.withAttributes(Attributes.inputBuffer(initial = 1, max = 1))
|
||||
val flow = section.via(Flow[Int].map(_ / 2)) // the buffer size of this map is the default
|
||||
//#section-buffer
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,7 +1,7 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.event.Logging
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.Attributes
|
||||
import akka.stream.scaladsl.{ Sink, Source }
|
||||
import akka.testkit.{ EventFilter, TestProbe }
|
||||
|
||||
|
|
@ -30,7 +30,7 @@ class RecipeLoggingElements extends RecipeSpec {
|
|||
//#log-custom
|
||||
// customise log levels
|
||||
mySource.log("before-map")
|
||||
.withAttributes(OperationAttributes.logLevels(onElement = Logging.WarningLevel))
|
||||
.withAttributes(Attributes.logLevels(onElement = Logging.WarningLevel))
|
||||
.map(analyse)
|
||||
|
||||
// or provide custom logging adapter
|
||||
|
|
|
|||
|
|
@ -56,7 +56,7 @@ class StreamFileDocSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
"configure dispatcher in code" in {
|
||||
//#custom-dispatcher-code
|
||||
SynchronousFileSink(file)
|
||||
.withAttributes(ActorOperationAttributes.dispatcher("custom-file-io-dispatcher"))
|
||||
.withAttributes(ActorAttributes.dispatcher("custom-file-io-dispatcher"))
|
||||
//#custom-dispatcher-code
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -109,7 +109,7 @@ private[http] object OutgoingConnectionBlueprint {
|
|||
// a simple merge stage that simply forwards its first input and ignores its second input
|
||||
// (the terminationBackchannelInput), but applies a special completion handling
|
||||
class TerminationMerge
|
||||
extends FlexiMerge[HttpRequest, FanInShape2[HttpRequest, HttpResponse, HttpRequest]](new FanInShape2("TerminationMerge"), OperationAttributes.name("TerminationMerge")) {
|
||||
extends FlexiMerge[HttpRequest, FanInShape2[HttpRequest, HttpResponse, HttpRequest]](new FanInShape2("TerminationMerge"), Attributes.name("TerminationMerge")) {
|
||||
import FlexiMerge._
|
||||
|
||||
def createMergeLogic(p: PortT) = new MergeLogic[HttpRequest] {
|
||||
|
|
@ -142,7 +142,7 @@ private[http] object OutgoingConnectionBlueprint {
|
|||
* 3. Go back to 1.
|
||||
*/
|
||||
class ResponseParsingMerge(rootParser: HttpResponseParser)
|
||||
extends FlexiMerge[List[ResponseOutput], FanInShape2[ByteString, HttpMethod, List[ResponseOutput]]](new FanInShape2("ResponseParsingMerge"), OperationAttributes.name("ResponsePersingMerge")) {
|
||||
extends FlexiMerge[List[ResponseOutput], FanInShape2[ByteString, HttpMethod, List[ResponseOutput]]](new FanInShape2("ResponseParsingMerge"), Attributes.name("ResponsePersingMerge")) {
|
||||
import FlexiMerge._
|
||||
|
||||
def createMergeLogic(p: PortT) = new MergeLogic[List[ResponseOutput]] {
|
||||
|
|
|
|||
|
|
@ -108,7 +108,7 @@ private object PoolConductor {
|
|||
|
||||
private class SlotSelector(slotCount: Int, maxRetries: Int, pipeliningLimit: Int, log: LoggingAdapter)
|
||||
extends FlexiMerge[SwitchCommand, FanInShape2[RequestContext, SlotEvent, SwitchCommand]](
|
||||
new FanInShape2("PoolConductor.SlotSelector"), OperationAttributes.name("PoolConductor.SlotSelector")) {
|
||||
new FanInShape2("PoolConductor.SlotSelector"), Attributes.name("PoolConductor.SlotSelector")) {
|
||||
import FlexiMerge._
|
||||
|
||||
def createMergeLogic(s: FanInShape2[RequestContext, SlotEvent, SwitchCommand]): MergeLogic[SwitchCommand] =
|
||||
|
|
@ -187,7 +187,7 @@ private object PoolConductor {
|
|||
}
|
||||
|
||||
private class Route(slotCount: Int) extends FlexiRoute[SwitchCommand, UniformFanOutShape[SwitchCommand, RequestContext]](
|
||||
new UniformFanOutShape(slotCount, "PoolConductor.Route"), OperationAttributes.name("PoolConductor.Route")) {
|
||||
new UniformFanOutShape(slotCount, "PoolConductor.Route"), Attributes.name("PoolConductor.Route")) {
|
||||
import FlexiRoute._
|
||||
|
||||
def createRouteLogic(s: UniformFanOutShape[SwitchCommand, RequestContext]): RouteLogic[SwitchCommand] =
|
||||
|
|
@ -208,7 +208,7 @@ private object PoolConductor {
|
|||
|
||||
// FIXME: remove when #17038 is cleared
|
||||
private class RetrySplit extends FlexiRoute[RawSlotEvent, FanOutShape2[RawSlotEvent, RawSlotEvent, RequestContext]](
|
||||
new FanOutShape2("PoolConductor.RetrySplit"), OperationAttributes.name("PoolConductor.RetrySplit")) {
|
||||
new FanOutShape2("PoolConductor.RetrySplit"), Attributes.name("PoolConductor.RetrySplit")) {
|
||||
import FlexiRoute._
|
||||
|
||||
def createRouteLogic(s: FanOutShape2[RawSlotEvent, RawSlotEvent, RequestContext]): RouteLogic[RawSlotEvent] =
|
||||
|
|
|
|||
|
|
@ -228,7 +228,7 @@ private object PoolSlot {
|
|||
|
||||
// FIXME: remove when #17038 is cleared
|
||||
private class SlotEventSplit extends FlexiRoute[ProcessorOut, FanOutShape2[ProcessorOut, ResponseContext, RawSlotEvent]](
|
||||
new FanOutShape2("PoolSlot.SlotEventSplit"), OperationAttributes.name("PoolSlot.SlotEventSplit")) {
|
||||
new FanOutShape2("PoolSlot.SlotEventSplit"), Attributes.name("PoolSlot.SlotEventSplit")) {
|
||||
import FlexiRoute._
|
||||
|
||||
def createRouteLogic(s: FanOutShape2[ProcessorOut, ResponseContext, RawSlotEvent]): RouteLogic[ProcessorOut] =
|
||||
|
|
|
|||
|
|
@ -158,7 +158,7 @@ private[http] object HttpServerBluePrint {
|
|||
}
|
||||
|
||||
class BypassMerge(settings: ServerSettings, log: LoggingAdapter)
|
||||
extends FlexiMerge[ResponseRenderingContext, FanInShape3[RequestOutput, OneHundredContinue.type, HttpResponse, ResponseRenderingContext]](new FanInShape3("BypassMerge"), OperationAttributes.name("BypassMerge")) {
|
||||
extends FlexiMerge[ResponseRenderingContext, FanInShape3[RequestOutput, OneHundredContinue.type, HttpResponse, ResponseRenderingContext]](new FanInShape3("BypassMerge"), Attributes.name("BypassMerge")) {
|
||||
import FlexiMerge._
|
||||
|
||||
def createMergeLogic(p: PortT) = new MergeLogic[ResponseRenderingContext] {
|
||||
|
|
@ -317,7 +317,7 @@ private[http] object HttpServerBluePrint {
|
|||
}
|
||||
}
|
||||
class WebsocketSwitchRouter
|
||||
extends FlexiRoute[AnyRef, FanOutShape2[AnyRef, ByteString, ByteString]](new FanOutShape2("websocketSplit"), OperationAttributes.name("websocketSplit")) {
|
||||
extends FlexiRoute[AnyRef, FanOutShape2[AnyRef, ByteString, ByteString]](new FanOutShape2("websocketSplit"), Attributes.name("websocketSplit")) {
|
||||
|
||||
override def createRouteLogic(shape: FanOutShape2[AnyRef, ByteString, ByteString]): RouteLogic[AnyRef] =
|
||||
new RouteLogic[AnyRef] {
|
||||
|
|
@ -342,7 +342,7 @@ private[http] object HttpServerBluePrint {
|
|||
}
|
||||
}
|
||||
}
|
||||
class WebsocketMerge(installHandler: Flow[FrameEvent, FrameEvent, Any] ⇒ Unit) extends FlexiMerge[ByteString, FanInShape2[ResponseRenderingOutput, ByteString, ByteString]](new FanInShape2("websocketMerge"), OperationAttributes.name("websocketMerge")) {
|
||||
class WebsocketMerge(installHandler: Flow[FrameEvent, FrameEvent, Any] ⇒ Unit) extends FlexiMerge[ByteString, FanInShape2[ResponseRenderingOutput, ByteString, ByteString]](new FanInShape2("websocketMerge"), Attributes.name("websocketMerge")) {
|
||||
def createMergeLogic(s: FanInShape2[ResponseRenderingOutput, ByteString, ByteString]): MergeLogic[ByteString] =
|
||||
new MergeLogic[ByteString] {
|
||||
var websocketHandlerWasInstalled: Boolean = false
|
||||
|
|
@ -376,7 +376,7 @@ private[http] object HttpServerBluePrint {
|
|||
}
|
||||
}
|
||||
/** A merge for two streams that just forwards all elements and closes the connection when the first input closes. */
|
||||
class CloseIfFirstClosesMerge2[T](name: String) extends FlexiMerge[T, FanInShape2[T, T, T]](new FanInShape2(name), OperationAttributes.name(name)) {
|
||||
class CloseIfFirstClosesMerge2[T](name: String) extends FlexiMerge[T, FanInShape2[T, T, T]](new FanInShape2(name), Attributes.name(name)) {
|
||||
def createMergeLogic(s: FanInShape2[T, T, T]): MergeLogic[T] =
|
||||
new MergeLogic[T] {
|
||||
def initialState: State[T] = State[T](ReadAny(s.in0, s.in1)) {
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import java.security.SecureRandom
|
|||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import akka.stream.{ OperationAttributes, FanOutShape2, FanInShape3, Inlet }
|
||||
import akka.stream.{ Attributes, FanOutShape2, FanInShape3, Inlet }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.stage._
|
||||
import FlexiRoute.{ DemandFrom, DemandFromAny, RouteLogic }
|
||||
|
|
@ -107,7 +107,7 @@ private[http] object Websocket {
|
|||
* Distributes output from the FrameHandler into bypass and userFlow.
|
||||
*/
|
||||
object BypassRouter
|
||||
extends FlexiRoute[Either[BypassEvent, MessagePart], FanOutShape2[Either[BypassEvent, MessagePart], BypassEvent, MessagePart]](new FanOutShape2("bypassRouter"), OperationAttributes.name("bypassRouter")) {
|
||||
extends FlexiRoute[Either[BypassEvent, MessagePart], FanOutShape2[Either[BypassEvent, MessagePart], BypassEvent, MessagePart]](new FanOutShape2("bypassRouter"), Attributes.name("bypassRouter")) {
|
||||
def createRouteLogic(s: FanOutShape2[Either[BypassEvent, MessagePart], BypassEvent, MessagePart]): RouteLogic[Either[BypassEvent, MessagePart]] =
|
||||
new RouteLogic[Either[BypassEvent, MessagePart]] {
|
||||
def initialState: State[_] = State(DemandFromAny(s)) { (ctx, out, ev) ⇒
|
||||
|
|
@ -135,7 +135,7 @@ private[http] object Websocket {
|
|||
/**
|
||||
* Merges bypass, user flow and tick source for consumption in the FrameOutHandler.
|
||||
*/
|
||||
object BypassMerge extends FlexiMerge[AnyRef, FanInShape3[BypassEvent, AnyRef, Tick.type, AnyRef]](new FanInShape3("bypassMerge"), OperationAttributes.name("bypassMerge")) {
|
||||
object BypassMerge extends FlexiMerge[AnyRef, FanInShape3[BypassEvent, AnyRef, Tick.type, AnyRef]](new FanInShape3("bypassMerge"), Attributes.name("bypassMerge")) {
|
||||
def createMergeLogic(s: FanInShape3[BypassEvent, AnyRef, Tick.type, AnyRef]): MergeLogic[AnyRef] =
|
||||
new MergeLogic[AnyRef] {
|
||||
def initialState: State[_] = Idle
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ import akka.stream.stage._
|
|||
* INTERNAL API
|
||||
*/
|
||||
private[http] object StreamUtils {
|
||||
import OperationAttributes.none
|
||||
import Attributes.none
|
||||
|
||||
/**
|
||||
* Creates a transformer that will call `f` for each incoming ByteString and output its result. After the complete
|
||||
|
|
@ -202,7 +202,7 @@ private[http] object StreamUtils {
|
|||
} else ByteString.empty
|
||||
}
|
||||
|
||||
Source(() ⇒ iterator).withAttributes(ActorOperationAttributes.dispatcher(fileIODispatcher))
|
||||
Source(() ⇒ iterator).withAttributes(ActorAttributes.dispatcher(fileIODispatcher))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -223,7 +223,7 @@ private[http] object StreamUtils {
|
|||
new Source[Out, Subscriber[Out]](new OneTimeSubscriberSource(none, SourceShape(new Outlet(name)), cell))
|
||||
|
||||
/** A copy of PublisherSink that allows access to the publisher through the cell but can only materialized once */
|
||||
private class OneTimePublisherSink[In](attributes: OperationAttributes, shape: SinkShape[In], cell: OneTimeWriteCell[Publisher[In]])
|
||||
private class OneTimePublisherSink[In](attributes: Attributes, shape: SinkShape[In], cell: OneTimeWriteCell[Publisher[In]])
|
||||
extends PublisherSink[In](attributes, shape) {
|
||||
override def create(context: MaterializationContext): (Subscriber[In], Publisher[In]) = {
|
||||
val results = super.create(context)
|
||||
|
|
@ -233,11 +233,11 @@ private[http] object StreamUtils {
|
|||
override protected def newInstance(shape: SinkShape[In]): SinkModule[In, Publisher[In]] =
|
||||
new OneTimePublisherSink[In](attributes, shape, cell)
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Module =
|
||||
override def withAttributes(attr: Attributes): Module =
|
||||
new OneTimePublisherSink[In](attr, amendShape(attr), cell)
|
||||
}
|
||||
/** A copy of SubscriberSource that allows access to the subscriber through the cell but can only materialized once */
|
||||
private class OneTimeSubscriberSource[Out](val attributes: OperationAttributes, shape: SourceShape[Out], cell: OneTimeWriteCell[Subscriber[Out]])
|
||||
private class OneTimeSubscriberSource[Out](val attributes: Attributes, shape: SourceShape[Out], cell: OneTimeWriteCell[Subscriber[Out]])
|
||||
extends SourceModule[Out, Subscriber[Out]](shape) {
|
||||
|
||||
override def create(context: MaterializationContext): (Publisher[Out], Subscriber[Out]) = {
|
||||
|
|
@ -258,7 +258,7 @@ private[http] object StreamUtils {
|
|||
|
||||
override protected def newInstance(shape: SourceShape[Out]): SourceModule[Out, Subscriber[Out]] =
|
||||
new OneTimeSubscriberSource[Out](attributes, shape, cell)
|
||||
override def withAttributes(attr: OperationAttributes): Module =
|
||||
override def withAttributes(attr: Attributes): Module =
|
||||
new OneTimeSubscriberSource[Out](attr, amendShape(attr), cell)
|
||||
}
|
||||
|
||||
|
|
@ -279,7 +279,7 @@ private[http] object StreamUtils {
|
|||
}
|
||||
|
||||
/** A merge for two streams that just forwards all elements and closes the connection eagerly. */
|
||||
class EagerCloseMerge2[T](name: String) extends FlexiMerge[T, FanInShape2[T, T, T]](new FanInShape2(name), OperationAttributes.name(name)) {
|
||||
class EagerCloseMerge2[T](name: String) extends FlexiMerge[T, FanInShape2[T, T, T]](new FanInShape2(name), Attributes.name(name)) {
|
||||
def createMergeLogic(s: FanInShape2[T, T, T]): MergeLogic[T] =
|
||||
new MergeLogic[T] {
|
||||
def initialState: State[T] = State[T](ReadAny(s.in0, s.in1)) {
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings }
|
|||
import akka.stream.impl.Stages.Identity
|
||||
import akka.stream.scaladsl.Flow
|
||||
import org.reactivestreams.Processor
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.Attributes
|
||||
|
||||
class FusableProcessorTest extends AkkaIdentityProcessorVerification[Int] {
|
||||
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.stream.tck
|
|||
import akka.stream.ActorFlowMaterializer
|
||||
import akka.stream.scaladsl.Flow
|
||||
import org.reactivestreams.Processor
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.Attributes
|
||||
|
||||
class MapTest extends AkkaIdentityProcessorVerification[Int] {
|
||||
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings }
|
|||
import akka.stream.impl.ActorFlowMaterializerImpl
|
||||
import akka.stream.impl.Stages.Identity
|
||||
import akka.stream.scaladsl.Flow
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.Attributes
|
||||
import akka.stream.stage.{ Context, PushStage }
|
||||
import org.reactivestreams.{ Processor, Publisher }
|
||||
|
||||
|
|
|
|||
|
|
@ -406,22 +406,22 @@ private[testkit] object StreamTestKit {
|
|||
def sendOnSubscribe(): Unit = subscriber.onSubscribe(this)
|
||||
}
|
||||
|
||||
final class ProbeSource[T](val attributes: OperationAttributes, shape: SourceShape[T])(implicit system: ActorSystem) extends SourceModule[T, TestPublisher.Probe[T]](shape) {
|
||||
final class ProbeSource[T](val attributes: Attributes, shape: SourceShape[T])(implicit system: ActorSystem) extends SourceModule[T, TestPublisher.Probe[T]](shape) {
|
||||
override def create(context: MaterializationContext) = {
|
||||
val probe = TestPublisher.probe[T]()
|
||||
(probe, probe)
|
||||
}
|
||||
override protected def newInstance(shape: SourceShape[T]): SourceModule[T, TestPublisher.Probe[T]] = new ProbeSource[T](attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module = new ProbeSource[T](attr, amendShape(attr))
|
||||
override def withAttributes(attr: Attributes): Module = new ProbeSource[T](attr, amendShape(attr))
|
||||
}
|
||||
|
||||
final class ProbeSink[T](val attributes: OperationAttributes, shape: SinkShape[T])(implicit system: ActorSystem) extends SinkModule[T, TestSubscriber.Probe[T]](shape) {
|
||||
final class ProbeSink[T](val attributes: Attributes, shape: SinkShape[T])(implicit system: ActorSystem) extends SinkModule[T, TestSubscriber.Probe[T]](shape) {
|
||||
override def create(context: MaterializationContext) = {
|
||||
val probe = TestSubscriber.probe[T]()
|
||||
(probe, probe)
|
||||
}
|
||||
override protected def newInstance(shape: SinkShape[T]): SinkModule[T, TestSubscriber.Probe[T]] = new ProbeSink[T](attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module = new ProbeSink[T](attr, amendShape(attr))
|
||||
override def withAttributes(attr: Attributes): Module = new ProbeSink[T](attr, amendShape(attr))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@
|
|||
package akka.stream.testkit.scaladsl
|
||||
|
||||
import akka.actor.ActorSystem
|
||||
import akka.stream.OperationAttributes.none
|
||||
import akka.stream.Attributes.none
|
||||
import akka.stream._
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit._
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@ package akka.stream.testkit.scaladsl
|
|||
|
||||
import akka.stream._
|
||||
import akka.stream.impl._
|
||||
import akka.stream.OperationAttributes.none
|
||||
import akka.stream.Attributes.none
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit._
|
||||
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@ import akka.stream.ActorFlowMaterializerSettings
|
|||
import akka.stream.scaladsl._
|
||||
import org.reactivestreams.Publisher
|
||||
import akka.stream.ActorFlowMaterializer
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.Attributes
|
||||
|
||||
class ChainSetup[In, Out](
|
||||
stream: Flow[In, In, _] ⇒ Flow[In, Out, _],
|
||||
|
|
|
|||
|
|
@ -5,9 +5,7 @@
|
|||
package akka.stream;
|
||||
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.event.Logging;
|
||||
import akka.stream.javadsl.AkkaJUnitActorSystemResource;
|
||||
import akka.stream.OperationAttributes;
|
||||
|
||||
public abstract class StreamTest {
|
||||
final protected ActorSystem system;
|
||||
|
|
|
|||
|
|
@ -0,0 +1,36 @@
|
|||
/**
|
||||
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.stream.javadsl;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import java.util.Arrays;
|
||||
import org.junit.Test;
|
||||
|
||||
import akka.stream.Attributes;
|
||||
|
||||
public class AttributesTest {
|
||||
|
||||
final Attributes attributes =
|
||||
Attributes.name("a")
|
||||
.and(Attributes.name("b"))
|
||||
.and(Attributes.inputBuffer(1, 2));
|
||||
|
||||
@Test
|
||||
public void mustGetAttributesByClass() {
|
||||
assertEquals(
|
||||
Arrays.asList(new Attributes.Name("a"), new Attributes.Name("b")),
|
||||
attributes.getAttributeList(Attributes.Name.class));
|
||||
assertEquals(
|
||||
Arrays.asList(new Attributes.InputBuffer(1, 2)),
|
||||
attributes.getAttributeList(Attributes.InputBuffer.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustGetAttributeByClass() {
|
||||
assertEquals(
|
||||
new Attributes.Name("a"),
|
||||
attributes.getAttribute(Attributes.Name.class, new Attributes.Name("default")));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -147,7 +147,7 @@ public class FlexiMergeTest {
|
|||
*/
|
||||
static public class Fair<T> extends FlexiMerge<T, T, UniformFanInShape<T, T>> {
|
||||
public Fair() {
|
||||
super(new UniformFanInShape<T, T>(2), OperationAttributes.name("Fair"));
|
||||
super(new UniformFanInShape<T, T>(2), Attributes.name("Fair"));
|
||||
}
|
||||
@Override
|
||||
public MergeLogic<T, T> createMergeLogic(final UniformFanInShape<T, T> s) {
|
||||
|
|
@ -173,7 +173,7 @@ public class FlexiMergeTest {
|
|||
*/
|
||||
static public class StrictRoundRobin<T> extends FlexiMerge<T, T, UniformFanInShape<T, T>> {
|
||||
public StrictRoundRobin() {
|
||||
super(new UniformFanInShape<T, T>(2), OperationAttributes.name("StrictRoundRobin"));
|
||||
super(new UniformFanInShape<T, T>(2), Attributes.name("StrictRoundRobin"));
|
||||
}
|
||||
@Override
|
||||
public MergeLogic<T, T> createMergeLogic(final UniformFanInShape<T, T> s) {
|
||||
|
|
@ -240,7 +240,7 @@ public class FlexiMergeTest {
|
|||
|
||||
static public class Zip<A, B> extends FlexiMerge<A, Pair<A, B>, FanInShape2<A, B, Pair<A, B>>> {
|
||||
public Zip() {
|
||||
super(new FanInShape2<A, B, Pair<A, B>>("Zip"), OperationAttributes.name("Zip"));
|
||||
super(new FanInShape2<A, B, Pair<A, B>>("Zip"), Attributes.name("Zip"));
|
||||
}
|
||||
@Override
|
||||
public MergeLogic<A, Pair<A, B>> createMergeLogic(final FanInShape2<A, B, Pair<A, B>> s) {
|
||||
|
|
@ -328,7 +328,7 @@ public class FlexiMergeTest {
|
|||
|
||||
static public class TripleZip<A, B, C> extends FlexiMerge<FlexiMerge.ReadAllInputs, Triple<A, B, C>, FanInShape3<A, B, C, Triple<A, B, C>>> {
|
||||
public TripleZip() {
|
||||
super(new FanInShape3<A, B, C, Triple<A, B, C>>("TripleZip"), OperationAttributes.name("TripleZip"));
|
||||
super(new FanInShape3<A, B, C, Triple<A, B, C>>("TripleZip"), Attributes.name("TripleZip"));
|
||||
}
|
||||
@Override
|
||||
public MergeLogic<ReadAllInputs, Triple<A, B, C>> createMergeLogic(final FanInShape3<A, B, C, Triple<A, B, C>> s) {
|
||||
|
|
|
|||
|
|
@ -139,7 +139,7 @@ public class FlexiRouteTest {
|
|||
*/
|
||||
static public class Fair<T> extends FlexiRoute<T, UniformFanOutShape<T, T>> {
|
||||
public Fair() {
|
||||
super(new UniformFanOutShape<T, T>(2), OperationAttributes.name("Fair"));
|
||||
super(new UniformFanOutShape<T, T>(2), Attributes.name("Fair"));
|
||||
}
|
||||
@Override
|
||||
public RouteLogic<T> createRouteLogic(final UniformFanOutShape<T, T> s) {
|
||||
|
|
@ -175,7 +175,7 @@ public class FlexiRouteTest {
|
|||
*/
|
||||
static public class StrictRoundRobin<T> extends FlexiRoute<T, UniformFanOutShape<T, T>> {
|
||||
public StrictRoundRobin() {
|
||||
super(new UniformFanOutShape<T, T>(2), OperationAttributes.name("StrictRoundRobin"));
|
||||
super(new UniformFanOutShape<T, T>(2), Attributes.name("StrictRoundRobin"));
|
||||
}
|
||||
@Override
|
||||
public RouteLogic<T> createRouteLogic(final UniformFanOutShape<T, T> s) {
|
||||
|
|
@ -207,7 +207,7 @@ public class FlexiRouteTest {
|
|||
|
||||
static public class Unzip<A, B> extends FlexiRoute<Pair<A, B>, FanOutShape2<Pair<A, B>, A, B>> {
|
||||
public Unzip() {
|
||||
super(new FanOutShape2<Pair<A, B>, A, B>("Unzip"), OperationAttributes.name("Unzip"));
|
||||
super(new FanOutShape2<Pair<A, B>, A, B>("Unzip"), Attributes.name("Unzip"));
|
||||
}
|
||||
@Override
|
||||
public RouteLogic<Pair<A, B>> createRouteLogic(final FanOutShape2<Pair<A, B>, A, B> s) {
|
||||
|
|
|
|||
|
|
@ -1,36 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.stream.javadsl;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import java.util.Arrays;
|
||||
import org.junit.Test;
|
||||
|
||||
import akka.stream.OperationAttributes;
|
||||
|
||||
public class OperationAttributesTest {
|
||||
|
||||
final OperationAttributes attributes =
|
||||
OperationAttributes.name("a")
|
||||
.and(OperationAttributes.name("b"))
|
||||
.and(OperationAttributes.inputBuffer(1, 2));
|
||||
|
||||
@Test
|
||||
public void mustGetAttributesByClass() {
|
||||
assertEquals(
|
||||
Arrays.asList(new OperationAttributes.Name("a"), new OperationAttributes.Name("b")),
|
||||
attributes.getAttributes(OperationAttributes.Name.class));
|
||||
assertEquals(
|
||||
Arrays.asList(new OperationAttributes.InputBuffer(1, 2)),
|
||||
attributes.getAttributes(OperationAttributes.InputBuffer.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustGetAttributeByClass() {
|
||||
assertEquals(
|
||||
new OperationAttributes.Name("a"),
|
||||
attributes.getAttribute(OperationAttributes.Name.class, new OperationAttributes.Name("default")));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -4,7 +4,7 @@
|
|||
package akka.stream.actor
|
||||
|
||||
import akka.actor.{ ActorRef, PoisonPill, Props }
|
||||
import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings, ActorOperationAttributes }
|
||||
import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings, ActorAttributes }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.testkit.Utils._
|
||||
|
|
@ -428,7 +428,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
|
|||
implicit val materializer = ActorFlowMaterializer()
|
||||
val s = TestSubscriber.manualProbe[String]()
|
||||
val ref = Source.actorPublisher(testPublisherProps(testActor, useTestDispatcher = false))
|
||||
.withAttributes(ActorOperationAttributes.dispatcher("my-dispatcher1"))
|
||||
.withAttributes(ActorAttributes.dispatcher("my-dispatcher1"))
|
||||
.to(Sink(s)).run()
|
||||
ref ! ThreadName
|
||||
expectMsgType[String] should include("my-dispatcher1")
|
||||
|
|
@ -438,7 +438,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
|
|||
implicit val materializer = ActorFlowMaterializer()
|
||||
val s = TestSubscriber.manualProbe[String]()
|
||||
val ref = Source.actorPublisher(testPublisherProps(testActor, useTestDispatcher = false).withDispatcher("my-dispatcher1"))
|
||||
.withAttributes(ActorOperationAttributes.dispatcher("my-dispatcher2"))
|
||||
.withAttributes(ActorAttributes.dispatcher("my-dispatcher2"))
|
||||
.to(Sink(s)).run()
|
||||
ref ! ThreadName
|
||||
expectMsgType[String] should include("my-dispatcher1")
|
||||
|
|
|
|||
|
|
@ -19,8 +19,8 @@ class StreamLayoutSpec extends AkkaSpec {
|
|||
|
||||
override def carbonCopy: Module = ???
|
||||
|
||||
override def attributes: OperationAttributes = OperationAttributes.none
|
||||
override def withAttributes(attributes: OperationAttributes): Module = this
|
||||
override def attributes: Attributes = Attributes.none
|
||||
override def withAttributes(attributes: Attributes): Module = this
|
||||
}
|
||||
|
||||
def testStage(): Module = testAtomic(1, 1)
|
||||
|
|
@ -174,7 +174,7 @@ class StreamLayoutSpec extends AkkaSpec {
|
|||
var publishers = Vector.empty[TestPublisher]
|
||||
var subscribers = Vector.empty[TestSubscriber]
|
||||
|
||||
override protected def materializeAtomic(atomic: Module, effectiveAttributes: OperationAttributes): Unit = {
|
||||
override protected def materializeAtomic(atomic: Module, effectiveAttributes: Attributes): Unit = {
|
||||
for (inPort ← atomic.inPorts) {
|
||||
val subscriber = TestSubscriber(atomic, inPort)
|
||||
subscribers :+= subscriber
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.stream.impl.fusing
|
|||
import akka.event.Logging
|
||||
import akka.stream.stage._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.{ ActorFlowMaterializer, OperationAttributes }
|
||||
import akka.stream.{ ActorFlowMaterializer, Attributes }
|
||||
import akka.testkit.TestProbe
|
||||
|
||||
trait InterpreterLifecycleSpecKit {
|
||||
|
|
@ -116,7 +116,7 @@ trait InterpreterSpecKit extends AkkaSpec with InterpreterLifecycleSpecKit {
|
|||
(op, ctx, event) ⇒ sidechannel.ref ! ActorInterpreter.AsyncInput(op, ctx, event),
|
||||
Logging(system, classOf[TestSetup]),
|
||||
ActorFlowMaterializer(),
|
||||
OperationAttributes.none,
|
||||
Attributes.none,
|
||||
forkLimit, overflowToHeap)
|
||||
interpreter.init()
|
||||
|
||||
|
|
|
|||
|
|
@ -15,7 +15,7 @@ import akka.stream.testkit.Utils._
|
|||
import akka.stream.testkit.StreamTestKit
|
||||
import akka.stream.ActorFlowMaterializer
|
||||
import akka.stream.ActorFlowMaterializerSettings
|
||||
import akka.stream.ActorOperationAttributes
|
||||
import akka.stream.ActorAttributes
|
||||
import akka.util.ByteString
|
||||
import akka.util.Timeout
|
||||
|
||||
|
|
@ -116,7 +116,7 @@ class SynchronousFileSinkSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
try {
|
||||
Source(() ⇒ Iterator.continually(TestByteStrings.head))
|
||||
.to(SynchronousFileSink(f))
|
||||
.withAttributes(ActorOperationAttributes.dispatcher("akka.actor.default-dispatcher"))
|
||||
.withAttributes(ActorAttributes.dispatcher("akka.actor.default-dispatcher"))
|
||||
.run()(mat)
|
||||
|
||||
mat.asInstanceOf[ActorFlowMaterializerImpl].supervisor.tell(StreamSupervisor.GetChildren, testActor)
|
||||
|
|
|
|||
|
|
@ -10,8 +10,8 @@ import java.util.Random
|
|||
import akka.actor.ActorSystem
|
||||
import akka.stream.ActorFlowMaterializer
|
||||
import akka.stream.ActorFlowMaterializerSettings
|
||||
import akka.stream.ActorOperationAttributes
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.ActorAttributes
|
||||
import akka.stream.Attributes
|
||||
import akka.stream.impl.ActorFlowMaterializerImpl
|
||||
import akka.stream.impl.StreamSupervisor
|
||||
import akka.stream.impl.StreamSupervisor.Children
|
||||
|
|
@ -72,7 +72,7 @@ class SynchronousFileSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
"File Source" must {
|
||||
"read contents from a file" in assertAllStagesStopped {
|
||||
val chunkSize = 512
|
||||
val bufferAttributes = OperationAttributes.inputBuffer(1, 2)
|
||||
val bufferAttributes = Attributes.inputBuffer(1, 2)
|
||||
|
||||
val p = SynchronousFileSource(testFile, chunkSize)
|
||||
.withAttributes(bufferAttributes)
|
||||
|
|
@ -107,7 +107,7 @@ class SynchronousFileSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
|
||||
"complete only when all contents of a file have been signalled" in assertAllStagesStopped {
|
||||
val chunkSize = 256
|
||||
val bufferAttributes = OperationAttributes.inputBuffer(4, 8)
|
||||
val bufferAttributes = Attributes.inputBuffer(4, 8)
|
||||
|
||||
val demandAllButOneChunks = TestText.length / chunkSize - 1
|
||||
|
||||
|
|
@ -157,7 +157,7 @@ class SynchronousFileSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
|
||||
s"count lines in real file (chunkSize = $chunkSize, readAhead = $readAhead)" in {
|
||||
val s = SynchronousFileSource(manyLines, chunkSize = chunkSize)
|
||||
.withAttributes(OperationAttributes.inputBuffer(readAhead, readAhead))
|
||||
.withAttributes(Attributes.inputBuffer(readAhead, readAhead))
|
||||
|
||||
val f = s.runWith(Sink.fold(0) { case (acc, l) ⇒ acc + l.utf8String.count(_ == '\n') })
|
||||
|
||||
|
|
@ -187,7 +187,7 @@ class SynchronousFileSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
|
||||
try {
|
||||
val p = SynchronousFileSource(manyLines)
|
||||
.withAttributes(ActorOperationAttributes.dispatcher("akka.actor.default-dispatcher"))
|
||||
.withAttributes(ActorAttributes.dispatcher("akka.actor.default-dispatcher"))
|
||||
.runWith(TestSink.probe())(mat)
|
||||
|
||||
mat.asInstanceOf[ActorFlowMaterializerImpl].supervisor.tell(StreamSupervisor.GetChildren, testActor)
|
||||
|
|
|
|||
|
|
@ -3,16 +3,16 @@
|
|||
*/
|
||||
package akka.stream.io
|
||||
|
||||
import akka.actor.{ActorSystem, Kill}
|
||||
import akka.actor.{ ActorSystem, Kill }
|
||||
import akka.io.Tcp._
|
||||
import akka.stream.scaladsl.Tcp.IncomingConnection
|
||||
import akka.stream.scaladsl.{Flow, _}
|
||||
import akka.stream.scaladsl.{ Flow, _ }
|
||||
import akka.stream.testkit.TestUtils.temporaryServerAddress
|
||||
import scala.util.control.NonFatal
|
||||
import akka.stream.testkit.Utils._
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.{ActorFlowMaterializer, BindFailedException, StreamTcpException}
|
||||
import akka.util.{ByteString, Helpers}
|
||||
import akka.stream.{ ActorFlowMaterializer, BindFailedException, StreamTcpException }
|
||||
import akka.util.{ ByteString, Helpers }
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.Await
|
||||
|
|
@ -393,7 +393,7 @@ class TcpSpec extends AkkaSpec("akka.io.tcp.windows-connection-abort-workaround-
|
|||
a[StreamTcpException] should be thrownBy
|
||||
Await.result(result, 3.seconds)
|
||||
|
||||
binding.map(_.unbind()).recover {case NonFatal(_) => ()} foreach(_ ⇒ system2.shutdown())
|
||||
binding.map(_.unbind()).recover { case NonFatal(_) ⇒ () } foreach (_ ⇒ system2.shutdown())
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -524,8 +524,6 @@ class TcpSpec extends AkkaSpec("akka.io.tcp.windows-connection-abort-workaround-
|
|||
|
||||
}
|
||||
|
||||
|
||||
|
||||
def validateServerClientCommunication(testData: ByteString,
|
||||
serverConnection: ServerConnection,
|
||||
readProbe: TcpReadProbe,
|
||||
|
|
|
|||
|
|
@ -12,10 +12,10 @@ import akka.stream.ActorFlowMaterializer
|
|||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.Attributes
|
||||
|
||||
class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
|
||||
import OperationAttributes._
|
||||
import Attributes._
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
implicit val mat = ActorFlowMaterializer()
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.ActorOperationAttributes._
|
||||
import akka.stream.ActorAttributes._
|
||||
import akka.stream.Supervision._
|
||||
import akka.stream.testkit.Utils._
|
||||
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.OperationAttributes._
|
||||
import akka.stream.Attributes._
|
||||
import akka.stream.ActorFlowMaterializer
|
||||
import akka.stream.OverflowStrategy
|
||||
import akka.stream.testkit._
|
||||
|
|
|
|||
|
|
@ -11,8 +11,8 @@ import akka.stream.Supervision.resumingDecider
|
|||
import akka.stream.testkit._
|
||||
import akka.stream.testkit.Utils._
|
||||
import org.reactivestreams.Publisher
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.ActorOperationAttributes
|
||||
import akka.stream.Attributes
|
||||
import akka.stream.ActorAttributes
|
||||
|
||||
class FlowGroupBySpec extends AkkaSpec {
|
||||
|
||||
|
|
@ -229,7 +229,7 @@ class FlowGroupBySpec extends AkkaSpec {
|
|||
val exc = TE("test")
|
||||
val publisher = Source(publisherProbeProbe)
|
||||
.groupBy(elem ⇒ if (elem == 2) throw exc else elem % 2)
|
||||
.withAttributes(ActorOperationAttributes.supervisionStrategy(resumingDecider))
|
||||
.withAttributes(ActorAttributes.supervisionStrategy(resumingDecider))
|
||||
.runWith(Sink.publisher)
|
||||
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, Unit])]()
|
||||
publisher.subscribe(subscriber)
|
||||
|
|
|
|||
|
|
@ -4,10 +4,10 @@
|
|||
package akka.stream.scaladsl
|
||||
|
||||
import akka.event.{ DummyClassForStringSources, Logging }
|
||||
import akka.stream.OperationAttributes.LogLevels
|
||||
import akka.stream.Attributes.LogLevels
|
||||
import akka.stream.testkit.{ AkkaSpec, ScriptedTest }
|
||||
import akka.stream.javadsl
|
||||
import akka.stream.{ ActorFlowMaterializer, FlowMaterializer, OperationAttributes }
|
||||
import akka.stream.{ ActorFlowMaterializer, FlowMaterializer, Attributes }
|
||||
import akka.testkit.TestProbe
|
||||
|
||||
import scala.util.control.NoStackTrace
|
||||
|
|
@ -40,7 +40,7 @@ class FlowLogSpec extends AkkaSpec("akka.loglevel = DEBUG") with ScriptedTest {
|
|||
}
|
||||
|
||||
"allow disabling element logging" in {
|
||||
val disableElementLogging = OperationAttributes.logLevels(
|
||||
val disableElementLogging = Attributes.logLevels(
|
||||
onElement = LogLevels.Off,
|
||||
onFinish = Logging.DebugLevel,
|
||||
onFailure = Logging.DebugLevel)
|
||||
|
|
@ -118,14 +118,14 @@ class FlowLogSpec extends AkkaSpec("akka.loglevel = DEBUG") with ScriptedTest {
|
|||
}
|
||||
|
||||
"allow configuring log levels via OperationAttributes" in {
|
||||
val logAttrs = OperationAttributes.logLevels(
|
||||
val logAttrs = Attributes.logLevels(
|
||||
onElement = Logging.WarningLevel,
|
||||
onFinish = Logging.InfoLevel,
|
||||
onFailure = Logging.DebugLevel)
|
||||
|
||||
Source.single(42)
|
||||
.log("flow-6")
|
||||
.withAttributes(OperationAttributes.logLevels(
|
||||
.withAttributes(Attributes.logLevels(
|
||||
onElement = Logging.WarningLevel,
|
||||
onFinish = Logging.InfoLevel,
|
||||
onFailure = Logging.DebugLevel))
|
||||
|
|
|
|||
|
|
@ -14,7 +14,7 @@ import akka.stream.testkit._
|
|||
import akka.stream.testkit.Utils._
|
||||
import akka.testkit.TestLatch
|
||||
import akka.testkit.TestProbe
|
||||
import akka.stream.ActorOperationAttributes.supervisionStrategy
|
||||
import akka.stream.ActorAttributes.supervisionStrategy
|
||||
import akka.stream.Supervision.resumingDecider
|
||||
import akka.stream.impl.ReactiveStreamsCompliance
|
||||
import scala.util.Try
|
||||
|
|
|
|||
|
|
@ -14,7 +14,7 @@ import akka.stream.testkit.scaladsl._
|
|||
import akka.stream.testkit.Utils._
|
||||
import akka.testkit.TestLatch
|
||||
import akka.testkit.TestProbe
|
||||
import akka.stream.ActorOperationAttributes.supervisionStrategy
|
||||
import akka.stream.ActorAttributes.supervisionStrategy
|
||||
import akka.stream.Supervision.resumingDecider
|
||||
import akka.stream.impl.ReactiveStreamsCompliance
|
||||
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import scala.concurrent.{ Future, Await }
|
|||
import scala.concurrent.duration._
|
||||
import scala.util.Try
|
||||
import scala.util.control.NoStackTrace
|
||||
import akka.stream.{ OperationAttributes, ActorFlowMaterializer, ActorFlowMaterializerSettings }
|
||||
import akka.stream.{ Attributes, ActorFlowMaterializer, ActorFlowMaterializerSettings }
|
||||
import org.reactivestreams.Subscriber
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.testkit.Utils._
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import akka.stream.ActorFlowMaterializer
|
|||
import akka.stream.ActorFlowMaterializerSettings
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.testkit.Utils._
|
||||
import akka.stream.ActorOperationAttributes
|
||||
import akka.stream.ActorAttributes
|
||||
import akka.stream.Supervision
|
||||
|
||||
class FlowScanSpec extends AkkaSpec {
|
||||
|
|
@ -47,7 +47,7 @@ class FlowScanSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"fail properly" in {
|
||||
import ActorOperationAttributes._
|
||||
import ActorAttributes._
|
||||
val scan = Flow[Int].scan(0) { (old, current) ⇒
|
||||
require(current > 0)
|
||||
old + current
|
||||
|
|
|
|||
|
|
@ -3,8 +3,8 @@
|
|||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.OperationAttributes._
|
||||
import akka.stream.ActorOperationAttributes._
|
||||
import akka.stream.Attributes._
|
||||
import akka.stream.ActorAttributes._
|
||||
import akka.stream.ActorFlowMaterializer
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.actor.ActorRef
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import akka.stream.stage.Stage
|
|||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor._
|
||||
import akka.stream.{ AbruptTerminationException, OperationAttributes, ActorFlowMaterializerSettings, ActorFlowMaterializer }
|
||||
import akka.stream.{ AbruptTerminationException, Attributes, ActorFlowMaterializerSettings, ActorFlowMaterializer }
|
||||
import akka.stream.impl._
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.testkit.Utils._
|
||||
|
|
@ -44,7 +44,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
|||
_settings: ActorFlowMaterializerSettings,
|
||||
_ops: Seq[Stage[_, _]],
|
||||
brokenMessage: Any)
|
||||
extends ActorInterpreter(_settings, _ops, mat, OperationAttributes.none) {
|
||||
extends ActorInterpreter(_settings, _ops, mat, Attributes.none) {
|
||||
|
||||
import akka.stream.actor.ActorSubscriberMessage._
|
||||
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@ package akka.stream.scaladsl
|
|||
|
||||
import akka.stream.ActorFlowMaterializer
|
||||
import akka.stream.ActorFlowMaterializerSettings
|
||||
import akka.stream.ActorOperationAttributes
|
||||
import akka.stream.ActorAttributes
|
||||
import akka.stream.Supervision.resumingDecider
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.testkit.TestPublisher
|
||||
|
|
@ -170,7 +170,7 @@ class FlowSplitAfterSpec extends AkkaSpec {
|
|||
val exc = TE("test")
|
||||
val publisher = Source(publisherProbeProbe)
|
||||
.splitAfter(elem ⇒ if (elem == 3) throw exc else elem % 3 == 0)
|
||||
.withAttributes(ActorOperationAttributes.supervisionStrategy(resumingDecider))
|
||||
.withAttributes(ActorAttributes.supervisionStrategy(resumingDecider))
|
||||
.runWith(Sink.publisher)
|
||||
val subscriber = TestSubscriber.manualProbe[Source[Int, Unit]]()
|
||||
publisher.subscribe(subscriber)
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@ package akka.stream.scaladsl
|
|||
|
||||
import akka.stream.ActorFlowMaterializer
|
||||
import akka.stream.ActorFlowMaterializerSettings
|
||||
import akka.stream.ActorOperationAttributes
|
||||
import akka.stream.ActorAttributes
|
||||
import akka.stream.Supervision.resumingDecider
|
||||
import akka.stream.testkit.Utils._
|
||||
import akka.stream.testkit._
|
||||
|
|
@ -167,7 +167,7 @@ class FlowSplitWhenSpec extends AkkaSpec {
|
|||
val exc = TE("test")
|
||||
val publisher = Source(publisherProbeProbe)
|
||||
.splitWhen(elem ⇒ if (elem == 3) throw exc else elem % 3 == 0)
|
||||
.withAttributes(ActorOperationAttributes.supervisionStrategy(resumingDecider))
|
||||
.withAttributes(ActorAttributes.supervisionStrategy(resumingDecider))
|
||||
.runWith(Sink.publisher)
|
||||
val subscriber = TestSubscriber.manualProbe[Source[Int, Unit]]()
|
||||
publisher.subscribe(subscriber)
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@
|
|||
package akka.stream.scaladsl
|
||||
|
||||
import akka.actor.PoisonPill
|
||||
import akka.stream.{ OperationAttributes, OverflowStrategy, ActorFlowMaterializer, ActorFlowMaterializerSettings }
|
||||
import akka.stream.{ Attributes, OverflowStrategy, ActorFlowMaterializer, ActorFlowMaterializerSettings }
|
||||
import akka.stream.stage._
|
||||
import scala.collection.immutable.Seq
|
||||
import scala.concurrent.duration._
|
||||
|
|
|
|||
|
|
@ -12,10 +12,10 @@ import scala.util.control.NoStackTrace
|
|||
import scala.concurrent.Await
|
||||
import akka.stream.Supervision
|
||||
import akka.stream.impl.ReactiveStreamsCompliance
|
||||
import akka.stream.ActorOperationAttributes
|
||||
import akka.stream.ActorAttributes
|
||||
|
||||
class FlowSupervisionSpec extends AkkaSpec {
|
||||
import ActorOperationAttributes.supervisionStrategy
|
||||
import ActorAttributes.supervisionStrategy
|
||||
|
||||
implicit val materializer = ActorFlowMaterializer()(system)
|
||||
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.ActorOperationAttributes._
|
||||
import akka.stream.ActorAttributes._
|
||||
import akka.stream.Supervision._
|
||||
import akka.stream.testkit.Utils._
|
||||
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ import scala.concurrent.duration._
|
|||
|
||||
object GraphFlexiMergeSpec {
|
||||
|
||||
class Fair[T] extends FlexiMerge[T, UniformFanInShape[T, T]](new UniformFanInShape(2), OperationAttributes.name("FairMerge")) {
|
||||
class Fair[T] extends FlexiMerge[T, UniformFanInShape[T, T]](new UniformFanInShape(2), Attributes.name("FairMerge")) {
|
||||
def createMergeLogic(p: PortT): MergeLogic[T] = new MergeLogic[T] {
|
||||
override def initialState = State[T](ReadAny(p.in(0), p.in(1))) { (ctx, input, element) ⇒
|
||||
ctx.emit(element)
|
||||
|
|
@ -28,7 +28,7 @@ object GraphFlexiMergeSpec {
|
|||
}
|
||||
}
|
||||
|
||||
class StrictRoundRobin[T] extends FlexiMerge[T, UniformFanInShape[T, T]](new UniformFanInShape(2), OperationAttributes.name("RoundRobinMerge")) {
|
||||
class StrictRoundRobin[T] extends FlexiMerge[T, UniformFanInShape[T, T]](new UniformFanInShape(2), Attributes.name("RoundRobinMerge")) {
|
||||
def createMergeLogic(p: PortT): MergeLogic[T] = new MergeLogic[T] {
|
||||
val emitOtherOnClose = CompletionHandling(
|
||||
onUpstreamFinish = { (ctx, input) ⇒
|
||||
|
|
@ -64,7 +64,7 @@ object GraphFlexiMergeSpec {
|
|||
}
|
||||
|
||||
class StartStopTest(lifecycleProbe: ActorRef)
|
||||
extends FlexiMerge[String, FanInShape2[String, String, String]](new FanInShape2("StartStopTest"), OperationAttributes.name("StartStopTest")) {
|
||||
extends FlexiMerge[String, FanInShape2[String, String, String]](new FanInShape2("StartStopTest"), Attributes.name("StartStopTest")) {
|
||||
|
||||
def createMergeLogic(p: PortT) = new MergeLogic[String] {
|
||||
|
||||
|
|
@ -82,7 +82,7 @@ object GraphFlexiMergeSpec {
|
|||
}
|
||||
}
|
||||
|
||||
class MyZip[A, B] extends FlexiMerge[(A, B), FanInShape2[A, B, (A, B)]](new FanInShape2("MyZip"), OperationAttributes.name("MyZip")) {
|
||||
class MyZip[A, B] extends FlexiMerge[(A, B), FanInShape2[A, B, (A, B)]](new FanInShape2("MyZip"), Attributes.name("MyZip")) {
|
||||
def createMergeLogic(p: PortT): MergeLogic[(A, B)] = new MergeLogic[(A, B)] {
|
||||
var lastInA: A = _
|
||||
|
||||
|
|
@ -103,7 +103,7 @@ object GraphFlexiMergeSpec {
|
|||
}
|
||||
|
||||
class TripleCancellingZip[A, B, C](var cancelAfter: Int = Int.MaxValue, defVal: Option[A] = None)
|
||||
extends FlexiMerge[(A, B, C), FanInShape3[A, B, C, (A, B, C)]](new FanInShape3("TripleCancellingZip"), OperationAttributes.name("TripleCancellingZip")) {
|
||||
extends FlexiMerge[(A, B, C), FanInShape3[A, B, C, (A, B, C)]](new FanInShape3("TripleCancellingZip"), Attributes.name("TripleCancellingZip")) {
|
||||
def createMergeLogic(p: PortT) = new MergeLogic[(A, B, C)] {
|
||||
override def initialState = State(ReadAll(p.in0, p.in1, p.in2)) {
|
||||
case (ctx, input, inputs) ⇒
|
||||
|
|
@ -123,7 +123,7 @@ object GraphFlexiMergeSpec {
|
|||
}
|
||||
}
|
||||
|
||||
object PreferringMerge extends FlexiMerge[Int, UniformFanInShape[Int, Int]](new UniformFanInShape(3), OperationAttributes.name("PreferringMerge")) {
|
||||
object PreferringMerge extends FlexiMerge[Int, UniformFanInShape[Int, Int]](new UniformFanInShape(3), Attributes.name("PreferringMerge")) {
|
||||
def createMergeLogic(p: PortT) = new MergeLogic[Int] {
|
||||
override def initialState = State(Read(p.in(0))) {
|
||||
(ctx, input, element) ⇒
|
||||
|
|
@ -139,7 +139,7 @@ object GraphFlexiMergeSpec {
|
|||
}
|
||||
|
||||
class TestMerge(completionProbe: ActorRef)
|
||||
extends FlexiMerge[String, UniformFanInShape[String, String]](new UniformFanInShape(3), OperationAttributes.name("TestMerge")) {
|
||||
extends FlexiMerge[String, UniformFanInShape[String, String]](new UniformFanInShape(3), Attributes.name("TestMerge")) {
|
||||
|
||||
def createMergeLogic(p: PortT) = new MergeLogic[String] {
|
||||
var throwFromOnComplete = false
|
||||
|
|
@ -697,7 +697,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
|
|||
protected override def construct(i: Init[T]) = new MShape(i)
|
||||
}
|
||||
class MyMerge[T] extends FlexiMerge[T, MShape[T]](
|
||||
new MShape, OperationAttributes.name("cmerge")) {
|
||||
new MShape, Attributes.name("cmerge")) {
|
||||
import akka.stream.scaladsl.FlexiMerge._
|
||||
override def createMergeLogic(p: PortT) = new MergeLogic[T] {
|
||||
override def initialState =
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ object GraphFlexiRouteSpec {
|
|||
* they are have requested elements. Or in other words, if all outputs have demand available at the same
|
||||
* time then in finite steps all elements are enqueued to them.
|
||||
*/
|
||||
class Fair[T] extends FlexiRoute[T, UniformFanOutShape[T, T]](new UniformFanOutShape(2), OperationAttributes.name("FairBalance")) {
|
||||
class Fair[T] extends FlexiRoute[T, UniformFanOutShape[T, T]](new UniformFanOutShape(2), Attributes.name("FairBalance")) {
|
||||
import FlexiRoute._
|
||||
|
||||
override def createRouteLogic(p: PortT): RouteLogic[T] = new RouteLogic[T] {
|
||||
|
|
@ -42,7 +42,7 @@ object GraphFlexiRouteSpec {
|
|||
* It never skips an output while cycling but waits on it instead (closed outputs are skipped though).
|
||||
* The fair route above is a non-strict round-robin (skips currently unavailable outputs).
|
||||
*/
|
||||
class StrictRoundRobin[T] extends FlexiRoute[T, UniformFanOutShape[T, T]](new UniformFanOutShape(2), OperationAttributes.name("RoundRobinBalance")) {
|
||||
class StrictRoundRobin[T] extends FlexiRoute[T, UniformFanOutShape[T, T]](new UniformFanOutShape(2), Attributes.name("RoundRobinBalance")) {
|
||||
import FlexiRoute._
|
||||
|
||||
override def createRouteLogic(p: PortT) = new RouteLogic[T] {
|
||||
|
|
@ -61,7 +61,7 @@ object GraphFlexiRouteSpec {
|
|||
}
|
||||
}
|
||||
|
||||
class Unzip[A, B] extends FlexiRoute[(A, B), FanOutShape2[(A, B), A, B]](new FanOutShape2("Unzip"), OperationAttributes.name("Unzip")) {
|
||||
class Unzip[A, B] extends FlexiRoute[(A, B), FanOutShape2[(A, B), A, B]](new FanOutShape2("Unzip"), Attributes.name("Unzip")) {
|
||||
import FlexiRoute._
|
||||
|
||||
override def createRouteLogic(p: PortT) = new RouteLogic[(A, B)] {
|
||||
|
|
@ -78,7 +78,7 @@ object GraphFlexiRouteSpec {
|
|||
}
|
||||
|
||||
class StartStopTestRoute(lifecycleProbe: ActorRef)
|
||||
extends FlexiRoute[String, FanOutShape2[String, String, String]](new FanOutShape2("StartStopTest"), OperationAttributes.name("StartStopTest")) {
|
||||
extends FlexiRoute[String, FanOutShape2[String, String, String]](new FanOutShape2("StartStopTest"), Attributes.name("StartStopTest")) {
|
||||
import FlexiRoute._
|
||||
|
||||
def createRouteLogic(p: PortT) = new RouteLogic[String] {
|
||||
|
|
@ -100,7 +100,7 @@ object GraphFlexiRouteSpec {
|
|||
}
|
||||
|
||||
class TestRoute(completionProbe: ActorRef)
|
||||
extends FlexiRoute[String, FanOutShape2[String, String, String]](new FanOutShape2("TestRoute"), OperationAttributes.name("TestRoute")) {
|
||||
extends FlexiRoute[String, FanOutShape2[String, String, String]](new FanOutShape2("TestRoute"), Attributes.name("TestRoute")) {
|
||||
import FlexiRoute._
|
||||
|
||||
var throwFromOnComplete = false
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.OperationAttributes._
|
||||
import akka.stream.Attributes._
|
||||
import akka.stream.ActorFlowMaterializer
|
||||
import akka.stream.ActorFlowMaterializerSettings
|
||||
import akka.stream.testkit._
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import akka.stream.testkit._
|
|||
import akka.util.ByteString
|
||||
import akka.stream.{ Inlet, Outlet, Shape, Graph }
|
||||
import org.scalactic.ConversionCheckedTripleEquals
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.Attributes
|
||||
|
||||
object GraphOpsIntegrationSpec {
|
||||
import FlowGraph.Implicits._
|
||||
|
|
@ -89,7 +89,7 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual
|
|||
}
|
||||
|
||||
"support wikipedia Topological_sorting 2" in {
|
||||
import OperationAttributes.name
|
||||
import Attributes.name
|
||||
// see https://en.wikipedia.org/wiki/Topological_sorting#mediaviewer/File:Directed_acyclic_graph.png
|
||||
val seqSink = Sink.head[Seq[Int]]
|
||||
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@
|
|||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.ActorFlowMaterializer
|
||||
import akka.stream.ActorOperationAttributes._
|
||||
import akka.stream.ActorAttributes._
|
||||
import akka.stream.Supervision._
|
||||
import akka.stream.testkit.Utils._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
|
|
|
|||
|
|
@ -8,8 +8,8 @@ import akka.actor.Deploy
|
|||
import akka.stream._
|
||||
import akka.stream.impl.Junctions.FanInModule
|
||||
import akka.stream.impl.StreamLayout.Module
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.OperationAttributes._
|
||||
import akka.stream.Attributes
|
||||
import akka.stream.Attributes._
|
||||
|
||||
/** INTERNAL API: Boilerplate generated Junctions */
|
||||
private[akka] object GenJunctions {
|
||||
|
|
@ -23,9 +23,9 @@ private[akka] object GenJunctions {
|
|||
final case class ZipWith1Module[[#A1#], B](
|
||||
shape: FanInShape1[[#A1#], B],
|
||||
f: ([#A1#]) ⇒ B,
|
||||
override val attributes: OperationAttributes = name("zipWith1")) extends FanInModule with ZipWithModule {
|
||||
override val attributes: Attributes = name("zipWith1")) extends FanInModule with ZipWithModule {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Module = copy(attributes = attr)
|
||||
override def withAttributes(attr: Attributes): Module = copy(attributes = attr)
|
||||
|
||||
override def carbonCopy: Module = ZipWith1Module(shape.deepCopy(), f, attributes)
|
||||
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@ package akka.stream.scaladsl
|
|||
|
||||
import akka.stream.impl.StreamLayout
|
||||
import akka.stream.impl.StreamLayout.Module
|
||||
import akka.stream.{ Graph, OperationAttributes, Shape }
|
||||
import akka.stream.{ Graph, Attributes, Shape }
|
||||
|
||||
trait GraphApply {
|
||||
|
||||
|
|
@ -108,10 +108,10 @@ private[stream] object GraphApply {
|
|||
class GraphImpl[S <: Shape, Mat](override val shape: S, private[stream] override val module: StreamLayout.Module)
|
||||
extends Graph[S, Mat] {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Graph[S, Mat] =
|
||||
override def withAttributes(attr: Attributes): Graph[S, Mat] =
|
||||
new GraphImpl(shape, module.withAttributes(attr).wrap())
|
||||
|
||||
override def named(name: String): Graph[S, Mat] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): Graph[S, Mat] = withAttributes(Attributes.name(name))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -17,7 +17,7 @@ trait ZipWithApply {
|
|||
*/
|
||||
def apply[[#A1#], O](zipper: ([#A1#]) ⇒ O): ZipWith1[[#A1#], O] = {
|
||||
val shape = new FanInShape1[[#A1#], O]("ZipWith1")
|
||||
new ZipWith1(shape, new ZipWith1Module(shape, zipper, OperationAttributes.name("ZipWith1")))
|
||||
new ZipWith1(shape, new ZipWith1Module(shape, zipper, Attributes.name("ZipWith1")))
|
||||
}
|
||||
#
|
||||
|
||||
|
|
@ -30,10 +30,10 @@ class ZipWith1[[#A1#], O] private[stream] (override val shape: FanInShape1[[#A1#
|
|||
private[stream] override val module: StreamLayout.Module)
|
||||
extends Graph[FanInShape1[[#A1#], O], Unit] {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): ZipWith1[[#A1#], O] =
|
||||
override def withAttributes(attr: Attributes): ZipWith1[[#A1#], O] =
|
||||
new ZipWith1(shape, module.withAttributes(attr).wrap())
|
||||
|
||||
override def named(name: String): ZipWith1[[#A1#], O] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): ZipWith1[[#A1#], O] = withAttributes(Attributes.name(name))
|
||||
}
|
||||
#
|
||||
]
|
||||
|
|
|
|||
|
|
@ -145,7 +145,7 @@ abstract class ActorFlowMaterializer extends FlowMaterializer {
|
|||
|
||||
def settings: ActorFlowMaterializerSettings
|
||||
|
||||
def effectiveSettings(opAttr: OperationAttributes): ActorFlowMaterializerSettings
|
||||
def effectiveSettings(opAttr: Attributes): ActorFlowMaterializerSettings
|
||||
|
||||
/**
|
||||
* Shuts down this materializer and all the stages that have been materialized through this materializer. After
|
||||
|
|
@ -291,7 +291,7 @@ final class ActorFlowMaterializerSettings(
|
|||
/**
|
||||
* Scala API: Decides how exceptions from application code are to be handled, unless
|
||||
* overridden for specific flows of the stream operations with
|
||||
* [[akka.stream.OperationAttributes#supervisionStrategy]].
|
||||
* [[akka.stream.Attributes#supervisionStrategy]].
|
||||
*/
|
||||
def withSupervisionStrategy(decider: Supervision.Decider): ActorFlowMaterializerSettings =
|
||||
copy(supervisionDecider = decider)
|
||||
|
|
@ -299,7 +299,7 @@ final class ActorFlowMaterializerSettings(
|
|||
/**
|
||||
* Java API: Decides how exceptions from application code are to be handled, unless
|
||||
* overridden for specific flows of the stream operations with
|
||||
* [[akka.stream.OperationAttributes#supervisionStrategy]].
|
||||
* [[akka.stream.Attributes#supervisionStrategy]].
|
||||
*/
|
||||
def withSupervisionStrategy(decider: function.Function[Throwable, Supervision.Directive]): ActorFlowMaterializerSettings = {
|
||||
import Supervision._
|
||||
|
|
|
|||
|
|
@ -13,29 +13,29 @@ import akka.japi.function
|
|||
* Holds attributes which can be used to alter [[akka.stream.scaladsl.Flow]] / [[akka.stream.javadsl.Flow]]
|
||||
* or [[akka.stream.scaladsl.FlowGraph]] / [[akka.stream.javadsl.FlowGraph]] materialization.
|
||||
*
|
||||
* Note that more attributes for the [[ActorFlowMaterializer]] are defined in [[ActorOperationAttributes]].
|
||||
* Note that more attributes for the [[ActorFlowMaterializer]] are defined in [[ActorAttributes]].
|
||||
*/
|
||||
final case class OperationAttributes private (attributes: immutable.Seq[OperationAttributes.Attribute] = Nil) {
|
||||
final case class Attributes private (attributeList: immutable.Seq[Attributes.Attribute] = Nil) {
|
||||
|
||||
import OperationAttributes._
|
||||
import Attributes._
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def getAttributes(): java.util.List[Attribute] = {
|
||||
def getAttributeList(): java.util.List[Attribute] = {
|
||||
import scala.collection.JavaConverters._
|
||||
attributes.asJava
|
||||
attributeList.asJava
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API: Get all attributes of a given `Class` or
|
||||
* subclass thereof.
|
||||
*/
|
||||
def getAttributes[T <: Attribute](c: Class[T]): java.util.List[T] =
|
||||
if (attributes.isEmpty) java.util.Collections.emptyList()
|
||||
def getAttributeList[T <: Attribute](c: Class[T]): java.util.List[T] =
|
||||
if (attributeList.isEmpty) java.util.Collections.emptyList()
|
||||
else {
|
||||
val result = new java.util.ArrayList[T]
|
||||
attributes.foreach { a ⇒
|
||||
attributeList.foreach { a ⇒
|
||||
if (c.isInstance(a))
|
||||
result.add(c.cast(a))
|
||||
}
|
||||
|
|
@ -47,7 +47,7 @@ final case class OperationAttributes private (attributes: immutable.Seq[Operatio
|
|||
* If no such attribute exists the `default` value is returned.
|
||||
*/
|
||||
def getAttribute[T <: Attribute](c: Class[T], default: T): T =
|
||||
attributes.find(c.isInstance) match {
|
||||
attributeList.find(c.isInstance) match {
|
||||
case Some(a) ⇒ c.cast(a)
|
||||
case None ⇒ default
|
||||
}
|
||||
|
|
@ -55,20 +55,20 @@ final case class OperationAttributes private (attributes: immutable.Seq[Operatio
|
|||
/**
|
||||
* Adds given attributes to the end of these attributes.
|
||||
*/
|
||||
def and(other: OperationAttributes): OperationAttributes =
|
||||
if (attributes.isEmpty) other
|
||||
else if (other.attributes.isEmpty) this
|
||||
else OperationAttributes(attributes ++ other.attributes)
|
||||
def and(other: Attributes): Attributes =
|
||||
if (attributeList.isEmpty) other
|
||||
else if (other.attributeList.isEmpty) this
|
||||
else Attributes(attributeList ++ other.attributeList)
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def nameLifted: Option[String] =
|
||||
if (attributes.isEmpty)
|
||||
if (attributeList.isEmpty)
|
||||
None
|
||||
else {
|
||||
val sb = new java.lang.StringBuilder
|
||||
val iter = attributes.iterator
|
||||
val iter = attributeList.iterator
|
||||
while (iter.hasNext) {
|
||||
iter.next() match {
|
||||
case Name(name) ⇒
|
||||
|
|
@ -93,54 +93,54 @@ final case class OperationAttributes private (attributes: immutable.Seq[Operatio
|
|||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def nameOption: Option[String] =
|
||||
attributes.collectFirst { case Name(name) ⇒ name }
|
||||
attributeList.collectFirst { case Name(name) ⇒ name }
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def logLevels: Option[LogLevels] =
|
||||
attributes.collectFirst { case l: LogLevels ⇒ l }
|
||||
attributeList.collectFirst { case l: LogLevels ⇒ l }
|
||||
|
||||
private[akka] def transform(node: StageModule): StageModule =
|
||||
if ((this eq OperationAttributes.none) || (this eq node.attributes)) node
|
||||
if ((this eq Attributes.none) || (this eq node.attributes)) node
|
||||
else node.withAttributes(attributes = this and node.attributes)
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Note that more attributes for the [[ActorFlowMaterializer]] are defined in [[ActorOperationAttributes]].
|
||||
* Note that more attributes for the [[ActorFlowMaterializer]] are defined in [[ActorAttributes]].
|
||||
*/
|
||||
object OperationAttributes {
|
||||
object Attributes {
|
||||
|
||||
trait Attribute
|
||||
final case class Name(n: String) extends Attribute
|
||||
final case class InputBuffer(initial: Int, max: Int) extends Attribute
|
||||
final case class LogLevels(onElement: Logging.LogLevel, onFinish: Logging.LogLevel, onFailure: Logging.LogLevel) extends Attribute
|
||||
object LogLevels {
|
||||
/** Use to disable logging on certain operations when configuring [[OperationAttributes.LogLevels]] */
|
||||
/** Use to disable logging on certain operations when configuring [[Attributes.LogLevels]] */
|
||||
final val Off: Logging.LogLevel = Logging.levelFor("off").get
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def apply(attribute: Attribute): OperationAttributes =
|
||||
private[akka] def apply(attribute: Attribute): Attributes =
|
||||
apply(List(attribute))
|
||||
|
||||
val none: OperationAttributes = OperationAttributes()
|
||||
val none: Attributes = Attributes()
|
||||
|
||||
/**
|
||||
* Specifies the name of the operation.
|
||||
* If the name is null or empty the name is ignored, i.e. [[#none]] is returned.
|
||||
*/
|
||||
def name(name: String): OperationAttributes =
|
||||
def name(name: String): Attributes =
|
||||
if (name == null || name.isEmpty) none
|
||||
else OperationAttributes(Name(name))
|
||||
else Attributes(Name(name))
|
||||
|
||||
/**
|
||||
* Specifies the initial and maximum size of the input buffer.
|
||||
*/
|
||||
def inputBuffer(initial: Int, max: Int): OperationAttributes = OperationAttributes(InputBuffer(initial, max))
|
||||
def inputBuffer(initial: Int, max: Int): Attributes = Attributes(InputBuffer(initial, max))
|
||||
|
||||
/**
|
||||
* Java API
|
||||
|
|
@ -161,38 +161,38 @@ object OperationAttributes {
|
|||
* Configures `log()` stage log-levels to be used when logging.
|
||||
* Logging a certain operation can be completely disabled by using [[LogLevels.Off]].
|
||||
*
|
||||
* See [[OperationAttributes.createLogLevels]] for Java API
|
||||
* See [[Attributes.createLogLevels]] for Java API
|
||||
*/
|
||||
def logLevels(onElement: Logging.LogLevel = Logging.DebugLevel, onFinish: Logging.LogLevel = Logging.DebugLevel, onFailure: Logging.LogLevel = Logging.ErrorLevel) =
|
||||
OperationAttributes(LogLevels(onElement, onFinish, onFailure))
|
||||
Attributes(LogLevels(onElement, onFinish, onFailure))
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Attributes for the [[ActorFlowMaterializer]].
|
||||
* Note that more attributes defined in [[OperationAttributes]].
|
||||
* Note that more attributes defined in [[Attributes]].
|
||||
*/
|
||||
object ActorOperationAttributes {
|
||||
import OperationAttributes._
|
||||
object ActorAttributes {
|
||||
import Attributes._
|
||||
final case class Dispatcher(dispatcher: String) extends Attribute
|
||||
final case class SupervisionStrategy(decider: Supervision.Decider) extends Attribute
|
||||
|
||||
/**
|
||||
* Specifies the name of the dispatcher.
|
||||
*/
|
||||
def dispatcher(dispatcher: String): OperationAttributes = OperationAttributes(Dispatcher(dispatcher))
|
||||
def dispatcher(dispatcher: String): Attributes = Attributes(Dispatcher(dispatcher))
|
||||
|
||||
/**
|
||||
* Scala API: Decides how exceptions from user are to be handled.
|
||||
*/
|
||||
def supervisionStrategy(decider: Supervision.Decider): OperationAttributes =
|
||||
OperationAttributes(SupervisionStrategy(decider))
|
||||
def supervisionStrategy(decider: Supervision.Decider): Attributes =
|
||||
Attributes(SupervisionStrategy(decider))
|
||||
|
||||
/**
|
||||
* Java API: Decides how exceptions from application code are to be handled.
|
||||
*/
|
||||
def withSupervisionStrategy(decider: function.Function[Throwable, Supervision.Directive]): OperationAttributes =
|
||||
ActorOperationAttributes.supervisionStrategy(decider.apply _)
|
||||
def withSupervisionStrategy(decider: function.Function[Throwable, Supervision.Directive]): Attributes =
|
||||
ActorAttributes.supervisionStrategy(decider.apply _)
|
||||
|
||||
/**
|
||||
* Java API
|
||||
|
|
@ -213,9 +213,9 @@ object ActorOperationAttributes {
|
|||
* Configures `log()` stage log-levels to be used when logging.
|
||||
* Logging a certain operation can be completely disabled by using [[LogLevels.Off]].
|
||||
*
|
||||
* See [[OperationAttributes.createLogLevels]] for Java API
|
||||
* See [[Attributes.createLogLevels]] for Java API
|
||||
*/
|
||||
def logLevels(onElement: Logging.LogLevel = Logging.DebugLevel, onFinish: Logging.LogLevel = Logging.DebugLevel, onFailure: Logging.LogLevel = Logging.ErrorLevel) =
|
||||
OperationAttributes(LogLevels(onElement, onFinish, onFailure))
|
||||
Attributes(LogLevels(onElement, onFinish, onFailure))
|
||||
|
||||
}
|
||||
|
|
@ -52,5 +52,5 @@ private[akka] object NoFlowMaterializer extends FlowMaterializer {
|
|||
*/
|
||||
private[akka] case class MaterializationContext(
|
||||
materializer: FlowMaterializer,
|
||||
effectiveAttributes: OperationAttributes,
|
||||
effectiveAttributes: Attributes,
|
||||
stageName: String)
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ trait Graph[+S <: Shape, +M] {
|
|||
*/
|
||||
private[stream] def module: StreamLayout.Module
|
||||
|
||||
def withAttributes(attr: OperationAttributes): Graph[S, M]
|
||||
def withAttributes(attr: Attributes): Graph[S, M]
|
||||
|
||||
def named(name: String): Graph[S, M] = withAttributes(OperationAttributes.name(name))
|
||||
def named(name: String): Graph[S, M] = withAttributes(Attributes.name(name))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ import java.util.concurrent.atomic.AtomicLong
|
|||
import scala.concurrent.duration._
|
||||
import scala.language.implicitConversions
|
||||
import scala.language.existentials
|
||||
import akka.stream.OperationAttributes._
|
||||
import akka.stream.Attributes._
|
||||
import akka.stream.scaladsl.{ Keep, Source, Flow }
|
||||
import akka.stream.stage._
|
||||
|
||||
|
|
|
|||
|
|
@ -51,10 +51,10 @@ private[akka] case class ActorFlowMaterializerImpl(
|
|||
|
||||
private[this] def createFlowName(): String = s"$namePrefix-${nextFlowNameCount()}"
|
||||
|
||||
override def effectiveSettings(opAttr: OperationAttributes): ActorFlowMaterializerSettings = {
|
||||
import OperationAttributes._
|
||||
import ActorOperationAttributes._
|
||||
opAttr.attributes.foldLeft(settings) { (s, attr) ⇒
|
||||
override def effectiveSettings(opAttr: Attributes): ActorFlowMaterializerSettings = {
|
||||
import Attributes._
|
||||
import ActorAttributes._
|
||||
opAttr.attributeList.foldLeft(settings) { (s, attr) ⇒
|
||||
attr match {
|
||||
case InputBuffer(initial, max) ⇒ s.withInputBuffer(initial, max)
|
||||
case Dispatcher(dispatcher) ⇒ s.withDispatcher(dispatcher)
|
||||
|
|
@ -73,13 +73,13 @@ private[akka] case class ActorFlowMaterializerImpl(
|
|||
val session = new MaterializerSession(runnableFlow.module) {
|
||||
private val flowName = createFlowName()
|
||||
private var nextId = 0
|
||||
private def stageName(attr: OperationAttributes): String = {
|
||||
private def stageName(attr: Attributes): String = {
|
||||
val name = s"$flowName-$nextId-${attr.nameOrDefault()}"
|
||||
nextId += 1
|
||||
name
|
||||
}
|
||||
|
||||
override protected def materializeAtomic(atomic: Module, effectiveAttributes: OperationAttributes): Any = {
|
||||
override protected def materializeAtomic(atomic: Module, effectiveAttributes: Attributes): Any = {
|
||||
|
||||
def newMaterializationContext() = new MaterializationContext(ActorFlowMaterializerImpl.this,
|
||||
effectiveAttributes, stageName(effectiveAttributes))
|
||||
|
|
@ -120,7 +120,7 @@ private[akka] case class ActorFlowMaterializerImpl(
|
|||
}
|
||||
|
||||
private def processorFor(op: StageModule,
|
||||
effectiveAttributes: OperationAttributes,
|
||||
effectiveAttributes: Attributes,
|
||||
effectiveSettings: ActorFlowMaterializerSettings): (Processor[Any, Any], Any) = op match {
|
||||
case DirectProcessor(processorFactory, _) ⇒ processorFactory()
|
||||
case Identity(attr) ⇒ (new VirtualProcessor, ())
|
||||
|
|
@ -134,7 +134,7 @@ private[akka] case class ActorFlowMaterializerImpl(
|
|||
}
|
||||
|
||||
private def materializeJunction(op: JunctionModule,
|
||||
effectiveAttributes: OperationAttributes,
|
||||
effectiveAttributes: Attributes,
|
||||
effectiveSettings: ActorFlowMaterializerSettings): Unit = {
|
||||
op match {
|
||||
case fanin: FanInModule ⇒
|
||||
|
|
@ -294,7 +294,7 @@ private[akka] object ActorProcessorFactory {
|
|||
|
||||
private val _identity = (x: Any) ⇒ x
|
||||
|
||||
def props(materializer: ActorFlowMaterializer, op: StageModule, parentAttributes: OperationAttributes): (Props, Any) = {
|
||||
def props(materializer: ActorFlowMaterializer, op: StageModule, parentAttributes: Attributes): (Props, Any) = {
|
||||
val att = parentAttributes and op.attributes
|
||||
// USE THIS TO AVOID CLOSING OVER THE MATERIALIZER BELOW
|
||||
// Also, otherwise the attributes will not affect the settings properly!
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@ package akka.stream.impl
|
|||
|
||||
import akka.stream.impl.StreamLayout.Module
|
||||
import akka.stream.scaladsl.FlexiRoute.RouteLogic
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.Attributes
|
||||
import akka.stream.{ Inlet, Outlet, Shape, InPort, OutPort }
|
||||
import akka.stream.scaladsl.FlexiMerge.MergeLogic
|
||||
import akka.stream.UniformFanInShape
|
||||
|
|
@ -19,7 +19,7 @@ import akka.event.Logging.simpleName
|
|||
*/
|
||||
private[stream] object Junctions {
|
||||
|
||||
import OperationAttributes._
|
||||
import Attributes._
|
||||
|
||||
sealed trait JunctionModule extends Module {
|
||||
override def subModules: Set[Module] = Set.empty
|
||||
|
|
@ -35,27 +35,27 @@ private[stream] object Junctions {
|
|||
|
||||
final case class MergeModule[T](
|
||||
shape: UniformFanInShape[T, T],
|
||||
override val attributes: OperationAttributes = name("merge")) extends FanInModule {
|
||||
override val attributes: Attributes = name("merge")) extends FanInModule {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Module = copy(attributes = attr)
|
||||
override def withAttributes(attr: Attributes): Module = copy(attributes = attr)
|
||||
|
||||
override def carbonCopy: Module = MergeModule(shape.deepCopy(), attributes)
|
||||
}
|
||||
|
||||
final case class BroadcastModule[T](
|
||||
shape: UniformFanOutShape[T, T],
|
||||
override val attributes: OperationAttributes = name("broadcast")) extends FanOutModule {
|
||||
override val attributes: Attributes = name("broadcast")) extends FanOutModule {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Module = copy(attributes = attr)
|
||||
override def withAttributes(attr: Attributes): Module = copy(attributes = attr)
|
||||
|
||||
override def carbonCopy: Module = BroadcastModule(shape.deepCopy(), attributes)
|
||||
}
|
||||
|
||||
final case class MergePreferredModule[T](
|
||||
shape: MergePreferred.MergePreferredShape[T],
|
||||
override val attributes: OperationAttributes = name("preferred")) extends FanInModule {
|
||||
override val attributes: Attributes = name("preferred")) extends FanInModule {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Module = copy(attributes = attr)
|
||||
override def withAttributes(attr: Attributes): Module = copy(attributes = attr)
|
||||
|
||||
override def carbonCopy: Module = MergePreferredModule(shape.deepCopy(), attributes)
|
||||
}
|
||||
|
|
@ -63,11 +63,11 @@ private[stream] object Junctions {
|
|||
final case class FlexiMergeModule[T, S <: Shape](
|
||||
shape: S,
|
||||
flexi: S ⇒ MergeLogic[T],
|
||||
override val attributes: OperationAttributes) extends FanInModule {
|
||||
override val attributes: Attributes) extends FanInModule {
|
||||
|
||||
require(shape.outlets.size == 1, "FlexiMerge can have only one output port")
|
||||
|
||||
override def withAttributes(attributes: OperationAttributes): Module = copy(attributes = attributes)
|
||||
override def withAttributes(attributes: Attributes): Module = copy(attributes = attributes)
|
||||
|
||||
override def carbonCopy: Module = FlexiMergeModule(shape.deepCopy().asInstanceOf[S], flexi, attributes)
|
||||
}
|
||||
|
|
@ -75,11 +75,11 @@ private[stream] object Junctions {
|
|||
final case class FlexiRouteModule[T, S <: Shape](
|
||||
shape: S,
|
||||
flexi: S ⇒ RouteLogic[T],
|
||||
override val attributes: OperationAttributes) extends FanOutModule {
|
||||
override val attributes: Attributes) extends FanOutModule {
|
||||
|
||||
require(shape.inlets.size == 1, "FlexiRoute can have only one input port")
|
||||
|
||||
override def withAttributes(attributes: OperationAttributes): Module = copy(attributes = attributes)
|
||||
override def withAttributes(attributes: Attributes): Module = copy(attributes = attributes)
|
||||
|
||||
override def carbonCopy: Module = FlexiRouteModule(shape.deepCopy().asInstanceOf[S], flexi, attributes)
|
||||
}
|
||||
|
|
@ -87,27 +87,27 @@ private[stream] object Junctions {
|
|||
final case class BalanceModule[T](
|
||||
shape: UniformFanOutShape[T, T],
|
||||
waitForAllDownstreams: Boolean,
|
||||
override val attributes: OperationAttributes = name("broadcast")) extends FanOutModule {
|
||||
override val attributes: Attributes = name("broadcast")) extends FanOutModule {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Module = copy(attributes = attr)
|
||||
override def withAttributes(attr: Attributes): Module = copy(attributes = attr)
|
||||
|
||||
override def carbonCopy: Module = BalanceModule(shape.deepCopy(), waitForAllDownstreams, attributes)
|
||||
}
|
||||
|
||||
final case class UnzipModule[A, B](
|
||||
shape: FanOutShape2[(A, B), A, B],
|
||||
override val attributes: OperationAttributes = name("unzip")) extends FanOutModule {
|
||||
override val attributes: Attributes = name("unzip")) extends FanOutModule {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Module = copy(attributes = attr)
|
||||
override def withAttributes(attr: Attributes): Module = copy(attributes = attr)
|
||||
|
||||
override def carbonCopy: Module = UnzipModule(shape.deepCopy(), attributes)
|
||||
}
|
||||
|
||||
final case class ConcatModule[T](
|
||||
shape: UniformFanInShape[T, T],
|
||||
override val attributes: OperationAttributes = name("concat")) extends FanInModule {
|
||||
override val attributes: Attributes = name("concat")) extends FanInModule {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Module = copy(attributes = attr)
|
||||
override def withAttributes(attr: Attributes): Module = copy(attributes = attr)
|
||||
|
||||
override def carbonCopy: Module = ConcatModule(shape.deepCopy(), attributes)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.stream.impl
|
|||
import java.io.{ InputStream, File }
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import akka.actor.{ ActorRef, Cancellable, PoisonPill, Props }
|
||||
import akka.stream.ActorOperationAttributes.Dispatcher
|
||||
import akka.stream.ActorAttributes.Dispatcher
|
||||
import akka.stream.impl.StreamLayout.Module
|
||||
import akka.stream._
|
||||
import akka.util.ByteString
|
||||
|
|
@ -37,7 +37,7 @@ private[akka] abstract class SourceModule[+Out, +Mat](val shape: SourceShape[Out
|
|||
|
||||
override def subModules: Set[Module] = Set.empty
|
||||
|
||||
def amendShape(attr: OperationAttributes): SourceShape[Out] = {
|
||||
def amendShape(attr: Attributes): SourceShape[Out] = {
|
||||
attr.nameOption match {
|
||||
case None ⇒ shape
|
||||
case s: Some[String] if s == attributes.nameOption ⇒ shape
|
||||
|
|
@ -52,7 +52,7 @@ private[akka] abstract class SourceModule[+Out, +Mat](val shape: SourceShape[Out
|
|||
* Holds a `Subscriber` representing the input side of the flow.
|
||||
* The `Subscriber` can later be connected to an upstream `Publisher`.
|
||||
*/
|
||||
private[akka] final class SubscriberSource[Out](val attributes: OperationAttributes, shape: SourceShape[Out]) extends SourceModule[Out, Subscriber[Out]](shape) {
|
||||
private[akka] final class SubscriberSource[Out](val attributes: Attributes, shape: SourceShape[Out]) extends SourceModule[Out, Subscriber[Out]](shape) {
|
||||
|
||||
override def create(context: MaterializationContext): (Publisher[Out], Subscriber[Out]) = {
|
||||
val processor = new VirtualProcessor[Out]
|
||||
|
|
@ -60,7 +60,7 @@ private[akka] final class SubscriberSource[Out](val attributes: OperationAttribu
|
|||
}
|
||||
|
||||
override protected def newInstance(shape: SourceShape[Out]): SourceModule[Out, Subscriber[Out]] = new SubscriberSource[Out](attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module = new SubscriberSource[Out](attr, amendShape(attr))
|
||||
override def withAttributes(attr: Attributes): Module = new SubscriberSource[Out](attr, amendShape(attr))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -70,17 +70,17 @@ private[akka] final class SubscriberSource[Out](val attributes: OperationAttribu
|
|||
* that mediate the flow of elements downstream and the propagation of
|
||||
* back-pressure upstream.
|
||||
*/
|
||||
private[akka] final class PublisherSource[Out](p: Publisher[Out], val attributes: OperationAttributes, shape: SourceShape[Out]) extends SourceModule[Out, Unit](shape) {
|
||||
private[akka] final class PublisherSource[Out](p: Publisher[Out], val attributes: Attributes, shape: SourceShape[Out]) extends SourceModule[Out, Unit](shape) {
|
||||
override def create(context: MaterializationContext) = (p, ())
|
||||
|
||||
override protected def newInstance(shape: SourceShape[Out]): SourceModule[Out, Unit] = new PublisherSource[Out](p, attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module = new PublisherSource[Out](p, attr, amendShape(attr))
|
||||
override def withAttributes(attr: Attributes): Module = new PublisherSource[Out](p, attr, amendShape(attr))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final class LazyEmptySource[Out](val attributes: OperationAttributes, shape: SourceShape[Out]) extends SourceModule[Out, Promise[Unit]](shape) {
|
||||
private[akka] final class LazyEmptySource[Out](val attributes: Attributes, shape: SourceShape[Out]) extends SourceModule[Out, Promise[Unit]](shape) {
|
||||
import ReactiveStreamsCompliance._
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
|
|
@ -104,7 +104,7 @@ private[akka] final class LazyEmptySource[Out](val attributes: OperationAttribut
|
|||
}
|
||||
|
||||
override protected def newInstance(shape: SourceShape[Out]): SourceModule[Out, Promise[Unit]] = new LazyEmptySource[Out](attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module = new LazyEmptySource(attr, amendShape(attr))
|
||||
override def withAttributes(attr: Attributes): Module = new LazyEmptySource(attr, amendShape(attr))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -115,7 +115,7 @@ private[akka] final class LazyEmptySource[Out](val attributes: OperationAttribut
|
|||
* element is produced it will not receive that tick element later. It will
|
||||
* receive new tick elements as soon as it has requested more elements.
|
||||
*/
|
||||
private[akka] final class TickSource[Out](initialDelay: FiniteDuration, interval: FiniteDuration, tick: Out, val attributes: OperationAttributes, shape: SourceShape[Out]) extends SourceModule[Out, Cancellable](shape) {
|
||||
private[akka] final class TickSource[Out](initialDelay: FiniteDuration, interval: FiniteDuration, tick: Out, val attributes: Attributes, shape: SourceShape[Out]) extends SourceModule[Out, Cancellable](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
val cancelled = new AtomicBoolean(false)
|
||||
|
|
@ -133,7 +133,7 @@ private[akka] final class TickSource[Out](initialDelay: FiniteDuration, interval
|
|||
}
|
||||
|
||||
override protected def newInstance(shape: SourceShape[Out]): SourceModule[Out, Cancellable] = new TickSource[Out](initialDelay, interval, tick, attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module = new TickSource(initialDelay, interval, tick, attr, amendShape(attr))
|
||||
override def withAttributes(attr: Attributes): Module = new TickSource(initialDelay, interval, tick, attr, amendShape(attr))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -141,7 +141,7 @@ private[akka] final class TickSource[Out](initialDelay: FiniteDuration, interval
|
|||
* Creates and wraps an actor into [[org.reactivestreams.Publisher]] from the given `props`,
|
||||
* which should be [[akka.actor.Props]] for an [[akka.stream.actor.ActorPublisher]].
|
||||
*/
|
||||
private[akka] final class ActorPublisherSource[Out](props: Props, val attributes: OperationAttributes, shape: SourceShape[Out]) extends SourceModule[Out, ActorRef](shape) {
|
||||
private[akka] final class ActorPublisherSource[Out](props: Props, val attributes: Attributes, shape: SourceShape[Out]) extends SourceModule[Out, ActorRef](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
val publisherRef = ActorFlowMaterializer.downcast(context.materializer).actorOf(context, props)
|
||||
|
|
@ -149,14 +149,14 @@ private[akka] final class ActorPublisherSource[Out](props: Props, val attributes
|
|||
}
|
||||
|
||||
override protected def newInstance(shape: SourceShape[Out]): SourceModule[Out, ActorRef] = new ActorPublisherSource[Out](props, attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module = new ActorPublisherSource(props, attr, amendShape(attr))
|
||||
override def withAttributes(attr: Attributes): Module = new ActorPublisherSource(props, attr, amendShape(attr))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final class ActorRefSource[Out](
|
||||
bufferSize: Int, overflowStrategy: OverflowStrategy, val attributes: OperationAttributes, shape: SourceShape[Out])
|
||||
bufferSize: Int, overflowStrategy: OverflowStrategy, val attributes: Attributes, shape: SourceShape[Out])
|
||||
extends SourceModule[Out, ActorRef](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
|
|
@ -167,6 +167,6 @@ private[akka] final class ActorRefSource[Out](
|
|||
|
||||
override protected def newInstance(shape: SourceShape[Out]): SourceModule[Out, ActorRef] =
|
||||
new ActorRefSource[Out](bufferSize, overflowStrategy, attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module =
|
||||
override def withAttributes(attr: Attributes): Module =
|
||||
new ActorRefSource(bufferSize, overflowStrategy, attr, amendShape(attr))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,9 +6,9 @@ package akka.stream.impl
|
|||
import java.io.File
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import akka.actor.{ Deploy, ActorRef, Props }
|
||||
import akka.stream.ActorOperationAttributes.Dispatcher
|
||||
import akka.stream.ActorAttributes.Dispatcher
|
||||
import akka.stream.impl.StreamLayout.Module
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.Attributes
|
||||
import akka.stream.{ Inlet, Shape, SinkShape }
|
||||
import akka.util.ByteString
|
||||
import org.reactivestreams.{ Publisher, Subscriber, Subscription }
|
||||
|
|
@ -39,7 +39,7 @@ private[akka] abstract class SinkModule[-In, Mat](val shape: SinkShape[In]) exte
|
|||
|
||||
override def subModules: Set[Module] = Set.empty
|
||||
|
||||
def amendShape(attr: OperationAttributes): SinkShape[In] = {
|
||||
def amendShape(attr: Attributes): SinkShape[In] = {
|
||||
attr.nameOption match {
|
||||
case None ⇒ shape
|
||||
case s: Some[String] if s == attributes.nameOption ⇒ shape
|
||||
|
|
@ -55,7 +55,7 @@ private[akka] abstract class SinkModule[-In, Mat](val shape: SinkShape[In]) exte
|
|||
* elements to fill the internal buffers it will assert back-pressure until
|
||||
* a subscriber connects and creates demand for elements to be emitted.
|
||||
*/
|
||||
private[akka] class PublisherSink[In](val attributes: OperationAttributes, shape: SinkShape[In]) extends SinkModule[In, Publisher[In]](shape) {
|
||||
private[akka] class PublisherSink[In](val attributes: Attributes, shape: SinkShape[In]) extends SinkModule[In, Publisher[In]](shape) {
|
||||
|
||||
override def toString: String = "PublisherSink"
|
||||
|
||||
|
|
@ -65,7 +65,7 @@ private[akka] class PublisherSink[In](val attributes: OperationAttributes, shape
|
|||
}
|
||||
|
||||
override protected def newInstance(shape: SinkShape[In]): SinkModule[In, Publisher[In]] = new PublisherSink[In](attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module = new PublisherSink[In](attr, amendShape(attr))
|
||||
override def withAttributes(attr: Attributes): Module = new PublisherSink[In](attr, amendShape(attr))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -74,7 +74,7 @@ private[akka] class PublisherSink[In](val attributes: OperationAttributes, shape
|
|||
private[akka] final class FanoutPublisherSink[In](
|
||||
initialBufferSize: Int,
|
||||
maximumBufferSize: Int,
|
||||
val attributes: OperationAttributes,
|
||||
val attributes: Attributes,
|
||||
shape: SinkShape[In])
|
||||
extends SinkModule[In, Publisher[In]](shape) {
|
||||
|
||||
|
|
@ -90,7 +90,7 @@ private[akka] final class FanoutPublisherSink[In](
|
|||
override protected def newInstance(shape: SinkShape[In]): SinkModule[In, Publisher[In]] =
|
||||
new FanoutPublisherSink[In](initialBufferSize, maximumBufferSize, attributes, shape)
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Module =
|
||||
override def withAttributes(attr: Attributes): Module =
|
||||
new FanoutPublisherSink[In](initialBufferSize, maximumBufferSize, attr, amendShape(attr))
|
||||
}
|
||||
|
||||
|
|
@ -130,7 +130,7 @@ private[akka] object HeadSink {
|
|||
* the Future into the corresponding failed state) or the end-of-stream
|
||||
* (failing the Future with a NoSuchElementException).
|
||||
*/
|
||||
private[akka] class HeadSink[In](val attributes: OperationAttributes, shape: SinkShape[In]) extends SinkModule[In, Future[In]](shape) {
|
||||
private[akka] class HeadSink[In](val attributes: Attributes, shape: SinkShape[In]) extends SinkModule[In, Future[In]](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
val p = Promise[In]()
|
||||
|
|
@ -139,7 +139,7 @@ private[akka] class HeadSink[In](val attributes: OperationAttributes, shape: Sin
|
|||
}
|
||||
|
||||
override protected def newInstance(shape: SinkShape[In]): SinkModule[In, Future[In]] = new HeadSink[In](attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module = new HeadSink[In](attr, amendShape(attr))
|
||||
override def withAttributes(attr: Attributes): Module = new HeadSink[In](attr, amendShape(attr))
|
||||
|
||||
override def toString: String = "HeadSink"
|
||||
}
|
||||
|
|
@ -149,7 +149,7 @@ private[akka] class HeadSink[In](val attributes: OperationAttributes, shape: Sin
|
|||
* Attaches a subscriber to this stream which will just discard all received
|
||||
* elements.
|
||||
*/
|
||||
private[akka] final class BlackholeSink(val attributes: OperationAttributes, shape: SinkShape[Any]) extends SinkModule[Any, Future[Unit]](shape) {
|
||||
private[akka] final class BlackholeSink(val attributes: Attributes, shape: SinkShape[Any]) extends SinkModule[Any, Future[Unit]](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
val effectiveSettings = ActorFlowMaterializer.downcast(context.materializer).effectiveSettings(context.effectiveAttributes)
|
||||
|
|
@ -158,26 +158,26 @@ private[akka] final class BlackholeSink(val attributes: OperationAttributes, sha
|
|||
}
|
||||
|
||||
override protected def newInstance(shape: SinkShape[Any]): SinkModule[Any, Future[Unit]] = new BlackholeSink(attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module = new BlackholeSink(attr, amendShape(attr))
|
||||
override def withAttributes(attr: Attributes): Module = new BlackholeSink(attr, amendShape(attr))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* Attaches a subscriber to this stream.
|
||||
*/
|
||||
private[akka] final class SubscriberSink[In](subscriber: Subscriber[In], val attributes: OperationAttributes, shape: SinkShape[In]) extends SinkModule[In, Unit](shape) {
|
||||
private[akka] final class SubscriberSink[In](subscriber: Subscriber[In], val attributes: Attributes, shape: SinkShape[In]) extends SinkModule[In, Unit](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = (subscriber, ())
|
||||
|
||||
override protected def newInstance(shape: SinkShape[In]): SinkModule[In, Unit] = new SubscriberSink[In](subscriber, attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module = new SubscriberSink[In](subscriber, attr, amendShape(attr))
|
||||
override def withAttributes(attr: Attributes): Module = new SubscriberSink[In](subscriber, attr, amendShape(attr))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* A sink that immediately cancels its upstream upon materialization.
|
||||
*/
|
||||
private[akka] final class CancelSink(val attributes: OperationAttributes, shape: SinkShape[Any]) extends SinkModule[Any, Unit](shape) {
|
||||
private[akka] final class CancelSink(val attributes: Attributes, shape: SinkShape[Any]) extends SinkModule[Any, Unit](shape) {
|
||||
|
||||
override def create(context: MaterializationContext): (Subscriber[Any], Unit) = {
|
||||
val subscriber = new Subscriber[Any] {
|
||||
|
|
@ -190,7 +190,7 @@ private[akka] final class CancelSink(val attributes: OperationAttributes, shape:
|
|||
}
|
||||
|
||||
override protected def newInstance(shape: SinkShape[Any]): SinkModule[Any, Unit] = new CancelSink(attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module = new CancelSink(attr, amendShape(attr))
|
||||
override def withAttributes(attr: Attributes): Module = new CancelSink(attr, amendShape(attr))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -198,7 +198,7 @@ private[akka] final class CancelSink(val attributes: OperationAttributes, shape:
|
|||
* Creates and wraps an actor into [[org.reactivestreams.Subscriber]] from the given `props`,
|
||||
* which should be [[akka.actor.Props]] for an [[akka.stream.actor.ActorSubscriber]].
|
||||
*/
|
||||
private[akka] final class ActorSubscriberSink[In](props: Props, val attributes: OperationAttributes, shape: SinkShape[In]) extends SinkModule[In, ActorRef](shape) {
|
||||
private[akka] final class ActorSubscriberSink[In](props: Props, val attributes: Attributes, shape: SinkShape[In]) extends SinkModule[In, ActorRef](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
val subscriberRef = ActorFlowMaterializer.downcast(context.materializer).actorOf(context, props)
|
||||
|
|
@ -206,14 +206,14 @@ private[akka] final class ActorSubscriberSink[In](props: Props, val attributes:
|
|||
}
|
||||
|
||||
override protected def newInstance(shape: SinkShape[In]): SinkModule[In, ActorRef] = new ActorSubscriberSink[In](props, attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module = new ActorSubscriberSink[In](props, attr, amendShape(attr))
|
||||
override def withAttributes(attr: Attributes): Module = new ActorSubscriberSink[In](props, attr, amendShape(attr))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final class ActorRefSink[In](ref: ActorRef, onCompleteMessage: Any,
|
||||
val attributes: OperationAttributes,
|
||||
val attributes: Attributes,
|
||||
shape: SinkShape[In]) extends SinkModule[In, Unit](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
|
|
@ -226,7 +226,7 @@ private[akka] final class ActorRefSink[In](ref: ActorRef, onCompleteMessage: Any
|
|||
|
||||
override protected def newInstance(shape: SinkShape[In]): SinkModule[In, Unit] =
|
||||
new ActorRefSink[In](ref, onCompleteMessage, attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module =
|
||||
override def withAttributes(attr: Attributes): Module =
|
||||
new ActorRefSink[In](ref, onCompleteMessage, attr, amendShape(attr))
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -6,8 +6,8 @@ package akka.stream.impl
|
|||
import akka.event.{ LoggingAdapter, Logging }
|
||||
import akka.stream.impl.SplitDecision.SplitDecision
|
||||
import akka.stream.{ OverflowStrategy, TimerTransformer }
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.OperationAttributes._
|
||||
import akka.stream.Attributes
|
||||
import akka.stream.Attributes._
|
||||
import akka.stream.stage.Stage
|
||||
import org.reactivestreams.Processor
|
||||
import StreamLayout._
|
||||
|
|
@ -91,32 +91,32 @@ private[stream] object Stages {
|
|||
|
||||
sealed trait StageModule extends FlowModule[Any, Any, Any] {
|
||||
|
||||
def attributes: OperationAttributes
|
||||
def withAttributes(attributes: OperationAttributes): StageModule
|
||||
def attributes: Attributes
|
||||
def withAttributes(attributes: Attributes): StageModule
|
||||
|
||||
protected def newInstance: StageModule
|
||||
override def carbonCopy: Module = newInstance
|
||||
}
|
||||
|
||||
final case class TimerTransform(mkStage: () ⇒ TimerTransformer[Any, Any], attributes: OperationAttributes = timerTransform) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class TimerTransform(mkStage: () ⇒ TimerTransformer[Any, Any], attributes: Attributes = timerTransform) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class StageFactory(mkStage: () ⇒ Stage[_, _], attributes: OperationAttributes = stageFactory) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class StageFactory(mkStage: () ⇒ Stage[_, _], attributes: Attributes = stageFactory) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class MaterializingStageFactory(
|
||||
mkStageAndMaterialized: () ⇒ (Stage[_, _], Any),
|
||||
attributes: OperationAttributes = stageFactory) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
attributes: Attributes = stageFactory) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class Identity(attributes: OperationAttributes = OperationAttributes.name("identity")) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class Identity(attributes: Attributes = Attributes.name("identity")) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
|
|
@ -125,114 +125,114 @@ private[stream] object Stages {
|
|||
Fused(ops, name(ops.iterator.map(x ⇒ Logging.simpleName(x).toLowerCase).mkString("+")))
|
||||
}
|
||||
|
||||
final case class Fused(ops: immutable.Seq[Stage[_, _]], attributes: OperationAttributes = fused) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class Fused(ops: immutable.Seq[Stage[_, _]], attributes: Attributes = fused) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class Map(f: Any ⇒ Any, attributes: OperationAttributes = map) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class Map(f: Any ⇒ Any, attributes: Attributes = map) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class Log(name: String, extract: Any ⇒ Any, loggingAdapter: Option[LoggingAdapter], attributes: OperationAttributes = map) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class Log(name: String, extract: Any ⇒ Any, loggingAdapter: Option[LoggingAdapter], attributes: Attributes = map) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class Filter(p: Any ⇒ Boolean, attributes: OperationAttributes = filter) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class Filter(p: Any ⇒ Boolean, attributes: Attributes = filter) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class Collect(pf: PartialFunction[Any, Any], attributes: OperationAttributes = collect) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class Collect(pf: PartialFunction[Any, Any], attributes: Attributes = collect) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class MapAsync(parallelism: Int, f: Any ⇒ Future[Any], attributes: OperationAttributes = mapAsync) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class MapAsync(parallelism: Int, f: Any ⇒ Future[Any], attributes: Attributes = mapAsync) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class MapAsyncUnordered(parallelism: Int, f: Any ⇒ Future[Any], attributes: OperationAttributes = mapAsyncUnordered) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class MapAsyncUnordered(parallelism: Int, f: Any ⇒ Future[Any], attributes: Attributes = mapAsyncUnordered) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class Grouped(n: Int, attributes: OperationAttributes = grouped) extends StageModule {
|
||||
final case class Grouped(n: Int, attributes: Attributes = grouped) extends StageModule {
|
||||
require(n > 0, "n must be greater than 0")
|
||||
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class Take(n: Long, attributes: OperationAttributes = take) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class Take(n: Long, attributes: Attributes = take) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class Drop(n: Long, attributes: OperationAttributes = drop) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class Drop(n: Long, attributes: Attributes = drop) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class TakeWhile(p: Any ⇒ Boolean, attributes: OperationAttributes = takeWhile) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class TakeWhile(p: Any ⇒ Boolean, attributes: Attributes = takeWhile) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class DropWhile(p: Any ⇒ Boolean, attributes: OperationAttributes = dropWhile) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class DropWhile(p: Any ⇒ Boolean, attributes: Attributes = dropWhile) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class Scan(zero: Any, f: (Any, Any) ⇒ Any, attributes: OperationAttributes = scan) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class Scan(zero: Any, f: (Any, Any) ⇒ Any, attributes: Attributes = scan) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class Buffer(size: Int, overflowStrategy: OverflowStrategy, attributes: OperationAttributes = buffer) extends StageModule {
|
||||
final case class Buffer(size: Int, overflowStrategy: OverflowStrategy, attributes: Attributes = buffer) extends StageModule {
|
||||
require(size > 0, s"Buffer size must be larger than zero but was [$size]")
|
||||
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
final case class Conflate(seed: Any ⇒ Any, aggregate: (Any, Any) ⇒ Any, attributes: OperationAttributes = conflate) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class Conflate(seed: Any ⇒ Any, aggregate: (Any, Any) ⇒ Any, attributes: Attributes = conflate) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
final case class Expand(seed: Any ⇒ Any, extrapolate: Any ⇒ (Any, Any), attributes: OperationAttributes = expand) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class Expand(seed: Any ⇒ Any, extrapolate: Any ⇒ (Any, Any), attributes: Attributes = expand) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
final case class MapConcat(f: Any ⇒ immutable.Iterable[Any], attributes: OperationAttributes = mapConcat) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class MapConcat(f: Any ⇒ immutable.Iterable[Any], attributes: Attributes = mapConcat) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class GroupBy(f: Any ⇒ Any, attributes: OperationAttributes = groupBy) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class GroupBy(f: Any ⇒ Any, attributes: Attributes = groupBy) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class PrefixAndTail(n: Int, attributes: OperationAttributes = prefixAndTail) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class PrefixAndTail(n: Int, attributes: Attributes = prefixAndTail) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class Split(p: Any ⇒ SplitDecision, attributes: OperationAttributes = split) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class Split(p: Any ⇒ SplitDecision, attributes: Attributes = split) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class ConcatAll(attributes: OperationAttributes = concatAll) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class ConcatAll(attributes: Attributes = concatAll) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
final case class DirectProcessor(p: () ⇒ (Processor[Any, Any], Any), attributes: OperationAttributes = processor) extends StageModule {
|
||||
def withAttributes(attributes: OperationAttributes) = copy(attributes = attributes)
|
||||
final case class DirectProcessor(p: () ⇒ (Processor[Any, Any], Any), attributes: Attributes = processor) extends StageModule {
|
||||
def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override protected def newInstance: StageModule = this.copy()
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -132,7 +132,7 @@ private[akka] object StreamLayout {
|
|||
* materialized value computation of its submodules.
|
||||
*/
|
||||
Atomic(this),
|
||||
OperationAttributes.none)
|
||||
Attributes.none)
|
||||
}
|
||||
|
||||
def subModules: Set[Module]
|
||||
|
|
@ -144,8 +144,8 @@ private[akka] object StreamLayout {
|
|||
def materializedValueComputation: MaterializedValueNode = Atomic(this)
|
||||
def carbonCopy: Module
|
||||
|
||||
def attributes: OperationAttributes
|
||||
def withAttributes(attributes: OperationAttributes): Module
|
||||
def attributes: Attributes
|
||||
def withAttributes(attributes: Attributes): Module
|
||||
|
||||
final override def hashCode(): Int = super.hashCode()
|
||||
final override def equals(obj: scala.Any): Boolean = super.equals(obj)
|
||||
|
|
@ -234,9 +234,9 @@ private[akka] object StreamLayout {
|
|||
|
||||
override def subModules: Set[Module] = Set.empty
|
||||
|
||||
override def withAttributes(attributes: OperationAttributes): Module =
|
||||
override def withAttributes(attributes: Attributes): Module =
|
||||
throw new UnsupportedOperationException("EmptyModule cannot carry attributes")
|
||||
override def attributes = OperationAttributes.none
|
||||
override def attributes = Attributes.none
|
||||
|
||||
override def carbonCopy: Module = this
|
||||
|
||||
|
|
@ -245,10 +245,10 @@ private[akka] object StreamLayout {
|
|||
override def materializedValueComputation: MaterializedValueNode = Ignore
|
||||
}
|
||||
|
||||
final case class CopiedModule(shape: Shape, attributes: OperationAttributes, copyOf: Module) extends Module {
|
||||
final case class CopiedModule(shape: Shape, attributes: Attributes, copyOf: Module) extends Module {
|
||||
override val subModules: Set[Module] = Set(copyOf)
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Module = this.copy(attributes = attr)
|
||||
override def withAttributes(attr: Attributes): Module = this.copy(attributes = attr)
|
||||
|
||||
override def carbonCopy: Module = this.copy(shape = shape.deepCopy())
|
||||
|
||||
|
|
@ -270,7 +270,7 @@ private[akka] object StreamLayout {
|
|||
override val downstreams: Map[OutPort, InPort],
|
||||
override val upstreams: Map[InPort, OutPort],
|
||||
override val materializedValueComputation: MaterializedValueNode,
|
||||
attributes: OperationAttributes) extends Module {
|
||||
attributes: Attributes) extends Module {
|
||||
|
||||
override def replaceShape(s: Shape): Module = {
|
||||
shape.requireSamePortsAs(s)
|
||||
|
|
@ -279,7 +279,7 @@ private[akka] object StreamLayout {
|
|||
|
||||
override def carbonCopy: Module = CopiedModule(shape.deepCopy(), attributes, copyOf = this)
|
||||
|
||||
override def withAttributes(attributes: OperationAttributes): Module = copy(attributes = attributes)
|
||||
override def withAttributes(attributes: Attributes): Module = copy(attributes = attributes)
|
||||
|
||||
override def toString =
|
||||
s"""
|
||||
|
|
@ -404,17 +404,17 @@ private[stream] final class VirtualProcessor[T] extends Processor[T, T] {
|
|||
*/
|
||||
private[stream] final case class MaterializedValueSource[M](
|
||||
shape: SourceShape[M] = SourceShape[M](new Outlet[M]("Materialized.out")),
|
||||
attributes: OperationAttributes = OperationAttributes.name("Materialized")) extends StreamLayout.Module {
|
||||
attributes: Attributes = Attributes.name("Materialized")) extends StreamLayout.Module {
|
||||
|
||||
override def subModules: Set[Module] = Set.empty
|
||||
override def withAttributes(attr: OperationAttributes): Module = this.copy(shape = amendShape(attr), attributes = attr)
|
||||
override def withAttributes(attr: Attributes): Module = this.copy(shape = amendShape(attr), attributes = attr)
|
||||
override def carbonCopy: Module = this.copy(shape = SourceShape(new Outlet[M]("Materialized.out")))
|
||||
|
||||
override def replaceShape(s: Shape): Module =
|
||||
if (s == shape) this
|
||||
else throw new UnsupportedOperationException("cannot replace the shape of MaterializedValueSource")
|
||||
|
||||
def amendShape(attr: OperationAttributes): SourceShape[M] = {
|
||||
def amendShape(attr: Attributes): SourceShape[M] = {
|
||||
attr.nameOption match {
|
||||
case None ⇒ shape
|
||||
case s: Some[String] if s == attributes.nameOption ⇒ shape
|
||||
|
|
@ -641,10 +641,10 @@ private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Mo
|
|||
}
|
||||
}
|
||||
|
||||
protected def mergeAttributes(parent: OperationAttributes, current: OperationAttributes): OperationAttributes =
|
||||
protected def mergeAttributes(parent: Attributes, current: Attributes): Attributes =
|
||||
parent and current
|
||||
|
||||
protected def materializeModule(module: Module, effectiveAttributes: OperationAttributes): Any = {
|
||||
protected def materializeModule(module: Module, effectiveAttributes: Attributes): Any = {
|
||||
val materializedValues = collection.mutable.HashMap.empty[Module, Any]
|
||||
var materializedValuePublishers: List[MaterializedValuePublisher] = Nil
|
||||
|
||||
|
|
@ -672,11 +672,11 @@ private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Mo
|
|||
mat
|
||||
}
|
||||
|
||||
protected def materializeComposite(composite: Module, effectiveAttributes: OperationAttributes): Any = {
|
||||
protected def materializeComposite(composite: Module, effectiveAttributes: Attributes): Any = {
|
||||
materializeModule(composite, effectiveAttributes)
|
||||
}
|
||||
|
||||
protected def materializeAtomic(atomic: Module, effectiveAttributes: OperationAttributes): Any
|
||||
protected def materializeAtomic(atomic: Module, effectiveAttributes: Attributes): Any
|
||||
|
||||
private def resolveMaterialized(matNode: MaterializedValueNode, materializedValues: collection.Map[Module, Any]): Any = matNode match {
|
||||
case Atomic(m) ⇒ materializedValues(m)
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.stream.impl.fusing
|
|||
import java.util.Arrays
|
||||
import akka.actor._
|
||||
import akka.stream.impl.ReactiveStreamsCompliance._
|
||||
import akka.stream.{ AbruptTerminationException, ActorFlowMaterializerSettings, OperationAttributes, ActorFlowMaterializer }
|
||||
import akka.stream.{ AbruptTerminationException, ActorFlowMaterializerSettings, Attributes, ActorFlowMaterializer }
|
||||
import akka.stream.actor.ActorSubscriber.OnSubscribe
|
||||
import akka.stream.actor.ActorSubscriberMessage.{ OnNext, OnError, OnComplete }
|
||||
import akka.stream.impl._
|
||||
|
|
@ -321,7 +321,7 @@ private[akka] class ActorOutputBoundary(val actor: ActorRef,
|
|||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object ActorInterpreter {
|
||||
def props(settings: ActorFlowMaterializerSettings, ops: Seq[Stage[_, _]], materializer: ActorFlowMaterializer, attributes: OperationAttributes = OperationAttributes.none): Props =
|
||||
def props(settings: ActorFlowMaterializerSettings, ops: Seq[Stage[_, _]], materializer: ActorFlowMaterializer, attributes: Attributes = Attributes.none): Props =
|
||||
Props(new ActorInterpreter(settings, ops, materializer, attributes)).withDeploy(Deploy.local)
|
||||
|
||||
case class AsyncInput(op: AsyncStage[Any, Any, Any], ctx: AsyncContext[Any, Any], event: Any) extends DeadLetterSuppression with NoSerializationVerificationNeeded
|
||||
|
|
@ -330,7 +330,7 @@ private[akka] object ActorInterpreter {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class ActorInterpreter(val settings: ActorFlowMaterializerSettings, val ops: Seq[Stage[_, _]], val materializer: ActorFlowMaterializer, val attributes: OperationAttributes)
|
||||
private[akka] class ActorInterpreter(val settings: ActorFlowMaterializerSettings, val ops: Seq[Stage[_, _]], val materializer: ActorFlowMaterializer, val attributes: Attributes)
|
||||
extends Actor with ActorLogging {
|
||||
import ActorInterpreter._
|
||||
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.stream.impl.fusing
|
|||
import akka.event.LoggingAdapter
|
||||
import akka.stream.impl.ReactiveStreamsCompliance
|
||||
import akka.stream.stage._
|
||||
import akka.stream.{ FlowMaterializer, OperationAttributes, Supervision }
|
||||
import akka.stream.{ FlowMaterializer, Attributes, Supervision }
|
||||
|
||||
import scala.annotation.{ switch, tailrec }
|
||||
import scala.collection.{ breakOut, immutable }
|
||||
|
|
@ -154,7 +154,7 @@ private[akka] class OneBoundedInterpreter(ops: Seq[Stage[_, _]],
|
|||
onAsyncInput: (AsyncStage[Any, Any, Any], AsyncContext[Any, Any], Any) ⇒ Unit,
|
||||
log: LoggingAdapter,
|
||||
materializer: FlowMaterializer,
|
||||
attributes: OperationAttributes = OperationAttributes.none,
|
||||
attributes: Attributes = Attributes.none,
|
||||
val forkLimit: Int = 100,
|
||||
val overflowToHeap: Boolean = true,
|
||||
val name: String = "") {
|
||||
|
|
@ -428,7 +428,7 @@ private[akka] class OneBoundedInterpreter(ops: Seq[Stage[_, _]],
|
|||
}
|
||||
|
||||
override def materializer: FlowMaterializer = OneBoundedInterpreter.this.materializer
|
||||
override def attributes: OperationAttributes = OneBoundedInterpreter.this.attributes
|
||||
override def attributes: Attributes = OneBoundedInterpreter.this.attributes
|
||||
}
|
||||
|
||||
private final val Pushing: State = new State {
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@ package akka.stream.impl.fusing
|
|||
|
||||
import akka.event.Logging.LogLevel
|
||||
import akka.event.{ LogSource, Logging, LoggingAdapter }
|
||||
import akka.stream.OperationAttributes.LogLevels
|
||||
import akka.stream.Attributes.LogLevels
|
||||
import akka.stream.impl.{ FixedSizeBuffer, ReactiveStreamsCompliance }
|
||||
import akka.stream.stage._
|
||||
import akka.stream.{ Supervision, _ }
|
||||
|
|
|
|||
|
|
@ -1,13 +1,13 @@
|
|||
package akka.stream.impl.io
|
||||
|
||||
import akka.stream.ActorOperationAttributes.Dispatcher
|
||||
import akka.stream.ActorAttributes.Dispatcher
|
||||
import akka.stream.{ ActorFlowMaterializer, MaterializationContext }
|
||||
|
||||
private[stream] object IOSettings {
|
||||
/** Picks default akka.stream.file-io-dispatcher or the OperationAttributes configured one */
|
||||
def fileIoDispatcher(context: MaterializationContext): String = {
|
||||
val mat = ActorFlowMaterializer.downcast(context.materializer)
|
||||
context.effectiveAttributes.attributes.collectFirst { case d: Dispatcher ⇒ d.dispatcher } getOrElse {
|
||||
context.effectiveAttributes.attributeList.collectFirst { case d: Dispatcher ⇒ d.dispatcher } getOrElse {
|
||||
mat.system.settings.config.getString("akka.stream.file-io-dispatcher")
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ import java.io.{ File, OutputStream }
|
|||
|
||||
import akka.stream.impl.SinkModule
|
||||
import akka.stream.impl.StreamLayout.Module
|
||||
import akka.stream.{ ActorFlowMaterializer, MaterializationContext, OperationAttributes, SinkShape }
|
||||
import akka.stream.{ ActorFlowMaterializer, MaterializationContext, Attributes, SinkShape }
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.concurrent.{ Future, Promise }
|
||||
|
|
@ -17,7 +17,7 @@ import scala.concurrent.{ Future, Promise }
|
|||
* Creates simple synchronous (Java 6 compatible) Sink which writes all incoming elements to the given file
|
||||
* (creating it before hand if neccessary).
|
||||
*/
|
||||
private[akka] final class SynchronousFileSink(f: File, append: Boolean, val attributes: OperationAttributes, shape: SinkShape[ByteString])
|
||||
private[akka] final class SynchronousFileSink(f: File, append: Boolean, val attributes: Attributes, shape: SinkShape[ByteString])
|
||||
extends SinkModule[ByteString, Future[Long]](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
|
|
@ -35,7 +35,7 @@ private[akka] final class SynchronousFileSink(f: File, append: Boolean, val attr
|
|||
override protected def newInstance(shape: SinkShape[ByteString]): SinkModule[ByteString, Future[Long]] =
|
||||
new SynchronousFileSink(f, append, attributes, shape)
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Module =
|
||||
override def withAttributes(attr: Attributes): Module =
|
||||
new SynchronousFileSink(f, append, attr, amendShape(attr))
|
||||
}
|
||||
|
||||
|
|
@ -44,7 +44,7 @@ private[akka] final class SynchronousFileSink(f: File, append: Boolean, val attr
|
|||
* Creates simple synchronous (Java 6 compatible) Sink which writes all incoming elements to the given file
|
||||
* (creating it before hand if neccessary).
|
||||
*/
|
||||
private[akka] final class OutputStreamSink(createOutput: () ⇒ OutputStream, val attributes: OperationAttributes, shape: SinkShape[ByteString])
|
||||
private[akka] final class OutputStreamSink(createOutput: () ⇒ OutputStream, val attributes: Attributes, shape: SinkShape[ByteString])
|
||||
extends SinkModule[ByteString, Future[Long]](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
|
|
@ -63,6 +63,6 @@ private[akka] final class OutputStreamSink(createOutput: () ⇒ OutputStream, va
|
|||
override protected def newInstance(shape: SinkShape[ByteString]): SinkModule[ByteString, Future[Long]] =
|
||||
new OutputStreamSink(createOutput, attributes, shape)
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Module =
|
||||
override def withAttributes(attr: Attributes): Module =
|
||||
new OutputStreamSink(createOutput, attr, amendShape(attr))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -17,7 +17,7 @@ import scala.concurrent.{ Future, Promise }
|
|||
* INTERNAL API
|
||||
* Creates simple synchronous (Java 6 compatible) Source backed by the given file.
|
||||
*/
|
||||
private[akka] final class SynchronousFileSource(f: File, chunkSize: Int, val attributes: OperationAttributes, shape: SourceShape[ByteString])
|
||||
private[akka] final class SynchronousFileSource(f: File, chunkSize: Int, val attributes: Attributes, shape: SourceShape[ByteString])
|
||||
extends SourceModule[ByteString, Future[Long]](shape) {
|
||||
override def create(context: MaterializationContext) = {
|
||||
val mat = ActorFlowMaterializer.downcast(context.materializer)
|
||||
|
|
@ -35,7 +35,7 @@ private[akka] final class SynchronousFileSource(f: File, chunkSize: Int, val att
|
|||
override protected def newInstance(shape: SourceShape[ByteString]): SourceModule[ByteString, Future[Long]] =
|
||||
new SynchronousFileSource(f, chunkSize, attributes, shape)
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Module =
|
||||
override def withAttributes(attr: Attributes): Module =
|
||||
new SynchronousFileSource(f, chunkSize, attr, amendShape(attr))
|
||||
}
|
||||
|
||||
|
|
@ -43,7 +43,7 @@ private[akka] final class SynchronousFileSource(f: File, chunkSize: Int, val att
|
|||
* INTERNAL API
|
||||
* Source backed by the given input stream.
|
||||
*/
|
||||
private[akka] final class InputStreamSource(createInputStream: () ⇒ InputStream, chunkSize: Int, val attributes: OperationAttributes, shape: SourceShape[ByteString])
|
||||
private[akka] final class InputStreamSource(createInputStream: () ⇒ InputStream, chunkSize: Int, val attributes: Attributes, shape: SourceShape[ByteString])
|
||||
extends SourceModule[ByteString, Future[Long]](shape) {
|
||||
override def create(context: MaterializationContext) = {
|
||||
val mat = ActorFlowMaterializer.downcast(context.materializer)
|
||||
|
|
@ -69,6 +69,6 @@ private[akka] final class InputStreamSource(createInputStream: () ⇒ InputStrea
|
|||
override protected def newInstance(shape: SourceShape[ByteString]): SourceModule[ByteString, Future[Long]] =
|
||||
new InputStreamSource(createInputStream, chunkSize, attributes, shape)
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Module =
|
||||
override def withAttributes(attr: Attributes): Module =
|
||||
new InputStreamSource(createInputStream, chunkSize, attr, amendShape(attr))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import akka.japi.function.Creator
|
|||
import akka.stream.impl.io.InputStreamSource
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.scaladsl.Source._
|
||||
import akka.stream.{ OperationAttributes, javadsl }
|
||||
import akka.stream.{ Attributes, javadsl }
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.concurrent.Future
|
||||
|
|
@ -17,7 +17,7 @@ import scala.concurrent.Future
|
|||
object InputStreamSource {
|
||||
|
||||
final val DefaultChunkSize = 8192
|
||||
final val DefaultAttributes = OperationAttributes.name("inputStreamSource")
|
||||
final val DefaultAttributes = Attributes.name("inputStreamSource")
|
||||
|
||||
/**
|
||||
* Creates a Source that will pull data out of the given input stream.
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import java.io.OutputStream
|
|||
import akka.japi.function.Creator
|
||||
import akka.stream.impl.io.OutputStreamSink
|
||||
import akka.stream.scaladsl.Sink
|
||||
import akka.stream.{ ActorOperationAttributes, OperationAttributes, javadsl }
|
||||
import akka.stream.{ ActorAttributes, Attributes, javadsl }
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.concurrent.Future
|
||||
|
|
@ -18,7 +18,7 @@ import scala.concurrent.Future
|
|||
*/
|
||||
object OutputStreamSink {
|
||||
|
||||
final val DefaultAttributes = OperationAttributes.name("outputStreamSink")
|
||||
final val DefaultAttributes = Attributes.name("outputStreamSink")
|
||||
|
||||
/**
|
||||
* Sink which writes incoming [[ByteString]]s to the given [[OutputStream]].
|
||||
|
|
@ -26,7 +26,7 @@ object OutputStreamSink {
|
|||
* Materializes a [[Future]] that will be completed with the size of the file (in bytes) at the streams completion.
|
||||
*
|
||||
* This source is backed by an Actor which will use the dedicated `akka.stream.file-io-dispatcher`,
|
||||
* unless configured otherwise by using [[ActorOperationAttributes]].
|
||||
* unless configured otherwise by using [[ActorAttributes]].
|
||||
*/
|
||||
def apply(output: () ⇒ OutputStream): Sink[ByteString, Future[Long]] =
|
||||
new Sink(new OutputStreamSink(output, DefaultAttributes, Sink.shape("OutputStreamSink")))
|
||||
|
|
|
|||
|
|
@ -65,7 +65,7 @@ object SslTls {
|
|||
*/
|
||||
def apply(sslContext: SSLContext, firstSession: NegotiateNewSession,
|
||||
role: Role, closing: Closing = IgnoreComplete): ScalaFlow =
|
||||
new scaladsl.BidiFlow(TlsModule(OperationAttributes.none, sslContext, firstSession, role, closing))
|
||||
new scaladsl.BidiFlow(TlsModule(Attributes.none, sslContext, firstSession, role, closing))
|
||||
|
||||
/**
|
||||
* Java API: create a StreamTls [[akka.stream.javadsl.BidiFlow]] in client mode. The
|
||||
|
|
@ -100,12 +100,12 @@ object SslTls {
|
|||
*/
|
||||
private[akka] case class TlsModule(plainIn: Inlet[SslTlsOutbound], plainOut: Outlet[SslTlsInbound],
|
||||
cipherIn: Inlet[ByteString], cipherOut: Outlet[ByteString],
|
||||
shape: Shape, attributes: OperationAttributes,
|
||||
shape: Shape, attributes: Attributes,
|
||||
sslContext: SSLContext, firstSession: NegotiateNewSession,
|
||||
role: Role, closing: Closing) extends Module {
|
||||
override def subModules: Set[Module] = Set.empty
|
||||
|
||||
override def withAttributes(att: OperationAttributes): Module = copy(attributes = att)
|
||||
override def withAttributes(att: Attributes): Module = copy(attributes = att)
|
||||
override def carbonCopy: Module = {
|
||||
val mod = TlsModule(attributes, sslContext, firstSession, role, closing)
|
||||
if (plainIn == shape.inlets(0)) mod
|
||||
|
|
@ -122,7 +122,7 @@ object SslTls {
|
|||
* INTERNAL API.
|
||||
*/
|
||||
private[akka] object TlsModule {
|
||||
def apply(attributes: OperationAttributes, sslContext: SSLContext, firstSession: NegotiateNewSession, role: Role, closing: Closing): TlsModule = {
|
||||
def apply(attributes: Attributes, sslContext: SSLContext, firstSession: NegotiateNewSession, role: Role, closing: Closing): TlsModule = {
|
||||
val name = attributes.nameOrDefault(s"StreamTls($role)")
|
||||
val cipherIn = new Inlet[ByteString](s"$name.cipherIn")
|
||||
val cipherOut = new Outlet[ByteString](s"$name.cipherOut")
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.stream.io
|
|||
import java.io.File
|
||||
|
||||
import akka.stream.impl.io.SynchronousFileSink
|
||||
import akka.stream.{ OperationAttributes, javadsl, ActorOperationAttributes }
|
||||
import akka.stream.{ Attributes, javadsl, ActorAttributes }
|
||||
import akka.stream.scaladsl.Sink
|
||||
import akka.util.ByteString
|
||||
|
||||
|
|
@ -17,7 +17,7 @@ import scala.concurrent.Future
|
|||
*/
|
||||
object SynchronousFileSink {
|
||||
|
||||
final val DefaultAttributes = OperationAttributes.name("synchronousFileSink")
|
||||
final val DefaultAttributes = Attributes.name("synchronousFileSink")
|
||||
|
||||
/**
|
||||
* Synchronous (Java 6 compatible) Sink that writes incoming [[ByteString]] elements to the given file.
|
||||
|
|
@ -25,7 +25,7 @@ object SynchronousFileSink {
|
|||
* Materializes a [[Future]] that will be completed with the size of the file (in bytes) at the streams completion.
|
||||
*
|
||||
* This source is backed by an Actor which will use the dedicated `akka.stream.file-io-dispatcher`,
|
||||
* unless configured otherwise by using [[ActorOperationAttributes]].
|
||||
* unless configured otherwise by using [[ActorAttributes]].
|
||||
*/
|
||||
def apply(f: File, append: Boolean = false): Sink[ByteString, Future[Long]] =
|
||||
new Sink(new SynchronousFileSink(f, append, DefaultAttributes, Sink.shape("SynchronousFileSink")))
|
||||
|
|
@ -39,7 +39,7 @@ object SynchronousFileSink {
|
|||
* Materializes a [[Future]] that will be completed with the size of the file (in bytes) at the streams completion.
|
||||
*
|
||||
* This source is backed by an Actor which will use the dedicated `akka.stream.file-io-dispatcher`,
|
||||
* unless configured otherwise by using [[ActorOperationAttributes]].
|
||||
* unless configured otherwise by using [[ActorAttributes]].
|
||||
*/
|
||||
def create(f: File): javadsl.Sink[ByteString, Future[java.lang.Long]] =
|
||||
apply(f, append = false).asJava.asInstanceOf[javadsl.Sink[ByteString, Future[java.lang.Long]]]
|
||||
|
|
@ -52,7 +52,7 @@ object SynchronousFileSink {
|
|||
* Materializes a [[Future]] that will be completed with the size of the file (in bytes) at the streams completion.
|
||||
*
|
||||
* This source is backed by an Actor which will use the dedicated `akka.stream.file-io-dispatcher`,
|
||||
* unless configured otherwise by using [[ActorOperationAttributes]].
|
||||
* unless configured otherwise by using [[ActorAttributes]].
|
||||
*/
|
||||
def appendTo(f: File): javadsl.Sink[ByteString, Future[java.lang.Long]] =
|
||||
apply(f, append = true).asInstanceOf[javadsl.Sink[ByteString, Future[java.lang.Long]]]
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.stream.io
|
|||
import java.io.File
|
||||
import akka.stream.impl.io.SynchronousFileSource
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.{ ActorOperationAttributes, OperationAttributes, javadsl }
|
||||
import akka.stream.{ ActorAttributes, Attributes, javadsl }
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.concurrent.Future
|
||||
|
|
@ -14,7 +14,7 @@ import scala.concurrent.Future
|
|||
object SynchronousFileSource {
|
||||
|
||||
final val DefaultChunkSize = 8192
|
||||
final val DefaultAttributes = OperationAttributes.name("synchronousFileSource")
|
||||
final val DefaultAttributes = Attributes.name("synchronousFileSource")
|
||||
|
||||
/**
|
||||
* Creates a synchronous (Java 6 compatible) Source from a Files contents.
|
||||
|
|
@ -22,7 +22,7 @@ object SynchronousFileSource {
|
|||
*
|
||||
* This source is backed by an Actor which will use the dedicated thread-pool base dispatcher.
|
||||
* You can configure the default dispatcher for this Source by changing the `akka.stream.file-io-dispatcher` or
|
||||
* set it for a given Source by using [[ActorOperationAttributes]].
|
||||
* set it for a given Source by using [[ActorAttributes]].
|
||||
*
|
||||
* It materializes a [[Future]] containing the number of bytes read from the source file upon completion.
|
||||
*/
|
||||
|
|
@ -36,7 +36,7 @@ object SynchronousFileSource {
|
|||
*
|
||||
* This source is backed by an Actor which will use the dedicated thread-pool base dispatcher.
|
||||
* You can configure the default dispatcher for this Source by changing the `akka.stream.file-io-dispatcher` or
|
||||
* set it for a given Source by using [[ActorOperationAttributes]].
|
||||
* set it for a given Source by using [[ActorAttributes]].
|
||||
*
|
||||
* It materializes a [[Future]] containing the number of bytes read from the source file upon completion.
|
||||
*/
|
||||
|
|
@ -49,7 +49,7 @@ object SynchronousFileSource {
|
|||
*
|
||||
* This source is backed by an Actor which will use the dedicated thread-pool base dispatcher.
|
||||
* You can configure the default dispatcher for this Source by changing the `akka.stream.file-io-dispatcher` or
|
||||
* set it for a given Source by using [[ActorOperationAttributes]].
|
||||
* set it for a given Source by using [[ActorAttributes]].
|
||||
*
|
||||
* It materializes a [[Future]] containing the number of bytes read from the source file upon completion.
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -152,6 +152,6 @@ class BidiFlow[-I1, +O1, -I2, +O2, +Mat](delegate: scaladsl.BidiFlow[I1, O1, I2,
|
|||
*/
|
||||
def reversed: BidiFlow[I2, O2, I1, O1, Mat] = new BidiFlow(delegate.reversed)
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): BidiFlow[I1, O1, I2, O2, Mat] =
|
||||
override def withAttributes(attr: Attributes): BidiFlow[I1, O1, I2, O2, Mat] =
|
||||
new BidiFlow(delegate.withAttributes(attr))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -328,7 +328,7 @@ object FlexiMerge {
|
|||
*
|
||||
* @param attributes optional attributes for this vertex
|
||||
*/
|
||||
abstract class FlexiMerge[T, Out, S <: Shape](val shape: S, val attributes: OperationAttributes) extends Graph[S, Unit] {
|
||||
abstract class FlexiMerge[T, Out, S <: Shape](val shape: S, val attributes: Attributes) extends Graph[S, Unit] {
|
||||
import FlexiMerge._
|
||||
|
||||
/**
|
||||
|
|
@ -345,9 +345,9 @@ abstract class FlexiMerge[T, Out, S <: Shape](val shape: S, val attributes: Oper
|
|||
case None ⇒ super.toString
|
||||
}
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Graph[S, Unit] =
|
||||
override def withAttributes(attr: Attributes): Graph[S, Unit] =
|
||||
throw new UnsupportedOperationException(
|
||||
"withAttributes not supported by default by FlexiMerge, subclass may override and implement it")
|
||||
|
||||
override def named(name: String): Graph[S, Unit] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): Graph[S, Unit] = withAttributes(Attributes.name(name))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -290,7 +290,7 @@ object FlexiRoute {
|
|||
*
|
||||
* @param attributes optional attributes for this vertex
|
||||
*/
|
||||
abstract class FlexiRoute[In, S <: Shape](val shape: S, val attributes: OperationAttributes) extends Graph[S, Unit] {
|
||||
abstract class FlexiRoute[In, S <: Shape](val shape: S, val attributes: Attributes) extends Graph[S, Unit] {
|
||||
import FlexiRoute._
|
||||
|
||||
/**
|
||||
|
|
@ -311,10 +311,10 @@ abstract class FlexiRoute[In, S <: Shape](val shape: S, val attributes: Operatio
|
|||
case None ⇒ super.toString
|
||||
}
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Graph[S, Unit] =
|
||||
override def withAttributes(attr: Attributes): Graph[S, Unit] =
|
||||
throw new UnsupportedOperationException(
|
||||
"withAttributes not supported by default by FlexiRoute, subclass may override and implement it")
|
||||
|
||||
override def named(name: String): Graph[S, Unit] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): Graph[S, Unit] = withAttributes(Attributes.name(name))
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -702,7 +702,7 @@ class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Graph
|
|||
def concatMat[M, M2](source: Graph[SourceShape[Out @uncheckedVariance], M], combine: function.Function2[Mat, M, M2]): javadsl.Flow[In, Out, M2] =
|
||||
new Flow(delegate.concatMat(source)(combinerToScala(combine)))
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): javadsl.Flow[In, Out, Mat] =
|
||||
override def withAttributes(attr: Attributes): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.withAttributes(attr))
|
||||
|
||||
override def named(name: String): javadsl.Flow[In, Out, Mat] =
|
||||
|
|
@ -712,7 +712,7 @@ class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Graph
|
|||
* Logs elements flowing through the stream as well as completion and erroring.
|
||||
*
|
||||
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
|
||||
* This can be adjusted according to your needs by providing a custom [[OperationAttributes.LogLevels]] atrribute on the given Flow:
|
||||
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] atrribute on the given Flow:
|
||||
*
|
||||
* The `extract` function will be applied to each element before logging, so it is possible to log only those fields
|
||||
* of a complex object flowing through this element.
|
||||
|
|
@ -734,7 +734,7 @@ class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Graph
|
|||
* Logs elements flowing through the stream as well as completion and erroring.
|
||||
*
|
||||
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
|
||||
* This can be adjusted according to your needs by providing a custom [[OperationAttributes.LogLevels]] atrribute on the given Flow:
|
||||
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] atrribute on the given Flow:
|
||||
*
|
||||
* The `extract` function will be applied to each element before logging, so it is possible to log only those fields
|
||||
* of a complex object flowing through this element.
|
||||
|
|
@ -756,7 +756,7 @@ class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Graph
|
|||
* Logs elements flowing through the stream as well as completion and erroring.
|
||||
*
|
||||
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
|
||||
* This can be adjusted according to your needs by providing a custom [[OperationAttributes.LogLevels]] atrribute on the given Flow:
|
||||
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] atrribute on the given Flow:
|
||||
*
|
||||
* Uses the given [[LoggingAdapter]] for logging.
|
||||
*
|
||||
|
|
@ -775,7 +775,7 @@ class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Graph
|
|||
* Logs elements flowing through the stream as well as completion and erroring.
|
||||
*
|
||||
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
|
||||
* This can be adjusted according to your needs by providing a custom [[OperationAttributes.LogLevels]] atrribute on the given Flow.
|
||||
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] atrribute on the given Flow.
|
||||
*
|
||||
* Uses an internally created [[LoggingAdapter]] which uses `akka.stream.Log` as it's source (use this class to configure slf4j loggers).
|
||||
*/
|
||||
|
|
@ -808,7 +808,7 @@ private[akka] class RunnableFlowAdapter[Mat](runnable: scaladsl.RunnableFlow[Mat
|
|||
new RunnableFlowAdapter(runnable.mapMaterializedValue(f.apply _))
|
||||
override def run(materializer: FlowMaterializer): Mat = runnable.run()(materializer)
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): RunnableFlow[Mat] =
|
||||
override def withAttributes(attr: Attributes): RunnableFlow[Mat] =
|
||||
new RunnableFlowAdapter(runnable.withAttributes(attr))
|
||||
|
||||
override def named(name: String): RunnableFlow[Mat] =
|
||||
|
|
|
|||
|
|
@ -165,7 +165,7 @@ class Sink[-In, +Mat](delegate: scaladsl.Sink[In, Mat]) extends Graph[SinkShape[
|
|||
def mapMaterializedValue[Mat2](f: function.Function[Mat, Mat2]): Sink[In, Mat2] =
|
||||
new Sink(delegate.mapMaterializedValue(f.apply _))
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): javadsl.Sink[In, Mat] =
|
||||
override def withAttributes(attr: Attributes): javadsl.Sink[In, Mat] =
|
||||
new Sink(delegate.withAttributes(attr))
|
||||
|
||||
override def named(name: String): javadsl.Sink[In, Mat] =
|
||||
|
|
|
|||
|
|
@ -10,7 +10,7 @@ import java.util.concurrent.Callable
|
|||
import akka.actor.{ Cancellable, ActorRef, Props }
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.japi.Util
|
||||
import akka.stream.OperationAttributes._
|
||||
import akka.stream.Attributes._
|
||||
import akka.stream._
|
||||
import akka.stream.impl.{ ActorPublisherSource, StreamLayout }
|
||||
import akka.util.ByteString
|
||||
|
|
@ -613,7 +613,7 @@ class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[Sour
|
|||
def flatten[U](strategy: FlattenStrategy[Out, U]): javadsl.Source[U, Mat] =
|
||||
new Source(delegate.flatten(strategy))
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): javadsl.Source[Out, Mat] =
|
||||
override def withAttributes(attr: Attributes): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.withAttributes(attr))
|
||||
|
||||
override def named(name: String): javadsl.Source[Out, Mat] =
|
||||
|
|
@ -623,7 +623,7 @@ class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[Sour
|
|||
* Logs elements flowing through the stream as well as completion and erroring.
|
||||
*
|
||||
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
|
||||
* This can be adjusted according to your needs by providing a custom [[OperationAttributes.LogLevels]] atrribute on the given Flow:
|
||||
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] atrribute on the given Flow:
|
||||
*
|
||||
* The `extract` function will be applied to each element before logging, so it is possible to log only those fields
|
||||
* of a complex object flowing through this element.
|
||||
|
|
@ -645,7 +645,7 @@ class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[Sour
|
|||
* Logs elements flowing through the stream as well as completion and erroring.
|
||||
*
|
||||
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
|
||||
* This can be adjusted according to your needs by providing a custom [[OperationAttributes.LogLevels]] atrribute on the given Flow:
|
||||
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] atrribute on the given Flow:
|
||||
*
|
||||
* The `extract` function will be applied to each element before logging, so it is possible to log only those fields
|
||||
* of a complex object flowing through this element.
|
||||
|
|
@ -667,7 +667,7 @@ class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[Sour
|
|||
* Logs elements flowing through the stream as well as completion and erroring.
|
||||
*
|
||||
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
|
||||
* This can be adjusted according to your needs by providing a custom [[OperationAttributes.LogLevels]] atrribute on the given Flow:
|
||||
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] atrribute on the given Flow:
|
||||
*
|
||||
* Uses the given [[LoggingAdapter]] for logging.
|
||||
*
|
||||
|
|
@ -686,7 +686,7 @@ class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[Sour
|
|||
* Logs elements flowing through the stream as well as completion and erroring.
|
||||
*
|
||||
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
|
||||
* This can be adjusted according to your needs by providing a custom [[OperationAttributes.LogLevels]] atrribute on the given Flow:
|
||||
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] atrribute on the given Flow:
|
||||
*
|
||||
* Uses an internally created [[LoggingAdapter]] which uses `akka.stream.Log` as it's source (use this class to configure slf4j loggers).
|
||||
*
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ import akka.stream.Graph
|
|||
import akka.stream.BidiShape
|
||||
import akka.stream.impl.StreamLayout.Module
|
||||
import akka.stream.FlowShape
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.Attributes
|
||||
|
||||
final class BidiFlow[-I1, +O1, -I2, +O2, +Mat](private[stream] override val module: Module) extends Graph[BidiShape[I1, O1, I2, O2], Mat] {
|
||||
override val shape = module.shape.asInstanceOf[BidiShape[I1, O1, I2, O2]]
|
||||
|
|
@ -117,11 +117,11 @@ final class BidiFlow[-I1, +O1, -I2, +O2, +Mat](private[stream] override val modu
|
|||
*/
|
||||
def reversed: BidiFlow[I2, O2, I1, O1, Mat] = new BidiFlow(module.replaceShape(shape.reversed))
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): BidiFlow[I1, O1, I2, O2, Mat] =
|
||||
override def withAttributes(attr: Attributes): BidiFlow[I1, O1, I2, O2, Mat] =
|
||||
new BidiFlow(module.withAttributes(attr).wrap())
|
||||
|
||||
override def named(name: String): BidiFlow[I1, O1, I2, O2, Mat] =
|
||||
withAttributes(OperationAttributes.name(name))
|
||||
withAttributes(Attributes.name(name))
|
||||
}
|
||||
|
||||
object BidiFlow extends BidiFlowApply {
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@
|
|||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.scaladsl.FlexiMerge.MergeLogic
|
||||
import akka.stream.{ Inlet, Shape, InPort, Graph, OperationAttributes }
|
||||
import akka.stream.{ Inlet, Shape, InPort, Graph, Attributes }
|
||||
import scala.collection.immutable
|
||||
import scala.collection.immutable.Seq
|
||||
import akka.stream.impl.StreamLayout
|
||||
|
|
@ -235,7 +235,7 @@ object FlexiMerge {
|
|||
* @param ports ports that this junction exposes
|
||||
* @param attributes optional attributes for this junction
|
||||
*/
|
||||
abstract class FlexiMerge[Out, S <: Shape](val shape: S, attributes: OperationAttributes) extends Graph[S, Unit] {
|
||||
abstract class FlexiMerge[Out, S <: Shape](val shape: S, attributes: Attributes) extends Graph[S, Unit] {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
|
|
@ -251,9 +251,9 @@ abstract class FlexiMerge[Out, S <: Shape](val shape: S, attributes: OperationAt
|
|||
case None ⇒ super.toString
|
||||
}
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Graph[S, Unit] =
|
||||
override def withAttributes(attr: Attributes): Graph[S, Unit] =
|
||||
throw new UnsupportedOperationException(
|
||||
"withAttributes not supported by default by FlexiMerge, subclass may override and implement it")
|
||||
|
||||
override def named(name: String): Graph[S, Unit] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): Graph[S, Unit] = withAttributes(Attributes.name(name))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@
|
|||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.impl.StreamLayout
|
||||
import akka.stream.{ Outlet, Shape, OutPort, Graph, OperationAttributes }
|
||||
import akka.stream.{ Outlet, Shape, OutPort, Graph, Attributes }
|
||||
import scala.collection.immutable
|
||||
import akka.stream.impl.Junctions.FlexiRouteModule
|
||||
import akka.stream.impl.Stages.DefaultAttributes
|
||||
|
|
@ -202,7 +202,7 @@ object FlexiRoute {
|
|||
* @param ports ports that this junction exposes
|
||||
* @param attributes optional attributes for this junction
|
||||
*/
|
||||
abstract class FlexiRoute[In, S <: Shape](val shape: S, attributes: OperationAttributes) extends Graph[S, Unit] {
|
||||
abstract class FlexiRoute[In, S <: Shape](val shape: S, attributes: Attributes) extends Graph[S, Unit] {
|
||||
import akka.stream.scaladsl.FlexiRoute._
|
||||
|
||||
/**
|
||||
|
|
@ -252,9 +252,9 @@ abstract class FlexiRoute[In, S <: Shape](val shape: S, attributes: OperationAtt
|
|||
case None ⇒ super.toString
|
||||
}
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Graph[S, Unit] =
|
||||
override def withAttributes(attr: Attributes): Graph[S, Unit] =
|
||||
throw new UnsupportedOperationException(
|
||||
"withAttributes not supported by default by FlexiRoute, subclass may override and implement it")
|
||||
|
||||
override def named(name: String): Graph[S, Unit] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): Graph[S, Unit] = withAttributes(Attributes.name(name))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import akka.event.LoggingAdapter
|
|||
import akka.stream.impl.Stages.{ MaterializingStageFactory, StageModule }
|
||||
import akka.stream.impl.StreamLayout.{ EmptyModule, Module }
|
||||
import akka.stream._
|
||||
import akka.stream.OperationAttributes._
|
||||
import akka.stream.Attributes._
|
||||
import akka.util.Collections.EmptyImmutableSeq
|
||||
import org.reactivestreams.Processor
|
||||
import scala.annotation.implicitNotFound
|
||||
|
|
@ -267,12 +267,12 @@ final class Flow[-In, +Out, +Mat](private[stream] override val module: Module)
|
|||
* operation has no effect on an empty Flow (because the attributes apply
|
||||
* only to the contained processing stages).
|
||||
*/
|
||||
override def withAttributes(attr: OperationAttributes): Repr[Out, Mat] = {
|
||||
override def withAttributes(attr: Attributes): Repr[Out, Mat] = {
|
||||
if (this.module eq EmptyModule) this
|
||||
else new Flow(module.withAttributes(attr).wrap())
|
||||
}
|
||||
|
||||
override def named(name: String): Repr[Out, Mat] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): Repr[Out, Mat] = withAttributes(Attributes.name(name))
|
||||
|
||||
/**
|
||||
* Connect the `Source` to this `Flow` and then connect it to the `Sink` and run it. The returned tuple contains
|
||||
|
|
@ -333,10 +333,10 @@ case class RunnableFlow[+Mat](private[stream] val module: StreamLayout.Module) e
|
|||
*/
|
||||
def run()(implicit materializer: FlowMaterializer): Mat = materializer.materialize(this)
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): RunnableFlow[Mat] =
|
||||
override def withAttributes(attr: Attributes): RunnableFlow[Mat] =
|
||||
new RunnableFlow(module.withAttributes(attr).wrap)
|
||||
|
||||
override def named(name: String): RunnableFlow[Mat] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): RunnableFlow[Mat] = withAttributes(Attributes.name(name))
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -943,7 +943,7 @@ trait FlowOps[+Out, +Mat] {
|
|||
* Logs elements flowing through the stream as well as completion and erroring.
|
||||
*
|
||||
* By default element and completion signals are logged on debug level, and errors are logged on Error level.
|
||||
* This can be adjusted according to your needs by providing a custom [[OperationAttributes.LogLevels]] atrribute on the given Flow:
|
||||
* This can be adjusted according to your needs by providing a custom [[Attributes.LogLevels]] atrribute on the given Flow:
|
||||
*
|
||||
* Uses implicit [[LoggingAdapter]] if available, otherwise uses an internally created one,
|
||||
* which uses `akka.stream.Log` as it's source (use this class to configure slf4j loggers).
|
||||
|
|
@ -959,7 +959,7 @@ trait FlowOps[+Out, +Mat] {
|
|||
def log(name: String, extract: Out ⇒ Any = _identity)(implicit log: LoggingAdapter = null): Repr[Out, Mat] =
|
||||
andThen(Stages.Log(name, extract.asInstanceOf[Any ⇒ Any], Option(log)))
|
||||
|
||||
def withAttributes(attr: OperationAttributes): Repr[Out, Mat]
|
||||
def withAttributes(attr: Attributes): Repr[Out, Mat]
|
||||
|
||||
/** INTERNAL API */
|
||||
private[scaladsl] def andThen[U](op: StageModule): Repr[U, Mat]
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import akka.stream.impl.Stages.{ MaterializingStageFactory, StageModule }
|
|||
import akka.stream.impl._
|
||||
import akka.stream.impl.StreamLayout._
|
||||
import akka.stream._
|
||||
import OperationAttributes.name
|
||||
import Attributes.name
|
||||
import scala.collection.immutable
|
||||
import scala.annotation.unchecked.uncheckedVariance
|
||||
import scala.annotation.tailrec
|
||||
|
|
@ -22,7 +22,7 @@ object Merge {
|
|||
*/
|
||||
def apply[T](inputPorts: Int): Merge[T] = {
|
||||
val shape = new UniformFanInShape[T, T](inputPorts)
|
||||
new Merge(inputPorts, shape, new MergeModule(shape, OperationAttributes.name("Merge")))
|
||||
new Merge(inputPorts, shape, new MergeModule(shape, Attributes.name("Merge")))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -44,10 +44,10 @@ class Merge[T] private (inputPorts: Int,
|
|||
private[stream] override val module: StreamLayout.Module)
|
||||
extends Graph[UniformFanInShape[T, T], Unit] {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Merge[T] =
|
||||
override def withAttributes(attr: Attributes): Merge[T] =
|
||||
new Merge(inputPorts, shape, module.withAttributes(attr).wrap())
|
||||
|
||||
override def named(name: String): Merge[T] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): Merge[T] = withAttributes(Attributes.name(name))
|
||||
}
|
||||
|
||||
object MergePreferred {
|
||||
|
|
@ -67,7 +67,7 @@ object MergePreferred {
|
|||
*/
|
||||
def apply[T](secondaryPorts: Int): MergePreferred[T] = {
|
||||
val shape = new MergePreferredShape[T](secondaryPorts, "MergePreferred")
|
||||
new MergePreferred(secondaryPorts, shape, new MergePreferredModule(shape, OperationAttributes.name("MergePreferred")))
|
||||
new MergePreferred(secondaryPorts, shape, new MergePreferredModule(shape, Attributes.name("MergePreferred")))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -93,10 +93,10 @@ class MergePreferred[T] private (secondaryPorts: Int,
|
|||
private[stream] override val module: StreamLayout.Module)
|
||||
extends Graph[MergePreferred.MergePreferredShape[T], Unit] {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): MergePreferred[T] =
|
||||
override def withAttributes(attr: Attributes): MergePreferred[T] =
|
||||
new MergePreferred(secondaryPorts, shape, module.withAttributes(attr).wrap())
|
||||
|
||||
override def named(name: String): MergePreferred[T] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): MergePreferred[T] = withAttributes(Attributes.name(name))
|
||||
}
|
||||
|
||||
object Broadcast {
|
||||
|
|
@ -107,7 +107,7 @@ object Broadcast {
|
|||
*/
|
||||
def apply[T](outputPorts: Int): Broadcast[T] = {
|
||||
val shape = new UniformFanOutShape[T, T](outputPorts)
|
||||
new Broadcast(outputPorts, shape, new BroadcastModule(shape, OperationAttributes.name("Broadcast")))
|
||||
new Broadcast(outputPorts, shape, new BroadcastModule(shape, Attributes.name("Broadcast")))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -128,10 +128,10 @@ class Broadcast[T] private (outputPorts: Int,
|
|||
private[stream] override val module: StreamLayout.Module)
|
||||
extends Graph[UniformFanOutShape[T, T], Unit] {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Broadcast[T] =
|
||||
override def withAttributes(attr: Attributes): Broadcast[T] =
|
||||
new Broadcast(outputPorts, shape, module.withAttributes(attr).wrap())
|
||||
|
||||
override def named(name: String): Broadcast[T] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): Broadcast[T] = withAttributes(Attributes.name(name))
|
||||
}
|
||||
|
||||
object Balance {
|
||||
|
|
@ -146,7 +146,7 @@ object Balance {
|
|||
def apply[T](outputPorts: Int, waitForAllDownstreams: Boolean = false): Balance[T] = {
|
||||
val shape = new UniformFanOutShape[T, T](outputPorts)
|
||||
new Balance(outputPorts, waitForAllDownstreams, shape,
|
||||
new BalanceModule(shape, waitForAllDownstreams, OperationAttributes.name("Balance")))
|
||||
new BalanceModule(shape, waitForAllDownstreams, Attributes.name("Balance")))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -171,10 +171,10 @@ class Balance[T] private (outputPorts: Int,
|
|||
private[stream] override val module: StreamLayout.Module)
|
||||
extends Graph[UniformFanOutShape[T, T], Unit] {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Balance[T] =
|
||||
override def withAttributes(attr: Attributes): Balance[T] =
|
||||
new Balance(outputPorts, waitForAllDownstreams, shape, module.withAttributes(attr).wrap())
|
||||
|
||||
override def named(name: String): Balance[T] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): Balance[T] = withAttributes(Attributes.name(name))
|
||||
}
|
||||
|
||||
object Zip {
|
||||
|
|
@ -183,7 +183,7 @@ object Zip {
|
|||
*/
|
||||
def apply[A, B](): Zip[A, B] = {
|
||||
val shape = new FanInShape2[A, B, (A, B)]("Zip")
|
||||
new Zip(shape, new ZipWith2Module[A, B, (A, B)](shape, Keep.both, OperationAttributes.name("Zip")))
|
||||
new Zip(shape, new ZipWith2Module[A, B, (A, B)](shape, Keep.both, Attributes.name("Zip")))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -204,10 +204,10 @@ class Zip[A, B] private (override val shape: FanInShape2[A, B, (A, B)],
|
|||
private[stream] override val module: StreamLayout.Module)
|
||||
extends Graph[FanInShape2[A, B, (A, B)], Unit] {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Zip[A, B] =
|
||||
override def withAttributes(attr: Attributes): Zip[A, B] =
|
||||
new Zip(shape, module.withAttributes(attr).wrap())
|
||||
|
||||
override def named(name: String): Zip[A, B] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): Zip[A, B] = withAttributes(Attributes.name(name))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -242,7 +242,7 @@ object Unzip {
|
|||
*/
|
||||
def apply[A, B](): Unzip[A, B] = {
|
||||
val shape = new FanOutShape2[(A, B), A, B]("Unzip")
|
||||
new Unzip(shape, new UnzipModule(shape, OperationAttributes.name("Unzip")))
|
||||
new Unzip(shape, new UnzipModule(shape, Attributes.name("Unzip")))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -253,10 +253,10 @@ class Unzip[A, B] private (override val shape: FanOutShape2[(A, B), A, B],
|
|||
private[stream] override val module: StreamLayout.Module)
|
||||
extends Graph[FanOutShape2[(A, B), A, B], Unit] {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Unzip[A, B] =
|
||||
override def withAttributes(attr: Attributes): Unzip[A, B] =
|
||||
new Unzip(shape, module.withAttributes(attr).wrap())
|
||||
|
||||
override def named(name: String): Unzip[A, B] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): Unzip[A, B] = withAttributes(Attributes.name(name))
|
||||
}
|
||||
|
||||
object Concat {
|
||||
|
|
@ -265,7 +265,7 @@ object Concat {
|
|||
*/
|
||||
def apply[T](): Concat[T] = {
|
||||
val shape = new UniformFanInShape[T, T](2)
|
||||
new Concat(shape, new ConcatModule(shape, OperationAttributes.name("Concat")))
|
||||
new Concat(shape, new ConcatModule(shape, Attributes.name("Concat")))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -288,10 +288,10 @@ class Concat[T] private (override val shape: UniformFanInShape[T, T],
|
|||
private[stream] override val module: StreamLayout.Module)
|
||||
extends Graph[UniformFanInShape[T, T], Unit] {
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Concat[T] =
|
||||
override def withAttributes(attr: Attributes): Concat[T] =
|
||||
new Concat(shape, module.withAttributes(attr).wrap())
|
||||
|
||||
override def named(name: String): Concat[T] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): Concat[T] = withAttributes(Attributes.name(name))
|
||||
}
|
||||
|
||||
object FlowGraph extends GraphApply {
|
||||
|
|
@ -553,7 +553,7 @@ object FlowGraph extends GraphApply {
|
|||
class PortOps[Out, Mat](val outlet: Outlet[Out], b: Builder[_]) extends FlowOps[Out, Mat] with CombinerBase[Out] {
|
||||
override type Repr[+O, +M] = PortOps[O, M] @uncheckedVariance
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Repr[Out, Mat] =
|
||||
override def withAttributes(attr: Attributes): Repr[Out, Mat] =
|
||||
throw new UnsupportedOperationException("Cannot set attributes on chained ops from a junction output port")
|
||||
|
||||
override private[scaladsl] def andThen[U](op: StageModule): Repr[U, Mat] = {
|
||||
|
|
|
|||
|
|
@ -12,8 +12,8 @@ import akka.stream.impl._
|
|||
import akka.stream.stage.Context
|
||||
import akka.stream.stage.PushStage
|
||||
import akka.stream.stage.SyncDirective
|
||||
import akka.stream.{ SinkShape, Inlet, Outlet, Graph, OperationAttributes }
|
||||
import akka.stream.OperationAttributes._
|
||||
import akka.stream.{ SinkShape, Inlet, Outlet, Graph, Attributes }
|
||||
import akka.stream.Attributes._
|
||||
import akka.stream.stage.{ TerminationDirective, Directive, Context, PushStage }
|
||||
import org.reactivestreams.{ Publisher, Subscriber }
|
||||
|
||||
|
|
@ -39,10 +39,10 @@ final class Sink[-In, +Mat](private[stream] override val module: Module)
|
|||
def mapMaterializedValue[Mat2](f: Mat ⇒ Mat2): Sink[In, Mat2] =
|
||||
new Sink(module.transformMaterializedValue(f.asInstanceOf[Any ⇒ Any]))
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Sink[In, Mat] =
|
||||
override def withAttributes(attr: Attributes): Sink[In, Mat] =
|
||||
new Sink(module.withAttributes(attr).wrap())
|
||||
|
||||
override def named(name: String): Sink[In, Mat] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): Sink[In, Mat] = withAttributes(Attributes.name(name))
|
||||
|
||||
/** Converts this Scala DSL element to it's Java DSL counterpart. */
|
||||
def asJava: javadsl.Sink[In, Mat] = new javadsl.Sink(this)
|
||||
|
|
|
|||
|
|
@ -32,7 +32,7 @@ import scala.concurrent.Promise
|
|||
import org.reactivestreams.Subscriber
|
||||
import akka.stream.stage.SyncDirective
|
||||
import akka.stream.OverflowStrategy
|
||||
import akka.stream.OperationAttributes
|
||||
import akka.stream.Attributes
|
||||
|
||||
/**
|
||||
* A `Source` is a set of stream processing steps that has one open output. It can comprise
|
||||
|
|
@ -153,10 +153,10 @@ final class Source[+Out, +Mat](private[stream] override val module: Module)
|
|||
*/
|
||||
def ++[Out2 >: Out, M](second: Graph[SourceShape[Out2], M]): Source[Out2, (Mat, M)] = concat(second)
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Repr[Out, Mat] =
|
||||
override def withAttributes(attr: Attributes): Repr[Out, Mat] =
|
||||
new Source(module.withAttributes(attr).wrap())
|
||||
|
||||
override def named(name: String): Repr[Out, Mat] = withAttributes(OperationAttributes.name(name))
|
||||
override def named(name: String): Repr[Out, Mat] = withAttributes(Attributes.name(name))
|
||||
|
||||
/** Converts this Scala DSL element to it's Java DSL counterpart. */
|
||||
def asJava: javadsl.Source[Out, Mat] = new javadsl.Source(this)
|
||||
|
|
|
|||
|
|
@ -78,7 +78,7 @@ class Tcp(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
val options: immutable.Traversable[SocketOption],
|
||||
val halfClose: Boolean,
|
||||
val idleTimeout: Duration = Duration.Inf,
|
||||
val attributes: OperationAttributes,
|
||||
val attributes: Attributes,
|
||||
_shape: SourceShape[IncomingConnection]) extends SourceModule[IncomingConnection, Future[ServerBinding]](_shape) {
|
||||
|
||||
override def create(context: MaterializationContext): (Publisher[IncomingConnection], Future[ServerBinding]) = {
|
||||
|
|
@ -112,7 +112,7 @@ class Tcp(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
|
||||
override protected def newInstance(s: SourceShape[IncomingConnection]): SourceModule[IncomingConnection, Future[ServerBinding]] =
|
||||
new BindSource(endpoint, backlog, options, halfClose, idleTimeout, attributes, shape)
|
||||
override def withAttributes(attr: OperationAttributes): Module =
|
||||
override def withAttributes(attr: Attributes): Module =
|
||||
new BindSource(endpoint, backlog, options, halfClose, idleTimeout, attr, shape)
|
||||
}
|
||||
|
||||
|
|
@ -140,7 +140,7 @@ class Tcp(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
halfClose: Boolean = false,
|
||||
idleTimeout: Duration = Duration.Inf): Source[IncomingConnection, Future[ServerBinding]] = {
|
||||
new Source(new BindSource(new InetSocketAddress(interface, port), backlog, options, halfClose, idleTimeout,
|
||||
OperationAttributes.none, SourceShape(new Outlet("BindSource.out"))))
|
||||
Attributes.none, SourceShape(new Outlet("BindSource.out"))))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@
|
|||
package akka.stream.stage
|
||||
|
||||
import akka.event.{ Logging, LogSource }
|
||||
import akka.stream.{ ActorFlowMaterializer, FlowMaterializer, OperationAttributes, Supervision }
|
||||
import akka.stream.{ ActorFlowMaterializer, FlowMaterializer, Attributes, Supervision }
|
||||
|
||||
/**
|
||||
* General interface for stream transformation.
|
||||
|
|
@ -540,7 +540,7 @@ trait LifecycleContext {
|
|||
def materializer: FlowMaterializer
|
||||
|
||||
/** Returns operation attributes associated with the this Stage */
|
||||
def attributes: OperationAttributes
|
||||
def attributes: Attributes
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue