diff --git a/akka-docs-dev/rst/scala/code/docs/stream/FlexiDocSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/FlexiDocSpec.scala index 022429c53a..f4cc54569c 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/FlexiDocSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/FlexiDocSpec.scala @@ -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 diff --git a/akka-docs-dev/rst/scala/code/docs/stream/FlowErrorDocSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/FlowErrorDocSpec.scala index 0a4ecc4fc9..c47577aa14 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/FlowErrorDocSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/FlowErrorDocSpec.scala @@ -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, diff --git a/akka-docs-dev/rst/scala/code/docs/stream/FlowGraphDocSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/FlowGraphDocSpec.scala index 9f02dc1313..6f74f9b091 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/FlowGraphDocSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/FlowGraphDocSpec.scala @@ -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 { diff --git a/akka-docs-dev/rst/scala/code/docs/stream/IntegrationDocSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/IntegrationDocSpec.scala index 0fb750d322..3806e4fcff 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/IntegrationDocSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/IntegrationDocSpec.scala @@ -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) diff --git a/akka-docs-dev/rst/scala/code/docs/stream/StreamBuffersRateSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/StreamBuffersRateSpec.scala index d8b63b350f..36ef60620a 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/StreamBuffersRateSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/StreamBuffersRateSpec.scala @@ -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 } diff --git a/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeLoggingElements.scala b/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeLoggingElements.scala index 9ca43de2d9..581d323ba6 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeLoggingElements.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeLoggingElements.scala @@ -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 diff --git a/akka-docs-dev/rst/scala/code/docs/stream/io/StreamFileDocSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/io/StreamFileDocSpec.scala index e212df54be..829b074de8 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/io/StreamFileDocSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/io/StreamFileDocSpec.scala @@ -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 } diff --git a/akka-http-core/src/main/scala/akka/http/impl/engine/client/OutgoingConnectionBlueprint.scala b/akka-http-core/src/main/scala/akka/http/impl/engine/client/OutgoingConnectionBlueprint.scala index 0efe26335f..e19abc372d 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/engine/client/OutgoingConnectionBlueprint.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/engine/client/OutgoingConnectionBlueprint.scala @@ -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]] { diff --git a/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolConductor.scala b/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolConductor.scala index 9347308fee..8b54e63b21 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolConductor.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolConductor.scala @@ -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] = diff --git a/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolSlot.scala b/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolSlot.scala index 45b5129d24..bcfbf1fc5e 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolSlot.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolSlot.scala @@ -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] = diff --git a/akka-http-core/src/main/scala/akka/http/impl/engine/server/HttpServerBluePrint.scala b/akka-http-core/src/main/scala/akka/http/impl/engine/server/HttpServerBluePrint.scala index 25b29accc8..0ed0974920 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/engine/server/HttpServerBluePrint.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/engine/server/HttpServerBluePrint.scala @@ -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)) { diff --git a/akka-http-core/src/main/scala/akka/http/impl/engine/ws/Websocket.scala b/akka-http-core/src/main/scala/akka/http/impl/engine/ws/Websocket.scala index 20f1933cbd..ea45530393 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/engine/ws/Websocket.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/engine/ws/Websocket.scala @@ -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 diff --git a/akka-http-core/src/main/scala/akka/http/impl/util/StreamUtils.scala b/akka-http-core/src/main/scala/akka/http/impl/util/StreamUtils.scala index 806597c9fe..62e84a7820 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/util/StreamUtils.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/util/StreamUtils.scala @@ -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)) { diff --git a/akka-stream-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala b/akka-stream-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala index 715868b497..97a355759b 100644 --- a/akka-stream-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala +++ b/akka-stream-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala @@ -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] { diff --git a/akka-stream-tck/src/test/scala/akka/stream/tck/MapTest.scala b/akka-stream-tck/src/test/scala/akka/stream/tck/MapTest.scala index b559c12dd1..eecf68febc 100644 --- a/akka-stream-tck/src/test/scala/akka/stream/tck/MapTest.scala +++ b/akka-stream-tck/src/test/scala/akka/stream/tck/MapTest.scala @@ -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] { diff --git a/akka-stream-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala b/akka-stream-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala index bb6259c54c..5ba0041593 100644 --- a/akka-stream-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala +++ b/akka-stream-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala @@ -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 } diff --git a/akka-stream-testkit/src/main/scala/akka/stream/testkit/StreamTestKit.scala b/akka-stream-testkit/src/main/scala/akka/stream/testkit/StreamTestKit.scala index 756c1e836d..da7b687238 100644 --- a/akka-stream-testkit/src/main/scala/akka/stream/testkit/StreamTestKit.scala +++ b/akka-stream-testkit/src/main/scala/akka/stream/testkit/StreamTestKit.scala @@ -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)) } } diff --git a/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSink.scala b/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSink.scala index dc62a462f7..ad5d7008d2 100644 --- a/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSink.scala +++ b/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSink.scala @@ -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._ diff --git a/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSource.scala b/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSource.scala index c31f2a20fe..b0de7d2ffa 100644 --- a/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSource.scala +++ b/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSource.scala @@ -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._ diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/ChainSetup.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/ChainSetup.scala index 774bd9f036..b041930b14 100644 --- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/ChainSetup.scala +++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/ChainSetup.scala @@ -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, _], diff --git a/akka-stream-tests/src/test/java/akka/stream/StreamTest.java b/akka-stream-tests/src/test/java/akka/stream/StreamTest.java index e26af67b99..e7746c39a1 100644 --- a/akka-stream-tests/src/test/java/akka/stream/StreamTest.java +++ b/akka-stream-tests/src/test/java/akka/stream/StreamTest.java @@ -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; diff --git a/akka-stream-tests/src/test/java/akka/stream/javadsl/AttributesTest.java b/akka-stream-tests/src/test/java/akka/stream/javadsl/AttributesTest.java new file mode 100644 index 0000000000..54e9d8c4ec --- /dev/null +++ b/akka-stream-tests/src/test/java/akka/stream/javadsl/AttributesTest.java @@ -0,0 +1,36 @@ +/** + * Copyright (C) 2015 Typesafe Inc. + */ +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"))); + } + +} diff --git a/akka-stream-tests/src/test/java/akka/stream/javadsl/FlexiMergeTest.java b/akka-stream-tests/src/test/java/akka/stream/javadsl/FlexiMergeTest.java index 697f53232e..f0052e33e0 100644 --- a/akka-stream-tests/src/test/java/akka/stream/javadsl/FlexiMergeTest.java +++ b/akka-stream-tests/src/test/java/akka/stream/javadsl/FlexiMergeTest.java @@ -147,7 +147,7 @@ public class FlexiMergeTest { */ static public class Fair extends FlexiMerge> { public Fair() { - super(new UniformFanInShape(2), OperationAttributes.name("Fair")); + super(new UniformFanInShape(2), Attributes.name("Fair")); } @Override public MergeLogic createMergeLogic(final UniformFanInShape s) { @@ -173,7 +173,7 @@ public class FlexiMergeTest { */ static public class StrictRoundRobin extends FlexiMerge> { public StrictRoundRobin() { - super(new UniformFanInShape(2), OperationAttributes.name("StrictRoundRobin")); + super(new UniformFanInShape(2), Attributes.name("StrictRoundRobin")); } @Override public MergeLogic createMergeLogic(final UniformFanInShape s) { @@ -240,7 +240,7 @@ public class FlexiMergeTest { static public class Zip extends FlexiMerge, FanInShape2>> { public Zip() { - super(new FanInShape2>("Zip"), OperationAttributes.name("Zip")); + super(new FanInShape2>("Zip"), Attributes.name("Zip")); } @Override public MergeLogic> createMergeLogic(final FanInShape2> s) { @@ -328,7 +328,7 @@ public class FlexiMergeTest { static public class TripleZip extends FlexiMerge, FanInShape3>> { public TripleZip() { - super(new FanInShape3>("TripleZip"), OperationAttributes.name("TripleZip")); + super(new FanInShape3>("TripleZip"), Attributes.name("TripleZip")); } @Override public MergeLogic> createMergeLogic(final FanInShape3> s) { diff --git a/akka-stream-tests/src/test/java/akka/stream/javadsl/FlexiRouteTest.java b/akka-stream-tests/src/test/java/akka/stream/javadsl/FlexiRouteTest.java index d9d4d1ee29..0bef0fe9b5 100644 --- a/akka-stream-tests/src/test/java/akka/stream/javadsl/FlexiRouteTest.java +++ b/akka-stream-tests/src/test/java/akka/stream/javadsl/FlexiRouteTest.java @@ -139,7 +139,7 @@ public class FlexiRouteTest { */ static public class Fair extends FlexiRoute> { public Fair() { - super(new UniformFanOutShape(2), OperationAttributes.name("Fair")); + super(new UniformFanOutShape(2), Attributes.name("Fair")); } @Override public RouteLogic createRouteLogic(final UniformFanOutShape s) { @@ -175,7 +175,7 @@ public class FlexiRouteTest { */ static public class StrictRoundRobin extends FlexiRoute> { public StrictRoundRobin() { - super(new UniformFanOutShape(2), OperationAttributes.name("StrictRoundRobin")); + super(new UniformFanOutShape(2), Attributes.name("StrictRoundRobin")); } @Override public RouteLogic createRouteLogic(final UniformFanOutShape s) { @@ -207,7 +207,7 @@ public class FlexiRouteTest { static public class Unzip extends FlexiRoute, FanOutShape2, A, B>> { public Unzip() { - super(new FanOutShape2, A, B>("Unzip"), OperationAttributes.name("Unzip")); + super(new FanOutShape2, A, B>("Unzip"), Attributes.name("Unzip")); } @Override public RouteLogic> createRouteLogic(final FanOutShape2, A, B> s) { diff --git a/akka-stream-tests/src/test/java/akka/stream/javadsl/OperationAttributesTest.java b/akka-stream-tests/src/test/java/akka/stream/javadsl/OperationAttributesTest.java deleted file mode 100644 index c16eae7b7d..0000000000 --- a/akka-stream-tests/src/test/java/akka/stream/javadsl/OperationAttributesTest.java +++ /dev/null @@ -1,36 +0,0 @@ -/** - * Copyright (C) 2015 Typesafe Inc. - */ -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"))); - } - -} diff --git a/akka-stream-tests/src/test/scala/akka/stream/actor/ActorPublisherSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/actor/ActorPublisherSpec.scala index 30bb0ac0bc..92f43eed9a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/actor/ActorPublisherSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/actor/ActorPublisherSpec.scala @@ -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") diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/StreamLayoutSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/StreamLayoutSpec.scala index e645e1f7d3..d6b7ebf461 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/StreamLayoutSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/StreamLayoutSpec.scala @@ -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 diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSpecKit.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSpecKit.scala index 252dd28130..aae59dd0e2 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSpecKit.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSpecKit.scala @@ -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() diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/SynchronousFileSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/SynchronousFileSinkSpec.scala index aebdaf8bd1..9754bff885 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/SynchronousFileSinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/SynchronousFileSinkSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/SynchronousFileSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/SynchronousFileSourceSpec.scala index bffbb5afbd..270d00128c 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/SynchronousFileSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/SynchronousFileSourceSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala index e9b45a8594..57361e37f0 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala @@ -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, diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala index 6b83d08f2b..68ecd3f821 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala @@ -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() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWhileSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWhileSpec.scala index 2f6552f661..24314a6970 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWhileSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWhileSpec.scala @@ -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._ diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGraphCompileSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGraphCompileSpec.scala index c65e6d9fe1..187e599143 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGraphCompileSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGraphCompileSpec.scala @@ -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._ diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala index 5a6801615d..b173079088 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala index 871822ac1d..8aaf919415 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala @@ -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)) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncSpec.scala index b2adba74d8..6fe95631a0 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncSpec.scala @@ -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 diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncUnorderedSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncUnorderedSpec.scala index 08bb4b4549..6ff3899668 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncUnorderedSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncUnorderedSpec.scala @@ -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 diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala index 3e7586188f..f5a75facf8 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala @@ -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._ diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala index e40a2c840f..c3b6882bfd 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala @@ -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 diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSectionSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSectionSpec.scala index c0bf45003c..c0c8a0b130 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSectionSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSectionSpec.scala @@ -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 diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala index d10779c9ec..a82b2c9e5c 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala @@ -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._ diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala index 6655936b85..4871c69b7a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala index 0a00fdf4b8..328e6ea78d 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowStageSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowStageSpec.scala index c39eb36068..3a7fd18c45 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowStageSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowStageSpec.scala @@ -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._ diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSupervisionSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSupervisionSpec.scala index b6ebe744a5..22962bc837 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSupervisionSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSupervisionSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWhileSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWhileSpec.scala index 9df07fc6b8..8398fa6a3b 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWhileSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWhileSpec.scala @@ -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._ diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphFlexiMergeSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphFlexiMergeSpec.scala index b7d6843413..e30e0ee940 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphFlexiMergeSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphFlexiMergeSpec.scala @@ -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 = diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphFlexiRouteSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphFlexiRouteSpec.scala index dbd61d9f9e..0dee0bc598 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphFlexiRouteSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphFlexiRouteSpec.scala @@ -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 diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphJunctionAttributesSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphJunctionAttributesSpec.scala index 267fa95289..76ebf6d4d4 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphJunctionAttributesSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphJunctionAttributesSpec.scala @@ -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._ diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala index 50fadbe6cf..0a55e8a668 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala @@ -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]] diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachParallelSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachParallelSpec.scala index 66fbb7dcfd..325890ff9a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachParallelSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachParallelSpec.scala @@ -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 diff --git a/akka-stream/src/main/boilerplate/akka/stream/impl/GenJunctions.scala.template b/akka-stream/src/main/boilerplate/akka/stream/impl/GenJunctions.scala.template index 49c224ada8..161c1784a0 100644 --- a/akka-stream/src/main/boilerplate/akka/stream/impl/GenJunctions.scala.template +++ b/akka-stream/src/main/boilerplate/akka/stream/impl/GenJunctions.scala.template @@ -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) diff --git a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/GraphApply.scala.template b/akka-stream/src/main/boilerplate/akka/stream/scaladsl/GraphApply.scala.template index 3542e789bf..b46188ee1e 100644 --- a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/GraphApply.scala.template +++ b/akka-stream/src/main/boilerplate/akka/stream/scaladsl/GraphApply.scala.template @@ -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)) } } diff --git a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/ZipWithApply.scala.template b/akka-stream/src/main/boilerplate/akka/stream/scaladsl/ZipWithApply.scala.template index aec5c994b3..037d91ac5e 100644 --- a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/ZipWithApply.scala.template +++ b/akka-stream/src/main/boilerplate/akka/stream/scaladsl/ZipWithApply.scala.template @@ -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)) } # ] diff --git a/akka-stream/src/main/scala/akka/stream/ActorFlowMaterializer.scala b/akka-stream/src/main/scala/akka/stream/ActorFlowMaterializer.scala index 4dd5c19658..be9f21013f 100644 --- a/akka-stream/src/main/scala/akka/stream/ActorFlowMaterializer.scala +++ b/akka-stream/src/main/scala/akka/stream/ActorFlowMaterializer.scala @@ -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._ diff --git a/akka-stream/src/main/scala/akka/stream/OperationAttributes.scala b/akka-stream/src/main/scala/akka/stream/Attributes.scala similarity index 72% rename from akka-stream/src/main/scala/akka/stream/OperationAttributes.scala rename to akka-stream/src/main/scala/akka/stream/Attributes.scala index dafbf1226a..103b24dcfe 100644 --- a/akka-stream/src/main/scala/akka/stream/OperationAttributes.scala +++ b/akka-stream/src/main/scala/akka/stream/Attributes.scala @@ -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)) } diff --git a/akka-stream/src/main/scala/akka/stream/FlowMaterializer.scala b/akka-stream/src/main/scala/akka/stream/FlowMaterializer.scala index 684b52fe5a..bef17bfefb 100644 --- a/akka-stream/src/main/scala/akka/stream/FlowMaterializer.scala +++ b/akka-stream/src/main/scala/akka/stream/FlowMaterializer.scala @@ -52,5 +52,5 @@ private[akka] object NoFlowMaterializer extends FlowMaterializer { */ private[akka] case class MaterializationContext( materializer: FlowMaterializer, - effectiveAttributes: OperationAttributes, + effectiveAttributes: Attributes, stageName: String) diff --git a/akka-stream/src/main/scala/akka/stream/Graph.scala b/akka-stream/src/main/scala/akka/stream/Graph.scala index 5cc15cb93d..8bc58d63a9 100644 --- a/akka-stream/src/main/scala/akka/stream/Graph.scala +++ b/akka-stream/src/main/scala/akka/stream/Graph.scala @@ -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)) } diff --git a/akka-stream/src/main/scala/akka/stream/extra/Timed.scala b/akka-stream/src/main/scala/akka/stream/extra/Timed.scala index a2968b8cad..58d2a1a806 100644 --- a/akka-stream/src/main/scala/akka/stream/extra/Timed.scala +++ b/akka-stream/src/main/scala/akka/stream/extra/Timed.scala @@ -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._ diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorFlowMaterializerImpl.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorFlowMaterializerImpl.scala index 4ad84b1337..7cb1712ef9 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/ActorFlowMaterializerImpl.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/ActorFlowMaterializerImpl.scala @@ -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! diff --git a/akka-stream/src/main/scala/akka/stream/impl/Junctions.scala b/akka-stream/src/main/scala/akka/stream/impl/Junctions.scala index 19eac25f08..56f65d1207 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/Junctions.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/Junctions.scala @@ -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) } diff --git a/akka-stream/src/main/scala/akka/stream/impl/Modules.scala b/akka-stream/src/main/scala/akka/stream/impl/Modules.scala index 48c1f4c22c..7af2fd1e2d 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/Modules.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/Modules.scala @@ -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)) } diff --git a/akka-stream/src/main/scala/akka/stream/impl/Sinks.scala b/akka-stream/src/main/scala/akka/stream/impl/Sinks.scala index 51d7a5def9..c9b5d5df18 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/Sinks.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/Sinks.scala @@ -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)) } diff --git a/akka-stream/src/main/scala/akka/stream/impl/Stages.scala b/akka-stream/src/main/scala/akka/stream/impl/Stages.scala index a503c0bf17..d0373c475f 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/Stages.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/Stages.scala @@ -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() } diff --git a/akka-stream/src/main/scala/akka/stream/impl/StreamLayout.scala b/akka-stream/src/main/scala/akka/stream/impl/StreamLayout.scala index c2cccbb051..e89f8688e3 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/StreamLayout.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/StreamLayout.scala @@ -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) diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/ActorInterpreter.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/ActorInterpreter.scala index bc53852e10..87c4ed4fd3 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/fusing/ActorInterpreter.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/ActorInterpreter.scala @@ -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._ diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/Interpreter.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/Interpreter.scala index ea4f862bb2..8acd0fb76a 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/fusing/Interpreter.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/Interpreter.scala @@ -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 { diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala index 1386ae4d74..9c37e87f4f 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala @@ -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, _ } diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/IOSettings.scala b/akka-stream/src/main/scala/akka/stream/impl/io/IOSettings.scala index 28bce437d4..191709414d 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/io/IOSettings.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/io/IOSettings.scala @@ -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") } } diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/IOSinks.scala b/akka-stream/src/main/scala/akka/stream/impl/io/IOSinks.scala index 1b1b0c8c54..e0feb63679 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/io/IOSinks.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/io/IOSinks.scala @@ -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)) } diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/IOSources.scala b/akka-stream/src/main/scala/akka/stream/impl/io/IOSources.scala index 2ef1ef5ead..0040cdee7c 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/io/IOSources.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/io/IOSources.scala @@ -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)) } diff --git a/akka-stream/src/main/scala/akka/stream/io/InputStreamSource.scala b/akka-stream/src/main/scala/akka/stream/io/InputStreamSource.scala index 81ce9bcf47..34eae30aeb 100644 --- a/akka-stream/src/main/scala/akka/stream/io/InputStreamSource.scala +++ b/akka-stream/src/main/scala/akka/stream/io/InputStreamSource.scala @@ -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. diff --git a/akka-stream/src/main/scala/akka/stream/io/OutputStreamSink.scala b/akka-stream/src/main/scala/akka/stream/io/OutputStreamSink.scala index 3df27882fd..bbe5f3f122 100644 --- a/akka-stream/src/main/scala/akka/stream/io/OutputStreamSink.scala +++ b/akka-stream/src/main/scala/akka/stream/io/OutputStreamSink.scala @@ -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"))) diff --git a/akka-stream/src/main/scala/akka/stream/io/SslTls.scala b/akka-stream/src/main/scala/akka/stream/io/SslTls.scala index 19c23bf0f3..afd762d159 100644 --- a/akka-stream/src/main/scala/akka/stream/io/SslTls.scala +++ b/akka-stream/src/main/scala/akka/stream/io/SslTls.scala @@ -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") diff --git a/akka-stream/src/main/scala/akka/stream/io/SynchronousFileSink.scala b/akka-stream/src/main/scala/akka/stream/io/SynchronousFileSink.scala index 23e5cf4540..bafcdf6f5f 100644 --- a/akka-stream/src/main/scala/akka/stream/io/SynchronousFileSink.scala +++ b/akka-stream/src/main/scala/akka/stream/io/SynchronousFileSink.scala @@ -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]]] diff --git a/akka-stream/src/main/scala/akka/stream/io/SynchronousFileSource.scala b/akka-stream/src/main/scala/akka/stream/io/SynchronousFileSource.scala index c319137b79..c66f9b124f 100644 --- a/akka-stream/src/main/scala/akka/stream/io/SynchronousFileSource.scala +++ b/akka-stream/src/main/scala/akka/stream/io/SynchronousFileSource.scala @@ -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. */ diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala b/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala index 766d3f7dab..723df5d6dd 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala @@ -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)) } diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/FlexiMerge.scala b/akka-stream/src/main/scala/akka/stream/javadsl/FlexiMerge.scala index 0509222cf0..8a29dd8328 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/FlexiMerge.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/FlexiMerge.scala @@ -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)) } diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/FlexiRoute.scala b/akka-stream/src/main/scala/akka/stream/javadsl/FlexiRoute.scala index e30d8fd0c5..c6c6c45d6a 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/FlexiRoute.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/FlexiRoute.scala @@ -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)) } diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala index eb25d765c7..e651aec45e 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala @@ -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] = diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala index 199e8eca8b..5b799b63ab 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala @@ -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] = diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala index d30a020790..5362784e90 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala @@ -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). * diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala index c59ac60c5c..37efc4787f 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala @@ -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 { diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/FlexiMerge.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/FlexiMerge.scala index c015656c36..144918cafb 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/FlexiMerge.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/FlexiMerge.scala @@ -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)) } diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/FlexiRoute.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/FlexiRoute.scala index 5d4a28ac50..3e99bedd2b 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/FlexiRoute.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/FlexiRoute.scala @@ -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)) } diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala index 55194db57f..ffec1eec2e 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala @@ -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] diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala index c509649609..18b3024340 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala @@ -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] = { diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala index 399cdfd805..1757e16d65 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala @@ -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) diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala index 02bdcc7ef0..6378cb53b1 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala @@ -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) diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Tcp.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Tcp.scala index a0aedf19b0..ffddbc49bc 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Tcp.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Tcp.scala @@ -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")))) } /** diff --git a/akka-stream/src/main/scala/akka/stream/stage/Stage.scala b/akka-stream/src/main/scala/akka/stream/stage/Stage.scala index 29cce67dbf..8bc5a2afef 100644 --- a/akka-stream/src/main/scala/akka/stream/stage/Stage.scala +++ b/akka-stream/src/main/scala/akka/stream/stage/Stage.scala @@ -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 } /**