From 3a6af76325a6a2adfe917c134cb007785eda5717 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Wed, 29 Jul 2020 22:04:38 +0700 Subject: [PATCH 01/23] Rebase to new dev --- .../BatchingSqliteCurrentAllEventsSpec.cs | 37 +++++++++++++++++++ .../Query/SqliteCurrentAllEventsSpec.cs | 2 +- 2 files changed, 38 insertions(+), 1 deletion(-) create mode 100644 src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/BatchingSqliteCurrentAllEventsSpec.cs diff --git a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/BatchingSqliteCurrentAllEventsSpec.cs b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/BatchingSqliteCurrentAllEventsSpec.cs new file mode 100644 index 00000000000..8da551deb13 --- /dev/null +++ b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/BatchingSqliteCurrentAllEventsSpec.cs @@ -0,0 +1,37 @@ +using System; +using System.Collections.Generic; +using System.Text; +using Akka.Configuration; +using Akka.Persistence.Query; +using Akka.Persistence.Query.Sql; +using Akka.Persistence.TCK.Query; +using Akka.Util.Internal; +using Xunit.Abstractions; + +namespace Akka.Persistence.Sqlite.Tests.Query +{ + public class BatchingCurrentSqliteAllEventsSpec : CurrentAllEventsSpec + { + public static readonly AtomicCounter Counter = new AtomicCounter(0); + + public static Config Config(int id) => ConfigurationFactory.ParseString($@" + akka.loglevel = INFO + akka.persistence.journal.plugin = ""akka.persistence.journal.sqlite"" + akka.persistence.journal.sqlite {{ + class = ""Akka.Persistence.Sqlite.Journal.BatchingSqliteJournal, Akka.Persistence.Sqlite"" + plugin-dispatcher = ""akka.actor.default-dispatcher"" + table-name = event_journal + metadata-table-name = journal_metadata + auto-initialize = on + connection-string = ""Filename=file:memdb-journal-eventsbytag-{id}.db;Mode=Memory;Cache=Shared"" + refresh-interval = 1s + }} + akka.test.single-expect-default = 10s") + .WithFallback(SqlReadJournal.DefaultConfiguration()); + + public BatchingCurrentSqliteAllEventsSpec(ITestOutputHelper output) : base(Config(Counter.GetAndIncrement()), nameof(BatchingSqliteAllEventsSpec), output) + { + ReadJournal = Sys.ReadJournalFor(SqlReadJournal.Identifier); + } + } +} diff --git a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteCurrentAllEventsSpec.cs b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteCurrentAllEventsSpec.cs index c48d678dca8..dc3e0db991f 100644 --- a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteCurrentAllEventsSpec.cs +++ b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteCurrentAllEventsSpec.cs @@ -29,7 +29,7 @@ class = ""Akka.Persistence.Sqlite.Journal.SqliteJournal, Akka.Persistence.Sqlite akka.test.single-expect-default = 10s") .WithFallback(SqlReadJournal.DefaultConfiguration()); - public SqliteCurrentAllEventsSpec(ITestOutputHelper output) : base(Config(Counter.GetAndIncrement()), nameof(SqliteCurrentAllEventsSpec), output) + public SqliteCurrentAllEventsSpec(ITestOutputHelper output) : base(Config(Counter.GetAndIncrement()), nameof(SqliteAllEventsSpec), output) { ReadJournal = Sys.ReadJournalFor(SqlReadJournal.Identifier); } From ef43856652d5e6447975f3be9027294da4b42ae4 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Wed, 29 Jul 2020 23:42:24 +0700 Subject: [PATCH 02/23] Re-implement PersistenceIds Query --- .../AllEventsPublisher.cs | 2 +- .../AllPersistenceIdsPublisher.cs | 200 ++++++++++--- .../SqlReadJournal.cs | 31 +- .../Journal/BatchingSqlJournal.cs | 148 +++------- .../Journal/QueryApi.cs | 48 +-- .../Journal/QueryExecutor.cs | 72 +++-- .../Journal/SqlJournal.cs | 146 +++------ .../BatchingSqliteCurrentAllEventsSpec.cs | 37 --- .../Query/SqliteCurrentPersistenceIdsSpec.cs | 5 - .../Query/CurrentPersistenceIdsSpec.cs | 8 +- .../Akka.Streams.TestKit/TestSubscriber.cs | 45 ++- .../DistinctRetainingMultiReaderBufferSpec.cs | 149 ++++++++++ .../ResizableMultiReaderRingBufferSpec.cs | 4 +- .../RetainingMultiReaderBufferSpec.cs | 249 ++++++++++++++++ src/core/Akka.Streams/Dsl/Sink.cs | 7 +- .../Implementation/ActorPublisher.cs | 4 +- .../Implementation/FanoutProcessorImpl.cs | 43 ++- .../ResizableMultiReaderRingBuffer.cs | 174 +++++++++-- src/core/Akka.Streams/Implementation/Sinks.cs | 9 +- .../Implementation/SubscriberManagement.cs | 69 ++--- .../Akka.Streams/Properties/AssemblyInfo.cs | 1 + src/core/Akka.Streams/Util/BitOperations.cs | 279 ++++++++++++++++++ 22 files changed, 1272 insertions(+), 458 deletions(-) delete mode 100644 src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/BatchingSqliteCurrentAllEventsSpec.cs create mode 100644 src/core/Akka.Streams.Tests/Implementation/DistinctRetainingMultiReaderBufferSpec.cs create mode 100644 src/core/Akka.Streams.Tests/Implementation/RetainingMultiReaderBufferSpec.cs create mode 100644 src/core/Akka.Streams/Util/BitOperations.cs diff --git a/src/contrib/persistence/Akka.Persistence.Query.Sql/AllEventsPublisher.cs b/src/contrib/persistence/Akka.Persistence.Query.Sql/AllEventsPublisher.cs index c55d2b9ec09..ea6fd7ce3df 100644 --- a/src/contrib/persistence/Akka.Persistence.Query.Sql/AllEventsPublisher.cs +++ b/src/contrib/persistence/Akka.Persistence.Query.Sql/AllEventsPublisher.cs @@ -141,7 +141,7 @@ internal sealed class LiveAllEventsPublisher : AbstractAllEventsPublisher public LiveAllEventsPublisher(long fromOffset, TimeSpan refreshInterval, int maxBufferSize, string writeJournalPluginId) : base(fromOffset, maxBufferSize, writeJournalPluginId) { - _tickCancelable = Context.System.Scheduler.ScheduleTellRepeatedlyCancelable(refreshInterval, refreshInterval, Self, EventsByTagPublisher.Continue.Instance, Self); + _tickCancelable = Context.System.Scheduler.ScheduleTellRepeatedlyCancelable(refreshInterval, refreshInterval, Self, AllEventsPublisher.Continue.Instance, Self); } protected override long ToOffset => long.MaxValue; diff --git a/src/contrib/persistence/Akka.Persistence.Query.Sql/AllPersistenceIdsPublisher.cs b/src/contrib/persistence/Akka.Persistence.Query.Sql/AllPersistenceIdsPublisher.cs index d4e36826f3b..45e3b18005b 100644 --- a/src/contrib/persistence/Akka.Persistence.Query.Sql/AllPersistenceIdsPublisher.cs +++ b/src/contrib/persistence/Akka.Persistence.Query.Sql/AllPersistenceIdsPublisher.cs @@ -5,64 +5,196 @@ // //----------------------------------------------------------------------- +using System; using Akka.Actor; using Akka.Persistence.Sql.Common.Journal; using Akka.Streams.Actors; namespace Akka.Persistence.Query.Sql { - internal sealed class AllPersistenceIdsPublisher : ActorPublisher + internal sealed class CurrentPersistenceIdsPublisher : ActorPublisher, IWithUnboundedStash { - public static Props Props(bool liveQuery, string writeJournalPluginId) + public static Props Props(string writeJournalPluginId) { - return Actor.Props.Create(() => new AllPersistenceIdsPublisher(liveQuery, writeJournalPluginId)); + return Actor.Props.Create(() => new CurrentPersistenceIdsPublisher(writeJournalPluginId)); } - private readonly bool _liveQuery; private readonly IActorRef _journalRef; private readonly DeliveryBuffer _buffer; - public AllPersistenceIdsPublisher(bool liveQuery, string writeJournalPluginId) + public IStash Stash { get; set; } + + public CurrentPersistenceIdsPublisher(string writeJournalPluginId) { - _liveQuery = liveQuery; _buffer = new DeliveryBuffer(OnNext); _journalRef = Persistence.Instance.Apply(Context.System).JournalFor(writeJournalPluginId); } - protected override bool Receive(object message) => message.Match() - .With(_ => + protected override bool Receive(object message) + { + switch (message) { - _journalRef.Tell(SubscribeAllPersistenceIds.Instance); - Become(Active); - }) - .With(_ => Context.Stop(Self)) - .WasHandled; - - private bool Active(object message) => message.Match() - .With(current => + case Request _: + _journalRef.Tell(new SelectCurrentPersistenceIds(0, Self)); + Become(Initializing); + return true; + case Cancel _: + Context.Stop(Self); + return true; + default: + return false; + } + } + + private bool Initializing(object message) + { + switch (message) { - _buffer.AddRange(current.AllPersistenceIds); - _buffer.DeliverBuffer(TotalDemand); + case CurrentPersistenceIds current: + _buffer.AddRange(current.AllPersistenceIds); + _buffer.DeliverBuffer(TotalDemand); - if (!_liveQuery && _buffer.IsEmpty) - OnCompleteThenStop(); - }) - .With(added => + if (_buffer.IsEmpty) + { + OnCompleteThenStop(); + return true; + } + + Become(Active); + Stash.UnstashAll(); + return true; + case Cancel _: + Context.Stop(Self); + return true; + default: + Stash.Stash(); + return true; + } + } + + private bool Active(object message) + { + switch (message) { - if (_liveQuery) - { - _buffer.Add(added.PersistenceId); + case Request _: _buffer.DeliverBuffer(TotalDemand); - } - }) - .With(_ => + if (_buffer.IsEmpty) + OnCompleteThenStop(); + return true; + case Cancel _: + Context.Stop(Self); + return true; + default: + return false; + } + } + } + + internal sealed class LivePersistenceIdsPublisher : ActorPublisher, IWithUnboundedStash + { + private class Continue + { + public static readonly Continue Instance = new Continue(); + + private Continue() { } + } + + public static Props Props(TimeSpan refreshInterval, string writeJournalPluginId, Action onTerminated) + { + return Actor.Props.Create(() => new LivePersistenceIdsPublisher(refreshInterval, writeJournalPluginId, onTerminated)); + } + + private long _lastOrderingOffset; + private readonly ICancelable _tickCancelable; + private readonly IActorRef _journalRef; + private readonly Action _onTerminated; + private readonly DeliveryBuffer _buffer; + + public IStash Stash { get; set; } + + public LivePersistenceIdsPublisher(TimeSpan refreshInterval, string writeJournalPluginId, Action onTerminated) + { + _tickCancelable = Context.System.Scheduler.ScheduleTellRepeatedlyCancelable( + refreshInterval, + refreshInterval, + Self, + Continue.Instance, + Self); + _buffer = new DeliveryBuffer(OnNext); + _journalRef = Persistence.Instance.Apply(Context.System).JournalFor(writeJournalPluginId); + _onTerminated = onTerminated; + } + + protected override void PostStop() + { + _tickCancelable.Cancel(); + _onTerminated(); + base.PostStop(); + } + + protected override bool Receive(object message) + { + switch (message) { - _buffer.DeliverBuffer(TotalDemand); - if (!_liveQuery && _buffer.IsEmpty) - OnCompleteThenStop(); - }) - .With(_ => Context.Stop(Self)) - .WasHandled; + case Request _: + _journalRef.Tell(new SelectCurrentPersistenceIds(0, Self)); + Become(Initializing); + return true; + case Continue _: + return true; + case Cancel _: + Context.Stop(Self); + return true; + default: + return false; + } + } + + private bool Initializing(object message) + { + switch (message) + { + case CurrentPersistenceIds current: + _lastOrderingOffset = current.HighestOrderingNumber; + _buffer.AddRange(current.AllPersistenceIds); + _buffer.DeliverBuffer(TotalDemand); + + Become(Active); + Stash.UnstashAll(); + return true; + case Continue _: + return true; + case Cancel _: + Context.Stop(Self); + return true; + default: + Stash.Stash(); + return true; + } + } + + private bool Active(object message) + { + switch (message) + { + case CurrentPersistenceIds added: + _lastOrderingOffset = added.HighestOrderingNumber; + _buffer.AddRange(added.AllPersistenceIds); + _buffer.DeliverBuffer(TotalDemand); + return true; + case Request _: + _buffer.DeliverBuffer(TotalDemand); + return true; + case Continue _: + _journalRef.Tell(new SelectCurrentPersistenceIds(_lastOrderingOffset, Self)); + return true; + case Cancel _: + Context.Stop(Self); + return true; + default: + return false; + } + } } } diff --git a/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs b/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs index b2fb8e19f75..0d12dc128f1 100644 --- a/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs +++ b/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs @@ -11,6 +11,7 @@ using Akka.Configuration; using Akka.Persistence.Journal; using Akka.Streams.Dsl; +using Akka.Streams; using Akka.Util.Internal; namespace Akka.Persistence.Query.Sql @@ -40,12 +41,16 @@ public static Config DefaultConfiguration() private readonly TimeSpan _refreshInterval; private readonly string _writeJournalPluginId; private readonly int _maxBufferSize; + private readonly ExtendedActorSystem _system; + + private IPublisher _persistenceIdsPublisher = null; public SqlReadJournal(ExtendedActorSystem system, Config config) { _refreshInterval = config.GetTimeSpan("refresh-interval", null); _writeJournalPluginId = config.GetString("write-plugin", null); _maxBufferSize = config.GetInt("max-buffer-size", 0); + _system = system; } /// @@ -68,18 +73,32 @@ public SqlReadJournal(ExtendedActorSystem system, Config config) /// backend journal. /// /// - public Source PersistenceIds() => - Source.ActorPublisher(AllPersistenceIdsPublisher.Props(true, _writeJournalPluginId)) - .MapMaterializedValue(_ => NotUsed.Instance) - .Named("AllPersistenceIds") as Source; + public Source PersistenceIds() + { + if (_persistenceIdsPublisher is null) + { + var graph = + Source.ActorPublisher( + LivePersistenceIdsPublisher.Props( + _refreshInterval, + _writeJournalPluginId, + () => _persistenceIdsPublisher = null)) + .ToMaterialized(Sink.DistinctRetainingFanOutPublisher(), Keep.Right); + _persistenceIdsPublisher = graph.Run(_system.Materializer()); + } + + return Source.FromPublisher(_persistenceIdsPublisher) + .MapMaterializedValue(_ => NotUsed.Instance) + .Named("AllPersistenceIds") as Source; + } /// /// Same type of query as but the stream /// is completed immediately when it reaches the end of the "result set". Persistent /// actors that are created after the query is completed are not included in the stream. /// - public Source CurrentPersistenceIds() => - Source.ActorPublisher(AllPersistenceIdsPublisher.Props(false, _writeJournalPluginId)) + public Source CurrentPersistenceIds() + => Source.ActorPublisher(CurrentPersistenceIdsPublisher.Props(_writeJournalPluginId)) .MapMaterializedValue(_ => NotUsed.Instance) .Named("CurrentPersistenceIds") as Source; diff --git a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/BatchingSqlJournal.cs b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/BatchingSqlJournal.cs index 92b2c4ba40e..37f3bbe47a3 100644 --- a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/BatchingSqlJournal.cs +++ b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/BatchingSqlJournal.cs @@ -383,13 +383,6 @@ public BatchComplete(int chunkId, int operationCount, TimeSpan timeSpent, Except } } - // this little guy will be called only once, only by the current journal - private sealed class GetCurrentPersistenceIds - { - public static readonly GetCurrentPersistenceIds Instance = new GetCurrentPersistenceIds(); - private GetCurrentPersistenceIds() { } - } - private struct RequestChunk { public readonly int ChunkId; @@ -463,7 +456,7 @@ public RequestChunk(int chunkId, IJournalRequest[] requests) protected virtual string InsertEventSql { get; } /// - /// SQL query executed as result of request to journal. + /// SQL query executed as result of request to journal. /// It's a part of persistence query protocol. /// protected virtual string AllPersistenceIdsSql { get; } @@ -518,12 +511,6 @@ public RequestChunk(int chunkId, IJournalRequest[] requests) /// protected bool HasTagSubscribers => _tagSubscribers.Count != 0; - /// - /// Flag determining if current journal has any subscribers for and - /// messages. - /// - protected bool HasAllIdsSubscribers => _allIdsSubscribers.Count != 0; - /// /// Flag determining if current journal has any subscribers for and /// @@ -548,8 +535,6 @@ public RequestChunk(int chunkId, IJournalRequest[] requests) private readonly Dictionary> _persistenceIdSubscribers; private readonly Dictionary> _tagSubscribers; - private readonly HashSet _allIdsSubscribers; - private readonly HashSet _allPersistenceIds; private readonly HashSet _newEventSubscriber; private readonly Akka.Serialization.Serialization _serialization; @@ -567,8 +552,6 @@ protected BatchingSqlJournal(BatchingSqlJournalSetup setup) _persistenceIdSubscribers = new Dictionary>(); _tagSubscribers = new Dictionary>(); - _allIdsSubscribers = new HashSet(); - _allPersistenceIds = new HashSet(); _newEventSubscriber = new HashSet(); _remainingOperations = Setup.MaxConcurrentOperations; @@ -637,8 +620,7 @@ WHERE e.{conventions.OrderingColumnName} > @Ordering HighestOrderingSql = $@" SELECT MAX(e.{conventions.OrderingColumnName}) as Ordering - FROM {conventions.FullJournalTableName} e - WHERE e.{conventions.OrderingColumnName} > @Ordering"; + FROM {conventions.FullJournalTableName} e"; InsertEventSql = $@" INSERT INTO {conventions.FullJournalTableName} ( @@ -710,15 +692,15 @@ protected sealed override bool Receive(object message) case ReplayAllEvents msg: BatchRequest(msg); return true; + case SelectCurrentPersistenceIds msg: + BatchRequest(msg); + return true; case BatchComplete msg: CompleteBatch(msg); return true; case SubscribePersistenceId msg: AddPersistenceIdSubscriber(msg); return true; - case SubscribeAllPersistenceIds msg: - AddAllPersistenceIdsSubscriber(msg); - return true; case SubscribeTag msg: AddTagSubscriber(msg); return true; @@ -728,12 +710,6 @@ protected sealed override bool Receive(object message) case Terminated msg: RemoveSubscriber(msg.ActorRef); return true; - case GetCurrentPersistenceIds _: - InitializePersistenceIds(); - return true; - case CurrentPersistenceIds msg: - SendCurrentPersistenceIds(msg); - return true; case ChunkExecutionFailure msg: FailChunkExecution(msg); return true; @@ -770,62 +746,9 @@ private void FailChunkExecution(ChunkExecutionFailure message) } } - private void SendCurrentPersistenceIds(CurrentPersistenceIds message) - { - foreach (var persistenceId in message.AllPersistenceIds) - { - _allPersistenceIds.Add(persistenceId); - } - - foreach (var subscriber in _allIdsSubscribers) - { - subscriber.Tell(message); - } - } - #region subscriptions - - private void InitializePersistenceIds() - { - var self = Self; - GetAllPersistenceIdsAsync() - .ContinueWith(task => - { - if (task.IsCanceled || task.IsFaulted) - { - var cause = (Exception)task.Exception ?? new OperationCanceledException("Cancellation occurred while trying to retrieve current persistence ids"); - Log.Error(cause, "Couldn't retrieve current persistence ids"); - } - else - { - self.Tell(new CurrentPersistenceIds(task.Result)); - } - }); - } - - private async Task> GetAllPersistenceIdsAsync() - { - var result = new List(256); - using (var connection = CreateConnection(Setup.ConnectionString)) - { - await connection.OpenAsync(); - using (var command = connection.CreateCommand()) - { - command.CommandText = AllPersistenceIdsSql; - - var reader = await command.ExecuteReaderAsync(); - while (await reader.ReadAsync()) - { - result.Add(reader.GetString(0)); - } - } - } - return result; - } - private void RemoveSubscriber(IActorRef subscriberRef) { - _allIdsSubscribers.Remove(subscriberRef); _persistenceIdSubscribers.RemoveItem(subscriberRef); _tagSubscribers.RemoveItem(subscriberRef); _newEventSubscriber.Remove(subscriberRef); @@ -845,18 +768,6 @@ private void AddTagSubscriber(SubscribeTag message) Context.Watch(subscriber); } - private void AddAllPersistenceIdsSubscriber(SubscribeAllPersistenceIds message) - { - if (!HasAllIdsSubscribers) - { - Self.Tell(GetCurrentPersistenceIds.Instance); - } - - var subscriber = Sender; - _allIdsSubscribers.Add(subscriber); - Context.Watch(subscriber); - } - private void AddPersistenceIdSubscriber(SubscribePersistenceId message) { var subscriber = Sender; @@ -895,18 +806,6 @@ private void NotifyPersistenceIdChanged(string persistenceId) } } - protected void NotifyNewPersistenceIdAdded(string persistenceId) - { - if (_allPersistenceIds.Add(persistenceId) && HasAllIdsSubscribers) - { - var added = new PersistenceIdAdded(persistenceId); - foreach (var subscriber in _allIdsSubscribers) - { - subscriber.Tell(added, ActorRefs.NoSender); - } - } - } - #endregion /// @@ -1004,6 +903,9 @@ private async Task ExecuteChunk(RequestChunk chunk, IActorContext case ReplayAllEvents msg: await HandleReplayAllMessages(msg, command); break; + case SelectCurrentPersistenceIds msg: + await HandleSelectCurrentPersistenceIds(msg, command); + break; default: Unhandled(req); break; @@ -1040,8 +942,6 @@ protected virtual async Task HandleDeleteMessagesTo(DeleteMessagesTo req, TComma var toSequenceNr = req.ToSequenceNr; var persistenceId = req.PersistenceId; - NotifyNewPersistenceIdAdded(persistenceId); - try { var highestSequenceNr = await ReadHighestSequenceNr(persistenceId, command); @@ -1086,6 +986,34 @@ protected virtual async Task ReadHighestSequenceNr(string persistenceId, T return highestSequenceNr; } + protected virtual async Task ReadHighestSequenceNr(TCommand command) + { + command.CommandText = HighestOrderingSql; + command.Parameters.Clear(); + + var result = await command.ExecuteScalarAsync(); + var highestSequenceNr = result is long ? Convert.ToInt64(result) : 0L; + return highestSequenceNr; + } + + protected virtual async Task HandleSelectCurrentPersistenceIds(SelectCurrentPersistenceIds message, TCommand command) + { + long highestOrderingNumber = await ReadHighestSequenceNr(command); + + var result = new List(256); + command.CommandText = AllPersistenceIdsSql; + command.Parameters.Clear(); + AddParameter(command, "@Ordering", DbType.Int64, message.Offset); + + var reader = await command.ExecuteReaderAsync(); + while (await reader.ReadAsync()) + { + result.Add(reader.GetString(0)); + } + + message.ReplyTo.Tell(new CurrentPersistenceIds(result, highestOrderingNumber)); + } + protected virtual async Task HandleReplayTaggedMessages(ReplayTaggedMessages req, TCommand command) { var replyTo = req.ReplyTo; @@ -1179,8 +1107,6 @@ protected virtual async Task HandleReplayMessages(ReplayMessages req, TCommand c : req.PersistentActor; var persistenceId = req.PersistenceId; - NotifyNewPersistenceIdAdded(persistenceId); - try { var highestSequenceNr = await ReadHighestSequenceNr(persistenceId, command); @@ -1268,8 +1194,6 @@ private async Task HandleWriteMessages(WriteMessages req, TCommand command) var response = (new WriteMessageSuccess(unadapted, actorInstanceId), unadapted.Sender); responses.Add(response); persistenceIds.Add(persistent.PersistenceId); - - NotifyNewPersistenceIdAdded(persistent.PersistenceId); } catch (DbException cause) { diff --git a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryApi.cs b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryApi.cs index 8ac1b79bdec..5e993190323 100644 --- a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryApi.cs +++ b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryApi.cs @@ -64,20 +64,17 @@ public EventAppended(string persistenceId) } } - /// - /// Subscribe the `sender` to current and new persistenceIds. - /// Used by query-side. The journal will send one to the - /// subscriber followed by messages when new persistenceIds - /// are created. - /// [Serializable] - public sealed class SubscribeAllPersistenceIds : ISubscriptionCommand + public sealed class SelectCurrentPersistenceIds : IJournalRequest { - /// - /// TBD - /// - public static readonly SubscribeAllPersistenceIds Instance = new SubscribeAllPersistenceIds(); - private SubscribeAllPersistenceIds() { } + public IActorRef ReplyTo { get; } + public long Offset { get; } + + public SelectCurrentPersistenceIds(long offset, IActorRef replyTo) + { + Offset = offset; + ReplyTo = replyTo; + } } /// @@ -91,34 +88,17 @@ public sealed class CurrentPersistenceIds : IDeadLetterSuppression /// public readonly IEnumerable AllPersistenceIds; + public readonly long HighestOrderingNumber; + /// /// TBD /// /// TBD - public CurrentPersistenceIds(IEnumerable allPersistenceIds) + /// TBD + public CurrentPersistenceIds(IEnumerable allPersistenceIds, long highestOrderingNumber) { AllPersistenceIds = allPersistenceIds.ToImmutableHashSet(); - } - } - - /// - /// TBD - /// - [Serializable] - public sealed class PersistenceIdAdded : IDeadLetterSuppression - { - /// - /// TBD - /// - public readonly string PersistenceId; - - /// - /// TBD - /// - /// TBD - public PersistenceIdAdded(string persistenceId) - { - PersistenceId = persistenceId; + HighestOrderingNumber = highestOrderingNumber; } } diff --git a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs index eb9ed40e93f..c15fbe05512 100644 --- a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs +++ b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs @@ -6,6 +6,7 @@ //----------------------------------------------------------------------- using System; +using System.Collections.Generic; using System.Collections.Immutable; using System.Data; using System.Data.Common; @@ -34,8 +35,9 @@ public interface IJournalQueryExecutor /// /// TBD /// TBD + /// TBD /// TBD - Task> SelectAllPersistenceIdsAsync(DbConnection connection, CancellationToken cancellationToken); + Task> SelectAllPersistenceIdsAsync(DbConnection connection, CancellationToken cancellationToken, long offset); /// /// Asynchronously replays a on all selected events for provided @@ -70,10 +72,11 @@ public interface IJournalQueryExecutor Task SelectAllEventsAsync( DbConnection connection, - CancellationToken cancellationToken, long fromOffset, + long toOffset, long max, - Action callback); + Action callback, + CancellationToken cancellationToken); /// /// Asynchronously returns single number considered as the highest sequence number in current journal for the provided . @@ -82,8 +85,10 @@ Task SelectAllEventsAsync( /// TBD /// TBD /// TBD - Task SelectHighestSequenceNrAsync(DbConnection connection, CancellationToken cancellationToken, string persistenceId); - + Task SelectHighestSequenceNrAsync(DbConnection connection, string persistenceId, CancellationToken cancellationToken); + + Task SelectHighestSequenceNrAsync(DbConnection connection, CancellationToken cancellationToken); + /// /// Asynchronously inserts a collection of events and theirs tags into a journal table. /// @@ -326,7 +331,8 @@ protected AbstractQueryExecutor(QueryConfiguration configuration, Akka.Serializa AllPersistenceIdsSql = $@" SELECT DISTINCT e.{Configuration.PersistenceIdColumnName} as PersistenceId - FROM {Configuration.FullJournalTableName} e;"; + FROM {Configuration.FullJournalTableName} e + WHERE e.{Configuration.OrderingColumnName} > @Ordering"; HighestSequenceNrSql = $@" SELECT MAX(u.SeqNr) as SequenceNr @@ -376,8 +382,7 @@ WHERE e.{Configuration.OrderingColumnName} > @Ordering HighestOrderingSql = $@" SELECT MAX(e.{Configuration.OrderingColumnName}) as Ordering - FROM {Configuration.FullJournalTableName} e - WHERE e.{Configuration.OrderingColumnName} > @Ordering"; + FROM {Configuration.FullJournalTableName} e"; InsertEventSql = $@" INSERT INTO {Configuration.FullJournalTableName} ( @@ -473,19 +478,23 @@ SELECT MAX(e.{Configuration.OrderingColumnName}) as Ordering /// /// TBD /// TBD + /// TBD /// TBD - public virtual async Task> SelectAllPersistenceIdsAsync(DbConnection connection, CancellationToken cancellationToken) + public virtual async Task> SelectAllPersistenceIdsAsync(DbConnection connection, CancellationToken cancellationToken, long offset) { using (var command = GetCommand(connection, AllPersistenceIdsSql)) - using (var reader = await command.ExecuteReaderAsync(cancellationToken)) { - var builder = ImmutableArray.CreateBuilder(); - while (await reader.ReadAsync(cancellationToken)) + AddParameter(command, "@Ordering", DbType.Int64, offset); + + using (var reader = await command.ExecuteReaderAsync(cancellationToken)) { - builder.Add(reader.GetString(0)); + var builder = ImmutableArray.CreateBuilder(); + while (await reader.ReadAsync(cancellationToken)) + { + builder.Add(reader.GetString(0)); + } + return builder.ToImmutable(); } - - return builder.ToImmutable(); } } @@ -586,15 +595,24 @@ public virtual async Task SelectByTagAsync(DbConnection connection, Cancel public async Task SelectAllEventsAsync( DbConnection connection, - CancellationToken cancellationToken, long fromOffset, + long toOffset, long max, - Action callback) + Action callback, + CancellationToken cancellationToken) { + long maxOrdering; + using (var command = GetCommand(connection, HighestOrderingSql)) + { + maxOrdering = (await command.ExecuteScalarAsync(cancellationToken)) as long? ?? 0L; + } + using (var command = GetCommand(connection, AllEventsSql)) { + var take = Math.Min(toOffset - fromOffset, max); + AddParameter(command, "@Ordering", DbType.Int64, fromOffset); - AddParameter(command, "@Take", DbType.Int64, max); + AddParameter(command, "@Take", DbType.Int64, take); var commandBehavior = Configuration.UseSequentialAccess ? CommandBehavior.SequentialAccess : @@ -611,12 +629,7 @@ public async Task SelectAllEventsAsync( } } - using (var command = GetCommand(connection, HighestOrderingSql)) - { - AddParameter(command, "@Ordering", DbType.Int64, fromOffset); - var maxOrdering = (await command.ExecuteScalarAsync(cancellationToken)) as long? ?? 0L; - return maxOrdering; - } + return maxOrdering; } /// @@ -626,7 +639,7 @@ public async Task SelectAllEventsAsync( /// TBD /// TBD /// TBD - public virtual async Task SelectHighestSequenceNrAsync(DbConnection connection, CancellationToken cancellationToken, string persistenceId) + public virtual async Task SelectHighestSequenceNrAsync(DbConnection connection, string persistenceId, CancellationToken cancellationToken) { using (var command = GetCommand(connection, HighestSequenceNrSql)) { @@ -637,6 +650,15 @@ public virtual async Task SelectHighestSequenceNrAsync(DbConnection connec } } + public virtual async Task SelectHighestSequenceNrAsync(DbConnection connection, CancellationToken cancellationToken) + { + using (var command = GetCommand(connection, HighestOrderingSql)) + { + var result = await command.ExecuteScalarAsync(cancellationToken); + return result is long ? Convert.ToInt64(result) : 0L; + } + } + /// /// TBD /// diff --git a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/SqlJournal.cs b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/SqlJournal.cs index b19a589440d..e61749279ed 100644 --- a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/SqlJournal.cs +++ b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/SqlJournal.cs @@ -10,6 +10,7 @@ using System.Collections.Immutable; using System.Data.Common; using System.Linq; +using System.Reflection; using System.Threading; using System.Threading.Tasks; using Akka.Actor; @@ -27,9 +28,6 @@ public abstract class SqlJournal : AsyncWriteJournal, IWithUnboundedStash private ImmutableDictionary> _persistenceIdSubscribers = ImmutableDictionary.Create>(); private ImmutableDictionary> _tagSubscribers = ImmutableDictionary.Create>(); private readonly HashSet _newEventsSubscriber = new HashSet(); - private readonly HashSet _allPersistenceIdSubscribers = new HashSet(); - private readonly ReaderWriterLockSlim _allPersistenceIdsLock = new ReaderWriterLockSlim(); - private HashSet _allPersistenceIds = new HashSet(); private IImmutableDictionary _tagSequenceNr = ImmutableDictionary.Empty; private readonly CancellationTokenSource _pendingRequestsCancellation; @@ -47,16 +45,8 @@ protected SqlJournal(Config journalConfig) _pendingRequestsCancellation = new CancellationTokenSource(); } - /// - /// TBD - /// public IStash Stash { get; set; } - /// - /// TBD - /// - public IEnumerable AllPersistenceIds => _allPersistenceIds; - /// /// TBD /// @@ -69,10 +59,6 @@ protected SqlJournal(Config journalConfig) /// TBD /// protected bool HasNewEventSubscribers => _newEventsSubscriber.Count != 0; - /// - /// TBD - /// - protected bool HasAllPersistenceIdSubscribers => _allPersistenceIdSubscribers.Count != 0; /// /// Returns a HOCON config path to associated journal. @@ -118,9 +104,9 @@ protected override bool ReceivePluginInternal(object message) AddPersistenceIdSubscriber(Sender, subscribe.PersistenceId); Context.Watch(Sender); return true; - case SubscribeAllPersistenceIds _: - AddAllPersistenceIdSubscriber(Sender); - Context.Watch(Sender); + case SelectCurrentPersistenceIds request: + SelectAllPersistenceIdsAsync(request.Offset) + .PipeTo(request.ReplyTo, success: result => new CurrentPersistenceIds(result.Ids, request.Offset)); return true; case SubscribeTag subscribe: AddTagSubscriber(Sender, subscribe.Tag); @@ -178,8 +164,6 @@ protected override async Task> WriteMessagesAsync(IEnu if (IsTagId(p.PersistenceId)) throw new InvalidOperationException($"Persistence Id {p.PersistenceId} must not start with {QueryExecutor.Configuration.TagsColumnName}"); - - NotifyNewPersistenceIdAdded(p.PersistenceId); } var batch = new WriteJournalBatch(eventToTags); @@ -247,15 +231,30 @@ protected virtual async Task ReplayAllEventsAsync(ReplayAllEvents replay) { return await QueryExecutor .SelectAllEventsAsync(connection, - cancellationToken.Token, replay.FromOffset, + replay.ToOffset, replay.Max, replayedEvent => { foreach (var adapted in AdaptFromJournal(replayedEvent.Persistent)) { replay.ReplyTo.Tell(new ReplayedEvent(adapted, replayedEvent.Offset), ActorRefs.NoSender); } - }); + }, + cancellationToken.Token); + } + } + } + + protected virtual async Task<(IEnumerable Ids, long LastOrdering)> SelectAllPersistenceIdsAsync(long offset) + { + using (var connection = CreateDbConnection()) + { + await connection.OpenAsync(); + using (var cancellationToken = CancellationTokenSource.CreateLinkedTokenSource(_pendingRequestsCancellation.Token)) + { + var lastOrdering = await QueryExecutor.SelectHighestSequenceNrAsync(connection, cancellationToken.Token); + var ids = await QueryExecutor.SelectAllPersistenceIdsAsync(connection, cancellationToken.Token, offset); + return (ids, lastOrdering); } } } @@ -273,7 +272,6 @@ protected virtual async Task ReplayAllEventsAsync(ReplayAllEvents replay) public override async Task ReplayMessagesAsync(IActorContext context, string persistenceId, long fromSequenceNr, long toSequenceNr, long max, Action recoveryCallback) { - NotifyNewPersistenceIdAdded(persistenceId); using (var connection = CreateDbConnection()) { await connection.OpenAsync(); @@ -310,24 +308,27 @@ protected override void PostStop() /// TBD protected bool WaitingForInitialization(object message) { - return message.Match() - .With(all => - { - _allPersistenceIds = new HashSet(all.Ids); + switch (message) + { + case Status.Success _: UnbecomeStacked(); Stash.UnstashAll(); - }) - .With(fail => - { - Log.Error(fail.Exception, "Failure during {0} initialization.", Self); + return true; + case Status.Failure fail: + Log.Error(fail.Cause, "Failure during {0} initialization.", Self); Context.Stop(Self); - }) - .Default(_ => Stash.Stash()) - .WasHandled; + return true; + default: + Stash.Stash(); + return true; + } } private async Task Initialize() { + if (!_settings.AutoInitialize) + return new Status.Success(NotUsed.Instance); + try { using (var connection = CreateDbConnection()) @@ -335,20 +336,15 @@ private async Task Initialize() await connection.OpenAsync(); using (var cancellationToken = CancellationTokenSource.CreateLinkedTokenSource(_pendingRequestsCancellation.Token)) { - if (_settings.AutoInitialize) - { - await QueryExecutor.CreateTablesAsync(connection, cancellationToken.Token); - } - - var ids = await QueryExecutor.SelectAllPersistenceIdsAsync(connection, cancellationToken.Token); - return new AllPersistenceIds(ids); + await QueryExecutor.CreateTablesAsync(connection, cancellationToken.Token); } } } catch (Exception e) { - return new Failure {Exception = e}; + return new Status.Failure(e); } + return new Status.Success(NotUsed.Instance); } /// @@ -375,8 +371,6 @@ public void RemoveSubscriber(IActorRef subscriber) .Where(kv => kv.Value.Contains(subscriber)) .Select(kv => new KeyValuePair>(kv.Key, kv.Value.Remove(subscriber)))); - _allPersistenceIdSubscribers.Remove(subscriber); - _newEventsSubscriber.Remove(subscriber); } @@ -402,16 +396,6 @@ public void AddTagSubscriber(IActorRef subscriber, string tag) } } - /// - /// TBD - /// - /// TBD - public void AddAllPersistenceIdSubscriber(IActorRef subscriber) - { - _allPersistenceIdSubscribers.Add(subscriber); - subscriber.Tell(new CurrentPersistenceIds(AllPersistenceIds)); - } - /// /// TBD /// @@ -441,44 +425,6 @@ private async Task NextTagSequenceNr(string tag) private string TagId(string tag) => QueryExecutor.Configuration.TagsColumnName + tag; - private void NotifyNewPersistenceIdAdded(string persistenceId) - { - var isNew = TryAddPersistenceId(persistenceId); - if (isNew && HasAllPersistenceIdSubscribers && !IsTagId(persistenceId)) - { - var added = new PersistenceIdAdded(persistenceId); - foreach (var subscriber in _allPersistenceIdSubscribers) - subscriber.Tell(added); - } - } - - private bool TryAddPersistenceId(string persistenceId) - { - try - { - _allPersistenceIdsLock.EnterUpgradeableReadLock(); - - if (_allPersistenceIds.Contains(persistenceId)) return false; - else - { - try - { - _allPersistenceIdsLock.EnterWriteLock(); - _allPersistenceIds.Add(persistenceId); - return true; - } - finally - { - _allPersistenceIdsLock.ExitWriteLock(); - } - } - } - finally - { - _allPersistenceIdsLock.ExitUpgradeableReadLock(); - } - } - private bool IsTagId(string persistenceId) { return persistenceId.StartsWith(QueryExecutor.Configuration.TagsColumnName); @@ -524,7 +470,6 @@ private void NotifyNewEventAppended() /// TBD protected override async Task DeleteMessagesToAsync(string persistenceId, long toSequenceNr) { - NotifyNewPersistenceIdAdded(persistenceId); using (var connection = CreateDbConnection()) { await connection.OpenAsync(); @@ -543,13 +488,12 @@ protected override async Task DeleteMessagesToAsync(string persistenceId, long t /// TBD public override async Task ReadHighestSequenceNrAsync(string persistenceId, long fromSequenceNr) { - NotifyNewPersistenceIdAdded(persistenceId); using (var connection = CreateDbConnection()) { await connection.OpenAsync(); using (var cancellationToken = CancellationTokenSource.CreateLinkedTokenSource(_pendingRequestsCancellation.Token)) { - return await QueryExecutor.SelectHighestSequenceNrAsync(connection, cancellationToken.Token, persistenceId); + return await QueryExecutor.SelectHighestSequenceNrAsync(connection, persistenceId, cancellationToken.Token); } } } @@ -582,14 +526,10 @@ protected virtual string GetConnectionString() protected ITimestampProvider GetTimestampProvider(string typeName) { var type = Type.GetType(typeName, true); - try - { - return (ITimestampProvider)Activator.CreateInstance(type, Context.System); - } - catch (Exception) - { - return (ITimestampProvider)Activator.CreateInstance(type); - } + var withSystem = type.GetConstructor(new[] { Context.System.GetType() }) != null; + return withSystem ? + (ITimestampProvider)Activator.CreateInstance(type, Context.System) : + (ITimestampProvider)Activator.CreateInstance(type); } #endregion } diff --git a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/BatchingSqliteCurrentAllEventsSpec.cs b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/BatchingSqliteCurrentAllEventsSpec.cs deleted file mode 100644 index 8da551deb13..00000000000 --- a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/BatchingSqliteCurrentAllEventsSpec.cs +++ /dev/null @@ -1,37 +0,0 @@ -using System; -using System.Collections.Generic; -using System.Text; -using Akka.Configuration; -using Akka.Persistence.Query; -using Akka.Persistence.Query.Sql; -using Akka.Persistence.TCK.Query; -using Akka.Util.Internal; -using Xunit.Abstractions; - -namespace Akka.Persistence.Sqlite.Tests.Query -{ - public class BatchingCurrentSqliteAllEventsSpec : CurrentAllEventsSpec - { - public static readonly AtomicCounter Counter = new AtomicCounter(0); - - public static Config Config(int id) => ConfigurationFactory.ParseString($@" - akka.loglevel = INFO - akka.persistence.journal.plugin = ""akka.persistence.journal.sqlite"" - akka.persistence.journal.sqlite {{ - class = ""Akka.Persistence.Sqlite.Journal.BatchingSqliteJournal, Akka.Persistence.Sqlite"" - plugin-dispatcher = ""akka.actor.default-dispatcher"" - table-name = event_journal - metadata-table-name = journal_metadata - auto-initialize = on - connection-string = ""Filename=file:memdb-journal-eventsbytag-{id}.db;Mode=Memory;Cache=Shared"" - refresh-interval = 1s - }} - akka.test.single-expect-default = 10s") - .WithFallback(SqlReadJournal.DefaultConfiguration()); - - public BatchingCurrentSqliteAllEventsSpec(ITestOutputHelper output) : base(Config(Counter.GetAndIncrement()), nameof(BatchingSqliteAllEventsSpec), output) - { - ReadJournal = Sys.ReadJournalFor(SqlReadJournal.Identifier); - } - } -} diff --git a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteCurrentPersistenceIdsSpec.cs b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteCurrentPersistenceIdsSpec.cs index cf2ad4a878a..ca46a9f0dee 100644 --- a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteCurrentPersistenceIdsSpec.cs +++ b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteCurrentPersistenceIdsSpec.cs @@ -38,10 +38,5 @@ public SqliteCurrentPersistenceIdsSpec(ITestOutputHelper output) : base(Config(C { ReadJournal = Sys.ReadJournalFor(SqlReadJournal.Identifier); } - - [Fact(Skip = "Not implemented, due to bugs on NetCore")] - public override void ReadJournal_query_CurrentPersistenceIds_should_not_see_new_events_after_complete() - { - } } } diff --git a/src/core/Akka.Persistence.TCK/Query/CurrentPersistenceIdsSpec.cs b/src/core/Akka.Persistence.TCK/Query/CurrentPersistenceIdsSpec.cs index e0014a27375..9d2b432839d 100644 --- a/src/core/Akka.Persistence.TCK/Query/CurrentPersistenceIdsSpec.cs +++ b/src/core/Akka.Persistence.TCK/Query/CurrentPersistenceIdsSpec.cs @@ -6,6 +6,7 @@ //----------------------------------------------------------------------- using System; +using System.Collections.Generic; using Akka.Actor; using Akka.Configuration; using Akka.Persistence.Query; @@ -78,16 +79,17 @@ public virtual void ReadJournal_query_CurrentPersistenceIds_should_not_see_new_e var greenSrc = queries.CurrentPersistenceIds(); var probe = greenSrc.RunWith(this.SinkProbe(), Materializer); + var set = new List { "a", "b", "c" }; probe.Request(2) - .ExpectNext("a") - .ExpectNext("c") + .ExpectNextWithinSet(set) + .ExpectNextWithinSet(set) .ExpectNoMsg(TimeSpan.FromMilliseconds(100)); Setup("d", 1); probe.ExpectNoMsg(TimeSpan.FromMilliseconds(100)); probe.Request(5) - .ExpectNext("b") + .ExpectNextWithinSet(set) .ExpectComplete(); } diff --git a/src/core/Akka.Streams.TestKit/TestSubscriber.cs b/src/core/Akka.Streams.TestKit/TestSubscriber.cs index 62ce71f513e..c5d9a5ac376 100644 --- a/src/core/Akka.Streams.TestKit/TestSubscriber.cs +++ b/src/core/Akka.Streams.TestKit/TestSubscriber.cs @@ -148,9 +148,9 @@ public T ExpectNext(TimeSpan timeout) /// /// Fluent DSL. Expect a stream element. /// - public ManualProbe ExpectNext(T element) + public ManualProbe ExpectNext(T element, TimeSpan? timeout = null) { - _probe.ExpectMsg>(x => AssertEquals(x.Element, element, "Expected '{0}', but got '{1}'", element, x.Element)); + _probe.ExpectMsg>(x => AssertEquals(x.Element, element, "Expected '{0}', but got '{1}'", element, x.Element), timeout); return this; } @@ -176,9 +176,12 @@ public ManualProbe ExpectNext(T element, TimeSpan timeout) /// Fluent DSL. Expect multiple stream elements. /// public ManualProbe ExpectNext(T e1, T e2, params T[] elems) + => ExpectNext(null, e1, e2, elems); + + public ManualProbe ExpectNext(TimeSpan? timeout, T e1, T e2, params T[] elems) { var len = elems.Length + 2; - var e = ExpectNextN(len).ToArray(); + var e = ExpectNextN(len, timeout).ToArray(); AssertEquals(e.Length, len, "expected to get {0} events, but got {1}", len, e.Length); AssertEquals(e[0], e1, "expected [0] element to be {0} but found {1}", e1, e[0]); AssertEquals(e[1], e2, "expected [1] element to be {0} but found {1}", e2, e[1]); @@ -195,27 +198,42 @@ public ManualProbe ExpectNext(T e1, T e2, params T[] elems) /// FluentDSL. Expect multiple stream elements in arbitrary order. /// public ManualProbe ExpectNextUnordered(T e1, T e2, params T[] elems) + { + return ExpectNextUnordered(null, e1, e2, elems); + } + + public ManualProbe ExpectNextUnordered(TimeSpan? timeout, T e1, T e2, params T[] elems) { var len = elems.Length + 2; - var e = ExpectNextN(len).ToArray(); + var e = ExpectNextN(len, timeout).ToArray(); AssertEquals(e.Length, len, "expected to get {0} events, but got {1}", len, e.Length); - var expectedSet = new HashSet(elems) {e1, e2}; + var expectedSet = new HashSet(elems) { e1, e2 }; expectedSet.ExceptWith(e); Assert(expectedSet.Count == 0, "unexpected elements [{0}] found in the result", string.Join(", ", expectedSet)); return this; } + public ManualProbe ExpectNextWithinSet(List elems) + { + var next = _probe.ExpectMsg>(); + if(!elems.Contains(next.Element)) + Assert(false, "unexpected elements [{0}] found in the result", next.Element); + elems.Remove(next.Element); + + return this; + } + /// /// Expect and return the next stream elements. /// - public IEnumerable ExpectNextN(long n) + public IEnumerable ExpectNextN(long n, TimeSpan? timeout = null) { var res = new List((int)n); for (int i = 0; i < n; i++) { - var next = _probe.ExpectMsg>(); + var next = _probe.ExpectMsg>(timeout); res.Add(next.Element); } return res; @@ -224,10 +242,10 @@ public IEnumerable ExpectNextN(long n) /// /// Fluent DSL. Expect the given elements to be signalled in order. /// - public ManualProbe ExpectNextN(IEnumerable all) + public ManualProbe ExpectNextN(IEnumerable all, TimeSpan? timeout = null) { foreach (var x in all) - _probe.ExpectMsg>(y => AssertEquals(y.Element, x, "Expected one of ({0}), but got '{1}'", string.Join(", ", all), y.Element)); + _probe.ExpectMsg>(y => AssertEquals(y.Element, x, "Expected one of ({0}), but got '{1}'", string.Join(", ", all), y.Element), timeout); return this; } @@ -235,12 +253,12 @@ public ManualProbe ExpectNextN(IEnumerable all) /// /// Fluent DSL. Expect the given elements to be signalled in any order. /// - public ManualProbe ExpectNextUnorderedN(IEnumerable all) + public ManualProbe ExpectNextUnorderedN(IEnumerable all, TimeSpan? timeout = null) { var collection = new HashSet(all); while (collection.Count > 0) { - var next = ExpectNext(); + var next = timeout.HasValue ? ExpectNext(timeout.Value) : ExpectNext(); Assert(collection.Contains(next), $"expected one of (${string.Join(", ", collection)}), but received {next}"); collection.Remove(next); } @@ -480,6 +498,11 @@ private void Assert(bool predicate, string format, params object[] args) if (!predicate) throw new Exception(string.Format(format, args)); } + private void Assert(Func predicate, string format, params object[] args) + { + if (!predicate()) throw new Exception(string.Format(format, args)); + } + private void AssertEquals(T1 x, T2 y, string format, params object[] args) { if (!Equals(x, y)) throw new Exception(string.Format(format, args)); diff --git a/src/core/Akka.Streams.Tests/Implementation/DistinctRetainingMultiReaderBufferSpec.cs b/src/core/Akka.Streams.Tests/Implementation/DistinctRetainingMultiReaderBufferSpec.cs new file mode 100644 index 00000000000..d56d8d42a82 --- /dev/null +++ b/src/core/Akka.Streams.Tests/Implementation/DistinctRetainingMultiReaderBufferSpec.cs @@ -0,0 +1,149 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; +using Akka.Streams.Implementation; +using FluentAssertions; +using FluentAssertions.Execution; +using Xunit; + +namespace Akka.Streams.Tests.Implementation +{ + public class DistinctRetainingMultiReaderBufferSpec + { + // The rest of the tests are covered by ResizableMultiReaderRingBufferSpec + + [Fact] + public void A_DistinctRetainingMultiReaderBuffer_should_store_distinct_values_only() + { + var test = new Test(4, 4, 3); + test.Write(1).Should().BeTrue(); + test.Write(2).Should().BeTrue(); + test.Write(3).Should().BeTrue(); + test.Write(2).Should().BeTrue(); + test.Write(2).Should().BeTrue(); + test.Write(1).Should().BeTrue(); + test.Inspect().Should().Be("1 2 3 0 (size=3, cursors=3)"); + test.Read(0).Should().Be(1); + test.Read(0).Should().Be(2); + test.Read(1).Should().Be(1); + test.Inspect().Should().Be("1 2 3 0 (size=3, cursors=3)"); + test.Read(0).Should().Be(3); + test.Read(0).Should().Be(null); + test.Read(1).Should().Be(2); + test.Inspect().Should().Be("1 2 3 0 (size=3, cursors=3)"); + test.Read(2).Should().Be(1); + test.Inspect().Should().Be("1 2 3 0 (size=3, cursors=3)"); + test.Read(1).Should().Be(3); + test.Read(1).Should().Be(null); + test.Read(2).Should().Be(2); + test.Read(2).Should().Be(3); + test.Inspect().Should().Be("1 2 3 0 (size=3, cursors=3)"); + } + + private class TestBuffer : DistinctRetainingMultiReaderBuffer + { + public ICursors UnderlyingCursors { get; } + + public TestBuffer(int initialSize, int maxSize, ICursors cursors) : base(initialSize, maxSize, cursors) + { + UnderlyingCursors = cursors; + } + + public string Inspect() + { + return Buffer.Select(x => x ?? 0).Aggregate("", (s, i) => s + i + " ") + + ToString().SkipWhile(c => c != '(').Aggregate("", (s, c) => s + c); + } + } + + private class Test : TestBuffer + { + public Test(int initialSize, int maxSize, int cursorCount) : base(initialSize, maxSize, new SimpleCursors(cursorCount)) + { + } + + public int? Read(int cursorIx) + { + try + { + return Read(Cursors.Cursors.ElementAt(cursorIx)); + } + catch (NothingToReadException) + { + return null; + } + } + } + + private class SimpleCursors : ICursors + { + public SimpleCursors(IEnumerable cursors) + { + Cursors = cursors; + } + + public SimpleCursors(int cursorCount) + { + Cursors = Enumerable.Range(0, cursorCount).Select(_ => new SimpleCursor()).ToList(); + } + + public IEnumerable Cursors { get; } + } + + private class SimpleCursor : ICursor + { + public long Cursor { get; set; } + } + + private class StressTestCursor : ICursor + { + private readonly int _cursorNr; + private readonly int _run; + private readonly Action _log; + private readonly int _counterLimit; + private readonly StringBuilder _sb; + private int _counter = 1; + + public StressTestCursor(int cursorNr, int run, Action log, int counterLimit, StringBuilder sb) + { + _cursorNr = cursorNr; + _run = run; + _log = log; + _counterLimit = counterLimit; + _sb = sb; + } + + public bool TryReadAndReturnTrueIfDone(TestBuffer buf) + { + _log($" Try reading of {this}: "); + try + { + var x = buf.Read(this); + _log("OK\n"); + if (x != _counter) + { + throw new AssertionFailedException( + $@"|Run {_run}, cursorNr {_cursorNr}, counter {_counter}: got unexpected {x} + | Buf: {buf.Inspect()} + | Cursors: {buf.UnderlyingCursors.Cursors.Aggregate(" ", (s, cursor) => s + cursor + "\n ")} + |Log: {_sb} + "); + } + _counter++; + return _counter == _counterLimit; + } + catch (NothingToReadException) + { + _log("FAILED\n"); + return false; // ok, we currently can't read, try again later + } + } + + public long Cursor { get; set; } + + public override string ToString() => $"cursorNr {_cursorNr}, ix {Cursor}, counter {_counter}"; + } + } +} diff --git a/src/core/Akka.Streams.Tests/Implementation/ResizableMultiReaderRingBufferSpec.cs b/src/core/Akka.Streams.Tests/Implementation/ResizableMultiReaderRingBufferSpec.cs index bdaebbdf75b..f04ac789ace 100644 --- a/src/core/Akka.Streams.Tests/Implementation/ResizableMultiReaderRingBufferSpec.cs +++ b/src/core/Akka.Streams.Tests/Implementation/ResizableMultiReaderRingBufferSpec.cs @@ -244,7 +244,7 @@ public SimpleCursors(int cursorCount) private class SimpleCursor : ICursor { - public int Cursor { get; set; } + public long Cursor { get; set; } } private class StressTestCursor : ICursor @@ -291,7 +291,7 @@ public bool TryReadAndReturnTrueIfDone(TestBuffer buf) } } - public int Cursor { get; set; } + public long Cursor { get; set; } public override string ToString() => $"cursorNr {_cursorNr}, ix {Cursor}, counter {_counter}"; } diff --git a/src/core/Akka.Streams.Tests/Implementation/RetainingMultiReaderBufferSpec.cs b/src/core/Akka.Streams.Tests/Implementation/RetainingMultiReaderBufferSpec.cs new file mode 100644 index 00000000000..64a9e6b9761 --- /dev/null +++ b/src/core/Akka.Streams.Tests/Implementation/RetainingMultiReaderBufferSpec.cs @@ -0,0 +1,249 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; +using Akka.Streams.Implementation; +using FluentAssertions; +using FluentAssertions.Execution; +using Xunit; + +namespace Akka.Streams.Tests.Implementation +{ + public class RetainingMultiReaderBufferSpec + { + [Theory] + [InlineData(2, 4, 1, "0 0 (size=0, cursors=1)")] + [InlineData(4, 4, 3, "0 0 0 0 (size=0, cursors=3)")] + public void A_RetainingMultiReaderBufferSpec_should_initially_be_empty(int iSize, int mSize, int cursorCount, string expected) + { + var test = new Test(iSize, mSize, cursorCount); + test.Inspect().Should().Be(expected); + } + + [Fact] + public void A_RetainingMultiReaderBufferSpec_should_fail_reads_if_nothing_can_be_read() + { + var test = new Test(4, 4, 3); + test.Write(1).Should().BeTrue(); + test.Write(2).Should().BeTrue(); + test.Write(3).Should().BeTrue(); + test.Inspect().Should().Be("1 2 3 0 (size=3, cursors=3)"); + test.Read(0).Should().Be(1); + test.Read(0).Should().Be(2); + test.Read(1).Should().Be(1); + test.Inspect().Should().Be("1 2 3 0 (size=3, cursors=3)"); + test.Read(0).Should().Be(3); + test.Read(0).Should().Be(null); + test.Read(1).Should().Be(2); + test.Inspect().Should().Be("1 2 3 0 (size=3, cursors=3)"); + test.Read(2).Should().Be(1); + test.Inspect().Should().Be("1 2 3 0 (size=3, cursors=3)"); + test.Read(1).Should().Be(3); + test.Read(1).Should().Be(null); + test.Read(2).Should().Be(2); + test.Read(2).Should().Be(3); + test.Inspect().Should().Be("1 2 3 0 (size=3, cursors=3)"); + } + + [Fact] + public void A_RetainingMultiReaderBufferSpec_should_automatically_grow_if_possible() + { + var test = new Test(2, 8, 2); + test.Write(1).Should().BeTrue(); + test.Inspect().Should().Be("1 0 (size=1, cursors=2)"); + test.Write(2).Should().BeTrue(); + test.Inspect().Should().Be("1 2 (size=2, cursors=2)"); + test.Write(3).Should().BeTrue(); + test.Inspect().Should().Be("1 2 3 0 (size=3, cursors=2)"); + test.Write(4).Should().BeTrue(); + test.Inspect().Should().Be("1 2 3 4 (size=4, cursors=2)"); + test.Read(0).Should().Be(1); + test.Read(0).Should().Be(2); + test.Read(0).Should().Be(3); + test.Read(1).Should().Be(1); + test.Read(1).Should().Be(2); + test.Write(5).Should().BeTrue(); + test.Inspect().Should().Be("1 2 3 4 5 0 0 0 (size=5, cursors=2)"); + test.Write(6).Should().BeTrue(); + test.Inspect().Should().Be("1 2 3 4 5 6 0 0 (size=6, cursors=2)"); + test.Write(7).Should().BeTrue(); + test.Inspect().Should().Be("1 2 3 4 5 6 7 0 (size=7, cursors=2)"); + test.Read(0).Should().Be(4); + test.Read(0).Should().Be(5); + test.Read(0).Should().Be(6); + test.Read(0).Should().Be(7); + test.Read(0).Should().Be(null); + test.Read(1).Should().Be(3); + test.Read(1).Should().Be(4); + test.Read(1).Should().Be(5); + test.Read(1).Should().Be(6); + test.Read(1).Should().Be(7); + test.Read(1).Should().Be(null); + test.Inspect().Should().Be("1 2 3 4 5 6 7 0 (size=7, cursors=2)"); + } + + [Fact] + public void A_RetainingMultiReaderBufferSpec_should_pass_the_stress_test() + { + // create 100 buffers with an initialSize of 1 and a maxSize of 1 to 64, + // for each one attach 1 to 8 cursors and randomly try reading and writing to the buffer; + // in total 200 elements need to be written to the buffer and read in the correct order by each cursor + var MAXSIZEBIT_LIMIT = 6; // 2 ^ (this number) + var COUNTER_LIMIT = 200; + var LOG = false; + var sb = new StringBuilder(); + var log = new Action(s => + { + if (LOG) + sb.Append(s); + }); + + var random = new Random(); + for (var bit = 1; bit <= MAXSIZEBIT_LIMIT; bit++) + for (var n = 1; n <= 2; n++) + { + var counter = 1; + var activeCoursors = + Enumerable.Range(0, random.Next(8) + 1) + .Select(i => new StressTestCursor(i, 1 << bit, log, COUNTER_LIMIT, sb)) + .ToList(); + var stillWriting = 2;// give writing a slight bias, so as to somewhat "stretch" the buffer + var buf = new TestBuffer(1, 1 << bit, new SimpleCursors(activeCoursors)); + sb.Clear(); + + while (activeCoursors.Count != 0) + { + log($"Buf: {buf.Inspect()}\n"); + var activeCursorCount = activeCoursors.Count; + var index = random.Next(activeCursorCount + stillWriting); + if (index >= activeCursorCount) + { + log($" Writing {counter}: "); + if (buf.Write(counter)) + { + log("OK\n"); + counter++; + } + else + { + log("FAILED\n"); + if (counter == COUNTER_LIMIT) + stillWriting = 0; + } + } + else + { + var cursor = activeCoursors[index]; + if (cursor.TryReadAndReturnTrueIfDone(buf)) + activeCoursors = activeCoursors.Where(c => c != cursor).ToList(); + } + } + } + } + + private class TestBuffer : RetainingMultiReaderBuffer + { + public ICursors UnderlyingCursors { get; } + + public TestBuffer(int initialSize, int maxSize, ICursors cursors) : base(initialSize, maxSize, cursors) + { + UnderlyingCursors = cursors; + } + + public string Inspect() + { + return Buffer.Select(x => x ?? 0).Aggregate("", (s, i) => s + i + " ") + + ToString().SkipWhile(c => c != '(').Aggregate("", (s, c) => s + c); + } + } + + private class Test : TestBuffer + { + public Test(int initialSize, int maxSize, int cursorCount) : base(initialSize, maxSize, new SimpleCursors(cursorCount)) + { + } + + public int? Read(int cursorIx) + { + try + { + return Read(Cursors.Cursors.ElementAt(cursorIx)); + } + catch (NothingToReadException) + { + return null; + } + } + } + + private class SimpleCursors : ICursors + { + public SimpleCursors(IEnumerable cursors) + { + Cursors = cursors; + } + + public SimpleCursors(int cursorCount) + { + Cursors = Enumerable.Range(0, cursorCount).Select(_ => new SimpleCursor()).ToList(); + } + + public IEnumerable Cursors { get; } + } + + private class SimpleCursor : ICursor + { + public long Cursor { get; set; } + } + + private class StressTestCursor : ICursor + { + private readonly int _cursorNr; + private readonly int _run; + private readonly Action _log; + private readonly int _counterLimit; + private readonly StringBuilder _sb; + private int _counter = 1; + + public StressTestCursor(int cursorNr, int run, Action log, int counterLimit, StringBuilder sb) + { + _cursorNr = cursorNr; + _run = run; + _log = log; + _counterLimit = counterLimit; + _sb = sb; + } + + public bool TryReadAndReturnTrueIfDone(TestBuffer buf) + { + _log($" Try reading of {this}: "); + try + { + var x = buf.Read(this); + _log("OK\n"); + if (x != _counter) + { + throw new AssertionFailedException( + $@"|Run {_run}, cursorNr {_cursorNr}, counter {_counter}: got unexpected {x} + | Buf: {buf.Inspect()} + | Cursors: {buf.UnderlyingCursors.Cursors.Aggregate(" ", (s, cursor) => s + cursor + "\n ")} + |Log: {_sb} + "); + } + _counter++; + return _counter == _counterLimit; + } + catch (NothingToReadException) + { + _log("FAILED\n"); + return false; // ok, we currently can't read, try again later + } + } + + public long Cursor { get; set; } + + public override string ToString() => $"cursorNr {_cursorNr}, ix {Cursor}, counter {_counter}"; + } + } +} diff --git a/src/core/Akka.Streams/Dsl/Sink.cs b/src/core/Akka.Streams/Dsl/Sink.cs index a7b400a8530..33073899db2 100644 --- a/src/core/Akka.Streams/Dsl/Sink.cs +++ b/src/core/Akka.Streams/Dsl/Sink.cs @@ -282,7 +282,10 @@ public static Sink> Publisher() /// TBD /// TBD public static Sink> FanoutPublisher() - => new Sink>(new FanoutPublisherSink(DefaultAttributes.FanoutPublisherSink, Shape("FanoutPublisherSink"))); + => new Sink>(new FanoutPublisherSink>(DefaultAttributes.FanoutPublisherSink, Shape("FanoutPublisherSink"))); + + internal static Sink> DistinctRetainingFanOutPublisher() + => new Sink>(new FanoutPublisherSink>(DefaultAttributes.FanoutPublisherSink, Shape("DistinctRetainingFanOutPublisherSink"))); /// /// A that will consume the stream and discard the elements. @@ -592,7 +595,7 @@ public static Sink> AsPublisher(bool fanout) { SinkModule> publisherSink; if (fanout) - publisherSink = new FanoutPublisherSink(DefaultAttributes.FanoutPublisherSink, Shape("FanoutPublisherSink")); + publisherSink = new FanoutPublisherSink>(DefaultAttributes.FanoutPublisherSink, Shape("FanoutPublisherSink")); else publisherSink = new PublisherSink(DefaultAttributes.PublisherSink, Shape("PublisherSink")); diff --git a/src/core/Akka.Streams/Implementation/ActorPublisher.cs b/src/core/Akka.Streams/Implementation/ActorPublisher.cs index f6b6e9dfbac..b8a273e64b1 100644 --- a/src/core/Akka.Streams/Implementation/ActorPublisher.cs +++ b/src/core/Akka.Streams/Implementation/ActorPublisher.cs @@ -390,7 +390,7 @@ bool ISubscriptionWithCursor.IsActive /// /// TBD /// - public int Cursor { get; private set; } + public long Cursor { get; private set; } long ISubscriptionWithCursor.TotalDemand { @@ -409,7 +409,7 @@ long ISubscriptionWithCursor.TotalDemand /// TBD public void Dispatch(TIn element) => ReactiveStreamsCompliance.TryOnNext(Subscriber, element); - int ICursor.Cursor + long ICursor.Cursor { get { return Cursor; } set { Cursor = value; } diff --git a/src/core/Akka.Streams/Implementation/FanoutProcessorImpl.cs b/src/core/Akka.Streams/Implementation/FanoutProcessorImpl.cs index 4f7312f341c..81dd495d605 100644 --- a/src/core/Akka.Streams/Implementation/FanoutProcessorImpl.cs +++ b/src/core/Akka.Streams/Implementation/FanoutProcessorImpl.cs @@ -17,7 +17,8 @@ namespace Akka.Streams.Implementation /// TBD /// /// TBD - internal class FanoutOutputs : SubscriberManagement, IOutputs + /// TBD + internal class FanoutOutputs : SubscriberManagement, IOutputs where TStreamBuffer : IStreamBuffer { private long _downstreamBufferSpace; private bool _downstreamCompleted; @@ -87,8 +88,7 @@ public FanoutOutputs(int maxBufferSize, int initialBufferSize, IActorRef self, I NeedsDemandOrCancel = DefaultOutputTransferStates.NeedsDemandOrCancel(this); SubReceive = new SubReceive(message => { - var publisher = message as ExposedPublisher; - if (publisher == null) + if (!(message is ExposedPublisher publisher)) throw new IllegalStateException($"The first message must be ExposedPublisher but was {message}"); ExposedPublisher = publisher.Publisher; @@ -112,24 +112,22 @@ protected override ISubscriptionWithCursor CreateSubscription(ISubscriber /// TBD protected bool DownstreamRunning(object message) { - if (message is SubscribePending) - SubscribePending(); - else if (message is RequestMore) + switch (message) { - var requestMore = (RequestMore) message; - MoreRequested((ActorSubscriptionWithCursor) requestMore.Subscription, requestMore.Demand); - _pump.Pump(); + case SubscribePending _: + SubscribePending(); + return true; + case RequestMore requestMore: + MoreRequested((ActorSubscriptionWithCursor) requestMore.Subscription, requestMore.Demand); + _pump.Pump(); + return true; + case Cancel cancel: + UnregisterSubscription((ActorSubscriptionWithCursor) cancel.Subscription); + _pump.Pump(); + return true; + default: + return false; } - else if (message is Cancel) - { - var cancel = (Cancel) message; - UnregisterSubscription((ActorSubscriptionWithCursor) cancel.Subscription); - _pump.Pump(); - } - else - return false; - - return true; } /// @@ -217,7 +215,8 @@ public void Error(Exception e) /// TBD /// /// TBD - internal sealed class FanoutProcessorImpl : ActorProcessorImpl + /// TBD + internal sealed class FanoutProcessorImpl : ActorProcessorImpl where TStreamBuffer : IStreamBuffer { /// /// TBD @@ -225,7 +224,7 @@ internal sealed class FanoutProcessorImpl : ActorProcessorImpl /// TBD /// TBD public static Props Props(ActorMaterializerSettings settings) - => Actor.Props.Create(() => new FanoutProcessorImpl(settings)).WithDeploy(Deploy.Local); + => Actor.Props.Create(() => new FanoutProcessorImpl(settings)).WithDeploy(Deploy.Local); /// /// TBD @@ -238,7 +237,7 @@ public static Props Props(ActorMaterializerSettings settings) /// TBD public FanoutProcessorImpl(ActorMaterializerSettings settings) : base(settings) { - PrimaryOutputs = new FanoutOutputs(settings.MaxInputBufferSize, + PrimaryOutputs = new FanoutOutputs(settings.MaxInputBufferSize, settings.InitialInputBufferSize, Self, this, AfterFlush); var running = new TransferPhase(PrimaryInputs.NeedsInput.And(PrimaryOutputs.NeedsDemand), diff --git a/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs b/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs index ccdf061e711..a4909200451 100644 --- a/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs +++ b/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs @@ -8,7 +8,6 @@ using System; using System.Collections.Generic; using System.Linq; -using System.Runtime.Serialization; using Akka.Annotations; using Akka.Streams.Util; @@ -61,7 +60,135 @@ public interface ICursor /// /// TBD /// - int Cursor { get; set; } + long Cursor { get; set; } + } + + public interface IStreamBuffer + { + bool IsEmpty { get; } + + long Length { get; } + + long CapacityLeft { get; } + + long Count(ICursor cursor); + + T Read(ICursor cursor); + + bool Write(T value); + + void InitCursor(ICursor cursor); + + void OnCursorRemoved(ICursor cursor); + } + + public class DistinctRetainingMultiReaderBuffer : RetainingMultiReaderBuffer + { + public DistinctRetainingMultiReaderBuffer(long initialSize, long maxSize, ICursors cursors) : base(initialSize, maxSize, cursors) + { } + + public override bool Write(T value) + { + return Buffer.Contains(value) || base.Write(value); + } + + /// + /// TBD + /// + /// TBD + public override string ToString() => $"DistinctRetainingMultiReaderBuffer(size={Length}, cursors={Cursors.Cursors.Count()})"; + } + + public class RetainingMultiReaderBuffer : IStreamBuffer + { + /// + /// TBD + /// + protected readonly ICursors Cursors; + + protected T[] Buffer { get; private set; } + + /// + /// The number of elements currently in the buffer. + /// + public long Length { get; private set; } + + public bool IsEmpty => Buffer.LongLength == 0; + + /// + /// The maximum number of elements the buffer can still take. + /// + public long CapacityLeft => long.MaxValue - Length; + + // DO NOT REMOVE maxSize parameter, the parameters are fixed and passed through reflection + public RetainingMultiReaderBuffer(long initialSize, long maxSize, ICursors cursors) + { + Cursors = cursors; + + if ((initialSize & (initialSize - 1)) != 0 || initialSize <= 0) + throw new ArgumentException("initialSize must be a power of 2 that is > 0"); + + // We don't care about the maximum size + Buffer = new T[initialSize]; + } + + /// + /// Returns the number of elements that the buffer currently contains for the given cursor. + /// + /// TBD + /// TBD + public long Count(ICursor cursor) => Length - cursor.Cursor; + + public T Read(ICursor cursor) + { + var c = cursor.Cursor; + if (c < Length) + { + cursor.Cursor++; + return Buffer[c]; + } + + throw NothingToReadException.Instance; + } + + public virtual bool Write(T value) + { + if (Length < Buffer.Length) + { + // if we have space left we can simply write and be done + Buffer[Length] = value; + Length++; + return true; + } + + if (Buffer.LongLength >= long.MaxValue) return false; + + // if we are full but can grow we do so + // Array.Resize() does not work here, because it is limited to int.MaxValue + var newLength = unchecked(Buffer.LongLength << 1); + if (newLength < 0) + newLength = long.MaxValue; + var newArray = new T[newLength]; + + Array.Copy(Buffer, newArray, Buffer.LongLength); + Buffer = newArray; + Buffer[Length] = value; + Length++; + return true; + } + + public void InitCursor(ICursor cursor) => cursor.Cursor = 0; + + public void OnCursorRemoved(ICursor cursor) + { + // no op + } + + /// + /// TBD + /// + /// TBD + public override string ToString() => $"RetainingMultiReaderBuffer(size={Length}, cursors={Cursors.Cursors.Count()})"; } /// @@ -72,27 +199,27 @@ public interface ICursor /// /// TBD [InternalApi] - public class ResizableMultiReaderRingBuffer + public class ResizableMultiReaderRingBuffer : IStreamBuffer { private readonly int _maxSizeBit; - private object[] _array; + private T[] _array; /// /// Two counters counting the number of elements ever written and read; wrap-around is /// handled by always looking at differences or masked values /// - private int _writeIndex; + private long _writeIndex; + + private long _readIndex; // the "oldest" of all read cursor indices, i.e. the one that is most behind - private int _readIndex; // the "oldest" of all read cursor indices, i.e. the one that is most behind - /// /// Current array.length log2, we don't keep it as an extra field because /// is a JVM intrinsic compiling down to a `BSF` instruction on x86, which is very fast on modern CPUs /// - private int LengthBit => _array.Length.NumberOfTrailingZeros(); + private int LengthBit => BitOperations.TrailingZeroCount(_array.LongLength); // bit mask for converting a cursor into an array index - private int Mask => int.MaxValue >> (31 - LengthBit); + private long Mask => long.MaxValue >> (63 - LengthBit); /// /// TBD @@ -101,7 +228,7 @@ public class ResizableMultiReaderRingBuffer /// TBD /// TBD /// TBD - public ResizableMultiReaderRingBuffer(int initialSize, int maxSize, ICursors cursors) + public ResizableMultiReaderRingBuffer(long initialSize, long maxSize, ICursors cursors) { Cursors = cursors; if ((initialSize & (initialSize - 1)) != 0 || initialSize <= 0 || initialSize > maxSize) @@ -111,8 +238,8 @@ public ResizableMultiReaderRingBuffer(int initialSize, int maxSize, ICursors cur if ((maxSize & (maxSize - 1)) != 0 || maxSize <= 0 || maxSize > int.MaxValue / 2) throw new ArgumentException("maxSize must be a power of 2 that is > 0 and < Int.MaxValue/2"); - _array = new object[initialSize]; - _maxSizeBit = maxSize.NumberOfTrailingZeros(); + _array = new T[initialSize]; + _maxSizeBit = BitOperations.TrailingZeroCount(maxSize); } /// @@ -123,12 +250,12 @@ public ResizableMultiReaderRingBuffer(int initialSize, int maxSize, ICursors cur /// /// TBD /// - protected object[] UnderlyingArray => _array; + protected T[] UnderlyingArray => _array; /// /// The number of elements currently in the buffer. /// - public int Length => _writeIndex - _readIndex; + public long Length => _writeIndex - _readIndex; /// /// TBD @@ -143,19 +270,19 @@ public ResizableMultiReaderRingBuffer(int initialSize, int maxSize, ICursors cur /// /// The number of elements the buffer can still take without having to be resized. /// - public int ImmediatelyAvailable => _array.Length - Length; + public long ImmediatelyAvailable => _array.Length - Length; /// /// The maximum number of elements the buffer can still take. /// - public int CapacityLeft => (1 << _maxSizeBit) - Length; + public long CapacityLeft => (1 << _maxSizeBit) - Length; /// /// Returns the number of elements that the buffer currently contains for the given cursor. /// /// TBD /// TBD - public int Count(ICursor cursor) => _writeIndex - cursor.Cursor; + public long Count(ICursor cursor) => _writeIndex - cursor.Cursor; /// /// Initializes the given Cursor to the oldest buffer entry that is still available. @@ -184,7 +311,12 @@ public bool Write(T value) // the growing logic is quite simple: we assemble all current buffer entries in the new array // in their natural order (removing potential wrap around) and rebase all indices to zero var r = _readIndex & Mask; - var newArray = new object[_array.Length << 1]; + + var newLength = unchecked(_array.LongLength << 1); + if (newLength < 0) + newLength = long.MaxValue; + var newArray = new T[newLength]; + Array.Copy(_array, r, newArray, 0, _array.Length - r); Array.Copy(_array, 0, newArray, _array.Length - r, r); RebaseCursors(Cursors.Cursors); @@ -219,7 +351,7 @@ public T Read(ICursor cursor) if (c - _writeIndex < 0) { cursor.Cursor += 1; - var ret = (T)_array[c & Mask]; + var ret = _array[c & Mask]; if(c == _readIndex) UpdateReadIndex(); return ret; @@ -243,12 +375,12 @@ private void UpdateReadIndex() var newReadIx = _writeIndex + MinCursor(Cursors.Cursors, 0); while (_readIndex != newReadIx) { - _array[_readIndex & Mask] = null; + _array[_readIndex & Mask] = default; _readIndex++; } } - private int MinCursor(IEnumerable remaining, int result) + private long MinCursor(IEnumerable remaining, long result) { foreach (var cursor in remaining) result = Math.Min(cursor.Cursor - _writeIndex, result); diff --git a/src/core/Akka.Streams/Implementation/Sinks.cs b/src/core/Akka.Streams/Implementation/Sinks.cs index 46e585325b5..b9314fbbdfd 100644 --- a/src/core/Akka.Streams/Implementation/Sinks.cs +++ b/src/core/Akka.Streams/Implementation/Sinks.cs @@ -205,7 +205,8 @@ public override object Create(MaterializationContext context, out IPublisher /// TBD - internal sealed class FanoutPublisherSink : SinkModule> + /// TBD + internal sealed class FanoutPublisherSink : SinkModule> where TStreamBuffer : IStreamBuffer { /// /// TBD @@ -228,7 +229,7 @@ public FanoutPublisherSink(Attributes attributes, SinkShape shape) : base(s /// TBD /// TBD public override IModule WithAttributes(Attributes attributes) - => new FanoutPublisherSink(attributes, AmendShape(attributes)); + => new FanoutPublisherSink(attributes, AmendShape(attributes)); /// /// TBD @@ -236,7 +237,7 @@ public override IModule WithAttributes(Attributes attributes) /// TBD /// TBD protected override SinkModule> NewInstance(SinkShape shape) - => new FanoutPublisherSink(Attributes, shape); + => new FanoutPublisherSink(Attributes, shape); /// /// TBD @@ -248,7 +249,7 @@ public override object Create(MaterializationContext context, out IPublisher.Props(settings)); + var impl = actorMaterializer.ActorOf(context, FanoutProcessorImpl.Props(settings)); var fanoutProcessor = new ActorProcessor(impl); impl.Tell(new ExposedPublisher(fanoutProcessor)); // Resolve cyclic dependency with actor. This MUST be the first message no matter what. diff --git a/src/core/Akka.Streams/Implementation/SubscriberManagement.cs b/src/core/Akka.Streams/Implementation/SubscriberManagement.cs index 7bdeaca776f..9706c4b48cf 100644 --- a/src/core/Akka.Streams/Implementation/SubscriberManagement.cs +++ b/src/core/Akka.Streams/Implementation/SubscriberManagement.cs @@ -37,7 +37,7 @@ internal interface ISubscriptionWithCursor : ISubscription, ICursor bool IsActive { get; set; } /// - /// Do not increment directly, use instead (it provides overflow protection)! + /// Do not increment directly, use instead (it provides overflow protection)! /// long TotalDemand { get; set; } // number of requested but not yet dispatched elements } @@ -143,9 +143,10 @@ public ErrorCompleted(Exception cause) /// TBD /// /// TBD - internal abstract class SubscriberManagement : ICursors + /// TBD + internal abstract class SubscriberManagement : ICursors where TStreamBuffer : IStreamBuffer { - private readonly Lazy> _buffer; + private readonly Lazy> _buffer; // optimize for small numbers of subscribers by keeping subscribers in a plain list private ICollection> _subscriptions = new List>(); @@ -161,8 +162,8 @@ internal abstract class SubscriberManagement : ICursors /// protected SubscriberManagement() { - _buffer = new Lazy>(() => - new ResizableMultiReaderRingBuffer(InitialBufferSize, MaxBufferSize, this)); + _buffer = new Lazy>(() + => (IStreamBuffer) Activator.CreateInstance(typeof(TStreamBuffer), InitialBufferSize, MaxBufferSize, this)); } /// @@ -213,40 +214,39 @@ protected SubscriberManagement() /// TBD protected void MoreRequested(ISubscriptionWithCursor subscription, long elements) { - if (subscription.IsActive) + if (!subscription.IsActive) return; + + // check for illegal demand See 3.9 + if (elements < 1) { - // check for illegal demand See 3.9 - if (elements < 1) + try { - try - { - ReactiveStreamsCompliance.TryOnError(subscription.Subscriber, ReactiveStreamsCompliance.NumberOfElementsInRequestMustBePositiveException); - } - finally + ReactiveStreamsCompliance.TryOnError(subscription.Subscriber, ReactiveStreamsCompliance.NumberOfElementsInRequestMustBePositiveException); + } + finally + { + UnregisterSubscriptionInternal(subscription); + } + } + else + { + if (_endOfStream is SubscriberManagement.NotReached || _endOfStream is SubscriberManagement.Completed) + { + var d = subscription.TotalDemand + elements; + // Long overflow, Reactive Streams Spec 3:17: effectively unbounded + var demand = d < 1 ? long.MaxValue : d; + subscription.TotalDemand = demand; + // returns Long.MinValue if the subscription is to be terminated + var remainingRequested = DispatchFromBufferAndReturnRemainingRequested(demand, subscription, _endOfStream); + if (remainingRequested == long.MinValue) { + _endOfStream.Apply(subscription.Subscriber); UnregisterSubscriptionInternal(subscription); } - } - else - { - if (_endOfStream is SubscriberManagement.NotReached || _endOfStream is SubscriberManagement.Completed) + else { - var d = subscription.TotalDemand + elements; - // Long overflow, Reactive Streams Spec 3:17: effectively unbounded - var demand = d < 1 ? long.MaxValue : d; - subscription.TotalDemand = demand; - // returns Long.MinValue if the subscription is to be terminated - var remainingRequested = DispatchFromBufferAndReturnRemainingRequested(demand, subscription, _endOfStream); - if (remainingRequested == long.MinValue) - { - _endOfStream.Apply(subscription.Subscriber); - UnregisterSubscriptionInternal(subscription); - } - else - { - subscription.TotalDemand = remainingRequested; - RequestFromUpstreamIfRequired(); - } + subscription.TotalDemand = remainingRequested; + RequestFromUpstreamIfRequired(); } } } @@ -313,9 +313,10 @@ protected void PushToDownstream(T value) if (_endOfStream is SubscriberManagement.NotReached) { _pendingFromUpstream--; + var oldBufferLength = _buffer.Value.Length; if (!_buffer.Value.Write(value)) throw new IllegalStateException("Output buffer overflow"); - if (Dispatch(_subscriptions)) + if (_buffer.Value.Length > oldBufferLength && Dispatch(_subscriptions)) RequestFromUpstreamIfRequired(); } else throw new IllegalStateException("PushToDownStream(...) after CompleteDownstream() or AbortDownstream(...)"); diff --git a/src/core/Akka.Streams/Properties/AssemblyInfo.cs b/src/core/Akka.Streams/Properties/AssemblyInfo.cs index ff5be28a670..ad09773634f 100644 --- a/src/core/Akka.Streams/Properties/AssemblyInfo.cs +++ b/src/core/Akka.Streams/Properties/AssemblyInfo.cs @@ -12,6 +12,7 @@ // General Information about an assembly is controlled through the following // set of attributes. Change these attribute values to modify the information // associated with an assembly. +[assembly: InternalsVisibleTo("Akka.Persistence.Query.Sql")] [assembly: InternalsVisibleTo("Akka.Streams.Tests")] [assembly: InternalsVisibleTo("Akka.Streams.TestKit")] [assembly: InternalsVisibleTo("Akka.Benchmarks")] diff --git a/src/core/Akka.Streams/Util/BitOperations.cs b/src/core/Akka.Streams/Util/BitOperations.cs new file mode 100644 index 00000000000..47cb3510d90 --- /dev/null +++ b/src/core/Akka.Streams/Util/BitOperations.cs @@ -0,0 +1,279 @@ +// Licensed to the .NET Foundation under one or more agreements. +// The .NET Foundation licenses this file to you under the MIT license. + +using System; +using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; + +// Some routines inspired by the Stanford Bit Twiddling Hacks by Sean Eron Anderson: +// http://graphics.stanford.edu/~seander/bithacks.html +namespace Akka.Streams.Util +{ + // TODO: replace this with the official System.Numerics.BitOperations when we move on to .NET Core 3.0 + /// + /// Utility methods for intrinsic bit-twiddling operations. + /// + /// A copy of Microsoft .NET core 3.0 implementation, without the hardware optimization + /// + internal static class BitOperations + { + // C# no-alloc optimization that directly wraps the data section of the dll (similar to string constants) + // https://github.com/dotnet/roslyn/pull/24621 + + private static ReadOnlySpan TrailingZeroCountDeBruijn => new byte[32] + { + 00, 01, 28, 02, 29, 14, 24, 03, + 30, 22, 20, 15, 25, 17, 04, 08, + 31, 27, 13, 23, 21, 19, 16, 07, + 26, 12, 18, 06, 11, 05, 10, 09 + }; + + private static ReadOnlySpan Log2DeBruijn => new byte[32] + { + 00, 09, 01, 10, 13, 21, 02, 29, + 11, 14, 16, 18, 22, 25, 03, 30, + 08, 12, 20, 28, 15, 17, 24, 07, + 19, 27, 23, 06, 26, 05, 04, 31 + }; + + /// + /// Count the number of leading zero bits in a mask. + /// Similar in behavior to the x86 instruction LZCNT. + /// + /// The value. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LeadingZeroCount(uint value) + { + // Unguarded fallback contract is 0->31, BSR contract is 0->undefined + if (value == 0) + return 32; + + return 31 ^ Log2SoftwareFallback(value); + } + + /// + /// Count the number of leading zero bits in a mask. + /// Similar in behavior to the x86 instruction LZCNT. + /// + /// The value. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LeadingZeroCount(ulong value) + { + var hi = (uint)(value >> 32); + + if (hi == 0) + return 32 + LeadingZeroCount((uint)value); + + return LeadingZeroCount(hi); + } + + /// + /// Returns the integer (floor) log of the specified value, base 2. + /// Note that by convention, input value 0 returns 0 since log(0) is undefined. + /// + /// The value. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Log2(uint value) + { + // The 0->0 contract is fulfilled by setting the LSB to 1. + // Log(1) is 0, and setting the LSB for values > 1 does not change the log2 result. + value |= 1; + + // value lzcnt actual expected + // ..0001 31 31-31 0 + // ..0010 30 31-30 1 + // 0010.. 2 31-2 29 + // 0100.. 1 31-1 30 + // 1000.. 0 31-0 31 + + // Fallback contract is 0->0 + return Log2SoftwareFallback(value); + } + + /// + /// Returns the integer (floor) log of the specified value, base 2. + /// Note that by convention, input value 0 returns 0 since log(0) is undefined. + /// + /// The value. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Log2(ulong value) + { + value |= 1; + + var hi = (uint)(value >> 32); + + if (hi == 0) + return Log2((uint)value); + + return 32 + Log2(hi); + } + + /// + /// Returns the integer (floor) log of the specified value, base 2. + /// Note that by convention, input value 0 returns 0 since Log(0) is undefined. + /// Does not directly use any hardware intrinsics, nor does it incur branching. + /// + /// The value. + private static int Log2SoftwareFallback(uint value) + { + // No AggressiveInlining due to large method size + // Has conventional contract 0->0 (Log(0) is undefined) + + // Fill trailing zeros with ones, eg 00010010 becomes 00011111 + value |= value >> 01; + value |= value >> 02; + value |= value >> 04; + value |= value >> 08; + value |= value >> 16; + + // uint.MaxValue >> 27 is always in range [0 - 31] so we use Unsafe.AddByteOffset to avoid bounds check + return Unsafe.AddByteOffset( + // Using deBruijn sequence, k=2, n=5 (2^5=32) : 0b_0000_0111_1100_0100_1010_1100_1101_1101u + ref MemoryMarshal.GetReference(Log2DeBruijn), + // uint|long -> IntPtr cast on 32-bit platforms does expensive overflow checks not needed here + (IntPtr)(int)((value * 0x07C4ACDDu) >> 27)); + } + + /// + /// Returns the population count (number of bits set) of a mask. + /// Similar in behavior to the x86 instruction POPCNT. + /// + /// The value. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int PopCount(uint value) + { + const uint c1 = 0x_55555555u; + const uint c2 = 0x_33333333u; + const uint c3 = 0x_0F0F0F0Fu; + const uint c4 = 0x_01010101u; + + value -= (value >> 1) & c1; + value = (value & c2) + ((value >> 2) & c2); + value = (((value + (value >> 4)) & c3) * c4) >> 24; + + return (int)value; + } + + /// + /// Returns the population count (number of bits set) of a mask. + /// Similar in behavior to the x86 instruction POPCNT. + /// + /// The value. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int PopCount(ulong value) + { + const ulong c1 = 0x_55555555_55555555ul; + const ulong c2 = 0x_33333333_33333333ul; + const ulong c3 = 0x_0F0F0F0F_0F0F0F0Ful; + const ulong c4 = 0x_01010101_01010101ul; + + value -= (value >> 1) & c1; + value = (value & c2) + ((value >> 2) & c2); + value = (((value + (value >> 4)) & c3) * c4) >> 56; + + return (int)value; + } + + /// + /// Count the number of trailing zero bits in an integer value. + /// Similar in behavior to the x86 instruction TZCNT. + /// + /// The value. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int TrailingZeroCount(int value) + => TrailingZeroCount((uint)value); + + /// + /// Count the number of trailing zero bits in an integer value. + /// Similar in behavior to the x86 instruction TZCNT. + /// + /// The value. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int TrailingZeroCount(uint value) + { + // Unguarded fallback contract is 0->0, BSF contract is 0->undefined + if (value == 0) + return 32; + + // uint.MaxValue >> 27 is always in range [0 - 31] so we use Unsafe.AddByteOffset to avoid bounds check + return Unsafe.AddByteOffset( + // Using deBruijn sequence, k=2, n=5 (2^5=32) : 0b_0000_0111_0111_1100_1011_0101_0011_0001u + ref MemoryMarshal.GetReference(TrailingZeroCountDeBruijn), + // uint|long -> IntPtr cast on 32-bit platforms does expensive overflow checks not needed here + (IntPtr)(int)(((value & (uint)-(int)value) * 0x077CB531u) >> 27)); // Multi-cast mitigates redundant conv.u8 + } + + /// + /// Count the number of trailing zero bits in a mask. + /// Similar in behavior to the x86 instruction TZCNT. + /// + /// The value. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int TrailingZeroCount(long value) + => TrailingZeroCount((ulong)value); + + /// + /// Count the number of trailing zero bits in a mask. + /// Similar in behavior to the x86 instruction TZCNT. + /// + /// The value. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int TrailingZeroCount(ulong value) + { + var lo = (uint)value; + + if (lo == 0) + return 32 + TrailingZeroCount((uint)(value >> 32)); + + return TrailingZeroCount(lo); + } + + /// + /// Rotates the specified value left by the specified number of bits. + /// Similar in behavior to the x86 instruction ROL. + /// + /// The value to rotate. + /// The number of bits to rotate by. + /// Any value outside the range [0..31] is treated as congruent mod 32. + /// The rotated value. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint RotateLeft(uint value, int offset) + => (value << offset) | (value >> (32 - offset)); + + /// + /// Rotates the specified value left by the specified number of bits. + /// Similar in behavior to the x86 instruction ROL. + /// + /// The value to rotate. + /// The number of bits to rotate by. + /// Any value outside the range [0..63] is treated as congruent mod 64. + /// The rotated value. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ulong RotateLeft(ulong value, int offset) + => (value << offset) | (value >> (64 - offset)); + + /// + /// Rotates the specified value right by the specified number of bits. + /// Similar in behavior to the x86 instruction ROR. + /// + /// The value to rotate. + /// The number of bits to rotate by. + /// Any value outside the range [0..31] is treated as congruent mod 32. + /// The rotated value. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint RotateRight(uint value, int offset) + => (value >> offset) | (value << (32 - offset)); + + /// + /// Rotates the specified value right by the specified number of bits. + /// Similar in behavior to the x86 instruction ROR. + /// + /// The value to rotate. + /// The number of bits to rotate by. + /// Any value outside the range [0..63] is treated as congruent mod 64. + /// The rotated value. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ulong RotateRight(ulong value, int offset) + => (value >> offset) | (value << (64 - offset)); + } +} From a7e2e5ffe6384ab21b9fc5f3a9ada95d7e740bbc Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Wed, 22 Jul 2020 02:26:20 +0700 Subject: [PATCH 03/23] Fix copy-paste error --- .../Query/SqliteCurrentAllEventsSpec.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteCurrentAllEventsSpec.cs b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteCurrentAllEventsSpec.cs index dc3e0db991f..c48d678dca8 100644 --- a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteCurrentAllEventsSpec.cs +++ b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteCurrentAllEventsSpec.cs @@ -29,7 +29,7 @@ class = ""Akka.Persistence.Sqlite.Journal.SqliteJournal, Akka.Persistence.Sqlite akka.test.single-expect-default = 10s") .WithFallback(SqlReadJournal.DefaultConfiguration()); - public SqliteCurrentAllEventsSpec(ITestOutputHelper output) : base(Config(Counter.GetAndIncrement()), nameof(SqliteAllEventsSpec), output) + public SqliteCurrentAllEventsSpec(ITestOutputHelper output) : base(Config(Counter.GetAndIncrement()), nameof(SqliteCurrentAllEventsSpec), output) { ReadJournal = Sys.ReadJournalFor(SqlReadJournal.Identifier); } From 7796f2f005e4d0d0f3482e8de77409f326cf7d3d Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Thu, 30 Jul 2020 00:57:43 +0700 Subject: [PATCH 04/23] Make sure APIs are as backward compatible as possible, update API approver list. --- .../Journal/QueryExecutor.cs | 16 +++--- .../Journal/SqlJournal.cs | 9 ++-- ...c.ApprovePersistenceSqlCommon.approved.txt | 39 +++++++------- .../CoreAPISpec.ApproveStreams.approved.txt | 51 +++++++++++++++---- 4 files changed, 75 insertions(+), 40 deletions(-) diff --git a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs index c15fbe05512..bb4027d4b98 100644 --- a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs +++ b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs @@ -71,12 +71,12 @@ public interface IJournalQueryExecutor Task SelectByTagAsync(DbConnection connection, CancellationToken cancellationToken, string tag, long fromOffset, long toOffset, long max, Action callback); Task SelectAllEventsAsync( - DbConnection connection, + DbConnection connection, + CancellationToken cancellationToken, long fromOffset, long toOffset, long max, - Action callback, - CancellationToken cancellationToken); + Action callback); /// /// Asynchronously returns single number considered as the highest sequence number in current journal for the provided . @@ -85,7 +85,7 @@ Task SelectAllEventsAsync( /// TBD /// TBD /// TBD - Task SelectHighestSequenceNrAsync(DbConnection connection, string persistenceId, CancellationToken cancellationToken); + Task SelectHighestSequenceNrAsync(DbConnection connection, CancellationToken cancellationToken, string persistenceId); Task SelectHighestSequenceNrAsync(DbConnection connection, CancellationToken cancellationToken); @@ -594,12 +594,12 @@ public virtual async Task SelectByTagAsync(DbConnection connection, Cancel } public async Task SelectAllEventsAsync( - DbConnection connection, + DbConnection connection, + CancellationToken cancellationToken, long fromOffset, long toOffset, long max, - Action callback, - CancellationToken cancellationToken) + Action callback) { long maxOrdering; using (var command = GetCommand(connection, HighestOrderingSql)) @@ -639,7 +639,7 @@ public async Task SelectAllEventsAsync( /// TBD /// TBD /// TBD - public virtual async Task SelectHighestSequenceNrAsync(DbConnection connection, string persistenceId, CancellationToken cancellationToken) + public virtual async Task SelectHighestSequenceNrAsync(DbConnection connection, CancellationToken cancellationToken, string persistenceId) { using (var command = GetCommand(connection, HighestSequenceNrSql)) { diff --git a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/SqlJournal.cs b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/SqlJournal.cs index e61749279ed..72c5ae0dff9 100644 --- a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/SqlJournal.cs +++ b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/SqlJournal.cs @@ -230,7 +230,9 @@ protected virtual async Task ReplayAllEventsAsync(ReplayAllEvents replay) using (var cancellationToken = CancellationTokenSource.CreateLinkedTokenSource(_pendingRequestsCancellation.Token)) { return await QueryExecutor - .SelectAllEventsAsync(connection, + .SelectAllEventsAsync( + connection, + cancellationToken.Token, replay.FromOffset, replay.ToOffset, replay.Max, @@ -239,8 +241,7 @@ protected virtual async Task ReplayAllEventsAsync(ReplayAllEvents replay) { replay.ReplyTo.Tell(new ReplayedEvent(adapted, replayedEvent.Offset), ActorRefs.NoSender); } - }, - cancellationToken.Token); + }); } } } @@ -493,7 +494,7 @@ public override async Task ReadHighestSequenceNrAsync(string persistenceId await connection.OpenAsync(); using (var cancellationToken = CancellationTokenSource.CreateLinkedTokenSource(_pendingRequestsCancellation.Token)) { - return await QueryExecutor.SelectHighestSequenceNrAsync(connection, persistenceId, cancellationToken.Token); + return await QueryExecutor.SelectHighestSequenceNrAsync(connection, cancellationToken.Token, persistenceId); } } } diff --git a/src/core/Akka.API.Tests/CoreAPISpec.ApprovePersistenceSqlCommon.approved.txt b/src/core/Akka.API.Tests/CoreAPISpec.ApprovePersistenceSqlCommon.approved.txt index 58e46427c5f..cd950333acd 100644 --- a/src/core/Akka.API.Tests/CoreAPISpec.ApprovePersistenceSqlCommon.approved.txt +++ b/src/core/Akka.API.Tests/CoreAPISpec.ApprovePersistenceSqlCommon.approved.txt @@ -39,11 +39,12 @@ namespace Akka.Persistence.Sql.Common.Journal protected System.Data.Common.DbCommand GetCommand(System.Data.Common.DbConnection connection, string sql) { } public virtual System.Threading.Tasks.Task InsertBatchAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken, Akka.Persistence.Sql.Common.Journal.WriteJournalBatch write) { } protected virtual Akka.Persistence.IPersistentRepresentation ReadEvent(System.Data.Common.DbDataReader reader) { } - public System.Threading.Tasks.Task SelectAllEventsAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken, long fromOffset, long max, System.Action callback) { } - public virtual System.Threading.Tasks.Task> SelectAllPersistenceIdsAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken) { } + public System.Threading.Tasks.Task SelectAllEventsAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken, long fromOffset, long toOffset, long max, System.Action callback) { } + public virtual System.Threading.Tasks.Task> SelectAllPersistenceIdsAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken, long offset) { } public virtual System.Threading.Tasks.Task SelectByPersistenceIdAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken, string persistenceId, long fromSequenceNr, long toSequenceNr, long max, System.Action callback) { } public virtual System.Threading.Tasks.Task SelectByTagAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken, string tag, long fromOffset, long toOffset, long max, System.Action callback) { } public virtual System.Threading.Tasks.Task SelectHighestSequenceNrAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken, string persistenceId) { } + public virtual System.Threading.Tasks.Task SelectHighestSequenceNrAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken) { } protected virtual void WriteEvent(System.Data.Common.DbCommand command, Akka.Persistence.IPersistentRepresentation e, System.Collections.Immutable.IImmutableSet tags) { } } public sealed class AllPersistenceIds @@ -87,7 +88,6 @@ namespace Akka.Persistence.Sql.Common.Journal protected virtual string ByPersistenceIdSql { get; } protected virtual string ByTagSql { get; } protected virtual string DeleteBatchSql { get; } - protected bool HasAllIdsSubscribers { get; } protected bool HasNewEventsSubscribers { get; } protected bool HasPersistenceIdSubscribers { get; } protected bool HasTagSubscribers { get; } @@ -104,11 +104,12 @@ namespace Akka.Persistence.Sql.Common.Journal protected virtual System.Threading.Tasks.Task HandleReplayAllMessages(Akka.Persistence.Sql.Common.Journal.ReplayAllEvents req, TCommand command) { } protected virtual System.Threading.Tasks.Task HandleReplayMessages(Akka.Persistence.ReplayMessages req, TCommand command, Akka.Actor.IActorContext context) { } protected virtual System.Threading.Tasks.Task HandleReplayTaggedMessages(Akka.Persistence.Sql.Common.Journal.ReplayTaggedMessages req, TCommand command) { } - protected void NotifyNewPersistenceIdAdded(string persistenceId) { } + protected virtual System.Threading.Tasks.Task HandleSelectCurrentPersistenceIds(Akka.Persistence.Sql.Common.Journal.SelectCurrentPersistenceIds message, TCommand command) { } protected virtual void OnBufferOverflow(Akka.Persistence.IJournalMessage request) { } protected override void PreStart() { } protected virtual Akka.Persistence.IPersistentRepresentation ReadEvent(System.Data.Common.DbDataReader reader) { } protected virtual System.Threading.Tasks.Task ReadHighestSequenceNr(string persistenceId, TCommand command) { } + protected virtual System.Threading.Tasks.Task ReadHighestSequenceNr(TCommand command) { } protected virtual bool Receive(object message) { } protected virtual void WriteEvent(TCommand command, Akka.Persistence.IPersistentRepresentation persistent, string tags = "") { } } @@ -123,7 +124,8 @@ namespace Akka.Persistence.Sql.Common.Journal public sealed class CurrentPersistenceIds : Akka.Event.IDeadLetterSuppression { public readonly System.Collections.Generic.IEnumerable AllPersistenceIds; - public CurrentPersistenceIds(System.Collections.Generic.IEnumerable allPersistenceIds) { } + public readonly long HighestOrderingNumber; + public CurrentPersistenceIds(System.Collections.Generic.IEnumerable allPersistenceIds, long highestOrderingNumber) { } } public sealed class DefaultTimestampProvider : Akka.Persistence.Sql.Common.Journal.ITimestampProvider { @@ -166,11 +168,12 @@ namespace Akka.Persistence.Sql.Common.Journal System.Threading.Tasks.Task CreateTablesAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken); System.Threading.Tasks.Task DeleteBatchAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken, string persistenceId, long toSequenceNr); System.Threading.Tasks.Task InsertBatchAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken, Akka.Persistence.Sql.Common.Journal.WriteJournalBatch write); - System.Threading.Tasks.Task SelectAllEventsAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken, long fromOffset, long max, System.Action callback); - System.Threading.Tasks.Task> SelectAllPersistenceIdsAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken); + System.Threading.Tasks.Task SelectAllEventsAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken, long fromOffset, long toOffset, long max, System.Action callback); + System.Threading.Tasks.Task> SelectAllPersistenceIdsAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken, long offset); System.Threading.Tasks.Task SelectByPersistenceIdAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken, string persistenceId, long fromSequenceNr, long toSequenceNr, long max, System.Action callback); System.Threading.Tasks.Task SelectByTagAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken, string tag, long fromOffset, long toOffset, long max, System.Action callback); System.Threading.Tasks.Task SelectHighestSequenceNrAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken, string persistenceId); + System.Threading.Tasks.Task SelectHighestSequenceNrAsync(System.Data.Common.DbConnection connection, System.Threading.CancellationToken cancellationToken); } public interface ISubscriptionCommand { } public interface ITimestampProvider @@ -196,11 +199,6 @@ namespace Akka.Persistence.Sql.Common.Journal { public static Akka.Persistence.Sql.Common.Journal.NewEventAppended Instance; } - public sealed class PersistenceIdAdded : Akka.Event.IDeadLetterSuppression - { - public readonly string PersistenceId; - public PersistenceIdAdded(string persistenceId) { } - } public class QueryConfiguration { public readonly string IsDeletedColumnName; @@ -262,11 +260,15 @@ namespace Akka.Persistence.Sql.Common.Journal public readonly string Tag; public ReplayedTaggedMessage(Akka.Persistence.IPersistentRepresentation persistent, string tag, long offset) { } } + public sealed class SelectCurrentPersistenceIds : Akka.Actor.INoSerializationVerificationNeeded, Akka.Persistence.IJournalMessage, Akka.Persistence.IJournalRequest, Akka.Persistence.IPersistenceMessage + { + public SelectCurrentPersistenceIds(long offset, Akka.Actor.IActorRef replyTo) { } + public long Offset { get; } + public Akka.Actor.IActorRef ReplyTo { get; } + } public abstract class SqlJournal : Akka.Persistence.Journal.AsyncWriteJournal, Akka.Actor.IActorStash, Akka.Actor.IWithUnboundedStash, Akka.Dispatch.IRequiresMessageQueue { protected SqlJournal(Akka.Configuration.Config journalConfig) { } - public System.Collections.Generic.IEnumerable AllPersistenceIds { get; } - protected bool HasAllPersistenceIdSubscribers { get; } protected bool HasNewEventSubscribers { get; } protected bool HasPersistenceIdSubscribers { get; } protected bool HasTagSubscribers { get; } @@ -274,7 +276,6 @@ namespace Akka.Persistence.Sql.Common.Journal protected Akka.Event.ILoggingAdapter Log { get; } public abstract Akka.Persistence.Sql.Common.Journal.IJournalQueryExecutor QueryExecutor { get; } public Akka.Actor.IStash Stash { get; set; } - public void AddAllPersistenceIdSubscriber(Akka.Actor.IActorRef subscriber) { } public void AddNewEventsSubscriber(Akka.Actor.IActorRef subscriber) { } public void AddPersistenceIdSubscriber(Akka.Actor.IActorRef subscriber, string persistenceId) { } public void AddTagSubscriber(Akka.Actor.IActorRef subscriber, string tag) { } @@ -291,13 +292,13 @@ namespace Akka.Persistence.Sql.Common.Journal protected virtual System.Threading.Tasks.Task ReplayAllEventsAsync(Akka.Persistence.Sql.Common.Journal.ReplayAllEvents replay) { } public override System.Threading.Tasks.Task ReplayMessagesAsync(Akka.Actor.IActorContext context, string persistenceId, long fromSequenceNr, long toSequenceNr, long max, System.Action recoveryCallback) { } protected virtual System.Threading.Tasks.Task ReplayTaggedMessagesAsync(Akka.Persistence.Sql.Common.Journal.ReplayTaggedMessages replay) { } + [return: System.Runtime.CompilerServices.TupleElementNamesAttribute(new string[] { + "Ids", + "LastOrdering"})] + protected virtual System.Threading.Tasks.Task, long>> SelectAllPersistenceIdsAsync(long offset) { } protected bool WaitingForInitialization(object message) { } protected override System.Threading.Tasks.Task> WriteMessagesAsync(System.Collections.Generic.IEnumerable messages) { } } - public sealed class SubscribeAllPersistenceIds : Akka.Persistence.Sql.Common.Journal.ISubscriptionCommand - { - public static readonly Akka.Persistence.Sql.Common.Journal.SubscribeAllPersistenceIds Instance; - } public sealed class SubscribeNewEvents : Akka.Persistence.Sql.Common.Journal.ISubscriptionCommand { public static Akka.Persistence.Sql.Common.Journal.SubscribeNewEvents Instance; diff --git a/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt b/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt index 1400a378611..e586d76bb92 100644 --- a/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt +++ b/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt @@ -1,4 +1,5 @@ [assembly: System.Runtime.CompilerServices.InternalsVisibleToAttribute("Akka.Benchmarks")] +[assembly: System.Runtime.CompilerServices.InternalsVisibleToAttribute("Akka.Persistence.Query.Sql")] [assembly: System.Runtime.CompilerServices.InternalsVisibleToAttribute("Akka.Streams.TestKit")] [assembly: System.Runtime.CompilerServices.InternalsVisibleToAttribute("Akka.Streams.Tests")] [assembly: System.Runtime.InteropServices.ComVisibleAttribute(false)] @@ -2632,7 +2633,7 @@ namespace Akka.Streams.Implementation public class ActorSubscriptionWithCursor : Akka.Streams.Implementation.ActorSubscription, Akka.Streams.Implementation.ICursor, Reactive.Streams.ISubscription { public ActorSubscriptionWithCursor(Akka.Actor.IActorRef implementor, Reactive.Streams.ISubscriber subscriber) { } - public int Cursor { get; } + public long Cursor { get; } public bool IsActive { get; } public long TotalDemand { get; } public void Dispatch(object element) { } @@ -2750,6 +2751,12 @@ namespace Akka.Streams.Implementation protected override Akka.Streams.Stage.GraphStageLogic CreateLogic(Akka.Streams.Attributes inheritedAttributes) { } public override string ToString() { } } + public class DistinctRetainingMultiReaderBuffer : Akka.Streams.Implementation.RetainingMultiReaderBuffer + { + public DistinctRetainingMultiReaderBuffer(long initialSize, long maxSize, Akka.Streams.Implementation.ICursors cursors) { } + public override string ToString() { } + public override bool Write(T value) { } + } public sealed class EmptyModule : Akka.Streams.Implementation.Module { public static readonly Akka.Streams.Implementation.EmptyModule Instance; @@ -2962,7 +2969,7 @@ namespace Akka.Streams.Implementation public interface IActorSubscription : Reactive.Streams.ISubscription { } public interface ICursor { - int Cursor { get; set; } + long Cursor { get; set; } } public interface ICursors { @@ -3016,6 +3023,17 @@ namespace Akka.Streams.Implementation void WaitForUpstream(int waitForUpstream); } public interface ISpecViolation { } + public interface IStreamBuffer + { + long CapacityLeft { get; } + bool IsEmpty { get; } + long Length { get; } + long Count(Akka.Streams.Implementation.ICursor cursor); + void InitCursor(Akka.Streams.Implementation.ICursor cursor); + void OnCursorRemoved(Akka.Streams.Implementation.ICursor cursor); + T Read(Akka.Streams.Implementation.ICursor cursor); + bool Write(T value); + } [Akka.Annotations.InternalApiAttribute()] public sealed class IdleInject : Akka.Streams.Stage.GraphStage> where TIn : TOut @@ -3317,23 +3335,38 @@ namespace Akka.Streams.Implementation public static void TryRequest(Reactive.Streams.ISubscription subscription, long demand) { } } [Akka.Annotations.InternalApiAttribute()] - public class ResizableMultiReaderRingBuffer + public class ResizableMultiReaderRingBuffer : Akka.Streams.Implementation.IStreamBuffer { protected readonly Akka.Streams.Implementation.ICursors Cursors; - public ResizableMultiReaderRingBuffer(int initialSize, int maxSize, Akka.Streams.Implementation.ICursors cursors) { } - public int CapacityLeft { get; } - public int ImmediatelyAvailable { get; } + public ResizableMultiReaderRingBuffer(long initialSize, long maxSize, Akka.Streams.Implementation.ICursors cursors) { } + public long CapacityLeft { get; } + public long ImmediatelyAvailable { get; } public bool IsEmpty { get; } - public int Length { get; } + public long Length { get; } public bool NonEmpty { get; } - protected object[] UnderlyingArray { get; } - public int Count(Akka.Streams.Implementation.ICursor cursor) { } + protected T[] UnderlyingArray { get; } + public long Count(Akka.Streams.Implementation.ICursor cursor) { } public void InitCursor(Akka.Streams.Implementation.ICursor cursor) { } public void OnCursorRemoved(Akka.Streams.Implementation.ICursor cursor) { } public T Read(Akka.Streams.Implementation.ICursor cursor) { } public override string ToString() { } public bool Write(T value) { } } + public class RetainingMultiReaderBuffer : Akka.Streams.Implementation.IStreamBuffer + { + protected readonly Akka.Streams.Implementation.ICursors Cursors; + public RetainingMultiReaderBuffer(long initialSize, long maxSize, Akka.Streams.Implementation.ICursors cursors) { } + protected T[] Buffer { get; } + public long CapacityLeft { get; } + public bool IsEmpty { get; } + public long Length { get; } + public long Count(Akka.Streams.Implementation.ICursor cursor) { } + public void InitCursor(Akka.Streams.Implementation.ICursor cursor) { } + public void OnCursorRemoved(Akka.Streams.Implementation.ICursor cursor) { } + public T Read(Akka.Streams.Implementation.ICursor cursor) { } + public override string ToString() { } + public virtual bool Write(T value) { } + } [Akka.Annotations.InternalApiAttribute()] public sealed class SeqStage : Akka.Streams.Stage.GraphStageWithMaterializedValue, System.Threading.Tasks.Task>> { From f98a9d730c923f1d39c427b8c5e096f353295266 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Fri, 31 Jul 2020 23:10:03 +0700 Subject: [PATCH 05/23] Add unit test for publisher (cache) deallocation. --- .../Query/PersistenceIdsSpec.cs | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs b/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs index 1a288c9c012..7729d018cfd 100644 --- a/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs +++ b/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs @@ -6,12 +6,16 @@ //----------------------------------------------------------------------- using System; +using System.Reflection; +using System.Threading.Tasks; using Akka.Actor; using Akka.Configuration; using Akka.Persistence.Query; +using Akka.Persistence.Query.Sql; using Akka.Streams; using Akka.Streams.TestKit; using Akka.Util.Internal; +using Reactive.Streams; using Xunit; using Xunit.Abstractions; @@ -134,6 +138,41 @@ public virtual void ReadJournal_AllPersistenceIds_should_deliver_persistenceId_o }); } + [Fact] + public virtual async Task ReadJournal_should_deallocate_AllPersistenceIds_publisher_when_the_last_subscriber_left() + { + var journal = (SqlReadJournal)ReadJournal; + + Setup("a", 1); + Setup("b", 1); + + var source = journal.PersistenceIds(); + var probe = source.RunWith(this.SinkProbe(), Materializer); + var probe2 = source.RunWith(this.SinkProbe(), Materializer); + + var fieldInfo = journal.GetType().GetField("_persistenceIdsPublisher", BindingFlags.NonPublic | BindingFlags.Instance); + Assert.True(fieldInfo != null); + + // Assert that publisher is running. + probe.Within(TimeSpan.FromSeconds(10), () => probe.Request(10) + .ExpectNextUnordered("a", "b") + .ExpectNoMsg(TimeSpan.FromMilliseconds(200))); + + probe.Cancel(); + + // Assert that publisher is still alive when it still have a subscriber + Assert.True(fieldInfo.GetValue(journal) is IPublisher); + + probe2.Within(TimeSpan.FromSeconds(10), () => probe2.Request(4) + .ExpectNextUnordered("a", "b") + .ExpectNoMsg(TimeSpan.FromMilliseconds(200))); + + // Assert that publisher is de-allocated when the last subscriber left + probe2.Cancel(); + await Task.Delay(400); + Assert.True(fieldInfo.GetValue(journal) is null); + } + private IActorRef Setup(string persistenceId, int n) { var pref = Sys.ActorOf(Query.TestActor.Props(persistenceId)); From edb8e5279bf634d25c940376c1a8eaecaaca79b2 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Fri, 31 Jul 2020 23:11:17 +0700 Subject: [PATCH 06/23] Move onTerminate callback from AllPersistenceIdsPublisher to FanoutProcessorImpl --- .../AllPersistenceIdsPublisher.cs | 9 +++------ .../SqlReadJournal.cs | 10 +++++++--- .../Akka.Persistence.Sqlite.Tests.csproj | 1 + .../Query/SqlitePersistenceIdsSpec.cs | 2 +- .../Akka.Persistence.TCK.csproj | 1 + src/core/Akka.Streams/Dsl/Sink.cs | 4 ++-- .../Implementation/FanoutProcessorImpl.cs | 18 ++++++++++++++---- src/core/Akka.Streams/Implementation/Sinks.cs | 12 ++++++++---- .../Akka.Streams/Properties/AssemblyInfo.cs | 1 + 9 files changed, 38 insertions(+), 20 deletions(-) diff --git a/src/contrib/persistence/Akka.Persistence.Query.Sql/AllPersistenceIdsPublisher.cs b/src/contrib/persistence/Akka.Persistence.Query.Sql/AllPersistenceIdsPublisher.cs index 45e3b18005b..74a2eba3c8f 100644 --- a/src/contrib/persistence/Akka.Persistence.Query.Sql/AllPersistenceIdsPublisher.cs +++ b/src/contrib/persistence/Akka.Persistence.Query.Sql/AllPersistenceIdsPublisher.cs @@ -100,20 +100,19 @@ private class Continue private Continue() { } } - public static Props Props(TimeSpan refreshInterval, string writeJournalPluginId, Action onTerminated) + public static Props Props(TimeSpan refreshInterval, string writeJournalPluginId) { - return Actor.Props.Create(() => new LivePersistenceIdsPublisher(refreshInterval, writeJournalPluginId, onTerminated)); + return Actor.Props.Create(() => new LivePersistenceIdsPublisher(refreshInterval, writeJournalPluginId)); } private long _lastOrderingOffset; private readonly ICancelable _tickCancelable; private readonly IActorRef _journalRef; - private readonly Action _onTerminated; private readonly DeliveryBuffer _buffer; public IStash Stash { get; set; } - public LivePersistenceIdsPublisher(TimeSpan refreshInterval, string writeJournalPluginId, Action onTerminated) + public LivePersistenceIdsPublisher(TimeSpan refreshInterval, string writeJournalPluginId) { _tickCancelable = Context.System.Scheduler.ScheduleTellRepeatedlyCancelable( refreshInterval, @@ -123,13 +122,11 @@ public LivePersistenceIdsPublisher(TimeSpan refreshInterval, string writeJournal Self); _buffer = new DeliveryBuffer(OnNext); _journalRef = Persistence.Instance.Apply(Context.System).JournalFor(writeJournalPluginId); - _onTerminated = onTerminated; } protected override void PostStop() { _tickCancelable.Cancel(); - _onTerminated(); base.PostStop(); } diff --git a/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs b/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs index 0d12dc128f1..1d4155022b0 100644 --- a/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs +++ b/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs @@ -81,9 +81,8 @@ public Source PersistenceIds() Source.ActorPublisher( LivePersistenceIdsPublisher.Props( _refreshInterval, - _writeJournalPluginId, - () => _persistenceIdsPublisher = null)) - .ToMaterialized(Sink.DistinctRetainingFanOutPublisher(), Keep.Right); + _writeJournalPluginId)) + .ToMaterialized(Sink.DistinctRetainingFanOutPublisher(PersistenceIdsShutdownCallback), Keep.Right); _persistenceIdsPublisher = graph.Run(_system.Materializer()); } @@ -92,6 +91,11 @@ public Source PersistenceIds() .Named("AllPersistenceIds") as Source; } + private void PersistenceIdsShutdownCallback() + { + _persistenceIdsPublisher = null; + } + /// /// Same type of query as but the stream /// is completed immediately when it reaches the end of the "result set". Persistent diff --git a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Akka.Persistence.Sqlite.Tests.csproj b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Akka.Persistence.Sqlite.Tests.csproj index 2084f8515f3..69486b773af 100644 --- a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Akka.Persistence.Sqlite.Tests.csproj +++ b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Akka.Persistence.Sqlite.Tests.csproj @@ -9,6 +9,7 @@ + diff --git a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqlitePersistenceIdsSpec.cs b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqlitePersistenceIdsSpec.cs index fdd5c91e611..f54a2f745bb 100644 --- a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqlitePersistenceIdsSpec.cs +++ b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqlitePersistenceIdsSpec.cs @@ -28,7 +28,7 @@ class = ""Akka.Persistence.Sqlite.Journal.SqliteJournal, Akka.Persistence.Sqlite metadata-table-name = journal_metadata auto-initialize = on connection-string = ""Filename=file:memdb-journal-persistenceids-{id}.db;Mode=Memory;Cache=Shared"" - refresh-interval = 1s + refresh-interval = 200ms }} akka.test.single-expect-default = 10s") .WithFallback(SqlReadJournal.DefaultConfiguration()); diff --git a/src/core/Akka.Persistence.TCK/Akka.Persistence.TCK.csproj b/src/core/Akka.Persistence.TCK/Akka.Persistence.TCK.csproj index 22ed0c1a51a..e8633b6e16c 100644 --- a/src/core/Akka.Persistence.TCK/Akka.Persistence.TCK.csproj +++ b/src/core/Akka.Persistence.TCK/Akka.Persistence.TCK.csproj @@ -10,6 +10,7 @@ + diff --git a/src/core/Akka.Streams/Dsl/Sink.cs b/src/core/Akka.Streams/Dsl/Sink.cs index 33073899db2..54925ccf2b4 100644 --- a/src/core/Akka.Streams/Dsl/Sink.cs +++ b/src/core/Akka.Streams/Dsl/Sink.cs @@ -284,8 +284,8 @@ public static Sink> Publisher() public static Sink> FanoutPublisher() => new Sink>(new FanoutPublisherSink>(DefaultAttributes.FanoutPublisherSink, Shape("FanoutPublisherSink"))); - internal static Sink> DistinctRetainingFanOutPublisher() - => new Sink>(new FanoutPublisherSink>(DefaultAttributes.FanoutPublisherSink, Shape("DistinctRetainingFanOutPublisherSink"))); + internal static Sink> DistinctRetainingFanOutPublisher(Action onTerminated = null) + => new Sink>(new FanoutPublisherSink>(DefaultAttributes.FanoutPublisherSink, Shape("DistinctRetainingFanOutPublisherSink"), onTerminated)); /// /// A that will consume the stream and discard the elements. diff --git a/src/core/Akka.Streams/Implementation/FanoutProcessorImpl.cs b/src/core/Akka.Streams/Implementation/FanoutProcessorImpl.cs index 81dd495d605..c7da56b2adc 100644 --- a/src/core/Akka.Streams/Implementation/FanoutProcessorImpl.cs +++ b/src/core/Akka.Streams/Implementation/FanoutProcessorImpl.cs @@ -218,13 +218,16 @@ public void Error(Exception e) /// TBD internal sealed class FanoutProcessorImpl : ActorProcessorImpl where TStreamBuffer : IStreamBuffer { + private readonly Action _onTerminated; + /// /// TBD /// /// TBD + /// TBD /// TBD - public static Props Props(ActorMaterializerSettings settings) - => Actor.Props.Create(() => new FanoutProcessorImpl(settings)).WithDeploy(Deploy.Local); + public static Props Props(ActorMaterializerSettings settings, Action onTerminated = null) + => Actor.Props.Create(() => new FanoutProcessorImpl(settings, onTerminated)).WithDeploy(Deploy.Local); /// /// TBD @@ -235,11 +238,14 @@ public static Props Props(ActorMaterializerSettings settings) /// TBD /// /// TBD - public FanoutProcessorImpl(ActorMaterializerSettings settings) : base(settings) + /// TBD + public FanoutProcessorImpl(ActorMaterializerSettings settings, Action onTerminated) : base(settings) { PrimaryOutputs = new FanoutOutputs(settings.MaxInputBufferSize, settings.InitialInputBufferSize, Self, this, AfterFlush); + _onTerminated = onTerminated; + var running = new TransferPhase(PrimaryInputs.NeedsInput.And(PrimaryOutputs.NeedsDemand), () => PrimaryOutputs.EnqueueOutputElement(PrimaryInputs.DequeueInputElement())); InitialPhase(1, running); @@ -268,6 +274,10 @@ public override void PumpFinished() PrimaryOutputs.Complete(); } - private void AfterFlush() => Context.Stop(Self); + private void AfterFlush() + { + _onTerminated?.Invoke(); + Context.Stop(Self); + } } } diff --git a/src/core/Akka.Streams/Implementation/Sinks.cs b/src/core/Akka.Streams/Implementation/Sinks.cs index b9314fbbdfd..e3ba7b8e030 100644 --- a/src/core/Akka.Streams/Implementation/Sinks.cs +++ b/src/core/Akka.Streams/Implementation/Sinks.cs @@ -208,14 +208,18 @@ public override object Create(MaterializationContext context, out IPublisherTBD internal sealed class FanoutPublisherSink : SinkModule> where TStreamBuffer : IStreamBuffer { + private readonly Action _onTerminated; + /// /// TBD /// /// TBD /// TBD - public FanoutPublisherSink(Attributes attributes, SinkShape shape) : base(shape) + /// TBD + public FanoutPublisherSink(Attributes attributes, SinkShape shape, Action onTerminated = null) : base(shape) { Attributes = attributes; + _onTerminated = onTerminated; } /// @@ -229,7 +233,7 @@ public FanoutPublisherSink(Attributes attributes, SinkShape shape) : base(s /// TBD /// TBD public override IModule WithAttributes(Attributes attributes) - => new FanoutPublisherSink(attributes, AmendShape(attributes)); + => new FanoutPublisherSink(attributes, AmendShape(attributes), _onTerminated); /// /// TBD @@ -237,7 +241,7 @@ public override IModule WithAttributes(Attributes attributes) /// TBD /// TBD protected override SinkModule> NewInstance(SinkShape shape) - => new FanoutPublisherSink(Attributes, shape); + => new FanoutPublisherSink(Attributes, shape, _onTerminated); /// /// TBD @@ -249,7 +253,7 @@ public override object Create(MaterializationContext context, out IPublisher.Props(settings)); + var impl = actorMaterializer.ActorOf(context, FanoutProcessorImpl.Props(settings, _onTerminated)); var fanoutProcessor = new ActorProcessor(impl); impl.Tell(new ExposedPublisher(fanoutProcessor)); // Resolve cyclic dependency with actor. This MUST be the first message no matter what. diff --git a/src/core/Akka.Streams/Properties/AssemblyInfo.cs b/src/core/Akka.Streams/Properties/AssemblyInfo.cs index ad09773634f..394606e42b5 100644 --- a/src/core/Akka.Streams/Properties/AssemblyInfo.cs +++ b/src/core/Akka.Streams/Properties/AssemblyInfo.cs @@ -13,6 +13,7 @@ // set of attributes. Change these attribute values to modify the information // associated with an assembly. [assembly: InternalsVisibleTo("Akka.Persistence.Query.Sql")] +[assembly: InternalsVisibleTo("Akka.Persistence.TCK")] [assembly: InternalsVisibleTo("Akka.Streams.Tests")] [assembly: InternalsVisibleTo("Akka.Streams.TestKit")] [assembly: InternalsVisibleTo("Akka.Benchmarks")] From 238fca8d128b60129151168fb4c213a55bb98cd3 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Fri, 31 Jul 2020 23:29:00 +0700 Subject: [PATCH 07/23] Update API approver list --- src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt b/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt index e586d76bb92..df0b0ff95fc 100644 --- a/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt +++ b/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt @@ -1,5 +1,6 @@ [assembly: System.Runtime.CompilerServices.InternalsVisibleToAttribute("Akka.Benchmarks")] [assembly: System.Runtime.CompilerServices.InternalsVisibleToAttribute("Akka.Persistence.Query.Sql")] +[assembly: System.Runtime.CompilerServices.InternalsVisibleToAttribute("Akka.Persistence.TCK")] [assembly: System.Runtime.CompilerServices.InternalsVisibleToAttribute("Akka.Streams.TestKit")] [assembly: System.Runtime.CompilerServices.InternalsVisibleToAttribute("Akka.Streams.Tests")] [assembly: System.Runtime.InteropServices.ComVisibleAttribute(false)] From 738f855b9b54bf218d02b57abb93cfb9b125802e Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Tue, 18 Aug 2020 01:33:13 +0700 Subject: [PATCH 08/23] Remove redundant `as` cast --- .../Akka.Persistence.Query.Sql/SqlReadJournal.cs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs b/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs index 1d4155022b0..f236eb77b0a 100644 --- a/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs +++ b/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs @@ -88,7 +88,7 @@ public Source PersistenceIds() return Source.FromPublisher(_persistenceIdsPublisher) .MapMaterializedValue(_ => NotUsed.Instance) - .Named("AllPersistenceIds") as Source; + .Named("AllPersistenceIds"); } private void PersistenceIdsShutdownCallback() @@ -103,8 +103,8 @@ private void PersistenceIdsShutdownCallback() /// public Source CurrentPersistenceIds() => Source.ActorPublisher(CurrentPersistenceIdsPublisher.Props(_writeJournalPluginId)) - .MapMaterializedValue(_ => NotUsed.Instance) - .Named("CurrentPersistenceIds") as Source; + .MapMaterializedValue(_ => NotUsed.Instance) + .Named("CurrentPersistenceIds"); /// /// is used for retrieving events for a specific @@ -135,7 +135,7 @@ public Source CurrentPersistenceIds() public Source EventsByPersistenceId(string persistenceId, long fromSequenceNr, long toSequenceNr) => Source.ActorPublisher(EventsByPersistenceIdPublisher.Props(persistenceId, fromSequenceNr, toSequenceNr, _refreshInterval, _maxBufferSize, _writeJournalPluginId)) .MapMaterializedValue(_ => NotUsed.Instance) - .Named("EventsByPersistenceId-" + persistenceId) as Source; + .Named("EventsByPersistenceId-" + persistenceId); /// /// Same type of query as but the event stream @@ -145,7 +145,7 @@ public Source EventsByPersistenceId(string persistenceId public Source CurrentEventsByPersistenceId(string persistenceId, long fromSequenceNr, long toSequenceNr) => Source.ActorPublisher(EventsByPersistenceIdPublisher.Props(persistenceId, fromSequenceNr, toSequenceNr, null, _maxBufferSize, _writeJournalPluginId)) .MapMaterializedValue(_ => NotUsed.Instance) - .Named("CurrentEventsByPersistenceId-" + persistenceId) as Source; + .Named("CurrentEventsByPersistenceId-" + persistenceId); /// /// is used for retrieving events that were marked with From 1b79bf5755c82b7485de70aaa7c931499c2b8533 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Tue, 18 Aug 2020 01:59:51 +0700 Subject: [PATCH 09/23] Remove reference to Akka.Persistence.Query.Sql from Akka.Persistence.TCK --- src/Akka.sln.DotSettings | 1 + src/core/Akka.Persistence.TCK/Akka.Persistence.TCK.csproj | 1 - src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs | 3 +-- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Akka.sln.DotSettings b/src/Akka.sln.DotSettings index 41a06c2ddfc..c5ab510a684 100644 --- a/src/Akka.sln.DotSettings +++ b/src/Akka.sln.DotSettings @@ -24,4 +24,5 @@ True True True + \ No newline at end of file diff --git a/src/core/Akka.Persistence.TCK/Akka.Persistence.TCK.csproj b/src/core/Akka.Persistence.TCK/Akka.Persistence.TCK.csproj index e8633b6e16c..22ed0c1a51a 100644 --- a/src/core/Akka.Persistence.TCK/Akka.Persistence.TCK.csproj +++ b/src/core/Akka.Persistence.TCK/Akka.Persistence.TCK.csproj @@ -10,7 +10,6 @@ - diff --git a/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs b/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs index 7729d018cfd..41d29fe916f 100644 --- a/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs +++ b/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs @@ -11,7 +11,6 @@ using Akka.Actor; using Akka.Configuration; using Akka.Persistence.Query; -using Akka.Persistence.Query.Sql; using Akka.Streams; using Akka.Streams.TestKit; using Akka.Util.Internal; @@ -141,7 +140,7 @@ public virtual void ReadJournal_AllPersistenceIds_should_deliver_persistenceId_o [Fact] public virtual async Task ReadJournal_should_deallocate_AllPersistenceIds_publisher_when_the_last_subscriber_left() { - var journal = (SqlReadJournal)ReadJournal; + var journal = ReadJournal.AsInstanceOf(); Setup("a", 1); Setup("b", 1); From 1352b0e3ec4ccf85a0c90378a6230b194427366e Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Tue, 18 Aug 2020 02:26:15 +0700 Subject: [PATCH 10/23] Add concurrency locking on the internal persistence ids publisher. --- .../SqlReadJournal.cs | 47 ++++++++++++++++--- 1 file changed, 40 insertions(+), 7 deletions(-) diff --git a/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs b/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs index f236eb77b0a..d142b54a855 100644 --- a/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs +++ b/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs @@ -6,18 +6,17 @@ //----------------------------------------------------------------------- using System; +using System.Threading; using Reactive.Streams; using Akka.Actor; using Akka.Configuration; using Akka.Persistence.Journal; using Akka.Streams.Dsl; using Akka.Streams; -using Akka.Util.Internal; namespace Akka.Persistence.Query.Sql { public class SqlReadJournal : - IReadJournal, IPersistenceIdsQuery, ICurrentPersistenceIdsQuery, IEventsByPersistenceIdQuery, @@ -43,7 +42,37 @@ public static Config DefaultConfiguration() private readonly int _maxBufferSize; private readonly ExtendedActorSystem _system; - private IPublisher _persistenceIdsPublisher = null; + private readonly ReaderWriterLockSlim _lock; + private IPublisher _persistenceIdsPublisherDoNotUseDirectly; + + private IPublisher PersistenceIdsPublisher + { + get + { + _lock.EnterReadLock(); + try + { + return _persistenceIdsPublisherDoNotUseDirectly; + } + finally + { + _lock.ExitReadLock(); + } + } + + set + { + _lock.EnterWriteLock(); + try + { + _persistenceIdsPublisherDoNotUseDirectly = value; + } + finally + { + _lock.ExitWriteLock(); + } + } + } public SqlReadJournal(ExtendedActorSystem system, Config config) { @@ -51,6 +80,9 @@ public SqlReadJournal(ExtendedActorSystem system, Config config) _writeJournalPluginId = config.GetString("write-plugin", null); _maxBufferSize = config.GetInt("max-buffer-size", 0); _system = system; + + _lock = new ReaderWriterLockSlim(); + PersistenceIdsPublisher = null; } /// @@ -75,7 +107,7 @@ public SqlReadJournal(ExtendedActorSystem system, Config config) /// public Source PersistenceIds() { - if (_persistenceIdsPublisher is null) + if (PersistenceIdsPublisher is null) { var graph = Source.ActorPublisher( @@ -83,17 +115,18 @@ public Source PersistenceIds() _refreshInterval, _writeJournalPluginId)) .ToMaterialized(Sink.DistinctRetainingFanOutPublisher(PersistenceIdsShutdownCallback), Keep.Right); - _persistenceIdsPublisher = graph.Run(_system.Materializer()); + + PersistenceIdsPublisher = graph.Run(_system.Materializer()); } - return Source.FromPublisher(_persistenceIdsPublisher) + return Source.FromPublisher(PersistenceIdsPublisher) .MapMaterializedValue(_ => NotUsed.Instance) .Named("AllPersistenceIds"); } private void PersistenceIdsShutdownCallback() { - _persistenceIdsPublisher = null; + PersistenceIdsPublisher = null; } /// From 1ec20e4ea9891bfff3ec41a1a2c216d221891661 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Tue, 18 Aug 2020 07:55:06 +0700 Subject: [PATCH 11/23] Make PersistenceIds to query both Journal and Metadata table. Add test for snapshot and journal case. --- .../Journal/QueryExecutor.cs | 15 +++-- .../BatchingSqlitePersistenceIdSpec.cs | 44 +++++++++++--- .../Query/SqlitePersistenceIdsSpec.cs | 44 +++++++++++--- .../Query/PersistenceIdsSpec.cs | 59 ++++++++++++++++++- .../Akka.Streams.TestKit/TestSubscriber.cs | 2 +- 5 files changed, 139 insertions(+), 25 deletions(-) diff --git a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs index bb4027d4b98..6cf02073fbd 100644 --- a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs +++ b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs @@ -330,9 +330,16 @@ protected AbstractQueryExecutor(QueryConfiguration configuration, Akka.Serializa e.{Configuration.SerializerIdColumnName} as SerializerId"; AllPersistenceIdsSql = $@" - SELECT DISTINCT e.{Configuration.PersistenceIdColumnName} as PersistenceId - FROM {Configuration.FullJournalTableName} e - WHERE e.{Configuration.OrderingColumnName} > @Ordering"; + SELECT DISTINCT u.Id as PersistenceId + FROM ( + SELECT DISTINCT e.{Configuration.PersistenceIdColumnName} as Id + FROM {Configuration.FullJournalTableName} e + WHERE e.{Configuration.SequenceNrColumnName} > @SequenceNr + UNION + SELECT DISTINCT e.{Configuration.PersistenceIdColumnName} as Id + FROM {Configuration.FullMetaTableName} e + WHERE e.{Configuration.SequenceNrColumnName} > @SequenceNr + ) as u"; HighestSequenceNrSql = $@" SELECT MAX(u.SeqNr) as SequenceNr @@ -484,7 +491,7 @@ public virtual async Task> SelectAllPersistenceIdsAsync(D { using (var command = GetCommand(connection, AllPersistenceIdsSql)) { - AddParameter(command, "@Ordering", DbType.Int64, offset); + AddParameter(command, "@SequenceNr", DbType.Int64, offset); using (var reader = await command.ExecuteReaderAsync(cancellationToken)) { diff --git a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Batching/BatchingSqlitePersistenceIdSpec.cs b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Batching/BatchingSqlitePersistenceIdSpec.cs index 9f9ffd335d3..139f62eef6c 100644 --- a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Batching/BatchingSqlitePersistenceIdSpec.cs +++ b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Batching/BatchingSqlitePersistenceIdSpec.cs @@ -17,17 +17,43 @@ namespace Akka.Persistence.Sqlite.Tests.Batching public class BatchingSqlitePersistenceIdSpec : PersistenceIdsSpec { public static readonly AtomicCounter Counter = new AtomicCounter(0); + + public static string ConnectionString(int id) => $"Filename=file:memdb-persistenceids-{id}.db;Mode=Memory;Cache=Shared"; + public static Config Config(int id) => ConfigurationFactory.ParseString($@" akka.loglevel = INFO - akka.persistence.journal.plugin = ""akka.persistence.journal.sqlite"" - akka.persistence.journal.sqlite {{ - class = ""Akka.Persistence.Sqlite.Journal.BatchingSqliteJournal, Akka.Persistence.Sqlite"" - plugin-dispatcher = ""akka.actor.default-dispatcher"" - table-name = event_journal - metadata-table-name = journal_metadata - auto-initialize = on - connection-string = ""Datasource=memdb-journal-batch-persistenceids-{id}.db;Mode=Memory;Cache=Shared"" - refresh-interval = 1s + akka.actor{{ + serializers{{ + persistence-tck-test=""Akka.Persistence.TCK.Serialization.TestSerializer,Akka.Persistence.TCK"" + }} + serialization-bindings {{ + ""Akka.Persistence.TCK.Serialization.TestPayload,Akka.Persistence.TCK"" = persistence-tck-test + }} + }} + akka.persistence {{ + publish-plugin-commands = on + journal {{ + plugin = ""akka.persistence.journal.sqlite"" + sqlite = {{ + class = ""Akka.Persistence.Sqlite.Journal.SqliteJournal, Akka.Persistence.Sqlite"" + plugin-dispatcher = ""akka.actor.default-dispatcher"" + table-name = event_journal + metadata-table-name = journal_metadata + auto-initialize = on + connection-string = ""{ConnectionString(id)}"" + refresh-interval = 200ms + }} + }} + snapshot-store {{ + plugin = ""akka.persistence.snapshot-store.sqlite"" + sqlite {{ + class = ""Akka.Persistence.Sqlite.Snapshot.SqliteSnapshotStore, Akka.Persistence.Sqlite"" + plugin-dispatcher = ""akka.actor.default-dispatcher"" + table-name = snapshot_store + auto-initialize = on + connection-string = ""{ConnectionString(id)}"" + }} + }} }} akka.test.single-expect-default = 10s") .WithFallback(SqlReadJournal.DefaultConfiguration()); diff --git a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqlitePersistenceIdsSpec.cs b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqlitePersistenceIdsSpec.cs index f54a2f745bb..b1552d61b17 100644 --- a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqlitePersistenceIdsSpec.cs +++ b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqlitePersistenceIdsSpec.cs @@ -5,6 +5,7 @@ // //----------------------------------------------------------------------- +using Akka.Actor; using Akka.Configuration; using Akka.Persistence.Query; using Akka.Persistence.Query.Sql; @@ -18,17 +19,42 @@ public class SqlitePersistenceIdsSpec : PersistenceIdsSpec { public static readonly AtomicCounter Counter = new AtomicCounter(0); + public static string ConnectionString(int id) => $"Filename=file:memdb-persistenceids-{id}.db;Mode=Memory;Cache=Shared"; + public static Config Config(int id) => ConfigurationFactory.ParseString($@" akka.loglevel = INFO - akka.persistence.journal.plugin = ""akka.persistence.journal.sqlite"" - akka.persistence.journal.sqlite {{ - class = ""Akka.Persistence.Sqlite.Journal.SqliteJournal, Akka.Persistence.Sqlite"" - plugin-dispatcher = ""akka.actor.default-dispatcher"" - table-name = event_journal - metadata-table-name = journal_metadata - auto-initialize = on - connection-string = ""Filename=file:memdb-journal-persistenceids-{id}.db;Mode=Memory;Cache=Shared"" - refresh-interval = 200ms + akka.actor{{ + serializers{{ + persistence-tck-test=""Akka.Persistence.TCK.Serialization.TestSerializer,Akka.Persistence.TCK"" + }} + serialization-bindings {{ + ""Akka.Persistence.TCK.Serialization.TestPayload,Akka.Persistence.TCK"" = persistence-tck-test + }} + }} + akka.persistence {{ + publish-plugin-commands = on + journal {{ + plugin = ""akka.persistence.journal.sqlite"" + sqlite = {{ + class = ""Akka.Persistence.Sqlite.Journal.SqliteJournal, Akka.Persistence.Sqlite"" + plugin-dispatcher = ""akka.actor.default-dispatcher"" + table-name = event_journal + metadata-table-name = journal_metadata + auto-initialize = on + connection-string = ""{ConnectionString(id)}"" + refresh-interval = 200ms + }} + }} + snapshot-store {{ + plugin = ""akka.persistence.snapshot-store.sqlite"" + sqlite {{ + class = ""Akka.Persistence.Sqlite.Snapshot.SqliteSnapshotStore, Akka.Persistence.Sqlite"" + plugin-dispatcher = ""akka.actor.default-dispatcher"" + table-name = snapshot_store + auto-initialize = on + connection-string = ""{ConnectionString(id)}"" + }} + }} }} akka.test.single-expect-default = 10s") .WithFallback(SqlReadJournal.DefaultConfiguration()); diff --git a/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs b/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs index 41d29fe916f..4a29235ca8e 100644 --- a/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs +++ b/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs @@ -6,6 +6,7 @@ //----------------------------------------------------------------------- using System; +using System.Collections.Generic; using System.Reflection; using System.Threading.Tasks; using Akka.Actor; @@ -13,6 +14,7 @@ using Akka.Persistence.Query; using Akka.Streams; using Akka.Streams.TestKit; +using Akka.TestKit; using Akka.Util.Internal; using Reactive.Streams; using Xunit; @@ -25,11 +27,17 @@ public abstract class PersistenceIdsSpec : Akka.TestKit.Xunit2.TestKit protected ActorMaterializer Materializer { get; } protected IReadJournal ReadJournal { get; set; } + protected IActorRef SnapshotStore => Extension.SnapshotStoreFor(null); + protected PersistenceExtension Extension { get; } + + private readonly TestProbe _senderProbe; protected PersistenceIdsSpec(Config config = null, string actorSystemName = null, ITestOutputHelper output = null) : base(config ?? Config.Empty, actorSystemName, output) { Materializer = Sys.Materializer(); + Extension = Persistence.Instance.Apply(Sys as ExtendedActorSystem); + _senderProbe = CreateTestProbe(); } [Fact] @@ -80,6 +88,36 @@ public virtual void ReadJournal_AllPersistenceIds_should_find_new_events_after_d }); } + [Fact] + public virtual void ReadJournal_AllPersistenceIds_should_find_events_on_both_journal_and_snapshot_store() + { + var queries = ReadJournal.AsInstanceOf(); + + WriteSnapshot("a", 2); + WriteSnapshot("b", 2); + WriteSnapshot("c", 2); + Setup("d", 2); + Setup("e", 2); + Setup("f", 2); + + var source = queries.PersistenceIds(); + var probe = source.RunWith(this.SinkProbe(), Materializer); + + var expectedUniqueList = new List(){"a", "b", "c", "d", "e", "f"}; + + probe.Within(TimeSpan.FromSeconds(10), () => probe.Request(3) + .ExpectNextWithinSet(expectedUniqueList) + .ExpectNextWithinSet(expectedUniqueList) + .ExpectNextWithinSet(expectedUniqueList) + .ExpectNoMsg(TimeSpan.FromMilliseconds(200))); + + probe.Within(TimeSpan.FromSeconds(10), () => probe.Request(3) + .ExpectNextWithinSet(expectedUniqueList) + .ExpectNextWithinSet(expectedUniqueList) + .ExpectNextWithinSet(expectedUniqueList) + .ExpectNoMsg(TimeSpan.FromMilliseconds(200))); + } + [Fact] public virtual void ReadJournal_AllPersistenceIds_should_only_deliver_what_requested_if_there_is_more_in_the_buffer() { @@ -149,7 +187,7 @@ public virtual async Task ReadJournal_should_deallocate_AllPersistenceIds_publis var probe = source.RunWith(this.SinkProbe(), Materializer); var probe2 = source.RunWith(this.SinkProbe(), Materializer); - var fieldInfo = journal.GetType().GetField("_persistenceIdsPublisher", BindingFlags.NonPublic | BindingFlags.Instance); + var fieldInfo = journal.GetType().GetProperty("PersistenceIdsPublisher", BindingFlags.NonPublic | BindingFlags.Instance); Assert.True(fieldInfo != null); // Assert that publisher is running. @@ -172,7 +210,7 @@ public virtual async Task ReadJournal_should_deallocate_AllPersistenceIds_publis Assert.True(fieldInfo.GetValue(journal) is null); } - private IActorRef Setup(string persistenceId, int n) + protected IActorRef Setup(string persistenceId, int n) { var pref = Sys.ActorOf(Query.TestActor.Props(persistenceId)); for (int i = 1; i <= n; i++) @@ -184,6 +222,23 @@ private IActorRef Setup(string persistenceId, int n) return pref; } + protected IActorRef WriteSnapshot(string persistenceId, int n) + { + var pref = Sys.ActorOf(Query.TestActor.Props(persistenceId)); + for (var i = 1; i <= n; i++) + { + pref.Tell($"{persistenceId}-{i}"); + ExpectMsg($"{persistenceId}-{i}-done"); + } + + var metadata = new SnapshotMetadata(persistenceId, n + 10); + SnapshotStore.Tell(new SaveSnapshot(metadata, $"s-{n}"), _senderProbe.Ref); + _senderProbe.ExpectMsg(); + + return pref; + } + + protected override void Dispose(bool disposing) { Materializer.Dispose(); diff --git a/src/core/Akka.Streams.TestKit/TestSubscriber.cs b/src/core/Akka.Streams.TestKit/TestSubscriber.cs index c5d9a5ac376..dd2c1ecdef2 100644 --- a/src/core/Akka.Streams.TestKit/TestSubscriber.cs +++ b/src/core/Akka.Streams.TestKit/TestSubscriber.cs @@ -221,7 +221,7 @@ public ManualProbe ExpectNextWithinSet(List elems) if(!elems.Contains(next.Element)) Assert(false, "unexpected elements [{0}] found in the result", next.Element); elems.Remove(next.Element); - + _probe.Log.Info($"Received '{next.Element}' within OnNext()."); return this; } From e983d47bdea0900ec0794493faa37bc1143078cd Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Tue, 18 Aug 2020 08:05:50 +0700 Subject: [PATCH 12/23] Change IStreamBuffer access to internal. --- .../Implementation/ResizableMultiReaderRingBuffer.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs b/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs index a4909200451..83d5da5da08 100644 --- a/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs +++ b/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs @@ -63,7 +63,7 @@ public interface ICursor long Cursor { get; set; } } - public interface IStreamBuffer + internal interface IStreamBuffer { bool IsEmpty { get; } From f6f1b602e39649e28798e37771552263548cb683 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Tue, 18 Aug 2020 08:28:09 +0700 Subject: [PATCH 13/23] Update API approver list --- .../CoreAPISpec.ApproveStreams.approved.txt | 15 ++------------- 1 file changed, 2 insertions(+), 13 deletions(-) diff --git a/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt b/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt index df0b0ff95fc..d4023d7eb91 100644 --- a/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt +++ b/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt @@ -3024,17 +3024,6 @@ namespace Akka.Streams.Implementation void WaitForUpstream(int waitForUpstream); } public interface ISpecViolation { } - public interface IStreamBuffer - { - long CapacityLeft { get; } - bool IsEmpty { get; } - long Length { get; } - long Count(Akka.Streams.Implementation.ICursor cursor); - void InitCursor(Akka.Streams.Implementation.ICursor cursor); - void OnCursorRemoved(Akka.Streams.Implementation.ICursor cursor); - T Read(Akka.Streams.Implementation.ICursor cursor); - bool Write(T value); - } [Akka.Annotations.InternalApiAttribute()] public sealed class IdleInject : Akka.Streams.Stage.GraphStage> where TIn : TOut @@ -3336,7 +3325,7 @@ namespace Akka.Streams.Implementation public static void TryRequest(Reactive.Streams.ISubscription subscription, long demand) { } } [Akka.Annotations.InternalApiAttribute()] - public class ResizableMultiReaderRingBuffer : Akka.Streams.Implementation.IStreamBuffer + public class ResizableMultiReaderRingBuffer { protected readonly Akka.Streams.Implementation.ICursors Cursors; public ResizableMultiReaderRingBuffer(long initialSize, long maxSize, Akka.Streams.Implementation.ICursors cursors) { } @@ -3353,7 +3342,7 @@ namespace Akka.Streams.Implementation public override string ToString() { } public bool Write(T value) { } } - public class RetainingMultiReaderBuffer : Akka.Streams.Implementation.IStreamBuffer + public class RetainingMultiReaderBuffer { protected readonly Akka.Streams.Implementation.ICursors Cursors; public RetainingMultiReaderBuffer(long initialSize, long maxSize, Akka.Streams.Implementation.ICursors cursors) { } From fabeaa1ffe8b0972bfd2427cbfb21d8e0c8f01a6 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Tue, 18 Aug 2020 23:55:08 +0700 Subject: [PATCH 14/23] Go back to using ordeting column to filter the journal table for performance reasons --- .../Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs index 6cf02073fbd..aa155b70483 100644 --- a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs +++ b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/QueryExecutor.cs @@ -334,11 +334,10 @@ SELECT DISTINCT u.Id as PersistenceId FROM ( SELECT DISTINCT e.{Configuration.PersistenceIdColumnName} as Id FROM {Configuration.FullJournalTableName} e - WHERE e.{Configuration.SequenceNrColumnName} > @SequenceNr + WHERE e.{Configuration.OrderingColumnName} > @Ordering UNION SELECT DISTINCT e.{Configuration.PersistenceIdColumnName} as Id FROM {Configuration.FullMetaTableName} e - WHERE e.{Configuration.SequenceNrColumnName} > @SequenceNr ) as u"; HighestSequenceNrSql = $@" @@ -491,7 +490,7 @@ public virtual async Task> SelectAllPersistenceIdsAsync(D { using (var command = GetCommand(connection, AllPersistenceIdsSql)) { - AddParameter(command, "@SequenceNr", DbType.Int64, offset); + AddParameter(command, "@Ordering", DbType.Int64, offset); using (var reader = await command.ExecuteReaderAsync(cancellationToken)) { From 386d6fc6368b8180534c2f15601ae581f4271557 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Wed, 19 Aug 2020 00:26:43 +0700 Subject: [PATCH 15/23] Change synchronization to a simpler lock --- .../SqlReadJournal.cs | 65 ++++++------------- 1 file changed, 21 insertions(+), 44 deletions(-) diff --git a/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs b/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs index d142b54a855..4bf46e0b5b2 100644 --- a/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs +++ b/src/contrib/persistence/Akka.Persistence.Query.Sql/SqlReadJournal.cs @@ -42,37 +42,8 @@ public static Config DefaultConfiguration() private readonly int _maxBufferSize; private readonly ExtendedActorSystem _system; - private readonly ReaderWriterLockSlim _lock; - private IPublisher _persistenceIdsPublisherDoNotUseDirectly; - - private IPublisher PersistenceIdsPublisher - { - get - { - _lock.EnterReadLock(); - try - { - return _persistenceIdsPublisherDoNotUseDirectly; - } - finally - { - _lock.ExitReadLock(); - } - } - - set - { - _lock.EnterWriteLock(); - try - { - _persistenceIdsPublisherDoNotUseDirectly = value; - } - finally - { - _lock.ExitWriteLock(); - } - } - } + private readonly object _lock = new object(); + private IPublisher _persistenceIdsPublisher; public SqlReadJournal(ExtendedActorSystem system, Config config) { @@ -82,7 +53,7 @@ public SqlReadJournal(ExtendedActorSystem system, Config config) _system = system; _lock = new ReaderWriterLockSlim(); - PersistenceIdsPublisher = null; + _persistenceIdsPublisher = null; } /// @@ -107,26 +78,32 @@ public SqlReadJournal(ExtendedActorSystem system, Config config) /// public Source PersistenceIds() { - if (PersistenceIdsPublisher is null) + lock (_lock) { - var graph = - Source.ActorPublisher( - LivePersistenceIdsPublisher.Props( - _refreshInterval, - _writeJournalPluginId)) - .ToMaterialized(Sink.DistinctRetainingFanOutPublisher(PersistenceIdsShutdownCallback), Keep.Right); + if (_persistenceIdsPublisher is null) + { + var graph = + Source.ActorPublisher( + LivePersistenceIdsPublisher.Props( + _refreshInterval, + _writeJournalPluginId)) + .ToMaterialized(Sink.DistinctRetainingFanOutPublisher(PersistenceIdsShutdownCallback), Keep.Right); - PersistenceIdsPublisher = graph.Run(_system.Materializer()); + _persistenceIdsPublisher = graph.Run(_system.Materializer()); + } + return Source.FromPublisher(_persistenceIdsPublisher) + .MapMaterializedValue(_ => NotUsed.Instance) + .Named("AllPersistenceIds"); } - return Source.FromPublisher(PersistenceIdsPublisher) - .MapMaterializedValue(_ => NotUsed.Instance) - .Named("AllPersistenceIds"); } private void PersistenceIdsShutdownCallback() { - PersistenceIdsPublisher = null; + lock (_lock) + { + _persistenceIdsPublisher = null; + } } /// From e7d174a9b9475ddcb7880aee55a419cd4c1cde07 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Wed, 19 Aug 2020 00:27:31 +0700 Subject: [PATCH 16/23] Change test to reflect changes in synchonization --- src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs b/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs index 4a29235ca8e..5045e77ce53 100644 --- a/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs +++ b/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs @@ -187,7 +187,7 @@ public virtual async Task ReadJournal_should_deallocate_AllPersistenceIds_publis var probe = source.RunWith(this.SinkProbe(), Materializer); var probe2 = source.RunWith(this.SinkProbe(), Materializer); - var fieldInfo = journal.GetType().GetProperty("PersistenceIdsPublisher", BindingFlags.NonPublic | BindingFlags.Instance); + var fieldInfo = journal.GetType().GetField("_persistenceIdsPublisher", BindingFlags.NonPublic | BindingFlags.Instance); Assert.True(fieldInfo != null); // Assert that publisher is running. From 6982944a25bb1048b04f9735268c67a5457fa5eb Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Wed, 19 Aug 2020 00:28:05 +0700 Subject: [PATCH 17/23] Update persistence ids SQL in BatchingSqlJournal --- .../Journal/BatchingSqlJournal.cs | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/BatchingSqlJournal.cs b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/BatchingSqlJournal.cs index 37f3bbe47a3..eef79a89168 100644 --- a/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/BatchingSqlJournal.cs +++ b/src/contrib/persistence/Akka.Persistence.Sql.Common/Journal/BatchingSqlJournal.cs @@ -575,8 +575,15 @@ protected BatchingSqlJournal(BatchingSqlJournalSetup setup) e.{conventions.SerializerIdColumnName} as SerializerId"; AllPersistenceIdsSql = $@" - SELECT DISTINCT e.{conventions.PersistenceIdColumnName} as PersistenceId - FROM {conventions.FullJournalTableName} e;"; + SELECT DISTINCT u.Id as PersistenceId + FROM ( + SELECT DISTINCT e.{conventions.PersistenceIdColumnName} as Id + FROM {conventions.FullJournalTableName} e + WHERE e.{conventions.OrderingColumnName} > @Ordering + UNION + SELECT DISTINCT e.{conventions.PersistenceIdColumnName} as Id + FROM {conventions.FullMetaTableName} e + ) as u"; HighestSequenceNrSql = $@" SELECT MAX(u.SeqNr) as SequenceNr From 5a08ec5607d8be9f9108f24174f5a5fa16a19b37 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Wed, 19 Aug 2020 23:57:36 +0700 Subject: [PATCH 18/23] Change how SubscribeManager checks for available data. --- .../ResizableMultiReaderRingBuffer.cs | 13 +++++++++++++ .../Implementation/SubscriberManagement.cs | 3 +-- 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs b/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs index 83d5da5da08..f7cdfd7398e 100644 --- a/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs +++ b/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs @@ -69,6 +69,8 @@ internal interface IStreamBuffer long Length { get; } + long Count(); + long CapacityLeft { get; } long Count(ICursor cursor); @@ -139,6 +141,15 @@ public RetainingMultiReaderBuffer(long initialSize, long maxSize, ICursors curso /// TBD public long Count(ICursor cursor) => Length - cursor.Cursor; + public long Count() + { + var lowest = 0L; + foreach (var cursor in Cursors.Cursors) + lowest = Math.Max(cursor.Cursor, lowest); + + return Length - lowest; + } + public T Read(ICursor cursor) { var c = cursor.Cursor; @@ -257,6 +268,8 @@ public ResizableMultiReaderRingBuffer(long initialSize, long maxSize, ICursors c /// public long Length => _writeIndex - _readIndex; + public long Count() => Length; + /// /// TBD /// diff --git a/src/core/Akka.Streams/Implementation/SubscriberManagement.cs b/src/core/Akka.Streams/Implementation/SubscriberManagement.cs index 9706c4b48cf..2e15d12474e 100644 --- a/src/core/Akka.Streams/Implementation/SubscriberManagement.cs +++ b/src/core/Akka.Streams/Implementation/SubscriberManagement.cs @@ -313,10 +313,9 @@ protected void PushToDownstream(T value) if (_endOfStream is SubscriberManagement.NotReached) { _pendingFromUpstream--; - var oldBufferLength = _buffer.Value.Length; if (!_buffer.Value.Write(value)) throw new IllegalStateException("Output buffer overflow"); - if (_buffer.Value.Length > oldBufferLength && Dispatch(_subscriptions)) + if (_buffer.Value.Count() > 0 && Dispatch(_subscriptions)) RequestFromUpstreamIfRequired(); } else throw new IllegalStateException("PushToDownStream(...) after CompleteDownstream() or AbortDownstream(...)"); From e5f7d98d28afd1d44fb413e83418332153f35205 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Thu, 20 Aug 2020 01:50:29 +0700 Subject: [PATCH 19/23] Change confusing `Count()` to `AvailableData` property --- .../ResizableMultiReaderRingBuffer.cs | 17 ++++++++++------- .../Implementation/SubscriberManagement.cs | 2 +- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs b/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs index f7cdfd7398e..7e885623213 100644 --- a/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs +++ b/src/core/Akka.Streams/Implementation/ResizableMultiReaderRingBuffer.cs @@ -69,7 +69,7 @@ internal interface IStreamBuffer long Length { get; } - long Count(); + long AvailableData { get; } long CapacityLeft { get; } @@ -141,13 +141,16 @@ public RetainingMultiReaderBuffer(long initialSize, long maxSize, ICursors curso /// TBD public long Count(ICursor cursor) => Length - cursor.Cursor; - public long Count() + public long AvailableData { - var lowest = 0L; - foreach (var cursor in Cursors.Cursors) - lowest = Math.Max(cursor.Cursor, lowest); + get + { + var lowest = 0L; + foreach (var cursor in Cursors.Cursors) + lowest = Math.Max(cursor.Cursor, lowest); - return Length - lowest; + return Length - lowest; + } } public T Read(ICursor cursor) @@ -268,7 +271,7 @@ public ResizableMultiReaderRingBuffer(long initialSize, long maxSize, ICursors c /// public long Length => _writeIndex - _readIndex; - public long Count() => Length; + public long AvailableData => Length; /// /// TBD diff --git a/src/core/Akka.Streams/Implementation/SubscriberManagement.cs b/src/core/Akka.Streams/Implementation/SubscriberManagement.cs index 2e15d12474e..1976c2ef71c 100644 --- a/src/core/Akka.Streams/Implementation/SubscriberManagement.cs +++ b/src/core/Akka.Streams/Implementation/SubscriberManagement.cs @@ -315,7 +315,7 @@ protected void PushToDownstream(T value) _pendingFromUpstream--; if (!_buffer.Value.Write(value)) throw new IllegalStateException("Output buffer overflow"); - if (_buffer.Value.Count() > 0 && Dispatch(_subscriptions)) + if (_buffer.Value.AvailableData > 0 && Dispatch(_subscriptions)) RequestFromUpstreamIfRequired(); } else throw new IllegalStateException("PushToDownStream(...) after CompleteDownstream() or AbortDownstream(...)"); From 8954464dffaa47b6bf0e8377506ae0c3f0e7f5d4 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Thu, 20 Aug 2020 01:56:03 +0700 Subject: [PATCH 20/23] Update API approver list --- src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt b/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt index d4023d7eb91..f446d828a43 100644 --- a/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt +++ b/src/core/Akka.API.Tests/CoreAPISpec.ApproveStreams.approved.txt @@ -3329,6 +3329,7 @@ namespace Akka.Streams.Implementation { protected readonly Akka.Streams.Implementation.ICursors Cursors; public ResizableMultiReaderRingBuffer(long initialSize, long maxSize, Akka.Streams.Implementation.ICursors cursors) { } + public long AvailableData { get; } public long CapacityLeft { get; } public long ImmediatelyAvailable { get; } public bool IsEmpty { get; } @@ -3346,6 +3347,7 @@ namespace Akka.Streams.Implementation { protected readonly Akka.Streams.Implementation.ICursors Cursors; public RetainingMultiReaderBuffer(long initialSize, long maxSize, Akka.Streams.Implementation.ICursors cursors) { } + public long AvailableData { get; } protected T[] Buffer { get; } public long CapacityLeft { get; } public bool IsEmpty { get; } From c20ddb331a07bc4f438bcc42612660b6f95b3828 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Thu, 20 Aug 2020 23:38:11 +0700 Subject: [PATCH 21/23] Make sure that tests are never run in parallel --- src/xunit.runner.json | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/xunit.runner.json b/src/xunit.runner.json index cafdde412f1..4a73b1e56a4 100644 --- a/src/xunit.runner.json +++ b/src/xunit.runner.json @@ -1,4 +1,6 @@ { - "$schema": "https://xunit.github.io/schema/current/xunit.runner.schema.json", - "parallelizeTestCollections": false + "$schema": "https://xunit.github.io/schema/current/xunit.runner.schema.json", + "longRunningTestSeconds": 60, + "parallelizeAssembly": false, + "parallelizeTestCollections": false } \ No newline at end of file From 56f4f47078e6b857d01a3289c73cd7b79f6e1bc6 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Thu, 20 Aug 2020 23:39:03 +0700 Subject: [PATCH 22/23] Prune excessive test --- src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs b/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs index 5045e77ce53..7017557cc0c 100644 --- a/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs +++ b/src/core/Akka.Persistence.TCK/Query/PersistenceIdsSpec.cs @@ -153,7 +153,7 @@ public virtual void ReadJournal_AllPersistenceIds_should_deliver_persistenceId_o { var queries = ReadJournal.AsInstanceOf(); - Setup("p", 1000); + Setup("p", 10); var source = queries.PersistenceIds(); var probe = source.RunWith(this.SinkProbe(), Materializer); @@ -162,16 +162,16 @@ public virtual void ReadJournal_AllPersistenceIds_should_deliver_persistenceId_o { return probe.Request(10) .ExpectNext("p") - .ExpectNoMsg(TimeSpan.FromMilliseconds(1000)); + .ExpectNoMsg(TimeSpan.FromMilliseconds(200)); }); - Setup("q", 1000); + Setup("q", 10); probe.Within(TimeSpan.FromSeconds(10), () => { return probe.Request(10) .ExpectNext("q") - .ExpectNoMsg(TimeSpan.FromMilliseconds(1000)); + .ExpectNoMsg(TimeSpan.FromMilliseconds(200)); }); } From d75c717616bb22aeb215cb39c67ef228cf054236 Mon Sep 17 00:00:00 2001 From: Gregorius Soedharmo Date: Thu, 20 Aug 2020 23:40:32 +0700 Subject: [PATCH 23/23] Append a GUID on the db filename to make doubly sure that db never get used twice --- .../Batching/BatchingSqliteAllEventsSpec.cs | 8 +++----- .../Batching/BatchingSqlitePersistenceIdSpec.cs | 13 ++++++------- .../Query/SqliteAllEventsSpec.cs | 8 +++----- .../Query/SqlitePersistenceIdsSpec.cs | 13 ++++++------- 4 files changed, 18 insertions(+), 24 deletions(-) diff --git a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Batching/BatchingSqliteAllEventsSpec.cs b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Batching/BatchingSqliteAllEventsSpec.cs index 878584f1b2f..567413d6374 100644 --- a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Batching/BatchingSqliteAllEventsSpec.cs +++ b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Batching/BatchingSqliteAllEventsSpec.cs @@ -12,9 +12,7 @@ namespace Akka.Persistence.Sqlite.Tests.Query { public class BatchingSqliteAllEventsSpec : AllEventsSpec { - public static readonly AtomicCounter Counter = new AtomicCounter(0); - - public static Config Config(int id) => ConfigurationFactory.ParseString($@" + public static Config Config => ConfigurationFactory.ParseString($@" akka.loglevel = INFO akka.persistence.journal.plugin = ""akka.persistence.journal.sqlite"" akka.persistence.journal.sqlite {{ @@ -23,13 +21,13 @@ class = ""Akka.Persistence.Sqlite.Journal.BatchingSqliteJournal, Akka.Persistenc table-name = event_journal metadata-table-name = journal_metadata auto-initialize = on - connection-string = ""Filename=file:memdb-journal-eventsbytag-{id}.db;Mode=Memory;Cache=Shared"" + connection-string = ""Filename=file:memdb-journal-eventsbytag-{Guid.NewGuid()}.db;Mode=Memory;Cache=Shared"" refresh-interval = 1s }} akka.test.single-expect-default = 10s") .WithFallback(SqlReadJournal.DefaultConfiguration()); - public BatchingSqliteAllEventsSpec(ITestOutputHelper output) : base(Config(Counter.GetAndIncrement()), nameof(BatchingSqliteAllEventsSpec), output) + public BatchingSqliteAllEventsSpec(ITestOutputHelper output) : base(Config, nameof(BatchingSqliteAllEventsSpec), output) { ReadJournal = Sys.ReadJournalFor(SqlReadJournal.Identifier); } diff --git a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Batching/BatchingSqlitePersistenceIdSpec.cs b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Batching/BatchingSqlitePersistenceIdSpec.cs index 139f62eef6c..abdd2f069bf 100644 --- a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Batching/BatchingSqlitePersistenceIdSpec.cs +++ b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Batching/BatchingSqlitePersistenceIdSpec.cs @@ -5,6 +5,7 @@ // //----------------------------------------------------------------------- +using System; using Akka.Configuration; using Akka.Persistence.Query; using Akka.Persistence.Query.Sql; @@ -16,11 +17,9 @@ namespace Akka.Persistence.Sqlite.Tests.Batching { public class BatchingSqlitePersistenceIdSpec : PersistenceIdsSpec { - public static readonly AtomicCounter Counter = new AtomicCounter(0); + public static string ConnectionString(string type) => $"Filename=file:memdb-persistenceids-{type}-{Guid.NewGuid()}.db;Mode=Memory;Cache=Shared"; - public static string ConnectionString(int id) => $"Filename=file:memdb-persistenceids-{id}.db;Mode=Memory;Cache=Shared"; - - public static Config Config(int id) => ConfigurationFactory.ParseString($@" + public static Config Config => ConfigurationFactory.ParseString($@" akka.loglevel = INFO akka.actor{{ serializers{{ @@ -40,7 +39,7 @@ class = ""Akka.Persistence.Sqlite.Journal.SqliteJournal, Akka.Persistence.Sqlite table-name = event_journal metadata-table-name = journal_metadata auto-initialize = on - connection-string = ""{ConnectionString(id)}"" + connection-string = ""{ConnectionString("journal")}"" refresh-interval = 200ms }} }} @@ -51,7 +50,7 @@ class = ""Akka.Persistence.Sqlite.Snapshot.SqliteSnapshotStore, Akka.Persistence plugin-dispatcher = ""akka.actor.default-dispatcher"" table-name = snapshot_store auto-initialize = on - connection-string = ""{ConnectionString(id)}"" + connection-string = ""{ConnectionString("snapshot")}"" }} }} }} @@ -59,7 +58,7 @@ class = ""Akka.Persistence.Sqlite.Snapshot.SqliteSnapshotStore, Akka.Persistence .WithFallback(SqlReadJournal.DefaultConfiguration()); - public BatchingSqlitePersistenceIdSpec(ITestOutputHelper output) : base(Config(Counter.GetAndIncrement()), nameof(BatchingSqlitePersistenceIdSpec), output) + public BatchingSqlitePersistenceIdSpec(ITestOutputHelper output) : base(Config, nameof(BatchingSqlitePersistenceIdSpec), output) { ReadJournal = Sys.ReadJournalFor(SqlReadJournal.Identifier); } diff --git a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteAllEventsSpec.cs b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteAllEventsSpec.cs index 1b7ebf08c95..186faae968b 100644 --- a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteAllEventsSpec.cs +++ b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqliteAllEventsSpec.cs @@ -12,9 +12,7 @@ namespace Akka.Persistence.Sqlite.Tests.Query { public class SqliteAllEventsSpec:AllEventsSpec { - public static readonly AtomicCounter Counter = new AtomicCounter(0); - - public static Config Config(int id) => ConfigurationFactory.ParseString($@" + public static Config Config => ConfigurationFactory.ParseString($@" akka.loglevel = INFO akka.persistence.journal.plugin = ""akka.persistence.journal.sqlite"" akka.persistence.journal.sqlite {{ @@ -23,13 +21,13 @@ class = ""Akka.Persistence.Sqlite.Journal.SqliteJournal, Akka.Persistence.Sqlite table-name = event_journal metadata-table-name = journal_metadata auto-initialize = on - connection-string = ""Filename=file:memdb-journal-eventsbytag-{id}.db;Mode=Memory;Cache=Shared"" + connection-string = ""Filename=file:memdb-journal-eventsbytag-{Guid.NewGuid()}.db;Mode=Memory;Cache=Shared"" refresh-interval = 1s }} akka.test.single-expect-default = 10s") .WithFallback(SqlReadJournal.DefaultConfiguration()); - public SqliteAllEventsSpec(ITestOutputHelper output) : base(Config(Counter.GetAndIncrement()), nameof(SqliteAllEventsSpec), output) + public SqliteAllEventsSpec(ITestOutputHelper output) : base(Config, nameof(SqliteAllEventsSpec), output) { ReadJournal = Sys.ReadJournalFor(SqlReadJournal.Identifier); } diff --git a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqlitePersistenceIdsSpec.cs b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqlitePersistenceIdsSpec.cs index b1552d61b17..43200b64713 100644 --- a/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqlitePersistenceIdsSpec.cs +++ b/src/contrib/persistence/Akka.Persistence.Sqlite.Tests/Query/SqlitePersistenceIdsSpec.cs @@ -5,6 +5,7 @@ // //----------------------------------------------------------------------- +using System; using Akka.Actor; using Akka.Configuration; using Akka.Persistence.Query; @@ -17,11 +18,9 @@ namespace Akka.Persistence.Sqlite.Tests.Query { public class SqlitePersistenceIdsSpec : PersistenceIdsSpec { - public static readonly AtomicCounter Counter = new AtomicCounter(0); + public static string ConnectionString(string type) => $"Filename=file:memdb-persistenceids-{type}-{Guid.NewGuid()}.db;Mode=Memory;Cache=Shared"; - public static string ConnectionString(int id) => $"Filename=file:memdb-persistenceids-{id}.db;Mode=Memory;Cache=Shared"; - - public static Config Config(int id) => ConfigurationFactory.ParseString($@" + public static Config Config => ConfigurationFactory.ParseString($@" akka.loglevel = INFO akka.actor{{ serializers{{ @@ -41,7 +40,7 @@ class = ""Akka.Persistence.Sqlite.Journal.SqliteJournal, Akka.Persistence.Sqlite table-name = event_journal metadata-table-name = journal_metadata auto-initialize = on - connection-string = ""{ConnectionString(id)}"" + connection-string = ""{ConnectionString("journal")}"" refresh-interval = 200ms }} }} @@ -52,14 +51,14 @@ class = ""Akka.Persistence.Sqlite.Snapshot.SqliteSnapshotStore, Akka.Persistence plugin-dispatcher = ""akka.actor.default-dispatcher"" table-name = snapshot_store auto-initialize = on - connection-string = ""{ConnectionString(id)}"" + connection-string = ""{ConnectionString("snapshot")}"" }} }} }} akka.test.single-expect-default = 10s") .WithFallback(SqlReadJournal.DefaultConfiguration()); - public SqlitePersistenceIdsSpec(ITestOutputHelper output) : base(Config(Counter.GetAndIncrement()), nameof(SqlitePersistenceIdsSpec), output) + public SqlitePersistenceIdsSpec(ITestOutputHelper output) : base(Config, nameof(SqlitePersistenceIdsSpec), output) { ReadJournal = Sys.ReadJournalFor(SqlReadJournal.Identifier); }