diff --git a/build.fsx b/build.fsx index 44c456b3cc2..a4cf3a1d15f 100644 --- a/build.fsx +++ b/build.fsx @@ -452,7 +452,8 @@ Target "Protobuf" <| fun _ -> ("DistributedPubSubMessages.proto", "/src/contrib/cluster/Akka.Cluster.Tools/PublishSubscribe/Serialization/Proto/"); ("ClusterShardingMessages.proto", "/src/contrib/cluster/Akka.Cluster.Sharding/Serialization/Proto/"); ("TestConductorProtocol.proto", "/src/core/Akka.Remote.TestKit/Proto/"); - ("Persistence.proto", "/src/core/Akka.Persistence/Serialization/Proto/") ] + ("Persistence.proto", "/src/core/Akka.Persistence/Serialization/Proto/"); + ("StreamRefMessages.proto", "/src/core/Akka.Streams/Serialization/Proto/")] printfn "Using proto.exe: %s" protocPath diff --git a/docs/articles/streams/streamrefs.md b/docs/articles/streams/streamrefs.md new file mode 100644 index 00000000000..c63a06bc44a --- /dev/null +++ b/docs/articles/streams/streamrefs.md @@ -0,0 +1,88 @@ +--- +uid: stream-ref +title: StreamRefs - Reactive Streams over the network +--- + +> **Warning** +This module is currently marked as may change in the sense of being the subject of active research. This means that API or semantics can change without warning or deprecation period and it is not recommended to use this module in production just yet—you have been warned. + +Stream references, or “stream refs” for short, allow running Akka Streams across multiple nodes within an Akka Remote boundaries. + +Unlike heavier “streaming data processing” frameworks, Akka Streams are not “deployed” nor automatically distributed. Akka stream refs are, as the name implies, references to existing parts of a stream, and can be used to create a distributed processing framework or introduce such capabilities in specific parts of your application. + +Stream refs are trivial to make use of in existing clustered Akka applications, and require no additional configuration or setup. They automatically maintain flow-control / back-pressure over the network, and employ Akka’s failure detection mechanisms to fail-fast (“let it crash!”) in the case of failures of remote nodes. They can be seen as an implementation of the Work Pulling Pattern, which one would otherwise implement manually. + +> **Note** +A useful way to think about stream refs is: “like an `IActorRef`, but for Akka Streams’s `Source` and `Sink`”. +Stream refs refer to an already existing, possibly remote, `Sink` or `Source`. This is not to be mistaken with deploying streams remotely, which this feature is not intended for. + +## Stream References + +The prime use case for stream refs is to replace raw actor or HTTP messaging between systems where a long running stream of data is expected between two entities. Often times, they can be used to effectively achieve point to point streaming without the need of setting up additional message brokers or similar secondary clusters. + +Stream refs are well suited for any system in which you need to send messages between nodes and need to do so in a flow-controlled fashion. Typical examples include sending work requests to worker nodes, as fast as possible, but not faster than the worker node can process them, or sending data elements which the downstream may be slow at processing. It is recommended to mix and introduce stream refs in Actor messaging based systems, where the actor messaging is used to orchestrate and prepare such message flows, and later the stream refs are used to do the flow-controlled message transfer. + +Stream refs are not persistent, however it is simple to build a recover-able stream by introducing such protocol on the actor messaging layer. Stream refs are absolutely expected to be sent over Akka remoting to other nodes within a cluster, and as such, complement and do not compete with plain Actor messaging. Actors would usually be used to establish the stream, by means of some initial message saying “I want to offer you many log elements (the stream ref)”, or alternatively in the opposite way “If you need to send me much data, here is the stream ref you can use to do so”. + +Since the two sides (“local” and “remote”) of each reference may be confusing to simply refer to as “remote” and “local” – since either side can be seen as “local” or “remote” depending how we look at it – we propose to use the terminology “origin” and “target”, which is defined by where the stream ref was created. For SourceRefs, the “origin” is the side which has the data that it is going to stream out. For SinkRefs the “origin” side is the actor system that is ready to receive the data and has allocated the ref. Those two may be seen as duals of each other, however to explain patterns about sharing references, we found this wording to be rather useful. + +### Source Refs - offering streaming data to a remote system + +A `SourceRef` can be offered to a remote actor system in order for it to consume some source of data that we have prepared locally. + +In order to share a `Source` with a remote endpoint you need to materialize it by running it into the `StreamRefs.SourceRef`. That sink materializes the `ISourceRef` that you can then send to other nodes. Please note that it materializes into a Task so you will have to use the continuation (either `PipeTo` or async/await pattern). + +[!code-csharp[StreamRefsDocTests.cs](../../examples/DocsExamples/Streams/StreamRefsDocTests.cs?name=data-source-actor)] + +The origin actor which creates and owns the `Source` could also perform some validation or additional setup when preparing the source. Once it has handed out the `ISourceRef` the remote side can run it like this: + +[!code-csharp[StreamRefsDocTests.cs](../../examples/DocsExamples/Streams/StreamRefsDocTests.cs?name=source-ref-materialization)] + +The process of preparing and running a `ISourceRef` powered distributed stream is shown by the animation below: + +![source ref](/images/source-ref-animation.gif) + +> **Warning** +A `ISourceRef` is by design “single-shot”. i.e. it may only be materialized once. This is in order to not complicate the mental model what materializing such value would mean. +While stream refs are designed to be single shot, you may use them to mimic multicast scenarios, simply by starting a `Broadcast` stage once, and attaching multiple new streams to it, for each emitting a new stream ref. This way each output of the broadcast is by itself an unique single-shot reference, however they can all be powered using a single `Source` – located before the `Broadcast` stage. + +### Sink Refs - offering to receive streaming data from a remote system + +They can be used to offer the other side the capability to send to the origin side data in a streaming, flow-controlled fashion. The origin here allocates a Sink, which could be as simple as a `Sink.ForEach` or as advanced as a complex sink which streams the incoming data into various other systems (e.g. any of the Alpakka provided Sinks). + +> **Note** +To form a good mental model of `SinkRef`s, you can think of them as being similar to “passive mode” in FTP. + +[!code-csharp[StreamRefsDocTests.cs](../../examples/DocsExamples/Streams/StreamRefsDocTests.cs?name=data-sink-actor)] + +Using the offered `ISinkRef<>` to send data to the origin of the `Sink` is also simple, as we can treat the `ISinkRef<>` just as any other sink and directly runWith or run with it. + +[!code-csharp[StreamRefsDocTests.cs](../../examples/DocsExamples/Streams/StreamRefsDocTests.cs?name=sink-ref-materialization)] + +The process of preparing and running a `ISinkRef<>` powered distributed stream is shown by the animation below: + +![sink ref](/images/sink-ref-animation.gif) + +> **Warning** +A `ISinkRef<>` is *by design* “single-shot”. i.e. it may only be materialized once. This is in order to not complicate the mental model what materializing such value would mean. If you have an use case for building a fan-in operation accepting writes from multiple remote nodes, you can build your `Sink` and prepend it with a `Merge` stage, each time materializing a new `ISinkRef<>` targeting that `Merge`. This has the added benefit of giving you full control how to merge these streams (i.e. by using “merge preferred” or any other variation of the fan-in stages). + +## Configuration + +### Stream reference subscription timeouts + +All stream references have a subscription timeout, which is intended to prevent resource leaks in situations in which a remote node would requests the allocation of many streams yet never actually run them. In order to prevent this, each stream reference has a default timeout (of 30 seconds), after which the origin will abort the stream offer if the target has not materialized the stream ref in time. After the timeout has triggered, materialization of the target side will fail pointing out that the origin is missing. + +Since these timeouts are often very different based on the kind of stream offered, and there can be many different kinds of them in the same application, it is possible to not only configure this setting globally (`akka.stream.materializer.stream-ref.subscription-timeout`), but also via attributes: + +```csharp +Source.Repeat("hello") + .RunWith(StreamRefs.SourceRef() + .AddAttributes(StreamRefAttributes + .SubscriptionTimeout(TimeSpan.FromSeconds(5))) + , materializer); + +StreamRefs.SinkRef() + .AddAttributes(StreamRefAttributes + .SubscriptionTimeout(TimeSpan.FromSeconds(5))) + .RunWith(Sink.Ignore(), materializer); +``` \ No newline at end of file diff --git a/docs/examples/DocsExamples/Streams/StreamRefsDocTests.cs b/docs/examples/DocsExamples/Streams/StreamRefsDocTests.cs new file mode 100644 index 00000000000..380ec2f8754 --- /dev/null +++ b/docs/examples/DocsExamples/Streams/StreamRefsDocTests.cs @@ -0,0 +1,140 @@ +using System; +using System.Collections.Concurrent; +using System.Collections.Generic; +using System.Threading.Tasks; +using Akka; +using Akka.Streams; +using Akka.Streams.Dsl; +using Akka.TestKit.Xunit2; +using Xunit; +using Xunit.Abstractions; +using Akka.Actor; +using Akka.IO; +using Akka.Util; +using System.Linq; + +namespace DocsExamples.Streams +{ + public class StreamRefsDocTests : TestKit + { + #region data-source-actor + public sealed class RequestLogs + { + public int StreamId { get; } + + public RequestLogs(int streamId) + { + StreamId = streamId; + } + } + + public sealed class LogsOffer + { + public int StreamId { get; } + public ISourceRef SourceRef { get; } + + public LogsOffer(int streamId, ISourceRef sourceRef) + { + StreamId = streamId; + SourceRef = sourceRef; + } + } + + public class DataSource : ReceiveActor + { + public DataSource() + { + Receive(request => + { + // create a source + StreamLogs(request.StreamId) + // materialize it using stream refs + .RunWith(StreamRefs.SourceRef(), Context.System.Materializer()) + // and send to sender + .PipeTo(Sender, success: sourceRef => new LogsOffer(request.StreamId, sourceRef)); + }); + } + + private Source StreamLogs(int streamId) => + Source.From(Enumerable.Range(1, 100)).Select(i => i.ToString()); + } + #endregion + + #region data-sink-actor + public sealed class PrepareUpload + { + public string Id { get; } + public PrepareUpload(string id) + { + Id = id; + } + } + + public sealed class MeasurementsSinkReady + { + public string Id { get; } + public ISinkRef SinkRef { get; } + public MeasurementsSinkReady(string id, ISinkRef sinkRef) + { + Id = id; + SinkRef = sinkRef; + } + } + + class DataReceiver : ReceiveActor + { + public DataReceiver() + { + Receive(prepare => + { + // obtain a source you want to offer + var sink = LogsSinksFor(prepare.Id); + + // materialize sink ref (remote is source data for us) + StreamRefs.SinkRef() + .To(sink) + .Run(Context.System.Materializer()) + .PipeTo(Sender, success: sinkRef => new MeasurementsSinkReady(prepare.Id, sinkRef)); + }); + } + + private Sink LogsSinksFor(string id) => + Sink.ForEach(Console.WriteLine); + } + #endregion + + private ActorMaterializer Materializer { get; } + + public StreamRefsDocTests(ITestOutputHelper output) + : base("", output) + { + Materializer = Sys.Materializer(); + } + + [Fact] + public async Task SourceRef_must_propagate_source_from_another_system() + { + #region source-ref-materialization + var sourceActor = Sys.ActorOf(Props.Create(), "dataSource"); + + var offer = await sourceActor.Ask(new RequestLogs(1337)); + await offer.SourceRef.Source.RunForeach(Console.WriteLine, Materializer); + #endregion + } + + [Fact] + public async Task SinkRef_must_receive_messages_from_another_system() + { + #region sink-ref-materialization + var receiver = Sys.ActorOf(Props.Create(), "receiver"); + + var ready = await receiver.Ask(new PrepareUpload("id"), timeout: TimeSpan.FromSeconds(30)); + + // stream local metrics to Sink's origin: + Source.From(Enumerable.Range(1, 100)) + .Select(i => i.ToString()) + .RunWith(ready.SinkRef.Sink, Materializer); + #endregion + } + } +} \ No newline at end of file diff --git a/docs/images/sink-ref-animation.gif b/docs/images/sink-ref-animation.gif new file mode 100644 index 00000000000..52e26d2103c Binary files /dev/null and b/docs/images/sink-ref-animation.gif differ diff --git a/docs/images/source-ref-animation.gif b/docs/images/source-ref-animation.gif new file mode 100644 index 00000000000..76fd2c936dc Binary files /dev/null and b/docs/images/source-ref-animation.gif differ diff --git a/src/Akka.sln b/src/Akka.sln index 1df2814800f..817a57953b1 100644 --- a/src/Akka.sln +++ b/src/Akka.sln @@ -57,6 +57,7 @@ Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Protobuf", "Protobuf", "{98 protobuf\SystemMessageFormats.proto = protobuf\SystemMessageFormats.proto protobuf\TestConductorProtocol.proto = protobuf\TestConductorProtocol.proto protobuf\WireFormats.proto = protobuf\WireFormats.proto + protobuf\StreamRefMessages.proto = protobuf\StreamRefMessages.proto EndProjectSection EndProject Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Build", "Build", "{10C5B1E8-15B5-4EB3-81AE-1FC054FE1305}" diff --git a/src/core/Akka.API.Tests/CoreAPISpec.ApproveCore.approved.txt b/src/core/Akka.API.Tests/CoreAPISpec.ApproveCore.approved.txt index b2a78a7fe9c..1acc2dc5509 100644 --- a/src/core/Akka.API.Tests/CoreAPISpec.ApproveCore.approved.txt +++ b/src/core/Akka.API.Tests/CoreAPISpec.ApproveCore.approved.txt @@ -113,6 +113,7 @@ namespace Akka.Actor protected void Stash(Akka.Dispatch.SysMsg.SystemMessage msg) { } public void Stop(Akka.Actor.IActorRef child) { } public void Stop() { } + protected void StopFunctionRefs() { } public void Suspend() { } protected void TellWatchersWeDied() { } public void TerminatedQueuedFor(Akka.Actor.IActorRef subject) { } @@ -1702,12 +1703,15 @@ namespace Akka.Actor protected override void PreRestart(System.Exception reason, object message) { } protected override bool Receive(object message) { } } - public sealed class Terminated : Akka.Actor.IAutoReceivedMessage, Akka.Actor.INoSerializationVerificationNeeded, Akka.Actor.IPossiblyHarmful, Akka.Event.IDeadLetterSuppression + public sealed class Terminated : Akka.Actor.IAutoReceivedMessage, Akka.Actor.INoSerializationVerificationNeeded, Akka.Actor.IPossiblyHarmful, Akka.Event.IDeadLetterSuppression, System.IEquatable { public Terminated(Akka.Actor.IActorRef actorRef, bool existenceConfirmed, bool addressTerminated) { } public Akka.Actor.IActorRef ActorRef { get; } public bool AddressTerminated { get; } public bool ExistenceConfirmed { get; } + public bool Equals(Akka.Actor.Terminated other) { } + public override bool Equals(object obj) { } + public override int GetHashCode() { } public override string ToString() { } } public class TerminatedProps : Akka.Actor.Props @@ -4554,6 +4558,7 @@ namespace Akka.Util { public const string Base64Chars = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789+~"; public static string Base64Encode(this long value) { } + public static System.Text.StringBuilder Base64Encode(this long value, System.Text.StringBuilder sb) { } public static string Base64Encode(this string s) { } } public class static BitArrayHelpers diff --git a/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt b/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt index 595d16b2a59..7cd7a64802f 100644 --- a/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt +++ b/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt @@ -72,10 +72,11 @@ namespace Akka.Streams public readonly int MaxFixedBufferSize; public readonly int MaxInputBufferSize; public readonly int OutputBurstLimit; + public readonly Akka.Streams.Dsl.StreamRefSettings StreamRefSettings; public readonly Akka.Streams.StreamSubscriptionTimeoutSettings SubscriptionTimeoutSettings; public readonly Akka.Streams.Supervision.Decider SupervisionDecider; public readonly int SyncProcessingLimit; - public ActorMaterializerSettings(int initialInputBufferSize, int maxInputBufferSize, string dispatcher, Akka.Streams.Supervision.Decider supervisionDecider, Akka.Streams.StreamSubscriptionTimeoutSettings subscriptionTimeoutSettings, bool isDebugLogging, int outputBurstLimit, bool isFuzzingMode, bool isAutoFusing, int maxFixedBufferSize, int syncProcessingLimit = 1000) { } + public ActorMaterializerSettings(int initialInputBufferSize, int maxInputBufferSize, string dispatcher, Akka.Streams.Supervision.Decider supervisionDecider, Akka.Streams.StreamSubscriptionTimeoutSettings subscriptionTimeoutSettings, Akka.Streams.Dsl.StreamRefSettings streamRefSettings, bool isDebugLogging, int outputBurstLimit, bool isFuzzingMode, bool isAutoFusing, int maxFixedBufferSize, int syncProcessingLimit = 1000) { } public static Akka.Streams.ActorMaterializerSettings Create(Akka.Actor.ActorSystem system) { } public Akka.Streams.ActorMaterializerSettings WithAutoFusing(bool isAutoFusing) { } public Akka.Streams.ActorMaterializerSettings WithDebugLogging(bool isEnabled) { } @@ -83,6 +84,7 @@ namespace Akka.Streams public Akka.Streams.ActorMaterializerSettings WithFuzzingMode(bool isFuzzingMode) { } public Akka.Streams.ActorMaterializerSettings WithInputBuffer(int initialSize, int maxSize) { } public Akka.Streams.ActorMaterializerSettings WithMaxFixedBufferSize(int maxFixedBufferSize) { } + public Akka.Streams.ActorMaterializerSettings WithStreamRefSettings(Akka.Streams.Dsl.StreamRefSettings settings) { } public Akka.Streams.ActorMaterializerSettings WithSubscriptionTimeoutSettings(Akka.Streams.StreamSubscriptionTimeoutSettings settings) { } public Akka.Streams.ActorMaterializerSettings WithSupervisionStrategy(Akka.Streams.Supervision.Decider decider) { } public Akka.Streams.ActorMaterializerSettings WithSyncProcessingLimit(int limit) { } @@ -605,11 +607,27 @@ namespace Akka.Streams { protected InPort() { } } + public sealed class InvalidPartnerActorException : Akka.Pattern.IllegalStateException + { + public InvalidPartnerActorException(Akka.Actor.IActorRef expectedRef, Akka.Actor.IActorRef gotRef, string message) { } + public Akka.Actor.IActorRef ExpectedRef { get; } + public Akka.Actor.IActorRef GotRef { get; } + } + public sealed class InvalidSequenceNumberException : Akka.Pattern.IllegalStateException + { + public InvalidSequenceNumberException(long expectedSeqNr, long gotSeqNr, string message) { } + public long ExpectedSeqNr { get; } + public long GotSeqNr { get; } + } public interface IQueueOfferResult { } public interface ISinkQueue { System.Threading.Tasks.Task> PullAsync(); } + public interface ISinkRef + { + Akka.Streams.Dsl.Sink Sink { get; } + } public interface ISourceQueue { System.Threading.Tasks.Task OfferAsync(T element); @@ -621,6 +639,10 @@ namespace Akka.Streams void Fail(System.Exception ex); new System.Threading.Tasks.Task WatchCompletionAsync(); } + public interface ISourceRef + { + Akka.Streams.Dsl.Source Source { get; } + } public interface ITransformerLike { bool IsComplete { get; } @@ -719,6 +741,10 @@ namespace Akka.Streams public static readonly Akka.Streams.QueueOfferResult.QueueClosed Instance; } } + public sealed class RemoteStreamRefActorTerminatedException : System.Exception + { + public RemoteStreamRefActorTerminatedException(string message) { } + } public abstract class Shape : System.ICloneable { protected Shape() { } @@ -765,6 +791,20 @@ namespace Akka.Streams public StreamLimitReachedException(long max) { } protected StreamLimitReachedException(System.Runtime.Serialization.SerializationInfo info, System.Runtime.Serialization.StreamingContext context) { } } + public class static StreamRefAttributes + { + public static Akka.Streams.Attributes CreateSubscriptionTimeout(System.TimeSpan timeout) { } + public interface IStreamRefAttribute : Akka.Streams.Attributes.IAttribute { } + public sealed class SubscriptionTimeout : Akka.Streams.Attributes.IAttribute, Akka.Streams.StreamRefAttributes.IStreamRefAttribute + { + public SubscriptionTimeout(System.TimeSpan timeout) { } + public System.TimeSpan Timeout { get; } + } + } + public sealed class StreamRefSubscriptionTimeoutException : Akka.Pattern.IllegalStateException + { + public StreamRefSubscriptionTimeoutException(string message) { } + } public sealed class StreamSubscriptionTimeoutSettings : System.IEquatable { public readonly Akka.Streams.StreamSubscriptionTimeoutTerminationMode Mode; @@ -793,6 +833,10 @@ namespace Akka.Streams Propagate = 0, Drain = 1, } + public sealed class TargetRefNotInitializedYetException : Akka.Pattern.IllegalStateException + { + public TargetRefNotInitializedYetException() { } + } public enum ThrottleMode { Shaping = 0, @@ -1631,7 +1675,7 @@ namespace Akka.Streams.Dsl public static Akka.Streams.Dsl.Source FromPublisher(Reactive.Streams.IPublisher publisher) { } public static Akka.Streams.Dsl.Source FromTask(System.Threading.Tasks.Task task) { } public static Akka.Streams.Dsl.Source> Lazily(System.Func> create) { } - public static Akka.Streams.Dsl.Source> Maybe() { } + public static Akka.Streams.Dsl.Source>> Maybe() { } public static Akka.Streams.Dsl.Source> Queue(int bufferSize, Akka.Streams.OverflowStrategy overflowStrategy) { } public static Akka.Streams.Dsl.Source Repeat(T element) { } public static Akka.Streams.SourceShape Shape(string name) { } @@ -1764,6 +1808,27 @@ namespace Akka.Streams.Dsl public static Akka.Streams.Dsl.Source> FromInputStream(System.Func createInputStream, int chunkSize = 8192) { } public static Akka.Streams.Dsl.Sink> FromOutputStream(System.Func createOutputStream, bool autoFlush = False) { } } + [Akka.Annotations.ApiMayChangeAttribute()] + public class static StreamRefs + { + [Akka.Annotations.ApiMayChangeAttribute()] + public static Akka.Streams.Dsl.Source>> SinkRef() { } + [Akka.Annotations.ApiMayChangeAttribute()] + public static Akka.Streams.Dsl.Sink>> SourceRef() { } + } + public sealed class StreamRefSettings + { + public StreamRefSettings(int bufferCapacity, System.TimeSpan demandRedeliveryInterval, System.TimeSpan subscriptionTimeout) { } + public int BufferCapacity { get; } + public System.TimeSpan DemandRedeliveryInterval { get; } + public string ProductPrefix { get; } + public System.TimeSpan SubscriptionTimeout { get; } + public Akka.Streams.Dsl.StreamRefSettings Copy(System.Nullable bufferCapacity = null, System.Nullable demandRedeliveryInterval = null, System.Nullable subscriptionTimeout = null) { } + public static Akka.Streams.Dsl.StreamRefSettings Create(Akka.Configuration.Config config) { } + public Akka.Streams.Dsl.StreamRefSettings WithBufferCapacity(int value) { } + public Akka.Streams.Dsl.StreamRefSettings WithDemandRedeliveryInterval(System.TimeSpan value) { } + public Akka.Streams.Dsl.StreamRefSettings WithSubscriptionTimeout(System.TimeSpan value) { } + } public abstract class SubFlow : Akka.Streams.Dsl.IFlow { protected SubFlow() { } @@ -2847,12 +2912,12 @@ namespace Akka.Streams.Implementation } } [Akka.Annotations.InternalApiAttribute()] - public sealed class MaybeSource : Akka.Streams.Implementation.SourceModule> + public sealed class MaybeSource : Akka.Streams.Implementation.SourceModule>> { public MaybeSource(Akka.Streams.Attributes attributes, Akka.Streams.SourceShape shape) { } public override Akka.Streams.Attributes Attributes { get; } public override Reactive.Streams.IPublisher Create(Akka.Streams.MaterializationContext context, out System.Threading.Tasks.TaskCompletionSource<> materializer) { } - protected override Akka.Streams.Implementation.SourceModule> NewInstance(Akka.Streams.SourceShape shape) { } + protected override Akka.Streams.Implementation.SourceModule>> NewInstance(Akka.Streams.SourceShape shape) { } public override Akka.Streams.Implementation.IModule WithAttributes(Akka.Streams.Attributes attributes) { } } public abstract class Module : Akka.Streams.Implementation.IModule, System.IComparable @@ -4114,6 +4179,16 @@ namespace Akka.Streams.IO public static Akka.Streams.IO.IOResult Success(long count) { } } } +namespace Akka.Streams.Serialization +{ + public sealed class StreamRefSerializer : Akka.Serialization.SerializerWithStringManifest + { + public StreamRefSerializer(Akka.Actor.ExtendedActorSystem system) { } + public override object FromBinary(byte[] bytes, string manifest) { } + public override string Manifest(object o) { } + public override byte[] ToBinary(object o) { } + } +} namespace Akka.Streams.Stage { [System.ObsoleteAttribute("Please use GraphStage instead. [1.1.2]")] @@ -4207,7 +4282,9 @@ namespace Akka.Streams.Stage public Akka.Event.ILoggingAdapter Log { get; } protected object LogSource { get; } protected Akka.Streams.IMaterializer Materializer { get; } - public Akka.Streams.Stage.StageActorRef StageActorRef { get; } + public Akka.Streams.Stage.StageActor StageActor { get; } + [Akka.Annotations.ApiMayChangeAttribute()] + protected virtual string StageActorName { get; } protected Akka.Streams.IMaterializer SubFusingMaterializer { get; } protected internal void AbortEmitting(Akka.Streams.Outlet outlet) { } protected void AbortReading(Akka.Streams.Inlet inlet) { } @@ -4232,7 +4309,7 @@ namespace Akka.Streams.Stage protected Akka.Streams.Stage.IInHandler GetHandler(Akka.Streams.Inlet inlet) { } protected Akka.Streams.Stage.IOutHandler GetHandler(Akka.Streams.Outlet outlet) { } [Akka.Annotations.ApiMayChangeAttribute()] - protected Akka.Streams.Stage.StageActorRef GetStageActorRef(Akka.Streams.Stage.StageActorRef.Receive receive) { } + protected Akka.Streams.Stage.StageActor GetStageActor(Akka.Streams.Stage.StageActorRef.Receive receive) { } protected internal T Grab(Akka.Streams.Inlet inlet) { } protected bool HasBeenPulled(Akka.Streams.Inlet inlet) { } protected internal bool IsAvailable(Akka.Streams.Inlet inlet) { } @@ -4480,21 +4557,17 @@ namespace Akka.Streams.Stage protected PushStage() { } public virtual Akka.Streams.Stage.ISyncDirective OnPull(Akka.Streams.Stage.IContext context) { } } - public sealed class StageActorRef : Akka.Actor.MinimalActorRef + public sealed class StageActor { - public readonly Akka.Event.ILoggingAdapter Log; - public static readonly Akka.Streams.Implementation.EnumerableActorName Name; - public readonly System.Collections.Immutable.IImmutableSet StageTerminatedTombstone; - public StageActorRef(Akka.Actor.IActorRefProvider provider, Akka.Event.ILoggingAdapter log, System.Func>> getAsyncCallback, Akka.Streams.Stage.StageActorRef.Receive initialReceive, Akka.Actor.ActorPath path) { } - public override bool IsTerminated { get; } - public override Akka.Actor.ActorPath Path { get; } - public override Akka.Actor.IActorRefProvider Provider { get; } - public void Become(Akka.Streams.Stage.StageActorRef.Receive behavior) { } - public override void SendSystemMessage(Akka.Dispatch.SysMsg.ISystemMessage message) { } - public override void Stop() { } - protected override void TellInternal(object message, Akka.Actor.IActorRef sender) { } + public StageActor(Akka.Streams.ActorMaterializer materializer, System.Func>> getAsyncCallback, Akka.Streams.Stage.StageActorRef.Receive initialReceive, string name = null) { } + public Akka.Actor.IActorRef Ref { get; } + public void Become(Akka.Streams.Stage.StageActorRef.Receive receive) { } + public void Stop() { } public void Unwatch(Akka.Actor.IActorRef actorRef) { } public void Watch(Akka.Actor.IActorRef actorRef) { } + } + public class static StageActorRef + { public delegate void Receive(System.Tuple args); } public class StageActorRefNotInitializedException : System.Exception diff --git a/src/core/Akka.Streams.TestKit/TestSink.cs b/src/core/Akka.Streams.TestKit/TestSink.cs index c7b0a21a852..52aef721225 100644 --- a/src/core/Akka.Streams.TestKit/TestSink.cs +++ b/src/core/Akka.Streams.TestKit/TestSink.cs @@ -5,6 +5,7 @@ // //----------------------------------------------------------------------- +using Akka.Actor; using Akka.Streams.Dsl; using Akka.TestKit; @@ -18,9 +19,7 @@ public static class TestSink /// /// /// - public static Sink> SinkProbe(this TestKitBase testKit) - { - return new Sink>(new StreamTestKit.ProbeSink(testKit, Attributes.None, new SinkShape(new Inlet("ProbeSink.in")))); - } + public static Sink> SinkProbe(this TestKitBase testKit) => + new Sink>(new StreamTestKit.ProbeSink(testKit, Attributes.None, new SinkShape(new Inlet("ProbeSink.in")))); } } diff --git a/src/core/Akka.Streams.Tests/Akka.Streams.Tests.csproj b/src/core/Akka.Streams.Tests/Akka.Streams.Tests.csproj index 60a7b72c1cf..c4ebd2dac7d 100644 --- a/src/core/Akka.Streams.Tests/Akka.Streams.Tests.csproj +++ b/src/core/Akka.Streams.Tests/Akka.Streams.Tests.csproj @@ -1,19 +1,17 @@  - Akka.Streams.Tests net452;netcoreapp1.1 - + - @@ -21,33 +19,27 @@ - - - - TRACE;DEBUG;SERIALIZATION;CONFIGURATION;UNSAFE_THREADING;NET452;NET452 $(DefineConstants);SERIALIZATION;CONFIGURATION;UNSAFE_THREADING;AKKAIO - $(DefineConstants);CORECLR - $(DefineConstants);RELEASE - + \ No newline at end of file diff --git a/src/core/Akka.Streams.Tests/Dsl/FlowScanSpec.cs b/src/core/Akka.Streams.Tests/Dsl/FlowScanSpec.cs index 6d3f08f7481..ffa41c30488 100644 --- a/src/core/Akka.Streams.Tests/Dsl/FlowScanSpec.cs +++ b/src/core/Akka.Streams.Tests/Dsl/FlowScanSpec.cs @@ -13,6 +13,7 @@ using Akka.Streams.Supervision; using Akka.Streams.TestKit; using Akka.Streams.TestKit.Tests; +using Akka.Streams.Util; using Akka.TestKit; using FluentAssertions; using Xunit; @@ -24,7 +25,7 @@ public class FlowScanSpec : AkkaSpec { private ActorMaterializer Materializer { get; } - public FlowScanSpec(ITestOutputHelper helper):base(helper) + public FlowScanSpec(ITestOutputHelper helper) : base(helper) { var settings = ActorMaterializerSettings.Create(Sys).WithInputBuffer(2, 16); Materializer = ActorMaterializer.Create(Sys, settings); @@ -43,13 +44,13 @@ private IEnumerable Scan(Source source, TimeSpan? duration = t.Wait(duration.Value).Should().BeTrue(); return t.Result; } - + [Fact] public void A_Scan_must_Scan() { Func scan = source => { - var result = new int[source.Length+1]; + var result = new int[source.Length + 1]; result[0] = 0; for (var i = 1; i <= source.Length; i++) @@ -79,19 +80,19 @@ public void A_Scan_must_Scan_empty_failed() [Fact] public void A_Scan_must_Scan_empty() => - this.AssertAllStagesStopped(() => Scan(Source.Empty()).ShouldAllBeEquivalentTo(new[] {0}), Materializer); + this.AssertAllStagesStopped(() => Scan(Source.Empty()).ShouldAllBeEquivalentTo(new[] { 0 }), Materializer); [Fact] public void A_Scan_must_emit_values_promptly() { - var task = Source.Single(1).MapMaterializedValue>(_ => null) + var task = Source.Single(1).MapMaterializedValue>>(_ => null) .Concat(Source.Maybe()) .Scan(0, (i, i1) => i + i1) .Take(2) .RunWith(Sink.Seq(), Materializer); task.Wait(TimeSpan.FromSeconds(1)).Should().BeTrue(); - task.Result.ShouldAllBeEquivalentTo(new[] {0, 1}); + task.Result.ShouldAllBeEquivalentTo(new[] { 0, 1 }); } [Fact] @@ -105,11 +106,11 @@ public void A_Scan_must_restart_properly() return old + current; }).WithAttributes(ActorAttributes.CreateSupervisionStrategy(Deciders.RestartingDecider)); - Source.From(new[] {1, 3, -1, 5, 7}) + Source.From(new[] { 1, 3, -1, 5, 7 }) .Via(scan) .RunWith(this.SinkProbe(), Materializer) .ToStrict(TimeSpan.FromSeconds(1)) - .ShouldAllBeEquivalentTo(new[] {0, 1, 4, 0, 5, 12}); + .ShouldAllBeEquivalentTo(new[] { 0, 1, 4, 0, 5, 12 }); } @@ -124,13 +125,13 @@ public void A_Scan_must_resume_properly() return old + current; }).WithAttributes(ActorAttributes.CreateSupervisionStrategy(Deciders.ResumingDecider)); - Source.From(new[] {1, 3, -1, 5, 7}) + Source.From(new[] { 1, 3, -1, 5, 7 }) .Via(scan) .RunWith(this.SinkProbe(), Materializer) .ToStrict(TimeSpan.FromSeconds(1)) - .ShouldAllBeEquivalentTo(new[] {0, 1, 4, 9, 16}); + .ShouldAllBeEquivalentTo(new[] { 0, 1, 4, 9, 16 }); } - + [Fact] public void A_Scan_must_scan_normally_for_empty_source() { diff --git a/src/core/Akka.Streams.Tests/Dsl/FlowSplitWhenSpec.cs b/src/core/Akka.Streams.Tests/Dsl/FlowSplitWhenSpec.cs index 7ca5b5f6c60..c4afe91165e 100644 --- a/src/core/Akka.Streams.Tests/Dsl/FlowSplitWhenSpec.cs +++ b/src/core/Akka.Streams.Tests/Dsl/FlowSplitWhenSpec.cs @@ -14,6 +14,7 @@ using Akka.Streams.Implementation; using Akka.Streams.TestKit; using Akka.Streams.TestKit.Tests; +using Akka.Streams.Util; using Akka.TestKit; using FluentAssertions; using Reactive.Streams; @@ -49,7 +50,7 @@ public StreamPuppet(IPublisher p, TestKitBase kit) p.Subscribe(_probe); _subscription = _probe.ExpectSubscription(); } - + public void Request(int demand) => _subscription.Request(demand); public void ExpectNext(int element) => _probe.ExpectNext(element); @@ -144,7 +145,7 @@ public void SplitWhen_must_work_when_first_element_is_split_by() WithSubstreamsSupport(1, 3, run: (masterSubscriber, masterSubscription, getSubFlow) => { var s1 = new StreamPuppet(getSubFlow().RunWith(Sink.AsPublisher(false), Materializer), this); - + s1.Request(5); s1.ExpectNext(1); s1.ExpectNext(2); @@ -365,7 +366,7 @@ public void SplitWhen_must_fail_stream_if_substream_not_materialized_in_time() var testSource = Source.Single(1) - .MapMaterializedValue>(_ => null) + .MapMaterializedValue>>(_ => null) .Concat(Source.Maybe()) .SplitWhen(_ => true); Action action = () => @@ -373,7 +374,7 @@ public void SplitWhen_must_fail_stream_if_substream_not_materialized_in_time() var task = testSource.Lift() .Delay(TimeSpan.FromSeconds(1)) - .ConcatMany(s => s.MapMaterializedValue>(_ => null)) + .ConcatMany(s => s.MapMaterializedValue>>(_ => null)) .RunWith(Sink.Ignore(), tightTimeoutMaterializer); task.Wait(TimeSpan.FromSeconds(3)).Should().BeTrue(); }; diff --git a/src/core/Akka.Streams.Tests/Dsl/GraphStageTimersSpec.cs b/src/core/Akka.Streams.Tests/Dsl/GraphStageTimersSpec.cs index 1eb5bba0bb5..6bf1b2605f7 100644 --- a/src/core/Akka.Streams.Tests/Dsl/GraphStageTimersSpec.cs +++ b/src/core/Akka.Streams.Tests/Dsl/GraphStageTimersSpec.cs @@ -13,6 +13,7 @@ using Akka.Streams.Stage; using Akka.Streams.TestKit; using Akka.Streams.TestKit.Tests; +using Akka.Streams.Util; using Akka.TestKit; using FluentAssertions; using Xunit; @@ -41,7 +42,7 @@ private SideChannel SetupIsolatedStage() .To(Sink.Ignore()) .Run(Materializer); channel.StopPromise = stopPromise; - AwaitCondition(()=>channel.IsReady); + AwaitCondition(() => channel.IsReady); return channel; } @@ -217,7 +218,7 @@ public override bool Equals(object obj) private sealed class SideChannel { public volatile Action AsyncCallback; - public volatile TaskCompletionSource StopPromise; + public volatile TaskCompletionSource> StopPromise; public bool IsReady => AsyncCallback != null; public void Tell(object message) => AsyncCallback(message); @@ -291,7 +292,7 @@ public TestStage(IActorRef probe, SideChannel sideChannel, TestKitBase testKit) protected override GraphStageLogic CreateLogic(Attributes inheritedAttributes) => new Logic(this); } - + private sealed class TestStage2 : SimpleLinearGraphStage { private sealed class Logic : TimerGraphStageLogic @@ -304,7 +305,7 @@ public Logic(TestStage2 stage) : base(stage.Shape) { _stage = stage; - SetHandler(stage.Inlet, onPush: DoNothing, + SetHandler(stage.Inlet, onPush: DoNothing, onUpstreamFinish: CompleteStage, onUpstreamFailure: FailStage); @@ -317,9 +318,9 @@ public Logic(TestStage2 stage) : base(stage.Shape) protected internal override void OnTimer(object timerKey) { _tickCount++; - if(IsAvailable(_stage.Outlet)) + if (IsAvailable(_stage.Outlet)) Push(_stage.Outlet, _tickCount); - if(_tickCount == 3) + if (_tickCount == 3) CancelTimer(TimerKey); } } diff --git a/src/core/Akka.Streams.Tests/Dsl/HubSpec.cs b/src/core/Akka.Streams.Tests/Dsl/HubSpec.cs index fb34210cf7f..f7a3821ed5e 100644 --- a/src/core/Akka.Streams.Tests/Dsl/HubSpec.cs +++ b/src/core/Akka.Streams.Tests/Dsl/HubSpec.cs @@ -18,6 +18,7 @@ using FluentAssertions; using Xunit; using Akka.Actor; +using Akka.Streams.Util; using Akka.Util.Internal; using Xunit.Abstractions; @@ -292,7 +293,7 @@ public void BroadcastHub_must_send_the_same_elements_to_consumers_attaching_arou this.AssertAllStagesStopped(() => { var other = Source.From(Enumerable.Range(2, 9)) - .MapMaterializedValue>(_ => null); + .MapMaterializedValue>>(_ => null); var t = Source.Maybe() .Concat(other) .ToMaterialized(BroadcastHub.Sink(8), Keep.Both) @@ -317,7 +318,7 @@ public void BroadcastHub_must_send_the_same_prefix_to_consumers_attaching_around this.AssertAllStagesStopped(() => { var other = Source.From(Enumerable.Range(2, 19)) - .MapMaterializedValue>(_ => null); + .MapMaterializedValue>>(_ => null); var t = Source.Maybe() .Concat(other) .ToMaterialized(BroadcastHub.Sink(8), Keep.Both) @@ -359,7 +360,7 @@ public void BroadcastHub_must_send_the_same_elements_to_consumers_of_different_s this.AssertAllStagesStopped(() => { var other = Source.From(Enumerable.Range(2, 9)) - .MapMaterializedValue>(_ => null); + .MapMaterializedValue>>(_ => null); var t = Source.Maybe() .Concat(other) .ToMaterialized(BroadcastHub.Sink(8), Keep.Both) @@ -385,7 +386,7 @@ public void BroadcastHub_must_send_the_same_elements_to_consumers_of_attaching_a this.AssertAllStagesStopped(() => { var other = Source.From(Enumerable.Range(2, 9)) - .MapMaterializedValue>(_ => null); + .MapMaterializedValue>>(_ => null); var t = Source.Maybe() .Concat(other) .Throttle(1, TimeSpan.FromMilliseconds(10), 3, ThrottleMode.Shaping) @@ -411,7 +412,7 @@ public void BroadcastHub_must_ensure_that_from_two_different_speed_consumers_the this.AssertAllStagesStopped(() => { var other = Source.From(Enumerable.Range(2, 19)) - .MapMaterializedValue>(_ => null); + .MapMaterializedValue>>(_ => null); var t = Source.Maybe() .Concat(other) .ToMaterialized(BroadcastHub.Sink(1), Keep.Both) @@ -441,7 +442,7 @@ public void BroadcastHub_must_send_the_same_elements_to_consumers_attaching_arou this.AssertAllStagesStopped(() => { var other = Source.From(Enumerable.Range(2, 9)) - .MapMaterializedValue>(_ => null); + .MapMaterializedValue>>(_ => null); var t = Source.Maybe() .Concat(other) .ToMaterialized(BroadcastHub.Sink(1), Keep.Both) diff --git a/src/core/Akka.Streams.Tests/Dsl/SourceSpec.cs b/src/core/Akka.Streams.Tests/Dsl/SourceSpec.cs index 56af56fb43f..13d41695ccf 100644 --- a/src/core/Akka.Streams.Tests/Dsl/SourceSpec.cs +++ b/src/core/Akka.Streams.Tests/Dsl/SourceSpec.cs @@ -107,7 +107,7 @@ public void Maybe_Source_must_complete_materialized_future_with_None_when_stream c.ExpectNoMsg(TimeSpan.FromMilliseconds(300)); subs.Cancel(); - f.Task.AwaitResult().Should().Be(null); + f.Task.AwaitResult().Should().Be(Util.Option.None); }, Materializer); } diff --git a/src/core/Akka.Streams.Tests/Dsl/StageActorRefSpec.cs b/src/core/Akka.Streams.Tests/Dsl/StageActorRefSpec.cs index 8180339e889..e17fdaf1ee3 100644 --- a/src/core/Akka.Streams.Tests/Dsl/StageActorRefSpec.cs +++ b/src/core/Akka.Streams.Tests/Dsl/StageActorRefSpec.cs @@ -34,26 +34,23 @@ private static GraphStageWithMaterializedValue, Task> SumSta => new SumTestStage(probe); [Fact] - public void A_Graph_stage_ActorRef_must_receive_messages() + public async Task A_Graph_stage_ActorRef_must_receive_messages() { var t = Source.Maybe().ToMaterialized(SumStage(TestActor), Keep.Both).Run(Materializer); - var res = t.Item2; var stageRef = ExpectMsg(); stageRef.Tell(new Add(1)); stageRef.Tell(new Add(2)); stageRef.Tell(new Add(3)); - stageRef.Tell(StopNow.Instance); - res.Wait(TimeSpan.FromSeconds(3)).Should().BeTrue(); - res.Result.Should().Be(6); + + (await t.Item2).Should().Be(6); } [Fact] - public void A_Graph_stage_ActorRef_must_be_able_to_be_replied_to() + public async Task A_Graph_stage_ActorRef_must_be_able_to_be_replied_to() { var t = Source.Maybe().ToMaterialized(SumStage(TestActor), Keep.Both).Run(Materializer); - var res = t.Item2; var stageRef = ExpectMsg(); stageRef.Tell(new AddAndTell(1)); @@ -63,15 +60,13 @@ public void A_Graph_stage_ActorRef_must_be_able_to_be_replied_to() ExpectMsg(10); stageRef.Tell(StopNow.Instance); - res.Wait(TimeSpan.FromSeconds(3)).Should().BeTrue(); - res.Result.Should().Be(10); + (await t.Item2).Should().Be(10); } [Fact] - public void A_Graph_stage_ActorRef_must_yield_the_same_self_ref_each_time() + public async Task A_Graph_stage_ActorRef_must_yield_the_same_self_ref_each_time() { var t = Source.Maybe().ToMaterialized(SumStage(TestActor), Keep.Both).Run(Materializer); - var res = t.Item2; var stageRef = ExpectMsg(); stageRef.Tell(CallInitStageActorRef.Instance); @@ -85,16 +80,15 @@ public void A_Graph_stage_ActorRef_must_yield_the_same_self_ref_each_time() ExpectMsg(6); stageRef.Tell(StopNow.Instance); - res.Wait(TimeSpan.FromSeconds(3)).Should().BeTrue(); - res.Result.Should().Be(6); + + (await t.Item2).Should().Be(6); } [Fact] - public void A_Graph_stage_ActorRef_must_be_watchable() + public async Task A_Graph_stage_ActorRef_must_be_watchable() { var t = Source.Maybe().ToMaterialized(SumStage(TestActor), Keep.Both).Run(Materializer); var source = t.Item1; - var res = t.Item2; var stageRef = ExpectMsg(); Watch(stageRef); @@ -102,17 +96,15 @@ public void A_Graph_stage_ActorRef_must_be_watchable() stageRef.Tell(new Add(1)); source.SetResult(0); - res.Wait(TimeSpan.FromSeconds(3)).Should().BeTrue(); - res.Result.Should().Be(1); + (await t.Item2).Should().Be(1); ExpectTerminated(stageRef); } [Fact] - public void A_Graph_stage_ActorRef_must_be_able_to_become() + public async Task A_Graph_stage_ActorRef_must_be_able_to_become() { var t = Source.Maybe().ToMaterialized(SumStage(TestActor), Keep.Both).Run(Materializer); var source = t.Item1; - var res = t.Item2; var stageRef = ExpectMsg(); Watch(stageRef); @@ -123,24 +115,24 @@ public void A_Graph_stage_ActorRef_must_be_able_to_become() ExpectMsg("42"); source.SetResult(0); - res.Wait(TimeSpan.FromSeconds(3)).Should().BeTrue(); - res.Result.Should().Be(1); + (await t.Item2).Should().Be(1); + ExpectTerminated(stageRef); } [Fact] - public void A_Graph_stage_ActorRef_must_reply_Terminated_when_terminated_stage_is_watched() + public async Task A_Graph_stage_ActorRef_must_reply_Terminated_when_terminated_stage_is_watched() { var t = Source.Maybe().ToMaterialized(SumStage(TestActor), Keep.Both).Run(Materializer); var source = t.Item1; - var res = t.Item2; var stageRef = ExpectMsg(); Watch(stageRef); - stageRef.Tell(new Add(1)); + stageRef.Tell(new AddAndTell(1)); + ExpectMsg(1); source.SetResult(0); - res.Wait(TimeSpan.FromSeconds(3)).Should().BeTrue(); - res.Result.Should().Be(1); + + (await t.Item2).Should().Be(1); ExpectTerminated(stageRef); var p = CreateTestProbe(); @@ -149,30 +141,29 @@ public void A_Graph_stage_ActorRef_must_reply_Terminated_when_terminated_stage_i } [Fact] - public void A_Graph_stage_ActorRef_must_be_unwatchable() + public async Task A_Graph_stage_ActorRef_must_be_unwatchable() { var t = Source.Maybe().ToMaterialized(SumStage(TestActor), Keep.Both).Run(Materializer); var source = t.Item1; - var res = t.Item2; var stageRef = ExpectMsg(); Watch(stageRef); Unwatch(stageRef); - stageRef.Tell(new Add(1)); + stageRef.Tell(new AddAndTell(1)); + ExpectMsg(1); source.SetResult(0); - res.Wait(TimeSpan.FromSeconds(3)).Should().BeTrue(); - res.Result.Should().Be(1); + + (await t.Item2).Should().Be(1); ExpectNoMsg(100); } [Fact] - public void A_Graph_stage_ActorRef_must_ignore_and_log_warnings_for_PoisonPill_and_Kill_messages() + public async Task A_Graph_stage_ActorRef_must_ignore_and_log_warnings_for_PoisonPill_and_Kill_messages() { var t = Source.Maybe().ToMaterialized(SumStage(TestActor), Keep.Both).Run(Materializer); var source = t.Item1; - var res = t.Item2; var stageRef = ExpectMsg(); stageRef.Tell(new Add(40)); @@ -187,12 +178,12 @@ public void A_Graph_stage_ActorRef_must_ignore_and_log_warnings_for_PoisonPill_a warn.Message.ToString() .Should() .MatchRegex( - " message sent to StageActorRef\\(akka\\://AkkaSpec/user/StreamSupervisor-[0-9]+/StageActorRef-[0-9]+\\) will be ignored, since it is not a real Actor. Use a custom message type to communicate with it instead."); + " message sent to StageActor\\(akka\\://AkkaSpec/user/StreamSupervisor-[0-9]+/\\$\\$[a-z]+\\) will be ignored, since it is not a real Actor. Use a custom message type to communicate with it instead."); #else warn.Message.ToString() .Should() .MatchRegex( - " message sent to StageActorRef\\(akka\\://StageActorRefSpec-[0-9]+/user/StreamSupervisor-[0-9]+/StageActorRef-[0-9]+\\) will be ignored, since it is not a real Actor. Use a custom message type to communicate with it instead."); + " message sent to StageActor\\(akka\\://StageActorRefSpec-[0-9]+/user/StreamSupervisor-[0-9]+/\\$\\$[a-z]+\\) will be ignored, since it is not a real Actor. Use a custom message type to communicate with it instead."); #endif stageRef.Tell(Kill.Instance); warn = ExpectMsg(TimeSpan.FromSeconds(1)); @@ -201,36 +192,17 @@ public void A_Graph_stage_ActorRef_must_ignore_and_log_warnings_for_PoisonPill_a warn.Message.ToString() .Should() .MatchRegex( - " message sent to StageActorRef\\(akka\\://AkkaSpec/user/StreamSupervisor-[0-9]+/StageActorRef-[0-9]+\\) will be ignored, since it is not a real Actor. Use a custom message type to communicate with it instead."); + " message sent to StageActor\\(akka\\://AkkaSpec/user/StreamSupervisor-[0-9]+/\\$\\$[a-z]+\\) will be ignored, since it is not a real Actor. Use a custom message type to communicate with it instead."); #else warn.Message.ToString() .Should() .MatchRegex( - " message sent to StageActorRef\\(akka\\://StageActorRefSpec-[0-9]+/user/StreamSupervisor-[0-9]+/StageActorRef-[0-9]+\\) will be ignored, since it is not a real Actor. Use a custom message type to communicate with it instead."); + " message sent to StageActor\\(akka\\://StageActorRefSpec-[0-9]+/user/StreamSupervisor-[0-9]+/\\$\\$[a-z]+\\) will be ignored, since it is not a real Actor. Use a custom message type to communicate with it instead."); #endif source.SetResult(2); - res.Wait(TimeSpan.FromSeconds(3)).Should().BeTrue(); - res.Result.Should().Be(42); - } - - [Fact] - public void A_Graph_stage_ActorRef_must_be_able_to_watch_other_actors() - { - var killMe = ActorOf(dsl => { }, "KilMe"); - var t = Source.Maybe().ToMaterialized(SumStage(TestActor), Keep.Both).Run(Materializer); - var source = t.Item1; - var res = t.Item2; - - var stageRef = ExpectMsg(); - stageRef.Tell(new WatchMe(killMe)); - stageRef.Tell(new Add(1)); - killMe.Tell(PoisonPill.Instance); - ExpectMsg().Watchee.Should().Be(killMe); - - source.SetResult(0); - res.Wait(TimeSpan.FromSeconds(3)).Should().BeTrue(); - res.Result.Should().Be(1); + + (await t.Item2).Should().Be(42); } private sealed class Add @@ -271,24 +243,6 @@ private sealed class StopNow public static readonly StopNow Instance = new StopNow(); private StopNow() { } } - private sealed class WatchMe - { - public WatchMe(IActorRef watchee) - { - Watchee = watchee; - } - - public IActorRef Watchee { get; } - } - private sealed class WatcheeTerminated - { - public WatcheeTerminated(IActorRef watchee) - { - Watchee = watchee; - } - - public IActorRef Watchee { get; } - } private class SumTestStage : GraphStageWithMaterializedValue, Task> { @@ -301,7 +255,7 @@ private class Logic : GraphStageLogic private readonly SumTestStage _stage; private readonly TaskCompletionSource _promise; private int _sum; - private StageActorRef _self; + private IActorRef Self => StageActor.Ref; public Logic(SumTestStage stage, TaskCompletionSource promise) : base(stage.Shape) { @@ -327,8 +281,8 @@ public Logic(SumTestStage stage, TaskCompletionSource promise) : base(stage public override void PreStart() { Pull(_stage._inlet); - _self = GetStageActorRef(Behaviour); - _stage._probe.Tell(_self); + GetStageActor(Behaviour); + _stage._probe.Tell(Self); } private void Behaviour(Tuple args) @@ -336,22 +290,26 @@ private void Behaviour(Tuple args) var msg = args.Item2; var sender = args.Item1; - msg.Match() - .With(a => _sum += a.N) - .With(() => Pull(_stage._inlet)) - .With(() => sender.Tell(GetStageActorRef(Behaviour), _self)) - .With(() => GetStageActorRef(tuple => tuple.Item1.Tell(tuple.Item2.ToString()))) - .With(() => + switch (msg) + { + case Add add: _sum += add.N; break; + case PullNow _: Pull(_stage._inlet); break; + case CallInitStageActorRef _: sender.Tell(GetStageActor(Behaviour).Ref, Self); break; + case BecomeStringEcho _: GetStageActor(tuple => { + var theSender = tuple.Item1; + var theMsg = tuple.Item2; + theSender.Tell(theMsg.ToString(), Self); + }); break; + case StopNow _: _promise.TrySetResult(_sum); CompleteStage(); - }).With(a => - { - _sum += a.N; - sender.Tell(_sum, _self); - }) - .With(w => _self.Watch(w.Watchee)) - .With(t => _stage._probe.Tell(new WatcheeTerminated(t.ActorRef))); + break; + case AddAndTell addAndTell: + _sum += addAndTell.N; + sender.Tell(_sum, Self); + break; + } } } #endregion diff --git a/src/core/Akka.Streams.Tests/Dsl/StreamRefsSpec.cs b/src/core/Akka.Streams.Tests/Dsl/StreamRefsSpec.cs new file mode 100644 index 00000000000..87ced56c8d7 --- /dev/null +++ b/src/core/Akka.Streams.Tests/Dsl/StreamRefsSpec.cs @@ -0,0 +1,405 @@ +//----------------------------------------------------------------------- +// +// Copyright (C) 2009-2018 Lightbend Inc. +// Copyright (C) 2013-2018 .NET Foundation +// +//----------------------------------------------------------------------- + +using System; +using System.Linq; +using System.Threading; +using System.Threading.Tasks; +using Akka.Actor; +using Akka.Actor.Internal; +using Akka.Configuration; +using Akka.IO; +using Akka.Streams.Dsl; +using Akka.Streams.Implementation; +using Akka.Streams.TestKit; +using Akka.TestKit; +using Xunit; +using Xunit.Abstractions; +using FluentAssertions; + +namespace Akka.Streams.Tests +{ + internal sealed class DataSourceActor : ActorBase + { + public static Props Props(IActorRef probe) => + Akka.Actor.Props.Create(() => new DataSourceActor(probe));//.WithDispatcher("akka.test.stream-dispatcher"); + + private readonly IActorRef _probe; + private readonly ActorMaterializer _materializer; + + public DataSourceActor(IActorRef probe) + { + _probe = probe; + _materializer = Context.System.Materializer(); + } + + protected override void PostStop() + { + base.PostStop(); + _materializer.Dispose(); + } + + protected override bool Receive(object message) + { + switch (message) + { + case "give": + { + /* + * Here we're able to send a source to a remote recipient + * For them it's a Source; for us it is a Sink we run data "into" + */ + var source = Source.From(new[] { "hello", "world" }); + var aref = source.RunWith(StreamRefs.SourceRef(), _materializer); + aref.PipeTo(Sender); + return true; + } + case "give-infinite": + { + var source = Source.From(Enumerable.Range(1, int.MaxValue).Select(i => "ping-" + i)); + var t = source.ToMaterialized(StreamRefs.SourceRef(), Keep.Right).Run(_materializer); + t.PipeTo(Sender); + return true; + } + case "give-fail": + { + var r = Source.Failed(new Exception("Boom!")) + .RunWith(StreamRefs.SourceRef(), _materializer); + r.PipeTo(Sender); + return true; + } + case "give-complete-asap": + { + var r = Source.Empty().RunWith(StreamRefs.SourceRef(), _materializer); + r.PipeTo(Sender); + return true; + } + case "give-subscribe-timeout": + { + var r = Source.Repeat("is anyone there?") + .ToMaterialized(StreamRefs.SourceRef(), Keep.Right) + .WithAttributes(StreamRefAttributes.CreateSubscriptionTimeout(TimeSpan.FromMilliseconds(500))) + .Run(_materializer); + r.PipeTo(Sender); + return true; + } + case "receive": + { + /* + * We write out code, knowing that the other side will stream the data into it. + * For them it's a Sink; for us it's a Source. + */ + var sink = StreamRefs.SinkRef().To(Sink.ActorRef(_probe, "")) + .Run(_materializer); + sink.PipeTo(Sender); + return true; + } + case "receive-subscribe-timeout": + { + var sink = StreamRefs.SinkRef() + .WithAttributes(StreamRefAttributes.CreateSubscriptionTimeout(TimeSpan.FromMilliseconds(500))) + .To(Sink.ActorRef(_probe, "")) + .Run(_materializer); + sink.PipeTo(Sender); + return true; + } + case "receive-32": + { +// var t = StreamRefs.SinkRef() +// .ToMaterialized(TestSink.SinkProbe(Context.System), Keep.Both) +// .Run(_materializer); +// +// var sink = t.Item1; +// var driver = t.Item2; +// Task.Run(() => +// { +// driver.EnsureSubscription(); +// driver.Request(2); +// driver.ExpectNext(); +// driver.ExpectNext(); +// driver.ExpectNoMsg(TimeSpan.FromMilliseconds(100)); +// driver.Request(30); +// driver.ExpectNextN(30); +// +// return ""; +// }).PipeTo(_probe); + + return true; + } + default: return false; + } + } + } + + internal sealed class SourceMsg + { + public ISourceRef DataSource { get; } + + public SourceMsg(ISourceRef dataSource) + { + DataSource = dataSource; + } + } + + internal sealed class BulkSourceMsg + { + public ISourceRef DataSource { get; } + + public BulkSourceMsg(ISourceRef dataSource) + { + DataSource = dataSource; + } + } + + internal sealed class SinkMsg + { + public ISinkRef DataSink { get; } + + public SinkMsg(ISinkRef dataSink) + { + DataSink = dataSink; + } + } + + internal sealed class BulkSinkMsg + { + public ISinkRef DataSink { get; } + + public BulkSinkMsg(ISinkRef dataSink) + { + DataSink = dataSink; + } + } + + public class StreamRefsSpec : AkkaSpec + { + public static Config Config() + { + var address = TestUtils.TemporaryServerAddress(); + return ConfigurationFactory.ParseString($@" + akka {{ + loglevel = INFO + actor {{ + provider = remote + serialize-messages = off + }} + remote.dot-netty.tcp {{ + port = {address.Port} + hostname = ""{address.Address}"" + }} + }}").WithFallback(ConfigurationFactory.Load()); + } + + public StreamRefsSpec(ITestOutputHelper output) : this(Config(), output: output) + { + } + + protected StreamRefsSpec(Config config, ITestOutputHelper output = null) : base(config, output) + { + Materializer = Sys.Materializer(); + RemoteSystem = ActorSystem.Create("remote-system", Config()); + InitializeLogger(RemoteSystem); + _probe = CreateTestProbe(); + + var it = RemoteSystem.ActorOf(DataSourceActor.Props(_probe.Ref), "remoteActor"); + var remoteAddress = ((ActorSystemImpl)RemoteSystem).Provider.DefaultAddress; + Sys.ActorSelection(it.Path.ToStringWithAddress(remoteAddress)).Tell(new Identify("hi")); + + _remoteActor = ExpectMsg().Subject; + } + + protected readonly ActorSystem RemoteSystem; + protected readonly ActorMaterializer Materializer; + private readonly TestProbe _probe; + private readonly IActorRef _remoteActor; + + protected override void BeforeTermination() + { + base.BeforeTermination(); + RemoteSystem.Dispose(); + Materializer.Dispose(); + } + + [Fact] + public void SourceRef_must_send_messages_via_remoting() + { + _remoteActor.Tell("give"); + var sourceRef = ExpectMsg>(); + + sourceRef.Source.RunWith(Sink.ActorRef(_probe.Ref, ""), Materializer); + + _probe.ExpectMsg("hello"); + _probe.ExpectMsg("world"); + _probe.ExpectMsg(""); + } + + [Fact] + public void SourceRef_must_fail_when_remote_source_failed() + { + _remoteActor.Tell("give-fail"); + var sourceRef = ExpectMsg>(); + + sourceRef.Source.RunWith(Sink.ActorRef(_probe.Ref, ""), Materializer); + + var f = _probe.ExpectMsg(); + f.Cause.Message.Should().Contain("Remote stream ("); + f.Cause.Message.Should().Contain("Boom!"); + } + + [Fact] + public void SourceRef_must_complete_properly_when_remote_source_is_empty() + { + // this is a special case since it makes sure that the remote stage is still there when we connect to it + _remoteActor.Tell("give-complete-asap"); + var sourceRef = ExpectMsg>(); + + sourceRef.Source.RunWith(Sink.ActorRef(_probe.Ref, ""), Materializer); + + _probe.ExpectMsg(""); + } + + [Fact] + public void SourceRef_must_respect_backpressure_from_implied_by_target_Sink() + { + _remoteActor.Tell("give-infinite"); + var sourceRef = ExpectMsg>(); + + var probe = sourceRef.Source.RunWith(this.SinkProbe(), Materializer); + + probe.EnsureSubscription(); + probe.ExpectNoMsg(TimeSpan.FromMilliseconds(100)); + + probe.Request(1); + probe.ExpectNext("ping-1"); + probe.ExpectNoMsg(TimeSpan.FromMilliseconds(100)); + + probe.Request(20); + probe.ExpectNextN(Enumerable.Range(1, 20).Select(i => "ping-" + (i + 1))); + probe.Cancel(); + + // since no demand anyway + probe.ExpectNoMsg(TimeSpan.FromMilliseconds(100)); + + // should not cause more pulling, since we issued a cancel already + probe.Request(10); + probe.ExpectNoMsg(TimeSpan.FromMilliseconds(100)); + } + + [Fact] + public void SourceRef_must_receive_timeout_if_subscribing_too_late_to_the_source_ref() + { + _remoteActor.Tell("give-subscribe-timeout"); + var sourceRef = ExpectMsg>(); + + + // not materializing it, awaiting the timeout... + Thread.Sleep(800); + + var probe = sourceRef.Source.RunWith(this.SinkProbe(), Materializer); + + // the local "remote sink" should cancel, since it should notice the origin target actor is dead + probe.EnsureSubscription(); + var ex = probe.ExpectError(); + ex.Message.Should().Contain("has terminated! Tearing down this side of the stream as well."); + } + + [Fact] + public void SinkRef_must_receive_elements_via_remoting() + { + _remoteActor.Tell("receive"); + var remoteSink = ExpectMsg>(); + + Source.From(new[] { "hello", "world" }) + .To(remoteSink.Sink) + .Run(Materializer); + + _probe.ExpectMsg("hello"); + _probe.ExpectMsg("world"); + _probe.ExpectMsg(""); + } + + [Fact] + public void SinkRef_must_fail_origin_if_remote_Sink_gets_a_failure() + { + _remoteActor.Tell("receive"); + var remoteSink = ExpectMsg>(); + + Source.Failed(new Exception("Boom!")) + .To(remoteSink.Sink) + .Run(Materializer); + + var failure = _probe.ExpectMsg(); + failure.Cause.Message.Should().Contain("Remote stream ("); + failure.Cause.Message.Should().Contain("Boom!"); + } + + [Fact] + public void SinkRef_must_receive_hundreds_of_elements_via_remoting() + { + _remoteActor.Tell("receive"); + var remoteSink = ExpectMsg>(); + + var msgs = Enumerable.Range(1, 100).Select(i => "payload-" + i).ToArray(); + + Source.From(msgs).RunWith(remoteSink.Sink, Materializer); + + foreach (var msg in msgs) + { + _probe.ExpectMsg(msg); + } + + _probe.ExpectMsg(""); + } + + [Fact] + public void SinkRef_must_receive_timeout_if_subscribing_too_late_to_the_sink_ref() + { + _remoteActor.Tell("receive-subscribe-timeout"); + var remoteSink = ExpectMsg>(); + + // not materializing it, awaiting the timeout... + Thread.Sleep(800); + + var probe = this.SourceProbe().To(remoteSink.Sink).Run(Materializer); + + var failure = _probe.ExpectMsg(); + failure.Cause.Message.Should().Contain("Remote side did not subscribe (materialize) handed out Sink reference"); + + // the local "remote sink" should cancel, since it should notice the origin target actor is dead + probe.ExpectCancellation(); + } + + [Fact(Skip="FIXME: how to pass test assertions to remote system?")] + public void SinkRef_must_respect_backpressure_implied_by_origin_Sink() + { + _remoteActor.Tell("receive-32"); + var sinkRef = ExpectMsg>(); + + Source.Repeat("hello").RunWith(sinkRef.Sink, Materializer); + + // if we get this message, it means no checks in the request/expect semantics were broken, good! + _probe.ExpectMsg(""); + } + + [Fact] + public void SinkRef_must_not_allow_materializing_multiple_times() + { + _remoteActor.Tell("receive-subscribe-timeout"); + var sinkRef = ExpectMsg>(); + + var p1 = this.SourceProbe().To(sinkRef.Sink).Run(Materializer); + var p2 = this.SourceProbe().To(sinkRef.Sink).Run(Materializer); + + p1.EnsureSubscription(); + var req = p1.ExpectRequest(); + + // will be cancelled immediately, since it's 2nd: + p2.EnsureSubscription(); + p2.ExpectCancellation(); + } + } +} \ No newline at end of file diff --git a/src/core/Akka.Streams.Tests/IO/TcpSpec.cs b/src/core/Akka.Streams.Tests/IO/TcpSpec.cs index 78beaebbcc8..fce7275697d 100644 --- a/src/core/Akka.Streams.Tests/IO/TcpSpec.cs +++ b/src/core/Akka.Streams.Tests/IO/TcpSpec.cs @@ -18,6 +18,7 @@ using Akka.Streams.Dsl; using Akka.Streams.TestKit; using Akka.Streams.TestKit.Tests; +using Akka.Streams.Util; using Akka.TestKit; using FluentAssertions; using Xunit; @@ -38,7 +39,7 @@ public void Outgoing_TCP_stream_must_work_in_the_happy_case() { this.AssertAllStagesStopped(() => { - var testData = ByteString.FromBytes(new byte[] {1, 2, 3, 4, 5}); + var testData = ByteString.FromBytes(new byte[] { 1, 2, 3, 4, 5 }); var server = new Server(this); @@ -62,7 +63,7 @@ public void Outgoing_TCP_stream_must_work_in_the_happy_case() public void Outgoing_TCP_stream_must_be_able_to_write_a_sequence_of_ByteStrings() { var server = new Server(this); - var testInput = Enumerable.Range(0, 256).Select(i => ByteString.FromBytes(new[] {Convert.ToByte(i)})); + var testInput = Enumerable.Range(0, 256).Select(i => ByteString.FromBytes(new[] { Convert.ToByte(i) })); var expectedOutput = ByteString.FromBytes(Enumerable.Range(0, 256).Select(Convert.ToByte).ToArray()); Source.From(testInput) @@ -74,7 +75,7 @@ public void Outgoing_TCP_stream_must_be_able_to_write_a_sequence_of_ByteStrings( serverConnection.Read(256); serverConnection.WaitRead().ShouldBeEquivalentTo(expectedOutput); } - + [Fact] public async Task Outgoing_TCP_stream_must_be_able_to_read_a_sequence_of_ByteStrings() { @@ -83,7 +84,7 @@ public async Task Outgoing_TCP_stream_must_be_able_to_read_a_sequence_of_ByteStr var testOutput = new byte[255]; for (byte i = 0; i < 255; i++) { - testInput[i] = ByteString.FromBytes(new [] {i}); + testInput[i] = ByteString.FromBytes(new[] { i }); testOutput[i] = i; } @@ -104,7 +105,7 @@ public async Task Outgoing_TCP_stream_must_be_able_to_read_a_sequence_of_ByteStr result.ShouldBe(expectedOutput); } - [Fact(Skip="FIXME .net core / linux")] + [Fact(Skip = "FIXME .net core / linux")] public void Outgoing_TCP_stream_must_fail_the_materialized_task_when_the_connection_fails() { this.AssertAllStagesStopped(() => @@ -169,7 +170,7 @@ public void Outgoing_TCP_stream_must_work_when_remote_closes_write_then_client_c { this.AssertAllStagesStopped(() => { - var testData = ByteString.FromBytes(new byte[] {1, 2, 3, 4, 5}); + var testData = ByteString.FromBytes(new byte[] { 1, 2, 3, 4, 5 }); var server = new Server(this); var tcpWriteProbe = new TcpWriteProbe(this); @@ -298,7 +299,7 @@ public void Outgoing_TCP_stream_must_shut_everything_down_if_client_signals_erro // Server can still write serverConnection.Write(testData); tcpReadProbe.Read(5).ShouldBeEquivalentTo(testData); - + // Client can still write tcpWriteProbe.Write(testData); serverConnection.Read(5); @@ -308,7 +309,7 @@ public void Outgoing_TCP_stream_must_shut_everything_down_if_client_signals_erro tcpWriteProbe.TcpWriteSubscription.Value.SendError(new IllegalStateException("test")); tcpReadProbe.SubscriberProbe.ExpectError(); - serverConnection.ExpectClosed(c=>c.IsErrorClosed); + serverConnection.ExpectClosed(c => c.IsErrorClosed); serverConnection.ExpectTerminated(); }, Materializer); } @@ -341,7 +342,7 @@ public void Outgoing_TCP_stream_must_shut_everything_down_if_client_signals_erro tcpWriteProbe.Write(testData); serverConnection.Read(5); serverConnection.WaitRead().ShouldBeEquivalentTo(testData); - + tcpWriteProbe.TcpWriteSubscription.Value.SendError(new IllegalStateException("test")); serverConnection.ExpectClosed(c => c.IsErrorClosed); serverConnection.ExpectTerminated(); @@ -376,7 +377,7 @@ public void Outgoing_TCP_stream_must_shut_down_both_streams_when_connection_is_a [Fact] public async Task Outgoing_TCP_stream_must_materialize_correctly_when_used_in_multiple_flows() { - var testData = ByteString.FromBytes(new byte[] {1, 2, 3, 4, 5}); + var testData = ByteString.FromBytes(new byte[] { 1, 2, 3, 4, 5 }); var server = new Server(this); var tcpWriteProbe1 = new TcpWriteProbe(this); @@ -399,14 +400,14 @@ public async Task Outgoing_TCP_stream_must_materialize_correctly_when_used_in_mu ValidateServerClientCommunication(testData, serverConnection1, tcpReadProbe1, tcpWriteProbe1); ValidateServerClientCommunication(testData, serverConnection2, tcpReadProbe2, tcpWriteProbe2); - + var conn1 = await conn1F; var conn2 = await conn2F; // Since we have already communicated over the connections we can have short timeouts for the tasks - ((IPEndPoint) conn1.RemoteAddress).Port.Should().Be(((IPEndPoint) server.Address).Port); - ((IPEndPoint) conn2.RemoteAddress).Port.Should().Be(((IPEndPoint) server.Address).Port); - ((IPEndPoint) conn1.LocalAddress).Port.Should().NotBe(((IPEndPoint) conn2.LocalAddress).Port); + ((IPEndPoint)conn1.RemoteAddress).Port.Should().Be(((IPEndPoint)server.Address).Port); + ((IPEndPoint)conn2.RemoteAddress).Port.Should().Be(((IPEndPoint)server.Address).Port); + ((IPEndPoint)conn1.LocalAddress).Port.Should().NotBe(((IPEndPoint)conn2.LocalAddress).Port); tcpWriteProbe1.Close(); tcpReadProbe1.Close(); @@ -423,7 +424,7 @@ public void Outgoing_TCP_stream_must_properly_full_close_if_requested() var writeButIgnoreRead = Flow.FromSinkAndSource(Sink.Ignore(), Source.Single(ByteString.FromString("Early response")), Keep.Right); - var task = + var task = Sys.TcpStream() .Bind(serverAddress.Address.ToString(), serverAddress.Port, halfClose: false) .ToMaterialized( @@ -442,7 +443,7 @@ public void Outgoing_TCP_stream_must_properly_full_close_if_requested() result.Wait(TimeSpan.FromSeconds(5)).Should().BeTrue(); result.Result.ShouldBeEquivalentTo(ByteString.FromString("Early response")); - promise.SetResult(null); // close client upstream, no more data + promise.SetResult(Option.None); // close client upstream, no more data binding.Unbind(); }, Materializer); @@ -461,10 +462,10 @@ public async Task Outgoing_TCP_stream_must_Echo_should_work_even_if_server_is_in .Run(Materializer); var result = await Source.From(Enumerable.Repeat(0, 1000) - .Select(i => ByteString.FromBytes(new[] {Convert.ToByte(i)}))) + .Select(i => ByteString.FromBytes(new[] { Convert.ToByte(i) }))) .Via(Sys.TcpStream().OutgoingConnection(serverAddress, halfClose: true)) .RunAggregate(0, (i, s) => i + s.Count, Materializer); - + result.Should().Be(1000); await binding.Unbind(); @@ -532,7 +533,7 @@ private void ValidateServerClientCommunication(ByteString testData, ServerConnec writeProbe.Write(testData); serverConnection.WaitRead().ShouldBeEquivalentTo(testData); } - + private Sink EchoHandler() => Sink.ForEach(c => c.Flow.Join(Flow.Create()).Run(Materializer)); @@ -550,7 +551,7 @@ public async Task Tcp_listen_stream_must_be_able_to_implement_echo() var echoServerFinish = t.Item2; var testInput = Enumerable.Range(0, 255) - .Select(i => ByteString.FromBytes(new[] {Convert.ToByte(i)})) + .Select(i => ByteString.FromBytes(new[] { Convert.ToByte(i) })) .ToList(); var expectedOutput = testInput.Aggregate(ByteString.Empty, (agg, b) => agg.Concat(b)); @@ -558,7 +559,7 @@ public async Task Tcp_listen_stream_must_be_able_to_implement_echo() var result = await Source.From(testInput) .Via(Sys.TcpStream().OutgoingConnection(serverAddress)) .RunAggregate(ByteString.Empty, (agg, b) => agg.Concat(b), Materializer); - + result.ShouldBeEquivalentTo(expectedOutput); await binding.Unbind(); await echoServerFinish; @@ -591,7 +592,7 @@ public async Task Tcp_listen_stream_must_work_with_a_chain_of_echoes() .Via(echoConnection) .Via(echoConnection) .RunAggregate(ByteString.Empty, (agg, b) => agg.Concat(b), Materializer); - + result.ShouldBeEquivalentTo(expectedOutput); await binding.Unbind(); await echoServerFinish; @@ -619,10 +620,10 @@ public void Tcp_listen_stream_must_bind_and_unbind_correctly() var probe3 = this.CreateManualSubscriberProbe(); var binding3F = bind.To(Sink.FromSubscriber(probe3)).Run(Materializer); probe3.ExpectSubscriptionAndError().Should().BeOfType(); - + binding2F.Invoking(x => x.Wait(TimeSpan.FromSeconds(3))).ShouldThrow(); binding3F.Invoking(x => x.Wait(TimeSpan.FromSeconds(3))).ShouldThrow(); - + // Now unbind first binding1.Unbind().Wait(TimeSpan.FromSeconds(3)).Should().BeTrue(); probe1.ExpectComplete(); @@ -675,8 +676,8 @@ public void Tcp_listen_stream_must_not_shut_down_connections_after_the_connectio }, Materializer); } - [Fact(Skip="FIXME")] - public void Tcp_listen_stream_must_shut_down_properly_even_if_some_accepted_connection_Flows_have_not_been_subscribed_to () + [Fact(Skip = "FIXME")] + public void Tcp_listen_stream_must_shut_down_properly_even_if_some_accepted_connection_Flows_have_not_been_subscribed_to() { this.AssertAllStagesStopped(() => { @@ -693,7 +694,7 @@ public void Tcp_listen_stream_must_shut_down_properly_even_if_some_accepted_conn .Via(takeTwoAndDropSecond) .RunForeach(c => c.Flow.Join(Flow.Create()).Run(Materializer), Materializer); - var folder = Source.From(Enumerable.Range(0, 100).Select(_ => ByteString.FromBytes(new byte[] {0}))) + var folder = Source.From(Enumerable.Range(0, 100).Select(_ => ByteString.FromBytes(new byte[] { 0 }))) .Via(Sys.TcpStream().OutgoingConnection(serverAddress)) .Aggregate(0, (i, s) => i + s.Count) .ToMaterialized(Sink.First(), Keep.Right); diff --git a/src/core/Akka.Streams.Tests/Implementation/Fusing/KeepGoingStageSpec.cs b/src/core/Akka.Streams.Tests/Implementation/Fusing/KeepGoingStageSpec.cs index fc7e373ef3c..99268264601 100644 --- a/src/core/Akka.Streams.Tests/Implementation/Fusing/KeepGoingStageSpec.cs +++ b/src/core/Akka.Streams.Tests/Implementation/Fusing/KeepGoingStageSpec.cs @@ -11,6 +11,7 @@ using Akka.Streams.Dsl; using Akka.Streams.Stage; using Akka.Streams.TestKit.Tests; +using Akka.Streams.Util; using Akka.TestKit; using FluentAssertions; using Xunit; @@ -126,7 +127,7 @@ public PingableLogic(PingableSink pingable) : base(pingable.Shape) { _pingable = pingable; - SetHandler(_pingable.Shape.Inlet, + SetHandler(_pingable.Shape.Inlet, () => Pull(_pingable.Shape.Inlet), //Ignore finish () => { _listener.Tell(UpstreamCompleted.Instance); }); @@ -213,7 +214,7 @@ public void A_stage_with_keep_going_must_still_be_alive_after_all_ports_have_bee pinger.Ping(); ExpectMsg(); - maybePromise.TrySetResult(0); + maybePromise.TrySetResult(Option.None); ExpectMsg(); ExpectNoMsg(200); @@ -248,7 +249,7 @@ public void A_stage_with_keep_going_must_still_be_alive_after_all_ports_have_bee pinger.Ping(); ExpectMsg(); - maybePromise.TrySetResult(0); + maybePromise.TrySetResult(Option.None); ExpectMsg(); ExpectNoMsg(200); @@ -287,7 +288,7 @@ public void A_stage_with_keep_going_must_still_be_alive_after_all_ports_have_bee pinger.Ping(); ExpectMsg(); - maybePromise.TrySetResult(0); + maybePromise.TrySetResult(Option.None); ExpectMsg(); ExpectNoMsg(200); @@ -300,7 +301,7 @@ public void A_stage_with_keep_going_must_still_be_alive_after_all_ports_have_bee // We need to catch the exception otherwise the test fails // ReSharper disable once EmptyGeneralCatchClause - try { pinger.ThrowEx();} catch { } + try { pinger.ThrowEx(); } catch { } // PostStop should not be concurrent with the event handler. This event here tests this. ExpectMsg(); ExpectMsg(); @@ -328,7 +329,7 @@ public void A_stage_with_keep_going_must_close_down_earls_if_keepAlive_is_not_re pinger.Ping(); ExpectMsg(); - maybePromise.TrySetResult(0); + maybePromise.TrySetResult(Option.None); ExpectMsg(); ExpectMsg(); }, Materializer); diff --git a/src/core/Akka.Streams/ActorMaterializer.cs b/src/core/Akka.Streams/ActorMaterializer.cs index 16f1b5d026c..d2ad60b091e 100644 --- a/src/core/Akka.Streams/ActorMaterializer.cs +++ b/src/core/Akka.Streams/ActorMaterializer.cs @@ -318,7 +318,8 @@ private static ActorMaterializerSettings Create(Config config) isFuzzingMode: config.GetBoolean("debug.fuzzing-mode"), isAutoFusing: config.GetBoolean("auto-fusing", true), maxFixedBufferSize: config.GetInt("max-fixed-buffer-size", 1000000000), - syncProcessingLimit: config.GetInt("sync-processing-limit", 1000)); + syncProcessingLimit: config.GetInt("sync-processing-limit", 1000), + streamRefSettings: StreamRefSettings.Create(config.GetConfig("stream-ref") ?? Config.Empty)); } private const int DefaultlMaxFixedbufferSize = 1000; @@ -367,6 +368,8 @@ private static ActorMaterializerSettings Create(Config config) /// public readonly int SyncProcessingLimit; + public readonly StreamRefSettings StreamRefSettings; + /// /// TBD /// @@ -381,7 +384,7 @@ private static ActorMaterializerSettings Create(Config config) /// TBD /// TBD /// TBD - public ActorMaterializerSettings(int initialInputBufferSize, int maxInputBufferSize, string dispatcher, Decider supervisionDecider, StreamSubscriptionTimeoutSettings subscriptionTimeoutSettings, bool isDebugLogging, int outputBurstLimit, bool isFuzzingMode, bool isAutoFusing, int maxFixedBufferSize, int syncProcessingLimit = DefaultlMaxFixedbufferSize) + public ActorMaterializerSettings(int initialInputBufferSize, int maxInputBufferSize, string dispatcher, Decider supervisionDecider, StreamSubscriptionTimeoutSettings subscriptionTimeoutSettings, StreamRefSettings streamRefSettings, bool isDebugLogging, int outputBurstLimit, bool isFuzzingMode, bool isAutoFusing, int maxFixedBufferSize, int syncProcessingLimit = DefaultlMaxFixedbufferSize) { InitialInputBufferSize = initialInputBufferSize; MaxInputBufferSize = maxInputBufferSize; @@ -394,7 +397,7 @@ public ActorMaterializerSettings(int initialInputBufferSize, int maxInputBufferS IsAutoFusing = isAutoFusing; MaxFixedBufferSize = maxFixedBufferSize; SyncProcessingLimit = syncProcessingLimit; - + StreamRefSettings = streamRefSettings; } /// @@ -405,7 +408,7 @@ public ActorMaterializerSettings(int initialInputBufferSize, int maxInputBufferS /// TBD public ActorMaterializerSettings WithInputBuffer(int initialSize, int maxSize) { - return new ActorMaterializerSettings(initialSize, maxSize, Dispatcher, SupervisionDecider, SubscriptionTimeoutSettings, IsDebugLogging, OutputBurstLimit, IsFuzzingMode, IsAutoFusing, MaxFixedBufferSize, SyncProcessingLimit); + return new ActorMaterializerSettings(initialSize, maxSize, Dispatcher, SupervisionDecider, SubscriptionTimeoutSettings, StreamRefSettings, IsDebugLogging, OutputBurstLimit, IsFuzzingMode, IsAutoFusing, MaxFixedBufferSize, SyncProcessingLimit); } /// @@ -415,7 +418,7 @@ public ActorMaterializerSettings WithInputBuffer(int initialSize, int maxSize) /// TBD public ActorMaterializerSettings WithDispatcher(string dispatcher) { - return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, dispatcher, SupervisionDecider, SubscriptionTimeoutSettings, IsDebugLogging, OutputBurstLimit, IsFuzzingMode, IsAutoFusing, MaxFixedBufferSize, SyncProcessingLimit); + return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, dispatcher, SupervisionDecider, SubscriptionTimeoutSettings, StreamRefSettings, IsDebugLogging, OutputBurstLimit, IsFuzzingMode, IsAutoFusing, MaxFixedBufferSize, SyncProcessingLimit); } /// @@ -425,7 +428,7 @@ public ActorMaterializerSettings WithDispatcher(string dispatcher) /// TBD public ActorMaterializerSettings WithSupervisionStrategy(Decider decider) { - return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, Dispatcher, decider, SubscriptionTimeoutSettings, IsDebugLogging, OutputBurstLimit, IsFuzzingMode, IsAutoFusing, MaxFixedBufferSize, SyncProcessingLimit); + return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, Dispatcher, decider, SubscriptionTimeoutSettings, StreamRefSettings, IsDebugLogging, OutputBurstLimit, IsFuzzingMode, IsAutoFusing, MaxFixedBufferSize, SyncProcessingLimit); } /// @@ -435,7 +438,7 @@ public ActorMaterializerSettings WithSupervisionStrategy(Decider decider) /// TBD public ActorMaterializerSettings WithDebugLogging(bool isEnabled) { - return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, Dispatcher, SupervisionDecider, SubscriptionTimeoutSettings, isEnabled, OutputBurstLimit, IsFuzzingMode, IsAutoFusing, MaxFixedBufferSize, SyncProcessingLimit); + return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, Dispatcher, SupervisionDecider, SubscriptionTimeoutSettings, StreamRefSettings, isEnabled, OutputBurstLimit, IsFuzzingMode, IsAutoFusing, MaxFixedBufferSize, SyncProcessingLimit); } /// @@ -445,7 +448,7 @@ public ActorMaterializerSettings WithDebugLogging(bool isEnabled) /// TBD public ActorMaterializerSettings WithFuzzingMode(bool isFuzzingMode) { - return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, Dispatcher, SupervisionDecider, SubscriptionTimeoutSettings, IsDebugLogging, OutputBurstLimit, isFuzzingMode, IsAutoFusing, MaxFixedBufferSize, SyncProcessingLimit); + return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, Dispatcher, SupervisionDecider, SubscriptionTimeoutSettings, StreamRefSettings, IsDebugLogging, OutputBurstLimit, isFuzzingMode, IsAutoFusing, MaxFixedBufferSize, SyncProcessingLimit); } /// @@ -455,7 +458,7 @@ public ActorMaterializerSettings WithFuzzingMode(bool isFuzzingMode) /// TBD public ActorMaterializerSettings WithAutoFusing(bool isAutoFusing) { - return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, Dispatcher, SupervisionDecider, SubscriptionTimeoutSettings, IsDebugLogging, OutputBurstLimit, IsFuzzingMode, isAutoFusing, MaxFixedBufferSize, SyncProcessingLimit); + return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, Dispatcher, SupervisionDecider, SubscriptionTimeoutSettings, StreamRefSettings, IsDebugLogging, OutputBurstLimit, IsFuzzingMode, isAutoFusing, MaxFixedBufferSize, SyncProcessingLimit); } /// @@ -465,7 +468,7 @@ public ActorMaterializerSettings WithAutoFusing(bool isAutoFusing) /// TBD public ActorMaterializerSettings WithMaxFixedBufferSize(int maxFixedBufferSize) { - return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, Dispatcher, SupervisionDecider, SubscriptionTimeoutSettings, IsDebugLogging, OutputBurstLimit, IsFuzzingMode, IsAutoFusing, maxFixedBufferSize, SyncProcessingLimit); + return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, Dispatcher, SupervisionDecider, SubscriptionTimeoutSettings, StreamRefSettings, IsDebugLogging, OutputBurstLimit, IsFuzzingMode, IsAutoFusing, maxFixedBufferSize, SyncProcessingLimit); } /// @@ -475,7 +478,7 @@ public ActorMaterializerSettings WithMaxFixedBufferSize(int maxFixedBufferSize) /// TBD public ActorMaterializerSettings WithSyncProcessingLimit(int limit) { - return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, Dispatcher, SupervisionDecider, SubscriptionTimeoutSettings, IsDebugLogging, OutputBurstLimit, IsFuzzingMode, IsAutoFusing, MaxFixedBufferSize, limit); + return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, Dispatcher, SupervisionDecider, SubscriptionTimeoutSettings, StreamRefSettings, IsDebugLogging, OutputBurstLimit, IsFuzzingMode, IsAutoFusing, MaxFixedBufferSize, limit); } /// @@ -488,7 +491,14 @@ public ActorMaterializerSettings WithSubscriptionTimeoutSettings(StreamSubscript if (Equals(settings, SubscriptionTimeoutSettings)) return this; - return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, Dispatcher, SupervisionDecider, settings, IsDebugLogging, OutputBurstLimit, IsFuzzingMode, IsAutoFusing, MaxFixedBufferSize, SyncProcessingLimit); + return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, Dispatcher, SupervisionDecider, settings, StreamRefSettings, IsDebugLogging, OutputBurstLimit, IsFuzzingMode, IsAutoFusing, MaxFixedBufferSize, SyncProcessingLimit); + } + + public ActorMaterializerSettings WithStreamRefSettings(StreamRefSettings settings) + { + if (settings == null) throw new ArgumentNullException(nameof(settings)); + if (ReferenceEquals(settings, this.StreamRefSettings)) return this; + return new ActorMaterializerSettings(InitialInputBufferSize, MaxInputBufferSize, Dispatcher, SupervisionDecider, SubscriptionTimeoutSettings, settings, IsDebugLogging, OutputBurstLimit, IsFuzzingMode, IsAutoFusing, MaxFixedBufferSize, SyncProcessingLimit); } } diff --git a/src/core/Akka.Streams/Akka.Streams.csproj b/src/core/Akka.Streams/Akka.Streams.csproj index 6c5fab2efe6..2b40d287c6b 100644 --- a/src/core/Akka.Streams/Akka.Streams.csproj +++ b/src/core/Akka.Streams/Akka.Streams.csproj @@ -66,8 +66,12 @@ + + + + $(DefineConstants);SERIALIZATION;CLONEABLE;AKKAIO diff --git a/src/core/Akka.Streams/Attributes.cs b/src/core/Akka.Streams/Attributes.cs index 8b248c512b5..a8944a9af46 100644 --- a/src/core/Akka.Streams/Attributes.cs +++ b/src/core/Akka.Streams/Attributes.cs @@ -471,4 +471,32 @@ public SupervisionStrategy(Decider decider) public static Attributes CreateSupervisionStrategy(Decider strategy) => new Attributes(new SupervisionStrategy(strategy)); } + + /// + /// Attributes for stream refs ( and ). + /// Note that more attributes defined in and . + /// + public static class StreamRefAttributes + { + /// + /// Attributes specific to stream refs. + /// + public interface IStreamRefAttribute : Attributes.IAttribute { } + + public sealed class SubscriptionTimeout : IStreamRefAttribute + { + public TimeSpan Timeout { get; } + + public SubscriptionTimeout(TimeSpan timeout) + { + Timeout = timeout; + } + } + + /// + /// Specifies the subscription timeout within which the remote side MUST subscribe to the handed out stream reference. + /// + public static Attributes CreateSubscriptionTimeout(TimeSpan timeout) => + new Attributes(new SubscriptionTimeout(timeout)); + } } diff --git a/src/core/Akka.Streams/Dsl/Source.cs b/src/core/Akka.Streams/Dsl/Source.cs index f394b194de3..f3cda634756 100644 --- a/src/core/Akka.Streams/Dsl/Source.cs +++ b/src/core/Akka.Streams/Dsl/Source.cs @@ -596,9 +596,9 @@ public static Source UnfoldInfinite(TState state, /// /// TBD /// TBD - public static Source> Maybe() + public static Source>> Maybe() { - return new Source>( + return new Source>>( new MaybeSource(DefaultAttributes.MaybeSource, new SourceShape(new Outlet("MaybeSource")))); } diff --git a/src/core/Akka.Streams/Dsl/StreamRefs.cs b/src/core/Akka.Streams/Dsl/StreamRefs.cs new file mode 100644 index 00000000000..e4e0a62eccb --- /dev/null +++ b/src/core/Akka.Streams/Dsl/StreamRefs.cs @@ -0,0 +1,731 @@ +//----------------------------------------------------------------------- +// +// Copyright (C) 2009-2018 Lightbend Inc. +// Copyright (C) 2013-2018 .NET Foundation +// +//----------------------------------------------------------------------- + +using System; +using System.Threading.Tasks; +using Akka.Actor; +using Akka.Annotations; +using Akka.Configuration; +using Akka.Event; +using Akka.Pattern; +using Akka.Streams.Actors; +using Akka.Streams.Dsl; +using Akka.Streams.Implementation; +using Akka.Streams.Stage; +using Akka.Util.Internal; +using Reactive.Streams; + +namespace Akka.Streams.Dsl +{ + + /// + /// API MAY CHANGE: The functionality of stream refs is working, however it is expected that the materialized value + /// will eventually be able to remove the Task wrapping the stream references. For this reason the API is now marked + /// as API may change. See ticket https://github.com/akka/akka/issues/24372 for more details. + /// + /// Factories for creating stream refs. + /// + [ApiMayChange] + public static class StreamRefs + { + /// + /// A local which materializes a which can be used by other streams (including remote ones), + /// to consume data from this local stream, as if they were attached in the spot of the local Sink directly. + /// + /// Adheres to . + /// + /// + [ApiMayChange] + public static Sink>> SourceRef() => + Sink.FromGraph>>(new SinkRefStageImpl(null)); + + /// + /// A local which materializes a which can be used by other streams (including remote ones), + /// to consume data from this local stream, as if they were attached in the spot of the local Sink directly. + /// + /// Adheres to . + /// + /// See more detailed documentation on [[SinkRef]]. + /// + /// + [ApiMayChange] + public static Source>> SinkRef() => + Source.FromGraph>>(new SourceRefStageImpl(null)); + } + + #region StreamRef messages + + internal interface IStreamRefsProtocol { } + + /// + /// Sequenced equivalent. + /// The receiving end of these messages MUST fail the stream if it observes gaps in the sequence, + /// as these messages will not be re-delivered. + /// + /// Sequence numbers start from `0`. + /// + internal sealed class SequencedOnNext : IStreamRefsProtocol, IDeadLetterSuppression + { + public long SeqNr { get; } + public object Payload { get; } + + public SequencedOnNext(long seqNr, object payload) + { + SeqNr = seqNr; + Payload = payload ?? throw ReactiveStreamsCompliance.ElementMustNotBeNullException; + } + } + + /// + /// Initial message sent to remote side to establish partnership between origin and remote stream refs. + /// + internal sealed class OnSubscribeHandshake : IStreamRefsProtocol, IDeadLetterSuppression + { + public OnSubscribeHandshake(IActorRef targetRef) + { + TargetRef = targetRef; + } + + public IActorRef TargetRef { get; } + } + + /// + /// Sent to a the receiver side of a stream ref, once the sending side of the SinkRef gets signalled a Failure. + /// + internal sealed class RemoteStreamFailure : IStreamRefsProtocol + { + public RemoteStreamFailure(string message) + { + Message = message; + } + + public string Message { get; } + } + + /// + /// Sent to a the receiver side of a stream ref, once the sending side of the SinkRef gets signalled a completion. + /// + internal sealed class RemoteStreamCompleted : IStreamRefsProtocol + { + public RemoteStreamCompleted(long seqNr) + { + SeqNr = seqNr; + } + + public long SeqNr { get; } + } + + /// + /// INTERNAL API: Cumulative demand, equivalent to sequence numbering all events in a stream. + /// + /// This message may be re-delivered. + /// + internal sealed class CumulativeDemand : IStreamRefsProtocol, IDeadLetterSuppression + { + public CumulativeDemand(long seqNr) + { + if (seqNr <= 0) throw ReactiveStreamsCompliance.NumberOfElementsInRequestMustBePositiveException; + SeqNr = seqNr; + } + + public long SeqNr { get; } + } + + #endregion + + #region extension + + internal sealed class StreamRefsMaster : IExtension + { + public static StreamRefsMaster Get(ActorSystem system) => + system.WithExtension(); + + private readonly EnumerableActorName sourceRefStageNames = new EnumerableActorNameImpl("SourceRef", new AtomicCounterLong(0L)); + private readonly EnumerableActorName sinkRefStageNames = new EnumerableActorNameImpl("SinkRef", new AtomicCounterLong(0L)); + + public StreamRefsMaster(ExtendedActorSystem system) + { + + } + + public string NextSourceRefName() => sourceRefStageNames.Next(); + public string NextSinkRefName() => sinkRefStageNames.Next(); + } + + internal sealed class StreamRefsMasterProvider : ExtensionIdProvider + { + public override StreamRefsMaster CreateExtension(ExtendedActorSystem system) => + new StreamRefsMaster(system); + } + + #endregion + + public sealed class StreamRefSettings + { + public static StreamRefSettings Create(Config config) + { + if (config == null) throw new ArgumentNullException(nameof(config), "`akka.stream.materializer.stream-ref` was not present"); + + return new StreamRefSettings( + bufferCapacity: config.GetInt("buffer-capacity", 32), + demandRedeliveryInterval: config.GetTimeSpan("demand-redelivery-interval", TimeSpan.FromSeconds(1)), + subscriptionTimeout: config.GetTimeSpan("subscription-timeout", TimeSpan.FromSeconds(30))); + } + + public int BufferCapacity { get; } + public TimeSpan DemandRedeliveryInterval { get; } + public TimeSpan SubscriptionTimeout { get; } + + public StreamRefSettings(int bufferCapacity, TimeSpan demandRedeliveryInterval, TimeSpan subscriptionTimeout) + { + BufferCapacity = bufferCapacity; + DemandRedeliveryInterval = demandRedeliveryInterval; + SubscriptionTimeout = subscriptionTimeout; + } + + public string ProductPrefix => nameof(StreamRefSettings); + + public StreamRefSettings WithBufferCapacity(int value) => Copy(bufferCapacity: value); + public StreamRefSettings WithDemandRedeliveryInterval(TimeSpan value) => Copy(demandRedeliveryInterval: value); + public StreamRefSettings WithSubscriptionTimeout(TimeSpan value) => Copy(subscriptionTimeout: value); + + public StreamRefSettings Copy(int? bufferCapacity = null, + TimeSpan? demandRedeliveryInterval = null, + TimeSpan? subscriptionTimeout = null) => new StreamRefSettings( + bufferCapacity: bufferCapacity ?? this.BufferCapacity, + demandRedeliveryInterval: demandRedeliveryInterval ?? this.DemandRedeliveryInterval, + subscriptionTimeout: subscriptionTimeout ?? this.SubscriptionTimeout); + } + + /// + /// Abstract class defined serialization purposes of . + /// + internal abstract class SourceRefImpl + { + public static SourceRefImpl Create(Type eventType, IActorRef initialPartnerRef) + { + var destType = typeof(SourceRefImpl<>).MakeGenericType(eventType); + return (SourceRefImpl)Activator.CreateInstance(destType, initialPartnerRef); + } + + protected SourceRefImpl(IActorRef initialPartnerRef) + { + InitialPartnerRef = initialPartnerRef; + } + + public IActorRef InitialPartnerRef { get; } + public abstract Type EventType { get; } + } + internal sealed class SourceRefImpl : SourceRefImpl, ISourceRef + { + public SourceRefImpl(IActorRef initialPartnerRef) : base(initialPartnerRef) { } + public override Type EventType => typeof(T); + public Source Source => + Dsl.Source.FromGraph(new SourceRefStageImpl(InitialPartnerRef)).MapMaterializedValue(_ => NotUsed.Instance); + } + + /// + /// Abstract class defined serialization purposes of . + /// + internal abstract class SinkRefImpl + { + public static SinkRefImpl Create(Type eventType, IActorRef initialPartnerRef) + { + var destType = typeof(SinkRefImpl<>).MakeGenericType(eventType); + return (SinkRefImpl)Activator.CreateInstance(destType, initialPartnerRef); + } + + protected SinkRefImpl(IActorRef initialPartnerRef) + { + InitialPartnerRef = initialPartnerRef; + } + + public IActorRef InitialPartnerRef { get; } + public abstract Type EventType { get; } + } + + internal sealed class SinkRefImpl : SinkRefImpl, ISinkRef + { + public SinkRefImpl(IActorRef initialPartnerRef) : base(initialPartnerRef) { } + public override Type EventType => typeof(T); + public Sink Sink => Dsl.Sink.FromGraph(new SinkRefStageImpl(InitialPartnerRef)).MapMaterializedValue(_ => NotUsed.Instance); + } + + /// + /// INTERNAL API: Actual stage implementation backing s. + /// + /// If initialPartnerRef is set, then the remote side is already set up. If it is none, then we are the side creating + /// the ref. + /// + /// + internal sealed class SinkRefStageImpl : GraphStageWithMaterializedValue, Task>> + { + #region logic + + private sealed class Logic : TimerGraphStageLogic, IInHandler + { + private const string SubscriptionTimeoutKey = "SubscriptionTimeoutKey"; + + private readonly SinkRefStageImpl _stage; + private readonly TaskCompletionSource> _promise; + private readonly Attributes _inheritedAttributes; + + private StreamRefsMaster _streamRefsMaster; + private StreamRefSettings _settings; + private StreamRefAttributes.SubscriptionTimeout _subscriptionTimeout; + private string _stageActorName; + + private StreamRefsMaster StreamRefsMaster => _streamRefsMaster ?? (_streamRefsMaster = StreamRefsMaster.Get(ActorMaterializerHelper.Downcast(Materializer).System)); + private StreamRefSettings Settings => _settings ?? (_settings = ActorMaterializerHelper.Downcast(Materializer).Settings.StreamRefSettings); + private StreamRefAttributes.SubscriptionTimeout SubscriptionTimeout => _subscriptionTimeout ?? (_subscriptionTimeout = + _inheritedAttributes.GetAttribute(new StreamRefAttributes.SubscriptionTimeout(Settings.SubscriptionTimeout))); + protected override string StageActorName => _stageActorName ?? (_stageActorName = StreamRefsMaster.NextSinkRefName()); + + private StageActor _stageActor; + + private IActorRef _partnerRef = null; + + #region demand management + private long _remoteCumulativeDemandReceived = 0L; + private long _remoteCumulativeDemandConsumed = 0L; + #endregion + + private Status _completedBeforeRemoteConnected = null; + + public IActorRef Self => _stageActor.Ref; + public IActorRef PartnerRef + { + get + { + if (_partnerRef == null) throw new TargetRefNotInitializedYetException(); + return _partnerRef; + } + } + + public Logic(SinkRefStageImpl stage, TaskCompletionSource> promise, + Attributes inheritedAttributes) : base(stage.Shape) + { + _stage = stage; + _promise = promise; + _inheritedAttributes = inheritedAttributes; + + this.SetHandler(_stage.Inlet, this); + } + + public override void PreStart() + { + _stageActor = GetStageActor(InitialReceive); + var initialPartnerRef = _stage._initialPartnerRef; + if (initialPartnerRef != null) + ObserveAndValidateSender(initialPartnerRef, "Illegal initialPartnerRef! This would be a bug in the SinkRef usage or impl."); + + Log.Debug("Created SinkRef, pointing to remote Sink receiver: {0}, local worker: {1}", initialPartnerRef, Self); + + _promise.SetResult(new SourceRefImpl(Self)); + + if (_partnerRef != null) + { + _partnerRef.Tell(new OnSubscribeHandshake(Self), Self); + TryPull(); + } + + ScheduleOnce(SubscriptionTimeoutKey, SubscriptionTimeout.Timeout); + } + + private void InitialReceive(Tuple args) + { + var sender = args.Item1; + var message = args.Item2; + + switch (message) + { + case Terminated terminated: + if (Equals(terminated.ActorRef, PartnerRef)) + FailStage(new RemoteStreamRefActorTerminatedException($"Remote target receiver of data {PartnerRef} terminated. " + + "Local stream terminating, message loss (on remote side) may have happened.")); + break; + case CumulativeDemand demand: + // the other side may attempt to "double subscribe", which we want to fail eagerly since we're 1:1 pairings + ObserveAndValidateSender(sender, "Illegal sender for CumulativeDemand"); + if (_remoteCumulativeDemandReceived < demand.SeqNr) + { + _remoteCumulativeDemandReceived = demand.SeqNr; + Log.Debug("Received cumulative demand [{0}], consumable demand: [{1}]", demand.SeqNr, _remoteCumulativeDemandReceived - _remoteCumulativeDemandConsumed); + } + TryPull(); + break; + } + } + + public void OnPush() + { + var element = GrabSequenced(_stage.Inlet); + PartnerRef.Tell(element, Self); + Log.Debug("Sending sequenced: {0} to {1}", element, PartnerRef); + TryPull(); + } + + private void TryPull() + { + if (_remoteCumulativeDemandConsumed < _remoteCumulativeDemandReceived && !HasBeenPulled(_stage.Inlet)) + { + Pull(_stage.Inlet); + } + } + + protected internal override void OnTimer(object timerKey) + { + if ((string)timerKey == SubscriptionTimeoutKey) + { + // we know the future has been competed by now, since it is in preStart + var ex = new StreamRefSubscriptionTimeoutException($"[{StageActorName}] Remote side did not subscribe (materialize) handed out Sink reference [${_promise.Task.Result}], " + + "within subscription timeout: ${PrettyDuration.format(subscriptionTimeout.timeout)}!"); + + throw ex; // this will also log the exception, unlike failStage; this should fail rarely, but would be good to have it "loud" + } + } + + private SequencedOnNext GrabSequenced(Inlet inlet) + { + var onNext = new SequencedOnNext(_remoteCumulativeDemandConsumed, Grab(inlet)); + _remoteCumulativeDemandConsumed++; + return onNext; + } + + public void OnUpstreamFailure(Exception cause) + { + if (_partnerRef != null) + { + _partnerRef.Tell(new RemoteStreamFailure(cause.ToString()), Self); + _stageActor.Unwatch(_partnerRef); + FailStage(cause); + } + else + { + _completedBeforeRemoteConnected = new Status.Failure(cause); + // not terminating on purpose, since other side may subscribe still and then we want to fail it + // the stage will be terminated either by timeout, or by the handling in `observeAndValidateSender` + SetKeepGoing(true); + } + } + + public void OnUpstreamFinish() + { + if (_partnerRef != null) + { + _partnerRef.Tell(new RemoteStreamCompleted(_remoteCumulativeDemandConsumed), Self); + _stageActor.Unwatch(_partnerRef); + CompleteStage(); + } + else + { + _completedBeforeRemoteConnected = new Status.Success(Done.Instance); + // not terminating on purpose, since other side may subscribe still and then we want to complete it + SetKeepGoing(true); + } + } + + private void ObserveAndValidateSender(IActorRef partner, string failureMessage) + { + if (_partnerRef == null) + { + _partnerRef = partner; + _stageActor.Watch(_partnerRef); + + switch (_completedBeforeRemoteConnected) + { + case Status.Failure failure: + Log.Warning("Stream already terminated with exception before remote side materialized, failing now."); + partner.Tell(new RemoteStreamFailure(failure.Cause.ToString()), Self); + FailStage(failure.Cause); + break; + case Status.Success _: + Log.Warning("Stream already completed before remote side materialized, failing now."); + partner.Tell(new RemoteStreamCompleted(_remoteCumulativeDemandConsumed), Self); + CompleteStage(); + break; + case null: + if (!Equals(partner, PartnerRef)) + { + var ex = new InvalidPartnerActorException(partner, PartnerRef, failureMessage); + partner.Tell(new RemoteStreamFailure(ex.ToString()), Self); + throw ex; + } + break; + } + } + } + } + + #endregion + + private readonly IActorRef _initialPartnerRef; + + public SinkRefStageImpl(IActorRef initialPartnerRef) + { + _initialPartnerRef = initialPartnerRef; + Shape = new SinkShape(Inlet); + } + + public Inlet Inlet { get; } = new Inlet("SinkRef.in"); + public override SinkShape Shape { get; } + public override ILogicAndMaterializedValue>> CreateLogicAndMaterializedValue(Attributes inheritedAttributes) + { + var promise = new TaskCompletionSource>(); + return new LogicAndMaterializedValue>>(new Logic(this, promise, inheritedAttributes), promise.Task); + } + } + + /// + /// INTERNAL API: Actual stage implementation backing [[SourceRef]]s. + /// + /// If initialPartnerRef is set, then the remote side is already set up. + /// If it is none, then we are the side creating the ref. + /// + internal sealed class SourceRefStageImpl : GraphStageWithMaterializedValue, Task>> + { + + #region logic + + private sealed class Logic : TimerGraphStageLogic, IOutHandler + { + private const string SubscriptionTimeoutKey = "SubscriptionTimeoutKey"; + private const string DemandRedeliveryTimerKey = "DemandRedeliveryTimerKey"; + + private readonly SourceRefStageImpl _stage; + private readonly TaskCompletionSource> _promise; + private readonly Attributes _inheritedAttributes; + + private StreamRefsMaster _streamRefsMaster; + private StreamRefSettings _settings; + private StreamRefAttributes.SubscriptionTimeout _subscriptionTimeout; + private string _stageActorName; + + private StageActor _stageActor; + private IActorRef _partnerRef = null; + + private StreamRefsMaster StreamRefsMaster => _streamRefsMaster ?? (_streamRefsMaster = StreamRefsMaster.Get(ActorMaterializerHelper.Downcast(Materializer).System)); + private StreamRefSettings Settings => _settings ?? (_settings = ActorMaterializerHelper.Downcast(Materializer).Settings.StreamRefSettings); + private StreamRefAttributes.SubscriptionTimeout SubscriptionTimeout => _subscriptionTimeout ?? (_subscriptionTimeout = + _inheritedAttributes.GetAttribute(new StreamRefAttributes.SubscriptionTimeout(Settings.SubscriptionTimeout))); + protected override string StageActorName => _stageActorName ?? (_stageActorName = StreamRefsMaster.NextSourceRefName()); + + public IActorRef Self => _stageActor.Ref; + public IActorRef PartnerRef + { + get + { + if (_partnerRef == null) throw new TargetRefNotInitializedYetException(); + return _partnerRef; + } + } + + #region demand management + + private bool _completed = false; + private long _expectingSeqNr = 0L; + private long _localCumulativeDemand = 0L; + private long _localRemainingRequested = 0L; + private FixedSizeBuffer _receiveBuffer; // initialized in preStart since depends on settings + private IRequestStrategy _requestStrategy; // initialized in preStart since depends on receiveBuffer's size + #endregion + + public Logic(SourceRefStageImpl stage, TaskCompletionSource> promise, Attributes inheritedAttributes) : base(stage.Shape) + { + _stage = stage; + _promise = promise; + _inheritedAttributes = inheritedAttributes; + + SetHandler(_stage.Outlet, this); + } + + public override void PreStart() + { + _receiveBuffer = new ModuloFixedSizeBuffer(Settings.BufferCapacity); + _requestStrategy = new WatermarkRequestStrategy(highWatermark: _receiveBuffer.Capacity); + + _stageActor = GetStageActor(InitialReceive); + + Log.Debug("[{0}] Allocated receiver: {1}", StageActorName, Self); + + var initialPartnerRef = _stage._initialPartnerRef; + if (initialPartnerRef != null) // this will set the partnerRef + ObserveAndValidateSender(initialPartnerRef, ""); + + _promise.SetResult(new SinkRefImpl(Self)); + + ScheduleOnce(SubscriptionTimeoutKey, SubscriptionTimeout.Timeout); + } + + public void OnPull() + { + TryPush(); + TriggerCumulativeDemand(); + } + + public void OnDownstreamFinish() + { + CompleteStage(); + } + + private void TriggerCumulativeDemand() + { + var i = _receiveBuffer.RemainingCapacity - _localRemainingRequested; + if (_partnerRef != null && i > 0) + { + var addDemand = _requestStrategy.RequestDemand((int)(_receiveBuffer.Used + _localRemainingRequested)); + + // only if demand has increased we shoot it right away + // otherwise it's the same demand level, so it'd be triggered via redelivery anyway + if (addDemand > 0) + { + _localCumulativeDemand += addDemand; + _localRemainingRequested += addDemand; + var demand = new CumulativeDemand(_localCumulativeDemand); + + Log.Debug("[{0}] Demanding until [{1}] (+{2})", _stageActorName, _localCumulativeDemand, addDemand); + PartnerRef.Tell(demand, Self); + ScheduleDemandRedelivery(); + } + } + } + + private void ScheduleDemandRedelivery() => + ScheduleOnce(DemandRedeliveryTimerKey, Settings.DemandRedeliveryInterval); + + protected internal override void OnTimer(object timerKey) + { + switch (timerKey) + { + case SubscriptionTimeoutKey: + var ex = new StreamRefSubscriptionTimeoutException( + // we know the future has been competed by now, since it is in preStart + $"[{StageActorName}] Remote side did not subscribe (materialize) handed out Sink reference [{_promise.Task.Result}]," + + $"within subscription timeout: {SubscriptionTimeout.Timeout}!"); + throw ex; + case DemandRedeliveryTimerKey: + Log.Debug("[{0}] Scheduled re-delivery of demand until [{1}]", StageActorName, _localCumulativeDemand); + PartnerRef.Tell(new CumulativeDemand(_localCumulativeDemand), Self); + ScheduleDemandRedelivery(); + break; + } + } + + private void InitialReceive(Tuple args) + { + var sender = args.Item1; + var message = args.Item2; + + switch (message) + { + case OnSubscribeHandshake handshake: + CancelTimer(SubscriptionTimeoutKey); + ObserveAndValidateSender(sender, "Illegal sender in OnSubscribeHandshake"); + Log.Debug("[{0}] Received handshake {1} from {2}", StageActorName, message, sender); + TriggerCumulativeDemand(); + break; + case SequencedOnNext onNext: + ObserveAndValidateSender(sender, "Illegal sender in SequencedOnNext"); + ObserveAndValidateSequenceNr(onNext.SeqNr, "Illegal sequence nr in SequencedOnNext"); + Log.Debug("[{0}] Received seq {1} from {2}", StageActorName, message, sender); + OnReceiveElement(onNext.Payload); + TriggerCumulativeDemand(); + break; + case RemoteStreamCompleted completed: + ObserveAndValidateSender(sender, "Illegal sender in RemoteStreamCompleted"); + ObserveAndValidateSequenceNr(completed.SeqNr, "Illegal sequence nr in RemoteStreamCompleted"); + Log.Debug("[{0}] The remote stream has completed, completing as well...", StageActorName); + _stageActor.Unwatch(sender); + _completed = true; + TryPush(); + break; + case RemoteStreamFailure failure: + ObserveAndValidateSender(sender, "Illegal sender in RemoteStreamFailure"); + Log.Warning("[{0}] The remote stream has failed, failing (reason: {1})", StageActorName, failure.Message); + _stageActor.Unwatch(sender); + FailStage(new RemoteStreamRefActorTerminatedException($"Remote stream ({sender.Path}) failed, reason: {failure.Message}")); + break; + case Terminated terminated: + if (Equals(_partnerRef, terminated.ActorRef)) + FailStage(new RemoteStreamRefActorTerminatedException( + $"The remote partner {terminated.ActorRef} has terminated! Tearing down this side of the stream as well.")); + else + FailStage(new RemoteStreamRefActorTerminatedException( + $"Received UNEXPECTED Terminated({terminated.ActorRef}) message! This actor was NOT our trusted remote partner, which was: {_partnerRef}. Tearing down.")); + + break; + } + } + + private void TryPush() + { + if (!_receiveBuffer.IsEmpty && IsAvailable(_stage.Outlet)) Push(_stage.Outlet, _receiveBuffer.Dequeue()); + else if (_receiveBuffer.IsEmpty && _completed) CompleteStage(); + } + + private void OnReceiveElement(object payload) + { + var outlet = _stage.Outlet; + _localRemainingRequested--; + if (_receiveBuffer.IsEmpty && IsAvailable(outlet)) + Push(outlet, (TOut)payload); + else if (_receiveBuffer.IsFull) + throw new IllegalStateException($"Attempted to overflow buffer! Capacity: {_receiveBuffer.Capacity}, incoming element: {payload}, localRemainingRequested: {_localRemainingRequested}, localCumulativeDemand: {_localCumulativeDemand}"); + else + _receiveBuffer.Enqueue((TOut)payload); + } + + /// + /// TBD + /// + /// Thrown when is invalid + private void ObserveAndValidateSender(IActorRef partner, string failureMessage) + { + if (_partnerRef == null) + { + Log.Debug("Received first message from {0}, assuming it to be the remote partner for this stage", partner); + _partnerRef = partner; + _stageActor.Watch(partner); + } + else if (!Equals(_partnerRef, partner)) + { + var ex = new InvalidPartnerActorException(partner, PartnerRef, failureMessage); + partner.Tell(new RemoteStreamFailure(ex.Message), Self); + throw ex; + } + } + + private void ObserveAndValidateSequenceNr(long seqNr, string failureMessage) + { + if (seqNr != _expectingSeqNr) + throw new InvalidSequenceNumberException(_expectingSeqNr, seqNr, failureMessage); + else + _expectingSeqNr++; + } + } + + #endregion + + private readonly IActorRef _initialPartnerRef; + + public SourceRefStageImpl(IActorRef initialPartnerRef) + { + _initialPartnerRef = initialPartnerRef; + + Shape = new SourceShape(Outlet); + } + + public Outlet Outlet { get; } = new Outlet("SourceRef.out"); + public override SourceShape Shape { get; } + + public override ILogicAndMaterializedValue>> CreateLogicAndMaterializedValue(Attributes inheritedAttributes) + { + var promise= new TaskCompletionSource>(); + return new LogicAndMaterializedValue>>(new Logic(this, promise, inheritedAttributes), promise.Task); + } + } +} \ No newline at end of file diff --git a/src/core/Akka.Streams/Implementation/ActorRefBackpressureSinkStage.cs b/src/core/Akka.Streams/Implementation/ActorRefBackpressureSinkStage.cs index 9f726e54ee6..0ae226d05fe 100644 --- a/src/core/Akka.Streams/Implementation/ActorRefBackpressureSinkStage.cs +++ b/src/core/Akka.Streams/Implementation/ActorRefBackpressureSinkStage.cs @@ -30,7 +30,8 @@ private sealed class Logic : InGraphStageLogic private readonly int _maxBuffer; private readonly List _buffer; private readonly Type _ackType; - private StageActorRef _self; + + public IActorRef Self => StageActor.Ref; public Logic(ActorRefBackpressureSinkStage stage, int maxBuffer) : base(stage.Shape) { @@ -65,7 +66,7 @@ public override void OnUpstreamFinish() public override void OnUpstreamFailure(Exception ex) { - _stage._actorRef.Tell(_stage._onFailureMessage(ex), _self); + _stage._actorRef.Tell(_stage._onFailureMessage(ex), Self); _completionSignalled = true; FailStage(ex); } @@ -100,9 +101,8 @@ private void Receive(Tuple evt) public override void PreStart() { SetKeepGoing(true); - _self = GetStageActorRef(Receive); - _self.Watch(_stage._actorRef); - _stage._actorRef.Tell(_stage._onInitMessage, _self); + GetStageActor(Receive).Watch(_stage._actorRef); + _stage._actorRef.Tell(_stage._onInitMessage, Self); Pull(_stage._inlet); } @@ -110,14 +110,14 @@ private void DequeueAndSend() { var msg = _buffer[0]; _buffer.RemoveAt(0); - _stage._actorRef.Tell(msg, _self); + _stage._actorRef.Tell(msg, Self); if (_buffer.Count == 0 && _completeReceived) Finish(); } private void Finish() { - _stage._actorRef.Tell(_stage._onCompleteMessage, _self); + _stage._actorRef.Tell(_stage._onCompleteMessage, Self); _completionSignalled = true; CompleteStage(); } @@ -125,7 +125,7 @@ private void Finish() public override void PostStop() { if(!_completionSignalled) - StageActorRef.Tell(_stage._onFailureMessage(new AbruptStageTerminationException(this))); + Self.Tell(_stage._onFailureMessage(new AbruptStageTerminationException(this))); } public override string ToString() => "ActorRefBackpressureSink"; diff --git a/src/core/Akka.Streams/Implementation/Buffers.cs b/src/core/Akka.Streams/Implementation/Buffers.cs index cec8295d7ef..ba08816131e 100644 --- a/src/core/Akka.Streams/Implementation/Buffers.cs +++ b/src/core/Akka.Streams/Implementation/Buffers.cs @@ -194,6 +194,8 @@ protected FixedSizeBuffer(int capacity) /// public bool NonEmpty => Used != 0; + public long RemainingCapacity => Capacity - Used; + // for the maintenance parameter see dropHead /// /// TBD diff --git a/src/core/Akka.Streams/Implementation/CompletedPublishers.cs b/src/core/Akka.Streams/Implementation/CompletedPublishers.cs index 26f7f647b87..3c081630612 100644 --- a/src/core/Akka.Streams/Implementation/CompletedPublishers.cs +++ b/src/core/Akka.Streams/Implementation/CompletedPublishers.cs @@ -113,10 +113,10 @@ internal sealed class MaybePublisher : IPublisher private class MaybeSubscription : ISubscription { private readonly ISubscriber _subscriber; - private readonly TaskCompletionSource _promise; + private readonly TaskCompletionSource> _promise; private bool _done; - public MaybeSubscription(ISubscriber subscriber, TaskCompletionSource promise) + public MaybeSubscription(ISubscriber subscriber, TaskCompletionSource> promise) { _subscriber = subscriber; _promise = promise; @@ -131,9 +131,9 @@ public void Request(long n) _done = true; _promise.Task.ContinueWith(t => { - if (!_promise.Task.Result.IsDefaultForType()) + if (!_promise.Task.Result.Equals(Option.None)) { - ReactiveStreamsCompliance.TryOnNext(_subscriber, _promise.Task.Result); + ReactiveStreamsCompliance.TryOnNext(_subscriber, _promise.Task.Result.Value); ReactiveStreamsCompliance.TryOnComplete(_subscriber); } else @@ -145,14 +145,14 @@ public void Request(long n) public void Cancel() { _done = true; - _promise.TrySetResult(default(T)); + _promise.TrySetResult(Option.None); } } /// /// TBD /// - public readonly TaskCompletionSource Promise; + public readonly TaskCompletionSource> Promise; /// /// TBD /// @@ -163,7 +163,7 @@ public void Cancel() /// /// TBD /// TBD - public MaybePublisher(TaskCompletionSource promise, string name) + public MaybePublisher(TaskCompletionSource> promise, string name) { Promise = promise; Name = name; diff --git a/src/core/Akka.Streams/Implementation/IO/TcpStages.cs b/src/core/Akka.Streams/Implementation/IO/TcpStages.cs index fe0330263f9..d724ca8cbfb 100644 --- a/src/core/Akka.Streams/Implementation/IO/TcpStages.cs +++ b/src/core/Akka.Streams/Implementation/IO/TcpStages.cs @@ -53,7 +53,7 @@ public ConnectionSourceStageLogic(Shape shape, ConnectionSourceStage stage, Task public void OnPull() { // Ignore if still binding - _listener?.Tell(new Tcp.ResumeAccepting(1), StageActorRef); + _listener?.Tell(new Tcp.ResumeAccepting(1), StageActor.Ref); } public void OnDownstreamFinish() => TryUnbind(); @@ -85,13 +85,13 @@ private void TryUnbind() { _unbindStarted = true; SetKeepGoing(true); - _listener.Tell(Tcp.Unbind.Instance, StageActorRef); + _listener.Tell(Tcp.Unbind.Instance, StageActor.Ref); } } private void UnbindCompleted() { - StageActorRef.Unwatch(_listener); + StageActor.Unwatch(_listener); if (_connectionFlowsAwaitingInitialization.Current == 0) CompleteStage(); else @@ -106,8 +106,8 @@ protected internal override void OnTimer(object timerKey) public override void PreStart() { - GetStageActorRef(Receive); - _stage._tcpManager.Tell(new Tcp.Bind(StageActorRef, _stage._endpoint, _stage._backlog, _stage._options, pullMode: true), StageActorRef); + GetStageActor(Receive); + _stage._tcpManager.Tell(new Tcp.Bind(StageActor.Ref, _stage._endpoint, _stage._backlog, _stage._options, pullMode: true), StageActor.Ref); } private void Receive(Tuple args) @@ -118,12 +118,12 @@ private void Receive(Tuple args) { var bound = (Tcp.Bound)msg; _listener = sender; - StageActorRef.Watch(_listener); + StageActor.Watch(_listener); if (IsAvailable(_stage._out)) - _listener.Tell(new Tcp.ResumeAccepting(1), StageActorRef); + _listener.Tell(new Tcp.ResumeAccepting(1), StageActor.Ref); - var thisStage = StageActorRef; + var thisStage = StageActor.Ref; _bindingPromise.TrySetResult(new StreamTcp.ServerBinding(bound.LocalAddress, () => { // Beware, sender must be explicit since stageActor.ref will be invalid to access after the stage stopped @@ -404,14 +404,14 @@ public TcpStreamLogic(FlowShape shape, ITcpRole role, En _bytesOut = shape.Outlet; _readHandler = new LambdaOutHandler( - onPull: () => _connection.Tell(Tcp.ResumeReading.Instance, StageActorRef), + onPull: () => _connection.Tell(Tcp.ResumeReading.Instance, StageActor.Ref), onDownstreamFinish: () => { if (!IsClosed(_bytesIn)) - _connection.Tell(Tcp.ResumeReading.Instance, StageActorRef); + _connection.Tell(Tcp.ResumeReading.Instance, StageActor.Ref); else { - _connection.Tell(Tcp.Abort.Instance, StageActorRef); + _connection.Tell(Tcp.Abort.Instance, StageActor.Ref); CompleteStage(); } }); @@ -423,17 +423,17 @@ public TcpStreamLogic(FlowShape shape, ITcpRole role, En { var elem = Grab(_bytesIn); ReactiveStreamsCompliance.RequireNonNullElement(elem); - _connection.Tell(Tcp.Write.Create(elem, WriteAck.Instance), StageActorRef); + _connection.Tell(Tcp.Write.Create(elem, WriteAck.Instance), StageActor.Ref); }, onUpstreamFinish: () => { // Reading has stopped before, either because of cancel, or PeerClosed, so just Close now // (or half-close is turned off) if (IsClosed(_bytesOut) || !_role.HalfClose) - _connection.Tell(Tcp.Close.Instance, StageActorRef); + _connection.Tell(Tcp.Close.Instance, StageActor.Ref); // We still read, so we only close the write side else if (_connection != null) - _connection.Tell(Tcp.ConfirmedClose.Instance, StageActorRef); + _connection.Tell(Tcp.ConfirmedClose.Instance, StageActor.Ref); else CompleteStage(); }, @@ -444,7 +444,7 @@ public TcpStreamLogic(FlowShape shape, ITcpRole role, En if (Interpreter.Log.IsDebugEnabled) Interpreter.Log.Debug( $"Aborting tcp connection to {_remoteAddress} because of upstream failure: {ex.Message}\n{ex.StackTrace}"); - _connection.Tell(Tcp.Abort.Instance, StageActorRef); + _connection.Tell(Tcp.Abort.Instance, StageActor.Ref); } else FailStage(ex); @@ -463,15 +463,15 @@ public override void PreStart() var inbound = (Inbound)_role; SetHandler(_bytesOut, _readHandler); _connection = inbound.Connection; - GetStageActorRef(Connected).Watch(_connection); - _connection.Tell(new Tcp.Register(StageActorRef, keepOpenOnPeerClosed: true, useResumeWriting: false), StageActorRef); + GetStageActor(Connected).Watch(_connection); + _connection.Tell(new Tcp.Register(StageActor.Ref, keepOpenOnPeerClosed: true, useResumeWriting: false), StageActor.Ref); Pull(_bytesIn); } else { var outbound = (Outbound)_role; - GetStageActorRef(Connecting(outbound)).Watch(outbound.Manager); - outbound.Manager.Tell(outbound.ConnectCmd, StageActorRef); + GetStageActor(Connecting(outbound)).Watch(outbound.Manager); + outbound.Manager.Tell(outbound.ConnectCmd, StageActor.Ref); } } @@ -506,13 +506,13 @@ private StageActorRef.Receive Connecting(Outbound outbound) ((Outbound)_role).LocalAddressPromise.TrySetResult(connected.LocalAddress); _connection = sender; SetHandler(_bytesOut, _readHandler); - StageActorRef.Unwatch(outbound.Manager); - StageActorRef.Become(Connected); - StageActorRef.Watch(_connection); - _connection.Tell(new Tcp.Register(StageActorRef, keepOpenOnPeerClosed: true, useResumeWriting: false), StageActorRef); + StageActor.Unwatch(outbound.Manager); + StageActor.Become(Connected); + StageActor.Watch(_connection); + _connection.Tell(new Tcp.Register(StageActor.Ref, keepOpenOnPeerClosed: true, useResumeWriting: false), StageActor.Ref); if (IsAvailable(_bytesOut)) - _connection.Tell(Tcp.ResumeReading.Instance, StageActorRef); + _connection.Tell(Tcp.ResumeReading.Instance, StageActor.Ref); Pull(_bytesIn); } @@ -535,7 +535,7 @@ private void Connected(Tuple args) { var received = (Tcp.Received)msg; // Keep on reading even when closed. There is no "close-read-side" in TCP - if (IsClosed(_bytesOut)) _connection.Tell(Tcp.ResumeReading.Instance, StageActorRef); + if (IsClosed(_bytesOut)) _connection.Tell(Tcp.ResumeReading.Instance, StageActor.Ref); else Push(_bytesOut, received.Data); } else if (msg is WriteAck) diff --git a/src/core/Akka.Streams/Implementation/Modules.cs b/src/core/Akka.Streams/Implementation/Modules.cs index 5c7a949dc33..b0798d8c712 100644 --- a/src/core/Akka.Streams/Implementation/Modules.cs +++ b/src/core/Akka.Streams/Implementation/Modules.cs @@ -10,6 +10,7 @@ using Akka.Actor; using Akka.Annotations; using Akka.Streams.Actors; +using Akka.Streams.Util; using Reactive.Streams; namespace Akka.Streams.Implementation @@ -252,7 +253,7 @@ public override IPublisher Create(MaterializationContext context, out NotU /// /// TBD [InternalApi] - public sealed class MaybeSource : SourceModule> + public sealed class MaybeSource : SourceModule>> { /// /// TBD @@ -282,7 +283,7 @@ public override IModule WithAttributes(Attributes attributes) /// /// TBD /// TBD - protected override SourceModule> NewInstance(SourceShape shape) + protected override SourceModule>> NewInstance(SourceShape shape) => new MaybeSource(Attributes, shape); /// @@ -291,9 +292,9 @@ protected override SourceModule> NewInstance(So /// TBD /// TBD /// TBD - public override IPublisher Create(MaterializationContext context, out TaskCompletionSource materializer) + public override IPublisher Create(MaterializationContext context, out TaskCompletionSource> materializer) { - materializer = new TaskCompletionSource(); + materializer = new TaskCompletionSource>(); return new MaybePublisher(materializer, Attributes.GetNameOrDefault("MaybeSource")); } } @@ -396,7 +397,7 @@ public ActorRefSource(int bufferSize, OverflowStrategy overflowStrategy, Attribu /// /// TBD /// TBD - public override IModule WithAttributes(Attributes attributes) + public override IModule WithAttributes(Attributes attributes) => new ActorRefSource(_bufferSize, _overflowStrategy, attributes, AmendShape(attributes)); /// @@ -404,7 +405,7 @@ public override IModule WithAttributes(Attributes attributes) /// /// TBD /// TBD - protected override SourceModule NewInstance(SourceShape shape) + protected override SourceModule NewInstance(SourceShape shape) => new ActorRefSource(_bufferSize, _overflowStrategy, Attributes, shape); /// diff --git a/src/core/Akka.Streams/Serialization/Proto/StreamRefMessages.g.cs b/src/core/Akka.Streams/Serialization/Proto/StreamRefMessages.g.cs new file mode 100644 index 00000000000..3854e88d887 --- /dev/null +++ b/src/core/Akka.Streams/Serialization/Proto/StreamRefMessages.g.cs @@ -0,0 +1,1413 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: StreamRefMessages.proto +#pragma warning disable 1591, 0612, 3021 +#region Designer generated code + +using pb = global::Google.Protobuf; +using pbc = global::Google.Protobuf.Collections; +using pbr = global::Google.Protobuf.Reflection; +using scg = global::System.Collections.Generic; +namespace Akka.Streams.Serialization.Proto.Msg { + + /// Holder for reflection information generated from StreamRefMessages.proto + internal static partial class StreamRefMessagesReflection { + + #region Descriptor + /// File descriptor for StreamRefMessages.proto + public static pbr::FileDescriptor Descriptor { + get { return descriptor; } + } + private static pbr::FileDescriptor descriptor; + + static StreamRefMessagesReflection() { + byte[] descriptorData = global::System.Convert.FromBase64String( + string.Concat( + "ChdTdHJlYW1SZWZNZXNzYWdlcy5wcm90bxIkQWtrYS5TdHJlYW1zLlNlcmlh", + "bGl6YXRpb24uUHJvdG8uTXNnIh0KCUV2ZW50VHlwZRIQCgh0eXBlTmFtZRgB", + "IAEoCSKQAQoHU2lua1JlZhJBCgl0YXJnZXRSZWYYASABKAsyLi5Ba2thLlN0", + "cmVhbXMuU2VyaWFsaXphdGlvbi5Qcm90by5Nc2cuQWN0b3JSZWYSQgoJZXZl", + "bnRUeXBlGAIgASgLMi8uQWtrYS5TdHJlYW1zLlNlcmlhbGl6YXRpb24uUHJv", + "dG8uTXNnLkV2ZW50VHlwZSKSAQoJU291cmNlUmVmEkEKCW9yaWdpblJlZhgB", + "IAEoCzIuLkFra2EuU3RyZWFtcy5TZXJpYWxpemF0aW9uLlByb3RvLk1zZy5B", + "Y3RvclJlZhJCCglldmVudFR5cGUYAiABKAsyLy5Ba2thLlN0cmVhbXMuU2Vy", + "aWFsaXphdGlvbi5Qcm90by5Nc2cuRXZlbnRUeXBlIhgKCEFjdG9yUmVmEgwK", + "BHBhdGgYASABKAkiUQoHUGF5bG9hZBIXCg9lbmNsb3NlZE1lc3NhZ2UYASAB", + "KAwSFAoMc2VyaWFsaXplcklkGAIgASgFEhcKD21lc3NhZ2VNYW5pZmVzdBgD", + "IAEoDCJZChRPblN1YnNjcmliZUhhbmRzaGFrZRJBCgl0YXJnZXRSZWYYASAB", + "KAsyLi5Ba2thLlN0cmVhbXMuU2VyaWFsaXphdGlvbi5Qcm90by5Nc2cuQWN0", + "b3JSZWYiIQoQQ3VtdWxhdGl2ZURlbWFuZBINCgVzZXFOchgBIAEoAyJgCg9T", + "ZXF1ZW5jZWRPbk5leHQSDQoFc2VxTnIYASABKAMSPgoHcGF5bG9hZBgCIAEo", + "CzItLkFra2EuU3RyZWFtcy5TZXJpYWxpemF0aW9uLlByb3RvLk1zZy5QYXls", + "b2FkIiQKE1JlbW90ZVN0cmVhbUZhaWx1cmUSDQoFY2F1c2UYASABKAwiJgoV", + "UmVtb3RlU3RyZWFtQ29tcGxldGVkEg0KBXNlcU5yGAEgASgDQgJIAWIGcHJv", + "dG8z")); + descriptor = pbr::FileDescriptor.FromGeneratedCode(descriptorData, + new pbr::FileDescriptor[] { }, + new pbr::GeneratedClrTypeInfo(null, new pbr::GeneratedClrTypeInfo[] { + new pbr::GeneratedClrTypeInfo(typeof(global::Akka.Streams.Serialization.Proto.Msg.EventType), global::Akka.Streams.Serialization.Proto.Msg.EventType.Parser, new[]{ "TypeName" }, null, null, null), + new pbr::GeneratedClrTypeInfo(typeof(global::Akka.Streams.Serialization.Proto.Msg.SinkRef), global::Akka.Streams.Serialization.Proto.Msg.SinkRef.Parser, new[]{ "TargetRef", "EventType" }, null, null, null), + new pbr::GeneratedClrTypeInfo(typeof(global::Akka.Streams.Serialization.Proto.Msg.SourceRef), global::Akka.Streams.Serialization.Proto.Msg.SourceRef.Parser, new[]{ "OriginRef", "EventType" }, null, null, null), + new pbr::GeneratedClrTypeInfo(typeof(global::Akka.Streams.Serialization.Proto.Msg.ActorRef), global::Akka.Streams.Serialization.Proto.Msg.ActorRef.Parser, new[]{ "Path" }, null, null, null), + new pbr::GeneratedClrTypeInfo(typeof(global::Akka.Streams.Serialization.Proto.Msg.Payload), global::Akka.Streams.Serialization.Proto.Msg.Payload.Parser, new[]{ "EnclosedMessage", "SerializerId", "MessageManifest" }, null, null, null), + new pbr::GeneratedClrTypeInfo(typeof(global::Akka.Streams.Serialization.Proto.Msg.OnSubscribeHandshake), global::Akka.Streams.Serialization.Proto.Msg.OnSubscribeHandshake.Parser, new[]{ "TargetRef" }, null, null, null), + new pbr::GeneratedClrTypeInfo(typeof(global::Akka.Streams.Serialization.Proto.Msg.CumulativeDemand), global::Akka.Streams.Serialization.Proto.Msg.CumulativeDemand.Parser, new[]{ "SeqNr" }, null, null, null), + new pbr::GeneratedClrTypeInfo(typeof(global::Akka.Streams.Serialization.Proto.Msg.SequencedOnNext), global::Akka.Streams.Serialization.Proto.Msg.SequencedOnNext.Parser, new[]{ "SeqNr", "Payload" }, null, null, null), + new pbr::GeneratedClrTypeInfo(typeof(global::Akka.Streams.Serialization.Proto.Msg.RemoteStreamFailure), global::Akka.Streams.Serialization.Proto.Msg.RemoteStreamFailure.Parser, new[]{ "Cause" }, null, null, null), + new pbr::GeneratedClrTypeInfo(typeof(global::Akka.Streams.Serialization.Proto.Msg.RemoteStreamCompleted), global::Akka.Streams.Serialization.Proto.Msg.RemoteStreamCompleted.Parser, new[]{ "SeqNr" }, null, null, null) + })); + } + #endregion + + } + #region Messages + internal sealed partial class EventType : pb::IMessage { + private static readonly pb::MessageParser _parser = new pb::MessageParser(() => new EventType()); + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pb::MessageParser Parser { get { return _parser; } } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pbr::MessageDescriptor Descriptor { + get { return global::Akka.Streams.Serialization.Proto.Msg.StreamRefMessagesReflection.Descriptor.MessageTypes[0]; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + pbr::MessageDescriptor pb::IMessage.Descriptor { + get { return Descriptor; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public EventType() { + OnConstruction(); + } + + partial void OnConstruction(); + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public EventType(EventType other) : this() { + typeName_ = other.typeName_; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public EventType Clone() { + return new EventType(this); + } + + /// Field number for the "typeName" field. + public const int TypeNameFieldNumber = 1; + private string typeName_ = ""; + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public string TypeName { + get { return typeName_; } + set { + typeName_ = pb::ProtoPreconditions.CheckNotNull(value, "value"); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override bool Equals(object other) { + return Equals(other as EventType); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public bool Equals(EventType other) { + if (ReferenceEquals(other, null)) { + return false; + } + if (ReferenceEquals(other, this)) { + return true; + } + if (TypeName != other.TypeName) return false; + return true; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override int GetHashCode() { + int hash = 1; + if (TypeName.Length != 0) hash ^= TypeName.GetHashCode(); + return hash; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override string ToString() { + return pb::JsonFormatter.ToDiagnosticString(this); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void WriteTo(pb::CodedOutputStream output) { + if (TypeName.Length != 0) { + output.WriteRawTag(10); + output.WriteString(TypeName); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public int CalculateSize() { + int size = 0; + if (TypeName.Length != 0) { + size += 1 + pb::CodedOutputStream.ComputeStringSize(TypeName); + } + return size; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(EventType other) { + if (other == null) { + return; + } + if (other.TypeName.Length != 0) { + TypeName = other.TypeName; + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(pb::CodedInputStream input) { + uint tag; + while ((tag = input.ReadTag()) != 0) { + switch(tag) { + default: + input.SkipLastField(); + break; + case 10: { + TypeName = input.ReadString(); + break; + } + } + } + } + + } + + internal sealed partial class SinkRef : pb::IMessage { + private static readonly pb::MessageParser _parser = new pb::MessageParser(() => new SinkRef()); + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pb::MessageParser Parser { get { return _parser; } } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pbr::MessageDescriptor Descriptor { + get { return global::Akka.Streams.Serialization.Proto.Msg.StreamRefMessagesReflection.Descriptor.MessageTypes[1]; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + pbr::MessageDescriptor pb::IMessage.Descriptor { + get { return Descriptor; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public SinkRef() { + OnConstruction(); + } + + partial void OnConstruction(); + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public SinkRef(SinkRef other) : this() { + TargetRef = other.targetRef_ != null ? other.TargetRef.Clone() : null; + EventType = other.eventType_ != null ? other.EventType.Clone() : null; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public SinkRef Clone() { + return new SinkRef(this); + } + + /// Field number for the "targetRef" field. + public const int TargetRefFieldNumber = 1; + private global::Akka.Streams.Serialization.Proto.Msg.ActorRef targetRef_; + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public global::Akka.Streams.Serialization.Proto.Msg.ActorRef TargetRef { + get { return targetRef_; } + set { + targetRef_ = value; + } + } + + /// Field number for the "eventType" field. + public const int EventTypeFieldNumber = 2; + private global::Akka.Streams.Serialization.Proto.Msg.EventType eventType_; + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public global::Akka.Streams.Serialization.Proto.Msg.EventType EventType { + get { return eventType_; } + set { + eventType_ = value; + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override bool Equals(object other) { + return Equals(other as SinkRef); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public bool Equals(SinkRef other) { + if (ReferenceEquals(other, null)) { + return false; + } + if (ReferenceEquals(other, this)) { + return true; + } + if (!object.Equals(TargetRef, other.TargetRef)) return false; + if (!object.Equals(EventType, other.EventType)) return false; + return true; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override int GetHashCode() { + int hash = 1; + if (targetRef_ != null) hash ^= TargetRef.GetHashCode(); + if (eventType_ != null) hash ^= EventType.GetHashCode(); + return hash; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override string ToString() { + return pb::JsonFormatter.ToDiagnosticString(this); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void WriteTo(pb::CodedOutputStream output) { + if (targetRef_ != null) { + output.WriteRawTag(10); + output.WriteMessage(TargetRef); + } + if (eventType_ != null) { + output.WriteRawTag(18); + output.WriteMessage(EventType); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public int CalculateSize() { + int size = 0; + if (targetRef_ != null) { + size += 1 + pb::CodedOutputStream.ComputeMessageSize(TargetRef); + } + if (eventType_ != null) { + size += 1 + pb::CodedOutputStream.ComputeMessageSize(EventType); + } + return size; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(SinkRef other) { + if (other == null) { + return; + } + if (other.targetRef_ != null) { + if (targetRef_ == null) { + targetRef_ = new global::Akka.Streams.Serialization.Proto.Msg.ActorRef(); + } + TargetRef.MergeFrom(other.TargetRef); + } + if (other.eventType_ != null) { + if (eventType_ == null) { + eventType_ = new global::Akka.Streams.Serialization.Proto.Msg.EventType(); + } + EventType.MergeFrom(other.EventType); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(pb::CodedInputStream input) { + uint tag; + while ((tag = input.ReadTag()) != 0) { + switch(tag) { + default: + input.SkipLastField(); + break; + case 10: { + if (targetRef_ == null) { + targetRef_ = new global::Akka.Streams.Serialization.Proto.Msg.ActorRef(); + } + input.ReadMessage(targetRef_); + break; + } + case 18: { + if (eventType_ == null) { + eventType_ = new global::Akka.Streams.Serialization.Proto.Msg.EventType(); + } + input.ReadMessage(eventType_); + break; + } + } + } + } + + } + + internal sealed partial class SourceRef : pb::IMessage { + private static readonly pb::MessageParser _parser = new pb::MessageParser(() => new SourceRef()); + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pb::MessageParser Parser { get { return _parser; } } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pbr::MessageDescriptor Descriptor { + get { return global::Akka.Streams.Serialization.Proto.Msg.StreamRefMessagesReflection.Descriptor.MessageTypes[2]; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + pbr::MessageDescriptor pb::IMessage.Descriptor { + get { return Descriptor; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public SourceRef() { + OnConstruction(); + } + + partial void OnConstruction(); + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public SourceRef(SourceRef other) : this() { + OriginRef = other.originRef_ != null ? other.OriginRef.Clone() : null; + EventType = other.eventType_ != null ? other.EventType.Clone() : null; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public SourceRef Clone() { + return new SourceRef(this); + } + + /// Field number for the "originRef" field. + public const int OriginRefFieldNumber = 1; + private global::Akka.Streams.Serialization.Proto.Msg.ActorRef originRef_; + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public global::Akka.Streams.Serialization.Proto.Msg.ActorRef OriginRef { + get { return originRef_; } + set { + originRef_ = value; + } + } + + /// Field number for the "eventType" field. + public const int EventTypeFieldNumber = 2; + private global::Akka.Streams.Serialization.Proto.Msg.EventType eventType_; + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public global::Akka.Streams.Serialization.Proto.Msg.EventType EventType { + get { return eventType_; } + set { + eventType_ = value; + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override bool Equals(object other) { + return Equals(other as SourceRef); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public bool Equals(SourceRef other) { + if (ReferenceEquals(other, null)) { + return false; + } + if (ReferenceEquals(other, this)) { + return true; + } + if (!object.Equals(OriginRef, other.OriginRef)) return false; + if (!object.Equals(EventType, other.EventType)) return false; + return true; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override int GetHashCode() { + int hash = 1; + if (originRef_ != null) hash ^= OriginRef.GetHashCode(); + if (eventType_ != null) hash ^= EventType.GetHashCode(); + return hash; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override string ToString() { + return pb::JsonFormatter.ToDiagnosticString(this); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void WriteTo(pb::CodedOutputStream output) { + if (originRef_ != null) { + output.WriteRawTag(10); + output.WriteMessage(OriginRef); + } + if (eventType_ != null) { + output.WriteRawTag(18); + output.WriteMessage(EventType); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public int CalculateSize() { + int size = 0; + if (originRef_ != null) { + size += 1 + pb::CodedOutputStream.ComputeMessageSize(OriginRef); + } + if (eventType_ != null) { + size += 1 + pb::CodedOutputStream.ComputeMessageSize(EventType); + } + return size; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(SourceRef other) { + if (other == null) { + return; + } + if (other.originRef_ != null) { + if (originRef_ == null) { + originRef_ = new global::Akka.Streams.Serialization.Proto.Msg.ActorRef(); + } + OriginRef.MergeFrom(other.OriginRef); + } + if (other.eventType_ != null) { + if (eventType_ == null) { + eventType_ = new global::Akka.Streams.Serialization.Proto.Msg.EventType(); + } + EventType.MergeFrom(other.EventType); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(pb::CodedInputStream input) { + uint tag; + while ((tag = input.ReadTag()) != 0) { + switch(tag) { + default: + input.SkipLastField(); + break; + case 10: { + if (originRef_ == null) { + originRef_ = new global::Akka.Streams.Serialization.Proto.Msg.ActorRef(); + } + input.ReadMessage(originRef_); + break; + } + case 18: { + if (eventType_ == null) { + eventType_ = new global::Akka.Streams.Serialization.Proto.Msg.EventType(); + } + input.ReadMessage(eventType_); + break; + } + } + } + } + + } + + internal sealed partial class ActorRef : pb::IMessage { + private static readonly pb::MessageParser _parser = new pb::MessageParser(() => new ActorRef()); + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pb::MessageParser Parser { get { return _parser; } } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pbr::MessageDescriptor Descriptor { + get { return global::Akka.Streams.Serialization.Proto.Msg.StreamRefMessagesReflection.Descriptor.MessageTypes[3]; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + pbr::MessageDescriptor pb::IMessage.Descriptor { + get { return Descriptor; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public ActorRef() { + OnConstruction(); + } + + partial void OnConstruction(); + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public ActorRef(ActorRef other) : this() { + path_ = other.path_; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public ActorRef Clone() { + return new ActorRef(this); + } + + /// Field number for the "path" field. + public const int PathFieldNumber = 1; + private string path_ = ""; + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public string Path { + get { return path_; } + set { + path_ = pb::ProtoPreconditions.CheckNotNull(value, "value"); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override bool Equals(object other) { + return Equals(other as ActorRef); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public bool Equals(ActorRef other) { + if (ReferenceEquals(other, null)) { + return false; + } + if (ReferenceEquals(other, this)) { + return true; + } + if (Path != other.Path) return false; + return true; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override int GetHashCode() { + int hash = 1; + if (Path.Length != 0) hash ^= Path.GetHashCode(); + return hash; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override string ToString() { + return pb::JsonFormatter.ToDiagnosticString(this); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void WriteTo(pb::CodedOutputStream output) { + if (Path.Length != 0) { + output.WriteRawTag(10); + output.WriteString(Path); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public int CalculateSize() { + int size = 0; + if (Path.Length != 0) { + size += 1 + pb::CodedOutputStream.ComputeStringSize(Path); + } + return size; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(ActorRef other) { + if (other == null) { + return; + } + if (other.Path.Length != 0) { + Path = other.Path; + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(pb::CodedInputStream input) { + uint tag; + while ((tag = input.ReadTag()) != 0) { + switch(tag) { + default: + input.SkipLastField(); + break; + case 10: { + Path = input.ReadString(); + break; + } + } + } + } + + } + + internal sealed partial class Payload : pb::IMessage { + private static readonly pb::MessageParser _parser = new pb::MessageParser(() => new Payload()); + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pb::MessageParser Parser { get { return _parser; } } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pbr::MessageDescriptor Descriptor { + get { return global::Akka.Streams.Serialization.Proto.Msg.StreamRefMessagesReflection.Descriptor.MessageTypes[4]; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + pbr::MessageDescriptor pb::IMessage.Descriptor { + get { return Descriptor; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public Payload() { + OnConstruction(); + } + + partial void OnConstruction(); + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public Payload(Payload other) : this() { + enclosedMessage_ = other.enclosedMessage_; + serializerId_ = other.serializerId_; + messageManifest_ = other.messageManifest_; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public Payload Clone() { + return new Payload(this); + } + + /// Field number for the "enclosedMessage" field. + public const int EnclosedMessageFieldNumber = 1; + private pb::ByteString enclosedMessage_ = pb::ByteString.Empty; + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public pb::ByteString EnclosedMessage { + get { return enclosedMessage_; } + set { + enclosedMessage_ = pb::ProtoPreconditions.CheckNotNull(value, "value"); + } + } + + /// Field number for the "serializerId" field. + public const int SerializerIdFieldNumber = 2; + private int serializerId_; + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public int SerializerId { + get { return serializerId_; } + set { + serializerId_ = value; + } + } + + /// Field number for the "messageManifest" field. + public const int MessageManifestFieldNumber = 3; + private pb::ByteString messageManifest_ = pb::ByteString.Empty; + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public pb::ByteString MessageManifest { + get { return messageManifest_; } + set { + messageManifest_ = pb::ProtoPreconditions.CheckNotNull(value, "value"); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override bool Equals(object other) { + return Equals(other as Payload); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public bool Equals(Payload other) { + if (ReferenceEquals(other, null)) { + return false; + } + if (ReferenceEquals(other, this)) { + return true; + } + if (EnclosedMessage != other.EnclosedMessage) return false; + if (SerializerId != other.SerializerId) return false; + if (MessageManifest != other.MessageManifest) return false; + return true; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override int GetHashCode() { + int hash = 1; + if (EnclosedMessage.Length != 0) hash ^= EnclosedMessage.GetHashCode(); + if (SerializerId != 0) hash ^= SerializerId.GetHashCode(); + if (MessageManifest.Length != 0) hash ^= MessageManifest.GetHashCode(); + return hash; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override string ToString() { + return pb::JsonFormatter.ToDiagnosticString(this); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void WriteTo(pb::CodedOutputStream output) { + if (EnclosedMessage.Length != 0) { + output.WriteRawTag(10); + output.WriteBytes(EnclosedMessage); + } + if (SerializerId != 0) { + output.WriteRawTag(16); + output.WriteInt32(SerializerId); + } + if (MessageManifest.Length != 0) { + output.WriteRawTag(26); + output.WriteBytes(MessageManifest); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public int CalculateSize() { + int size = 0; + if (EnclosedMessage.Length != 0) { + size += 1 + pb::CodedOutputStream.ComputeBytesSize(EnclosedMessage); + } + if (SerializerId != 0) { + size += 1 + pb::CodedOutputStream.ComputeInt32Size(SerializerId); + } + if (MessageManifest.Length != 0) { + size += 1 + pb::CodedOutputStream.ComputeBytesSize(MessageManifest); + } + return size; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(Payload other) { + if (other == null) { + return; + } + if (other.EnclosedMessage.Length != 0) { + EnclosedMessage = other.EnclosedMessage; + } + if (other.SerializerId != 0) { + SerializerId = other.SerializerId; + } + if (other.MessageManifest.Length != 0) { + MessageManifest = other.MessageManifest; + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(pb::CodedInputStream input) { + uint tag; + while ((tag = input.ReadTag()) != 0) { + switch(tag) { + default: + input.SkipLastField(); + break; + case 10: { + EnclosedMessage = input.ReadBytes(); + break; + } + case 16: { + SerializerId = input.ReadInt32(); + break; + } + case 26: { + MessageManifest = input.ReadBytes(); + break; + } + } + } + } + + } + + internal sealed partial class OnSubscribeHandshake : pb::IMessage { + private static readonly pb::MessageParser _parser = new pb::MessageParser(() => new OnSubscribeHandshake()); + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pb::MessageParser Parser { get { return _parser; } } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pbr::MessageDescriptor Descriptor { + get { return global::Akka.Streams.Serialization.Proto.Msg.StreamRefMessagesReflection.Descriptor.MessageTypes[5]; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + pbr::MessageDescriptor pb::IMessage.Descriptor { + get { return Descriptor; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public OnSubscribeHandshake() { + OnConstruction(); + } + + partial void OnConstruction(); + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public OnSubscribeHandshake(OnSubscribeHandshake other) : this() { + TargetRef = other.targetRef_ != null ? other.TargetRef.Clone() : null; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public OnSubscribeHandshake Clone() { + return new OnSubscribeHandshake(this); + } + + /// Field number for the "targetRef" field. + public const int TargetRefFieldNumber = 1; + private global::Akka.Streams.Serialization.Proto.Msg.ActorRef targetRef_; + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public global::Akka.Streams.Serialization.Proto.Msg.ActorRef TargetRef { + get { return targetRef_; } + set { + targetRef_ = value; + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override bool Equals(object other) { + return Equals(other as OnSubscribeHandshake); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public bool Equals(OnSubscribeHandshake other) { + if (ReferenceEquals(other, null)) { + return false; + } + if (ReferenceEquals(other, this)) { + return true; + } + if (!object.Equals(TargetRef, other.TargetRef)) return false; + return true; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override int GetHashCode() { + int hash = 1; + if (targetRef_ != null) hash ^= TargetRef.GetHashCode(); + return hash; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override string ToString() { + return pb::JsonFormatter.ToDiagnosticString(this); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void WriteTo(pb::CodedOutputStream output) { + if (targetRef_ != null) { + output.WriteRawTag(10); + output.WriteMessage(TargetRef); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public int CalculateSize() { + int size = 0; + if (targetRef_ != null) { + size += 1 + pb::CodedOutputStream.ComputeMessageSize(TargetRef); + } + return size; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(OnSubscribeHandshake other) { + if (other == null) { + return; + } + if (other.targetRef_ != null) { + if (targetRef_ == null) { + targetRef_ = new global::Akka.Streams.Serialization.Proto.Msg.ActorRef(); + } + TargetRef.MergeFrom(other.TargetRef); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(pb::CodedInputStream input) { + uint tag; + while ((tag = input.ReadTag()) != 0) { + switch(tag) { + default: + input.SkipLastField(); + break; + case 10: { + if (targetRef_ == null) { + targetRef_ = new global::Akka.Streams.Serialization.Proto.Msg.ActorRef(); + } + input.ReadMessage(targetRef_); + break; + } + } + } + } + + } + + internal sealed partial class CumulativeDemand : pb::IMessage { + private static readonly pb::MessageParser _parser = new pb::MessageParser(() => new CumulativeDemand()); + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pb::MessageParser Parser { get { return _parser; } } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pbr::MessageDescriptor Descriptor { + get { return global::Akka.Streams.Serialization.Proto.Msg.StreamRefMessagesReflection.Descriptor.MessageTypes[6]; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + pbr::MessageDescriptor pb::IMessage.Descriptor { + get { return Descriptor; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public CumulativeDemand() { + OnConstruction(); + } + + partial void OnConstruction(); + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public CumulativeDemand(CumulativeDemand other) : this() { + seqNr_ = other.seqNr_; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public CumulativeDemand Clone() { + return new CumulativeDemand(this); + } + + /// Field number for the "seqNr" field. + public const int SeqNrFieldNumber = 1; + private long seqNr_; + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public long SeqNr { + get { return seqNr_; } + set { + seqNr_ = value; + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override bool Equals(object other) { + return Equals(other as CumulativeDemand); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public bool Equals(CumulativeDemand other) { + if (ReferenceEquals(other, null)) { + return false; + } + if (ReferenceEquals(other, this)) { + return true; + } + if (SeqNr != other.SeqNr) return false; + return true; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override int GetHashCode() { + int hash = 1; + if (SeqNr != 0L) hash ^= SeqNr.GetHashCode(); + return hash; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override string ToString() { + return pb::JsonFormatter.ToDiagnosticString(this); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void WriteTo(pb::CodedOutputStream output) { + if (SeqNr != 0L) { + output.WriteRawTag(8); + output.WriteInt64(SeqNr); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public int CalculateSize() { + int size = 0; + if (SeqNr != 0L) { + size += 1 + pb::CodedOutputStream.ComputeInt64Size(SeqNr); + } + return size; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(CumulativeDemand other) { + if (other == null) { + return; + } + if (other.SeqNr != 0L) { + SeqNr = other.SeqNr; + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(pb::CodedInputStream input) { + uint tag; + while ((tag = input.ReadTag()) != 0) { + switch(tag) { + default: + input.SkipLastField(); + break; + case 8: { + SeqNr = input.ReadInt64(); + break; + } + } + } + } + + } + + internal sealed partial class SequencedOnNext : pb::IMessage { + private static readonly pb::MessageParser _parser = new pb::MessageParser(() => new SequencedOnNext()); + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pb::MessageParser Parser { get { return _parser; } } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pbr::MessageDescriptor Descriptor { + get { return global::Akka.Streams.Serialization.Proto.Msg.StreamRefMessagesReflection.Descriptor.MessageTypes[7]; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + pbr::MessageDescriptor pb::IMessage.Descriptor { + get { return Descriptor; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public SequencedOnNext() { + OnConstruction(); + } + + partial void OnConstruction(); + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public SequencedOnNext(SequencedOnNext other) : this() { + seqNr_ = other.seqNr_; + Payload = other.payload_ != null ? other.Payload.Clone() : null; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public SequencedOnNext Clone() { + return new SequencedOnNext(this); + } + + /// Field number for the "seqNr" field. + public const int SeqNrFieldNumber = 1; + private long seqNr_; + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public long SeqNr { + get { return seqNr_; } + set { + seqNr_ = value; + } + } + + /// Field number for the "payload" field. + public const int PayloadFieldNumber = 2; + private global::Akka.Streams.Serialization.Proto.Msg.Payload payload_; + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public global::Akka.Streams.Serialization.Proto.Msg.Payload Payload { + get { return payload_; } + set { + payload_ = value; + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override bool Equals(object other) { + return Equals(other as SequencedOnNext); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public bool Equals(SequencedOnNext other) { + if (ReferenceEquals(other, null)) { + return false; + } + if (ReferenceEquals(other, this)) { + return true; + } + if (SeqNr != other.SeqNr) return false; + if (!object.Equals(Payload, other.Payload)) return false; + return true; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override int GetHashCode() { + int hash = 1; + if (SeqNr != 0L) hash ^= SeqNr.GetHashCode(); + if (payload_ != null) hash ^= Payload.GetHashCode(); + return hash; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override string ToString() { + return pb::JsonFormatter.ToDiagnosticString(this); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void WriteTo(pb::CodedOutputStream output) { + if (SeqNr != 0L) { + output.WriteRawTag(8); + output.WriteInt64(SeqNr); + } + if (payload_ != null) { + output.WriteRawTag(18); + output.WriteMessage(Payload); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public int CalculateSize() { + int size = 0; + if (SeqNr != 0L) { + size += 1 + pb::CodedOutputStream.ComputeInt64Size(SeqNr); + } + if (payload_ != null) { + size += 1 + pb::CodedOutputStream.ComputeMessageSize(Payload); + } + return size; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(SequencedOnNext other) { + if (other == null) { + return; + } + if (other.SeqNr != 0L) { + SeqNr = other.SeqNr; + } + if (other.payload_ != null) { + if (payload_ == null) { + payload_ = new global::Akka.Streams.Serialization.Proto.Msg.Payload(); + } + Payload.MergeFrom(other.Payload); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(pb::CodedInputStream input) { + uint tag; + while ((tag = input.ReadTag()) != 0) { + switch(tag) { + default: + input.SkipLastField(); + break; + case 8: { + SeqNr = input.ReadInt64(); + break; + } + case 18: { + if (payload_ == null) { + payload_ = new global::Akka.Streams.Serialization.Proto.Msg.Payload(); + } + input.ReadMessage(payload_); + break; + } + } + } + } + + } + + internal sealed partial class RemoteStreamFailure : pb::IMessage { + private static readonly pb::MessageParser _parser = new pb::MessageParser(() => new RemoteStreamFailure()); + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pb::MessageParser Parser { get { return _parser; } } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pbr::MessageDescriptor Descriptor { + get { return global::Akka.Streams.Serialization.Proto.Msg.StreamRefMessagesReflection.Descriptor.MessageTypes[8]; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + pbr::MessageDescriptor pb::IMessage.Descriptor { + get { return Descriptor; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public RemoteStreamFailure() { + OnConstruction(); + } + + partial void OnConstruction(); + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public RemoteStreamFailure(RemoteStreamFailure other) : this() { + cause_ = other.cause_; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public RemoteStreamFailure Clone() { + return new RemoteStreamFailure(this); + } + + /// Field number for the "cause" field. + public const int CauseFieldNumber = 1; + private pb::ByteString cause_ = pb::ByteString.Empty; + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public pb::ByteString Cause { + get { return cause_; } + set { + cause_ = pb::ProtoPreconditions.CheckNotNull(value, "value"); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override bool Equals(object other) { + return Equals(other as RemoteStreamFailure); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public bool Equals(RemoteStreamFailure other) { + if (ReferenceEquals(other, null)) { + return false; + } + if (ReferenceEquals(other, this)) { + return true; + } + if (Cause != other.Cause) return false; + return true; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override int GetHashCode() { + int hash = 1; + if (Cause.Length != 0) hash ^= Cause.GetHashCode(); + return hash; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override string ToString() { + return pb::JsonFormatter.ToDiagnosticString(this); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void WriteTo(pb::CodedOutputStream output) { + if (Cause.Length != 0) { + output.WriteRawTag(10); + output.WriteBytes(Cause); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public int CalculateSize() { + int size = 0; + if (Cause.Length != 0) { + size += 1 + pb::CodedOutputStream.ComputeBytesSize(Cause); + } + return size; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(RemoteStreamFailure other) { + if (other == null) { + return; + } + if (other.Cause.Length != 0) { + Cause = other.Cause; + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(pb::CodedInputStream input) { + uint tag; + while ((tag = input.ReadTag()) != 0) { + switch(tag) { + default: + input.SkipLastField(); + break; + case 10: { + Cause = input.ReadBytes(); + break; + } + } + } + } + + } + + internal sealed partial class RemoteStreamCompleted : pb::IMessage { + private static readonly pb::MessageParser _parser = new pb::MessageParser(() => new RemoteStreamCompleted()); + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pb::MessageParser Parser { get { return _parser; } } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public static pbr::MessageDescriptor Descriptor { + get { return global::Akka.Streams.Serialization.Proto.Msg.StreamRefMessagesReflection.Descriptor.MessageTypes[9]; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + pbr::MessageDescriptor pb::IMessage.Descriptor { + get { return Descriptor; } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public RemoteStreamCompleted() { + OnConstruction(); + } + + partial void OnConstruction(); + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public RemoteStreamCompleted(RemoteStreamCompleted other) : this() { + seqNr_ = other.seqNr_; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public RemoteStreamCompleted Clone() { + return new RemoteStreamCompleted(this); + } + + /// Field number for the "seqNr" field. + public const int SeqNrFieldNumber = 1; + private long seqNr_; + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public long SeqNr { + get { return seqNr_; } + set { + seqNr_ = value; + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override bool Equals(object other) { + return Equals(other as RemoteStreamCompleted); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public bool Equals(RemoteStreamCompleted other) { + if (ReferenceEquals(other, null)) { + return false; + } + if (ReferenceEquals(other, this)) { + return true; + } + if (SeqNr != other.SeqNr) return false; + return true; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override int GetHashCode() { + int hash = 1; + if (SeqNr != 0L) hash ^= SeqNr.GetHashCode(); + return hash; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public override string ToString() { + return pb::JsonFormatter.ToDiagnosticString(this); + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void WriteTo(pb::CodedOutputStream output) { + if (SeqNr != 0L) { + output.WriteRawTag(8); + output.WriteInt64(SeqNr); + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public int CalculateSize() { + int size = 0; + if (SeqNr != 0L) { + size += 1 + pb::CodedOutputStream.ComputeInt64Size(SeqNr); + } + return size; + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(RemoteStreamCompleted other) { + if (other == null) { + return; + } + if (other.SeqNr != 0L) { + SeqNr = other.SeqNr; + } + } + + [global::System.Diagnostics.DebuggerNonUserCodeAttribute] + public void MergeFrom(pb::CodedInputStream input) { + uint tag; + while ((tag = input.ReadTag()) != 0) { + switch(tag) { + default: + input.SkipLastField(); + break; + case 8: { + SeqNr = input.ReadInt64(); + break; + } + } + } + } + + } + + #endregion + +} + +#endregion Designer generated code diff --git a/src/core/Akka.Streams/Serialization/StreamRefSerializer.cs b/src/core/Akka.Streams/Serialization/StreamRefSerializer.cs new file mode 100644 index 00000000000..14196a3188e --- /dev/null +++ b/src/core/Akka.Streams/Serialization/StreamRefSerializer.cs @@ -0,0 +1,235 @@ +//----------------------------------------------------------------------- +// +// Copyright (C) 2009-2018 Lightbend Inc. +// Copyright (C) 2013-2018 .NET Foundation +// +//----------------------------------------------------------------------- + +using System; +using System.Text; +using Akka.Actor; +using Akka.Serialization; +using Akka.Streams.Implementation; +using Akka.Streams.Serialization.Proto.Msg; +using Akka.Util; +using Google.Protobuf; +using Akka.Streams.Dsl; +using CumulativeDemand = Akka.Streams.Dsl.CumulativeDemand; +using OnSubscribeHandshake = Akka.Streams.Dsl.OnSubscribeHandshake; +using RemoteStreamCompleted = Akka.Streams.Dsl.RemoteStreamCompleted; +using RemoteStreamFailure = Akka.Streams.Dsl.RemoteStreamFailure; +using SequencedOnNext = Akka.Streams.Dsl.SequencedOnNext; + +namespace Akka.Streams.Serialization +{ + public sealed class StreamRefSerializer : SerializerWithStringManifest + { + private readonly ExtendedActorSystem _system; + private readonly Akka.Serialization.Serialization _serialization; + + private const string SequencedOnNextManifest = "A"; + private const string CumulativeDemandManifest = "B"; + private const string RemoteSinkFailureManifest = "C"; + private const string RemoteSinkCompletedManifest = "D"; + private const string SourceRefManifest = "E"; + private const string SinkRefManifest = "F"; + private const string OnSubscribeHandshakeManifest = "G"; + + public StreamRefSerializer(ExtendedActorSystem system) : base(system) + { + _system = system; + _serialization = system.Serialization; + } + + public override string Manifest(object o) + { + switch (o) + { + case SequencedOnNext _: return SequencedOnNextManifest; + case CumulativeDemand _: return CumulativeDemandManifest; + case OnSubscribeHandshake _: return OnSubscribeHandshakeManifest; + case RemoteStreamFailure _: return RemoteSinkFailureManifest; + case RemoteStreamCompleted _: return RemoteSinkCompletedManifest; + case SourceRefImpl _: return SourceRefManifest; + case SinkRefImpl _: return SinkRefManifest; + default: throw new ArgumentException($"Unsupported object of type {o.GetType()}", nameof(o)); + } + } + + public override byte[] ToBinary(object o) + { + switch (o) + { + case SequencedOnNext onNext: return SerializeSequencedOnNext(onNext).ToByteArray(); + case CumulativeDemand demand: return SerializeCumulativeDemand(demand).ToByteArray(); + case OnSubscribeHandshake handshake: return SerializeOnSubscribeHandshake(handshake).ToByteArray(); + case RemoteStreamFailure failure: return SerializeRemoteStreamFailure(failure).ToByteArray(); + case RemoteStreamCompleted completed: return SerializeRemoteStreamCompleted(completed).ToByteArray(); + case SourceRefImpl sourceRef: return SerializeSourceRef(sourceRef).ToByteArray(); + case SinkRefImpl sinkRef: return SerializeSinkRef(sinkRef).ToByteArray(); + default: throw new ArgumentException($"Unsupported object of type {o.GetType()}", nameof(o)); + } + } + + public override object FromBinary(byte[] bytes, string manifest) + { + switch (manifest) + { + case SequencedOnNextManifest: return DeserializeSequenceOnNext(bytes); + case CumulativeDemandManifest: return DeserializeCumulativeDemand(bytes); + case OnSubscribeHandshakeManifest: return DeserializeOnSubscribeHandshake(bytes); + case RemoteSinkFailureManifest: return DeserializeRemoteSinkFailure(bytes); + case RemoteSinkCompletedManifest: return DeserializeRemoteSinkCompleted(bytes); + case SourceRefManifest: return DeserializeSourceRef(bytes); + case SinkRefManifest: return DeserializeSinkRef(bytes); + default: throw new ArgumentException($"Unsupported manifest '{manifest}'", nameof(manifest)); + } + } + + private Type TypeFromProto(Proto.Msg.EventType eventType) + { + var typeName = eventType.TypeName; + return Type.GetType(typeName, throwOnError: true); + } + + private Proto.Msg.EventType TypeToProto(Type clrType) => new Proto.Msg.EventType + { + TypeName = clrType.TypeQualifiedName() + }; + + private SinkRefImpl DeserializeSinkRef(byte[] bytes) + { + var sinkRef = SinkRef.Parser.ParseFrom(bytes); + var type = TypeFromProto(sinkRef.EventType); + var targetRef = _system.Provider.ResolveActorRef(sinkRef.TargetRef.Path); + return SinkRefImpl.Create(type, targetRef); + } + + private SourceRefImpl DeserializeSourceRef(byte[] bytes) + { + var sourceRef = SourceRef.Parser.ParseFrom(bytes); + var type = TypeFromProto(sourceRef.EventType); + var originRef = _system.Provider.ResolveActorRef(sourceRef.OriginRef.Path); + return SourceRefImpl.Create(type, originRef); + } + + private RemoteStreamCompleted DeserializeRemoteSinkCompleted(byte[] bytes) + { + var completed = Proto.Msg.RemoteStreamCompleted.Parser.ParseFrom(bytes); + return new RemoteStreamCompleted(completed.SeqNr); + } + + private RemoteStreamFailure DeserializeRemoteSinkFailure(byte[] bytes) + { + var failure = Proto.Msg.RemoteStreamFailure.Parser.ParseFrom(bytes); + var errorMessage = Encoding.UTF8.GetString(failure.Cause.ToByteArray()); + return new RemoteStreamFailure(errorMessage); + } + + private OnSubscribeHandshake DeserializeOnSubscribeHandshake(byte[] bytes) + { + var handshake = Proto.Msg.OnSubscribeHandshake.Parser.ParseFrom(bytes); + var targetRef = _system.Provider.ResolveActorRef(handshake.TargetRef.Path); + return new OnSubscribeHandshake(targetRef); + } + + private CumulativeDemand DeserializeCumulativeDemand(byte[] bytes) + { + var demand = Proto.Msg.CumulativeDemand.Parser.ParseFrom(bytes); + return new CumulativeDemand(demand.SeqNr); + } + + private SequencedOnNext DeserializeSequenceOnNext(byte[] bytes) + { + var onNext = Proto.Msg.SequencedOnNext.Parser.ParseFrom(bytes); + var serializer = _serialization.GetSerializerById(onNext.Payload.SerializerId); + object payload; + if (onNext.Payload.MessageManifest != null) + { + var manifest = Encoding.UTF8.GetString(onNext.Payload.MessageManifest.ToByteArray()); + if (serializer is SerializerWithStringManifest s) + { + payload = s.FromBinary(onNext.Payload.EnclosedMessage.ToByteArray(), manifest); + } + else + { + var type = Type.GetType(manifest, throwOnError: true); + payload = serializer.FromBinary(onNext.Payload.EnclosedMessage.ToByteArray(), type); + } + } + else + { + payload = serializer.FromBinary(onNext.Payload.EnclosedMessage.ToByteArray(), null); + } + + return new SequencedOnNext(onNext.SeqNr, payload); + } + + private ByteString SerializeSinkRef(SinkRefImpl sinkRef) => new SinkRef + { + EventType = TypeToProto(sinkRef.EventType), + TargetRef = new ActorRef + { + Path = Akka.Serialization.Serialization.SerializedActorPath(sinkRef.InitialPartnerRef) + } + }.ToByteString(); + + private ByteString SerializeSourceRef(SourceRefImpl sourceRef) + { + return new SourceRef + { + EventType = TypeToProto(sourceRef.EventType), + OriginRef = new ActorRef + { + Path = Akka.Serialization.Serialization.SerializedActorPath(sourceRef.InitialPartnerRef) + } + }.ToByteString(); + } + + private ByteString SerializeRemoteStreamCompleted(RemoteStreamCompleted completed) => + new Proto.Msg.RemoteStreamCompleted { SeqNr = completed.SeqNr }.ToByteString(); + + private ByteString SerializeRemoteStreamFailure(RemoteStreamFailure failure) => new Proto.Msg.RemoteStreamFailure + { + Cause = ByteString.CopyFromUtf8(failure.Message) + }.ToByteString(); + + private ByteString SerializeOnSubscribeHandshake(OnSubscribeHandshake handshake) => + new Proto.Msg.OnSubscribeHandshake + { + TargetRef = new ActorRef + { Path = Akka.Serialization.Serialization.SerializedActorPath(handshake.TargetRef) } + }.ToByteString(); + + private ByteString SerializeCumulativeDemand(CumulativeDemand demand) => + new Proto.Msg.CumulativeDemand { SeqNr = demand.SeqNr }.ToByteString(); + + private ByteString SerializeSequencedOnNext(SequencedOnNext onNext) + { + var payload = onNext.Payload; + var serializer = _serialization.FindSerializerFor(payload); + string manifest = null; + if (serializer.IncludeManifest) + { + manifest = serializer is SerializerWithStringManifest s + ? s.Manifest(payload) + : payload.GetType().TypeQualifiedName(); + } + + var p = new Payload + { + EnclosedMessage = ByteString.CopyFrom(serializer.ToBinary(payload)), + SerializerId = serializer.Identifier + }; + + if (!string.IsNullOrEmpty(manifest)) + p.MessageManifest = ByteString.CopyFromUtf8(manifest); + + return new Proto.Msg.SequencedOnNext + { + SeqNr = onNext.SeqNr, + Payload = p + }.ToByteString(); + } + } +} \ No newline at end of file diff --git a/src/core/Akka.Streams/Stage/GraphStage.cs b/src/core/Akka.Streams/Stage/GraphStage.cs index 08613227e97..55c091d9b50 100644 --- a/src/core/Akka.Streams/Stage/GraphStage.cs +++ b/src/core/Akka.Streams/Stage/GraphStage.cs @@ -859,18 +859,18 @@ protected GraphStageLogic(Shape shape) : this(shape.Inlets.Count(), shape.Outlet /// public virtual bool KeepGoingAfterAllPortsClosed => false; - private StageActorRef _stageActorRef; + private StageActor _stageActor; /// /// TBD /// - public StageActorRef StageActorRef + public StageActor StageActor { get { - if (_stageActorRef == null) + if (_stageActor == null) throw StageActorRefNotInitializedException.Instance; - return _stageActorRef; + return _stageActor; } } @@ -1619,21 +1619,35 @@ protected Action GetAsyncCallback(Action handler) /// Callback that will be called upon receiving of a message by this special Actor /// Minimal actor with watch method [ApiMayChange] - protected StageActorRef GetStageActorRef(StageActorRef.Receive receive) + protected StageActor GetStageActor(StageActorRef.Receive receive) { - if (_stageActorRef == null) + if (_stageActor == null) { var actorMaterializer = ActorMaterializerHelper.Downcast(Interpreter.Materializer); - var provider = ((IInternalActorRef)actorMaterializer.Supervisor).Provider; - var path = actorMaterializer.Supervisor.Path / StageActorRef.Name.Next(); - _stageActorRef = new StageActorRef(provider, actorMaterializer.Logger, r => GetAsyncCallback>(tuple => r(tuple)), receive, path); + _stageActor = new StageActor( + actorMaterializer, + r => GetAsyncCallback>(message => r(message)), + receive, + StageActorName); } else - _stageActorRef.Become(receive); + _stageActor.Become(receive); - return _stageActorRef; + return _stageActor; } + /// + /// Override and return a name to be given to the StageActor of this stage. + /// + /// This method will be only invoked and used once, during the first + /// invocation whichc reates the actor, since subsequent `getStageActors` calls function + /// like `become`, rather than creating new actors. + /// + /// Returns an empty string by default, which means that the name will a unique generated String (e.g. "$$a"). + /// + [ApiMayChange] + protected virtual string StageActorName => ""; + /// /// TBD /// @@ -1644,10 +1658,10 @@ protected internal virtual void BeforePreStart() { } /// protected internal virtual void AfterPostStop() { - if (_stageActorRef != null) + if (_stageActor != null) { - _stageActorRef.Stop(); - _stageActorRef = null; + _stageActor.Stop(); + _stageActor = null; } } @@ -2349,225 +2363,97 @@ public override void OnDownstreamFinish() } } + public static class StageActorRef + { + public delegate void Receive(Tuple args); + } + /// /// Minimal actor to work with other actors and watch them in a synchronous ways. /// - public sealed class StageActorRef : MinimalActorRef + public sealed class StageActor { - /// - /// TBD - /// - /// TBD - public delegate void Receive(Tuple args); - - /// - /// TBD - /// - public readonly IImmutableSet StageTerminatedTombstone = null; - - /// - /// TBD - /// - public static readonly EnumerableActorName Name = new EnumerableActorNameImpl("StageActorRef", new AtomicCounterLong(0L)); - - /// - /// TBD - /// - public readonly ILoggingAdapter Log; private readonly Action> _callback; - private readonly AtomicReference> _watchedBy = new AtomicReference>(ImmutableHashSet.Empty); - - private volatile Receive _behavior; - private IImmutableSet _watching = ImmutableHashSet.Empty; - - /// - /// TBD - /// - /// TBD - /// TBD - /// TBD - /// TBD - /// TBD - /// TBD - public StageActorRef(IActorRefProvider provider, ILoggingAdapter log, Func>> getAsyncCallback, Receive initialReceive, ActorPath path) + private readonly ActorCell _cell; + private readonly FunctionRef _functionRef; + private StageActorRef.Receive _behavior; + + public StageActor( + ActorMaterializer materializer, + Func>> getAsyncCallback, + StageActorRef.Receive initialReceive, + string name = null) { - Log = log; - Provider = provider; + _callback = getAsyncCallback(InternalReceive); _behavior = initialReceive; - Path = path; - - _callback = getAsyncCallback(args => _behavior(args)); - } - - /// - /// TBD - /// - public override ActorPath Path { get; } - /// - /// TBD - /// - public override IActorRefProvider Provider { get; } - - /// - /// TBD - /// - public override bool IsTerminated => _watchedBy.Value == StageTerminatedTombstone; + switch (materializer.Supervisor) + { + case LocalActorRef r: _cell = r.Cell; break; + case RepointableActorRef r: _cell = (ActorCell)r.Underlying; break; + default: throw new IllegalStateException($"Stream supervisor must be a local actor, was [{materializer.Supervisor.GetType()}]"); + } - private void LogIgnored(object message) => Log.Warning($"{message} message sent to StageActorRef({Path}) will be ignored, since it is not a real Actor. Use a custom message type to communicate with it instead."); - /// - /// TBD - /// - /// TBD - /// TBD - protected override void TellInternal(object message, IActorRef sender) - { - switch (message) + _functionRef = _cell.AddFunctionRef((sender, message) => { - case PoisonPill _: - case Kill _: - LogIgnored(message); - return; - case Terminated t: - if (_watching.Contains(t.ActorRef)) - { - _watching.Remove(t.ActorRef); - _callback(Tuple.Create(sender, message)); + switch (message) + { + case PoisonPill _: + case Kill _: + materializer.Logger.Warning("{0} message sent to StageActor({1}) will be ignored, since it is not a real Actor. " + + "Use a custom message type to communicate with it instead.", message, _functionRef.Path); break; - } - else return; - default: - _callback(Tuple.Create(sender, message)); - break; - } + default: _callback(Tuple.Create(sender, message)); break; + } + }); } /// - /// TBD + /// The by which this can be contacted from the outside. + /// This is a full-fledged that supports watching and being watched + /// as well as location transparent (remote) communication. /// - /// TBD - public override void SendSystemMessage(ISystemMessage message) - { - if (message is DeathWatchNotification death) - Tell(new Terminated(death.Actor, true, false), ActorRefs.NoSender); - else if (message is Watch w) - AddWatcher(w.Watchee, w.Watcher); - else if (message is Unwatch u) - RemoveWatcher(u.Watchee, u.Watcher); - } + public IActorRef Ref => _functionRef; /// - /// TBD + /// Special `Become` allowing to swap the behaviour of this . + /// Unbecome is not available. /// - /// TBD - public void Become(Receive behavior) => _behavior = behavior; - - private void SendTerminated() - { - var watchedBy = _watchedBy.GetAndSet(StageTerminatedTombstone); - if (watchedBy != StageTerminatedTombstone) - { - foreach (var actorRef in watchedBy.Cast()) - SendTerminated(actorRef); - - foreach (var actorRef in _watching.Cast()) - UnwatchWatched(actorRef); - - _watching = ImmutableHashSet.Empty; - } - } + public void Become(StageActorRef.Receive receive) => Volatile.Write(ref _behavior, receive); /// - /// TBD + /// Stops current . /// - /// TBD - public void Watch(IActorRef actorRef) - { - var iw = (IInternalActorRef) actorRef; - _watching = _watching.Add(actorRef); - iw.SendSystemMessage(new Watch(iw, this)); - } + public void Stop() => _cell.RemoveFunctionRef(_functionRef); /// - /// TBD + /// Makes current watch over given . + /// It will be notified when an underlying actor is . /// - /// TBD - public void Unwatch(IActorRef actorRef) - { - var iw = (IInternalActorRef)actorRef; - _watching = _watching.Remove(actorRef); - iw.SendSystemMessage(new Unwatch(iw, this)); - } - + /// + public void Watch(IActorRef actorRef) => _functionRef.Watch(actorRef); + /// - /// TBD + /// Makes current stop watching previously ed . + /// If was not watched over, this method has no result. /// - public override void Stop() => SendTerminated(); - - private void SendTerminated(IInternalActorRef actorRef) - => actorRef.SendSystemMessage(new DeathWatchNotification(this, true, false)); - - private void UnwatchWatched(IInternalActorRef actorRef) => actorRef.SendSystemMessage(new Unwatch(actorRef, this)); + /// + public void Unwatch(IActorRef actorRef) => _functionRef.Unwatch(actorRef); - private void AddWatcher(IInternalActorRef watchee, IInternalActorRef watcher) + internal void InternalReceive(Tuple pack) { - while (true) + if (pack.Item2 is Terminated terminated) { - var watchedBy = _watchedBy.Value; - if (watchedBy == StageTerminatedTombstone) - SendTerminated(watcher); - else + if (_functionRef.IsWatching(terminated.ActorRef)) { - var isWatcheeSelf = Equals(watchee, this); - var isWatcherSelf = Equals(watcher, this); - - if (isWatcheeSelf && !isWatcherSelf) - { - if (!watchedBy.Contains(watcher)) - if (!_watchedBy.CompareAndSet(watchedBy, watchedBy.Add(watcher))) - continue; // try again - } - else if (!isWatcheeSelf && isWatcherSelf) - Log.Warning("externally triggered watch from {0} to {1} is illegal on StageActorRef", - watcher, watchee); - else - Log.Error("BUG: illegal Watch({0}, {1}) for {2}", watchee, watcher, this); + _functionRef.Unwatch(terminated.ActorRef); + _behavior(pack); } - - break; } + else _behavior(pack); } - - private void RemoveWatcher(IInternalActorRef watchee, IInternalActorRef watcher) - { - while (true) - { - var watchedBy = _watchedBy.Value; - if (watchedBy == null) - SendTerminated(watcher); - else - { - var isWatcheeSelf = Equals(watchee, this); - var isWatcherSelf = Equals(watcher, this); - - if (isWatcheeSelf && !isWatcherSelf) - { - if (!watchedBy.Contains(watcher)) - if (!_watchedBy.CompareAndSet(watchedBy, watchedBy.Remove(watcher))) - continue; // try again - } - else if (!isWatcheeSelf && isWatcherSelf) - Log.Warning("externally triggered unwatch from {0} to {1} is illegal on StageActorRef", - watcher, watchee); - else - Log.Error("BUG: illegal Watch({0}, {1}) for {2}", watchee, watcher, this); - } - - break; - } - } - } - + } + /// /// /// This class wraps callback for instances and gracefully handles diff --git a/src/core/Akka.Streams/StreamRefs.cs b/src/core/Akka.Streams/StreamRefs.cs new file mode 100644 index 00000000000..5ad80c7d7a1 --- /dev/null +++ b/src/core/Akka.Streams/StreamRefs.cs @@ -0,0 +1,112 @@ +//----------------------------------------------------------------------- +// +// Copyright (C) 2015-2016 Lightbend Inc. +// Copyright (C) 2013-2016 Akka.NET project +// +//----------------------------------------------------------------------- + +using System; +using Akka.Actor; +using Akka.Pattern; +using Akka.Streams.Dsl; +using Akka.Streams.Implementation; + +namespace Akka.Streams +{ + /// + /// A allows sharing a "reference" to a with others, + /// with the main purpose of crossing a network boundary. Usually obtaining a SinkRef would be done via Actor messaging, + /// in which one system asks a remote one, to accept some data from it, and the remote one decides to accept the + /// request to send data in a back-pressured streaming fashion -- using a sink ref. + /// + /// To create a you have to materialize the that you want to obtain + /// a reference to by attaching it to a . + /// + /// Stream refs can be seen as Reactive Streams over network boundaries. + /// + /// For additional configuration see `reference.conf` as well as . + /// + /// + public interface ISinkRef + { + Sink Sink { get; } + } + + /// + /// A SourceRef allows sharing a "reference" with others, with the main purpose of crossing a network boundary. + /// Usually obtaining a SourceRef would be done via Actor messaging, in which one system asks a remote one, + /// to share some data with it, and the remote one decides to do so in a back-pressured streaming fashion -- using a stream ref. + /// + /// To create a you have to materialize the that you want to + /// obtain a reference to by attaching it to a . + /// + /// Stream refs can be seen as Reactive Streams over network boundaries. + /// + /// For additional configuration see `reference.conf` as well as . + /// + /// + public interface ISourceRef + { + Source Source { get; } + } + + public sealed class TargetRefNotInitializedYetException : IllegalStateException + { + public TargetRefNotInitializedYetException() : base( + "Internal remote target actor ref not yet resolved, yet attempted to send messages to it. " + + "This should not happen due to proper flow-control, please open a ticket on the issue tracker: https://github.com/akkadotnet/akka.net") + { + } + } + + public sealed class StreamRefSubscriptionTimeoutException : IllegalStateException + { + public StreamRefSubscriptionTimeoutException(string message) : base(message) + { + } + } + + public sealed class RemoteStreamRefActorTerminatedException : Exception + { + public RemoteStreamRefActorTerminatedException(string message) : base(message) + { + } + } + + public sealed class InvalidSequenceNumberException : IllegalStateException + { + public long ExpectedSeqNr { get; } + public long GotSeqNr { get; } + + public InvalidSequenceNumberException(long expectedSeqNr, long gotSeqNr, string message) : base( + $"{message} (expected: {expectedSeqNr}, got: {gotSeqNr}). In most cases this means that message loss on this connection has occurred and the stream will fail eagerly.") + { + ExpectedSeqNr = expectedSeqNr; + GotSeqNr = gotSeqNr; + } + } + + /// + /// Stream refs establish a connection between a local and remote actor, representing the origin and remote sides + /// of a stream. Each such actor refers to the other side as its "partner". We make sure that no other actor than + /// the initial partner can send demand/messages to the other side accidentally. + /// + /// This exception is thrown when a message is recived from a non-partner actor, + /// which could mean a bug or some actively malicient behavior from the other side. + /// + /// This is not meant as a security feature, but rather as plain sanity-check. + /// + public sealed class InvalidPartnerActorException : IllegalStateException + { + public IActorRef ExpectedRef { get; } + public IActorRef GotRef { get; } + + public InvalidPartnerActorException(IActorRef expectedRef, IActorRef gotRef, string message) : base( + $"{message} (expected: {expectedRef}, got: {gotRef}). "+ + "This may happen due to 'double-materialization' on the other side of this stream ref. " + + "Do note that stream refs are one-shot references and have to be paired up in 1:1 pairs. " + + "Multi-cast such as broadcast etc can be implemented by sharing multiple new stream references. ") + { + } + } +} \ No newline at end of file diff --git a/src/core/Akka.Streams/reference.conf b/src/core/Akka.Streams/reference.conf index f6815aa6b88..3c48cf4183e 100644 --- a/src/core/Akka.Streams/reference.conf +++ b/src/core/Akka.Streams/reference.conf @@ -83,6 +83,30 @@ akka { # Note: If you change this value also change the fallback value in ActorMaterializerSettings fuzzing-mode = off } + + stream-ref { + # Buffer of a SinkRef that is used to batch Request elements from the other side of the stream ref + # + # The buffer will be attempted to be filled eagerly even while the local stage did not request elements, + # because the delay of requesting over network boundaries is much higher. + buffer-capacity = 32 + + # Demand is signalled by sending a cumulative demand message ("requesting messages until the n-th sequence number) + # Using a cumulative demand model allows us to re-deliver the demand message in case of message loss (which should + # be very rare in any case, yet possible -- mostly under connection break-down and re-establishment). + # + # The semantics of handling and updating the demand however are in-line with what Reactive Streams dictates. + # + # In normal operation, demand is signalled in response to arriving elements, however if no new elements arrive + # within `demand-redelivery-interval` a re-delivery of the demand will be triggered, assuming that it may have gotten lost. + demand-redelivery-interval = 1 second + + # Subscription timeout, during which the "remote side" MUST subscribe (materialize) the handed out stream ref. + # This timeout does not have to be very low in normal situations, since the remote side may also need to + # prepare things before it is ready to materialize the reference. However the timeout is needed to avoid leaking + # in-active streams which are never subscribed to. + subscription-timeout = 30 seconds + } } # Fully qualified config path which holds the dispatcher configuration @@ -102,9 +126,26 @@ akka { } } } - + # configure overrides to ssl-configuration here (to be used by akka-streams, and akka-http – i.e. when serving https connections) ssl-config { protocol = "TLSv1" } + + actor { + + serializers { + akka-stream-ref = "Akka.Streams.Serialization.StreamRefSerializer, Akka.Streams" + } + + serialization-bindings { + "Akka.Streams.Dsl.SinkRefImpl, Akka.Streams" = akka-stream-ref + "Akka.Streams.Dsl.SourceRefImpl, Akka.Streams" = akka-stream-ref + "Akka.Streams.Dsl.IStreamRefsProtocol, Akka.Streams" = akka-stream-ref + } + + serialization-identifiers { + "Akka.Streams.Serialization.StreamRefSerializer, Akka.Streams" = 30 + } + } } diff --git a/src/core/Akka.Tests/Actor/FunctionRefSpec.cs b/src/core/Akka.Tests/Actor/FunctionRefSpec.cs new file mode 100644 index 00000000000..f860a0b7f78 --- /dev/null +++ b/src/core/Akka.Tests/Actor/FunctionRefSpec.cs @@ -0,0 +1,253 @@ +//----------------------------------------------------------------------- +// +// Copyright (C) 2009-2018 Lightbend Inc. +// Copyright (C) 2018 Akka.NET project +// +//----------------------------------------------------------------------- + +using System; +using Akka.Actor; +using Akka.Configuration; +using Akka.TestKit; +using Xunit; +using Xunit.Abstractions; + +namespace Akka.Tests.Actor +{ + public class FunctionRefSpec : AkkaSpec + { + #region internal classes + + sealed class GetForwarder : IEquatable + { + public IActorRef ReplyTo { get; } + + public GetForwarder(IActorRef replyTo) + { + ReplyTo = replyTo; + } + + public bool Equals(GetForwarder other) + { + if (ReferenceEquals(null, other)) return false; + if (ReferenceEquals(this, other)) return true; + return Equals(ReplyTo, other.ReplyTo); + } + + public override bool Equals(object obj) => obj is GetForwarder forwarder && Equals(forwarder); + + public override int GetHashCode() => (ReplyTo != null ? ReplyTo.GetHashCode() : 0); + } + + sealed class DropForwarder : IEquatable + { + public FunctionRef Ref { get; } + + public DropForwarder(FunctionRef @ref) + { + Ref = @ref; + } + + public bool Equals(DropForwarder other) + { + if (ReferenceEquals(null, other)) return false; + if (ReferenceEquals(this, other)) return true; + return Equals(Ref, other.Ref); + } + + public override bool Equals(object obj) => obj is DropForwarder forwarder && Equals(forwarder); + + public override int GetHashCode() => (Ref != null ? Ref.GetHashCode() : 0); + } + + sealed class Forwarded : IEquatable + { + public object Message { get; } + public IActorRef Sender { get; } + + public Forwarded(object message, IActorRef sender) + { + Message = message; + Sender = sender; + } + + public bool Equals(Forwarded other) + { + if (ReferenceEquals(null, other)) return false; + if (ReferenceEquals(this, other)) return true; + return Equals(Message, other.Message) && Equals(Sender, other.Sender); + } + + public override bool Equals(object obj) => obj is Forwarded forwarded && Equals(forwarded); + + public override int GetHashCode() + { + unchecked + { + return ((Message != null ? Message.GetHashCode() : 0) * 397) ^ (Sender != null ? Sender.GetHashCode() : 0); + } + } + } + + sealed class Super : ReceiveActor + { + public Super() + { + Receive(get => + { + var cell = (ActorCell)Context; + var fref = cell.AddFunctionRef((sender, msg) => + { + get.ReplyTo.Tell(new Forwarded(msg, sender)); + }); + get.ReplyTo.Tell(fref); + }); + Receive(drop => { + var cell = (ActorCell)Context; + cell.RemoveFunctionRef(drop.Ref); + }); + } + } + + sealed class SupSuper : ReceiveActor + { + public SupSuper() + { + var s = Context.ActorOf(Props.Create(), "super"); + ReceiveAny(msg => s.Tell(msg)); + } + } + + #endregion + + public FunctionRefSpec(ITestOutputHelper output) : base(output, null) + { + } + + #region top level + + [Fact] + public void FunctionRef_created_by_top_level_actor_must_forward_messages() + { + var s = SuperActor(); + var forwarder = GetFunctionRef(s); + + forwarder.Tell("hello"); + ExpectMsg(new Forwarded("hello", TestActor)); + } + + [Fact] + public void FunctionRef_created_by_top_level_actor_must_be_watchable() + { + var s = SuperActor(); + var forwarder = GetFunctionRef(s); + + s.Tell(new GetForwarder(TestActor)); + var f = ExpectMsg(); + Watch(f); + s.Tell(new DropForwarder(f)); + ExpectTerminated(f); + } + + [Fact] + public void FunctionRef_created_by_top_level_actor_must_be_able_to_watch() + { + var s = SuperActor(); + var forwarder = GetFunctionRef(s); + + s.Tell(new GetForwarder(TestActor)); + var f = ExpectMsg(); + forwarder.Watch(f); + s.Tell(new DropForwarder(f)); + ExpectMsg(new Forwarded(new Terminated(f, true, false), f)); + } + + [Fact] + public void FunctionRef_created_by_top_level_actor_must_terminate_when_their_parent_terminates() + { + var s = SuperActor(); + var forwarder = GetFunctionRef(s); + + Watch(forwarder); + s.Tell(PoisonPill.Instance); + ExpectTerminated(forwarder); + } + + private FunctionRef GetFunctionRef(IActorRef s) + { + s.Tell(new GetForwarder(TestActor)); + return ExpectMsg(); + } + + private IActorRef SuperActor() => Sys.ActorOf(Props.Create(), "super"); + + #endregion + + #region non-top level + + [Fact] + public void FunctionRef_created_by_non_top_level_actor_must_forward_messages() + { + var s = SupSuperActor(); + var forwarder = GetFunctionRef(s); + + forwarder.Tell("hello"); + ExpectMsg(new Forwarded("hello", TestActor)); + } + + [Fact] + public void FunctionRef_created_by_non_top_level_actor_must_be_watchable() + { + var s = SupSuperActor(); + var forwarder = GetFunctionRef(s); + + s.Tell(new GetForwarder(TestActor)); + var f = ExpectMsg(); + Watch(f); + s.Tell(new DropForwarder(f)); + ExpectTerminated(f); + } + + [Fact] + public void FunctionRef_created_by_non_top_level_actor_must_be_able_to_watch() + { + var s = SupSuperActor(); + var forwarder = GetFunctionRef(s); + + s.Tell(new GetForwarder(TestActor)); + var f = ExpectMsg(); + forwarder.Watch(f); + s.Tell(new DropForwarder(f)); + ExpectMsg(new Forwarded(new Terminated(f, true, false), f)); + } + + [Fact] + public void FunctionRef_created_by_non_top_level_actor_must_terminate_when_their_parent_terminates() + { + var s = SupSuperActor(); + var forwarder = GetFunctionRef(s); + + Watch(forwarder); + s.Tell(PoisonPill.Instance); + ExpectTerminated(forwarder); + } + + private IActorRef SupSuperActor() => Sys.ActorOf(Props.Create(), "supsuper"); + + #endregion + + [Fact(Skip = "FIXME")] + public void FunctionRef_when_not_registered_must_not_be_found() + { + var provider = ((ExtendedActorSystem) Sys).Provider; + var fref = new FunctionRef(TestActor.Path / "blabla", provider, Sys.EventStream, (x, y) => { }); + EventFilter.Exception().ExpectOne(() => + { + // needs to be something that fails when the deserialized form is not a FunctionRef + // this relies upon serialize-messages during tests + TestActor.Tell(new DropForwarder(fref)); + ExpectNoMsg(TimeSpan.FromSeconds(1)); + }); + } + } +} \ No newline at end of file diff --git a/src/core/Akka.Tests/Akka.Tests.csproj b/src/core/Akka.Tests/Akka.Tests.csproj index bd521e79bbf..7a89113d73a 100644 --- a/src/core/Akka.Tests/Akka.Tests.csproj +++ b/src/core/Akka.Tests/Akka.Tests.csproj @@ -1,17 +1,14 @@  - Akka.Tests net452;netcoreapp1.1 - - @@ -21,28 +18,22 @@ - - - - $(DefineConstants);SERIALIZATION;CONFIGURATION;UNSAFE_THREADING - $(DefineConstants);CORECLR - $(DefineConstants);RELEASE diff --git a/src/core/Akka/Actor/ActorCell.Children.cs b/src/core/Akka/Actor/ActorCell.Children.cs index 6682b11ecf1..6ffeeaa8813 100644 --- a/src/core/Akka/Actor/ActorCell.Children.cs +++ b/src/core/Akka/Actor/ActorCell.Children.cs @@ -7,6 +7,8 @@ using System; using System.Collections.Generic; +using System.Collections.Immutable; +using System.Text; using System.Threading; using Akka.Actor.Internal; using Akka.Serialization; @@ -19,18 +21,53 @@ public partial class ActorCell { private volatile IChildrenContainer _childrenContainerDoNotCallMeDirectly = EmptyChildrenContainer.Instance; private long _nextRandomNameDoNotCallMeDirectly = -1; // Interlocked.Increment automatically adds 1 to this value. Allows us to start from 0. + private ImmutableDictionary _functionRefsDoNotCallMeDirectly = ImmutableDictionary.Empty; /// /// The child container collection, used to house information about all child actors. /// - public IChildrenContainer ChildrenContainer + public IChildrenContainer ChildrenContainer => _childrenContainerDoNotCallMeDirectly; + + private IReadOnlyCollection Children => ChildrenContainer.Children; + + private ImmutableDictionary FunctionRefs => Volatile.Read(ref _functionRefsDoNotCallMeDirectly); + + internal bool TryGetFunctionRef(string name, out FunctionRef functionRef) => + FunctionRefs.TryGetValue(name, out functionRef); + + internal bool TryGetFunctionRef(string name, int uid, out FunctionRef functionRef) => + FunctionRefs.TryGetValue(name, out functionRef) && (uid == ActorCell.UndefinedUid || uid == functionRef.Path.Uid); + + internal FunctionRef AddFunctionRef(Action tell, string suffix = "") { - get { return _childrenContainerDoNotCallMeDirectly; } + var r = GetRandomActorName("$$"); + var n = string.IsNullOrEmpty(suffix) ? r : r + "-" + suffix; + var childPath = new ChildActorPath(Self.Path, n, NewUid()); + var functionRef = new FunctionRef(childPath, SystemImpl.Provider, SystemImpl.EventStream, tell); + + return ImmutableInterlocked.GetOrAdd(ref _functionRefsDoNotCallMeDirectly, childPath.Name, functionRef); + } + + internal bool RemoveFunctionRef(FunctionRef functionRef) + { + if (functionRef.Path.Parent != Self.Path) throw new InvalidOperationException($"Trying to remove FunctionRef {functionRef.Path} from wrong ActorCell"); + + var name = functionRef.Path.Name; + if (ImmutableInterlocked.TryRemove(ref _functionRefsDoNotCallMeDirectly, name, out var fref)) + { + fref.Stop(); + return true; + } + else return false; } - private IReadOnlyCollection Children + protected void StopFunctionRefs() { - get { return ChildrenContainer.Children; } + var refs = Interlocked.Exchange(ref _functionRefsDoNotCallMeDirectly, ImmutableDictionary.Empty); + foreach (var pair in refs) + { + pair.Value.Stop(); + } } /// @@ -87,10 +124,11 @@ private IActorRef ActorOf(Props props, string name, bool isAsync, bool isSystemS return MakeChild(props, name, isAsync, isSystemService); } - private string GetRandomActorName() + private string GetRandomActorName(string prefix = "$") { var id = Interlocked.Increment(ref _nextRandomNameDoNotCallMeDirectly); - return "$" + id.Base64Encode(); + var sb = new StringBuilder(prefix); + return id.Base64Encode(sb).ToString(); } /// @@ -331,8 +369,7 @@ protected bool TryGetChildStatsByRef(IActorRef actor, out ChildRestartStats chil [Obsolete("Use TryGetSingleChild [0.7.1]")] public IInternalActorRef GetSingleChild(string name) { - IInternalActorRef child; - return TryGetSingleChild(name, out child) ? child : ActorRefs.Nobody; + return TryGetSingleChild(name, out var child) ? child : ActorRefs.Nobody; } /// @@ -346,18 +383,21 @@ public bool TryGetSingleChild(string name, out IInternalActorRef child) if (name.IndexOf('#') < 0) { // optimization for the non-uid case - ChildRestartStats stats; - if (TryGetChildRestartStatsByName(name, out stats)) + if (TryGetChildRestartStatsByName(name, out var stats)) { child = stats.Child; return true; } + else if (TryGetFunctionRef(name, out var functionRef)) + { + child = functionRef; + return true; + } } else { var nameAndUid = SplitNameAndUid(name); - ChildRestartStats stats; - if (TryGetChildRestartStatsByName(nameAndUid.Name, out stats)) + if (TryGetChildRestartStatsByName(nameAndUid.Name, out var stats)) { var uid = nameAndUid.Uid; if (uid == ActorCell.UndefinedUid || uid == stats.Uid) @@ -366,6 +406,11 @@ public bool TryGetSingleChild(string name, out IInternalActorRef child) return true; } } + else if (TryGetFunctionRef(nameAndUid.Name, nameAndUid.Uid, out var functionRef)) + { + child = functionRef; + return true; + } } child = ActorRefs.Nobody; return false; @@ -378,8 +423,7 @@ public bool TryGetSingleChild(string name, out IInternalActorRef child) /// TBD protected SuspendReason RemoveChildAndGetStateChange(IActorRef child) { - var terminating = ChildrenContainer as TerminatingChildrenContainer; - if (terminating != null) + if (ChildrenContainer is TerminatingChildrenContainer terminating) { var newContainer = UpdateChildrenRefs(c => c.Remove(child)); if (newContainer is TerminatingChildrenContainer) return null; diff --git a/src/core/Akka/Actor/ActorCell.FaultHandling.cs b/src/core/Akka/Actor/ActorCell.FaultHandling.cs index 09a5eb17ef4..a8e185b3b97 100644 --- a/src/core/Akka/Actor/ActorCell.FaultHandling.cs +++ b/src/core/Akka/Actor/ActorCell.FaultHandling.cs @@ -304,21 +304,25 @@ private void FinishTerminate() try { Parent.SendSystemMessage(new DeathWatchNotification(_self, existenceConfirmed: true, addressTerminated: false)); } finally { - try { TellWatchersWeDied(); } + try { StopFunctionRefs(); } finally { - try { UnwatchWatchedActors(a); } // stay here as we expect an emergency stop from HandleInvokeFailure + try { TellWatchersWeDied(); } finally { - if (System.Settings.DebugLifecycle) - Publish(new Debug(_self.Path.ToString(), ActorType, "Stopped")); + try { UnwatchWatchedActors(a); } // stay here as we expect an emergency stop from HandleInvokeFailure + finally + { + if (System.Settings.DebugLifecycle) + Publish(new Debug(_self.Path.ToString(), ActorType, "Stopped")); - ClearActor(a); - ClearActorCell(); + ClearActor(a); + ClearActorCell(); - _actor = null; + _actor = null; - } + } + } } } } diff --git a/src/core/Akka/Actor/ActorRef.cs b/src/core/Akka/Actor/ActorRef.cs index 9f0007f585d..77c01cd7b7b 100644 --- a/src/core/Akka/Actor/ActorRef.cs +++ b/src/core/Akka/Actor/ActorRef.cs @@ -9,6 +9,7 @@ using System.Collections; using System.Collections.Concurrent; using System.Collections.Generic; +using System.Collections.Immutable; using System.Linq; using System.Threading; using System.Threading.Tasks; @@ -841,5 +842,204 @@ IEnumerator IEnumerable.GetEnumerator() } } } + + /// + /// INTERNAL API + /// + /// This kind of ActorRef passes all received messages to the given function for + /// performing a non-blocking side-effect. The intended use is to transform the + /// message before sending to the real target actor. Such references can be created + /// by calling and must be deregistered when no longer + /// needed by calling . FunctionRefs do not count + /// towards the live children of an actor, they do not receive the Terminate command + /// and do not prevent the parent from terminating. FunctionRef is properly + /// registered for remote lookup and ActorSelection. + /// + /// When using the feature you must ensure that upon reception of the + /// Terminated message the watched actorRef is ed. + /// + internal sealed class FunctionRef : MinimalActorRef + { + private readonly EventStream _eventStream; + private readonly Action _tell; + + private ImmutableHashSet _watching = ImmutableHashSet.Empty; + private ImmutableHashSet _watchedBy = ImmutableHashSet.Empty; + + public FunctionRef(ActorPath path, IActorRefProvider provider, EventStream eventStream, Action tell) + { + _eventStream = eventStream; + _tell = tell; + Path = path; + Provider = provider; + } + + public override ActorPath Path { get; } + public override IActorRefProvider Provider { get; } + public override bool IsTerminated => Volatile.Read(ref _watchedBy) == null; + + /// + /// Have this FunctionRef watch the given Actor. This method must not be + /// called concurrently from different threads, it should only be called by + /// its parent Actor. + /// + /// Upon receiving the Terminated message, must be called from a + /// safe context (i.e. normally from the parent Actor). + /// + public void Watch(IActorRef actorRef) + { + _watching = _watching.Add(actorRef); + var internalRef = (IInternalActorRef) actorRef; + internalRef.SendSystemMessage(new Watch(internalRef, this)); + } + + /// + /// Have this FunctionRef unwatch the given Actor. This method must not be + /// called concurrently from different threads, it should only be called by + /// its parent Actor. + /// + public void Unwatch(IActorRef actorRef) + { + _watching = _watching.Remove(actorRef); + var internalRef = (IInternalActorRef) actorRef; + internalRef.SendSystemMessage(new Unwatch(internalRef, this)); + + } + + /// + /// Query whether this FunctionRef is currently watching the given Actor. This + /// method must not be called concurrently from different threads, it should + /// only be called by its parent Actor. + /// + public bool IsWatching(IActorRef actorRef) => _watching.Contains(actorRef); + + protected override void TellInternal(object message, IActorRef sender) => _tell(sender, message); + + public override void SendSystemMessage(ISystemMessage message) + { + switch (message) + { + case Watch watch: + AddWatcher(watch.Watchee, watch.Watcher); + break; + case Unwatch unwatch: + RemoveWatcher(unwatch.Watchee, unwatch.Watcher); + break; + case DeathWatchNotification deathWatch: + this.Tell(new Terminated(deathWatch.Actor, existenceConfirmed: true, addressTerminated: false), deathWatch.Actor); + break; + } + } + + private void SendTerminated() + { + var watchedBy = Interlocked.Exchange(ref _watchedBy, null); + if (watchedBy != null) + { + if (!watchedBy.IsEmpty) + { + foreach (var watcher in watchedBy) + SendTerminated(watcher); + } + + if (!_watching.IsEmpty) + { + foreach (var watched in _watching) + UnwatchWatched(watched); + + _watching = ImmutableHashSet.Empty; + } + } + } + + private void SendTerminated(IActorRef watcher) + { + if (watcher is IInternalActorRef scope) + scope.SendSystemMessage(new DeathWatchNotification(this, existenceConfirmed: true, addressTerminated: false)); + } + + private void UnwatchWatched(IActorRef watched) + { + if (watched is IInternalActorRef internalActorRef) + internalActorRef.SendSystemMessage(new Unwatch(internalActorRef, this)); + } + + public override void Stop() => SendTerminated(); + + private void AddWatcher(IInternalActorRef watchee, IInternalActorRef watcher) + { + while (true) + { + var watchedBy = Volatile.Read(ref _watchedBy); + if (watchedBy == null) + SendTerminated(watcher); + else + { + var watcheeSelf = Equals(watchee, this); + var watcherSelf = Equals(watcher, this); + + if (watcheeSelf && !watcherSelf) + { + if (!watchedBy.Contains(watcher) && !ReferenceEquals(watchedBy, Interlocked.CompareExchange(ref _watchedBy, watchedBy.Add(watcher), watchedBy))) + { + continue; + } + } + else if (!watcheeSelf && watcherSelf) + { + Publish(new Warning(Path.ToString(), typeof(FunctionRef), $"Externally triggered watch from {watcher} to {watchee} is illegal on FunctionRef")); + } + else + { + Publish(new Warning(Path.ToString(), typeof(FunctionRef), $"BUG: illegal Watch({watchee},{watcher}) for {this}")); + } + } + + break; + } + } + + private void RemoveWatcher(IInternalActorRef watchee, IInternalActorRef watcher) + { + while (true) + { + var watchedBy = Volatile.Read(ref _watchedBy); + if (watchedBy == null) + SendTerminated(watcher); + else + { + var watcheeSelf = Equals(watchee, this); + var watcherSelf = Equals(watcher, this); + + if (watcheeSelf && !watcherSelf) + { + if (!watchedBy.Contains(watcher) && !ReferenceEquals(watchedBy, Interlocked.CompareExchange(ref _watchedBy, watchedBy.Remove(watcher), watchedBy))) + { + continue; + } + } + else if (!watcheeSelf && watcherSelf) + { + Publish(new Warning(Path.ToString(), typeof(FunctionRef), $"Externally triggered watch from {watcher} to {watchee} is illegal on FunctionRef")); + } + else + { + Publish(new Warning(Path.ToString(), typeof(FunctionRef), $"BUG: illegal Watch({watchee},{watcher}) for {this}")); + } + } + + break; + } + } + + private void Publish(LogEvent e) + { + try + { + _eventStream.Publish(e); + } + catch (Exception) { } + } + } } diff --git a/src/core/Akka/Actor/IAutoReceivedMessage.cs b/src/core/Akka/Actor/IAutoReceivedMessage.cs index 4759f0f303e..58284fe1587 100644 --- a/src/core/Akka/Actor/IAutoReceivedMessage.cs +++ b/src/core/Akka/Actor/IAutoReceivedMessage.cs @@ -5,6 +5,7 @@ // //----------------------------------------------------------------------- +using System; using Akka.Event; namespace Akka.Actor @@ -21,7 +22,7 @@ public interface IAutoReceivedMessage /// Terminated message can't be forwarded to another actor, since that actor might not be watching the subject. /// Instead, if you need to forward Terminated to another actor you should send the information in your own message. /// - public sealed class Terminated : IAutoReceivedMessage, IPossiblyHarmful, IDeadLetterSuppression, INoSerializationVerificationNeeded + public sealed class Terminated : IAutoReceivedMessage, IPossiblyHarmful, IDeadLetterSuppression, INoSerializationVerificationNeeded, IEquatable { /// /// Initializes a new instance of the class. @@ -59,7 +60,27 @@ public Terminated(IActorRef actorRef, bool existenceConfirmed, bool addressTermi /// A that represents this instance. public override string ToString() { - return $": {ActorRef} - ExistenceConfirmed={ExistenceConfirmed}"; + return $"Terminated(ref: {ActorRef}, existenceConfirmed: {ExistenceConfirmed}, addressTerminated: {AddressTerminated})"; + } + + public bool Equals(Terminated other) + { + if (ReferenceEquals(null, other)) return false; + if (ReferenceEquals(this, other)) return true; + return Equals(ActorRef, other.ActorRef) && AddressTerminated == other.AddressTerminated && ExistenceConfirmed == other.ExistenceConfirmed; + } + + public override bool Equals(object obj) => obj is Terminated terminated && Equals(terminated); + + public override int GetHashCode() + { + unchecked + { + var hashCode = (ActorRef != null ? ActorRef.GetHashCode() : 0); + hashCode = (hashCode * 397) ^ AddressTerminated.GetHashCode(); + hashCode = (hashCode * 397) ^ ExistenceConfirmed.GetHashCode(); + return hashCode; + } } } diff --git a/src/core/Akka/Actor/RepointableActorRef.cs b/src/core/Akka/Actor/RepointableActorRef.cs index 8c5f44bd169..e2f9ab5e48b 100644 --- a/src/core/Akka/Actor/RepointableActorRef.cs +++ b/src/core/Akka/Actor/RepointableActorRef.cs @@ -293,8 +293,7 @@ public override IActorRef GetChild(IEnumerable name) return ActorRefs.Nobody; default: var nameAndUid = ActorCell.SplitNameAndUid(next); - IChildStats stats; - if (Lookup.TryGetChildStatsByName(nameAndUid.Name, out stats)) + if (Lookup.TryGetChildStatsByName(nameAndUid.Name, out var stats)) { var crs = stats as ChildRestartStats; var uid = nameAndUid.Uid; @@ -306,6 +305,10 @@ public override IActorRef GetChild(IEnumerable name) return crs.Child; } } + else if (Lookup is ActorCell cell && cell.TryGetFunctionRef(nameAndUid.Name, nameAndUid.Uid, out var functionRef)) + { + return functionRef; + } return ActorRefs.Nobody; } } diff --git a/src/core/Akka/Util/Base64Encoding.cs b/src/core/Akka/Util/Base64Encoding.cs index 13d7a6d465f..d9e8ff661c4 100644 --- a/src/core/Akka/Util/Base64Encoding.cs +++ b/src/core/Akka/Util/Base64Encoding.cs @@ -24,9 +24,15 @@ public static class Base64Encoding /// /// TBD /// TBD - public static string Base64Encode(this long value) + public static string Base64Encode(this long value) => Base64Encode(value, new StringBuilder()).ToString(); + + /// + /// TBD + /// + /// TBD + /// TBD + public static StringBuilder Base64Encode(this long value, StringBuilder sb) { - var sb = new StringBuilder(); var next = value; do { @@ -34,7 +40,7 @@ public static string Base64Encode(this long value) sb.Append(Base64Chars[index]); next = next >> 6; } while(next != 0); - return sb.ToString(); + return sb; } /// diff --git a/src/protobuf/StreamRefMessages.proto b/src/protobuf/StreamRefMessages.proto new file mode 100644 index 00000000000..69a5194a299 --- /dev/null +++ b/src/protobuf/StreamRefMessages.proto @@ -0,0 +1,58 @@ +/** + * Copyright (C) 2009-2017 Lightbend Inc. + * Copyright (C) 2017-2018 Akka.NET project + */ + +syntax = 'proto3'; +package Akka.Streams.Serialization.Proto.Msg; +option optimize_for = SPEED; + +/************************************************* + StreamRefs (SourceRef / SinkRef) related formats +**************************************************/ + +message EventType { + string typeName = 1; +} + +message SinkRef { + ActorRef targetRef = 1; + EventType eventType = 2; +} + +message SourceRef { + ActorRef originRef = 1; + EventType eventType = 2; +} + +message ActorRef { + string path = 1; +} + +message Payload { + bytes enclosedMessage = 1; + int32 serializerId = 2; + bytes messageManifest = 3; +} + +// stream refs protocol + +message OnSubscribeHandshake { + ActorRef targetRef = 1; +} +message CumulativeDemand { + int64 seqNr = 1; +} + +message SequencedOnNext { + int64 seqNr = 1; + Payload payload = 2; +} + +message RemoteStreamFailure { + bytes cause = 1; +} + +message RemoteStreamCompleted { + int64 seqNr = 1; +} \ No newline at end of file