From 3469037722ad27ad831dc3fa44b1d661633df563 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Wed, 26 Jun 2024 17:01:47 -0700 Subject: [PATCH 01/29] avoid full metadata call --- core/Processors/DefaultTopicManager.cs | 2 +- samples/sample-stream/Program.cs | 6 ++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/core/Processors/DefaultTopicManager.cs b/core/Processors/DefaultTopicManager.cs index 5fab99a6..364551cc 100644 --- a/core/Processors/DefaultTopicManager.cs +++ b/core/Processors/DefaultTopicManager.cs @@ -49,7 +49,6 @@ async Task> Run() var defaultConfig = new Dictionary(); var topicsNewCreated = new List(); var topicsToCreate = new List(); - var metadata = AdminClient.GetMetadata(timeout); // 1. get source topic partition // 2. check if changelog exist, : @@ -58,6 +57,7 @@ async Task> Run() // 3. if changelog doesn't exist, create it with partition number and configuration foreach (var t in topics) { + var metadata = AdminClient.GetMetadata(t.Key, timeout); var numberPartitions = GetNumberPartitionForTopic(metadata, t.Key); if (numberPartitions == 0) { diff --git a/samples/sample-stream/Program.cs b/samples/sample-stream/Program.cs index d6e54372..59468230 100644 --- a/samples/sample-stream/Program.cs +++ b/samples/sample-stream/Program.cs @@ -25,7 +25,6 @@ public static async Task Main(string[] args) PartitionAssignmentStrategy = PartitionAssignmentStrategy.RoundRobin, Partitioner = Partitioner.ConsistentRandom, Debug = "broker,topic,msg", - ClientId = "ttoot", Logger = LoggerFactory.Create((b) => { b.AddConsole(); @@ -50,7 +49,7 @@ private static Topology BuildTopology() TimeSpan windowSize = TimeSpan.FromHours(1); var builder = new StreamBuilder(); - /* builder.Stream("input") + builder.Stream("input") .GroupByKey() .WindowedBy(TumblingWindowOptions.Of(windowSize)) .Count(RocksDbWindows.As("count-store") @@ -61,9 +60,8 @@ private static Topology BuildTopology() .Map((k,v) => new KeyValuePair(k.ToString(), v.ToString())) .To("output", new StringSerDes(), - new StringSerDes());*/ + new StringSerDes()); - builder.GlobalTable("global"); return builder.Build(); } From 1c5f3492262111eac1c6106cbb719e50736cdc2b Mon Sep 17 00:00:00 2001 From: LGouellec Date: Thu, 27 Jun 2024 10:48:39 -0700 Subject: [PATCH 02/29] fix unit test --- core/Mock/Sync/SyncAdminClient.cs | 29 +++++++++++++++++++---------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/core/Mock/Sync/SyncAdminClient.cs b/core/Mock/Sync/SyncAdminClient.cs index f59a3c16..7db9398a 100644 --- a/core/Mock/Sync/SyncAdminClient.cs +++ b/core/Mock/Sync/SyncAdminClient.cs @@ -2,6 +2,7 @@ using Confluent.Kafka.Admin; using System; using System.Collections.Generic; +using System.Linq; using System.Threading.Tasks; using Streamiz.Kafka.Net.Mock.Kafka; @@ -66,20 +67,28 @@ public override void Dispose() public override Metadata GetMetadata(string topic, TimeSpan timeout) { var error = new Error(ErrorCode.NoError); - - var brokersMetadata = new List { - new BrokerMetadata(1, "localhost", 9092) - }; - - var partitionsMetadata = new List + var topics = producer.GetAllTopics(); + var brokersMetadata = new List { - new PartitionMetadata(1, 1, new int[1]{1}, new int[1]{1}, error) + new(1, "localhost", 9092) }; + + if (topics.Contains(topic)) + { + var partitionsMetadata = new List + { + new(1, 1, new int[1] { 1 }, new int[1] { 1 }, error) + }; - var topicMetadata = new TopicMetadata(topic, partitionsMetadata, error); - + var topicMetadata = new TopicMetadata(topic, partitionsMetadata, error); + + return new Metadata(brokersMetadata, + new List() { topicMetadata }, + 1, "localhost"); + } + return new Metadata(brokersMetadata, - new List() { topicMetadata }, + new List(), 1, "localhost"); } From 5b61ccc78a68b5bd9de3034f03fcf3d42b6fe9b8 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Fri, 28 Jun 2024 15:59:53 -0700 Subject: [PATCH 03/29] Update unit test --- .gitignore | 4 +++- core/Mock/Sync/SyncAdminClient.cs | 11 ++++++++++- core/Mock/Sync/SyncKafkaSupplier.cs | 4 ++-- core/State/RocksDb/RocksDbOptions.cs | 4 ++-- core/Streamiz.Kafka.Net.csproj | 2 +- .../Private/DefaultTopicManagerTests.cs | 5 +++-- .../Public/KafkaStreamTests.cs | 2 +- .../Public/RocksDbOptionsTests.cs | 8 +------- 8 files changed, 23 insertions(+), 17 deletions(-) diff --git a/.gitignore b/.gitignore index c95e4f75..77a1b76b 100644 --- a/.gitignore +++ b/.gitignore @@ -9,4 +9,6 @@ build TestResults .idea/ -.vscode/ \ No newline at end of file +.vscode/ + +confidential \ No newline at end of file diff --git a/core/Mock/Sync/SyncAdminClient.cs b/core/Mock/Sync/SyncAdminClient.cs index 7db9398a..d6741e10 100644 --- a/core/Mock/Sync/SyncAdminClient.cs +++ b/core/Mock/Sync/SyncAdminClient.cs @@ -11,11 +11,13 @@ namespace Streamiz.Kafka.Net.Mock.Sync internal class SyncAdminClient : BasedAdminClient { private readonly SyncProducer producer; + private readonly bool _autoCreateTopic; private AdminClientConfig config; - public SyncAdminClient(SyncProducer producer) + public SyncAdminClient(SyncProducer producer, bool autoCreateTopic) { this.producer = producer; + _autoCreateTopic = autoCreateTopic; } internal void UseConfig(AdminClientConfig config) @@ -86,6 +88,13 @@ public override Metadata GetMetadata(string topic, TimeSpan timeout) new List() { topicMetadata }, 1, "localhost"); } + + if (_autoCreateTopic) + { + // auto create topic if not exist + producer.CreateTopic(topic); + return GetMetadata(topic, timeout); + } return new Metadata(brokersMetadata, new List(), diff --git a/core/Mock/Sync/SyncKafkaSupplier.cs b/core/Mock/Sync/SyncKafkaSupplier.cs index 03a46da4..d080c60e 100644 --- a/core/Mock/Sync/SyncKafkaSupplier.cs +++ b/core/Mock/Sync/SyncKafkaSupplier.cs @@ -14,10 +14,10 @@ internal class SyncKafkaSupplier : IKafkaSupplier protected SyncProducer producer = null; protected SyncAdminClient admin = null; - public SyncKafkaSupplier() + public SyncKafkaSupplier(bool autoCreateTopic = true) { producer = new SyncProducer(); - admin = new SyncAdminClient(producer); + admin = new SyncAdminClient(producer, autoCreateTopic); } public virtual IAdminClient GetAdmin(AdminClientConfig config) diff --git a/core/State/RocksDb/RocksDbOptions.cs b/core/State/RocksDb/RocksDbOptions.cs index 24f3f161..5d98510d 100644 --- a/core/State/RocksDb/RocksDbOptions.cs +++ b/core/State/RocksDb/RocksDbOptions.cs @@ -1191,9 +1191,9 @@ public RocksDbOptions SetOptimizeFiltersForHits(int value) /// /// /// the instance of the current object - public RocksDbOptions SetPlainTableFactory(uint p1, int p2, double p3, ulong p4) + public RocksDbOptions SetPlainTableFactory(uint user_key_len, int bloom_bits_per_key, double hash_table_ratio, int index_sparseness, int huge_page_tlb_size, char encoding_type, bool full_scan_mode, bool store_index_in_file) { - columnFamilyOptions.SetPlainTableFactory(p1, p2, p3, p4); + columnFamilyOptions.SetPlainTableFactory(user_key_len, bloom_bits_per_key, hash_table_ratio, index_sparseness, huge_page_tlb_size, encoding_type, full_scan_mode, store_index_in_file); return this; } diff --git a/core/Streamiz.Kafka.Net.csproj b/core/Streamiz.Kafka.Net.csproj index ee73700f..8606b4d9 100644 --- a/core/Streamiz.Kafka.Net.csproj +++ b/core/Streamiz.Kafka.Net.csproj @@ -42,7 +42,7 @@ - + diff --git a/test/Streamiz.Kafka.Net.Tests/Private/DefaultTopicManagerTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/DefaultTopicManagerTests.cs index 4551db3f..650441e0 100644 --- a/test/Streamiz.Kafka.Net.Tests/Private/DefaultTopicManagerTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Private/DefaultTopicManagerTests.cs @@ -16,7 +16,7 @@ public class DefaultTopicManagerTests [SetUp] public void Begin() { - kafkaSupplier = new SyncKafkaSupplier(); + kafkaSupplier = new SyncKafkaSupplier(false); } [TearDown] @@ -47,7 +47,8 @@ public void ApplyInternalChangelogTopics() NumberPartitions = 1 }); - var r = manager.ApplyAsync(0, topics).GetAwaiter().GetResult().ToList(); + var r = manager.ApplyAsync(0, topics) + .GetAwaiter().GetResult().ToList(); Assert.AreEqual(2, r.Count); Assert.AreEqual("topic", r[0]); diff --git a/test/Streamiz.Kafka.Net.Tests/Public/KafkaStreamTests.cs b/test/Streamiz.Kafka.Net.Tests/Public/KafkaStreamTests.cs index 681586e6..94f6c47a 100644 --- a/test/Streamiz.Kafka.Net.Tests/Public/KafkaStreamTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Public/KafkaStreamTests.cs @@ -633,7 +633,7 @@ public async Task BuildGlobalStateStore() config.PollMs = 1; var builder = new StreamBuilder(); - builder.GlobalTable("test", InMemory.As("store")); + builder.GlobalTable("test", InMemory.As("store")); var supplier = new SyncKafkaSupplier(); var producer = supplier.GetProducer(new ProducerConfig()); diff --git a/test/Streamiz.Kafka.Net.Tests/Public/RocksDbOptionsTests.cs b/test/Streamiz.Kafka.Net.Tests/Public/RocksDbOptionsTests.cs index 3a3052d7..a29c4ca1 100644 --- a/test/Streamiz.Kafka.Net.Tests/Public/RocksDbOptionsTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Public/RocksDbOptionsTests.cs @@ -65,11 +65,8 @@ public void Begin() .SetArenaBlockSize(arenaBlockSize) .SetBloomLocality(bloomLocality) .SetBytesPerSync(bytesPerSync) - // .SetCompactionFilter(IntPtr.Zero) - // .SetCompactionFilterFactory(IntPtr.Zero) .SetCompactionReadaheadSize(1200) .SetCompactionStyle(RocksDbSharp.Compaction.Level) - //.SetComparator(IntPtr.Zero) .SetCompression(RocksDbSharp.Compression.Lz4) .SetCompressionOptions(1, 2, 3, 4) .SetCompressionPerLevel(new[] { RocksDbSharp.Compression.Lz4 }, 1) @@ -80,9 +77,7 @@ public void Begin() .SetDeleteObsoleteFilesPeriodMicros(50) .SetDisableAutoCompactions(1) .SetEnableWriteThreadAdaptiveYield(true) - //.SetEnv(IntPtr.Zero) - .SetErrorIfExists() - //.SetFifoCompactionOptions(IntPtr.Zero) + .SetErrorIfExists(false) .SetHardPendingCompactionBytesLimit(1) .SetHashLinkListRep(12) .SetHashSkipListRep(56, 4, 2) @@ -140,7 +135,6 @@ public void Begin() .SetPrefixExtractor(SliceTransform.CreateNoOp()) .SetRecycleLogFileNum(1) .SetReportBgIoStats(true) - .SetPlainTableFactory(1, 23, 4, 2) .SetMaxBackgroundCompactions(1); }; From 44f96d6533d5dd764103d18e6b951bc4fa91a995 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Fri, 28 Jun 2024 17:09:40 -0700 Subject: [PATCH 04/29] WIP --- .../Internal/DefaultKafkaClientSupplier.cs | 1 - core/Stream/IKStream.cs | 173 +++++++++--------- 2 files changed, 87 insertions(+), 87 deletions(-) diff --git a/core/Kafka/Internal/DefaultKafkaClientSupplier.cs b/core/Kafka/Internal/DefaultKafkaClientSupplier.cs index f2606f52..df80259a 100644 --- a/core/Kafka/Internal/DefaultKafkaClientSupplier.cs +++ b/core/Kafka/Internal/DefaultKafkaClientSupplier.cs @@ -110,7 +110,6 @@ public IProducer GetProducer(ProducerConfig config) producerStatisticsHandler.Publish(statistics); }); } - return builder.Build(); } diff --git a/core/Stream/IKStream.cs b/core/Stream/IKStream.cs index f2972074..b551939d 100644 --- a/core/Stream/IKStream.cs +++ b/core/Stream/IKStream.cs @@ -7,6 +7,7 @@ using System.Collections; using System.Collections.Generic; using System.Threading.Tasks; +using Confluent.Kafka; using Streamiz.Kafka.Net.Processors.Public; using Streamiz.Kafka.Net.Processors.Internal; @@ -161,46 +162,46 @@ public interface IKStream /// Key serializer /// Value serializer /// A config used to name the processor in the topology. Default : null - void To(ITopicNameExtractor topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, string named = null); - - /// - /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. - /// The topic names for each record to send to is dynamically determined based on the Func<K, V, IRecordContext, string>. - /// - /// Extractor function to determine the name of the Kafka topic to write to for each record - /// Extractor function to determine the timestamp of the record stored in the Kafka topic - /// A config used to name the processor in the topology. Default : null - void To(Func topicExtractor, Func recordTimestampExtractor, string named = null); - - /// - /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. - /// The topic names for each record to send to is dynamically determined based on the Func<K, V, IRecordContext, string>. - /// - /// Extractor function to determine the name of the Kafka topic to write to for each record - /// Key serializer - /// Value serializer - /// Extractor function to determine the timestamp of the record stored in the Kafka topic - /// A config used to name the processor in the topology. Default : null - void To(Func topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, Func recordTimestampExtractor, string named = null); - - /// - /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. - /// The topic names for each record to send to is dynamically determined based on the }. - /// - /// The extractor to determine the name of the Kafka topic to write to for each record - /// Extractor function to determine the timestamp of the record stored in the Kafka topic - /// A config used to name the processor in the topology. Default : null - void To(ITopicNameExtractor topicExtractor, IRecordTimestampExtractor recordTimestampExtractor, string named = null); - - /// - /// Dynamically materialize this stream to a topic using serializers specified in the method parameters. - /// The topic names for each record to send to is dynamically determined based on the }. - /// - /// The extractor to determine the name of the Kafka topic to write to for each record4 - /// Key serializer - /// Value serializer - /// Extractor function to determine the timestamp of the record stored in the Kafka topic - /// A config used to name the processor in the topology. Default : null + void To(ITopicNameExtractor topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, string named = null); + + /// + /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. + /// The topic names for each record to send to is dynamically determined based on the Func<K, V, IRecordContext, string>. + /// + /// Extractor function to determine the name of the Kafka topic to write to for each record + /// Extractor function to determine the timestamp of the record stored in the Kafka topic + /// A config used to name the processor in the topology. Default : null + void To(Func topicExtractor, Func recordTimestampExtractor, string named = null); + + /// + /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. + /// The topic names for each record to send to is dynamically determined based on the Func<K, V, IRecordContext, string>. + /// + /// Extractor function to determine the name of the Kafka topic to write to for each record + /// Key serializer + /// Value serializer + /// Extractor function to determine the timestamp of the record stored in the Kafka topic + /// A config used to name the processor in the topology. Default : null + void To(Func topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, Func recordTimestampExtractor, string named = null); + + /// + /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. + /// The topic names for each record to send to is dynamically determined based on the }. + /// + /// The extractor to determine the name of the Kafka topic to write to for each record + /// Extractor function to determine the timestamp of the record stored in the Kafka topic + /// A config used to name the processor in the topology. Default : null + void To(ITopicNameExtractor topicExtractor, IRecordTimestampExtractor recordTimestampExtractor, string named = null); + + /// + /// Dynamically materialize this stream to a topic using serializers specified in the method parameters. + /// The topic names for each record to send to is dynamically determined based on the }. + /// + /// The extractor to determine the name of the Kafka topic to write to for each record4 + /// Key serializer + /// Value serializer + /// Extractor function to determine the timestamp of the record stored in the Kafka topic + /// A config used to name the processor in the topology. Default : null void To(ITopicNameExtractor topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, IRecordTimestampExtractor recordTimestampExtractor, string named = null); /// @@ -232,34 +233,34 @@ public interface IKStream /// New type value serializer /// The extractor to determine the name of the Kafka topic to write to for each record /// A config used to name the processor in the topology. Default : null - void To(ITopicNameExtractor topicExtractor, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); - - /// - /// Dynamically materialize this stream to a topic using and serializers specified in the method parameters. - /// The topic names for each record to send to is dynamically determined based on the Func<K, V, IRecordContext, string>. - /// - /// New type key serializer - /// New type value serializer - /// Extractor function to determine the name of the Kafka topic to write to for each record - /// Extractor function to determine the timestamp of the record stored in the Kafka topic - /// A config used to name the processor in the topology. Default : null - void To(Func topicExtractor, Func recordTimestampExtractor, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); - - /// - /// Dynamically materialize this stream to a topic using and serializers specified in the method parameters. - /// The topic names for each record to send to is dynamically determined based on the }. - /// - /// New type key serializer - /// New type value serializer - /// The extractor to determine the name of the Kafka topic to write to for each record - /// Extractor function to determine the timestamp of the record stored in the Kafka topic - /// A config used to name the processor in the topology. Default : null - void To(ITopicNameExtractor topicExtractor, IRecordTimestampExtractor recordTimestampExtractor, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + void To(ITopicNameExtractor topicExtractor, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + + /// + /// Dynamically materialize this stream to a topic using and serializers specified in the method parameters. + /// The topic names for each record to send to is dynamically determined based on the Func<K, V, IRecordContext, string>. + /// + /// New type key serializer + /// New type value serializer + /// Extractor function to determine the name of the Kafka topic to write to for each record + /// Extractor function to determine the timestamp of the record stored in the Kafka topic + /// A config used to name the processor in the topology. Default : null + void To(Func topicExtractor, Func recordTimestampExtractor, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + + /// + /// Dynamically materialize this stream to a topic using and serializers specified in the method parameters. + /// The topic names for each record to send to is dynamically determined based on the }. + /// + /// New type key serializer + /// New type value serializer + /// The extractor to determine the name of the Kafka topic to write to for each record + /// Extractor function to determine the timestamp of the record stored in the Kafka topic + /// A config used to name the processor in the topology. Default : null + void To(ITopicNameExtractor topicExtractor, IRecordTimestampExtractor recordTimestampExtractor, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); #endregion - + #region FlatMap - + /// /// Transform each record of the input stream into zero or more records in the output stream (bot /// can be altered arbitrarily). @@ -1982,12 +1983,12 @@ void ForeachAsync( Func, ExternalContext, Task> asyncAction, RetryPolicy retryPolicy = null, RequestSerDes requestSerDes = null, - string named = null); - + string named = null); + #endregion - + #region Process - + /// /// Process all records in this stream, one record at a time, by applying a processor (provided by the given /// . @@ -2026,12 +2027,12 @@ void ForeachAsync( /// an instance of which contains the processor and a potential state store. Use to build this supplier. /// A config used to name the processor in the topology. Default : null /// The names of the state stores used by the processor. - void Process(ProcessorSupplier processorSupplier, string named = null, params string[] storeNames); - + void Process(ProcessorSupplier processorSupplier, string named = null, params string[] storeNames); + #endregion - + #region Transform - + /// /// Transform each record of the input stream into zero or one record in the output stream (both key and value type /// can be altered arbitrarily). @@ -2076,8 +2077,8 @@ void ForeachAsync( /// the key type of the new stream /// the value type of the new stream /// a that contains more or less records with new key and value (possibly of different type) - IKStream Transform(TransformerSupplier transformerSupplier, string named = null, params string[] storeNames); - + IKStream Transform(TransformerSupplier transformerSupplier, string named = null, params string[] storeNames); + /// /// Transform each record of the input stream into zero or one record in the output stream (only value type, the original key will be through to the upstream processors ). /// A (provided by the given ) is applied to each input record and @@ -2120,20 +2121,20 @@ void ForeachAsync( /// The names of the state stores used by the processor. /// the value type of the new stream /// a that contains more or less records with new key and value (possibly of different type) - IKStream TransformValues(TransformerSupplier transformerSupplier, string named = null, params string[] storeNames); - + IKStream TransformValues(TransformerSupplier transformerSupplier, string named = null, params string[] storeNames); + #endregion - + #region WithRecordTimestamp - - /// - /// Updates the timestamp of the record with one provided by the function. Negative timestamps will be ignored. - /// - /// + + /// + /// Updates the timestamp of the record with one provided by the function. Negative timestamps will be ignored. + /// + /// /// A config used to name the processor in the topology. Default : null - /// a that has records with timestamp explicitly provided by - IKStream WithRecordTimestamp(Func timestampExtractor, string named = null); - + /// a that has records with timestamp explicitly provided by + IKStream WithRecordTimestamp(Func timestampExtractor, string named = null); + #endregion } } From b32d2b641364289cf9ad392141e0cad2f0ed1ef6 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Tue, 2 Jul 2024 14:03:23 -0700 Subject: [PATCH 05/29] #310 - WIP --- core/Kafka/IRecordCollector.cs | 3 +- core/Kafka/Internal/RecordCollector.cs | 30 ++- core/Mock/Kafka/MockCluster.cs | 3 + core/Mock/TaskSynchronousTopologyDriver.cs | 5 +- .../ExternalProcessorTopologyExecutor.cs | 5 +- core/Processors/ExternalStreamThread.cs | 3 +- core/Processors/IStreamPartitioner.cs | 9 + .../Internal/DefaultStreamPartitioner.cs | 10 + core/Processors/Internal/NodeFactory.cs | 19 +- .../Internal/WrapperStreamPartitioner.cs | 18 ++ core/Processors/SinkProcessor.cs | 26 +- core/Processors/StreamTask.cs | 3 +- core/Stream/IKStream.cs | 154 ++++++++++- .../Internal/Graph/Nodes/RepartitionNode.cs | 3 +- .../Internal/Graph/Nodes/StreamSinkNode.cs | 7 +- core/Stream/Internal/KStream.cs | 244 +++++++++++++----- core/Stream/Internal/Produced.cs | 7 +- core/Stream/Repartitioned.cs | 31 ++- .../Private/RecordCollectorTests.cs | 5 +- .../Processors/KStreamPassThoughTests.cs | 2 +- .../Processors/KStreamRepartitionTests.cs | 2 +- .../Public/ProcessorContextTests.cs | 3 +- ...ggingTimestampedKeyValueBytesStoreTests.cs | 9 +- ...LoggingTimestampedWindowBytesStoreTests.cs | 9 +- .../ChangeLoggingWindowBytesStoreTests.cs | 9 +- 25 files changed, 508 insertions(+), 111 deletions(-) create mode 100644 core/Processors/IStreamPartitioner.cs create mode 100644 core/Processors/Internal/DefaultStreamPartitioner.cs create mode 100644 core/Processors/Internal/WrapperStreamPartitioner.cs diff --git a/core/Kafka/IRecordCollector.cs b/core/Kafka/IRecordCollector.cs index f2d5f85a..4f03c8a8 100644 --- a/core/Kafka/IRecordCollector.cs +++ b/core/Kafka/IRecordCollector.cs @@ -1,4 +1,4 @@ -using System.Collections.Generic; +using System.Collections.Generic; using Confluent.Kafka; using Streamiz.Kafka.Net.SerDes; @@ -12,5 +12,6 @@ internal interface IRecordCollector void Close(); void Send(string topic, K key, V value, Headers headers, long timestamp, ISerDes keySerializer, ISerDes valueSerializer); void Send(string topic, K key, V value, Headers headers, int partition, long timestamp, ISerDes keySerializer, ISerDes valueSerializer); + int PartitionsFor(string topic); } } diff --git a/core/Kafka/Internal/RecordCollector.cs b/core/Kafka/Internal/RecordCollector.cs index e0be4679..a05d320c 100644 --- a/core/Kafka/Internal/RecordCollector.cs +++ b/core/Kafka/Internal/RecordCollector.cs @@ -7,6 +7,7 @@ using System.Collections.Concurrent; using System.Collections.Generic; using System.Diagnostics; +using System.Linq; using System.Text; using Microsoft.Extensions.Logging; using Streamiz.Kafka.Net.Metrics; @@ -81,24 +82,28 @@ public void Clear() private readonly IStreamConfig configuration; private readonly TaskId id; private readonly Sensor droppedRecordsSensor; + private readonly IAdminClient _adminClient; private Exception exception = null; private readonly string logPrefix; private readonly ILogger log = Logger.GetLogger(typeof(RecordCollector)); - private readonly ConcurrentDictionary collectorsOffsets = - new ConcurrentDictionary(); + private readonly ConcurrentDictionary collectorsOffsets = new(); - private readonly RetryRecordContext retryRecordContext = new RetryRecordContext(); + private readonly RetryRecordContext retryRecordContext = new(); public IDictionary CollectorOffsets => collectorsOffsets.ToDictionary(); - public RecordCollector(string logPrefix, IStreamConfig configuration, TaskId id, Sensor droppedRecordsSensor) + public IDictionary cachePartitionsForTopics = + new Dictionary(); + + public RecordCollector(string logPrefix, IStreamConfig configuration, TaskId id, Sensor droppedRecordsSensor, IAdminClient adminClient) { this.logPrefix = $"{logPrefix}"; this.configuration = configuration; this.id = id; this.droppedRecordsSensor = droppedRecordsSensor; + _adminClient = adminClient; } public void Init(ref IProducer producer) @@ -136,6 +141,8 @@ public void Close() } } } + + _adminClient?.Dispose(); } public void Flush() @@ -416,5 +423,20 @@ private void CheckForException() throw e; } } + + public int PartitionsFor(string topic) + { + var adminConfig = configuration.ToAdminConfig(""); + var refreshInterval = adminConfig.TopicMetadataRefreshIntervalMs ?? 5 * 60 * 1000; + + if (cachePartitionsForTopics.ContainsKey(topic) && + cachePartitionsForTopics[topic].Item2 + TimeSpan.FromMilliseconds(refreshInterval) > DateTime.Now) + return cachePartitionsForTopics[topic].Item1; + + var metadata = _adminClient.GetMetadata(topic, TimeSpan.FromSeconds(5)); + var partitionCount = metadata.Topics.FirstOrDefault(t => t.Topic.Equals(topic))!.Partitions.Count; + cachePartitionsForTopics.Add(topic, (partitionCount, DateTime.Now)); + return partitionCount; + } } } \ No newline at end of file diff --git a/core/Mock/Kafka/MockCluster.cs b/core/Mock/Kafka/MockCluster.cs index 806a59d1..6a0b5f3a 100644 --- a/core/Mock/Kafka/MockCluster.cs +++ b/core/Mock/Kafka/MockCluster.cs @@ -809,6 +809,9 @@ internal DeliveryReport Produce(string topic, Message Produce(TopicPartition topicPartition, Message message) { + if (topicPartition.Partition.Equals(Partition.Any)) + return Produce(topicPartition.Topic, message); + DeliveryReport r = new DeliveryReport(); r.Status = PersistenceStatus.NotPersisted; CreateTopic(topicPartition.Topic); diff --git a/core/Mock/TaskSynchronousTopologyDriver.cs b/core/Mock/TaskSynchronousTopologyDriver.cs index df9b06a4..08200679 100644 --- a/core/Mock/TaskSynchronousTopologyDriver.cs +++ b/core/Mock/TaskSynchronousTopologyDriver.cs @@ -78,13 +78,13 @@ public TaskSynchronousTopologyDriver(string clientId, InternalTopologyBuilder to partitionsByTaskId.Add(taskId, new List {part}); } + var adminClient = this.supplier.GetAdmin(configuration.ToAdminConfig($"{clientId}-admin")); ProcessorTopology globalTaskTopology = topologyBuilder.BuildGlobalStateTopology(); hasGlobalTopology = globalTaskTopology != null; if (hasGlobalTopology) { var globalConsumer = this.supplier.GetGlobalConsumer(configuration.ToGlobalConsumerConfig($"{clientId}-global-consumer")); - var adminClient = this.supplier.GetAdmin(configuration.ToAdminConfig($"{clientId}-admin")); var stateManager = new GlobalStateManager(globalConsumer, globalTaskTopology, adminClient, configuration); globalProcessorContext = new GlobalProcessorContext(configuration, stateManager, metricsRegistry); @@ -105,7 +105,8 @@ public TaskSynchronousTopologyDriver(string clientId, InternalTopologyBuilder to topologyBuilder.BuildTopology(taskId).GetSourceProcessor(requestTopic), this.supplier.GetProducer(configuration.ToProducerConfig($"ext-thread-producer-{requestTopic}")), configuration, - metricsRegistry)); + metricsRegistry, + adminClient)); } } diff --git a/core/Processors/ExternalProcessorTopologyExecutor.cs b/core/Processors/ExternalProcessorTopologyExecutor.cs index d3f9f6bf..76e681d2 100644 --- a/core/Processors/ExternalProcessorTopologyExecutor.cs +++ b/core/Processors/ExternalProcessorTopologyExecutor.cs @@ -125,7 +125,8 @@ public ExternalProcessorTopologyExecutor( ISourceProcessor sourceProcessor, IProducer producer, IStreamConfig config, - StreamMetricsRegistry streamMetricsRegistry) + StreamMetricsRegistry streamMetricsRegistry, + IAdminClient adminClient) { this.threadId = threadId; this.producerId = producer.Name; @@ -140,7 +141,7 @@ public ExternalProcessorTopologyExecutor( processSensor = TaskMetrics.ProcessSensor(threadId, taskId, streamMetricsRegistry); processLatencySensor = TaskMetrics.ProcessLatencySensor(threadId, taskId, streamMetricsRegistry); - recordCollector = new RecordCollector(logPrefix, config, taskId, droppedRecordsSensor); + recordCollector = new RecordCollector(logPrefix, config, taskId, droppedRecordsSensor, adminClient); recordCollector.Init(ref producer); context = new ProcessorContext(ExternalStreamTask.Create(taskId), config, null, streamMetricsRegistry); diff --git a/core/Processors/ExternalStreamThread.cs b/core/Processors/ExternalStreamThread.cs index 9020636b..480045fb 100644 --- a/core/Processors/ExternalStreamThread.cs +++ b/core/Processors/ExternalStreamThread.cs @@ -372,7 +372,8 @@ private ExternalProcessorTopologyExecutor GetExternalProcessorTopology(string to topology.GetSourceProcessor(topic), producer, configuration, - streamMetricsRegistry); + streamMetricsRegistry, + adminClient); externalProcessorTopologies.Add(topic, externalProcessorTopologyExecutor); return externalProcessorTopologyExecutor; diff --git a/core/Processors/IStreamPartitioner.cs b/core/Processors/IStreamPartitioner.cs new file mode 100644 index 00000000..ad67c0c9 --- /dev/null +++ b/core/Processors/IStreamPartitioner.cs @@ -0,0 +1,9 @@ +using Confluent.Kafka; + +namespace Streamiz.Kafka.Net.Processors +{ + public interface IStreamPartitioner + { + Partition Partition(string topic, K key, V value, int numPartitions); + } +} \ No newline at end of file diff --git a/core/Processors/Internal/DefaultStreamPartitioner.cs b/core/Processors/Internal/DefaultStreamPartitioner.cs new file mode 100644 index 00000000..14627afb --- /dev/null +++ b/core/Processors/Internal/DefaultStreamPartitioner.cs @@ -0,0 +1,10 @@ +using Confluent.Kafka; + +namespace Streamiz.Kafka.Net.Processors.Internal +{ + internal class DefaultStreamPartitioner : IStreamPartitioner + { + public Partition Partition(string topic, K key, V value, int numPartitions) + => Confluent.Kafka.Partition.Any; + } +} \ No newline at end of file diff --git a/core/Processors/Internal/NodeFactory.cs b/core/Processors/Internal/NodeFactory.cs index 3527a292..b92abc06 100644 --- a/core/Processors/Internal/NodeFactory.cs +++ b/core/Processors/Internal/NodeFactory.cs @@ -80,21 +80,16 @@ internal class SinkNodeFactory : NodeFactory, ISinkNodeFactory public IRecordTimestampExtractor TimestampExtractor { get; } public ISerDes KeySerdes { get; } public ISerDes ValueSerdes { get; } - public Func ProducedPartitioner { get; } + public IStreamPartitioner ProducedPartitioner { get; } - public string Topic - { - get - { - return TopicExtractor is StaticTopicNameExtractor ? - ((StaticTopicNameExtractor)TopicExtractor).TopicName : - null; - } - } + public string Topic => + (TopicExtractor as StaticTopicNameExtractor)?.TopicName; - public SinkNodeFactory(string name, string[] previous, ITopicNameExtractor topicExtractor, + public SinkNodeFactory(string name, string[] previous, ITopicNameExtractor topicExtractor, IRecordTimestampExtractor timestampExtractor, - ISerDes keySerdes, ISerDes valueSerdes, Func producedPartitioner) + ISerDes keySerdes, + ISerDes valueSerdes, + IStreamPartitioner producedPartitioner) : base(name, previous) { TopicExtractor = topicExtractor; diff --git a/core/Processors/Internal/WrapperStreamPartitioner.cs b/core/Processors/Internal/WrapperStreamPartitioner.cs new file mode 100644 index 00000000..8206ca44 --- /dev/null +++ b/core/Processors/Internal/WrapperStreamPartitioner.cs @@ -0,0 +1,18 @@ +using System; +using Confluent.Kafka; + +namespace Streamiz.Kafka.Net.Processors.Internal +{ + internal class WrapperStreamPartitioner : IStreamPartitioner + { + private readonly Func _partitioner; + + public WrapperStreamPartitioner(Func partitioner) + { + _partitioner = partitioner; + } + + public Partition Partition(string topic, K key, V value, int numPartitions) + => _partitioner(topic, key, value, numPartitions); + } +} \ No newline at end of file diff --git a/core/Processors/SinkProcessor.cs b/core/Processors/SinkProcessor.cs index 63c4a128..f85db35d 100644 --- a/core/Processors/SinkProcessor.cs +++ b/core/Processors/SinkProcessor.cs @@ -1,4 +1,5 @@ using System; +using Confluent.Kafka; using Microsoft.Extensions.Logging; using Streamiz.Kafka.Net.Errors; using Streamiz.Kafka.Net.Processors.Internal; @@ -15,9 +16,15 @@ internal class SinkProcessor : AbstractProcessor, ISinkProcessor { private ITopicNameExtractor topicNameExtractor; private readonly IRecordTimestampExtractor timestampExtractor; - private readonly Func partitioner; + private readonly IStreamPartitioner partitioner; - internal SinkProcessor(string name, ITopicNameExtractor topicNameExtractor, IRecordTimestampExtractor timestampExtractor, ISerDes keySerdes, ISerDes valueSerdes, Func partitioner = null) + internal SinkProcessor( + string name, + ITopicNameExtractor topicNameExtractor, + IRecordTimestampExtractor timestampExtractor, + ISerDes keySerdes, + ISerDes valueSerdes, + IStreamPartitioner partitioner = null) : base(name, keySerdes, valueSerdes) { this.topicNameExtractor = topicNameExtractor; @@ -62,11 +69,20 @@ public override void Process(K key, V value) { throw new StreamsException($"Invalid (negative) timestamp of {timestamp} for output record <{key}:{value}>."); } - + if (partitioner != null) { - int partition = partitioner.Invoke(topicName, key, value); - Context.RecordCollector.Send(topicName, key, value, Context.RecordContext.Headers, partition, timestamp, KeySerDes, + Partition partition = partitioner.Partition( + topicName, + key, value, + Context.RecordCollector.PartitionsFor(topicName)); + + Context.RecordCollector.Send( + topicName, + key, value, + Context.RecordContext.Headers, + partition, + timestamp, KeySerDes, ValueSerDes); } else diff --git a/core/Processors/StreamTask.cs b/core/Processors/StreamTask.cs index 10b004c0..7c2c4096 100644 --- a/core/Processors/StreamTask.cs +++ b/core/Processors/StreamTask.cs @@ -73,7 +73,8 @@ public StreamTask(string threadId, TaskId id, IEnumerable partit } var droppedRecordsSensor = TaskMetrics.DroppedRecordsSensor(this.threadId, Id, this.streamMetricsRegistry); - collector = new RecordCollector(logPrefix, configuration, id, droppedRecordsSensor); + var adminClient = kafkaSupplier.GetAdmin(configuration.ToAdminConfig(this.threadId)); + collector = new RecordCollector(logPrefix, configuration, id, droppedRecordsSensor, adminClient); collector.Init(ref this.producer); Context = new ProcessorContext(this, configuration, stateMgr, streamMetricsRegistry) diff --git a/core/Stream/IKStream.cs b/core/Stream/IKStream.cs index b551939d..04f24222 100644 --- a/core/Stream/IKStream.cs +++ b/core/Stream/IKStream.cs @@ -127,6 +127,20 @@ public interface IKStream /// Throw if is null /// /// Throw if is incorrect void To(string topicName, ISerDes keySerdes, ISerDes valueSerdes, string named = null); + + /// + /// Materialize this stream to a topic using serializers specified in the method parameters. + /// The specified topic should be manually created before it is used(i.e., before the Kafka Streams application is + /// started). + /// + /// the topic name + /// The function used to determine how records are distributed among partitions of the topic + /// Key serializer + /// Value serializer + /// A config used to name the processor in the topology. Default : null + /// Throw if is null + /// /// Throw if is incorrect + void To(string topicName, Func partitioner, ISerDes keySerdes, ISerDes valueSerdes, string named = null); /// /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. @@ -135,6 +149,15 @@ public interface IKStream /// Extractor function to determine the name of the Kafka topic to write to for each record /// A config used to name the processor in the topology. Default : null void To(Func topicExtractor, string named = null); + + /// + /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. + /// The topic names for each record to send to is dynamically determined based on the Func<K, V, IRecordContext, string>. + /// + /// Extractor function to determine the name of the Kafka topic to write to for each record + /// The function used to determine how records are distributed among partitions of the topic + /// A config used to name the processor in the topology. Default : null + void To(Func topicExtractor, Func partitioner, string named = null); /// /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. @@ -146,6 +169,18 @@ public interface IKStream /// A config used to name the processor in the topology. Default : null void To(Func topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, string named = null); + /// + /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. + /// The topic names for each record to send to is dynamically determined based on the Func<K, V, IRecordContext, string>. + /// + /// Extractor function to determine the name of the Kafka topic to write to for each record + /// The function used to determine how records are distributed among partitions of the topic + /// Key serializer + /// Value serializer + /// A config used to name the processor in the topology. Default : null + void To(Func topicExtractor, Func partitioner, ISerDes keySerdes, ISerDes valueSerdes, string named = null); + + /// /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. /// The topic names for each record to send to is dynamically determined based on the }. @@ -153,17 +188,29 @@ public interface IKStream /// The extractor to determine the name of the Kafka topic to write to for each record /// A config used to name the processor in the topology. Default : null void To(ITopicNameExtractor topicExtractor, string named = null); - + /// /// Dynamically materialize this stream to a topic using serializers specified in the method parameters. /// The topic names for each record to send to is dynamically determined based on the }. /// - /// The extractor to determine the name of the Kafka topic to write to for each record4 + /// The extractor to determine the name of the Kafka topic to write to for each record /// Key serializer /// Value serializer - /// A config used to name the processor in the topology. Default : null + /// A onfig used to name the processor in the topology. Default : null void To(ITopicNameExtractor topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, string named = null); + + /// + /// Dynamically materialize this stream to a topic using serializers specified in the method parameters. + /// The topic names for each record to send to is dynamically determined based on the }. + /// + /// The extractor to determine the name of the Kafka topic to write to for each record + /// The function used to determine how records are distributed among partitions of the topic + /// Key serializer + /// Value serializer + /// A onfig used to name the processor in the topology. Default : null + void To(ITopicNameExtractor topicExtractor, IStreamPartitioner partitioner, ISerDes keySerdes, ISerDes valueSerdes, string named = null); + /// /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. /// The topic names for each record to send to is dynamically determined based on the Func<K, V, IRecordContext, string>. @@ -172,6 +219,16 @@ public interface IKStream /// Extractor function to determine the timestamp of the record stored in the Kafka topic /// A config used to name the processor in the topology. Default : null void To(Func topicExtractor, Func recordTimestampExtractor, string named = null); + + /// + /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. + /// The topic names for each record to send to is dynamically determined based on the Func<K, V, IRecordContext, string>. + /// + /// Extractor function to determine the name of the Kafka topic to write to for each record + /// Extractor function to determine the timestamp of the record stored in the Kafka topic + /// The function used to determine how records are distributed among partitions of the topic + /// A config used to name the processor in the topology. Default : null + void To(Func topicExtractor, Func recordTimestampExtractor, Func partitioner, string named = null); /// /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. @@ -193,6 +250,25 @@ public interface IKStream /// A config used to name the processor in the topology. Default : null void To(ITopicNameExtractor topicExtractor, IRecordTimestampExtractor recordTimestampExtractor, string named = null); + /// + /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. + /// The topic names for each record to send to is dynamically determined based on the }. + /// + /// The extractor to determine the name of the Kafka topic to write to for each record + /// The function used to determine how records are distributed among partitions of the topic + /// A config used to name the processor in the topology. Default : null + void To(ITopicNameExtractor topicExtractor, IStreamPartitioner partitioner, string named = null); + + /// + /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. + /// The topic names for each record to send to is dynamically determined based on the }. + /// + /// The extractor to determine the name of the Kafka topic to write to for each record + /// Extractor function to determine the timestamp of the record stored in the Kafka topic + /// The function used to determine how records are distributed among partitions of the topic + /// A config used to name the processor in the topology. Default : null + void To(ITopicNameExtractor topicExtractor, IRecordTimestampExtractor recordTimestampExtractor, IStreamPartitioner partitioner, string named = null); + /// /// Dynamically materialize this stream to a topic using serializers specified in the method parameters. /// The topic names for each record to send to is dynamically determined based on the }. @@ -204,6 +280,18 @@ public interface IKStream /// A config used to name the processor in the topology. Default : null void To(ITopicNameExtractor topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, IRecordTimestampExtractor recordTimestampExtractor, string named = null); + /// + /// Dynamically materialize this stream to a topic using serializers specified in the method parameters. + /// The topic names for each record to send to is dynamically determined based on the }. + /// + /// The extractor to determine the name of the Kafka topic to write to for each record4 + /// Key serializer + /// Value serializer + /// Extractor function to determine the timestamp of the record stored in the Kafka topic + /// The function used to determine how records are distributed among partitions of the topic + /// A config used to name the processor in the topology. Default : null + void To(ITopicNameExtractor topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, IRecordTimestampExtractor recordTimestampExtractor, IStreamPartitioner partitioner, string named = null); + /// /// Materialize this stream to a topic using and serializers specified in the method parameters. /// The specified topic should be manually created before it is used(i.e., before the Kafka Streams application is @@ -214,6 +302,18 @@ public interface IKStream /// the topic name /// A config used to name the processor in the topology. Default : null void To(string topicName, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + + /// + /// Materialize this stream to a topic using and serializers specified in the method parameters. + /// The specified topic should be manually created before it is used(i.e., before the Kafka Streams application is + /// started). + /// + /// New type key serializer + /// New type value serializer + /// the topic name + /// The function used to determine how records are distributed among partitions of the topic + /// A config used to name the processor in the topology. Default : null + void To(string topicName, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); /// /// Dynamically materialize this stream to a topic using and serializers specified in the method parameters. @@ -222,9 +322,20 @@ public interface IKStream /// New type key serializer /// New type value serializer /// Extractor function to determine the name of the Kafka topic to write to for each record + /// The function used to determine how records are distributed among partitions of the topic /// A config used to name the processor in the topology. Default : null - void To(Func topicExtractor, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + void To(Func topicExtractor, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + /// + /// Dynamically materialize this stream to a topic using and serializers specified in the method parameters. + /// The topic names for each record to send to is dynamically determined based on the Func<K, V, IRecordContext, string>. + /// + /// New type key serializer + /// New type value serializer + /// Extractor function to determine the name of the Kafka topic to write to for each record + /// A config used to name the processor in the topology. Default : null + void To(Func topicExtractor, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + /// /// Dynamically materialize this stream to a topic using and serializers specified in the method parameters. /// The topic names for each record to send to is dynamically determined based on the }. @@ -235,6 +346,17 @@ public interface IKStream /// A config used to name the processor in the topology. Default : null void To(ITopicNameExtractor topicExtractor, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + /// + /// Dynamically materialize this stream to a topic using and serializers specified in the method parameters. + /// The topic names for each record to send to is dynamically determined based on the }. + /// + /// New type key serializer + /// New type value serializer + /// The extractor to determine the name of the Kafka topic to write to for each record + /// The function used to determine how records are distributed among partitions of the topic + /// A config used to name the processor in the topology. Default : null + void To(ITopicNameExtractor topicExtractor, IStreamPartitioner partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + /// /// Dynamically materialize this stream to a topic using and serializers specified in the method parameters. /// The topic names for each record to send to is dynamically determined based on the Func<K, V, IRecordContext, string>. @@ -246,6 +368,18 @@ public interface IKStream /// A config used to name the processor in the topology. Default : null void To(Func topicExtractor, Func recordTimestampExtractor, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + /// + /// Dynamically materialize this stream to a topic using and serializers specified in the method parameters. + /// The topic names for each record to send to is dynamically determined based on the Func<K, V, IRecordContext, string>. + /// + /// New type key serializer + /// New type value serializer + /// Extractor function to determine the name of the Kafka topic to write to for each record + /// Extractor function to determine the timestamp of the record stored in the Kafka topic + /// The function used to determine how records are distributed among partitions of the topic + /// A config used to name the processor in the topology. Default : null + void To(Func topicExtractor, Func recordTimestampExtractor, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + /// /// Dynamically materialize this stream to a topic using and serializers specified in the method parameters. /// The topic names for each record to send to is dynamically determined based on the }. @@ -257,6 +391,18 @@ public interface IKStream /// A config used to name the processor in the topology. Default : null void To(ITopicNameExtractor topicExtractor, IRecordTimestampExtractor recordTimestampExtractor, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + /// + /// Dynamically materialize this stream to a topic using and serializers specified in the method parameters. + /// The topic names for each record to send to is dynamically determined based on the }. + /// + /// New type key serializer + /// New type value serializer + /// The extractor to determine the name of the Kafka topic to write to for each record + /// Extractor function to determine the timestamp of the record stored in the Kafka topic + /// The function used to determine how records are distributed among partitions of the topic + /// A config used to name the processor in the topology. Default : null + void To(ITopicNameExtractor topicExtractor, IRecordTimestampExtractor recordTimestampExtractor, IStreamPartitioner partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + #endregion #region FlatMap diff --git a/core/Stream/Internal/Graph/Nodes/RepartitionNode.cs b/core/Stream/Internal/Graph/Nodes/RepartitionNode.cs index ce4f14e9..292bb265 100644 --- a/core/Stream/Internal/Graph/Nodes/RepartitionNode.cs +++ b/core/Stream/Internal/Graph/Nodes/RepartitionNode.cs @@ -1,4 +1,5 @@ using System; +using Streamiz.Kafka.Net.Processors; using Streamiz.Kafka.Net.Processors.Internal; using Streamiz.Kafka.Net.SerDes; @@ -12,7 +13,7 @@ public RepartitionNode(string streamGraphNode, string sourceName, ProcessorParam } public int? NumberOfPartition { get; set; } - public Func StreamPartitioner { get; set; } + public IStreamPartitioner StreamPartitioner { get; set; } public override void WriteToTopology(InternalTopologyBuilder builder) { diff --git a/core/Stream/Internal/Graph/Nodes/StreamSinkNode.cs b/core/Stream/Internal/Graph/Nodes/StreamSinkNode.cs index 2ed39ac2..0ccb0663 100644 --- a/core/Stream/Internal/Graph/Nodes/StreamSinkNode.cs +++ b/core/Stream/Internal/Graph/Nodes/StreamSinkNode.cs @@ -17,7 +17,10 @@ internal class StreamSinkNode : StreamSinkNode private readonly Produced produced; private readonly IRecordTimestampExtractor timestampExtractor; - public StreamSinkNode(ITopicNameExtractor topicNameExtractor, IRecordTimestampExtractor timestampExtractor, string streamGraphNode, Produced produced) + public StreamSinkNode( + ITopicNameExtractor topicNameExtractor, + IRecordTimestampExtractor timestampExtractor, + string streamGraphNode, Produced produced) : base(streamGraphNode) { this.topicNameExtractor = topicNameExtractor; @@ -27,7 +30,7 @@ public StreamSinkNode(ITopicNameExtractor topicNameExtractor, IRecordTimes public override void WriteToTopology(InternalTopologyBuilder builder) { - builder.AddSinkOperator(topicNameExtractor, timestampExtractor, this.streamGraphNode, produced, ParentNodeNames()); + builder.AddSinkOperator(topicNameExtractor, timestampExtractor, streamGraphNode, produced, ParentNodeNames()); } } } diff --git a/core/Stream/Internal/KStream.cs b/core/Stream/Internal/KStream.cs index b3bb2076..50a2da76 100644 --- a/core/Stream/Internal/KStream.cs +++ b/core/Stream/Internal/KStream.cs @@ -13,6 +13,7 @@ using System.Collections.Generic; using System.Linq; using System.Threading.Tasks; +using Confluent.Kafka; using Streamiz.Kafka.Net.Processors.Public; namespace Streamiz.Kafka.Net.Stream.Internal @@ -178,55 +179,151 @@ public IKStream Repartition(Repartitioned repartitioned = null) public void To(string topicName, string named = null) { - if (topicName == null) - { - throw new ArgumentNullException(nameof(topicName)); - } - if (string.IsNullOrEmpty(topicName)) { - throw new ArgumentException("topicName must be empty"); + throw new ArgumentException("topicName must not be null or empty"); } - To(new StaticTopicNameExtractor(topicName), named); + To(new StaticTopicNameExtractor(topicName), new DefaultStreamPartitioner(), named); } public void To(string topicName, ISerDes keySerdes, ISerDes valueSerdes, string named = null) { - if (topicName == null) - { - throw new ArgumentNullException(nameof(topicName)); - } + if (string.IsNullOrEmpty(topicName)) + throw new ArgumentException("topicName must not be null or empty"); + + To(new StaticTopicNameExtractor(topicName), new DefaultStreamPartitioner(), keySerdes, valueSerdes, named); + } + public void To(string topicName, Func partitioner, ISerDes keySerdes, ISerDes valueSerdes, string named = null) + { if (string.IsNullOrEmpty(topicName)) - { throw new ArgumentException("topicName must be empty"); - } - To(new StaticTopicNameExtractor(topicName), keySerdes, valueSerdes, named); + To(new StaticTopicNameExtractor(topicName), + new WrapperStreamPartitioner(partitioner), + keySerdes, valueSerdes, named); } - public void To(ITopicNameExtractor topicExtractor, string named = null) => DoTo(topicExtractor, new DefaultRecordTimestampExtractor(), Produced.Create(KeySerdes, ValueSerdes).WithName(named)); + public void To(Func topicExtractor, + Func partitioner, ISerDes keySerdes, ISerDes valueSerdes, + string named = null) + => To( + new WrapperTopicNameExtractor(topicExtractor), + KeySerdes, ValueSerdes, + new DefaultRecordTimestampExtractor(), + new WrapperStreamPartitioner(partitioner), + named); - public void To(ITopicNameExtractor topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, string named = null) - => DoTo(topicExtractor, new DefaultRecordTimestampExtractor(), Produced.Create(keySerdes, valueSerdes).WithName(named)); + public void To(ITopicNameExtractor topicExtractor, string named = null) + => To( + topicExtractor, + KeySerdes, ValueSerdes, + new DefaultRecordTimestampExtractor(), + new DefaultStreamPartitioner(), + named); - public void To(Func topicExtractor, string named = null) => To(new WrapperTopicNameExtractor(topicExtractor), named); + public void To(ITopicNameExtractor topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, + string named = null) + => To(topicExtractor, + keySerdes, valueSerdes, + new DefaultRecordTimestampExtractor(), + new DefaultStreamPartitioner(), + named); + + public void To(ITopicNameExtractor topicExtractor, IStreamPartitioner partitioner, + ISerDes keySerdes, ISerDes valueSerdes, + string named = null) + => To(topicExtractor, + keySerdes, valueSerdes, + new DefaultRecordTimestampExtractor(), + partitioner, + named); + + public void To(Func topicExtractor, string named = null) + => To(new WrapperTopicNameExtractor(topicExtractor), named); + + public void To(Func topicExtractor, + Func partitioner, string named = null) + => To(new WrapperTopicNameExtractor(topicExtractor), + new WrapperStreamPartitioner(partitioner), + named); public void To(Func topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, string named = null) - => To(new WrapperTopicNameExtractor(topicExtractor), keySerdes, valueSerdes, named); - - public void To(Func topicExtractor, Func recordTimestampExtractor, string named = null) => - DoTo(new WrapperTopicNameExtractor(topicExtractor), new WrapperRecordTimestampExtractor(recordTimestampExtractor), Produced.Create(KeySerdes, ValueSerdes).WithName(named)); - - public void To(Func topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, Func recordTimestampExtractor, string named = null) => - DoTo(new WrapperTopicNameExtractor(topicExtractor), new WrapperRecordTimestampExtractor(recordTimestampExtractor), Produced.Create(keySerdes, valueSerdes).WithName(named)); - - public void To(ITopicNameExtractor topicExtractor, IRecordTimestampExtractor recordTimestampExtractor, string named = null) => - DoTo(topicExtractor, recordTimestampExtractor, Produced.Create(KeySerdes, ValueSerdes).WithName(named)); - - public void To(ITopicNameExtractor topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, IRecordTimestampExtractor recordTimestampExtractor, string named = null) => - DoTo(topicExtractor, recordTimestampExtractor, Produced.Create(keySerdes, valueSerdes).WithName(named)); + => To(new WrapperTopicNameExtractor(topicExtractor), keySerdes, valueSerdes, named); + + public void To(Func topicExtractor, + Func recordTimestampExtractor, string named = null) + => To( + new WrapperTopicNameExtractor(topicExtractor), + KeySerdes, ValueSerdes, + new WrapperRecordTimestampExtractor(recordTimestampExtractor), + new DefaultStreamPartitioner(), + named); + + public void To(Func topicExtractor, + Func recordTimestampExtractor, Func partitioner, + string named = null) + => To( + new WrapperTopicNameExtractor(topicExtractor), + KeySerdes, ValueSerdes, + new WrapperRecordTimestampExtractor(recordTimestampExtractor), + new WrapperStreamPartitioner(partitioner), + named); + + public void To(Func topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, + Func recordTimestampExtractor, string named = null) + => To(new WrapperTopicNameExtractor(topicExtractor), + keySerdes, valueSerdes, + new WrapperRecordTimestampExtractor(recordTimestampExtractor), + new DefaultStreamPartitioner(), + named); + + public void To(ITopicNameExtractor topicExtractor, + IRecordTimestampExtractor recordTimestampExtractor, string named = null) + => To(topicExtractor, + KeySerdes, ValueSerdes, + recordTimestampExtractor, + new DefaultStreamPartitioner(), + named); + + public void To(ITopicNameExtractor topicExtractor, IStreamPartitioner partitioner, + string named = null) + => To(topicExtractor, + KeySerdes, ValueSerdes, + new DefaultRecordTimestampExtractor(), + partitioner, + named); + + public void To(ITopicNameExtractor topicExtractor, IRecordTimestampExtractor recordTimestampExtractor, + IStreamPartitioner partitioner, string named = null) + => To(topicExtractor, + KeySerdes, ValueSerdes, + recordTimestampExtractor, + partitioner, + named); + + public void To(ITopicNameExtractor topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, IRecordTimestampExtractor recordTimestampExtractor, string named = null) + => To( + topicExtractor, + keySerdes, valueSerdes, + recordTimestampExtractor, + new DefaultStreamPartitioner(), + named); + + public void To(ITopicNameExtractor topicExtractor, ISerDes keySerdes, ISerDes valueSerdes, + IRecordTimestampExtractor recordTimestampExtractor, IStreamPartitioner partitioner, string named = null) + => DoTo( + topicExtractor, + recordTimestampExtractor, + partitioner, + Produced.Create(keySerdes, valueSerdes).WithName(named)); + + public void To(Func topicExtractor, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new() + => To( + new WrapperTopicNameExtractor(topicExtractor), + new WrapperStreamPartitioner(partitioner), + named); public void To(Func topicExtractor, string named = null) where KS : ISerDes, new() @@ -238,20 +335,44 @@ public void To(string topicName, string named = null) where VS : ISerDes, new() => To(new StaticTopicNameExtractor(topicName), named); + public void To(string topicName, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new() + => To(new StaticTopicNameExtractor(topicName), + new WrapperStreamPartitioner(partitioner), + named); + public void To(ITopicNameExtractor topicExtractor, string named = null) where KS : ISerDes, new() where VS : ISerDes, new() - => DoTo(topicExtractor, new DefaultRecordTimestampExtractor(), Produced.Create().WithName(named)); - - public void To(Func topicExtractor, Func recordTimestampExtractor, string named = null) - where KS : ISerDes, new() - where VS : ISerDes, new() - => To(new WrapperTopicNameExtractor(topicExtractor), new WrapperRecordTimestampExtractor(recordTimestampExtractor), named); - - public void To(ITopicNameExtractor topicExtractor, IRecordTimestampExtractor recordTimestampExtractor, string named = null) - where KS : ISerDes, new() - where VS : ISerDes, new() - => DoTo(topicExtractor, recordTimestampExtractor, Produced.Create().WithName(named)); + => To(topicExtractor, new DefaultStreamPartitioner(), named); + + public void To(ITopicNameExtractor topicExtractor, IStreamPartitioner partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new() + => To( + topicExtractor, + new DefaultRecordTimestampExtractor(), + partitioner, + named); + + public void To(Func topicExtractor, Func recordTimestampExtractor, string named = null) + where KS : ISerDes, new() + where VS : ISerDes, new() + => To(new WrapperTopicNameExtractor(topicExtractor), new WrapperRecordTimestampExtractor(recordTimestampExtractor), named); + + public void To(Func topicExtractor, Func recordTimestampExtractor, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new() + => To(new WrapperTopicNameExtractor(topicExtractor), new WrapperRecordTimestampExtractor(recordTimestampExtractor), new WrapperStreamPartitioner(partitioner), named); + + public void To(ITopicNameExtractor topicExtractor, + IRecordTimestampExtractor recordTimestampExtractor, string named = null) + where KS : ISerDes, new() + where VS : ISerDes, new() + => To(topicExtractor, recordTimestampExtractor, new DefaultStreamPartitioner(), named); + + public void To(ITopicNameExtractor topicExtractor, IRecordTimestampExtractor recordTimestampExtractor, + IStreamPartitioner partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new() + => DoTo( + topicExtractor, + recordTimestampExtractor, + partitioner, + Produced.Create().WithName(named)); #endregion @@ -1018,10 +1139,10 @@ public IKTable ToTable(Materialized> m tableSource, tableNode, builder); - } - + } + #endregion - + #region WithRecordTimestamp public IKStream WithRecordTimestamp(Func timestampExtractor, string named = null) @@ -1029,22 +1150,22 @@ public IKStream WithRecordTimestamp(Func timestampExtractor, s if (timestampExtractor == null) { throw new ArgumentNullException($"Extractor function can't be null"); - } - + } + String name = new Named(named).OrElseGenerateWithPrefix(builder, KStream.RECORD_TIMESTAMP_NAME); ProcessorParameters processorParameters = new ProcessorParameters(new KStreamTimestampExtractor(timestampExtractor), name); ProcessorGraphNode timestampExtractorNode = new ProcessorGraphNode(name, processorParameters); - builder.AddGraphNode(Node, timestampExtractorNode); - + builder.AddGraphNode(Node, timestampExtractorNode); + return new KStream(name, KeySerdes, ValueSerdes, SetSourceNodes, RepartitionRequired, timestampExtractorNode, builder); } - + #endregion - + #region Private - + private IKStream DoFilter(Func predicate, string named, bool not) { if (predicate == null) @@ -1058,13 +1179,17 @@ private IKStream DoFilter(Func predicate, string named, bool n builder.AddGraphNode(Node, filterProcessorNode); return new KStream(name, KeySerdes, ValueSerdes, SetSourceNodes, RepartitionRequired, filterProcessorNode, builder); - } - - private void DoTo(ITopicNameExtractor topicExtractor, IRecordTimestampExtractor timestampExtractor, Produced produced) + } + + private void DoTo(ITopicNameExtractor topicExtractor, IRecordTimestampExtractor timestampExtractor, IStreamPartitioner partitioner, Produced produced) { string name = new Named(produced.Named).OrElseGenerateWithPrefix(builder, KStream.SINK_NAME); - StreamSinkNode sinkNode = new StreamSinkNode(topicExtractor, timestampExtractor, name, produced); + StreamSinkNode sinkNode = new StreamSinkNode( + topicExtractor, + timestampExtractor, + name, + produced.WithPartitioner(partitioner)); builder.AddGraphNode(Node, sinkNode); } @@ -1342,8 +1467,11 @@ string asyncProcessorName responseSinkProcessorName, responseSourceProcessorName, asyncProcessorName); - } - + } + #endregion + + + } } \ No newline at end of file diff --git a/core/Stream/Internal/Produced.cs b/core/Stream/Internal/Produced.cs index 35f79a94..dcb96a5c 100644 --- a/core/Stream/Internal/Produced.cs +++ b/core/Stream/Internal/Produced.cs @@ -1,4 +1,5 @@ using System; +using Streamiz.Kafka.Net.Processors; using Streamiz.Kafka.Net.SerDes; namespace Streamiz.Kafka.Net.Stream.Internal @@ -9,7 +10,7 @@ internal class Produced internal ISerDes ValueSerdes { get; } internal string Named { get; private set; } - internal Func Partitioner { get; private set; } + internal IStreamPartitioner Partitioner { get; private set; } internal Produced(ISerDes keySerdes, ISerDes valueSerdes) { @@ -35,9 +36,9 @@ internal Produced WithName(string named) return this; } - internal Produced WithPartitioner(Func partitioner) + internal Produced WithPartitioner(IStreamPartitioner partitioner) { - this.Partitioner = partitioner; + Partitioner = partitioner; return this; } } diff --git a/core/Stream/Repartitioned.cs b/core/Stream/Repartitioned.cs index 0ab205f9..9337bbbd 100644 --- a/core/Stream/Repartitioned.cs +++ b/core/Stream/Repartitioned.cs @@ -1,4 +1,7 @@ using System; +using Confluent.Kafka; +using Streamiz.Kafka.Net.Processors; +using Streamiz.Kafka.Net.Processors.Internal; using Streamiz.Kafka.Net.SerDes; namespace Streamiz.Kafka.Net.Stream @@ -14,14 +17,14 @@ public class Repartitioned internal ISerDes KeySerdes { get; private set; } internal ISerDes ValueSerdes { get; private set; } internal int? NumberOfPartition { get; private set; } - internal Func StreamPartitioner { get; private set; } + internal IStreamPartitioner StreamPartitioner { get; private set; } private Repartitioned( string named, ISerDes keySerdes, ISerDes valueSerdes, int? numberOfPartition, - Func streamPartitioner) + IStreamPartitioner streamPartitioner) { Named = named; KeySerdes = keySerdes; @@ -72,7 +75,16 @@ public static Repartitioned Create() /// /// the function used to determine how records are distributed among partitions of the topic /// A new instance of - public static Repartitioned Partitioner(Func streamPartitioner) => new(null, null, null, null, streamPartitioner); + public static Repartitioned Partitioner(IStreamPartitioner streamPartitioner) => new(null, null, null, null, streamPartitioner); + + /// + /// Create a instance with provided partitioner. + /// + /// the function used to determine how records are distributed among partitions of the topic + /// A new instance of + public static Repartitioned Partitioner(Func streamPartitioner) + => Partitioner(new WrapperStreamPartitioner(streamPartitioner)); + #endregion @@ -127,12 +139,21 @@ public Repartitioned WithNumberOfPartitions(int numberOfPartitions) /// /// Function to determine the partition where the repartition record will be persist /// this - public Repartitioned WithStreamPartitioner(Func streamPartitioner) + public Repartitioned WithStreamPartitioner(IStreamPartitioner streamPartitioner) { StreamPartitioner = streamPartitioner; return this; } - + + /// + /// Set the stream partitioner + /// + /// Function to determine the partition where the repartition record will be persist + /// this + public Repartitioned WithStreamPartitioner(Func streamPartitioner) + => WithStreamPartitioner(new WrapperStreamPartitioner(streamPartitioner)); + + #endregion } } \ No newline at end of file diff --git a/test/Streamiz.Kafka.Net.Tests/Private/RecordCollectorTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/RecordCollectorTests.cs index 6bc2c6f8..827c8209 100644 --- a/test/Streamiz.Kafka.Net.Tests/Private/RecordCollectorTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Private/RecordCollectorTests.cs @@ -1,6 +1,7 @@ using System; using Confluent.Kafka; using NUnit.Framework; +using Moq; using Streamiz.Kafka.Net.Crosscutting; using Streamiz.Kafka.Net.Kafka.Internal; using Streamiz.Kafka.Net.Metrics.Internal; @@ -21,11 +22,13 @@ public void Init() config = new StreamConfig(); config.ApplicationId = "collector-unit-test"; config.ProductionExceptionHandler = (_) => ProductionExceptionHandlerResponse.RETRY; + var client = new Mock(); collector = new RecordCollector( "test-collector", config, new TaskId {Id = 0, Partition = 0}, - NoRunnableSensor.Empty); + NoRunnableSensor.Empty, + client.Object); } [TearDown] diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamPassThoughTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamPassThoughTests.cs index a66faf25..9a84b876 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamPassThoughTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamPassThoughTests.cs @@ -73,7 +73,7 @@ public void ShouldNotAllowNullTopicDest() var builder = new StreamBuilder(); var stream = builder.Stream("topic"); string topicDes = null; - Assert.Throws(() => stream.To(topicDes)); + Assert.Throws(() => stream.To(topicDes)); } [Test] diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamRepartitionTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamRepartitionTests.cs index f4817ba0..55dc19e7 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamRepartitionTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamRepartitionTests.cs @@ -89,7 +89,7 @@ public void RepartitionWithPartitioner() { builder .Stream("topic") - .Repartition(Repartitioned.Empty().WithStreamPartitioner((t,k,v) => 0)) + .Repartition(Repartitioned.Empty().WithStreamPartitioner((t,k,v, c) => 0)) .To("output"); }, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, 10); diff --git a/test/Streamiz.Kafka.Net.Tests/Public/ProcessorContextTests.cs b/test/Streamiz.Kafka.Net.Tests/Public/ProcessorContextTests.cs index 59c30e9d..075f1fee 100644 --- a/test/Streamiz.Kafka.Net.Tests/Public/ProcessorContextTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Public/ProcessorContextTests.cs @@ -79,12 +79,13 @@ private StreamTask GetTask(TaskId taskId) config.ClientId = "test"; config.ApplicationId = "test-app"; + var syncKafkaSupplier = new SyncKafkaSupplier(true); var streamTask = new StreamTask( "thread", taskId, new List(), new Stream.Internal.ProcessorTopology(null, null, null, null, null, null, null, null), - null, config , null, new SyncProducer(config.ToProducerConfig()), new MockChangelogRegister(), new StreamMetricsRegistry()); + null, config , syncKafkaSupplier, new SyncProducer(config.ToProducerConfig()), new MockChangelogRegister(), new StreamMetricsRegistry()); return streamTask; } diff --git a/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedKeyValueBytesStoreTests.cs b/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedKeyValueBytesStoreTests.cs index be7e7e9b..a0a14c16 100644 --- a/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedKeyValueBytesStoreTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedKeyValueBytesStoreTests.cs @@ -51,9 +51,14 @@ public void Begin() var producerConfig = new ProducerConfig(); producerConfig.ClientId = "producer-1"; - var producerClient = kafkaSupplier.GetProducer(producerConfig); - recordCollector = new RecordCollector("p-1", config, id, new NoRunnableSensor("s", "s", MetricsRecordingLevel.DEBUG)); + var adminConfig = new AdminClientConfig(); + adminConfig.ClientId = "admin-client"; + + var producerClient = kafkaSupplier.GetProducer(producerConfig); + var adminClient = kafkaSupplier.GetAdmin(adminConfig); + + recordCollector = new RecordCollector("p-1", config, id, new NoRunnableSensor("s", "s", MetricsRecordingLevel.DEBUG), adminClient); recordCollector.Init(ref producerClient); var changelogsTopics = new Dictionary{ diff --git a/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedWindowBytesStoreTests.cs b/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedWindowBytesStoreTests.cs index 35d4da6d..5870c20c 100644 --- a/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedWindowBytesStoreTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedWindowBytesStoreTests.cs @@ -50,9 +50,14 @@ public void Begin() var producerConfig = new ProducerConfig(); producerConfig.ClientId = "producer-1"; + + var adminConfig = new AdminClientConfig(); + adminConfig.ClientId = "admin-client"; + var producerClient = kafkaSupplier.GetProducer(producerConfig); - - recordCollector = new RecordCollector("p-1", config, id, new NoRunnableSensor("s", "s", MetricsRecordingLevel.DEBUG)); + var adminClient = kafkaSupplier.GetAdmin(adminConfig); + + recordCollector = new RecordCollector("p-1", config, id, new NoRunnableSensor("s", "s", MetricsRecordingLevel.DEBUG), adminClient); recordCollector.Init(ref producerClient); var changelogsTopics = new Dictionary{ diff --git a/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingWindowBytesStoreTests.cs b/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingWindowBytesStoreTests.cs index 329029f7..1d45821c 100644 --- a/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingWindowBytesStoreTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingWindowBytesStoreTests.cs @@ -47,9 +47,14 @@ public void Begin() var producerConfig = new ProducerConfig(); producerConfig.ClientId = "producer-1"; + + var adminConfig = new AdminClientConfig(); + adminConfig.ClientId = "admin-client"; + var producerClient = kafkaSupplier.GetProducer(producerConfig); - - recordCollector = new RecordCollector("p-1", config, id, new NoRunnableSensor("s", "s", MetricsRecordingLevel.DEBUG)); + var adminClient = kafkaSupplier.GetAdmin(adminConfig); + + recordCollector = new RecordCollector("p-1", config, id, new NoRunnableSensor("s", "s", MetricsRecordingLevel.DEBUG), adminClient); recordCollector.Init(ref producerClient); var changelogsTopics = new Dictionary{ From c861e962f36b3898683213da6f050a3efc297af9 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Tue, 2 Jul 2024 15:39:57 -0700 Subject: [PATCH 06/29] add unit test with partitioner --- core/Stream/IKStream.cs | 24 ++++ core/Stream/Internal/KStream.cs | 26 ++++ .../Processors/KStreamToTests.cs | 133 ++++++++++++++++++ 3 files changed, 183 insertions(+) create mode 100644 test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs diff --git a/core/Stream/IKStream.cs b/core/Stream/IKStream.cs index 04f24222..c739b59a 100644 --- a/core/Stream/IKStream.cs +++ b/core/Stream/IKStream.cs @@ -114,6 +114,30 @@ public interface IKStream /// Throw if is null /// /// Throw if is incorrect void To(string topicName, string named = null); + + /// + /// Materialize this stream to a topic using default serializers specified in the config and producer's. + /// The specified topic should be manually created before it is used(i.e., before the Kafka Streams application is + /// started). + /// + /// the topic name + /// The function used to determine how records are distributed among partitions of the topic + /// A config used to name the processor in the topology. Default : null + /// Throw if is null + /// /// Throw if is incorrect + void To(string topicName, Func partitioner, string named = null); + + /// + /// Materialize this stream to a topic using default serializers specified in the config and producer's. + /// The specified topic should be manually created before it is used(i.e., before the Kafka Streams application is + /// started). + /// + /// the topic name + /// The function used to determine how records are distributed among partitions of the topic + /// A config used to name the processor in the topology. Default : null + /// Throw if is null + /// /// Throw if is incorrect + void To(string topicName, IStreamPartitioner partitioner, string named = null); /// /// Materialize this stream to a topic using serializers specified in the method parameters. diff --git a/core/Stream/Internal/KStream.cs b/core/Stream/Internal/KStream.cs index 50a2da76..482a3729 100644 --- a/core/Stream/Internal/KStream.cs +++ b/core/Stream/Internal/KStream.cs @@ -187,6 +187,32 @@ public void To(string topicName, string named = null) To(new StaticTopicNameExtractor(topicName), new DefaultStreamPartitioner(), named); } + public void To(string topicName, Func partitioner, string named = null) + { + if (string.IsNullOrEmpty(topicName)) + { + throw new ArgumentException("topicName must not be null or empty"); + } + + To( + new StaticTopicNameExtractor(topicName), + new WrapperStreamPartitioner(partitioner), + named); + } + + public void To(string topicName, IStreamPartitioner partitioner, string named = null) + { + if (string.IsNullOrEmpty(topicName)) + { + throw new ArgumentException("topicName must not be null or empty"); + } + + To( + new StaticTopicNameExtractor(topicName), + partitioner, + named); + } + public void To(string topicName, ISerDes keySerdes, ISerDes valueSerdes, string named = null) { if (string.IsNullOrEmpty(topicName)) diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs new file mode 100644 index 00000000..af02daea --- /dev/null +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs @@ -0,0 +1,133 @@ +using System; +using System.Linq; +using Confluent.Kafka; +using NUnit.Framework; +using Streamiz.Kafka.Net.Mock; +using Streamiz.Kafka.Net.Mock.Kafka; +using Streamiz.Kafka.Net.Processors; +using Streamiz.Kafka.Net.SerDes; +using Streamiz.Kafka.Net.Stream; + +namespace Streamiz.Kafka.Net.Tests.Processors; + +public class KStreamToTests +{ + private class MyStreamPartitioner : IStreamPartitioner + { + public Partition Partition(string topic, string key, string value, int numPartitions) + { + switch (key) + { + case "order": + return new Partition(0); + case "sale": + return new Partition(1); + default: + return Confluent.Kafka.Partition.Any; + } + } + } + + [Test] + public void StreamToLambdaCustomPartitionerTest() + { + var config = new StreamConfig + { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To("output", + (_, _, _, _) => new Partition(0)); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = outputTopic.ReadKeyValueList().ToList(); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual(0, record[0].Partition.Value); + + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual(0, record[1].Partition.Value); + + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual(0, record[2].Partition.Value); + } + + [Test] + public void StreamToCustomPartitionerTest() + { + var config = new StreamConfig + { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To("output", new MyStreamPartitioner()); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output", TimeSpan.FromSeconds(2)); + inputTopic.PipeInput("order", "order1"); + inputTopic.PipeInput("order", "order2"); + inputTopic.PipeInput("sale", "sale1"); + inputTopic.PipeInput("other", "other"); + inputTopic.PipeInput("test", "test1"); + + var record = outputTopic.ReadKeyValueList() + .ToList(); + + var orderRecord = record.Where(o => o.Message.Key.Equals("order")).ToList(); + var saleRecord = record.Where(o => o.Message.Key.Equals("sale")).ToList(); + var otherRecord = record.Where(o => o.Message.Key.Equals("other") || o.Message.Key.Equals("test")).ToList(); + + Assert.IsNotNull(record); + Assert.AreEqual(5, record.Count); + + Assert.AreEqual("order", orderRecord[0].Message.Key); + Assert.AreEqual("ORDER1", orderRecord[0].Message.Value); + Assert.AreEqual(0, orderRecord[0].Partition.Value); + + + Assert.AreEqual("order", orderRecord[1].Message.Key); + Assert.AreEqual("ORDER2", orderRecord[1].Message.Value); + Assert.AreEqual(0, orderRecord[1].Partition.Value); + + + Assert.AreEqual("sale", saleRecord[0].Message.Key); + Assert.AreEqual("SALE1", saleRecord[0].Message.Value); + Assert.AreEqual(1, saleRecord[0].Partition.Value); + + Assert.AreEqual(2, otherRecord.Count); + } +} \ No newline at end of file From bc8a371a1b430d6da5e0c2c03c2de7436ab7f144 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Tue, 2 Jul 2024 17:01:06 -0700 Subject: [PATCH 07/29] add unit tests --- core/Mock/Kafka/MockCluster.cs | 13 +- core/Mock/Kafka/MockPartition.cs | 13 +- core/Mock/Kafka/MockTopic.cs | 4 +- .../Processors/KStreamToTests.cs | 482 ++++++++++++++++++ 4 files changed, 502 insertions(+), 10 deletions(-) diff --git a/core/Mock/Kafka/MockCluster.cs b/core/Mock/Kafka/MockCluster.cs index 6a0b5f3a..70d559da 100644 --- a/core/Mock/Kafka/MockCluster.cs +++ b/core/Mock/Kafka/MockCluster.cs @@ -756,7 +756,12 @@ internal ConsumeResult Consume(MockConsumer mockConsumer, TimeSp Topic = p.Topic, Partition = p.Partition, Message = new Message - {Key = record.Key, Value = record.Value} + { + Key = record.Key, + Value = record.Value, + Timestamp = new Timestamp( + record.Timestamp ?? DateTime.Now, TimestampType.NotAvailable) + } }; ++offset.OffsetConsumed; log.LogDebug($"Consumer {mockConsumer.Name} consume message from topic/partition {p}, offset {offset.OffsetConsumed}"); @@ -795,7 +800,7 @@ internal DeliveryReport Produce(string topic, Message Produce(TopicPartition topicPartition, M CreateTopic(topicPartition.Topic); if (topics[topicPartition.Topic].PartitionNumber > topicPartition.Partition) { - topics[topicPartition.Topic].AddMessage(message.Key, message.Value, topicPartition.Partition); + topics[topicPartition.Topic].AddMessage(message.Key, message.Value, topicPartition.Partition, message.Timestamp.UnixTimestampMs); r.Status = PersistenceStatus.Persisted; } else { topics[topicPartition.Topic].CreateNewPartitions(topicPartition.Partition); - topics[topicPartition.Topic].AddMessage(message.Key, message.Value, topicPartition.Partition); + topics[topicPartition.Topic].AddMessage(message.Key, message.Value, topicPartition.Partition, message.Timestamp.UnixTimestampMs); r.Status = PersistenceStatus.Persisted; } diff --git a/core/Mock/Kafka/MockPartition.cs b/core/Mock/Kafka/MockPartition.cs index a7bd00b9..987ab39f 100644 --- a/core/Mock/Kafka/MockPartition.cs +++ b/core/Mock/Kafka/MockPartition.cs @@ -7,7 +7,7 @@ namespace Streamiz.Kafka.Net.Mock.Kafka { internal class MockPartition { - private readonly List<(byte[], byte[])> log = new(); + private readonly List<(byte[], byte[], long)> log = new(); private readonly Dictionary mappingOffsets = new(); public MockPartition(int indice) @@ -21,10 +21,10 @@ public MockPartition(int indice) public long LowOffset { get; private set; } = Offset.Unset; public long HighOffset { get; private set; } = Offset.Unset; - internal void AddMessageInLog(byte[] key, byte[] value) + internal void AddMessageInLog(byte[] key, byte[] value, long timestamp) { mappingOffsets.Add(Size, log.Count); - log.Add((key, value)); + log.Add((key, value, timestamp)); ++Size; UpdateOffset(); } @@ -43,7 +43,12 @@ internal TestRecord GetMessage(long offset) if (mappingOffsets.ContainsKey(offset)) { var record = log[(int) mappingOffsets[offset]]; - return new TestRecord {Key = record.Item1, Value = record.Item2}; + return new TestRecord + { + Key = record.Item1, + Value = record.Item2, + Timestamp = record.Item3.FromMilliseconds() + }; } return null; diff --git a/core/Mock/Kafka/MockTopic.cs b/core/Mock/Kafka/MockTopic.cs index dfda305f..d96f82fa 100644 --- a/core/Mock/Kafka/MockTopic.cs +++ b/core/Mock/Kafka/MockTopic.cs @@ -23,9 +23,9 @@ public MockTopic(string topic, int part) public int PartitionNumber { get; private set; } public IEnumerable Partitions => partitions.AsReadOnly(); - public void AddMessage(byte[] key, byte[] value, int partition) + public void AddMessage(byte[] key, byte[] value, int partition, long timestamp = 0) { - partitions[partition].AddMessageInLog(key, value); + partitions[partition].AddMessageInLog(key, value, timestamp); } public TestRecord GetMessage(int partition, long consumerOffset) diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs index af02daea..84576ab3 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs @@ -1,12 +1,15 @@ using System; using System.Linq; +using System.Runtime.InteropServices.ComTypes; using Confluent.Kafka; using NUnit.Framework; +using Streamiz.Kafka.Net.Crosscutting; using Streamiz.Kafka.Net.Mock; using Streamiz.Kafka.Net.Mock.Kafka; using Streamiz.Kafka.Net.Processors; using Streamiz.Kafka.Net.SerDes; using Streamiz.Kafka.Net.Stream; +using Streamiz.Kafka.Net.Stream.Internal; namespace Streamiz.Kafka.Net.Tests.Processors; @@ -27,6 +30,26 @@ public Partition Partition(string topic, string key, string value, int numPartit } } } + + private class MyTopicNameExtractor : ITopicNameExtractor + { + public string Extract(string key, string value, IRecordContext recordContext) + => "output"; + } + + private class MyRecordTimestampExtractor : IRecordTimestampExtractor + { + private readonly long _ts; + + public MyRecordTimestampExtractor(long ts) + { + _ts = ts; + } + + public long Extract(string key, string value, IRecordContext recordContext) + => _ts; + } + [Test] public void StreamToLambdaCustomPartitionerTest() @@ -74,6 +97,189 @@ public void StreamToLambdaCustomPartitionerTest() Assert.AreEqual(0, record[2].Partition.Value); } + [Test] + public void StreamToLambdaCustomPartitionerWithSerdesTest() + { + var config = new StreamConfig + { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To("output", + (_, _, _, _) => new Partition(0), + new StringSerDes(), + new StringSerDes()); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = outputTopic.ReadKeyValueList().ToList(); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual(0, record[0].Partition.Value); + + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual(0, record[1].Partition.Value); + + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual(0, record[2].Partition.Value); + } + + [Test] + public void StreamToTopicExtractorLambdaCustomPartitionerWithSerdesTest() + { + var config = new StreamConfig + { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To((_,_,_) => "output", + (_, _, _, _) => new Partition(0), + new StringSerDes(), + new StringSerDes()); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = outputTopic.ReadKeyValueList().ToList(); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual(0, record[0].Partition.Value); + + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual(0, record[1].Partition.Value); + + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual(0, record[2].Partition.Value); + } + + [Test] + public void StreamToTopicExtractorLambdaCustomPartitionerTest() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To((_,_,_) => "output", + (_, _, _, _) => new Partition(0)); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = outputTopic.ReadKeyValueList().ToList(); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual(0, record[0].Partition.Value); + + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual(0, record[1].Partition.Value); + + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual(0, record[2].Partition.Value); + } + + [Test] + public void StreamToTopicExtractorWithSerdesTest() + { + var config = new StreamConfig + { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To(new MyTopicNameExtractor(), + new StringSerDes(), + new StringSerDes()); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = outputTopic.ReadKeyValueList().ToList(); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + } + [Test] public void StreamToCustomPartitionerTest() { @@ -130,4 +336,280 @@ public void StreamToCustomPartitionerTest() Assert.AreEqual(2, otherRecord.Count); } + + [Test] + public void StreamToTopicExtractorLambdaCustomPartitionerRecordExtractorTimestampTest() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + var tst = DateTime.Now.GetMilliseconds(); + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To((_,_,_) => "output", + (_,_,_) => tst, + (_, _, _, _) => new Partition(0)); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = outputTopic.ReadKeyValueList().ToList(); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual(0, record[0].Partition.Value); + + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual(0, record[1].Partition.Value); + + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual(0, record[2].Partition.Value); + } + + [Test] + public void StreamToTopicExtractorRecordExtractorTimestampWithSerdesTest() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + var tst = DateTime.Now.GetMilliseconds(); + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To((_,_,_) => "output", + new StringSerDes(), new StringSerDes(), + (_,_,_) => tst); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = outputTopic.ReadKeyValueList().ToList(); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); + } + + [Test] + public void StreamToTopicExtractorRecordExtractorTimestampTest() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + var tst = DateTime.Now.GetMilliseconds(); + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To(new MyTopicNameExtractor(), + new MyRecordTimestampExtractor(tst)); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = outputTopic.ReadKeyValueList().ToList(); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); + } + + [Test] + public void StreamToTopicExtractorStreamPartitionerTest() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + var tst = DateTime.Now.GetMilliseconds(); + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To(new MyTopicNameExtractor(), + new MyStreamPartitioner()); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = outputTopic.ReadKeyValueList().ToList(); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.IsTrue(record[0].TopicPartition.Partition.Value >= 0); + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.IsTrue(record[1].TopicPartition.Partition.Value >= 0); + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.IsTrue(record[2].TopicPartition.Partition.Value >= 0); + } + + [Test] + public void StreamToTopicExtractorRecordExtractorTimestampStreamPartitionerTest() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + var tst = DateTime.Now.GetMilliseconds(); + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To(new MyTopicNameExtractor(), + new MyRecordTimestampExtractor(tst), + new MyStreamPartitioner()); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = outputTopic.ReadKeyValueList().ToList(); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); + Assert.IsTrue(record[0].TopicPartition.Partition.Value >= 0); + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); + Assert.IsTrue(record[1].TopicPartition.Partition.Value >= 0); + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); + Assert.IsTrue(record[2].TopicPartition.Partition.Value >= 0); + } + + [Test] + public void StreamToTopicExtractorRecordExtractorTimestampWithSerdesWrapperTest() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + var tst = DateTime.Now.GetMilliseconds(); + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To(new MyTopicNameExtractor(), + new StringSerDes(), new StringSerDes(), + new MyRecordTimestampExtractor(tst)); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = outputTopic.ReadKeyValueList().ToList(); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); + } } \ No newline at end of file From 75cd375de053f5da4203651d0f8eac17319c1216 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Tue, 2 Jul 2024 17:15:25 -0700 Subject: [PATCH 08/29] fix latency metrics --- core/Metrics/StreamMetric.cs | 2 +- .../Metrics/StateStoreMetricTests.cs | 8 ++++---- .../Metrics/TaskMetricsTests.cs | 2 +- .../Metrics/ThreadMetricsTests.cs | 16 ++++++++-------- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/core/Metrics/StreamMetric.cs b/core/Metrics/StreamMetric.cs index f57283cf..23bcef8f 100644 --- a/core/Metrics/StreamMetric.cs +++ b/core/Metrics/StreamMetric.cs @@ -15,7 +15,7 @@ public class StreamMetric private readonly IMetricValueProvider metricValueProvider; private readonly MetricConfig config; - private object lastValue; + private object lastValue = -1L; internal StreamMetric( MetricName metricName, diff --git a/test/Streamiz.Kafka.Net.Tests/Metrics/StateStoreMetricTests.cs b/test/Streamiz.Kafka.Net.Tests/Metrics/StateStoreMetricTests.cs index fb4ca9e5..9fa7a3f2 100644 --- a/test/Streamiz.Kafka.Net.Tests/Metrics/StateStoreMetricTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Metrics/StateStoreMetricTests.cs @@ -241,8 +241,8 @@ public void KeyValueStoreMetricsTest() StreamMetricsRegistry.LATENCY_SUFFIX + StreamMetricsRegistry.MAX_SUFFIX, StreamMetricsRegistry.STATE_STORE_LEVEL_GROUP); - Assert.IsTrue((double)latencyAvg.Value > 0); - Assert.IsTrue((double)latencyMax.Value > 0); + Assert.IsTrue((double)latencyAvg.Value >= 0); + Assert.IsTrue((double)latencyMax.Value >= 0); for(int i = 0 ; i < nbMessage2 ; ++i) meteredKeyValueStore.Put($"test{i}", $"test{i}"); @@ -346,8 +346,8 @@ private void AssertAvgAndMaxLatency(string sensorName) sensorName, StreamMetricsRegistry.LATENCY_SUFFIX + StreamMetricsRegistry.MAX_SUFFIX, StreamMetricsRegistry.STATE_STORE_LEVEL_GROUP); - Assert.IsTrue((double)latencyAvg.Value > 0); - Assert.IsTrue((double)latencyMax.Value > 0); + Assert.IsTrue((double)latencyAvg.Value >= 0); + Assert.IsTrue((double)latencyMax.Value >= 0); } private StreamMetric GetSensorMetric(string sensorName, string metricSuffix, string group) diff --git a/test/Streamiz.Kafka.Net.Tests/Metrics/TaskMetricsTests.cs b/test/Streamiz.Kafka.Net.Tests/Metrics/TaskMetricsTests.cs index 09218b55..75136493 100644 --- a/test/Streamiz.Kafka.Net.Tests/Metrics/TaskMetricsTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Metrics/TaskMetricsTests.cs @@ -169,7 +169,7 @@ public void TaskMetricsTest() Assert.IsTrue( (double)processLatency.Metrics[MetricName.NameAndGroup( TaskMetrics.PROCESS_LATENCY + StreamMetricsRegistry.MAX_SUFFIX, - StreamMetricsRegistry.TASK_LEVEL_GROUP)].Value > 0d); + StreamMetricsRegistry.TASK_LEVEL_GROUP)].Value >= 0d); var commitSensor = sensors.FirstOrDefault(s => s.Name.Equals(GetSensorName(TaskMetrics.COMMIT))); Assert.AreEqual(2, commitSensor.Metrics.Count()); diff --git a/test/Streamiz.Kafka.Net.Tests/Metrics/ThreadMetricsTests.cs b/test/Streamiz.Kafka.Net.Tests/Metrics/ThreadMetricsTests.cs index d831c293..30f813ad 100644 --- a/test/Streamiz.Kafka.Net.Tests/Metrics/ThreadMetricsTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Metrics/ThreadMetricsTests.cs @@ -172,11 +172,11 @@ public void ThreadMetricsTest() Assert.IsTrue( (double)commitSensor.Metrics[MetricName.NameAndGroup( ThreadMetrics.COMMIT + StreamMetricsRegistry.LATENCY_SUFFIX + StreamMetricsRegistry.AVG_SUFFIX, - StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value > 0d); + StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value >= 0d); Assert.IsTrue( (double)commitSensor.Metrics[MetricName.NameAndGroup( ThreadMetrics.COMMIT + StreamMetricsRegistry.LATENCY_SUFFIX + StreamMetricsRegistry.MAX_SUFFIX, - StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value > 0d); + StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value >= 0d); var pollSensor = sensors.FirstOrDefault(s => s.Name.Equals(GetSensorName(ThreadMetrics.POLL))); Assert.AreEqual(4, pollSensor.Metrics.Count()); @@ -191,11 +191,11 @@ public void ThreadMetricsTest() Assert.IsTrue( (double)pollSensor.Metrics[MetricName.NameAndGroup( ThreadMetrics.POLL + StreamMetricsRegistry.LATENCY_SUFFIX + StreamMetricsRegistry.AVG_SUFFIX, - StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value > 0d); + StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value >= 0d); Assert.IsTrue( (double)pollSensor.Metrics[MetricName.NameAndGroup( ThreadMetrics.POLL + StreamMetricsRegistry.LATENCY_SUFFIX + StreamMetricsRegistry.MAX_SUFFIX, - StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value > 0d); + StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value >= 0d); var pollRecordsSensor = sensors.FirstOrDefault(s => s.Name.Equals(GetSensorName(ThreadMetrics.POLL + StreamMetricsRegistry.RECORDS_SUFFIX))); Assert.AreEqual(2, pollRecordsSensor.Metrics.Count()); @@ -235,11 +235,11 @@ public void ThreadMetricsTest() Assert.IsTrue( (double)processLatencySensor.Metrics[MetricName.NameAndGroup( ThreadMetrics.PROCESS + StreamMetricsRegistry.LATENCY_SUFFIX + StreamMetricsRegistry.AVG_SUFFIX, - StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value > 0d); + StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value >= 0d); Assert.IsTrue( (double)processLatencySensor.Metrics[MetricName.NameAndGroup( ThreadMetrics.PROCESS + StreamMetricsRegistry.LATENCY_SUFFIX + StreamMetricsRegistry.MAX_SUFFIX, - StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value > 0d); + StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value >= 0d); // Punctuate sensor var punctuateSensor = sensors.FirstOrDefault(s => s.Name.Equals(GetSensorName(ThreadMetrics.PUNCTUATE))); @@ -255,11 +255,11 @@ public void ThreadMetricsTest() Assert.IsTrue( (double)punctuateSensor.Metrics[MetricName.NameAndGroup( ThreadMetrics.PUNCTUATE + StreamMetricsRegistry.LATENCY_SUFFIX + StreamMetricsRegistry.AVG_SUFFIX, - StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value > 0d); + StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value >= 0d); Assert.IsTrue( (double)punctuateSensor.Metrics[MetricName.NameAndGroup( ThreadMetrics.PUNCTUATE + StreamMetricsRegistry.LATENCY_SUFFIX + StreamMetricsRegistry.MAX_SUFFIX, - StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value > 0d); + StreamMetricsRegistry.THREAD_LEVEL_GROUP)].Value >= 0d); // ratio sensors var processRatioSensor = sensors.FirstOrDefault(s => s.Name.Equals(GetSensorName(ThreadMetrics.PROCESS + StreamMetricsRegistry.RATIO_SUFFIX))); From ede1efd4d1d53a6533db10addf1e17beb32584fb Mon Sep 17 00:00:00 2001 From: LGouellec Date: Wed, 3 Jul 2024 10:18:38 -0700 Subject: [PATCH 09/29] add unit tests --- .../Helpers/ExecutorService.cs | 72 ----- .../Processors/KStreamToTests.cs | 247 +++++++++++++++++- 2 files changed, 233 insertions(+), 86 deletions(-) delete mode 100644 test/Streamiz.Kafka.Net.Tests/Helpers/ExecutorService.cs diff --git a/test/Streamiz.Kafka.Net.Tests/Helpers/ExecutorService.cs b/test/Streamiz.Kafka.Net.Tests/Helpers/ExecutorService.cs deleted file mode 100644 index f7684a5b..00000000 --- a/test/Streamiz.Kafka.Net.Tests/Helpers/ExecutorService.cs +++ /dev/null @@ -1,72 +0,0 @@ -using System; -using System.Threading; -using System.Threading.Tasks; -using NUnit.Framework; - -namespace Streamiz.Kafka.Net.Tests.Helpers -{ - internal class ExecutorService : IDisposable - { - private Action action; - private readonly TimeSpan timeout; - private Thread _thread; - private bool disposed = false; - - private ExecutorService(Action action, TimeSpan timeout) - { - this.action = action; - this.timeout = timeout; - _thread = new Thread(() => - { - DateTime dt = DateTime.Now; - bool @continue = true; - try - { - while (@continue) - { - try - { - action(); - @continue = false; - } - catch (AssertionException e) - { - if (dt.Add(timeout) < DateTime.Now) - { - @continue = false; - throw; - } - } - } - } - catch (Exception e) - { - Console.WriteLine(e.Message); - throw; - } - }); - } - - public static ExecutorService NewSingleThreadExecutor(Action action, TimeSpan timeout) - { - return new ExecutorService(action, timeout); - } - - public void Start() - { - _thread.Start(); - } - - public void Stop() - { - _thread.Join(); - disposed = true; - } - - public void Dispose() - { - if(!disposed) - Stop(); - } - } -} diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs index 84576ab3..14845609 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs @@ -1,6 +1,5 @@ using System; using System.Linq; -using System.Runtime.InteropServices.ComTypes; using Confluent.Kafka; using NUnit.Framework; using Streamiz.Kafka.Net.Crosscutting; @@ -9,7 +8,6 @@ using Streamiz.Kafka.Net.Processors; using Streamiz.Kafka.Net.SerDes; using Streamiz.Kafka.Net.Stream; -using Streamiz.Kafka.Net.Stream.Internal; namespace Streamiz.Kafka.Net.Tests.Processors; @@ -79,7 +77,7 @@ public void StreamToLambdaCustomPartitionerTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = outputTopic.ReadKeyValueList().ToList(); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); Assert.AreEqual("test1", record[0].Message.Key); @@ -127,7 +125,53 @@ public void StreamToLambdaCustomPartitionerWithSerdesTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = outputTopic.ReadKeyValueList().ToList(); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual(0, record[0].Partition.Value); + + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual(0, record[1].Partition.Value); + + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual(0, record[2].Partition.Value); + } + + [Test] + public void StreamToLambdaCustomPartitionerWithSerdesTypeTest() + { + var config = new StreamConfig + { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To("output", + (_, _, _, _) => new Partition(0)); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); Assert.AreEqual("test1", record[0].Message.Key); @@ -175,7 +219,53 @@ public void StreamToTopicExtractorLambdaCustomPartitionerWithSerdesTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = outputTopic.ReadKeyValueList().ToList(); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual(0, record[0].Partition.Value); + + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual(0, record[1].Partition.Value); + + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual(0, record[2].Partition.Value); + } + + [Test] + public void StreamToTopicExtractorLambdaCustomPartitionerWithSerdesTypeTest() + { + var config = new StreamConfig + { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To((_,_,_) => "output", + (_, _, _, _) => new Partition(0)); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); Assert.AreEqual("test1", record[0].Message.Key); @@ -220,7 +310,7 @@ public void StreamToTopicExtractorLambdaCustomPartitionerTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = outputTopic.ReadKeyValueList().ToList(); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); Assert.AreEqual("test1", record[0].Message.Key); @@ -267,7 +357,7 @@ public void StreamToTopicExtractorWithSerdesTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = outputTopic.ReadKeyValueList().ToList(); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); Assert.AreEqual("test1", record[0].Message.Key); @@ -280,6 +370,46 @@ public void StreamToTopicExtractorWithSerdesTest() Assert.AreEqual("TEST3", record[2].Message.Value); } + [Test] + public void StreamToTopicExtractorWithSerdesTypeTest() + { + var config = new StreamConfig + { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To(new MyTopicNameExtractor()); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + } + [Test] public void StreamToCustomPartitionerTest() { @@ -310,7 +440,7 @@ public void StreamToCustomPartitionerTest() inputTopic.PipeInput("other", "other"); inputTopic.PipeInput("test", "test1"); - var record = outputTopic.ReadKeyValueList() + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 5) .ToList(); var orderRecord = record.Where(o => o.Message.Key.Equals("order")).ToList(); @@ -366,7 +496,7 @@ public void StreamToTopicExtractorLambdaCustomPartitionerRecordExtractorTimestam inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = outputTopic.ReadKeyValueList().ToList(); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); Assert.AreEqual("test1", record[0].Message.Key); @@ -416,7 +546,7 @@ public void StreamToTopicExtractorRecordExtractorTimestampWithSerdesTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = outputTopic.ReadKeyValueList().ToList(); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); Assert.AreEqual("test1", record[0].Message.Key); @@ -432,6 +562,50 @@ public void StreamToTopicExtractorRecordExtractorTimestampWithSerdesTest() Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); } + [Test] + public void StreamToTopicExtractorRecordExtractorTimestampWithSerdesTypeTest() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + var tst = DateTime.Now.GetMilliseconds(); + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To((_,_,_) => "output", + (_,_,_) => tst); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); + } + [Test] public void StreamToTopicExtractorRecordExtractorTimestampTest() { @@ -460,7 +634,7 @@ public void StreamToTopicExtractorRecordExtractorTimestampTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = outputTopic.ReadKeyValueList().ToList(); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); Assert.AreEqual("test1", record[0].Message.Key); @@ -504,7 +678,7 @@ public void StreamToTopicExtractorStreamPartitionerTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = outputTopic.ReadKeyValueList().ToList(); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); Assert.AreEqual("test1", record[0].Message.Key); @@ -549,7 +723,7 @@ public void StreamToTopicExtractorRecordExtractorTimestampStreamPartitionerTest( inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = outputTopic.ReadKeyValueList().ToList(); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); Assert.AreEqual("test1", record[0].Message.Key); @@ -597,7 +771,7 @@ public void StreamToTopicExtractorRecordExtractorTimestampWithSerdesWrapperTest( inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = outputTopic.ReadKeyValueList().ToList(); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); Assert.AreEqual("test1", record[0].Message.Key); @@ -612,4 +786,49 @@ public void StreamToTopicExtractorRecordExtractorTimestampWithSerdesWrapperTest( Assert.AreEqual("TEST3", record[2].Message.Value); Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); } + + [Test] + public void StreamToTopicExtractorRecordExtractorTimestampWithSerdesTypeWrapperTest() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + var tst = DateTime.Now.GetMilliseconds(); + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To(new MyTopicNameExtractor(), + new MyRecordTimestampExtractor(tst)); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); + } + } \ No newline at end of file From 3630cc42a895872ab573685ca32ad6942aaccb14 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Wed, 3 Jul 2024 11:26:15 -0700 Subject: [PATCH 10/29] add unit tests --- .../Processors/KStreamToTests.cs | 197 ++++++++++++++++++ 1 file changed, 197 insertions(+) diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs index 14845609..802f47bd 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs @@ -831,4 +831,201 @@ public void StreamToTopicExtractorRecordExtractorTimestampWithSerdesTypeWrapperT Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); } + + [Test] + public void StreamToLambdaTopicExtractorWithSerdesTest() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + var tst = DateTime.Now.GetMilliseconds(); + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To((_, _, _) => "output", new StringSerDes(), new StringSerDes()); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + } + + [Test] + public void StreamToLambdaTopicExtractorWithSerdesTypeTest() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + var tst = DateTime.Now.GetMilliseconds(); + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To((_, _, _) => "output"); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + } + + [Test] + public void StreamToAllLambdaWithSerdesTypeTest() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + var tst = DateTime.Now.GetMilliseconds(); + builder + .Stream("stream") + .MapValues((v) => v.ToUpper()) + .To( + (_, _, _) => "output", + (_,_,_) => tst, + (_,_,_,_) => new Partition(0)); + + Topology t = builder.Build(); + + var mockSupplier = new MockKafkaSupplier(4); + mockSupplier.CreateTopic("output"); + mockSupplier.CreateTopic("stream"); + + using var driver = new TopologyTestDriver(t.Builder, config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, mockSupplier); + var inputTopic = driver.CreateInputTopic("stream"); + var outputTopic = driver.CreateOuputTopic("output"); + inputTopic.PipeInput("test1", "test1"); + inputTopic.PipeInput("test2", "test2"); + inputTopic.PipeInput("test3", "test3"); + var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + Assert.IsNotNull(record); + Assert.AreEqual(3, record.Count); + Assert.AreEqual("test1", record[0].Message.Key); + Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual(0, record[0].Partition.Value); + + Assert.AreEqual("test2", record[1].Message.Key); + Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual(0, record[1].Partition.Value); + + Assert.AreEqual("test3", record[2].Message.Key); + Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual(0, record[2].Partition.Value); + } + + [Test] + public void StreamArgumentExceptionTopicEmptyTest() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + var stream = builder + .Stream("stream") + .MapValues((v) => v.ToUpper()); + + Assert.Throws(() => stream.To( + string.Empty, (_,_,_,_) => Partition.Any)); + } + + [Test] + public void StreamArgumentExceptionTopicEmpty2Test() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + var stream = builder + .Stream("stream") + .MapValues((v) => v.ToUpper()); + + Assert.Throws(() => stream.To( + string.Empty, new MyStreamPartitioner())); + } + + [Test] + public void StreamArgumentExceptionTopicEmpty3Test() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + var stream = builder + .Stream("stream") + .MapValues((v) => v.ToUpper()); + + Assert.Throws(() => stream.To( + string.Empty, new StringSerDes(), new StringSerDes())); + } + + [Test] + public void StreamArgumentExceptionTopicEmpty4Test() + { + var config = new StreamConfig { + ApplicationId = "test-stream-table-left-join" + }; + + StreamBuilder builder = new StreamBuilder(); + + var stream = builder + .Stream("stream") + .MapValues((v) => v.ToUpper()); + + Assert.Throws(() => stream.To( + string.Empty, (_,_,_,_) => Partition.Any, new StringSerDes(), new StringSerDes())); + } } \ No newline at end of file From c601bd5e9dfa97dc9dfa7a7ed73d4f12ecb45157 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Wed, 3 Jul 2024 13:24:22 -0700 Subject: [PATCH 11/29] Add the source record partition --- core/Processors/IStreamPartitioner.cs | 2 +- .../Internal/DefaultStreamPartitioner.cs | 9 ++++++-- .../Internal/WrapperStreamPartitioner.cs | 8 +++---- core/Processors/SinkProcessor.cs | 1 + core/Stream/IKStream.cs | 16 +++++++------- core/Stream/Internal/KStream.cs | 16 +++++++------- core/Stream/Repartitioned.cs | 4 ++-- samples/sample-stream/Program.cs | 13 +++++------ .../Processors/KStreamRepartitionTests.cs | 2 +- .../Processors/KStreamToTests.cs | 22 +++++++++---------- 10 files changed, 49 insertions(+), 44 deletions(-) diff --git a/core/Processors/IStreamPartitioner.cs b/core/Processors/IStreamPartitioner.cs index ad67c0c9..c8e2d12a 100644 --- a/core/Processors/IStreamPartitioner.cs +++ b/core/Processors/IStreamPartitioner.cs @@ -4,6 +4,6 @@ namespace Streamiz.Kafka.Net.Processors { public interface IStreamPartitioner { - Partition Partition(string topic, K key, V value, int numPartitions); + Partition Partition(string topic, K key, V value, Partition sourcePartition, int numPartitions); } } \ No newline at end of file diff --git a/core/Processors/Internal/DefaultStreamPartitioner.cs b/core/Processors/Internal/DefaultStreamPartitioner.cs index 14627afb..aa96a0b5 100644 --- a/core/Processors/Internal/DefaultStreamPartitioner.cs +++ b/core/Processors/Internal/DefaultStreamPartitioner.cs @@ -2,9 +2,14 @@ namespace Streamiz.Kafka.Net.Processors.Internal { + /// + /// Forward the source partition as the sink partition of the record + /// + /// Key record type + /// Value record type internal class DefaultStreamPartitioner : IStreamPartitioner { - public Partition Partition(string topic, K key, V value, int numPartitions) - => Confluent.Kafka.Partition.Any; + public Partition Partition(string topic, K key, V value, Partition sourcePartition, int numPartitions) + => sourcePartition; } } \ No newline at end of file diff --git a/core/Processors/Internal/WrapperStreamPartitioner.cs b/core/Processors/Internal/WrapperStreamPartitioner.cs index 8206ca44..608c8ec2 100644 --- a/core/Processors/Internal/WrapperStreamPartitioner.cs +++ b/core/Processors/Internal/WrapperStreamPartitioner.cs @@ -5,14 +5,14 @@ namespace Streamiz.Kafka.Net.Processors.Internal { internal class WrapperStreamPartitioner : IStreamPartitioner { - private readonly Func _partitioner; + private readonly Func _partitioner; - public WrapperStreamPartitioner(Func partitioner) + public WrapperStreamPartitioner(Func partitioner) { _partitioner = partitioner; } - public Partition Partition(string topic, K key, V value, int numPartitions) - => _partitioner(topic, key, value, numPartitions); + public Partition Partition(string topic, K key, V value, Partition sourcePartition, int numPartitions) + => _partitioner(topic, key, value, sourcePartition, numPartitions); } } \ No newline at end of file diff --git a/core/Processors/SinkProcessor.cs b/core/Processors/SinkProcessor.cs index f85db35d..9470a9f6 100644 --- a/core/Processors/SinkProcessor.cs +++ b/core/Processors/SinkProcessor.cs @@ -75,6 +75,7 @@ public override void Process(K key, V value) Partition partition = partitioner.Partition( topicName, key, value, + Context.Partition, Context.RecordCollector.PartitionsFor(topicName)); Context.RecordCollector.Send( diff --git a/core/Stream/IKStream.cs b/core/Stream/IKStream.cs index c739b59a..04c830a3 100644 --- a/core/Stream/IKStream.cs +++ b/core/Stream/IKStream.cs @@ -125,7 +125,7 @@ public interface IKStream /// A config used to name the processor in the topology. Default : null /// Throw if is null /// /// Throw if is incorrect - void To(string topicName, Func partitioner, string named = null); + void To(string topicName, Func partitioner, string named = null); /// /// Materialize this stream to a topic using default serializers specified in the config and producer's. @@ -164,7 +164,7 @@ public interface IKStream /// A config used to name the processor in the topology. Default : null /// Throw if is null /// /// Throw if is incorrect - void To(string topicName, Func partitioner, ISerDes keySerdes, ISerDes valueSerdes, string named = null); + void To(string topicName, Func partitioner, ISerDes keySerdes, ISerDes valueSerdes, string named = null); /// /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. @@ -181,7 +181,7 @@ public interface IKStream /// Extractor function to determine the name of the Kafka topic to write to for each record /// The function used to determine how records are distributed among partitions of the topic /// A config used to name the processor in the topology. Default : null - void To(Func topicExtractor, Func partitioner, string named = null); + void To(Func topicExtractor, Func partitioner, string named = null); /// /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. @@ -202,7 +202,7 @@ public interface IKStream /// Key serializer /// Value serializer /// A config used to name the processor in the topology. Default : null - void To(Func topicExtractor, Func partitioner, ISerDes keySerdes, ISerDes valueSerdes, string named = null); + void To(Func topicExtractor, Func partitioner, ISerDes keySerdes, ISerDes valueSerdes, string named = null); /// @@ -252,7 +252,7 @@ public interface IKStream /// Extractor function to determine the timestamp of the record stored in the Kafka topic /// The function used to determine how records are distributed among partitions of the topic /// A config used to name the processor in the topology. Default : null - void To(Func topicExtractor, Func recordTimestampExtractor, Func partitioner, string named = null); + void To(Func topicExtractor, Func recordTimestampExtractor, Func partitioner, string named = null); /// /// Dynamically materialize this stream to topics using default serializers specified in the config and producer's. @@ -337,7 +337,7 @@ public interface IKStream /// the topic name /// The function used to determine how records are distributed among partitions of the topic /// A config used to name the processor in the topology. Default : null - void To(string topicName, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + void To(string topicName, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); /// /// Dynamically materialize this stream to a topic using and serializers specified in the method parameters. @@ -348,7 +348,7 @@ public interface IKStream /// Extractor function to determine the name of the Kafka topic to write to for each record /// The function used to determine how records are distributed among partitions of the topic /// A config used to name the processor in the topology. Default : null - void To(Func topicExtractor, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + void To(Func topicExtractor, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); /// /// Dynamically materialize this stream to a topic using and serializers specified in the method parameters. @@ -402,7 +402,7 @@ public interface IKStream /// Extractor function to determine the timestamp of the record stored in the Kafka topic /// The function used to determine how records are distributed among partitions of the topic /// A config used to name the processor in the topology. Default : null - void To(Func topicExtractor, Func recordTimestampExtractor, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); + void To(Func topicExtractor, Func recordTimestampExtractor, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new(); /// /// Dynamically materialize this stream to a topic using and serializers specified in the method parameters. diff --git a/core/Stream/Internal/KStream.cs b/core/Stream/Internal/KStream.cs index 482a3729..f21ed209 100644 --- a/core/Stream/Internal/KStream.cs +++ b/core/Stream/Internal/KStream.cs @@ -187,7 +187,7 @@ public void To(string topicName, string named = null) To(new StaticTopicNameExtractor(topicName), new DefaultStreamPartitioner(), named); } - public void To(string topicName, Func partitioner, string named = null) + public void To(string topicName, Func partitioner, string named = null) { if (string.IsNullOrEmpty(topicName)) { @@ -221,7 +221,7 @@ public void To(string topicName, ISerDes keySerdes, ISerDes valueSerdes, s To(new StaticTopicNameExtractor(topicName), new DefaultStreamPartitioner(), keySerdes, valueSerdes, named); } - public void To(string topicName, Func partitioner, ISerDes keySerdes, ISerDes valueSerdes, string named = null) + public void To(string topicName, Func partitioner, ISerDes keySerdes, ISerDes valueSerdes, string named = null) { if (string.IsNullOrEmpty(topicName)) throw new ArgumentException("topicName must be empty"); @@ -232,7 +232,7 @@ public void To(string topicName, Func partitioner, } public void To(Func topicExtractor, - Func partitioner, ISerDes keySerdes, ISerDes valueSerdes, + Func partitioner, ISerDes keySerdes, ISerDes valueSerdes, string named = null) => To( new WrapperTopicNameExtractor(topicExtractor), @@ -270,7 +270,7 @@ public void To(Func topicExtractor, string named = => To(new WrapperTopicNameExtractor(topicExtractor), named); public void To(Func topicExtractor, - Func partitioner, string named = null) + Func partitioner, string named = null) => To(new WrapperTopicNameExtractor(topicExtractor), new WrapperStreamPartitioner(partitioner), named); @@ -288,7 +288,7 @@ public void To(Func topicExtractor, named); public void To(Func topicExtractor, - Func recordTimestampExtractor, Func partitioner, + Func recordTimestampExtractor, Func partitioner, string named = null) => To( new WrapperTopicNameExtractor(topicExtractor), @@ -345,7 +345,7 @@ public void To(ITopicNameExtractor topicExtractor, ISerDes keySerdes, I partitioner, Produced.Create(keySerdes, valueSerdes).WithName(named)); - public void To(Func topicExtractor, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new() + public void To(Func topicExtractor, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new() => To( new WrapperTopicNameExtractor(topicExtractor), new WrapperStreamPartitioner(partitioner), @@ -361,7 +361,7 @@ public void To(string topicName, string named = null) where VS : ISerDes, new() => To(new StaticTopicNameExtractor(topicName), named); - public void To(string topicName, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new() + public void To(string topicName, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new() => To(new StaticTopicNameExtractor(topicName), new WrapperStreamPartitioner(partitioner), named); @@ -383,7 +383,7 @@ public void To(Func topicExtractor, Func, new() => To(new WrapperTopicNameExtractor(topicExtractor), new WrapperRecordTimestampExtractor(recordTimestampExtractor), named); - public void To(Func topicExtractor, Func recordTimestampExtractor, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new() + public void To(Func topicExtractor, Func recordTimestampExtractor, Func partitioner, string named = null) where KS : ISerDes, new() where VS : ISerDes, new() => To(new WrapperTopicNameExtractor(topicExtractor), new WrapperRecordTimestampExtractor(recordTimestampExtractor), new WrapperStreamPartitioner(partitioner), named); public void To(ITopicNameExtractor topicExtractor, diff --git a/core/Stream/Repartitioned.cs b/core/Stream/Repartitioned.cs index 9337bbbd..acde2cba 100644 --- a/core/Stream/Repartitioned.cs +++ b/core/Stream/Repartitioned.cs @@ -82,7 +82,7 @@ public static Repartitioned Create() /// /// the function used to determine how records are distributed among partitions of the topic /// A new instance of - public static Repartitioned Partitioner(Func streamPartitioner) + public static Repartitioned Partitioner(Func streamPartitioner) => Partitioner(new WrapperStreamPartitioner(streamPartitioner)); @@ -150,7 +150,7 @@ public Repartitioned WithStreamPartitioner(IStreamPartitioner stream /// /// Function to determine the partition where the repartition record will be persist /// this - public Repartitioned WithStreamPartitioner(Func streamPartitioner) + public Repartitioned WithStreamPartitioner(Func streamPartitioner) => WithStreamPartitioner(new WrapperStreamPartitioner(streamPartitioner)); diff --git a/samples/sample-stream/Program.cs b/samples/sample-stream/Program.cs index 59468230..19060913 100644 --- a/samples/sample-stream/Program.cs +++ b/samples/sample-stream/Program.cs @@ -7,9 +7,7 @@ using Streamiz.Kafka.Net.Metrics; using Streamiz.Kafka.Net.Metrics.Prometheus; using Streamiz.Kafka.Net.Stream; -using System.Collections.Generic; using Microsoft.Extensions.Logging; -using Streamiz.Kafka.Net.Metrics.Prometheus; using Streamiz.Kafka.Net.Table; namespace sample_stream @@ -22,9 +20,6 @@ public static async Task Main(string[] args) ApplicationId = $"test-app", BootstrapServers = "localhost:9092", AutoOffsetReset = AutoOffsetReset.Earliest, - PartitionAssignmentStrategy = PartitionAssignmentStrategy.RoundRobin, - Partitioner = Partitioner.ConsistentRandom, - Debug = "broker,topic,msg", Logger = LoggerFactory.Create((b) => { b.AddConsole(); @@ -49,7 +44,7 @@ private static Topology BuildTopology() TimeSpan windowSize = TimeSpan.FromHours(1); var builder = new StreamBuilder(); - builder.Stream("input") + /*builder.Stream("input") .GroupByKey() .WindowedBy(TumblingWindowOptions.Of(windowSize)) .Count(RocksDbWindows.As("count-store") @@ -60,7 +55,11 @@ private static Topology BuildTopology() .Map((k,v) => new KeyValuePair(k.ToString(), v.ToString())) .To("output", new StringSerDes(), - new StringSerDes()); + new StringSerDes());*/ + + builder.Stream("input") + .To( + "output");//, (s, s1, arg3, arg4) => new Partition(0)); return builder.Build(); diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamRepartitionTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamRepartitionTests.cs index 55dc19e7..d26c2ea1 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamRepartitionTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamRepartitionTests.cs @@ -89,7 +89,7 @@ public void RepartitionWithPartitioner() { builder .Stream("topic") - .Repartition(Repartitioned.Empty().WithStreamPartitioner((t,k,v, c) => 0)) + .Repartition(Repartitioned.Empty().WithStreamPartitioner((t,k,v,_, c) => 0)) .To("output"); }, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY, 10); diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs index 802f47bd..d3e73c98 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs @@ -15,7 +15,7 @@ public class KStreamToTests { private class MyStreamPartitioner : IStreamPartitioner { - public Partition Partition(string topic, string key, string value, int numPartitions) + public Partition Partition(string topic, string key, string value, Partition sourcePartition, int numPartitions) { switch (key) { @@ -63,7 +63,7 @@ public void StreamToLambdaCustomPartitionerTest() .Stream("stream") .MapValues((v) => v.ToUpper()) .To("output", - (_, _, _, _) => new Partition(0)); + (_, _, _,_, _) => new Partition(0)); Topology t = builder.Build(); @@ -109,7 +109,7 @@ public void StreamToLambdaCustomPartitionerWithSerdesTest() .Stream("stream") .MapValues((v) => v.ToUpper()) .To("output", - (_, _, _, _) => new Partition(0), + (_, _, _,_, _) => new Partition(0), new StringSerDes(), new StringSerDes()); @@ -157,7 +157,7 @@ public void StreamToLambdaCustomPartitionerWithSerdesTypeTest() .Stream("stream") .MapValues((v) => v.ToUpper()) .To("output", - (_, _, _, _) => new Partition(0)); + (_, _,_, _, _) => new Partition(0)); Topology t = builder.Build(); @@ -203,7 +203,7 @@ public void StreamToTopicExtractorLambdaCustomPartitionerWithSerdesTest() .Stream("stream") .MapValues((v) => v.ToUpper()) .To((_,_,_) => "output", - (_, _, _, _) => new Partition(0), + (_, _,_, _, _) => new Partition(0), new StringSerDes(), new StringSerDes()); @@ -251,7 +251,7 @@ public void StreamToTopicExtractorLambdaCustomPartitionerWithSerdesTypeTest() .Stream("stream") .MapValues((v) => v.ToUpper()) .To((_,_,_) => "output", - (_, _, _, _) => new Partition(0)); + (_, _,_, _, _) => new Partition(0)); Topology t = builder.Build(); @@ -296,7 +296,7 @@ public void StreamToTopicExtractorLambdaCustomPartitionerTest() .Stream("stream") .MapValues((v) => v.ToUpper()) .To((_,_,_) => "output", - (_, _, _, _) => new Partition(0)); + (_, _,_, _, _) => new Partition(0)); Topology t = builder.Build(); @@ -482,7 +482,7 @@ public void StreamToTopicExtractorLambdaCustomPartitionerRecordExtractorTimestam .MapValues((v) => v.ToUpper()) .To((_,_,_) => "output", (_,_,_) => tst, - (_, _, _, _) => new Partition(0)); + (_, _,_, _, _) => new Partition(0)); Topology t = builder.Build(); @@ -928,7 +928,7 @@ public void StreamToAllLambdaWithSerdesTypeTest() .To( (_, _, _) => "output", (_,_,_) => tst, - (_,_,_,_) => new Partition(0)); + (_,_,_,_,_) => new Partition(0)); Topology t = builder.Build(); @@ -975,7 +975,7 @@ public void StreamArgumentExceptionTopicEmptyTest() .MapValues((v) => v.ToUpper()); Assert.Throws(() => stream.To( - string.Empty, (_,_,_,_) => Partition.Any)); + string.Empty, (_,_,_,_,_) => Partition.Any)); } [Test] @@ -1026,6 +1026,6 @@ public void StreamArgumentExceptionTopicEmpty4Test() .MapValues((v) => v.ToUpper()); Assert.Throws(() => stream.To( - string.Empty, (_,_,_,_) => Partition.Any, new StringSerDes(), new StringSerDes())); + string.Empty, (_,_,_,_,_) => Partition.Any, new StringSerDes(), new StringSerDes())); } } \ No newline at end of file From ef9dea7d0a6c70c6e68df53df62af38e94afb35a Mon Sep 17 00:00:00 2001 From: LGouellec Date: Wed, 3 Jul 2024 14:22:49 -0700 Subject: [PATCH 12/29] fix unit tests --- core/Processors/DefaultStreamPartitioner.cs | 28 +++++++++++++++++++ core/Processors/IStreamPartitioner.cs | 9 ++++++ .../Internal/DefaultStreamPartitioner.cs | 15 ---------- 3 files changed, 37 insertions(+), 15 deletions(-) create mode 100644 core/Processors/DefaultStreamPartitioner.cs delete mode 100644 core/Processors/Internal/DefaultStreamPartitioner.cs diff --git a/core/Processors/DefaultStreamPartitioner.cs b/core/Processors/DefaultStreamPartitioner.cs new file mode 100644 index 00000000..3345b6ac --- /dev/null +++ b/core/Processors/DefaultStreamPartitioner.cs @@ -0,0 +1,28 @@ +using Confluent.Kafka; + +namespace Streamiz.Kafka.Net.Processors +{ + /// + /// Forward the source partition as the sink partition of the record if there is enough sink partitions, Partition.Any otherwise + /// + /// Key record type + /// Value record type + public class DefaultStreamPartitioner : IStreamPartitioner + { + /// + /// Function used to determine how records are distributed among partitions of the topic + /// + /// Sink topic name + /// record's key + /// record's value + /// record's source partition + /// number partitions of the sink topic + /// Return the source partition as the sink partition of the record if there is enough sink partitions, Partition.Any otherwise + public Partition Partition(string topic, K key, V value, Partition sourcePartition, int numPartitions) + { + if (sourcePartition.Value <= numPartitions - 1) + return sourcePartition; + return Confluent.Kafka.Partition.Any; + } + } +} \ No newline at end of file diff --git a/core/Processors/IStreamPartitioner.cs b/core/Processors/IStreamPartitioner.cs index c8e2d12a..c29d5318 100644 --- a/core/Processors/IStreamPartitioner.cs +++ b/core/Processors/IStreamPartitioner.cs @@ -4,6 +4,15 @@ namespace Streamiz.Kafka.Net.Processors { public interface IStreamPartitioner { + /// + /// Function used to determine how records are distributed among partitions of the topic + /// + /// Sink topic name + /// record's key + /// record's value + /// record's source partition + /// number partitions of the sink topic + /// Return the destination partition for the current record Partition Partition(string topic, K key, V value, Partition sourcePartition, int numPartitions); } } \ No newline at end of file diff --git a/core/Processors/Internal/DefaultStreamPartitioner.cs b/core/Processors/Internal/DefaultStreamPartitioner.cs deleted file mode 100644 index aa96a0b5..00000000 --- a/core/Processors/Internal/DefaultStreamPartitioner.cs +++ /dev/null @@ -1,15 +0,0 @@ -using Confluent.Kafka; - -namespace Streamiz.Kafka.Net.Processors.Internal -{ - /// - /// Forward the source partition as the sink partition of the record - /// - /// Key record type - /// Value record type - internal class DefaultStreamPartitioner : IStreamPartitioner - { - public Partition Partition(string topic, K key, V value, Partition sourcePartition, int numPartitions) - => sourcePartition; - } -} \ No newline at end of file From 5d3207a9c5d8af9e41c4649e0e1061ea045cd542 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Thu, 4 Jul 2024 10:21:38 -0700 Subject: [PATCH 13/29] Add CacheSize options --- core/ProcessorContext.cs | 2 - core/Processors/AbstractKTableProcessor.cs | 3 +- .../Internal/TimestampedTupleForwarder.cs | 8 +- .../KStreamWindowAggregateProcessor.cs | 3 +- core/Processors/KTableSourceProcessor.cs | 3 +- core/Processors/StatefullProcessor.cs | 3 +- core/State/AbstractStoreBuilder.cs | 9 +- core/State/Cache/CachingKeyValueStore.cs | 15 +- core/State/Cache/CachingWindowStore.cs | 15 +- core/State/IStoreBuilder.cs | 7 +- .../TimestampedKeyValueStoreMaterializer.cs | 2 +- .../TimestampedWindowStoreMaterializer.cs | 2 +- core/State/KeyValueStoreBuilder.cs | 2 +- core/State/TimestampedKeyValueStoreBuilder.cs | 2 +- core/State/TimestampedWindowStoreBuilder.cs | 3 +- core/State/WindowStoreBuilder.cs | 3 +- core/StreamConfig.cs | 10 +- core/Table/CacheSize.cs | 12 + core/Table/Materialized.cs | 19 +- .../Processors/KStreamToTests.cs | 490 ++++++++++-------- .../AbstractPersistentWindowStoreTests.cs | 4 +- .../Stores/CacheKeyValueStoreTests.cs | 8 +- 22 files changed, 357 insertions(+), 268 deletions(-) create mode 100644 core/Table/CacheSize.cs diff --git a/core/ProcessorContext.cs b/core/ProcessorContext.cs index 5d1033d5..66d08573 100644 --- a/core/ProcessorContext.cs +++ b/core/ProcessorContext.cs @@ -67,8 +67,6 @@ public class ProcessorContext /// Returns the state directory for the partition. /// public virtual string StateDir => $"{Path.Combine(Configuration.StateDir, Configuration.ApplicationId, Id.ToString())}"; - - internal bool ConfigEnableCache => Configuration.StateStoreCacheMaxBytes > 0; // FOR TESTING internal ProcessorContext() diff --git a/core/Processors/AbstractKTableProcessor.cs b/core/Processors/AbstractKTableProcessor.cs index c1eb653e..4be70e38 100644 --- a/core/Processors/AbstractKTableProcessor.cs +++ b/core/Processors/AbstractKTableProcessor.cs @@ -36,8 +36,7 @@ public override void Init(ProcessorContext context) new Change(sendOldValues ? kv.Value.OldValue.Value : default, kv.Value.NewValue.Value), kv.Value.NewValue.Timestamp); }, - sendOldValues, - context.ConfigEnableCache); + sendOldValues); } if (throwException && (queryableStoreName == null || store == null || tupleForwarder == null)) diff --git a/core/Processors/Internal/TimestampedTupleForwarder.cs b/core/Processors/Internal/TimestampedTupleForwarder.cs index 51958ada..77164e9c 100644 --- a/core/Processors/Internal/TimestampedTupleForwarder.cs +++ b/core/Processors/Internal/TimestampedTupleForwarder.cs @@ -17,14 +17,12 @@ public TimestampedTupleForwarder( IStateStore store, IProcessor processor, Action>>> listener, - bool sendOldValues, - bool configCachingEnabled) + bool sendOldValues) { this.processor = processor; this.sendOldValues = sendOldValues; - cachingEnabled = configCachingEnabled && - ((IWrappedStateStore)store).IsCachedStore && - ((ICachedStateStore>)store).SetFlushListener(listener, sendOldValues); + cachingEnabled = ((IWrappedStateStore)store).IsCachedStore && + ((ICachedStateStore>)store).SetFlushListener(listener, sendOldValues); } public void MaybeForward(K key, V newValue, V oldValue) diff --git a/core/Processors/KStreamWindowAggregateProcessor.cs b/core/Processors/KStreamWindowAggregateProcessor.cs index 2afce3da..6f73ef2e 100644 --- a/core/Processors/KStreamWindowAggregateProcessor.cs +++ b/core/Processors/KStreamWindowAggregateProcessor.cs @@ -47,8 +47,7 @@ public override void Init(ProcessorContext context) new Change(sendOldValues ? kv.Value.OldValue.Value : default, kv.Value.NewValue.Value), kv.Value.NewValue.Timestamp); }, - sendOldValues, - context.ConfigEnableCache); + sendOldValues); } public override void Process(K key, V value) diff --git a/core/Processors/KTableSourceProcessor.cs b/core/Processors/KTableSourceProcessor.cs index d7002e0c..e018e8d4 100644 --- a/core/Processors/KTableSourceProcessor.cs +++ b/core/Processors/KTableSourceProcessor.cs @@ -37,8 +37,7 @@ public override void Init(ProcessorContext context) new Change(sendOldValues ? (kv.Value.OldValue != null ? kv.Value.OldValue.Value : default) : default, kv.Value.NewValue.Value), kv.Value.NewValue.Timestamp); }, - sendOldValues, - context.ConfigEnableCache); + sendOldValues); } } diff --git a/core/Processors/StatefullProcessor.cs b/core/Processors/StatefullProcessor.cs index 1ffde514..9bbba94c 100644 --- a/core/Processors/StatefullProcessor.cs +++ b/core/Processors/StatefullProcessor.cs @@ -32,8 +32,7 @@ public override void Init(ProcessorContext context) new Change(sendOldValues ? kv.Value.OldValue.Value : default, kv.Value.NewValue.Value), kv.Value.NewValue.Timestamp); }, - sendOldValues, - context.ConfigEnableCache); + sendOldValues); } } } diff --git a/core/State/AbstractStoreBuilder.cs b/core/State/AbstractStoreBuilder.cs index 72169347..5bfe15f2 100644 --- a/core/State/AbstractStoreBuilder.cs +++ b/core/State/AbstractStoreBuilder.cs @@ -2,6 +2,7 @@ using Streamiz.Kafka.Net.SerDes; using System; using System.Collections.Generic; +using Streamiz.Kafka.Net.Table; namespace Streamiz.Kafka.Net.State { @@ -64,6 +65,11 @@ public abstract class AbstractStoreBuilder : IStoreBuilder /// public bool CachingEnabled => enableCaching; + /// + /// Cache size of the storage + /// + public CacheSize CacheSize { get; private set; } + /// /// /// @@ -81,9 +87,10 @@ protected AbstractStoreBuilder(String name, ISerDes keySerde, ISerDes valu /// Activate caching /// /// - public IStoreBuilder WithCachingEnabled() + public IStoreBuilder WithCachingEnabled(CacheSize cacheSize = null) { enableCaching = true; + CacheSize = cacheSize; return this; } diff --git a/core/State/Cache/CachingKeyValueStore.cs b/core/State/Cache/CachingKeyValueStore.cs index 7ea8e479..24aefec9 100644 --- a/core/State/Cache/CachingKeyValueStore.cs +++ b/core/State/Cache/CachingKeyValueStore.cs @@ -8,6 +8,7 @@ using Streamiz.Kafka.Net.State.Cache.Internal; using Streamiz.Kafka.Net.State.Enumerator; using Streamiz.Kafka.Net.State.Internal; +using Streamiz.Kafka.Net.Table; using Streamiz.Kafka.Net.Table.Internal; namespace Streamiz.Kafka.Net.State.Cache @@ -17,6 +18,7 @@ internal class CachingKeyValueStore : IKeyValueStore, ICachedStateStore { + private readonly CacheSize _cacheSize; private MemoryCache cache; private Action>> flushListener; private bool sendOldValue; @@ -25,9 +27,11 @@ internal class CachingKeyValueStore : private Sensor hitRatioSensor = NoRunnableSensor.Empty; private Sensor totalCacheSizeSensor = NoRunnableSensor.Empty; - public CachingKeyValueStore(IKeyValueStore wrapped) + public CachingKeyValueStore(IKeyValueStore wrapped, CacheSize cacheSize) : base(wrapped) - { } + { + _cacheSize = cacheSize; + } protected virtual void RegisterMetrics() { @@ -39,7 +43,7 @@ protected virtual void RegisterMetrics() } } - public override bool IsCachedStore => true; + public override bool IsCachedStore => cachingEnabled; public bool SetFlushListener(Action>> listener, bool sendOldChanges) { @@ -51,10 +55,11 @@ public bool SetFlushListener(Action>> listen // Only for testing internal void CreateCache(ProcessorContext context) { - cachingEnabled = context.Configuration.StateStoreCacheMaxBytes > 0; + cachingEnabled = context.Configuration.DefaultStateStoreCacheMaxBytes > 0 || + _cacheSize is { CacheSizeBytes: > 0 }; if(cachingEnabled) cache = new MemoryCache(new MemoryCacheOptions { - SizeLimit = context.Configuration.StateStoreCacheMaxBytes, + SizeLimit = _cacheSize is { CacheSizeBytes: > 0 } ? _cacheSize.CacheSizeBytes : context.Configuration.DefaultStateStoreCacheMaxBytes, CompactionPercentage = .20 }, new BytesComparer()); } diff --git a/core/State/Cache/CachingWindowStore.cs b/core/State/Cache/CachingWindowStore.cs index 05997ea4..8eac5502 100644 --- a/core/State/Cache/CachingWindowStore.cs +++ b/core/State/Cache/CachingWindowStore.cs @@ -10,6 +10,7 @@ using Streamiz.Kafka.Net.State.Enumerator; using Streamiz.Kafka.Net.State.Helper; using Streamiz.Kafka.Net.State.Internal; +using Streamiz.Kafka.Net.Table; using Streamiz.Kafka.Net.Table.Internal; namespace Streamiz.Kafka.Net.State.Cache @@ -21,7 +22,8 @@ internal class CachingWindowStore : ICachedStateStore { private readonly long _windowSize; - + private readonly CacheSize _cacheSize; + private MemoryCache cache; private bool cachingEnabled; private bool sendOldValue; @@ -35,10 +37,12 @@ public CachingWindowStore( IWindowStore wrapped, long windowSize, long segmentInterval, - IKeySchema keySchema) + IKeySchema keySchema, + CacheSize cacheSize) : base(wrapped) { _windowSize = windowSize; + _cacheSize = cacheSize; SegmentInterval = segmentInterval; KeySchema = keySchema; SegmentCacheFunction = new SegmentedCacheFunction(KeySchema, segmentInterval); @@ -51,7 +55,7 @@ public CachingWindowStore( internal ICacheFunction SegmentCacheFunction { get; } internal IKeySchema KeySchema { get; } - public override bool IsCachedStore => true; + public override bool IsCachedStore => cachingEnabled; public override void Init(ProcessorContext context, IStateStore root) { @@ -73,10 +77,11 @@ protected virtual void RegisterMetrics() // Only for testing internal void CreateCache(ProcessorContext context) { - cachingEnabled = context.Configuration.StateStoreCacheMaxBytes > 0; + cachingEnabled = context.Configuration.DefaultStateStoreCacheMaxBytes > 0 || + _cacheSize is { CacheSizeBytes: > 0 }; if(cachingEnabled) cache = new MemoryCache(new MemoryCacheOptions { - SizeLimit = context.Configuration.StateStoreCacheMaxBytes, + SizeLimit = _cacheSize is { CacheSizeBytes: > 0 } ? _cacheSize.CacheSizeBytes : context.Configuration.DefaultStateStoreCacheMaxBytes, CompactionPercentage = .20 }, new BytesComparer()); } diff --git a/core/State/IStoreBuilder.cs b/core/State/IStoreBuilder.cs index f34002e8..a87ec9ea 100644 --- a/core/State/IStoreBuilder.cs +++ b/core/State/IStoreBuilder.cs @@ -1,6 +1,7 @@ using Streamiz.Kafka.Net.Processors; using System; using System.Collections.Generic; +using Streamiz.Kafka.Net.Table; namespace Streamiz.Kafka.Net.State { @@ -30,6 +31,10 @@ public interface IStoreBuilder /// string Name { get; } /// + /// Cache size of the storage + /// + CacheSize CacheSize { get; } + /// /// Build the state store /// /// @@ -47,7 +52,7 @@ public interface IStoreBuilder : IStoreBuilder /// Activate caching /// /// - IStoreBuilder WithCachingEnabled(); + IStoreBuilder WithCachingEnabled(CacheSize cacheSize = null); /// /// Disable caching /// diff --git a/core/State/Internal/TimestampedKeyValueStoreMaterializer.cs b/core/State/Internal/TimestampedKeyValueStoreMaterializer.cs index 1eada8f1..017b9cb9 100644 --- a/core/State/Internal/TimestampedKeyValueStoreMaterializer.cs +++ b/core/State/Internal/TimestampedKeyValueStoreMaterializer.cs @@ -37,7 +37,7 @@ public IStoreBuilder> Materialize() if (materialized.CachingEnabled) { - builder.WithCachingEnabled(); + builder.WithCachingEnabled(materialized.CacheSize); } else { diff --git a/core/State/Internal/TimestampedWindowStoreMaterializer.cs b/core/State/Internal/TimestampedWindowStoreMaterializer.cs index 07e23678..135a8fe3 100644 --- a/core/State/Internal/TimestampedWindowStoreMaterializer.cs +++ b/core/State/Internal/TimestampedWindowStoreMaterializer.cs @@ -42,7 +42,7 @@ public IStoreBuilder> Materialize() builder.WithLoggingDisabled(); if (materializedInternal.CachingEnabled) - builder.WithCachingEnabled(); + builder.WithCachingEnabled(materializedInternal.CacheSize); return builder; } diff --git a/core/State/KeyValueStoreBuilder.cs b/core/State/KeyValueStoreBuilder.cs index 447ac3bb..a0a95494 100644 --- a/core/State/KeyValueStoreBuilder.cs +++ b/core/State/KeyValueStoreBuilder.cs @@ -63,7 +63,7 @@ private IKeyValueStore WrapLogging(IKeyValueStore private IKeyValueStore WrapCaching(IKeyValueStore inner) { - return !CachingEnabled ? inner : new CachingKeyValueStore(inner); + return !CachingEnabled ? inner : new CachingKeyValueStore(inner, CacheSize); } } } \ No newline at end of file diff --git a/core/State/TimestampedKeyValueStoreBuilder.cs b/core/State/TimestampedKeyValueStoreBuilder.cs index a965d856..23c90bf5 100644 --- a/core/State/TimestampedKeyValueStoreBuilder.cs +++ b/core/State/TimestampedKeyValueStoreBuilder.cs @@ -64,7 +64,7 @@ private IKeyValueStore WrapLogging(IKeyValueStore private IKeyValueStore WrapCaching(IKeyValueStore inner) { - return !CachingEnabled ? inner : new CachingKeyValueStore(inner); + return !CachingEnabled ? inner : new CachingKeyValueStore(inner, CacheSize); } } } diff --git a/core/State/TimestampedWindowStoreBuilder.cs b/core/State/TimestampedWindowStoreBuilder.cs index b420f181..29ac1c5e 100644 --- a/core/State/TimestampedWindowStoreBuilder.cs +++ b/core/State/TimestampedWindowStoreBuilder.cs @@ -71,7 +71,8 @@ private IWindowStore WrapCaching(IWindowStore inne inner, supplier.WindowSize.Value, supplier.SegmentInterval, - new WindowKeySchema()); + new WindowKeySchema(), + CacheSize); } } } diff --git a/core/State/WindowStoreBuilder.cs b/core/State/WindowStoreBuilder.cs index 1ec759f3..6d907b51 100644 --- a/core/State/WindowStoreBuilder.cs +++ b/core/State/WindowStoreBuilder.cs @@ -72,7 +72,8 @@ private IWindowStore WrapCaching(IWindowStore inne inner, supplier.WindowSize.Value, supplier.SegmentInterval, - new WindowKeySchema()); + new WindowKeySchema(), + CacheSize); } } } \ No newline at end of file diff --git a/core/StreamConfig.cs b/core/StreamConfig.cs index 3ee6378a..0e4319a3 100644 --- a/core/StreamConfig.cs +++ b/core/StreamConfig.cs @@ -343,9 +343,9 @@ public interface IStreamConfig : ICloneable TimeSpan LogProcessingSummary { get; set; } /// - /// Maximum number of memory bytes to be used for state stores cache for a single store. (default: 5Mb) + /// Define the default maximum number of memory bytes to be used for state stores cache for a single store. (default: 5Mb) /// - long StateStoreCacheMaxBytes { get; set; } + long DefaultStateStoreCacheMaxBytes { get; set; } #endregion @@ -2282,12 +2282,12 @@ public StreamConfig(IDictionary properties) MetricsIntervalMs = (long)TimeSpan.FromSeconds(30).TotalMilliseconds; MetricsRecording = MetricsRecordingLevel.INFO; LogProcessingSummary = TimeSpan.FromMinutes(1); - MetricsReporter = (_) => { }; // nothing by default, maybe another behavior in future + MetricsReporter = _ => { }; // nothing by default, maybe another behavior in future ExposeLibrdKafkaStats = false; StartTaskDelayMs = 5000; ParallelProcessing = false; MaxDegreeOfParallelism = 8; - StateStoreCacheMaxBytes = 5 * 1024 * 1024; + DefaultStateStoreCacheMaxBytes = 5 * 1024 * 1024; _consumerConfig = new ConsumerConfig(); _producerConfig = new ProducerConfig(); @@ -2761,7 +2761,7 @@ public TimeSpan LogProcessingSummary /// [StreamConfigProperty("" + stateStoreCacheMaxBytesCst)] - public long StateStoreCacheMaxBytes + public long DefaultStateStoreCacheMaxBytes { get => configProperties[stateStoreCacheMaxBytesCst]; set => configProperties.AddOrUpdate(stateStoreCacheMaxBytesCst, value); diff --git a/core/Table/CacheSize.cs b/core/Table/CacheSize.cs new file mode 100644 index 00000000..50efab44 --- /dev/null +++ b/core/Table/CacheSize.cs @@ -0,0 +1,12 @@ +namespace Streamiz.Kafka.Net.Table +{ + public class CacheSize + { + internal long CacheSizeBytes { get; private set; } + + public static CacheSize OfB(int bytes) => new CacheSize { CacheSizeBytes = bytes }; + public static CacheSize OfKb(int kilobytes) => new CacheSize { CacheSizeBytes = kilobytes * 1024 }; + public static CacheSize OfMb(int megabytes) => new CacheSize { CacheSizeBytes = megabytes * 1024 * 1024 }; + public static CacheSize OfGb(int gigabytes) => new CacheSize { CacheSizeBytes = gigabytes * 1024 * 1024 * 1024 }; + } +} \ No newline at end of file diff --git a/core/Table/Materialized.cs b/core/Table/Materialized.cs index 59caab43..ee9810c4 100644 --- a/core/Table/Materialized.cs +++ b/core/Table/Materialized.cs @@ -41,6 +41,11 @@ public class Materialized /// private TimeSpan retention; + /// + /// Cache size + /// + private CacheSize cacheSize; + #region Ctor /// @@ -252,6 +257,11 @@ public static Materialized> Create(I /// public TimeSpan Retention => retention; + /// + /// Cache size (in bytes) of the storage (default: null). + /// + public CacheSize CacheSize => cacheSize; + #endregion #region Methods @@ -289,14 +299,16 @@ public Materialized WithLoggingDisabled() TopicConfig?.Clear(); return this; } - + /// - /// Enable caching for this + /// Enable caching for this and specify the cache size /// + /// Cache size of the storage /// Itself - public Materialized WithCachingEnabled() + public Materialized WithCachingEnabled(CacheSize cacheSize = null) { CachingEnabled = true; + this.cacheSize = cacheSize; return this; } @@ -306,6 +318,7 @@ public Materialized WithCachingEnabled() /// Itself public Materialized WithCachingDisabled() { + this.cacheSize = null; CachingEnabled = false; return this; } diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs index d3e73c98..1f238448 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamToTests.cs @@ -77,22 +77,24 @@ public void StreamToLambdaCustomPartitionerTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - Assert.AreEqual(0, record[0].Partition.Value); - - - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); - Assert.AreEqual(0, record[1].Partition.Value); - - - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); - Assert.AreEqual(0, record[2].Partition.Value); + + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); + Assert.AreEqual(0, record["test1"].Partition.Value); + + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + Assert.AreEqual(0, record["test2"].Partition.Value); + + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); + Assert.AreEqual(0, record["test3"].Partition.Value); } [Test] @@ -125,22 +127,24 @@ public void StreamToLambdaCustomPartitionerWithSerdesTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - Assert.AreEqual(0, record[0].Partition.Value); - - - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); - Assert.AreEqual(0, record[1].Partition.Value); - - - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); - Assert.AreEqual(0, record[2].Partition.Value); + + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); + Assert.AreEqual(0, record["test1"].Partition.Value); + + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + Assert.AreEqual(0, record["test2"].Partition.Value); + + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); + Assert.AreEqual(0, record["test3"].Partition.Value); } [Test] @@ -171,22 +175,24 @@ public void StreamToLambdaCustomPartitionerWithSerdesTypeTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - Assert.AreEqual(0, record[0].Partition.Value); - - - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); - Assert.AreEqual(0, record[1].Partition.Value); - - - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); - Assert.AreEqual(0, record[2].Partition.Value); + + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); + Assert.AreEqual(0, record["test1"].Partition.Value); + + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + Assert.AreEqual(0, record["test2"].Partition.Value); + + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); + Assert.AreEqual(0, record["test3"].Partition.Value); } [Test] @@ -219,22 +225,23 @@ public void StreamToTopicExtractorLambdaCustomPartitionerWithSerdesTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - Assert.AreEqual(0, record[0].Partition.Value); - - - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); - Assert.AreEqual(0, record[1].Partition.Value); - - - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); - Assert.AreEqual(0, record[2].Partition.Value); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); + Assert.AreEqual(0, record["test1"].Partition.Value); + + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + Assert.AreEqual(0, record["test2"].Partition.Value); + + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); + Assert.AreEqual(0, record["test3"].Partition.Value); } [Test] @@ -265,22 +272,23 @@ public void StreamToTopicExtractorLambdaCustomPartitionerWithSerdesTypeTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - Assert.AreEqual(0, record[0].Partition.Value); - - - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); - Assert.AreEqual(0, record[1].Partition.Value); - - - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); - Assert.AreEqual(0, record[2].Partition.Value); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); + Assert.AreEqual(0, record["test1"].Partition.Value); + + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + Assert.AreEqual(0, record["test2"].Partition.Value); + + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); + Assert.AreEqual(0, record["test3"].Partition.Value); } [Test] @@ -310,22 +318,23 @@ public void StreamToTopicExtractorLambdaCustomPartitionerTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - Assert.AreEqual(0, record[0].Partition.Value); - - - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); - Assert.AreEqual(0, record[1].Partition.Value); - - - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); - Assert.AreEqual(0, record[2].Partition.Value); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); + Assert.AreEqual(0, record["test1"].Partition.Value); + + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + Assert.AreEqual(0, record["test2"].Partition.Value); + + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); + Assert.AreEqual(0, record["test3"].Partition.Value); } [Test] @@ -357,17 +366,20 @@ public void StreamToTopicExtractorWithSerdesTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); } [Test] @@ -397,17 +409,20 @@ public void StreamToTopicExtractorWithSerdesTypeTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); } [Test] @@ -496,25 +511,26 @@ public void StreamToTopicExtractorLambdaCustomPartitionerRecordExtractorTimestam inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual(0, record[0].Partition.Value); - - - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); - Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual(0, record[1].Partition.Value); - - - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); - Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual(0, record[2].Partition.Value); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); + Assert.AreEqual(0, record["test1"].Partition.Value); + Assert.AreEqual(tst, record["test1"].Message.Timestamp.UnixTimestampMs); + + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + Assert.AreEqual(0, record["test2"].Partition.Value); + Assert.AreEqual(tst, record["test2"].Message.Timestamp.UnixTimestampMs); + + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); + Assert.AreEqual(0, record["test3"].Partition.Value); + Assert.AreEqual(tst, record["test3"].Message.Timestamp.UnixTimestampMs); } [Test] @@ -546,20 +562,24 @@ public void StreamToTopicExtractorRecordExtractorTimestampWithSerdesTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); - Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); + Assert.AreEqual(tst, record["test1"].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); - Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + Assert.AreEqual(tst, record["test2"].Message.Timestamp.UnixTimestampMs); + + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); + Assert.AreEqual(tst, record["test3"].Message.Timestamp.UnixTimestampMs); } [Test] @@ -590,20 +610,24 @@ public void StreamToTopicExtractorRecordExtractorTimestampWithSerdesTypeTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); - Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); + Assert.AreEqual(tst, record["test1"].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); - Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + Assert.AreEqual(tst, record["test2"].Message.Timestamp.UnixTimestampMs); + + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); + Assert.AreEqual(tst, record["test3"].Message.Timestamp.UnixTimestampMs); } [Test] @@ -634,20 +658,23 @@ public void StreamToTopicExtractorRecordExtractorTimestampTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); + Assert.AreEqual(tst, record["test1"].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); - Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + Assert.AreEqual(tst, record["test2"].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); - Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); + Assert.AreEqual(tst, record["test3"].Message.Timestamp.UnixTimestampMs); } [Test] @@ -678,20 +705,23 @@ public void StreamToTopicExtractorStreamPartitionerTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - Assert.IsTrue(record[0].TopicPartition.Partition.Value >= 0); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); + Assert.IsTrue(record["test1"].TopicPartition.Partition.Value >= 0); - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); - Assert.IsTrue(record[1].TopicPartition.Partition.Value >= 0); + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + Assert.IsTrue(record["test2"].TopicPartition.Partition.Value >= 0); - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); - Assert.IsTrue(record[2].TopicPartition.Partition.Value >= 0); + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); + Assert.IsTrue(record["test3"].TopicPartition.Partition.Value >= 0); } [Test] @@ -723,23 +753,26 @@ public void StreamToTopicExtractorRecordExtractorTimestampStreamPartitionerTest( inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); - Assert.IsTrue(record[0].TopicPartition.Partition.Value >= 0); - - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); - Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); - Assert.IsTrue(record[1].TopicPartition.Partition.Value >= 0); - - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); - Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); - Assert.IsTrue(record[2].TopicPartition.Partition.Value >= 0); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); + Assert.AreEqual(tst, record["test1"].Message.Timestamp.UnixTimestampMs); + Assert.IsTrue(record["test1"].TopicPartition.Partition.Value >= 0); + + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + Assert.AreEqual(tst, record["test2"].Message.Timestamp.UnixTimestampMs); + Assert.IsTrue(record["test2"].TopicPartition.Partition.Value >= 0); + + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); + Assert.AreEqual(tst, record["test3"].Message.Timestamp.UnixTimestampMs); + Assert.IsTrue(record["test3"].TopicPartition.Partition.Value >= 0); } [Test] @@ -771,20 +804,23 @@ public void StreamToTopicExtractorRecordExtractorTimestampWithSerdesWrapperTest( inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); + Assert.AreEqual(tst, record["test1"].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); - Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + Assert.AreEqual(tst, record["test2"].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); - Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); + Assert.AreEqual(tst, record["test3"].Message.Timestamp.UnixTimestampMs); } [Test] @@ -815,20 +851,23 @@ public void StreamToTopicExtractorRecordExtractorTimestampWithSerdesTypeWrapperT inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); + Assert.AreEqual(tst, record["test1"].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); - Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + Assert.AreEqual(tst, record["test2"].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); - Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); + Assert.AreEqual(tst, record["test3"].Message.Timestamp.UnixTimestampMs); } @@ -859,17 +898,20 @@ public void StreamToLambdaTopicExtractorWithSerdesTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); } [Test] @@ -899,17 +941,20 @@ public void StreamToLambdaTopicExtractorWithSerdesTypeTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); } [Test] @@ -942,23 +987,26 @@ public void StreamToAllLambdaWithSerdesTypeTest() inputTopic.PipeInput("test1", "test1"); inputTopic.PipeInput("test2", "test2"); inputTopic.PipeInput("test3", "test3"); - var record = IntegrationTestUtils.WaitUntilMinKeyValueRecordsReceived(outputTopic, 3); + var record = IntegrationTestUtils + .WaitUntilMinKeyValueRecordsReceived(outputTopic, 3) + .ToDictionary(c => c.Message.Key, c => c); + Assert.IsNotNull(record); Assert.AreEqual(3, record.Count); - Assert.AreEqual("test1", record[0].Message.Key); - Assert.AreEqual("TEST1", record[0].Message.Value); - Assert.AreEqual(tst, record[0].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual(0, record[0].Partition.Value); - - Assert.AreEqual("test2", record[1].Message.Key); - Assert.AreEqual("TEST2", record[1].Message.Value); - Assert.AreEqual(tst, record[1].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual(0, record[1].Partition.Value); - - Assert.AreEqual("test3", record[2].Message.Key); - Assert.AreEqual("TEST3", record[2].Message.Value); - Assert.AreEqual(tst, record[2].Message.Timestamp.UnixTimestampMs); - Assert.AreEqual(0, record[2].Partition.Value); + Assert.AreEqual("test1", record["test1"].Message.Key); + Assert.AreEqual("TEST1", record["test1"].Message.Value); + Assert.AreEqual(tst, record["test1"].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual(0, record["test1"].TopicPartition.Partition.Value); + + Assert.AreEqual("test2", record["test2"].Message.Key); + Assert.AreEqual("TEST2", record["test2"].Message.Value); + Assert.AreEqual(tst, record["test2"].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual(0, record["test2"].TopicPartition.Partition.Value); + + Assert.AreEqual("test3", record["test3"].Message.Key); + Assert.AreEqual("TEST3", record["test3"].Message.Value); + Assert.AreEqual(tst, record["test3"].Message.Timestamp.UnixTimestampMs); + Assert.AreEqual(0, record["test3"].TopicPartition.Partition.Value); } [Test] diff --git a/test/Streamiz.Kafka.Net.Tests/Stores/AbstractPersistentWindowStoreTests.cs b/test/Streamiz.Kafka.Net.Tests/Stores/AbstractPersistentWindowStoreTests.cs index 69a4c2aa..b3911fe5 100644 --- a/test/Streamiz.Kafka.Net.Tests/Stores/AbstractPersistentWindowStoreTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Stores/AbstractPersistentWindowStoreTests.cs @@ -43,7 +43,7 @@ public void Begin() { config = new StreamConfig(); config.ApplicationId = "unit-test-rocksdb-w"; - config.StateStoreCacheMaxBytes = MAX_CACHE_SIZE_BYTES; + config.DefaultStateStoreCacheMaxBytes = MAX_CACHE_SIZE_BYTES; id = new TaskId { Id = 0, Partition = 0 }; partition = new TopicPartition("source", 0); @@ -62,7 +62,7 @@ public void Begin() keySchema = new WindowKeySchema(); store = GetBackWindowStore(); cachingStore = new CachingWindowStore(store, - WINDOW_SIZE, SEGMENT_INTERVAL, keySchema); + WINDOW_SIZE, SEGMENT_INTERVAL, keySchema, null); if (store.Persistent) config.UseRandomRocksDbConfigForTest(); diff --git a/test/Streamiz.Kafka.Net.Tests/Stores/CacheKeyValueStoreTests.cs b/test/Streamiz.Kafka.Net.Tests/Stores/CacheKeyValueStoreTests.cs index b937d29d..5275ceac 100644 --- a/test/Streamiz.Kafka.Net.Tests/Stores/CacheKeyValueStoreTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Stores/CacheKeyValueStoreTests.cs @@ -63,7 +63,7 @@ public void Begin() { config = new StreamConfig(); config.ApplicationId = "unit-test-cachestore-kv"; - config.StateStoreCacheMaxBytes = 1000; + config.DefaultStateStoreCacheMaxBytes = 1000; threadId = Thread.CurrentThread.Name ?? StreamMetricsRegistry.UNKNOWN_THREAD; id = new TaskId { Id = 0, Partition = 0 }; @@ -82,7 +82,7 @@ public void Begin() context = new ProcessorContext(task.Object, config, stateManager, metricsRegistry); inMemoryKeyValue = new InMemoryKeyValueStore("store"); - cache = new CachingKeyValueStore(inMemoryKeyValue); + cache = new CachingKeyValueStore(inMemoryKeyValue, null); cache.Init(context, cache); } @@ -99,7 +99,7 @@ public void End() [Test] public void ExpiryCapacityTest() { - config.StateStoreCacheMaxBytes = 30; + config.DefaultStateStoreCacheMaxBytes = 30; cache.CreateCache(context); bool checkListener = true; cache.SetFlushListener(record => { @@ -240,7 +240,7 @@ public void RangeTest() [Test] public void DisabledCachingTest() { - config.StateStoreCacheMaxBytes = 0; + config.DefaultStateStoreCacheMaxBytes = 0; cache.CreateCache(context); context.SetRecordMetaData(new RecordContext(new Headers(), 0, 100, 0, "topic")); From d04be34fdf833eac0b49b0e4f5f04bfc957ef3aa Mon Sep 17 00:00:00 2001 From: LGouellec Date: Thu, 4 Jul 2024 15:32:17 -0700 Subject: [PATCH 14/29] Add cache size unit tests --- core/ProcessorContext.cs | 6 +- core/State/Cache/CachingKeyValueStore.cs | 39 ++++++------ core/State/Cache/Internal/MemoryCache.cs | 4 +- core/Table/CacheSize.cs | 2 +- .../Helpers/MockProcessorContext.cs | 20 ++++++ .../Public/CacheSizeTests.cs | 61 +++++++++++++++++++ .../Stores/CacheKeyValueStoreTests.cs | 1 - 7 files changed, 108 insertions(+), 25 deletions(-) create mode 100644 test/Streamiz.Kafka.Net.Tests/Helpers/MockProcessorContext.cs create mode 100644 test/Streamiz.Kafka.Net.Tests/Public/CacheSizeTests.cs diff --git a/core/ProcessorContext.cs b/core/ProcessorContext.cs index 66d08573..35f6cce7 100644 --- a/core/ProcessorContext.cs +++ b/core/ProcessorContext.cs @@ -67,12 +67,12 @@ public class ProcessorContext /// Returns the state directory for the partition. /// public virtual string StateDir => $"{Path.Combine(Configuration.StateDir, Configuration.ApplicationId, Id.ToString())}"; - - // FOR TESTING + internal ProcessorContext() { + } - + internal ProcessorContext(AbstractTask task, IStreamConfig configuration, IStateManager stateManager, StreamMetricsRegistry streamMetricsRegistry) { diff --git a/core/State/Cache/CachingKeyValueStore.cs b/core/State/Cache/CachingKeyValueStore.cs index 24aefec9..08f93bea 100644 --- a/core/State/Cache/CachingKeyValueStore.cs +++ b/core/State/Cache/CachingKeyValueStore.cs @@ -19,14 +19,15 @@ internal class CachingKeyValueStore : ICachedStateStore { private readonly CacheSize _cacheSize; - private MemoryCache cache; private Action>> flushListener; private bool sendOldValue; private bool cachingEnabled; private Sensor hitRatioSensor = NoRunnableSensor.Empty; private Sensor totalCacheSizeSensor = NoRunnableSensor.Empty; - + + internal MemoryCache Cache { get; private set; } + public CachingKeyValueStore(IKeyValueStore wrapped, CacheSize cacheSize) : base(wrapped) { @@ -58,7 +59,7 @@ internal void CreateCache(ProcessorContext context) cachingEnabled = context.Configuration.DefaultStateStoreCacheMaxBytes > 0 || _cacheSize is { CacheSizeBytes: > 0 }; if(cachingEnabled) - cache = new MemoryCache(new MemoryCacheOptions { + Cache = new MemoryCache(new MemoryCacheOptions { SizeLimit = _cacheSize is { CacheSizeBytes: > 0 } ? _cacheSize.CacheSizeBytes : context.Configuration.DefaultStateStoreCacheMaxBytes, CompactionPercentage = .20 }, new BytesComparer()); @@ -70,7 +71,7 @@ private byte[] GetInternal(Bytes key) { byte[] value; - if (cache.TryGetValue(key, out CacheEntryValue priorEntry)) + if (Cache.TryGetValue(key, out CacheEntryValue priorEntry)) value = priorEntry.Value; else { @@ -79,7 +80,7 @@ private byte[] GetInternal(Bytes key) PutInternal(key, new CacheEntryValue(value), true); } - var currentStat = cache.GetCurrentStatistics(); + var currentStat = Cache.GetCurrentStatistics(); hitRatioSensor.Record((double)currentStat.TotalHits / (currentStat.TotalMisses + currentStat.TotalHits)); return value; @@ -97,7 +98,7 @@ public override void Init(ProcessorContext context, IStateStore root) private void UpdateRatioSensor() { - var currentStat = cache.GetCurrentStatistics(); + var currentStat = Cache.GetCurrentStatistics(); hitRatioSensor.Record((double)currentStat.TotalHits / (currentStat.TotalMisses + currentStat.TotalHits)); } @@ -131,14 +132,14 @@ private void CacheEntryEviction(Bytes key, CacheEntryValue value, EvictionReason context.SetRecordMetaData(currentContext); } - totalCacheSizeSensor.Record(cache.Size); + totalCacheSizeSensor.Record(Cache.Size); } public override void Flush() { if (cachingEnabled) { - cache.Compact(1); // Compact 100% of the cache + Cache.Compact(1); // Compact 100% of the cache base.Flush(); } else @@ -155,8 +156,8 @@ public IKeyValueEnumerator Range(Bytes from, Bytes to) var storeEnumerator = wrapped.Range(from, to); var cacheEnumerator = new CacheEnumerator( - cache.KeyRange(from, to, true, true), - cache, + Cache.KeyRange(from, to, true, true), + Cache, UpdateRatioSensor); return new MergedStoredCacheKeyValueEnumerator(cacheEnumerator, storeEnumerator, true); @@ -171,8 +172,8 @@ public IKeyValueEnumerator ReverseRange(Bytes from, Bytes to) var storeEnumerator = wrapped.ReverseRange(from, to); var cacheEnumerator = new CacheEnumerator( - cache.KeyRange(from, to, true, false), - cache, + Cache.KeyRange(from, to, true, false), + Cache, UpdateRatioSensor); return new MergedStoredCacheKeyValueEnumerator(cacheEnumerator, storeEnumerator, false); @@ -185,8 +186,8 @@ private IEnumerable> InternalAll(bool reverse) { var storeEnumerator = new WrapEnumerableKeyValueEnumerator(wrapped.All()); var cacheEnumerator = new CacheEnumerator( - cache.KeySetEnumerable(reverse), - cache, + Cache.KeySetEnumerable(reverse), + Cache, UpdateRatioSensor); var mergedEnumerator = new MergedStoredCacheKeyValueEnumerator(cacheEnumerator, storeEnumerator, reverse); @@ -210,7 +211,7 @@ public IEnumerable> ReverseAll() return wrapped.ReverseAll(); } - public long ApproximateNumEntries() => cachingEnabled ? cache.Count : wrapped.ApproximateNumEntries(); + public long ApproximateNumEntries() => cachingEnabled ? Cache.Count : wrapped.ApproximateNumEntries(); public void Put(Bytes key, byte[] value) { @@ -236,10 +237,10 @@ private void PutInternal(Bytes key, CacheEntryValue entry, bool fromWrappedCache var memoryCacheEntryOptions = new MemoryCacheEntryOptions() .SetSize(totalSize) - .RegisterPostEvictionCallback(CacheEntryEviction, cache); + .RegisterPostEvictionCallback(CacheEntryEviction, Cache); - cache.Set(key, entry, memoryCacheEntryOptions, fromWrappedCache ? EvictionReason.None : EvictionReason.Setted); - totalCacheSizeSensor.Record(cache.Size); + Cache.Set(key, entry, memoryCacheEntryOptions, fromWrappedCache ? EvictionReason.None : EvictionReason.Setted); + totalCacheSizeSensor.Record(Cache.Size); } public byte[] PutIfAbsent(Bytes key, byte[] value) @@ -282,7 +283,7 @@ public byte[] Delete(Bytes key) { if (cachingEnabled) { - cache.Dispose(); + Cache.Dispose(); base.Close(); } else diff --git a/core/State/Cache/Internal/MemoryCache.cs b/core/State/Cache/Internal/MemoryCache.cs index 51f1a718..69e678f8 100644 --- a/core/State/Cache/Internal/MemoryCache.cs +++ b/core/State/Cache/Internal/MemoryCache.cs @@ -25,9 +25,11 @@ internal sealed class MemoryCache : IMemoryCache private readonly IComparer _keyComparer; private readonly IClockTime _clockTime; internal readonly ILogger Logger; - + private readonly MemoryCacheOptions _options; + internal long Capacity => _options.SizeLimit; + private readonly List> _allStats; private readonly Stats _accumulatedStats; private readonly ThreadLocal _stats; diff --git a/core/Table/CacheSize.cs b/core/Table/CacheSize.cs index 50efab44..a9c951d6 100644 --- a/core/Table/CacheSize.cs +++ b/core/Table/CacheSize.cs @@ -2,7 +2,7 @@ namespace Streamiz.Kafka.Net.Table { public class CacheSize { - internal long CacheSizeBytes { get; private set; } + public long CacheSizeBytes { get; private set; } public static CacheSize OfB(int bytes) => new CacheSize { CacheSizeBytes = bytes }; public static CacheSize OfKb(int kilobytes) => new CacheSize { CacheSizeBytes = kilobytes * 1024 }; diff --git a/test/Streamiz.Kafka.Net.Tests/Helpers/MockProcessorContext.cs b/test/Streamiz.Kafka.Net.Tests/Helpers/MockProcessorContext.cs new file mode 100644 index 00000000..28244dfb --- /dev/null +++ b/test/Streamiz.Kafka.Net.Tests/Helpers/MockProcessorContext.cs @@ -0,0 +1,20 @@ +using System.Collections.Generic; +using Confluent.Kafka; +using Streamiz.Kafka.Net.Metrics; +using Streamiz.Kafka.Net.Mock; +using Streamiz.Kafka.Net.Processors; +using Streamiz.Kafka.Net.Processors.Internal; + +namespace Streamiz.Kafka.Net.Tests.Helpers; + +public class MockProcessorContext : ProcessorContext +{ + public MockProcessorContext(TaskId id, StreamConfig config) + : base(UnassignedStreamTask.Create(), config, new ProcessorStateManager( + id, + new List(), + new Dictionary(), + new MockChangelogRegister(), + new MockOffsetCheckpointManager()), new StreamMetricsRegistry()) + { } +} \ No newline at end of file diff --git a/test/Streamiz.Kafka.Net.Tests/Public/CacheSizeTests.cs b/test/Streamiz.Kafka.Net.Tests/Public/CacheSizeTests.cs new file mode 100644 index 00000000..006f5ef7 --- /dev/null +++ b/test/Streamiz.Kafka.Net.Tests/Public/CacheSizeTests.cs @@ -0,0 +1,61 @@ +using NUnit.Framework; +using Streamiz.Kafka.Net.Processors.Internal; +using Streamiz.Kafka.Net.State.Cache; +using Streamiz.Kafka.Net.State.InMemory; +using Streamiz.Kafka.Net.Table; +using Streamiz.Kafka.Net.Tests.Helpers; + +namespace Streamiz.Kafka.Net.Tests.Public; + +public class CacheSizeTests +{ + [Test] + public void CacheSizeKVStoreEnabledWithDefaultConf() + { + var config = new StreamConfig(); + config.DefaultStateStoreCacheMaxBytes = CacheSize.OfMb(1).CacheSizeBytes; + var context = new MockProcessorContext(new TaskId { Id = 0, Partition = 0 }, config); + var inMemoryKeyValue = new InMemoryKeyValueStore("store"); + var cache = new CachingKeyValueStore(inMemoryKeyValue, null); + cache.Init(context, cache); + Assert.IsTrue(cache.IsCachedStore); + Assert.AreEqual(CacheSize.OfMb(1).CacheSizeBytes, cache.Cache.Capacity); + } + + [Test] + public void CacheSizeKVStoreEnabledWithSpecificConf() + { + var config = new StreamConfig(); + config.DefaultStateStoreCacheMaxBytes = CacheSize.OfMb(1).CacheSizeBytes; + var context = new MockProcessorContext(new TaskId { Id = 0, Partition = 0 }, config); + var inMemoryKeyValue = new InMemoryKeyValueStore("store"); + var cache = new CachingKeyValueStore(inMemoryKeyValue, CacheSize.OfMb(10)); + cache.Init(context, cache); + Assert.IsTrue(cache.IsCachedStore); + Assert.AreEqual(CacheSize.OfMb(10).CacheSizeBytes, cache.Cache.Capacity); + } + + [Test] + public void CacheSizeKVStoreDisabled() + { + var config = new StreamConfig(); + config.DefaultStateStoreCacheMaxBytes = 0L; + var context = new MockProcessorContext(new TaskId { Id = 0, Partition = 0 }, config); + var inMemoryKeyValue = new InMemoryKeyValueStore("store"); + var cache = new CachingKeyValueStore(inMemoryKeyValue, null); + cache.Init(context, cache); + Assert.IsFalse(cache.IsCachedStore); + } + + [Test] + public void CacheSizeKVStoreDisabledExplicitConf() + { + var config = new StreamConfig(); + config.DefaultStateStoreCacheMaxBytes = 0L; + var context = new MockProcessorContext(new TaskId { Id = 0, Partition = 0 }, config); + var inMemoryKeyValue = new InMemoryKeyValueStore("store"); + var cache = new CachingKeyValueStore(inMemoryKeyValue, CacheSize.OfB(0)); + cache.Init(context, cache); + Assert.IsFalse(cache.IsCachedStore); + } +} \ No newline at end of file diff --git a/test/Streamiz.Kafka.Net.Tests/Stores/CacheKeyValueStoreTests.cs b/test/Streamiz.Kafka.Net.Tests/Stores/CacheKeyValueStoreTests.cs index 5275ceac..6151feb2 100644 --- a/test/Streamiz.Kafka.Net.Tests/Stores/CacheKeyValueStoreTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Stores/CacheKeyValueStoreTests.cs @@ -18,7 +18,6 @@ namespace Streamiz.Kafka.Net.Tests.Stores { - // TODO : add test add event from internal wrapped store and flush cache store public class CacheKeyValueStoreTests { private StreamConfig config; From 3580eb1261b4d06ae78f1ab3a212e5a40ad46fa8 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Fri, 5 Jul 2024 15:51:12 -0700 Subject: [PATCH 15/29] #270 - WIP --- .../KStreamDropDuplicateProcessor.cs | 65 ++++++++++ .../State/Internal/WindowStoreMaterializer.cs | 50 ++++++++ core/Stream/IKStream.cs | 8 ++ .../Internal/Graph/KStreamDropDuplicate.cs | 29 +++++ core/Stream/Internal/KStream.cs | 63 +++++++++- .../Processors/KStreamDropDuplicateTests.cs | 111 ++++++++++++++++++ 6 files changed, 323 insertions(+), 3 deletions(-) create mode 100644 core/Processors/KStreamDropDuplicateProcessor.cs create mode 100644 core/State/Internal/WindowStoreMaterializer.cs create mode 100644 core/Stream/Internal/Graph/KStreamDropDuplicate.cs create mode 100644 test/Streamiz.Kafka.Net.Tests/Processors/KStreamDropDuplicateTests.cs diff --git a/core/Processors/KStreamDropDuplicateProcessor.cs b/core/Processors/KStreamDropDuplicateProcessor.cs new file mode 100644 index 00000000..a768d72e --- /dev/null +++ b/core/Processors/KStreamDropDuplicateProcessor.cs @@ -0,0 +1,65 @@ +using System; +using Streamiz.Kafka.Net.State; + +namespace Streamiz.Kafka.Net.Processors +{ + internal class KStreamDropDuplicateProcessor : AbstractProcessor + { + private IWindowStore windowEventStore; + private ProcessorContext context; + private readonly long leftDurationMs; + private readonly long rightDurationMs; + private readonly string storeName; + private readonly Func valueComparer; + + public KStreamDropDuplicateProcessor( + string name, + string storeName, + Func valueComparer, + long maintainDurationMs) + : base(name) + { + leftDurationMs = maintainDurationMs / 2; + rightDurationMs = maintainDurationMs - leftDurationMs; + this.storeName = storeName; + this.valueComparer = valueComparer; + } + + public override void Init(ProcessorContext context) + { + base.Init(context); + this.context = context; + windowEventStore = (IWindowStore)context.GetStateStore(storeName); + } + + public override void Process(K key, V value) + { + LogProcessingKeyValue(key, value); + + if (value != null) + { + if (IsDuplicate(key, value)) { + windowEventStore.Put(key, value, context.Timestamp); + } else { + windowEventStore.Put(key, value, context.Timestamp); + Forward(key, value); + } + } + } + + private bool IsDuplicate(K key, V value) { + long eventTime = context.Timestamp; + + using var iterator = windowEventStore.Fetch( + key, + eventTime - leftDurationMs, + eventTime + rightDurationMs); + bool isDuplicate = iterator.MoveNext(); + + return isDuplicate && + iterator.Current != null && + valueComparer(key, iterator.Current.Value.Value, value); + } + } + +} \ No newline at end of file diff --git a/core/State/Internal/WindowStoreMaterializer.cs b/core/State/Internal/WindowStoreMaterializer.cs new file mode 100644 index 00000000..fb10be69 --- /dev/null +++ b/core/State/Internal/WindowStoreMaterializer.cs @@ -0,0 +1,50 @@ +using System; +using Streamiz.Kafka.Net.Crosscutting; +using Streamiz.Kafka.Net.State.Supplier; +using Streamiz.Kafka.Net.Stream; +using Streamiz.Kafka.Net.Table; + +namespace Streamiz.Kafka.Net.State.Internal +{ + internal class WindowStoreMaterializer + where W : Window + { + private readonly WindowOptions windowsOptions; + private readonly Materialized> materializedInternal; + + public WindowStoreMaterializer(WindowOptions windowsOptions, Materialized> materializedInternal) + { + this.windowsOptions = windowsOptions; + this.materializedInternal = materializedInternal; + } + + public IStoreBuilder> Materialize() + { + IWindowBytesStoreSupplier supplier = (IWindowBytesStoreSupplier)materializedInternal.StoreSupplier; + if (supplier == null) + { + if (windowsOptions.Size + windowsOptions.GracePeriodMs > materializedInternal.Retention.TotalMilliseconds) + throw new ArgumentException($"The retention period of the window store { materializedInternal.StoreName } must be no smaller than its window size plus the grace period. Got size=[{windowsOptions.Size}], grace=[{windowsOptions.GracePeriodMs}], retention=[{materializedInternal.Retention.TotalMilliseconds}]."); + + supplier = Stores.DefaultWindowStore( + materializedInternal.StoreName, + materializedInternal.Retention, + TimeSpan.FromMilliseconds(windowsOptions.Size)); + } + else + supplier.WindowSize = !supplier.WindowSize.HasValue ? windowsOptions.Size : supplier.WindowSize.Value; + + var builder = Stores.WindowStoreBuilder(supplier, materializedInternal.KeySerdes, materializedInternal.ValueSerdes); + + if (materializedInternal.LoggingEnabled) + builder.WithLoggingEnabled(materializedInternal.TopicConfig); + else + builder.WithLoggingDisabled(); + + if (materializedInternal.CachingEnabled) + builder.WithCachingEnabled(materializedInternal.CacheSize); + + return builder; + } + } +} \ No newline at end of file diff --git a/core/Stream/IKStream.cs b/core/Stream/IKStream.cs index 04c830a3..41031a12 100644 --- a/core/Stream/IKStream.cs +++ b/core/Stream/IKStream.cs @@ -2306,5 +2306,13 @@ void ForeachAsync( IKStream WithRecordTimestamp(Func timestampExtractor, string named = null); #endregion + + #region DropDuplicates + + IKStream DropDuplicate(Func valueComparer, TimeSpan interval, string named = null); + + IKStream DropDuplicate(Func valueComparer, TimeSpan interval, Materialized> materialized, string named = null); + + #endregion } } diff --git a/core/Stream/Internal/Graph/KStreamDropDuplicate.cs b/core/Stream/Internal/Graph/KStreamDropDuplicate.cs new file mode 100644 index 00000000..a4290913 --- /dev/null +++ b/core/Stream/Internal/Graph/KStreamDropDuplicate.cs @@ -0,0 +1,29 @@ +using System; +using Streamiz.Kafka.Net.Processors; + +namespace Streamiz.Kafka.Net.Stream.Internal.Graph +{ + internal class KStreamDropDuplicate : IProcessorSupplier + { + private readonly string _name; + private readonly string _storeName; + private readonly Func _valueComparer; + private readonly TimeSpan _interval; + + public KStreamDropDuplicate( + string name, + string storeName, + Func valueComparer, + TimeSpan interval) + + { + _name = name; + _storeName = storeName; + _valueComparer = valueComparer; + _interval = interval; + } + + public IProcessor Get() + => new KStreamDropDuplicateProcessor(_name, _storeName, _valueComparer, (long)_interval.TotalMilliseconds); + } +} \ No newline at end of file diff --git a/core/Stream/Internal/KStream.cs b/core/Stream/Internal/KStream.cs index f21ed209..74701f84 100644 --- a/core/Stream/Internal/KStream.cs +++ b/core/Stream/Internal/KStream.cs @@ -15,6 +15,7 @@ using System.Threading.Tasks; using Confluent.Kafka; using Streamiz.Kafka.Net.Processors.Public; +using Streamiz.Kafka.Net.State.Supplier; namespace Streamiz.Kafka.Net.Stream.Internal { @@ -55,6 +56,7 @@ internal static class KStream internal static readonly string FLATMAPVALUES_ASYNC_NAME = "KSTREAM-FLATMAPVALUES-ASYNC-"; internal static readonly string FOREACH_ASYNC_NAME = "KSTREAM-FOREACH-ASYNC-"; internal static readonly string RECORD_TIMESTAMP_NAME = "KSTREAM-RECORDTIMESTAMP-"; + internal static readonly string DROP_DUPLICATES_NAME = "KSTREAM-DROPDUPLICATES-"; internal static readonly string REQUEST_SINK_SUFFIX = "-request-sink"; internal static readonly string RESPONSE_SINK_SUFFIX = "-response-sink"; @@ -1188,6 +1190,64 @@ public IKStream WithRecordTimestamp(Func timestampExtractor, s return new KStream(name, KeySerdes, ValueSerdes, SetSourceNodes, RepartitionRequired, timestampExtractorNode, builder); } + #endregion + + #region DropDuplicates + + public IKStream DropDuplicate(Func valueComparer, TimeSpan interval, string named = null) + => DropDuplicate(valueComparer, interval, null, named); + + public IKStream DropDuplicate(Func valueComparer, TimeSpan interval, Materialized> materialized, string named = null) + { + materialized ??= Materialized>.Create(); + + var name = new Named(named).OrElseGenerateWithPrefix(builder, KStream.DROP_DUPLICATES_NAME); + materialized.UseProvider(builder, KStream.DROP_DUPLICATES_NAME); + StreamGraphNode tableParentNode = null; + IEnumerable subTopologySourceNodes = null; + + if (RepartitionRequired) + { + (string repartitionSourceName, RepartitionNode node) = CreateRepartitionSource( + name, + KeySerdes, + ValueSerdes, builder); + tableParentNode = node; + builder.AddGraphNode(Node, tableParentNode); + subTopologySourceNodes = repartitionSourceName.ToSingle(); + } + else{ + tableParentNode = Node; + subTopologySourceNodes = SetSourceNodes; + } + + var options = TumblingWindowOptions.Of((long)interval.TotalMilliseconds); + var storeBuilder = new WindowStoreMaterializer(options, materialized).Materialize(); + + var processorSupplier = new KStreamDropDuplicate( + name, + storeBuilder.Name, + valueComparer, + interval); + var processorParameters = new ProcessorParameters(processorSupplier, name); + + StatefulProcessorNode dropDuplicateProcessorNode = + new StatefulProcessorNode( + name, + processorParameters, + storeBuilder); + + builder.AddGraphNode(tableParentNode, dropDuplicateProcessorNode); + + return new KStream( + name, + KeySerdes, + ValueSerdes, + subTopologySourceNodes.ToList(), + dropDuplicateProcessorNode, + builder); + } + #endregion #region Private @@ -1496,8 +1556,5 @@ string asyncProcessorName } #endregion - - - } } \ No newline at end of file diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamDropDuplicateTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamDropDuplicateTests.cs new file mode 100644 index 00000000..687e5ef3 --- /dev/null +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamDropDuplicateTests.cs @@ -0,0 +1,111 @@ +using System; +using System.Linq; +using NUnit.Framework; +using Streamiz.Kafka.Net.Crosscutting; +using Streamiz.Kafka.Net.Mock; +using Streamiz.Kafka.Net.SerDes; +using Streamiz.Kafka.Net.State; +using Streamiz.Kafka.Net.Table; + +namespace Streamiz.Kafka.Net.Tests.Processors; + +public class KStreamDropDuplicateTests +{ + [Test] + public void NoDuplicate() + { + var config = new StreamConfig(); + config.ApplicationId = "test-drop-duplicate"; + + var builder = new StreamBuilder(); + Materialized> m = null; + + builder + .Stream("input") + .DropDuplicate((k,v1,v2) => v1.Equals(v2), + TimeSpan.FromMinutes(1), + InMemoryWindows.As("duplicate-store") + .WithCachingDisabled()) + .To("output"); + + using var driver = new TopologyTestDriver(builder.Build(), config); + var input = driver.CreateInputTopic("input"); + var output = driver.CreateOuputTopic("output"); + input.PipeInput("key1", "test1"); + input.PipeInput("key2", "test2"); + var records = output.ReadKeyValueList().ToList(); + Assert.AreEqual(2, records.Count); + + Assert.AreEqual("key1", records[0].Message.Key); + Assert.AreEqual("test1", records[0].Message.Value); + Assert.AreEqual(0, records[0].Offset.Value); + + Assert.AreEqual("key2", records[1].Message.Key); + Assert.AreEqual("test2", records[1].Message.Value); + Assert.AreEqual(1, records[1].Offset.Value); + } + + [Test] + public void DuplicateSameKeyValue() + { + var config = new StreamConfig(); + config.ApplicationId = "test-drop-duplicate"; + + var builder = new StreamBuilder(); + Materialized> m = null; + + builder + .Stream("input") + .DropDuplicate((k,v1,v2) => v1.Equals(v2), + TimeSpan.FromMinutes(1), + InMemoryWindows.As("duplicate-store") + .WithCachingDisabled()) + .To("output"); + + using var driver = new TopologyTestDriver(builder.Build(), config); + var input = driver.CreateInputTopic("input"); + var output = driver.CreateOuputTopic("output"); + input.PipeInput("key1", "test1"); + input.PipeInput("key1", "test1"); + var records = output.ReadKeyValueList().ToList(); + Assert.AreEqual(1, records.Count); + + Assert.AreEqual("key1", records[0].Message.Key); + Assert.AreEqual("test1", records[0].Message.Value); + Assert.AreEqual(0, records[0].Offset.Value); + } + + [Test] + public void NoDuplicateSameKeyButDifferentValue() + { + var config = new StreamConfig(); + config.ApplicationId = "test-drop-duplicate"; + + var builder = new StreamBuilder(); + Materialized> m = null; + + builder + .Stream("input") + .DropDuplicate((k,v1,v2) => v1.Equals(v2), + TimeSpan.FromMinutes(1), + InMemoryWindows.As("duplicate-store") + .WithCachingDisabled()) + .To("output"); + + using var driver = new TopologyTestDriver(builder.Build(), config); + var input = driver.CreateInputTopic("input"); + var output = driver.CreateOuputTopic("output"); + input.PipeInput("key1", "test1"); + input.PipeInput("key1", "test2"); + var records = output.ReadKeyValueList().ToList(); + Assert.AreEqual(2, records.Count); + + Assert.AreEqual("key1", records[0].Message.Key); + Assert.AreEqual("test1", records[0].Message.Value); + Assert.AreEqual(0, records[0].Offset.Value); + + Assert.AreEqual("key1", records[1].Message.Key); + Assert.AreEqual("test2", records[1].Message.Value); + Assert.AreEqual(1, records[1].Offset.Value); + } +} \ No newline at end of file From 4df0c251ef8aba24b3aa998bc83520c494fad4fe Mon Sep 17 00:00:00 2001 From: LGouellec Date: Fri, 5 Jul 2024 16:24:19 -0700 Subject: [PATCH 16/29] Fix Sonar Lint --- core/Mock/ClusterInMemoryTopologyDriver.cs | 1 - core/Mock/MockChangelogRegister.cs | 3 +-- core/Mock/MockOffsetCheckpointManager.cs | 3 +-- core/Mock/TaskSynchronousTopologyDriver.cs | 3 --- core/Mock/TestMultiInputTopic.cs | 3 +-- core/Mock/TestOutputTopic.cs | 1 - core/Processors/KStreamDropDuplicateProcessor.cs | 7 ++++++- 7 files changed, 9 insertions(+), 12 deletions(-) diff --git a/core/Mock/ClusterInMemoryTopologyDriver.cs b/core/Mock/ClusterInMemoryTopologyDriver.cs index e47fcff7..8b22280b 100644 --- a/core/Mock/ClusterInMemoryTopologyDriver.cs +++ b/core/Mock/ClusterInMemoryTopologyDriver.cs @@ -8,7 +8,6 @@ using System; using System.Collections.Generic; using System.IO; -using System.Linq; using System.Threading; using Streamiz.Kafka.Net.Metrics; using Streamiz.Kafka.Net.Stream.Internal; diff --git a/core/Mock/MockChangelogRegister.cs b/core/Mock/MockChangelogRegister.cs index 2c99f774..fce9b8f5 100644 --- a/core/Mock/MockChangelogRegister.cs +++ b/core/Mock/MockChangelogRegister.cs @@ -1,5 +1,4 @@ -using System; -using System.Collections.Generic; +using System.Collections.Generic; using Confluent.Kafka; using Streamiz.Kafka.Net.Processors.Internal; diff --git a/core/Mock/MockOffsetCheckpointManager.cs b/core/Mock/MockOffsetCheckpointManager.cs index 8467e7e3..3487bd13 100644 --- a/core/Mock/MockOffsetCheckpointManager.cs +++ b/core/Mock/MockOffsetCheckpointManager.cs @@ -1,5 +1,4 @@ -using System; -using System.Collections.Generic; +using System.Collections.Generic; using Confluent.Kafka; using Streamiz.Kafka.Net.Processors.Internal; using Streamiz.Kafka.Net.State; diff --git a/core/Mock/TaskSynchronousTopologyDriver.cs b/core/Mock/TaskSynchronousTopologyDriver.cs index 08200679..8b19368c 100644 --- a/core/Mock/TaskSynchronousTopologyDriver.cs +++ b/core/Mock/TaskSynchronousTopologyDriver.cs @@ -7,13 +7,10 @@ using Streamiz.Kafka.Net.SerDes; using System; using System.Collections.Generic; -using System.ComponentModel; -using System.Data.SqlTypes; using System.IO; using System.Linq; using System.Threading; using Streamiz.Kafka.Net.Crosscutting; -using Streamiz.Kafka.Net.Kafka.Internal; using Streamiz.Kafka.Net.Metrics; using Streamiz.Kafka.Net.Stream.Internal; diff --git a/core/Mock/TestMultiInputTopic.cs b/core/Mock/TestMultiInputTopic.cs index 4afa4827..492ed3b7 100644 --- a/core/Mock/TestMultiInputTopic.cs +++ b/core/Mock/TestMultiInputTopic.cs @@ -1,5 +1,4 @@ - -using Confluent.Kafka; +using Confluent.Kafka; using Streamiz.Kafka.Net.Mock.Pipes; using Streamiz.Kafka.Net.SerDes; using System; diff --git a/core/Mock/TestOutputTopic.cs b/core/Mock/TestOutputTopic.cs index 19d9494f..5c1e1de3 100644 --- a/core/Mock/TestOutputTopic.cs +++ b/core/Mock/TestOutputTopic.cs @@ -3,7 +3,6 @@ using Streamiz.Kafka.Net.Errors; using Streamiz.Kafka.Net.Mock.Pipes; using Streamiz.Kafka.Net.SerDes; -using System; using System.Collections.Generic; using System.Linq; using Microsoft.Extensions.Logging; diff --git a/core/Processors/KStreamDropDuplicateProcessor.cs b/core/Processors/KStreamDropDuplicateProcessor.cs index a768d72e..520399d4 100644 --- a/core/Processors/KStreamDropDuplicateProcessor.cs +++ b/core/Processors/KStreamDropDuplicateProcessor.cs @@ -1,4 +1,5 @@ using System; +using System.Collections.Generic; using Streamiz.Kafka.Net.State; namespace Streamiz.Kafka.Net.Processors @@ -36,7 +37,7 @@ public override void Process(K key, V value) { LogProcessingKeyValue(key, value); - if (value != null) + if (!EqualityComparer.Default.Equals(value, default(V))) { if (IsDuplicate(key, value)) { windowEventStore.Put(key, value, context.Timestamp); @@ -45,6 +46,10 @@ public override void Process(K key, V value) Forward(key, value); } } + else + { + Forward(key, value); + } } private bool IsDuplicate(K key, V value) { From a2efbff0543825fe1986760d68437f15139d4a56 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Mon, 8 Jul 2024 14:35:21 -0700 Subject: [PATCH 17/29] Avoid duplicated codes --- .../TimestampedWindowStoreMaterializer.cs | 42 ++++--------------- .../State/Internal/WindowStoreMaterializer.cs | 42 ++++++++++++------- core/Stream/Internal/KStream.cs | 2 +- .../Metrics/StateStoreMetricTests.cs | 4 +- 4 files changed, 38 insertions(+), 52 deletions(-) diff --git a/core/State/Internal/TimestampedWindowStoreMaterializer.cs b/core/State/Internal/TimestampedWindowStoreMaterializer.cs index 135a8fe3..147600ef 100644 --- a/core/State/Internal/TimestampedWindowStoreMaterializer.cs +++ b/core/State/Internal/TimestampedWindowStoreMaterializer.cs @@ -6,45 +6,17 @@ namespace Streamiz.Kafka.Net.State.Internal { - internal class TimestampedWindowStoreMaterializer + internal class TimestampedWindowStoreMaterializer : + WindowStoreMaterializer> where W : Window { - private readonly WindowOptions windowsOptions; - private readonly Materialized> materializedInternal; - public TimestampedWindowStoreMaterializer(WindowOptions windowsOptions, Materialized> materializedInternal) + : base(windowsOptions, materializedInternal) { - this.windowsOptions = windowsOptions; - this.materializedInternal = materializedInternal; + } - public IStoreBuilder> Materialize() - { - IWindowBytesStoreSupplier supplier = (IWindowBytesStoreSupplier)materializedInternal.StoreSupplier; - if (supplier == null) - { - if (windowsOptions.Size + windowsOptions.GracePeriodMs > materializedInternal.Retention.TotalMilliseconds) - throw new ArgumentException($"The retention period of the window store { materializedInternal.StoreName } must be no smaller than its window size plus the grace period. Got size=[{windowsOptions.Size}], grace=[{windowsOptions.GracePeriodMs}], retention=[{materializedInternal.Retention.TotalMilliseconds}]."); - - supplier = Stores.DefaultWindowStore( - materializedInternal.StoreName, - materializedInternal.Retention, - TimeSpan.FromMilliseconds(windowsOptions.Size)); - } - else - supplier.WindowSize = !supplier.WindowSize.HasValue ? windowsOptions.Size : supplier.WindowSize.Value; - - var builder = Stores.TimestampedWindowStoreBuilder(supplier, materializedInternal.KeySerdes, materializedInternal.ValueSerdes); - - if (materializedInternal.LoggingEnabled) - builder.WithLoggingEnabled(materializedInternal.TopicConfig); - else - builder.WithLoggingDisabled(); - - if (materializedInternal.CachingEnabled) - builder.WithCachingEnabled(materializedInternal.CacheSize); - - return builder; - } - } + protected override IStoreBuilder> GetWindowStoreBuilder(IWindowBytesStoreSupplier supplier) + => Stores.TimestampedWindowStoreBuilder(supplier, materializedInternal.KeySerdes, materializedInternal.ValueSerdes); + } } diff --git a/core/State/Internal/WindowStoreMaterializer.cs b/core/State/Internal/WindowStoreMaterializer.cs index fb10be69..952252a1 100644 --- a/core/State/Internal/WindowStoreMaterializer.cs +++ b/core/State/Internal/WindowStoreMaterializer.cs @@ -1,16 +1,18 @@ using System; using Streamiz.Kafka.Net.Crosscutting; +using Streamiz.Kafka.Net.Processors; using Streamiz.Kafka.Net.State.Supplier; using Streamiz.Kafka.Net.Stream; using Streamiz.Kafka.Net.Table; namespace Streamiz.Kafka.Net.State.Internal { - internal class WindowStoreMaterializer + internal class WindowStoreMaterializer + where WS : IStateStore where W : Window { - private readonly WindowOptions windowsOptions; - private readonly Materialized> materializedInternal; + protected readonly WindowOptions windowsOptions; + protected readonly Materialized> materializedInternal; public WindowStoreMaterializer(WindowOptions windowsOptions, Materialized> materializedInternal) { @@ -18,7 +20,23 @@ public WindowStoreMaterializer(WindowOptions windowsOptions, Materialized> Materialize() + public IStoreBuilder Materialize() + { + var supplier = GetWindowSupplier(); + var builder = GetWindowStoreBuilder(supplier); + + if (materializedInternal.LoggingEnabled) + builder.WithLoggingEnabled(materializedInternal.TopicConfig); + else + builder.WithLoggingDisabled(); + + if (materializedInternal.CachingEnabled) + builder.WithCachingEnabled(materializedInternal.CacheSize); + + return builder; + } + + protected IWindowBytesStoreSupplier GetWindowSupplier() { IWindowBytesStoreSupplier supplier = (IWindowBytesStoreSupplier)materializedInternal.StoreSupplier; if (supplier == null) @@ -34,17 +52,13 @@ public IStoreBuilder> Materialize() else supplier.WindowSize = !supplier.WindowSize.HasValue ? windowsOptions.Size : supplier.WindowSize.Value; - var builder = Stores.WindowStoreBuilder(supplier, materializedInternal.KeySerdes, materializedInternal.ValueSerdes); - - if (materializedInternal.LoggingEnabled) - builder.WithLoggingEnabled(materializedInternal.TopicConfig); - else - builder.WithLoggingDisabled(); - - if (materializedInternal.CachingEnabled) - builder.WithCachingEnabled(materializedInternal.CacheSize); + return supplier; + } - return builder; + protected virtual IStoreBuilder GetWindowStoreBuilder(IWindowBytesStoreSupplier supplier) + { + return (IStoreBuilder) + Stores.WindowStoreBuilder(supplier, materializedInternal.KeySerdes, materializedInternal.ValueSerdes); } } } \ No newline at end of file diff --git a/core/Stream/Internal/KStream.cs b/core/Stream/Internal/KStream.cs index 74701f84..ccbfaf5d 100644 --- a/core/Stream/Internal/KStream.cs +++ b/core/Stream/Internal/KStream.cs @@ -1222,7 +1222,7 @@ public IKStream DropDuplicate(Func valueComparer, TimeSpan } var options = TumblingWindowOptions.Of((long)interval.TotalMilliseconds); - var storeBuilder = new WindowStoreMaterializer(options, materialized).Materialize(); + var storeBuilder = new WindowStoreMaterializer>(options, materialized).Materialize(); var processorSupplier = new KStreamDropDuplicate( name, diff --git a/test/Streamiz.Kafka.Net.Tests/Metrics/StateStoreMetricTests.cs b/test/Streamiz.Kafka.Net.Tests/Metrics/StateStoreMetricTests.cs index 9fa7a3f2..c3b70ccb 100644 --- a/test/Streamiz.Kafka.Net.Tests/Metrics/StateStoreMetricTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Metrics/StateStoreMetricTests.cs @@ -346,8 +346,8 @@ private void AssertAvgAndMaxLatency(string sensorName) sensorName, StreamMetricsRegistry.LATENCY_SUFFIX + StreamMetricsRegistry.MAX_SUFFIX, StreamMetricsRegistry.STATE_STORE_LEVEL_GROUP); - Assert.IsTrue((double)latencyAvg.Value >= 0); - Assert.IsTrue((double)latencyMax.Value >= 0); + Assert.GreaterOrEqual((double)latencyAvg.Value, 0); + Assert.GreaterOrEqual((double)latencyMax.Value, 0); } private StreamMetric GetSensorMetric(string sensorName, string metricSuffix, string group) From cc2ae6cc211c7fe32339a5e879d447673c14f518 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Mon, 8 Jul 2024 15:18:48 -0700 Subject: [PATCH 18/29] Add documentation --- core/Stream/IKStream.cs | 15 +++++++++++++++ core/Stream/Internal/KStream.cs | 15 +++++++++++++++ core/Table/CacheSize.cs | 29 +++++++++++++++++++++++++++++ docs/stateful-processors.md | 20 +++++++++++++++++++- samples/sample-stream/Program.cs | 2 ++ 5 files changed, 80 insertions(+), 1 deletion(-) diff --git a/core/Stream/IKStream.cs b/core/Stream/IKStream.cs index 41031a12..05b06374 100644 --- a/core/Stream/IKStream.cs +++ b/core/Stream/IKStream.cs @@ -2309,8 +2309,23 @@ void ForeachAsync( #region DropDuplicates + /// + /// This function removes duplicate records from the same key based on the time interval mentioned. + /// + /// Lambda function which determine if the old value and the new value for the same key is equal + /// Time elapsed between two records are considered duplicates + /// A config used to name the processor in the topology. Default : null + /// a which rejects duplicates records IKStream DropDuplicate(Func valueComparer, TimeSpan interval, string named = null); + /// + /// This function removes duplicate records from the same key based on the time interval mentioned. + /// + /// Lambda function which determine if the old value and the new value for the same key is equal + /// Time elapsed between two records are considered duplicates + /// an instance of used to describe how the window state store should be materialized. + /// A config used to name the processor in the topology. Default : null + /// a which rejects duplicates records IKStream DropDuplicate(Func valueComparer, TimeSpan interval, Materialized> materialized, string named = null); #endregion diff --git a/core/Stream/Internal/KStream.cs b/core/Stream/Internal/KStream.cs index ccbfaf5d..c2ee034d 100644 --- a/core/Stream/Internal/KStream.cs +++ b/core/Stream/Internal/KStream.cs @@ -1194,9 +1194,24 @@ public IKStream WithRecordTimestamp(Func timestampExtractor, s #region DropDuplicates + /// + /// This function removes duplicate records from the same key based on the time interval mentioned. + /// + /// Lambda function which determine if the old value and the new value for the same key is equal + /// Time elapsed between two records are considered duplicates + /// A config used to name the processor in the topology. Default : null + /// a which rejects duplicates records public IKStream DropDuplicate(Func valueComparer, TimeSpan interval, string named = null) => DropDuplicate(valueComparer, interval, null, named); + /// + /// This function removes duplicate records from the same key based on the time interval mentioned. + /// + /// Lambda function which determine if the old value and the new value for the same key is equal + /// Time elapsed between two records are considered duplicates + /// an instance of used to describe how the window state store should be materialized. + /// A config used to name the processor in the topology. Default : null + /// a which rejects duplicates records public IKStream DropDuplicate(Func valueComparer, TimeSpan interval, Materialized> materialized, string named = null) { materialized ??= Materialized>.Create(); diff --git a/core/Table/CacheSize.cs b/core/Table/CacheSize.cs index a9c951d6..8f7b0178 100644 --- a/core/Table/CacheSize.cs +++ b/core/Table/CacheSize.cs @@ -1,12 +1,41 @@ namespace Streamiz.Kafka.Net.Table { + /// + /// Specify the maximum cache size + /// public class CacheSize { + /// + /// Number of binary bytes + /// public long CacheSizeBytes { get; private set; } + /// + /// Create a based on the paramters. + /// + /// Number of bytes (in binary) of your cache size + /// public static CacheSize OfB(int bytes) => new CacheSize { CacheSizeBytes = bytes }; + + /// + /// Create a based on the paramters. + /// + /// Number of kilobytes (in binary) of your cache size + /// public static CacheSize OfKb(int kilobytes) => new CacheSize { CacheSizeBytes = kilobytes * 1024 }; + + /// + /// Create a based on the paramters. + /// + /// Number of megabytes (in binary) of your cache size + /// public static CacheSize OfMb(int megabytes) => new CacheSize { CacheSizeBytes = megabytes * 1024 * 1024 }; + + /// + /// Create a based on the paramters. + /// + /// Number of gigabytes (in binary) of your cache size + /// public static CacheSize OfGb(int gigabytes) => new CacheSize { CacheSizeBytes = gigabytes * 1024 * 1024 * 1024 }; } } \ No newline at end of file diff --git a/docs/stateful-processors.md b/docs/stateful-processors.md index 055dc983..3accf30e 100644 --- a/docs/stateful-processors.md +++ b/docs/stateful-processors.md @@ -458,4 +458,22 @@ Detailed behavior: - The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied ValueJoiner will be called to produce join output records. - Input records with a null key are ignored and do not trigger the join. - Input records with a null value are interpreted as tombstones for the corresponding key, which indicate the deletion of the key from the table. Tombstones do not trigger the join. When an input tombstone is received, then an output tombstone is forwarded directly to the join result IKTable if required (i.e. only if the corresponding key actually exists already in the join result IKTable). -- For each input record on one side that does not have any match on the other side, the ValueJoiner will be called with (leftRecord.value, null) or (null, rightRecord.value), respectively; this explains the rows with timestamp=3 and timestamp=7 in the table below, which list [A, null] and [null, b], respectively, in the OUTER JOIN column. \ No newline at end of file +- For each input record on one side that does not have any match on the other side, the ValueJoiner will be called with (leftRecord.value, null) or (null, rightRecord.value), respectively; this explains the rows with timestamp=3 and timestamp=7 in the table below, which list [A, null] and [null, b], respectively, in the OUTER JOIN column. + + +## Drop Duplicates + +This function removes duplicate records from the same key based on the time interval mentioned. Duplicate records are detected based on the kafka key plus the lambda returns; In this simplified example, the record value is the event ID. The internal processor remembers the old event IDs in an associated window state store, which automatically purges/expires event IDs from the store after a certain amount of time has passed to prevent the store from growing indefinitely. + +``` csharp +builder.Stream("input") + .DropDuplicate((key, value1, value2) => value1.Equals(value2), TimeSpan.FromMinutes(1)) + .To("output"); +``` + +| Timestamp | Input Topic | Window Store | Output Topic | +|------------|-------------------------------|----------------------------------------|------------------------------------------| +| t0 | key: key1 ; value : eventID1 | key: key1 ; value : eventID1 ; t0 | key: key1 ; value : eventID1 | +| t0+10s | key: key1 ; value : eventID1 | key: key1 ; value : eventID1 ; t0 | X | +| t0+30s | key: key1 ; value : eventID2 | key: key1 ; value : eventID2 ; t0+30s| key: key1 ; value : eventID2 | +| t0+95s | key: key1 ; value : eventID2 | key: key1 ; value : eventID2 ; t0+95s| key: key1 ; value : eventID2 | \ No newline at end of file diff --git a/samples/sample-stream/Program.cs b/samples/sample-stream/Program.cs index 19060913..3238a83a 100644 --- a/samples/sample-stream/Program.cs +++ b/samples/sample-stream/Program.cs @@ -58,6 +58,8 @@ private static Topology BuildTopology() new StringSerDes());*/ builder.Stream("input") + .DropDuplicate((key, value1, value2) => value1.Equals(value2), + TimeSpan.FromMinutes(1)) .To( "output");//, (s, s1, arg3, arg4) => new Partition(0)); From 6358a7fd238d954f8c87ebfd38c8304555e78cb0 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Mon, 8 Jul 2024 16:16:17 -0700 Subject: [PATCH 19/29] fix #338 --- core/Mock/Kafka/MockCluster.cs | 9 +- core/Mock/Kafka/MockPartition.cs | 9 +- core/Mock/Kafka/MockTopic.cs | 4 +- core/Mock/Sync/SyncPipeBuilder.cs | 14 ++-- .../Processors/HeaderMetadataTests.cs | 82 +++++++++++++++++++ 5 files changed, 102 insertions(+), 16 deletions(-) create mode 100644 test/Streamiz.Kafka.Net.Tests/Processors/HeaderMetadataTests.cs diff --git a/core/Mock/Kafka/MockCluster.cs b/core/Mock/Kafka/MockCluster.cs index 70d559da..63e0e783 100644 --- a/core/Mock/Kafka/MockCluster.cs +++ b/core/Mock/Kafka/MockCluster.cs @@ -760,7 +760,8 @@ internal ConsumeResult Consume(MockConsumer mockConsumer, TimeSp Key = record.Key, Value = record.Value, Timestamp = new Timestamp( - record.Timestamp ?? DateTime.Now, TimestampType.NotAvailable) + record.Timestamp ?? DateTime.Now, TimestampType.NotAvailable), + Headers = record.Headers } }; ++offset.OffsetConsumed; @@ -800,7 +801,7 @@ internal DeliveryReport Produce(string topic, Message Produce(TopicPartition topicPartition, M CreateTopic(topicPartition.Topic); if (topics[topicPartition.Topic].PartitionNumber > topicPartition.Partition) { - topics[topicPartition.Topic].AddMessage(message.Key, message.Value, topicPartition.Partition, message.Timestamp.UnixTimestampMs); + topics[topicPartition.Topic].AddMessage(message.Key, message.Value, topicPartition.Partition, message.Timestamp.UnixTimestampMs, message.Headers); r.Status = PersistenceStatus.Persisted; } else { topics[topicPartition.Topic].CreateNewPartitions(topicPartition.Partition); - topics[topicPartition.Topic].AddMessage(message.Key, message.Value, topicPartition.Partition, message.Timestamp.UnixTimestampMs); + topics[topicPartition.Topic].AddMessage(message.Key, message.Value, topicPartition.Partition, message.Timestamp.UnixTimestampMs, message.Headers); r.Status = PersistenceStatus.Persisted; } diff --git a/core/Mock/Kafka/MockPartition.cs b/core/Mock/Kafka/MockPartition.cs index 987ab39f..37cc555c 100644 --- a/core/Mock/Kafka/MockPartition.cs +++ b/core/Mock/Kafka/MockPartition.cs @@ -7,7 +7,7 @@ namespace Streamiz.Kafka.Net.Mock.Kafka { internal class MockPartition { - private readonly List<(byte[], byte[], long)> log = new(); + private readonly List<(byte[], byte[], long, Headers)> log = new(); private readonly Dictionary mappingOffsets = new(); public MockPartition(int indice) @@ -21,10 +21,10 @@ public MockPartition(int indice) public long LowOffset { get; private set; } = Offset.Unset; public long HighOffset { get; private set; } = Offset.Unset; - internal void AddMessageInLog(byte[] key, byte[] value, long timestamp) + internal void AddMessageInLog(byte[] key, byte[] value, long timestamp, Headers headers) { mappingOffsets.Add(Size, log.Count); - log.Add((key, value, timestamp)); + log.Add((key, value, timestamp, headers)); ++Size; UpdateOffset(); } @@ -47,7 +47,8 @@ internal TestRecord GetMessage(long offset) { Key = record.Item1, Value = record.Item2, - Timestamp = record.Item3.FromMilliseconds() + Timestamp = record.Item3.FromMilliseconds(), + Headers = record.Item4 }; } diff --git a/core/Mock/Kafka/MockTopic.cs b/core/Mock/Kafka/MockTopic.cs index d96f82fa..c933c22a 100644 --- a/core/Mock/Kafka/MockTopic.cs +++ b/core/Mock/Kafka/MockTopic.cs @@ -23,9 +23,9 @@ public MockTopic(string topic, int part) public int PartitionNumber { get; private set; } public IEnumerable Partitions => partitions.AsReadOnly(); - public void AddMessage(byte[] key, byte[] value, int partition, long timestamp = 0) + public void AddMessage(byte[] key, byte[] value, int partition, long timestamp = 0, Headers headers = null) { - partitions[partition].AddMessageInLog(key, value, timestamp); + partitions[partition].AddMessageInLog(key, value, timestamp, headers); } public TestRecord GetMessage(int partition, long consumerOffset) diff --git a/core/Mock/Sync/SyncPipeBuilder.cs b/core/Mock/Sync/SyncPipeBuilder.cs index 6750288c..e236596a 100644 --- a/core/Mock/Sync/SyncPipeBuilder.cs +++ b/core/Mock/Sync/SyncPipeBuilder.cs @@ -24,21 +24,23 @@ public StreamTaskPublisher(StreamTask task) private int offset = 0; public void PublishRecord(string topic, byte[] key, byte[] value, DateTime timestamp, Headers headers) - => task.AddRecord(new ConsumeResult - { - Topic = topic, - TopicPartitionOffset = new TopicPartitionOffset(new TopicPartition(topic, task.Id.Partition), offset++), - Message = new Message { Key = key, Value = value, Timestamp = new Timestamp(timestamp), Headers = headers } - }); + => task.AddRecord(new ConsumeResult + { + Topic = topic, + TopicPartitionOffset = new TopicPartitionOffset(new TopicPartition(topic, task.Id.Partition), offset++), + Message = new Message { Key = key, Value = value, Timestamp = new Timestamp(timestamp), Headers = headers } + }); public void Flush() { long now = DateTime.Now.GetMilliseconds(); + TaskManager.CurrentTask = task; while (task.CanProcess(now)) task.Process(); task.PunctuateStreamTime(); task.PunctuateSystemTime(); + TaskManager.CurrentTask = null; } public void Close() diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/HeaderMetadataTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/HeaderMetadataTests.cs new file mode 100644 index 00000000..14070d43 --- /dev/null +++ b/test/Streamiz.Kafka.Net.Tests/Processors/HeaderMetadataTests.cs @@ -0,0 +1,82 @@ +using System.Text; +using Confluent.Kafka; +using NUnit.Framework; +using Streamiz.Kafka.Net.Mock; +using Streamiz.Kafka.Net.SerDes; + +namespace Streamiz.Kafka.Net.Tests.Processors; + +public class HeaderMetadataTests +{ + [Test] + public void TestIssue338Sync() + { + var config = new StreamConfig + { + ApplicationId = "test-topic-splitter", + SchemaRegistryUrl = "mock://test", + FollowMetadata = true, + MaxPollIntervalMs = 100 + }; + Headers Headers = new(); + Headers.Add("MessageType", Encoding.ASCII.GetBytes("headers-type")); + + var builder = new StreamBuilder(); + builder + .Stream("topic") + .MapValues((v) => + { + var h = StreamizMetadata.GetCurrentHeadersMetadata(); + h.Add("header1", Encoding.UTF8.GetBytes("value1")); + return v; + }) + .To("output"); + + using var driver = new TopologyTestDriver(builder.Build(), config, TopologyTestDriver.Mode.SYNC_TASK); + var inputTopic = driver.CreateInputTopic("topic"); + var outputTopic = driver.CreateOuputTopic("output"); + + inputTopic.PipeInput("test-1", "order", Headers); + + var sinkRecord = outputTopic.ReadKeyValue(); + Assert.AreEqual(2, sinkRecord.Message.Headers.Count); + Assert.IsNotNull( sinkRecord.Message.Headers.GetLastBytes("header1")); + Assert.IsNotNull( sinkRecord.Message.Headers.GetLastBytes("MessageType")); + } + + [Test] + public void TestIssue338Async() + { + var config = new StreamConfig + { + ApplicationId = "test-topic-splitter", + SchemaRegistryUrl = "mock://test", + FollowMetadata = true, + MaxPollIntervalMs = 100 + }; + Headers Headers = new(); + Headers.Add("MessageType", Encoding.ASCII.GetBytes("headers-type")); + + var builder = new StreamBuilder(); + builder + .Stream("topic") + .MapValues((v) => + { + var h = StreamizMetadata.GetCurrentHeadersMetadata(); + h.Add("header1", Encoding.UTF8.GetBytes("value1")); + return v; + }) + .To("output"); + + using var driver = new TopologyTestDriver(builder.Build(), config, TopologyTestDriver.Mode.ASYNC_CLUSTER_IN_MEMORY); + var inputTopic = driver.CreateInputTopic("topic"); + var outputTopic = driver.CreateOuputTopic("output"); + + inputTopic.PipeInput("test-1", "order", Headers); + + var sinkRecord = outputTopic.ReadKeyValue(); + Assert.AreEqual(2, sinkRecord.Message.Headers.Count); + Assert.IsNotNull( sinkRecord.Message.Headers.GetLastBytes("header1")); + Assert.IsNotNull( sinkRecord.Message.Headers.GetLastBytes("MessageType")); + } +} \ No newline at end of file From 5eee3be9a39cb2f28425d7f7149eccd56bfc12cb Mon Sep 17 00:00:00 2001 From: LGouellec Date: Tue, 9 Jul 2024 16:31:28 -0700 Subject: [PATCH 20/29] Reproducer #314 --- core/State/InMemory/InMemoryWindowStore.cs | 7 +- .../InMemory/Internal/ConcurrentHashSet.cs | 244 ++++++++++++++++++ environment/datagen_connector.json | 14 + environment/docker-compose-with-connect.yml | 8 +- samples/sample-stream/Program.cs | 2 +- samples/sample-stream/Reproducer314.cs | 75 ++++++ 6 files changed, 342 insertions(+), 8 deletions(-) create mode 100644 core/State/InMemory/Internal/ConcurrentHashSet.cs create mode 100644 environment/datagen_connector.json create mode 100644 samples/sample-stream/Reproducer314.cs diff --git a/core/State/InMemory/InMemoryWindowStore.cs b/core/State/InMemory/InMemoryWindowStore.cs index 997c9401..7ab51a39 100644 --- a/core/State/InMemory/InMemoryWindowStore.cs +++ b/core/State/InMemory/InMemoryWindowStore.cs @@ -12,6 +12,7 @@ using Streamiz.Kafka.Net.Metrics; using Streamiz.Kafka.Net.Metrics.Internal; using Streamiz.Kafka.Net.State.Helper; +using Streamiz.Kafka.Net.State.InMemory.Internal; namespace Streamiz.Kafka.Net.State.InMemory { @@ -260,7 +261,8 @@ public Windowed PeekNextKey() #endregion - internal class InMemoryWindowStore : IWindowStore + internal class + InMemoryWindowStore : IWindowStore { private readonly TimeSpan retention; private readonly long size; @@ -272,8 +274,7 @@ internal class InMemoryWindowStore : IWindowStore private int seqnum = 0; private readonly ConcurrentDictionary> map = new(); - - private readonly ISet openIterators = new HashSet(); + private readonly ConcurrentSet openIterators = new(); private readonly ILogger logger = Logger.GetLogger(typeof(InMemoryWindowStore)); diff --git a/core/State/InMemory/Internal/ConcurrentHashSet.cs b/core/State/InMemory/Internal/ConcurrentHashSet.cs new file mode 100644 index 00000000..57ed5895 --- /dev/null +++ b/core/State/InMemory/Internal/ConcurrentHashSet.cs @@ -0,0 +1,244 @@ +using System; +using System.Collections; +using System.Collections.Concurrent; +using System.Collections.Generic; +using System.Linq; + +namespace Streamiz.Kafka.Net.State.InMemory.Internal +{ + internal class ConcurrentSet : IEnumerable, ISet, ICollection + { + private readonly ConcurrentDictionary _dictionary = new(); + + /// + /// Returns an enumerator that iterates through the collection. + /// + /// + /// A that can be used to iterate through the collection. + /// + public IEnumerator GetEnumerator() + { + return _dictionary.Keys.GetEnumerator(); + } + + /// + /// Returns an enumerator that iterates through a collection. + /// + /// + /// An object that can be used to iterate through the collection. + /// + IEnumerator IEnumerable.GetEnumerator() + { + return GetEnumerator(); + } + + /// + /// Removes the first occurrence of a specific object from the . + /// + /// + /// true if was successfully removed from the ; otherwise, false. This method also returns false if is not found in the original . + /// + /// The object to remove from the .The is read-only. + public bool Remove(T item) + { + return TryRemove(item); + } + + /// + /// Gets the number of elements in the set. + /// + public int Count => _dictionary.Count; + + /// + /// Gets a value indicating whether the is read-only. + /// + /// + /// true if the is read-only; otherwise, false. + /// + public bool IsReadOnly => false; + + /// + /// Gets a value that indicates if the set is empty. + /// + public bool IsEmpty => _dictionary.IsEmpty; + + private ICollection Values => _dictionary.Keys; + + /// + /// Adds an item to the . + /// + /// The object to add to the .The is read-only. + void ICollection.Add(T item) + { + if (!Add(item)) + throw new ArgumentException("Item already exists in set."); + } + + /// + /// Modifies the current set so that it contains all elements that are present in both the current set and in the specified collection. + /// + /// The collection to compare to the current set. is null. + public void UnionWith(IEnumerable other) + { + foreach (var item in other) + TryAdd(item); + } + + /// + /// Modifies the current set so that it contains only elements that are also in a specified collection. + /// + /// The collection to compare to the current set. is null. + public void IntersectWith(IEnumerable other) + { + var enumerable = other as IList ?? other.ToArray(); + foreach (var item in this) + { + if (!enumerable.Contains(item)) + TryRemove(item); + } + } + + /// + /// Removes all elements in the specified collection from the current set. + /// + /// The collection of items to remove from the set. is null. + public void ExceptWith(IEnumerable other) + { + foreach (var item in other) + TryRemove(item); + } + + /// + /// Modifies the current set so that it contains only elements that are present either in the current set or in the specified collection, but not both. + /// + /// The collection to compare to the current set. is null. + public void SymmetricExceptWith(IEnumerable other) + { + throw new NotImplementedException(); + } + + /// + /// Determines whether a set is a subset of a specified collection. + /// + /// + /// true if the current set is a subset of ; otherwise, false. + /// + /// The collection to compare to the current set. is null. + public bool IsSubsetOf(IEnumerable other) + { + var enumerable = other as IList ?? other.ToArray(); + return this.AsParallel().All(enumerable.Contains); + } + + /// + /// Determines whether the current set is a superset of a specified collection. + /// + /// + /// true if the current set is a superset of ; otherwise, false. + /// + /// The collection to compare to the current set. is null. + public bool IsSupersetOf(IEnumerable other) + { + return other.AsParallel().All(Contains); + } + + /// + /// Determines whether the current set is a correct superset of a specified collection. + /// + /// + /// true if the object is a correct superset of ; otherwise, false. + /// + /// The collection to compare to the current set. is null. + public bool IsProperSupersetOf(IEnumerable other) + { + var enumerable = other as IList ?? other.ToArray(); + return this.Count != enumerable.Count && IsSupersetOf(enumerable); + } + + /// + /// Determines whether the current set is a property (strict) subset of a specified collection. + /// + /// + /// true if the current set is a correct subset of ; otherwise, false. + /// + /// The collection to compare to the current set. is null. + public bool IsProperSubsetOf(IEnumerable other) + { + var enumerable = other as IList ?? other.ToArray(); + return Count != enumerable.Count && IsSubsetOf(enumerable); + } + + /// + /// Determines whether the current set overlaps with the specified collection. + /// + /// + /// true if the current set and share at least one common element; otherwise, false. + /// + /// The collection to compare to the current set. is null. + public bool Overlaps(IEnumerable other) + { + return other.AsParallel().Any(Contains); + } + + /// + /// Determines whether the current set and the specified collection contain the same elements. + /// + /// + /// true if the current set is equal to ; otherwise, false. + /// + /// The collection to compare to the current set. is null. + public bool SetEquals(IEnumerable other) + { + var enumerable = other as IList ?? other.ToArray(); + return Count == enumerable.Count && enumerable.AsParallel().All(Contains); + } + + /// + /// Adds an element to the current set and returns a value to indicate if the element was successfully added. + /// + /// + /// true if the element is added to the set; false if the element is already in the set. + /// + /// The element to add to the set. + public bool Add(T item) + { + return TryAdd(item); + } + + public void Clear() + { + _dictionary.Clear(); + } + + public bool Contains(T item) + { + return _dictionary.ContainsKey(item); + } + + /// + /// Copies the elements of the to an , starting at a particular index. + /// + /// The one-dimensional that is the destination of the elements copied from . The must have zero-based indexing.The zero-based index in at which copying begins. is null. is less than 0. is multidimensional.-or-The number of elements in the source is greater than the available space from to the end of the destination .-or-Type cannot be cast automatically to the type of the destination . + public void CopyTo(T[] array, int arrayIndex) + { + Values.CopyTo(array, arrayIndex); + } + + public T[] ToArray() + { + return _dictionary.Keys.ToArray(); + } + + public bool TryAdd(T item) + { + return _dictionary.TryAdd(item, default(byte)); + } + + public bool TryRemove(T item) + { + byte donotcare; + return _dictionary.TryRemove(item, out donotcare); + } + } + +} \ No newline at end of file diff --git a/environment/datagen_connector.json b/environment/datagen_connector.json new file mode 100644 index 00000000..66abd216 --- /dev/null +++ b/environment/datagen_connector.json @@ -0,0 +1,14 @@ +{ + "name": "datagen-users", + "config": { + "connector.class": "io.confluent.kafka.connect.datagen.DatagenConnector", + "kafka.topic": "users", + "quickstart": "users", + "key.converter": "org.apache.kafka.connect.storage.StringConverter", + "value.converter": "org.apache.kafka.connect.json.JsonConverter", + "value.converter.schemas.enable": "false", + "max.interval": 1000, + "iterations": 10000000, + "tasks.max": "1" + } +} \ No newline at end of file diff --git a/environment/docker-compose-with-connect.yml b/environment/docker-compose-with-connect.yml index c65dad8d..3a4489ca 100644 --- a/environment/docker-compose-with-connect.yml +++ b/environment/docker-compose-with-connect.yml @@ -2,7 +2,7 @@ version: '2' services: zookeeper: - image: confluentinc/cp-zookeeper:7.4.0 + image: confluentinc/cp-zookeeper:7.6.1 hostname: zookeeper container_name: zookeeper ports: @@ -13,7 +13,7 @@ services: KAFKA_OPTS: "-Dzookeeper.4lw.commands.whitelist=*" broker: - image: confluentinc/cp-server:7.4.0 + image: confluentinc/cp-server:7.6.1 hostname: broker container_name: broker depends_on: @@ -42,7 +42,7 @@ services: CONFLUENT_SUPPORT_CUSTOMER_ID: 'anonymous' schema-registry: - image: confluentinc/cp-schema-registry:7.4.0 + image: confluentinc/cp-schema-registry:7.6.1 hostname: schema-registry container_name: schema-registry depends_on: @@ -55,7 +55,7 @@ services: SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:8081 connect: - image: cnfldemos/kafka-connect-datagen:0.6.0-7.3.0 + image: cnfldemos/kafka-connect-datagen:0.6.4-7.6.0 container_name: connect depends_on: - broker diff --git a/samples/sample-stream/Program.cs b/samples/sample-stream/Program.cs index 3238a83a..b1af4a78 100644 --- a/samples/sample-stream/Program.cs +++ b/samples/sample-stream/Program.cs @@ -14,7 +14,7 @@ namespace sample_stream { public static class Program { - public static async Task Main(string[] args) + public static async Task Main2(string[] args) { var config = new StreamConfig{ ApplicationId = $"test-app", diff --git a/samples/sample-stream/Reproducer314.cs b/samples/sample-stream/Reproducer314.cs new file mode 100644 index 00000000..44230759 --- /dev/null +++ b/samples/sample-stream/Reproducer314.cs @@ -0,0 +1,75 @@ +using System; +using System.Threading; +using System.Threading.Tasks; +using Confluent.Kafka; +using Streamiz.Kafka.Net; +using Streamiz.Kafka.Net.SerDes; +using Streamiz.Kafka.Net.State; +using Streamiz.Kafka.Net.Stream; +using Streamiz.Kafka.Net.Table; + +namespace sample_stream; + +public class Reproducer314 +{ + public static async Task Main(string[] args) + { + Console.WriteLine("Hello Streams"); + + var config = new StreamConfig + { + ApplicationId = $"test-windowedtable-bis", + BootstrapServers = "localhost:9092", + AutoOffsetReset = AutoOffsetReset.Earliest + }; + + var builder = CreateWindowedStore(); + var t = builder.Build(); + var windowedTableStream = new KafkaStream(t, config); + + await windowedTableStream.StartAsync(); + + // wait for the store to be restored and ready + Thread.Sleep(10000); + + GetValueFromWindowedStore(windowedTableStream, DateTime.UtcNow.AddHours(-1), new CancellationToken()); + + Console.WriteLine("Finished"); + } + + private static void GetValueFromWindowedStore(KafkaStream windowedTableStream, DateTime startUtcForWindowLookup, CancellationToken cancellationToken) + { + var windowedStore = windowedTableStream.Store(StoreQueryParameters.FromNameAndType("store", QueryableStoreTypes.WindowStore())); + + while (!cancellationToken.IsCancellationRequested) + { + var records = windowedStore.FetchAll(startUtcForWindowLookup, DateTime.UtcNow).ToList(); + + if (records.Count > 0) + { + foreach (var item in records) + { + Console.WriteLine($"Value from windowed store : KEY = {item.Key} VALUE = {item.Value}"); + } + + startUtcForWindowLookup = DateTime.UtcNow; + } + } + } + + private static StreamBuilder CreateWindowedStore() + { + var builder = new StreamBuilder(); + + builder + .Stream("users") + .GroupByKey() + .WindowedBy(TumblingWindowOptions.Of(60000)) + .Aggregate( + () => 0, + (k, v, agg) => Math.Max(v.Length, agg), + InMemoryWindows.As("store").WithValueSerdes()); + + return builder; + } +} \ No newline at end of file From 52bca511a9bc580dfa1d76425e0c9e613db30d3f Mon Sep 17 00:00:00 2001 From: LGouellec Date: Wed, 10 Jul 2024 15:16:26 -0700 Subject: [PATCH 21/29] Add unit tests --- core/State/InMemory/InMemoryWindowStore.cs | 5 +- .../InMemory/Internal/ConcurrentHashSet.cs | 183 +----------------- .../Private/ConcurrentSetTests.cs | 93 +++++++++ 3 files changed, 101 insertions(+), 180 deletions(-) create mode 100644 test/Streamiz.Kafka.Net.Tests/Private/ConcurrentSetTests.cs diff --git a/core/State/InMemory/InMemoryWindowStore.cs b/core/State/InMemory/InMemoryWindowStore.cs index 7ab51a39..fb981db5 100644 --- a/core/State/InMemory/InMemoryWindowStore.cs +++ b/core/State/InMemory/InMemoryWindowStore.cs @@ -310,8 +310,9 @@ public void Close() if (openIterators.Count != 0) { logger.LogWarning("Closing {OpenIteratorCount} open iterators for store {Name}", openIterators.Count, Name); - for (int i = 0; i< openIterators.Count; ++i) - openIterators.ElementAt(i).Close(); + foreach(var iterator in openIterators) + iterator.Close(); + openIterators.Clear(); } map.Clear(); diff --git a/core/State/InMemory/Internal/ConcurrentHashSet.cs b/core/State/InMemory/Internal/ConcurrentHashSet.cs index 57ed5895..e898b396 100644 --- a/core/State/InMemory/Internal/ConcurrentHashSet.cs +++ b/core/State/InMemory/Internal/ConcurrentHashSet.cs @@ -1,12 +1,9 @@ -using System; -using System.Collections; using System.Collections.Concurrent; using System.Collections.Generic; -using System.Linq; namespace Streamiz.Kafka.Net.State.InMemory.Internal { - internal class ConcurrentSet : IEnumerable, ISet, ICollection + internal class ConcurrentSet { private readonly ConcurrentDictionary _dictionary = new(); @@ -21,17 +18,6 @@ public IEnumerator GetEnumerator() return _dictionary.Keys.GetEnumerator(); } - /// - /// Returns an enumerator that iterates through a collection. - /// - /// - /// An object that can be used to iterate through the collection. - /// - IEnumerator IEnumerable.GetEnumerator() - { - return GetEnumerator(); - } - /// /// Removes the first occurrence of a specific object from the . /// @@ -49,150 +35,6 @@ public bool Remove(T item) /// public int Count => _dictionary.Count; - /// - /// Gets a value indicating whether the is read-only. - /// - /// - /// true if the is read-only; otherwise, false. - /// - public bool IsReadOnly => false; - - /// - /// Gets a value that indicates if the set is empty. - /// - public bool IsEmpty => _dictionary.IsEmpty; - - private ICollection Values => _dictionary.Keys; - - /// - /// Adds an item to the . - /// - /// The object to add to the .The is read-only. - void ICollection.Add(T item) - { - if (!Add(item)) - throw new ArgumentException("Item already exists in set."); - } - - /// - /// Modifies the current set so that it contains all elements that are present in both the current set and in the specified collection. - /// - /// The collection to compare to the current set. is null. - public void UnionWith(IEnumerable other) - { - foreach (var item in other) - TryAdd(item); - } - - /// - /// Modifies the current set so that it contains only elements that are also in a specified collection. - /// - /// The collection to compare to the current set. is null. - public void IntersectWith(IEnumerable other) - { - var enumerable = other as IList ?? other.ToArray(); - foreach (var item in this) - { - if (!enumerable.Contains(item)) - TryRemove(item); - } - } - - /// - /// Removes all elements in the specified collection from the current set. - /// - /// The collection of items to remove from the set. is null. - public void ExceptWith(IEnumerable other) - { - foreach (var item in other) - TryRemove(item); - } - - /// - /// Modifies the current set so that it contains only elements that are present either in the current set or in the specified collection, but not both. - /// - /// The collection to compare to the current set. is null. - public void SymmetricExceptWith(IEnumerable other) - { - throw new NotImplementedException(); - } - - /// - /// Determines whether a set is a subset of a specified collection. - /// - /// - /// true if the current set is a subset of ; otherwise, false. - /// - /// The collection to compare to the current set. is null. - public bool IsSubsetOf(IEnumerable other) - { - var enumerable = other as IList ?? other.ToArray(); - return this.AsParallel().All(enumerable.Contains); - } - - /// - /// Determines whether the current set is a superset of a specified collection. - /// - /// - /// true if the current set is a superset of ; otherwise, false. - /// - /// The collection to compare to the current set. is null. - public bool IsSupersetOf(IEnumerable other) - { - return other.AsParallel().All(Contains); - } - - /// - /// Determines whether the current set is a correct superset of a specified collection. - /// - /// - /// true if the object is a correct superset of ; otherwise, false. - /// - /// The collection to compare to the current set. is null. - public bool IsProperSupersetOf(IEnumerable other) - { - var enumerable = other as IList ?? other.ToArray(); - return this.Count != enumerable.Count && IsSupersetOf(enumerable); - } - - /// - /// Determines whether the current set is a property (strict) subset of a specified collection. - /// - /// - /// true if the current set is a correct subset of ; otherwise, false. - /// - /// The collection to compare to the current set. is null. - public bool IsProperSubsetOf(IEnumerable other) - { - var enumerable = other as IList ?? other.ToArray(); - return Count != enumerable.Count && IsSubsetOf(enumerable); - } - - /// - /// Determines whether the current set overlaps with the specified collection. - /// - /// - /// true if the current set and share at least one common element; otherwise, false. - /// - /// The collection to compare to the current set. is null. - public bool Overlaps(IEnumerable other) - { - return other.AsParallel().Any(Contains); - } - - /// - /// Determines whether the current set and the specified collection contain the same elements. - /// - /// - /// true if the current set is equal to ; otherwise, false. - /// - /// The collection to compare to the current set. is null. - public bool SetEquals(IEnumerable other) - { - var enumerable = other as IList ?? other.ToArray(); - return Count == enumerable.Count && enumerable.AsParallel().All(Contains); - } - /// /// Adds an element to the current set and returns a value to indicate if the element was successfully added. /// @@ -215,29 +57,14 @@ public bool Contains(T item) return _dictionary.ContainsKey(item); } - /// - /// Copies the elements of the to an , starting at a particular index. - /// - /// The one-dimensional that is the destination of the elements copied from . The must have zero-based indexing.The zero-based index in at which copying begins. is null. is less than 0. is multidimensional.-or-The number of elements in the source is greater than the available space from to the end of the destination .-or-Type cannot be cast automatically to the type of the destination . - public void CopyTo(T[] array, int arrayIndex) - { - Values.CopyTo(array, arrayIndex); - } - - public T[] ToArray() - { - return _dictionary.Keys.ToArray(); - } - - public bool TryAdd(T item) + private bool TryAdd(T item) { - return _dictionary.TryAdd(item, default(byte)); + return _dictionary.TryAdd(item, default); } - public bool TryRemove(T item) + private bool TryRemove(T item) { - byte donotcare; - return _dictionary.TryRemove(item, out donotcare); + return _dictionary.TryRemove(item, out _); } } diff --git a/test/Streamiz.Kafka.Net.Tests/Private/ConcurrentSetTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/ConcurrentSetTests.cs new file mode 100644 index 00000000..d76168a0 --- /dev/null +++ b/test/Streamiz.Kafka.Net.Tests/Private/ConcurrentSetTests.cs @@ -0,0 +1,93 @@ +using System; +using System.Collections.Generic; +using System.Threading.Tasks; +using NUnit.Framework; +using Streamiz.Kafka.Net.State.InMemory.Internal; + +namespace Streamiz.Kafka.Net.Tests.Private; + +public class ConcurrentSetTests +{ + private ConcurrentSet concurrentSet; + + [SetUp] + public void Init() + { + concurrentSet = new(); + } + + [TearDown] + public void Dispose() + { + concurrentSet.Clear(); + } + + [TestCase(1000)] + public void ConcurrencyAdded(int numberTasks) + { + var taskList = new List(); + for (int i = 0; i < numberTasks; i++) + { + taskList.Add(Task.Factory.StartNew((Object obj) => + { + concurrentSet.Add(Guid.NewGuid().ToString()); + }, null)); + } + Task.WaitAll(taskList.ToArray()); + Assert.AreEqual(numberTasks, concurrentSet.Count); + } + + [TestCase(1000)] + public void ConcurrencyRemoved(int numberTasks) + { + for (int i = 0; i < numberTasks; i++) + concurrentSet.Add(i.ToString()); + + var taskList = new List(); + for (int i = 0; i < numberTasks; i++) + { + taskList.Add(Task.Factory.StartNew((Object obj) => + { + concurrentSet.Remove(obj.ToString()); + }, i)); + } + + Task.WaitAll(taskList.ToArray()); + Assert.AreEqual(0, concurrentSet.Count); + } + + [TestCase(10000)] + public void ConcurrencyAddedAndForeach(int numberTasks) + { + var taskList = new List(); + for (int i = 0; i < numberTasks; i++) + { + taskList.Add(Task.Factory.StartNew((Object obj) => + { + concurrentSet.Add(Guid.NewGuid().ToString()); + foreach (var c in concurrentSet) + ; + }, null)); + } + Task.WaitAll(taskList.ToArray()); + Assert.AreEqual(numberTasks, concurrentSet.Count); + } + + [TestCase(10000)] + public void ConcurrencyAddedAndContains(int numberTasks) + { + var taskList = new List(); + for (int i = 0; i < numberTasks; i++) + { + taskList.Add(Task.Factory.StartNew((Object obj) => + { + var guid = Guid.NewGuid().ToString(); + concurrentSet.Add(guid); + Assert.IsTrue(concurrentSet.Contains(guid)); + }, null)); + } + Task.WaitAll(taskList.ToArray()); + Assert.AreEqual(numberTasks, concurrentSet.Count); + } + +} \ No newline at end of file From da9b9925e00918272ec7316aca570ccf8f832da5 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Fri, 12 Jul 2024 13:52:14 -0700 Subject: [PATCH 22/29] Fix cache concurrent issue --- core/Crosscutting/Bytes.cs | 4 + .../SortedDictionaryExtensions.cs | 33 +- core/Kafka/Internal/RecordCollector.cs | 7 +- .../Enumerator/AbstractMergedEnumerator.cs | 6 +- .../State/Cache/Enumerator/CacheEnumerator.cs | 2 +- core/State/Cache/Internal/CacheEntry.cs | 2 +- .../Cache/Internal/CacheEntryExtensions.cs | 12 +- .../Internal/ConcurrentSortedDictionary.cs | 2649 +++++++++++++++++ core/State/Cache/Internal/ICacheEntry.cs | 2 +- core/State/Cache/Internal/IMemoryCache.cs | 2 +- core/State/Cache/Internal/MemoryCache.cs | 58 +- .../Internal/MemoryCacheEntryExtensions.cs | 4 +- .../Cache/Internal/MemoryCacheEntryOptions.cs | 2 +- .../Cache/Internal/MemoryCacheExtensions.cs | 12 +- .../PostEvictionCallbackRegistration.cs | 4 +- .../Cache/Internal/PostEvictionDelegate.cs | 4 +- environment/datagen_connector.json | 2 +- samples/sample-stream/Reproducer314.cs | 42 +- .../ConcurrentSortedDictionaryTests.cs | 23 + 19 files changed, 2789 insertions(+), 81 deletions(-) create mode 100644 core/State/Cache/Internal/ConcurrentSortedDictionary.cs create mode 100644 test/Streamiz.Kafka.Net.Tests/Private/ConcurrentSortedDictionaryTests.cs diff --git a/core/Crosscutting/Bytes.cs b/core/Crosscutting/Bytes.cs index 6e0c1f8a..22f7a133 100644 --- a/core/Crosscutting/Bytes.cs +++ b/core/Crosscutting/Bytes.cs @@ -127,6 +127,10 @@ internal static Bytes Wrap(byte[] bytes) public virtual int CompareTo(Bytes other) { BytesComparer comparer = new BytesComparer(); + if (other == null || other.Get == null) + return 1; + if (Get == null) + return -1; return comparer.Compare(this, other); } } diff --git a/core/Crosscutting/SortedDictionaryExtensions.cs b/core/Crosscutting/SortedDictionaryExtensions.cs index d946f324..9f4cae2a 100644 --- a/core/Crosscutting/SortedDictionaryExtensions.cs +++ b/core/Crosscutting/SortedDictionaryExtensions.cs @@ -1,32 +1,36 @@ -using System.Collections.Generic; +using System; +using System.Collections.Generic; namespace Streamiz.Kafka.Net.Crosscutting { internal static class SortedDictionaryExtensions { - internal static IEnumerable> HeadMap(this SortedDictionary sortedDic, K key, bool inclusive) + internal static IEnumerable> HeadMap(this IEnumerable> enumerable, K key, bool inclusive) + where K : IComparable { - foreach (K k in sortedDic.Keys) { - int r = sortedDic.Comparer.Compare(key, k); + foreach (var kv in enumerable) + { + int r = key.CompareTo(kv.Key); if ((inclusive && r >= 0) || (!inclusive && r > 0)) - yield return new KeyValuePair(k, sortedDic[k]); + yield return new KeyValuePair(kv.Key, kv.Value); else break; } } - internal static IEnumerable> SubMap(this SortedDictionary sortedDic, K keyFrom, K keyTo , bool inclusiveFrom, bool inclusiveTo) + internal static IEnumerable> SubMap(this IEnumerable> enumerable, K keyFrom, K keyTo , bool inclusiveFrom, bool inclusiveTo) + where K : IComparable { - foreach (K k in sortedDic.Keys) + foreach (var kv in enumerable) { - int rF = sortedDic.Comparer.Compare(keyFrom, k); - int rT = sortedDic.Comparer.Compare(keyTo, k); + int rF = keyFrom.CompareTo(kv.Key); + int rT = keyTo.CompareTo(kv.Key); if((inclusiveFrom && rF <= 0) || (!inclusiveFrom && rF < 0)) { if ((inclusiveTo && rT >= 0) || (!inclusiveTo && rT > 0)) { - yield return new KeyValuePair(k, sortedDic[k]); + yield return new KeyValuePair(kv.Key, kv.Value); } else break; @@ -34,16 +38,17 @@ internal static IEnumerable> SubMap(this SortedDictiona } } - internal static IEnumerable> TailMap(this SortedDictionary sortedDic, K keyFrom, + internal static IEnumerable> TailMap(this IEnumerable> enumerable, K keyFrom, bool inclusive) + where K : IComparable { - foreach (K k in sortedDic.Keys) + foreach (var kv in enumerable) { - int rT = sortedDic.Comparer.Compare(keyFrom, k); + int rT = keyFrom.CompareTo(kv.Key); if ((inclusive && rT <= 0) || (!inclusive && rT < 0)) { - yield return new KeyValuePair(k, sortedDic[k]); + yield return new KeyValuePair(kv.Key, kv.Value); } } } diff --git a/core/Kafka/Internal/RecordCollector.cs b/core/Kafka/Internal/RecordCollector.cs index a05d320c..205cf110 100644 --- a/core/Kafka/Internal/RecordCollector.cs +++ b/core/Kafka/Internal/RecordCollector.cs @@ -325,7 +325,12 @@ private void HandleError(DeliveryReport report) log.LogDebug( "{LogPrefix}Record persisted: (timestamp {Timestamp}) topic=[{Topic}] partition=[{Partition}] offset=[{Offset}]", logPrefix, report.Message.Timestamp.UnixTimestampMs, report.Topic, report.Partition, report.Offset); - collectorsOffsets.AddOrUpdate(report.TopicPartition, report.Offset.Value); + if (collectorsOffsets.ContainsKey(report.TopicPartition) && + collectorsOffsets[report.TopicPartition] < report.Offset.Value) + collectorsOffsets.TryUpdate(report.TopicPartition, report.Offset.Value, + collectorsOffsets[report.TopicPartition]); + else + collectorsOffsets.TryAdd(report.TopicPartition, report.Offset); retryRecordContext.AckRecord(report); } } diff --git a/core/State/Cache/Enumerator/AbstractMergedEnumerator.cs b/core/State/Cache/Enumerator/AbstractMergedEnumerator.cs index bc08a472..4ef02927 100644 --- a/core/State/Cache/Enumerator/AbstractMergedEnumerator.cs +++ b/core/State/Cache/Enumerator/AbstractMergedEnumerator.cs @@ -32,7 +32,7 @@ protected AbstractMergedEnumerator( } private bool IsDeletedCacheEntry(KeyValuePair? nextFromCache) - => nextFromCache?.Value.Value == null; + => nextFromCache?.Value?.Value == null; public K PeekNextKey() => Current.Value.Key; @@ -45,7 +45,9 @@ public bool MoveNext() if (_lastChoice is LastChoice.NONE or LastChoice.CACHE or LastChoice.BOTH) { // skip over items deleted from cache, and corresponding store items if they have the same key - while (cacheEnumerator.MoveNext() && IsDeletedCacheEntry(cacheEnumerator.Current)) + while (cacheEnumerator.MoveNext() + && cacheEnumerator.Current != null + && IsDeletedCacheEntry(cacheEnumerator.Current)) { var currentKeyStore = storeEnumerator.Current; // advance the store enumerator if the key is the same as the deleted cache key diff --git a/core/State/Cache/Enumerator/CacheEnumerator.cs b/core/State/Cache/Enumerator/CacheEnumerator.cs index f916de4f..25884d93 100644 --- a/core/State/Cache/Enumerator/CacheEnumerator.cs +++ b/core/State/Cache/Enumerator/CacheEnumerator.cs @@ -7,7 +7,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Enumerator { internal class CacheEnumerator : IKeyValueEnumerator - where K : class + where K : class, IComparable where V : class { private IEnumerator keys; diff --git a/core/State/Cache/Internal/CacheEntry.cs b/core/State/Cache/Internal/CacheEntry.cs index d06ff742..523e554b 100644 --- a/core/State/Cache/Internal/CacheEntry.cs +++ b/core/State/Cache/Internal/CacheEntry.cs @@ -11,7 +11,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal { internal sealed partial class CacheEntry : ICacheEntry - where K : class + where K : IComparable where V : class { //private static readonly Action ExpirationCallback = ExpirationTokensExpired; diff --git a/core/State/Cache/Internal/CacheEntryExtensions.cs b/core/State/Cache/Internal/CacheEntryExtensions.cs index e609039f..94b5138e 100644 --- a/core/State/Cache/Internal/CacheEntryExtensions.cs +++ b/core/State/Cache/Internal/CacheEntryExtensions.cs @@ -21,7 +21,7 @@ internal static class CacheEntryExtensions internal static ICacheEntry RegisterPostEvictionCallback( this ICacheEntry entry, PostEvictionDelegate callback) - where K : class + where K : IComparable where V : class { return entry.RegisterPostEvictionCallbackNoValidation(callback, state: null); @@ -38,7 +38,7 @@ public static ICacheEntry RegisterPostEvictionCallback( this ICacheEntry entry, PostEvictionDelegate callback, MemoryCache? state) - where K : class + where K : IComparable where V : class { return entry.RegisterPostEvictionCallbackNoValidation(callback, state); @@ -48,7 +48,7 @@ private static ICacheEntry RegisterPostEvictionCallbackNoValidation( this ICacheEntry entry, PostEvictionDelegate callback, MemoryCache? state) - where K : class + where K : IComparable where V : class { entry.PostEvictionCallbacks.Add(new PostEvictionCallbackRegistration() @@ -68,7 +68,7 @@ private static ICacheEntry RegisterPostEvictionCallbackNoValidation( public static ICacheEntry SetValue( this ICacheEntry entry, V value) - where K : class + where K : IComparable where V : class { entry.Value = value; @@ -84,7 +84,7 @@ public static ICacheEntry SetValue( public static ICacheEntry SetSize( this ICacheEntry entry, long size) - where K : class + where K : IComparable where V : class { if (size < 0) @@ -103,7 +103,7 @@ public static ICacheEntry SetSize( /// Set the values of these options on the . /// The for chaining. public static ICacheEntry SetOptions(this ICacheEntry entry, MemoryCacheEntryOptions options) - where K : class + where K : IComparable where V : class { entry.Size = options.Size; diff --git a/core/State/Cache/Internal/ConcurrentSortedDictionary.cs b/core/State/Cache/Internal/ConcurrentSortedDictionary.cs new file mode 100644 index 00000000..66713e86 --- /dev/null +++ b/core/State/Cache/Internal/ConcurrentSortedDictionary.cs @@ -0,0 +1,2649 @@ +using System; +using System.Collections; +using System.Collections.Generic; +using System.Linq; +using System.Threading; + +namespace Streamiz.Kafka.Net.State.Cache.Internal +{ /* +MIT License + +Copyright (c) 2023 Matthew Krebser (https://github.com/mkrebser) + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. +*/ + +// Used for more nuanced lock testing and sanity test +// #define ConcurrentSortedDictionary_DEBUG + + public enum InsertResult + { + /// + /// Operation completed successfully. + /// + success = 0, + + /// + /// Value was not inserted because it already exists. + /// + alreadyExists = 1, + + /// + /// Value ws not inserted due to timeout. + /// + timedOut = 2 + } + + public enum RemoveResult + { + /// + /// Successfully deleted. + /// + success = 0, + + /// + /// key was not found. No deletion occured. + /// + notFound = 1, + + /// + /// Value ws not deleted due to timeout. + /// + timedOut = 2, + } + + public enum SearchResult + { + /// + /// Successfully found key. + /// + success = 0, + + /// + /// key was not found. + /// + notFound = 1, + + /// + /// Couldn't complete search due to timeout. + /// + timedOut = 2, + } + +// not using the Nullable notation (eg myType? ) because it adds a small overhead to everything +// eg, Nullable[] vs int[] -> Nullable will take up double the memory on many x64 systems due to extra boolean flag +#nullable disable + +// Locking Scheme: Multiple Readers & Single Writer +// However, locks are at a granularity of each node- so multiple parts of the tree can be written concurrently +// +// Read Access Scheme: While traversing down the tree hierarchy, reads will aquire the lock of the next +// node they move to before releasing the lock of their current node. This guarentees +// that no writers will skip ahead of any readers. It also guarentees readers will not +// skip ahead of any writers traversing down. +// +// Write Access Scheme: Writers use 'latching'. While traversing downwards.. They will only release a parent node +// if an insert/delete will not cause a spit or merge. (This means that the insert/delete won't +// need to traverse up the B+Tree) +// +// Locks: All locks used are ReaderWriterSlim locks which will prioritize writers +// https://learn.microsoft.com/en-us/dotnet/api/system.threading.readerwriterlockslim?view=net-7.0 + +// An excellent lecture on B+Trees: +// https://courses.cs.washington.edu/courses/cse332/20au/lectures/cse332-20au-lec09-BTrees.pdf + +// slides that go over latching (latch-crabbing) https://15721.courses.cs.cmu.edu/spring2017/slides/06-latching.pdf + + /// + /// Implementation of a concurrent B+Tree. https://en.wikipedia.org/wiki/B+tree# + /// + public partial class ConcurrentSortedDictionary : IEnumerable> + where Key : IComparable + { + private volatile ConcurrentKTreeNode _root; + + public void setRoot(object o) + { + this._root = (ConcurrentKTreeNode)o; + } + + private readonly ReaderWriterLockSlim _rootLock; + private long _count; + + /// + /// Number of key-value pairs in the collection. Value may be stale in concurrent access. + /// + public long Count + { + get { return _count; } + } + + /// + /// Is collection empty? Value may be stale in concurrent access. + /// + public bool IsEmpty + { + get { return this.Count <= 0; } + } + + private volatile int _depth; + + /// + /// Approximate depth of the search tree. Value may be stale in concurrent access. + /// + public int Depth + { + get { return _depth + 1; } + } // A tree with only root node has _depth = 0 + + /// + /// Width of each node in the tree. + /// + public readonly int k; + + /// + /// Create a new instance of ConcurrentSortedDictionary + /// + /// Number of children per node. + public ConcurrentSortedDictionary(int k = 32) + { + if (k < 3) // Don't allow '2', it creates potentially many leafs with only 1 item due to b+ tree requirements + throw new ArgumentException("Invalid k specified"); + _rootLock = new ReaderWriterLockSlim(LockRecursionPolicy.NoRecursion); + _root = new ConcurrentKTreeNode(k, isLeaf: true); + _count = 0; + _depth = 0; + this.k = k; + } + + public ConcurrentSortedDictionary(ICollection> collection, int k = 32) : this(k) + { + if (ReferenceEquals(null, collection)) + throw new ArgumentException("Cannot input null collection"); + foreach (var pair in collection) + AddOrUpdate(pair.Key, pair.Value); + } + + public ConcurrentSortedDictionary(ICollection> collection, int k = 32) : this(k) + { + if (ReferenceEquals(null, collection)) + throw new ArgumentException("Cannot input null collection"); + foreach (var pair in collection) + AddOrUpdate(pair.Item1, pair.Item2); + } + + public ConcurrentSortedDictionary(ICollection> collection, int k = 32) : this(k) + { + if (ReferenceEquals(null, collection)) + throw new ArgumentException("Cannot input null collection"); + foreach (var pair in collection) + AddOrUpdate(pair.Item1, pair.Item2); + } + + public ConcurrentSortedDictionary(ICollection keys, ICollection values, int k = 32) : this(k) + { + if (ReferenceEquals(null, keys) || ReferenceEquals(null, values)) + throw new ArgumentException("Cannot input null collection"); + foreach (var pair in keys.Zip(values, (key, value) => (key, value))) + AddOrUpdate(pair.Item1, pair.Item2); + } + + void assertTimeoutArg(in int timeoutMs) + { + if (timeoutMs < 0) + throw new ArgumentException("Timeout cannot be negative!"); + } + + InsertResult ToInsertResult(in ConcurrentTreeResult_Extended result) + { + if (result == ConcurrentTreeResult_Extended.success) + { + return InsertResult.success; + } + else if (result == ConcurrentTreeResult_Extended.alreadyExists) + { + return InsertResult.alreadyExists; + } + else + { + return InsertResult.timedOut; + } + } + + RemoveResult ToRemoveResult(in ConcurrentTreeResult_Extended result) + { + if (result == ConcurrentTreeResult_Extended.success) + { + return RemoveResult.success; + } + else if (result == ConcurrentTreeResult_Extended.notFound) + { + return RemoveResult.notFound; + } + else + { + return RemoveResult.timedOut; + } + } + + /// + /// Insert a Key-Value pair and overwrite if it already exists. + /// + public InsertResult AddOrUpdate( + in Key key, + in Value value, + in int timeoutMs + ) + { + assertTimeoutArg(timeoutMs); + Value retrievedValue; + return ToInsertResult(writeToTree(in key, in value, in timeoutMs, LatchAccessType.insert, + out retrievedValue, true)); + } + + /// + /// Insert a Key-Value pair and overwrite if it already exists. Waits forever for mutex. + /// + public void AddOrUpdate( + in Key key, + in Value value + ) + { + Value retrievedValue; + writeToTree(in key, in value, -1, LatchAccessType.insert, out retrievedValue, true); + } + + /// + /// Insert a Key-Value pair or return the existing pair if key already exists. + /// + public InsertResult TryAdd( + in Key key, + in Value value, + in int timeoutMs + ) + { + assertTimeoutArg(timeoutMs); + Value retrievedValue; + return ToInsertResult(writeToTree(in key, in value, in timeoutMs, LatchAccessType.insertTest, + out retrievedValue, false)); + } + + /// + /// Insert a Key-Value pair. Return false if not inserted due to existing value. + /// + public bool TryAdd( + in Key key, + in Value value + ) + { + Value retrievedValue; + return ToInsertResult(writeToTree(in key, in value, -1, LatchAccessType.insertTest, out retrievedValue, + false)) == InsertResult.success; + } + + /// + /// Insert a Key-Value pair or output the existing pair if key already exists. + /// + public InsertResult GetOrAdd( + in Key key, + in Value value, + in int timeoutMs, + out Value retrievedValue + ) + { + assertTimeoutArg(timeoutMs); + return ToInsertResult(writeToTree(in key, in value, in timeoutMs, LatchAccessType.insertTest, + out retrievedValue, false)); + } + + /// + /// Insert a Key-Value pair if it doesn't exist and return the value. If it does exist, the existing value is returned. + /// + public Value GetOrAdd( + in Key key, + in Value value + ) + { + Value retrievedValue; + writeToTree(in key, in value, -1, LatchAccessType.insertTest, out retrievedValue, false); + return retrievedValue; + } + + void tryUpdateDepth( + int newSearchDepth, + ref Latch latch, + ref LockBuffer lockBuffer + ) where LockBuffer : ILockBuffer + { + if (newSearchDepth >= 30) + { + latch.ExitLatchChain(ref lockBuffer); + throw new ArgumentException("Reached 31 tree limit depth. Only a max of " + + (int)Math.Pow(this.k, 31) + + " items is supported. Increasing 'k' will increase limit."); + } + + this._depth = newSearchDepth; + } + + /// + /// Perform a insert or delete on the tree depending on the LatchAccessType. + /// + private ConcurrentTreeResult_Extended writeToTree( + in Key key, + in Value value, + in int timeoutMs, + in LatchAccessType accessType, + out Value retrievedValue, + in bool overwrite = false + ) + { + if (!typeof(Key).IsValueType && ReferenceEquals(null, key)) + { + throw new ArgumentException("Cannot have null key"); + } + + SearchResultInfo info = default(SearchResultInfo); + Value currentValue; + // Optmistic latching + var rwLatch = new Latch(accessType, this._rootLock, assumeLeafIsSafe: true); + var rwLockBuffer = new LockBuffer2(); + long startTime = DateTimeOffset.Now.ToUnixTimeMilliseconds(); + int remainingMs = getRemainingMs(in startTime, in timeoutMs); + var searchOptions = new SearchOptions(remainingMs, startTime: startTime); + // Perform a query to recurse to the deepest node, latching downwards optimistically + var getResult = ConcurrentKTreeNode.TryGetValue(in key, out currentValue, + ref info, ref rwLatch, ref rwLockBuffer, this, searchOptions); + + // Timeout! + if (getResult == ConcurrentTreeResult_Extended.timedOut) + { + retrievedValue = default(Value); + return ConcurrentTreeResult_Extended.timedOut; + } + + // If this is a test-before-write- then determine if the test failed + bool exitOnTest = false; + if (accessType == LatchAccessType.insertTest) + { + exitOnTest = info.index > -1; // if it is found.. it must have a positive index + } + else if (accessType == LatchAccessType.deleteTest) + { + exitOnTest = info.index < 0; // if it is not found, it will have a negative index + } + + try + { + // If we were able to optimistally acquire latch... (or test op was successful) + // The write to tree + if (getResult != ConcurrentTreeResult_Extended.notSafeToUpdateLeaf || exitOnTest) + { +#if ConcurrentSortedDictionary_DEBUG + info.node.assertWriterLockHeld(); +#endif + + if (rwLatch.isInsertAccess) + { + tryUpdateDepth(info.depth, ref rwLatch, ref rwLockBuffer); + return writeInsertion(in key, in value, in info, in getResult, in overwrite, out retrievedValue, + exitOnTest, ref rwLockBuffer, ref rwLatch); + } + else + { + retrievedValue = default(Value); + return writeDeletion(in key, in info, in getResult, exitOnTest, ref rwLockBuffer, ref rwLatch); + } + } + else + { + rwLatch.ExitLatchChain(ref rwLockBuffer); +#if ConcurrentSortedDictionary_DEBUG + Test.Assert(!rwLatch.HoldingRootLock); + Test.Assert(ReferenceEquals(null, rwLockBuffer.peek())); +#endif + } + } + finally + { +#if ConcurrentSortedDictionary_DEBUG + Test.Assert(rwLockBuffer.peek() == null); + Test.Assert(!rwLatch.HoldingRootLock); +#endif + } + + // Otherwise, try to acquire write access using a full write latch chain + // Note* forcing test to false + var writeLatchAccessType = accessType == LatchAccessType.insertTest ? LatchAccessType.insert : accessType; + writeLatchAccessType = accessType == LatchAccessType.deleteTest ? LatchAccessType.delete : accessType; + var writeLatch = new Latch(writeLatchAccessType, this._rootLock, assumeLeafIsSafe: false); + var writeLockBuffer = new LockBuffer32(); + remainingMs = getRemainingMs(in startTime, in timeoutMs); + searchOptions = new SearchOptions(remainingMs, startTime: startTime); + getResult = ConcurrentKTreeNode.TryGetValue(in key, out currentValue, + ref info, ref writeLatch, ref writeLockBuffer, this, searchOptions); + + // Check if timed out... + if (getResult == ConcurrentTreeResult_Extended.timedOut) + { + retrievedValue = default(Value); + return ConcurrentTreeResult_Extended.timedOut; + } + + try + { +#if ConcurrentSortedDictionary_DEBUG + info.node.assertWriterLockHeld(); +#endif + + if (writeLatch.isInsertAccess) + { + tryUpdateDepth(info.depth, ref writeLatch, ref writeLockBuffer); + return writeInsertion(in key, in value, in info, in getResult, in overwrite, out retrievedValue, + false, ref writeLockBuffer, ref writeLatch); + } + else + { + retrievedValue = default(Value); + return writeDeletion(in key, in info, in getResult, false, ref writeLockBuffer, ref writeLatch); + } + } + finally + { +#if ConcurrentSortedDictionary_DEBUG + Test.Assert(writeLockBuffer.peek() == null); + Test.Assert(!writeLatch.HoldingRootLock); +#endif + } + } + + private ConcurrentTreeResult_Extended writeInsertion( + in Key key, + in Value value, + in SearchResultInfo info, + in ConcurrentTreeResult_Extended getResult, + in bool overwrite, + out Value retrievedValue, + in bool exitOnTest, // if exitOnTest is true, then this function should never perform a write + ref LockBuffer lockBuffer, + ref Latch latch + ) where LockBuffer : ILockBuffer + { + // If the vaue already exists... + if (getResult == ConcurrentTreeResult_Extended.success || exitOnTest) + { + if (overwrite && !exitOnTest) + { + info.node.SetValue(info.index, in key, in value); + retrievedValue = value; + + latch.ExitLatchChain(ref lockBuffer); +#if ConcurrentSortedDictionary_DEBUG + Test.Assert(!latch.HoldingRootLock); + Test.Assert(ReferenceEquals(null, lockBuffer.peek())); +#endif + + return ConcurrentTreeResult_Extended.success; + } + + retrievedValue = info.node.GetValue(info.index).value; + + latch.ExitLatchChain(ref lockBuffer); +#if ConcurrentSortedDictionary_DEBUG + Test.Assert(!latch.HoldingRootLock); + Test.Assert(ReferenceEquals(null, lockBuffer.peek())); +#endif + + return ConcurrentTreeResult_Extended.alreadyExists; + } + + info.node.sync_InsertAtThisNode(in key, in value, this, ref lockBuffer, ref latch); + // NOTE* tree is unlocked after sync_Insert + Interlocked.Increment(ref this._count); // increase count + retrievedValue = value; + return ConcurrentTreeResult_Extended.success; + } + + private ConcurrentTreeResult_Extended writeDeletion( + in Key key, + in SearchResultInfo info, + in ConcurrentTreeResult_Extended getResult, + in bool exitOnTest, // if exitOnTest is true, then this function should never perform a write + ref LockBuffer lockBuffer, + ref Latch latch + ) where LockBuffer : ILockBuffer + { + if (getResult == ConcurrentTreeResult_Extended.notFound || exitOnTest) + { + latch.ExitLatchChain(ref lockBuffer); +#if ConcurrentSortedDictionary_DEBUG + Test.Assert(!latch.HoldingRootLock); + Test.Assert(ReferenceEquals(null, lockBuffer.peek())); +#endif + return ConcurrentTreeResult_Extended.notFound; + } + + info.node.sync_DeleteAtThisNode(in key, this, ref lockBuffer, ref latch); + // NOTE* tree is unlocked after sync_delete + Interlocked.Decrement(ref this._count); // decrement count + return ConcurrentTreeResult_Extended.success; + } + + /// + /// Remove a key-value pair from the tree. + /// + public RemoveResult TryRemove(in Key key, int timeoutMs) + { + assertTimeoutArg(timeoutMs); + Value v = default(Value); + return ToRemoveResult(writeToTree(in key, in v, in timeoutMs, LatchAccessType.deleteTest, out v)); + } + + /// + /// Remove a key-value pair from the tree. Waits forever until mutex(s) are acquired. + /// + public bool TryRemove(in Key key) + { + Value v = default(Value); + return ToRemoveResult(writeToTree(in key, in v, -1, LatchAccessType.deleteTest, out v)) == + RemoveResult.success; + } + + /// + /// Search for input key and outputs the value. Returns false if not found. Waits forever until search mutex(s) are acquired. + /// + public bool TryGetValue(in Key key, out Value value) + { + return tryGetValue(in key, out value, -1) == SearchResult.success; + } + + /// + /// Search for input key and outputs the value. Returns if it was successful. + /// + public SearchResult TryGetValue(in Key key, out Value value, in int timeoutMs) + { + assertTimeoutArg(timeoutMs); + return tryGetValue(in key, out value, timeoutMs); + } + + /// + /// Check if the input key is in this collection. + /// + public SearchResult ContainsKey(in Key key, in int timeoutMs) + { + assertTimeoutArg(timeoutMs); + Value value; + return tryGetValue(in key, out value, in timeoutMs); + } + + /// + /// Check if the input key is in this collection. Wait forever to acquire mutex(s) + /// + public bool ContainsKey(in Key key) + { + Value value; + return tryGetValue(in key, out value, -1) == SearchResult.success; + } + + /// + /// Search for input key and output the value. + /// + private SearchResult tryGetValue(in Key key, out Value value, in int timeoutMs = -1) + { + if (!typeof(Key).IsValueType && ReferenceEquals(null, key)) + { + throw new ArgumentException("Cannot have null key"); + } + + SearchResultInfo searchInfo = default(SearchResultInfo); + var searchOptions = new SearchOptions(timeoutMs); + var latch = new Latch(LatchAccessType.read, this._rootLock); + var readLockBuffer = new LockBuffer2(); + var result = ConcurrentKTreeNode.TryGetValue(in key, out value, + ref searchInfo, ref latch, ref readLockBuffer, this, searchOptions); + if (result == ConcurrentTreeResult_Extended.timedOut) + { + return SearchResult.timedOut; + } + + this._depth = searchInfo.depth; // Note* Int32 read/write is atomic + return result == ConcurrentTreeResult_Extended.success ? SearchResult.success : SearchResult.notFound; + } + + public Value this[in Key key] + { + get + { + Value value; + if (!this.TryGetValue(in key, out value)) + { + throw new ArgumentException("Input key does not exist!"); + } + + return value; + } + set { this.AddOrUpdate(in key, in value); } + } + + /// + /// Can be used to iterate though all items in the Dictionary with optional timeout and subtree depth. + /// + public IEnumerable> Items(int itemTimeoutMs = -1) + { + using (var it = GetEnumerator(itemTimeoutMs)) + { + while (it.MoveNext()) + { + yield return it.Current; + } + } + + ; + } + + public IEnumerable> Reversed(int itemTimeoutMs = -1) + { + using (var it = GetEnumerator(itemTimeoutMs, reversed: true)) + { + while (it.MoveNext()) + { + yield return it.Current; + } + } + + ; + } + + public IEnumerable Keys + { + get + { + foreach (var pair in this) + { + yield return pair.Key; + } + } + } + + public IEnumerable Values + { + get + { + foreach (var pair in this) + { + yield return pair.Value; + } + } + } + + public IEnumerator> GetEnumerator() + { + return GetEnumerator(-1, false); + } + + public IEnumerator> GetEnumerator(int itemTimeoutMs = -1, bool reversed = false) + { + return ConcurrentKTreeNode.AllItems(this, itemTimeoutMs, reversed).GetEnumerator(); + } + + IEnumerator IEnumerable.GetEnumerator() + { + return this.GetEnumerator(); + } + + public void Clear() + { + clear(); + } + + public bool Clear(int timeoutMs) + { + assertTimeoutArg(timeoutMs); + return clear(); + } + + private bool clear(int timeoutMs = -1) + { + // Try to enter the root lock + var latch = new Latch(LatchAccessType.delete, this._rootLock, assumeLeafIsSafe: false); + if (!latch.TryEnterRootLock(timeoutMs)) + { + return false; + } + + try + { + // Make a new root... + var newRoot = new ConcurrentKTreeNode(_root.k, isLeaf: true); + this.setRoot(newRoot); + this._count = 0; + this._depth = 0; + return true; + } + finally + { + latch.ExitRootLock(); + } + } + + /// + /// Struct that contains meta data about the TryGetValue search attempt. + /// + private struct SearchResultInfo where K : IComparable + { + /// + /// Index of found item. -1 if not found. + /// + public int index; + + /// + /// Node that the search stopped at. + /// + public ConcurrentKTreeNode node; + + /// + /// Depth where search stopped + /// + public int depth; + + /// + /// Next key of next node + /// + public K nextSubTreeKey; + + /// + /// Is there a subtree after this one? + /// + public bool hasNextSubTree; + } + + private enum SearchType + { + search = 0, + findMin = 1, + findMax = 2 + } + + private struct SearchOptions + { + private const int kDefaultMaxDepth = int.MaxValue - 1; + public readonly int timeoutMs; + public readonly int maxDepth; + public readonly long startTime; + public SearchType type; + + public SearchOptions(in int timeoutMs = -1, in int maxDepth = kDefaultMaxDepth, + in SearchType type = SearchType.search, in long startTime = -1) + { + this.timeoutMs = timeoutMs; + this.maxDepth = maxDepth; + this.type = type; + this.startTime = startTime < 0 ? DateTimeOffset.Now.ToUnixTimeMilliseconds() : startTime; + } + + public void assertValid(bool isReadAccess) + { + if (this.maxDepth != kDefaultMaxDepth && !isReadAccess) + throw new ArgumentException("Can only set maxDepth for read access"); + if (maxDepth > kDefaultMaxDepth || maxDepth < 0) + throw new ArgumentException("Invalid maxDepth specified"); + } + } + + /// + /// Key-Value pair that is used to store all items in the tree. + /// + private struct NodeData where K : IComparable + { + public readonly V value; + public readonly K key; + + public NodeData(in K key, in V value) + { + this.value = value; + this.key = key; + } + } + + private enum ConcurrentTreeResult_Extended + { + success = 0, + notFound = 1, + timedOut = 2, + alreadyExists = 3, + notSafeToUpdateLeaf = 4 + } + + private enum LatchAccessType + { + read = 0, + insert = 1, + delete = 2, + insertTest = 3, + deleteTest = 4 + } + + private enum LatchAccessResult + { + timedOut = 0, + acquired = 1, + notSafeToUpdateLeaf = 2, + + // Test Result.. This is returned when it is not safe to update the leaf + // but we want to retain the lock on the leaf for the purpose of testing if the + // desired key is present + notSafeToUpdateLeafTest = 3 + } + + private interface ILockBuffer where K : IComparable + { + public ConcurrentKTreeNode peek(); + public void push(in ConcurrentKTreeNode node); + public ConcurrentKTreeNode pop(); + public ConcurrentKTreeNode peekParent(); + } + + private struct LockBuffer2 : ILockBuffer where K : IComparable + { + public ConcurrentKTreeNode peek() + { + if (!ReferenceEquals(null, r1)) return r1; + else return r0; + } + + public ConcurrentKTreeNode peekParent() + { + if (!ReferenceEquals(null, r1)) return r0; + return null; + } + + public void push(in ConcurrentKTreeNode node) + { + if (ReferenceEquals(null, r0)) r0 = node; + else if (ReferenceEquals(null, r1)) r1 = node; + else throw new ArgumentException("Lock stack is full"); + } + + public ConcurrentKTreeNode pop() + { + //Note* pop returns null on empty, this is intentional + // See Latch.ExitLatchChain- it will just iterate until pop() returns null + if (!ReferenceEquals(null, r1)) + { + var result = r1; + r1 = null; + return result; + } + else + { + var result = r0; + r0 = null; + return result; + } + } + + private ConcurrentKTreeNode r0; + private ConcurrentKTreeNode r1; + } + + // Doing this nonsense because c# doesn't allow stacalloc of reference type arrays. + // -And don't want to force users to use unsafe if they arent compiling with it. + // Another alternative is to create a pool or linked list nodes- + // but this would potentially create unexpected memory usage by this data structure. + // This struct should be 256 bytes and is used whenever a write forces changing the tree structure. + private struct LockBuffer32 : ILockBuffer where K : IComparable + { + public ConcurrentKTreeNode peek() + { + if (this.Count <= 0) + return null; + return get(this.Count - 1); + } + + public ConcurrentKTreeNode peekParent() + { + if (this.Count <= 1) + return null; + return get(this.Count - 2); + } + + public int Count { get; private set; } + + public void push(in ConcurrentKTreeNode node) + { + if (this.Count >= 32) + throw new ArgumentException("Cannot push, reach lock buffer limit"); + set(this.Count, in node); + this.Count++; + } + + public ConcurrentKTreeNode pop() + { + //Note* pop returns null on empty, this is intentional + if (this.Count <= 0) + return null; + var topNode = get(this.Count - 1); + set(this.Count - 1, null); + this.Count--; + return topNode; + } + + private ConcurrentKTreeNode r0; + private ConcurrentKTreeNode r1; + private ConcurrentKTreeNode r2; + private ConcurrentKTreeNode r3; + private ConcurrentKTreeNode r4; + private ConcurrentKTreeNode r5; + private ConcurrentKTreeNode r6; + private ConcurrentKTreeNode r7; + private ConcurrentKTreeNode r8; + private ConcurrentKTreeNode r9; + private ConcurrentKTreeNode r10; + private ConcurrentKTreeNode r11; + private ConcurrentKTreeNode r12; + private ConcurrentKTreeNode r13; + private ConcurrentKTreeNode r14; + private ConcurrentKTreeNode r15; + private ConcurrentKTreeNode r16; + private ConcurrentKTreeNode r17; + private ConcurrentKTreeNode r18; + private ConcurrentKTreeNode r19; + private ConcurrentKTreeNode r20; + private ConcurrentKTreeNode r21; + private ConcurrentKTreeNode r22; + private ConcurrentKTreeNode r23; + private ConcurrentKTreeNode r24; + private ConcurrentKTreeNode r25; + private ConcurrentKTreeNode r26; + private ConcurrentKTreeNode r27; + private ConcurrentKTreeNode r28; + private ConcurrentKTreeNode r29; + private ConcurrentKTreeNode r30; + private ConcurrentKTreeNode r31; + + private void set(in int i, in ConcurrentKTreeNode value) + { + switch (i) + { + case 0: + this.r0 = value; + return; + case 1: + this.r1 = value; + return; + case 2: + this.r2 = value; + return; + case 3: + this.r3 = value; + return; + case 4: + this.r4 = value; + return; + case 5: + this.r5 = value; + return; + case 6: + this.r6 = value; + return; + case 7: + this.r7 = value; + return; + case 8: + this.r8 = value; + return; + case 9: + this.r9 = value; + return; + case 10: + this.r10 = value; + return; + case 11: + this.r11 = value; + return; + case 12: + this.r12 = value; + return; + case 13: + this.r13 = value; + return; + case 14: + this.r14 = value; + return; + case 15: + this.r15 = value; + return; + case 16: + this.r16 = value; + return; + case 17: + this.r17 = value; + return; + case 18: + this.r18 = value; + return; + case 19: + this.r19 = value; + return; + case 20: + this.r20 = value; + return; + case 21: + this.r21 = value; + return; + case 22: + this.r22 = value; + return; + case 23: + this.r23 = value; + return; + case 24: + this.r24 = value; + return; + case 25: + this.r25 = value; + return; + case 26: + this.r26 = value; + return; + case 27: + this.r27 = value; + return; + case 28: + this.r28 = value; + return; + case 29: + this.r29 = value; + return; + case 30: + this.r30 = value; + return; + case 31: + this.r31 = value; + return; + } + } + + private ConcurrentKTreeNode get(in int i) + { + switch (i) + { + case 0: return this.r0; + case 1: return this.r1; + case 2: return this.r2; + case 3: return this.r3; + case 4: return this.r4; + case 5: return this.r5; + case 6: return this.r6; + case 7: return this.r7; + case 8: return this.r8; + case 9: return this.r9; + case 10: return this.r10; + case 11: return this.r11; + case 12: return this.r12; + case 13: return this.r13; + case 14: return this.r14; + case 15: return this.r15; + case 16: return this.r16; + case 17: return this.r17; + case 18: return this.r18; + case 19: return this.r19; + case 20: return this.r20; + case 21: return this.r21; + case 22: return this.r22; + case 23: return this.r23; + case 24: return this.r24; + case 25: return this.r25; + case 26: return this.r26; + case 27: return this.r27; + case 28: return this.r28; + case 29: return this.r29; + case 30: return this.r30; + case 31: return this.r31; + default: throw new IndexOutOfRangeException(); + } + } + } + + private struct Latch where K : IComparable + { + /// + /// if 'true', write operations will acquire read locks all the way to the leaf- and then acquire + /// a write lock only on the leaf. if 'false' then write locks will be used to traverse down the + /// while latching. + /// + public readonly bool assumeLeafIsSafe; + + /// + /// type of latch + /// + private readonly LatchAccessType accessType; + + /// + /// Retain the reader lock on the found node after finishing a tree search? + /// + public readonly bool retainReaderLock; + + private ReaderWriterLockSlim _rootLock; + + public bool isInsertAccess + { + get + { + return this.accessType == LatchAccessType.insert || this.accessType == LatchAccessType.insertTest; + } + } + + public bool isDeleteAccess + { + get + { + return this.accessType == LatchAccessType.delete || this.accessType == LatchAccessType.deleteTest; + } + } + + public bool isReadAccess + { + get { return this.accessType == LatchAccessType.read; } + } + + public bool TryEnterRootLock(int timeoutMs = -1) + { + if (this.isReadAccess || this.assumeLeafIsSafe) + { + return this._rootLock.TryEnterReadLock(timeoutMs); + } + else + { + return this._rootLock.TryEnterWriteLock(timeoutMs); + } + } + + /// + /// exits the rootLock or does nothing if it was already exited. + /// + public void ExitRootLock() + { + if (!ReferenceEquals(null, this._rootLock)) + { + if (this.isReadAccess || this.assumeLeafIsSafe) + { + this._rootLock.ExitReadLock(); + } + else + { + this._rootLock.ExitWriteLock(); + } + + this._rootLock = null; + } + } + +#if ConcurrentSortedDictionary_DEBUG + public bool HoldingRootLock { get { return !ReferenceEquals(null, this._rootLock); } } +#endif + + /// + /// Exit the latch at this level and every parent including the rootLock + /// + public void ExitLatchChain(ref LockBuffer lockBuffer) where LockBuffer : ILockBuffer + { + ConcurrentKTreeNode node = lockBuffer.pop(); + while (node != null) + { + if (this.isReadAccess || + (this.assumeLeafIsSafe && !node.isLeaf) + ) + { + node._rwLock.ExitReadLock(); + } + else + { + node._rwLock.ExitWriteLock(); + } + + node = lockBuffer.pop(); + } + + ExitRootLock(); + } + + /// + /// Exit the latch at the node at the top of the stack + /// + public void PopLatch(ref LockBuffer lockBuffer) where LockBuffer : ILockBuffer + { + ConcurrentKTreeNode node = lockBuffer.pop(); + if (!ReferenceEquals(null, node)) + { + if (this.isReadAccess || + (this.assumeLeafIsSafe && !node.isLeaf) + ) + { + node._rwLock.ExitReadLock(); + } + else + { + node._rwLock.ExitWriteLock(); + } + } + + // If there is nothing left in the buffer... + if (ReferenceEquals(null, lockBuffer.peek())) + { + ExitRootLock(); + } + } + + public LatchAccessResult TryEnterLatch( + ref LockBuffer lockBuffer, + in ConcurrentKTreeNode node, + in int timeoutMs, + bool isRoot + ) where LockBuffer : ILockBuffer + { + if (this.isReadAccess || + (this.assumeLeafIsSafe && !node.isLeaf) + ) + { + // Try to acquire read lock... + bool acquired = node._rwLock.TryEnterReadLock(timeoutMs); + // Always release existing locks, even if failed to acquire + ExitLatchChain(ref lockBuffer); + if (acquired) + { + lockBuffer.push(node); + } + + return acquired ? LatchAccessResult.acquired : LatchAccessResult.timedOut; + } + else + { + // try to acquire a write lock... + if (!node._rwLock.TryEnterWriteLock(timeoutMs)) + { + // If failed to get the lock.. release locks + ExitLatchChain(ref lockBuffer); + return LatchAccessResult.timedOut; + } + + // Check if it is safe to update node + if (node.NodeIsSafe(this.isInsertAccess, this.isDeleteAccess, isRoot)) + { + ExitLatchChain(ref lockBuffer); // Exit existing locks + lockBuffer.push(node); // push newly acquired lock to chain + return LatchAccessResult.acquired; + } + + // Not safe to update.. + if (this.assumeLeafIsSafe) + { + lockBuffer.push(node); // push newly acquired lock to chain + + // if test... retain the write lock! (this way we can read the leaf to test it) + if (this.accessType == LatchAccessType.insertTest || + this.accessType == LatchAccessType.deleteTest) + { + return LatchAccessResult.notSafeToUpdateLeafTest; + } + + // if assumingLeafIsafe, then exit latch and return not safe + ExitLatchChain(ref lockBuffer); + return LatchAccessResult.notSafeToUpdateLeaf; + } + + // Otherwise... return acquired and don't release any locks + lockBuffer.push(node); + return LatchAccessResult.acquired; + } + } + + public Latch( + LatchAccessType type, + ReaderWriterLockSlim rootLock, + bool assumeLeafIsSafe = true, + bool retainReaderLock = false + ) + { + this.accessType = type; + this._rootLock = rootLock; + this.assumeLeafIsSafe = assumeLeafIsSafe; + this.retainReaderLock = retainReaderLock; + } + } + + /// + /// Tree Node with N children. Can be a leaf or an internal node. + /// + private partial class ConcurrentKTreeNode where K : IComparable + { + public ConcurrentKTreeNode(int k, bool isLeaf = false) + { + if (isLeaf) + { + this._values = new NodeData[k + 1]; + this._children = null; + } + else + { + this._children = new NodeData>[k + 1]; + this._values = null; + } + + this._rwLock = new ReaderWriterLockSlim(LockRecursionPolicy.NoRecursion); + this._count = 0; + } + + private LeafSiblingNodes siblings; + private NodeData[] _values; + private NodeData>[] _children; + private volatile int _count; + public ReaderWriterLockSlim _rwLock { get; private set; } // Each node has its own lock + +#if ConcurrentSortedDictionary_DEBUG + private volatile int _version; +#endif + + public bool isLeaf + { + get { return ReferenceEquals(null, this._children); } + } + + public int k + { + get { return this.isLeaf ? this._values.Length - 1 : this._children.Length - 1; } + } + + public void SetValue(in int index, in K key, in V value) + { + this._values[index] = new NodeData(key, value); + } + + public ref NodeData GetValue(in int index) + { + return ref this._values[index]; + } + + public int Count + { + get { return _count; } + set { this._count = value; } + } + + /// + /// Perform insert starting at leaf and recurse up. + /// **WARNING**. This method assumes the calling thread has acquired all write locks needed + /// for this write. + /// + public void sync_InsertAtThisNode( + in K key, + in V value, + in ConcurrentSortedDictionary tree, + ref LockBuffer lockBuffer, + ref Latch latch + ) where LockBuffer : ILockBuffer + { + if (!this.isLeaf) + { + throw new Exception("Can only insert at leaf node"); + } + + this.orderedInsert(in key, in this._values, in value); + trySplit(in tree, ref lockBuffer, ref latch); + } + + /// + /// Returns index in the array for the input key + /// + private int searchRangeIndex(in K key, in NodeData[] array, out int compareResult) + { + // Perform a modified binary search to find the 'key' in the array + // This binary search will return the index of the last bucket that 'key' is greater than or equal to + // eg, for key = 5, and arr = 1, 2, 4, 6, 7, the result is index=2, for key = 5 and arr = 1, 2, 4, 5, 7, 8, the reuslt is index = 3 + // this search function expects unique array entries! + int lo = 0; + int hi = this.Count - 1; + compareResult = -1; + if (hi < 0) return 0; + int index = lo; + while (lo <= hi) + { + index = lo + ((hi - lo) >> 1); + compareResult = key.CompareTo(array[index].key); + if (compareResult == 0) return index; + if (compareResult > 0) + { + lo = index + 1; + } + else + { + hi = index - 1; + } + } + + if (compareResult < 0 && index > 0) + { + compareResult = key.CompareTo(array[index - 1].key); + return index - 1; + } + + return index; + } + + /// + /// Returns index in the array for the input key + /// + private int searchRangeIndex(in K key, in NodeData[] array) + { + int cmp; + return searchRangeIndex(in key, in array, out cmp); + } + + private int indexOfNode(in ConcurrentKTreeNode node) + { + int cmp; + // The index of node in 'this' can be found via a bsearch by using any key that is underneath 'node' + if (node.isLeaf) + { + return searchRangeIndex(in node._values[0].key, this._children, out cmp); + } + else + { + K key; + if (node.Count <= 1) + { + // In the event that node recently had a merge... It could have a count of '1' + // so we need to fetch from any of node's children keys + // the children of node are guarenteed to be well formed! (eg have atleast k/2 children) + key = node._children[0].value.isLeaf + ? node._children[0].value._values[0].key + : node._children[0].value._children[1].key; + } + else + { + key = node._children[1].key; + } + + return searchRangeIndex(in key, in this._children, out cmp); + } + } + + /// + /// Insert item into array and shift array right. Returns index of inserted item. + /// + /// insert in the bucket this key belongs to + /// array to insert into + /// value to insert + private int orderedInsert( + in K key, + in NodeData[] array, + in VType value + ) + { + // Get index that key belongs in + int compareResult; + int index = this.searchRangeIndex(in key, in array, out compareResult); + if (compareResult > 0 && this.Count > 0) + index++; // insertion happens after found index + // shift the array right + return indexInsert(in index, in key, in array, in value); + } + + /// + /// Insert key-value at index. Shift array right after index. + /// + /// insert in the bucket this key belongs to + /// array to insert into + /// value to insert + private int indexInsert( + in int index, + in K key, + in NodeData[] array, + in VType value + ) + { + if (index < this.Count) + { + // shift the array right + Array.Copy(array, index, array, index + 1, this.Count - index); + } + + //assign value at index + array[index] = new NodeData(key, value); + this.Count++; + return index; + } + + /// + /// Try to split this node into two nodes. The new node will have k/2 highest children. + /// The current node (this) will keep k/2 lowest children. + /// The new node will have the same parent as this node. + /// + private static void trySplit( + in ConcurrentSortedDictionary tree, + ref LockBuffer lockBuffer, + ref Latch latch + ) where LockBuffer : ILockBuffer + { + /// + /// copy right half of array from -> to and zero-initialize copied indices in 'from' + /// + void splitCopy( + in ConcurrentKTreeNode from, + in ConcurrentKTreeNode to, + out K newNodeMinKey + ) + { + int arrLength = from.k + 1; + int half = arrLength / 2; + if (from.isLeaf) + { + for (int i = half; i < arrLength; i++) + { + to._values[i - half] = from._values[i]; + from._values[i] = default(NodeData); // 0 init + } + + newNodeMinKey = to._values[0].key; + } + else + { + // set default key for new node + newNodeMinKey = from._children[half].key; + to._children[0] = + new NodeData>(default(K), from._children[half].value); + from._children[half] = default(NodeData>); // default + // set the rest + for (int i = half + 1; i < arrLength; i++) + { + to._children[i - half] = new NodeData>( + from._children[i].key, from._children[i].value); + from._children[i] = default(NodeData>); // 0 init + } + } + + from.Count = half; + to.Count = arrLength - half; + } + + var node = lockBuffer.peek(); + var parent = lockBuffer.peekParent(); + +#if ConcurrentSortedDictionary_DEBUG + int version = node.assertWriterLock(beginWrite: true); +#endif + + // Check if this node needs to split + if (!node.canSplit()) + { +#if ConcurrentSortedDictionary_DEBUG + node.assertWriterLock(version); +#endif + + latch.PopLatch(ref lockBuffer); + +#if ConcurrentSortedDictionary_DEBUG + Test.Assert(!latch.HoldingRootLock); + Test.Assert(ReferenceEquals(null, lockBuffer.peek())); +#endif + + return; + } + + bool isRoot = ReferenceEquals(null, parent); + + // 1. Make empty new node with the same parent as this node + var newNode = new ConcurrentKTreeNode(node.k, node.isLeaf); + + // 2. Copy k/2 largest from this node to the new node + K newNodeMinKey; + splitCopy(node, in newNode, out newNodeMinKey); + + // 3a. Handle root edge case + if (isRoot) + { + var newRoot = new ConcurrentKTreeNode(node.k, false); + newRoot._children[0] = new NodeData>(default(K), node); + newRoot._children[1] = new NodeData>(newNodeMinKey, in newNode); + newRoot.Count = 2; + tree.setRoot( + newRoot); // Note* newRoot is not locked.. but noone else has ref to it since the root ptr is locked + + if (node.isLeaf) LeafSiblingNodes.AtomicUpdateSplitNodes(node, newNode); + +#if ConcurrentSortedDictionary_DEBUG + node.assertRootWriteLockHeld(tree); +#endif + + // pop node off of latch (also unlocks node) + latch.PopLatch(ref lockBuffer); + +#if ConcurrentSortedDictionary_DEBUG + Test.Assert(!latch.HoldingRootLock); + Test.Assert(ReferenceEquals(null, lockBuffer.peek())); +#endif + // 3b. Otherwise, handle internal node parent + } + else + { + var thisNodeIndex = parent.indexOfNode(node); + // Insert new node into the parent + parent.indexInsert(thisNodeIndex + 1, newNodeMinKey, in parent._children, in newNode); + + if (node.isLeaf) LeafSiblingNodes.AtomicUpdateSplitNodes(node, newNode); + + // pop node off of latch (also unlocks node) + latch.PopLatch(ref lockBuffer); + +#if ConcurrentSortedDictionary_DEBUG + parent.assertWriterLockHeld(); + Test.Assert(ReferenceEquals(parent, lockBuffer.peek())); +#endif + + // Try recurse on parent + trySplit(in tree, ref lockBuffer, ref latch); + } + } + + /// + /// Perform deletion starting at this node and recurse up. + /// **WARNING**. This method assumes the calling thread has acquired all write locks needed + /// for this write. + /// + public void sync_DeleteAtThisNode( + in K key, + in ConcurrentSortedDictionary tree, + ref LockBuffer lockBuffer, + ref Latch latch + ) where LockBuffer : ILockBuffer + { + if (!this.isLeaf) + { + throw new Exception("Can only delete at leaf node"); + } + + this.orderedDelete(in key, in this._values); + tryMerge(in tree, ref lockBuffer, ref latch); + } + + /// + /// This node will merge/adopt from siblings to maintain tree balane + /// + private static void tryMerge( + in ConcurrentSortedDictionary tree, + ref LockBuffer lockBuffer, + ref Latch latch + ) where LockBuffer : ILockBuffer + { + /// + /// Merge 'left' into 'right'. Update parent accordingly. + /// + void mergeLeft( + in ConcurrentKTreeNode left, + in ConcurrentKTreeNode right, + in int leftNodeIndex, + in int rightNodeIndex, + in ConcurrentKTreeNode parent + ) + { + var leftAncestorKey = parent._children[leftNodeIndex].key; + var rightAncestorKey = parent._children[rightNodeIndex].key; + + // Perform Copy + if (left.isLeaf) + { + for (int i = right.Count - 1; i >= 0; i--) + { + // Shift rightArray right by left.Count + right._values[i + left.Count] = right._values[i]; + } + + for (int i = 0; i < left.Count; i++) + { + right._values[i] = left._values[i]; + left._values[i] = default(NodeData); + } + } + else + { + // Update right key to be non default in preparation for being (not in the front anymore) + right._children[0] = + new NodeData>(rightAncestorKey, right._children[0].value); + for (int i = right.Count - 1; i >= 0; i--) + { + right._children[i + left.Count] = right._children[i]; + } + + for (int i = 0; i < left.Count; i++) + { + right._children[i] = new NodeData>( + left._children[i].key, left._children[i].value); + left._children[i] = default(NodeData>); // clear + } + } + + // Update Counts + right.Count += left.Count; + left.Count = 0; + // De-parent the left node + parent.deleteIndex(in leftNodeIndex, parent._children); + // right is shifted left by one + parent._children[rightNodeIndex - 1] = + new NodeData>(leftAncestorKey, right); + } + + /// + /// Merge 'right' into 'left'. Upate parent accordingly. + /// + void mergeRight( + in ConcurrentKTreeNode left, + in ConcurrentKTreeNode right, + in int leftNodeIndex, + in int rightNodeIndex, + in ConcurrentKTreeNode parent + ) + { + var rightAncestorKey = parent._children[rightNodeIndex].key; + + // Perform copy + if (right.isLeaf) + { + for (int i = 0; i < right.Count; i++) + { + left._values[left.Count + i] = right._values[i]; + right._values[i] = default(NodeData); + } + } + else + { + // Update right key to be non default in preparation for being (not in the front anymore) + right._children[0] = + new NodeData>(rightAncestorKey, right._children[0].value); + for (int i = 0; i < right.Count; i++) + { + left._children[left.Count + i] = new NodeData>( + right._children[i].key, right._children[i].value); + right._children[i] = default(NodeData>); // clear + } + } + + // Update Counts + left.Count += right.Count; + right.Count = 0; + // De-Parent the right node + parent.deleteIndex(in rightNodeIndex, in parent._children); + } + + /// + /// Adopt left to right. + /// + void adoptLeft( + in ConcurrentKTreeNode left, + in ConcurrentKTreeNode right, + in int leftNodeIndex, + in int rightNodeIndex, + in ConcurrentKTreeNode parent + ) + { + int leftArrayIndex = left.Count - 1; // (index of max in left node) + + K newParentMin; + var rightAncestorKey = parent._children[rightNodeIndex].key; + + if (left.isLeaf) + { + // copy from left[count-1] to right [0] + newParentMin = left._values[leftArrayIndex].key; + right.indexInsert(0, in left._values[leftArrayIndex].key, in right._values, + in left._values[leftArrayIndex].value); + left._values[leftArrayIndex] = default(NodeData); + } + else + { + newParentMin = left._children[leftArrayIndex].key; + // Update right key to be non default in preparation for being (not in the front anymore) + right._children[0] = + new NodeData>(rightAncestorKey, right._children[0].value); + int insertedIndex = right.indexInsert(0, default(K), + in right._children, in left._children[leftArrayIndex].value); + left._children[leftArrayIndex] = default(NodeData>); + } + + // Update parent keys + parent._children[rightNodeIndex] = new NodeData>(newParentMin, right); + // Update counts + left.Count--; + } + + /// + /// Adopt right into left. + /// + void adoptRight( + in ConcurrentKTreeNode left, + in ConcurrentKTreeNode right, + in int leftNodeIndex, + in int rightNodeIndex, + in ConcurrentKTreeNode parent + ) + { + K newParentMin; + var rightAncestorKey = parent._children[rightNodeIndex].key; + + if (right.isLeaf) + { + // copy right[0] to left[count] + newParentMin = right._values[1].key; // new parent min will be the next key... + left._values[left.Count] = right._values[0]; + right.deleteIndex(0, right._values); + } + else + { + newParentMin = right._children[1].key; + left._children[left.Count] = new NodeData>(rightAncestorKey, + right._children[0].value); + right.deleteIndex(0, right._children); + // reset the key on the first index in the right array + right._children[0] = + new NodeData>(default(K), right._children[0].value); + } + + // Update parent keys + parent._children[rightNodeIndex] = new NodeData>(newParentMin, right); + // update counts + left.Count++; + } + + var node = lockBuffer.peek(); + var parent = lockBuffer.peekParent(); + bool isRoot = ReferenceEquals(null, parent); + +#if ConcurrentSortedDictionary_DEBUG + int version = node.assertWriterLock(beginWrite: true); +#endif + + // 1. Check if this node needs to merge or adopt + if (!node.canMerge(isRoot)) + { +#if ConcurrentSortedDictionary_DEBUG + node.assertWriterLock(version); +#endif + + latch.PopLatch(ref lockBuffer); + +#if ConcurrentSortedDictionary_DEBUG + Test.Assert(!latch.HoldingRootLock); + Test.Assert(ReferenceEquals(null, lockBuffer.peek())); +#endif + + return; + } + + bool isLeaf = node.isLeaf; + + // 2. Handle root edge case + if (isRoot) + { + // Try to select new root if this root only has 1 child + if (!isLeaf) + { +#if ConcurrentSortedDictionary_DEBUG + Test.Assert(node.Count == 1); +#endif + tree.setRoot(node._children[0].value); // set new root + +#if ConcurrentSortedDictionary_DEBUG + node.assertWriterLock(version); + node.assertRootWriteLockHeld(tree); +#endif + + latch.PopLatch(ref lockBuffer); + +#if ConcurrentSortedDictionary_DEBUG + Test.Assert(!latch.HoldingRootLock); + Test.Assert(ReferenceEquals(null, lockBuffer.peek())); +#endif + + return; + } +#if ConcurrentSortedDictionary_DEBUG + node.assertWriterLock(version); +#endif + + latch.PopLatch(ref lockBuffer); + +#if ConcurrentSortedDictionary_DEBUG + Test.Assert(!latch.HoldingRootLock); + Test.Assert(ReferenceEquals(null, lockBuffer.peek())); +#endif + + // Otherwise, root remains... + return; + } + +#if ConcurrentSortedDictionary_DEBUG + parent.assertWriterLockHeld(); +#endif + + int nodeIndex = parent.indexOfNode(node); + int leftIndex = nodeIndex - 1; + int rightIndex = nodeIndex + 1; + var left = nodeIndex > 0 ? parent._children[leftIndex].value : null; + var right = nodeIndex < parent.Count - 1 ? parent._children[rightIndex].value : null; + + try + { + // Always write lock on nodes- this is due to the leaf iterator! + // To safely iterate side-to-side, all leaf nodes that are being modified need to be write locked + // Deadlock is guarenteed to not occur here because we hold a write-lock on the parent of this node! + // Or for internal nodes, there are cases where subsequent in-progress writes will have race conditions if not locked + if (!ReferenceEquals(null, left)) left._rwLock.EnterWriteLock(); + if (!ReferenceEquals(null, right)) right._rwLock.EnterWriteLock(); + + // 3. Try to Adopt from left + if (!ReferenceEquals(null, left) && left.canSafelyDelete(isRoot)) + { + adoptLeft(in left, node, in leftIndex, in nodeIndex, parent); + +#if ConcurrentSortedDictionary_DEBUG + node.assertWriterLock(version); +#endif + + latch.ExitLatchChain(ref lockBuffer); + +#if ConcurrentSortedDictionary_DEBUG + Test.Assert(!latch.HoldingRootLock); + Test.Assert(ReferenceEquals(null, lockBuffer.peek())); +#endif + + return; + } + + // 4. Try to Adopt from right + if (!ReferenceEquals(null, right) && right.canSafelyDelete(isRoot)) + { + adoptRight(node, in right, in nodeIndex, in rightIndex, parent); + +#if ConcurrentSortedDictionary_DEBUG + node.assertWriterLock(version); +#endif + + latch.ExitLatchChain(ref lockBuffer); + +#if ConcurrentSortedDictionary_DEBUG + Test.Assert(!latch.HoldingRootLock); + Test.Assert(ReferenceEquals(null, lockBuffer.peek())); +#endif + + return; + } + + // 5a. Merge Right if possible + if (!ReferenceEquals(null, right)) + { + mergeRight(node, in right, in nodeIndex, in rightIndex, parent); + if (right.isLeaf) LeafSiblingNodes.AtomicUpdateMergeNodes(in right); + } + // 5b. Otherwise Merge left + else + { + mergeLeft(in left, node, in leftIndex, in nodeIndex, parent); + if (left.isLeaf) LeafSiblingNodes.AtomicUpdateMergeNodes(in left); + } + +#if ConcurrentSortedDictionary_DEBUG + node.assertWriterLock(version); +#endif + } + finally + { + if (!ReferenceEquals(null, left)) left._rwLock.ExitWriteLock(); + if (!ReferenceEquals(null, right)) right._rwLock.ExitWriteLock(); + } + + // pop node + latch.PopLatch(ref lockBuffer); + +#if ConcurrentSortedDictionary_DEBUG + Test.Assert(ReferenceEquals(parent, lockBuffer.peek())); +#endif + + // 6. Try to merge recurse on parent + tryMerge(in tree, ref lockBuffer, ref latch); + } + + /// + /// Removes the key from the array. + /// + private void orderedDelete( + in K key, + in NodeData[] array + ) + { + // Get index of key + int index = this.searchRangeIndex(in key, in array); + // shift the array left + this.deleteIndex(in index, in array); + } + + /// + /// Remove index from array and shift left starting at removed index + /// + private void deleteIndex( + in int index, + in NodeData[] array + ) + { + // shift the array left + if (index < this.Count) + { + Array.Copy(array, index + 1, array, index, this.Count - index); + } + + // unassign value in the array + array[this.Count] = default(NodeData); + this.Count--; + } + + /// + /// Recurse down the tree searching for a value starting at the root. + /// + /// key of the item to be inserted + /// item to be inserted + /// contains search meta data + /// latch used to secure concurrent node access. TryGetValue will always release the entire latch upon exiting except in the case where it is in (insert|delete) and returned notFound or success. + public static ConcurrentTreeResult_Extended TryGetValue( + in K key, + out V value, + ref SearchResultInfo info, + ref Latch latch, + ref LockBuffer lockBuffer, + in ConcurrentSortedDictionary tree, + in SearchOptions options = new SearchOptions() + ) where LockBuffer : ILockBuffer + { + options.assertValid(latch.isReadAccess); + + // Try to retrieve a value. + // When this function returns: + // timedOut => latch is fully released + // notFound,success => if this is a write operation then it is not released + // => if read lock retaining was specified then it is not released + // => otherise, it is released + // notSafeToUpdateLeaf => if this is a write-test, then it is not released + // => otherwise is is released + + // Try to enter the root lock + if (!latch.TryEnterRootLock(options.timeoutMs)) + { + value = default(V); + info.index = -1; + return ConcurrentTreeResult_Extended.timedOut; + } + + // Init + info.depth = 0; + info.node = tree._root as ConcurrentKTreeNode; + info.index = 0; + info.nextSubTreeKey = default(K); + info.hasNextSubTree = false; + int remainingMs = getRemainingMs(in options.startTime, in options.timeoutMs); + + // Try enter latch on this (ie the root node) + LatchAccessResult result = latch.TryEnterLatch(ref lockBuffer, in info.node, in remainingMs, true); + if (result == LatchAccessResult.timedOut || result == LatchAccessResult.notSafeToUpdateLeaf) + { + value = default(V); + info.index = -1; + return result == LatchAccessResult.timedOut + ? ConcurrentTreeResult_Extended.timedOut + : ConcurrentTreeResult_Extended.notSafeToUpdateLeaf; + } + +#if ConcurrentSortedDictionary_DEBUG + int version = info.node.assertLatchLock(ref latch, beginRead: true); +#endif + + for (int depth = 0; depth < options.maxDepth; depth++) + { + if (info.node.isLeaf) + { + int compareResult = 0; + info.index = options.type == SearchType.search + ? info.node.searchRangeIndex(key, info.node._values, out compareResult) + : (options.type == SearchType.findMin ? 0 : info.node.Count - 1); + info.depth = depth; + var searchResult = ConcurrentTreeResult_Extended.success; + if (compareResult == 0) + { + value = info.node._values[info.index].value; + } + else + { + value = default(V); + info.index = -1; + searchResult = ConcurrentTreeResult_Extended.notFound; + } + +#if ConcurrentSortedDictionary_DEBUG + info.node.assertLatchLock(ref latch, version); +#endif + + // Exit latch if reading and not retaining + if (latch.isReadAccess && !latch.retainReaderLock) + { + latch.ExitLatchChain(ref lockBuffer); + } + + if (result == LatchAccessResult.notSafeToUpdateLeafTest) + { + return ConcurrentTreeResult_Extended.notSafeToUpdateLeaf; + } + + return searchResult; + } + else + { +#if ConcurrentSortedDictionary_DEBUG + if (result == LatchAccessResult.notSafeToUpdateLeafTest) + throw new Exception("Failed sanity test"); + info.node.assertLatchLock(ref latch, version); +#endif + + int nextIndex = options.type == SearchType.search + ? info.node.searchRangeIndex(in key, in info.node._children) + : (options.type == SearchType.findMin ? 0 : info.node.Count - 1); + // get next sibling subtree + if (nextIndex + 1 < info.node.Count) + { + info.hasNextSubTree = true; + info.nextSubTreeKey = info.node._children[nextIndex + 1].key; + } + + // Move to next node + info.node = info.node._children[nextIndex].value; + info.depth = depth + 1; + + // Try Enter latch on next node (which will also atomically exit latch on parent) + result = latch.TryEnterLatch(ref lockBuffer, in info.node, in remainingMs, false); + if (result == LatchAccessResult.timedOut || result == LatchAccessResult.notSafeToUpdateLeaf) + { + value = default(V); + info.index = -1; + return result == LatchAccessResult.timedOut + ? ConcurrentTreeResult_Extended.timedOut + : ConcurrentTreeResult_Extended.notSafeToUpdateLeaf; + } + +#if ConcurrentSortedDictionary_DEBUG + version = info.node.assertLatchLock(ref latch, beginRead: true); +#endif + } + + remainingMs = getRemainingMs(in options.startTime, in options.timeoutMs); + } + + // Sanity check + if (info.depth >= int.MaxValue - 1) + { + throw new Exception("Bad Tree State, reached integer max depth limit"); + } + + // maxDepth was reached before finding a result! + if (latch.isReadAccess && !latch.retainReaderLock) + { + latch.ExitLatchChain(ref lockBuffer); + } + + value = default(V); + return ConcurrentTreeResult_Extended.notFound; + } + + /// + /// Get all items starting from this node. This method will not read lock the entire tree. + /// It will instead lock subtrees as it iterates through the entire tree. + /// + /// tree reference + /// depth subtrees which get read locked. (eg 1=k values locked, 2=k^2 locked, 3=k^3 locked), etc.. + /// key of the item to be inserted + public static IEnumerable> AllItems( + ConcurrentSortedDictionary tree, + int itemTimeoutMs = -1, + bool reversed = false + ) + { + bool acquiredNextNode(ConcurrentKTreeNode node, out ConcurrentKTreeNode next) + { + // Now get the next node + next = reversed ? node.siblings.Prev : node.siblings.Next; + if (ReferenceEquals(null, next)) + { + return false; + } + + // Try to acquire read lock... (dont wait at all if lock is held by writer.. just quit) + if (!next._rwLock.TryEnterReadLock(0)) + { + return true; // failed to acquire- need to retry search + } + + return false; // acquired, don't need to retry search + } + + ConcurrentKTreeNode node = null; + bool retry = true; + V _ = default(V); + K cmpKey = default(K); + bool startedSearch = false; + SearchResultInfo subtree = default(SearchResultInfo); + + do + { + if (retry) + { + var searchType = startedSearch + ? SearchType.search + : (reversed ? SearchType.findMax : SearchType.findMin); + var searchOptions = new SearchOptions(itemTimeoutMs, type: searchType); + var latch = new Latch(LatchAccessType.read, tree._rootLock, retainReaderLock: true); + var readLockBuffer = new LockBuffer2(); + + // Recurse to leaf + var searchResult = TryGetValue(cmpKey, out _, ref subtree, + ref latch, ref readLockBuffer, in tree, searchOptions); + if (searchResult == ConcurrentTreeResult_Extended.timedOut) + { + throw new TimeoutException(); + } + else if (searchResult == ConcurrentTreeResult_Extended.notSafeToUpdateLeaf) + { + throw new Exception("Bad Tree State, unexpected search result"); + } + + startedSearch = true; + node = subtree.node; + +#if ConcurrentSortedDictionary_DEBUG + int version = subtree.node.assertLatchLock(ref latch, beginRead: true); +#endif + + try + { + int start, end, increment; + if (reversed) + { + start = node.Count - 1; + end = -1; + increment = -1; + } + else + { + start = 0; + end = node.Count; + increment = 1; + } + + for (int i = start; (!reversed && i < end) || (reversed && i > end); i += increment) + { + var pair = new KeyValuePair(node._values[i].key, node._values[i].value); + bool notProceding = + reversed ? pair.Key.CompareTo(cmpKey) < 0 : pair.Key.CompareTo(cmpKey) > 0; + if (!startedSearch || notProceding) + { + yield return pair; + startedSearch = true; + cmpKey = pair.Key; + } + } + + retry = acquiredNextNode(node, out node); + } + finally + { +#if ConcurrentSortedDictionary_DEBUG + subtree.node.assertLatchLock(ref latch, version); +#endif + + // Release Latch on subtree + latch.ExitLatchChain(ref readLockBuffer); + } + // Otherwise.. just try getting next from sibling... + } + else + { + var prev = node; + try + { + int start, end, increment; + if (reversed) + { + start = node.Count - 1; + end = -1; + increment = -1; + } + else + { + start = 0; + end = node.Count; + increment = 1; + } + +#if ConcurrentSortedDictionary_DEBUG + int version = node.assertReadLock(beginRead: true); +#endif + + for (int i = start; (!reversed && i < end) || (reversed && i > end); i += increment) + { + var pair = new KeyValuePair(node._values[i].key, node._values[i].value); + bool notProceding = + reversed ? pair.Key.CompareTo(cmpKey) < 0 : pair.Key.CompareTo(cmpKey) > 0; + if (!startedSearch || notProceding) + { + yield return pair; + startedSearch = true; + cmpKey = pair.Key; + } + } + +#if ConcurrentSortedDictionary_DEBUG + node.assertReadLock(version); +#endif + + retry = acquiredNextNode(node, out node); + } + finally + { + prev._rwLock.ExitReadLock(); + } + } + } while (node != null); + } + + private bool canSafelyInsert() + { + return this.Count < this.k; // it is safe to insert if (count + 1 <= k) + } + + private bool canSplit() + { + return this.Count > this.k; // split if we exceeded allowed count + } + + private bool canSafelyDelete(bool isRoot) + { + if (isRoot) + { + return this.Count > 2; // root optimization (root only gets deleted on 1 node) + } + + int k = this.k; + // Example: (L=3, safe to release at C=3), (L=4, C=4,3), (L=5, C=5,4), (L=6, C=6,5,4) etc... + int checkLength = k % 2 == 0 ? k / 2 : k / 2 + 1; + return this.Count > checkLength; + } + + private bool canMerge(bool isRoot) + { + if (isRoot) + { + return this.Count < 2; // root optimization (root only gets deleted on 1 node) + } + + int k = this.k; // merge if less than k/2 items in array + int checkLength = k % 2 == 0 ? k / 2 : k / 2 + 1; + return this.Count < checkLength; + } + + /// + /// Check if inserting/deleting on this node will cause a split or merge to parent + /// + public bool NodeIsSafe(bool isInsertAccess, bool isDeleteAccess, bool isRoot) + { + if (isInsertAccess) + { + return canSafelyInsert(); + } + else if (isDeleteAccess) + { + return canSafelyDelete(isRoot); + } + else + { + throw new ArgumentException("Unsupported latch access type"); + } + } + + private partial struct LeafSiblingNodes + { + private volatile ConcurrentKTreeNode next; + private volatile ConcurrentKTreeNode prev; + private volatile int _mutex; + +#if ConcurrentSortedDictionary_DEBUG + private volatile int _version; +#endif + + public ConcurrentKTreeNode Next + { + get { return this.next; } + } + + public ConcurrentKTreeNode Prev + { + get { return this.prev; } + } + + static bool TryAcquire(ref LeafSiblingNodes node) + { +#if ConcurrentSortedDictionary_DEBUG + int set = System.Environment.CurrentManagedThreadId; +#else + int set = 1; +#endif + + return Interlocked.CompareExchange(ref node._mutex, set, 0) == 0; + } + + static void Release(ref LeafSiblingNodes node) + { +#if ConcurrentSortedDictionary_DEBUG + AssertMutexHeld(ref node); +#endif + + node._mutex = 0; + } + + public static void AtomicUpdateSplitNodes(in ConcurrentKTreeNode node, + in ConcurrentKTreeNode splitNode) + { + ConcurrentKTreeNode next; + AcquireSplitMultiLock(in node, in splitNode, out next); + +#if ConcurrentSortedDictionary_DEBUG + int v1, v2, v3; + assertStartWriter(node, splitNode, next, out v1, out v2, out v3); +#endif + + // Fully assign valid points to the new node before allowing it to be reached + splitNode.siblings.next = next; + splitNode.siblings.prev = node; + // Now, leaf searches will find the correct nodes + node.siblings.next = splitNode; + if (!ReferenceEquals(null, next)) + next.siblings.prev = splitNode; + +#if ConcurrentSortedDictionary_DEBUG + assertEndWriter(node, splitNode, next, v1, v2, v3); +#endif + + ReleaseMultiLock(in node, in splitNode, in next); + } + + public static void AtomicUpdateMergeNodes(in ConcurrentKTreeNode deleteNode) + { + ConcurrentKTreeNode prevNode, nextNode; + AcquireMergeMultiLock(in deleteNode, out prevNode, out nextNode); + +#if ConcurrentSortedDictionary_DEBUG + int v1, v2, v3; + assertStartWriter(prevNode, deleteNode, nextNode, out v1, out v2, out v3); +#endif + + if (!ReferenceEquals(null, prevNode)) prevNode.siblings.next = nextNode; + if (!ReferenceEquals(null, nextNode)) nextNode.siblings.prev = prevNode; + +#if ConcurrentSortedDictionary_DEBUG + assertEndWriter(prevNode, deleteNode, nextNode, v1, v2, v3); +#endif + + // For leaf searches- it is possible that the leaf search acquires the deleted node.. + // however, the deleted node will still have the correct siblings, so it can continue + + ReleaseMultiLock(in prevNode, in deleteNode, in nextNode); + } + + static void AcquireSplitMultiLock( + in ConcurrentKTreeNode node, + in ConcurrentKTreeNode splitNode, + out ConcurrentKTreeNode next + ) + { + // Spin forever until acquired... + // The critical section for this lock is expected to only be assigning + // referenes in the siblings struct (very brief critical section) + // The lock is only acquired if all three are acquired simultaneously, otherwise + // try again + int spinCount = 0; + while (true) + { + // Spin backoff... + if (spinCount > 1000000) + { + // This should rarely (if ever) occur naturally as the entire critical section is just a couple of assignments + // Its here to act as a form of backoff... + Thread.Sleep(1); + } + else if (spinCount > 0) + { + while (spinCount < spinCount + 1000) spinCount++; // spin 1000 + } + + spinCount++; + + if (!TryAcquire(ref node.siblings)) + { + continue; + } + + if (!TryAcquire(ref splitNode.siblings)) + { + Release(ref node.siblings); + continue; + } + + next = node.siblings.next; + if (!ReferenceEquals(null, next) && !TryAcquire(ref next.siblings)) + { + Release(ref node.siblings); + Release(ref splitNode.siblings); + continue; + } + + // Acquired all, so just return + return; + } + } + + static void AcquireMergeMultiLock( + in ConcurrentKTreeNode deleteNode, + out ConcurrentKTreeNode prevNode, + out ConcurrentKTreeNode nextNode + ) + { + // Spin forever until acquired... + // The critical section for this lock is expected to only be assigning + // referenes in the siblings struct (very brief critical section) + // The lock is only acquired if all three are acquired simultaneously, otherwise + // try again + int spinCount = 0; + while (true) + { + // Spin backoff... + if (spinCount > 1000000) + { + Thread.Sleep( + 1); // This should rarely (if ever) occur as the entire critical section is just a couple of assignments + } + else if (spinCount > 0) + { + while (spinCount < spinCount + 1000) spinCount++; // spin 1000 + } + + spinCount++; + + if (!TryAcquire(ref deleteNode.siblings)) + { + continue; + } + + prevNode = deleteNode.siblings.prev; + if (!ReferenceEquals(null, prevNode) && !TryAcquire(ref prevNode.siblings)) + { + Release(ref deleteNode.siblings); + continue; + } + + nextNode = deleteNode.siblings.next; + if (!ReferenceEquals(null, nextNode) && !TryAcquire(ref nextNode.siblings)) + { + Release(ref deleteNode.siblings); + Release(ref prevNode.siblings); + continue; + } + + // Acquired all, so just return + return; + } + } + + static void ReleaseMultiLock( + in ConcurrentKTreeNode node1, + in ConcurrentKTreeNode node2, + in ConcurrentKTreeNode node3 + ) + { + bool node1Exists = !ReferenceEquals(null, node1); + bool node2Exists = !ReferenceEquals(null, node2); + bool node3Exists = !ReferenceEquals(null, node3); + if (node1Exists) Release(ref node1.siblings); + if (node2Exists) Release(ref node2.siblings); + if (node3Exists) Release(ref node3.siblings); + } + } + } + + private static int getRemainingMs(in long startTime, in int timeoutMs) + { + return timeoutMs < 0 + ? -1 + : Math.Max(0, timeoutMs - ((int)(DateTimeOffset.Now.ToUnixTimeMilliseconds() - startTime))); + } + } + +#nullable restore +} \ No newline at end of file diff --git a/core/State/Cache/Internal/ICacheEntry.cs b/core/State/Cache/Internal/ICacheEntry.cs index 07bbc6a0..555b1889 100644 --- a/core/State/Cache/Internal/ICacheEntry.cs +++ b/core/State/Cache/Internal/ICacheEntry.cs @@ -13,7 +13,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// When Disposed, is committed to the cache. /// internal interface ICacheEntry : IDisposable - where K : class + where K : IComparable where V : class { /// diff --git a/core/State/Cache/Internal/IMemoryCache.cs b/core/State/Cache/Internal/IMemoryCache.cs index 2a2c172e..99a399c2 100644 --- a/core/State/Cache/Internal/IMemoryCache.cs +++ b/core/State/Cache/Internal/IMemoryCache.cs @@ -11,7 +11,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// Represents a local in-memory cache whose values are not serialized. /// internal interface IMemoryCache : IDisposable - where K : class + where K : IComparable where V : class { /// diff --git a/core/State/Cache/Internal/MemoryCache.cs b/core/State/Cache/Internal/MemoryCache.cs index 69e678f8..d61b7b46 100644 --- a/core/State/Cache/Internal/MemoryCache.cs +++ b/core/State/Cache/Internal/MemoryCache.cs @@ -19,7 +19,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// store its entries. /// internal sealed class MemoryCache : IMemoryCache - where K : class + where K : IComparable where V : class { private readonly IComparer _keyComparer; @@ -95,12 +95,12 @@ private MemoryCache( /// /// Gets the count of the current entries for diagnostic purposes. /// - public int Count => _coherentState.Count; + public long Count => _coherentState.Count; /// /// Gets an enumerable of the all the keys in the . /// - private IEnumerable Keys => _coherentState.Entries.Keys; + private IEnumerable Keys => _coherentState.EntriesBis.Keys.ToList(); internal IEnumerable KeySetEnumerable(bool forward) { @@ -116,21 +116,22 @@ internal IEnumerable KeyRange(K from, K to, bool inclusive, bool forward) IEnumerable selectedKeys; if (from == null) - selectedKeys = _coherentState.Entries + selectedKeys = _coherentState.EntriesBis .HeadMap(to, inclusive) .Select(kv => kv.Key); else if (to == null) - selectedKeys = _coherentState.Entries + selectedKeys = _coherentState.EntriesBis .TailMap(from, true) .Select(kv => kv.Key); else if (_keyComparer.Compare(from, to) > 0) selectedKeys = new List(); else - selectedKeys = _coherentState.Entries + selectedKeys = _coherentState.EntriesBis .SubMap(from, to, true, inclusive) .Select(kv => kv.Key); - return forward ? selectedKeys : selectedKeys.OrderByDescending(k => k, _keyComparer); + selectedKeys= forward ? selectedKeys : selectedKeys.OrderByDescending(k => k, _keyComparer); + return selectedKeys.ToList(); } /// @@ -164,7 +165,7 @@ internal void SetEntry(CacheEntry entry) entry.LastAccessed = utcNow; CoherentState coherentState = _coherentState; // Clear() can update the reference in the meantime - if (coherentState.Entries.TryGetValue(entry.Key, out CacheEntry priorEntry)) + if (coherentState.EntriesBis.TryGetValue(entry.Key, out CacheEntry priorEntry)) priorEntry.SetExpired(EvictionReason.Replaced); if (!UpdateCacheSizeExceedsCapacity(entry, coherentState)) @@ -172,12 +173,12 @@ internal void SetEntry(CacheEntry entry) if (priorEntry == null) { // Try to add the new entry if no previous entries exist. - coherentState.Entries.TryAdd(entry.Key, entry); + coherentState.EntriesBis.TryAdd(entry.Key, entry); } else { // Try to update with the new entry if a previous entries exist. - coherentState.Entries.AddOrUpdate(entry.Key, entry); + coherentState.EntriesBis.AddOrUpdate(entry.Key, entry); // The prior entry was removed, decrease the by the prior entry's size Interlocked.Add(ref coherentState.CacheSize, -priorEntry.Size); } @@ -187,7 +188,7 @@ internal void SetEntry(CacheEntry entry) else { TriggerOvercapacityCompaction(); - coherentState.Entries.TryAdd(entry.Key, entry); + coherentState.EntriesBis.TryAdd(entry.Key, entry); Interlocked.Add(ref coherentState.CacheSize, entry.Size); } } @@ -203,7 +204,7 @@ public bool TryGetValue(K key, out V value) DateTime utcNow = UtcNow; CoherentState coherentState = _coherentState; // Clear() can update the reference in the meantime - if (coherentState.Entries.TryGetValue(key, out CacheEntry tmp)) + if (coherentState.EntriesBis.TryGetValue(key, out CacheEntry tmp)) { CacheEntry entry = tmp; entry.LastAccessed = utcNow; @@ -249,7 +250,8 @@ public void Remove(K key) CheckDisposed(); CoherentState coherentState = _coherentState; // Clear() can update the reference in the meantime - if (coherentState.Entries.Remove(key, out CacheEntry entry)) + if (coherentState.EntriesBis.TryGetValue(key, out CacheEntry entry) && + coherentState.EntriesBis.TryRemove(key)) { Interlocked.Add(ref coherentState.CacheSize, -entry.Size); entry.SetExpired(EvictionReason.Removed); @@ -265,7 +267,7 @@ public void Clear() CheckDisposed(); CoherentState oldState = Interlocked.Exchange(ref _coherentState, new CoherentState()); - foreach (KeyValuePair> entry in oldState.Entries) + foreach (KeyValuePair> entry in oldState.EntriesBis) { entry.Value.SetExpired(EvictionReason.Removed); entry.Value.InvokeEvictionCallbacks(); @@ -438,13 +440,10 @@ public void Compact(double percentage) private void Flush() { var entriesToRemove = new List>(); - using var enumerator = _coherentState - .Entries - .OrderBy(e => e.Value.LastAccessed) - .GetEnumerator(); - while (enumerator.MoveNext()) - entriesToRemove.Add(enumerator.Current.Value); + foreach(var entry in + _coherentState.EntriesBis.OrderBy(e => e.Value.LastAccessed)) + entriesToRemove.Add(entry.Value); foreach (CacheEntry entry in entriesToRemove) _coherentState.RemoveEntry(entry, _options); @@ -455,13 +454,13 @@ private void Compact(long removalSizeTarget, Func, long> comput var entriesToRemove = new List>(); long removedSize = 0; - ExpireLruBucket(ref removedSize, removalSizeTarget, computeEntrySize, entriesToRemove, coherentState.Entries); + ExpireLruBucket(ref removedSize, removalSizeTarget, computeEntrySize, entriesToRemove, coherentState.EntriesBis); foreach (CacheEntry entry in entriesToRemove) coherentState.RemoveEntry(entry, _options); // Expire the least recently used objects. - static void ExpireLruBucket(ref long removedSize, long removalSizeTarget, Func, long> computeEntrySize, List> entriesToRemove, SortedDictionary> priorityEntries) + static void ExpireLruBucket(ref long removedSize, long removalSizeTarget, Func, long> computeEntrySize, List> entriesToRemove, ConcurrentSortedDictionary> priorityEntries) { // Do we meet our quota by just removing expired entries? if (removalSizeTarget <= removedSize) @@ -541,22 +540,21 @@ private static void ValidateCacheKey(object key) /// Clearing the cache simply replaces the object, so that any still in progress updates do not affect the overall size value for /// the new backing collection. /// - private sealed class CoherentState - where K : class + private sealed class CoherentState + where K : IComparable where V : class { - internal readonly SortedDictionary> Entries = new(); + //internal readonly SortedDictionary> EntriesBis = new(); + internal readonly ConcurrentSortedDictionary> EntriesBis = new(); internal long CacheSize; - - private ICollection>> EntriesCollection => Entries; - - internal int Count => Entries.Count; + + internal long Count => EntriesBis.Count; internal long Size => Volatile.Read(ref CacheSize); internal void RemoveEntry(CacheEntry entry, MemoryCacheOptions options) { - if (EntriesCollection.Remove(new KeyValuePair>(entry.Key, entry))) + if (EntriesBis.TryRemove(entry.Key)) { Interlocked.Add(ref CacheSize, -entry.Size); entry.InvokeEvictionCallbacks(); diff --git a/core/State/Cache/Internal/MemoryCacheEntryExtensions.cs b/core/State/Cache/Internal/MemoryCacheEntryExtensions.cs index d84f3d7b..d6240b0a 100644 --- a/core/State/Cache/Internal/MemoryCacheEntryExtensions.cs +++ b/core/State/Cache/Internal/MemoryCacheEntryExtensions.cs @@ -21,7 +21,7 @@ internal static class MemoryCacheEntryExtensions internal static MemoryCacheEntryOptions SetSize( this MemoryCacheEntryOptions options, long size) - where K : class + where K : IComparable where V : class { if (size < 0) @@ -44,7 +44,7 @@ internal static MemoryCacheEntryOptions RegisterPostEvictionCallback this MemoryCacheEntryOptions options, PostEvictionDelegate callback, MemoryCache state) - where K : class + where K : IComparable where V : class { options.PostEvictionCallbacks.Add(new PostEvictionCallbackRegistration() diff --git a/core/State/Cache/Internal/MemoryCacheEntryOptions.cs b/core/State/Cache/Internal/MemoryCacheEntryOptions.cs index cab859fe..f7fe8f5d 100644 --- a/core/State/Cache/Internal/MemoryCacheEntryOptions.cs +++ b/core/State/Cache/Internal/MemoryCacheEntryOptions.cs @@ -13,7 +13,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// Represents the cache options applied to an entry of the instance. /// internal class MemoryCacheEntryOptions - where K : class + where K : IComparable where V : class { private long? _size; diff --git a/core/State/Cache/Internal/MemoryCacheExtensions.cs b/core/State/Cache/Internal/MemoryCacheExtensions.cs index 5b2ad3a8..d9db090e 100644 --- a/core/State/Cache/Internal/MemoryCacheExtensions.cs +++ b/core/State/Cache/Internal/MemoryCacheExtensions.cs @@ -19,7 +19,7 @@ internal static class CacheExtensions /// The key of the value to get. /// The value associated with this key, or null if the key is not present. internal static V Get(this IMemoryCache cache, K key) - where K : class + where K : IComparable where V : class { cache.TryGetValue(key, out V value); @@ -34,7 +34,7 @@ internal static V Get(this IMemoryCache cache, K key) /// The value associated with the given key. /// true if the key was found. false otherwise. internal static bool TryGetValue(this IMemoryCache cache, K key, out V value) - where K : class + where K : IComparable where V : class { if (cache.TryGetValue(key, out V result)) @@ -64,7 +64,7 @@ internal static bool TryGetValue(this IMemoryCache cache, K key, out /// The value to associate with the key. /// The value that was set. internal static V Set(this IMemoryCache cache, K key, V value) - where K : class + where K : IComparable where V : class { using ICacheEntry entry = cache.CreateEntry(key); @@ -84,7 +84,7 @@ internal static V Set(this IMemoryCache cache, K key, V value) /// Initial eviction reason /// The value that was set. internal static V Set(this IMemoryCache cache, K key, V value, MemoryCacheEntryOptions options, EvictionReason reason) - where K : class + where K : IComparable where V : class { using ICacheEntry entry = cache.CreateEntry(key); @@ -107,7 +107,7 @@ internal static V Set(this IMemoryCache cache, K key, V value, Memor /// The factory that creates the value associated with this key if the key does not exist in the cache. /// The value associated with this key. internal static V GetOrCreate(this IMemoryCache cache, K key, Func, V> factory) - where K : class + where K : IComparable where V : class { return GetOrCreate(cache, key, factory, null); @@ -121,7 +121,7 @@ internal static V GetOrCreate(this IMemoryCache cache, K key, FuncThe factory that creates the value associated with this key if the key does not exist in the cache. /// The options to be applied to the if the key does not exist in the cache. internal static V GetOrCreate(this IMemoryCache cache, K key, Func, V> factory, MemoryCacheEntryOptions? createOptions) - where K : class + where K : IComparable where V : class { if (!cache.TryGetValue(key, out V result)) diff --git a/core/State/Cache/Internal/PostEvictionCallbackRegistration.cs b/core/State/Cache/Internal/PostEvictionCallbackRegistration.cs index 77f33a4d..d426a2cf 100644 --- a/core/State/Cache/Internal/PostEvictionCallbackRegistration.cs +++ b/core/State/Cache/Internal/PostEvictionCallbackRegistration.cs @@ -3,13 +3,15 @@ // This is a fork from Microsoft.Extensions.Caching.Memory.MemoryCache https://github.com/dotnet/runtime/blob/main/src/libraries/Microsoft.Extensions.Caching.Memory // The only difference is the compaction process and eviction callback is synchronous whereas the .NET repo is asyncrhonous +using System; + namespace Streamiz.Kafka.Net.State.Cache.Internal { /// /// Represents a callback delegate that will be fired after an entry is evicted from the cache. /// internal class PostEvictionCallbackRegistration - where K : class + where K : IComparable where V : class { /// diff --git a/core/State/Cache/Internal/PostEvictionDelegate.cs b/core/State/Cache/Internal/PostEvictionDelegate.cs index 86f7c8b7..920948ea 100644 --- a/core/State/Cache/Internal/PostEvictionDelegate.cs +++ b/core/State/Cache/Internal/PostEvictionDelegate.cs @@ -3,6 +3,8 @@ // This is a fork from Microsoft.Extensions.Caching.Memory.MemoryCache https://github.com/dotnet/runtime/blob/main/src/libraries/Microsoft.Extensions.Caching.Memory // The only difference is the compaction process and eviction callback is synchronous whereas the .NET repo is asyncrhonous +using System; + namespace Streamiz.Kafka.Net.State.Cache.Internal { /// @@ -13,6 +15,6 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// The . /// The information that was passed when registering the callback. internal delegate void PostEvictionDelegate(K key, V? value, EvictionReason reason, MemoryCache state) - where K : class + where K : IComparable where V : class; } diff --git a/environment/datagen_connector.json b/environment/datagen_connector.json index 66abd216..048e6214 100644 --- a/environment/datagen_connector.json +++ b/environment/datagen_connector.json @@ -7,7 +7,7 @@ "key.converter": "org.apache.kafka.connect.storage.StringConverter", "value.converter": "org.apache.kafka.connect.json.JsonConverter", "value.converter.schemas.enable": "false", - "max.interval": 1000, + "max.interval": 50, "iterations": 10000000, "tasks.max": "1" } diff --git a/samples/sample-stream/Reproducer314.cs b/samples/sample-stream/Reproducer314.cs index 44230759..4c3e9112 100644 --- a/samples/sample-stream/Reproducer314.cs +++ b/samples/sample-stream/Reproducer314.cs @@ -1,8 +1,10 @@ using System; +using System.Linq; using System.Threading; using System.Threading.Tasks; using Confluent.Kafka; using Streamiz.Kafka.Net; +using Streamiz.Kafka.Net.Errors; using Streamiz.Kafka.Net.SerDes; using Streamiz.Kafka.Net.State; using Streamiz.Kafka.Net.Stream; @@ -23,52 +25,68 @@ public static async Task Main(string[] args) AutoOffsetReset = AutoOffsetReset.Earliest }; - var builder = CreateWindowedStore(); + var builder = CreateKVStore(); var t = builder.Build(); var windowedTableStream = new KafkaStream(t, config); + Console.CancelKeyPress += (_,_) => { + windowedTableStream.Dispose(); + }; + await windowedTableStream.StartAsync(); // wait for the store to be restored and ready Thread.Sleep(10000); - GetValueFromWindowedStore(windowedTableStream, DateTime.UtcNow.AddHours(-1), new CancellationToken()); - + GetValueFromKVStore(windowedTableStream, DateTime.UtcNow.AddHours(-1), new CancellationToken()); + Console.WriteLine("Finished"); } - private static void GetValueFromWindowedStore(KafkaStream windowedTableStream, DateTime startUtcForWindowLookup, CancellationToken cancellationToken) + private static void GetValueFromKVStore(KafkaStream windowedTableStream, DateTime startUtcForWindowLookup, CancellationToken cancellationToken) { - var windowedStore = windowedTableStream.Store(StoreQueryParameters.FromNameAndType("store", QueryableStoreTypes.WindowStore())); + IReadOnlyKeyValueStore windowedStore = null; + while (windowedStore == null) + { + try + { + windowedStore = + windowedTableStream.Store(StoreQueryParameters.FromNameAndType("store", + QueryableStoreTypes.KeyValueStore())); + + } + catch (InvalidStateStoreException e) + { + } + } while (!cancellationToken.IsCancellationRequested) { - var records = windowedStore.FetchAll(startUtcForWindowLookup, DateTime.UtcNow).ToList(); + var records = windowedStore.All(); - if (records.Count > 0) + if (records.Any()) { foreach (var item in records) { Console.WriteLine($"Value from windowed store : KEY = {item.Key} VALUE = {item.Value}"); } - - startUtcForWindowLookup = DateTime.UtcNow; } } } - private static StreamBuilder CreateWindowedStore() + private static StreamBuilder CreateKVStore() { var builder = new StreamBuilder(); builder .Stream("users") .GroupByKey() - .WindowedBy(TumblingWindowOptions.Of(60000)) .Aggregate( () => 0, (k, v, agg) => Math.Max(v.Length, agg), - InMemoryWindows.As("store").WithValueSerdes()); + InMemory.As("store") + .WithValueSerdes() + .WithCachingEnabled()); return builder; } diff --git a/test/Streamiz.Kafka.Net.Tests/Private/ConcurrentSortedDictionaryTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/ConcurrentSortedDictionaryTests.cs new file mode 100644 index 00000000..c7232258 --- /dev/null +++ b/test/Streamiz.Kafka.Net.Tests/Private/ConcurrentSortedDictionaryTests.cs @@ -0,0 +1,23 @@ +using NUnit.Framework; +using Streamiz.Kafka.Net.Crosscutting; +using Streamiz.Kafka.Net.State.Cache.Internal; + +namespace Streamiz.Kafka.Net.Tests.Private; + +// TODO : +public class ConcurrentSortedDictionaryTests +{ + private ConcurrentSortedDictionary concurrentSet; + + [SetUp] + public void Init() + { + concurrentSet = new ConcurrentSortedDictionary(); + } + + [TearDown] + public void Dispose() + { + concurrentSet.Clear(); + } +} \ No newline at end of file From 07dbfe493e251078347f2dc4848e543bc183b80e Mon Sep 17 00:00:00 2001 From: LGouellec Date: Tue, 16 Jul 2024 11:19:01 -0700 Subject: [PATCH 23/29] Change openIterator set type --- core/State/InMemory/InMemoryWindowStore.cs | 1 + core/State/{InMemory => }/Internal/ConcurrentHashSet.cs | 2 +- core/State/RocksDb/RocksDbKeyValueStore.cs | 4 ++-- 3 files changed, 4 insertions(+), 3 deletions(-) rename core/State/{InMemory => }/Internal/ConcurrentHashSet.cs (97%) diff --git a/core/State/InMemory/InMemoryWindowStore.cs b/core/State/InMemory/InMemoryWindowStore.cs index fb981db5..f5570b59 100644 --- a/core/State/InMemory/InMemoryWindowStore.cs +++ b/core/State/InMemory/InMemoryWindowStore.cs @@ -13,6 +13,7 @@ using Streamiz.Kafka.Net.Metrics.Internal; using Streamiz.Kafka.Net.State.Helper; using Streamiz.Kafka.Net.State.InMemory.Internal; +using Streamiz.Kafka.Net.State.Internal; namespace Streamiz.Kafka.Net.State.InMemory { diff --git a/core/State/InMemory/Internal/ConcurrentHashSet.cs b/core/State/Internal/ConcurrentHashSet.cs similarity index 97% rename from core/State/InMemory/Internal/ConcurrentHashSet.cs rename to core/State/Internal/ConcurrentHashSet.cs index e898b396..e9a25aa0 100644 --- a/core/State/InMemory/Internal/ConcurrentHashSet.cs +++ b/core/State/Internal/ConcurrentHashSet.cs @@ -1,7 +1,7 @@ using System.Collections.Concurrent; using System.Collections.Generic; -namespace Streamiz.Kafka.Net.State.InMemory.Internal +namespace Streamiz.Kafka.Net.State.Internal { internal class ConcurrentSet { diff --git a/core/State/RocksDb/RocksDbKeyValueStore.cs b/core/State/RocksDb/RocksDbKeyValueStore.cs index cab1a2f2..79636141 100644 --- a/core/State/RocksDb/RocksDbKeyValueStore.cs +++ b/core/State/RocksDb/RocksDbKeyValueStore.cs @@ -8,6 +8,7 @@ using System.Collections.Generic; using System.IO; using Microsoft.Extensions.Logging; +using Streamiz.Kafka.Net.State.Internal; namespace Streamiz.Kafka.Net.State { @@ -61,8 +62,7 @@ public class RocksDbKeyValueStore : IKeyValueStore { private static readonly ILogger log = Logger.GetLogger(typeof(RocksDbKeyValueStore)); - private readonly ISet openIterators - = new HashSet(); + private readonly ConcurrentSet openIterators = new(); private const Compression COMPRESSION_TYPE = Compression.No; From fbb64328718df2ebd16dd886a2676a4d5242ad18 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Wed, 17 Jul 2024 16:02:15 -0700 Subject: [PATCH 24/29] Fix self relationship exception + avoid multiple full metadata call --- core/Crosscutting/DictionaryExtensions.cs | 19 +++++ core/KafkaStream.cs | 12 ++- .../Internal/InternalTopicManagerUtils.cs | 63 +++++++------- .../Internal/InternalTopologyBuilder.cs | 6 +- core/StreamConfig.cs | 12 ++- environment/docker-compose.yml | 2 + samples/sample-stream/Program.cs | 4 +- samples/sample-stream/Reproducer314.cs | 75 ----------------- .../Base/JoinSlidingWindowOptions.cs | 8 +- .../Base/KStreamKStreamWindowedTestsBase.cs | 30 +++---- ...StreamKStreamWindowedFullOuterJoinTests.cs | 28 +++---- .../KStreamKStreamWindowedInnerJoinTests.cs | 14 ++-- .../KStreamKStreamWindowedLeftJoinTests.cs | 14 ++-- .../Reproducer/FIxIssue319Tests.cs | 84 +++++++++++++++++++ .../{ => Reproducer}/FixIssue158Tests.cs | 11 +-- .../{ => Reproducer}/FixIssue221Tests.cs | 4 +- .../{ => Reproducer}/Reproducer229Tests.cs | 2 +- .../{ => Reproducer}/Reproducer275Tests.cs | 2 +- 18 files changed, 212 insertions(+), 178 deletions(-) delete mode 100644 samples/sample-stream/Reproducer314.cs create mode 100644 test/Streamiz.Kafka.Net.Tests/Reproducer/FIxIssue319Tests.cs rename test/Streamiz.Kafka.Net.Tests/{ => Reproducer}/FixIssue158Tests.cs (92%) rename test/Streamiz.Kafka.Net.Tests/{ => Reproducer}/FixIssue221Tests.cs (97%) rename test/Streamiz.Kafka.Net.Tests/{ => Reproducer}/Reproducer229Tests.cs (99%) rename test/Streamiz.Kafka.Net.Tests/{ => Reproducer}/Reproducer275Tests.cs (99%) diff --git a/core/Crosscutting/DictionaryExtensions.cs b/core/Crosscutting/DictionaryExtensions.cs index 6a3cd1f1..10bb6913 100644 --- a/core/Crosscutting/DictionaryExtensions.cs +++ b/core/Crosscutting/DictionaryExtensions.cs @@ -32,6 +32,25 @@ public static bool AddOrUpdate(this IDictionary map, K key, V value) return true; } + /// + /// Add the element if the key doesn't exist + /// + /// Key type + /// Value type + /// Source dictionary + /// New key + /// Value + /// Return true if the key|value was added, false otherwise + public static bool AddIfNotExist(this IDictionary map, K key, V value) + { + if (!map.ContainsKey(key)) + { + map.Add(key, value); + return true; + } + return false; + } + /// /// Convert enumerable of to /// diff --git a/core/KafkaStream.cs b/core/KafkaStream.cs index babef907..2958963a 100644 --- a/core/KafkaStream.cs +++ b/core/KafkaStream.cs @@ -430,7 +430,8 @@ public async Task StartAsync(CancellationToken? token = null) Dispose(); }); } - await Task.Factory.StartNew(async () => + + await Task.Factory.StartNew(() => { if (SetState(State.REBALANCING)) { @@ -448,7 +449,8 @@ await Task.Factory.StartNew(async () => SetState(State.PENDING_SHUTDOWN); SetState(State.ERROR); } - return; + + return Task.CompletedTask; } RunMiddleware(true, true); @@ -463,6 +465,8 @@ await Task.Factory.StartNew(async () => RunMiddleware(false, true); } + + return Task.CompletedTask; }, token ?? _cancelSource.Token); @@ -624,8 +628,8 @@ private async Task InitializeInternalTopicManagerAsync() { // Create internal topics (changelogs & repartition) if need var adminClientInternalTopicManager = kafkaSupplier.GetAdmin(configuration.ToAdminConfig(StreamThread.GetSharedAdminClientId($"{configuration.ApplicationId.ToLower()}-admin-internal-topic-manager"))); - using(var internalTopicManager = new DefaultTopicManager(configuration, adminClientInternalTopicManager)) - await InternalTopicManagerUtils.New().CreateInternalTopicsAsync(internalTopicManager, topology.Builder); + using var internalTopicManager = new DefaultTopicManager(configuration, adminClientInternalTopicManager); + await InternalTopicManagerUtils.New().CreateInternalTopicsAsync(internalTopicManager, topology.Builder); } private void RunMiddleware(bool before, bool start) diff --git a/core/Processors/Internal/InternalTopicManagerUtils.cs b/core/Processors/Internal/InternalTopicManagerUtils.cs index 2f450ccb..dc66b707 100644 --- a/core/Processors/Internal/InternalTopicManagerUtils.cs +++ b/core/Processors/Internal/InternalTopicManagerUtils.cs @@ -25,75 +25,74 @@ static InternalTopicManagerUtils() }; } - internal static InternalTopicManagerUtils New() => new InternalTopicManagerUtils(); + internal static InternalTopicManagerUtils New() => new(); internal async Task CreateInternalTopicsAsync( ITopicManager topicManager, InternalTopologyBuilder builder) { - var clusterMetadata = topicManager.AdminClient.GetMetadata(timeout); + //var clusterMetadata = topicManager.AdminClient.GetMetadata(timeout); var internalTopicsGroups = builder.MakeInternalTopicGroups(); - - var resultsConf = new List(); - if (internalTopicsGroups.Any(internalTopic => - !clusterMetadata.Topics.Exists(t => t.Topic.Equals(internalTopic.Value.SourceTopics.First())))) - { - brokerConfigResource.Name = clusterMetadata.Brokers[0].BrokerId.ToString(); - resultsConf = await topicManager.AdminClient.DescribeConfigsAsync(new List { brokerConfigResource }); - } - + foreach (var entry in internalTopicsGroups) { - ComputeRepartitionTopicConfig(entry.Value, internalTopicsGroups, clusterMetadata); - ComputeChangelogTopicConfig(entry.Value, clusterMetadata, resultsConf); + ComputeRepartitionTopicConfig(entry.Value, internalTopicsGroups, topicManager); + ComputeChangelogTopicConfig(entry.Value, topicManager); var internalTopics = entry.Value.ChangelogTopics.Union(entry.Value.RepartitionTopics).ToDictionary(); await topicManager.ApplyAsync(entry.Key, internalTopics); - // refresh metadata - clusterMetadata = topicManager.AdminClient.GetMetadata(timeout); } } - private static void ComputeChangelogTopicConfig(InternalTopologyBuilder.TopologyTopicsInfo topicsInfo, - Metadata clusterMetadata, List resultsConf) + private static void ComputeChangelogTopicConfig( + InternalTopologyBuilder.TopologyTopicsInfo topicsInfo, + ITopicManager topicManager) { - var topic = clusterMetadata.Topics.FirstOrDefault(t => t.Topic.Equals(topicsInfo.SourceTopics.First())); - if (topic != null) + var metadata = + topicManager.AdminClient.GetMetadata(topicsInfo.SourceTopics.First(), TimeSpan.FromSeconds(10)); + var topicMetadata = metadata.Topics.FirstOrDefault(t => t.Topic.Equals(topicsInfo.SourceTopics.First())); + if (topicMetadata != null) { topicsInfo .ChangelogTopics .Values - .ForEach(c => c.NumberPartitions = topic.Partitions.Count); + .ForEach(c => c.NumberPartitions = topicMetadata.Partitions.Count); } else { topicsInfo .ChangelogTopics .Values - .ForEach(c => c.NumberPartitions = DefaultPartitionNumber(resultsConf)); + .ForEach(c => c.NumberPartitions = -1); } } private static void ComputeRepartitionTopicConfig( InternalTopologyBuilder.TopologyTopicsInfo topicsInfo, IDictionary topologyTopicInfos, - Metadata clusterMetadata) + ITopicManager topicManager) { if (topicsInfo.RepartitionTopics.Any()) { - CheckIfExternalSourceTopicsExist(topicsInfo, clusterMetadata); - SetRepartitionSourceTopicPartitionCount(topicsInfo.RepartitionTopics, topologyTopicInfos, - clusterMetadata); + CheckIfExternalSourceTopicsExist(topicsInfo, topicManager); + SetRepartitionSourceTopicPartitionCount(topicsInfo.RepartitionTopics, topologyTopicInfos, topicManager); } } - private static void CheckIfExternalSourceTopicsExist(InternalTopologyBuilder.TopologyTopicsInfo topicsInfo, Metadata clusterMetadata) + private static void CheckIfExternalSourceTopicsExist( + InternalTopologyBuilder.TopologyTopicsInfo topicsInfo, + ITopicManager topicManager) { - List sourcesTopics = new List(topicsInfo.SourceTopics); - sourcesTopics.RemoveAll(topicsInfo.RepartitionTopics.Keys); - sourcesTopics.RemoveAll(clusterMetadata.Topics.Select(t => t.Topic)); - if (sourcesTopics.Any()) + List sourcesTopics = new List(); + foreach (var s in topicsInfo.SourceTopics) + { + var metadata = topicManager.AdminClient.GetMetadata(s, TimeSpan.FromSeconds(10)); + if (metadata.PartitionCountForTopic(s).HasValue) + sourcesTopics.Add(s); + } + + if (!sourcesTopics.Any()) { log.LogError($"Topology use one (or multiple) repartition topic(s)." + $" The following source topics ({string.Join(",", sourcesTopics)}) are missing/unknown." + @@ -105,7 +104,7 @@ private static void CheckIfExternalSourceTopicsExist(InternalTopologyBuilder.Top private static void SetRepartitionSourceTopicPartitionCount( IDictionary repartitionTopics, IDictionary topologyTopicInfos, - Metadata clusterMetadata) + ITopicManager topicManager) { #region Compute topic partition count @@ -126,6 +125,8 @@ private static void SetRepartitionSourceTopicPartitionCount( } else { + var clusterMetadata = + topicManager.AdminClient.GetMetadata(upstreamSourceTopic, TimeSpan.FromSeconds(10)); var count = clusterMetadata.PartitionCountForTopic(upstreamSourceTopic); if (count == null) count = ComputePartitionCount(upstreamSourceTopic); diff --git a/core/Processors/Internal/InternalTopologyBuilder.cs b/core/Processors/Internal/InternalTopologyBuilder.cs index e426295a..7f124be1 100644 --- a/core/Processors/Internal/InternalTopologyBuilder.cs +++ b/core/Processors/Internal/InternalTopologyBuilder.cs @@ -585,11 +585,11 @@ string AddRepartitionTopic(string topic) int? internalTopicPartition = internalTopics[topic]; if (internalTopicPartition.HasValue) - repartitionTopics.Add(internalTopic, + repartitionTopics.AddIfNotExist(internalTopic, new RepartitionTopicConfig() {Name = internalTopic, NumberPartitions = internalTopicPartition.Value}); else - repartitionTopics.Add(internalTopic, + repartitionTopics.AddIfNotExist(internalTopic, new RepartitionTopicConfig() {Name = internalTopic}); return internalTopic; } @@ -804,6 +804,8 @@ internal TaskId GetTaskIdFromPartition(TopicPartition topicPartition) // FOR TESTING internal void GetLinkTopics(string topic, IList linkTopics) { + if (linkTopics.Contains(topic)) + return; var subTopo = GetSubTopologyDescription(topic); if (subTopo != null) { diff --git a/core/StreamConfig.cs b/core/StreamConfig.cs index 0e4319a3..11a124b9 100644 --- a/core/StreamConfig.cs +++ b/core/StreamConfig.cs @@ -2003,7 +2003,16 @@ public IsolationLevel? IsolationLevel /// low /// [StreamConfigProperty("allow.auto.create.topics")] - public bool? AllowAutoCreateTopics { get { return _consumerConfig.AllowAutoCreateTopics; } set { _consumerConfig.AllowAutoCreateTopics = value; } } + public bool? AllowAutoCreateTopics + { + get { return _consumerConfig.AllowAutoCreateTopics; } + set + { + _consumerConfig.AllowAutoCreateTopics = value; + _adminClientConfig.AllowAutoCreateTopics = value; + _producerConfig.AllowAutoCreateTopics = value; + } + } #endregion @@ -2297,6 +2306,7 @@ public StreamConfig(IDictionary properties) MaxPollIntervalMs = 300000; EnableAutoCommit = false; EnableAutoOffsetStore = false; + AllowAutoCreateTopics = false; PartitionAssignmentStrategy = Confluent.Kafka.PartitionAssignmentStrategy.CooperativeSticky; Partitioner = Confluent.Kafka.Partitioner.Murmur2Random; diff --git a/environment/docker-compose.yml b/environment/docker-compose.yml index 49fb3e24..f9227850 100644 --- a/environment/docker-compose.yml +++ b/environment/docker-compose.yml @@ -28,6 +28,7 @@ services: KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://broker-1:29092,PLAINTEXT_HOST://localhost:9092 KAFKA_METRIC_REPORTERS: io.confluent.metrics.reporter.ConfluentMetricsReporter KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 2 + KAFKA_ALLOW_AUTO_CREATE_TOPICS: false KAFKA_DEFAULT_REPLICATION_FACTOR: 2 KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS: 0 KAFKA_CONFLUENT_LICENSE_TOPIC_REPLICATION_FACTOR: 2 @@ -59,6 +60,7 @@ services: KAFKA_METRIC_REPORTERS: io.confluent.metrics.reporter.ConfluentMetricsReporter KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 2 KAFKA_DEFAULT_REPLICATION_FACTOR: 2 + KAFKA_ALLOW_AUTO_CREATE_TOPICS: false KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS: 0 KAFKA_CONFLUENT_LICENSE_TOPIC_REPLICATION_FACTOR: 2 KAFKA_CONFLUENT_BALANCER_TOPIC_REPLICATION_FACTOR: 2 diff --git a/samples/sample-stream/Program.cs b/samples/sample-stream/Program.cs index b1af4a78..74672a21 100644 --- a/samples/sample-stream/Program.cs +++ b/samples/sample-stream/Program.cs @@ -2,19 +2,17 @@ using Streamiz.Kafka.Net; using Streamiz.Kafka.Net.SerDes; using System; -using System.Collections.Generic; using System.Threading.Tasks; using Streamiz.Kafka.Net.Metrics; using Streamiz.Kafka.Net.Metrics.Prometheus; using Streamiz.Kafka.Net.Stream; using Microsoft.Extensions.Logging; -using Streamiz.Kafka.Net.Table; namespace sample_stream { public static class Program { - public static async Task Main2(string[] args) + public static async Task Main(string[] args) { var config = new StreamConfig{ ApplicationId = $"test-app", diff --git a/samples/sample-stream/Reproducer314.cs b/samples/sample-stream/Reproducer314.cs deleted file mode 100644 index 44230759..00000000 --- a/samples/sample-stream/Reproducer314.cs +++ /dev/null @@ -1,75 +0,0 @@ -using System; -using System.Threading; -using System.Threading.Tasks; -using Confluent.Kafka; -using Streamiz.Kafka.Net; -using Streamiz.Kafka.Net.SerDes; -using Streamiz.Kafka.Net.State; -using Streamiz.Kafka.Net.Stream; -using Streamiz.Kafka.Net.Table; - -namespace sample_stream; - -public class Reproducer314 -{ - public static async Task Main(string[] args) - { - Console.WriteLine("Hello Streams"); - - var config = new StreamConfig - { - ApplicationId = $"test-windowedtable-bis", - BootstrapServers = "localhost:9092", - AutoOffsetReset = AutoOffsetReset.Earliest - }; - - var builder = CreateWindowedStore(); - var t = builder.Build(); - var windowedTableStream = new KafkaStream(t, config); - - await windowedTableStream.StartAsync(); - - // wait for the store to be restored and ready - Thread.Sleep(10000); - - GetValueFromWindowedStore(windowedTableStream, DateTime.UtcNow.AddHours(-1), new CancellationToken()); - - Console.WriteLine("Finished"); - } - - private static void GetValueFromWindowedStore(KafkaStream windowedTableStream, DateTime startUtcForWindowLookup, CancellationToken cancellationToken) - { - var windowedStore = windowedTableStream.Store(StoreQueryParameters.FromNameAndType("store", QueryableStoreTypes.WindowStore())); - - while (!cancellationToken.IsCancellationRequested) - { - var records = windowedStore.FetchAll(startUtcForWindowLookup, DateTime.UtcNow).ToList(); - - if (records.Count > 0) - { - foreach (var item in records) - { - Console.WriteLine($"Value from windowed store : KEY = {item.Key} VALUE = {item.Value}"); - } - - startUtcForWindowLookup = DateTime.UtcNow; - } - } - } - - private static StreamBuilder CreateWindowedStore() - { - var builder = new StreamBuilder(); - - builder - .Stream("users") - .GroupByKey() - .WindowedBy(TumblingWindowOptions.Of(60000)) - .Aggregate( - () => 0, - (k, v, agg) => Math.Max(v.Length, agg), - InMemoryWindows.As("store").WithValueSerdes()); - - return builder; - } -} \ No newline at end of file diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/Base/JoinSlidingWindowOptions.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/Base/JoinSlidingWindowOptions.cs index ebee61c2..e9ee0ba2 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/Base/JoinSlidingWindowOptions.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/Base/JoinSlidingWindowOptions.cs @@ -1,12 +1,12 @@ -using Streamiz.Kafka.Net.Stream; +using Streamiz.Kafka.Net.Stream; -namespace Streamiz.Kafka.Net.Tests.Processors -{ +namespace Streamiz.Kafka.Net.Tests.Processors +{ internal class JoinSlidingWindowOptions : JoinWindowOptions { public JoinSlidingWindowOptions(long beforeMs, long afterMs, long graceMs, long maintainDurationMs) : base(beforeMs, afterMs, graceMs, maintainDurationMs) { } - } + } } \ No newline at end of file diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/Base/KStreamKStreamWindowedTestsBase.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/Base/KStreamKStreamWindowedTestsBase.cs index 22fa5836..701261fa 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/Base/KStreamKStreamWindowedTestsBase.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/Base/KStreamKStreamWindowedTestsBase.cs @@ -1,11 +1,11 @@ using NUnit.Framework; using Streamiz.Kafka.Net.Mock; using Streamiz.Kafka.Net.SerDes; -using Streamiz.Kafka.Net.Stream; -using System; +using Streamiz.Kafka.Net.Stream; +using System; -namespace Streamiz.Kafka.Net.Tests.Processors -{ +namespace Streamiz.Kafka.Net.Tests.Processors +{ public abstract class KStreamKStreamWindowedTestsBase { private TopologyTestDriver driver; @@ -21,20 +21,20 @@ public abstract class KStreamKStreamWindowedTestsBase [SetUp] public void Setup() { - var config = new StreamConfig - { - ApplicationId = $"test-stream-stream-{GetType().Name}" - }; - - StreamBuilder builder = new StreamBuilder(); - - var stream1 = builder.Stream("topic1"); - var stream2 = builder.Stream("topic2"); + var config = new StreamConfig + { + ApplicationId = $"test-stream-stream-{GetType().Name}" + }; + + StreamBuilder builder = new StreamBuilder(); + + var stream1 = builder.Stream("topic1"); + var stream2 = builder.Stream("topic2"); var window = CreateJoinWindow(); var joinDelegate = GetJoinDelegate(stream1); - CreateOutputStream(stream2, joinDelegate, window).To("output-join"); - + CreateOutputStream(stream2, joinDelegate, window).To("output-join"); + var topology = builder.Build(); driver = new TopologyTestDriver(topology, config); diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/KStreamKStreamWindowedFullOuterJoinTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/KStreamKStreamWindowedFullOuterJoinTests.cs index 7ff6eb37..7f0ca036 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/KStreamKStreamWindowedFullOuterJoinTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/KStreamKStreamWindowedFullOuterJoinTests.cs @@ -1,9 +1,9 @@ using NUnit.Framework; -using Streamiz.Kafka.Net.Stream; +using Streamiz.Kafka.Net.Stream; using System.Linq; -namespace Streamiz.Kafka.Net.Tests.Processors -{ +namespace Streamiz.Kafka.Net.Tests.Processors +{ public class KStreamKStreamWindowedFullOuterJoinTests : KStreamKStreamWindowedTestsBase { protected override JoinWindowOptions CreateJoinWindow() @@ -12,9 +12,9 @@ protected override JoinWindowOptions CreateJoinWindow() protected override JoinDelegate GetJoinDelegate(IKStream stream1) => stream1.OuterJoin; - [Test] - public void StreamStreamFullOuterJoin() - { + [Test] + public void StreamStreamFullOuterJoin() + { inputTopic2.PipeInput("test", "right-0", TestTime.AddSeconds(0)); inputTopic1.PipeInput("test", "left-1", TestTime.AddSeconds(1)); inputTopic2.PipeInput("test", "right-2", TestTime.AddSeconds(2)); @@ -26,14 +26,14 @@ public void StreamStreamFullOuterJoin() var records = outputTopic.ReadValueList().ToArray(); Assert.AreEqual(8, records.Length); - Assert.That(records[0], Is.EqualTo("-right-0")); + Assert.That(records[0], Is.EqualTo("-right-0")); Assert.That(records[1], Is.EqualTo("left-1-")); - Assert.That(records[2], Is.EqualTo("left-1-right-2")); - Assert.That(records[3], Is.EqualTo("-right-3")); - Assert.That(records[4], Is.EqualTo("left-5.1-")); - Assert.That(records[5], Is.EqualTo("left-5.2-")); - Assert.That(records[6], Is.EqualTo("left-5.1-right-6")); - Assert.That(records[7], Is.EqualTo("left-5.2-right-6")); + Assert.That(records[2], Is.EqualTo("left-1-right-2")); + Assert.That(records[3], Is.EqualTo("-right-3")); + Assert.That(records[4], Is.EqualTo("left-5.1-")); + Assert.That(records[5], Is.EqualTo("left-5.2-")); + Assert.That(records[6], Is.EqualTo("left-5.1-right-6")); + Assert.That(records[7], Is.EqualTo("left-5.2-right-6")); } - } + } } \ No newline at end of file diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/KStreamKStreamWindowedInnerJoinTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/KStreamKStreamWindowedInnerJoinTests.cs index 9911e895..94745929 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/KStreamKStreamWindowedInnerJoinTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/KStreamKStreamWindowedInnerJoinTests.cs @@ -1,9 +1,9 @@ using NUnit.Framework; -using Streamiz.Kafka.Net.Stream; +using Streamiz.Kafka.Net.Stream; using System.Linq; -namespace Streamiz.Kafka.Net.Tests.Processors -{ +namespace Streamiz.Kafka.Net.Tests.Processors +{ public class KStreamKStreamWindowedInnerJoinTests : KStreamKStreamWindowedTestsBase { protected override JoinWindowOptions CreateJoinWindow() @@ -12,9 +12,9 @@ protected override JoinWindowOptions CreateJoinWindow() protected override JoinDelegate GetJoinDelegate(IKStream stream1) => stream1.Join; - [Test] - public void StreamStreamInnerJoin() - { + [Test] + public void StreamStreamInnerJoin() + { inputTopic2.PipeInput("test", "right-0", TestTime.AddSeconds(0)); inputTopic1.PipeInput("test", "left-1", TestTime.AddSeconds(1)); inputTopic2.PipeInput("test", "right-2", TestTime.AddSeconds(2)); @@ -23,7 +23,7 @@ public void StreamStreamInnerJoin() var records = outputTopic.ReadValueList().ToArray(); Assert.AreEqual(1, records.Length); - Assert.That(records[0], Is.EqualTo("left-1-right-2")); + Assert.That(records[0], Is.EqualTo("left-1-right-2")); } } } \ No newline at end of file diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/KStreamKStreamWindowedLeftJoinTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/KStreamKStreamWindowedLeftJoinTests.cs index c43af8e4..cdae7d60 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/KStreamKStreamWindowedLeftJoinTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/KStreamKStreamWindowedLeftJoinTests.cs @@ -1,9 +1,9 @@ using NUnit.Framework; -using Streamiz.Kafka.Net.Stream; +using Streamiz.Kafka.Net.Stream; using System.Linq; -namespace Streamiz.Kafka.Net.Tests.Processors -{ +namespace Streamiz.Kafka.Net.Tests.Processors +{ public class KStreamKStreamWindowedLeftJoinTests : KStreamKStreamWindowedTestsBase { protected override JoinWindowOptions CreateJoinWindow() @@ -12,9 +12,9 @@ protected override JoinWindowOptions CreateJoinWindow() protected override JoinDelegate GetJoinDelegate(IKStream stream1) => stream1.LeftJoin; - [Test] - public void StreamStreamLeftJoin() - { + [Test] + public void StreamStreamLeftJoin() + { inputTopic2.PipeInput("test", "right-0", TestTime.AddSeconds(0)); inputTopic1.PipeInput("test", "left-1", TestTime.AddSeconds(1)); inputTopic2.PipeInput("test", "right-2", TestTime.AddSeconds(2)); @@ -24,7 +24,7 @@ public void StreamStreamLeftJoin() Assert.AreEqual(2, records.Length); Assert.That(records[0], Is.EqualTo("left-1-")); - Assert.That(records[1], Is.EqualTo("left-1-right-2")); + Assert.That(records[1], Is.EqualTo("left-1-right-2")); } } } \ No newline at end of file diff --git a/test/Streamiz.Kafka.Net.Tests/Reproducer/FIxIssue319Tests.cs b/test/Streamiz.Kafka.Net.Tests/Reproducer/FIxIssue319Tests.cs new file mode 100644 index 00000000..ab93747e --- /dev/null +++ b/test/Streamiz.Kafka.Net.Tests/Reproducer/FIxIssue319Tests.cs @@ -0,0 +1,84 @@ +using NUnit.Framework; +using Streamiz.Kafka.Net.Mock; +using Streamiz.Kafka.Net.SerDes; +using Streamiz.Kafka.Net.Table; + +namespace Streamiz.Kafka.Net.Tests.Reproducer; + +public class FIxIssue319Tests +{ + public class SelfId + { + public int Id { get; set; } + } + + public class SelfRelation + { + public int Id { get; set; } + public string Name { get; set; } + public int? Relation { get; set; } + } + + public class Container + { + public SelfRelation Node { get; set; } + public SelfRelation Dependency { get; set; } + } + + [Test] + public void SelfRelationshipTest() + { + var builder = new StreamBuilder(); + + var stream = + builder.Stream("self", new JsonSerDes(), new JsonSerDes()); + + var filtrate = stream.Filter((k, v) => v.Relation.HasValue); + var withRelationKeyStream = filtrate.SelectKey((k, v) => new SelfId { Id = v.Relation!.Value }); + var withRelationKeyTable = withRelationKeyStream.ToTable( + InMemory.As() + .WithKeySerdes(new JsonSerDes()) + .WithValueSerdes(new JsonSerDes())); + + var table = stream.ToTable( + InMemory.As() + .WithKeySerdes(new JsonSerDes()) + .WithValueSerdes(new JsonSerDes())); + + withRelationKeyTable + .Join(table, + (left, right) => new Container { Node = left, Dependency = right }) + .ToStream() + .To("output", new JsonSerDes(), new JsonSerDes()); + + var topology = builder.Build(); + + var config = new StreamConfig(); + config.ApplicationId = "test-issue-319-driver-app"; + + var driver = new TopologyTestDriver(topology, config); + + // create the test input topic + var inputTopic = + driver.CreateInputTopic( + "self", new JsonSerDes(), new JsonSerDes()); + + var outputTopic = + driver.CreateOuputTopic, JsonSerDes>( + "output"); + + inputTopic.PipeInput(new SelfId { Id = 1 }, new SelfRelation { Id = 1, Name = "self", Relation = 2 }); + inputTopic.PipeInput(new SelfId { Id = 2 }, new SelfRelation { Id = 2, Name = "rel" }); + + var output = outputTopic.ReadValue(); + + Assert.AreEqual(1, output.Node.Id); + Assert.AreEqual("self", output.Node.Name); + Assert.IsNotNull(output.Node.Relation); + Assert.AreEqual(2, output.Node.Relation.Value); + Assert.AreEqual(2, output.Dependency.Id); + Assert.AreEqual("rel", output.Dependency.Name); + Assert.IsNull(output.Dependency.Relation); + } + +} \ No newline at end of file diff --git a/test/Streamiz.Kafka.Net.Tests/FixIssue158Tests.cs b/test/Streamiz.Kafka.Net.Tests/Reproducer/FixIssue158Tests.cs similarity index 92% rename from test/Streamiz.Kafka.Net.Tests/FixIssue158Tests.cs rename to test/Streamiz.Kafka.Net.Tests/Reproducer/FixIssue158Tests.cs index f9f774f9..59ec2ff9 100644 --- a/test/Streamiz.Kafka.Net.Tests/FixIssue158Tests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Reproducer/FixIssue158Tests.cs @@ -1,19 +1,10 @@ using System; -using System.Collections.Generic; -using System.Reflection.Metadata.Ecma335; -using Confluent.Kafka; -using Google.Protobuf.Reflection; -using Microsoft.VisualBasic; using NUnit.Framework; -using Streamiz.Kafka.Net.Kafka; -using Streamiz.Kafka.Net.Kafka.Internal; -using Streamiz.Kafka.Net.Metrics; using Streamiz.Kafka.Net.Mock; using Streamiz.Kafka.Net.SerDes; using Streamiz.Kafka.Net.Table; -using Streamiz.Kafka.Net.Tests.Helpers; -namespace Streamiz.Kafka.Net.Tests +namespace Streamiz.Kafka.Net.Tests.Reproducer { public class FixIssue158Tests { diff --git a/test/Streamiz.Kafka.Net.Tests/FixIssue221Tests.cs b/test/Streamiz.Kafka.Net.Tests/Reproducer/FixIssue221Tests.cs similarity index 97% rename from test/Streamiz.Kafka.Net.Tests/FixIssue221Tests.cs rename to test/Streamiz.Kafka.Net.Tests/Reproducer/FixIssue221Tests.cs index eeaeee8d..7a3718da 100644 --- a/test/Streamiz.Kafka.Net.Tests/FixIssue221Tests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Reproducer/FixIssue221Tests.cs @@ -1,14 +1,12 @@ using System; using System.Collections.Generic; -using System.Diagnostics.CodeAnalysis; -using System.Linq; using NUnit.Framework; using Streamiz.Kafka.Net.Mock; using Streamiz.Kafka.Net.SerDes; using Streamiz.Kafka.Net.Stream; using Streamiz.Kafka.Net.Table; -namespace Streamiz.Kafka.Net.Tests +namespace Streamiz.Kafka.Net.Tests.Reproducer { public class FixIssue221Tests { diff --git a/test/Streamiz.Kafka.Net.Tests/Reproducer229Tests.cs b/test/Streamiz.Kafka.Net.Tests/Reproducer/Reproducer229Tests.cs similarity index 99% rename from test/Streamiz.Kafka.Net.Tests/Reproducer229Tests.cs rename to test/Streamiz.Kafka.Net.Tests/Reproducer/Reproducer229Tests.cs index e8595e49..eae02904 100644 --- a/test/Streamiz.Kafka.Net.Tests/Reproducer229Tests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Reproducer/Reproducer229Tests.cs @@ -6,7 +6,7 @@ using Streamiz.Kafka.Net.SerDes; using Streamiz.Kafka.Net.Stream; -namespace Streamiz.Kafka.Net.Tests +namespace Streamiz.Kafka.Net.Tests.Reproducer { public class Reproducer229Tests { diff --git a/test/Streamiz.Kafka.Net.Tests/Reproducer275Tests.cs b/test/Streamiz.Kafka.Net.Tests/Reproducer/Reproducer275Tests.cs similarity index 99% rename from test/Streamiz.Kafka.Net.Tests/Reproducer275Tests.cs rename to test/Streamiz.Kafka.Net.Tests/Reproducer/Reproducer275Tests.cs index ea11fdba..51695b21 100644 --- a/test/Streamiz.Kafka.Net.Tests/Reproducer275Tests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Reproducer/Reproducer275Tests.cs @@ -13,7 +13,7 @@ using Streamiz.Kafka.Net.SerDes; using Streamiz.Kafka.Net.Stream; -namespace Streamiz.Kafka.Net.Tests +namespace Streamiz.Kafka.Net.Tests.Reproducer { internal class CloudEventSerDes2 : AbstractSerDes { From 9643ed17588b539409160d96cffabf9fdeaab26e Mon Sep 17 00:00:00 2001 From: Ubuntu Date: Thu, 18 Jul 2024 18:07:39 +0000 Subject: [PATCH 25/29] Fix unit test --- test/Streamiz.Kafka.Net.Tests/Private/ConcurrentSetTests.cs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/Streamiz.Kafka.Net.Tests/Private/ConcurrentSetTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/ConcurrentSetTests.cs index d76168a0..639e355b 100644 --- a/test/Streamiz.Kafka.Net.Tests/Private/ConcurrentSetTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Private/ConcurrentSetTests.cs @@ -2,7 +2,7 @@ using System.Collections.Generic; using System.Threading.Tasks; using NUnit.Framework; -using Streamiz.Kafka.Net.State.InMemory.Internal; +using Streamiz.Kafka.Net.State.Internal; namespace Streamiz.Kafka.Net.Tests.Private; @@ -90,4 +90,4 @@ public void ConcurrencyAddedAndContains(int numberTasks) Assert.AreEqual(numberTasks, concurrentSet.Count); } -} \ No newline at end of file +} From 26c0e47655accb5309c3f95a22a13e2da2ab62f5 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Mon, 22 Jul 2024 22:40:39 +0000 Subject: [PATCH 26/29] fix sonar --- core/Crosscutting/Bytes.cs | 5 +++++ core/State/Cache/Enumerator/CacheEnumerator.cs | 2 +- core/State/Cache/Internal/CacheEntry.cs | 2 +- core/State/Cache/Internal/CacheEntryExtensions.cs | 12 ++++++------ core/State/Cache/Internal/ICacheEntry.cs | 2 +- core/State/Cache/Internal/IMemoryCache.cs | 2 +- core/State/Cache/Internal/MemoryCache.cs | 4 ++-- .../Cache/Internal/MemoryCacheEntryExtensions.cs | 4 ++-- core/State/Cache/Internal/MemoryCacheEntryOptions.cs | 2 +- core/State/Cache/Internal/MemoryCacheExtensions.cs | 12 ++++++------ .../Internal/PostEvictionCallbackRegistration.cs | 2 +- core/State/Cache/Internal/PostEvictionDelegate.cs | 2 +- 12 files changed, 28 insertions(+), 23 deletions(-) diff --git a/core/Crosscutting/Bytes.cs b/core/Crosscutting/Bytes.cs index 22f7a133..40b3aae9 100644 --- a/core/Crosscutting/Bytes.cs +++ b/core/Crosscutting/Bytes.cs @@ -82,6 +82,11 @@ public Bytes(byte[] bytes) { Get = bytes; } + + /// + /// Create a Bytes using the byte array. + /// + public Bytes(){} /// /// diff --git a/core/State/Cache/Enumerator/CacheEnumerator.cs b/core/State/Cache/Enumerator/CacheEnumerator.cs index 25884d93..cd7860f2 100644 --- a/core/State/Cache/Enumerator/CacheEnumerator.cs +++ b/core/State/Cache/Enumerator/CacheEnumerator.cs @@ -7,7 +7,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Enumerator { internal class CacheEnumerator : IKeyValueEnumerator - where K : class, IComparable + where K : class, IComparable, new() where V : class { private IEnumerator keys; diff --git a/core/State/Cache/Internal/CacheEntry.cs b/core/State/Cache/Internal/CacheEntry.cs index 523e554b..f0e5fec8 100644 --- a/core/State/Cache/Internal/CacheEntry.cs +++ b/core/State/Cache/Internal/CacheEntry.cs @@ -11,7 +11,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal { internal sealed partial class CacheEntry : ICacheEntry - where K : IComparable + where K : IComparable, new() where V : class { //private static readonly Action ExpirationCallback = ExpirationTokensExpired; diff --git a/core/State/Cache/Internal/CacheEntryExtensions.cs b/core/State/Cache/Internal/CacheEntryExtensions.cs index 94b5138e..f91ab109 100644 --- a/core/State/Cache/Internal/CacheEntryExtensions.cs +++ b/core/State/Cache/Internal/CacheEntryExtensions.cs @@ -21,7 +21,7 @@ internal static class CacheEntryExtensions internal static ICacheEntry RegisterPostEvictionCallback( this ICacheEntry entry, PostEvictionDelegate callback) - where K : IComparable + where K : IComparable, new() where V : class { return entry.RegisterPostEvictionCallbackNoValidation(callback, state: null); @@ -38,7 +38,7 @@ public static ICacheEntry RegisterPostEvictionCallback( this ICacheEntry entry, PostEvictionDelegate callback, MemoryCache? state) - where K : IComparable + where K : IComparable, new() where V : class { return entry.RegisterPostEvictionCallbackNoValidation(callback, state); @@ -48,7 +48,7 @@ private static ICacheEntry RegisterPostEvictionCallbackNoValidation( this ICacheEntry entry, PostEvictionDelegate callback, MemoryCache? state) - where K : IComparable + where K : IComparable, new() where V : class { entry.PostEvictionCallbacks.Add(new PostEvictionCallbackRegistration() @@ -68,7 +68,7 @@ private static ICacheEntry RegisterPostEvictionCallbackNoValidation( public static ICacheEntry SetValue( this ICacheEntry entry, V value) - where K : IComparable + where K : IComparable, new() where V : class { entry.Value = value; @@ -84,7 +84,7 @@ public static ICacheEntry SetValue( public static ICacheEntry SetSize( this ICacheEntry entry, long size) - where K : IComparable + where K : IComparable, new() where V : class { if (size < 0) @@ -103,7 +103,7 @@ public static ICacheEntry SetSize( /// Set the values of these options on the . /// The for chaining. public static ICacheEntry SetOptions(this ICacheEntry entry, MemoryCacheEntryOptions options) - where K : IComparable + where K : IComparable, new() where V : class { entry.Size = options.Size; diff --git a/core/State/Cache/Internal/ICacheEntry.cs b/core/State/Cache/Internal/ICacheEntry.cs index 555b1889..dd43edf7 100644 --- a/core/State/Cache/Internal/ICacheEntry.cs +++ b/core/State/Cache/Internal/ICacheEntry.cs @@ -13,7 +13,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// When Disposed, is committed to the cache. /// internal interface ICacheEntry : IDisposable - where K : IComparable + where K : IComparable, new() where V : class { /// diff --git a/core/State/Cache/Internal/IMemoryCache.cs b/core/State/Cache/Internal/IMemoryCache.cs index 99a399c2..8def99c7 100644 --- a/core/State/Cache/Internal/IMemoryCache.cs +++ b/core/State/Cache/Internal/IMemoryCache.cs @@ -11,7 +11,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// Represents a local in-memory cache whose values are not serialized. /// internal interface IMemoryCache : IDisposable - where K : IComparable + where K : IComparable, new() where V : class { /// diff --git a/core/State/Cache/Internal/MemoryCache.cs b/core/State/Cache/Internal/MemoryCache.cs index d61b7b46..4ea0f339 100644 --- a/core/State/Cache/Internal/MemoryCache.cs +++ b/core/State/Cache/Internal/MemoryCache.cs @@ -19,7 +19,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// store its entries. /// internal sealed class MemoryCache : IMemoryCache - where K : IComparable + where K : IComparable, new() where V : class { private readonly IComparer _keyComparer; @@ -541,7 +541,7 @@ private static void ValidateCacheKey(object key) /// the new backing collection. /// private sealed class CoherentState - where K : IComparable + where K : IComparable, new() where V : class { //internal readonly SortedDictionary> EntriesBis = new(); diff --git a/core/State/Cache/Internal/MemoryCacheEntryExtensions.cs b/core/State/Cache/Internal/MemoryCacheEntryExtensions.cs index d6240b0a..90402d91 100644 --- a/core/State/Cache/Internal/MemoryCacheEntryExtensions.cs +++ b/core/State/Cache/Internal/MemoryCacheEntryExtensions.cs @@ -21,7 +21,7 @@ internal static class MemoryCacheEntryExtensions internal static MemoryCacheEntryOptions SetSize( this MemoryCacheEntryOptions options, long size) - where K : IComparable + where K : IComparable, new() where V : class { if (size < 0) @@ -44,7 +44,7 @@ internal static MemoryCacheEntryOptions RegisterPostEvictionCallback this MemoryCacheEntryOptions options, PostEvictionDelegate callback, MemoryCache state) - where K : IComparable + where K : IComparable, new() where V : class { options.PostEvictionCallbacks.Add(new PostEvictionCallbackRegistration() diff --git a/core/State/Cache/Internal/MemoryCacheEntryOptions.cs b/core/State/Cache/Internal/MemoryCacheEntryOptions.cs index f7fe8f5d..0e8c1f39 100644 --- a/core/State/Cache/Internal/MemoryCacheEntryOptions.cs +++ b/core/State/Cache/Internal/MemoryCacheEntryOptions.cs @@ -13,7 +13,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// Represents the cache options applied to an entry of the instance. /// internal class MemoryCacheEntryOptions - where K : IComparable + where K : IComparable, new() where V : class { private long? _size; diff --git a/core/State/Cache/Internal/MemoryCacheExtensions.cs b/core/State/Cache/Internal/MemoryCacheExtensions.cs index d9db090e..478e2c26 100644 --- a/core/State/Cache/Internal/MemoryCacheExtensions.cs +++ b/core/State/Cache/Internal/MemoryCacheExtensions.cs @@ -19,7 +19,7 @@ internal static class CacheExtensions /// The key of the value to get. /// The value associated with this key, or null if the key is not present. internal static V Get(this IMemoryCache cache, K key) - where K : IComparable + where K : IComparable, new() where V : class { cache.TryGetValue(key, out V value); @@ -34,7 +34,7 @@ internal static V Get(this IMemoryCache cache, K key) /// The value associated with the given key. /// true if the key was found. false otherwise. internal static bool TryGetValue(this IMemoryCache cache, K key, out V value) - where K : IComparable + where K : IComparable, new() where V : class { if (cache.TryGetValue(key, out V result)) @@ -64,7 +64,7 @@ internal static bool TryGetValue(this IMemoryCache cache, K key, out /// The value to associate with the key. /// The value that was set. internal static V Set(this IMemoryCache cache, K key, V value) - where K : IComparable + where K : IComparable, new() where V : class { using ICacheEntry entry = cache.CreateEntry(key); @@ -84,7 +84,7 @@ internal static V Set(this IMemoryCache cache, K key, V value) /// Initial eviction reason /// The value that was set. internal static V Set(this IMemoryCache cache, K key, V value, MemoryCacheEntryOptions options, EvictionReason reason) - where K : IComparable + where K : IComparable, new() where V : class { using ICacheEntry entry = cache.CreateEntry(key); @@ -107,7 +107,7 @@ internal static V Set(this IMemoryCache cache, K key, V value, Memor /// The factory that creates the value associated with this key if the key does not exist in the cache. /// The value associated with this key. internal static V GetOrCreate(this IMemoryCache cache, K key, Func, V> factory) - where K : IComparable + where K : IComparable, new() where V : class { return GetOrCreate(cache, key, factory, null); @@ -121,7 +121,7 @@ internal static V GetOrCreate(this IMemoryCache cache, K key, FuncThe factory that creates the value associated with this key if the key does not exist in the cache. /// The options to be applied to the if the key does not exist in the cache. internal static V GetOrCreate(this IMemoryCache cache, K key, Func, V> factory, MemoryCacheEntryOptions? createOptions) - where K : IComparable + where K : IComparable, new() where V : class { if (!cache.TryGetValue(key, out V result)) diff --git a/core/State/Cache/Internal/PostEvictionCallbackRegistration.cs b/core/State/Cache/Internal/PostEvictionCallbackRegistration.cs index d426a2cf..046c42b7 100644 --- a/core/State/Cache/Internal/PostEvictionCallbackRegistration.cs +++ b/core/State/Cache/Internal/PostEvictionCallbackRegistration.cs @@ -11,7 +11,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// Represents a callback delegate that will be fired after an entry is evicted from the cache. /// internal class PostEvictionCallbackRegistration - where K : IComparable + where K : IComparable, new() where V : class { /// diff --git a/core/State/Cache/Internal/PostEvictionDelegate.cs b/core/State/Cache/Internal/PostEvictionDelegate.cs index 920948ea..079733d6 100644 --- a/core/State/Cache/Internal/PostEvictionDelegate.cs +++ b/core/State/Cache/Internal/PostEvictionDelegate.cs @@ -15,6 +15,6 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// The . /// The information that was passed when registering the callback. internal delegate void PostEvictionDelegate(K key, V? value, EvictionReason reason, MemoryCache state) - where K : IComparable + where K : IComparable, new() where V : class; } From fe127affe0a9458552cfccf69a3b8e7a2559404b Mon Sep 17 00:00:00 2001 From: LGouellec Date: Tue, 23 Jul 2024 15:08:32 -0700 Subject: [PATCH 27/29] fix sonar and unit test --- core/Crosscutting/Bytes.cs | 5 ++ .../State/Cache/Enumerator/CacheEnumerator.cs | 2 +- core/State/Cache/Internal/CacheEntry.cs | 2 +- .../Cache/Internal/CacheEntryExtensions.cs | 12 +-- .../Internal/ConcurrentSortedDictionary.cs | 22 +++--- core/State/Cache/Internal/ICacheEntry.cs | 2 +- core/State/Cache/Internal/IMemoryCache.cs | 2 +- core/State/Cache/Internal/MemoryCache.cs | 39 +++++----- .../Internal/MemoryCacheEntryExtensions.cs | 4 +- .../Cache/Internal/MemoryCacheEntryOptions.cs | 2 +- .../Cache/Internal/MemoryCacheExtensions.cs | 12 +-- .../PostEvictionCallbackRegistration.cs | 2 +- .../Cache/Internal/PostEvictionDelegate.cs | 2 +- .../Metrics/StateStoreMetricTests.cs | 74 ++++++++----------- 14 files changed, 88 insertions(+), 94 deletions(-) diff --git a/core/Crosscutting/Bytes.cs b/core/Crosscutting/Bytes.cs index 22f7a133..40b3aae9 100644 --- a/core/Crosscutting/Bytes.cs +++ b/core/Crosscutting/Bytes.cs @@ -82,6 +82,11 @@ public Bytes(byte[] bytes) { Get = bytes; } + + /// + /// Create a Bytes using the byte array. + /// + public Bytes(){} /// /// diff --git a/core/State/Cache/Enumerator/CacheEnumerator.cs b/core/State/Cache/Enumerator/CacheEnumerator.cs index 25884d93..cd7860f2 100644 --- a/core/State/Cache/Enumerator/CacheEnumerator.cs +++ b/core/State/Cache/Enumerator/CacheEnumerator.cs @@ -7,7 +7,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Enumerator { internal class CacheEnumerator : IKeyValueEnumerator - where K : class, IComparable + where K : class, IComparable, new() where V : class { private IEnumerator keys; diff --git a/core/State/Cache/Internal/CacheEntry.cs b/core/State/Cache/Internal/CacheEntry.cs index 523e554b..afd62b6d 100644 --- a/core/State/Cache/Internal/CacheEntry.cs +++ b/core/State/Cache/Internal/CacheEntry.cs @@ -11,7 +11,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal { internal sealed partial class CacheEntry : ICacheEntry - where K : IComparable + where K : class, IComparable where V : class { //private static readonly Action ExpirationCallback = ExpirationTokensExpired; diff --git a/core/State/Cache/Internal/CacheEntryExtensions.cs b/core/State/Cache/Internal/CacheEntryExtensions.cs index 94b5138e..086f11b3 100644 --- a/core/State/Cache/Internal/CacheEntryExtensions.cs +++ b/core/State/Cache/Internal/CacheEntryExtensions.cs @@ -21,7 +21,7 @@ internal static class CacheEntryExtensions internal static ICacheEntry RegisterPostEvictionCallback( this ICacheEntry entry, PostEvictionDelegate callback) - where K : IComparable + where K : class, IComparable where V : class { return entry.RegisterPostEvictionCallbackNoValidation(callback, state: null); @@ -38,7 +38,7 @@ public static ICacheEntry RegisterPostEvictionCallback( this ICacheEntry entry, PostEvictionDelegate callback, MemoryCache? state) - where K : IComparable + where K : class, IComparable where V : class { return entry.RegisterPostEvictionCallbackNoValidation(callback, state); @@ -48,7 +48,7 @@ private static ICacheEntry RegisterPostEvictionCallbackNoValidation( this ICacheEntry entry, PostEvictionDelegate callback, MemoryCache? state) - where K : IComparable + where K : class, IComparable where V : class { entry.PostEvictionCallbacks.Add(new PostEvictionCallbackRegistration() @@ -68,7 +68,7 @@ private static ICacheEntry RegisterPostEvictionCallbackNoValidation( public static ICacheEntry SetValue( this ICacheEntry entry, V value) - where K : IComparable + where K : class, IComparable where V : class { entry.Value = value; @@ -84,7 +84,7 @@ public static ICacheEntry SetValue( public static ICacheEntry SetSize( this ICacheEntry entry, long size) - where K : IComparable + where K : class, IComparable where V : class { if (size < 0) @@ -103,7 +103,7 @@ public static ICacheEntry SetSize( /// Set the values of these options on the . /// The for chaining. public static ICacheEntry SetOptions(this ICacheEntry entry, MemoryCacheEntryOptions options) - where K : IComparable + where K : class, IComparable where V : class { entry.Size = options.Size; diff --git a/core/State/Cache/Internal/ConcurrentSortedDictionary.cs b/core/State/Cache/Internal/ConcurrentSortedDictionary.cs index 66713e86..ebc7be77 100644 --- a/core/State/Cache/Internal/ConcurrentSortedDictionary.cs +++ b/core/State/Cache/Internal/ConcurrentSortedDictionary.cs @@ -1,11 +1,4 @@ -using System; -using System.Collections; -using System.Collections.Generic; -using System.Linq; -using System.Threading; - -namespace Streamiz.Kafka.Net.State.Cache.Internal -{ /* +/* MIT License Copyright (c) 2023 Matthew Krebser (https://github.com/mkrebser) @@ -29,6 +22,14 @@ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ +using System; +using System.Collections; +using System.Collections.Generic; +using System.Linq; +using System.Threading; + +namespace Streamiz.Kafka.Net.State.Cache.Internal +{ // Used for more nuanced lock testing and sanity test // #define ConcurrentSortedDictionary_DEBUG @@ -2003,7 +2004,7 @@ in ConcurrentKTreeNode parent else { mergeLeft(in left, node, in leftIndex, in nodeIndex, parent); - if (left.isLeaf) LeafSiblingNodes.AtomicUpdateMergeNodes(in left); + if (left is { isLeaf: true }) LeafSiblingNodes.AtomicUpdateMergeNodes(in left); } #if ConcurrentSortedDictionary_DEBUG @@ -2612,7 +2613,8 @@ out ConcurrentKTreeNode nextNode if (!ReferenceEquals(null, nextNode) && !TryAcquire(ref nextNode.siblings)) { Release(ref deleteNode.siblings); - Release(ref prevNode.siblings); + if (prevNode != null) + Release(ref prevNode.siblings); continue; } diff --git a/core/State/Cache/Internal/ICacheEntry.cs b/core/State/Cache/Internal/ICacheEntry.cs index 555b1889..b7cac8eb 100644 --- a/core/State/Cache/Internal/ICacheEntry.cs +++ b/core/State/Cache/Internal/ICacheEntry.cs @@ -13,7 +13,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// When Disposed, is committed to the cache. /// internal interface ICacheEntry : IDisposable - where K : IComparable + where K : class, IComparable where V : class { /// diff --git a/core/State/Cache/Internal/IMemoryCache.cs b/core/State/Cache/Internal/IMemoryCache.cs index 99a399c2..7b87a05c 100644 --- a/core/State/Cache/Internal/IMemoryCache.cs +++ b/core/State/Cache/Internal/IMemoryCache.cs @@ -11,7 +11,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// Represents a local in-memory cache whose values are not serialized. /// internal interface IMemoryCache : IDisposable - where K : IComparable + where K : class, IComparable where V : class { /// diff --git a/core/State/Cache/Internal/MemoryCache.cs b/core/State/Cache/Internal/MemoryCache.cs index d61b7b46..7fb33116 100644 --- a/core/State/Cache/Internal/MemoryCache.cs +++ b/core/State/Cache/Internal/MemoryCache.cs @@ -19,7 +19,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// store its entries. /// internal sealed class MemoryCache : IMemoryCache - where K : IComparable + where K : class, IComparable where V : class { private readonly IComparer _keyComparer; @@ -100,7 +100,7 @@ private MemoryCache( /// /// Gets an enumerable of the all the keys in the . /// - private IEnumerable Keys => _coherentState.EntriesBis.Keys.ToList(); + private IEnumerable Keys => _coherentState.Entries.Keys.ToList(); internal IEnumerable KeySetEnumerable(bool forward) { @@ -116,17 +116,17 @@ internal IEnumerable KeyRange(K from, K to, bool inclusive, bool forward) IEnumerable selectedKeys; if (from == null) - selectedKeys = _coherentState.EntriesBis + selectedKeys = _coherentState.Entries .HeadMap(to, inclusive) .Select(kv => kv.Key); else if (to == null) - selectedKeys = _coherentState.EntriesBis + selectedKeys = _coherentState.Entries .TailMap(from, true) .Select(kv => kv.Key); else if (_keyComparer.Compare(from, to) > 0) selectedKeys = new List(); else - selectedKeys = _coherentState.EntriesBis + selectedKeys = _coherentState.Entries .SubMap(from, to, true, inclusive) .Select(kv => kv.Key); @@ -165,7 +165,7 @@ internal void SetEntry(CacheEntry entry) entry.LastAccessed = utcNow; CoherentState coherentState = _coherentState; // Clear() can update the reference in the meantime - if (coherentState.EntriesBis.TryGetValue(entry.Key, out CacheEntry priorEntry)) + if (coherentState.Entries.TryGetValue(entry.Key, out CacheEntry priorEntry)) priorEntry.SetExpired(EvictionReason.Replaced); if (!UpdateCacheSizeExceedsCapacity(entry, coherentState)) @@ -173,12 +173,12 @@ internal void SetEntry(CacheEntry entry) if (priorEntry == null) { // Try to add the new entry if no previous entries exist. - coherentState.EntriesBis.TryAdd(entry.Key, entry); + coherentState.Entries.TryAdd(entry.Key, entry); } else { // Try to update with the new entry if a previous entries exist. - coherentState.EntriesBis.AddOrUpdate(entry.Key, entry); + coherentState.Entries.AddOrUpdate(entry.Key, entry); // The prior entry was removed, decrease the by the prior entry's size Interlocked.Add(ref coherentState.CacheSize, -priorEntry.Size); } @@ -188,7 +188,7 @@ internal void SetEntry(CacheEntry entry) else { TriggerOvercapacityCompaction(); - coherentState.EntriesBis.TryAdd(entry.Key, entry); + coherentState.Entries.TryAdd(entry.Key, entry); Interlocked.Add(ref coherentState.CacheSize, entry.Size); } } @@ -204,7 +204,7 @@ public bool TryGetValue(K key, out V value) DateTime utcNow = UtcNow; CoherentState coherentState = _coherentState; // Clear() can update the reference in the meantime - if (coherentState.EntriesBis.TryGetValue(key, out CacheEntry tmp)) + if (coherentState.Entries.TryGetValue(key, out CacheEntry tmp)) { CacheEntry entry = tmp; entry.LastAccessed = utcNow; @@ -250,8 +250,8 @@ public void Remove(K key) CheckDisposed(); CoherentState coherentState = _coherentState; // Clear() can update the reference in the meantime - if (coherentState.EntriesBis.TryGetValue(key, out CacheEntry entry) && - coherentState.EntriesBis.TryRemove(key)) + if (coherentState.Entries.TryGetValue(key, out CacheEntry entry) && + coherentState.Entries.TryRemove(key)) { Interlocked.Add(ref coherentState.CacheSize, -entry.Size); entry.SetExpired(EvictionReason.Removed); @@ -267,7 +267,7 @@ public void Clear() CheckDisposed(); CoherentState oldState = Interlocked.Exchange(ref _coherentState, new CoherentState()); - foreach (KeyValuePair> entry in oldState.EntriesBis) + foreach (KeyValuePair> entry in oldState.Entries) { entry.Value.SetExpired(EvictionReason.Removed); entry.Value.InvokeEvictionCallbacks(); @@ -442,7 +442,7 @@ private void Flush() var entriesToRemove = new List>(); foreach(var entry in - _coherentState.EntriesBis.OrderBy(e => e.Value.LastAccessed)) + _coherentState.Entries.OrderBy(e => e.Value.LastAccessed)) entriesToRemove.Add(entry.Value); foreach (CacheEntry entry in entriesToRemove) @@ -454,7 +454,7 @@ private void Compact(long removalSizeTarget, Func, long> comput var entriesToRemove = new List>(); long removedSize = 0; - ExpireLruBucket(ref removedSize, removalSizeTarget, computeEntrySize, entriesToRemove, coherentState.EntriesBis); + ExpireLruBucket(ref removedSize, removalSizeTarget, computeEntrySize, entriesToRemove, coherentState.Entries); foreach (CacheEntry entry in entriesToRemove) coherentState.RemoveEntry(entry, _options); @@ -541,20 +541,19 @@ private static void ValidateCacheKey(object key) /// the new backing collection. /// private sealed class CoherentState - where K : IComparable + where K : class, IComparable where V : class { - //internal readonly SortedDictionary> EntriesBis = new(); - internal readonly ConcurrentSortedDictionary> EntriesBis = new(); + internal readonly ConcurrentSortedDictionary> Entries = new(); internal long CacheSize; - internal long Count => EntriesBis.Count; + internal long Count => Entries.Count; internal long Size => Volatile.Read(ref CacheSize); internal void RemoveEntry(CacheEntry entry, MemoryCacheOptions options) { - if (EntriesBis.TryRemove(entry.Key)) + if (Entries.TryRemove(entry.Key)) { Interlocked.Add(ref CacheSize, -entry.Size); entry.InvokeEvictionCallbacks(); diff --git a/core/State/Cache/Internal/MemoryCacheEntryExtensions.cs b/core/State/Cache/Internal/MemoryCacheEntryExtensions.cs index d6240b0a..564d5a8c 100644 --- a/core/State/Cache/Internal/MemoryCacheEntryExtensions.cs +++ b/core/State/Cache/Internal/MemoryCacheEntryExtensions.cs @@ -21,7 +21,7 @@ internal static class MemoryCacheEntryExtensions internal static MemoryCacheEntryOptions SetSize( this MemoryCacheEntryOptions options, long size) - where K : IComparable + where K : class, IComparable where V : class { if (size < 0) @@ -44,7 +44,7 @@ internal static MemoryCacheEntryOptions RegisterPostEvictionCallback this MemoryCacheEntryOptions options, PostEvictionDelegate callback, MemoryCache state) - where K : IComparable + where K : class, IComparable where V : class { options.PostEvictionCallbacks.Add(new PostEvictionCallbackRegistration() diff --git a/core/State/Cache/Internal/MemoryCacheEntryOptions.cs b/core/State/Cache/Internal/MemoryCacheEntryOptions.cs index f7fe8f5d..1c6f51ca 100644 --- a/core/State/Cache/Internal/MemoryCacheEntryOptions.cs +++ b/core/State/Cache/Internal/MemoryCacheEntryOptions.cs @@ -13,7 +13,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// Represents the cache options applied to an entry of the instance. /// internal class MemoryCacheEntryOptions - where K : IComparable + where K : class, IComparable where V : class { private long? _size; diff --git a/core/State/Cache/Internal/MemoryCacheExtensions.cs b/core/State/Cache/Internal/MemoryCacheExtensions.cs index d9db090e..e331d8f1 100644 --- a/core/State/Cache/Internal/MemoryCacheExtensions.cs +++ b/core/State/Cache/Internal/MemoryCacheExtensions.cs @@ -19,7 +19,7 @@ internal static class CacheExtensions /// The key of the value to get. /// The value associated with this key, or null if the key is not present. internal static V Get(this IMemoryCache cache, K key) - where K : IComparable + where K : class, IComparable where V : class { cache.TryGetValue(key, out V value); @@ -34,7 +34,7 @@ internal static V Get(this IMemoryCache cache, K key) /// The value associated with the given key. /// true if the key was found. false otherwise. internal static bool TryGetValue(this IMemoryCache cache, K key, out V value) - where K : IComparable + where K : class, IComparable where V : class { if (cache.TryGetValue(key, out V result)) @@ -64,7 +64,7 @@ internal static bool TryGetValue(this IMemoryCache cache, K key, out /// The value to associate with the key. /// The value that was set. internal static V Set(this IMemoryCache cache, K key, V value) - where K : IComparable + where K : class, IComparable where V : class { using ICacheEntry entry = cache.CreateEntry(key); @@ -84,7 +84,7 @@ internal static V Set(this IMemoryCache cache, K key, V value) /// Initial eviction reason /// The value that was set. internal static V Set(this IMemoryCache cache, K key, V value, MemoryCacheEntryOptions options, EvictionReason reason) - where K : IComparable + where K : class, IComparable where V : class { using ICacheEntry entry = cache.CreateEntry(key); @@ -107,7 +107,7 @@ internal static V Set(this IMemoryCache cache, K key, V value, Memor /// The factory that creates the value associated with this key if the key does not exist in the cache. /// The value associated with this key. internal static V GetOrCreate(this IMemoryCache cache, K key, Func, V> factory) - where K : IComparable + where K : class, IComparable where V : class { return GetOrCreate(cache, key, factory, null); @@ -121,7 +121,7 @@ internal static V GetOrCreate(this IMemoryCache cache, K key, FuncThe factory that creates the value associated with this key if the key does not exist in the cache. /// The options to be applied to the if the key does not exist in the cache. internal static V GetOrCreate(this IMemoryCache cache, K key, Func, V> factory, MemoryCacheEntryOptions? createOptions) - where K : IComparable + where K : class, IComparable where V : class { if (!cache.TryGetValue(key, out V result)) diff --git a/core/State/Cache/Internal/PostEvictionCallbackRegistration.cs b/core/State/Cache/Internal/PostEvictionCallbackRegistration.cs index d426a2cf..a84537db 100644 --- a/core/State/Cache/Internal/PostEvictionCallbackRegistration.cs +++ b/core/State/Cache/Internal/PostEvictionCallbackRegistration.cs @@ -11,7 +11,7 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// Represents a callback delegate that will be fired after an entry is evicted from the cache. /// internal class PostEvictionCallbackRegistration - where K : IComparable + where K : class, IComparable where V : class { /// diff --git a/core/State/Cache/Internal/PostEvictionDelegate.cs b/core/State/Cache/Internal/PostEvictionDelegate.cs index 920948ea..7dcb9d1e 100644 --- a/core/State/Cache/Internal/PostEvictionDelegate.cs +++ b/core/State/Cache/Internal/PostEvictionDelegate.cs @@ -15,6 +15,6 @@ namespace Streamiz.Kafka.Net.State.Cache.Internal /// The . /// The information that was passed when registering the callback. internal delegate void PostEvictionDelegate(K key, V? value, EvictionReason reason, MemoryCache state) - where K : IComparable + where K : class, IComparable where V : class; } diff --git a/test/Streamiz.Kafka.Net.Tests/Metrics/StateStoreMetricTests.cs b/test/Streamiz.Kafka.Net.Tests/Metrics/StateStoreMetricTests.cs index c3b70ccb..a693d48b 100644 --- a/test/Streamiz.Kafka.Net.Tests/Metrics/StateStoreMetricTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Metrics/StateStoreMetricTests.cs @@ -8,7 +8,6 @@ using Streamiz.Kafka.Net.Metrics; using Streamiz.Kafka.Net.Metrics.Internal; using Streamiz.Kafka.Net.Mock; -using Streamiz.Kafka.Net.Processors; using Streamiz.Kafka.Net.Processors.Internal; using Streamiz.Kafka.Net.SerDes; using Streamiz.Kafka.Net.State; @@ -154,18 +153,13 @@ public override IKeyValueEnumerator, byte[]> FetchAll(DateTime @ } } - private StreamMetricsRegistry streamMetricsRegistry = null; - - private readonly StreamConfig config = - new StreamConfig(); + private StreamMetricsRegistry streamMetricsRegistry; + private readonly StreamConfig config = new(); private string threadId = StreamMetricsRegistry.UNKNOWN_THREAD; private TopicPartition topicPartition; private TaskId id; - private string storeScope = "store-scope"; - private string storeName = "my-store"; private ProcessorContext context; - private IStateStore store; [SetUp] public void Initialize() @@ -205,22 +199,20 @@ public void Initialize() [TearDown] public void Dispose() { - store.Close(); } [Test] public void KeyValueStoreMetricsTest() { + var storeName = "key-value-store"; var random = new Random(); MeteredKeyValueStore meteredKeyValueStore = new MeteredKeyValueStore( new MockInMemoryStore(storeName), new StringSerDes(), new StringSerDes(), - storeScope); - store = meteredKeyValueStore; + "mock-in-memory"); meteredKeyValueStore.Init(context, meteredKeyValueStore); - int nbMessage = random.Next(0, 30000); int nbMessage2 = random.Next(0, 30); // produce ${nbMessage} messages to input topic @@ -231,57 +223,49 @@ public void KeyValueStoreMetricsTest() meteredKeyValueStore.PutAll(messages); meteredKeyValueStore.Flush(); - // AssertAvgAndMaxLatency(StateStoreMetrics.PUT_ALL); - var latencyAvg = GetSensorMetric( - StateStoreMetrics.PUT_ALL, - StreamMetricsRegistry.LATENCY_SUFFIX + StreamMetricsRegistry.AVG_SUFFIX, - StreamMetricsRegistry.STATE_STORE_LEVEL_GROUP); - var latencyMax = GetSensorMetric( - StateStoreMetrics.PUT_ALL, - StreamMetricsRegistry.LATENCY_SUFFIX + StreamMetricsRegistry.MAX_SUFFIX, - StreamMetricsRegistry.STATE_STORE_LEVEL_GROUP); - - Assert.IsTrue((double)latencyAvg.Value >= 0); - Assert.IsTrue((double)latencyMax.Value >= 0); + AssertAvgAndMaxLatency(storeName, StateStoreMetrics.PUT_ALL); for(int i = 0 ; i < nbMessage2 ; ++i) meteredKeyValueStore.Put($"test{i}", $"test{i}"); meteredKeyValueStore.Flush(); - AssertAvgAndMaxLatency(StateStoreMetrics.PUT); + AssertAvgAndMaxLatency(storeName, StateStoreMetrics.PUT); for(int i = 0 ; i < nbMessage2 ; ++i) meteredKeyValueStore.PutIfAbsent($"test{i}", $"test{i}"); meteredKeyValueStore.Flush(); - AssertAvgAndMaxLatency(StateStoreMetrics.PUT_IF_ABSENT); + AssertAvgAndMaxLatency(storeName, StateStoreMetrics.PUT_IF_ABSENT); for (int i = 0; i < nbMessage2; ++i) meteredKeyValueStore.Get($"test{i}"); - AssertAvgAndMaxLatency(StateStoreMetrics.GET); + AssertAvgAndMaxLatency(storeName, StateStoreMetrics.GET); for(int i = 0 ; i < 5 ; ++i) meteredKeyValueStore.All(); - AssertAvgAndMaxLatency(StateStoreMetrics.ALL); + AssertAvgAndMaxLatency(storeName, StateStoreMetrics.ALL); var results1 = meteredKeyValueStore.Range($"key0", $"key{nbMessage - 1}").ToList(); var results2 = meteredKeyValueStore.Range($"test0", $"test{nbMessage2 - 1}").ToList(); - AssertAvgAndMaxLatency(StateStoreMetrics.RANGE); + AssertAvgAndMaxLatency(storeName, StateStoreMetrics.RANGE); for (int i = 0; i < nbMessage2; ++i) meteredKeyValueStore.Delete($"key{i + 1}"); - AssertAvgAndMaxLatency(StateStoreMetrics.DELETE); - AssertAvgAndMaxLatency(StateStoreMetrics.FLUSH); + AssertAvgAndMaxLatency(storeName, StateStoreMetrics.DELETE); + AssertAvgAndMaxLatency(storeName, StateStoreMetrics.FLUSH); + + meteredKeyValueStore.Close(); } [Test] public void WindowStoreMetricsTest() { + var storeName = "window-store"; long windowSize = 1000 * 60; var random = new Random(); MeteredWindowStore meteredWindowStore = new MeteredWindowStore( @@ -289,8 +273,8 @@ public void WindowStoreMetricsTest() windowSize, new StringSerDes(), new StringSerDes(), - storeScope); - store = meteredWindowStore; + "in-memory-window"); + meteredWindowStore.Init(context, meteredWindowStore); int nbMessage = random.Next(0, 30); @@ -308,41 +292,45 @@ public void WindowStoreMetricsTest() meteredWindowStore.Flush(); - AssertAvgAndMaxLatency(StateStoreMetrics.PUT); - AssertAvgAndMaxLatency(StateStoreMetrics.FLUSH); + AssertAvgAndMaxLatency(storeName, StateStoreMetrics.PUT); + AssertAvgAndMaxLatency(storeName, StateStoreMetrics.FLUSH); for (int i = 0; i < nbMessage; ++i) meteredWindowStore.Fetch($"test{i}", now1); - AssertAvgAndMaxLatency(StateStoreMetrics.FETCH); + AssertAvgAndMaxLatency(storeName, StateStoreMetrics.FETCH); meteredWindowStore.Fetch($"test0", now1.FromMilliseconds().AddSeconds(-10), now1.FromMilliseconds().AddSeconds(10)).ToList(); - AssertAvgAndMaxLatency(StateStoreMetrics.FETCH); + AssertAvgAndMaxLatency(storeName, StateStoreMetrics.FETCH); meteredWindowStore.Fetch($"test0", now1 - 10000, now1 + 10000).ToList(); - AssertAvgAndMaxLatency(StateStoreMetrics.FETCH); + AssertAvgAndMaxLatency(storeName, StateStoreMetrics.FETCH); var nb = meteredWindowStore.FetchAll(now1.FromMilliseconds().AddSeconds(-10), now2.FromMilliseconds().AddSeconds(10)).ToList().Count(); Assert.AreEqual(nbMessage * 2 , nb); - AssertAvgAndMaxLatency(StateStoreMetrics.FETCH); + AssertAvgAndMaxLatency(storeName, StateStoreMetrics.FETCH); meteredWindowStore.All().ToList(); + + meteredWindowStore.Close(); } - private void AssertAvgAndMaxLatency(string sensorName) + private void AssertAvgAndMaxLatency(string storeName, string sensorName) { var latencyAvg = GetSensorMetric( + storeName, sensorName, StreamMetricsRegistry.LATENCY_SUFFIX + StreamMetricsRegistry.AVG_SUFFIX, StreamMetricsRegistry.STATE_STORE_LEVEL_GROUP); var latencyMax = GetSensorMetric( + storeName, sensorName, StreamMetricsRegistry.LATENCY_SUFFIX + StreamMetricsRegistry.MAX_SUFFIX, StreamMetricsRegistry.STATE_STORE_LEVEL_GROUP); @@ -350,10 +338,10 @@ private void AssertAvgAndMaxLatency(string sensorName) Assert.GreaterOrEqual((double)latencyMax.Value, 0); } - private StreamMetric GetSensorMetric(string sensorName, string metricSuffix, string group) + private StreamMetric GetSensorMetric(string storeName, string sensorName, string metricSuffix, string group) { long now = DateTime.Now.GetMilliseconds(); - var sensor = streamMetricsRegistry.GetSensors().FirstOrDefault(s => s.Name.Equals(GetSensorName(sensorName))); + var sensor = streamMetricsRegistry.GetSensors().FirstOrDefault(s => s.Name.Equals(GetSensorName(storeName, sensorName))); if (sensor == null) throw new NullReferenceException($"sensor {sensorName} not found"); @@ -367,7 +355,7 @@ private StreamMetric GetSensorMetric(string sensorName, string metricSuffix, str return sensor.Metrics[keyMetric]; } - private string GetSensorName(string sensorName) + private string GetSensorName(string storeName, string sensorName) => streamMetricsRegistry.FullSensorName( sensorName, streamMetricsRegistry.StoreSensorPrefix(threadId, id.ToString(), storeName)); From 520097d5355b5ee7128ca67f92f6608c325d5bc0 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Tue, 23 Jul 2024 22:27:40 +0000 Subject: [PATCH 28/29] fix --- .../Cache/Internal/CacheEntryExtensions.cs | 20 ------------------- 1 file changed, 20 deletions(-) diff --git a/core/State/Cache/Internal/CacheEntryExtensions.cs b/core/State/Cache/Internal/CacheEntryExtensions.cs index 481edf20..086f11b3 100644 --- a/core/State/Cache/Internal/CacheEntryExtensions.cs +++ b/core/State/Cache/Internal/CacheEntryExtensions.cs @@ -38,11 +38,7 @@ public static ICacheEntry RegisterPostEvictionCallback( this ICacheEntry entry, PostEvictionDelegate callback, MemoryCache? state) -<<<<<<< HEAD where K : class, IComparable -======= - where K : IComparable, new() ->>>>>>> 26c0e47655accb5309c3f95a22a13e2da2ab62f5 where V : class { return entry.RegisterPostEvictionCallbackNoValidation(callback, state); @@ -52,11 +48,7 @@ private static ICacheEntry RegisterPostEvictionCallbackNoValidation( this ICacheEntry entry, PostEvictionDelegate callback, MemoryCache? state) -<<<<<<< HEAD where K : class, IComparable -======= - where K : IComparable, new() ->>>>>>> 26c0e47655accb5309c3f95a22a13e2da2ab62f5 where V : class { entry.PostEvictionCallbacks.Add(new PostEvictionCallbackRegistration() @@ -76,11 +68,7 @@ private static ICacheEntry RegisterPostEvictionCallbackNoValidation( public static ICacheEntry SetValue( this ICacheEntry entry, V value) -<<<<<<< HEAD where K : class, IComparable -======= - where K : IComparable, new() ->>>>>>> 26c0e47655accb5309c3f95a22a13e2da2ab62f5 where V : class { entry.Value = value; @@ -96,11 +84,7 @@ public static ICacheEntry SetValue( public static ICacheEntry SetSize( this ICacheEntry entry, long size) -<<<<<<< HEAD where K : class, IComparable -======= - where K : IComparable, new() ->>>>>>> 26c0e47655accb5309c3f95a22a13e2da2ab62f5 where V : class { if (size < 0) @@ -119,11 +103,7 @@ public static ICacheEntry SetSize( /// Set the values of these options on the . /// The for chaining. public static ICacheEntry SetOptions(this ICacheEntry entry, MemoryCacheEntryOptions options) -<<<<<<< HEAD where K : class, IComparable -======= - where K : IComparable, new() ->>>>>>> 26c0e47655accb5309c3f95a22a13e2da2ab62f5 where V : class { entry.Size = options.Size; From d41c355822ca8e9b2bb2f643cf82214b59386d58 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Mon, 29 Jul 2024 16:32:01 -0700 Subject: [PATCH 29/29] fix unit tests --- core/Processors/Internal/InternalTopologyBuilder.cs | 6 ++++-- test/Streamiz.Kafka.Net.Tests/Public/KafkaStreamTests.cs | 7 ++++++- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/core/Processors/Internal/InternalTopologyBuilder.cs b/core/Processors/Internal/InternalTopologyBuilder.cs index 7f124be1..85cd2935 100644 --- a/core/Processors/Internal/InternalTopologyBuilder.cs +++ b/core/Processors/Internal/InternalTopologyBuilder.cs @@ -804,8 +804,6 @@ internal TaskId GetTaskIdFromPartition(TopicPartition topicPartition) // FOR TESTING internal void GetLinkTopics(string topic, IList linkTopics) { - if (linkTopics.Contains(topic)) - return; var subTopo = GetSubTopologyDescription(topic); if (subTopo != null) { @@ -820,6 +818,10 @@ internal void GetLinkTopics(string topic, IList linkTopics) var sinkTopic = internalTopics.ContainsKey(sinkNode.Topic) ? DecorateTopic(sinkNode.Topic) : sinkNode.Topic; + + if (linkTopics.Contains(sinkTopic)) + return; + linkTopics.Add(sinkTopic); GetLinkTopics(sinkTopic, linkTopics); } diff --git a/test/Streamiz.Kafka.Net.Tests/Public/KafkaStreamTests.cs b/test/Streamiz.Kafka.Net.Tests/Public/KafkaStreamTests.cs index 94f6c47a..ad0e08ab 100644 --- a/test/Streamiz.Kafka.Net.Tests/Public/KafkaStreamTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Public/KafkaStreamTests.cs @@ -8,6 +8,7 @@ using Streamiz.Kafka.Net.Table; using System; using System.Collections.Generic; +using System.IO; using System.Linq; using System.Threading; using System.Threading.Tasks; @@ -15,6 +16,7 @@ using Streamiz.Kafka.Net.Processors.Public; using Streamiz.Kafka.Net.State.Enumerator; using Streamiz.Kafka.Net.Stream.Internal.Graph; +using Streamiz.Kafka.Net.Tests.Helpers; namespace Streamiz.Kafka.Net.Tests.Public { @@ -813,7 +815,8 @@ public async Task OpenIteratorsWithTheProcessAPI() config.ApplicationId = "test-open-iterator"; config.BootstrapServers = "127.0.0.1"; config.PollMs = 10; - + config.UseRandomRocksDbConfigForTest(); + var supplier = new SyncKafkaSupplier(); var producer = supplier.GetProducer(config.ToProducerConfig()); @@ -873,6 +876,8 @@ public async Task OpenIteratorsWithTheProcessAPI() stream.Dispose(); foreach(var e in currentEnumerators) Assert.Throws(() => e.Dispose()); + + //Directory.Delete(config.StateDir, true); } } } \ No newline at end of file