!str #17393: Make stream-tests pass with serialize-messages=on

This commit is contained in:
Endre Sándor Varga 2015-05-29 16:43:02 +02:00
parent 0e6694a102
commit f4c83771bb
37 changed files with 121 additions and 159 deletions

View file

@ -4,8 +4,7 @@
package akka.stream.testkit package akka.stream.testkit
import scala.language.existentials import scala.language.existentials
import akka.actor.ActorSystem import akka.actor.{ NoSerializationVerificationNeeded, ActorSystem, DeadLetterSuppression }
import akka.actor.DeadLetterSuppression
import akka.stream._ import akka.stream._
import akka.stream.impl._ import akka.stream.impl._
import akka.testkit.TestProbe import akka.testkit.TestProbe
@ -359,12 +358,12 @@ private[testkit] object StreamTestKit {
import TestPublisher._ import TestPublisher._
import TestSubscriber._ import TestSubscriber._
sealed trait PublisherEvent extends DeadLetterSuppression sealed trait PublisherEvent extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class Subscribe(subscription: Subscription) extends PublisherEvent final case class Subscribe(subscription: Subscription) extends PublisherEvent
final case class CancelSubscription(subscription: Subscription) extends PublisherEvent final case class CancelSubscription(subscription: Subscription) extends PublisherEvent
final case class RequestMore(subscription: Subscription, elements: Long) extends PublisherEvent final case class RequestMore(subscription: Subscription, elements: Long) extends PublisherEvent
sealed trait SubscriberEvent extends DeadLetterSuppression sealed trait SubscriberEvent extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class OnSubscribe(subscription: Subscription) extends SubscriberEvent final case class OnSubscribe(subscription: Subscription) extends SubscriberEvent
final case class OnNext[I](element: I) extends SubscriberEvent final case class OnNext[I](element: I) extends SubscriberEvent
final case object OnComplete extends SubscriberEvent final case object OnComplete extends SubscriberEvent

View file

@ -5,6 +5,7 @@ package akka.stream.javadsl;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.japi.Pair; import akka.japi.Pair;
import akka.pattern.Patterns;
import akka.stream.*; import akka.stream.*;
import akka.stream.javadsl.FlowGraph.Builder; import akka.stream.javadsl.FlowGraph.Builder;
import akka.stream.stage.*; import akka.stream.stage.*;
@ -223,7 +224,7 @@ public class FlowGraphTest extends StreamTest {
b.from(Source.single(1)).to(out); b.from(Source.single(1)).to(out);
b.from(b.materializedValue()).to(Sink.foreach(new Procedure<Future<Integer>>(){ b.from(b.materializedValue()).to(Sink.foreach(new Procedure<Future<Integer>>(){
public void apply(Future<Integer> mat) throws Exception { public void apply(Future<Integer> mat) throws Exception {
probe.ref().tell(mat, ActorRef.noSender()); Patterns.pipe(mat, system.dispatcher()).to(probe.ref());
} }
})); }));
} }
@ -232,10 +233,7 @@ public class FlowGraphTest extends StreamTest {
final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS)); final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS));
assertEquals(1, (int) result); assertEquals(1, (int) result);
final Future<Integer> future2 = probe.expectMsgClass(Future.class); probe.expectMsg(1);
final Integer result2 = Await.result(future2, Duration.create(300, TimeUnit.MILLISECONDS));
assertEquals(1, (int) result2);
} }
} }

View file

@ -0,0 +1,6 @@
akka {
actor {
serialize-creators = on
serialize-messages = on
}
}

View file

@ -3,7 +3,7 @@
*/ */
package akka.stream.io package akka.stream.io
import akka.actor.{ Actor, ActorRef, Props } import akka.actor._
import akka.io.Tcp.{ ResumeReading, Register, ConnectionClosed, Closed } import akka.io.Tcp.{ ResumeReading, Register, ConnectionClosed, Closed }
import akka.io.{ IO, Tcp } import akka.io.{ IO, Tcp }
import akka.stream.testkit._ import akka.stream.testkit._
@ -17,9 +17,10 @@ import akka.stream.testkit.TestUtils.temporaryServerAddress
import scala.concurrent.duration._ import scala.concurrent.duration._
object TcpHelper { object TcpHelper {
case class ClientWrite(bytes: ByteString) case class ClientWrite(bytes: ByteString) extends NoSerializationVerificationNeeded
case class ClientRead(count: Int, readTo: ActorRef) case class ClientRead(count: Int, readTo: ActorRef) extends NoSerializationVerificationNeeded
case class ClientClose(cmd: Tcp.CloseCommand) case class ClientClose(cmd: Tcp.CloseCommand) extends NoSerializationVerificationNeeded
case class ReadResult(bytes: ByteString) extends NoSerializationVerificationNeeded
// FIXME: Workaround object just to force a ResumeReading that will poll for a possibly pending close event // FIXME: Workaround object just to force a ResumeReading that will poll for a possibly pending close event
// See https://github.com/akka/akka/issues/16552 // See https://github.com/akka/akka/issues/16552
@ -69,7 +70,7 @@ object TcpHelper {
case Tcp.Received(bytes) case Tcp.Received(bytes)
readBuffer ++= bytes readBuffer ++= bytes
if (readBuffer.size >= toRead) { if (readBuffer.size >= toRead) {
readTo ! readBuffer readTo ! ReadResult(readBuffer)
readBuffer = ByteString.empty readBuffer = ByteString.empty
toRead = 0 toRead = 0
readTo = context.system.deadLetters readTo = context.system.deadLetters
@ -136,7 +137,7 @@ trait TcpHelper { this: TestKitBase ⇒
def read(count: Int): Unit = connectionActor ! ClientRead(count, connectionProbe.ref) def read(count: Int): Unit = connectionActor ! ClientRead(count, connectionProbe.ref)
def waitRead(): ByteString = connectionProbe.expectMsgType[ByteString] def waitRead(): ByteString = connectionProbe.expectMsgType[ReadResult].bytes
def confirmedClose(): Unit = connectionActor ! ClientClose(Tcp.ConfirmedClose) def confirmedClose(): Unit = connectionActor ! ClientClose(Tcp.ConfirmedClose)
def close(): Unit = connectionActor ! ClientClose(Tcp.Close) def close(): Unit = connectionActor ! ClientClose(Tcp.Close)
def abort(): Unit = connectionActor ! ClientClose(Tcp.Abort) def abort(): Unit = connectionActor ! ClientClose(Tcp.Abort)

View file

@ -59,7 +59,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
val faultyFlow: Flow[Any, Any, _] Flow[Any, Any, _] = in in.andThenMat { () val faultyFlow: Flow[Any, Any, _] Flow[Any, Any, _] = in in.andThenMat { ()
val props = Props(new BrokenActorInterpreter(settings, List(fusing.Map({ x: Any x }, stoppingDecider)), "a3")) val props = Props(new BrokenActorInterpreter(settings, List(fusing.Map({ x: Any x }, stoppingDecider)), "a3"))
.withDispatcher("akka.test.stream-dispatcher") .withDispatcher("akka.test.stream-dispatcher").withDeploy(Deploy.local)
val processor = ActorProcessorFactory[Any, Any](system.actorOf( val processor = ActorProcessorFactory[Any, Any](system.actorOf(
props, props,
"borken-stage-actor")) "borken-stage-actor"))

View file

@ -4,6 +4,7 @@
package akka.stream.impl package akka.stream.impl
import akka.actor.Props import akka.actor.Props
import akka.actor.Deploy
import akka.stream._ import akka.stream._
import akka.stream.impl.Junctions.FanInModule import akka.stream.impl.Junctions.FanInModule
import akka.stream.impl.StreamLayout.Module import akka.stream.impl.StreamLayout.Module
@ -29,7 +30,7 @@ private[akka] object GenJunctions {
override def carbonCopy: Module = ZipWith1Module(shape.deepCopy(), f, attributes) override def carbonCopy: Module = ZipWith1Module(shape.deepCopy(), f, attributes)
override def props(settings: ActorFlowMaterializerSettings): Props = override def props(settings: ActorFlowMaterializerSettings): Props =
Props(new Zip1With(settings, f.asInstanceOf[Function1[[#Any#], Any]])) Props(new Zip1With(settings, f.asInstanceOf[Function1[[#Any#], Any]])).withDeploy(Deploy.local)
}# }#
] ]

View file

@ -4,21 +4,11 @@
package akka.stream.actor package akka.stream.actor
import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentHashMap
import akka.actor.Cancellable import akka.actor._
import akka.stream.impl.{ ReactiveStreamsCompliance, StreamSubscriptionTimeoutSupport } import akka.stream.impl.{ ReactiveStreamsCompliance, StreamSubscriptionTimeoutSupport }
import org.reactivestreams.{ Publisher, Subscriber, Subscription } import org.reactivestreams.{ Publisher, Subscriber, Subscription }
import akka.actor.AbstractActor
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
import akka.actor.UntypedActor
import concurrent.duration.Duration import concurrent.duration.Duration
import concurrent.duration.FiniteDuration import concurrent.duration.FiniteDuration
import akka.actor.DeadLetterSuppression
import akka.stream.impl.CancelledSubscription import akka.stream.impl.CancelledSubscription
import akka.stream.impl.ReactiveStreamsCompliance._ import akka.stream.impl.ReactiveStreamsCompliance._
@ -35,7 +25,7 @@ object ActorPublisher {
* INTERNAL API * INTERNAL API
*/ */
private[akka] object Internal { private[akka] object Internal {
final case class Subscribe(subscriber: Subscriber[Any]) extends DeadLetterSuppression final case class Subscribe(subscriber: Subscriber[Any]) extends DeadLetterSuppression with NoSerializationVerificationNeeded
sealed trait LifecycleState sealed trait LifecycleState
case object PreSubscriber extends LifecycleState case object PreSubscriber extends LifecycleState
@ -55,20 +45,20 @@ object ActorPublisherMessage {
* more elements. * more elements.
* @param n number of requested elements * @param n number of requested elements
*/ */
@SerialVersionUID(1L) final case class Request(n: Long) extends ActorPublisherMessage final case class Request(n: Long) extends ActorPublisherMessage with NoSerializationVerificationNeeded
/** /**
* This message is delivered to the [[ActorPublisher]] actor when the stream subscriber cancels the * This message is delivered to the [[ActorPublisher]] actor when the stream subscriber cancels the
* subscription. * subscription.
*/ */
@SerialVersionUID(1L) final case object Cancel extends Cancel final case object Cancel extends Cancel with NoSerializationVerificationNeeded
sealed class Cancel extends ActorPublisherMessage sealed class Cancel extends ActorPublisherMessage
/** /**
* This message is delivered to the [[ActorPublisher]] actor in order to signal the exceeding of an subscription timeout. * This message is delivered to the [[ActorPublisher]] actor in order to signal the exceeding of an subscription timeout.
* Once the actor receives this message, this publisher will already be in cancelled state, thus the actor should clean-up and stop itself. * Once the actor receives this message, this publisher will already be in cancelled state, thus the actor should clean-up and stop itself.
*/ */
@SerialVersionUID(1L) final case object SubscriptionTimeoutExceeded extends SubscriptionTimeoutExceeded final case object SubscriptionTimeoutExceeded extends SubscriptionTimeoutExceeded with NoSerializationVerificationNeeded
sealed abstract class SubscriptionTimeoutExceeded extends ActorPublisherMessage sealed abstract class SubscriptionTimeoutExceeded extends ActorPublisherMessage
/** /**

View file

@ -5,16 +5,7 @@ package akka.stream.actor
import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentHashMap
import org.reactivestreams.{ Subscriber, Subscription } import org.reactivestreams.{ Subscriber, Subscription }
import akka.actor.AbstractActor import akka.actor._
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
import akka.actor.UntypedActor
import akka.actor.DeadLetterSuppression
import akka.stream.impl.ReactiveStreamsCompliance import akka.stream.impl.ReactiveStreamsCompliance
object ActorSubscriber { object ActorSubscriber {
@ -28,17 +19,17 @@ object ActorSubscriber {
/** /**
* INTERNAL API * INTERNAL API
*/ */
@SerialVersionUID(1L) private[akka] final case class OnSubscribe(subscription: Subscription) private[akka] final case class OnSubscribe(subscription: Subscription)
extends DeadLetterSuppression extends DeadLetterSuppression with NoSerializationVerificationNeeded
} }
sealed abstract class ActorSubscriberMessage extends DeadLetterSuppression sealed abstract class ActorSubscriberMessage extends DeadLetterSuppression with NoSerializationVerificationNeeded
object ActorSubscriberMessage { object ActorSubscriberMessage {
@SerialVersionUID(1L) final case class OnNext(element: Any) extends ActorSubscriberMessage final case class OnNext(element: Any) extends ActorSubscriberMessage
@SerialVersionUID(1L) final case class OnError(cause: Throwable) extends ActorSubscriberMessage final case class OnError(cause: Throwable) extends ActorSubscriberMessage
@SerialVersionUID(1L) case object OnComplete extends ActorSubscriberMessage case object OnComplete extends ActorSubscriberMessage
/** /**
* Java API: get the singleton instance of the `OnComplete` message * Java API: get the singleton instance of the `OnComplete` message

View file

@ -246,7 +246,7 @@ private[akka] class FlowNameCounter extends Extension {
* INTERNAL API * INTERNAL API
*/ */
private[akka] object StreamSupervisor { private[akka] object StreamSupervisor {
def props(settings: ActorFlowMaterializerSettings): Props = Props(new StreamSupervisor(settings)) def props(settings: ActorFlowMaterializerSettings): Props = Props(new StreamSupervisor(settings)).withDeploy(Deploy.local)
final case class Materialize(props: Props, name: String) extends DeadLetterSuppression final case class Materialize(props: Props, name: String) extends DeadLetterSuppression

View file

@ -5,14 +5,14 @@ package akka.stream.impl
import akka.stream.ActorFlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
import akka.actor.Props import akka.actor.{ Deploy, Props }
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[akka] object ConcatAllImpl { private[akka] object ConcatAllImpl {
def props(materializer: ActorFlowMaterializer): Props = def props(materializer: ActorFlowMaterializer): Props =
Props(new ConcatAllImpl(materializer)) Props(new ConcatAllImpl(materializer)).withDeploy(Deploy.local)
} }
/** /**

View file

@ -3,13 +3,11 @@
*/ */
package akka.stream.impl package akka.stream.impl
import akka.actor.{ ActorRef, ActorLogging, Actor } import akka.actor._
import akka.actor.Props
import akka.stream.{ AbruptTerminationException, ActorFlowMaterializerSettings, InPort, Shape } import akka.stream.{ AbruptTerminationException, ActorFlowMaterializerSettings, InPort, Shape }
import akka.stream.actor.{ ActorSubscriberMessage, ActorSubscriber } import akka.stream.actor.{ ActorSubscriberMessage, ActorSubscriber }
import akka.stream.scaladsl.FlexiMerge.MergeLogic import akka.stream.scaladsl.FlexiMerge.MergeLogic
import org.reactivestreams.{ Subscription, Subscriber } import org.reactivestreams.{ Subscription, Subscriber }
import akka.actor.DeadLetterSuppression
import scala.collection.immutable import scala.collection.immutable
@ -18,10 +16,10 @@ import scala.collection.immutable
*/ */
private[akka] object FanIn { private[akka] object FanIn {
final case class OnError(id: Int, cause: Throwable) extends DeadLetterSuppression final case class OnError(id: Int, cause: Throwable) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class OnComplete(id: Int) extends DeadLetterSuppression final case class OnComplete(id: Int) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class OnNext(id: Int, e: Any) extends DeadLetterSuppression final case class OnNext(id: Int, e: Any) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class OnSubscribe(id: Int, subscription: Subscription) extends DeadLetterSuppression final case class OnSubscribe(id: Int, subscription: Subscription) extends DeadLetterSuppression with NoSerializationVerificationNeeded
private[akka] final case class SubInput[T](impl: ActorRef, id: Int) extends Subscriber[T] { private[akka] final case class SubInput[T](impl: ActorRef, id: Int) extends Subscriber[T] {
override def onError(cause: Throwable): Unit = { override def onError(cause: Throwable): Unit = {
@ -264,7 +262,7 @@ private[akka] abstract class FanIn(val settings: ActorFlowMaterializerSettings,
*/ */
private[akka] object FairMerge { private[akka] object FairMerge {
def props(settings: ActorFlowMaterializerSettings, inputPorts: Int): Props = def props(settings: ActorFlowMaterializerSettings, inputPorts: Int): Props =
Props(new FairMerge(settings, inputPorts)) Props(new FairMerge(settings, inputPorts)).withDeploy(Deploy.local)
} }
/** /**
@ -287,7 +285,7 @@ private[akka] object UnfairMerge {
val DefaultPreferred = 0 val DefaultPreferred = 0
def props(settings: ActorFlowMaterializerSettings, inputPorts: Int): Props = def props(settings: ActorFlowMaterializerSettings, inputPorts: Int): Props =
Props(new UnfairMerge(settings, inputPorts, DefaultPreferred)) Props(new UnfairMerge(settings, inputPorts, DefaultPreferred)).withDeploy(Deploy.local)
} }
/** /**
@ -309,14 +307,14 @@ private[akka] final class UnfairMerge(_settings: ActorFlowMaterializerSettings,
*/ */
private[akka] object FlexiMerge { private[akka] object FlexiMerge {
def props[T, S <: Shape](settings: ActorFlowMaterializerSettings, ports: S, mergeLogic: MergeLogic[T]): Props = def props[T, S <: Shape](settings: ActorFlowMaterializerSettings, ports: S, mergeLogic: MergeLogic[T]): Props =
Props(new FlexiMergeImpl(settings, ports, mergeLogic)) Props(new FlexiMergeImpl(settings, ports, mergeLogic)).withDeploy(Deploy.local)
} }
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[akka] object Concat { private[akka] object Concat {
def props(settings: ActorFlowMaterializerSettings): Props = Props(new Concat(settings)) def props(settings: ActorFlowMaterializerSettings): Props = Props(new Concat(settings)).withDeploy(Deploy.local)
} }
/** /**

View file

@ -7,21 +7,17 @@ import akka.stream.scaladsl.FlexiRoute.RouteLogic
import akka.stream.{ AbruptTerminationException, Shape, ActorFlowMaterializerSettings } import akka.stream.{ AbruptTerminationException, Shape, ActorFlowMaterializerSettings }
import scala.collection.immutable import scala.collection.immutable
import akka.actor.Actor import akka.actor._
import akka.actor.ActorLogging
import akka.actor.ActorRef
import akka.actor.Props
import org.reactivestreams.Subscription import org.reactivestreams.Subscription
import akka.actor.DeadLetterSuppression
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[akka] object FanOut { private[akka] object FanOut {
final case class SubstreamRequestMore(id: Int, demand: Long) extends DeadLetterSuppression final case class SubstreamRequestMore(id: Int, demand: Long) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class SubstreamCancel(id: Int) extends DeadLetterSuppression final case class SubstreamCancel(id: Int) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class SubstreamSubscribePending(id: Int) extends DeadLetterSuppression final case class SubstreamSubscribePending(id: Int) extends DeadLetterSuppression with NoSerializationVerificationNeeded
class SubstreamSubscription(val parent: ActorRef, val id: Int) extends Subscription { class SubstreamSubscription(val parent: ActorRef, val id: Int) extends Subscription {
override def request(elements: Long): Unit = parent ! SubstreamRequestMore(id, elements) override def request(elements: Long): Unit = parent ! SubstreamRequestMore(id, elements)
@ -33,7 +29,7 @@ private[akka] object FanOut {
override def createSubscription(): Subscription = new SubstreamSubscription(actor, id) override def createSubscription(): Subscription = new SubstreamSubscription(actor, id)
} }
final case class ExposedPublishers(publishers: immutable.Seq[ActorPublisher[Any]]) extends DeadLetterSuppression final case class ExposedPublishers(publishers: immutable.Seq[ActorPublisher[Any]]) extends DeadLetterSuppression with NoSerializationVerificationNeeded
class OutputBunch(outputCount: Int, impl: ActorRef, pump: Pump) { class OutputBunch(outputCount: Int, impl: ActorRef, pump: Pump) {
private var bunchCancelled = false private var bunchCancelled = false
@ -294,7 +290,7 @@ private[akka] abstract class FanOut(val settings: ActorFlowMaterializerSettings,
*/ */
private[akka] object Broadcast { private[akka] object Broadcast {
def props(settings: ActorFlowMaterializerSettings, outputPorts: Int): Props = def props(settings: ActorFlowMaterializerSettings, outputPorts: Int): Props =
Props(new Broadcast(settings, outputPorts)) Props(new Broadcast(settings, outputPorts)).withDeploy(Deploy.local)
} }
/** /**
@ -314,7 +310,7 @@ private[akka] class Broadcast(_settings: ActorFlowMaterializerSettings, _outputP
*/ */
private[akka] object Balance { private[akka] object Balance {
def props(settings: ActorFlowMaterializerSettings, outputPorts: Int, waitForAllDownstreams: Boolean): Props = def props(settings: ActorFlowMaterializerSettings, outputPorts: Int, waitForAllDownstreams: Boolean): Props =
Props(new Balance(settings, outputPorts, waitForAllDownstreams)) Props(new Balance(settings, outputPorts, waitForAllDownstreams)).withDeploy(Deploy.local)
} }
/** /**
@ -341,7 +337,7 @@ private[akka] class Balance(_settings: ActorFlowMaterializerSettings, _outputPor
*/ */
private[akka] object Unzip { private[akka] object Unzip {
def props(settings: ActorFlowMaterializerSettings): Props = def props(settings: ActorFlowMaterializerSettings): Props =
Props(new Unzip(settings)) Props(new Unzip(settings)).withDeploy(Deploy.local)
} }
/** /**
@ -373,5 +369,5 @@ private[akka] class Unzip(_settings: ActorFlowMaterializerSettings) extends FanO
*/ */
private[akka] object FlexiRoute { private[akka] object FlexiRoute {
def props[T, S <: Shape](settings: ActorFlowMaterializerSettings, ports: S, routeLogic: RouteLogic[T]): Props = def props[T, S <: Shape](settings: ActorFlowMaterializerSettings, ports: S, routeLogic: RouteLogic[T]): Props =
Props(new FlexiRouteImpl(settings, ports, routeLogic)) Props(new FlexiRouteImpl(settings, ports, routeLogic)).withDeploy(Deploy.local)
} }

View file

@ -7,16 +7,11 @@ import scala.concurrent.Future
import scala.util.Failure import scala.util.Failure
import scala.util.Success import scala.util.Success
import scala.util.Try import scala.util.Try
import akka.actor.Actor import akka.actor._
import akka.actor.ActorRef
import akka.actor.Props
import akka.actor.Status
import akka.actor.SupervisorStrategy
import akka.stream.ActorFlowMaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.pattern.pipe import akka.pattern.pipe
import org.reactivestreams.Subscriber import org.reactivestreams.Subscriber
import org.reactivestreams.Subscription import org.reactivestreams.Subscription
import akka.actor.DeadLetterSuppression
import scala.util.control.NonFatal import scala.util.control.NonFatal
/** /**
@ -24,11 +19,11 @@ import scala.util.control.NonFatal
*/ */
private[akka] object FuturePublisher { private[akka] object FuturePublisher {
def props(future: Future[Any], settings: ActorFlowMaterializerSettings): Props = def props(future: Future[Any], settings: ActorFlowMaterializerSettings): Props =
Props(new FuturePublisher(future, settings)).withDispatcher(settings.dispatcher) Props(new FuturePublisher(future, settings)).withDispatcher(settings.dispatcher).withDeploy(Deploy.local)
object FutureSubscription { object FutureSubscription {
final case class Cancel(subscription: FutureSubscription) extends DeadLetterSuppression final case class Cancel(subscription: FutureSubscription) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class RequestMore(subscription: FutureSubscription, elements: Long) extends DeadLetterSuppression final case class RequestMore(subscription: FutureSubscription, elements: Long) extends DeadLetterSuppression with NoSerializationVerificationNeeded
} }
class FutureSubscription(ref: ActorRef) extends Subscription { class FutureSubscription(ref: ActorRef) extends Subscription {

View file

@ -4,7 +4,7 @@
package akka.stream.impl package akka.stream.impl
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.actor.Props import akka.actor.{ Deploy, Props }
import akka.stream.ActorFlowMaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.Supervision import akka.stream.Supervision
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
@ -14,7 +14,7 @@ import akka.stream.scaladsl.Source
*/ */
private[akka] object GroupByProcessorImpl { private[akka] object GroupByProcessorImpl {
def props(settings: ActorFlowMaterializerSettings, keyFor: Any Any): Props = def props(settings: ActorFlowMaterializerSettings, keyFor: Any Any): Props =
Props(new GroupByProcessorImpl(settings, keyFor)) Props(new GroupByProcessorImpl(settings, keyFor)).withDeploy(Deploy.local)
private case object Drop private case object Drop
} }

View file

@ -5,28 +5,28 @@ package akka.stream.impl
import language.existentials import language.existentials
import org.reactivestreams.Subscription import org.reactivestreams.Subscription
import akka.actor.DeadLetterSuppression import akka.actor.{ NoSerializationVerificationNeeded, DeadLetterSuppression }
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[akka] case object SubscribePending extends DeadLetterSuppression private[akka] case object SubscribePending extends DeadLetterSuppression with NoSerializationVerificationNeeded
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[akka] final case class RequestMore(subscription: ActorSubscription[_], demand: Long) private[akka] final case class RequestMore(subscription: ActorSubscription[_], demand: Long)
extends DeadLetterSuppression extends DeadLetterSuppression with NoSerializationVerificationNeeded
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[akka] final case class Cancel(subscription: ActorSubscription[_]) private[akka] final case class Cancel(subscription: ActorSubscription[_])
extends DeadLetterSuppression extends DeadLetterSuppression with NoSerializationVerificationNeeded
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[akka] final case class ExposedPublisher(publisher: ActorPublisher[Any]) private[akka] final case class ExposedPublisher(publisher: ActorPublisher[Any])
extends DeadLetterSuppression extends DeadLetterSuppression with NoSerializationVerificationNeeded

View file

@ -6,14 +6,14 @@ package akka.stream.impl
import scala.collection.immutable import scala.collection.immutable
import akka.stream.ActorFlowMaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.actor.Props import akka.actor.{ Deploy, Props }
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[akka] object PrefixAndTailImpl { private[akka] object PrefixAndTailImpl {
def props(settings: ActorFlowMaterializerSettings, takeMax: Int): Props = def props(settings: ActorFlowMaterializerSettings, takeMax: Int): Props =
Props(new PrefixAndTailImpl(settings, takeMax)) Props(new PrefixAndTailImpl(settings, takeMax)).withDeploy(Deploy.local)
} }
/** /**

View file

@ -5,7 +5,7 @@ package akka.stream.impl
import java.io.File import java.io.File
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import akka.actor.{ ActorRef, Props } import akka.actor.{ Deploy, ActorRef, Props }
import akka.stream.ActorOperationAttributes.Dispatcher import akka.stream.ActorOperationAttributes.Dispatcher
import akka.stream.impl.StreamLayout.Module import akka.stream.impl.StreamLayout.Module
import akka.stream.OperationAttributes import akka.stream.OperationAttributes
@ -83,7 +83,7 @@ private[akka] final class FanoutPublisherSink[In](
val actorMaterializer = ActorFlowMaterializer.downcast(context.materializer) val actorMaterializer = ActorFlowMaterializer.downcast(context.materializer)
val fanoutActor = actorMaterializer.actorOf(context, val fanoutActor = actorMaterializer.actorOf(context,
Props(new FanoutProcessorImpl(actorMaterializer.effectiveSettings(context.effectiveAttributes), Props(new FanoutProcessorImpl(actorMaterializer.effectiveSettings(context.effectiveAttributes),
initialBufferSize, maximumBufferSize))) initialBufferSize, maximumBufferSize)).withDeploy(Deploy.local))
val fanoutProcessor = ActorProcessorFactory[In, In](fanoutActor) val fanoutProcessor = ActorProcessorFactory[In, In](fanoutActor)
(fanoutProcessor, fanoutProcessor) (fanoutProcessor, fanoutProcessor)
} }

View file

@ -4,24 +4,21 @@
package akka.stream.impl package akka.stream.impl
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import akka.actor.ActorLogging import akka.actor._
import akka.actor.Cancellable
import akka.actor.{ Actor, ActorRef }
import akka.stream.ActorFlowMaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import org.reactivestreams.{ Publisher, Subscriber, Subscription } import org.reactivestreams.{ Publisher, Subscriber, Subscription }
import scala.collection.mutable import scala.collection.mutable
import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.FiniteDuration
import akka.actor.DeadLetterSuppression
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[akka] object MultiStreamOutputProcessor { private[akka] object MultiStreamOutputProcessor {
final case class SubstreamKey(id: Long) final case class SubstreamKey(id: Long)
final case class SubstreamRequestMore(substream: SubstreamKey, demand: Long) extends DeadLetterSuppression final case class SubstreamRequestMore(substream: SubstreamKey, demand: Long) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class SubstreamCancel(substream: SubstreamKey) extends DeadLetterSuppression final case class SubstreamCancel(substream: SubstreamKey) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class SubstreamSubscribe(substream: SubstreamKey, subscriber: Subscriber[Any]) extends DeadLetterSuppression final case class SubstreamSubscribe(substream: SubstreamKey, subscriber: Subscriber[Any]) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class SubstreamSubscriptionTimeout(substream: SubstreamKey) extends DeadLetterSuppression final case class SubstreamSubscriptionTimeout(substream: SubstreamKey) extends DeadLetterSuppression with NoSerializationVerificationNeeded
class SubstreamSubscription(val parent: ActorRef, val substreamKey: SubstreamKey) extends Subscription { class SubstreamSubscription(val parent: ActorRef, val substreamKey: SubstreamKey) extends Subscription {
override def request(elements: Long): Unit = parent ! SubstreamRequestMore(substreamKey, elements) override def request(elements: Long): Unit = parent ! SubstreamRequestMore(substreamKey, elements)
@ -316,10 +313,10 @@ private[akka] object MultiStreamInputProcessor {
} }
} }
case class SubstreamOnComplete(key: SubstreamKey) extends DeadLetterSuppression case class SubstreamOnComplete(key: SubstreamKey) extends DeadLetterSuppression with NoSerializationVerificationNeeded
case class SubstreamOnNext(key: SubstreamKey, element: Any) extends DeadLetterSuppression case class SubstreamOnNext(key: SubstreamKey, element: Any) extends DeadLetterSuppression with NoSerializationVerificationNeeded
case class SubstreamOnError(key: SubstreamKey, e: Throwable) extends DeadLetterSuppression case class SubstreamOnError(key: SubstreamKey, e: Throwable) extends DeadLetterSuppression with NoSerializationVerificationNeeded
case class SubstreamStreamOnSubscribe(key: SubstreamKey, subscription: Subscription) extends DeadLetterSuppression case class SubstreamStreamOnSubscribe(key: SubstreamKey, subscription: Subscription) extends DeadLetterSuppression with NoSerializationVerificationNeeded
class SubstreamInput(val key: SubstreamKey, bufferSize: Int, processor: MultiStreamInputProcessorLike, pump: Pump) extends BatchingInputBuffer(bufferSize, pump) { class SubstreamInput(val key: SubstreamKey, bufferSize: Int, processor: MultiStreamInputProcessorLike, pump: Pump) extends BatchingInputBuffer(bufferSize, pump) {
// Not driven directly // Not driven directly

View file

@ -4,13 +4,12 @@
package akka.stream.impl package akka.stream.impl
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import akka.actor.{ Actor, ActorRef, Cancellable, Props, SupervisorStrategy } import akka.actor._
import akka.stream.ActorFlowMaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import org.reactivestreams.{ Subscriber, Subscription } import org.reactivestreams.{ Subscriber, Subscription }
import scala.collection.mutable import scala.collection.mutable
import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.FiniteDuration
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.actor.DeadLetterSuppression
import akka.event.Logging import akka.event.Logging
/** /**
@ -19,7 +18,9 @@ import akka.event.Logging
private[akka] object TickPublisher { private[akka] object TickPublisher {
def props(initialDelay: FiniteDuration, interval: FiniteDuration, tick: Any, def props(initialDelay: FiniteDuration, interval: FiniteDuration, tick: Any,
settings: ActorFlowMaterializerSettings, cancelled: AtomicBoolean): Props = settings: ActorFlowMaterializerSettings, cancelled: AtomicBoolean): Props =
Props(new TickPublisher(initialDelay, interval, tick, settings, cancelled)).withDispatcher(settings.dispatcher) Props(new TickPublisher(initialDelay, interval, tick, settings, cancelled))
.withDispatcher(settings.dispatcher)
.withDeploy(Deploy.local)
object TickPublisherSubscription { object TickPublisherSubscription {
case object Cancel extends DeadLetterSuppression case object Cancel extends DeadLetterSuppression

View file

@ -7,11 +7,11 @@ import java.util.LinkedList
import akka.stream.ActorFlowMaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.TimerTransformer import akka.stream.TimerTransformer
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.actor.Props import akka.actor.{ Deploy, Props }
private[akka] object TimerTransformerProcessorsImpl { private[akka] object TimerTransformerProcessorsImpl {
def props(settings: ActorFlowMaterializerSettings, transformer: TimerTransformer[Any, Any]): Props = def props(settings: ActorFlowMaterializerSettings, transformer: TimerTransformer[Any, Any]): Props =
Props(new TimerTransformerProcessorsImpl(settings, transformer)) Props(new TimerTransformerProcessorsImpl(settings, transformer)).withDeploy(Deploy.local)
} }
/** /**

View file

@ -154,7 +154,7 @@ private[akka] object ActorOutputBoundary {
/** /**
* INTERNAL API. * INTERNAL API.
*/ */
private case object ContinuePulling extends DeadLetterSuppression private case object ContinuePulling extends DeadLetterSuppression with NoSerializationVerificationNeeded
} }
/** /**
@ -308,9 +308,9 @@ private[akka] class ActorOutputBoundary(val actor: ActorRef,
*/ */
private[akka] object ActorInterpreter { 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: OperationAttributes = OperationAttributes.none): Props =
Props(new ActorInterpreter(settings, ops, materializer, attributes)) 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 case class AsyncInput(op: AsyncStage[Any, Any, Any], ctx: AsyncContext[Any, Any], event: Any) extends DeadLetterSuppression with NoSerializationVerificationNeeded
} }
/** /**

View file

@ -1,4 +1,4 @@
package akka.stream.io.impl package akka.stream.impl.io
import akka.stream.ActorOperationAttributes.Dispatcher import akka.stream.ActorOperationAttributes.Dispatcher
import akka.stream.{ ActorFlowMaterializer, MaterializationContext } import akka.stream.{ ActorFlowMaterializer, MaterializationContext }

View file

@ -1,13 +1,12 @@
/** /**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.stream.io.impl package akka.stream.impl.io
import java.io.{ File, OutputStream } import java.io.{ File, OutputStream }
import akka.stream.impl.SinkModule import akka.stream.impl.SinkModule
import akka.stream.impl.StreamLayout.Module import akka.stream.impl.StreamLayout.Module
import akka.stream.io.impl.IOSettings._
import akka.stream.{ ActorFlowMaterializer, MaterializationContext, OperationAttributes, SinkShape } import akka.stream.{ ActorFlowMaterializer, MaterializationContext, OperationAttributes, SinkShape }
import akka.util.ByteString import akka.util.ByteString
@ -27,7 +26,7 @@ private[akka] final class SynchronousFileSink(f: File, append: Boolean, val attr
val bytesWrittenPromise = Promise[Long]() val bytesWrittenPromise = Promise[Long]()
val props = SynchronousFileSubscriber.props(f, bytesWrittenPromise, settings.maxInputBufferSize, append) val props = SynchronousFileSubscriber.props(f, bytesWrittenPromise, settings.maxInputBufferSize, append)
val dispatcher = fileIoDispatcher(context) val dispatcher = IOSettings.fileIoDispatcher(context)
val ref = mat.actorOf(context, props.withDispatcher(dispatcher)) val ref = mat.actorOf(context, props.withDispatcher(dispatcher))
(akka.stream.actor.ActorSubscriber[ByteString](ref), bytesWrittenPromise.future) (akka.stream.actor.ActorSubscriber[ByteString](ref), bytesWrittenPromise.future)

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.stream.io.impl package akka.stream.impl.io
import java.io.{ File, InputStream } import java.io.{ File, InputStream }

View file

@ -1,11 +1,11 @@
/** /**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.stream.io.impl package akka.stream.impl.io
import java.io.InputStream import java.io.InputStream
import akka.actor.{ ActorLogging, DeadLetterSuppression, Props } import akka.actor.{ Deploy, ActorLogging, DeadLetterSuppression, Props }
import akka.io.DirectByteBufferPool import akka.io.DirectByteBufferPool
import akka.stream.actor.ActorPublisherMessage import akka.stream.actor.ActorPublisherMessage
import akka.util.ByteString import akka.util.ByteString
@ -22,7 +22,7 @@ private[akka] object InputStreamPublisher {
require(initialBuffer > 0, s"initialBuffer must be > 0 (was $initialBuffer)") require(initialBuffer > 0, s"initialBuffer must be > 0 (was $initialBuffer)")
require(maxBuffer >= initialBuffer, s"maxBuffer must be >= initialBuffer (was $maxBuffer)") require(maxBuffer >= initialBuffer, s"maxBuffer must be >= initialBuffer (was $maxBuffer)")
Props(classOf[InputStreamPublisher], is, completionPromise, chunkSize, initialBuffer, maxBuffer) Props(classOf[InputStreamPublisher], is, completionPromise, chunkSize, initialBuffer, maxBuffer).withDeploy(Deploy.local)
} }
private final case object Continue extends DeadLetterSuppression private final case object Continue extends DeadLetterSuppression

View file

@ -1,11 +1,11 @@
/** /**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.stream.io.impl package akka.stream.impl.io
import java.io.OutputStream import java.io.OutputStream
import akka.actor.{ ActorLogging, Props } import akka.actor.{ Deploy, ActorLogging, Props }
import akka.stream.actor.{ ActorSubscriberMessage, WatermarkRequestStrategy } import akka.stream.actor.{ ActorSubscriberMessage, WatermarkRequestStrategy }
import akka.util.ByteString import akka.util.ByteString
@ -15,7 +15,7 @@ import scala.concurrent.Promise
private[akka] object OutputStreamSubscriber { private[akka] object OutputStreamSubscriber {
def props(os: OutputStream, completionPromise: Promise[Long], bufSize: Int) = { def props(os: OutputStream, completionPromise: Promise[Long], bufSize: Int) = {
require(bufSize > 0, "buffer size must be > 0") require(bufSize > 0, "buffer size must be > 0")
Props(classOf[OutputStreamSubscriber], os, completionPromise, bufSize) Props(classOf[OutputStreamSubscriber], os, completionPromise, bufSize).withDeploy(Deploy.local)
} }
} }

View file

@ -10,7 +10,7 @@ import javax.net.ssl.SSLEngineResult.HandshakeStatus
import javax.net.ssl.SSLEngineResult.HandshakeStatus._ import javax.net.ssl.SSLEngineResult.HandshakeStatus._
import javax.net.ssl.SSLEngineResult.Status._ import javax.net.ssl.SSLEngineResult.Status._
import javax.net.ssl._ import javax.net.ssl._
import akka.actor.{ Props, Actor, ActorLogging, ActorRef } import akka.actor._
import akka.stream.ActorFlowMaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.impl.FanIn.InputBunch import akka.stream.impl.FanIn.InputBunch
import akka.stream.impl.FanOut.OutputBunch import akka.stream.impl.FanOut.OutputBunch
@ -35,7 +35,7 @@ private[akka] object SslTlsCipherActor {
tracing: Boolean, tracing: Boolean,
role: Role, role: Role,
closing: Closing): Props = closing: Closing): Props =
Props(new SslTlsCipherActor(settings, sslContext, firstSession, tracing, role, closing)) Props(new SslTlsCipherActor(settings, sslContext, firstSession, tracing, role, closing)).withDeploy(Deploy.local)
final val TransportIn = 0 final val TransportIn = 0
final val TransportOut = 0 final val TransportOut = 0

View file

@ -10,7 +10,7 @@ import scala.concurrent.Future
import scala.concurrent.Promise import scala.concurrent.Promise
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.FiniteDuration
import akka.actor.Actor import akka.actor.{ NoSerializationVerificationNeeded, Actor, DeadLetterSuppression }
import akka.io.Inet.SocketOption import akka.io.Inet.SocketOption
import akka.io.Tcp import akka.io.Tcp
import akka.stream.ActorFlowMaterializerSettings import akka.stream.ActorFlowMaterializerSettings
@ -20,7 +20,6 @@ import akka.stream.scaladsl.{ Tcp ⇒ StreamTcp }
import akka.util.ByteString import akka.util.ByteString
import org.reactivestreams.Processor import org.reactivestreams.Processor
import org.reactivestreams.Subscriber import org.reactivestreams.Subscriber
import akka.actor.DeadLetterSuppression
/** /**
* INTERNAL API * INTERNAL API
@ -37,7 +36,7 @@ private[akka] object StreamTcpManager {
options: immutable.Traversable[SocketOption], options: immutable.Traversable[SocketOption],
connectTimeout: Duration, connectTimeout: Duration,
idleTimeout: Duration) idleTimeout: Duration)
extends DeadLetterSuppression extends DeadLetterSuppression with NoSerializationVerificationNeeded
/** /**
* INTERNAL API * INTERNAL API
@ -50,13 +49,13 @@ private[akka] object StreamTcpManager {
backlog: Int, backlog: Int,
options: immutable.Traversable[SocketOption], options: immutable.Traversable[SocketOption],
idleTimeout: Duration) idleTimeout: Duration)
extends DeadLetterSuppression extends DeadLetterSuppression with NoSerializationVerificationNeeded
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[akka] final case class ExposedProcessor(processor: Processor[ByteString, ByteString]) private[akka] final case class ExposedProcessor(processor: Processor[ByteString, ByteString])
extends DeadLetterSuppression extends DeadLetterSuppression with NoSerializationVerificationNeeded
} }

View file

@ -1,13 +1,13 @@
/** /**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.stream.io.impl package akka.stream.impl.io
import java.io.{ File, RandomAccessFile } import java.io.{ File, RandomAccessFile }
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.nio.channels.FileChannel import java.nio.channels.FileChannel
import akka.actor.{ ActorLogging, DeadLetterSuppression, Props } import akka.actor.{ Deploy, ActorLogging, DeadLetterSuppression, Props }
import akka.stream.actor.ActorPublisherMessage import akka.stream.actor.ActorPublisherMessage
import akka.util.ByteString import akka.util.ByteString
@ -22,6 +22,7 @@ private[akka] object SynchronousFilePublisher {
require(maxBuffer >= initialBuffer, s"maxBuffer must be >= initialBuffer (was $maxBuffer)") require(maxBuffer >= initialBuffer, s"maxBuffer must be >= initialBuffer (was $maxBuffer)")
Props(classOf[SynchronousFilePublisher], f, completionPromise, chunkSize, initialBuffer, maxBuffer) Props(classOf[SynchronousFilePublisher], f, completionPromise, chunkSize, initialBuffer, maxBuffer)
.withDeploy(Deploy.local)
} }
private final case object Continue extends DeadLetterSuppression private final case object Continue extends DeadLetterSuppression

View file

@ -1,12 +1,12 @@
/** /**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.stream.io.impl package akka.stream.impl.io
import java.io.{ File, RandomAccessFile } import java.io.{ File, RandomAccessFile }
import java.nio.channels.FileChannel import java.nio.channels.FileChannel
import akka.actor.{ ActorLogging, Props } import akka.actor.{ Deploy, ActorLogging, Props }
import akka.stream.actor.{ ActorSubscriberMessage, WatermarkRequestStrategy } import akka.stream.actor.{ ActorSubscriberMessage, WatermarkRequestStrategy }
import akka.util.ByteString import akka.util.ByteString
@ -16,7 +16,7 @@ import scala.concurrent.Promise
private[akka] object SynchronousFileSubscriber { private[akka] object SynchronousFileSubscriber {
def props(f: File, completionPromise: Promise[Long], bufSize: Int, append: Boolean) = { def props(f: File, completionPromise: Promise[Long], bufSize: Int, append: Boolean) = {
require(bufSize > 0, "buffer size must be > 0") require(bufSize > 0, "buffer size must be > 0")
Props(classOf[SynchronousFileSubscriber], f, completionPromise, bufSize, append) Props(classOf[SynchronousFileSubscriber], f, completionPromise, bufSize, append).withDeploy(Deploy.local)
} }
} }

View file

@ -27,10 +27,10 @@ private[akka] object TcpStreamActor {
connectCmd: Connect, connectCmd: Connect,
materializerSettings: ActorFlowMaterializerSettings): Props = materializerSettings: ActorFlowMaterializerSettings): Props =
Props(new OutboundTcpStreamActor(processorPromise, localAddressPromise, connectCmd, Props(new OutboundTcpStreamActor(processorPromise, localAddressPromise, connectCmd,
materializerSettings)).withDispatcher(materializerSettings.dispatcher) materializerSettings)).withDispatcher(materializerSettings.dispatcher).withDeploy(Deploy.local)
def inboundProps(connection: ActorRef, settings: ActorFlowMaterializerSettings): Props = def inboundProps(connection: ActorRef, settings: ActorFlowMaterializerSettings): Props =
Props(new InboundTcpStreamActor(connection, settings)).withDispatcher(settings.dispatcher) Props(new InboundTcpStreamActor(connection, settings)).withDispatcher(settings.dispatcher).withDeploy(Deploy.local)
} }
/** /**

View file

@ -5,9 +5,7 @@ package akka.stream.impl.io
import java.net.InetSocketAddress import java.net.InetSocketAddress
import scala.concurrent.{ Future, Promise } import scala.concurrent.{ Future, Promise }
import akka.actor.Actor import akka.actor._
import akka.actor.ActorRef
import akka.actor.Props
import akka.io.{ IO, Tcp } import akka.io.{ IO, Tcp }
import akka.io.Tcp._ import akka.io.Tcp._
import akka.stream.{ FlowMaterializer, ActorFlowMaterializerSettings } import akka.stream.{ FlowMaterializer, ActorFlowMaterializerSettings }
@ -18,7 +16,6 @@ import akka.util.ByteString
import org.reactivestreams.Subscriber import org.reactivestreams.Subscriber
import akka.stream.ConnectionException import akka.stream.ConnectionException
import akka.stream.BindFailedException import akka.stream.BindFailedException
import akka.actor.ActorLogging
/** /**
* INTERNAL API * INTERNAL API
@ -29,6 +26,7 @@ private[akka] object TcpListenStreamActor {
flowSubscriber: Subscriber[StreamTcp.IncomingConnection], flowSubscriber: Subscriber[StreamTcp.IncomingConnection],
bindCmd: Tcp.Bind, materializerSettings: ActorFlowMaterializerSettings): Props = { bindCmd: Tcp.Bind, materializerSettings: ActorFlowMaterializerSettings): Props = {
Props(new TcpListenStreamActor(localAddressPromise, unbindPromise, flowSubscriber, bindCmd, materializerSettings)) Props(new TcpListenStreamActor(localAddressPromise, unbindPromise, flowSubscriber, bindCmd, materializerSettings))
.withDeploy(Deploy.local)
} }
} }

View file

@ -6,7 +6,7 @@ package akka.stream.io
import java.io.InputStream import java.io.InputStream
import akka.japi.function.Creator import akka.japi.function.Creator
import akka.stream.io.impl.InputStreamSource import akka.stream.impl.io.InputStreamSource
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.stream.scaladsl.Source._ import akka.stream.scaladsl.Source._
import akka.stream.{ OperationAttributes, javadsl } import akka.stream.{ OperationAttributes, javadsl }

View file

@ -6,7 +6,7 @@ package akka.stream.io
import java.io.OutputStream import java.io.OutputStream
import akka.japi.function.Creator import akka.japi.function.Creator
import akka.stream.io.impl.OutputStreamSink import akka.stream.impl.io.OutputStreamSink
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
import akka.stream.{ ActorOperationAttributes, OperationAttributes, javadsl } import akka.stream.{ ActorOperationAttributes, OperationAttributes, javadsl }
import akka.util.ByteString import akka.util.ByteString

View file

@ -5,8 +5,8 @@ package akka.stream.io
import java.io.File import java.io.File
import akka.stream.impl.io.SynchronousFileSink
import akka.stream.{ OperationAttributes, javadsl, ActorOperationAttributes } import akka.stream.{ OperationAttributes, javadsl, ActorOperationAttributes }
import akka.stream.io.impl.SynchronousFileSink
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
import akka.util.ByteString import akka.util.ByteString
@ -28,7 +28,7 @@ object SynchronousFileSink {
* unless configured otherwise by using [[ActorOperationAttributes]]. * unless configured otherwise by using [[ActorOperationAttributes]].
*/ */
def apply(f: File, append: Boolean = false): Sink[ByteString, Future[Long]] = def apply(f: File, append: Boolean = false): Sink[ByteString, Future[Long]] =
new Sink(new impl.SynchronousFileSink(f, append, DefaultAttributes, Sink.shape("SynchronousFileSink"))) new Sink(new SynchronousFileSink(f, append, DefaultAttributes, Sink.shape("SynchronousFileSink")))
/** /**
* Java API * Java API

View file

@ -4,8 +4,7 @@
package akka.stream.io package akka.stream.io
import java.io.File import java.io.File
import akka.stream.impl.io.SynchronousFileSource
import akka.stream.io.impl.SynchronousFileSource
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.stream.{ ActorOperationAttributes, OperationAttributes, javadsl } import akka.stream.{ ActorOperationAttributes, OperationAttributes, javadsl }
import akka.util.ByteString import akka.util.ByteString

View file

@ -10,13 +10,7 @@ import scala.concurrent.{ Promise, ExecutionContext, Future }
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import scala.util.{ Failure, Success } import scala.util.{ Failure, Success }
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.actor.Actor import akka.actor._
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.ExtendedActorSystem
import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
import akka.actor.Props
import akka.io.Inet.SocketOption import akka.io.Inet.SocketOption
import akka.io.{ Tcp IoTcp } import akka.io.{ Tcp IoTcp }
import akka.stream._ import akka.stream._
@ -25,7 +19,6 @@ import akka.stream.impl.ReactiveStreamsCompliance._
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.util.ByteString import akka.util.ByteString
import org.reactivestreams.{ Publisher, Processor, Subscriber, Subscription } import org.reactivestreams.{ Publisher, Processor, Subscriber, Subscription }
import akka.actor.actorRef2Scala
import akka.stream.impl.io.TcpStreamActor import akka.stream.impl.io.TcpStreamActor
import akka.stream.impl.io.TcpListenStreamActor import akka.stream.impl.io.TcpListenStreamActor
import akka.stream.impl.io.DelayedInitProcessor import akka.stream.impl.io.DelayedInitProcessor
@ -77,7 +70,7 @@ class Tcp(system: ExtendedActorSystem) extends akka.actor.Extension {
import Tcp._ import Tcp._
private val manager: ActorRef = system.systemActorOf(Props[StreamTcpManager] private val manager: ActorRef = system.systemActorOf(Props[StreamTcpManager]
.withDispatcher(IoTcp(system).Settings.ManagementDispatcher), name = "IO-TCP-STREAM") .withDispatcher(IoTcp(system).Settings.ManagementDispatcher).withDeploy(Deploy.local), name = "IO-TCP-STREAM")
private class BindSource( private class BindSource(
val endpoint: InetSocketAddress, val endpoint: InetSocketAddress,