Rename OperationAttributes to Attributes

This commit is contained in:
Endre Sándor Varga 2015-06-23 17:32:55 +02:00
parent e6f2db127c
commit dc7269e620
92 changed files with 458 additions and 462 deletions

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -0,0 +1,36 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.javadsl;
import static org.junit.Assert.assertEquals;
import java.util.Arrays;
import org.junit.Test;
import akka.stream.Attributes;
public class AttributesTest {
final Attributes attributes =
Attributes.name("a")
.and(Attributes.name("b"))
.and(Attributes.inputBuffer(1, 2));
@Test
public void mustGetAttributesByClass() {
assertEquals(
Arrays.asList(new Attributes.Name("a"), new Attributes.Name("b")),
attributes.getAttributeList(Attributes.Name.class));
assertEquals(
Arrays.asList(new Attributes.InputBuffer(1, 2)),
attributes.getAttributeList(Attributes.InputBuffer.class));
}
@Test
public void mustGetAttributeByClass() {
assertEquals(
new Attributes.Name("a"),
attributes.getAttribute(Attributes.Name.class, new Attributes.Name("default")));
}
}

View file

@ -147,7 +147,7 @@ public class FlexiMergeTest {
*/
static public class Fair<T> extends FlexiMerge<T, T, UniformFanInShape<T, T>> {
public Fair() {
super(new UniformFanInShape<T, T>(2), OperationAttributes.name("Fair"));
super(new UniformFanInShape<T, T>(2), Attributes.name("Fair"));
}
@Override
public MergeLogic<T, T> createMergeLogic(final UniformFanInShape<T, T> s) {
@ -173,7 +173,7 @@ public class FlexiMergeTest {
*/
static public class StrictRoundRobin<T> extends FlexiMerge<T, T, UniformFanInShape<T, T>> {
public StrictRoundRobin() {
super(new UniformFanInShape<T, T>(2), OperationAttributes.name("StrictRoundRobin"));
super(new UniformFanInShape<T, T>(2), Attributes.name("StrictRoundRobin"));
}
@Override
public MergeLogic<T, T> createMergeLogic(final UniformFanInShape<T, T> s) {
@ -240,7 +240,7 @@ public class FlexiMergeTest {
static public class Zip<A, B> extends FlexiMerge<A, Pair<A, B>, FanInShape2<A, B, Pair<A, B>>> {
public Zip() {
super(new FanInShape2<A, B, Pair<A, B>>("Zip"), OperationAttributes.name("Zip"));
super(new FanInShape2<A, B, Pair<A, B>>("Zip"), Attributes.name("Zip"));
}
@Override
public MergeLogic<A, Pair<A, B>> createMergeLogic(final FanInShape2<A, B, Pair<A, B>> s) {
@ -328,7 +328,7 @@ public class FlexiMergeTest {
static public class TripleZip<A, B, C> extends FlexiMerge<FlexiMerge.ReadAllInputs, Triple<A, B, C>, FanInShape3<A, B, C, Triple<A, B, C>>> {
public TripleZip() {
super(new FanInShape3<A, B, C, Triple<A, B, C>>("TripleZip"), OperationAttributes.name("TripleZip"));
super(new FanInShape3<A, B, C, Triple<A, B, C>>("TripleZip"), Attributes.name("TripleZip"));
}
@Override
public MergeLogic<ReadAllInputs, Triple<A, B, C>> createMergeLogic(final FanInShape3<A, B, C, Triple<A, B, C>> s) {

View file

@ -139,7 +139,7 @@ public class FlexiRouteTest {
*/
static public class Fair<T> extends FlexiRoute<T, UniformFanOutShape<T, T>> {
public Fair() {
super(new UniformFanOutShape<T, T>(2), OperationAttributes.name("Fair"));
super(new UniformFanOutShape<T, T>(2), Attributes.name("Fair"));
}
@Override
public RouteLogic<T> createRouteLogic(final UniformFanOutShape<T, T> s) {
@ -175,7 +175,7 @@ public class FlexiRouteTest {
*/
static public class StrictRoundRobin<T> extends FlexiRoute<T, UniformFanOutShape<T, T>> {
public StrictRoundRobin() {
super(new UniformFanOutShape<T, T>(2), OperationAttributes.name("StrictRoundRobin"));
super(new UniformFanOutShape<T, T>(2), Attributes.name("StrictRoundRobin"));
}
@Override
public RouteLogic<T> createRouteLogic(final UniformFanOutShape<T, T> s) {
@ -207,7 +207,7 @@ public class FlexiRouteTest {
static public class Unzip<A, B> extends FlexiRoute<Pair<A, B>, FanOutShape2<Pair<A, B>, A, B>> {
public Unzip() {
super(new FanOutShape2<Pair<A, B>, A, B>("Unzip"), OperationAttributes.name("Unzip"));
super(new FanOutShape2<Pair<A, B>, A, B>("Unzip"), Attributes.name("Unzip"));
}
@Override
public RouteLogic<Pair<A, B>> createRouteLogic(final FanOutShape2<Pair<A, B>, A, B> s) {

View file

@ -1,36 +0,0 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.javadsl;
import static org.junit.Assert.assertEquals;
import java.util.Arrays;
import org.junit.Test;
import akka.stream.OperationAttributes;
public class OperationAttributesTest {
final OperationAttributes attributes =
OperationAttributes.name("a")
.and(OperationAttributes.name("b"))
.and(OperationAttributes.inputBuffer(1, 2));
@Test
public void mustGetAttributesByClass() {
assertEquals(
Arrays.asList(new OperationAttributes.Name("a"), new OperationAttributes.Name("b")),
attributes.getAttributes(OperationAttributes.Name.class));
assertEquals(
Arrays.asList(new OperationAttributes.InputBuffer(1, 2)),
attributes.getAttributes(OperationAttributes.InputBuffer.class));
}
@Test
public void mustGetAttributeByClass() {
assertEquals(
new OperationAttributes.Name("a"),
attributes.getAttribute(OperationAttributes.Name.class, new OperationAttributes.Name("default")));
}
}

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -52,5 +52,5 @@ private[akka] object NoFlowMaterializer extends FlowMaterializer {
*/
private[akka] case class MaterializationContext(
materializer: FlowMaterializer,
effectiveAttributes: OperationAttributes,
effectiveAttributes: Attributes,
stageName: String)

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -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"))))
}
/**

View file

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