From 3469037722ad27ad831dc3fa44b1d661633df563 Mon Sep 17 00:00:00 2001 From: LGouellec Date: Wed, 26 Jun 2024 17:01:47 -0700 Subject: [PATCH 01/20] 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/20] 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/20] 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/20] 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/20] #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/20] 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/20] 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/20] 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/20] 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/20] 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/20] 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/20] 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/20] 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/20] 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/20] #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/20] 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/20] 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/20] 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 5eee3be9a39cb2f28425d7f7149eccd56bfc12cb Mon Sep 17 00:00:00 2001 From: LGouellec Date: Tue, 9 Jul 2024 16:31:28 -0700 Subject: [PATCH 19/20] 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 20/20] 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