From 8ed429c4f8961f4e2eb7b6f04651ab546aacea48 Mon Sep 17 00:00:00 2001 From: Ruben Bartelink Date: Thu, 25 Aug 2022 14:10:22 +0100 Subject: [PATCH] V3 cleanup and signature changes (#169) Lots of restructuring prompted by a) signature changes driven by profiling b) implementation of FeedSourceBase.AwaitCompletion See also #116: cleanup naming and signatures for Sinks rename ProjectorPipeline<'T> -> Sink name Ingester batch tuples Restructure Scheduler modules rename Feed.Internal to Feed.Core convert all Tuples to struct tuples change StreamSpan from a record to a type alias Linting e.g. remove ., ref etc Extensive profiling-driven impl changes --- CHANGELOG.md | 3 + README.md | 6 +- src/Propulsion.Cosmos/CosmosPruner.fs | 36 +- src/Propulsion.Cosmos/CosmosSink.fs | 81 +- .../Propulsion.Cosmos.fsproj | 1 + .../CosmosStoreParser.fs | 31 +- .../CosmosStorePruner.fs | 43 +- src/Propulsion.CosmosStore/CosmosStoreSink.fs | 99 +- .../CosmosStoreSource.fs | 31 +- .../ReaderCheckpoint.fs | 4 +- .../Propulsion.CosmosStore3.fsproj | 1 + src/Propulsion.DynamoStore/AppendsEpoch.fs | 2 +- src/Propulsion.DynamoStore/DynamoDbExport.fs | 2 +- .../DynamoStoreIndex.fs | 7 +- .../DynamoStoreSource.fs | 25 +- src/Propulsion.DynamoStore/Types.fs | 10 +- src/Propulsion.EventStore/EventStoreReader.fs | 63 +- src/Propulsion.EventStore/EventStoreSink.fs | 99 +- src/Propulsion.EventStore/EventStoreSource.fs | 8 +- .../Propulsion.EventStore.fsproj | 2 +- src/Propulsion.EventStore/StripedIngester.fs | 42 +- .../EventStoreSource.fs | 17 +- src/Propulsion.Feed/FeedPrometheus.fs | 2 +- src/Propulsion.Feed/FeedReader.fs | 20 +- src/Propulsion.Feed/FeedSource.fs | 45 +- src/Propulsion.Feed/PeriodicSource.fs | 23 +- src/Propulsion.Kafka/Codec.fs | 38 +- src/Propulsion.Kafka/Consumers.fs | 145 +- src/Propulsion.Kafka/ProducerSinks.fs | 26 +- src/Propulsion.Kafka/Producers.fs | 22 +- .../MemoryStoreSource.fs | 72 +- .../SqlStreamStoreSource.fs | 18 +- src/Propulsion/Feed.fs | 2 +- src/Propulsion/Infrastructure.fs | 5 + src/Propulsion/Ingestion.fs | 66 +- src/Propulsion/Internal.fs | 127 +- src/Propulsion/Parallel.fs | 60 +- src/Propulsion/{Projector.fs => Pipeline.fs} | 4 +- src/Propulsion/Propulsion.fsproj | 2 +- src/Propulsion/Streams.fs | 1503 ++++++++--------- src/Propulsion/Submission.fs | 83 +- .../ConsumersIntegration.fs | 29 +- tests/Propulsion.Tests/ProgressTests.fs | 18 +- tests/Propulsion.Tests/StreamStateTests.fs | 59 +- tools/Propulsion.Tool/Infrastructure.fs | 2 +- tools/Propulsion.Tool/Program.fs | 15 +- 46 files changed, 1497 insertions(+), 1502 deletions(-) rename src/Propulsion/{Projector.fs => Pipeline.fs} (96%) diff --git a/CHANGELOG.md b/CHANGELOG.md index f8cb675f..21cb1f8a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -24,6 +24,9 @@ The `Unreleased` section name is replaced by the expected version of next releas ### Changed - Targeted `net6.0` with `6.0.300` SDK, `FSharp.Core` v `6.0.0`, `FSharp.Control.AsyncSeq` v `3.2.1`, `MathNet.Numerics` v `4.15.0` +- Changed all `Tuple` types to `struct` tuples (`System.ValueTuple`) [#169](https://github.com/jet/propulsion/pull/169) +- Changed dominant `ITimelineEvent` EventBody type from `byte array` to `System.ReadOnlyMemory` [#169](https://github.com/jet/propulsion/pull/169) +- `StreamSpan`: Changed from a record to a type alias [#169](https://github.com/jet/propulsion/pull/169) - `Propulsion.CosmosStore`: Changed to target `Equinox.CosmosStore` v `4.0.0` [#139](https://github.com/jet/propulsion/pull/139) - `Propulsion.CosmosStore.CosmosSource`: Changed parsing to use `System.Text.Json` [#139](https://github.com/jet/propulsion/pull/139) - `Propulsion.EventStore`: Pinned to target `Equinox.EventStore` v `[3.0.7`-`3.99.0]` **Deprecated; Please migrate to `Propulsion.EventStoreDb`** [#139](https://github.com/jet/propulsion/pull/139) diff --git a/README.md b/README.md index b5e90f60..c1fad535 100644 --- a/README.md +++ b/README.md @@ -14,7 +14,7 @@ The components within this repository are delivered as a multi-targeted Nuget pa - `Propulsion.MemoryStore` [![NuGet](https://img.shields.io/nuget/v/Propulsion.MemoryStore.svg)](https://www.nuget.org/packages/Propulsion.MemoryStore/). Provides bindings to `Equinox.MemoryStore`. [Depends](https://www.fuget.org/packages/Propulsion.MemoryStore) on `Equinox.MemoryStore` v `4.0.0`, `FsCodec.Box`, `Propulsion` - 1. `MemoryStoreSource`: Forwarding from an `Equinox.MemoryStore` into a `Propulsion.ProjectorPipeline`, in order to enable maximum speed integration testing. + 1. `MemoryStoreSource`: Forwarding from an `Equinox.MemoryStore` into a `Propulsion.Sink`, in order to enable maximum speed integration testing. - `Propulsion.CosmosStore` [![NuGet](https://img.shields.io/nuget/v/Propulsion.CosmosStore.svg)](https://www.nuget.org/packages/Propulsion.CosmosStore/) Provides bindings to Azure CosmosDB. [Depends](https://www.fuget.org/packages/Propulsion.CosmosStore) on `Equinox.CosmosStore` v `4.0.0` @@ -45,7 +45,7 @@ The components within this repository are delivered as a multi-targeted Nuget pa - `Propulsion.DynamoStore.Constructs` [![NuGet](https://img.shields.io/nuget/v/Propulsion.DynamoStore.Constructs.svg)](https://www.nuget.org/packages/Propulsion.DynamoStore.Constructs/) AWS Lambda CDK deploy logic. [Depends](https://www.fuget.org/packages/Propulsion.DynamoStore.Constructs) on `Amazon.CDK.Lib` (and, indirectly, on the binary assets included as content in `Propulsion.DynamoStore.Lambda`) - `Propulsion.EventStoreDb` [![NuGet](https://img.shields.io/nuget/v/Propulsion.EventStoreDb.svg)](https://www.nuget.org/packages/Propulsion.EventStoreDb/). Provides bindings to [EventStore](https://www.eventstore.org), writing via `Propulsion.EventStore.EventStoreSink` [Depends](https://www.fuget.org/packages/Propulsion.EventStoreDb) on `Equinox.EventStoreDb` v `4.0.0`, `Serilog` - 1. `EventStoreSource`: reading from an EventStoreDB >= `20.10` `$all` stream into a `Propulsion.ProjectorPipeline` using the gRPC interface. Provides throughput metrics via `Propulsion.Feed.Prometheus` + 1. `EventStoreSource`: reading from an EventStoreDB >= `20.10` `$all` stream into a `Propulsion.Sink` using the gRPC interface. Provides throughput metrics via `Propulsion.Feed.Prometheus` 2. `EventStoreSink`: writing to `Equinox.EventStoreDb` v `4.0.0` (Reading and position metrics are exposed via `Propulsion.Feed.Prometheus`) @@ -60,7 +60,7 @@ The components within this repository are delivered as a multi-targeted Nuget pa - `Propulsion.SqlStreamStore` [![NuGet](https://img.shields.io/nuget/v/Propulsion.SqlStreamStore.svg)](https://www.nuget.org/packages/Propulsion.SqlStreamStore/). Provides bindings to [SqlStreamStore](https://github.com/SQLStreamStore/SQLStreamStore), maintaining checkpoints in a SQL table using Dapper [Depends](https://www.fuget.org/packages/Propulsion.SqlStreamStore) on `Propulsion.Feed`, `SqlStreamStore`, `Dapper` v `2.0`, `Microsoft.Data.SqlClient` v `1.1.3`, `Serilog` - 1. `SqlStreamStoreSource`: reading from a SqlStreamStore `$all` stream into a `Propulsion.ProjectorPipeline` + 1. `SqlStreamStoreSource`: reading from a SqlStreamStore `$all` stream into a `Propulsion.Sink` 2. `ReaderCheckpoint`: checkpoint storage for `Propulsion.Feed`/`SqlStreamSteamStore`/`EventStoreDb` using `Dapper`, `Microsoft.Data.SqlClient` (Reading and position metrics are exposed via `Propulsion.Feed.Prometheus`) diff --git a/src/Propulsion.Cosmos/CosmosPruner.fs b/src/Propulsion.Cosmos/CosmosPruner.fs index 2bdc0347..7659dfb9 100644 --- a/src/Propulsion.Cosmos/CosmosPruner.fs +++ b/src/Propulsion.Cosmos/CosmosPruner.fs @@ -78,9 +78,9 @@ module Pruner = Equinox.Cosmos.Store.Log.InternalMetrics.dump log // Per set of accumulated events per stream (selected via `selectExpired`), attempt to prune up to the high water mark - let handle pruneUntil (stream, span: StreamSpan<_>) = async { + let handle pruneUntil struct (stream, span: StreamSpan<_>) = async { // The newest event eligible for deletion defines the cutoff point - let untilIndex = span.events[span.events.Length - 1].Index + let untilIndex = span[span.Length - 1].Index // Depending on the way the events are batched, requests break into three groupings: // 1. All requested events already deleted, no writes took place // (if trimmedPos is beyond requested Index, Propulsion will discard the requests via the OverrideWritePosition) @@ -91,20 +91,20 @@ module Pruner = // in this case, we mark the event as handled and await a successor event triggering another attempt let! deleted, deferred, trimmedPos = pruneUntil (FsCodec.StreamName.toString stream) untilIndex // Categorize the outcome so the stats handler can summarize the work being carried out - let res = if deleted = 0 && deferred = 0 then Nop span.events.Length else Ok (deleted, deferred) + let res = if deleted = 0 && deferred = 0 then Nop span.Length else Ok (deleted, deferred) // For case where we discover events have already been deleted beyond our requested position, signal to reader to drop events let writePos = max trimmedPos (untilIndex + 1L) - return writePos, res + return struct (writePos, res) } type StreamSchedulingEngine = static member Create(pruneUntil, itemDispatcher, stats : Stats, dumpStreams, ?maxBatches, ?purgeInterval, ?wakeForResults, ?idleDelay) - : Scheduling.StreamSchedulingEngine<_, _, _> = - let interpret (stream, span) = - let stats = Buffering.StreamSpan.stats span - stats, (stream, span) - let dispatcher = Scheduling.MultiDispatcher<_, _, _>.Create(itemDispatcher, handle pruneUntil, interpret, (fun _ -> id), stats, dumpStreams) + : Scheduling.StreamSchedulingEngine<_, _, _, _> = + let interpret struct (stream, span) = + let metrics = StreamSpan.metrics Default.eventSize span + struct (metrics, struct (stream, span)) + let dispatcher = Scheduling.Dispatcher.MultiDispatcher<_, _, _, _>.Create(itemDispatcher, handle pruneUntil, interpret, (fun _ -> id), stats, dumpStreams) Scheduling.StreamSchedulingEngine( dispatcher, ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay, @@ -114,7 +114,7 @@ module Pruner = type CosmosPruner = /// DANGER: this API DELETES events - use with care - /// Starts a StreamsProjectorPipeline that prunes _all submitted events from the supplied context_ + /// Starts a Sink that prunes _all submitted events from the supplied context_ static member Start ( log : ILogger, maxReadAhead, context, maxConcurrentStreams, // Default 5m @@ -122,24 +122,20 @@ type CosmosPruner = // Default 5m ?stateInterval, ?maxSubmissionsPerPartition, - ?maxBatches, ?purgeInterval, ?wakeForResults, - // Delay when no items available. Default 10ms. - ?idleDelay, + ?maxBatches, ?purgeInterval, ?wakeForResults, ?idleDelay, // Defaults to statsInterval ?ingesterStatsInterval) - : Propulsion.ProjectorPipeline<_> = - let idleDelay = defaultArg idleDelay (TimeSpan.FromMilliseconds 10.) + : Default.Sink = let statsInterval, stateInterval = defaultArg statsInterval (TimeSpan.FromMinutes 5.), defaultArg stateInterval (TimeSpan.FromMinutes 5.) let stats = Pruner.Stats(log.ForContext(), statsInterval, stateInterval) - let dispatcher = Scheduling.ItemDispatcher<_>(maxConcurrentStreams) - let dumpStreams struct (s : Scheduling.StreamStates<_>, totalPruned) log = - s.Dump(log, totalPruned, Buffering.StreamState.eventsSize) + let dispatcher = Dispatch.ItemDispatcher<_, _>(maxConcurrentStreams) + let dumpStreams logStreamStates _log = logStreamStates Default.eventSize let pruneUntil stream index = Equinox.Cosmos.Core.Events.pruneUntil context stream index let streamScheduler = Pruner.StreamSchedulingEngine.Create( pruneUntil, dispatcher, stats, dumpStreams, - ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, idleDelay = idleDelay) - Projector.StreamsProjectorPipeline.Start( + ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay) + Projector.Pipeline.Start( log, dispatcher.Pump, streamScheduler.Pump, maxReadAhead, streamScheduler.Submit, statsInterval, ?maxSubmissionsPerPartition = maxSubmissionsPerPartition, ?ingesterStatsInterval = ingesterStatsInterval) diff --git a/src/Propulsion.Cosmos/CosmosSink.fs b/src/Propulsion.Cosmos/CosmosSink.fs index b255862d..479ba606 100644 --- a/src/Propulsion.Cosmos/CosmosSink.fs +++ b/src/Propulsion.Cosmos/CosmosSink.fs @@ -4,16 +4,16 @@ open Equinox.Cosmos.Core open Equinox.Cosmos.Store open FsCodec open Propulsion +open Propulsion.Internal // Helpers open Propulsion.Streams -open Propulsion.Streams.Internal // Helpers open Serilog open System.Collections.Generic open System open System.Threading -[] -module private Impl = - let inline mb x = float x / 1024. / 1024. +module private StreamSpan = + + let defaultToNative_ = FsCodec.Core.TimelineEvent.Map (fun (xs : ReadOnlyMemory) -> xs.ToArray()) module Internal = @@ -24,34 +24,34 @@ module Internal = type [] Result = | Ok of updatedPos : int64 | Duplicate of updatedPos : int64 - | PartialDuplicate of overage : StreamSpan - | PrefixMissing of batch : StreamSpan * writePos : int64 - let logTo (log : ILogger) malformed (res : StreamName * Choice) = + | PartialDuplicate of overage : Default.StreamSpan + | PrefixMissing of batch : Default.StreamSpan * writePos : int64 + let logTo (log : ILogger) malformed (res : StreamName * Choice) = match res with | stream, Choice1Of2 (_, Ok pos) -> log.Information("Wrote {stream} up to {pos}", stream, pos) | stream, Choice1Of2 (_, Duplicate updatedPos) -> log.Information("Ignored {stream} (synced up to {pos})", stream, updatedPos) | stream, Choice1Of2 (_, PartialDuplicate overage) -> - log.Information("Requeuing {stream} {pos} ({count} events)", stream, overage.index, overage.events.Length) + log.Information("Requeuing {stream} {pos} ({count} events)", stream, overage[0].Index, overage.Length) | stream, Choice1Of2 (_, PrefixMissing (batch, pos)) -> - log.Information("Waiting {stream} missing {gap} events ({count} events @ {pos})", stream, batch.index-pos, batch.events.Length, batch.index) + log.Information("Waiting {stream} missing {gap} events ({count} events @ {pos})", stream, batch[0].Index - pos, batch.Length, batch[0].Index) | stream, Choice2Of2 (_, exn) -> let level = if malformed then Events.LogEventLevel.Warning else Events.LogEventLevel.Information log.Write(level, exn, "Writing {stream} failed, retrying", stream) - let write (log : ILogger) (ctx : Context) stream span = async { - log.Debug("Writing {s}@{i}x{n}", stream, span.index, span.events.Length) + let write (log : ILogger) (ctx : Context) stream (span : Default.StreamSpan) = async { + log.Debug("Writing {s}@{i}x{n}", stream, span[0].Index, span.Length) let stream = ctx.CreateStream stream - let! res = ctx.Sync(stream, { index = span.index; etag = None }, span.events |> Array.map (fun x -> x :> _)) + let! res = ctx.Sync(stream, { index = span[0].Index; etag = None }, span |> Array.map (fun x -> StreamSpan.defaultToNative_ x :> _)) let res' = match res with | AppendResult.Ok pos -> Ok pos.index | AppendResult.Conflict (pos, _) | AppendResult.ConflictUnknown pos -> match pos.index with - | actual when actual < span.index -> PrefixMissing (span, actual) - | actual when actual >= span.index + span.events.LongLength -> Duplicate actual - | actual -> PartialDuplicate { index = actual; events = span.events |> Array.skip (actual-span.index |> int) } + | actual when actual < span[0].Index -> PrefixMissing (span, actual) + | actual when actual >= span[0].Index + span.LongLength -> Duplicate actual + | actual -> PartialDuplicate (span |> Array.skip (actual - span[0].Index |> int)) log.Debug("Result: {res}", res') return res' } let (|TimedOutMessage|RateLimitedMessage|TooLargeMessage|MalformedMessage|Other|) (e : exn) = @@ -77,9 +77,9 @@ module Internal = | ResultKind.TooLarge | ResultKind.Malformed -> true type Stats(log : ILogger, statsInterval, stateInterval) = - inherit Scheduling.Stats(log, statsInterval, stateInterval) + inherit Scheduling.Stats(log, statsInterval, stateInterval) let mutable okStreams, resultOk, resultDup, resultPartialDup, resultPrefix, resultExnOther = HashSet(), 0, 0, 0, 0, 0 - let mutable badCats, failStreams, rateLimited, timedOut, tooLarge, malformed = CatStats(), HashSet(), 0, 0, 0, 0 + let mutable badCats, failStreams, rateLimited, timedOut, tooLarge, malformed = Stats.CatStats(), HashSet(), 0, 0, 0, 0 let rlStreams, toStreams, tlStreams, mfStreams, oStreams = HashSet(), HashSet(), HashSet(), HashSet(), HashSet() let mutable okEvents, okBytes, exnEvents, exnBytes = 0, 0L, 0, 0L @@ -134,40 +134,42 @@ module Internal = type StreamSchedulingEngine = static member Create( - log : ILogger, cosmosContexts : _ [], itemDispatcher, stats : Stats, dumpStreams, - ?maxBatches, ?purgeInterval, ?wakeForResults, ?idleDelay, ?maxEvents, ?maxBytes, ?prioritizeLargePayloads) - : Scheduling.StreamSchedulingEngine<_, _, _> = + log : ILogger, cosmosContexts : _ [], itemDispatcher, stats : Stats, dumpStreams, + ?maxBatches, ?purgeInterval, ?wakeForResults, ?idleDelay, ?maxEvents, ?maxBytes) + : Scheduling.StreamSchedulingEngine<_, _, _, _> = let maxEvents, maxBytes = defaultArg maxEvents 16384, defaultArg maxBytes (1024 * 1024 - (*fudge*)4096) let writerResultLog = log.ForContext() let mutable robin = 0 - let attemptWrite (stream, span) ct = task { + let attemptWrite struct (stream, span) ct = task { let index = Interlocked.Increment(&robin) % cosmosContexts.Length let selectedConnection = cosmosContexts[index] - let met, span' = Buffering.StreamSpan.slice (maxEvents, maxBytes) span + let struct (met, span') = StreamSpan.slice Default.jsonSize (maxEvents, maxBytes) span try let! res = Writer.write log selectedConnection (StreamName.toString stream) span' |> fun f -> Async.StartAsTask(f, cancellationToken = ct) - return span'.events.Length > 0, Choice1Of2 (met, res) - with e -> return false, Choice2Of2 (met, e) } + return struct (span'.Length > 0, Choice1Of2 struct (met, res)) + with e -> return false, Choice2Of2 struct (met, e) } let interpretWriteResultProgress (streams: Scheduling.StreamStates<_>) stream res = let applyResultToStreamState = function - | Choice1Of2 (_stats, Writer.Ok pos) -> streams.InternalUpdate stream pos null, false - | Choice1Of2 (_stats, Writer.Duplicate pos) -> streams.InternalUpdate stream pos null, false - | Choice1Of2 (_stats, Writer.PartialDuplicate overage) -> streams.InternalUpdate stream overage.index [|overage|], false - | Choice1Of2 (_stats, Writer.PrefixMissing (overage, pos)) -> streams.InternalUpdate stream pos [|overage|], false - | Choice2Of2 (_stats, exn) -> + | Choice1Of2 struct (_stats, Writer.Ok pos) -> struct (streams.RecordWriteProgress(stream, pos, null), false) + | Choice1Of2 (_stats, Writer.Duplicate pos) -> streams.RecordWriteProgress(stream, pos, null), false + | Choice1Of2 (_stats, Writer.PartialDuplicate overage) -> streams.RecordWriteProgress(stream, overage[0].Index, [| overage |]), false + | Choice1Of2 (_stats, Writer.PrefixMissing (overage, pos)) -> streams.RecordWriteProgress(stream, pos, [| overage |]), false + | Choice2Of2 struct (_stats, exn) -> let malformed = Writer.classify exn |> Writer.isMalformed streams.SetMalformed(stream, malformed), malformed - let (_stream, ss), malformed = applyResultToStreamState res + let struct (ss, malformed) = applyResultToStreamState res Writer.logTo writerResultLog malformed (stream, res) - ss.Write, res - let dispatcher = Scheduling.MultiDispatcher<_, _, _>.Create(itemDispatcher, attemptWrite, interpretWriteResultProgress, stats, dumpStreams) + struct (ss.WritePos, res) + let dispatcher = + Scheduling.Dispatcher.MultiDispatcher<_, _, _, _> + .Create(itemDispatcher, attemptWrite, interpretWriteResultProgress, stats, dumpStreams) Scheduling.StreamSchedulingEngine( dispatcher, ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay, - enableSlipstreaming = true, ?prioritizeLargePayloads = prioritizeLargePayloads) + enableSlipstreaming = true, prioritizeStreamsBy = Default.eventDataSize) type CosmosSink = - /// Starts a StreamsProjectorPipeline that ingests all submitted events into the supplied context + /// Starts a Sink that ingests all submitted events into the supplied context static member Start ( log : ILogger, maxReadAhead, cosmosContexts, maxConcurrentStreams, // Default 5m @@ -181,18 +183,17 @@ type CosmosSink = // Default: 1MB (limited by maximum size of a CosmosDB stored procedure invocation) ?maxBytes, ?ingesterStatsInterval) - : ProjectorPipeline<_> = + : Default.Sink = let statsInterval, stateInterval = defaultArg statsInterval (TimeSpan.FromMinutes 5.), defaultArg stateInterval (TimeSpan.FromMinutes 5.) let stats = Internal.Stats(log.ForContext(), statsInterval, stateInterval) - let dispatcher = Scheduling.ItemDispatcher<_>(maxConcurrentStreams) - let dumpStreams struct (s : Scheduling.StreamStates<_>, totalPruned) log = - s.Dump(log, totalPruned, Buffering.StreamState.eventsSize) + let dispatcher = Dispatch.ItemDispatcher<_, _>(maxConcurrentStreams) + let dumpStreams logStreamStates _log = logStreamStates Default.eventSize let streamScheduler = Internal.StreamSchedulingEngine.Create( log, cosmosContexts, dispatcher, stats, dumpStreams, ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay, - ?maxEvents=maxEvents, ?maxBytes=maxBytes, prioritizeLargePayloads = true) - Projector.StreamsProjectorPipeline.Start( + ?maxEvents=maxEvents, ?maxBytes = maxBytes) + Projector.Pipeline.Start( log, dispatcher.Pump, streamScheduler.Pump, maxReadAhead, streamScheduler.Submit, statsInterval, ?maxSubmissionsPerPartition = maxSubmissionsPerPartition, ?ingesterStatsInterval = ingesterStatsInterval) diff --git a/src/Propulsion.Cosmos/Propulsion.Cosmos.fsproj b/src/Propulsion.Cosmos/Propulsion.Cosmos.fsproj index a278c9d2..4037b7b4 100644 --- a/src/Propulsion.Cosmos/Propulsion.Cosmos.fsproj +++ b/src/Propulsion.Cosmos/Propulsion.Cosmos.fsproj @@ -31,6 +31,7 @@ + diff --git a/src/Propulsion.CosmosStore/CosmosStoreParser.fs b/src/Propulsion.CosmosStore/CosmosStoreParser.fs index ce5e5519..2eb08a81 100644 --- a/src/Propulsion.CosmosStore/CosmosStoreParser.fs +++ b/src/Propulsion.CosmosStore/CosmosStoreParser.fs @@ -29,10 +29,26 @@ module EquinoxSystemTextJsonParser = unixEpoch.AddSeconds(ts.GetDouble()) /// Sanity check to determine whether the Document represents an `Equinox.Cosmos` >= 1.0 based batch - let isEquinoxBatch (d : System.Text.Json.JsonDocument) = + let tryParseEquinoxBatch streamFilter (d : System.Text.Json.JsonDocument) = let r = d.RootElement - let hasProp (id : string) = match r.TryGetProperty id with true, _ -> true | _ -> false - hasProp "p" && hasProp "i" && hasProp "n" && hasProp "e" + let tryProp (id : string) : ValueOption = + let mutable p = Unchecked.defaultof<_> + if r.TryGetProperty(id, &p) then ValueSome p else ValueNone + let hasProp (id : string) : bool = tryProp id |> ValueOption.isSome + + match tryProp "p" with + | ValueSome je when je.ValueKind = System.Text.Json.JsonValueKind.String && hasProp "i" && hasProp "n" && hasProp "e" -> + let streamName = je.GetString() |> FsCodec.StreamName.parse // we expect all Equinox data to adhere to "{category}-{aggregateId}" form (or we'll throw) + if streamFilter streamName then ValueSome (struct (streamName, d.Cast())) else ValueNone + | _ -> ValueNone + + /// Enumerates the events represented within a batch + let enumEquinoxCosmosEvents struct (streamName, batch : Batch) : Default.StreamEvent seq = + batch.e |> Seq.mapi (fun offset x -> streamName, FsCodec.Core.TimelineEvent.Create(batch.i + int64 offset, x.c, batch.MapData x.d, batch.MapData x.m, timestamp = x.t)) + + /// Collects all events with a Document [typically obtained via the CosmosDb ChangeFeed] that potentially represents an Equinox.Cosmos event-batch + let enumStreamEvents streamFilter d : Default.StreamEvent seq = + tryParseEquinoxBatch streamFilter d |> ValueOption.map enumEquinoxCosmosEvents |> ValueOption.defaultValue Seq.empty #else #if COSMOSV2 module EquinoxCosmosParser = @@ -65,16 +81,15 @@ module EquinoxNewtonsoftParser = /// Sanity check to determine whether the Document represents an `Equinox.Cosmos` >= 1.0 based batch let isEquinoxBatch (d : Newtonsoft.Json.Linq.JObject) = d.ContainsKey "p" && d.ContainsKey "i" && d.ContainsKey "n" && d.ContainsKey "e" -#endif #endif /// Enumerates the events represented within a batch - let enumEquinoxCosmosEvents (batch : Batch) : StreamEvent seq = + let enumEquinoxCosmosEvents (batch : Batch) : Default.StreamEvent seq = let streamName = FsCodec.StreamName.parse batch.p // we expect all Equinox data to adhere to "{category}-{aggregateId}" form (or we'll throw) - batch.e |> Seq.mapi (fun offset x -> { stream = streamName; event = FsCodec.Core.TimelineEvent.Create(batch.i + int64 offset, x.c, batch.MapData x.d, batch.MapData x.m, timestamp=x.t) }) + batch.e |> Seq.mapi (fun offset x -> streamName, FsCodec.Core.TimelineEvent.Create(batch.i + int64 offset, x.c, batch.MapData x.d, batch.MapData x.m, timestamp=x.t)) /// Collects all events with a Document [typically obtained via the CosmosDb ChangeFeed] that potentially represents an Equinox.Cosmos event-batch - let enumStreamEvents d : StreamEvent seq = + let enumStreamEvents d : Default.StreamEvent seq = if isEquinoxBatch d then d.Cast() |> enumEquinoxCosmosEvents else Seq.empty - +#endif diff --git a/src/Propulsion.CosmosStore/CosmosStorePruner.fs b/src/Propulsion.CosmosStore/CosmosStorePruner.fs index da8c02a0..384e47cc 100644 --- a/src/Propulsion.CosmosStore/CosmosStorePruner.fs +++ b/src/Propulsion.CosmosStore/CosmosStorePruner.fs @@ -1,6 +1,7 @@ // Implements a Sink that removes every submitted event (and all preceding events) from the relevant stream namespace Propulsion.CosmosStore +open Propulsion.Streams open Serilog open System open System.Collections.Generic @@ -54,7 +55,7 @@ module Pruner = let inline adds x (set:HashSet<_>) = set.Add x |> ignore base.Handle message match message with - | Propulsion.Streams.Scheduling.InternalMessage.Result (_duration, stream, _progressed, Choice2Of2 (_, exn)) -> + | Scheduling.InternalMessage.Result (_duration, stream, _progressed, Choice2Of2 (_, exn)) -> match classify exn with | ExceptionKind.RateLimited -> adds stream rlStreams; rateLimited <- rateLimited + 1 @@ -76,9 +77,9 @@ module Pruner = Equinox.CosmosStore.Core.Log.InternalMetrics.dump log // Per set of accumulated events per stream (selected via `selectExpired`), attempt to prune up to the high water mark - let handle pruneUntil (stream, span: Propulsion.Streams.StreamSpan<_>) = async { + let handle pruneUntil struct (stream, span: Propulsion.Streams.StreamSpan<_>) = async { // The newest event eligible for deletion defines the cutoff point - let untilIndex = span.events[span.events.Length - 1].Index + let untilIndex = span[span.Length - 1].Index // Depending on the way the events are batched, requests break into three groupings: // 1. All requested events already deleted, no writes took place // (if trimmedPos is beyond requested Index, Propulsion will discard the requests via the OverrideWritePosition) @@ -89,21 +90,23 @@ module Pruner = // in this case, we mark the event as handled and await a successor event triggering another attempt let! deleted, deferred, trimmedPos = pruneUntil (FsCodec.StreamName.toString stream) untilIndex // Categorize the outcome so the stats handler can summarize the work being carried out - let res = if deleted = 0 && deferred = 0 then Nop span.events.Length else Ok (deleted, deferred) + let res = if deleted = 0 && deferred = 0 then Nop span.Length else Ok (deleted, deferred) // For case where we discover events have already been deleted beyond our requested position, signal to reader to drop events let writePos = max trimmedPos (untilIndex + 1L) - return writePos, res + return struct (writePos, res) } type StreamSchedulingEngine = static member Create(pruneUntil, itemDispatcher, stats : Stats, dumpStreams, ?maxBatches, ?purgeInterval, ?wakeForResults, ?idleDelay) - : Propulsion.Streams.Scheduling.StreamSchedulingEngine<_, _, _> = - let interpret (stream, span) = - let stats = Propulsion.Streams.Buffering.StreamSpan.stats span - stats, (stream, span) - let dispatcher = Propulsion.Streams.Scheduling.MultiDispatcher<_, _, _>.Create(itemDispatcher, handle pruneUntil, interpret, (fun _ -> id), stats, dumpStreams) - Propulsion.Streams.Scheduling.StreamSchedulingEngine( + : Scheduling.StreamSchedulingEngine<_, _, _, _> = + let interpret struct (stream, span) = + let metrics = StreamSpan.metrics Default.eventSize span + struct (metrics, struct (stream, span)) + let dispatcher = + Scheduling.Dispatcher.MultiDispatcher<_, _, _, _> + .Create(itemDispatcher, handle pruneUntil, interpret, (fun _ -> id), stats, dumpStreams) + Scheduling.StreamSchedulingEngine( dispatcher, ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay, enableSlipstreaming = false) @@ -112,7 +115,7 @@ module Pruner = type CosmosStorePruner = /// DANGER: this API DELETES events - use with care - /// Starts a StreamsProjectorPipeline that prunes _all submitted events from the supplied context_ + /// Starts a Sink that prunes _all submitted events from the supplied context_ static member Start ( log : ILogger, maxReadAhead, context, maxConcurrentStreams, // Default 5m @@ -120,24 +123,20 @@ type CosmosStorePruner = // Default 5m ?stateInterval, ?maxSubmissionsPerPartition, - ?maxBatches, ?purgeInterval, ?wakeForResults, - // Delay when no items available. Default 10ms. - ?idleDelay, + ?maxBatches, ?purgeInterval, ?wakeForResults, ?idleDelay, // Defaults to statsInterval ?ingesterStatsInterval) - : Propulsion.ProjectorPipeline<_> = - let idleDelay = defaultArg idleDelay (TimeSpan.FromMilliseconds 10.) + : Default.Sink = let statsInterval, stateInterval = defaultArg statsInterval (TimeSpan.FromMinutes 5.), defaultArg stateInterval (TimeSpan.FromMinutes 5.) let stats = Pruner.Stats(log.ForContext(), statsInterval, stateInterval) - let dispatcher = Propulsion.Streams.Scheduling.ItemDispatcher<_>(maxConcurrentStreams) - let dumpStreams struct (s : Propulsion.Streams.Scheduling.StreamStates<_>, totalPurged) logger = - s.Dump(logger, totalPurged, Propulsion.Streams.Buffering.StreamState.eventsSize) + let dispatcher = Dispatch.ItemDispatcher<_, _>(maxConcurrentStreams) + let dumpStreams logStreamStates _log = logStreamStates Default.eventSize let pruneUntil stream index = Equinox.CosmosStore.Core.Events.pruneUntil context stream index let streamScheduler = Pruner.StreamSchedulingEngine.Create( pruneUntil, dispatcher, stats, dumpStreams, - ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, idleDelay = idleDelay) - Propulsion.Streams.Projector.StreamsProjectorPipeline.Start( + ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay) + Projector.Pipeline.Start( log, dispatcher.Pump, streamScheduler.Pump, maxReadAhead, streamScheduler.Submit, statsInterval, ?maxSubmissionsPerPartition = maxSubmissionsPerPartition, ?ingesterStatsInterval = ingesterStatsInterval) diff --git a/src/Propulsion.CosmosStore/CosmosStoreSink.fs b/src/Propulsion.CosmosStore/CosmosStoreSink.fs index a98fd217..7d97e3eb 100644 --- a/src/Propulsion.CosmosStore/CosmosStoreSink.fs +++ b/src/Propulsion.CosmosStore/CosmosStoreSink.fs @@ -2,26 +2,30 @@ namespace Propulsion.CosmosStore open Equinox.CosmosStore.Core open FsCodec -open Propulsion +open Propulsion.Internal // Helpers open Propulsion.Streams -open Propulsion.Streams.Internal // Helpers open Serilog open System open System.Collections.Generic [] module private Impl = - let inline mb x = float x / 1024. / 1024. -#if COSMOSV3 || COSMOSV2 - // The event bodies the store deals with are byte arrays - let toNativeEvent = id + +#if COSMOSV3 + type EventBody = byte array // V4 defines one directly, here we shim it + module StreamSpan = + + let private toNativeEventBody (xs : Default.EventBody) : byte array = xs.ToArray() + let defaultToNative_ = FsCodec.Core.TimelineEvent.Map toNativeEventBody #else - // v4 and later use JsonElement, but Propulsion is sticking with byte arrays until 3.x (at which point it'll probably shift to ReadOnlyMemory rather than assuming and/or offering optimization for JSON bodies) - open System.Text.Json - let toNativeEventBody (x : byte[]) : JsonElement = - if x = null then JsonElement() - else JsonSerializer.Deserialize(System.ReadOnlySpan.op_Implicit x) - let toNativeEvent = FsCodec.Core.TimelineEvent.Map toNativeEventBody + module StreamSpan = + + // v4 and later use JsonElement, but Propulsion is sticking with byte arrays until 3.x (at which point it'll probably shift to ReadOnlyMemory rather than assuming and/or offering optimization for JSON bodies) + open System.Text.Json + let private toNativeEventBody (x : Default.EventBody) : JsonElement = + if x.IsEmpty then JsonElement() + else JsonSerializer.Deserialize(x.Span) + let defaultToNative_ = FsCodec.Core.TimelineEvent.Map toNativeEventBody #endif module Internal = @@ -33,33 +37,33 @@ module Internal = type [] Result = | Ok of updatedPos : int64 | Duplicate of updatedPos : int64 - | PartialDuplicate of overage : StreamSpan - | PrefixMissing of batch : StreamSpan * writePos : int64 - let logTo (log : ILogger) malformed (res : StreamName * Choice) = + | PartialDuplicate of overage : StreamSpan + | PrefixMissing of batch : StreamSpan * writePos : int64 + let logTo (log : ILogger) malformed (res : StreamName * Choice) = match res with | stream, Choice1Of2 (_, Ok pos) -> log.Information("Wrote {stream} up to {pos}", stream, pos) | stream, Choice1Of2 (_, Duplicate updatedPos) -> log.Information("Ignored {stream} (synced up to {pos})", stream, updatedPos) | stream, Choice1Of2 (_, PartialDuplicate overage) -> - log.Information("Requeuing {stream} {pos} ({count} events)", stream, overage.index, overage.events.Length) + log.Information("Requeuing {stream} {pos} ({count} events)", stream, overage[0].Index, overage.Length) | stream, Choice1Of2 (_, PrefixMissing (batch, pos)) -> - log.Information("Waiting {stream} missing {gap} events ({count} events @ {pos})", stream, batch.index-pos, batch.events.Length, batch.index) + log.Information("Waiting {stream} missing {gap} events ({count} events @ {pos})", stream, batch[0].Index - pos, batch.Length, batch[0].Index) | stream, Choice2Of2 (_, exn) -> let level = if malformed then Events.LogEventLevel.Warning else Events.LogEventLevel.Information log.Write(level, exn, "Writing {stream} failed, retrying", stream) - let write (log : ILogger) (ctx : EventsContext) stream span = async { - log.Debug("Writing {s}@{i}x{n}", stream, span.index, span.events.Length) - let! res = ctx.Sync(stream, { index = span.index; etag = None }, span.events |> Array.map (fun x -> toNativeEvent x :> _)) + let write (log : ILogger) (ctx : EventsContext) stream (span : Default.StreamSpan) = async { + log.Debug("Writing {s}@{i}x{n}", stream, span[0].Index, span.Length) + let! res = ctx.Sync(stream, { index = span[0].Index; etag = None }, span |> Array.map (fun x -> StreamSpan.defaultToNative_ x :> _)) let res' = match res with | AppendResult.Ok pos -> Ok pos.index | AppendResult.Conflict (pos, _) | AppendResult.ConflictUnknown pos -> match pos.index with - | actual when actual < span.index -> PrefixMissing (span, actual) - | actual when actual >= span.index + span.events.LongLength -> Duplicate actual - | actual -> PartialDuplicate { index = actual; events = span.events |> Array.skip (actual-span.index |> int) } + | actual when actual < span[0].Index -> PrefixMissing (span, actual) + | actual when actual >= span[0].Index + span.LongLength -> Duplicate actual + | actual -> PartialDuplicate (span |> Array.skip (actual - span[0].Index |> int)) log.Debug("Result: {res}", res') return res' } let (|TimedOutMessage|RateLimitedMessage|TooLargeMessage|MalformedMessage|Other|) (e : exn) = @@ -85,9 +89,9 @@ module Internal = | ResultKind.TooLarge | ResultKind.Malformed -> true type Stats(log : ILogger, statsInterval, stateInterval) = - inherit Scheduling.Stats(log, statsInterval, stateInterval) + inherit Scheduling.Stats(log, statsInterval, stateInterval) let mutable okStreams, resultOk, resultDup, resultPartialDup, resultPrefix, resultExnOther = HashSet(), 0, 0, 0, 0, 0 - let mutable badCats, failStreams, rateLimited, timedOut, tooLarge, malformed = CatStats(), HashSet(), 0, 0, 0, 0 + let mutable badCats, failStreams, rateLimited, timedOut, tooLarge, malformed = Stats.CatStats(), HashSet(), 0, 0, 0, 0 let rlStreams, toStreams, tlStreams, mfStreams, oStreams = HashSet(), HashSet(), HashSet(), HashSet(), HashSet() let mutable okEvents, okBytes, exnEvents, exnBytes = 0, 0L, 0, 0L @@ -142,37 +146,37 @@ module Internal = type StreamSchedulingEngine = static member Create( - log : ILogger, eventsContext, itemDispatcher, stats : Stats, dumpStreams, - ?maxBatches, ?purgeInterval, ?wakeForResults, ?idleDelay, ?maxEvents, ?maxBytes, ?prioritizeLargePayloads) - : Scheduling.StreamSchedulingEngine<_, _, _> = + log : ILogger, eventsContext, itemDispatcher, stats : Stats, dumpStreams, + ?maxBatches, ?purgeInterval, ?wakeForResults, ?idleDelay, ?maxEvents, ?maxBytes, ?prioritizeStreamsBy) + : Scheduling.StreamSchedulingEngine<_, _, _, _> = let maxEvents, maxBytes = defaultArg maxEvents 16384, defaultArg maxBytes (1024 * 1024 - (*fudge*)4096) let writerResultLog = log.ForContext() - let attemptWrite (stream, span) ct = task { - let met, span' = Buffering.StreamSpan.slice (maxEvents, maxBytes) span + let attemptWrite struct (stream, span) ct = task { + let struct (met, span') = StreamSpan.slice Default.jsonSize (maxEvents, maxBytes) span try let! res = Writer.write log eventsContext (StreamName.toString stream) span' |> fun f -> Async.StartAsTask(f, cancellationToken = ct) - return span'.events.Length > 0, Choice1Of2 (met, res) - with e -> return false, Choice2Of2 (met, e) } + return struct (span'.Length > 0, Choice1Of2 struct (met, res)) + with e -> return struct (false, Choice2Of2 struct (met, e)) } let interpretWriteResultProgress (streams: Scheduling.StreamStates<_>) stream res = let applyResultToStreamState = function - | Choice1Of2 (_stats, Writer.Ok pos) -> streams.InternalUpdate stream pos null, false - | Choice1Of2 (_stats, Writer.Duplicate pos) -> streams.InternalUpdate stream pos null, false - | Choice1Of2 (_stats, Writer.PartialDuplicate overage) -> streams.InternalUpdate stream overage.index [|overage|], false - | Choice1Of2 (_stats, Writer.PrefixMissing (overage, pos)) -> streams.InternalUpdate stream pos [|overage|], false - | Choice2Of2 (_stats, exn) -> + | Choice1Of2 struct (_stats, Writer.Ok pos) -> struct (streams.RecordWriteProgress(stream, pos, null), false) + | Choice1Of2 (_stats, Writer.Duplicate pos) -> streams.RecordWriteProgress(stream, pos, null), false + | Choice1Of2 (_stats, Writer.PartialDuplicate overage) -> streams.RecordWriteProgress(stream, overage[0].Index, [| overage |]), false + | Choice1Of2 (_stats, Writer.PrefixMissing (overage, pos)) -> streams.RecordWriteProgress(stream, pos, [|overage|]), false + | Choice2Of2 struct (_stats, exn) -> let malformed = Writer.classify exn |> Writer.isMalformed streams.SetMalformed(stream, malformed), malformed - let (_stream, ss), malformed = applyResultToStreamState res + let struct (ss, malformed) = applyResultToStreamState res Writer.logTo writerResultLog malformed (stream, res) - ss.Write, res - let dispatcher = Scheduling.MultiDispatcher<_, _, _>.Create(itemDispatcher, attemptWrite, interpretWriteResultProgress, stats, dumpStreams) + struct (ss.WritePos, res) + let dispatcher = Scheduling.Dispatcher.MultiDispatcher<_, _, _, _>.Create(itemDispatcher, attemptWrite, interpretWriteResultProgress, stats, dumpStreams) Scheduling.StreamSchedulingEngine( dispatcher, ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay, - enableSlipstreaming = true, ?prioritizeLargePayloads = prioritizeLargePayloads) + enableSlipstreaming = true, ?prioritizeStreamsBy = prioritizeStreamsBy) type CosmosStoreSink = - /// Starts a StreamsProjectorPipeline that ingests all submitted events into the supplied context + /// Starts a Sink that ingests all submitted events into the supplied context static member Start ( log : ILogger, maxReadAhead, eventsContext, maxConcurrentStreams, // Default 5m @@ -186,18 +190,17 @@ type CosmosStoreSink = // Default: 1MB (limited by maximum size of a CosmosDB stored procedure invocation) ?maxBytes, ?ingesterStatsInterval) - : ProjectorPipeline<_> = + : Default.Sink = let statsInterval, stateInterval = defaultArg statsInterval (TimeSpan.FromMinutes 5.), defaultArg stateInterval (TimeSpan.FromMinutes 5.) let stats = Internal.Stats(log.ForContext(), statsInterval, stateInterval) - let dispatcher = Scheduling.ItemDispatcher<_>(maxConcurrentStreams) - let dumpStreams struct (s : Scheduling.StreamStates<_>, totalPurged) logger = - s.Dump(logger, totalPurged, Buffering.StreamState.eventsSize) + let dispatcher = Dispatch.ItemDispatcher<_, _>(maxConcurrentStreams) + let dumpStreams logStreamStates _log = logStreamStates Default.eventSize let streamScheduler = Internal.StreamSchedulingEngine.Create( log, eventsContext, dispatcher, stats, dumpStreams, ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay, - ?maxEvents=maxEvents, ?maxBytes=maxBytes, prioritizeLargePayloads = true) - Projector.StreamsProjectorPipeline.Start( + ?maxEvents = maxEvents, ?maxBytes = maxBytes, prioritizeStreamsBy = Default.eventDataSize) + Projector.Pipeline.Start( log, dispatcher.Pump, streamScheduler.Pump, maxReadAhead, streamScheduler.Submit, statsInterval, ?maxSubmissionsPerPartition = maxSubmissionsPerPartition, ?ingesterStatsInterval = ingesterStatsInterval) diff --git a/src/Propulsion.CosmosStore/CosmosStoreSource.fs b/src/Propulsion.CosmosStore/CosmosStoreSource.fs index fbe1e5e3..7bc0c1a6 100644 --- a/src/Propulsion.CosmosStore/CosmosStoreSource.fs +++ b/src/Propulsion.CosmosStore/CosmosStoreSource.fs @@ -11,6 +11,7 @@ open Microsoft.Azure.Cosmos #endif open Equinox.Core // Stopwatch.Time +open Propulsion.Internal open Serilog open System open System.Collections.Generic @@ -52,19 +53,19 @@ module Log = #if COSMOSV2 type CosmosSource = - static member CreateObserver<'Items,'Batch> + static member CreateObserver<'Items> ( log : ILogger, context : ChangeFeedObserverContext, - createIngester : ILogger * int -> Propulsion.Ingestion.Ingester<'Items,'Batch>, + startIngester : ILogger * int -> Propulsion.Ingestion.Ingester<'Items>, mapContent : IReadOnlyList -> 'Items) = let mutable rangeIngester = Unchecked.defaultof<_> - let init rangeLog partitionId = rangeIngester <- createIngester (rangeLog, partitionId) + let init rangeLog partitionId = rangeIngester <- startIngester (rangeLog, partitionId) let dispose () = rangeIngester.Stop() let sw = Stopwatch.StartNew() // we'll end up reporting the warmup/connect time on the first batch, but that's ok let ingest (log : ILogger) (ctx : IChangeFeedObserverContext) (docs : IReadOnlyList) = async { sw.Stop() // Stop the clock after ChangeFeedProcessor hands off to us let epoch, age = ctx.FeedResponse.ResponseContinuation.Trim[|'"'|] |> int64, DateTime.UtcNow - docs.[docs.Count-1].Timestamp - let! pt, (cur,max) = rangeIngester.Submit(epoch, ctx.Checkpoint(), mapContent docs) |> Stopwatch.Time - let readS, postS, rc = float sw.ElapsedMilliseconds / 1000., (let e = pt.Elapsed in e.TotalSeconds), ctx.FeedResponse.RequestCharge + let! pt, (cur,max) = rangeIngester.Ingest {epoch = epoch; checkpoint = ctx.Checkpoint(); items = mapContent docs; onCompletion = ignore } |> Stopwatch.Time + let readS, postS, rc = sw.ElapsedSeconds, (let e = pt.Elapsed in e.TotalSeconds), ctx.FeedResponse.RequestCharge let m = Log.Metric.Read { database = context.source.database; container = context.source.container; group = context.leasePrefix; rangeId = int ctx.PartitionKeyRangeId token = epoch; latency = sw.Elapsed; rc = rc; age = age; docs = docs.Count @@ -78,16 +79,16 @@ type CosmosSource = #else type CosmosStoreSource = - static member private CreateTrancheObserver<'Items,'Batch> + static member private CreateTrancheObserver<'Items> ( log : ILogger, - trancheIngester : Propulsion.Ingestion.Ingester<'Items,'Batch>, + trancheIngester : Propulsion.Ingestion.Ingester<'Items>, mapContent : IReadOnlyCollection<_> -> 'Items) : IChangeFeedObserver = let sw = Stopwatch.StartNew() // we'll end up reporting the warmup/connect time on the first batch, but that's ok let ingest (ctx : ChangeFeedObserverContext) checkpoint (docs : IReadOnlyCollection<_>) = async { sw.Stop() // Stop the clock after ChangeFeedProcessor hands off to us let readElapsed, age = sw.Elapsed, DateTime.UtcNow - ctx.timestamp - let! pt, (cur,max) = trancheIngester.Submit(ctx.epoch, checkpoint, mapContent docs) |> Stopwatch.Time + let! pt, struct (cur, max) = trancheIngester.Ingest { epoch = ctx.epoch; checkpoint = checkpoint; items = mapContent docs; onCompletion = ignore } |> Stopwatch.Time let postElapsed = pt.Elapsed let m = Log.Metric.Read { database = ctx.source.Database.Id; container = ctx.source.Id; group = ctx.group; rangeId = int ctx.rangeId @@ -103,9 +104,9 @@ type CosmosStoreSource = interface IDisposable with member _.Dispose() = trancheIngester.Stop() } - static member CreateObserver<'Items,'Batch> + static member CreateObserver<'Items> ( log : ILogger, - startIngester : ILogger * int -> Propulsion.Ingestion.Ingester<'Items,'Batch>, + startIngester : ILogger * int -> Propulsion.Ingestion.Ingester<'Items>, mapContent : IReadOnlyCollection<_> -> 'Items) : IChangeFeedObserver = // Its important we don't risk >1 instance https://andrewlock.net/making-getoradd-on-concurrentdictionary-thread-safe-using-lazy/ @@ -121,7 +122,7 @@ type CosmosStoreSource = { new IChangeFeedObserver with member _.Ingest(context, checkpoint, docs) = ingest context checkpoint docs interface IDisposable with - member _.Dispose() = dispose() } + member _.Dispose() = dispose () } #endif #if COSMOSV2 @@ -138,14 +139,14 @@ type CosmosStoreSource = let databaseId, containerId = monitored.Database.Id, monitored.Id #endif let logLag (interval : TimeSpan) (remainingWork : (int*int64) list) = async { - let synced, lagged, count, total = ResizeArray(), ResizeArray(), ref 0, ref 0L + let mutable synced, lagged, count, total = ResizeArray(), ResizeArray(), 0, 0L for partitionId, gap as partitionAndGap in remainingWork do - total := !total + gap - incr count + total <- total + gap + count <- count + 1 if gap = 0L then synced.Add partitionId else lagged.Add partitionAndGap let m = Log.Metric.Lag { database = databaseId; container = containerId; group = processorName; rangeLags = remainingWork |> Array.ofList } (log |> Log.metric m).Information("ChangeFeed {processorName} Lag Partitions {partitions} Gap {gapDocs:n0} docs {@laggingPartitions} Synced {@syncedPartitions}", - processorName, !count, !total, lagged, synced) + processorName, count, total, lagged, synced) return! Async.Sleep interval } let maybeLogLag = lagReportFreq |> Option.map logLag #if COSMOSV2 diff --git a/src/Propulsion.CosmosStore/ReaderCheckpoint.fs b/src/Propulsion.CosmosStore/ReaderCheckpoint.fs index c4a9cae5..58f1995c 100644 --- a/src/Propulsion.CosmosStore/ReaderCheckpoint.fs +++ b/src/Propulsion.CosmosStore/ReaderCheckpoint.fs @@ -80,7 +80,7 @@ let decideStart establishOrigin at freq state = async { | Fold.NotStarted -> let! origin = establishOrigin let config, checkpoint = mk at freq origin - return (configFreq config, checkpoint.pos), [Events.Started { config = config; origin = checkpoint}] + return struct (configFreq config, checkpoint.pos), [Events.Started { config = config; origin = checkpoint}] | Fold.Running s -> return (configFreq s.config, s.state.pos), [] } @@ -114,7 +114,7 @@ type Service internal (resolve : SourceId * TrancheId * string -> Decider = + member _.Start(source, tranche, ?establishOrigin) : Async = let decider = resolve (source, tranche, consumerGroupName) let establishOrigin = match establishOrigin with None -> async { return Position.initial } | Some f -> f decider.TransactAsync(decideStart establishOrigin DateTimeOffset.UtcNow defaultCheckpointFrequency) diff --git a/src/Propulsion.CosmosStore3/Propulsion.CosmosStore3.fsproj b/src/Propulsion.CosmosStore3/Propulsion.CosmosStore3.fsproj index 14c07e98..487dff87 100644 --- a/src/Propulsion.CosmosStore3/Propulsion.CosmosStore3.fsproj +++ b/src/Propulsion.CosmosStore3/Propulsion.CosmosStore3.fsproj @@ -36,6 +36,7 @@ + diff --git a/src/Propulsion.DynamoStore/AppendsEpoch.fs b/src/Propulsion.DynamoStore/AppendsEpoch.fs index 2f6c4e8c..46b05a89 100644 --- a/src/Propulsion.DynamoStore/AppendsEpoch.fs +++ b/src/Propulsion.DynamoStore/AppendsEpoch.fs @@ -131,7 +131,7 @@ module Config = /// only deserializing events pertaining to things we have not seen before module Reader = - type Event = int64 * Events.Event + type Event = (struct (int64 * Events.Event)) let codec : FsCodec.IEventCodec = EventCodec.withIndex type State = { changes : struct (int * Events.StreamSpan array) array; closed : bool } diff --git a/src/Propulsion.DynamoStore/DynamoDbExport.fs b/src/Propulsion.DynamoStore/DynamoDbExport.fs index 8a6658b4..55f79874 100644 --- a/src/Propulsion.DynamoStore/DynamoDbExport.fs +++ b/src/Propulsion.DynamoStore/DynamoDbExport.fs @@ -12,7 +12,7 @@ module DynamoDbJsonParser = and [] NumVal = { N : string } and ListVal<'t> = { L : 't[] } - let read (path : string) : seq = seq { + let read (path : string) : seq = seq { use r = new StreamReader(path) let mutable more = true while more do diff --git a/src/Propulsion.DynamoStore/DynamoStoreIndex.fs b/src/Propulsion.DynamoStore/DynamoStoreIndex.fs index 72c01e5e..ba26abf3 100644 --- a/src/Propulsion.DynamoStore/DynamoStoreIndex.fs +++ b/src/Propulsion.DynamoStore/DynamoStoreIndex.fs @@ -101,10 +101,10 @@ module Reader = let totalEvents = spans |> Array.sumBy (fun x -> x.c.Length) let totalStreams = spans |> AppendsEpoch.flatten |> Seq.length log.Information("Epoch {epochId} {totalE} events {totalS} streams ({spans} spans, {batches} batches, {k:n3} MiB) {loadS:n1}s", - string epochId, totalEvents, totalStreams, spans.Length, state.changes.Length, float sizeB / 1024. / 1024., t.TotalSeconds) + string epochId, totalEvents, totalStreams, spans.Length, state.changes.Length, Propulsion.Internal.mb sizeB, t.TotalSeconds) return spans, state.closed, sizeB } - let loadIndex (log, storeLog, context) trancheId gapsLimit: Async = async { + let loadIndex (log, storeLog, context) trancheId gapsLimit: Async = async { let indexEpochs = AppendsEpoch.Reader.Config.create storeLog context let mutable epochId, more, totalB, totalSpans = AppendsEpochId.initial, true, 0L, 0L let state = Buffer() @@ -124,6 +124,5 @@ module Reader = else totalSpans <- totalSpans + 1L more <- closed epochId <- AppendsEpochId.next epochId - let totalMib = float totalB / 1024. / 1024. - log.Information("Tranche {tranche} Current Index size {mib:n1} MiB; {gapped} Invalid spans", string trancheId, totalMib, invalidSpans) + log.Information("Tranche {tranche} Current Index size {mib:n1} MiB; {gapped} Invalid spans", string trancheId, Propulsion.Internal.mb totalB, invalidSpans) return state, totalSpans } diff --git a/src/Propulsion.DynamoStore/DynamoStoreSource.fs b/src/Propulsion.DynamoStore/DynamoStoreSource.fs index 642cfadf..47607a7e 100644 --- a/src/Propulsion.DynamoStore/DynamoStoreSource.fs +++ b/src/Propulsion.DynamoStore/DynamoStoreSource.fs @@ -1,7 +1,5 @@ namespace Propulsion.DynamoStore -type StreamEvent = Propulsion.Streams.StreamEvent - open Equinox.DynamoStore open FSharp.Control open System.Collections.Concurrent @@ -58,18 +56,18 @@ module private Impl = | Exceptions.ProvisionedThroughputExceeded when not force -> () | e -> storeLog.Warning(e, "DynamoStoreSource commit failure") - let mkBatch checkpoint isTail items : Propulsion.Feed.Internal.Batch<_> = + let mkBatch checkpoint isTail items : Propulsion.Feed.Core.Batch<_> = { items = items; checkpoint = Checkpoint.toPosition checkpoint; isTail = isTail } let sliceBatch epochId offset items = mkBatch (Checkpoint.ofEpochAndOffset epochId offset) false items - let finalBatch epochId (version, state : AppendsEpoch.Reader.State) items : Propulsion.Feed.Internal.Batch<_> = + let finalBatch epochId (version, state : AppendsEpoch.Reader.State) items : Propulsion.Feed.Core.Batch<_> = mkBatch (Checkpoint.ofEpochClosedAndVersion epochId state.closed version) (not state.closed) items // Includes optional hydrating of events with event bodies and/or metadata (controlled via hydrating/maybeLoad args) let materializeIndexEpochAsBatchesOfStreamEvents (log : Serilog.ILogger, sourceId, storeLog) (hydrating, maybeLoad, loadDop) batchCutoff (context : DynamoStoreContext) (AppendsTrancheId.Parse tid, Checkpoint.Parse (epochId, offset)) - : AsyncSeq> = asyncSeq { + : AsyncSeq)> = asyncSeq { let epochs = AppendsEpoch.Reader.Config.create storeLog context let sw = System.Diagnostics.Stopwatch.StartNew() let! _maybeSize, version, state = epochs.Read(tid, epochId, offset) @@ -93,7 +91,7 @@ module private Impl = sourceId, string tid, string epochId, offset, (if hydrating then "Hydrating" else "Feeding"), totalChanges, streamEvents.Count, totalStreams, chosenEvents, totalEvents) let buffer, cache = ResizeArray(), ConcurrentDictionary() // For each batch we produce, we load any streams we have not already loaded at this time - let materializeSpans : Async = async { + let materializeSpans : Async = async { let loadsRequired = buffer |> Seq.distinctBy (fun x -> x.p) @@ -110,12 +108,12 @@ module private Impl = for span in buffer do match cache.TryGetValue span.p with | false, _ -> () - | true, (items : FsCodec.ITimelineEvent<_>[]) -> + | true, (items : FsCodec.ITimelineEvent<_> array) -> // NOTE this could throw if a span has been indexed, but the stream read is from a replica that does not yet have it // the exception in that case will trigger a safe re-read from the last saved read position that a consumer has forwarded // TOCONSIDER revise logic to share session key etc to rule this out let events = Array.sub items (span.i - items[0].Index |> int) span.c.Length - for e in events do ({ stream = IndexStreamId.toStreamName span.p; event = e } : StreamEvent) |] } + for e in events -> IndexStreamId.toStreamName span.p, e |] } let mutable prevLoaded, batchIndex = 0L, 0 let report (i : int option) len = if largeEnough && hydrating then @@ -132,7 +130,7 @@ module private Impl = if buffer.Count <> 0 && buffer.Count + pending.Length > batchCutoff then let! hydrated = materializeSpans report (Some i) hydrated.Length - yield sw.Elapsed, sliceBatch epochId i hydrated // not i + 1 as the batch does not include these changes + yield struct (sw.Elapsed, sliceBatch epochId i hydrated) // not i + 1 as the batch does not include these changes sw.Reset() buffer.Clear() buffer.AddRange(pending) @@ -149,9 +147,7 @@ type LoadMode = * /// Defines the Context to use when loading the bodies storeContext : DynamoStoreContext module internal LoadMode = - let private mapTimelineEvent = - let mapBodyToBytes = (fun (x : System.ReadOnlyMemory) -> x.ToArray()) - FsCodec.Core.TimelineEvent.Map (FsCodec.Deflate.EncodedToUtf8 >> mapBodyToBytes) // TODO replace with FsCodec.Deflate.EncodedToByteArray + let private mapTimelineEvent = FsCodec.Core.TimelineEvent.Map FsCodec.Deflate.EncodedToUtf8 let private withBodies (eventsContext : Equinox.DynamoStore.Core.EventsContext) filter = fun sn (i, cs : string array) -> if filter sn then Some (async { let! _pos, events = eventsContext.Read(FsCodec.StreamName.toString sn, i, maxCount = cs.Length) @@ -171,8 +167,7 @@ module internal LoadMode = type DynamoStoreSource ( log : Serilog.ILogger, statsInterval, indexClient : DynamoStoreClient, batchSizeCutoff, tailSleepInterval, - checkpoints : Propulsion.Feed.IFeedCheckpointStore, - sink : Propulsion.ProjectorPipeline, Propulsion.Submission.SubmissionBatch>>, + checkpoints : Propulsion.Feed.IFeedCheckpointStore, sink : Propulsion.Streams.Default.Sink, // If the Handler does not utilize the bodies of the events, we can avoid loading them from the Store loadMode : LoadMode, // Override default start position to be at the tail of the index (Default: Always replay all events) @@ -182,7 +177,7 @@ type DynamoStoreSource ?readFailureSleepInterval, ?sourceId, ?trancheIds) = - inherit Propulsion.Feed.Internal.TailingFeedSource + inherit Propulsion.Feed.Core.TailingFeedSource ( log, statsInterval, defaultArg sourceId FeedSourceId.wellKnownId, tailSleepInterval, Impl.materializeIndexEpochAsBatchesOfStreamEvents (log, defaultArg sourceId FeedSourceId.wellKnownId, defaultArg storeLog log) diff --git a/src/Propulsion.DynamoStore/Types.fs b/src/Propulsion.DynamoStore/Types.fs index b504a523..48e533a7 100644 --- a/src/Propulsion.DynamoStore/Types.fs +++ b/src/Propulsion.DynamoStore/Types.fs @@ -67,10 +67,10 @@ module internal Config = let createWithOriginIndex codec initial fold context minIndex = // TOCONSIDER include way to limit item count being read // TOCONSIDER implement a loader hint to pass minIndex to the query as an additional filter - let isOrigin (i, _) = i <= minIndex + let isOrigin struct (i, _) = i <= minIndex // There _should_ always be an event at minIndex - if there isn't for any reason, the load might go back one event too far // Here we trim it for correctness (although Propulsion would technically ignore it) - let trimPotentialOverstep = Seq.filter (fun (i, _e) -> i >= minIndex) + let trimPotentialOverstep = Seq.filter (fun struct (i, _e) -> i >= minIndex) let accessStrategy = AccessStrategy.MultiSnapshot (isOrigin, fun _ -> failwith "writing not applicable") create codec initial (fun s -> trimPotentialOverstep >> fold s) accessStrategy (context, None) @@ -81,9 +81,9 @@ module internal EventCodec = let private withUpconverter<'c, 'e when 'c :> TypeShape.UnionContract.IUnionContract> up : FsCodec.IEventCodec<'e, _, _> = let down (_ : 'e) = failwith "Unexpected" FsCodec.SystemTextJson.Codec.Create<'e, 'c, _>(up, down) |> FsCodec.Deflate.EncodeTryDeflate - let withIndex<'c when 'c :> TypeShape.UnionContract.IUnionContract> : FsCodec.IEventCodec = - let up (raw : FsCodec.ITimelineEvent<_>, e) = raw.Index, e - withUpconverter<'c, int64 * 'c> up + let withIndex<'c when 'c :> TypeShape.UnionContract.IUnionContract> : FsCodec.IEventCodec = + let up (raw : FsCodec.ITimelineEvent<_>, e) = struct (raw.Index, e) + withUpconverter<'c, struct (int64 * 'c)> up module internal Async = diff --git a/src/Propulsion.EventStore/EventStoreReader.fs b/src/Propulsion.EventStore/EventStoreReader.fs index b86547c4..c18907e2 100755 --- a/src/Propulsion.EventStore/EventStoreReader.fs +++ b/src/Propulsion.EventStore/EventStoreReader.fs @@ -12,9 +12,8 @@ open System.Diagnostics open System.Threading let inline arrayBytes (x : byte[]) = match x with null -> 0 | x -> x.Length -let inline recPayloadBytes (x : EventStore.ClientAPI.RecordedEvent) = arrayBytes x.Data + arrayBytes x.Metadata -let inline payloadBytes (x : EventStore.ClientAPI.ResolvedEvent) = recPayloadBytes x.Event + x.OriginalStreamId.Length * sizeof -let inline mb x = float x / 1024. / 1024. +let inline recPayloadBytes (x : RecordedEvent) = arrayBytes x.Data + arrayBytes x.Metadata +let inline payloadBytes (x : ResolvedEvent) = recPayloadBytes x.Event + x.OriginalStreamId.Length * sizeof let private dash = [|'-'|] // Bespoke algorithm suited to grouping streams as observed in EventStore, where {category}-{aggregateId} is expected, but definitely not guaranteed @@ -27,14 +26,14 @@ type OverallStats(?statsInterval) = let overallStart, progressStart = Stopwatch.StartNew(), Stopwatch.StartNew() let mutable totalEvents, totalBytes = 0L, 0L - member __.Ingest(batchEvents, batchBytes) = + member _.Ingest(batchEvents, batchBytes) = Interlocked.Add(&totalEvents, batchEvents) |> ignore Interlocked.Add(&totalBytes, batchBytes) |> ignore - member __.Bytes = totalBytes - member __.Events = totalEvents + member _.Bytes = totalBytes + member _.Events = totalEvents - member __.DumpIfIntervalExpired(?force) = + member _.DumpIfIntervalExpired(?force) = if progressStart.ElapsedMilliseconds > intervalMs || force = Some true then if totalEvents <> 0L then let totalMb = mb totalBytes @@ -54,13 +53,13 @@ type SliceStatsBuffer(?interval) = let cat = categorizeEventStoreStreamId x.OriginalStreamId let eventBytes = payloadBytes x match recentCats.TryGetValue cat with - | true, (currCount, currSize) -> recentCats.[cat] <- (currCount + 1, currSize + eventBytes) - | false, _ -> recentCats.[cat] <- (1, eventBytes) + | true, (currCount, currSize) -> recentCats[cat] <- (currCount + 1, currSize + eventBytes) + | false, _ -> recentCats[cat] <- (1, eventBytes) batchBytes <- batchBytes + eventBytes __.DumpIfIntervalExpired() slice.Events.Length, int64 batchBytes - member __.DumpIfIntervalExpired(?force) = + member _.DumpIfIntervalExpired(?force) = if accStart.ElapsedMilliseconds > intervalMs || defaultArg force false then lock recentCats <| fun () -> let log kind limit xs = @@ -105,7 +104,7 @@ let chunk (pos : Position) = uint64 pos.CommitPosition >>> 28 let posFromChunk (chunk : int) = let chunkBase = int64 chunk * 1024L * 1024L * 256L Position(chunkBase, 0L) -let posFromChunkAfter (pos : EventStore.ClientAPI.Position) = +let posFromChunkAfter (pos : Position) = let nextChunk = 1 + int (chunk pos) posFromChunk nextChunk let posFromPercentage (pct, max : Position) = @@ -132,12 +131,12 @@ let establishMax (conn : IEventStoreConnection) = async { /// Walks a stream within the specified constraints; used to grab data when writing to a stream for which a prefix is missing /// Can throw (in which case the caller is in charge of retrying, possibly with a smaller batch size) -let pullStream (conn : IEventStoreConnection, batchSize) (stream, pos, limit : int option) mapEvent (postBatch : string * StreamSpan<_> -> Async) = +let pullStream (conn : IEventStoreConnection, batchSize) (stream, pos, limit : int option) mapEvent (postBatch : string * Default.StreamSpan -> Async) = let rec fetchFrom pos limit = async { let reqLen = match limit with Some limit -> min limit batchSize | None -> batchSize let! currentSlice = conn.ReadStreamEventsForwardAsync(stream, pos, reqLen, resolveLinkTos=true) |> Async.AwaitTaskCorrect let events = currentSlice.Events |> Array.map (fun x -> mapEvent x.Event) - do! postBatch (stream, { index = currentSlice.FromEventNumber; events = events }) + do! postBatch (stream, events) match limit with | None when currentSlice.IsEndOfStream -> return () | None -> return! fetchFrom currentSlice.NextEventNumber None @@ -149,7 +148,7 @@ let pullStream (conn : IEventStoreConnection, batchSize) (stream, pos, limit : i /// Can throw (in which case the caller is in charge of retrying, possibly with a smaller batch size) type [] PullResult = Exn of exn: exn | Eof | EndOfTranche let pullAll (slicesStats : SliceStatsBuffer, overallStats : OverallStats) (conn : IEventStoreConnection, batchSize) - (range:Range, once) (tryMapEvent : ResolvedEvent -> StreamEvent<_> option) (postBatch : Position -> StreamEvent<_>[] -> Async) = + (range:Range, once) (tryMapEvent : ResolvedEvent -> StreamEvent<_> option) (postBatch : Position -> StreamEvent<_>[] -> Async) = let sw = Stopwatch.StartNew() // we'll report the warmup/connect time on the first batch let streams, cats = HashSet(), HashSet() let rec aux () = async { @@ -159,10 +158,10 @@ let pullAll (slicesStats : SliceStatsBuffer, overallStats : OverallStats) (conn let batchEvents, batchBytes = slicesStats.Ingest currentSlice in overallStats.Ingest(int64 batchEvents, batchBytes) let events = currentSlice.Events |> Seq.choose tryMapEvent |> Array.ofSeq streams.Clear(); cats.Clear() - for x in events do - if streams.Add x.stream then - cats.Add (StreamName.categorize x.stream) |> ignore - let! (cur, max) = postBatch currentSlice.NextPosition events + for struct (stream, _) in events do + if streams.Add stream then + cats.Add (StreamName.categorize stream) |> ignore + let! cur, max = postBatch currentSlice.NextPosition events Log.Information("Read {pos,10} {pct:p1} {ft:n3}s {mb:n1}MB {count,4} {categories,4}c {streams,4}s {events,4}e Post {pt:n3}s {cur}/{max}", range.Current.CommitPosition, range.PositionAsRangePercentage, (let e = sw.Elapsed in e.TotalSeconds), mb batchBytes, batchEvents, cats.Count, streams.Count, events.Length, (let e = postSw.Elapsed in e.TotalSeconds), cur, max) @@ -182,9 +181,9 @@ type Req = | EofDetected /// Tail from a given start position, at intervals of the specified timespan (no waiting if catching up) | Tail of seriesId : int * startPos : Position * max : Position * interval : TimeSpan * batchSize : int - /// Read a given segment of a stream (used when a stream needs to be rolled forward to lay down an event for which the preceding events are missing) + // Read a given segment of a stream (used when a stream needs to be rolled forward to lay down an event for which the preceding events are missing) //| StreamPrefix of name: string * pos: int64 * len: int * batchSize: int - /// Read the entirity of a stream in blocks of the specified batchSize (TODO wire to commandline request) + // Read the entirety of a stream in blocks of the specified batchSize (TODO wire to commandline request) //| Stream of name: string * batchSize: int /// Read a specific chunk (min-max range), posting batches tagged with that chunk number | Chunk of seriesId : int * range: Range * batchSize : int @@ -193,15 +192,15 @@ type Req = [] type Res = /// A batch read from a Chunk - | Batch of seriesId : int * pos : Position * items : StreamEvent seq + | Batch of seriesId : int * pos : Position * items : Default.StreamEvent seq /// Ingestion buffer requires an explicit end of chunk message before next chunk can commence processing | EndOfChunk of seriesId : int - /// A Batch read from a Stream or StreamPrefix + // A Batch read from a Stream or StreamPrefix //| StreamSpan of span: State.StreamSpan /// Holds work queue, together with stats relating to the amount and/or categories of data being traversed /// Processing is driven by external callers running multiple concurrent invocations of `Process` -type EventStoreReader(conns : _ [], defaultBatchSize, minBatchSize, tryMapEvent, post : Res -> Async, tailInterval, dop, ?statsInterval) = +type EventStoreReader(connections : _ [], defaultBatchSize, minBatchSize, tryMapEvent, post : Res -> Async, tailInterval, dop, ?statsInterval) = let work = System.Collections.Concurrent.ConcurrentQueue() let sleepIntervalMs = 100 let overallStats = OverallStats(?statsInterval=statsInterval) @@ -282,20 +281,20 @@ type EventStoreReader(conns : _ [], defaultBatchSize, minBatchSize, tryMapEvent, batchSize <- adjust batchSize Log.Warning(e, "Tail $all failed, adjusting batch size to {bs}", batchSize) } - member __.Pump(initialSeriesId, initialPos, max) = async { + member _.Pump(initialSeriesId, initialPos, max) = async { let mutable robin = 0 let selectConn () = - let connIndex = Interlocked.Increment(&robin) % conns.Length - conns.[connIndex] + let connIndex = Interlocked.Increment(&robin) % connections.Length + connections[connIndex] let dop = Sem dop let forkRunRelease = - let r = new Random() + let r = Random() fun req -> async { // this is not called in parallel hence no need to lock `r` - let capacity = let used, max = dop.State in max - used + let capacity = let struct (used, max) = dop.State in max - used // Jitter is most relevant when processing commences - any commencement of a chunk can trigger significant page faults on server // which we want to attempt to limit the effects of - let jitterMs = match capacity with 0 -> 200 | x -> r.Next(1000, 2000) + let jitterMs = if capacity = 0 then 200 else r.Next(1000, 2000) Log.Information("Waiting {jitter}ms to jitter reader stripes, {currentCount} further reader stripes awaiting start", jitterMs, capacity) do! Async.Sleep jitterMs let! _ = Async.StartChild <| async { @@ -305,9 +304,9 @@ type EventStoreReader(conns : _ [], defaultBatchSize, minBatchSize, tryMapEvent, let mutable seriesId = initialSeriesId let mutable remainder = - if conns.Length > 1 then + if connections.Length > 1 then let nextPos = posFromChunkAfter initialPos - work.Enqueue <| Req.Chunk (seriesId, new Range(initialPos, Some nextPos, max), defaultBatchSize) + work.Enqueue <| Req.Chunk (seriesId, Range(initialPos, Some nextPos, max), defaultBatchSize) Some nextPos else work.Enqueue <| Req.Tail (seriesId, initialPos, max, tailInterval, defaultBatchSize) @@ -328,7 +327,7 @@ type EventStoreReader(conns : _ [], defaultBatchSize, minBatchSize, tryMapEvent, endDetected <- true remainder <- None seriesId <- seriesId + 1 - /// TODO shed excess connections as transitioning + // TODO shed excess connections as transitioning do! forkRunRelease <| Req.Tail (seriesId, nextChunk, nextChunk, tailInterval, defaultBatchSize) // Process requeuing etc | (true, task), _ -> diff --git a/src/Propulsion.EventStore/EventStoreSink.fs b/src/Propulsion.EventStore/EventStoreSink.fs index f81716f8..09739cf0 100755 --- a/src/Propulsion.EventStore/EventStoreSink.fs +++ b/src/Propulsion.EventStore/EventStoreSink.fs @@ -9,16 +9,25 @@ open Equinox.EventStoreDb #endif open Propulsion +open Propulsion.Internal // Helpers open Propulsion.Streams -open Propulsion.Streams.Internal // Helpers open Serilog open System.Collections.Generic open System open System.Threading -[] -module private Impl2 = - let inline mb x = float x / 1024. / 1024. +module private StreamSpan = + +#if EVENTSTORE_LEGACY + let private nativeToDefault_ = FsCodec.Core.TimelineEvent.Map (fun (xs : byte array) -> ReadOnlyMemory xs) + let inline nativeToDefault span = Array.map nativeToDefault_ span + let defaultToNative_ = FsCodec.Core.TimelineEvent.Map (fun (xs : ReadOnlyMemory) -> xs.ToArray()) + let inline defaultToNative span = Array.map defaultToNative_ span +#else + let nativeToDefault = id + let defaultToNative_ = id + let defaultToNative = id +#endif module Internal = @@ -27,32 +36,26 @@ module Internal = type [] Result = | Ok of updatedPos : int64 | Duplicate of updatedPos : int64 - | PartialDuplicate of overage : StreamSpan - | PrefixMissing of batch : StreamSpan * writePos : int64 + | PartialDuplicate of overage : Default.StreamSpan + | PrefixMissing of batch : Default.StreamSpan * writePos : int64 - let logTo (log : ILogger) (res : FsCodec.StreamName * Choice) = + let logTo (log : ILogger) (res : FsCodec.StreamName * Choice) = match res with | stream, Choice1Of2 (_, Ok pos) -> log.Information("Wrote {stream} up to {pos}", stream, pos) | stream, Choice1Of2 (_, Duplicate updatedPos) -> log.Information("Ignored {stream} (synced up to {pos})", stream, updatedPos) | stream, Choice1Of2 (_, PartialDuplicate overage) -> - log.Information("Requeuing {stream} {pos} ({count} events)", stream, overage.index, overage.events.Length) + log.Information("Requeuing {stream} {pos} ({count} events)", stream, overage[0].Index, overage.Length) | stream, Choice1Of2 (_, PrefixMissing (batch, pos)) -> - log.Information("Waiting {stream} missing {gap} events ({count} events @ {pos})", stream, batch.index - pos, batch.events.Length, batch.index) + log.Information("Waiting {stream} missing {gap} events ({count} events @ {pos})", + stream, batch[0].Index - pos, batch.Length, batch[0].Index) | stream, Choice2Of2 (_, exn) -> log.Warning(exn,"Writing {stream} failed, retrying", stream) -#if !EVENTSTORE_LEGACY - let inline mapBodyToRom xs = FsCodec.Core.TimelineEvent.Map (fun (xs : byte[]) -> ReadOnlyMemory xs) xs - let inline mapStreamSpanToRom (span : StreamSpan) : StreamSpan> = { index = span.index; events = span.events |> Array.map mapBodyToRom } - let inline mapBodyToBytes xs = FsCodec.Core.TimelineEvent.Map (fun (xs : ReadOnlyMemory) -> xs.ToArray()) xs - let inline mapStreamSpanToBytes (span : StreamSpan>) : StreamSpan = { index = span.index; events = span.events |> Array.map mapBodyToBytes } -#endif - - let write (log : ILogger) (context : EventStoreContext) stream span = async { - log.Debug("Writing {s}@{i}x{n}", stream, span.index, span.events.Length) - let! res = context.Sync(log, stream, span.index - 1L, span.events |> Array.map (fun x -> x :> _)) + let write (log : ILogger) (context : EventStoreContext) stream (span : Default.StreamSpan) = async { + log.Debug("Writing {s}@{i}x{n}", stream, span[0].Index, span.Length) + let! res = context.Sync(log, stream, span[0].Index - 1L, (span |> Array.map (fun span -> StreamSpan.defaultToNative_ span :> _))) let ress = match res with | GatewaySyncResult.Written (Token.Unpack pos') -> @@ -65,12 +68,11 @@ module Internal = #if EVENTSTORE_LEGACY match pos.pos.streamVersion + 1L with #else - let span = mapStreamSpanToBytes span match pos.streamVersion + 1L with #endif - | actual when actual < span.index -> PrefixMissing (span, actual) - | actual when actual >= span.index + span.events.LongLength -> Duplicate actual - | actual -> PartialDuplicate { index = actual; events = span.events |> Array.skip (actual - span.index |> int) } + | actual when actual < span[0].Index -> PrefixMissing (span, actual) + | actual when actual >= span[0].Index + span.LongLength -> Duplicate actual + | actual -> PartialDuplicate (span |> Array.skip (actual - span[0].Index |> int)) log.Debug("Result: {res}", ress) return ress } @@ -82,8 +84,8 @@ module Internal = | _ -> ResultKind.Other type Stats(log : ILogger, statsInterval, stateInterval) = - inherit Scheduling.Stats(log, statsInterval, stateInterval) - let mutable okStreams, badCats, failStreams, toStreams, oStreams = HashSet(), CatStats(), HashSet(), HashSet(), HashSet() + inherit Scheduling.Stats(log, statsInterval, stateInterval) + let mutable okStreams, badCats, failStreams, toStreams, oStreams = HashSet(), Stats.CatStats(), HashSet(), HashSet(), HashSet() let mutable resultOk, resultDup, resultPartialDup, resultPrefix, resultExnOther, timedOut = 0, 0, 0, 0, 0, 0 let mutable okEvents, okBytes, exnEvents, exnBytes = 0, 0L, 0, 0L @@ -134,39 +136,37 @@ module Internal = type EventStoreSchedulingEngine = static member Create(log : ILogger, storeLog, connections : _ [], itemDispatcher, stats : Stats, dumpStreams, ?maxBatches, ?idleDelay, ?purgeInterval) - : Scheduling.StreamSchedulingEngine<_, _, _> = + : Scheduling.StreamSchedulingEngine<_, _, _, _> = let writerResultLog = log.ForContext() let mutable robin = 0 - let attemptWrite (stream, span) ct = task { + let attemptWrite struct (stream, span) ct = task { let index = Interlocked.Increment(&robin) % connections.Length let selectedConnection = connections[index] let maxEvents, maxBytes = 65536, 4 * 1024 * 1024 - (*fudge*)4096 - let met, span' = Buffering.StreamSpan.slice (maxEvents, maxBytes) span -#if !EVENTSTORE_LEGACY - let span' = mapStreamSpanToRom span' -#endif - try let! res = Writer.write storeLog selectedConnection (FsCodec.StreamName.toString stream) span' |> fun f -> Async.StartAsTask(f, cancellationToken = ct) - return span'.events.Length > 0, Choice1Of2 (met, res) - with e -> return false, Choice2Of2 (met, e) } + let struct (met, span') = StreamSpan.slice Default.jsonSize (maxEvents, maxBytes) span + try let! res = Writer.write storeLog selectedConnection (FsCodec.StreamName.toString stream) span' + |> fun f -> Async.StartAsTask(f, cancellationToken = ct) + return struct (span'.Length > 0, Choice1Of2 struct (met, res)) + with e -> return false, Choice2Of2 struct (met, e) } let interpretWriteResultProgress (streams : Scheduling.StreamStates<_>) stream res = let applyResultToStreamState = function - | Choice1Of2 (_stats, Writer.Ok pos) -> streams.InternalUpdate stream pos null - | Choice1Of2 (_stats, Writer.Duplicate pos) -> streams.InternalUpdate stream pos null - | Choice1Of2 (_stats, Writer.PartialDuplicate overage) -> streams.InternalUpdate stream overage.index [|overage|] - | Choice1Of2 (_stats, Writer.PrefixMissing (overage, pos)) -> streams.InternalUpdate stream pos [|overage|] - | Choice2Of2 (_stats, _exn) -> streams.SetMalformed(stream, false) - let _stream, ss = applyResultToStreamState res + | Choice1Of2 struct (_stats, Writer.Ok pos) -> streams.RecordWriteProgress(stream, pos, null) + | Choice1Of2 (_stats, Writer.Duplicate pos) -> streams.RecordWriteProgress(stream, pos, null) + | Choice1Of2 (_stats, Writer.PartialDuplicate overage) -> streams.RecordWriteProgress(stream, overage[0].Index, [| overage |]) + | Choice1Of2 (_stats, Writer.PrefixMissing (overage, pos)) -> streams.RecordWriteProgress(stream, pos, [| overage |]) + | Choice2Of2 struct (_stats, _exn) -> streams.SetMalformed(stream, false) + let ss = applyResultToStreamState res Writer.logTo writerResultLog (stream, res) - ss.Write, res + struct (ss.WritePos, res) - let dispatcher = Scheduling.MultiDispatcher<_, _, _>.Create(itemDispatcher, attemptWrite, interpretWriteResultProgress, stats, dumpStreams) - Scheduling.StreamSchedulingEngine(dispatcher, enableSlipstreaming=true, ?maxBatches=maxBatches, ?idleDelay=idleDelay, ?purgeInterval=purgeInterval) + let dispatcher = Scheduling.Dispatcher.MultiDispatcher<_, _, _, _>.Create(itemDispatcher, attemptWrite, interpretWriteResultProgress, stats, dumpStreams) + Scheduling.StreamSchedulingEngine(dispatcher, enableSlipstreaming = true, ?maxBatches = maxBatches, ?idleDelay = idleDelay, ?purgeInterval = purgeInterval) type EventStoreSink = - /// Starts a StreamsProjectorPipeline that ingests all submitted events into the supplied connections + /// Starts a Sink that ingests all submitted events into the supplied connections static member Start ( log : ILogger, storeLog, maxReadAhead, connections, maxConcurrentStreams, // Default 5m @@ -180,14 +180,13 @@ type EventStoreSink = // NOTE: Can impair performance and/or increase costs of writes as it inhibits the ability of the ingester to discard redundant inputs ?purgeInterval, ?ingesterStatsInterval) - : ProjectorPipeline<_> = + : Default.Sink = let statsInterval, stateInterval = defaultArg statsInterval (TimeSpan.FromMinutes 5.), defaultArg stateInterval (TimeSpan.FromMinutes 5.) let stats = Internal.Stats(log.ForContext(), statsInterval, stateInterval) - let dispatcher = Scheduling.ItemDispatcher<_>(maxConcurrentStreams) - let dumpStats struct (s : Scheduling.StreamStates<_>, totalPurged) logger = - s.Dump(logger, totalPurged, Buffering.StreamState.eventsSize) - let streamScheduler = Internal.EventStoreSchedulingEngine.Create(log, storeLog, connections, dispatcher, stats, dumpStats, ?idleDelay=idleDelay, ?purgeInterval=purgeInterval) - Projector.StreamsProjectorPipeline.Start( + let dispatcher = Dispatch.ItemDispatcher<_, _>(maxConcurrentStreams) + let dumpStreams logStreamStates _log = logStreamStates Default.eventSize + let streamScheduler = Internal.EventStoreSchedulingEngine.Create(log, storeLog, connections, dispatcher, stats, dumpStreams, ?idleDelay = idleDelay, ?purgeInterval = purgeInterval) + Projector.Pipeline.Start( log, dispatcher.Pump, streamScheduler.Pump, maxReadAhead, streamScheduler.Submit, statsInterval, ?maxSubmissionsPerPartition = maxSubmissionsPerPartition, ?ingesterStatsInterval = ingesterStatsInterval) diff --git a/src/Propulsion.EventStore/EventStoreSource.fs b/src/Propulsion.EventStore/EventStoreSource.fs index f9c3c470..450b5adc 100755 --- a/src/Propulsion.EventStore/EventStoreSource.fs +++ b/src/Propulsion.EventStore/EventStoreSource.fs @@ -39,12 +39,12 @@ module Mapping = let inline len0ToNull (x : _[]) = match x with null -> null | x when x.Length = 0 -> null | x -> x FsCodec.Core.TimelineEvent.Create(x.EventNumber, x.EventType, len0ToNull x.Data, len0ToNull x.Metadata, timestamp = x.Timestamp) :> _ - let (|PropulsionStreamEvent|) (x : RecordedEvent) : Propulsion.Streams.StreamEvent<_> = - { stream = StreamName.internalParseSafe x.EventStreamId; event = (|PropulsionTimelineEvent|) x } + let (|PropulsionStreamEvent|) (x : RecordedEvent) : StreamEvent<_> = + StreamName.internalParseSafe x.EventStreamId, (|PropulsionTimelineEvent|) x type EventStoreSource = static member Run - ( log : Serilog.ILogger, sink : Propulsion.ProjectorPipeline<_>, checkpoints : Checkpoint.CheckpointSeries, + ( log : Serilog.ILogger, sink : Default.Sink, checkpoints : Checkpoint.CheckpointSeries, connect, spec, tryMapEvent, maxReadAhead, statsInterval) = async { let conn = connect () @@ -86,7 +86,7 @@ type EventStoreSource = log.Information("Commencing Gorging with {stripes} $all reader stripes covering a 256MB chunk each", factor) let extraConns = Seq.init (factor- 1 ) (ignore >> connect) let conns = [| yield conn; yield! extraConns |] - Reader.chunk startPos |> int, conns, (max (conns.Length) (spec.streamReaders+1)) + Reader.chunk startPos |> int, conns, (max conns.Length (spec.streamReaders+1)) | None -> 0, [|conn|], spec.streamReaders + 1 diff --git a/src/Propulsion.EventStore/Propulsion.EventStore.fsproj b/src/Propulsion.EventStore/Propulsion.EventStore.fsproj index 2cc10aab..e72a082d 100644 --- a/src/Propulsion.EventStore/Propulsion.EventStore.fsproj +++ b/src/Propulsion.EventStore/Propulsion.EventStore.fsproj @@ -20,7 +20,7 @@ - + diff --git a/src/Propulsion.EventStore/StripedIngester.fs b/src/Propulsion.EventStore/StripedIngester.fs index ae45adba..4b84587c 100755 --- a/src/Propulsion.EventStore/StripedIngester.fs +++ b/src/Propulsion.EventStore/StripedIngester.fs @@ -9,31 +9,31 @@ open System.Collections.Concurrent open System.Threading type [] Message = - | Batch of seriesIndex : int * epoch : int64 * checkpoint : Async * items : StreamEvent seq + | Batch of seriesIndex : int * epoch : int64 * checkpoint : Async * items : Default.StreamEvent seq | CloseSeries of seriesIndex : int module StripedIngesterImpl = type Stats(log : ILogger, statsInterval) = - let statsDue = intervalCheck statsInterval + let interval = IntervalTimer statsInterval let mutable cycles, ingested = 0, 0 - let dumpStats activeSeries (readingAhead, ready) (currentBuffer, maxBuffer) = + let dumpStats activeSeries (readingAhead, ready) struct (currentBuffer, maxBuffer) = let mutable buffered = 0 let count (xs : IDictionary>) = seq { for x in xs do buffered <- buffered + x.Value.Count; yield x.Key, x.Value.Count } |> Seq.sortBy fst |> Seq.toArray let ahead, ready = count readingAhead, count ready log.Information("Read {ingested} Cycles {cycles} Series {series} Holding {buffered} Reading {@reading} Ready {@ready} Active {currentBuffer}/{maxBuffer}", ingested, cycles, activeSeries, buffered, ahead, ready, currentBuffer, maxBuffer) ingested <- 0; cycles <- 0 - member __.Handle : InternalMessage -> unit = function + member _.Handle : InternalMessage -> unit = function | Batch _ -> ingested <- ingested + 1 | ActivateSeries _ | CloseSeries _ -> () - member __.TryDump(activeSeries, readingAhead, ready, readMaxState) = + member _.TryDump(activeSeries, readingAhead, ready, readMaxState) = cycles <- cycles + 1 - if statsDue () then + if interval.IfDueRestart() then dumpStats activeSeries (readingAhead, ready) readMaxState and [] InternalMessage = - | Batch of seriesIndex : int * epoch : int64 * checkpoint : Async * items : StreamEvent seq + | Batch of seriesIndex : int * epoch : int64 * checkpoint : Async * items : Default.StreamEvent seq | CloseSeries of seriesIndex : int | ActivateSeries of seriesIndex : int @@ -48,14 +48,14 @@ open StripedIngesterImpl /// Holds batches away from Core processing to limit in-flight processing type StripedIngester - ( log : ILogger, inner : Propulsion.Ingestion.Ingester>, Propulsion.Submission.SubmissionBatch>>, + ( log : ILogger, inner : Propulsion.Ingestion.Ingester, maxInFlightBatches, initialSeriesIndex : int, statsInterval : TimeSpan, ?pumpInterval) = let cts = new CancellationTokenSource() let pumpInterval = defaultArg pumpInterval (TimeSpan.FromMilliseconds 5.) let work = ConcurrentQueue() // Queue as need ordering semantically let maxInFlightBatches = Sem maxInFlightBatches let stats = Stats(log, statsInterval) - let pending = Queue<_>() + let pending = Queue seq>>() let readingAhead, ready = Dictionary>(), Dictionary>() let mutable activeSeries = initialSeriesIndex @@ -65,10 +65,8 @@ type StripedIngester let handle = function | Batch (seriesId, epoch, checkpoint, items) -> let isForActiveStripe = activeSeries = seriesId - let batchInfo = - let items = Array.ofSeq items - let onCompleted = - if isForActiveStripe then + let onCompletion = + if isForActiveStripe then // If this read represents a batch that we will immediately submit for processing, we will defer the releasing of the batch in out buffer // limit only when the batch's processing has concluded releaseInFlightBatchAllocation @@ -78,13 +76,14 @@ type StripedIngester // any ones we hold and forward through `readingAhead` are processed) // - yield a null function as the onCompleted callback to be triggered when the batch's processing has concluded id - epoch, checkpoint, items, onCompleted + let batchInfo : Propulsion.Ingestion.Batch<_ seq> = + { epoch = epoch; items = Array.ofSeq items; checkpoint = checkpoint; onCompletion = onCompletion } if isForActiveStripe then pending.Enqueue batchInfo else match readingAhead.TryGetValue seriesId with - | false, _ -> readingAhead.[seriesId] <- ResizeArray[|batchInfo|] + | false, _ -> readingAhead[seriesId] <- ResizeArray[|batchInfo|] | true,current -> current.Add(batchInfo) // As we'll be submitting `id` as the onCompleted callback, we now immediately release the allocation that gets `Await`ed in `Submit()` releaseInFlightBatchAllocation() @@ -96,10 +95,10 @@ type StripedIngester else match readingAhead |> tryTake seriesIndex with | Some batchesRead -> - ready.[seriesIndex] <- batchesRead + ready[seriesIndex] <- batchesRead log.Information("Completed reading {series}, marking {buffered} buffered items ready", seriesIndex, batchesRead.Count) | None -> - ready.[seriesIndex] <- ResizeArray() + ready[seriesIndex] <- ResizeArray() log.Information("Completed reading {series}, leaving empty batch list", seriesIndex) | ActivateSeries newActiveSeries -> @@ -117,7 +116,7 @@ type StripedIngester log.Information("Moving to series {activeChunk}, releasing {buffered} buffered batches, {ready} others ready, {ahead} reading ahead", newActiveSeries, buffered, ready.Count, readingAhead.Count) - member __.Pump = async { + member _.Pump = async { while not cts.IsCancellationRequested do let mutable itemLimit = 1024 while itemLimit > 0 do @@ -125,14 +124,13 @@ type StripedIngester | true, x -> handle x; stats.Handle x; itemLimit <- itemLimit - 1 | false, _ -> itemLimit <- 0 while pending.Count <> 0 do - let epoch, checkpoint, items, markCompleted = pending.Dequeue() - let! _, _ = inner.Submit(epoch, checkpoint, items, markCompleted) in () + let! _, _ = inner.Ingest(pending.Dequeue()) in () stats.TryDump(activeSeries, readingAhead, ready, maxInFlightBatches.State) do! Async.Sleep pumpInterval } /// Yields (used, maximum) of in-flight batches limit /// return can be delayed where we're over the limit until such time as the background processing ingests the batch - member __.Submit(content : Message) = async { + member _.Submit(content : Message) = async { match content with | Message.Batch (seriesId, epoch, checkpoint, events) -> work.Enqueue <| Batch (seriesId, epoch, checkpoint, events) @@ -143,4 +141,4 @@ type StripedIngester return maxInFlightBatches.State } /// As range assignments get revoked, a user is expected to `Stop `the active processing thread for the Ingester before releasing references to it - member __.Stop() = cts.Cancel() + member _.Stop() = cts.Cancel() diff --git a/src/Propulsion.EventStoreDb/EventStoreSource.fs b/src/Propulsion.EventStoreDb/EventStoreSource.fs index bdf8885f..c34862fa 100644 --- a/src/Propulsion.EventStoreDb/EventStoreSource.fs +++ b/src/Propulsion.EventStoreDb/EventStoreSource.fs @@ -1,21 +1,17 @@ namespace Propulsion.EventStoreDb -type StreamEvent = Propulsion.Streams.StreamEvent - module private Impl = open FSharp.Control - let toStreamEvent (x : EventStore.Client.ResolvedEvent) : StreamEvent = + let toStreamEvent (x : EventStore.Client.ResolvedEvent) : Propulsion.Streams.Default.StreamEvent = let e = x.Event // TOCONSIDER wire e.Metadata["$correlationId"] and ["$causationId"] into correlationId and causationId // https://eventstore.org/docs/server/metadata-and-reserved-names/index.html#event-metadata let n, d, m, eu, ts = e.EventNumber, e.Data, e.Metadata, e.EventId, System.DateTimeOffset e.Created - let inline (|Len0ToNull|) (x : _[]) = match x with null -> null | x when x.Length = 0 -> null | x -> x - let Len0ToNull d, Len0ToNull m = d.ToArray(), m.ToArray() - { stream = Propulsion.Streams.StreamName.internalParseSafe x.Event.EventStreamId - event = FsCodec.Core.TimelineEvent.Create(n.ToInt64(), e.EventType, d, m, eu.ToGuid(), correlationId = null, causationId = null, timestamp = ts) } - let readBatch hydrateBodies batchSize (store : EventStore.Client.EventStoreClient) pos : Async> = async { + let e = FsCodec.Core.TimelineEvent.Create(n.ToInt64(), e.EventType, d, m, eu.ToGuid(), correlationId = null, causationId = null, timestamp = ts) + Propulsion.Streams.StreamName.internalParseSafe x.Event.EventStreamId, e + let readBatch hydrateBodies batchSize (store : EventStore.Client.EventStoreClient) pos : Async> = async { let! ct = Async.CancellationToken let pos = let p = pos |> Propulsion.Feed.Position.toInt64 |> uint64 in EventStore.Client.Position(p, p) let res = store.ReadAllAsync(EventStore.Client.Direction.Forwards, pos, batchSize, hydrateBodies, cancellationToken = ct) @@ -29,13 +25,12 @@ module private Impl = type EventStoreSource ( log : Serilog.ILogger, statsInterval, store : EventStore.Client.EventStoreClient, batchSize, tailSleepInterval, - checkpoints : Propulsion.Feed.IFeedCheckpointStore, - sink : Propulsion.ProjectorPipeline, Propulsion.Submission.SubmissionBatch>>, + checkpoints : Propulsion.Feed.IFeedCheckpointStore, sink : Propulsion.Streams.Default.Sink, // If the Handler does not utilize the bodies of the events, we can avoid shipping them from the Store in the first instance. Default false. ?hydrateBodies, // TODO borrow impl of determining tail from Propulsion.EventStore, pass that to base as ?establishOrigin // ?fromTail, ?sourceId) = - inherit Propulsion.Feed.Internal.AllFeedSource + inherit Propulsion.Feed.Core.AllFeedSource ( log, statsInterval, defaultArg sourceId FeedSourceId.wellKnownId, tailSleepInterval, Impl.readBatch (hydrateBodies = Some true) batchSize store, checkpoints, sink) diff --git a/src/Propulsion.Feed/FeedPrometheus.fs b/src/Propulsion.Feed/FeedPrometheus.fs index 92622911..54179515 100644 --- a/src/Propulsion.Feed/FeedPrometheus.fs +++ b/src/Propulsion.Feed/FeedPrometheus.fs @@ -74,7 +74,7 @@ module private Histogram = let latencyBuckets = [| 0.0005; 0.001; 0.002; 0.004; 0.008; 0.016; 0.5; 1.; 2.; 4.; 8.; 16. |] let latency = create' latencyBuckets secondsStat latencyDesc -open Propulsion.Feed.Internal.Log +open Propulsion.Feed.Core.Log /// An ILogEventSink that publishes to Prometheus /// Custom tags to annotate the metric we're publishing where such tag manipulation cannot better be achieved via the Prometheus scraper config. diff --git a/src/Propulsion.Feed/FeedReader.fs b/src/Propulsion.Feed/FeedReader.fs index d0f7bf93..ad71559e 100644 --- a/src/Propulsion.Feed/FeedReader.fs +++ b/src/Propulsion.Feed/FeedReader.fs @@ -1,4 +1,4 @@ -namespace Propulsion.Feed.Internal +namespace Propulsion.Feed.Core open FSharp.Control open Propulsion // Async.Sleep, Raise @@ -7,7 +7,7 @@ open Serilog open System [] -type Batch<'e> = { items : Propulsion.Streams.StreamEvent<'e>[]; checkpoint : Position; isTail : bool } +type Batch<'F> = { items : Propulsion.Streams.StreamEvent<'F>[]; checkpoint : Position; isTail : bool } module internal TimelineEvent = @@ -101,17 +101,13 @@ type FeedReader crawl : bool // lastWasTail : may be used to induce a suitable backoff when repeatedly reading from tail * Position // checkpointPosition - -> AsyncSeq>, + -> AsyncSeq)>, // Feed a batch into the ingester. Internal checkpointing decides which Commit callback will be called // Throwing will tear down the processing loop, which is intended; we fail fast on poison messages // In the case where the number of batches reading has gotten ahead of processing exceeds the limit, // submitBatch triggers the backoff of the reading ahead loop by sleeping prior to returning - submitBatch : - int64 // unique tag used to identify batch in internal logging - * Async // commit callback. Internal checkpointing dictates when it will be called. - * seq> - // Yields (current batches pending,max readAhead) for logging purposes - -> Async, + // Yields (current batches pending,max readAhead) for logging purposes + submitBatch : Ingestion.Batch -> Async, // Periodically triggered, asynchronously, by the scheduler as processing of submitted batches progresses // Should make one attempt to persist a checkpoint // Throwing exceptions is acceptable; retrying and handling of exceptions is managed by the internal loop @@ -135,17 +131,17 @@ type FeedReader match logCommitFailure with None -> log.ForContext().Debug(e, "Exception while committing checkpoint {position}", position) | Some l -> l e return! Async.Raise e } - let submitPage (readLatency, batch : Batch) = async { + let submitPage (readLatency, batch : Batch<_>) = async { stats.RecordBatch(readLatency, batch) match Array.length batch.items with | 0 -> log.Verbose("Page {latency:f0}ms Checkpoint {checkpoint} Empty", readLatency.TotalMilliseconds, batch.checkpoint) | c -> if log.IsEnabled(Serilog.Events.LogEventLevel.Debug) then - let streamsCount = batch.items |> Seq.distinctBy (fun x -> x.stream) |> Seq.length + let streamsCount = batch.items |> Seq.distinctBy ValueTuple.fst |> Seq.length log.Debug("Page {latency:f0}ms Checkpoint {checkpoint} {eventCount}e {streamCount}s", readLatency.TotalMilliseconds, batch.checkpoint, c, streamsCount) let epoch, streamEvents : int64 * Propulsion.Streams.StreamEvent<_> seq = int64 batch.checkpoint, Seq.ofArray batch.items let ingestTimer = System.Diagnostics.Stopwatch.StartNew() - let! cur, max = submitBatch (epoch, commit batch.checkpoint, streamEvents) + let! cur, max = submitBatch { epoch = epoch; checkpoint = commit batch.checkpoint; items = streamEvents; onCompletion = ignore } stats.UpdateCurMax(ingestTimer.Elapsed, cur, max) } member _.Pump(initialPosition : Position) = async { diff --git a/src/Propulsion.Feed/FeedSource.fs b/src/Propulsion.Feed/FeedSource.fs index f6448cdf..cd2f9b56 100644 --- a/src/Propulsion.Feed/FeedSource.fs +++ b/src/Propulsion.Feed/FeedSource.fs @@ -1,9 +1,9 @@ -namespace Propulsion.Feed.Internal +namespace Propulsion.Feed.Core open FSharp.Control open Propulsion open Propulsion.Feed -open Propulsion.Streams +open Propulsion.Internal open System open System.Threading.Tasks @@ -11,33 +11,31 @@ open System.Threading.Tasks type FeedSourceBase internal ( log : Serilog.ILogger, statsInterval : TimeSpan, sourceId, checkpoints : IFeedCheckpointStore, establishOrigin : (TrancheId -> Async) option, - sink : ProjectorPipeline>, Submission.SubmissionBatch>>>, + sink : Propulsion.Streams.Default.Sink, renderPos : Position -> string, ?logCommitFailure) = - let log = log.ForContext("source", sourceId) let pumpPartition crawl partitionId trancheId = async { let log = log.ForContext("tranche", trancheId) - let ingester : Ingestion.Ingester<_, _> = sink.StartIngester(log, partitionId) - let reader = FeedReader(log, sourceId, trancheId, statsInterval, crawl trancheId, ingester.Submit, checkpoints.Commit, renderPos, ?logCommitFailure = logCommitFailure) - try let! freq, pos = checkpoints.Start(sourceId, trancheId, ?establishOrigin = (match establishOrigin with None -> None | Some f -> Some (f trancheId))) + let ingester : Ingestion.Ingester<_> = sink.StartIngester(log, partitionId) + let reader = FeedReader(log, sourceId, trancheId, statsInterval, crawl trancheId, ingester.Ingest, checkpoints.Commit, renderPos, ?logCommitFailure = logCommitFailure) + try let! freq, pos = checkpoints.Start(sourceId, trancheId, ?establishOrigin = (establishOrigin |> Option.map (fun f -> f trancheId))) log.Information("Reading {source:l}/{tranche:l} From {pos} Checkpoint Event interval {checkpointFreq:n1}m", sourceId, trancheId, renderPos pos, freq.TotalMinutes) return! reader.Pump(pos) with e -> log.Warning(e, "Exception encountered while running reader, exiting loop") - return! Async.Raise e - } + return! Async.Raise e } /// Propagates exceptions raised by readTranches or crawl, member internal _.Pump ( readTranches : unit -> Async, // Responsible for managing retries and back offs; yielding an exception will result in abend of the read loop - crawl : TrancheId -> bool * Position -> AsyncSeq>) = async { + crawl : TrancheId -> bool * Position -> AsyncSeq)>) = async { // TODO implement behavior to pick up newly added tranches by periodically re-running readTranches // TODO when that's done, remove workaround in readTranches - try let! tranches = readTranches () + try let! (tranches : TrancheId array) = readTranches () log.Information("Starting {tranches} tranche readers...", tranches.Length) return! Async.Parallel(tranches |> Seq.mapi (pumpPartition crawl)) |> Async.Ignore with e -> @@ -49,9 +47,8 @@ type FeedSourceBase internal type TailingFeedSource ( log : Serilog.ILogger, statsInterval : TimeSpan, sourceId, tailSleepInterval : TimeSpan, - crawl : TrancheId * Position -> AsyncSeq>, - checkpoints : IFeedCheckpointStore, establishOrigin : (TrancheId -> Async) option, - sink : ProjectorPipeline>, Submission.SubmissionBatch>>>, + crawl : TrancheId * Position -> AsyncSeq)>, + checkpoints : IFeedCheckpointStore, establishOrigin : (TrancheId -> Async) option, sink : Propulsion.Streams.Default.Sink, renderPos, ?logReadFailure, ?readFailureSleepInterval : TimeSpan, @@ -79,7 +76,7 @@ type TailingFeedSource let ct = cts.Token let tcs = System.Threading.Tasks.TaskCompletionSource() - let propagateExceptionToPipelineOutcome f = async { try do! f with e -> tcs.SetException(e) } + let propagateExceptionToPipelineOutcome inner = async { try do! inner with e -> tcs.SetException(e) } let startPump () = Async.Start(propagateExceptionToPipelineOutcome pump, cancellationToken = ct) @@ -100,8 +97,7 @@ type AllFeedSource ( log : Serilog.ILogger, statsInterval : TimeSpan, sourceId, tailSleepInterval : TimeSpan, readBatch : Position -> Async>, - checkpoints : IFeedCheckpointStore, - sink : ProjectorPipeline>, Submission.SubmissionBatch>>>, + checkpoints : IFeedCheckpointStore, sink : Propulsion.Streams.Default.Sink, // Custom checkpoint rendering logic ?renderPos, // Custom logic to derive an origin position if the checkpoint store doesn't have one @@ -125,12 +121,10 @@ type AllFeedSource namespace Propulsion.Feed open FSharp.Control -open Propulsion -open Propulsion.Streams open System [] -type Page<'e> = { items : FsCodec.ITimelineEvent<'e>[]; checkpoint : Position; isTail : bool } +type Page<'F> = { items : FsCodec.ITimelineEvent<'F>[]; checkpoint : Position; isTail : bool } /// Drives reading and checkpointing for a set of change feeds (tranches) of a custom data source that can represent their /// content as an append-only data source with a change feed wherein each FsCodec.ITimelineEvent has a monotonically increasing Index.
@@ -138,12 +132,11 @@ type Page<'e> = { items : FsCodec.ITimelineEvent<'e>[]; checkpoint : Position; i type FeedSource ( log : Serilog.ILogger, statsInterval : TimeSpan, sourceId, tailSleepInterval : TimeSpan, - checkpoints : IFeedCheckpointStore, + checkpoints : IFeedCheckpointStore, sink : Propulsion.Streams.Default.Sink, // Responsible for managing retries and back offs; yielding an exception will result in abend of the read loop - readPage : TrancheId * Position -> Async>, - sink : ProjectorPipeline>, Submission.SubmissionBatch>>>, + readPage : TrancheId * Position -> Async>, ?renderPos) = - inherit Internal.FeedSourceBase(log, statsInterval, sourceId, checkpoints, None, sink, defaultArg renderPos string) + inherit Core.FeedSourceBase(log, statsInterval, sourceId, checkpoints, None, sink, defaultArg renderPos string) let crawl trancheId = let streamName = FsCodec.StreamName.compose "Messages" [SourceId.toString sourceId; TrancheId.toString trancheId] @@ -152,8 +145,8 @@ type FeedSource do! Async.Sleep tailSleepInterval let sw = System.Diagnostics.Stopwatch.StartNew() let! page = readPage (trancheId, pos) - let items' = page.items |> Array.map (fun x -> { stream = streamName; event = x }) - yield sw.Elapsed, ({ items = items'; checkpoint = page.checkpoint; isTail = page.isTail } : Internal.Batch<_>) + let items' = page.items |> Array.map (fun x -> struct (streamName, x)) + yield struct (sw.Elapsed, ({ items = items'; checkpoint = page.checkpoint; isTail = page.isTail } : Core.Batch<_>)) } /// Drives the continual loop of reading and checkpointing each tranche until a fault occurs.
diff --git a/src/Propulsion.Feed/PeriodicSource.fs b/src/Propulsion.Feed/PeriodicSource.fs index 55489ee5..b8deb170 100644 --- a/src/Propulsion.Feed/PeriodicSource.fs +++ b/src/Propulsion.Feed/PeriodicSource.fs @@ -5,8 +5,6 @@ namespace Propulsion.Feed open FSharp.Control -open Propulsion -open Propulsion.Streams open System /// Int64.MaxValue = 9223372036854775807 @@ -41,7 +39,7 @@ module private TimelineEvent = baseIndex + i, x.EventType, x.Data, x.Meta, x.EventId, x.CorrelationId, x.CausationId, x.Timestamp, isUnfold = true, context = context) [] -type SourceItem = { streamName : FsCodec.StreamName; eventData : FsCodec.IEventData; context : obj } +type SourceItem<'F> = { streamName : FsCodec.StreamName; eventData : FsCodec.IEventData<'F>; context : obj } /// Drives reading and checkpointing for a custom source which does not have a way to incrementally query the data within as a change feed.
/// Reads the supplied `source` at `pollInterval` intervals, offsetting the `Index` of the events read based on the start time of the traversal @@ -51,14 +49,13 @@ type PeriodicSource ( log : Serilog.ILogger, statsInterval : TimeSpan, sourceId, // The AsyncSeq is expected to manage its own resilience strategy (retries etc).
// Yielding an exception will result in the Pump loop terminating, tearing down the source pipeline - crawl : TrancheId -> AsyncSeq, refreshInterval : TimeSpan, - checkpoints : IFeedCheckpointStore, - sink : ProjectorPipeline>, Submission.SubmissionBatch>>>, + crawl : TrancheId -> AsyncSeq array)>, refreshInterval : TimeSpan, + checkpoints : IFeedCheckpointStore, sink : Propulsion.Streams.Default.Sink, ?renderPos) = - inherit Internal.FeedSourceBase(log, statsInterval, sourceId, checkpoints, None, sink, defaultArg renderPos DateTimeOffsetPosition.render) + inherit Core.FeedSourceBase(log, statsInterval, sourceId, checkpoints, None, sink, defaultArg renderPos DateTimeOffsetPosition.render) // We don't want to checkpoint for real until we know the scheduler has handled the full set of pages in the crawl. - let crawl trancheId (_wasLast, position) : AsyncSeq> = asyncSeq { + let crawl trancheId (_wasLast, position) : AsyncSeq)> = asyncSeq { let startDate = DateTimeOffsetPosition.getDateTimeOffset position let dueDate = startDate + refreshInterval match dueDate - DateTimeOffset.UtcNow with @@ -74,11 +71,11 @@ type PeriodicSource let mutable elapsed = TimeSpan.Zero for ts, xs in crawl trancheId do elapsed <- elapsed + ts - let streamEvents = seq { + let streamEvents : Propulsion.Streams.StreamEvent<_> seq = seq { for si in xs -> let i = index index <- index + 1L - { StreamEvent.stream = si.streamName; event = mkTimelineEvent (i, si.eventData, si.context) } + si.streamName, mkTimelineEvent (i, si.eventData, si.context) } buffer.AddRange(streamEvents) match buffer.Count - 1 with @@ -86,14 +83,14 @@ type PeriodicSource let items = Array.zeroCreate ready buffer.CopyTo(0, items, 0, ready) buffer.RemoveRange(0, ready) - yield elapsed, ({ items = items; checkpoint = position; isTail = false } : Internal.Batch<_> ) + yield struct (elapsed, ({ items = items; checkpoint = position; isTail = false } : Core.Batch<_>)) elapsed <- TimeSpan.Zero | _ -> () let items, checkpoint = match buffer.ToArray() with | [||] as noItems -> noItems, basePosition - | finalItem -> finalItem, (Array.last finalItem).event |> Internal.TimelineEvent.toCheckpointPosition - yield elapsed, ({ items = items; checkpoint = checkpoint; isTail = true } : Internal.Batch<_>) } + | finalItem -> finalItem, let struct (_s, e) = Array.last finalItem in e |> Core.TimelineEvent.toCheckpointPosition + yield elapsed, ({ items = items; checkpoint = checkpoint; isTail = true } : Core.Batch<_>) } /// Drives the continual loop of reading and checkpointing each tranche until a fault occurs.
/// The readTranches and crawl functions are expected to manage their own resilience strategies (retries etc).
diff --git a/src/Propulsion.Kafka/Codec.fs b/src/Propulsion.Kafka/Codec.fs index 6b6bed3a..1c0a136b 100644 --- a/src/Propulsion.Kafka/Codec.fs +++ b/src/Propulsion.Kafka/Codec.fs @@ -3,7 +3,6 @@ // (and/or series of nugets, with an implementation per concrete serialization stack) namespace Propulsion.Codec.NewtonsoftJson -open FsCodec open FsCodec.NewtonsoftJson open Newtonsoft.Json open Propulsion.Streams @@ -11,7 +10,7 @@ open Propulsion.Streams /// Prepackaged serialization helpers with appropriate settings given the types will roundtrip correctly with default Json.net settings type Serdes private () = - static let serdes = lazy NewtonsoftJson.Serdes Options.Default + static let serdes = lazy FsCodec.NewtonsoftJson.Serdes Options.Default static member Serialize<'T>(value : 'T) : string = serdes.Value.Serialize(value) static member Deserialize(json : string) : 'T = serdes.Value.Deserialize(json) @@ -34,13 +33,13 @@ type [] RenderedEvent = m: byte[] } interface FsCodec.IEventData with - member __.EventType = __.c - member __.Data = __.d - member __.Meta = __.m - member __.EventId = System.Guid.Empty - member __.CorrelationId = null - member __.CausationId = null - member __.Timestamp = __.t + member x.EventType = x.c + member x.Data = x.d + member x.Meta = x.m + member _.EventId = System.Guid.Empty + member _.CorrelationId = null + member _.CausationId = null + member x.Timestamp = x.t /// Rendition of a contiguous span of events for a given stream type [] RenderedSpan = @@ -59,15 +58,16 @@ type [] RenderedSpan = /// Helpers for mapping to/from `Propulsion.Streams` canonical event types module RenderedSpan = - let ofStreamSpan (streamName : StreamName) (span : StreamSpan<_>) : RenderedSpan = - { s = StreamName.toString streamName - i = span.index - e = span.events |> Array.map (fun x -> { c = x.EventType; t = x.Timestamp; d = x.Data; m = x.Meta }) } + let ofStreamSpan (streamName : FsCodec.StreamName) (span : Default.StreamSpan) : RenderedSpan = + let ta (x : Default.EventBody) = x.ToArray() + { s = FsCodec.StreamName.toString streamName + i = span[0].Index + e = span |> Array.map (fun x -> { c = x.EventType; t = x.Timestamp; d = ta x.Data; m = ta x.Meta }) } - let enum (span: RenderedSpan) : StreamEvent<_> seq = + let enum (span: RenderedSpan) : StreamEvent seq = let streamName = StreamName.internalParseSafe span.s let inline mkEvent offset (e : RenderedEvent) = FsCodec.Core.TimelineEvent.Create(span.i+int64 offset, e.c, e.d, e.m, timestamp=e.t) - span.e |> Seq.mapi (fun i e -> { stream = streamName; event = mkEvent i e }) + span.e |> Seq.mapi (fun i e -> streamName, mkEvent i e) let parse (spanJson: string) : StreamEvent<_> seq = spanJson |> RenderedSpan.Parse |> enum @@ -89,17 +89,17 @@ type [] RenderedSummary = /// Helpers for mapping to/from `Propulsion.Streams` canonical event contract module RenderedSummary = - let ofStreamEvents (streamName : StreamName) (index : int64) (events : FsCodec.IEventData seq) : RenderedSummary = - { s = StreamName.toString streamName + let ofStreamEvents (streamName : FsCodec.StreamName) (index : int64) (events : FsCodec.IEventData seq) : RenderedSummary = + { s = FsCodec.StreamName.toString streamName i = index u = [| for x in events -> { c = x.EventType; t = x.Timestamp; d = x.Data; m = x.Meta } |] } - let ofStreamEvent (streamName : StreamName) (index : int64) (event : FsCodec.IEventData) : RenderedSummary = + let ofStreamEvent (streamName : FsCodec.StreamName) (index : int64) (event : FsCodec.IEventData) : RenderedSummary = ofStreamEvents streamName index (Seq.singleton event) let enum (span: RenderedSummary) : StreamEvent<_> seq = let streamName = StreamName.internalParseSafe span.s - seq { for e in span.u -> { stream = streamName; event = FsCodec.Core.TimelineEvent.Create(span.i, e.c, e.d, e.m, timestamp=e.t, isUnfold=true) } } + seq { for e in span.u -> streamName, FsCodec.Core.TimelineEvent.Create(span.i, e.c, e.d, e.m, timestamp=e.t, isUnfold=true) } let parse (spanJson: string) : StreamEvent<_> seq = spanJson |> RenderedSummary.Parse |> enum diff --git a/src/Propulsion.Kafka/Consumers.fs b/src/Propulsion.Kafka/Consumers.fs index 160f9631..2d06c7e9 100644 --- a/src/Propulsion.Kafka/Consumers.fs +++ b/src/Propulsion.Kafka/Consumers.fs @@ -7,6 +7,7 @@ open FsCodec open FsKafka open Propulsion open Propulsion.Internal // intervalCheck, AwaitTaskCorrect +open Propulsion.Streams open Serilog open System open System.Collections.Generic @@ -46,7 +47,6 @@ module private Impl = let approximateMessageBytes (m : Message) = let inline len (x : string) = match x with null -> 0 | x -> sizeof * x.Length 16 + len m.Key + len m.Value |> int64 - let inline mb x = float x / 1024. / 1024. module private Binding = @@ -60,7 +60,7 @@ module private Binding = /// Pauses if in-flight upper threshold is breached until such time as it drops below that the lower limit type KafkaIngestionEngine<'Info> ( log : ILogger, counter : Core.InFlightMessageCounter, consumer : IConsumer<_, _>, closeConsumer, - mapMessage : ConsumeResult<_, _> -> 'Info, emit : Submission.SubmissionBatch[] -> unit, + mapMessage : ConsumeResult<_, _> -> 'Info, emit : Submission.Batch[] -> unit, maxBatchSize, emitInterval, statsInterval) = let acc = Dictionary() let remainingIngestionWindow = intervalTimer emitInterval @@ -71,9 +71,9 @@ type KafkaIngestionEngine<'Info> intervalMsgs, intervalChars, counter.InFlightMb, totalMessages, totalChars) intervalMsgs <- 0L; intervalChars <- 0L let maybeLogStats = - let due = intervalCheck statsInterval - fun () -> if due () then dumpStats () - let mkSubmission topicPartition span : Submission.SubmissionBatch<'S, 'M> = + let interval = IntervalTimer statsInterval + fun () -> if interval.IfDueRestart() then dumpStats () + let mkSubmission topicPartition span : Submission.Batch<'S, 'M> = let checkpoint () = counter.Delta(-span.reservation) // counterbalance Delta(+) per ingest, below try consumer.StoreOffset(span.highWaterMark) @@ -99,7 +99,7 @@ type KafkaIngestionEngine<'Info> match acc.Count with | 0 -> () | topicPartitionsWithMessagesThisInterval -> - let tmp = ResizeArray>(topicPartitionsWithMessagesThisInterval) + let tmp = ResizeArray>(topicPartitionsWithMessagesThisInterval) for KeyValue(tp, span) in acc do tmp.Add(mkSubmission tp span) acc.Clear() @@ -114,8 +114,8 @@ type KafkaIngestionEngine<'Info> maybeLogStats() counter.AwaitThreshold(ct, consumer, busyWork) | false, None -> - submit() - maybeLogStats() + submit () + maybeLogStats () | false, Some intervalRemainder -> try match consumer.Consume(intervalRemainder) with | null -> () @@ -125,7 +125,7 @@ type KafkaIngestionEngine<'Info> finally submit () // We don't want to leak our reservations against the counter and want to pass of messages we ingested dumpStats () // Unconditional logging when completing - closeConsumer() (* Orderly Close() before Dispose() is critical *) } + closeConsumer () (* Orderly Close() before Dispose() is critical *) } /// Consumes according to the `config` supplied to `Start`, until `Stop()` is requested or `handle` yields a fault. /// Conclusion of processing can be awaited by via `AwaitShutdown` or `AwaitWithStopOnCancellation`. @@ -202,7 +202,7 @@ type ParallelConsumer private () = let dispatcher = Parallel.Scheduling.Dispatcher maxDop let scheduler = Parallel.Scheduling.PartitionedSchedulingEngine<_, 'Msg>(log, handle, dispatcher.TryAdd, statsInterval, ?logExternalStats=logExternalStats) - let mapBatch onCompletion (x : Submission.SubmissionBatch<_, _>) : Parallel.Scheduling.Batch<_, 'Msg> = + let mapBatch onCompletion (x : Submission.Batch<_, _>) : Parallel.Scheduling.Batch<_, 'Msg> = let onCompletion' () = x.onCompletion(); onCompletion() { source = x.source; messages = x.messages; onCompletion = onCompletion'; } let submitBatch (x : Parallel.Scheduling.Batch<_, _>) : int = @@ -222,8 +222,6 @@ type ParallelConsumer private () = ParallelConsumer.Start>(log, config, maxDop, Binding.mapConsumeResult, handle >> Async.Catch, ?maxSubmissionsPerPartition=maxSubmissionsPerPartition, ?statsInterval=statsInterval, ?logExternalStats=logExternalStats) -type EventMetrics = Streams.EventMetrics - /// APIs only required for advanced scenarios (specifically the integration tests) /// APIs within are not part of the stable API and are subject to unlimited change module Core = @@ -233,23 +231,24 @@ module Core = static member Start<'Info, 'Outcome> ( log : ILogger, config : KafkaConsumerConfig, resultToInfo, infoToStreamEvents, prepare, handle, maxDop, - stats : Streams.Scheduling.Stats, statsInterval, + stats : Scheduling.Stats, statsInterval, ?maxSubmissionsPerPartition, ?logExternalState, ?maxBatches, ?purgeInterval, ?wakeForResults, ?idleDelay, ?maximizeOffsetWriting) = let maxSubmissionsPerPartition = defaultArg maxSubmissionsPerPartition 5 - let dispatcher = Streams.Scheduling.ItemDispatcher<_> maxDop - let dumpStreams struct (streams : Streams.Scheduling.StreamStates<_>, totalPurged) log = + let dispatcher = Dispatch.ItemDispatcher<_, _> maxDop + let dumpStreams logStreamStates log = logExternalState |> Option.iter (fun f -> f log) - streams.Dump(log, totalPurged, Streams.Buffering.StreamState.eventsSize) - let streamsScheduler = Streams.Scheduling.StreamSchedulingEngine.Create<_, _, _>( - dispatcher, stats, prepare, handle, Streams.SpanResult.toIndex, dumpStreams, - ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay) - let mapConsumedMessagesToStreamsBatch onCompletion (x : Submission.SubmissionBatch) : Streams.Scheduling.StreamsBatch<_> = + logStreamStates Default.eventSize + let streamsScheduler = + Scheduling.StreamSchedulingEngine.Create<_, _, _, _>( + dispatcher, stats, prepare, handle, SpanResult.toIndex, dumpStreams, + ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay) + let mapConsumedMessagesToStreamsBatch onCompletion (x : Submission.Batch) : Buffer.Batch<_> = let onCompletion () = x.onCompletion(); onCompletion() - Streams.Scheduling.StreamsBatch.Create(onCompletion, Seq.collect infoToStreamEvents x.messages) |> fst + Buffer.Batch.Create(onCompletion, Seq.collect infoToStreamEvents x.messages) |> ValueTuple.fst let submitter = - Streams.Projector.StreamsSubmitter.Create + Projector.StreamsSubmitter.Create ( log, maxSubmissionsPerPartition, mapConsumedMessagesToStreamsBatch, streamsScheduler.Submit, statsInterval, ?disableCompaction=maximizeOffsetWriting) @@ -257,14 +256,14 @@ module Core = static member Start<'Info, 'Outcome> ( log : ILogger, config : KafkaConsumerConfig, consumeResultToInfo, infoToStreamEvents, - handle : StreamName * Streams.StreamSpan<_> -> Async, maxDop, - stats : Streams.Scheduling.Stats, statsInterval, + handle : struct (StreamName * StreamSpan<_>) -> Async, maxDop, + stats : Scheduling.Stats, statsInterval, ?maxSubmissionsPerPartition, ?logExternalState, ?maxBatches, ?purgeInterval, ?wakeForResults, ?idleDelay, ?maximizeOffsetWriting) = - let prepare (streamName, span) = - let stats = Streams.Buffering.StreamSpan.stats span - stats, (streamName, span) + let prepare struct (streamName, span) = + let metrics = StreamSpan.metrics Default.eventSize span + struct (metrics, struct (streamName, span)) StreamsConsumer.Start<'Info, 'Outcome>( log, config, consumeResultToInfo, infoToStreamEvents, prepare, handle, maxDop, stats, statsInterval, @@ -283,9 +282,9 @@ module Core = ( log : ILogger, config : KafkaConsumerConfig, // often implemented via StreamNameSequenceGenerator.KeyValueToStreamEvent keyValueToStreamEvents, - prepare, handle : StreamName * Streams.StreamSpan<_> -> Async, + prepare, handle : struct (StreamName * StreamSpan<_>) -> Async, maxDop, - stats : Streams.Scheduling.Stats, statsInterval, + stats : Scheduling.Stats, statsInterval, ?maxSubmissionsPerPartition, ?logExternalState, ?maxBatches, ?purgeInterval, ?wakeForResults, ?idleDelay, ?maximizeOffsetWriting) = @@ -300,9 +299,9 @@ module Core = static member Start<'Outcome> ( log : ILogger, config : KafkaConsumerConfig, // often implemented via StreamNameSequenceGenerator.KeyValueToStreamEvent - keyValueToStreamEvents : KeyValuePair -> Propulsion.Streams.StreamEvent<_> seq, - handle : StreamName * Streams.StreamSpan<_> -> Async, maxDop, - stats : Streams.Scheduling.Stats, statsInterval, + keyValueToStreamEvents : KeyValuePair -> StreamEvent<_> seq, + handle : struct (StreamName * StreamSpan<_>) -> Async, maxDop, + stats : Scheduling.Stats, statsInterval, ?maxSubmissionsPerPartition, ?logExternalState, ?maxBatches, ?purgeInterval, ?wakeForResults, ?idleDelay, ?maximizeOffsetWriting) = @@ -316,7 +315,7 @@ module Core = // Maps a (potentially `null`) message key to a valid {Category}-{StreamId} StreamName for routing and/or propagation through StreamsProjector let parseMessageKey defaultCategory = function | null -> FsCodec.StreamName.create defaultCategory "" - | key -> Propulsion.Streams.StreamName.parseWithDefaultCategory defaultCategory key + | key -> StreamName.parseWithDefaultCategory defaultCategory key let toTimelineEvent toDataAndContext (result : ConsumeResult, index) = let data, context = toDataAndContext result FsCodec.Core.TimelineEvent.Create(index, String.Empty, data, context = context) @@ -329,12 +328,12 @@ module Core = // This type is not a formal part of the API; the intent is to provide context as to the origin of the event for insertion into a parser error message // Arguably it should be an anonymous record... type ConsumeResultContext = { topic : string; partition : int; offset : int64 } - let toDataAndContext (result : ConsumeResult) : byte[] * obj = + let toDataAndContext (result : ConsumeResult) : Default.EventBody * obj = let m = Binding.message result if m = null then invalidOp "Cannot dereference null message" let data = System.Text.Encoding.UTF8.GetBytes m.Value let context = { topic = result.Topic; partition = Binding.partitionValue result.Partition; offset = Binding.offsetValue result.Offset } - (data, box context) + (ReadOnlyMemory data, box context) /// StreamsProjector buffers and deduplicates messages from a contiguous stream with each event bearing a monotonically incrementing `index`. /// Where the messages we consume don't have such characteristics, we need to maintain a fake `index` by keeping an int per stream in a dictionary @@ -342,7 +341,7 @@ module Core = type StreamNameSequenceGenerator() = // Last-used index per streamName - let indices = System.Collections.Generic.Dictionary() + let indices = Dictionary() /// Generates an index for the specified StreamName. Sequence starts at 0, incrementing per call. member _.GenerateIndex(streamName : StreamName) = @@ -355,36 +354,36 @@ type StreamNameSequenceGenerator() = member __.ConsumeResultToStreamEvent ( toStreamName : ConsumeResult<_, _> -> StreamName, toTimelineEvent : ConsumeResult<_, _> * int64 -> ITimelineEvent<_>) - : ConsumeResult<_, _> -> Propulsion.Streams.StreamEvent seq = + : ConsumeResult<_, _> -> Default.StreamEvent seq = fun consumeResult -> let sn = toStreamName consumeResult let e = toTimelineEvent (consumeResult, __.GenerateIndex sn) - Seq.singleton { stream = sn; event = e } + Seq.singleton (sn, e) /// Enables customizing of mapping from ConsumeResult to
/// 1) The StreamName
/// 2) The ITimelineEvent.Data : byte[], which bears the (potentially transformed in toDataAndContext) UTF-8 payload
/// 3) The ITimelineEvent.Context : obj, which can be used to include any metadata - member __.ConsumeResultToStreamEvent + member x.ConsumeResultToStreamEvent ( toStreamName : ConsumeResult<_, _> -> StreamName, - toDataAndContext : ConsumeResult<_, _> -> byte[] * obj) - : ConsumeResult -> Propulsion.Streams.StreamEvent seq = - __.ConsumeResultToStreamEvent(toStreamName, Core.toTimelineEvent toDataAndContext) + toDataAndContext : ConsumeResult<_, _> -> Default.EventBody * obj) + : ConsumeResult -> Default.StreamEvent seq = + x.ConsumeResultToStreamEvent(toStreamName, Core.toTimelineEvent toDataAndContext) /// Enables customizing of mapping from ConsumeResult to
/// 1) The ITimelineEvent.Data : byte[], which bears the (potentially transformed in toDataAndContext) UTF-8 payload
/// 2) The ITimelineEvent.Context : obj, which can be used to include any metadata - member __.ConsumeResultToStreamEvent(toDataAndContext : ConsumeResult<_, _> -> byte[] * obj, ?defaultCategory) - : ConsumeResult -> Propulsion.Streams.StreamEvent seq = + member x.ConsumeResultToStreamEvent(toDataAndContext : ConsumeResult<_, _> -> Default.EventBody * obj, ?defaultCategory) + : ConsumeResult -> Default.StreamEvent seq = let defaultCategory = defaultArg defaultCategory "" - __.ConsumeResultToStreamEvent(Core.toStreamName defaultCategory, Core.toTimelineEvent toDataAndContext) + x.ConsumeResultToStreamEvent(Core.toStreamName defaultCategory, Core.toTimelineEvent toDataAndContext) /// Enables customizing of mapping from ConsumeResult to the StreamName
/// The body of the message is passed as the ITimelineEvent.Data
/// Stores the topic, partition and offset as a ConsumeResultContext in the ITimelineEvent.Context - member __.ConsumeResultToStreamEvent(toStreamName : ConsumeResult<_, _> -> StreamName) - : ConsumeResult -> Propulsion.Streams.StreamEvent seq = - __.ConsumeResultToStreamEvent(toStreamName, Core.toDataAndContext) + member x.ConsumeResultToStreamEvent(toStreamName : ConsumeResult<_, _> -> StreamName) + : ConsumeResult -> Default.StreamEvent seq = + x.ConsumeResultToStreamEvent(toStreamName, Core.toDataAndContext) /// Default Mapping:
/// - Treats null keys as having streamId of ""
@@ -392,15 +391,15 @@ type StreamNameSequenceGenerator() = /// - Stores the topic, partition and offset as a ConsumeResultContext in the ITimelineEvent.Context member x.ConsumeResultToStreamEvent( // Placeholder category to use for StreamName where key is null and/or does not adhere to standard {category}-{streamId} form - ?defaultCategory) : ConsumeResult -> Propulsion.Streams.StreamEvent seq = + ?defaultCategory) : ConsumeResult -> StreamEvent seq = let defaultCategory = defaultArg defaultCategory "" x.ConsumeResultToStreamEvent(Core.toStreamName defaultCategory) /// Takes the key and value as extracted from the ConsumeResult, mapping them respectively to the StreamName and ITimelineEvent.Data - member x.KeyValueToStreamEvent(KeyValue (k, v : string), ?eventType, ?defaultCategory) : Propulsion.Streams.StreamEvent seq = + member x.KeyValueToStreamEvent(KeyValue (k, v : string), ?eventType, ?defaultCategory) : StreamEvent seq = let sn = Core.parseMessageKey (defaultArg defaultCategory String.Empty) k - let e = FsCodec.Core.TimelineEvent.Create(x.GenerateIndex sn, defaultArg eventType String.Empty, System.Text.Encoding.UTF8.GetBytes v) - Seq.singleton { stream = sn; event = e } + let e = FsCodec.Core.TimelineEvent.Create(x.GenerateIndex sn, defaultArg eventType String.Empty, System.Text.Encoding.UTF8.GetBytes v |> ReadOnlyMemory) + Seq.singleton (sn, e) type StreamsConsumer = @@ -414,19 +413,19 @@ type StreamsConsumer = static member Start<'Outcome> ( log : ILogger, config : KafkaConsumerConfig, // often implemented via StreamNameSequenceGenerator.ConsumeResultToStreamEvent where the incoming message does not have an embedded sequence number - consumeResultToStreamEvents : ConsumeResult<_, _> -> Propulsion.Streams.StreamEvent<_> seq, + consumeResultToStreamEvents : ConsumeResult<_, _> -> StreamEvent<_> seq, // Handler responses: // - first component: Index at which next processing will proceed (which can trigger discarding of earlier items on that stream) // - second component: Outcome (can be simply unit), to pass to the stats processor // - throwing marks the processing of a stream as having faulted (the stream's pending events and/or // new ones that arrived while the handler was processing are then eligible for retry purposes in the next dispatch cycle) - handle : StreamName * Streams.StreamSpan<_> -> Async, + handle : struct (StreamName * StreamSpan<_>) -> Async, // The maximum number of instances of handle that are permitted to be dispatched at any point in time. // The scheduler seeks to maximise the in-flight handlers at any point in time. // The scheduler guarantees to never schedule two concurrent handler invocations for the same stream. maxDop, // The 'Outcome from each handler invocation is passed to the Statistics processor by the scheduler for periodic emission - stats : Streams.Scheduling.Stats, statsInterval, + stats : Scheduling.Stats, statsInterval, ?maxSubmissionsPerPartition, ?logExternalState, ?maxBatches, ?purgeInterval, ?wakeForResults, ?idleDelay, // Prevent batches being consolidated prior to scheduling in order to maximize granularity of consumer offset updates @@ -457,9 +456,9 @@ type BatchesConsumer = // - Choice1Of2: Index at which next processing will proceed (which can trigger discarding of earlier items on that stream) // - Choice2Of2: Records the processing of the stream in question as having faulted (the stream's pending events and/or // new ones that arrived while the handler was processing are then eligible for retry purposes in the next dispatch cycle) - handle : Streams.Scheduling.DispatchItem<_>[] -> Async>>, + handle : Dispatch.Item<_>[] -> Async>>, // The responses from each handle invocation are passed to stats for periodic emission - stats : Streams.Scheduling.Stats, statsInterval, + stats : Scheduling.Stats, statsInterval, // Maximum number of batches to ingest for scheduling at any one time (Default: 24.) // NOTE Stream-wise consumption defaults to taking 5 batches each time replenishment is required ?schedulerIngestionBatchCount, @@ -471,11 +470,11 @@ type BatchesConsumer = ?logExternalState) = let maxBatches = defaultArg schedulerIngestionBatchCount 24 let maxSubmissionsPerPartition = defaultArg maxSubmissionsPerPartition 5 - let dumpStreams struct (streams : Streams.Scheduling.StreamStates<_>, totalPurged) log = + let dumpStreams logStreamStates log = logExternalState |> Option.iter (fun f -> f log) - streams.Dump(log, totalPurged, Streams.Buffering.StreamState.eventsSize) - let handle (items : Streams.Scheduling.DispatchItem[]) ct - : Task<(TimeSpan * StreamName * bool * Choice)[]> = task { + logStreamStates Default.eventSize + let handle (items : Dispatch.Item[]) ct + : Task)[]> = task { let sw = Stopwatch.StartNew() let avgElapsed () = let tot = let e = sw.Elapsed in e.TotalMilliseconds @@ -486,25 +485,25 @@ type BatchesConsumer = [| for x in Seq.zip items results -> match x with | item, Choice1Of2 index' -> - let used : Streams.StreamSpan<_> = { item.span with events = item.span.events |> Seq.takeWhile (fun e -> e.Index <> index' ) |> Array.ofSeq } - let s = Streams.Buffering.StreamSpan.stats used - ae, item.stream, true, Choice1Of2 (index', (s, ())) + let used : StreamSpan<_> = item.span |> Seq.takeWhile (fun e -> e.Index <> index' ) |> Array.ofSeq + let metrics = StreamSpan.metrics Default.eventSize used + struct (ae, item.stream, true, Choice1Of2 struct (index', struct (metrics, ()))) | item, Choice2Of2 exn -> - let s = Streams.Buffering.StreamSpan.stats item.span - ae, item.stream, false, Choice2Of2 (s, exn) |] + let metrics = StreamSpan.metrics Default.jsonSize item.span + ae, item.stream, false, Choice2Of2 struct (metrics, exn) |] with e -> let ae = avgElapsed () return [| for x in items -> - let s = Streams.Buffering.StreamSpan.stats x.span - ae, x.stream, false, Choice2Of2 (s, e) |] } - let dispatcher = Streams.Scheduling.BatchedDispatcher(select, handle, stats, dumpStreams) - let streamsScheduler = Streams.Scheduling.StreamSchedulingEngine.Create( + let metrics = StreamSpan.metrics Default.jsonSize x.span + ae, x.stream, false, Choice2Of2 struct (metrics, e) |] } + let dispatcher = Scheduling.Dispatcher.BatchedDispatcher(select, handle, stats, dumpStreams) + let streamsScheduler = Scheduling.StreamSchedulingEngine.Create( dispatcher, maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay) - let mapConsumedMessagesToStreamsBatch onCompletion (x : Submission.SubmissionBatch) : Streams.Scheduling.StreamsBatch<_> = + let mapConsumedMessagesToStreamsBatch onCompletion (x : Submission.Batch) : Buffer.Batch<_> = let onCompletion () = x.onCompletion(); onCompletion() - Streams.Scheduling.StreamsBatch.Create(onCompletion, Seq.collect infoToStreamEvents x.messages) |> fst - let submitter = Streams.Projector.StreamsSubmitter.Create(log, maxSubmissionsPerPartition, mapConsumedMessagesToStreamsBatch, streamsScheduler.Submit, statsInterval) + Buffer.Batch.Create(onCompletion, Seq.collect infoToStreamEvents x.messages) |> ValueTuple.fst + let submitter = Projector.StreamsSubmitter.Create(log, maxSubmissionsPerPartition, mapConsumedMessagesToStreamsBatch, streamsScheduler.Submit, statsInterval) ConsumerPipeline.Start(log, config, consumeResultToInfo, submitter.Ingest, submitter.Pump, streamsScheduler.Pump, dispatcher.Pump, statsInterval) diff --git a/src/Propulsion.Kafka/ProducerSinks.fs b/src/Propulsion.Kafka/ProducerSinks.fs index cc0eeb89..bdb0c8b7 100644 --- a/src/Propulsion.Kafka/ProducerSinks.fs +++ b/src/Propulsion.Kafka/ProducerSinks.fs @@ -8,12 +8,12 @@ open System type ParallelProducerSink = static member Start(maxReadAhead, maxDop, render, producer : Producer, ?statsInterval) - : ProjectorPipeline<_> = + : Sink> = let statsInterval = defaultArg statsInterval (TimeSpan.FromMinutes 5.) let handle item = async { let key, value = render item do! producer.Produce (key, value) } - Parallel.ParallelProjector.Start(Log.Logger, maxReadAhead, maxDop, handle >> Async.Catch, statsInterval=statsInterval, logExternalStats=producer.DumpStats) + Parallel.ParallelSink.Start(Log.Logger, maxReadAhead, maxDop, handle >> Async.Catch, statsInterval=statsInterval, logExternalStats=producer.DumpStats) type StreamsProducerSink = @@ -35,28 +35,28 @@ type StreamsProducerSink = ?maxBatches, // Max inner cycles per loop. Default 128. ?maxCycles) - : ProjectorPipeline<_> = + : Default.Sink = let maxBytes = (defaultArg maxBytes (1024*1024 - (*fudge*)4096)) - let handle (stream : StreamName, span) = async { + let handle struct (stream : StreamName, span) = async { let! (maybeMsg, outcome : 'Outcome) = prepare (stream, span) match maybeMsg with | Some (key : string, message : string) -> match message.Length with - | x when x > maxBytes -> log.Warning("Message on {stream} had String.Length {length} Queue length {queueLen}", stream, x, span.events.Length) + | x when x > maxBytes -> log.Warning("Message on {stream} had String.Length {length} Queue length {queueLen}", stream, x, span.Length) | _ -> () do! producer.Produce(key, message) | None -> () - return SpanResult.AllProcessed, outcome + return struct (SpanResult.AllProcessed, outcome) } Sync.StreamsSync.Start ( log, maxReadAhead, maxConcurrentStreams, handle, - stats, statsInterval=statsInterval, - maxBytes=maxBytes, ?idleDelay=idleDelay,?purgeInterval=purgeInterval, - ?maxEvents=maxEvents, ?maxBatches=maxBatches, ?maxCycles=maxCycles, dumpExternalStats=producer.DumpStats) + stats, statsInterval, Default.jsonSize, Default.eventSize, + maxBytes = maxBytes, ?idleDelay = idleDelay,?purgeInterval = purgeInterval, + ?maxEvents=maxEvents, ?maxBatches = maxBatches, ?maxCycles = maxCycles, dumpExternalStats = producer.DumpStats) static member Start ( log : ILogger, maxReadAhead, maxConcurrentStreams, - prepare : StreamName * StreamSpan<_> -> Async, + prepare : struct (StreamName * StreamSpan<_>) -> Async, producer : Producer, stats : Sync.Stats, statsInterval, // Default 1 ms @@ -72,7 +72,7 @@ type StreamsProducerSink = ?maxBatches, // Max inner cycles per loop. Default 128. ?maxCycles) - : ProjectorPipeline<_> = + : Default.Sink = let prepare (stream, span) = async { let! k, v = prepare (stream, span) return Some (k, v), () @@ -80,5 +80,5 @@ type StreamsProducerSink = StreamsProducerSink.Start ( log, maxReadAhead, maxConcurrentStreams, prepare, producer, stats, statsInterval, - ?idleDelay=idleDelay, ?purgeInterval=purgeInterval, ?maxBytes=maxBytes, - ?maxEvents=maxEvents, ?maxBatches=maxBatches, ?maxCycles=maxCycles) + ?idleDelay = idleDelay, ?purgeInterval = purgeInterval, ?maxBytes = maxBytes, + ?maxEvents = maxEvents, ?maxBatches = maxBatches, ?maxCycles = maxCycles) diff --git a/src/Propulsion.Kafka/Producers.fs b/src/Propulsion.Kafka/Producers.fs index e1a3f73b..49bf6940 100644 --- a/src/Propulsion.Kafka/Producers.fs +++ b/src/Propulsion.Kafka/Producers.fs @@ -1,44 +1,42 @@ namespace Propulsion.Kafka open Confluent.Kafka // required for shimming -open FsKafka -open Propulsion open Serilog open System /// Methods are intended to be used safely from multiple threads concurrently type Producer ( log : ILogger, clientId, bootstrapServers, acks, topic, - /// Linger period (larger values improve compression value and throughput, lower values improve best case latency). Default 5ms (librdkafka < 1.5 default: 0.5ms, librdkafka >= 1.5 default: 5ms) + // Linger period (larger values improve compression value and throughput, lower values improve best case latency). Default 5ms (librdkafka < 1.5 default: 0.5ms, librdkafka >= 1.5 default: 5ms) ?linger, - /// Default: LZ4 + // Default: LZ4 ?compression, // Deprecated; there's a good chance this will be removed ?degreeOfParallelism, - /// Miscellaneous configuration parameters to be passed to the underlying Confluent.Kafka producer configuration. Same as constructor argument for Confluent.Kafka >=1.2. + // Miscellaneous configuration parameters to be passed to the underlying Confluent.Kafka producer configuration. Same as constructor argument for Confluent.Kafka >=1.2. ?config, - /// Miscellaneous configuration parameters to be passed to the underlying Confluent.Kafka producer configuration. + // Miscellaneous configuration parameters to be passed to the underlying Confluent.Kafka producer configuration. ?custom, - /// Postprocesses the ProducerConfig after the rest of the rules have been applied + // Postprocesses the ProducerConfig after the rest of the rules have been applied ?customize) = let batching = let linger = defaultArg linger (TimeSpan.FromMilliseconds 5.) FsKafka.Batching.Linger linger let compression = defaultArg compression CompressionType.Lz4 - let cfg = KafkaProducerConfig.Create(clientId, bootstrapServers, acks, batching, compression, ?config = config, ?custom = custom, ?customize=customize) + let cfg = FsKafka.KafkaProducerConfig.Create(clientId, bootstrapServers, acks, batching, compression, ?config = config, ?custom = custom, ?customize=customize) // NB having multiple producers has yet to be proved necessary at this point // - the theory is that because each producer gets a dedicated rdkafka context, compression thread and set of sockets, better throughput can be attained // - we should consider removing the degreeOfParallelism argument and this associated logic unless we actually get to the point of leaning on this - let producers = Array.init (defaultArg degreeOfParallelism 1) (fun _i -> KafkaProducer.Create(log, cfg, topic)) - let produceStats = Streams.Internal.ConcurrentLatencyStats(sprintf "producers(%d)" producers.Length) + let producers = Array.init (defaultArg degreeOfParallelism 1) (fun _i -> FsKafka.KafkaProducer.Create(log, cfg, topic)) + let produceStats = Propulsion.Internal.Stats.ConcurrentLatencyStats(sprintf "producers(%d)" producers.Length) let mutable robin = 0 member _.DumpStats log = produceStats.Dump log /// Execute a producer operation, including recording of the latency statistics for the operation /// NOTE: the `execute` function is expected to throw in the event of a failure to produce (this is the standard semantic for all Confluent.Kafka ProduceAsync APIs) - member _.Produce(execute : KafkaProducer -> Async<'r>) : Async<'r> = async { - let producer = producers.[System.Threading.Interlocked.Increment(&robin) % producers.Length] + member _.Produce(execute : FsKafka.KafkaProducer -> Async<'r>) : Async<'r> = async { + let producer = producers[System.Threading.Interlocked.Increment(&robin) % producers.Length] let sw = System.Diagnostics.Stopwatch.StartNew() let! res = execute producer produceStats.Record sw.Elapsed diff --git a/src/Propulsion.MemoryStore/MemoryStoreSource.fs b/src/Propulsion.MemoryStore/MemoryStoreSource.fs index 05a220b1..dec31352 100644 --- a/src/Propulsion.MemoryStore/MemoryStoreSource.fs +++ b/src/Propulsion.MemoryStore/MemoryStoreSource.fs @@ -12,75 +12,63 @@ module MemoryStoreLogger = let items = seq { for kv in xs do yield sprintf "{\"%s\": %s}" kv.Key kv.Value } log.ForContext(name, sprintf "[%s]" (String.concat ",\n\r" items)) - let private propEventJsonUtf8 name (events : Propulsion.Streams.StreamEvent> array) (log : Serilog.ILogger) = + let private propEventJsonUtf8 name (events : FsCodec.ITimelineEvent> array) (log : Serilog.ILogger) = log |> propEvents name (seq { - for { event = e } in events do + for e in events do let d = e.Data if not d.IsEmpty then System.Collections.Generic.KeyValuePair<_,_>(e.EventType, System.Text.Encoding.UTF8.GetString d.Span) }) - let renderSubmit (log : Serilog.ILogger) (epoch, stream, events : Propulsion.Streams.StreamEvent<'F> array) = + let renderSubmit (log : Serilog.ILogger) struct (epoch, stream, events : FsCodec.ITimelineEvent<'F> array) = if log.IsEnabled Serilog.Events.LogEventLevel.Verbose then let log = if (not << log.IsEnabled) Serilog.Events.LogEventLevel.Debug then log elif typedefof<'F> <> typeof> then log else log |> propEventJsonUtf8 "Json" (unbox events) - let types = seq { for x in events -> x.event.EventType } + let types = events |> Seq.map (fun e -> e.EventType) log.ForContext("types", types).Debug("Submit #{epoch} {stream}x{count}", epoch, stream, events.Length) elif log.IsEnabled Serilog.Events.LogEventLevel.Debug then - let types = seq { for x in events -> x.event.EventType } |> Seq.truncate 5 + let types = seq { for e in events -> e.EventType } |> Seq.truncate 5 log.Debug("Submit #{epoch} {stream}x{count} {types}", epoch, stream, events.Length, types) let renderCompleted (log : Serilog.ILogger) (epoch, stream) = log.Verbose("Done! #{epoch} {stream}", epoch, stream) - let toStreamEvents stream (events : FsCodec.ITimelineEvent<'F> seq) = - [| for x in events -> { stream = stream; event = x } : Propulsion.Streams.StreamEvent<'F> |] - /// Wires specified Observable source (e.g. VolatileStore.Committed) to the Logger let subscribe log source = let mutable epoch = -1L let aux (stream, events) = - let events = toStreamEvents stream events let epoch = Interlocked.Increment &epoch renderSubmit log (epoch, stream, events) if log.IsEnabled Serilog.Events.LogEventLevel.Debug then Observable.subscribe aux source else { new IDisposable with member _.Dispose() = () } -module TimelineEvent = - - let mapEncoded = - let mapBodyToBytes = (fun (x : ReadOnlyMemory) -> x.ToArray()) - FsCodec.Core.TimelineEvent.Map (FsCodec.Deflate.EncodedToUtf8 >> mapBodyToBytes) // TODO replace with FsCodec.Deflate.EncodedToByteArray - /// Coordinates forwarding of a VolatileStore's Committed events to a supplied Sink /// Supports awaiting the (asynchronous) handling by the Sink of all Committed events from a given point in time -type MemoryStoreSource<'F, 'B>(log, store : Equinox.MemoryStore.VolatileStore<'F>, streamFilter, - mapTimelineEvent : FsCodec.ITimelineEvent<'F> -> FsCodec.ITimelineEvent, - sink : ProjectorPipeline seq, 'B>>) = - let ingester = sink.StartIngester(log, 0) +type MemoryStoreSource<'F>(log, store : Equinox.MemoryStore.VolatileStore<'F>, streamFilter, + mapTimelineEvent : FsCodec.ITimelineEvent<'F> -> FsCodec.ITimelineEvent, + sink : Propulsion.Streams.Default.Sink) = + let ingester : Ingestion.Ingester<_> = sink.StartIngester(log, 0) // epoch index of most recently prepared and completed submissions let mutable prepared, completed = -1L, -1L let enqueueSubmission, awaitSubmissions, tryDequeueSubmission = - let c = Channel.unboundedSr - Channel.write c, Channel.awaitRead c, c.Reader.TryRead + let c = Channel.unboundedSr seq>> in let r, w = c.Reader, c.Writer + Channel.write w, Channel.awaitRead r, r.TryRead - let handleCommitted (stream, events : FsCodec.ITimelineEvent<_> seq) = + let handleStoreCommitted (stream, events : FsCodec.ITimelineEvent<_> []) = let epoch = Interlocked.Increment &prepared - let events = MemoryStoreLogger.toStreamEvents stream events MemoryStoreLogger.renderSubmit log (epoch, stream, events) let markCompleted () = MemoryStoreLogger.renderCompleted log (epoch, stream) Volatile.Write(&completed, epoch) // We don't have anything Async to do, so we pass a null checkpointing function - let checkpoint = async { () } - enqueueSubmission (epoch, checkpoint, events, markCompleted) + enqueueSubmission { epoch = epoch; checkpoint = async.Zero (); items = events |> Array.map (fun e -> stream, e); onCompletion = markCompleted } let storeCommitsSubscription = let mapBody (s, e) = s, e |> Array.map mapTimelineEvent store.Committed |> Observable.filter (fst >> streamFilter) - |> Observable.subscribe (mapBody >> handleCommitted) + |> Observable.subscribe (mapBody >> handleStoreCommitted) member private _.Pump(ct : CancellationToken) = task { while not ct.IsCancellationRequested do @@ -88,7 +76,7 @@ type MemoryStoreSource<'F, 'B>(log, store : Equinox.MemoryStore.VolatileStore<'F while more do match tryDequeueSubmission () with | false, _ -> more <- false - | true, (epoch, checkpoint, events, markCompleted) -> do! ingester.Submit(epoch, checkpoint, events, markCompleted) |> Async.Ignore + | true, batch -> do! ingester.Ingest(batch) |> Async.Ignore do! awaitSubmissions ct :> Task } member x.Start() = @@ -111,7 +99,7 @@ type MemoryStoreSource<'F, 'B>(log, store : Equinox.MemoryStore.VolatileStore<'F storeCommitsSubscription.Dispose() } new Pipeline(Task.Run(supervise), stop) - /// Waits until all Submitted batches have been successfully processed via the Sink + /// Waits until all Ingested batches have been successfully processed via the Sink /// NOTE this relies on specific guarantees the MemoryStore's Committed event affords us /// 1. a Decider's Transact will not return until such time as the Committed events have been handled /// (i.e., we have prepared the batch for submission) @@ -132,30 +120,30 @@ type MemoryStoreSource<'F, 'B>(log, store : Equinox.MemoryStore.VolatileStore<'F let delayMs = let delay = defaultArg delay TimeSpan.FromMilliseconds 1. int delay.TotalMilliseconds - let maybeLog = - let logInterval = defaultArg logInterval (TimeSpan.FromSeconds 10.) - let logDue = intervalCheck logInterval - fun () -> - if logDue () then - let completed = match Volatile.Read &completed with -1L -> Nullable() | x -> Nullable x - if includeSubsequent then - log.Information("Awaiting Completion of all Batches. Starting Epoch {epoch} Current Epoch {current} Completed Epoch {completed}", - startingEpoch, Volatile.Read &prepared, completed) - else log.Information("Awaiting Completion of Starting Epoch {startingEpoch} Completed Epoch {completed}", startingEpoch, completed) + let logInterval = IntervalTimer(defaultArg logInterval (TimeSpan.FromSeconds 10.)) + let logStatus () = + let completed = match Volatile.Read &completed with -1L -> Nullable() | x -> Nullable x + if includeSubsequent then + log.Information("Awaiting Completion of all Batches. Starting Epoch {epoch} Current Epoch {current} Completed Epoch {completed}", + startingEpoch, Volatile.Read &prepared, completed) + else log.Information("Awaiting Completion of Starting Epoch {startingEpoch} Completed Epoch {completed}", startingEpoch, completed) let isComplete () = let currentCompleted = Volatile.Read &completed Volatile.Read &prepared = currentCompleted // All submitted work (including follow-on work), completed || (currentCompleted >= startingEpoch && not includeSubsequent) // At or beyond starting point while not (isComplete ()) && not sink.IsCompleted do - maybeLog () + if logInterval.IfDueRestart() then logStatus () do! Async.Sleep delayMs // If the sink Faulted, let the awaiter observe the associated Exception that triggered the shutdown if sink.IsCompleted && not sink.RanToCompletion then return! sink.AwaitShutdown() } +module TimelineEvent = + + let mapEncoded = FsCodec.Core.TimelineEvent.Map FsCodec.Deflate.EncodedToUtf8 + /// Coordinates forwarding of a VolatileStore's Committed events to a supplied Sink /// Supports awaiting the (asynchronous) handling by the Sink of all Committed events from a given point in time -type MemoryStoreSource<'B>(log, store : Equinox.MemoryStore.VolatileStore)>, filter, - sink : ProjectorPipeline seq, 'B>>) = - inherit MemoryStoreSource), 'B>(log, store, filter, TimelineEvent.mapEncoded, sink) +type MemoryStoreSource(log, store : Equinox.MemoryStore.VolatileStore)>, filter, sink) = + inherit MemoryStoreSource)>(log, store, filter, TimelineEvent.mapEncoded, sink) diff --git a/src/Propulsion.SqlStreamStore/SqlStreamStoreSource.fs b/src/Propulsion.SqlStreamStore/SqlStreamStoreSource.fs index 784340b8..cba0e893 100644 --- a/src/Propulsion.SqlStreamStore/SqlStreamStoreSource.fs +++ b/src/Propulsion.SqlStreamStore/SqlStreamStoreSource.fs @@ -1,25 +1,22 @@ namespace Propulsion.SqlStreamStore -type StreamEvent = Propulsion.Streams.StreamEvent - module private Impl = open Propulsion.Infrastructure // AwaitTaskCorrect - let toStreamEvent (dataJson : string) (msg: SqlStreamStore.Streams.StreamMessage) : StreamEvent = - let inline len0ToNull (x : _[]) = match x with null -> null | x when x.Length = 0 -> null | x -> x + let toStreamEvent (dataJson : string) (msg: SqlStreamStore.Streams.StreamMessage) : Propulsion.Streams.Default.StreamEvent = let e = FsCodec.Core.TimelineEvent.Create ( int64 msg.StreamVersion, msg.Type, - (match dataJson with null -> null | x -> x |> System.Text.Encoding.UTF8.GetBytes |> len0ToNull), - msg.JsonMetadata |> System.Text.Encoding.UTF8.GetBytes |> len0ToNull, + (match dataJson with null -> System.ReadOnlyMemory.Empty | x -> x |> System.Text.Encoding.UTF8.GetBytes |> System.ReadOnlyMemory), + msg.JsonMetadata |> System.Text.Encoding.UTF8.GetBytes |> System.ReadOnlyMemory, msg.MessageId, timestamp = System.DateTimeOffset(msg.CreatedUtc)) - { stream = Propulsion.Streams.StreamName.internalParseSafe msg.StreamId; event = e } + Propulsion.Streams.StreamName.internalParseSafe msg.StreamId, e let readWithDataAsStreamEvent (msg : SqlStreamStore.Streams.StreamMessage) = async { let! json = msg.GetJsonData() |> Async.AwaitTaskCorrect return toStreamEvent json msg } - let readBatch hydrateBodies batchSize (store : SqlStreamStore.IStreamStore) pos : Async> = async { + let readBatch hydrateBodies batchSize (store : SqlStreamStore.IStreamStore) pos : Async> = async { let! ct = Async.CancellationToken let! page = store.ReadAllForwards(Propulsion.Feed.Position.toInt64 pos, batchSize, hydrateBodies, ct) |> Async.AwaitTaskCorrect let! items = @@ -30,13 +27,12 @@ module private Impl = type SqlStreamStoreSource ( log : Serilog.ILogger, statsInterval, store : SqlStreamStore.IStreamStore, batchSize, tailSleepInterval, - checkpoints : Propulsion.Feed.IFeedCheckpointStore, - sink : Propulsion.ProjectorPipeline, Propulsion.Submission.SubmissionBatch>>, + checkpoints : Propulsion.Feed.IFeedCheckpointStore, sink : Propulsion.Streams.Default.Sink, // If the Handler does not require the bodies of the events, we can save significant Read Capacity by not having to load them. Default: false ?hydrateBodies, // TODO borrow impl of determining tail from Propulsion.EventStoreDb // ?fromTail, ?sourceId) = - inherit Propulsion.Feed.Internal.AllFeedSource + inherit Propulsion.Feed.Core.AllFeedSource ( log, statsInterval, defaultArg sourceId FeedSourceId.wellKnownId, tailSleepInterval, Impl.readBatch (hydrateBodies = Some true) batchSize store, checkpoints, sink) diff --git a/src/Propulsion/Feed.fs b/src/Propulsion/Feed.fs index 8e7b6907..a19a5f1c 100644 --- a/src/Propulsion/Feed.fs +++ b/src/Propulsion/Feed.fs @@ -26,5 +26,5 @@ module Position = type IFeedCheckpointStore = /// Determines the starting position, and checkpointing frequency for a given tranche - abstract member Start: source: SourceId * tranche : TrancheId * ?establishOrigin : Async -> Async + abstract member Start: source: SourceId * tranche : TrancheId * ?establishOrigin : Async -> Async abstract member Commit: source: SourceId * tranche : TrancheId * pos: Position -> Async diff --git a/src/Propulsion/Infrastructure.fs b/src/Propulsion/Infrastructure.fs index 91db2bf6..fd85dbcd 100755 --- a/src/Propulsion/Infrastructure.fs +++ b/src/Propulsion/Infrastructure.fs @@ -1,6 +1,11 @@ [] module internal Propulsion.Infrastructure +module ValueTuple = + + let inline fst struct (f, _s) = f + let inline snd struct (_f, s) = s + open System open System.Threading.Tasks diff --git a/src/Propulsion/Ingestion.fs b/src/Propulsion/Ingestion.fs index 4e32550d..95abbbd0 100755 --- a/src/Propulsion/Ingestion.fs +++ b/src/Propulsion/Ingestion.fs @@ -17,7 +17,7 @@ type ProgressWriter<'Res when 'Res : equality>(?period) = let commitIfDirty ct = task { match Volatile.Read &validatedPos with - | Some (v,f) when Volatile.Read(&committedEpoch) <> Some v -> + | Some (v, f) when Volatile.Read(&committedEpoch) <> Some v -> try do! Async.StartAsTask(f, cancellationToken = ct) Volatile.Write(&committedEpoch, Some v) result.Trigger (Choice1Of2 v) @@ -26,8 +26,8 @@ type ProgressWriter<'Res when 'Res : equality>(?period) = [] member _.Result = result.Publish - member _.Post(version,f) = - Volatile.Write(&validatedPos,Some (version, f)) + member _.Post(version, f) = + Volatile.Write(&validatedPos, Some (version, f)) member _.CommittedEpoch = Volatile.Read(&committedEpoch) @@ -45,13 +45,16 @@ type private InternalMessage = /// Internal message for stats purposes | Added of streams : int * events : int +[] +type Batch<'Items> = { epoch : int64; items : 'Items; onCompletion : unit -> unit; checkpoint : Async } + type private Stats(log : ILogger, partitionId, statsInterval : TimeSpan) = let mutable validatedEpoch, committedEpoch : int64 option * int64 option = None, None let mutable commitFails, commits = 0, 0 let mutable cycles, batchesPended, streamsPended, eventsPended = 0, 0, 0, 0 - let statsInterval = timeRemaining statsInterval + member val Interval = IntervalTimer statsInterval - let dumpStats (activeReads, maxReads) = + member _.DumpStats(activeReads, maxReads) = log.Information("Ingester {partitionId} Ahead {activeReads}/{maxReads} @ {validated} (committed: {committed}, {commits} commits) Ingested {batches} ({streams:n0}s {events:n0}e) Cycles {cycles}", partitionId, activeReads, maxReads, Option.toNullable validatedEpoch, Option.toNullable committedEpoch, commits, batchesPended, streamsPended, eventsPended, cycles) cycles <- 0; batchesPended <- 0; streamsPended <- 0; eventsPended <- 0 @@ -73,47 +76,44 @@ type private Stats(log : ILogger, partitionId, statsInterval : TimeSpan) = streamsPended <- streamsPended + streams eventsPended <- eventsPended + events - member _.TryDump(readState) = + member x.RecordCycle() = cycles <- cycles + 1 - let struct (due, remaining) = statsInterval () - if due then dumpStats readState - remaining /// Buffers items read from a range, unpacking them out of band from the reading so that can overlap /// On completion of the unpacking, they get submitted onward to the Submitter which will buffer them for us -type Ingester<'Items, 'Batch> private - ( stats : Stats, maxRead, - makeBatch : (unit -> unit) -> 'Items -> 'Batch * (int * int), - submit : 'Batch -> unit, +type Ingester<'Items> private + ( stats : Stats, maxReadAhead, + // forwards a set of items and the completion callback, yielding streams count * event count + submitBatch : 'Items * (unit -> unit) -> struct (int * int), cts : CancellationTokenSource) = - let maxRead = Sem maxRead + let maxRead = Sem maxReadAhead let awaitIncoming, applyIncoming, enqueueIncoming = - let c = Channel.unboundedSwSr - Channel.awaitRead c, Channel.apply c, Channel.write c + let c = Channel.unboundedSwSr> in let r, w = c.Reader, c.Writer + Channel.awaitRead r, Channel.apply r, Channel.write w let awaitMessage, applyMessages, enqueueMessage = - let c = Channel.unboundedSr - Channel.awaitRead c, Channel.apply c, Channel.write c + let c = Channel.unboundedSr in let r, w = c.Reader, c.Writer + Channel.awaitRead r, Channel.apply r, Channel.write w let progressWriter = ProgressWriter<_>() - let handleIncoming (epoch, checkpoint, items, outerMarkCompleted) = + let handleIncoming (batch : Batch<'Items>) = let markCompleted () = maxRead.Release() - outerMarkCompleted |> Option.iter (fun f -> f ()) // we guarantee this happens before checkpoint can be called - enqueueMessage <| Validated epoch - progressWriter.Post(epoch, checkpoint) - let batch, (streamCount, itemCount) = makeBatch markCompleted items - submit batch - enqueueMessage <| Added (streamCount,itemCount) + batch.onCompletion () // we guarantee this happens before checkpoint can be called + enqueueMessage <| Validated batch.epoch + progressWriter.Post(batch.epoch, batch.checkpoint) + let struct (streamCount, itemCount) = submitBatch (batch.items, markCompleted) + enqueueMessage <| Added (streamCount, itemCount) member private _.Pump(ct) = task { use _ = progressWriter.Result.Subscribe(ProgressResult >> enqueueMessage) Task.start (fun () -> progressWriter.Pump ct) while not ct.IsCancellationRequested do while applyIncoming handleIncoming || applyMessages stats.Handle do () - let nextStatsIntervalMs = stats.TryDump(maxRead.State) - do! Task.WhenAny(awaitIncoming ct, awaitMessage ct, Task.Delay(int nextStatsIntervalMs)) :> Task } + stats.RecordCycle() + if stats.Interval.IfDueRestart() then let struct (active, max) = maxRead.State in stats.DumpStats(active, max) + do! Task.WhenAny(awaitIncoming ct, awaitMessage ct, Task.Delay(stats.Interval.RemainingMs)) :> Task } // arguably the impl should be submitting while unpacking but // - maintaining consistency between incoming order and submit order is required // - in general maxRead will be double maxSubmit so this will only be relevant in catchup situations @@ -121,10 +121,10 @@ type Ingester<'Items, 'Batch> private /// Starts an independent Task that handles /// a) `unpack`ing of `incoming` items /// b) `submit`ting them onward (assuming there is capacity within the `readLimit`) - static member Start<'Item>(log, partitionId, maxRead, makeBatch, submit, statsInterval) = + static member Start<'Items>(log, partitionId, maxRead, submitBatch, statsInterval) = let cts = new CancellationTokenSource() let stats = Stats(log, partitionId, statsInterval) - let instance = Ingester<_, _>(stats, maxRead, makeBatch, submit, cts) + let instance = Ingester<'Items>(stats, maxRead, submitBatch, cts) let pump () = task { try do! instance.Pump cts.Token finally log.Information("Exiting {name}", "ingester") } @@ -133,11 +133,9 @@ type Ingester<'Items, 'Batch> private /// Submits a batch as read for unpacking and submission; will only return after the in-flight reads drops below the limit /// Returns (reads in flight, maximum reads in flight) - /// markCompleted will (if supplied) be triggered when the supplied batch has completed processing - /// (but prior to the calling of the checkpoint method, which will take place asynchronously) - member _.Submit(epoch, checkpoint, items, ?markCompleted) = async { - // If we've read it, feed it into the queue for unpacking - enqueueIncoming (epoch, checkpoint, items, markCompleted) + member _.Ingest(batch : Batch<'Items>) = async { + // It's been read... feed it into the queue for unpacking + enqueueIncoming batch // ... but we might hold off on yielding if we're at capacity do! maxRead.Await(cts.Token) return maxRead.State } diff --git a/src/Propulsion/Internal.fs b/src/Propulsion/Internal.fs index 6158adb5..f63b1b8f 100644 --- a/src/Propulsion/Internal.fs +++ b/src/Propulsion/Internal.fs @@ -1,23 +1,27 @@ module Propulsion.Internal open System -open System.Diagnostics -open System.Threading -open System.Threading.Tasks -/// Maintains a Stopwatch such that invoking will yield true at intervals defined by `period` -let intervalCheck (period : TimeSpan) = - let timer, max = Stopwatch.StartNew(), int64 period.TotalMilliseconds - fun () -> - let due = timer.ElapsedMilliseconds > max - if due then timer.Restart() +/// Manages a time cycle defined by `period`. Can be explicitly Trigger()ed prematurely +type IntervalTimer(period : TimeSpan) = + + let timer, periodMs = System.Diagnostics.Stopwatch.StartNew(), int64 period.TotalMilliseconds + let mutable force = false + + member val Period = period + member _.RemainingMs = periodMs - timer.ElapsedMilliseconds |> int |> max 0 + + member _.Trigger() = force <- true + + // NOTE asking the question is destructive - the timer is reset as a side effect + member _.IfDueRestart() = + let due = force || timer.ElapsedMilliseconds > periodMs + if due then timer.Restart(); force <- false due -let timeRemaining (period : TimeSpan) = - let timer, max = Stopwatch.StartNew(), int64 period.TotalMilliseconds - fun () -> - match max - timer.ElapsedMilliseconds |> int with - | rem when rem <= 0 -> timer.Restart(); struct (true, max) - | rem -> (false, rem) + +let inline mb x = float x / 1024. / 1024. + +type System.Diagnostics.Stopwatch with member x.ElapsedSeconds = float x.ElapsedMilliseconds / 1000. module Channel = @@ -26,24 +30,30 @@ module Channel = let unboundedSr<'t> = Channel.CreateUnbounded<'t>(UnboundedChannelOptions(SingleReader = true)) let unboundedSw<'t> = Channel.CreateUnbounded<'t>(UnboundedChannelOptions(SingleWriter = true)) let unboundedSwSr<'t> = Channel.CreateUnbounded<'t>(UnboundedChannelOptions(SingleWriter = true, SingleReader = true)) - let write (c : Channel<_>) = c.Writer.TryWrite >> ignore - let awaitRead (c : Channel<_>) ct = let vt = c.Reader.WaitToReadAsync(ct) in vt.AsTask() - let tryRead (c : Channel<_>) () = match c.Reader.TryRead() with true, m -> ValueSome m | false, _ -> ValueNone - let apply (c : Channel<_>) f = - let mutable worked, msg = false, Unchecked.defaultof<_> - while c.Reader.TryRead(&msg) do + let write (w : ChannelWriter<_>) = w.TryWrite >> ignore + let inline awaitRead (r : ChannelReader<_>) ct = let vt = r.WaitToReadAsync(ct) in vt.AsTask() + let inline tryRead (r : ChannelReader<_>) () = + let mutable msg = Unchecked.defaultof<_> + if r.TryRead(&msg) then ValueSome msg else ValueNone + let inline apply (r : ChannelReader<_>) f = + let mutable worked = false + let mutable msg = Unchecked.defaultof<_> + while r.TryRead(&msg) do worked <- true f msg worked +open System.Threading +open System.Threading.Tasks + module Task = - let start create = Task.Run(Func> create) |> ignore + let inline start create = Task.Run(Func> create) |> ignore type Sem(max) = let inner = new SemaphoreSlim(max) member _.HasCapacity = inner.CurrentCount <> 0 - member _.State = max-inner.CurrentCount,max + member _.State = struct(max - inner.CurrentCount, max) member _.Await(ct : CancellationToken) = inner.WaitAsync(ct) |> Async.AwaitTaskCorrect member x.AwaitButRelease() = // see https://stackoverflow.com/questions/31621644/task-whenany-and-semaphoreslim-class/73197290?noredirect=1#comment129334330_73197290 inner.WaitAsync().ContinueWith((fun _ -> x.Release()), CancellationToken.None, TaskContinuationOptions.ExecuteSynchronously, TaskScheduler.Default) @@ -65,3 +75,74 @@ type Async with tcs.TrySetException(TaskCanceledException "Execution cancelled via Ctrl-C/Break; exiting...") |> ignore) return! Async.AwaitTaskCorrect tcs.Task } +module Stats = + + open System.Collections.Generic + + let toValueTuple (x : KeyValuePair<_, _>) = struct (x.Key, x.Value) + let statsDescending (xs : Dictionary<_, _>) = xs |> Seq.map toValueTuple |> Seq.sortByDescending ValueTuple.snd + + /// Gathers stats relating to how many items of a given category have been observed + type CatStats() = + let cats = Dictionary() + + member _.Ingest(cat, ?weight) = + let weight = defaultArg weight 1L + match cats.TryGetValue cat with + | true, catCount -> cats[cat] <- catCount + weight + | false, _ -> cats[cat] <- weight + + member _.Count = cats.Count + member _.Any = (not << Seq.isEmpty) cats + member _.Clear() = cats.Clear() + member _.StatsDescending = statsDescending cats + + type private Data = + { min : TimeSpan + p50 : TimeSpan + p95 : TimeSpan + p99 : TimeSpan + max : TimeSpan + avg : TimeSpan + stddev : TimeSpan option } + + open MathNet.Numerics.Statistics + let private dumpStats (kind : string) (xs : TimeSpan seq) (log : Serilog.ILogger) = + let sortedLatencies = xs |> Seq.map (fun r -> r.TotalSeconds) |> Seq.sort |> Seq.toArray + + let pc p = SortedArrayStatistics.Percentile(sortedLatencies, p) |> TimeSpan.FromSeconds + let l = { + avg = ArrayStatistics.Mean sortedLatencies |> TimeSpan.FromSeconds + stddev = + let stdDev = ArrayStatistics.StandardDeviation sortedLatencies + // stddev of singletons is NaN + if Double.IsNaN stdDev then None else TimeSpan.FromSeconds stdDev |> Some + + min = SortedArrayStatistics.Minimum sortedLatencies |> TimeSpan.FromSeconds + max = SortedArrayStatistics.Maximum sortedLatencies |> TimeSpan.FromSeconds + p50 = pc 50 + p95 = pc 95 + p99 = pc 99 } + let inline sec (t : TimeSpan) = t.TotalSeconds + let stdDev = match l.stddev with None -> Double.NaN | Some d -> sec d + log.Information(" {kind} {count} : max={max:n3}s p99={p99:n3}s p95={p95:n3}s p50={p50:n3}s min={min:n3}s avg={avg:n3}s stddev={stddev:n3}s", + kind, sortedLatencies.Length, sec l.max, sec l.p99, sec l.p95, sec l.p50, sec l.min, sec l.avg, stdDev) + + /// Operations on an instance are safe cross-thread + type ConcurrentLatencyStats(kind) = + let buffer = System.Collections.Concurrent.ConcurrentStack() + member _.Record value = buffer.Push value + member _.Dump(log : Serilog.ILogger) = + if not buffer.IsEmpty then + dumpStats kind buffer log + buffer.Clear() // yes, there is a race + + /// Should only be used on one thread + type LatencyStats(kind) = + let buffer = ResizeArray() + member _.Record value = buffer.Add value + member _.Dump(log : Serilog.ILogger) = + if buffer.Count <> 0 then + dumpStats kind buffer log + buffer.Clear() + diff --git a/src/Propulsion/Parallel.fs b/src/Propulsion/Parallel.fs index 296cf2e2..5b7f4ab4 100755 --- a/src/Propulsion/Parallel.fs +++ b/src/Propulsion/Parallel.fs @@ -7,16 +7,9 @@ open Serilog open System open System.Collections.Concurrent open System.Collections.Generic -open System.Diagnostics open System.Threading open System.Threading.Tasks -[] -module private Helpers = - - /// Can't figure out a cleaner way to shim it :( - let tryPeek (x : Queue<_>) = if x.Count = 0 then None else Some (x.Peek()) - /// Deals with dispatch and result handling, triggering completion callbacks as batches reach completed state module Scheduling = @@ -24,7 +17,9 @@ module Scheduling = /// Semaphore is allocated on queueing, deallocated on completion of the processing type Dispatcher(maxDop) = // Using a Queue as a) the ordering is more correct, favoring more important work b) we are adding from many threads so no value in ConcurrentBag's thread-affinity - let tryWrite, wait, apply = let c = Channel.unboundedSwSr<_> in c.Writer.TryWrite, Channel.awaitRead c, Channel.apply c + let tryWrite, wait, apply = + let c = Channel.unboundedSwSr<_> in let r, w = c.Reader, c.Writer + w.TryWrite, Channel.awaitRead r, Channel.apply r let dop = Sem maxDop let wrap computation = async { @@ -68,7 +63,7 @@ module Scheduling = let x = { elapsedMs = 0L; remaining = batch.messages.Length; faults = ConcurrentStack(); batch = batch } x, seq { for item in batch.messages -> async { - let sw = Stopwatch.StartNew() + let sw = System.Diagnostics.Stopwatch.StartNew() try let! res = handle item let elapsed = sw.Elapsed match res with @@ -87,7 +82,7 @@ module Scheduling = /// - replenishing the Dispatcher /// - determining when WipBatches attain terminal state in order to triggering completion callbacks at the earliest possible opportunity /// - triggering abend of the processing should any dispatched tasks start to fault - type PartitionedSchedulingEngine<'S, 'M when 'S : equality>(log : ILogger, handle, tryDispatch : (Async) -> bool, statsInterval, ?logExternalStats) = + type PartitionedSchedulingEngine<'S, 'M when 'S : equality>(log : ILogger, handle, tryDispatch : Async -> bool, statsInterval, ?logExternalStats) = // Submitters dictate batch commencement order by supply batches in a fair order; should never be empty if there is work in the system let incoming = ConcurrentQueue>() // Prepared work items ready to feed to Dispatcher (only created on demand in order to ensure we maximize overall progress and fairness) @@ -115,10 +110,10 @@ module Scheduling = logExternalStats |> Option.iter (fun f -> f log) // doing this in here allows stats intervals to be aligned with that of the scheduler engine let maybeLogStats : unit -> bool = - let due = intervalCheck statsInterval + let timer = IntervalTimer statsInterval fun () -> cycles <- cycles + 1 - if due () then dumpStats (); true else false + if timer.IfDueRestart() then dumpStats (); true else false /// Inspects the oldest in-flight batch per partition to determine if it's reached a terminal state; if it has, remove and trigger completion callback let drainCompleted abend = @@ -126,12 +121,12 @@ module Scheduling = while more do more <- false for queue in active.Values do - match tryPeek queue with - | None // empty - | Some Busy -> () // still working - | Some (Faulted exns) -> // outer layers will react to this by tearing us down + match queue.TryPeek() with + | false, _ // empty + | true, Busy -> () // still working + | true, Faulted exns -> // outer layers will react to this by tearing us down abend (AggregateException(exns)) - | Some (Completed batchProcessingDuration) -> // call completion function asap + | true, Completed batchProcessingDuration -> // call completion function asap let partitionId, markCompleted, itemCount = let { batch = { source = p; onCompletion = f; messages = msgs } } = queue.Dequeue() p, f, msgs.LongLength @@ -153,7 +148,7 @@ module Scheduling = let wipBatch, runners = WipBatch.Create(batch, handle) runners |> Seq.iter waiting.Enqueue match active.TryGetValue pid with - | false, _ -> let q = Queue(1024) in active.[pid] <- q; q.Enqueue wipBatch + | false, _ -> let q = Queue(1024) in active[pid] <- q; q.Enqueue wipBatch | true, q -> q.Enqueue wipBatch true @@ -161,10 +156,10 @@ module Scheduling = let reprovisionDispatcher () = let mutable more, worked = true, false while more do - match tryPeek waiting with - | None -> // Crack open a new batch if we don't have anything ready + match waiting.TryPeek() with + | false, _ -> // Crack open a new batch if we don't have anything ready more <- tryPrepareNext () - | Some pending -> // Dispatch until we reach capacity if we do have something + | true, pending -> // Dispatch until we reach capacity if we do have something if tryDispatch pending then worked <- true waiting.Dequeue() |> ignore @@ -172,7 +167,7 @@ module Scheduling = more <- false worked - /// Main pumping loop; `abend` is a callback triggered by a faulted task which the outer controler can use to shut down the processing + /// Main pumping loop; `abend` is a callback triggered by a faulted task which the outer controller can use to shut down the processing member _.Pump abend (ct : CancellationToken) = task { while not ct.IsCancellationRequested do let hadResults = drainCompleted abend @@ -181,20 +176,21 @@ module Scheduling = if not hadResults && not queuedWork && not loggedStats then Thread.Sleep 1 } // not Async.Sleep, we like this context and/or cache state if nobody else needs it - /// Feeds a batch of work into the queue; the caller is expected to ensure sumbissions are timely to avoid starvation, but throttled to ensure fair ordering - member __.Submit(batches : Batch<'S, 'M>) = + /// Feeds a batch of work into the queue; the caller is expected to ensure submissions are timely to avoid starvation, but throttled to ensure fair ordering + member _.Submit(batches : Batch<'S, 'M>) = incoming.Enqueue batches type ParallelIngester<'Item> = static member Start(log, partitionId, maxRead, submit, statsInterval) = - let makeBatch onCompletion (items : 'Item seq) = + let submitBatch (items : 'Item seq, onCompletion) = let items = Array.ofSeq items - let batch : Submission.SubmissionBatch<_, 'Item> = { source = partitionId; onCompletion = onCompletion; messages = items } - batch,(items.Length,items.Length) - Ingestion.Ingester<'Item seq,Submission.SubmissionBatch<_, 'Item>>.Start(log, partitionId, maxRead, makeBatch, submit, statsInterval) + let batch : Submission.Batch<_, 'Item> = { source = partitionId; onCompletion = onCompletion; messages = items } + submit batch + struct (items.Length, items.Length) + Ingestion.Ingester<'Item seq>.Start(log, partitionId, maxRead, submitBatch, statsInterval) -type ParallelProjector = +type ParallelSink = static member Start ( log : ILogger, maxReadAhead, maxDop, handle, @@ -202,14 +198,14 @@ type ParallelProjector = // Default 5 ?maxSubmissionsPerPartition, ?logExternalStats, ?ingesterStatsInterval) - : ProjectorPipeline<_> = + : Sink> = let maxSubmissionsPerPartition = defaultArg maxSubmissionsPerPartition 5 let ingesterStatsInterval = defaultArg ingesterStatsInterval statsInterval let dispatcher = Scheduling.Dispatcher maxDop let scheduler = Scheduling.PartitionedSchedulingEngine<_, 'Item>(log, handle, dispatcher.TryAdd, statsInterval, ?logExternalStats=logExternalStats) - let mapBatch onCompletion (x : Submission.SubmissionBatch<_, 'Item>) : Scheduling.Batch<_, 'Item> = + let mapBatch onCompletion (x : Submission.Batch<_, 'Item>) : Scheduling.Batch<_, 'Item> = let onCompletion () = x.onCompletion(); onCompletion() { source = x.source; onCompletion = onCompletion; messages = x.messages} @@ -219,4 +215,4 @@ type ParallelProjector = let submitter = Submission.SubmissionEngine<_, _, _>(log, maxSubmissionsPerPartition, mapBatch, submitBatch, statsInterval) let startIngester (rangeLog, partitionId) = ParallelIngester<'Item>.Start(rangeLog, partitionId, maxReadAhead, submitter.Ingest, ingesterStatsInterval) - ProjectorPipeline.Start(log, dispatcher.Pump, scheduler.Pump, submitter.Pump, startIngester) + Sink.Start(log, dispatcher.Pump, scheduler.Pump, submitter.Pump, startIngester) diff --git a/src/Propulsion/Projector.fs b/src/Propulsion/Pipeline.fs similarity index 96% rename from src/Propulsion/Projector.fs rename to src/Propulsion/Pipeline.fs index 6e27e794..54fa60bf 100755 --- a/src/Propulsion/Projector.fs +++ b/src/Propulsion/Pipeline.fs @@ -33,7 +33,7 @@ type Pipeline (task : Task, triggerStop) = use _ = ct.Register(fun () -> x.Stop()) return! x.AwaitShutdown() } -type ProjectorPipeline<'Ingester> private (task : Task, triggerStop, startIngester) = +type Sink<'Ingester> private (task : Task, triggerStop, startIngester) = inherit Pipeline(task, triggerStop) member _.StartIngester(rangeLog : ILogger, partitionId : int) : 'Ingester = startIngester (rangeLog, partitionId) @@ -78,4 +78,4 @@ type ProjectorPipeline<'Ingester> private (task : Task, triggerStop, start let task = Task.Run(supervise) - new ProjectorPipeline<_>(task, triggerStop, startIngester) + new Sink<'Ingester>(task, triggerStop, startIngester) diff --git a/src/Propulsion/Propulsion.fsproj b/src/Propulsion/Propulsion.fsproj index 4dfe246d..933a94e9 100644 --- a/src/Propulsion/Propulsion.fsproj +++ b/src/Propulsion/Propulsion.fsproj @@ -9,7 +9,7 @@ - + diff --git a/src/Propulsion/Streams.fs b/src/Propulsion/Streams.fs index 24781606..f2391014 100755 --- a/src/Propulsion/Streams.fs +++ b/src/Propulsion/Streams.fs @@ -1,9 +1,9 @@ namespace Propulsion.Streams open Propulsion +open Propulsion.Internal // Helpers open Serilog open System -open System.Collections.Concurrent open System.Collections.Generic open System.Threading open System.Threading.Tasks @@ -70,98 +70,6 @@ module Log = | Some m -> Some (m, tryGetScalar GroupTag logEvent) | None -> None -/// A Single Event from an Ordered stream -[] -type StreamEvent<'Format> = { stream : FsCodec.StreamName; event : FsCodec.ITimelineEvent<'Format> } - -/// Span of events from an Ordered Stream -[] -type StreamSpan<'Format> = - { index : int64; events : FsCodec.ITimelineEvent<'Format>[] } - member x.Version = x.events[x.events.Length - 1].Index + 1L - -module Internal = - - /// Gathers stats relating to how many items of a given category have been observed - type CatStats() = - let cats = Dictionary() - - member _.Ingest(cat, ?weight) = - let weight = defaultArg weight 1L - match cats.TryGetValue cat with - | true, catCount -> cats[cat] <- catCount + weight - | false, _ -> cats[cat] <- weight - - member _.Count = cats.Count - member _.Any = (not << Seq.isEmpty) cats - member _.Clear() = cats.Clear() - member _.StatsDescending = Submission.Helpers.statsDescending cats - - type private Data = - { min : TimeSpan - p50 : TimeSpan - p95 : TimeSpan - p99 : TimeSpan - max : TimeSpan - avg : TimeSpan - stddev : TimeSpan option } - - open MathNet.Numerics.Statistics - let private dumpStats (kind : string) (xs : TimeSpan seq) (log : ILogger) = - let sortedLatencies = xs |> Seq.map (fun r -> r.TotalSeconds) |> Seq.sort |> Seq.toArray - - let pc p = SortedArrayStatistics.Percentile(sortedLatencies, p) |> TimeSpan.FromSeconds - let l = { - avg = ArrayStatistics.Mean sortedLatencies |> TimeSpan.FromSeconds - stddev = - let stdDev = ArrayStatistics.StandardDeviation sortedLatencies - // stddev of singletons is NaN - if Double.IsNaN stdDev then None else TimeSpan.FromSeconds stdDev |> Some - - min = SortedArrayStatistics.Minimum sortedLatencies |> TimeSpan.FromSeconds - max = SortedArrayStatistics.Maximum sortedLatencies |> TimeSpan.FromSeconds - p50 = pc 50 - p95 = pc 95 - p99 = pc 99 } - let inline sec (t : TimeSpan) = t.TotalSeconds - let stdDev = match l.stddev with None -> Double.NaN | Some d -> sec d - log.Information(" {kind} {count} : max={max:n3}s p99={p99:n3}s p95={p95:n3}s p50={p50:n3}s min={min:n3}s avg={avg:n3}s stddev={stddev:n3}s", - kind, sortedLatencies.Length, sec l.max, sec l.p99, sec l.p95, sec l.p50, sec l.min, sec l.avg, stdDev) - - /// Operations on an instance are safe cross-thread - type ConcurrentLatencyStats(kind) = - let buffer = ConcurrentStack() - member _.Record value = buffer.Push value - member _.Dump(log : ILogger) = - if not buffer.IsEmpty then - dumpStats kind buffer log - buffer.Clear() // yes, there is a race - - /// Should only be used on one thread - type LatencyStats(kind) = - let buffer = ResizeArray() - member _.Record value = buffer.Add value - member _.Dump(log : ILogger) = - if buffer.Count <> 0 then - dumpStats kind buffer log - buffer.Clear() - -open Internal - -[] -module private Impl = - - let inline arrayBytes (x : _ []) = if obj.ReferenceEquals(null, x) then 0 else x.Length - let inline stringBytes (x : string) = match x with null -> 0 | x -> x.Length * sizeof - let inline dataSize (x : FsCodec.IEventData) = arrayBytes x.Data + stringBytes x.EventType + 16 - let inline eventSize (x : FsCodec.IEventData) = dataSize x + arrayBytes x.Meta - let inline mb x = float x / 1024. / 1024. - let inline accStopwatch (f : unit -> 't) at = - let sw = System.Diagnostics.Stopwatch.StartNew() - let r = f () - at sw.Elapsed - r - module StreamName = /// Despite conventions, there's no guarantee that an arbitrary Kafka `key`, EventStore `StreamId` etc. @@ -181,185 +89,97 @@ module StreamName = | FsCodec.StreamName.CategoryAndId ("", aggregateId) -> aggregateId | FsCodec.StreamName.CategoryAndId (category, _) -> category -module Progress = - - type [] internal BatchState = { markCompleted : unit -> unit; streamToRequiredIndex : Dictionary } - - type ProgressState<'Pos>() = - let pending = Queue() - let trim () = - while pending.Count <> 0 && pending.Peek().streamToRequiredIndex.Count = 0 do - let batch = pending.Dequeue() - batch.markCompleted() - - // We potentially traverse the pending streams thousands of times per second - // so we reuse `InScheduledOrder`'s temps: sortBuffer and streamsBuffer for better L2 caching properties - let sortBuffer = ResizeArray() - let streamsBuffer = HashSet() - - member _.AppendBatch(markCompleted, reqs : Dictionary) = - pending.Enqueue { markCompleted = markCompleted; streamToRequiredIndex = reqs } - trim () - - member _.MarkStreamProgress(stream, index) = - let mutable requiredIndex = Unchecked.defaultof<_> - for x in pending do - if x.streamToRequiredIndex.TryGetValue(stream, &requiredIndex) && requiredIndex <= index then - x.streamToRequiredIndex.Remove stream |> ignore - trim () - - member _.IsEmpty = pending.Count = 0 - // NOTE internal reuse of `sortBuffer` and `streamsBuffer` means it's critical to never have >1 of these in flight - member _.InScheduledOrder(getStreamWeight : (FsCodec.StreamName -> int64) option) : seq = - trim () - // sortBuffer is used once per invocation, but the result is lazy so we can only clear it on entry - sortBuffer.Clear() - let mutable batch = 0 - let weight = match getStreamWeight with None -> (fun _s -> 0) | Some f -> (fun s -> -f s |> int) - for x in pending do - for s in x.streamToRequiredIndex.Keys do - if streamsBuffer.Add s then - // Within the head batch, expedite work on longest streams, where requested - let w = if batch = 0 then weight s else 0 // For anything other than the head batch, submitted order is just fine - sortBuffer.Add((struct (s, batch, w))) - if batch = 0 && sortBuffer.Count > 0 then - let c = Comparer<_>.Default - sortBuffer.Sort(fun struct (_, _ab, _aw) struct (_, _bb, _bw) -> c.Compare(struct(_ab, _aw), struct(_bb, _bw))) - batch <- batch + 1 - // We reuse this buffer next time around, but clear it now as it has no further use - streamsBuffer.Clear() - sortBuffer |> Seq.map (fun struct(s, _, _) -> s) - -module Buffering = - - module StreamSpan = - let (|End|) (x : StreamSpan<_>) = x.index + if x.events = null then 0L else x.events.LongLength - - let dropBeforeIndex min : StreamSpan<_> -> StreamSpan<_> = function - | x when x.index >= min -> x // don't adjust if min not within - | End n when n < min -> { index = min; events = [||] } // throw away if before min - | x -> { index = min; events = x.events |> Array.skip (min - x.index |> int) } // slice - - let merge min (items : StreamSpan<_> seq) = - let candidates = ResizeArray() - for x in items do - if x.events <> null then - let trimmed = dropBeforeIndex min x - if trimmed.events.Length > 0 then candidates.Add trimmed - let mutable buffer = null - let mutable curr = ValueNone - for x in Seq.sortBy (fun x -> x.index) candidates do - match curr, x with - // Not overlapping, no data buffered -> buffer - | ValueNone, _ -> - curr <- ValueSome x - // Gap - | ValueSome (End nextIndex as c), x when x.index > nextIndex -> - if buffer = null then buffer <- ResizeArray() - buffer.Add c - curr <- ValueSome x - // Overlapping, join - | ValueSome (End nextIndex as c), x -> - curr <- ValueSome { c with events = Array.append c.events (dropBeforeIndex nextIndex x).events } - match curr, buffer with - | ValueSome x, null -> Array.singleton x - | ValueSome x, b -> b.Add x; b.ToArray() - | ValueNone, null -> null - | ValueNone, b -> b.ToArray() - - let inline estimateBytesAsJsonUtf8 (x : FsCodec.IEventData<_>) = eventSize x + 80 - - let stats (x : StreamSpan<_>) = - x.events.Length, x.events |> Seq.sumBy estimateBytesAsJsonUtf8 - - let slice (maxEvents, maxBytes) streamSpan = - let mutable count, bytes = 0, 0 - let mutable countBudget, bytesBudget = maxEvents, maxBytes - let withinLimits y = - countBudget <- countBudget - 1 - let eventBytes = estimateBytesAsJsonUtf8 y - bytesBudget <- bytesBudget - eventBytes - // always send at least one event in order to surface the problem and have the stream marked malformed - let res = count = 0 || (countBudget >= 0 && bytesBudget >= 0) - if res then count <- count + 1; bytes <- bytes + eventBytes - res - let trimmed = { streamSpan with events = streamSpan.events |> Array.takeWhile withinLimits } - stats trimmed, trimmed - -(* // ORIGINAL StreamState memory representation: - - Type layout for 'StreamState`1' - Size: 24 bytes. Paddings: 7 bytes (%29 of empty space) - |========================================| - | Object Header (8 bytes) | - |----------------------------------------| - | Method Table Ptr (8 bytes) | - |========================================| - | 0-7: FSharpOption`1 write@ (8 bytes) | - |----------------------------------------| - | 8-15: StreamSpan`1[] queue@ (8 bytes) | - |----------------------------------------| - | 16: Boolean isMalformed@ (1 byte) | - |----------------------------------------| - | 17-23: padding (7 bytes) | - |========================================| - - // CURRENT layout: - - Type layout for 'StreamState`1' - Size: 16 bytes. Paddings: 0 bytes (%0 of empty space) - |========================================| - | 0-7: StreamSpan`1[] queue@ (8 bytes) | - |----------------------------------------| - | 8-15: Int64 write@ (8 bytes) | - |========================================| - *) - - // NOTE: Optimized Representation as we can have a lot of these - // 1. -2 sentinel value for write position signifying `None` (no write position yet established) - // 2. -3 sentinel value for malformed data +/// A contiguous set of Events from a Ordered stream, as held internally within this module +type StreamSpan<'Format> = FsCodec.ITimelineEvent<'Format> array + +module StreamSpan = + + type Metrics = (struct (int * int)) + let metrics eventSize (xs : FsCodec.ITimelineEvent<'F> array) : Metrics = + struct (xs.Length, xs |> Seq.sumBy eventSize) + let slice<'F> eventSize (maxEvents, maxBytes) (span : FsCodec.ITimelineEvent<'F> array) : struct (Metrics * FsCodec.ITimelineEvent<'F> array) = + let mutable count, bytes = 0, 0 + let mutable countBudget, bytesBudget = maxEvents, maxBytes + let withinLimits y = + countBudget <- countBudget - 1 + let eventBytes = eventSize y + bytesBudget <- bytesBudget - eventBytes + // always send at least one event in order to surface the problem and have the stream marked malformed + let res = count = 0 || (countBudget >= 0 && bytesBudget >= 0) + if res then count <- count + 1; bytes <- bytes + eventBytes + res + let trimmed = span |> Array.takeWhile withinLimits + metrics eventSize trimmed, trimmed + + let (|Ver|) (span : StreamSpan<'F>) = span[0].Index + span.LongLength + let dropBeforeIndex min : FsCodec.ITimelineEvent<_> array -> FsCodec.ITimelineEvent<_> array = function + | xs when xs.Length = 0 -> null + | xs when xs[0].Index >= min -> xs // don't adjust if min not within + | xs when (|Ver|) xs <= min -> null // throw away if before min + | xs -> xs |> Array.skip (min - xs[0].Index |> int) // slice + + let merge min (spans : FsCodec.ITimelineEvent<_> array seq) = + let candidates = ResizeArray() + for span in spans do + let trimmed = if span = null then null else dropBeforeIndex min span + if trimmed <> null then + if trimmed.Length = 0 then invalidOp "Cant add empty" + candidates.Add trimmed + let mutable buffer = null + let mutable curr = ValueNone + for x in candidates |> Seq.sortBy (fun x -> x[0].Index) do + match curr with + // Not overlapping, no data buffered -> buffer + | ValueNone -> + curr <- ValueSome x + // Gap + | ValueSome (Ver nextIndex as c) when x[0].Index > nextIndex -> + if buffer = null then buffer <- ResizeArray() + buffer.Add c + curr <- ValueSome x + // Overlapping, join + | ValueSome (Ver nextIndex as c) when (|Ver|) x > nextIndex -> + curr <- ValueSome (Array.append c (dropBeforeIndex nextIndex x)) + | _ -> () // drop + match curr, buffer with + | ValueSome x, null -> Array.singleton x + | ValueSome x, b -> b.Add x; b.ToArray() + | ValueNone, null -> null + | ValueNone, b -> b.ToArray() + +/// A Single Event from an Ordered stream being supplied for ingestion into the internal data structures +type StreamEvent<'Format> = (struct (FsCodec.StreamName * FsCodec.ITimelineEvent<'Format>)) + +module Buffer = + + /// NOTE: Optimized Representation as this is the dominant data structure in terms of memory usage - takes it from 24b to a cache-friendlier 16b + let [] WritePosUnknown = -2L // sentinel value for write position signifying `None` (no write position yet established) + let [] WritePosMalformed = -3L // sentinel value for malformed data [] - type StreamState<'Format> = private { write : int64; queue : StreamSpan<'Format>[] } with - static member Create(write, queue, ?malformed) = - let effWrite = - match write with - | _ when defaultArg malformed false -> -3L - | ValueNone -> -2L - | ValueSome w -> w - { write = effWrite; queue = queue } + type StreamState<'Format> = private { write : int64; queue : FsCodec.ITimelineEvent<'Format> array array } with + static member Create(write, queue, malformed) = + if malformed then { write = WritePosMalformed; queue = queue } + else StreamState<'Format>.Create(write, queue) + static member Create(write, queue) = { write = (match write with ValueSome x -> x | ValueNone -> WritePosUnknown); queue = queue } member x.IsEmpty = obj.ReferenceEquals(null, x.queue) - member x.IsPurgeable = x.IsEmpty && not x.IsMalformed - member x.IsMalformed = not x.IsEmpty && -3L = x.write + member x.EventsSumBy(f) = if x.IsEmpty then 0L else x.queue |> Seq.map (Seq.sumBy f) |> Seq.sum |> int64 + member x.EventsCount = if x.IsEmpty then 0 else x.queue |> Seq.sumBy Array.length + + member x.HeadSpan = x.queue[0] + member x.IsMalformed = not x.IsEmpty && WritePosMalformed = x.write member x.HasValid = not x.IsEmpty && not x.IsMalformed - member x.Write = match x.write with -2L -> ValueNone | x -> ValueSome x - member x.Queue : StreamSpan<'Format>[] = x.queue - member x.IsReady = - if not x.HasValid then false else - match x.Write, Array.head x.queue with - | ValueSome w, { index = i } -> i = w - | ValueNone, _ -> true + member x.WritePos = match x.write with WritePosUnknown -> ValueNone | x -> ValueSome x + member x.HasGap = match x.write with w when w = WritePosUnknown -> false | w -> w <> x.HeadSpan[0].Index + member x.CanPurge = x.IsEmpty && not x.IsMalformed module StreamState = - let eventsSize (x : StreamState) = - if x.IsEmpty then 0L else - - let mutable acc = 0 - for x in x.queue do - for x in x.events do - acc <- acc + eventSize x - int64 acc - - let inline private optionCombine f (r1 : 'a ValueOption) (r2 : 'a ValueOption) = - match r1, r2 with - | ValueSome x, ValueSome y -> f x y |> ValueSome - | ValueNone, ValueNone -> ValueNone - | ValueNone, x | x, ValueNone -> x let combine (s1 : StreamState<_>) (s2 : StreamState<_>) : StreamState<'Format> = - let writePos = optionCombine max s1.Write s2.Write + let writePos = max s1.WritePos s2.WritePos let malformed = s1.IsMalformed || s2.IsMalformed - let any1, any2 = not (isNull s1.queue), not (isNull s2.queue) + let any1 = not (isNull s1.queue) + let any2 = not (isNull s2.queue) if any1 || any2 then let items = if any1 && any2 then Seq.append s1.queue s2.queue elif any1 then s1.queue else s2.queue StreamState<'Format>.Create(writePos, StreamSpan.merge (defaultValueArg writePos 0L) items, malformed) @@ -372,107 +192,185 @@ module Buffering = if states.TryGetValue(stream, ¤t) then states[stream] <- StreamState.combine current state else states.Add(stream, state) - member _.Merge(item : StreamEvent<'Format>) = - merge item.stream (StreamState<'Format>.Create(ValueNone, [| { index = item.event.Index; events = [| item.event |] } |])) - - member _.Items = states :> seq>> + member _.Merge(stream, event : FsCodec.ITimelineEvent<'Format>) = + merge stream (StreamState<'Format>.Create(ValueNone, [| [| event |] |])) - member private _.States = states + member _.States = states :> seq>> member _.Merge(other : Streams<'Format>) = for x in other.States do merge x.Key x.Value member _.Dump(log : ILogger, estimateSize, categorize) = let mutable waiting, waitingE, waitingB = 0, 0, 0L - let waitingCats, waitingStreams = CatStats(), CatStats() + let waitingCats, waitingStreams = Stats.CatStats(), Stats.CatStats() for KeyValue (stream, state) in states do - let sz = estimateSize state - waitingCats.Ingest(categorize stream) - if sz <> 0L then - let sn, wp = FsCodec.StreamName.toString stream, defaultValueArg state.Write 0L - waitingStreams.Ingest(sprintf "%s@%dx%d" sn wp state.queue[0].events.Length, (sz + 512L) / 1024L) - waiting <- waiting + 1 - waitingE <- waitingE + (state.queue |> Array.sumBy (fun x -> x.events.Length)) - waitingB <- waitingB + sz + if not state.IsEmpty then + let sz = estimateSize state + waitingCats.Ingest(categorize stream) + if sz <> 0L then + let sn, wp = FsCodec.StreamName.toString stream, defaultValueArg state.WritePos 0L + waitingStreams.Ingest(sprintf "%s@%dx%d" sn wp state.queue[0].Length, (sz + 512L) / 1024L) + waiting <- waiting + 1 + waitingE <- waitingE + (state.queue |> Array.sumBy (fun x -> x.Length)) + waitingB <- waitingB + sz let m = Log.Metric.BufferReport { cats = waitingCats.Count; streams = waiting; events = waitingE; bytes = waitingB } - (log |> Log.metric m).Information(" Streams Waiting {busy:n0}/{busyMb:n1}MB ", waiting, mb waitingB) + (log |> Log.metric m).Information(" Streams Waiting {busy:n0}/{busyMb:n1}MB", waiting, mb waitingB) if waitingCats.Any then log.Information(" Waiting Categories, events {@readyCats}", Seq.truncate 5 waitingCats.StatsDescending) if waitingCats.Any then log.Information(" Waiting Streams, KB {@readyStreams}", Seq.truncate 5 waitingStreams.StatsDescending) -type EventMetrics = int * int + [] + type Batch<'Format> private (onCompletion, buffer, reqs) = + let mutable buffer = ValueSome buffer + static member Create(onCompletion, streamEvents : StreamEvent<'Format> seq) = + let buffer, reqs = Streams<'Format>(), Dictionary() + let mutable itemCount = 0 + for struct (stream, event) in streamEvents do + itemCount <- itemCount + 1 + buffer.Merge(stream, event) + match reqs.TryGetValue(stream), event.Index + 1L with + | (false, _), required -> reqs[stream] <- required + | (true, actual), required when actual < required -> reqs[stream] <- required + | (true, _), _ -> () // replayed same or earlier item + let batch = Batch(onCompletion, buffer, reqs) + struct (batch, struct (batch.RemainingStreamsCount, itemCount)) -module Scheduling = + member _.OnCompletion = onCompletion + member _.Reqs = reqs :> seq> + member _.RemainingStreamsCount = reqs.Count + member _.TryTakeStreams() = let t = buffer in buffer <- ValueNone; t + member _.TryMerge(other : Batch<_>) = + match buffer, other.TryTakeStreams() with + | ValueSome x, ValueSome y -> x.Merge(y); true + | ValueSome _, ValueNone -> false + | ValueNone, x -> buffer <- x; false + +/// A separate Async pump manages dispatching of scheduled work via the ThreadPool , managing the concurrency limits +module Dispatch = + + [] + type Item<'Format> = { stream : FsCodec.StreamName; writePos : int64 voption; span : FsCodec.ITimelineEvent<'Format> array } + + /// Coordinates the dispatching of work and emission of results, subject to the maxDop concurrent processors constraint + type internal DopDispatcher<'R>(maxDop : int) = + let tryWrite, wait, apply = + let c = Channel.unboundedSwSr Task<'R>> in let r, w = c.Reader, c.Writer + w.TryWrite, Channel.awaitRead r, Channel.apply r >> ignore + let result = Event<'R>() + let dop = Sem maxDop + + // NOTE this obviously depends on the passed computation never throwing, or we'd leak dop + let wrap ct (computation : CancellationToken -> Task<'R>) () = task { + let! res = computation ct + dop.Release() + result.Trigger res } + + [] member _.Result = result.Publish + member _.HasCapacity = dop.HasCapacity + member _.AwaitButRelease() = dop.AwaitButRelease() + member _.State = dop.State + + member _.TryAdd(item) = + dop.TryTake() && tryWrite item + + member _.Pump(ct : CancellationToken) = task { + while not ct.IsCancellationRequested do + try do! wait ct :> Task + with :? OperationCanceledException -> () + let run (f : CancellationToken -> Task<'R>) = Task.start (wrap ct f) + apply run } + + /// Kicks off enough work to fill the inner Dispatcher up to capacity + type ItemDispatcher<'R, 'F>(maxDop) = + let inner = DopDispatcher(maxDop) + + // On each iteration, we try to fill the in-flight queue, taking the oldest and/or heaviest streams first + let tryFillDispatcher (potential : seq>) markStarted project markBusy = + let xs = potential.GetEnumerator() + let ts = System.Diagnostics.Stopwatch.GetTimestamp() + let mutable hasCapacity, dispatched = true, false + while xs.MoveNext() && hasCapacity do + let item = xs.Current + let succeeded = inner.TryAdd(project ts item) + if succeeded then + markBusy item.stream + markStarted (item.stream, ts) + hasCapacity <- succeeded + dispatched <- dispatched || succeeded // if we added any request, we'll skip sleeping + struct (dispatched, hasCapacity) + + member _.Pump ct = inner.Pump ct + [] member _.Result = inner.Result + member _.State = inner.State + member _.TryReplenish(pending, markStarted, project, markStreamBusy) = + tryFillDispatcher pending markStarted project markStreamBusy + member _.HasCapacity = inner.HasCapacity + member _.AwaitCapacity() = inner.AwaitButRelease() - open Buffering +module Scheduling = - type [] DispatchItem<'Format> = { stream : FsCodec.StreamName; writePos : int64 voption; span : StreamSpan<'Format> } + open Buffer type StreamStates<'Format>() = let states = Dictionary>() - let update stream (state : StreamState<_>) = - let mutable current = Unchecked.defaultof<_> - if states.TryGetValue(stream, ¤t) then + let tryGetItem stream = + let mutable x = Unchecked.defaultof<_> + if states.TryGetValue(stream, &x) then ValueSome x else ValueNone + let merge stream (state : StreamState<_>) = + match tryGetItem stream with + | ValueSome current -> let updated = StreamState.combine current state states[stream] <- updated - stream, updated - else + updated + | ValueNone -> states.Add(stream, state) - stream, state - let merge_ stream (state : StreamState<_>) = - let mutable current = Unchecked.defaultof<_> - if states.TryGetValue(stream, ¤t) then states[stream] <- StreamState.combine current state - else states.Add(stream, state) - let merge (buffer : Streams<'Format>) = for x in buffer.Items do merge_ x.Key x.Value - let markCompleted stream index = merge_ stream (StreamState<'Format>.Create(ValueSome index, queue = null, malformed = false)) - - let updateWritePos stream isMalformed pos span = update stream (StreamState<'Format>.Create(pos, span, isMalformed)) + state + let markCompleted stream index = + merge stream (StreamState<'Format>.Create(ValueSome index, queue = null, malformed = false)) |> ignore + let updateWritePos stream isMalformed pos span = + merge stream (StreamState<'Format>.Create(pos, span, isMalformed)) let purge () = let mutable purged = 0 for x in states do let v = x.Value - if v.IsPurgeable then + if v.CanPurge then states.Remove x.Key |> ignore // Safe to do while iterating on netcore >=3.0 purged <- purged + 1 states.Count, purged let busy = HashSet() - let pending trySlipstreamed (requestedOrder : FsCodec.StreamName seq) : seq> = seq { + let pending trySlipstreamed (requestedOrder : FsCodec.StreamName seq) : seq> = seq { let proposed = HashSet() for s in requestedOrder do - match states.TryGetValue s with - | true, ss when ss.HasValid && not (busy.Contains s) -> + match tryGetItem s with + | ValueSome ss when ss.HasValid && not (busy.Contains s) -> proposed.Add s |> ignore // should always be true - yield { writePos = ss.Write; stream = s; span = Array.head ss.Queue } + yield { writePos = ss.WritePos; stream = s; span = ss.HeadSpan } | _ -> () if trySlipstreamed then // [lazily] slipstream in further events that are not yet referenced by in-scope batches - for KeyValue(s, v) in states do - if v.HasValid && not (busy.Contains s) && proposed.Add s then - yield { writePos = v.Write; stream = s; span = Array.head v.Queue } } - + for KeyValue(s, ss) in states do + if ss.HasValid && not (busy.Contains s) && proposed.Add s then + yield { writePos = ss.WritePos; stream = s; span = ss.HeadSpan } } let markBusy stream = busy.Add stream |> ignore let markNotBusy stream = busy.Remove stream |> ignore - member _.InternalMerge buffer = merge buffer - member _.Purge() = purge () - member _.InternalUpdate stream pos queue = update stream (StreamState<'Format>.Create(ValueSome pos, queue)) - - member _.Add(stream, index, event, ?isMalformed) = - updateWritePos stream (defaultArg isMalformed false) ValueNone [| { index = index; events = [| event |] } |] - - member _.Add(stream, span: StreamSpan<_>, isMalformed) = - updateWritePos stream isMalformed ValueNone [| span |] - + member _.WritePositionIsAlreadyBeyond(stream, required) = + match tryGetItem stream with + | ValueSome ss -> match ss.WritePos with ValueSome cw -> cw >= required | _ -> false + | _ -> false + member _.Merge(streams : Streams<'Format>) = + for kv in streams.States do + merge kv.Key kv.Value |> ignore + member _.RecordWriteProgress(stream, pos, queue) = + merge stream (StreamState<'Format>.Create(ValueSome pos, queue)) member _.SetMalformed(stream, isMalformed) = - updateWritePos stream isMalformed ValueNone [| { index = 0L; events = null } |] + updateWritePos stream isMalformed ValueNone null + member _.Purge() = + purge () - member _.TryGetItem(stream) = - let mutable x = Unchecked.defaultof<_> - if states.TryGetValue(stream, &x) then ValueSome x else ValueNone - - member _.WritePositionIsAlreadyBeyond(stream, required) = - let mutable streamState = Unchecked.defaultof<_> - states.TryGetValue(stream, &streamState) && - match streamState.Write with ValueSome cw -> cw >= required | ValueNone -> false + member _.HeadSpanSizeBy(stream, f : _ -> int) = + match tryGetItem stream with + | ValueSome state when not state.IsEmpty -> state.HeadSpan |> Array.sumBy f |> int64 + | _ -> 0L member _.MarkBusy stream = markBusy stream @@ -484,53 +382,53 @@ module Scheduling = member _.MarkFailed stream = markNotBusy stream - member _.Pending(trySlipstreamed, byQueuedPriority : FsCodec.StreamName seq) : DispatchItem<'Format> seq = + member _.Pending(trySlipstreamed, byQueuedPriority : FsCodec.StreamName seq) : Dispatch.Item<'Format> seq = pending trySlipstreamed byQueuedPriority - member _.Dump(log : ILogger, totalPurged, estimateSize) = + member _.Dump(log : ILogger, totalPurged : int, eventSize) = let mutable (busyCount, busyE, busyB), (ready, readyE, readyB), synced = (0, 0, 0L), (0, 0, 0L), 0 - let mutable (unprefixed, unprefixedE, unprefixedB), (malformed, malformedE, malformedB) = (0, 0, 0L), (0, 0, 0L) - let busyCats, readyCats, readyStreams = CatStats(), CatStats(), CatStats() - let unprefixedCats, unprefixedStreams, malformedCats, malformedStreams = CatStats(), CatStats(), CatStats(), CatStats() + let mutable (gaps, gapsE, gapsB), (malformed, malformedE, malformedB) = (0, 0, 0L), (0, 0, 0L) + let busyCats, readyCats, readyStreams = Stats.CatStats(), Stats.CatStats(), Stats.CatStats() + let gapCats, gapStreams, malformedCats, malformedStreams = Stats.CatStats(), Stats.CatStats(), Stats.CatStats(), Stats.CatStats() let kb sz = (sz + 512L) / 1024L for KeyValue (stream, state) in states do - match estimateSize state with + match state.EventsSumBy(eventSize) with | 0L -> synced <- synced + 1 | sz when busy.Contains stream -> busyCats.Ingest(StreamName.categorize stream) busyCount <- busyCount + 1 busyB <- busyB + sz - busyE <- busyE + (state.Queue |> Array.sumBy (fun x -> x.events.Length)) + busyE <- busyE + state.EventsCount | sz when state.IsMalformed -> malformedCats.Ingest(StreamName.categorize stream) malformedStreams.Ingest(FsCodec.StreamName.toString stream, mb sz |> int64) malformed <- malformed + 1 malformedB <- malformedB + sz - malformedE <- malformedE + (state.Queue |> Array.sumBy (fun x -> x.events.Length)) - | sz when not state.IsReady -> - unprefixedCats.Ingest(StreamName.categorize stream) - unprefixedStreams.Ingest(FsCodec.StreamName.toString stream, mb sz |> int64) - unprefixed <- unprefixed + 1 - unprefixedB <- unprefixedB + sz - unprefixedE <- unprefixedE + (state.Queue |> Array.sumBy (fun x -> x.events.Length)) + malformedE <- malformedE + state.EventsCount + | sz when state.HasGap -> + gapCats.Ingest(StreamName.categorize stream) + gapStreams.Ingest(FsCodec.StreamName.toString stream, mb sz |> int64) + gaps <- gaps + 1 + gapsB <- gapsB + sz + gapsE <- gapsE + state.EventsCount | sz -> readyCats.Ingest(StreamName.categorize stream) - readyStreams.Ingest(sprintf "%s@%dx%d" (FsCodec.StreamName.toString stream) (defaultValueArg state.Write 0L) state.Queue[0].events.Length, kb sz) + readyStreams.Ingest(sprintf "%s@%dx%d" (FsCodec.StreamName.toString stream) (defaultValueArg state.WritePos 0L) state.HeadSpan.Length, kb sz) ready <- ready + 1 readyB <- readyB + sz - readyE <- readyE + (state.Queue |> Array.sumBy (fun x -> x.events.Length)) + readyE <- readyE + state.EventsCount let busyStats : Log.BufferMetric = { cats = busyCats.Count; streams = busyCount; events = busyE; bytes = busyB } let readyStats : Log.BufferMetric = { cats = readyCats.Count; streams = readyStreams.Count; events = readyE; bytes = readyB } - let bufferingStats : Log.BufferMetric = { cats = unprefixedCats.Count; streams = unprefixedStreams.Count; events = unprefixedE; bytes = unprefixedB } + let bufferingStats : Log.BufferMetric = { cats = gapCats.Count; streams = gapStreams.Count; events = gapsE; bytes = gapsB } let malformedStats : Log.BufferMetric = { cats = malformedCats.Count; streams = malformedStreams.Count; events = malformedE; bytes = malformedB } let m = Log.Metric.SchedulerStateReport (synced, busyStats, readyStats, bufferingStats, malformedStats) (log |> Log.metric m).Information("Streams Synced {synced:n0} Purged {purged:n0} Active {busy:n0}/{busyMb:n1}MB Ready {ready:n0}/{readyMb:n1}MB Waiting {waiting}/{waitingMb:n1}MB Malformed {malformed}/{malformedMb:n1}MB", - synced, totalPurged, busyCount, mb busyB, ready, mb readyB, unprefixed, mb unprefixedB, malformed, mb malformedB) + synced, totalPurged, busyCount, mb busyB, ready, mb readyB, gaps, mb gapsB, malformed, mb malformedB) if busyCats.Any then log.Information(" Active Categories, events {@busyCats}", Seq.truncate 5 busyCats.StatsDescending) if readyCats.Any then log.Information(" Ready Categories, events {@readyCats}", Seq.truncate 5 readyCats.StatsDescending) if readyCats.Any then log.Information(" Ready Streams, KB {@readyStreams}", Seq.truncate 5 readyStreams.StatsDescending) - if unprefixedStreams.Any then log.Information(" Waiting Streams, KB {@missingStreams}", Seq.truncate 3 unprefixedStreams.StatsDescending) + if gapStreams.Any then log.Information(" Buffering Streams, KB {@missingStreams}", Seq.truncate 3 gapStreams.StatsDescending) if malformedStreams.Any then log.Information(" Malformed Streams, MB {@malformedStreams}", malformedStreams.StatsDescending) /// Messages used internally by projector, including synthetic ones for the purposes of the `Stats` listeners @@ -542,98 +440,116 @@ module Scheduling = | Result of duration : TimeSpan * stream : FsCodec.StreamName * progressed : bool * result : 'R type [] BufferState = Idle | Active | Full | Slipstreaming - type StateStats() = - let mutable idle, active, full, slip = 0, 0, 0, 0 - member _.Clear() = idle <- 0; active <- 0; full <- 0; slip <- 0 - member _.Ingest state = - match state with - | Idle -> idle <- idle + 1 - | Active -> active <- active + 1 - | Full -> full <- full + 1 - | Slipstreaming -> slip <- slip + 1 - member _.StatsDescending = - let t = CatStats() - if idle > 0 then t.Ingest(nameof Idle, idle) - if active > 0 then t.Ingest(nameof Active, active) - if full > 0 then t.Ingest(nameof Full, full) - if slip > 0 then t.Ingest(nameof Slipstreaming, slip) - t.StatsDescending - - /// Manages state used to generate metrics (and summary logs) regarding streams currently being processed by a Handler - module Busy = - let private ticksPerSecond = double System.Diagnostics.Stopwatch.Frequency - let timeSpanFromStopwatchTicks = function - | ticks when ticks > 0L -> TimeSpan.FromSeconds(double ticks / ticksPerSecond) - | _ -> TimeSpan.Zero - type private StreamState = { ts : int64; mutable count : int } - let private walkAges (state : Dictionary<_, _>) = - let now = System.Diagnostics.Stopwatch.GetTimestamp() - if state.Count = 0 then Seq.empty else - seq { for x in state.Values -> struct (now - x.ts, x.count) } - let private renderState agesAndCounts = - let mutable oldest, newest, streams, attempts = Int64.MinValue, Int64.MaxValue, 0, 0 - for struct (diff, count) in agesAndCounts do - oldest <- max oldest diff - newest <- min newest diff - streams <- streams + 1 - attempts <- attempts + count - if streams = 0 then oldest <- 0L; newest <- 0L - (streams, attempts), (timeSpanFromStopwatchTicks oldest, timeSpanFromStopwatchTicks newest) - /// Manages the list of currently dispatched Handlers - /// NOTE we are guaranteed we'll hear about a Start before a Finish (or another Start) per stream by the design of the Dispatcher - type private Active() = - let state = Dictionary() - member _.HandleStarted(sn, ts) = state.Add(sn, { ts = ts; count = 1 }) - member _.TakeFinished(sn) = - let res = state[sn] - state.Remove sn |> ignore - res.ts - member _.State = walkAges state |> renderState - /// Represents state of streams where the handler did not make progress on the last execution either intentionally or due to an exception - type private Repeating() = - let state = Dictionary() - member _.HandleResult(sn, isStuck, startTs) = - if not isStuck then state.Remove sn |> ignore - else match state.TryGetValue sn with - | true, v -> v.count <- v.count + 1 - | false, _ -> state.Add(sn, { ts = startTs; count = 1 }) - member _.State = walkAges state |> renderState - /// Collates all state and reactions to manage the list of busy streams based on callbacks/notifications from the Dispatcher - type Monitor() = - let active, failing, stuck = Active(), Repeating(), Repeating() - let emit (log : ILogger) state (streams, attempts) (oldest : TimeSpan, newest : TimeSpan) = - log.Information(" {state} {streams} for {newest:n1}-{oldest:n1}s, {attempts} attempts", - state, streams, newest.TotalSeconds, oldest.TotalSeconds, attempts) - member _.HandleStarted(sn, ts) = - active.HandleStarted(sn, ts) - member _.HandleResult(sn, succeeded, progressed) = - let startTs = active.TakeFinished(sn) - failing.HandleResult(sn, not succeeded, startTs) - stuck.HandleResult(sn, succeeded && not progressed, startTs) - member _.DumpState(log : ILogger) = - let inline dump state (streams, attempts) ages = - if streams <> 0 then - emit log state (streams, attempts) ages - active.State ||> dump "active" - failing.State ||> dump "failing" - stuck.State ||> dump "stalled" - member _.EmitMetrics(log : ILogger) = - let inline report state (streams, attempts) (oldest : TimeSpan, newest : TimeSpan) = - let m = Log.Metric.StreamsBusy (state, streams, oldest.TotalSeconds, newest.TotalSeconds) - emit (log |> Log.metric m) state (streams, attempts) (oldest, newest) - active.State ||> report "active" - failing.State ||> report "failing" - stuck.State ||> report "stalled" + + module Stats = + + /// Manages state used to generate metrics (and summary logs) regarding streams currently being processed by a Handler + module Busy = + + let private ticksPerSecond = double System.Diagnostics.Stopwatch.Frequency + let timeSpanFromStopwatchTicks = function + | ticks when ticks > 0L -> TimeSpan.FromSeconds(double ticks / ticksPerSecond) + | _ -> TimeSpan.Zero + type private StreamState = { ts : int64; mutable count : int } + let private walkAges (state : Dictionary<_, _>) = + if state.Count = 0 then Seq.empty else + let now = System.Diagnostics.Stopwatch.GetTimestamp() + seq { for x in state.Values -> struct (now - x.ts, x.count) } + let private renderState agesAndCounts = + let mutable oldest, newest, streams, attempts = Int64.MinValue, Int64.MaxValue, 0, 0 + for struct (diff, count) in agesAndCounts do + oldest <- max oldest diff + newest <- min newest diff + streams <- streams + 1 + attempts <- attempts + count + if streams = 0 then oldest <- 0L; newest <- 0L + struct (streams, attempts), struct (timeSpanFromStopwatchTicks oldest, timeSpanFromStopwatchTicks newest) + /// Manages the list of currently dispatched Handlers + /// NOTE we are guaranteed we'll hear about a Start before a Finish (or another Start) per stream by the design of the Dispatcher + type private Active() = + let state = Dictionary() + member _.HandleStarted(sn, ts) = state.Add(sn, { ts = ts; count = 1 }) + member _.TakeFinished(sn) = + let res = state[sn] + state.Remove sn |> ignore + res.ts + member _.State = walkAges state |> renderState + /// Represents state of streams where the handler did not make progress on the last execution either intentionally or due to an exception + type private Repeating() = + let state = Dictionary() + member _.HandleResult(sn, isStuck, startTs) = + if not isStuck then state.Remove sn |> ignore + else match state.TryGetValue sn with + | true, v -> v.count <- v.count + 1 + | false, _ -> state.Add(sn, { ts = startTs; count = 1 }) + member _.State = walkAges state |> renderState + /// Collates all state and reactions to manage the list of busy streams based on callbacks/notifications from the Dispatcher + type Monitor() = + let active, failing, stuck = Active(), Repeating(), Repeating() + let emit (log : ILogger) state struct (streams, attempts) struct (oldest : TimeSpan, newest : TimeSpan) = + log.Information(" {state} {streams} for {newest:n1}-{oldest:n1}s, {attempts} attempts", + state, streams, newest.TotalSeconds, oldest.TotalSeconds, attempts) + member _.HandleStarted(sn, ts) = + active.HandleStarted(sn, ts) + member _.HandleResult(sn, succeeded, progressed) = + let startTs = active.TakeFinished(sn) + failing.HandleResult(sn, not succeeded, startTs) + stuck.HandleResult(sn, succeeded && not progressed, startTs) + member _.DumpState(log : ILogger) = + let dump state struct (streams, attempts) ages = + if streams <> 0 then + emit log state (streams, attempts) ages + active.State ||> dump "active" + failing.State ||> dump "failing" + stuck.State ||> dump "stalled" + member _.EmitMetrics(log : ILogger) = + let report state struct (streams, attempts) struct (oldest : TimeSpan, newest : TimeSpan) = + let m = Log.Metric.StreamsBusy (state, streams, oldest.TotalSeconds, newest.TotalSeconds) + emit (log |> Log.metric m) state (streams, attempts) (oldest, newest) + active.State ||> report "active" + failing.State ||> report "failing" + stuck.State ||> report "stalled" + + type StateStats() = + let mutable idle, active, full, slip = 0, 0, 0, 0 + member _.Clear() = idle <- 0; active <- 0; full <- 0; slip <- 0 + member _.Ingest state = + match state with + | Idle -> idle <- idle + 1 + | Active -> active <- active + 1 + | Full -> full <- full + 1 + | Slipstreaming -> slip <- slip + 1 + member _.StatsDescending = + let t = Stats.CatStats() + if idle > 0 then t.Ingest(nameof Idle, idle) + if active > 0 then t.Ingest(nameof Active, active) + if full > 0 then t.Ingest(nameof Full, full) + if slip > 0 then t.Ingest(nameof Slipstreaming, slip) + t.StatsDescending + + type [] Timers = + { mutable dt : TimeSpan; mutable ft : TimeSpan; mutable mt : TimeSpan; mutable it : TimeSpan; mutable st : TimeSpan; mutable zt : TimeSpan } + member x.AddResults(y) = x.dt <- x.dt + y + member x.AddDispatch(y) = x.ft <- x.ft + y + member x.AddMerge(y) = x.mt <- x.mt + y + member x.AddIngest(y) = x.it <- x.it + y + member x.AddStats(y) = x.st <- x.st + y + member x.AddSleep(y) = x.zt <- x.zt + y + member x.Add(y : Timers) = x.AddResults y.dt; x.AddDispatch y.ft; x.AddMerge y.mt; x.AddIngest y.it; x.AddStats y.st; x.AddSleep y.zt + member x.Dump(log : ILogger) = + let m = Log.Metric.SchedulerCpu (x.mt, x.it, x.ft, x.dt, x.st) + (log |> Log.metric m).Information(" Cpu Streams {mt:n1}s Batches {it:n1}s Dispatch {ft:n1}s Results {dt:n1}s Stats {st:n1}s Sleep {zt:n1}s", + x.mt.TotalSeconds, x.it.TotalSeconds, x.ft.TotalSeconds, x.dt.TotalSeconds, x.st.TotalSeconds, x.zt.TotalSeconds) /// Gathers stats pertaining to the core projection/ingestion activity [] type Stats<'R, 'E>(log : ILogger, statsInterval : TimeSpan, stateInterval : TimeSpan) = let mutable cycles, fullCycles = 0, 0 - let stateStats, oks, exns, mon = StateStats(), LatencyStats("ok"), LatencyStats("exceptions"), Busy.Monitor() + let stateStats, oks, exns, mon = Stats.StateStats(), Stats.LatencyStats("ok"), Stats.LatencyStats("exceptions"), Stats.Busy.Monitor() let mutable batchesPended, streamsPended, eventsSkipped, eventsPended = 0, 0, 0, 0 - let statsDue, stateDue, stucksDue = intervalCheck statsInterval, intervalCheck stateInterval, intervalCheck (TimeSpan.FromSeconds 1.) let metricsLog = log.ForContext("isMetric", true) - let mutable dt, ft, mt, it, st, zt = TimeSpan.Zero, TimeSpan.Zero, TimeSpan.Zero, TimeSpan.Zero, TimeSpan.Zero, TimeSpan.Zero + let mutable t = Unchecked.defaultof + let monitorInterval = IntervalTimer(TimeSpan.FromSeconds 1.) let dumpStats (dispatchActive, dispatchMax) batchesWaiting = log.Information("Scheduler {cycles} cycles ({fullCycles} full) {@states} Running {busy}/{processors}", cycles, fullCycles, stateStats.StatsDescending, dispatchActive, dispatchMax) @@ -642,10 +558,12 @@ module Scheduling = log.Information(" Batches Holding {batchesWaiting} Started {batches} ({streams:n0}s {events:n0}-{skipped:n0}e)", batchesWaiting, batchesPended, streamsPended, eventsSkipped + eventsPended, eventsSkipped) batchesPended <- 0; streamsPended <- 0; eventsSkipped <- 0; eventsPended <- 0 - let m = Log.Metric.SchedulerCpu (mt, it, ft, dt, st) - (log |> Log.metric m).Information(" Cpu Streams {mt:n1}s Batches {it:n1}s Dispatch {ft:n1}s Results {dt:n1}s Stats {st:n1}s Sleep {zt:n1}s", - mt.TotalSeconds, it.TotalSeconds, ft.TotalSeconds, dt.TotalSeconds, st.TotalSeconds, zt.TotalSeconds) - dt <- TimeSpan.Zero; ft <- TimeSpan.Zero; mt <- TimeSpan.Zero; it <- TimeSpan.Zero; st <- TimeSpan.Zero; zt <- TimeSpan.Zero + t.Dump log + t <- Unchecked.defaultof<_> + + member val Log = log + member val StatsInterval = IntervalTimer statsInterval + member val StateInterval = IntervalTimer stateInterval abstract member Handle : InternalMessage> -> unit default x.Handle msg = msg |> function @@ -662,224 +580,210 @@ module Scheduling = x.HandleResult(stream, duration, false, progressed) abstract HandleResult : FsCodec.StreamName * TimeSpan * progressed : bool * succeeded : bool -> unit - default _.HandleResult(stream, duration, succeeded, progressed) = + default x.HandleResult(stream, duration, succeeded, progressed) = mon.HandleResult(stream, succeeded, progressed) if metricsLog.IsEnabled Serilog.Events.LogEventLevel.Information then let outcomeKind = if succeeded then "ok" else "exceptions" let m = Log.Metric.HandlerResult (outcomeKind, duration.TotalSeconds) (metricsLog |> Log.metric m).Information("Outcome {kind} in {ms:n0}ms, progressed: {progressed}", outcomeKind, duration.TotalMilliseconds, progressed) - if stucksDue () then + if monitorInterval.IfDueRestart() then mon.EmitMetrics metricsLog abstract MarkStarted : stream : FsCodec.StreamName * stopwatchTicks : int64 -> unit default _.MarkStarted(stream, stopwatchTicks) = mon.HandleStarted(stream, stopwatchTicks) - member x.DumpStats((used, max), batchesWaiting) = + member x.RecordStats() = cycles <- cycles + 1 - if statsDue () then - dumpStats (used, max) batchesWaiting - mon.DumpState log - x.DumpStats() - - member _.TryDumpState(state, dump, struct (_dt, _ft, _mt, _it, _st, _zt)) = - dt <- dt + _dt - ft <- ft + _ft - mt <- mt + _mt - it <- it + _it - st <- st + _st - zt <- zt + _zt + x.StatsInterval.IfDueRestart() + + member x.DumpStats(struct (used, max), batchesWaiting) = + dumpStats (used, max) batchesWaiting + mon.DumpState x.Log + x.DumpStats() + + member x.RecordState(state, timers) = fullCycles <- fullCycles + 1 + t.Add(timers) stateStats.Ingest(state) - - let due = stateDue () - if due then - dump log - due + x.StateInterval.IfDueRestart() /// Allows an ingester or projector to wire in custom stats (typically based on data gathered in a `Handle` override) abstract DumpStats : unit -> unit default _.DumpStats () = () - /// Coordinates the dispatching of work and emission of results, subject to the maxDop concurrent processors constraint - type private DopDispatcher<'R>(maxDop : int) = - let tryWrite, wait, apply = - let c = Channel.unboundedSwSr Task<'R>> - c.Writer.TryWrite, Channel.awaitRead c, Channel.apply c >> ignore - let result = Event<'R>() - let dop = Sem maxDop - - // NOTE this obviously depends on the passed computation never throwing, or we'd leak dop - let wrap ct (computation : CancellationToken -> Task<'R>) () = task { - let! res = computation ct - dop.Release() - result.Trigger res } - - [] member _.Result = result.Publish - member _.HasCapacity = dop.HasCapacity - member _.AwaitButRelease() = dop.AwaitButRelease() - member _.State = dop.State - - member _.TryAdd(item) = - dop.TryTake() && tryWrite item - - member _.Pump(ct : CancellationToken) = task { - while not ct.IsCancellationRequested do - try do! wait ct :> Task - with :? OperationCanceledException -> () - let run (f : CancellationToken -> Task<'R>) = Task.start (wrap ct f) - apply run } - - /// Kicks off enough work to fill the inner Dispatcher up to capacity - type ItemDispatcher<'R>(maxDop) = - let inner = DopDispatcher(maxDop) - - // On each iteration, we try to fill the in-flight queue, taking the oldest and/or heaviest streams first - let tryFillDispatcher (potential : seq>, markStarted) project markBusy = - let xs = potential.GetEnumerator() - let ts = System.Diagnostics.Stopwatch.GetTimestamp() - let mutable hasCapacity, dispatched = true, false - while xs.MoveNext() && hasCapacity do - let item = xs.Current - let succeeded = inner.TryAdd(project ts item) + /// Allows serialization of the emission of statistics where multiple Schedulers are active (via an externally managed lock object) + abstract Serialize : (unit -> unit) -> unit + default _.Serialize(f) = f () + + module Dispatcher = + + /// Defines interface between Scheduler (which owns the pending work) and the Dispatcher which periodically selects work to commence based on a policy + type IDispatcher<'P, 'R, 'E, 'F> = + abstract member HasCapacity : bool with get + abstract member AwaitCapacity : unit -> Task + abstract member TryReplenish : pending : seq> * markStreamBusy : (FsCodec.StreamName -> unit) -> struct (bool * bool) + [] abstract member Result : IEvent)> + abstract member InterpretProgress : StreamStates<'F> * FsCodec.StreamName * Choice<'P, 'E> -> struct (int64 voption * Choice<'R, 'E>) + abstract member RecordResultStats : InternalMessage> -> unit + abstract member RecordStats : int -> unit + abstract member RecordState : BufferState * StreamStates<'F> * int * Timers -> bool + + /// Implementation of IDispatcher that feeds items to an item dispatcher that maximizes concurrent requests (within a limit) + type MultiDispatcher<'P, 'R, 'E, 'F> + ( inner : Dispatch.ItemDispatcher, 'F>, + project : int64 -> Dispatch.Item<'F> -> CancellationToken -> Task)>, + interpretProgress : StreamStates<'F> -> FsCodec.StreamName -> Choice<'P, 'E> -> struct (int64 voption * Choice<'R, 'E>), + stats : Stats<'R, 'E>, + dumpState : ((FsCodec.ITimelineEvent<'F> -> int) -> unit) -> ILogger -> unit) = + static member Create(inner, + project : struct (FsCodec.StreamName * StreamSpan<'F>) -> CancellationToken -> Task)>, + interpretProgress, stats, dumpStreams) = + let project sw (item : Dispatch.Item<'F>) (ct : CancellationToken) = task { + let! progressed, res = project (item.stream, item.span) ct + let now = System.Diagnostics.Stopwatch.GetTimestamp() + return struct (Stats.Busy.timeSpanFromStopwatchTicks (now - sw), item.stream, progressed, res) } + MultiDispatcher<_, _, _, _>(inner, project, interpretProgress, stats, dumpStreams) + static member Create(inner, handle, interpret, toIndex, stats, dumpStreams) = + let project item ct = task { + let struct (met, (struct (_sn, span : StreamSpan<'F>) as ss)) = interpret item + try let! struct (spanResult, outcome) = handle ss |> fun f -> Async.StartAsTask(f, cancellationToken = ct) + let index' = toIndex span spanResult + return struct (index' > span[0].Index, Choice1Of2 struct (index', met, outcome)) + with e -> return struct (false, Choice2Of2 struct (met, e)) } + let interpretProgress (_streams : StreamStates<'F>) _stream = function + | Choice1Of2 struct (index, met, outcome) -> struct (ValueSome index, Choice1Of2 struct (met, outcome)) + | Choice2Of2 struct (stats, exn) -> ValueNone, Choice2Of2 struct (stats, exn) + MultiDispatcher<_, _, _, 'F>.Create(inner, project, interpretProgress, stats, dumpStreams) + interface IDispatcher<'P, 'R, 'E, 'F> with + override _.HasCapacity = inner.HasCapacity + override _.AwaitCapacity() = inner.AwaitCapacity() + override _.TryReplenish(pending, markStreamBusy) = inner.TryReplenish(pending, stats.MarkStarted, project, markStreamBusy) + [] override _.Result = inner.Result + override _.InterpretProgress(streams, stream, res) = interpretProgress streams stream res + override _.RecordResultStats msg = stats.Handle msg + override _.RecordStats pendingCount = + if stats.RecordStats() then + stats.Serialize(fun () -> stats.DumpStats(inner.State, pendingCount)) + override _.RecordState(dispatcherState, streams, totalPurged, timers) = + if stats.RecordState(dispatcherState, timers) then + let log = stats.Log + let dumpStreamStates (eventSize : FsCodec.ITimelineEvent<'F> -> int) = streams.Dump(log, totalPurged, eventSize) + dumpState dumpStreamStates log + true + else false + + /// Implementation of IDispatcher that allows a supplied handler select work and declare completion based on arbitrarily defined criteria + type BatchedDispatcher<'F> + ( select : Dispatch.Item<'F> seq -> Dispatch.Item<'F> array, + handle : Dispatch.Item<'F> array -> CancellationToken -> + Task) array>, + stats : Stats<_, _>, + dumpState) = + let dop = Dispatch.DopDispatcher 1 + let result = Event)>() + + // On each iteration, we offer the ordered work queue to the selector + // we propagate the selected streams to the handler + let trySelect (potential : seq>) markBusy = + let mutable hasCapacity, dispatched = true, false + let streams : Dispatch.Item<'F>[] = select potential + let succeeded = (not << Array.isEmpty) streams if succeeded then - markBusy item.stream - markStarted (item.stream, ts) - hasCapacity <- succeeded - dispatched <- dispatched || succeeded // if we added any request, we'll skip sleeping - struct (dispatched, hasCapacity) - - member _.Pump ct = inner.Pump ct - [] member _.Result = inner.Result - member _.State = inner.State - member _.TryReplenish(pending, markStarted, project, markStreamBusy) = - tryFillDispatcher (pending, markStarted) project markStreamBusy - member _.HasCapacity = inner.HasCapacity - member _.AwaitCapacity() = inner.AwaitButRelease() - - /// Defines interface between Scheduler (which owns the pending work) and the Dispatcher which periodically selects work to commence based on a policy - type IDispatcher<'P, 'R, 'E> = - abstract member HasCapacity : bool with get - abstract member AwaitCapacity : unit -> Task - abstract member TryReplenish : pending : seq> * markStreamBusy : (FsCodec.StreamName -> unit) -> struct (bool * bool) - [] abstract member Result : IEvent> - abstract member InterpretProgress : StreamStates * FsCodec.StreamName * Choice<'P, 'E> -> int64 voption * Choice<'R, 'E> - abstract member RecordResultStats : InternalMessage> -> unit - abstract member DumpStats : int -> unit - abstract member TryDumpState : BufferState * struct (StreamStates * int) * struct (TimeSpan * TimeSpan * TimeSpan * TimeSpan * TimeSpan * TimeSpan) -> bool - - /// Implementation of IDispatcher that feeds items to an item dispatcher that maximizes concurrent requests (within a limit) - type MultiDispatcher<'P, 'R, 'E> - ( inner : ItemDispatcher>, - project : int64 -> DispatchItem -> CancellationToken -> Task>, - interpretProgress : StreamStates -> FsCodec.StreamName -> Choice<'P, 'E> -> int64 voption * Choice<'R, 'E>, - stats : Stats<'R, 'E>, - dumpStreams) = - static member Create(inner, project : _ * _ -> CancellationToken -> Task>, interpretProgress, stats, dumpStreams) = - let project sw (item : DispatchItem) (ct : CancellationToken) (*: Task>*) = task { - let! progressed, res = project (item.stream, item.span) ct - let now = System.Diagnostics.Stopwatch.GetTimestamp() - return Busy.timeSpanFromStopwatchTicks (now - sw), item.stream, progressed, res } - MultiDispatcher(inner, project, interpretProgress, stats, dumpStreams) - static member Create(inner, handle, interpret, toIndex, stats, dumpStreams) = - let project item ct = task { - let met, (stream, span) = interpret item - try let! spanResult, outcome = handle (stream, span) |> fun f -> Async.StartAsTask(f, cancellationToken = ct) - let index' = toIndex (stream, span) spanResult - return index' > span.index, Choice1Of2 (index', met, outcome) - with e -> return false, Choice2Of2 (met, e) } - let interpretProgress (_streams : StreamStates<_>) _stream = function - | Choice1Of2 (index, met, outcome) -> ValueSome index, Choice1Of2 (met, outcome) - | Choice2Of2 (stats, exn) -> ValueNone, Choice2Of2 (stats, exn) - MultiDispatcher<_, _, _>.Create(inner, project, interpretProgress, stats, dumpStreams) - - interface IDispatcher<'P, 'R, 'E> with - override _.HasCapacity = inner.HasCapacity - override _.AwaitCapacity() = inner.AwaitCapacity() - override _.TryReplenish(pending, markStreamBusy) = inner.TryReplenish(pending, stats.MarkStarted, project, markStreamBusy) - [] override _.Result = inner.Result - override _.InterpretProgress(streams : StreamStates<_>, stream : FsCodec.StreamName, res : Choice<'P, 'E>) = - interpretProgress streams stream res - override _.RecordResultStats msg = stats.Handle msg - override _.DumpStats pendingCount = stats.DumpStats(inner.State, pendingCount) - override _.TryDumpState(dispatcherState, streams, (dt, ft, mt, it, st, zt)) = - stats.TryDumpState(dispatcherState, dumpStreams streams, (dt, ft, mt, it, st, zt)) - - /// Implementation of IDispatcher that allows a supplied handler select work and declare completion based on arbitrarily defined criteria - type BatchedDispatcher - ( select : DispatchItem seq -> DispatchItem[], - handle : DispatchItem[] -> CancellationToken -> Task<(TimeSpan * FsCodec.StreamName * bool * Choice)[]>, - stats : Stats<_, _>, - dumpStreams) = - let dop = DopDispatcher 1 - let result = Event>() - - // On each iteration, we offer the ordered work queue to the selector - // we propagate the selected streams to the handler - let trySelect (potential : seq>) markBusy = - let mutable hasCapacity, dispatched = true, false - let streams : DispatchItem[] = select potential - let succeeded = (not << Array.isEmpty) streams - if succeeded then - let res = dop.TryAdd(handle streams) - if not res then failwith "Checked we can add, what gives?" - for x in streams do - markBusy x.stream - dispatched <- true // if we added any request, we'll skip sleeping - hasCapacity <- false - struct (dispatched, hasCapacity) - - member _.Pump ct = task { - use _ = dop.Result.Subscribe(Array.iter result.Trigger) - return! dop.Pump ct } - - interface IDispatcher with - override _.HasCapacity = dop.HasCapacity - override _.AwaitCapacity() = dop.AwaitButRelease() - override _.TryReplenish(pending, markStreamBusy) = trySelect pending markStreamBusy - [] override _.Result = result.Publish - override _.InterpretProgress(_streams : StreamStates<_>, _stream : FsCodec.StreamName, res : Choice<_, _>) = - match res with - | Choice1Of2 (pos', (stats, outcome)) -> ValueSome pos', Choice1Of2 (stats, outcome) - | Choice2Of2 (stats, exn) -> ValueNone, Choice2Of2 (stats, exn) - override _.RecordResultStats msg = stats.Handle msg - override _.DumpStats pendingCount = stats.DumpStats(dop.State, pendingCount) - override _.TryDumpState(dispatcherState, streams, (dt, ft, mt, it, st, zt)) = - stats.TryDumpState(dispatcherState, dumpStreams streams, (dt, ft, mt, it, st, zt)) - - [] - type StreamsBatch<'Format> private (onCompletion, buffer, reqs) = - let mutable buffer = Some buffer - static member Create(onCompletion, items : StreamEvent<'Format> seq) = - let buffer, reqs = Streams<'Format>(), Dictionary() - let mutable itemCount = 0 - for item in items do - itemCount <- itemCount + 1 - buffer.Merge(item) - match reqs.TryGetValue(item.stream), item.event.Index + 1L with - | (false, _), required -> reqs[item.stream] <- required - | (true, actual), required when actual < required -> reqs[item.stream] <- required - | (true, _), _ -> () // replayed same or earlier item - let batch = StreamsBatch(onCompletion, buffer, reqs) - batch, (batch.RemainingStreamsCount, itemCount) - - member _.OnCompletion = onCompletion - member _.Reqs = reqs :> seq> - member _.RemainingStreamsCount = reqs.Count - member _.TryTakeStreams() = let t = buffer in buffer <- None; t - member _.TryMerge(other : StreamsBatch<_>) = - match buffer, other.TryTakeStreams() with - | Some x, Some y -> x.Merge(y); true - | Some _, None -> false - | None, x -> buffer <- x; false + let res = dop.TryAdd(handle streams) + if not res then failwith "Checked we can add, what gives?" + for x in streams do + markBusy x.stream + dispatched <- true // if we added any request, we'll skip sleeping + hasCapacity <- false + struct (dispatched, hasCapacity) + + member _.Pump ct = task { + use _ = dop.Result.Subscribe(Array.iter result.Trigger) + return! dop.Pump ct } + + interface IDispatcher with + override _.HasCapacity = dop.HasCapacity + override _.AwaitCapacity() = dop.AwaitButRelease() + override _.TryReplenish(pending, markStreamBusy) = trySelect pending markStreamBusy + [] override _.Result = result.Publish + override _.InterpretProgress(_streams : StreamStates<_>, _stream : FsCodec.StreamName, res : Choice<_, _>) = + match res with + | Choice1Of2 (pos', (stats, outcome)) -> ValueSome pos', Choice1Of2 (stats, outcome) + | Choice2Of2 (stats, exn) -> ValueNone, Choice2Of2 (stats, exn) + override _.RecordResultStats msg = stats.Handle msg + override _.RecordStats pendingCount = + if stats.RecordStats() then + stats.DumpStats(dop.State, pendingCount) + override _.RecordState(dispatcherState, streams, totalPurged, timers) = + if stats.RecordState(dispatcherState, timers) then + let log = stats.Log + let dumpStreamStates (eventSize : FsCodec.ITimelineEvent<'F> -> int) = streams.Dump(log, totalPurged, eventSize) + dumpState dumpStreamStates log + true + else false + + module Progress = + + type [] internal BatchState = { markCompleted : unit -> unit; streamToRequiredIndex : Dictionary } + + type ProgressState<'Pos>() = + let pending = Queue() + let trim () = + while pending.Count <> 0 && pending.Peek().streamToRequiredIndex.Count = 0 do + let batch = pending.Dequeue() + batch.markCompleted() + + // We potentially traverse the pending streams thousands of times per second + // so we reuse `InScheduledOrder`'s temps: sortBuffer and streamsBuffer for better L2 caching properties + let sortBuffer = ResizeArray() + let streamsBuffer = HashSet() + + member _.AppendBatch(markCompleted, reqs : Dictionary) = + pending.Enqueue { markCompleted = markCompleted; streamToRequiredIndex = reqs } + trim () + + member _.MarkStreamProgress(stream, index) = + let mutable requiredIndex = Unchecked.defaultof<_> + for x in pending do + if x.streamToRequiredIndex.TryGetValue(stream, &requiredIndex) && requiredIndex <= index then + x.streamToRequiredIndex.Remove stream |> ignore + trim () + + member _.IsEmpty = pending.Count = 0 + // NOTE internal reuse of `sortBuffer` and `streamsBuffer` means it's critical to never have >1 of these in flight + member _.InScheduledOrder(getStreamWeight : (FsCodec.StreamName -> int64) option) : seq = + trim () + // sortBuffer is used once per invocation, but the result is lazy so we can only clear it on entry + sortBuffer.Clear() + let mutable batch = 0 + let weight = match getStreamWeight with None -> (fun _s -> 0) | Some f -> (fun s -> -f s |> int) + for x in pending do + for s in x.streamToRequiredIndex.Keys do + if streamsBuffer.Add s then + // Within the head batch, expedite work on longest streams, where requested + let w = if batch = 0 then weight s else 0 // For anything other than the head batch, submitted order is just fine + sortBuffer.Add(struct (s, batch, w)) + if batch = 0 && sortBuffer.Count > 0 then + let c = Comparer<_>.Default + sortBuffer.Sort(fun struct (_, _ab, _aw) struct (_, _bb, _bw) -> c.Compare(struct(_ab, _aw), struct(_bb, _bw))) + batch <- batch + 1 + // We reuse this buffer next time around, but clear it now as it has no further use + streamsBuffer.Clear() + sortBuffer |> Seq.map (fun struct(s, _, _) -> s) + + (* These become individually allocated FSharpRef cells that get allocated for every cycle; until https://github.com/fsharp/fslang-suggestions/issues/732 we do this... *) + type [] private State = + { mutable idle : bool; mutable dispatcherState : BufferState; mutable remaining : int; mutable waitForCapacity : bool; mutable waitForPending : bool } + member x.IsSlipStreaming = x.dispatcherState = Slipstreaming /// Consolidates ingested events into streams; coordinates dispatching of these to projector/ingester in the order implied by the submission order /// a) does not itself perform any reading activities /// b) triggers synchronous callbacks as batches complete; writing of progress is managed asynchronously by the TrancheEngine(s) /// c) submits work to the supplied Dispatcher (which it triggers pumping of) /// d) periodically reports state (with hooks for ingestion engines to report same) - type StreamSchedulingEngine<'P, 'R, 'E> - ( dispatcher : IDispatcher<'P, 'R, 'E>, + type StreamSchedulingEngine<'P, 'R, 'E, 'F> + ( dispatcher : Dispatcher.IDispatcher<'P, 'R, 'E, 'F>, // Tune number of batches to ingest at a time. Default 1. ?maxBatches, // Tune the max number of check/dispatch cycles. Default 2. @@ -893,39 +797,38 @@ module Scheduling = ?idleDelay, // Tune the number of times to attempt handling results before sleeping. Default 3. ?maxCycles, - // Prioritize processing of the largest Payloads (in bytes) when scheduling work + // Prioritize processing of the largest Payloads within the Tip batch when scheduling work // Can yield significant throughput improvement when ingesting large batches in the face of rate limiting - ?prioritizeLargePayloads, + ?prioritizeStreamsBy, // Opt-in to allowing items to be processed independent of batch sequencing - requires upstream/projection function to be able to identify gaps. Default false. ?enableSlipstreaming) = - let purgeDue = purgeInterval |> Option.map intervalCheck + let purgeDue = + match purgeInterval with + | Some x -> let i = IntervalTimer x + i.IfDueRestart + | None -> fun () -> false let sleepIntervalMs = let idleDelay = defaultArg idleDelay (TimeSpan.FromSeconds 1.) int idleDelay.TotalMilliseconds let wakeForResults = defaultArg wakeForResults false let maxCycles, maxBatches, slipstreamingEnabled = defaultArg maxCycles 3, defaultArg maxBatches 1, defaultArg enableSlipstreaming false - let writeResult, awaitResults, tryApplyResults = let c = Channel.unboundedSr in Channel.write c >> ignore, Channel.awaitRead c, Channel.apply c + let writeResult, awaitResults, tryApplyResults = + let r, w = let c = Channel.unboundedSr in c.Reader, c.Writer + Channel.write w >> ignore, Channel.awaitRead r, Channel.apply r let writePending, pendingCount, awaitPending, tryReadPending = - let c = Channel.unboundedSw> // Actually SingleReader too, but Count throws if you use that - Channel.write c >> ignore, (fun () -> c.Reader.Count), Channel.awaitRead c, Channel.tryRead c - let streams = StreamStates() + let r, w = let c = Channel.unboundedSw> in c.Reader, c.Writer // Actually SingleReader too, but Count throws if you use that + Channel.write w >> ignore, (fun () -> r.Count), Channel.awaitRead r, Channel.tryRead r + let streams = StreamStates<'F>() + let weight f stream = streams.HeadSpanSizeBy(stream, f) + let maybePrioritizeLargePayloads = match prioritizeStreamsBy with Some f -> Some (weight f) | None -> None let progressState = Progress.ProgressState() let mutable totalPurged = 0 - let weight stream = - match streams.TryGetItem stream with - | ValueSome state when not state.IsEmpty -> - let firstSpan = Array.head state.Queue - let mutable acc = 0 - for x in firstSpan.events do acc <- acc + dataSize x - int64 acc - | _ -> 0L - let maybePrioritizeLargePayloads = if defaultArg prioritizeLargePayloads false then Some weight else None - let tryDispatch isSlipStreaming () = + let tryDispatch isSlipStreaming = let hasCapacity = dispatcher.HasCapacity if not hasCapacity || (progressState.IsEmpty && not isSlipStreaming) then struct (false, hasCapacity) else - let pending : seq> = streams.Pending(isSlipStreaming, progressState.InScheduledOrder maybePrioritizeLargePayloads) + let pending : seq> = streams.Pending(isSlipStreaming, progressState.InScheduledOrder maybePrioritizeLargePayloads) dispatcher.TryReplenish(pending, streams.MarkBusy) // ingest information to be gleaned from processing the results into `streams` @@ -945,7 +848,8 @@ module Scheduling = | _, Choice2Of2 exn -> streams.MarkFailed(stream) Result (duration, stream, progressed, Choice2Of2 exn) - let tryHandleResults () = tryApplyResults (mapResult >> dispatcher.RecordResultStats) + let mapRecord = mapResult >> dispatcher.RecordResultStats + let tryHandleResults () = tryApplyResults mapRecord // Take an incoming batch of events, correlating it against our known stream state to yield a set of remaining work let ingestPendingBatch feedStats (markCompleted, items : seq>) = @@ -959,115 +863,110 @@ module Scheduling = reqs[item.Key] <- item.Value progressState.AppendBatch(markCompleted, reqs) feedStats <| Added (reqs.Count, skipCount, count) - let ingestBatch (batch : StreamsBatch<_>) () = ingestPendingBatch dispatcher.RecordResultStats (batch.OnCompletion, batch.Reqs) - - let maybePurge () = - // After we've dumped the state, it may also be due for pruning - match purgeDue with - | Some dueNow when dueNow () -> - let remaining, purged = streams.Purge() - totalPurged <- totalPurged + purged - let l = if purged = 0 then Events.LogEventLevel.Debug else Events.LogEventLevel.Information - Log.Write(l, "PURGED Remaining {buffered:n0} Purged now {count:n0} Purged total {total:n0}", remaining, purged, totalPurged) - | _ -> () + let ingestBatch (batch : Batch<_>) = ingestPendingBatch dispatcher.RecordResultStats (batch.OnCompletion, batch.Reqs) + let ingestBatches mergeStreams ingestBatch = + let rec aux batchesRemaining = + match tryReadPending () with + | ValueSome batch -> + // Accommodate where stream-wise merges have been performed preemptively in the ingester + batch.TryTakeStreams() |> ValueOption.iter mergeStreams + ingestBatch batch + match batchesRemaining - 1 with + | 0 -> struct (true, true) // no need to wait, we filled the capacity + | r -> aux r + | ValueNone -> + if batchesRemaining <> maxBatches then true, false // already added some items, so we don't need to wait for pending + else false, true + aux + + let purge () = + let remaining, purged = streams.Purge() + totalPurged <- totalPurged + purged + let l = if purged = 0 then Events.LogEventLevel.Debug else Events.LogEventLevel.Information + Log.Write(l, "PURGED Remaining {buffered:n0} Purged now {count:n0} Purged total {total:n0}", remaining, purged, totalPurged) member _.Pump _abend (ct : CancellationToken) = task { - use _ = dispatcher.Result.Subscribe(Result >> writeResult) - let mutable zt = TimeSpan.Zero + use _ = dispatcher.Result.Subscribe(fun struct (t, s, pr, r) -> writeResult (Result (t, s, pr, r))) + let inline startSw () = System.Diagnostics.Stopwatch.StartNew() while not ct.IsCancellationRequested do - let mutable idle, dispatcherState, remaining = true, Idle, maxCycles - let mutable dt, ft, mt, it, st = TimeSpan.Zero, TimeSpan.Zero, TimeSpan.Zero, TimeSpan.Zero, TimeSpan.Zero - let tryIngestMaxBatches () = - // If we're going to fill the write queue with random work, we should bring all read events into the state first - // Hence we potentially take more than one batch at a time based on maxBatches (but less buffered work is more optimal) - let mutable more, batchesTaken, ok = true, 0, true - while more do - match tryReadPending () with - | ValueSome batch -> - // Accommodate where stream-wise merges have been performed preemptively in the ingester - match batch.TryTakeStreams() with - | Some s -> (fun () -> streams.InternalMerge(s)) |> accStopwatch <| fun t -> mt <- mt + t - | None -> () - ingestBatch batch |> accStopwatch <| fun t -> it <- it + t - batchesTaken <- batchesTaken + 1 - more <- batchesTaken < maxBatches - | ValueNone -> - more <- false - if batchesTaken <> 0 then () - elif slipstreamingEnabled then dispatcherState <- Slipstreaming - else remaining <- 0; ok <- false - ok - let mutable waitForPending, waitForCapacity = false, false - while remaining <> 0 do - remaining <- remaining - 1 + let mutable t = Unchecked.defaultof + let mutable s = { idle = true; dispatcherState = Idle; remaining = maxCycles; waitForPending = false; waitForCapacity = false } + while s.remaining <> 0 do + s.remaining <- s.remaining - 1 // 1. propagate write write outcomes to buffer (can mark batches completed etc) - let processedResults = tryHandleResults |> accStopwatch <| fun x -> dt <- dt + x + let processedResults = let sw = startSw () in let r = tryHandleResults () in t.AddResults sw.Elapsed; r // 2. top up provisioning of writers queue // On each iteration, we try to fill the in-flight queue, taking the oldest and/or heaviest streams first - let struct (dispatched, hasCapacity) = tryDispatch (dispatcherState = Slipstreaming) |> accStopwatch <| fun x -> ft <- ft + x - idle <- idle && not processedResults && not dispatched - match dispatcherState with + let struct (dispatched, hasCapacity) = let sw = startSw () in let r = tryDispatch s.IsSlipStreaming in t.AddDispatch sw.Elapsed; r + s.idle <- s.idle && not processedResults && not dispatched + match s.dispatcherState with | Idle when not hasCapacity -> // If we've achieved full state, spin around the loop to dump stats and ingest reader data - dispatcherState <- Full - remaining <- 0 - | Idle when remaining = 0 -> - dispatcherState <- Active + s.dispatcherState <- Full + s.remaining <- 0 + | Idle when s.remaining = 0 -> + s.dispatcherState <- Active | Idle -> // need to bring more work into the pool as we can't fill the work queue from what we have - waitForPending <- not (tryIngestMaxBatches ()) + // If we're going to fill the write queue with random work, we should bring all read events into the state first + // Hence we potentially take more than one batch at a time based on maxBatches (but less buffered work is more optimal) + let mergeStreams batchStreams = let sw = startSw () in streams.Merge batchStreams; t.AddMerge sw.Elapsed + let ingestBatch batch = let sw = startSw () in ingestBatch batch; t.AddIngest sw.Elapsed + let struct (ingested, filled) = ingestBatches mergeStreams ingestBatch maxBatches + if ingested then s.waitForPending <- not filled // no need to wait if we ingested as many as needed + elif slipstreamingEnabled then s.dispatcherState <- Slipstreaming; s.waitForPending <- true // try some slip-streaming, but wait for proper items too + else s.remaining <- 0; s.waitForPending <- true // definitely need to wait as there were no items | Slipstreaming -> // only do one round of slipstreaming - remaining <- 0 + s.remaining <- 0 | Active | Full -> failwith "Not handled here" - if remaining = 0 && hasCapacity then waitForPending <- true - if remaining = 0 && not hasCapacity && not wakeForResults then waitForCapacity <- true + if s.remaining = 0 && hasCapacity then s.waitForPending <- true + if s.remaining = 0 && not hasCapacity && not wakeForResults then s.waitForCapacity <- true // While the loop can take a long time, we don't attempt logging of stats per iteration on the basis that the maxCycles should be low - (fun () -> dispatcher.DumpStats(pendingCount())) |> accStopwatch <| fun t -> st <- st + t - // 3. Record completion state once per full iteration; dumping streams is expensive so needs to be done infrequently - let dumped = dispatcher.TryDumpState(dispatcherState, (streams, totalPurged), (dt, ft, mt, it, st, zt)) - zt <- TimeSpan.Zero - if dumped then maybePurge () - elif idle then + let sw = startSw () in dispatcher.RecordStats(pendingCount()); t.AddStats sw.Elapsed + let sleepSw = System.Diagnostics.Stopwatch.StartNew() + if s.idle then // 4. Do a minimal sleep so we don't run completely hot when empty (unless we did something non-trivial) let wakeConditions : Task array = [| if wakeForResults then awaitResults ct - elif waitForCapacity then dispatcher.AwaitCapacity() - if waitForPending then awaitPending ct + elif s.waitForCapacity then dispatcher.AwaitCapacity() + if s.waitForPending then awaitPending ct Task.Delay(int sleepIntervalMs) |] - let sw = System.Diagnostics.Stopwatch.StartNew() do! Task.WhenAny(wakeConditions) :> Task - zt <- sw.Elapsed } + // 3. Record completion state once per full iteration; dumping streams is expensive so needs to be done infrequently + t.AddSleep sleepSw.Elapsed + if dispatcher.RecordState(s.dispatcherState, streams, totalPurged, t) && purgeDue () then + purge () } - member _.Submit(x : StreamsBatch<_>) = + member _.Submit(x : Batch<_>) = writePending x type StreamSchedulingEngine = - static member Create<'Metrics, 'Progress, 'Outcome> - ( itemDispatcher : ItemDispatcher>, - stats : Stats<'Metrics * 'Outcome, 'Metrics * exn>, - prepare : FsCodec.StreamName * StreamSpan<_> -> 'Metrics * (FsCodec.StreamName * StreamSpan<_>), - handle : FsCodec.StreamName * StreamSpan<_> -> Async<'Progress * 'Outcome>, - toIndex : FsCodec.StreamName * StreamSpan<_> -> 'Progress -> int64, + static member Create<'Metrics, 'Progress, 'Outcome, 'F> + ( itemDispatcher : Dispatch.ItemDispatcher, 'F>, + stats : Stats, + prepare : struct (FsCodec.StreamName * FsCodec.ITimelineEvent<'F> array) -> struct ('Metrics * struct (FsCodec.StreamName * FsCodec.ITimelineEvent<'F> array)), + handle : struct (FsCodec.StreamName * FsCodec.ITimelineEvent<'F> array) -> Async, + toIndex : StreamSpan<'F> -> 'Progress -> int64, dumpStreams, ?maxBatches, ?purgeInterval, ?wakeForResults, ?idleDelay, ?enableSlipstreaming) - : StreamSchedulingEngine = + : StreamSchedulingEngine = - let dispatcher = MultiDispatcher<_, _, _>.Create(itemDispatcher, handle, prepare, toIndex, stats, dumpStreams) - StreamSchedulingEngine<_, _, _>( + let dispatcher = Dispatcher.MultiDispatcher<_, _, _, 'F>.Create(itemDispatcher, handle, prepare, toIndex, stats, dumpStreams) + StreamSchedulingEngine<_, _, _, 'F>( dispatcher, ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay, ?enableSlipstreaming = enableSlipstreaming) static member Create(dispatcher, ?maxBatches, ?purgeInterval, ?wakeForResults, ?idleDelay, ?enableSlipstreaming) - : StreamSchedulingEngine = - StreamSchedulingEngine<_, _, _>( + : StreamSchedulingEngine = + StreamSchedulingEngine<_, _, _, 'F>( dispatcher, ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay, ?enableSlipstreaming = enableSlipstreaming) [] type Stats<'Outcome>(log : ILogger, statsInterval, statesInterval) = - inherit Scheduling.Stats(log, statsInterval, statesInterval) - let okStreams, failStreams, badCats = HashSet(), HashSet(), CatStats() + inherit Scheduling.Stats(log, statsInterval, statesInterval) + let okStreams, failStreams, badCats = HashSet(), HashSet(), Stats.CatStats() let mutable resultOk, resultExnOther, okEvents, okBytes, exnEvents, exnBytes = 0, 0, 0, 0L, 0, 0L override _.DumpStats() = @@ -1105,22 +1004,23 @@ module Projector = type StreamsIngester = static member Start(log, partitionId, maxRead, submit, statsInterval) = - let makeBatch onCompletion (items : StreamEvent<_> seq) = + let submitBatch (items : StreamEvent<_> seq, onCompletion) = let items = Array.ofSeq items - let streams = HashSet(seq { for x in items -> x.stream }) - let batch : Submission.SubmissionBatch<_, _> = { source = partitionId; onCompletion = onCompletion; messages = items } - batch, (streams.Count, items.Length) - Ingestion.Ingester seq, Submission.SubmissionBatch<_, StreamEvent<_>>>.Start(log, partitionId, maxRead, makeBatch, submit, statsInterval) + let streams = items |> Seq.map ValueTuple.fst |> HashSet + let batch : Submission.Batch<_, _> = { source = partitionId; onCompletion = onCompletion; messages = items } + submit batch + struct (streams.Count, items.Length) + Ingestion.Ingester seq>.Start(log, partitionId, maxRead, submitBatch, statsInterval) type StreamsSubmitter = static member Create ( log : ILogger, maxSubmissionsPerPartition, mapBatch, submitStreamsBatch, statsInterval, ?disableCompaction) = - let submitBatch (x : Scheduling.StreamsBatch<_>) : int = + let submitBatch (x : Buffer.Batch<_>) : int = submitStreamsBatch x x.RemainingStreamsCount - let tryCompactQueueImpl (queue : Queue>) = + let tryCompactQueueImpl (queue : Queue>) = let mutable acc, worked = None, false for x in queue do match acc with @@ -1130,28 +1030,28 @@ module Projector = let tryCompactQueue = if defaultArg disableCompaction false then None else Some tryCompactQueueImpl Submission.SubmissionEngine<_, _, _>(log, maxSubmissionsPerPartition, mapBatch, submitBatch, statsInterval, ?tryCompactQueue=tryCompactQueue) - type StreamsProjectorPipeline = + type Pipeline = static member Start ( log : ILogger, pumpDispatcher, pumpScheduler, maxReadAhead, submitStreamsBatch, statsInterval, // Limits number of batches passed to the scheduler. // Holding items back makes scheduler processing more efficient as less state needs to be traversed. - // Holding items back is also key to the submitter's compaction mechanism working best. + // Holding items back is also key to enabling the compaction mechanism in the submitter to take effect. // Defaults to holding back 20% of maxReadAhead per partition ?maxSubmissionsPerPartition, ?ingesterStatsInterval) = let ingesterStatsInterval = defaultArg ingesterStatsInterval statsInterval - let mapBatch onCompletion (x : Submission.SubmissionBatch<_, StreamEvent<_>>) : Scheduling.StreamsBatch<_> = + let mapBatch onCompletion (x : Submission.Batch<_, StreamEvent<'F>>) : Buffer.Batch<'F> = let onCompletion () = x.onCompletion(); onCompletion() - Scheduling.StreamsBatch.Create(onCompletion, x.messages) |> fst + Buffer.Batch.Create(onCompletion, x.messages) |> fun struct (f, _s) -> f let maxSubmissionsPerPartition = defaultArg maxSubmissionsPerPartition (maxReadAhead - maxReadAhead/5) // NOTE needs to handle overflow if maxReadAhead is Int32.MaxValue let submitter = StreamsSubmitter.Create(log, maxSubmissionsPerPartition, mapBatch, submitStreamsBatch, statsInterval) let startIngester (rangeLog, projectionId) = StreamsIngester.Start(rangeLog, projectionId, maxReadAhead, submitter.Ingest, ingesterStatsInterval) - ProjectorPipeline.Start(log, pumpDispatcher, pumpScheduler, submitter.Pump, startIngester) + Sink.Start(log, pumpDispatcher, pumpScheduler, submitter.Pump, startIngester) /// Represents progress attained during the processing of the supplied StreamSpan for a given StreamName. /// This will be reflected in adjustments to the Write Position for the stream in question. -/// Incoming StreamEvents with Indexes prior to the Write Position implied by the result are proactively +/// Incoming StreamEvents with Indexes prior to the Write Position implied by the result are proactively /// dropped from incoming buffers, yielding increased throughput due to reduction of redundant processing. type SpanResult = /// Indicates no events where processed. @@ -1170,20 +1070,20 @@ type SpanResult = module SpanResult = - let toIndex (_sn, span : StreamSpan) = function - | NoneProcessed -> span.index - | AllProcessed -> span.index + span.events.LongLength - | PartiallyProcessed count -> span.index + int64 count + let toIndex<'F> (span : StreamSpan<'F>) = function + | NoneProcessed -> span[0].Index + | AllProcessed -> span[0].Index + span.LongLength + | PartiallyProcessed count -> span[0].Index + int64 count | OverrideWritePosition index -> index -type StreamsProjector = +type StreamsSink = /// Custom projection mechanism that divides work into a prepare phase that selects the prefix of the queued StreamSpan to handle, /// and a handle function that yields a Write Position representing the next event that's to be handled on this Stream - static member StartEx<'Progress, 'Outcome> + static member StartEx<'Progress, 'Outcome, 'F> ( log : ILogger, maxReadAhead, maxConcurrentStreams, prepare, handle, toIndex, - stats, statsInterval, + stats, statsInterval, eventSize, ?maxSubmissionsPerPartition, // Tune the number of batches the Scheduler should ingest at a time. Can be useful to compensate for small batches ?maxBatches, @@ -1195,24 +1095,24 @@ type StreamsProjector = // Tune the sleep time when there are no items to schedule or responses to process. Default 1s. ?idleDelay, ?ingesterStatsInterval) - : ProjectorPipeline<_> = - let dispatcher = Scheduling.ItemDispatcher<_>(maxConcurrentStreams) + : Sink seq>> = + let dispatcher = Dispatch.ItemDispatcher<_, 'F>(maxConcurrentStreams) let streamScheduler = - Scheduling.StreamSchedulingEngine.Create<_, 'Progress, 'Outcome> + Scheduling.StreamSchedulingEngine.Create<_, 'Progress, 'Outcome, 'F> ( dispatcher, stats, prepare, handle, toIndex, - (fun struct (s, totalPurged) logger -> s.Dump(logger, totalPurged, Buffering.StreamState.eventsSize)), + (fun logStreamStates _log -> logStreamStates eventSize), ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay) - Projector.StreamsProjectorPipeline.Start( - log, dispatcher.Pump, streamScheduler.Pump, maxReadAhead, streamScheduler.Submit, statsInterval, - ?maxSubmissionsPerPartition = maxSubmissionsPerPartition, - ?ingesterStatsInterval = ingesterStatsInterval) + Projector.Pipeline.Start( + log, dispatcher.Pump, streamScheduler.Pump, maxReadAhead, streamScheduler.Submit, statsInterval, + ?maxSubmissionsPerPartition = maxSubmissionsPerPartition, + ?ingesterStatsInterval = ingesterStatsInterval) /// Project StreamSpans using a handle function that yields a Write Position representing the next event that's to be handled on this Stream - static member Start<'Outcome> + static member Start<'Outcome, 'F> ( log : ILogger, maxReadAhead, maxConcurrentStreams, - handle : FsCodec.StreamName * StreamSpan<_> -> Async, - stats, statsInterval, + handle : struct (FsCodec.StreamName * FsCodec.ITimelineEvent<'F> array) -> Async, + stats, statsInterval, eventSize, // Limits number of batches passed to the scheduler. // Holding items back makes scheduler processing more efficient as less state needs to be traversed. // Holding items back is also key to the compaction mechanism working best. @@ -1228,21 +1128,21 @@ type StreamsProjector = // Tune the sleep time when there are no items to schedule or responses to process. Default 1s. ?idleDelay, ?ingesterStatsInterval) - : ProjectorPipeline<_> = - let prepare (streamName, span) = - let stats = Buffering.StreamSpan.stats span - stats, (streamName, span) - StreamsProjector.StartEx( - log, maxReadAhead, maxConcurrentStreams, prepare, handle, SpanResult.toIndex, stats, statsInterval, + : Sink seq>> = + let prepare struct (streamName, span) = + let metrics = StreamSpan.metrics eventSize span + struct (metrics, struct (streamName, span)) + StreamsSink.StartEx( + log, maxReadAhead, maxConcurrentStreams, prepare, handle, SpanResult.toIndex, stats, statsInterval, eventSize, ?maxSubmissionsPerPartition = maxSubmissionsPerPartition, ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay, ?ingesterStatsInterval = ingesterStatsInterval) /// Project StreamSpans using a handle function that guarantees to always handles all events in the span - static member Start<'Outcome> + static member Start<'Outcome, 'F> ( log : ILogger, maxReadAhead, maxConcurrentStreams, - handle : FsCodec.StreamName * StreamSpan<_> -> Async<'Outcome>, - stats, statsInterval, + handle : FsCodec.StreamName * FsCodec.ITimelineEvent<'F> array -> Async<'Outcome>, + stats, statsInterval, eventSize, // Limits number of batches passed to the scheduler. // Holding items back makes scheduler processing more efficient as less state needs to be traversed. // Holding items back is also key to the compaction mechanism working best. @@ -1258,12 +1158,12 @@ type StreamsProjector = // Tune the sleep time when there are no items to schedule or responses to process. Default 1s. ?idleDelay, ?ingesterStatsInterval) - : ProjectorPipeline<_> = - let handle (streamName, span : StreamSpan<_>) = async { + : Sink seq>> = + let handle struct (streamName, span : FsCodec.ITimelineEvent<'F> array) = async { let! res = handle (streamName, span) - return SpanResult.AllProcessed, res } - StreamsProjector.Start<'Outcome>( - log, maxReadAhead, maxConcurrentStreams, handle, stats, statsInterval, + return struct (SpanResult.AllProcessed, res) } + StreamsSink.Start<'Outcome, 'F>( + log, maxReadAhead, maxConcurrentStreams, handle, stats, statsInterval, eventSize, ?maxSubmissionsPerPartition = maxSubmissionsPerPartition, ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, ?wakeForResults = wakeForResults, ?idleDelay = idleDelay, ?ingesterStatsInterval = ingesterStatsInterval) @@ -1272,9 +1172,9 @@ module Sync = [] type Stats<'Outcome>(log : ILogger, statsInterval, stateInterval) = - inherit Scheduling.Stats<(EventMetrics * TimeSpan) * 'Outcome, EventMetrics * exn>(log, statsInterval, stateInterval) + inherit Scheduling.Stats(log, statsInterval, stateInterval) let okStreams, failStreams = HashSet(), HashSet() - let prepareStats = LatencyStats("prepare") + let prepareStats = Stats.LatencyStats("prepare") let mutable okEvents, okBytes, exnEvents, exnBytes = 0, 0L, 0, 0L override _.DumpStats() = @@ -1307,15 +1207,15 @@ module Sync = static member Start ( log : ILogger, maxReadAhead, maxConcurrentStreams, - handle : FsCodec.StreamName * StreamSpan<_> -> Async, - stats : Stats<'Outcome>, statsInterval, + handle : struct (FsCodec.StreamName * FsCodec.ITimelineEvent<'F> array) -> Async, + stats : Stats<'Outcome>, statsInterval, sliceSize, eventSize, // Default 1 ms ?idleDelay, // Default 1 MiB ?maxBytes, // Default 16384 ?maxEvents, - // Max scheduling readahead. Default 128. + // Max scheduling read ahead. Default 128. ?maxBatches, // Max inner cycles per loop. Default 128. ?maxCycles, @@ -1324,36 +1224,67 @@ module Sync = // Frequency of jettisoning Write Position state of inactive streams (held by the scheduler for deduplication purposes) to limit memory consumption // NOTE: Purging can impair performance, increase write costs or result in duplicate event emissions due to redundant inputs not being deduplicated ?purgeInterval) - : ProjectorPipeline<_> = + : Sink seq>> = let maxBatches, maxEvents, maxBytes = defaultArg maxBatches 128, defaultArg maxEvents 16384, (defaultArg maxBytes (1024 * 1024 - (*fudge*)4096)) - let attemptWrite (stream, span) ct = task { - let met, span' = Buffering.StreamSpan.slice (maxEvents, maxBytes) span + let attemptWrite struct (stream, span : FsCodec.ITimelineEvent<'F> array) ct = task { + let struct (met, span') = StreamSpan.slice<'F> sliceSize (maxEvents, maxBytes) span let sw = System.Diagnostics.Stopwatch.StartNew() - try let req = (stream, span') + try let req = struct (stream, span') let! res, outcome = Async.StartAsTask(handle req, cancellationToken = ct) let prepareElapsed = sw.Elapsed - let index' = SpanResult.toIndex req res - return index' > span.index, Choice1Of2 (index', (met, prepareElapsed), outcome) - with e -> return false, Choice2Of2 (met, e) } + let index' = SpanResult.toIndex span' res + return struct (index' > span[0].Index, Choice1Of2 struct (index', struct (met, prepareElapsed), outcome)) + with e -> return struct (false, Choice2Of2 struct (met, e)) } let interpretWriteResultProgress _streams (stream : FsCodec.StreamName) = function - | Choice1Of2 (i', stats, outcome) -> - ValueSome i', Choice1Of2 (stats, outcome) - | Choice2Of2 (eventCount, bytesCount as stats, exn : exn) -> + | Choice1Of2 struct (i', stats, outcome) -> + struct (ValueSome i', Choice1Of2 struct (stats, outcome)) + | Choice2Of2 struct (struct (eventCount, bytesCount) as stats, exn : exn) -> log.Warning(exn, "Handling {events:n0}e {bytes:n0}b for {stream} failed, retrying", eventCount, bytesCount, stream) - ValueNone, Choice2Of2 (stats, exn) + ValueNone, Choice2Of2 struct (stats, exn) - let itemDispatcher = Scheduling.ItemDispatcher<_>(maxConcurrentStreams) - let dumpStreams struct (s : Scheduling.StreamStates<_>, totalPurged) logger = - s.Dump(logger, totalPurged, Buffering.StreamState.eventsSize) - match dumpExternalStats with Some f -> f logger | None -> () + let itemDispatcher = Dispatch.ItemDispatcher<_, 'F>(maxConcurrentStreams) + let dumpStreams logStreamStates log = + logStreamStates eventSize + match dumpExternalStats with Some f -> f log | None -> () - let dispatcher = Scheduling.MultiDispatcher<_, _, _>.Create(itemDispatcher, attemptWrite, interpretWriteResultProgress, stats, dumpStreams) + let dispatcher = Scheduling.Dispatcher.MultiDispatcher<_, _, _, 'F>.Create(itemDispatcher, attemptWrite, interpretWriteResultProgress, stats, dumpStreams) let streamScheduler = - Scheduling.StreamSchedulingEngine - ( dispatcher, maxBatches=maxBatches, maxCycles=defaultArg maxCycles 128, ?idleDelay=idleDelay, ?purgeInterval=purgeInterval) + Scheduling.StreamSchedulingEngine + ( dispatcher, maxBatches = maxBatches, maxCycles = defaultArg maxCycles 128, ?idleDelay = idleDelay, ?purgeInterval = purgeInterval) - Projector.StreamsProjectorPipeline.Start( + Projector.Pipeline.Start( log, itemDispatcher.Pump, streamScheduler.Pump, maxReadAhead, streamScheduler.Submit, statsInterval, maxSubmissionsPerPartition = maxBatches) + +module Default = + + /// Canonical Data/Meta type supplied by the majority of Sources + type EventBody = ReadOnlyMemory + /// A contiguous set of Events from a Ordered stream, using the Canonical Data/Meta type + type StreamSpan = StreamSpan + /// A Single Event from an Ordered stream, using the Canonical Data/Meta type + type StreamEvent = StreamEvent + let inline private eventBodyBytes (x : EventBody) = x.Length + let inline private stringBytes (x : string) = match x with null -> 0 | x -> x.Length * sizeof + let eventDataSize (x : FsCodec.IEventData) = eventBodyBytes x.Data + stringBytes x.EventType + 16 + let eventSize (x : FsCodec.IEventData) = eventDataSize x + eventBodyBytes x.Meta + let jsonSize (x : FsCodec.IEventData) = eventSize x + 80 + + type Sink = Sink> + + type Config = + + static member Start<'Outcome> + ( log, maxReadAhead, maxConcurrentStreams, + handle : struct (FsCodec.StreamName * StreamSpan) -> Async, + stats, statsInterval, + ?maxSubmissionsPerPartition, ?maxBatches, + ?purgeInterval, + ?wakeForResults, ?idleDelay, ?ingesterStatsInterval) + : Sink = + StreamsSink.Start<'Outcome, EventBody>( + log, maxReadAhead, maxConcurrentStreams, handle, stats, statsInterval, eventSize, + ?maxSubmissionsPerPartition = maxSubmissionsPerPartition, ?maxBatches = maxBatches, ?purgeInterval = purgeInterval, + ?wakeForResults = wakeForResults, ?idleDelay = idleDelay, ?ingesterStatsInterval = ingesterStatsInterval) diff --git a/src/Propulsion/Submission.fs b/src/Propulsion/Submission.fs index 3e9b7d77..5aa56f0e 100755 --- a/src/Propulsion/Submission.fs +++ b/src/Propulsion/Submission.fs @@ -1,7 +1,7 @@ /// Holds batches from the Ingestion pipe, feeding them continuously to the scheduler in an appropriate order module Propulsion.Submission -open Propulsion.Internal // Helper types +open Propulsion.Internal // Helpers open Serilog open System open System.Collections.Generic @@ -12,9 +12,7 @@ open System.Threading.Tasks [] module Helpers = - let sortByVsndDescending (xs : seq) = xs |> Seq.sortByDescending (fun struct (_k, v) -> v) - let statsDescending (xs : Dictionary<_, _>) = xs |> Seq.map (fun x -> struct (x.Key, x.Value)) |> sortByVsndDescending - let statsTotal (xs : struct (_ * int64) array) = xs |> Array.sumBy (fun struct (_k, v) -> v) + let statsTotal (xs : struct (_ * int64) array) = xs |> Array.sumBy ValueTuple.snd /// Gathers stats relating to how many items of a given partition have been observed type PartitionStats<'S when 'S : equality>() = @@ -27,7 +25,7 @@ module Helpers = | false, _ -> partitions[partitionId] <- weight member _.Clear() = partitions.Clear() - member _.StatsDescending = statsDescending partitions + member _.StatsDescending = Stats.statsDescending partitions let atTimedIntervals (period : TimeSpan) = let timer, max = Stopwatch.StartNew(), int64 period.TotalMilliseconds @@ -39,7 +37,7 @@ module Helpers = /// Batch of work as passed from the Submitter to the Scheduler comprising messages with their associated checkpointing/completion callback [] -type SubmissionBatch<'S, 'M> = { source : 'S; onCompletion : unit -> unit; messages : 'M [] } +type Batch<'S, 'M> = { source : 'S; onCompletion : unit -> unit; messages : 'M [] } /// Holds the queue for a given partition, together with a semaphore we use to ensure the number of in-flight batches per partition is constrained [] @@ -47,30 +45,42 @@ type PartitionQueue<'B> = { submissions : Sem; queue : Queue<'B> } with member x.Append(batch) = x.queue.Enqueue batch static member Create(maxSubmits) = { submissions = Sem maxSubmits; queue = Queue(maxSubmits) } +type internal Stats<'S when 'S : equality>(log : ILogger, interval) = + + let mutable cycles, ingested, completed, compacted = 0, 0, 0, 0 + let submittedBatches,submittedMessages = PartitionStats<'S>(), PartitionStats<'S>() + + member val Interval = IntervalTimer interval + + member _.Dump(waiting : seq) = + log.Information("Submitter ingested {ingested} compacted {compacted} completed {completed} Events {items} Batches {batches} Holding {holding} Cycles {cycles}", + ingested, compacted, completed, submittedMessages.StatsDescending, submittedBatches.StatsDescending, waiting, cycles) + cycles <- 0; ingested <- 0; compacted <- 0; completed <- 0; submittedBatches.Clear(); submittedMessages.Clear() + + member _.RecordCompacted() = + compacted <- compacted + 1 + member _.RecordBatchIngested() = + ingested <- ingested + 1 + member _.RecordBatchCompleted() = + Interlocked.Increment(&completed) |> ignore + member _.RecordBatch(pi, count : int64) = + submittedBatches.Record(pi) + submittedMessages.Record(pi, count) + member _.RecordCycle() = + cycles <- cycles + 1 + /// Holds the stream of incoming batches, grouping by partition /// Manages the submission of batches into the Scheduler in a fair manner type SubmissionEngine<'S, 'M, 'B when 'S : equality> - ( log : ILogger, maxSubmitsPerPartition, mapBatch : (unit -> unit) -> SubmissionBatch<'S, 'M> -> 'B, submitBatch : 'B -> int, statsInterval, + ( log : ILogger, maxSubmitsPerPartition, mapBatch : (unit -> unit) -> Batch<'S, 'M> -> 'B, submitBatch : 'B -> int, statsInterval, ?tryCompactQueue) = let awaitIncoming, applyIncoming, enqueueIncoming = - let c = Channel.unboundedSr - Channel.awaitRead c, Channel.apply c, Channel.write c + let c = Channel.unboundedSr in let r, w = c.Reader, c.Writer + Channel.awaitRead r, Channel.apply r, Channel.write w let buffer = Dictionary<'S, PartitionQueue<'B>>() - - let mutable cycles, ingested, completed, compacted = 0, 0, 0, 0 - let submittedBatches,submittedMessages = PartitionStats(), PartitionStats() - let statsInterval = timeRemaining statsInterval - let dumpStats () = - let waiting = seq { for x in buffer do if x.Value.queue.Count <> 0 then yield struct (x.Key, x.Value.queue.Count) } |> sortByVsndDescending - log.Information("Submitter ingested {ingested} compacted {compacted} completed {completed} Events {items} Batches {batches} Holding {holding} Cycles {cycles}", - ingested, compacted, completed, submittedMessages.StatsDescending, submittedBatches.StatsDescending, waiting, cycles) - cycles <- 0; ingested <- 0; compacted <- 0; completed <- 0; submittedBatches.Clear(); submittedMessages.Clear() - let maybeDumpStats () = - cycles <- cycles + 1 - let struct (due, remaining) = statsInterval () - if due then dumpStats () - int remaining + let queueStats = seq { for x in buffer do if x.Value.queue.Count <> 0 then struct (x.Key, x.Value.queue.Count) } |> Seq.sortByDescending ValueTuple.snd + let stats = Stats<'S>(log, statsInterval) // Loop, submitting 0 or 1 item per partition per iteration to ensure // - each partition has a controlled maximum number of entrants in the scheduler queue @@ -78,25 +88,25 @@ type SubmissionEngine<'S, 'M, 'B when 'S : equality> let tryPropagate (waiting : ResizeArray) = waiting.Clear() let mutable worked = false - for KeyValue (pi, pq) in buffer do + for kv in buffer do + let pi, pq = kv.Key, kv.Value if pq.queue.Count <> 0 then if pq.submissions.TryTake() then worked <- true let count = submitBatch <| pq.queue.Dequeue() - submittedBatches.Record(pi) - submittedMessages.Record(pi, int64 count) + stats.RecordBatch(pi, count) else waiting.Add(pq.submissions) worked - let ingest (partitionBatches : SubmissionBatch<'S, 'M>[]) = - ingested <- ingested + 1 + let ingest (partitionBatches : Batch<'S, 'M>[]) = + stats.RecordBatchIngested() for { source = pid } as batch in partitionBatches do let mutable pq = Unchecked.defaultof<_> if not (buffer.TryGetValue(pid, &pq)) then pq <- PartitionQueue<_>.Create(maxSubmitsPerPartition) buffer[pid] <- pq let markCompleted () = - Interlocked.Increment(&completed) |> ignore + stats.RecordBatchCompleted() pq.submissions.Release() let mapped = mapBatch markCompleted batch pq.Append(mapped) @@ -104,10 +114,10 @@ type SubmissionEngine<'S, 'M, 'B when 'S : equality> /// We use timeslices where we're we've fully provisioned the scheduler to index any waiting Batches let compact f = let mutable worked = false - for KeyValue(_, pq) in buffer do - if f pq.queue then + for kv in buffer do + if f kv.Value.queue then worked <- true - if worked then compacted <- compacted + 1 + if worked then stats.RecordCompacted() worked let maybeCompact () = match tryCompactQueue with @@ -121,13 +131,14 @@ type SubmissionEngine<'S, 'M, 'B when 'S : equality> let submitCapacityAvailable : seq = seq { for w in waitingSubmissions -> w.AwaitButRelease() } while not ct.IsCancellationRequested do while applyIncoming ingest || tryPropagate waitingSubmissions || maybeCompact () do () - let nextStatsIntervalMs = maybeDumpStats () - do! Task.WhenAny[| awaitIncoming ct :> Task; yield! submitCapacityAvailable; Task.Delay(nextStatsIntervalMs) |] :> Task } + stats.RecordCycle() + if stats.Interval.IfDueRestart() then stats.Dump(queueStats) + do! Task.WhenAny[| awaitIncoming ct :> Task; yield! submitCapacityAvailable; Task.Delay(stats.Interval.RemainingMs) |] :> Task } /// Supplies a set of Batches for holding and forwarding to scheduler at the right time - member _.Ingest(items : SubmissionBatch<'S, 'M>[]) = + member _.Ingest(items : Batch<'S, 'M>[]) = enqueueIncoming items /// Supplies an incoming Batch for holding and forwarding to scheduler at the right time - member x.Ingest(batch : SubmissionBatch<'S, 'M>) = + member x.Ingest(batch : Batch<'S, 'M>) = x.Ingest [| batch |] diff --git a/tests/Propulsion.Kafka.Integration/ConsumersIntegration.fs b/tests/Propulsion.Kafka.Integration/ConsumersIntegration.fs index ed7d114e..bfac4bf4 100644 --- a/tests/Propulsion.Kafka.Integration/ConsumersIntegration.fs +++ b/tests/Propulsion.Kafka.Integration/ConsumersIntegration.fs @@ -112,8 +112,9 @@ module Helpers = do! Async.Parallel [for i in 1 .. numConsumers -> mkConsumer i] |> Async.Ignore } - let deserialize consumerId (e : FsCodec.ITimelineEvent) : ConsumedTestMessage = - let d = serdes.Deserialize(System.Text.Encoding.UTF8.GetString e.Data) + let deserialize consumerId (e : ITimelineEvent) : ConsumedTestMessage = + let d = e.Data + let d = serdes.Deserialize(System.Text.Encoding.UTF8.GetString d.Span) { consumerId = consumerId; meta = d; payload = unbox e.Context } type Stats(log, statsInterval, stateInterval) = @@ -136,14 +137,14 @@ module Helpers = // When offered, take whatever is pending let select = Array.ofSeq // when processing, declare all items processed each time we're invoked - let handle (streams : Propulsion.Streams.Scheduling.DispatchItem[]) = async { + let handle (streams : Propulsion.Streams.Dispatch.Item[]) = async { let mutable c = 0 for stream in streams do - for event in stream.span.events do + for event in stream.span do c <- c + 1 do! handler (getConsumer()) (deserialize consumerId event) - (log : Serilog.ILogger).Information("BATCHED CONSUMER Handled {c} events in {l} streams", c, streams.Length ) - return [| for x in streams -> Choice1Of2 (x.span.events.[x.span.events.Length-1].Index+1L) |] |> Seq.ofArray } + (log : ILogger).Information("BATCHED CONSUMER Handled {c} events in {l} streams", c, streams.Length ) + return [| for x in streams -> Choice1Of2 (x.span[x.span.Length-1].Index+1L) |] |> Seq.ofArray } let stats = Stats(log, TimeSpan.FromSeconds 5.,TimeSpan.FromSeconds 5.) let messageIndexes = StreamNameSequenceGenerator() let consumer = @@ -162,9 +163,9 @@ module Helpers = do! Async.Parallel [for i in 1 .. numConsumers -> mkConsumer i] |> Async.Ignore } - let mapStreamConsumeResultToDataAndContext (x: ConsumeResult<_,string>) : byte[] * obj = + let mapStreamConsumeResultToDataAndContext (x: ConsumeResult<_,string>) : Propulsion.Streams.Default.EventBody * obj = let m = Binding.message x - System.Text.Encoding.UTF8.GetBytes(m.Value), + System.Text.Encoding.UTF8.GetBytes(m.Value) |> ReadOnlyMemory, box { key = m.Key; value = m.Value; partition = Binding.partitionValue x.Partition; offset = let o = x.Offset in o.Value } let runConsumersStream log (config : KafkaConsumerConfig) (numConsumers : int) (timeout : TimeSpan option) (handler : ConsumerCallback) = async { @@ -179,10 +180,10 @@ module Helpers = | Some c -> c // when processing, declare all items processed each time we're invoked - let handle (streamName : StreamName, span : Propulsion.Streams.StreamSpan) = async { - for event in span.events do + let handle struct (streamName : StreamName, span : Propulsion.Streams.Default.StreamSpan) = async { + for event in span do do! handler (getConsumer()) (deserialize consumerId event) - return Propulsion.Streams.SpanResult.AllProcessed, () } + return struct (Propulsion.Streams.SpanResult.AllProcessed, ()) } let stats = Stats(log, TimeSpan.FromSeconds 5.,TimeSpan.FromSeconds 5.) let messageIndexes = StreamNameSequenceGenerator() let consumer = @@ -243,7 +244,7 @@ and [] ConsumerIntegration(testOutputHelper, expectConcurrentSche member __.RunProducers(log, bootstrapServers, topic, numProducers, messagesPerProducer) : Async = runProducers log bootstrapServers topic numProducers messagesPerProducer |> Async.Ignore - abstract RunConsumers: Serilog.ILogger * KafkaConsumerConfig * int * ConsumerCallback * TimeSpan option -> Async + abstract RunConsumers: ILogger * KafkaConsumerConfig * int * ConsumerCallback * TimeSpan option -> Async member __.RunConsumers(log,config,count,cb) = __.RunConsumers(log,config,count,cb,None) [] @@ -259,7 +260,7 @@ and [] ConsumerIntegration(testOutputHelper, expectConcurrentSche let consumedBatches = ConcurrentBag() let expectedUniqueMessages = numProducers * messagesPerProducer let consumerCallback (consumer:ConsumerPipeline) msg = async { - itemsSeen.[msg.payload] <- () + itemsSeen[msg.payload] <- () consumedBatches.Add msg // signal cancellation if consumed items reaches expected size if itemsSeen.Count >= expectedUniqueMessages then @@ -297,7 +298,7 @@ and [] ConsumerIntegration(testOutputHelper, expectConcurrentSche |> Array.where (fun gp -> gp.Length <> messagesPerProducer) let unconsumedCounts = unconsumed - |> Seq.map (fun gp -> gp.[0].payload.producerId, gp.Length) + |> Seq.map (fun gp -> gp[0].payload.producerId, gp.Length) |> Array.ofSeq test <@ Array.isEmpty unconsumedCounts @> } diff --git a/tests/Propulsion.Tests/ProgressTests.fs b/tests/Propulsion.Tests/ProgressTests.fs index 30e9f82e..ab1f85f0 100644 --- a/tests/Propulsion.Tests/ProgressTests.fs +++ b/tests/Propulsion.Tests/ProgressTests.fs @@ -1,7 +1,7 @@ module Propulsion.Tests.ProgressTests open FsCodec -open Propulsion.Streams +open Propulsion.Streams.Scheduling.Progress open Swensen.Unquote open System.Collections.Generic open Xunit @@ -10,27 +10,27 @@ let sn x = StreamName.create x x let mkDictionary xs = Dictionary(dict xs) let [] ``Empty has zero streams pending or progress to write`` () = - let sut = Progress.ProgressState<_>() + let sut = ProgressState<_>() let queue = sut.InScheduledOrder(None) test <@ Seq.isEmpty queue @> let [] ``Can add multiple batches with overlapping streams`` () = - let sut = Progress.ProgressState<_>() + let sut = ProgressState<_>() let noBatchesComplete () = failwith "No bathes should complete" sut.AppendBatch(noBatchesComplete, mkDictionary [sn "a",1L; sn "b",2L]) sut.AppendBatch(noBatchesComplete, mkDictionary [sn "b",2L; sn "c",3L]) let [] ``Marking Progress removes batches and triggers the callbacks`` () = - let sut = Progress.ProgressState<_>() + let sut = ProgressState<_>() let mutable callbacks = 0 let complete () = callbacks <- callbacks + 1 sut.AppendBatch(complete, mkDictionary [sn "a",1L; sn "b",2L]) - sut.MarkStreamProgress(sn "a",1L) |> ignore - sut.MarkStreamProgress(sn "b",3L) |> ignore + sut.MarkStreamProgress(sn "a",1L) + sut.MarkStreamProgress(sn "b",3L) 1 =! callbacks let [] ``Empty batches get removed immediately`` () = - let sut = Progress.ProgressState<_>() + let sut = ProgressState<_>() let mutable callbacks = 0 let complete () = callbacks <- callbacks + 1 sut.AppendBatch(complete, mkDictionary [||]) @@ -38,11 +38,11 @@ let [] ``Empty batches get removed immediately`` () = 2 =! callbacks let [] ``Marking progress is not persistent`` () = - let sut = Progress.ProgressState<_>() + let sut = ProgressState<_>() let mutable callbacks = 0 let complete () = callbacks <- callbacks + 1 sut.AppendBatch(complete, mkDictionary [sn "a",1L]) - sut.MarkStreamProgress(sn "a",2L) |> ignore + sut.MarkStreamProgress(sn "a",2L) sut.AppendBatch(complete, mkDictionary [sn "a",1L; sn "b",2L]) 1 =! callbacks diff --git a/tests/Propulsion.Tests/StreamStateTests.fs b/tests/Propulsion.Tests/StreamStateTests.fs index 10641c08..7e60e007 100644 --- a/tests/Propulsion.Tests/StreamStateTests.fs +++ b/tests/Propulsion.Tests/StreamStateTests.fs @@ -1,86 +1,87 @@ module StreamStateTests open Propulsion.Streams -open Propulsion.Streams.Buffering +open Propulsion.Streams.Buffer open Swensen.Unquote open Xunit let canonicalTime = System.DateTimeOffset.UtcNow let mk p c : StreamSpan = - { index = p - events = [| for x in 0..c-1 -> FsCodec.Core.TimelineEvent.Create(int64 x, p + int64 x |> string, null, timestamp = canonicalTime) |] } -let mergeSpans = StreamSpan.merge -let trimSpans = StreamSpan.dropBeforeIndex + [| for x in 0..c-1 -> FsCodec.Core.TimelineEvent.Create(p + int64 x, p + int64 x |> string, null, timestamp = canonicalTime) |] +let merge = StreamSpan.merge +let dropBeforeIndex = StreamSpan.dropBeforeIndex let is (xs : StreamSpan[]) (res : StreamSpan[]) = - (xs, res) ||> Seq.forall2 (fun x y -> x.index = y.index && (x.events, y.events) ||> Seq.forall2 (fun x y -> x.EventType = y.EventType)) + (xs = null && res = null) + || (xs, res) ||> Seq.forall2 (fun x y -> (x = null && y = null) + || (x[0].Index = y[0].Index && (x, y) ||> Seq.forall2 (fun x y -> x.EventType = y.EventType))) -let [] ``nothing`` () = - let r = mergeSpans 0L [ mk 0L 0; mk 0L 0 ] +let [] nothing () = + let r = merge 0L [ mk 0L 0; mk 0L 0 ] test <@ obj.ReferenceEquals(null, r) @> -let [] ``synced`` () = - let r = mergeSpans 1L [ mk 0L 1; mk 0L 0 ] +let [] synced () = + let r = merge 1L [ mk 0L 1; mk 0L 0 ] test <@ obj.ReferenceEquals(null, r) @> let [] ``no overlap`` () = - let r = mergeSpans 0L [ mk 0L 1; mk 2L 2 ] + let r = merge 0L [ mk 0L 1; mk 2L 2 ] test <@ r |> is [| mk 0L 1; mk 2L 2 |] @> -let [] ``overlap`` () = - let r = mergeSpans 0L [ mk 0L 1; mk 0L 2 ] +let [] overlap () = + let r = merge 0L [ mk 0L 1; mk 0L 2 ] test <@ r |> is [| mk 0L 2 |] @> let [] ``remove nulls`` () = - let r = mergeSpans 1L [ mk 0L 1; mk 0L 2 ] + let r = merge 1L [ mk 0L 1; mk 0L 2 ] test <@ r |> is [| mk 1L 1 |] @> -let [] ``adjacent`` () = - let r = mergeSpans 0L [ mk 0L 1; mk 1L 2 ] +let [] adjacent () = + let r = merge 0L [ mk 0L 1; mk 1L 2 ] test <@ r |> is [| mk 0L 3 |] @> let [] ``adjacent to min`` () = - let r = Array.map (trimSpans 2L) [| mk 0L 1; mk 1L 2 |] - test <@ r |> is [| mk 2L 0; mk 2L 1 |] @> + let r = Array.map (dropBeforeIndex 2L) [| mk 0L 1; mk 1L 2 |] + test <@ r |> is [| null; mk 2L 1 |] @> let [] ``adjacent to min merge`` () = - let r = mergeSpans 2L [ mk 0L 1; mk 1L 2 ] + let r = merge 2L [ mk 0L 1; mk 1L 2 ] test <@ r |> is [| mk 2L 1 |] @> let [] ``adjacent to min no overlap`` () = - let r = mergeSpans 2L [ mk 0L 1; mk 2L 1 ] + let r = merge 2L [ mk 0L 1; mk 2L 1 ] test <@ r |> is [| mk 2L 1|] @> let [] ``adjacent trim`` () = - let r = Array.map (trimSpans 1L) [| mk 0L 2; mk 2L 2 |] + let r = Array.map (dropBeforeIndex 1L) [| mk 0L 2; mk 2L 2 |] test <@ r |> is [| mk 1L 1; mk 2L 2 |] @> let [] ``adjacent trim merge`` () = - let r = mergeSpans 1L [ mk 0L 2; mk 2L 2 ] + let r = merge 1L [ mk 0L 2; mk 2L 2 ] test <@ r |> is [| mk 1L 3 |] @> let [] ``adjacent trim append`` () = - let r = Array.map (trimSpans 1L) [| mk 0L 2; mk 2L 2; mk 5L 1 |] + let r = Array.map (dropBeforeIndex 1L) [| mk 0L 2; mk 2L 2; mk 5L 1 |] test <@ r |> is [| mk 1L 1; mk 2L 2; mk 5L 1 |] @> let [] ``adjacent trim append merge`` () = - let r = mergeSpans 1L [ mk 0L 2; mk 2L 2; mk 5L 1] + let r = merge 1L [ mk 0L 2; mk 2L 2; mk 5L 1] test <@ r |> is [| mk 1L 3; mk 5L 1 |] @> let [] ``mixed adjacent trim append`` () = - let r = Array.map (trimSpans 1L) [| mk 0L 2; mk 5L 1; mk 2L 2 |] + let r = Array.map (dropBeforeIndex 1L) [| mk 0L 2; mk 5L 1; mk 2L 2 |] test <@ r |> is [| mk 1L 1; mk 5L 1; mk 2L 2 |] @> let [] ``mixed adjacent trim append merge`` () = - let r = mergeSpans 1L [ mk 0L 2; mk 5L 1; mk 2L 2] + let r = merge 1L [ mk 0L 2; mk 5L 1; mk 2L 2] test <@ r |> is [| mk 1L 3; mk 5L 1 |] @> -let [] ``fail`` () = - let r = mergeSpans 11614L [ {index=11614L; events=null}; mk 11614L 1 ] +let [] fail () = + let r = merge 11614L [ null; mk 11614L 1 ] test <@ r |> is [| mk 11614L 1 |] @> let [] ``fail 2`` () = - let r = mergeSpans 11613L [ mk 11614L 1; {index=11614L; events=null} ] + let r = merge 11613L [ mk 11614L 1; null ] test <@ r |> is [| mk 11614L 1 |] @> #if MEMORY_USAGE_ANALYSIS diff --git a/tools/Propulsion.Tool/Infrastructure.fs b/tools/Propulsion.Tool/Infrastructure.fs index 32a6fb32..8abd057c 100644 --- a/tools/Propulsion.Tool/Infrastructure.fs +++ b/tools/Propulsion.Tool/Infrastructure.fs @@ -34,7 +34,7 @@ type Logging() = let removeMetricsProps (c : LoggerConfiguration) : LoggerConfiguration = let trim (e : Serilog.Events.LogEvent) = e.RemovePropertyIfPresent(Propulsion.Streams.Log.PropertyTag) - e.RemovePropertyIfPresent(Propulsion.Feed.Internal.Log.PropertyTag) + e.RemovePropertyIfPresent(Propulsion.Feed.Core.Log.PropertyTag) e.RemovePropertyIfPresent(Propulsion.CosmosStore.Log.PropertyTag) e.RemovePropertyIfPresent(Equinox.CosmosStore.Core.Log.PropertyTag) e.RemovePropertyIfPresent(Equinox.DynamoStore.Core.Log.PropertyTag) diff --git a/tools/Propulsion.Tool/Program.fs b/tools/Propulsion.Tool/Program.fs index de8f849b..0110932c 100644 --- a/tools/Propulsion.Tool/Program.fs +++ b/tools/Propulsion.Tool/Program.fs @@ -322,24 +322,24 @@ module Project = let stats = Stats(TimeSpan.FromMinutes 1., TimeSpan.FromMinutes 5., logExternalStats = dumpStoreStats) let sink = let maxReadAhead, maxConcurrentStreams = 2, 16 - let handle (stream : FsCodec.StreamName, span : Propulsion.Streams.StreamSpan<_>) = async { + let handle (stream : FsCodec.StreamName, span : Propulsion.Streams.Default.StreamSpan) = async { match producer with | None -> () | Some producer -> let json = Propulsion.Codec.NewtonsoftJson.RenderedSpan.ofStreamSpan stream span |> Newtonsoft.Json.JsonConvert.SerializeObject let! _ = producer.ProduceAsync(FsCodec.StreamName.toString stream, json) in () } - Propulsion.Streams.StreamsProjector.Start(Log.Logger, maxReadAhead, maxConcurrentStreams, handle, stats, stats.StatsInterval, idleDelay = a.IdleDelay) + Propulsion.Streams.StreamsSink.Start(Log.Logger, maxReadAhead, maxConcurrentStreams, handle, stats, stats.StatsInterval, Propulsion.Streams.Default.eventSize, idleDelay = a.IdleDelay) let source = + let nullFilter _ = true match storeArgs with | Choice1Of2 sa -> let monitored = sa.MonitoredContainer() let leases = sa.ConnectLeases() - let maybeLogLagInterval = sa.MaybeLogLagInterval - let transformOrFilter = Propulsion.CosmosStore.EquinoxSystemTextJsonParser.enumStreamEvents - let observer = Propulsion.CosmosStore.CosmosStoreSource.CreateObserver(Log.Logger, sink.StartIngester, Seq.collect transformOrFilter) + let parseFeedDoc = Propulsion.CosmosStore.EquinoxSystemTextJsonParser.enumStreamEvents nullFilter + let observer = Propulsion.CosmosStore.CosmosStoreSource.CreateObserver(Log.Logger, sink.StartIngester, Seq.collect parseFeedDoc) Propulsion.CosmosStore.CosmosStoreSource.Start ( Log.Logger, monitored, leases, group, observer, - startFromTail = startFromTail, ?maxItems = maxItems, ?lagReportFreq = maybeLogLagInterval) + startFromTail = startFromTail, ?maxItems = maxItems, ?lagReportFreq = sa.MaybeLogLagInterval) | Choice2Of2 sa -> let (indexStore, indexFilter), maybeHydrate = sa.MonitoringParams() let checkpoints = @@ -348,7 +348,6 @@ module Project = let loadMode = match maybeHydrate with | Some (context, streamsDop) -> - let nullFilter _ = true Propulsion.DynamoStore.LoadMode.Hydrated (nullFilter, streamsDop, context) | None -> Propulsion.DynamoStore.LoadMode.All Propulsion.DynamoStore.DynamoStoreSource( @@ -366,7 +365,7 @@ module Project = /// Parse the commandline; can throw exceptions in response to missing arguments and/or `-h`/`--help` args let parseCommandLine argv = let programName = Reflection.Assembly.GetEntryAssembly().GetName().Name - let parser = ArgumentParser.Create(programName=programName) + let parser = ArgumentParser.Create(programName = programName) parser.ParseCommandLine argv []