!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
import scala.language.existentials
import akka.actor.ActorSystem
import akka.actor.DeadLetterSuppression
import akka.actor.{ NoSerializationVerificationNeeded, ActorSystem, DeadLetterSuppression }
import akka.stream._
import akka.stream.impl._
import akka.testkit.TestProbe
@ -359,12 +358,12 @@ private[testkit] object StreamTestKit {
import TestPublisher._
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 CancelSubscription(subscription: Subscription) 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 OnNext[I](element: I) extends SubscriberEvent
final case object OnComplete extends SubscriberEvent

View file

@ -5,6 +5,7 @@ package akka.stream.javadsl;
import akka.actor.ActorRef;
import akka.japi.Pair;
import akka.pattern.Patterns;
import akka.stream.*;
import akka.stream.javadsl.FlowGraph.Builder;
import akka.stream.stage.*;
@ -223,7 +224,7 @@ public class FlowGraphTest extends StreamTest {
b.from(Source.single(1)).to(out);
b.from(b.materializedValue()).to(Sink.foreach(new Procedure<Future<Integer>>(){
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));
assertEquals(1, (int) result);
final Future<Integer> future2 = probe.expectMsgClass(Future.class);
final Integer result2 = Await.result(future2, Duration.create(300, TimeUnit.MILLISECONDS));
assertEquals(1, (int) result2);
probe.expectMsg(1);
}
}

View file

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

View file

@ -3,7 +3,7 @@
*/
package akka.stream.io
import akka.actor.{ Actor, ActorRef, Props }
import akka.actor._
import akka.io.Tcp.{ ResumeReading, Register, ConnectionClosed, Closed }
import akka.io.{ IO, Tcp }
import akka.stream.testkit._
@ -17,9 +17,10 @@ import akka.stream.testkit.TestUtils.temporaryServerAddress
import scala.concurrent.duration._
object TcpHelper {
case class ClientWrite(bytes: ByteString)
case class ClientRead(count: Int, readTo: ActorRef)
case class ClientClose(cmd: Tcp.CloseCommand)
case class ClientWrite(bytes: ByteString) extends NoSerializationVerificationNeeded
case class ClientRead(count: Int, readTo: ActorRef) extends NoSerializationVerificationNeeded
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
// See https://github.com/akka/akka/issues/16552
@ -69,7 +70,7 @@ object TcpHelper {
case Tcp.Received(bytes)
readBuffer ++= bytes
if (readBuffer.size >= toRead) {
readTo ! readBuffer
readTo ! ReadResult(readBuffer)
readBuffer = ByteString.empty
toRead = 0
readTo = context.system.deadLetters
@ -136,7 +137,7 @@ trait TcpHelper { this: TestKitBase ⇒
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 close(): Unit = connectionActor ! ClientClose(Tcp.Close)
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 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(
props,
"borken-stage-actor"))

View file

@ -4,6 +4,7 @@
package akka.stream.impl
import akka.actor.Props
import akka.actor.Deploy
import akka.stream._
import akka.stream.impl.Junctions.FanInModule
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 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
import java.util.concurrent.ConcurrentHashMap
import akka.actor.Cancellable
import akka.actor._
import akka.stream.impl.{ ReactiveStreamsCompliance, StreamSubscriptionTimeoutSupport }
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.FiniteDuration
import akka.actor.DeadLetterSuppression
import akka.stream.impl.CancelledSubscription
import akka.stream.impl.ReactiveStreamsCompliance._
@ -35,7 +25,7 @@ object ActorPublisher {
* INTERNAL API
*/
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
case object PreSubscriber extends LifecycleState
@ -55,20 +45,20 @@ object ActorPublisherMessage {
* more 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
* subscription.
*/
@SerialVersionUID(1L) final case object Cancel extends Cancel
final case object Cancel extends Cancel with NoSerializationVerificationNeeded
sealed class Cancel extends ActorPublisherMessage
/**
* 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.
*/
@SerialVersionUID(1L) final case object SubscriptionTimeoutExceeded extends SubscriptionTimeoutExceeded
final case object SubscriptionTimeoutExceeded extends SubscriptionTimeoutExceeded with NoSerializationVerificationNeeded
sealed abstract class SubscriptionTimeoutExceeded extends ActorPublisherMessage
/**

View file

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

View file

@ -246,7 +246,7 @@ private[akka] class FlowNameCounter extends Extension {
* INTERNAL API
*/
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

View file

@ -5,14 +5,14 @@ package akka.stream.impl
import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Sink
import akka.actor.Props
import akka.actor.{ Deploy, Props }
/**
* INTERNAL API
*/
private[akka] object ConcatAllImpl {
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
import akka.actor.{ ActorRef, ActorLogging, Actor }
import akka.actor.Props
import akka.actor._
import akka.stream.{ AbruptTerminationException, ActorFlowMaterializerSettings, InPort, Shape }
import akka.stream.actor.{ ActorSubscriberMessage, ActorSubscriber }
import akka.stream.scaladsl.FlexiMerge.MergeLogic
import org.reactivestreams.{ Subscription, Subscriber }
import akka.actor.DeadLetterSuppression
import scala.collection.immutable
@ -18,10 +16,10 @@ import scala.collection.immutable
*/
private[akka] object FanIn {
final case class OnError(id: Int, cause: Throwable) extends DeadLetterSuppression
final case class OnComplete(id: Int) extends DeadLetterSuppression
final case class OnNext(id: Int, e: Any) extends DeadLetterSuppression
final case class OnSubscribe(id: Int, subscription: Subscription) extends DeadLetterSuppression
final case class OnError(id: Int, cause: Throwable) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class OnComplete(id: Int) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class OnNext(id: Int, e: Any) extends DeadLetterSuppression with NoSerializationVerificationNeeded
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] {
override def onError(cause: Throwable): Unit = {
@ -264,7 +262,7 @@ private[akka] abstract class FanIn(val settings: ActorFlowMaterializerSettings,
*/
private[akka] object FairMerge {
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
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 {
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
*/
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 scala.collection.immutable
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorRef
import akka.actor.Props
import akka.actor._
import org.reactivestreams.Subscription
import akka.actor.DeadLetterSuppression
/**
* INTERNAL API
*/
private[akka] object FanOut {
final case class SubstreamRequestMore(id: Int, demand: Long) extends DeadLetterSuppression
final case class SubstreamCancel(id: Int) extends DeadLetterSuppression
final case class SubstreamSubscribePending(id: Int) extends DeadLetterSuppression
final case class SubstreamRequestMore(id: Int, demand: Long) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class SubstreamCancel(id: Int) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class SubstreamSubscribePending(id: Int) extends DeadLetterSuppression with NoSerializationVerificationNeeded
class SubstreamSubscription(val parent: ActorRef, val id: Int) extends Subscription {
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)
}
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) {
private var bunchCancelled = false
@ -294,7 +290,7 @@ private[akka] abstract class FanOut(val settings: ActorFlowMaterializerSettings,
*/
private[akka] object Broadcast {
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 {
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 {
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 {
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.Success
import scala.util.Try
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
import akka.actor.Status
import akka.actor.SupervisorStrategy
import akka.actor._
import akka.stream.ActorFlowMaterializerSettings
import akka.pattern.pipe
import org.reactivestreams.Subscriber
import org.reactivestreams.Subscription
import akka.actor.DeadLetterSuppression
import scala.util.control.NonFatal
/**
@ -24,11 +19,11 @@ import scala.util.control.NonFatal
*/
private[akka] object FuturePublisher {
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 {
final case class Cancel(subscription: FutureSubscription) extends DeadLetterSuppression
final case class RequestMore(subscription: FutureSubscription, elements: Long) extends DeadLetterSuppression
final case class Cancel(subscription: FutureSubscription) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class RequestMore(subscription: FutureSubscription, elements: Long) extends DeadLetterSuppression with NoSerializationVerificationNeeded
}
class FutureSubscription(ref: ActorRef) extends Subscription {

View file

@ -4,7 +4,7 @@
package akka.stream.impl
import scala.util.control.NonFatal
import akka.actor.Props
import akka.actor.{ Deploy, Props }
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.Supervision
import akka.stream.scaladsl.Source
@ -14,7 +14,7 @@ import akka.stream.scaladsl.Source
*/
private[akka] object GroupByProcessorImpl {
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
}

View file

@ -5,28 +5,28 @@ package akka.stream.impl
import language.existentials
import org.reactivestreams.Subscription
import akka.actor.DeadLetterSuppression
import akka.actor.{ NoSerializationVerificationNeeded, DeadLetterSuppression }
/**
* INTERNAL API
*/
private[akka] case object SubscribePending extends DeadLetterSuppression
private[akka] case object SubscribePending extends DeadLetterSuppression with NoSerializationVerificationNeeded
/**
* INTERNAL API
*/
private[akka] final case class RequestMore(subscription: ActorSubscription[_], demand: Long)
extends DeadLetterSuppression
extends DeadLetterSuppression with NoSerializationVerificationNeeded
/**
* INTERNAL API
*/
private[akka] final case class Cancel(subscription: ActorSubscription[_])
extends DeadLetterSuppression
extends DeadLetterSuppression with NoSerializationVerificationNeeded
/**
* INTERNAL API
*/
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 akka.stream.ActorFlowMaterializerSettings
import akka.stream.scaladsl.Source
import akka.actor.Props
import akka.actor.{ Deploy, Props }
/**
* INTERNAL API
*/
private[akka] object PrefixAndTailImpl {
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.util.concurrent.atomic.AtomicReference
import akka.actor.{ ActorRef, Props }
import akka.actor.{ Deploy, ActorRef, Props }
import akka.stream.ActorOperationAttributes.Dispatcher
import akka.stream.impl.StreamLayout.Module
import akka.stream.OperationAttributes
@ -83,7 +83,7 @@ private[akka] final class FanoutPublisherSink[In](
val actorMaterializer = ActorFlowMaterializer.downcast(context.materializer)
val fanoutActor = actorMaterializer.actorOf(context,
Props(new FanoutProcessorImpl(actorMaterializer.effectiveSettings(context.effectiveAttributes),
initialBufferSize, maximumBufferSize)))
initialBufferSize, maximumBufferSize)).withDeploy(Deploy.local))
val fanoutProcessor = ActorProcessorFactory[In, In](fanoutActor)
(fanoutProcessor, fanoutProcessor)
}

View file

@ -4,24 +4,21 @@
package akka.stream.impl
import java.util.concurrent.atomic.AtomicReference
import akka.actor.ActorLogging
import akka.actor.Cancellable
import akka.actor.{ Actor, ActorRef }
import akka.actor._
import akka.stream.ActorFlowMaterializerSettings
import org.reactivestreams.{ Publisher, Subscriber, Subscription }
import scala.collection.mutable
import scala.concurrent.duration.FiniteDuration
import akka.actor.DeadLetterSuppression
/**
* INTERNAL API
*/
private[akka] object MultiStreamOutputProcessor {
final case class SubstreamKey(id: Long)
final case class SubstreamRequestMore(substream: SubstreamKey, demand: Long) extends DeadLetterSuppression
final case class SubstreamCancel(substream: SubstreamKey) extends DeadLetterSuppression
final case class SubstreamSubscribe(substream: SubstreamKey, subscriber: Subscriber[Any]) extends DeadLetterSuppression
final case class SubstreamSubscriptionTimeout(substream: SubstreamKey) extends DeadLetterSuppression
final case class SubstreamRequestMore(substream: SubstreamKey, demand: Long) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class SubstreamCancel(substream: SubstreamKey) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class SubstreamSubscribe(substream: SubstreamKey, subscriber: Subscriber[Any]) extends DeadLetterSuppression with NoSerializationVerificationNeeded
final case class SubstreamSubscriptionTimeout(substream: SubstreamKey) extends DeadLetterSuppression with NoSerializationVerificationNeeded
class SubstreamSubscription(val parent: ActorRef, val substreamKey: SubstreamKey) extends Subscription {
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 SubstreamOnNext(key: SubstreamKey, element: Any) extends DeadLetterSuppression
case class SubstreamOnError(key: SubstreamKey, e: Throwable) extends DeadLetterSuppression
case class SubstreamStreamOnSubscribe(key: SubstreamKey, subscription: Subscription) extends DeadLetterSuppression
case class SubstreamOnComplete(key: SubstreamKey) extends DeadLetterSuppression with NoSerializationVerificationNeeded
case class SubstreamOnNext(key: SubstreamKey, element: Any) extends DeadLetterSuppression with NoSerializationVerificationNeeded
case class SubstreamOnError(key: SubstreamKey, e: Throwable) extends DeadLetterSuppression with NoSerializationVerificationNeeded
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) {
// Not driven directly

View file

@ -4,13 +4,12 @@
package akka.stream.impl
import java.util.concurrent.atomic.AtomicBoolean
import akka.actor.{ Actor, ActorRef, Cancellable, Props, SupervisorStrategy }
import akka.actor._
import akka.stream.ActorFlowMaterializerSettings
import org.reactivestreams.{ Subscriber, Subscription }
import scala.collection.mutable
import scala.concurrent.duration.FiniteDuration
import scala.util.control.NonFatal
import akka.actor.DeadLetterSuppression
import akka.event.Logging
/**
@ -19,7 +18,9 @@ import akka.event.Logging
private[akka] object TickPublisher {
def props(initialDelay: FiniteDuration, interval: FiniteDuration, tick: Any,
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 {
case object Cancel extends DeadLetterSuppression

View file

@ -7,11 +7,11 @@ import java.util.LinkedList
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.TimerTransformer
import scala.util.control.NonFatal
import akka.actor.Props
import akka.actor.{ Deploy, Props }
private[akka] object TimerTransformerProcessorsImpl {
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.
*/
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 {
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.{ ActorFlowMaterializer, MaterializationContext }

View file

@ -1,13 +1,12 @@
/**
* 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 akka.stream.impl.SinkModule
import akka.stream.impl.StreamLayout.Module
import akka.stream.io.impl.IOSettings._
import akka.stream.{ ActorFlowMaterializer, MaterializationContext, OperationAttributes, SinkShape }
import akka.util.ByteString
@ -27,7 +26,7 @@ private[akka] final class SynchronousFileSink(f: File, append: Boolean, val attr
val bytesWrittenPromise = Promise[Long]()
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))
(akka.stream.actor.ActorSubscriber[ByteString](ref), bytesWrittenPromise.future)

View file

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

View file

@ -1,11 +1,11 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.io.impl
package akka.stream.impl.io
import java.io.InputStream
import akka.actor.{ ActorLogging, DeadLetterSuppression, Props }
import akka.actor.{ Deploy, ActorLogging, DeadLetterSuppression, Props }
import akka.io.DirectByteBufferPool
import akka.stream.actor.ActorPublisherMessage
import akka.util.ByteString
@ -22,7 +22,7 @@ private[akka] object InputStreamPublisher {
require(initialBuffer > 0, s"initialBuffer must be > 0 (was $initialBuffer)")
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

View file

@ -1,11 +1,11 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.io.impl
package akka.stream.impl.io
import java.io.OutputStream
import akka.actor.{ ActorLogging, Props }
import akka.actor.{ Deploy, ActorLogging, Props }
import akka.stream.actor.{ ActorSubscriberMessage, WatermarkRequestStrategy }
import akka.util.ByteString
@ -15,7 +15,7 @@ import scala.concurrent.Promise
private[akka] object OutputStreamSubscriber {
def props(os: OutputStream, completionPromise: Promise[Long], bufSize: Int) = {
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.Status._
import javax.net.ssl._
import akka.actor.{ Props, Actor, ActorLogging, ActorRef }
import akka.actor._
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.impl.FanIn.InputBunch
import akka.stream.impl.FanOut.OutputBunch
@ -35,7 +35,7 @@ private[akka] object SslTlsCipherActor {
tracing: Boolean,
role: Role,
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 TransportOut = 0

View file

@ -10,7 +10,7 @@ import scala.concurrent.Future
import scala.concurrent.Promise
import scala.concurrent.duration.Duration
import scala.concurrent.duration.FiniteDuration
import akka.actor.Actor
import akka.actor.{ NoSerializationVerificationNeeded, Actor, DeadLetterSuppression }
import akka.io.Inet.SocketOption
import akka.io.Tcp
import akka.stream.ActorFlowMaterializerSettings
@ -20,7 +20,6 @@ import akka.stream.scaladsl.{ Tcp ⇒ StreamTcp }
import akka.util.ByteString
import org.reactivestreams.Processor
import org.reactivestreams.Subscriber
import akka.actor.DeadLetterSuppression
/**
* INTERNAL API
@ -37,7 +36,7 @@ private[akka] object StreamTcpManager {
options: immutable.Traversable[SocketOption],
connectTimeout: Duration,
idleTimeout: Duration)
extends DeadLetterSuppression
extends DeadLetterSuppression with NoSerializationVerificationNeeded
/**
* INTERNAL API
@ -50,13 +49,13 @@ private[akka] object StreamTcpManager {
backlog: Int,
options: immutable.Traversable[SocketOption],
idleTimeout: Duration)
extends DeadLetterSuppression
extends DeadLetterSuppression with NoSerializationVerificationNeeded
/**
* INTERNAL API
*/
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>
*/
package akka.stream.io.impl
package akka.stream.impl.io
import java.io.{ File, RandomAccessFile }
import java.nio.ByteBuffer
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.util.ByteString
@ -22,6 +22,7 @@ private[akka] object SynchronousFilePublisher {
require(maxBuffer >= initialBuffer, s"maxBuffer must be >= initialBuffer (was $maxBuffer)")
Props(classOf[SynchronousFilePublisher], f, completionPromise, chunkSize, initialBuffer, maxBuffer)
.withDeploy(Deploy.local)
}
private final case object Continue extends DeadLetterSuppression

View file

@ -1,12 +1,12 @@
/**
* 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.nio.channels.FileChannel
import akka.actor.{ ActorLogging, Props }
import akka.actor.{ Deploy, ActorLogging, Props }
import akka.stream.actor.{ ActorSubscriberMessage, WatermarkRequestStrategy }
import akka.util.ByteString
@ -16,7 +16,7 @@ import scala.concurrent.Promise
private[akka] object SynchronousFileSubscriber {
def props(f: File, completionPromise: Promise[Long], bufSize: Int, append: Boolean) = {
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,
materializerSettings: ActorFlowMaterializerSettings): Props =
Props(new OutboundTcpStreamActor(processorPromise, localAddressPromise, connectCmd,
materializerSettings)).withDispatcher(materializerSettings.dispatcher)
materializerSettings)).withDispatcher(materializerSettings.dispatcher).withDeploy(Deploy.local)
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 scala.concurrent.{ Future, Promise }
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
import akka.actor._
import akka.io.{ IO, Tcp }
import akka.io.Tcp._
import akka.stream.{ FlowMaterializer, ActorFlowMaterializerSettings }
@ -18,7 +16,6 @@ import akka.util.ByteString
import org.reactivestreams.Subscriber
import akka.stream.ConnectionException
import akka.stream.BindFailedException
import akka.actor.ActorLogging
/**
* INTERNAL API
@ -29,6 +26,7 @@ private[akka] object TcpListenStreamActor {
flowSubscriber: Subscriber[StreamTcp.IncomingConnection],
bindCmd: Tcp.Bind, materializerSettings: ActorFlowMaterializerSettings): Props = {
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 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.{ OperationAttributes, javadsl }

View file

@ -6,7 +6,7 @@ package akka.stream.io
import java.io.OutputStream
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.{ ActorOperationAttributes, OperationAttributes, javadsl }
import akka.util.ByteString

View file

@ -5,8 +5,8 @@ package akka.stream.io
import java.io.File
import akka.stream.impl.io.SynchronousFileSink
import akka.stream.{ OperationAttributes, javadsl, ActorOperationAttributes }
import akka.stream.io.impl.SynchronousFileSink
import akka.stream.scaladsl.Sink
import akka.util.ByteString
@ -28,7 +28,7 @@ object SynchronousFileSink {
* unless configured otherwise by using [[ActorOperationAttributes]].
*/
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

View file

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

View file

@ -10,13 +10,7 @@ import scala.concurrent.{ Promise, ExecutionContext, Future }
import scala.concurrent.duration.Duration
import scala.util.{ Failure, Success }
import scala.util.control.NoStackTrace
import akka.actor.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.actor._
import akka.io.Inet.SocketOption
import akka.io.{ Tcp IoTcp }
import akka.stream._
@ -25,7 +19,6 @@ import akka.stream.impl.ReactiveStreamsCompliance._
import akka.stream.scaladsl._
import akka.util.ByteString
import org.reactivestreams.{ Publisher, Processor, Subscriber, Subscription }
import akka.actor.actorRef2Scala
import akka.stream.impl.io.TcpStreamActor
import akka.stream.impl.io.TcpListenStreamActor
import akka.stream.impl.io.DelayedInitProcessor
@ -77,7 +70,7 @@ class Tcp(system: ExtendedActorSystem) extends akka.actor.Extension {
import Tcp._
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(
val endpoint: InetSocketAddress,