diff --git a/.gitignore b/.gitignore index 77a1b76b..f5dd442e 100644 --- a/.gitignore +++ b/.gitignore @@ -11,4 +11,5 @@ TestResults .idea/ .vscode/ -confidential \ No newline at end of file +confidential +roadmap.md \ No newline at end of file diff --git a/.gitpod.yml b/.gitpod.yml index b85e07b8..f5c5dfdf 100644 --- a/.gitpod.yml +++ b/.gitpod.yml @@ -22,7 +22,7 @@ tasks: echo "1" > run echo "🚀 Enjoy Streamiz the .NET Stream processing library for Apache Kafka (TM)"; sleep 2 - dotnet run -f net6.0 --project samples/sample-stream-demo/sample-stream-demo.csproj --no-build --no-restore + dotnet run -f net6.0 --project launcher/sample-stream-demo/sample-stream-demo.csproj --no-build --no-restore - name: producer command: while [ ! -f run ]; do sleep 1; done; docker-compose -f environment/docker-compose.yml exec broker kafka-console-producer --bootstrap-server broker:29092 --topic input - name: consumer diff --git a/Pointers.md b/Pointers.md deleted file mode 100644 index 9da18369..00000000 --- a/Pointers.md +++ /dev/null @@ -1,4 +0,0 @@ -- Process more than one record of one task at time : [KAFKA-9756](https://issues.apache.org/jira/browse/KAFKA-9756) | [PR #8358](https://github.com/apache/kafka/pull/8358) -- EOS Producer could throw illegal state if trying to complete a failed batch after fatal error : [KAFKA-9605](https://issues.apache.org/jira/browse/KAFKA-9605) | [PR #8177](https://github.com/apache/kafka/pull/8177) -- KIP-450 : Sliding windows -- KIP-612 : end-to-end latency metrics \ No newline at end of file diff --git a/core/Crosscutting/Bytes.cs b/core/Crosscutting/Bytes.cs index 40b3aae9..26eec94b 100644 --- a/core/Crosscutting/Bytes.cs +++ b/core/Crosscutting/Bytes.cs @@ -77,7 +77,7 @@ public class Bytes : IEquatable, IComparable /// Create a Bytes using the byte array. /// /// This array becomes the backing storage for the object. - [Obsolete("Will be removed last release version")] + [Obsolete("Will be removed in 1.8.0")] public Bytes(byte[] bytes) { Get = bytes; diff --git a/core/Errors/StreamProducerException.cs b/core/Errors/StreamProducerException.cs new file mode 100644 index 00000000..c4282656 --- /dev/null +++ b/core/Errors/StreamProducerException.cs @@ -0,0 +1,17 @@ +using System; +using Confluent.Kafka; + +namespace Streamiz.Kafka.Net.Errors +{ + internal class StreamProducerException : Exception + { + public ProduceException OriginalProduceException { get; set; } + public ProductionExceptionHandlerResponse Response { get; set; } + + public StreamProducerException(ProduceException originalProduceException, ProductionExceptionHandlerResponse response) + { + OriginalProduceException = originalProduceException; + Response = response; + } + } +} \ No newline at end of file diff --git a/core/Kafka/IRecordCollector.cs b/core/Kafka/IRecordCollector.cs index 4f03c8a8..38f089a6 100644 --- a/core/Kafka/IRecordCollector.cs +++ b/core/Kafka/IRecordCollector.cs @@ -7,9 +7,9 @@ namespace Streamiz.Kafka.Net.Kafka internal interface IRecordCollector { IDictionary CollectorOffsets { get; } - void Init(ref IProducer producer); + void Initialize(); void Flush(); - void Close(); + void Close(bool dirty); 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 205cf110..226b11d4 100644 --- a/core/Kafka/Internal/RecordCollector.cs +++ b/core/Kafka/Internal/RecordCollector.cs @@ -1,20 +1,18 @@ -using Confluent.Kafka; -using Streamiz.Kafka.Net.Crosscutting; -using Streamiz.Kafka.Net.Errors; -using Streamiz.Kafka.Net.Processors.Internal; -using Streamiz.Kafka.Net.SerDes; -using System; +using System; using System.Collections.Concurrent; using System.Collections.Generic; using System.Diagnostics; -using System.Linq; using System.Text; +using Confluent.Kafka; using Microsoft.Extensions.Logging; +using Streamiz.Kafka.Net.Crosscutting; +using Streamiz.Kafka.Net.Errors; using Streamiz.Kafka.Net.Metrics; +using Streamiz.Kafka.Net.Processors.Internal; +using Streamiz.Kafka.Net.SerDes; namespace Streamiz.Kafka.Net.Kafka.Internal { - // TODO : Need to refactor, not necessary now to have one producer by thread if EOS is enable, see EOS_V2 internal class RecordCollector : IRecordCollector { private sealed class RetryRecord @@ -47,6 +45,7 @@ public void AddRecord(RetryRecord record) record.Headers.Remove(oldGuidKey); records.Remove(oldGuidKey); } + var newKey = Guid.NewGuid().ToString(); record.Headers.AddOrUpdate(RETRY_HEADER_KEY, Encoding.UTF8.GetBytes(newKey)); records.Add(newKey, record); @@ -72,96 +71,67 @@ public void Clear() } } - // IF EOS DISABLED, ONE PRODUCER BY TASK BUT ONE INSTANCE RECORD COLLECTOR BY TASK - // WHEN CLOSING TASK, WE MUST DISPOSE PRODUCER WHEN NO MORE INSTANCE OF RECORD COLLECTOR IS PRESENT - // IT'S A GARBAGE COLLECTOR LIKE - private static IDictionary instanceProducer = new Dictionary(); - private readonly object _lock = new(); + private StreamsProducer _producer; + private readonly IStreamConfig _configuration; + private readonly TaskId _id; + private readonly Sensor _droppedRecordsSensor; + private Exception _exception; - private IProducer producer; - 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 string _logPrefix; private readonly ILogger log = Logger.GetLogger(typeof(RecordCollector)); - private readonly ConcurrentDictionary collectorsOffsets = new(); - - private readonly RetryRecordContext retryRecordContext = new(); + private readonly ConcurrentDictionary _collectorsOffsets = new(); - public IDictionary CollectorOffsets => collectorsOffsets.ToDictionary(); + private readonly RetryRecordContext _retryRecordContext = new(); - public IDictionary cachePartitionsForTopics = - new Dictionary(); + public IDictionary CollectorOffsets => _collectorsOffsets.ToDictionary(); - public RecordCollector(string logPrefix, IStreamConfig configuration, TaskId id, Sensor droppedRecordsSensor, IAdminClient adminClient) + public RecordCollector(string logPrefix, IStreamConfig configuration, TaskId id, StreamsProducer producer, + Sensor droppedRecordsSensor) { - this.logPrefix = $"{logPrefix}"; - this.configuration = configuration; - this.id = id; - this.droppedRecordsSensor = droppedRecordsSensor; - _adminClient = adminClient; + _logPrefix = $"{logPrefix}"; + _configuration = configuration; + _id = id; + _droppedRecordsSensor = droppedRecordsSensor; + _producer = producer; } - public void Init(ref IProducer producer) + public void Initialize() { - this.producer = producer; - - string producerName = producer.Name.Split('#')[0]; - lock (_lock) - { - if (instanceProducer.ContainsKey(producerName)) - ++instanceProducer[producerName]; - else - instanceProducer.Add(producerName, 1); - } + if (_producer.EosEnabled) + _producer.InitTransaction(); } - public void Close() + public void Close(bool dirty) { - log.LogDebug("{LogPrefix}Closing producer", logPrefix); - if (producer != null) + log.LogDebug($"{_logPrefix}Closing producer"); + + if (_retryRecordContext.HasNext) + log.LogWarning( + "There are messages still pending to retry in the backpressure queue. These messages won't longer flush into the corresponding topic !"); + + _retryRecordContext.Clear(); + + if (dirty) { - lock (_lock) - { - string producerName = producer.Name.Split('#')[0]; - if (instanceProducer.ContainsKey(producerName) && --instanceProducer[producerName] <= 0) - { - if (retryRecordContext.HasNext) - log.LogWarning( - "There are messages still pending to retry in the backpressure queue. These messages won't longer flush into the corresponding topic !"); - - retryRecordContext.Clear(); - producer.Dispose(); - producer = null; - CheckForException(); - } - } + if (_producer.EosEnabled) + _producer.AbortTransaction(); } - - _adminClient?.Dispose(); + + _collectorsOffsets.Clear(); + CheckForException(); } public void Flush() { - log.LogDebug("{LogPrefix}Flushing producer", logPrefix); - if (producer != null) + log.LogDebug("{LogPrefix}Flushing producer", _logPrefix); + if (_producer != null) { - try - { - while (retryRecordContext.HasNext) - ProduceRetryRecord(); - - producer.Flush(); - CheckForException(); - } - catch (ObjectDisposedException) - { - // has been disposed - } + while (_retryRecordContext.HasNext) + ProduceRetryRecord(); + + _producer.Flush(); + CheckForException(); } } @@ -190,37 +160,30 @@ private bool IsFatalError(DeliveryReport report) private bool IsRecoverableError(DeliveryReport report) { - return IsRecoverableError(report.Error); - } - - private bool IsRecoverableError(Error error) - { - return error.Code == ErrorCode.TransactionCoordinatorFenced || - error.Code == ErrorCode.UnknownProducerId || - error.Code == ErrorCode.OutOfOrderSequenceNumber; + return StreamsProducer.IsRecoverable(report.Error); } private void SendInternal(string topic, K key, V value, Headers headers, int? partition, long timestamp, ISerDes keySerializer, ISerDes valueSerializer) { - Debug.Assert(producer != null, nameof(producer) + " != null"); + CheckForException(); + + Debug.Assert(_producer != null, nameof(_producer) + " != null"); var k = key != null ? keySerializer.Serialize(key, new SerializationContext(MessageComponentType.Key, topic, headers)) : null; var v = value != null ? valueSerializer.Serialize(value, new SerializationContext(MessageComponentType.Value, topic, headers)) : null; - - CheckForException(); - - while (retryRecordContext.HasNext) - ProduceRetryRecord(); + while (_retryRecordContext.HasNext) + ProduceRetryRecord(); + try { if (partition.HasValue) { - producer.Produce( + _producer.Produce( new TopicPartition(topic, partition.Value), new Message { @@ -233,7 +196,7 @@ private void SendInternal(string topic, K key, V value, Headers headers, i } else { - producer.Produce( + _producer.Produce( topic, new Message { @@ -247,6 +210,10 @@ private void SendInternal(string topic, K key, V value, Headers headers, i } catch (ProduceException produceException) { + if (StreamsProducer.IsRecoverable(produceException.Error)) + throw new TaskMigratedException( + $"Producer got fenced trying to send a record: {produceException.Message}"); + ManageProduceException(produceException); } } @@ -257,43 +224,40 @@ private void HandleError(DeliveryReport report) { StringBuilder sb = new StringBuilder(); sb.AppendLine( - $"{logPrefix}Error encountered sending record to topic {report.Topic} for task {id} due to:"); - sb.AppendLine($"{logPrefix}Error Code : {report.Error.Code.ToString()}"); - sb.AppendLine($"{logPrefix}Message : {report.Error.Reason}"); + $"{_logPrefix}Error encountered sending record to topic {report.Topic} for task {_id} due to:"); + sb.AppendLine($"{_logPrefix}Error Code : {report.Error.Code.ToString()}"); + sb.AppendLine($"{_logPrefix}Message : {report.Error.Reason}"); if (IsFatalError(report)) { sb.AppendLine( - $"{logPrefix}Written offsets would not be recorded and no more records would be sent since this is a fatal error."); + $"{_logPrefix}Written offsets would not be recorded and no more records would be sent since this is a fatal error."); log.LogError(sb.ToString()); - lock (_lock) - exception = new StreamsException(sb.ToString()); + _exception = new StreamsException(sb.ToString()); } else if (IsRecoverableError(report)) { sb.AppendLine( - $"{logPrefix}Written offsets would not be recorded and no more records would be sent since the producer is fenced, indicating the task may be migrated out"); + $"{_logPrefix}Written offsets would not be recorded and no more records would be sent since the producer is fenced, indicating the task may be migrated out"); log.LogError(sb.ToString()); - lock (_lock) - exception = new TaskMigratedException(sb.ToString()); + _exception = new TaskMigratedException(sb.ToString()); } else { - var exceptionResponse = configuration.ProductionExceptionHandler(report); + var exceptionResponse = _configuration.ProductionExceptionHandler(report); if (exceptionResponse == ProductionExceptionHandlerResponse.FAIL) { sb.AppendLine( - $"{logPrefix}Exception handler choose to FAIL the processing, no more records would be sent."); + $"{_logPrefix}Exception handler choose to FAIL the processing, no more records would be sent."); log.LogError(sb.ToString()); - lock (_lock) - exception = new ProductionException(sb.ToString()); + _exception = new ProductionException(sb.ToString()); } else if (exceptionResponse == ProductionExceptionHandlerResponse.RETRY) { sb.AppendLine( - $"{logPrefix}Exception handler choose to RETRY sending the message to the next iteration"); + $"{_logPrefix}Exception handler choose to RETRY sending the message to the next iteration"); log.LogWarning(sb.ToString()); - var retryRecord = new RetryRecord() + var retryRecord = new RetryRecord { Key = report.Key, Value = report.Value, @@ -302,14 +266,14 @@ private void HandleError(DeliveryReport report) Partition = report.Partition, Topic = report.Topic }; - retryRecordContext.AddRecord(retryRecord); + _retryRecordContext.AddRecord(retryRecord); } else { sb.AppendLine( - $"{logPrefix}Exception handler choose to CONTINUE processing in spite of this error but written offsets would not be recorded."); + $"{_logPrefix}Exception handler choose to CONTINUE processing in spite of this error but written offsets would not be recorded."); log.LogError(sb.ToString()); - droppedRecordsSensor.Record(); + _droppedRecordsSensor.Record(); } } } @@ -318,31 +282,33 @@ private void HandleError(DeliveryReport report) { log.LogWarning( "{logPrefix}Record not persisted or possibly persisted: (timestamp {Timestamp}) topic=[{Topic}] partition=[{Partition}] offset=[{Offset}]. May config Retry configuration, depends your use case", - logPrefix, report.Message.Timestamp.UnixTimestampMs, report.Topic, report.Partition, report.Offset); + _logPrefix, report.Message.Timestamp.UnixTimestampMs, report.Topic, report.Partition, + report.Offset); } else if (report.Status == PersistenceStatus.Persisted) { log.LogDebug( "{LogPrefix}Record persisted: (timestamp {Timestamp}) topic=[{Topic}] partition=[{Partition}] offset=[{Offset}]", - logPrefix, report.Message.Timestamp.UnixTimestampMs, report.Topic, report.Partition, report.Offset); - if (collectorsOffsets.ContainsKey(report.TopicPartition) && - collectorsOffsets[report.TopicPartition] < report.Offset.Value) - collectorsOffsets.TryUpdate(report.TopicPartition, report.Offset.Value, - collectorsOffsets[report.TopicPartition]); + _logPrefix, report.Message.Timestamp.UnixTimestampMs, report.Topic, report.Partition, + report.Offset); + if (_collectorsOffsets.ContainsKey(report.TopicPartition) && + _collectorsOffsets[report.TopicPartition] < report.Offset.Value) + _collectorsOffsets.TryUpdate(report.TopicPartition, report.Offset.Value, + _collectorsOffsets[report.TopicPartition]); else - collectorsOffsets.TryAdd(report.TopicPartition, report.Offset); - retryRecordContext.AckRecord(report); + _collectorsOffsets.TryAdd(report.TopicPartition, report.Offset); + _retryRecordContext.AckRecord(report); } } private void ProduceRetryRecord() { - var retryRecord = retryRecordContext.NextRecord(); + var retryRecord = _retryRecordContext.NextRecord(); if (retryRecord != null) { try { - producer.Produce( + _producer.Produce( new TopicPartition(retryRecord.Topic, retryRecord.Partition), new Message { @@ -354,6 +320,10 @@ private void ProduceRetryRecord() } catch (ProduceException produceException) { + if (StreamsProducer.IsRecoverable(produceException.Error)) + throw new TaskMigratedException( + $"Producer got fenced trying to send a record: {produceException.Message}"); + ManageProduceException(produceException); } } @@ -361,17 +331,11 @@ private void ProduceRetryRecord() private void ManageProduceException(ProduceException produceException) { - if (IsRecoverableError(produceException.Error)) - { - throw new TaskMigratedException( - $"Producer got fenced trying to send a record [{logPrefix}] : {produceException.Message}"); - } - StringBuilder sb = new StringBuilder(); sb.AppendLine( - $"{logPrefix}Error encountered sending record to topic {produceException.DeliveryResult.Topic} for task {id} due to:"); - sb.AppendLine($"{logPrefix}Error Code : {produceException.Error.Code.ToString()}"); - sb.AppendLine($"{logPrefix}Message : {produceException.Error.Reason}"); + $"{_logPrefix}Error encountered sending record to topic {produceException.DeliveryResult.Topic} for task {_id} due to:"); + sb.AppendLine($"{_logPrefix}Error Code : {produceException.Error.Code.ToString()}"); + sb.AppendLine($"{_logPrefix}Message : {produceException.Error.Reason}"); var buildDeliveryReport = new DeliveryReport @@ -383,21 +347,21 @@ private void ManageProduceException(ProduceException produceExce produceException.Error) }; - var exceptionHandlerResponse = configuration.ProductionExceptionHandler(buildDeliveryReport); + var exceptionHandlerResponse = _configuration.ProductionExceptionHandler(buildDeliveryReport); if (exceptionHandlerResponse == ProductionExceptionHandlerResponse.FAIL) { sb.AppendLine( - $"{logPrefix}Exception handler choose to FAIL the processing, no more records would be sent."); + $"{_logPrefix}Exception handler choose to FAIL the processing, no more records would be sent."); log.LogError(sb.ToString()); throw new StreamsException( - $"Error encountered trying to send record to topic {produceException.DeliveryResult.Topic} [{logPrefix}] : {produceException.Message}"); + $"Error encountered trying to send record to topic {produceException.DeliveryResult.Topic} [{_logPrefix}] : {produceException.Message}"); } if (exceptionHandlerResponse == ProductionExceptionHandlerResponse.RETRY) { sb.AppendLine( - $"{logPrefix}Exception handler choose to RETRY sending the message to the next iteration"); + $"{_logPrefix}Exception handler choose to RETRY sending the message to the next iteration"); log.LogWarning(sb.ToString()); var retryRecord = new RetryRecord { @@ -408,40 +372,25 @@ private void ManageProduceException(ProduceException produceExce Partition = buildDeliveryReport.Partition, Topic = buildDeliveryReport.Topic }; - retryRecordContext.AddRecord(retryRecord); + _retryRecordContext.AddRecord(retryRecord); } else if (exceptionHandlerResponse == ProductionExceptionHandlerResponse.CONTINUE) { sb.AppendLine( - $"{logPrefix}Exception handler choose to CONTINUE processing in spite of this error but written offsets would not be recorded."); + $"{_logPrefix}Exception handler choose to CONTINUE processing in spite of this error but written offsets would not be recorded."); log.LogError(sb.ToString()); } } private void CheckForException() { - lock (_lock) - { - if (exception == null) return; - var e = exception; - exception = null; - throw e; - } + if (_exception == null) return; + var e = _exception; + _exception = null; + 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; - } + => _producer.PartitionsFor(topic); } } \ No newline at end of file diff --git a/core/Kafka/Internal/StreamsProducer.cs b/core/Kafka/Internal/StreamsProducer.cs new file mode 100644 index 00000000..0fe43ac8 --- /dev/null +++ b/core/Kafka/Internal/StreamsProducer.cs @@ -0,0 +1,232 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using Confluent.Kafka; +using Microsoft.Extensions.Logging; +using Streamiz.Kafka.Net.Errors; +using Streamiz.Kafka.Net.Processors; +using Streamiz.Kafka.Net.Crosscutting; + +namespace Streamiz.Kafka.Net.Kafka.Internal +{ + /// + /// Manages the producers within a Kafka Streams application. + /// If EOS is enabled, it is responsible to init and begin transactions if necessary. + /// It also tracks the transaction status, ie, if a transaction is in-fight. + /// For non-EOS, the user should not call transaction related methods. + /// + internal class StreamsProducer : IDisposable + { + private readonly ILogger log = Logger.GetLogger(typeof(StreamsProducer)); + private readonly string _logPrefix; + private IProducer _producer; + private bool _transactionInitialized = false; + private readonly IStreamConfig _configuration; + private readonly IKafkaSupplier _kafkaSupplier; + private readonly ProducerConfig _producerConfig; + private readonly IAdminClient _adminClient; + + public bool EosEnabled { get; } + public bool TransactionInFlight { get; private set; } + + // for testing + internal IProducer Producer => _producer; + + public string Name => _producer.Name; + + private IDictionary _cachePartitionsForTopics = + new Dictionary(); + + public StreamsProducer( + IStreamConfig config, + String threadId, + Guid processId, + IKafkaSupplier kafkaSupplier, + String logPrefix) + { + _logPrefix = logPrefix; + _configuration = config; + _kafkaSupplier = kafkaSupplier; + + _producerConfig = config.ToProducerConfig(StreamThread.GetThreadProducerClientId(threadId)) + .Wrap(threadId); + + _adminClient = kafkaSupplier.GetAdmin(config.ToAdminConfig(threadId)); + + switch (config.Guarantee) + { + case ProcessingGuarantee.AT_LEAST_ONCE: + break; + case ProcessingGuarantee.EXACTLY_ONCE: + _producerConfig.TransactionalId = $"{config.ApplicationId}-{processId}"; + break; + default: + throw new StreamsException($"Guarantee {config.Guarantee} is not supported yet"); + } + + _producer = _kafkaSupplier.GetProducer(_producerConfig); + EosEnabled = config.Guarantee == ProcessingGuarantee.EXACTLY_ONCE; + } + + internal static bool IsRecoverable(Error error) + { + return error.Code == ErrorCode.InvalidProducerIdMapping || + error.Code == ErrorCode.ProducerFenced || + error.Code == ErrorCode.InvalidProducerEpoch || + error.Code == ErrorCode.UnknownProducerId; + } + + public void InitTransaction() + { + if (!EosEnabled) + throw new IllegalStateException("Exactly-once is not enabled"); + + if (!_transactionInitialized) + { + try + { + _producer.InitTransactions(_configuration.TransactionTimeout); + _transactionInitialized = true; + } + catch (KafkaRetriableException) + { + log.LogWarning( + "Timeout exception caught trying to initialize transactions. " + + "The broker is either slow or in bad state (like not having enough replicas) in " + + "responding to the request, or the connection to broker was interrupted sending " + + "the request or receiving the response. " + + "Will retry initializing the task in the next loop. " + + "Consider overwriting 'transaction.timeout' to a larger value to avoid timeout errors" + ); + throw; + } + catch (KafkaException kafkaException) + { + throw new StreamsException("Error encountered trying to initialize transactions", kafkaException); + } + } + } + + public void ResetProducer() + { + Close(); + _producer = _kafkaSupplier.GetProducer(_producerConfig); + } + + private void StartTransaction() + { + if (EosEnabled && !TransactionInFlight) + { + try + { + _producer.BeginTransaction(); + TransactionInFlight = true; + } + catch (KafkaException kafkaException) + { + if (IsRecoverable(kafkaException.Error)) + throw new TaskMigratedException( + $"Producer got fenced trying to begin a new transaction : {kafkaException.Message}"); + + throw new StreamsException("Error encountered trying to begin a new transaction", kafkaException); + } + } + } + + public void Produce(TopicPartition topicPartition, Message message, + Action> deliveryReport) + { + StartTransaction(); + _producer.Produce(topicPartition, message, deliveryReport); + } + + public void Produce(string topic, Message message, + Action> deliveryReport) + { + StartTransaction(); + _producer.Produce(topic, message, deliveryReport); + } + + public void CommitTransaction(IEnumerable offsets, IConsumerGroupMetadata groupMetadata) + { + if (!EosEnabled) + throw new IllegalStateException("Exactly-once is not enabled"); + + StartTransaction(); + try + { + _producer.SendOffsetsToTransaction(offsets, groupMetadata, _configuration.TransactionTimeout); + _producer.CommitTransaction(); + TransactionInFlight = false; + } + catch (KafkaException kafkaException) + { + if (IsRecoverable(kafkaException.Error)) + throw new TaskMigratedException( + $"Producer got fenced trying to commit a transaction : {kafkaException.Message}"); + + throw new StreamsException("Error encountered trying to commit a transaction", kafkaException); + } + } + + public void AbortTransaction() + { + if (!EosEnabled) + throw new IllegalStateException("Exactly-once is not enabled"); + + if (TransactionInFlight) + { + try + { + _producer.AbortTransaction(); + } + catch (KafkaException kafkaException) + { + if (IsRecoverable(kafkaException.Error)) + log.LogDebug( + $"Encountered {kafkaException.Message} while aborting the transaction; this is expected and hence swallowed"); + else + throw new StreamsException("Error encounter trying to abort a transaction", kafkaException); + + } + finally + { + TransactionInFlight = false; + } + } + } + + public void Flush() + { + _producer.Flush(); + } + + private void Close() + { + TransactionInFlight = false; + _transactionInitialized = false; + } + + public void Dispose() + { + Close(); + _producer.Dispose(); + _adminClient?.Dispose(); + } + + 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.AddOrUpdate(topic, (partitionCount, DateTime.Now)); + return partitionCount; + } + } +} \ No newline at end of file diff --git a/core/Kafka/Internal/StreamsRebalanceListener.cs b/core/Kafka/Internal/StreamsRebalanceListener.cs index a8f25f75..39b51e01 100644 --- a/core/Kafka/Internal/StreamsRebalanceListener.cs +++ b/core/Kafka/Internal/StreamsRebalanceListener.cs @@ -30,19 +30,26 @@ public void PartitionsAssigned(IConsumer consumer, List /// Token for propagates notification that the stream should be canceled. - [Obsolete("This method is deprecated, please use StartAsync(...) instead. It will be removed in next release version.")] + [Obsolete("This method is deprecated, please use StartAsync(...) instead. It will be removed in 1.8.0.")] public void Start(CancellationToken? token = null) { StartAsync(token).ConfigureAwait(false).GetAwaiter().GetResult(); diff --git a/core/Mock/ClusterInMemoryTopologyDriver.cs b/core/Mock/ClusterInMemoryTopologyDriver.cs index 63100725..ce63c794 100644 --- a/core/Mock/ClusterInMemoryTopologyDriver.cs +++ b/core/Mock/ClusterInMemoryTopologyDriver.cs @@ -64,6 +64,7 @@ public ClusterInMemoryTopologyDriver(string clientId, InternalTopologyBuilder to threadTopology = StreamThread.Create( $"{this.configuration.ApplicationId.ToLower()}-stream-thread-0", + Guid.NewGuid(), clientId, topologyBuilder, metricsRegistry, diff --git a/core/Mock/Pipes/SyncPipeOutput.cs b/core/Mock/Pipes/SyncPipeOutput.cs index 905bc51b..3230e9e0 100644 --- a/core/Mock/Pipes/SyncPipeOutput.cs +++ b/core/Mock/Pipes/SyncPipeOutput.cs @@ -4,6 +4,7 @@ using System; using System.Collections.Generic; using System.Threading; +using Streamiz.Kafka.Net.Kafka.Internal; namespace Streamiz.Kafka.Net.Mock.Pipes { diff --git a/core/Mock/Sync/SyncConsumer.cs b/core/Mock/Sync/SyncConsumer.cs index 355f2dc1..eab9cec3 100644 --- a/core/Mock/Sync/SyncConsumer.cs +++ b/core/Mock/Sync/SyncConsumer.cs @@ -6,12 +6,15 @@ using System.Globalization; using System.Linq; using System.Threading; +using Microsoft.Extensions.Logging; +using Streamiz.Kafka.Net.Crosscutting; +using Streamiz.Kafka.Net.Kafka.Internal; namespace Streamiz.Kafka.Net.Mock.Sync { internal class SyncConsumer : IConsumer, IConsumerGroupMetadata { - internal class SyncConsumerOffset + private class SyncConsumerOffset { public long OffsetCommitted { get; set; } = 0L; public long OffsetConsumed { get; set; } = 0L; @@ -319,6 +322,7 @@ private ConsumeResult ConsumeInternal(TimeSpan timeout) Partition = 0, Message = messages[kp.Value.OffsetConsumed] }; + ++kp.Value.OffsetConsumed; return result; } diff --git a/core/Mock/Sync/SyncPipeBuilder.cs b/core/Mock/Sync/SyncPipeBuilder.cs index e236596a..c4840dad 100644 --- a/core/Mock/Sync/SyncPipeBuilder.cs +++ b/core/Mock/Sync/SyncPipeBuilder.cs @@ -6,6 +6,7 @@ using System.Threading.Tasks; using Confluent.Kafka; using Streamiz.Kafka.Net.Crosscutting; +using Streamiz.Kafka.Net.Kafka.Internal; using Streamiz.Kafka.Net.Processors.Internal; namespace Streamiz.Kafka.Net.Mock.Sync diff --git a/core/Mock/Sync/SyncProducer.cs b/core/Mock/Sync/SyncProducer.cs index dada1fa7..240563d2 100644 --- a/core/Mock/Sync/SyncProducer.cs +++ b/core/Mock/Sync/SyncProducer.cs @@ -4,6 +4,8 @@ using System.Security.Cryptography; using System.Threading; using System.Threading.Tasks; +using Microsoft.Extensions.Logging; +using Streamiz.Kafka.Net.Crosscutting; namespace Streamiz.Kafka.Net.Mock.Sync { @@ -28,14 +30,11 @@ public void Commit() (metadata as SyncConsumer).Commit(offsets); } } - - private readonly static object _lock = new object(); - private readonly Dictionary>> topics = new Dictionary>>(); - - private SyncTransaction transaction = null; - + private static readonly object _lock = new(); + private readonly Dictionary>> topics = new(); + private SyncTransaction transaction; private ProducerConfig config; - + public SyncProducer(){} public SyncProducer(ProducerConfig config) { this.config = config; } @@ -113,8 +112,9 @@ public void InitTransactions(TimeSpan timeout) public void Produce(string topic, Message message, Action> deliveryHandler = null) { CreateTopic(topic); - - topics[topic].Add(message); + + lock (_lock) + topics[topic].Add(message); DeliveryReport r = new DeliveryReport(); diff --git a/core/Mock/TaskSynchronousTopologyDriver.cs b/core/Mock/TaskSynchronousTopologyDriver.cs index 8b19368c..9b3d0fa2 100644 --- a/core/Mock/TaskSynchronousTopologyDriver.cs +++ b/core/Mock/TaskSynchronousTopologyDriver.cs @@ -11,6 +11,7 @@ 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; @@ -33,7 +34,7 @@ internal sealed class TaskSynchronousTopologyDriver : IBehaviorTopologyTestDrive private readonly IDictionary> partitionsByTaskId = new Dictionary>(); - private readonly SyncProducer producer = null; + private readonly StreamsProducer producer = null; private readonly bool hasGlobalTopology = false; private ITopicManager internalTopicManager; @@ -62,7 +63,10 @@ public TaskSynchronousTopologyDriver(string clientId, InternalTopologyBuilder to builder = topologyBuilder; this.supplier = supplier ?? new SyncKafkaSupplier(); this.supplier.MetricsRegistry = metricsRegistry; - producer = this.supplier.GetProducer(configuration.ToProducerConfig()) as SyncProducer; + + producer = new StreamsProducer( + configuration, + clientId, Guid.NewGuid(), this.supplier, string.Empty); foreach (var sourceTopic in builder .GetSourceTopics()) @@ -100,7 +104,7 @@ public TaskSynchronousTopologyDriver(string clientId, InternalTopologyBuilder to "ext-thread-0", taskId, topologyBuilder.BuildTopology(taskId).GetSourceProcessor(requestTopic), - this.supplier.GetProducer(configuration.ToProducerConfig($"ext-thread-producer-{requestTopic}")), + producer, configuration, metricsRegistry, adminClient)); @@ -246,7 +250,7 @@ public TestInputTopic CreateInputTopic(string topicName, ISerDes public TestOutputTopic CreateOutputTopic(string topicName, TimeSpan consumeTimeout, ISerDes keySerdes = null, ISerDes valueSerdes = null) { - var pipeBuilder = new SyncPipeBuilder(producer); + var pipeBuilder = new SyncPipeBuilder(producer.Producer as SyncProducer); var pipeOutput = pipeBuilder.Output(topicName, consumeTimeout, configuration, token); return new TestOutputTopic(pipeOutput, topicConfiguration, keySerdes, valueSerdes); } @@ -258,7 +262,8 @@ public void Dispose() foreach (var t in tasks.Values) { bool persistent = t.IsPersistent; - t.Close(); + t.Suspend(); + t.Close(false); // Remove local state store for this task if (persistent) @@ -304,9 +309,9 @@ public void TriggerCommit() foreach (var task in tasks.Values) { - var consumer = supplier.GetConsumer(topicConfiguration.ToConsumerConfig("consumer-repartition-forwarder"), - null); - task.Commit(); + // var consumer = supplier.GetConsumer(topicConfiguration.ToConsumerConfig("consumer-repartition-forwarder"),null); + var offsets = task.PrepareCommit(); + task.PostCommit(false); } globalTask?.FlushState(); diff --git a/core/Processors/AbstractTask.cs b/core/Processors/AbstractTask.cs index de62321f..2dddefdc 100644 --- a/core/Processors/AbstractTask.cs +++ b/core/Processors/AbstractTask.cs @@ -86,13 +86,13 @@ public bool IsPersistent public abstract IDictionary PurgeOffsets { get; } public abstract PartitionGrouper Grouper { get; } public abstract bool CanProcess(long now); - public abstract void Close(); - public abstract void Commit(); + public abstract void Close(bool dirty); + public abstract IEnumerable PrepareCommit(); + public abstract void PostCommit(bool enforceCheckpoint); public abstract IStateStore GetStore(string name); public abstract void InitializeTopology(); public abstract void RestorationIfNeeded(); public abstract bool InitializeStateStores(); - public abstract void Resume(); public abstract void Suspend(); public abstract void MayWriteCheckpoint(bool force = false); public abstract TaskScheduled RegisterScheduleTask(TimeSpan interval, PunctuationType punctuationType, Action punctuator); diff --git a/core/Processors/ExternalProcessorTopologyExecutor.cs b/core/Processors/ExternalProcessorTopologyExecutor.cs index 76e681d2..27f9b6d6 100644 --- a/core/Processors/ExternalProcessorTopologyExecutor.cs +++ b/core/Processors/ExternalProcessorTopologyExecutor.cs @@ -32,6 +32,7 @@ internal enum ExternalProcessorTopologyState } // TODO : maybe use this task instead of ExternalProcessorTopologyExecutor + // TODO : need to refactor #360 internal class ExternalStreamTask : AbstractTask { public override TaskId Id { get; } @@ -49,56 +50,58 @@ public override bool CanProcess(long now) throw new NotImplementedException(); } - public override void Suspend() + public override void Close(bool dirty) { throw new NotImplementedException(); } - public override TaskScheduled RegisterScheduleTask(TimeSpan interval, PunctuationType punctuationType, - Action punctuator) + public override IEnumerable PrepareCommit() { throw new NotImplementedException(); } - - public override void MayWriteCheckpoint(bool force = false) + + public override void PostCommit(bool enforceCheckpoint) { throw new NotImplementedException(); } - public override void Close() + public override void Suspend() { throw new NotImplementedException(); } - public override IStateStore GetStore(string name) + public override TaskScheduled RegisterScheduleTask(TimeSpan interval, PunctuationType punctuationType, + Action punctuator) { throw new NotImplementedException(); } - - public override void InitializeTopology() + + public override void MayWriteCheckpoint(bool force = false) { throw new NotImplementedException(); } + - public override void RestorationIfNeeded() + public override IStateStore GetStore(string name) { throw new NotImplementedException(); } - public override bool InitializeStateStores() + public override void InitializeTopology() { throw new NotImplementedException(); } - public override void Resume() + public override void RestorationIfNeeded() { throw new NotImplementedException(); } - public override void Commit() + public override bool InitializeStateStores() { throw new NotImplementedException(); } + #endregion } @@ -123,13 +126,13 @@ public ExternalProcessorTopologyExecutor( string threadId, TaskId taskId, ISourceProcessor sourceProcessor, - IProducer producer, + StreamsProducer producer, IStreamConfig config, StreamMetricsRegistry streamMetricsRegistry, IAdminClient adminClient) { this.threadId = threadId; - this.producerId = producer.Name; + producerId = producer.Name; this.streamMetricsRegistry = streamMetricsRegistry; State = ExternalProcessorTopologyState.RUNNING; Processor = sourceProcessor; @@ -141,8 +144,7 @@ public ExternalProcessorTopologyExecutor( processSensor = TaskMetrics.ProcessSensor(threadId, taskId, streamMetricsRegistry); processLatencySensor = TaskMetrics.ProcessLatencySensor(threadId, taskId, streamMetricsRegistry); - recordCollector = new RecordCollector(logPrefix, config, taskId, droppedRecordsSensor, adminClient); - recordCollector.Init(ref producer); + recordCollector = new RecordCollector(logPrefix, config, taskId, producer, droppedRecordsSensor); context = new ProcessorContext(ExternalStreamTask.Create(taskId), config, null, streamMetricsRegistry); context.UseRecordCollector(recordCollector); @@ -272,7 +274,7 @@ public void Close() { log.LogInformation($"{logPrefix}Closing"); Processor.Close(); - recordCollector.Close(); + recordCollector.Close(false); streamMetricsRegistry.RemoveTaskSensors(threadId, context.Id.ToString()); streamMetricsRegistry.RemoveLibrdKafkaSensors(threadId, producerId); log.LogInformation($"{logPrefix}Closed"); diff --git a/core/Processors/ExternalStreamThread.cs b/core/Processors/ExternalStreamThread.cs index 480045fb..fb085875 100644 --- a/core/Processors/ExternalStreamThread.cs +++ b/core/Processors/ExternalStreamThread.cs @@ -47,7 +47,7 @@ internal class ExternalStreamThread : IThread private readonly Sensor pollSensor; private readonly Sensor processLatencySensor; private readonly Sensor processRateSensor; - private IProducer producer; + private StreamsProducer producer; public ExternalStreamThread( string threadId, @@ -335,7 +335,10 @@ public void Start(CancellationToken token) currentConsumer = GetConsumer(); adminClient = kafkaSupplier.GetAdmin(configuration.ToAdminConfig(clientId)); - producer = kafkaSupplier.GetProducer(configuration.ToExternalProducerConfig($"{thread.Name}-producer").Wrap(Name, configuration)); + + producer = new StreamsProducer(configuration, Name, Guid.NewGuid(), kafkaSupplier, logPrefix); + + //producer = kafkaSupplier.GetProducer(configuration.ToExternalProducerConfig($"{thread.Name}-producer").Wrap(Name, configuration)); SetState(ThreadState.PARTITIONS_ASSIGNED); thread.Start(); diff --git a/core/Processors/ITask.cs b/core/Processors/ITask.cs index f6e61b85..7fbe78ee 100644 --- a/core/Processors/ITask.cs +++ b/core/Processors/ITask.cs @@ -22,13 +22,13 @@ internal interface ITask bool InitializeStateStores(); - void Commit(); - - void Resume(); + IEnumerable PrepareCommit(); + void PostCommit(bool enforceCheckpoint); + void Suspend(); - void Close(); + void Close(bool dirty); IStateStore GetStore(String name); diff --git a/core/Processors/Internal/AbstractTaskCreator.cs b/core/Processors/Internal/AbstractTaskCreator.cs index cd1505e5..1739d004 100644 --- a/core/Processors/Internal/AbstractTaskCreator.cs +++ b/core/Processors/Internal/AbstractTaskCreator.cs @@ -2,6 +2,7 @@ using Confluent.Kafka; using Microsoft.Extensions.Logging; using Streamiz.Kafka.Net.Crosscutting; +using Streamiz.Kafka.Net.Kafka.Internal; namespace Streamiz.Kafka.Net.Processors.Internal { @@ -14,12 +15,12 @@ protected AbstractTaskCreator() log = Logger.GetLogger(GetType()); } - public ICollection CreateTasks(IConsumer consumer, IDictionary> tasksToBeCreated) + public ICollection CreateTasks(IConsumer consumer, StreamsProducer producer, IDictionary> tasksToBeCreated) { List createdTasks = new List(); foreach (var newTaskAndPartitions in tasksToBeCreated) { - T task = CreateTask(consumer, newTaskAndPartitions.Key, newTaskAndPartitions.Value); + T task = CreateTask(consumer, producer, newTaskAndPartitions.Key, newTaskAndPartitions.Value); if (task != null) { log.LogDebug("Created task {NewTaskAndPartitionsKey} with assigned partition {PartitionValues}", newTaskAndPartitions.Key, @@ -31,6 +32,6 @@ public ICollection CreateTasks(IConsumer consumer, IDictionar return createdTasks; } - public abstract T CreateTask(IConsumer consumer, TaskId id, IEnumerable partition); + public abstract T CreateTask(IConsumer consumer, StreamsProducer producer, TaskId id, IEnumerable partition); } } \ No newline at end of file diff --git a/core/Processors/Internal/InternalTopologyBuilder.cs b/core/Processors/Internal/InternalTopologyBuilder.cs index 85cd2935..91d3588c 100644 --- a/core/Processors/Internal/InternalTopologyBuilder.cs +++ b/core/Processors/Internal/InternalTopologyBuilder.cs @@ -462,6 +462,7 @@ private void BuildProcessorNode(IDictionary processors, IDic { if (!stateStores.ContainsKey(stateStoreName)) { + IStateStore store; if (stateFactories.ContainsKey(stateStoreName)) { StateStoreFactory stateStoreFactory = stateFactories[stateStoreName]; @@ -472,12 +473,17 @@ private void BuildProcessorNode(IDictionary processors, IDic storesToTopics.Add(stateStoreName, changelogTopic); } - stateStores.Add(stateStoreName, stateStoreFactory.Build(taskId)); + store = stateStoreFactory.Build(taskId); + stateStores.Add(stateStoreName, store); } else { - stateStores.Add(stateStoreName, GlobalStateStores[stateStoreName]); + store = GlobalStateStores[stateStoreName]; + stateStores.Add(stateStoreName, store); } + + if (store.Persistent) + HasPersistentStores = true; } } } @@ -511,6 +517,8 @@ internal void BuildTopology(RootNode root, IList nodes) .SelectMany(group => group.Value) .ToHashSet(); + public bool HasPersistentStores { get; private set; } = false; + private bool IsGlobalSource(string node) { var factory = nodeFactories[node]; diff --git a/core/Processors/Internal/RecordQueue.cs b/core/Processors/Internal/RecordQueue.cs index 7c355756..ee0779e5 100644 --- a/core/Processors/Internal/RecordQueue.cs +++ b/core/Processors/Internal/RecordQueue.cs @@ -1,5 +1,6 @@ using System; using System.Collections.Generic; +using System.Diagnostics; using Confluent.Kafka; using Microsoft.Extensions.Logging; using Streamiz.Kafka.Net.Crosscutting; @@ -122,6 +123,13 @@ private void UpdateHeadRecord() private ConsumeResult ToConsumeObject(ConsumeResult record) { + if (record == null) + throw new ArgumentNullException("record"); + if (record.Message == null) + throw new ArgumentNullException("record.Message"); + if (sourceProcessor == null) + throw new ArgumentNullException("sourceProcessor"); + return new ConsumeResult { Topic = record.Topic, diff --git a/core/Processors/Internal/StateDirectory.cs b/core/Processors/Internal/StateDirectory.cs new file mode 100644 index 00000000..420b0643 --- /dev/null +++ b/core/Processors/Internal/StateDirectory.cs @@ -0,0 +1,87 @@ +using System; +using System.IO; +using System.Runtime.CompilerServices; +using Microsoft.Extensions.Logging; +using Newtonsoft.Json; +using Streamiz.Kafka.Net.Crosscutting; + +namespace Streamiz.Kafka.Net.Processors.Internal +{ + /// + /// Manages the directories where the state of Tasks owned by a are stored. + /// This class is thread-safe. + /// + internal class StateDirectory + { + private static StateDirectory staticDirectoryInstance; + + private static ILogger log = Logger.GetLogger(typeof(StateDirectory)); + private bool hasPersistentStores; + private IStreamConfig config; + + internal StateDirectory(IStreamConfig streamConfig, bool hasPersistentStores) + { + this.hasPersistentStores = hasPersistentStores; + config = streamConfig; + } + + private const String PROCESS_FILE_NAME = "streamiz-process-metadata"; + + private class StateDirectoryProcessFile { + + [JsonProperty("processId")] + public Guid ProcessId { get; set; } + + public StateDirectoryProcessFile(Guid processId) + { + ProcessId = processId; + } + + public StateDirectoryProcessFile() { + + } + } + + public static StateDirectory GetInstance(IStreamConfig config, bool hasPersistentStores) + { + if (staticDirectoryInstance == null) + staticDirectoryInstance = new StateDirectory(config, hasPersistentStores); + + return staticDirectoryInstance; + } + + [MethodImpl(MethodImplOptions.Synchronized)] + public Guid InitializeProcessId() + { + if (!hasPersistentStores) + { + Guid processId = Guid.NewGuid(); + log.LogInformation($"Created new process id: {processId}"); + return processId; + } + + string path = Path.Combine(config.StateDir, PROCESS_FILE_NAME); + if (File.Exists(path)) + { + try + { + string jsonString = File.ReadAllText(path); + StateDirectoryProcessFile processFile = + JsonConvert.DeserializeObject(jsonString); + log.LogInformation($"Reading UUID from process file: {processFile.ProcessId}"); + return processFile.ProcessId; + } + catch (Exception e) + { + log.LogWarning($"Failed to read json process file : {e.Message}"); + } + } + + StateDirectoryProcessFile processFileData = new StateDirectoryProcessFile(Guid.NewGuid()); + log.LogInformation($"No process id found on disk, got fresh process id {processFileData.ProcessId}"); + + File.WriteAllText(path, JsonConvert.SerializeObject(processFileData)); + return processFileData.ProcessId; + } + } +} \ No newline at end of file diff --git a/core/Processors/Internal/TaskCreator.cs b/core/Processors/Internal/TaskCreator.cs index c4f9a2df..a52fa9e0 100644 --- a/core/Processors/Internal/TaskCreator.cs +++ b/core/Processors/Internal/TaskCreator.cs @@ -2,6 +2,7 @@ using Confluent.Kafka; using Microsoft.Extensions.Logging; using Streamiz.Kafka.Net.Kafka; +using Streamiz.Kafka.Net.Kafka.Internal; using Streamiz.Kafka.Net.Metrics; using Streamiz.Kafka.Net.Metrics.Internal; @@ -18,22 +19,23 @@ internal class TaskCreator : AbstractTaskCreator private readonly StreamMetricsRegistry streamMetricsRegistry; private readonly Sensor createTaskSensor; + internal IStreamConfig Configuration => configuration; + public TaskCreator(InternalTopologyBuilder builder, IStreamConfig configuration, string threadId, - IKafkaSupplier kafkaSupplier, IProducer producer, StoreChangelogReader storeChangelogReader, + IKafkaSupplier kafkaSupplier, StoreChangelogReader storeChangelogReader, StreamMetricsRegistry streamMetricsRegistry) { this.builder = builder; this.configuration = configuration; this.threadId = threadId; this.kafkaSupplier = kafkaSupplier; - this.producer = producer; this.storeChangelogReader = storeChangelogReader; this.streamMetricsRegistry = streamMetricsRegistry; createTaskSensor = ThreadMetrics.CreateTaskSensor(threadId, streamMetricsRegistry); } - public override StreamTask CreateTask(IConsumer consumer, TaskId id, IEnumerable partitions) + public override StreamTask CreateTask(IConsumer consumer, StreamsProducer producer, TaskId id, IEnumerable partitions) { log.LogDebug($"Created task {id} with assigned partition {string.Join(",", partitions)}"); var task = new StreamTask( diff --git a/core/Processors/Internal/TaskManager.cs b/core/Processors/Internal/TaskManager.cs index 0f8c0e73..99be0aaf 100644 --- a/core/Processors/Internal/TaskManager.cs +++ b/core/Processors/Internal/TaskManager.cs @@ -6,8 +6,10 @@ using Confluent.Kafka; using Confluent.Kafka.Admin; using Microsoft.Extensions.Logging; +using Microsoft.Win32.SafeHandles; using Streamiz.Kafka.Net.Crosscutting; using Streamiz.Kafka.Net.Errors; +using Streamiz.Kafka.Net.Kafka.Internal; namespace Streamiz.Kafka.Net.Processors.Internal { @@ -42,21 +44,24 @@ internal static StreamTask CurrentTask public IDictionary Tasks => activeTasks.ToDictionary(i => i.Key, i => (ITask)i.Value); public IConsumer Consumer { get; internal set; } + public StreamsProducer Producer { get; } public IEnumerable ActiveTaskIds => activeTasks.Keys; public bool RebalanceInProgress { get; internal set; } internal readonly object _lock = new object(); internal TaskManager(InternalTopologyBuilder builder, TaskCreator taskCreator, IAdminClient adminClient, - IChangelogReader changelogReader) + IChangelogReader changelogReader, StreamsProducer producer) { this.builder = builder; this.taskCreator = taskCreator; this.adminClient = adminClient; this.changelogReader = changelogReader; + + Producer = producer; } - internal TaskManager(InternalTopologyBuilder builder, TaskCreator taskCreator, IAdminClient adminClient, IConsumer consumer, IChangelogReader changelogReader) - : this(builder, taskCreator, adminClient, changelogReader) + internal TaskManager(InternalTopologyBuilder builder, TaskCreator taskCreator, IAdminClient adminClient, IConsumer consumer, IChangelogReader changelogReader, StreamsProducer producer) + : this(builder, taskCreator, adminClient, changelogReader, producer) { Consumer = consumer; } @@ -82,7 +87,7 @@ public void CreateTasks(ICollection assignment) if (tasksToBeCreated.Count > 0) { - var tasks = taskCreator.CreateTasks(Consumer, tasksToBeCreated); + var tasks = taskCreator.CreateTasks(Consumer, Producer, tasksToBeCreated); foreach (var task in tasks) { task.InitializeStateStores(); @@ -92,21 +97,52 @@ public void CreateTasks(ICollection assignment) } } - public void RevokeTasks(ICollection assignment) + public void RevokeTasks(ICollection revokeAssignment) { CurrentTask = null; - foreach (var p in assignment) + List revokedTask = new List(); + List commitNeededActiveTask = new List(); + List consumedOffsetsToCommit = new List(); + + foreach (var p in revokeAssignment) { var taskId = builder.GetTaskIdFromPartition(p); if (activeTasks.TryGetValue(taskId, out StreamTask task)) - { - task.MayWriteCheckpoint(true); - task.Close(); + { + revokedTask.Add(task); partitionsToTaskId.TryRemove(p, out _); activeTasks.TryRemove(taskId, out _); } } + + foreach (var activeTask in activeTasks) + { + if(activeTask.Value.CommitNeeded) + commitNeededActiveTask.Add(activeTask.Value); + } + + foreach (var rvTask in revokedTask) + consumedOffsetsToCommit.AddRange(rvTask.PrepareCommit()); + + if (consumedOffsetsToCommit.Any()) + { + foreach(var acT in commitNeededActiveTask) + consumedOffsetsToCommit.AddRange(acT.PrepareCommit()); + } + + CommitOffsetsOrTransaction(consumedOffsetsToCommit); + + foreach (var rvTask in revokedTask) + { + rvTask.PostCommit(true); + rvTask.Suspend(); + rvTask.Close(false); + } + + foreach(var acT in commitNeededActiveTask) + acT.PostCommit(false); + } public StreamTask ActiveTaskFor(TopicPartition partition) @@ -124,18 +160,36 @@ public StreamTask ActiveTaskFor(TopicPartition partition) public void Close() { + List tasksToCommit = new List(); + List consumedOffsets = new List(); CurrentTask = null; foreach (var t in activeTasks) { CurrentTask = t.Value; - t.Value.MayWriteCheckpoint(true); - t.Value.Close(); + tasksToCommit.Add(t.Value); + consumedOffsets.AddRange(t.Value.PrepareCommit()); + } + + CurrentTask = null; + + CommitOffsetsOrTransaction(consumedOffsets); + + foreach (var task in activeTasks.Values) // flush all active tasks + { + CurrentTask = task; + task.PostCommit(true); + task.Suspend(); + task.Close(false); } + + Producer.Dispose(); activeTasks.Clear(); CurrentTask = null; partitionsToTaskId.Clear(); + + changelogReader.Clear(); // if one delete request is in progress, we wait the result before closing the manager if (currentDeleteTask is {IsCompleted: false}) @@ -151,26 +205,65 @@ internal int CommitAll() return -1; } + List tasksToCommit = new List(); + List consumedOffsets = new List(); + foreach (var t in ActiveTasks) { CurrentTask = t; if (t.CommitNeeded || t.CommitRequested) { purgeOffsets.AddRange(t.PurgeOffsets); - t.Commit(); - t.MayWriteCheckpoint(); - ++committed; + tasksToCommit.Add(t); + consumedOffsets.AddRange(t.PrepareCommit()); } } CurrentTask = null; + CommitOffsetsOrTransaction(consumedOffsets); + + foreach (var task in tasksToCommit) + { + CurrentTask = task; + task.PostCommit(false); + ++committed; + } + + CurrentTask = null; + if (committed > 0) // try to purge the committed records for repartition topics if possible PurgeCommittedRecords(purgeOffsets); return committed; } + private void CommitOffsetsOrTransaction(IEnumerable offsets) + { + var offsetsToCommit = offsets.ToList(); + if (taskCreator.Configuration.Guarantee == ProcessingGuarantee.AT_LEAST_ONCE) + { + if (offsetsToCommit.Any()) + { + try + { + Consumer.Commit(offsetsToCommit); + } + catch (KafkaException kafkaException) + { + throw new TaskMigratedException( + $"Consumer committing offsets failed, indicating the corresponding thread is no longer part of the group : {kafkaException.Message}"); + } + } + }else if (taskCreator.Configuration.Guarantee == ProcessingGuarantee.EXACTLY_ONCE) + { + if (offsetsToCommit.Any() || Producer.TransactionInFlight) + { + Producer.CommitTransaction(offsetsToCommit, Consumer.ConsumerGroupMetadata); + } + } + } + internal int Process(long now) { int processed = 0; @@ -238,19 +331,27 @@ internal void HandleLostAll() log.LogDebug("Closing lost active tasks as zombies"); CurrentTask = null; + if (Producer.EosEnabled) + Producer.AbortTransaction(); + var enumerator = activeTasks.GetEnumerator(); while (enumerator.MoveNext()) { var task = enumerator.Current.Value; + task.PrepareCommit(); task.Suspend(); + task.PostCommit(true); + task.Close(true); + foreach(var part in task.Partition) { partitionsToTaskId.TryRemove(part, out TaskId taskId); } - task.MayWriteCheckpoint(true); - task.Close(); } activeTasks.Clear(); + + if (Producer.EosEnabled) + Producer.ResetProducer(); } internal bool NeedRestoration() diff --git a/core/Processors/Internal/UnassignedStreamTask.cs b/core/Processors/Internal/UnassignedStreamTask.cs index 12125a8e..79a8f60c 100644 --- a/core/Processors/Internal/UnassignedStreamTask.cs +++ b/core/Processors/Internal/UnassignedStreamTask.cs @@ -1,13 +1,13 @@ using System.Collections.Generic; using Confluent.Kafka; using Streamiz.Kafka.Net.Kafka; -using Streamiz.Kafka.Net.Kafka.Internal; using Streamiz.Kafka.Net.Metrics; using Streamiz.Kafka.Net.Mock.Sync; using Streamiz.Kafka.Net.Stream.Internal; namespace Streamiz.Kafka.Net.Processors.Internal { + // TODO : remove soon internal class UnassignedStreamTask : StreamTask { private readonly IStreamConfig config; @@ -24,7 +24,7 @@ internal UnassignedProcessorContext(AbstractTask task, IStreamConfig configurati } UnassignedStreamTask(string threadId, TaskId id, IEnumerable partitions, ProcessorTopology processorTopology, IConsumer consumer, IStreamConfig configuration, IKafkaSupplier kafkaSupplier, IProducer producer) - : base(threadId, id, partitions, processorTopology, consumer, configuration, kafkaSupplier, producer, null, new StreamMetricsRegistry()) + : base(threadId, id, partitions, processorTopology, consumer, configuration, kafkaSupplier, null, null, new StreamMetricsRegistry()) { config = configuration; } diff --git a/core/Processors/StreamTask.cs b/core/Processors/StreamTask.cs index 7c2c4096..9f9818fb 100644 --- a/core/Processors/StreamTask.cs +++ b/core/Processors/StreamTask.cs @@ -1,6 +1,7 @@ using System; using System.Collections.Generic; using System.Linq; +using System.Threading.Tasks; using Confluent.Kafka; using Microsoft.Extensions.Logging; using Streamiz.Kafka.Net.Crosscutting; @@ -31,7 +32,6 @@ internal class StreamTask : AbstractTask private readonly List systemTimePunctuationQueue = new(); private long idleStartTime; - private IProducer producer; private bool transactionInFlight; private readonly string threadId; @@ -47,7 +47,7 @@ internal class StreamTask : AbstractTask public StreamTask(string threadId, TaskId id, IEnumerable partitions, ProcessorTopology processorTopology, IConsumer consumer, IStreamConfig configuration, - IKafkaSupplier kafkaSupplier, IProducer producer, IChangelogRegister changelogRegister, + IKafkaSupplier kafkaSupplier, StreamsProducer producer, IChangelogRegister changelogRegister, StreamMetricsRegistry streamMetricsRegistry) : base(id, partitions, processorTopology, consumer, configuration, changelogRegister) { @@ -60,22 +60,9 @@ public StreamTask(string threadId, TaskId id, IEnumerable partit followMetadata = configuration.FollowMetadata; idleStartTime = -1; - // eos enabled - if (producer == null) - { - this.producer = CreateEOSProducer(); - InitializeTransaction(); - eosEnabled = true; - } - else - { - this.producer = producer; - } - + eosEnabled = configuration.Guarantee == ProcessingGuarantee.EXACTLY_ONCE; var droppedRecordsSensor = TaskMetrics.DroppedRecordsSensor(this.threadId, Id, this.streamMetricsRegistry); - var adminClient = kafkaSupplier.GetAdmin(configuration.ToAdminConfig(this.threadId)); - collector = new RecordCollector(logPrefix, configuration, id, droppedRecordsSensor, adminClient); - collector.Init(ref this.producer); + collector = new RecordCollector(logPrefix, configuration, id, producer, droppedRecordsSensor); Context = new ProcessorContext(this, configuration, stateMgr, streamMetricsRegistry) .UseRecordCollector(collector); @@ -131,119 +118,6 @@ private IEnumerable GetPartitionsWithOffset() } } - private void Commit(bool startNewTransaction) - { - log.LogDebug($"{logPrefix}Comitting"); - - if (state == TaskState.CLOSED) - throw new IllegalStateException($"Illegal state {state} while committing active task {Id}"); - if (state == TaskState.SUSPENDED || state == TaskState.CREATED - || state == TaskState.RUNNING || state == TaskState.RESTORING) - { - FlushState(); - if (eosEnabled) - { - bool repeat = false; - do - { - try - { - var offsets = GetPartitionsWithOffset().ToList(); - log.LogDebug($"Send offsets to transactions : {string.Join(",", offsets)}"); - producer.SendOffsetsToTransaction(offsets, consumer.ConsumerGroupMetadata, - configuration.TransactionTimeout); - producer.CommitTransaction(configuration.TransactionTimeout); - transactionInFlight = false; - } - catch (KafkaTxnRequiresAbortException e) - { - log.LogWarning( - $"{logPrefix}Committing failed with a non-fatal error: {e.Message}, the transaction will be aborted"); - producer.AbortTransaction(configuration.TransactionTimeout); - transactionInFlight = false; - } - catch (KafkaRetriableException e) - { - log.LogDebug($"{logPrefix}Committing failed with a non-fatal error: {e.Message}, going to repeat the operation"); - repeat = true; - } - } while (repeat); - - if (startNewTransaction) - { - producer.BeginTransaction(); - transactionInFlight = true; - } - - consumedOffsets.Clear(); - } - else - { - try - { - consumer.Commit(GetPartitionsWithOffset()); - consumedOffsets.Clear(); - } - catch (TopicPartitionOffsetException e) - { - log.LogError($"{logPrefix}Committing failed with a non-fatal error: {e.Message}, we can ignore this since commit may succeed still"); - } - catch (KafkaException e) - { - if (!e.Error.IsFatal) - { - if (e.Error.Code == - ErrorCode.IllegalGeneration) // Broker: Specified group generation id is not valid - { - log.LogWarning($"{logPrefix}Error with a non-fatal error during committing offset (ignore this, and try to commit during next time): {e.Message}"); - return; - } - log.LogWarning($"{logPrefix}Error with a non-fatal error during committing offset (ignore this, and try to commit during next time): {e.Message}"); - } - else - throw; - } - } - commitNeeded = false; - commitRequested = false; - commitSensor.Record(); - } - else - throw new IllegalStateException($"Unknown state {state} while committing active task {Id}"); - } - - private IProducer CreateEOSProducer() - { - IProducer tmpProducer = null; - var newConfig = configuration.Clone(); - log.LogInformation($"${logPrefix}Creating producer client for task {Id}"); - newConfig.TransactionalId = $"{newConfig.ApplicationId}-{Id}"; - tmpProducer = kafkaSupplier.GetProducer(newConfig.ToProducerConfig(StreamThread.GetTaskProducerClientId(threadId, Id)).Wrap(threadId, Id)); - return tmpProducer; - } - - private void InitializeTransaction() - { - bool initTransaction = false; - while (!initTransaction) - { - try - { - producer.InitTransactions(configuration.TransactionTimeout); - initTransaction = true; - } - catch (KafkaRetriableException) - { - initTransaction = false; - } - catch (KafkaException e) - { - throw new StreamsException($"{logPrefix}Failed to initialize task {Id} due to timeout ({configuration.TransactionTimeout}).", e); - } - } - - } - private TaskScheduled ScheduleTask(long startTime, TimeSpan interval, PunctuationType punctuationType, Action punctuator) { var taskScheduled = new TaskScheduled( @@ -314,12 +188,10 @@ public override bool CanProcess(long now) return false; } - public override void Close() + public override void Close(bool dirty) { log.LogInformation($"{logPrefix}Closing"); - Suspend(); - if (state == TaskState.CREATED || state == TaskState.RESTORING || state == TaskState.RUNNING) { throw new IllegalStateException($"Illegal state {state} while closing active task {Id}"); @@ -333,6 +205,8 @@ public override void Close() if (state == TaskState.SUSPENDED) { + partitionGrouper.Close(); + foreach (var kp in processors) { kp.Close(); @@ -341,9 +215,8 @@ public override void Close() streamTimePunctuationQueue.ForEach(t => t.Close()); systemTimePunctuationQueue.ForEach(t => t.Close()); - partitionGrouper.Close(); - - collector.Close(); + collector.Close(dirty); + CloseStateManager(); TransitTo(TaskState.CLOSED); @@ -362,8 +235,64 @@ public override void Close() systemTimePunctuationQueue.Clear(); } - public override void Commit() => Commit(true); + private IEnumerable CommittableTopicPartitionOffsets() + { + if (state == TaskState.CREATED || state == TaskState.RESTORING) + return new List(); + if (state == TaskState.RUNNING || state == TaskState.SUSPENDED) + return GetPartitionsWithOffset(); + + throw new IllegalStateException( + $"Illegal state {state} while getting committable offsets for active task {Id}"); + } + + public override IEnumerable PrepareCommit() + { + log.LogDebug($"{logPrefix}Preparing to commit"); + + if (state == TaskState.CREATED || + state == TaskState.RUNNING || + state == TaskState.RESTORING || + state == TaskState.SUSPENDED) + { + if (commitNeeded || commitRequested) + { + FlushState(); + return CommittableTopicPartitionOffsets(); + } + + log.LogDebug($"Skipped preparing {state} task for commit since there is nothing to commit"); + return new List(); + } + + throw new IllegalStateException($"Illegal state {state} while preparing active task {Id} for committing"); + } + public override void PostCommit(bool enforceCheckpoint) + { + if (state == TaskState.CREATED) + log.LogDebug($"Skipped writing checkpoint for {state} task"); + else if (state == TaskState.RESTORING || state == TaskState.SUSPENDED) + { + MayWriteCheckpoint(enforceCheckpoint); + log.LogDebug($"Finalized commit for {state} task with enforce checkpoint {enforceCheckpoint}"); + } + else if (state == TaskState.RUNNING) + { + if(enforceCheckpoint || !eosEnabled) + MayWriteCheckpoint(enforceCheckpoint); + + consumedOffsets.Clear(); + commitNeeded = false; + commitRequested = false; + commitSensor.Record(); + + log.LogDebug($"Finalized commit for {state} task with enforce checkpoint {enforceCheckpoint}"); + } + else + throw new IllegalStateException($"Illegal state {state} while post committing active task {Id}"); + } + public override IStateStore GetStore(string name) { return Context.GetStateStore(name); @@ -399,11 +328,7 @@ public override void InitializeTopology() } Context.CurrentProcessor = null; - if (eosEnabled) - { - producer.BeginTransaction(); - transactionInFlight = true; - } + collector.Initialize(); taskInitialized = true; } @@ -414,110 +339,21 @@ public override bool InitializeStateStores() RegisterStateStores(); return false; } - - public override void Resume() - { - if (state == TaskState.CREATED || - state == TaskState.RESTORING || - state == TaskState.RUNNING) - { - log.LogDebug($"{logPrefix}Skip resuming since state is {state}"); - } - else if (state == TaskState.SUSPENDED) - { - log.LogDebug($"{logPrefix}Resuming"); - InitializeStateStores(); - if (eosEnabled) - { - if (producer != null) - { - throw new IllegalStateException("Task producer should be null."); - } - - producer = CreateEOSProducer(); - InitializeTransaction(); - collector.Init(ref producer); - } - - RegisterSensors(); - - Context.CurrentProcessor = null; - - foreach (var p in processors) - p.Init(Context); - - Context.CurrentProcessor = null; - - TransitTo(TaskState.CREATED); - } - else if (state == TaskState.CLOSED) - { - throw new IllegalStateException($"Illegal state {state} while resuming active task {Id}"); - } - else - { - throw new IllegalStateException($"Unknow state {state} while resuming active task {Id}"); - } - } - + public override void Suspend() { log.LogDebug($"{logPrefix}Suspending"); if (state == TaskState.CREATED || state == TaskState.RESTORING) { - log.LogInformation($"{logPrefix}Suspended {(state == TaskState.CREATED ? "created" : "restoring")}"); - - // TODO : remove when stream task refactoring is finished - if (eosEnabled) - { - if (transactionInFlight) - { - producer.AbortTransaction(configuration.TransactionTimeout); - } - - collector.Close(); - producer = null; - } - - FlushState(); - MayWriteCheckpoint(true); - CloseStateManager(); - streamMetricsRegistry.RemoveTaskSensors(threadId, Id.ToString()); - foreach (var kp in processors) - kp.Close(); - + log.LogInformation($"{logPrefix}Suspended {state}"); TransitTo(TaskState.SUSPENDED); } else if (state == TaskState.RUNNING) { - try - { - Commit(false); - } - finally - { - partitionGrouper.Clear(); - - if (eosEnabled) - { - if (transactionInFlight) - { - producer.AbortTransaction(configuration.TransactionTimeout); - } - - collector.Close(); - producer = null; - } - - // duplicate FlushState(); - MayWriteCheckpoint(true); - CloseStateManager(); - streamMetricsRegistry.RemoveTaskSensors(threadId, Id.ToString()); - foreach (var kp in processors) - kp.Close(); - } - + foreach (var kp in processors) + kp.Close(); + partitionGrouper.Clear(); log.LogInformation($"{logPrefix}Suspended running"); TransitTo(TaskState.SUSPENDED); } diff --git a/core/Processors/StreamThread.cs b/core/Processors/StreamThread.cs index 8574e6c9..680e8362 100644 --- a/core/Processors/StreamThread.cs +++ b/core/Processors/StreamThread.cs @@ -17,10 +17,10 @@ internal class StreamThread : IThread { #region Static - public static string GetTaskProducerClientId(string threadClientId, TaskId taskId) - { - return threadClientId + "-" + taskId + "-streamiz-producer"; - } + //public static string GetTaskProducerClientId(string threadClientId, TaskId taskId) + //{ + // return threadClientId + "-" + taskId + "-streamiz-producer"; + //} public static string GetThreadProducerClientId(string threadClientId) { @@ -43,29 +43,15 @@ public static string GetSharedAdminClientId(string clientId) return clientId + "-streamiz-admin"; } - internal static IThread Create(string threadId, string clientId, InternalTopologyBuilder builder, + internal static IThread Create(string threadId, + Guid processId, + string clientId, InternalTopologyBuilder builder, StreamMetricsRegistry streamMetricsRegistry, IStreamConfig configuration, IKafkaSupplier kafkaSupplier, IAdminClient adminClient, int threadInd) { string logPrefix = $"stream-thread[{threadId}] "; var log = Logger.GetLogger(typeof(StreamThread)); var customerID = $"{clientId}-StreamThread-{threadInd}"; - IProducer producer = null; - - // TODO : remove this limitations depends version of Kafka Cluster - // Due to limitations outlined in KIP-447 (which KIP-447 overcomes), it is - // currently necessary to use a separate producer per input partition. The - // producerState dictionary is used to keep track of these, and the current - // consumed offset. - // https://cwiki.apache.org/confluence/display/KAFKA/KIP-447%3A+Producer+scalability+for+exactly+once+semantics - // IF Guarantee is AT_LEAST_ONCE, producer is the same of all StreamTasks in this thread, - // ELSE one producer by StreamTask. - if (configuration.Guarantee == ProcessingGuarantee.AT_LEAST_ONCE) - { - log.LogInformation("{LogPrefix}Creating shared producer client", logPrefix); - producer = kafkaSupplier.GetProducer(configuration.ToProducerConfig(GetThreadProducerClientId(threadId)) - .Wrap(threadId)); - } var restoreConfig = configuration.ToRestoreConsumerConfig(GetRestoreConsumerClientId(customerID)); var restoreConsumer = kafkaSupplier.GetRestoreConsumer(restoreConfig); @@ -76,9 +62,12 @@ internal static IThread Create(string threadId, string clientId, InternalTopolog threadId, streamMetricsRegistry); - var taskCreator = new TaskCreator(builder, configuration, threadId, kafkaSupplier, producer, + var taskCreator = new TaskCreator(builder, configuration, threadId, kafkaSupplier, storeChangelogReader, streamMetricsRegistry); - var manager = new TaskManager(builder, taskCreator, adminClient, storeChangelogReader); + + var producerThread = new StreamsProducer(configuration, threadId, processId, kafkaSupplier, logPrefix); + + var manager = new TaskManager(builder, taskCreator, adminClient, storeChangelogReader, producerThread); var listener = new StreamsRebalanceListener(manager); diff --git a/core/State/RocksDb/RocksDbKeyValueStore.cs b/core/State/RocksDb/RocksDbKeyValueStore.cs index 5ee75f4e..a6f624de 100644 --- a/core/State/RocksDb/RocksDbKeyValueStore.cs +++ b/core/State/RocksDb/RocksDbKeyValueStore.cs @@ -345,7 +345,7 @@ public IEnumerable> ReverseAll() tableConfig.SetBlockCache(RocksDbSharp.Cache.CreateLru(BLOCK_CACHE_SIZE)); tableConfig.SetBlockSize(BLOCK_SIZE); tableConfig.SetFilterPolicy(BloomFilterPolicy.Create()); - + rocksDbOptions.SetOptimizeFiltersForHits(1); rocksDbOptions.SetCompression(COMPRESSION_TYPE); rocksDbOptions.SetCompactionStyle(COMPACTION_STYLE); diff --git a/core/Stream/Topology.cs b/core/Stream/Topology.cs index a796bd89..abf894f1 100644 --- a/core/Stream/Topology.cs +++ b/core/Stream/Topology.cs @@ -18,7 +18,7 @@ namespace Streamiz.Kafka.Net.Stream /// public class Topology { - internal InternalTopologyBuilder Builder { get; } = new InternalTopologyBuilder(); + internal InternalTopologyBuilder Builder { get; } = new(); internal Topology() { diff --git a/core/StreamConfig.cs b/core/StreamConfig.cs index b1ad2eae..67062336 100644 --- a/core/StreamConfig.cs +++ b/core/StreamConfig.cs @@ -270,7 +270,7 @@ public interface IStreamConfig : ICloneable /// Authorize your streams application to follow metadata (timestamp, topic, partition, offset and headers) during processing record. /// You can use to get these metadatas. (Default : false) /// - [Obsolete("Plan to remove in the next release")] + [Obsolete("Plan to remove in 1.8.0")] bool FollowMetadata { get; set; } /// @@ -2599,7 +2599,7 @@ public long MaxTaskIdleMs /// Maximum number of records to buffer per partition. (Default: 1000) /// [StreamConfigProperty("" + bufferedRecordsPerPartitionCst)] - [Obsolete("Librdkafka clients manage internally a backpressure. So this configuration will be remove in the next release.")] + [Obsolete("Librdkafka clients manage internally a backpressure. So this configuration will be remove in 1.8.0")] public long BufferedRecordsPerPartition { get => configProperties[bufferedRecordsPerPartitionCst]; @@ -2742,7 +2742,7 @@ public MetricsRecordingLevel MetricsRecording /// (default: 5000) /// [StreamConfigProperty("" + startTaskDelayMsCst)] - [Obsolete] + [Obsolete("Remove in 1.8.0")] public long StartTaskDelayMs { get => configProperties[startTaskDelayMsCst]; diff --git a/core/StreamizMetadata.cs b/core/StreamizMetadata.cs index 237f2ddb..633a6895 100644 --- a/core/StreamizMetadata.cs +++ b/core/StreamizMetadata.cs @@ -32,7 +32,7 @@ namespace Streamiz.Kafka.Net /// /// /// - [Obsolete("Plan to remove in the next release")] + [Obsolete("Plan to remove in 1.8.0")] public static class StreamizMetadata { /// diff --git a/environment/start.sh b/environment/start.sh index 765f503e..1074d89f 100644 --- a/environment/start.sh +++ b/environment/start.sh @@ -17,4 +17,5 @@ curl -i -X PUT http://localhost:8083/connectors/datagen_product/config \ # curl -i -X PUT http://localhost:8083/connectors/datagen_product/pause -# curl -i -X PUT http://localhost:8083/connectors/datagen_product/resume \ No newline at end of file +# curl -i -X PUT http://localhost:8083/connectors/datagen_product/resume +# curl -X DELETE http://localhost:8083/connectors/datagen_product \ No newline at end of file diff --git a/feature.md b/feature.md deleted file mode 100644 index 3b48cd00..00000000 --- a/feature.md +++ /dev/null @@ -1,16 +0,0 @@ -# Roadmap Feature - -- [ ] Priority queue feature to process unordered records -- [ ] Dynamic windowing size -- [ ] Dead letter queue mecanism -- [ ] Backpressure pattern -- [ ] External Source Connector -- [ ] Request/Reply | External system inside the topology (refactor with a real async approach) -- [ ] Session Windows -- [ ] Dynamic Session Windows -- [ ] Singleton Registered SerDes -- [ ] Auto scaling consumption - - -- [ ] Evict cache store as a batch of records instead of unitary records -- [ ] State store restore handler with a batch approach \ No newline at end of file diff --git a/kafka-stream-net.sln b/kafka-stream-net.sln index 07abddf2..e776a6aa 100644 --- a/kafka-stream-net.sln +++ b/kafka-stream-net.sln @@ -5,41 +5,27 @@ VisualStudioVersion = 16.0.29519.87 MinimumVisualStudioVersion = 15.0.26124.0 Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Streamiz.Kafka.Net", "core\Streamiz.Kafka.Net.csproj", "{4812257D-B650-44DA-AD06-1E9EFC4E7992}" EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "sample-stream", "samples\sample-stream\sample-stream.csproj", "{57CA84AD-2E9F-4E4D-94B7-B8CF534B0B0E}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Producer", "test\Producer\Producer.csproj", "{A8367BFB-F9E0-43D6-AE4E-DA4FC353D413}" +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "sample-stream", "launcher\sample-stream\sample-stream.csproj", "{57CA84AD-2E9F-4E4D-94B7-B8CF534B0B0E}" EndProject Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Core", "Core", "{3C748A6C-9316-4507-8FEE-77C684C4F028}" EndProject -Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Sample", "Sample", "{FFB1B148-D781-4D08-A1E8-4A7F7147FCD4}" +Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Launcher", "Launcher", "{FFB1B148-D781-4D08-A1E8-4A7F7147FCD4}" EndProject Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Test", "Test", "{9526F3C9-8E0C-490B-A2B0-F6DA233314D0}" EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Consumer", "test\Consumer\Consumer.csproj", "{F725913E-F02E-4E77-9F99-A219C453E111}" -EndProject -Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "GenerateData", "GenerateData", "{E3C25908-114E-4CD0-BD07-4660152F3A70}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "sample-test-driver", "samples\sample-test-driver\sample-test-driver.csproj", "{5671E175-D880-4B49-A0ED-47F6531DDF32}" -EndProject Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Streamiz.Kafka.Net.Tests", "test\Streamiz.Kafka.Net.Tests\Streamiz.Kafka.Net.Tests.csproj", "{6AE2D31F-20D4-4A9B-8B90-62DE180C7E5C}" EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "sample-stream-registry", "samples\sample-stream-registry\sample-stream-registry.csproj", "{FFD63973-67BE-4142-AC82-A74D487B22B3}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Producer-Avro", "test\Producer-Avro\Producer-Avro.csproj", "{A8D0F9DD-AFC2-42E7-BD5B-8E667EEBBFDA}" -EndProject Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Streamiz.Kafka.Net.SchemaRegistry.SerDes.Avro", "serdes\Streamiz.Kafka.Net.SchemaRegistry.SerDes.Avro\Streamiz.Kafka.Net.SchemaRegistry.SerDes.Avro.csproj", "{2BC1A318-1AE9-4A32-B8B1-D8296FABFF8B}" EndProject Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Streamiz.Kafka.Net.SchemaRegistry.SerDes", "serdes\Streamiz.Kafka.Net.SchemaRegistry.SerDes\Streamiz.Kafka.Net.SchemaRegistry.SerDes.csproj", "{4D0FD00A-D2C5-470A-A808-C23F1448E68E}" EndProject Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Streamiz.Kafka.Net.SchemaRegistry.SerDes.Protobuf", "serdes\Streamiz.Kafka.Net.SchemaRegistry.SerDes.Protobuf\Streamiz.Kafka.Net.SchemaRegistry.SerDes.Protobuf.csproj", "{21CD1F56-F1AF-42BD-877B-43AC36A43793}" EndProject -Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "sample-micro-service", "samples\sample-micro-service\sample-micro-service.csproj", "{1D4EA25C-E34D-4B0D-BE81-826D4D36AC03}" -EndProject Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Crosscutting", "Crosscutting", "{A34792BA-25DD-44A5-B420-9CC810379BFE}" EndProject Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Streamiz.Kafka.Net.Metrics.Prometheus", "metrics\Streamiz.Kafka.Net.Metrics.Prometheus\Streamiz.Kafka.Net.Metrics.Prometheus.csproj", "{29CB687B-951E-4B3E-AD1C-13B222490091}" EndProject -Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "sample-stream-demo", "samples\sample-stream-demo\sample-stream-demo.csproj", "{55E20242-D62A-4E5C-8D14-5DC6E544B6FB}" +Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "sample-stream-demo", "launcher\sample-stream-demo\sample-stream-demo.csproj", "{55E20242-D62A-4E5C-8D14-5DC6E544B6FB}" EndProject Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Streamiz.Kafka.Net.SchemaRegistry.SerDes.Json", "serdes\Streamiz.Kafka.Net.SchemaRegistry.SerDes.Json\Streamiz.Kafka.Net.SchemaRegistry.SerDes.Json.csproj", "{84DDE273-6B34-40D7-80CC-E2C3BB541F30}" EndProject @@ -87,42 +73,6 @@ Global {57CA84AD-2E9F-4E4D-94B7-B8CF534B0B0E}.Release|x64.Build.0 = Release|Any CPU {57CA84AD-2E9F-4E4D-94B7-B8CF534B0B0E}.Release|x86.ActiveCfg = Release|Any CPU {57CA84AD-2E9F-4E4D-94B7-B8CF534B0B0E}.Release|x86.Build.0 = Release|Any CPU - {A8367BFB-F9E0-43D6-AE4E-DA4FC353D413}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {A8367BFB-F9E0-43D6-AE4E-DA4FC353D413}.Debug|Any CPU.Build.0 = Debug|Any CPU - {A8367BFB-F9E0-43D6-AE4E-DA4FC353D413}.Debug|x64.ActiveCfg = Debug|Any CPU - {A8367BFB-F9E0-43D6-AE4E-DA4FC353D413}.Debug|x64.Build.0 = Debug|Any CPU - {A8367BFB-F9E0-43D6-AE4E-DA4FC353D413}.Debug|x86.ActiveCfg = Debug|Any CPU - {A8367BFB-F9E0-43D6-AE4E-DA4FC353D413}.Debug|x86.Build.0 = Debug|Any CPU - {A8367BFB-F9E0-43D6-AE4E-DA4FC353D413}.Release|Any CPU.ActiveCfg = Release|Any CPU - {A8367BFB-F9E0-43D6-AE4E-DA4FC353D413}.Release|Any CPU.Build.0 = Release|Any CPU - {A8367BFB-F9E0-43D6-AE4E-DA4FC353D413}.Release|x64.ActiveCfg = Release|Any CPU - {A8367BFB-F9E0-43D6-AE4E-DA4FC353D413}.Release|x64.Build.0 = Release|Any CPU - {A8367BFB-F9E0-43D6-AE4E-DA4FC353D413}.Release|x86.ActiveCfg = Release|Any CPU - {A8367BFB-F9E0-43D6-AE4E-DA4FC353D413}.Release|x86.Build.0 = Release|Any CPU - {F725913E-F02E-4E77-9F99-A219C453E111}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {F725913E-F02E-4E77-9F99-A219C453E111}.Debug|Any CPU.Build.0 = Debug|Any CPU - {F725913E-F02E-4E77-9F99-A219C453E111}.Debug|x64.ActiveCfg = Debug|Any CPU - {F725913E-F02E-4E77-9F99-A219C453E111}.Debug|x64.Build.0 = Debug|Any CPU - {F725913E-F02E-4E77-9F99-A219C453E111}.Debug|x86.ActiveCfg = Debug|Any CPU - {F725913E-F02E-4E77-9F99-A219C453E111}.Debug|x86.Build.0 = Debug|Any CPU - {F725913E-F02E-4E77-9F99-A219C453E111}.Release|Any CPU.ActiveCfg = Release|Any CPU - {F725913E-F02E-4E77-9F99-A219C453E111}.Release|Any CPU.Build.0 = Release|Any CPU - {F725913E-F02E-4E77-9F99-A219C453E111}.Release|x64.ActiveCfg = Release|Any CPU - {F725913E-F02E-4E77-9F99-A219C453E111}.Release|x64.Build.0 = Release|Any CPU - {F725913E-F02E-4E77-9F99-A219C453E111}.Release|x86.ActiveCfg = Release|Any CPU - {F725913E-F02E-4E77-9F99-A219C453E111}.Release|x86.Build.0 = Release|Any CPU - {5671E175-D880-4B49-A0ED-47F6531DDF32}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {5671E175-D880-4B49-A0ED-47F6531DDF32}.Debug|Any CPU.Build.0 = Debug|Any CPU - {5671E175-D880-4B49-A0ED-47F6531DDF32}.Debug|x64.ActiveCfg = Debug|Any CPU - {5671E175-D880-4B49-A0ED-47F6531DDF32}.Debug|x64.Build.0 = Debug|Any CPU - {5671E175-D880-4B49-A0ED-47F6531DDF32}.Debug|x86.ActiveCfg = Debug|Any CPU - {5671E175-D880-4B49-A0ED-47F6531DDF32}.Debug|x86.Build.0 = Debug|Any CPU - {5671E175-D880-4B49-A0ED-47F6531DDF32}.Release|Any CPU.ActiveCfg = Release|Any CPU - {5671E175-D880-4B49-A0ED-47F6531DDF32}.Release|Any CPU.Build.0 = Release|Any CPU - {5671E175-D880-4B49-A0ED-47F6531DDF32}.Release|x64.ActiveCfg = Release|Any CPU - {5671E175-D880-4B49-A0ED-47F6531DDF32}.Release|x64.Build.0 = Release|Any CPU - {5671E175-D880-4B49-A0ED-47F6531DDF32}.Release|x86.ActiveCfg = Release|Any CPU - {5671E175-D880-4B49-A0ED-47F6531DDF32}.Release|x86.Build.0 = Release|Any CPU {6AE2D31F-20D4-4A9B-8B90-62DE180C7E5C}.Debug|Any CPU.ActiveCfg = Debug|Any CPU {6AE2D31F-20D4-4A9B-8B90-62DE180C7E5C}.Debug|Any CPU.Build.0 = Debug|Any CPU {6AE2D31F-20D4-4A9B-8B90-62DE180C7E5C}.Debug|x64.ActiveCfg = Debug|Any CPU @@ -135,30 +85,6 @@ Global {6AE2D31F-20D4-4A9B-8B90-62DE180C7E5C}.Release|x64.Build.0 = Release|Any CPU {6AE2D31F-20D4-4A9B-8B90-62DE180C7E5C}.Release|x86.ActiveCfg = Release|Any CPU {6AE2D31F-20D4-4A9B-8B90-62DE180C7E5C}.Release|x86.Build.0 = Release|Any CPU - {FFD63973-67BE-4142-AC82-A74D487B22B3}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {FFD63973-67BE-4142-AC82-A74D487B22B3}.Debug|Any CPU.Build.0 = Debug|Any CPU - {FFD63973-67BE-4142-AC82-A74D487B22B3}.Debug|x64.ActiveCfg = Debug|Any CPU - {FFD63973-67BE-4142-AC82-A74D487B22B3}.Debug|x64.Build.0 = Debug|Any CPU - {FFD63973-67BE-4142-AC82-A74D487B22B3}.Debug|x86.ActiveCfg = Debug|Any CPU - {FFD63973-67BE-4142-AC82-A74D487B22B3}.Debug|x86.Build.0 = Debug|Any CPU - {FFD63973-67BE-4142-AC82-A74D487B22B3}.Release|Any CPU.ActiveCfg = Release|Any CPU - {FFD63973-67BE-4142-AC82-A74D487B22B3}.Release|Any CPU.Build.0 = Release|Any CPU - {FFD63973-67BE-4142-AC82-A74D487B22B3}.Release|x64.ActiveCfg = Release|Any CPU - {FFD63973-67BE-4142-AC82-A74D487B22B3}.Release|x64.Build.0 = Release|Any CPU - {FFD63973-67BE-4142-AC82-A74D487B22B3}.Release|x86.ActiveCfg = Release|Any CPU - {FFD63973-67BE-4142-AC82-A74D487B22B3}.Release|x86.Build.0 = Release|Any CPU - {A8D0F9DD-AFC2-42E7-BD5B-8E667EEBBFDA}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {A8D0F9DD-AFC2-42E7-BD5B-8E667EEBBFDA}.Debug|Any CPU.Build.0 = Debug|Any CPU - {A8D0F9DD-AFC2-42E7-BD5B-8E667EEBBFDA}.Debug|x64.ActiveCfg = Debug|Any CPU - {A8D0F9DD-AFC2-42E7-BD5B-8E667EEBBFDA}.Debug|x64.Build.0 = Debug|Any CPU - {A8D0F9DD-AFC2-42E7-BD5B-8E667EEBBFDA}.Debug|x86.ActiveCfg = Debug|Any CPU - {A8D0F9DD-AFC2-42E7-BD5B-8E667EEBBFDA}.Debug|x86.Build.0 = Debug|Any CPU - {A8D0F9DD-AFC2-42E7-BD5B-8E667EEBBFDA}.Release|Any CPU.ActiveCfg = Release|Any CPU - {A8D0F9DD-AFC2-42E7-BD5B-8E667EEBBFDA}.Release|Any CPU.Build.0 = Release|Any CPU - {A8D0F9DD-AFC2-42E7-BD5B-8E667EEBBFDA}.Release|x64.ActiveCfg = Release|Any CPU - {A8D0F9DD-AFC2-42E7-BD5B-8E667EEBBFDA}.Release|x64.Build.0 = Release|Any CPU - {A8D0F9DD-AFC2-42E7-BD5B-8E667EEBBFDA}.Release|x86.ActiveCfg = Release|Any CPU - {A8D0F9DD-AFC2-42E7-BD5B-8E667EEBBFDA}.Release|x86.Build.0 = Release|Any CPU {2BC1A318-1AE9-4A32-B8B1-D8296FABFF8B}.Debug|Any CPU.ActiveCfg = Debug|Any CPU {2BC1A318-1AE9-4A32-B8B1-D8296FABFF8B}.Debug|Any CPU.Build.0 = Debug|Any CPU {2BC1A318-1AE9-4A32-B8B1-D8296FABFF8B}.Debug|x64.ActiveCfg = Debug|Any CPU @@ -195,18 +121,6 @@ Global {21CD1F56-F1AF-42BD-877B-43AC36A43793}.Release|x64.Build.0 = Release|Any CPU {21CD1F56-F1AF-42BD-877B-43AC36A43793}.Release|x86.ActiveCfg = Release|Any CPU {21CD1F56-F1AF-42BD-877B-43AC36A43793}.Release|x86.Build.0 = Release|Any CPU - {1D4EA25C-E34D-4B0D-BE81-826D4D36AC03}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {1D4EA25C-E34D-4B0D-BE81-826D4D36AC03}.Debug|Any CPU.Build.0 = Debug|Any CPU - {1D4EA25C-E34D-4B0D-BE81-826D4D36AC03}.Debug|x64.ActiveCfg = Debug|Any CPU - {1D4EA25C-E34D-4B0D-BE81-826D4D36AC03}.Debug|x64.Build.0 = Debug|Any CPU - {1D4EA25C-E34D-4B0D-BE81-826D4D36AC03}.Debug|x86.ActiveCfg = Debug|Any CPU - {1D4EA25C-E34D-4B0D-BE81-826D4D36AC03}.Debug|x86.Build.0 = Debug|Any CPU - {1D4EA25C-E34D-4B0D-BE81-826D4D36AC03}.Release|Any CPU.ActiveCfg = Release|Any CPU - {1D4EA25C-E34D-4B0D-BE81-826D4D36AC03}.Release|Any CPU.Build.0 = Release|Any CPU - {1D4EA25C-E34D-4B0D-BE81-826D4D36AC03}.Release|x64.ActiveCfg = Release|Any CPU - {1D4EA25C-E34D-4B0D-BE81-826D4D36AC03}.Release|x64.Build.0 = Release|Any CPU - {1D4EA25C-E34D-4B0D-BE81-826D4D36AC03}.Release|x86.ActiveCfg = Release|Any CPU - {1D4EA25C-E34D-4B0D-BE81-826D4D36AC03}.Release|x86.Build.0 = Release|Any CPU {29CB687B-951E-4B3E-AD1C-13B222490091}.Debug|Any CPU.ActiveCfg = Debug|Any CPU {29CB687B-951E-4B3E-AD1C-13B222490091}.Debug|Any CPU.Build.0 = Debug|Any CPU {29CB687B-951E-4B3E-AD1C-13B222490091}.Debug|x64.ActiveCfg = Debug|Any CPU @@ -298,16 +212,10 @@ Global GlobalSection(NestedProjects) = preSolution {4812257D-B650-44DA-AD06-1E9EFC4E7992} = {3C748A6C-9316-4507-8FEE-77C684C4F028} {57CA84AD-2E9F-4E4D-94B7-B8CF534B0B0E} = {FFB1B148-D781-4D08-A1E8-4A7F7147FCD4} - {A8367BFB-F9E0-43D6-AE4E-DA4FC353D413} = {E3C25908-114E-4CD0-BD07-4660152F3A70} - {F725913E-F02E-4E77-9F99-A219C453E111} = {E3C25908-114E-4CD0-BD07-4660152F3A70} - {5671E175-D880-4B49-A0ED-47F6531DDF32} = {FFB1B148-D781-4D08-A1E8-4A7F7147FCD4} {6AE2D31F-20D4-4A9B-8B90-62DE180C7E5C} = {9526F3C9-8E0C-490B-A2B0-F6DA233314D0} - {FFD63973-67BE-4142-AC82-A74D487B22B3} = {FFB1B148-D781-4D08-A1E8-4A7F7147FCD4} - {A8D0F9DD-AFC2-42E7-BD5B-8E667EEBBFDA} = {E3C25908-114E-4CD0-BD07-4660152F3A70} {2BC1A318-1AE9-4A32-B8B1-D8296FABFF8B} = {3C748A6C-9316-4507-8FEE-77C684C4F028} {4D0FD00A-D2C5-470A-A808-C23F1448E68E} = {3C748A6C-9316-4507-8FEE-77C684C4F028} {21CD1F56-F1AF-42BD-877B-43AC36A43793} = {3C748A6C-9316-4507-8FEE-77C684C4F028} - {1D4EA25C-E34D-4B0D-BE81-826D4D36AC03} = {FFB1B148-D781-4D08-A1E8-4A7F7147FCD4} {29CB687B-951E-4B3E-AD1C-13B222490091} = {A34792BA-25DD-44A5-B420-9CC810379BFE} {55E20242-D62A-4E5C-8D14-5DC6E544B6FB} = {FFB1B148-D781-4D08-A1E8-4A7F7147FCD4} {84DDE273-6B34-40D7-80CC-E2C3BB541F30} = {3C748A6C-9316-4507-8FEE-77C684C4F028} diff --git a/samples/sample-micro-service/.dockerignore b/launcher/sample-stream-demo/.dockerignore similarity index 100% rename from samples/sample-micro-service/.dockerignore rename to launcher/sample-stream-demo/.dockerignore diff --git a/samples/sample-stream-demo/Dockerfile b/launcher/sample-stream-demo/Dockerfile similarity index 100% rename from samples/sample-stream-demo/Dockerfile rename to launcher/sample-stream-demo/Dockerfile diff --git a/samples/sample-stream-demo/Program.cs b/launcher/sample-stream-demo/Program.cs similarity index 100% rename from samples/sample-stream-demo/Program.cs rename to launcher/sample-stream-demo/Program.cs diff --git a/samples/sample-stream-demo/sample-stream-demo.csproj b/launcher/sample-stream-demo/sample-stream-demo.csproj similarity index 100% rename from samples/sample-stream-demo/sample-stream-demo.csproj rename to launcher/sample-stream-demo/sample-stream-demo.csproj diff --git a/samples/sample-stream-demo/.dockerignore b/launcher/sample-stream/.dockerignore similarity index 100% rename from samples/sample-stream-demo/.dockerignore rename to launcher/sample-stream/.dockerignore diff --git a/samples/sample-stream/Dockerfile b/launcher/sample-stream/Dockerfile similarity index 100% rename from samples/sample-stream/Dockerfile rename to launcher/sample-stream/Dockerfile diff --git a/samples/sample-stream/Program.cs b/launcher/sample-stream/Program.cs similarity index 68% rename from samples/sample-stream/Program.cs rename to launcher/sample-stream/Program.cs index ea94829e..9b3556fb 100644 --- a/samples/sample-stream/Program.cs +++ b/launcher/sample-stream/Program.cs @@ -1,5 +1,6 @@ using Streamiz.Kafka.Net; using System; +using System.Collections.Generic; using System.Threading.Tasks; using Confluent.Kafka; using Microsoft.Extensions.Logging; @@ -15,7 +16,7 @@ namespace sample_stream { public static class Program { - public static async Task Main(string[] args) + public static async Task Main(string[] args) { var rocksDbHandler = new BoundMemoryRocksDbConfigHandler() .ConfigureNumThreads(2) @@ -27,7 +28,7 @@ public static async Task Main(string[] args) ApplicationId = $"test-app", BootstrapServers = "localhost:9092", AutoOffsetReset = AutoOffsetReset.Earliest, - Logger = LoggerFactory.Create(b => + Logger = LoggerFactory.Create((b) => { b.AddConsole(); b.SetMinimumLevel(LogLevel.Information); @@ -51,30 +52,34 @@ public static async Task Main(string[] args) private static Topology BuildTopology() { var builder = new StreamBuilder(); - /*builder.Stream("input") + + builder.Stream("input3") + .Peek((k,v,c) => Console.WriteLine($"Key : {k} Context : {c.Topic}:{c.Partition}:{c.Offset}")) .GroupByKey() - .WindowedBy(TumblingWindowOptions.Of(windowSize)) + .WindowedBy(TumblingWindowOptions.Of(TimeSpan.FromMinutes(1))) .Count(RocksDbWindows.As("count-store") .WithKeySerdes(new StringSerDes()) - .WithValueSerdes(new Int64SerDes()) - .WithCachingEnabled()) + .WithValueSerdes(new Int64SerDes())) + //.WithCachingEnabled() .ToStream() - .Map((k,v) => new KeyValuePair(k.ToString(), v.ToString())) - .To("output", + .Map((k,v, _) => new KeyValuePair(k.ToString(), v.ToString())) + .To("output3", + new StringSerDes(), + new StringSerDes()); + + /*builder.Stream("input3") + .Peek((k,v,c) => Console.WriteLine($"Key : {k} Context : {c.Topic}:{c.Partition}:{c.Offset}")) + .To("output3", new StringSerDes(), - new StringSerDes());*/ + new StringSerDes()); + /*builder.Stream("input") .DropDuplicate((key, value1, value2) => value1.Equals(value2), TimeSpan.FromMinutes(1)) .To( - "output");*/ - - builder.Stream("users") - .GroupByKey() - .Count() - .ToStream("count_users"); - + "output");//, (s, s1, arg3, arg4) => new Partition(0)); + */ return builder.Build(); } } diff --git a/samples/sample-stream/cacert.pem b/launcher/sample-stream/cacert.pem similarity index 100% rename from samples/sample-stream/cacert.pem rename to launcher/sample-stream/cacert.pem diff --git a/samples/sample-stream/log4net.config b/launcher/sample-stream/log4net.config similarity index 100% rename from samples/sample-stream/log4net.config rename to launcher/sample-stream/log4net.config diff --git a/samples/sample-stream/sample-stream.csproj b/launcher/sample-stream/sample-stream.csproj similarity index 100% rename from samples/sample-stream/sample-stream.csproj rename to launcher/sample-stream/sample-stream.csproj diff --git a/run-demo.sh b/run-demo.sh index ecc605a7..3e73553d 100755 --- a/run-demo.sh +++ b/run-demo.sh @@ -53,5 +53,5 @@ dotnet build -f net6.0 --no-restore if [ $# -gt 0 ] then - dotnet run -f net6.0 --project samples/sample-stream-demo/sample-stream-demo.csproj --no-build --no-restore + dotnet run -f net6.0 --project launcher/sample-stream-demo/sample-stream-demo.csproj --no-build --no-restore fi \ No newline at end of file diff --git a/samples/sample-micro-service/Controllers/WeatherForecastController.cs b/samples/sample-micro-service/Controllers/WeatherForecastController.cs deleted file mode 100644 index 4d910045..00000000 --- a/samples/sample-micro-service/Controllers/WeatherForecastController.cs +++ /dev/null @@ -1,39 +0,0 @@ -using System; -using System.Collections.Generic; -using System.Linq; -using System.Threading.Tasks; -using Microsoft.AspNetCore.Mvc; -using Microsoft.Extensions.Logging; - -namespace sample_micro_service.Controllers -{ - [ApiController] - [Route("[controller]")] - public class WeatherForecastController : ControllerBase - { - private static readonly string[] Summaries = new[] - { - "Freezing", "Bracing", "Chilly", "Cool", "Mild", "Warm", "Balmy", "Hot", "Sweltering", "Scorching" - }; - - private readonly ILogger _logger; - - public WeatherForecastController(ILogger logger) - { - _logger = logger; - } - - [HttpGet] - public IEnumerable Get() - { - var rng = new Random(); - return Enumerable.Range(1, 5).Select(index => new WeatherForecast - { - Date = DateTime.Now.AddDays(index), - TemperatureC = rng.Next(-20, 55), - Summary = Summaries[rng.Next(Summaries.Length)] - }) - .ToArray(); - } - } -} \ No newline at end of file diff --git a/samples/sample-micro-service/Dockerfile b/samples/sample-micro-service/Dockerfile deleted file mode 100644 index aefa3b18..00000000 --- a/samples/sample-micro-service/Dockerfile +++ /dev/null @@ -1,20 +0,0 @@ -FROM mcr.microsoft.com/dotnet/aspnet:5.0 AS base -WORKDIR /app -EXPOSE 80 -EXPOSE 443 - -FROM mcr.microsoft.com/dotnet/sdk:5.0 AS build -WORKDIR /src -COPY ["samples/sample-micro-service/sample-micro-service.csproj", "sample-micro-service/"] -RUN dotnet restore "samples/sample-micro-service/sample-micro-service.csproj" -COPY . . -WORKDIR "/src/sample-micro-service" -RUN dotnet build "sample-micro-service.csproj" -c Release -o /app/build - -FROM build AS publish -RUN dotnet publish "sample-micro-service.csproj" -c Release -o /app/publish - -FROM base AS final -WORKDIR /app -COPY --from=publish /app/publish . -ENTRYPOINT ["dotnet", "sample-micro-service.dll"] diff --git a/samples/sample-micro-service/Program.cs b/samples/sample-micro-service/Program.cs deleted file mode 100644 index dbae2f61..00000000 --- a/samples/sample-micro-service/Program.cs +++ /dev/null @@ -1,23 +0,0 @@ -using System; -using System.Collections.Generic; -using System.Linq; -using System.Threading.Tasks; -using Microsoft.AspNetCore.Hosting; -using Microsoft.Extensions.Configuration; -using Microsoft.Extensions.Hosting; -using Microsoft.Extensions.Logging; - -namespace sample_micro_service -{ - public class Program - { - public static void Main(string[] args) - { - CreateHostBuilder(args).Build().Run(); - } - - public static IHostBuilder CreateHostBuilder(string[] args) => - Host.CreateDefaultBuilder(args) - .ConfigureWebHostDefaults(webBuilder => { webBuilder.UseStartup(); }); - } -} \ No newline at end of file diff --git a/samples/sample-micro-service/Properties/launchSettings.json b/samples/sample-micro-service/Properties/launchSettings.json deleted file mode 100644 index 03d87f66..00000000 --- a/samples/sample-micro-service/Properties/launchSettings.json +++ /dev/null @@ -1,31 +0,0 @@ -{ - "$schema": "http://json.schemastore.org/launchsettings.json", - "iisSettings": { - "windowsAuthentication": false, - "anonymousAuthentication": true, - "iisExpress": { - "applicationUrl": "http://localhost:22908", - "sslPort": 44352 - } - }, - "profiles": { - "IIS Express": { - "commandName": "IISExpress", - "launchBrowser": true, - "launchUrl": "swagger", - "environmentVariables": { - "ASPNETCORE_ENVIRONMENT": "Development" - } - }, - "sample_micro_service": { - "commandName": "Project", - "dotnetRunMessages": "true", - "launchBrowser": true, - "launchUrl": "swagger", - "applicationUrl": "https://localhost:5001;http://localhost:5000", - "environmentVariables": { - "ASPNETCORE_ENVIRONMENT": "Development" - } - } - } -} diff --git a/samples/sample-micro-service/Service/StreamizService.cs b/samples/sample-micro-service/Service/StreamizService.cs deleted file mode 100644 index ae802986..00000000 --- a/samples/sample-micro-service/Service/StreamizService.cs +++ /dev/null @@ -1,36 +0,0 @@ -using System; -using System.IO; -using System.Threading; -using System.Threading.Tasks; -using Microsoft.Extensions.Hosting; -using Streamiz.Kafka.Net; -using Streamiz.Kafka.Net.SerDes; -using Streamiz.Kafka.Net.Stream; -using Streamiz.Kafka.Net.Table; - -namespace sample_micro_service.Service -{ - public class StreamizService : BackgroundService - { - protected override async Task ExecuteAsync(CancellationToken stoppingToken) - { - var config = new StreamConfig(); - config.ApplicationId = "test-app"; - config.BootstrapServers = "localhost:9092"; - config.AutoOffsetReset = Confluent.Kafka.AutoOffsetReset.Earliest; - config.StateDir = Path.Combine(".", Guid.NewGuid().ToString()); - - StreamBuilder builder = new StreamBuilder(); - - builder - .Table("table", RocksDb.As("table-store")) - .ToStream() - .Print(Printed.ToOut()); - - Topology t = builder.Build(); - - KafkaStream stream = new KafkaStream(t, config); - await stream.StartAsync(stoppingToken); - } - } -} \ No newline at end of file diff --git a/samples/sample-micro-service/Startup.cs b/samples/sample-micro-service/Startup.cs deleted file mode 100644 index 040faedf..00000000 --- a/samples/sample-micro-service/Startup.cs +++ /dev/null @@ -1,57 +0,0 @@ -using System; -using System.Collections.Generic; -using System.Linq; -using System.Threading.Tasks; -using Microsoft.AspNetCore.Builder; -using Microsoft.AspNetCore.Hosting; -using Microsoft.AspNetCore.HttpsPolicy; -using Microsoft.AspNetCore.Mvc; -using Microsoft.Extensions.Configuration; -using Microsoft.Extensions.DependencyInjection; -using Microsoft.Extensions.Hosting; -using Microsoft.Extensions.Logging; -using Microsoft.OpenApi.Models; -using sample_micro_service.Service; - -namespace sample_micro_service -{ - public class Startup - { - public Startup(IConfiguration configuration) - { - Configuration = configuration; - } - - public IConfiguration Configuration { get; } - - // This method gets called by the runtime. Use this method to add services to the container. - public void ConfigureServices(IServiceCollection services) - { - services.AddControllers(); - services.AddSwaggerGen(c => - { - c.SwaggerDoc("v1", new OpenApiInfo {Title = "sample_micro_service", Version = "v1"}); - }); - services.AddHostedService(); - } - - // This method gets called by the runtime. Use this method to configure the HTTP request pipeline. - public void Configure(IApplicationBuilder app, IWebHostEnvironment env) - { - if (env.IsDevelopment()) - { - app.UseDeveloperExceptionPage(); - app.UseSwagger(); - app.UseSwaggerUI(c => c.SwaggerEndpoint("/swagger/v1/swagger.json", "sample_micro_service v1")); - } - - app.UseHttpsRedirection(); - - app.UseRouting(); - - app.UseAuthorization(); - - app.UseEndpoints(endpoints => { endpoints.MapControllers(); }); - } - } -} \ No newline at end of file diff --git a/samples/sample-micro-service/WeatherForecast.cs b/samples/sample-micro-service/WeatherForecast.cs deleted file mode 100644 index 6f1c7f8a..00000000 --- a/samples/sample-micro-service/WeatherForecast.cs +++ /dev/null @@ -1,15 +0,0 @@ -using System; - -namespace sample_micro_service -{ - public class WeatherForecast - { - public DateTime Date { get; set; } - - public int TemperatureC { get; set; } - - public int TemperatureF => 32 + (int) (TemperatureC / 0.5556); - - public string Summary { get; set; } - } -} \ No newline at end of file diff --git a/samples/sample-micro-service/appsettings.Development.json b/samples/sample-micro-service/appsettings.Development.json deleted file mode 100644 index 8983e0fc..00000000 --- a/samples/sample-micro-service/appsettings.Development.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "Logging": { - "LogLevel": { - "Default": "Information", - "Microsoft": "Warning", - "Microsoft.Hosting.Lifetime": "Information" - } - } -} diff --git a/samples/sample-micro-service/appsettings.json b/samples/sample-micro-service/appsettings.json deleted file mode 100644 index d9d9a9bf..00000000 --- a/samples/sample-micro-service/appsettings.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "Logging": { - "LogLevel": { - "Default": "Information", - "Microsoft": "Warning", - "Microsoft.Hosting.Lifetime": "Information" - } - }, - "AllowedHosts": "*" -} diff --git a/samples/sample-micro-service/sample-micro-service.csproj b/samples/sample-micro-service/sample-micro-service.csproj deleted file mode 100644 index ed46b18b..00000000 --- a/samples/sample-micro-service/sample-micro-service.csproj +++ /dev/null @@ -1,17 +0,0 @@ - - - - net6.0 - sample_micro_service - Linux - - - - - - - - - - - diff --git a/samples/sample-stream-registry/Program.cs b/samples/sample-stream-registry/Program.cs deleted file mode 100644 index 5628ee8c..00000000 --- a/samples/sample-stream-registry/Program.cs +++ /dev/null @@ -1,73 +0,0 @@ -using com.avro.bean; -using Confluent.Kafka; -using Streamiz.Kafka.Net; -using Streamiz.Kafka.Net.SchemaRegistry.SerDes.Avro; -using Streamiz.Kafka.Net.SerDes; -using Streamiz.Kafka.Net.Stream; -using Streamiz.Kafka.Net.Table; -using System; -using System.Threading; -using System.Threading.Tasks; - -namespace sample_stream_registry -{ - class Program - { - static async Task Main(string[] args) - { - CancellationTokenSource source = new CancellationTokenSource(); - - var config = new StreamConfig(); - config.ApplicationId = "test-app"; - config.BootstrapServers = "localhost:9092"; - // NEED FOR SchemaAvroSerDes - config.SchemaRegistryUrl = "http://localhost:8081"; - config.AutoRegisterSchemas = true; - - StreamBuilder builder = new StreamBuilder(); - - var table = builder.Table("product", - new Int32SerDes(), - new SchemaAvroSerDes(), - InMemory.As("product-store")); - - var orders = builder.Stream>("orders"); - - orders.Join(table, (order, product) => new OrderProduct - { - order_id = order.order_id, - price = order.price, - product_id = product.product_id, - product_name = product.name, - product_price = product.price - }) - .To>("orders-output"); - - orders - .GroupByKey() - .Aggregate>( - () => new OrderAgg(), - (key, order, agg) => - { - agg.order_id = order.order_id; - agg.price = order.price; - agg.product_id = order.product_id; - agg.totalPrice += order.price; - return agg; - }) - .ToStream() - .Print(Printed.ToOut()); - - Topology t = builder.Build(); - - KafkaStream stream = new KafkaStream(t, config); - - Console.CancelKeyPress += (o, e) => - { - stream.Dispose(); - }; - - await stream.StartAsync(); - } - } -} diff --git a/samples/sample-stream-registry/com/avro/bean/Order.cs b/samples/sample-stream-registry/com/avro/bean/Order.cs deleted file mode 100644 index 65b33703..00000000 --- a/samples/sample-stream-registry/com/avro/bean/Order.cs +++ /dev/null @@ -1,85 +0,0 @@ -// ------------------------------------------------------------------------------ -// -// Generated by avrogen, version 1.7.7.5 -// Changes to this file may cause incorrect behavior and will be lost if code -// is regenerated -// -// ------------------------------------------------------------------------------ -namespace com.avro.bean -{ - using System; - using System.Collections.Generic; - using System.Text; - using global::Avro; - using global::Avro.Specific; - - public partial class Order : ISpecificRecord - { - public static Schema _SCHEMA = Schema.Parse("{\"type\":\"record\",\"name\":\"Order\",\"namespace\":\"com.avro.bean\",\"fields\":[{\"name\":\"or" + - "der_id\",\"type\":\"int\"},{\"name\":\"price\",\"type\":\"float\"},{\"name\":\"product_id\",\"type" + - "\":\"int\"}]}"); - private int _order_id; - private float _price; - private int _product_id; - public virtual Schema Schema - { - get - { - return Order._SCHEMA; - } - } - public int order_id - { - get - { - return this._order_id; - } - set - { - this._order_id = value; - } - } - public float price - { - get - { - return this._price; - } - set - { - this._price = value; - } - } - public int product_id - { - get - { - return this._product_id; - } - set - { - this._product_id = value; - } - } - public virtual object Get(int fieldPos) - { - switch (fieldPos) - { - case 0: return this.order_id; - case 1: return this.price; - case 2: return this.product_id; - default: throw new AvroRuntimeException("Bad index " + fieldPos + " in Get()"); - }; - } - public virtual void Put(int fieldPos, object fieldValue) - { - switch (fieldPos) - { - case 0: this.order_id = (System.Int32)fieldValue; break; - case 1: this.price = (System.Single)fieldValue; break; - case 2: this.product_id = (System.Int32)fieldValue; break; - default: throw new AvroRuntimeException("Bad index " + fieldPos + " in Put()"); - }; - } - } -} diff --git a/samples/sample-stream-registry/com/avro/bean/OrderAgg.cs b/samples/sample-stream-registry/com/avro/bean/OrderAgg.cs deleted file mode 100644 index 85498207..00000000 --- a/samples/sample-stream-registry/com/avro/bean/OrderAgg.cs +++ /dev/null @@ -1,99 +0,0 @@ -// ------------------------------------------------------------------------------ -// -// Generated by avrogen, version 1.11.0.0 -// Changes to this file may cause incorrect behavior and will be lost if code -// is regenerated -// -// ------------------------------------------------------------------------------ -namespace com.avro.bean -{ - using System; - using System.Collections.Generic; - using System.Text; - using Avro; - using Avro.Specific; - - public partial class OrderAgg : ISpecificRecord - { - public static Schema _SCHEMA = Avro.Schema.Parse("{\"type\":\"record\",\"name\":\"OrderAgg\",\"namespace\":\"com.avro.bean\",\"fields\":[{\"name\":" + - "\"order_id\",\"type\":\"int\"},{\"name\":\"price\",\"type\":\"float\"},{\"name\":\"product_id\",\"t" + - "ype\":\"int\"},{\"name\":\"totalPrice\",\"type\":\"float\"}]}"); - private int _order_id; - private float _price; - private int _product_id; - private float _totalPrice; - public virtual Schema Schema - { - get - { - return OrderAgg._SCHEMA; - } - } - public int order_id - { - get - { - return this._order_id; - } - set - { - this._order_id = value; - } - } - public float price - { - get - { - return this._price; - } - set - { - this._price = value; - } - } - public int product_id - { - get - { - return this._product_id; - } - set - { - this._product_id = value; - } - } - public float totalPrice - { - get - { - return this._totalPrice; - } - set - { - this._totalPrice = value; - } - } - public virtual object Get(int fieldPos) - { - switch (fieldPos) - { - case 0: return this.order_id; - case 1: return this.price; - case 2: return this.product_id; - case 3: return this.totalPrice; - default: throw new AvroRuntimeException("Bad index " + fieldPos + " in Get()"); - }; - } - public virtual void Put(int fieldPos, object fieldValue) - { - switch (fieldPos) - { - case 0: this.order_id = (System.Int32)fieldValue; break; - case 1: this.price = (System.Single)fieldValue; break; - case 2: this.product_id = (System.Int32)fieldValue; break; - case 3: this.totalPrice = (System.Single)fieldValue; break; - default: throw new AvroRuntimeException("Bad index " + fieldPos + " in Put()"); - }; - } - } -} diff --git a/samples/sample-stream-registry/com/avro/bean/OrderProduct.cs b/samples/sample-stream-registry/com/avro/bean/OrderProduct.cs deleted file mode 100644 index 8db29ebf..00000000 --- a/samples/sample-stream-registry/com/avro/bean/OrderProduct.cs +++ /dev/null @@ -1,111 +0,0 @@ -// ------------------------------------------------------------------------------ -// -// Generated by avrogen, version 1.7.7.5 -// Changes to this file may cause incorrect behavior and will be lost if code -// is regenerated -// -// ------------------------------------------------------------------------------ -namespace com.avro.bean -{ - using System; - using System.Collections.Generic; - using System.Text; - using global::Avro; - using global::Avro.Specific; - - public partial class OrderProduct : ISpecificRecord - { - public static Schema _SCHEMA = Schema.Parse(@"{""type"":""record"",""name"":""OrderProduct"",""namespace"":""com.avro.bean"",""fields"":[{""name"":""order_id"",""type"":""int""},{""name"":""price"",""type"":""float""},{""name"":""product_id"",""type"":""int""},{""name"":""product_name"",""type"":""string""},{""name"":""product_price"",""type"":""float""}]}"); - private int _order_id; - private float _price; - private int _product_id; - private string _product_name; - private float _product_price; - public virtual Schema Schema - { - get - { - return OrderProduct._SCHEMA; - } - } - public int order_id - { - get - { - return this._order_id; - } - set - { - this._order_id = value; - } - } - public float price - { - get - { - return this._price; - } - set - { - this._price = value; - } - } - public int product_id - { - get - { - return this._product_id; - } - set - { - this._product_id = value; - } - } - public string product_name - { - get - { - return this._product_name; - } - set - { - this._product_name = value; - } - } - public float product_price - { - get - { - return this._product_price; - } - set - { - this._product_price = value; - } - } - public virtual object Get(int fieldPos) - { - switch (fieldPos) - { - case 0: return this.order_id; - case 1: return this.price; - case 2: return this.product_id; - case 3: return this.product_name; - case 4: return this.product_price; - default: throw new AvroRuntimeException("Bad index " + fieldPos + " in Get()"); - }; - } - public virtual void Put(int fieldPos, object fieldValue) - { - switch (fieldPos) - { - case 0: this.order_id = (System.Int32)fieldValue; break; - case 1: this.price = (System.Single)fieldValue; break; - case 2: this.product_id = (System.Int32)fieldValue; break; - case 3: this.product_name = (System.String)fieldValue; break; - case 4: this.product_price = (System.Single)fieldValue; break; - default: throw new AvroRuntimeException("Bad index " + fieldPos + " in Put()"); - }; - } - } -} diff --git a/samples/sample-stream-registry/com/avro/bean/Product.cs b/samples/sample-stream-registry/com/avro/bean/Product.cs deleted file mode 100644 index 5fca1877..00000000 --- a/samples/sample-stream-registry/com/avro/bean/Product.cs +++ /dev/null @@ -1,85 +0,0 @@ -// ------------------------------------------------------------------------------ -// -// Generated by avrogen, version 1.7.7.5 -// Changes to this file may cause incorrect behavior and will be lost if code -// is regenerated -// -// ------------------------------------------------------------------------------ -namespace com.avro.bean -{ - using System; - using System.Collections.Generic; - using System.Text; - using global::Avro; - using global::Avro.Specific; - - public partial class Product : ISpecificRecord - { - public static Schema _SCHEMA = Schema.Parse("{\"type\":\"record\",\"name\":\"Product\",\"namespace\":\"com.avro.bean\",\"fields\":[{\"name\":\"" + - "product_id\",\"type\":\"int\"},{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"price\",\"type\"" + - ":\"float\"}]}"); - private int _product_id; - private string _name; - private float _price; - public virtual Schema Schema - { - get - { - return Product._SCHEMA; - } - } - public int product_id - { - get - { - return this._product_id; - } - set - { - this._product_id = value; - } - } - public string name - { - get - { - return this._name; - } - set - { - this._name = value; - } - } - public float price - { - get - { - return this._price; - } - set - { - this._price = value; - } - } - public virtual object Get(int fieldPos) - { - switch (fieldPos) - { - case 0: return this.product_id; - case 1: return this.name; - case 2: return this.price; - default: throw new AvroRuntimeException("Bad index " + fieldPos + " in Get()"); - }; - } - public virtual void Put(int fieldPos, object fieldValue) - { - switch (fieldPos) - { - case 0: this.product_id = (System.Int32)fieldValue; break; - case 1: this.name = (System.String)fieldValue; break; - case 2: this.price = (System.Single)fieldValue; break; - default: throw new AvroRuntimeException("Bad index " + fieldPos + " in Put()"); - }; - } - } -} diff --git a/samples/sample-stream-registry/order-agg.avsc b/samples/sample-stream-registry/order-agg.avsc deleted file mode 100644 index cb4f17d1..00000000 --- a/samples/sample-stream-registry/order-agg.avsc +++ /dev/null @@ -1,23 +0,0 @@ -{ - "type": "record", - "name": "OrderAgg", - "namespace": "com.avro.bean", - "fields": [ - { - "name": "order_id", - "type": "int" - }, - { - "name": "price", - "type": "float" - }, - { - "name": "product_id", - "type": "int" - }, - { - "name": "totalPrice", - "type": "float" - } - ] -} \ No newline at end of file diff --git a/samples/sample-stream-registry/order.avsc b/samples/sample-stream-registry/order.avsc deleted file mode 100644 index 3b14f9ed..00000000 --- a/samples/sample-stream-registry/order.avsc +++ /dev/null @@ -1,19 +0,0 @@ -{ - "type": "record", - "name": "Order", - "namespace": "com.avro.bean", - "fields": [ - { - "name": "order_id", - "type": "int" - }, - { - "name": "price", - "type": "float" - }, - { - "name": "product_id", - "type": "int" - } - ] -} \ No newline at end of file diff --git a/samples/sample-stream-registry/order_product.avsc b/samples/sample-stream-registry/order_product.avsc deleted file mode 100644 index be0d11c3..00000000 --- a/samples/sample-stream-registry/order_product.avsc +++ /dev/null @@ -1,27 +0,0 @@ -{ - "type": "record", - "name": "OrderProduct", - "namespace": "com.avro.bean", - "fields": [ - { - "name": "order_id", - "type": "int" - }, - { - "name": "price", - "type": "float" - }, - { - "name": "product_id", - "type": "int" - }, - { - "name": "product_name", - "type": "string" - }, - { - "name": "product_price", - "type": "float" - } - ] -} \ No newline at end of file diff --git a/samples/sample-stream-registry/product.avsc b/samples/sample-stream-registry/product.avsc deleted file mode 100644 index ddf461e0..00000000 --- a/samples/sample-stream-registry/product.avsc +++ /dev/null @@ -1,19 +0,0 @@ -{ - "type": "record", - "name": "Product", - "namespace": "com.avro.bean", - "fields": [ - { - "name": "product_id", - "type": "int" - }, - { - "name": "name", - "type": "string" - }, - { - "name": "price", - "type": "float" - } - ] -} \ No newline at end of file diff --git a/samples/sample-stream-registry/sample-stream-registry.csproj b/samples/sample-stream-registry/sample-stream-registry.csproj deleted file mode 100644 index eedc3fcd..00000000 --- a/samples/sample-stream-registry/sample-stream-registry.csproj +++ /dev/null @@ -1,18 +0,0 @@ - - - - Exe - net6.0 - sample_stream_registry - - - - - - - - - - - - diff --git a/samples/sample-stream/.dockerignore b/samples/sample-stream/.dockerignore deleted file mode 100644 index 38bece4e..00000000 --- a/samples/sample-stream/.dockerignore +++ /dev/null @@ -1,25 +0,0 @@ -**/.dockerignore -**/.env -**/.git -**/.gitignore -**/.project -**/.settings -**/.toolstarget -**/.vs -**/.vscode -**/.idea -**/*.*proj.user -**/*.dbmdl -**/*.jfm -**/azds.yaml -**/bin -**/charts -**/docker-compose* -**/Dockerfile* -**/node_modules -**/npm-debug.log -**/obj -**/secrets.dev.yaml -**/values.dev.yaml -LICENSE -README.md \ No newline at end of file diff --git a/samples/sample-test-driver/Program.cs b/samples/sample-test-driver/Program.cs deleted file mode 100644 index 1668335b..00000000 --- a/samples/sample-test-driver/Program.cs +++ /dev/null @@ -1,44 +0,0 @@ -using Streamiz.Kafka.Net; -using Streamiz.Kafka.Net.Mock; -using Streamiz.Kafka.Net.SerDes; -using Streamiz.Kafka.Net.State; -using Streamiz.Kafka.Net.Stream; -using Streamiz.Kafka.Net.Table; -using System; -using System.Threading; - -namespace sample_test_driver -{ - internal class Program - { - private static void Main(string[] args) - { - var config = new StreamConfig - { - ApplicationId = "test-test-driver-app" - }; - - StreamBuilder builder = new StreamBuilder(); - - var table = builder - .Stream("test") - .GroupByKey() - .WindowedBy(TumblingWindowOptions.Of(TimeSpan.FromSeconds(10))) - .Count(InMemoryWindows.As("count-store")); - - Topology t = builder.Build(); - - using (var driver = new TopologyTestDriver(t, config)) - { - DateTime dt = DateTime.Now; - var inputTopic = driver.CreateInputTopic("test"); - inputTopic.PipeInput("renault", "clio", dt); - inputTopic.PipeInput("renault", "megane", dt.AddMilliseconds(10)); - Thread.Sleep((int)TimeSpan.FromSeconds(10).TotalMilliseconds); - inputTopic.PipeInput("renault", "scenic", dt.AddSeconds(1)); - var store = driver.GetWindowStore("count-store"); - var elements = store.All().ToList(); - } - } - } -} \ No newline at end of file diff --git a/samples/sample-test-driver/sample-test-driver.csproj b/samples/sample-test-driver/sample-test-driver.csproj deleted file mode 100644 index 1aa15baf..00000000 --- a/samples/sample-test-driver/sample-test-driver.csproj +++ /dev/null @@ -1,17 +0,0 @@ - - - - Exe - net6.0 - sample_test_driver - - - - - - - - - - - diff --git a/test/Consumer/Consumer.csproj b/test/Consumer/Consumer.csproj deleted file mode 100644 index d0a21bb3..00000000 --- a/test/Consumer/Consumer.csproj +++ /dev/null @@ -1,14 +0,0 @@ - - - - Exe - net6.0 - Consumer - Consumer - - - - - - - diff --git a/test/Consumer/Program.cs b/test/Consumer/Program.cs deleted file mode 100644 index 08537ad3..00000000 --- a/test/Consumer/Program.cs +++ /dev/null @@ -1,155 +0,0 @@ -using System; -using System.Collections.Generic; -using System.Text; -using Confluent.Kafka; - -namespace Consumer -{ - class Program - { - static void Main(string[] args) - { - if (args.Length == 2) - { - string topic = args[0]; - string mode = args[1]; - - var config = new ConsumerConfig - { - GroupId = "test-app", - BootstrapServers = "localhost:9092", - Debug = "generic, broker, topic, metadata, consumer" - }; - - var builder = new ConsumerBuilder(config); - - if (mode.ToUpper().Equals("ASSIGN")) - Assign(builder, config.BootstrapServers, config.GroupId, topic); - else if (mode.ToUpper().Equals("CONSUME")) - Consume(builder, topic); - } - } - - static void Assign(ConsumerBuilder builder, string bootstrap, string groupId, string topic) - { - var offsets = new List - { - new TopicPartitionOffset(new TopicPartition(topic, 0), 2), - new TopicPartitionOffset(new TopicPartition(topic, 1), 1), - }; - - // Check if none consumer is register for this consumer group - var adminClientConfig = new AdminClientConfig(); - adminClientConfig.BootstrapServers = bootstrap; - var adminClientBuilder = new AdminClientBuilder(adminClientConfig); - - using (var adminClient = adminClientBuilder.Build()) - { - var groupInfo = adminClient.ListGroup(groupId, TimeSpan.FromSeconds(10)); - if (groupInfo.Members.Count > 0) - { - Console.WriteLine($"Error consumers already exist in this consumer group {groupId}"); - foreach (var member in groupInfo.Members) - Console.WriteLine( - $"Member {member.MemberId} (client.id:{member.ClientId}#client.host:{member.ClientHost}) =" + - $" Assigment {DecodeMemberAssignment(member.MemberAssignment)}"); - return; - } - } - - using (var consumer = builder.Build()) - { - consumer.Commit(offsets); - } - } - - static void Consume(ConsumerBuilder builder, string topic) - { - bool run = true; - - var consumer = builder.Build(); - consumer.Subscribe(topic); - - Console.CancelKeyPress += (p, e) => - { - run = false; - consumer.Unsubscribe(); - consumer.Close(); - consumer.Dispose(); - }; - - while (run) - { - consumer.Consume(TimeSpan.FromSeconds(1)); - } - } - - // From : https://github.com/confluentinc/confluent-kafka-dotnet/issues/1307 - static string DecodeMemberAssignment(byte[] b) - { - /* - https://kafka.apache.org/protocol - STRING Represents a sequence of characters. First the length N is given as an INT16. Then N bytes follow which are the UTF-8 encoding of the character sequence. Length must not be negative. - INT16 Represents an integer between -2^15 and 2^15-1 inclusive. The values are encoded using two bytes in network byte order (big-endian). - INT32 Represents an integer between -2^31 and 2^31-1 inclusive. The values are encoded using four bytes in network byte order (big-endian). - BYTES Represents a raw sequence of bytes. First the length N is given as an INT32. Then N bytes follow. - - https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol - MemberAssignment => Version PartitionAssignment - Version => int16 - PartitionAssignment => [Topic [Partition]] - Topic => string - Partition => int32 - UserData => bytes - - Note: [] probably denotes a sequence of same type items, begining with Int32 of the # of such items - */ - - UTF8Encoding enc = new UTF8Encoding(); - StringBuilder s = new StringBuilder(); - try - { - short version = SwapEndianness(BitConverter.ToInt16(b, 0)); - int num_topic_assignments = SwapEndianness(BitConverter.ToInt32(b, 2)); - int i = 6; - for (int t = 0; t < num_topic_assignments; t++) - { - short topic_len = SwapEndianness(BitConverter.ToInt16(b, i)); - byte[] str = new byte[topic_len]; - Array.Copy(b, i + 2, str, 0, topic_len); - string topic = enc.GetString(str); - i += (topic_len + 2); - int num_partition = SwapEndianness(BitConverter.ToInt32(b, i)); - if (s.Length > 0) s.Append($"; "); - s.Append($"{topic}: "); - for (int j = 0; j < num_partition; j++) - { - i += 4; - s.Append( - $"{SwapEndianness(BitConverter.ToInt32(b, i))}{(j < num_partition - 1 ? "," : "")}"); - } - } - - return s.ToString(); - } - catch - { - return ""; - } - } - - static int SwapEndianness(int value) - { - var b1 = (value >> 0) & 0xff; - var b2 = (value >> 8) & 0xff; - var b3 = (value >> 16) & 0xff; - var b4 = (value >> 24) & 0xff; - return b1 << 24 | b2 << 16 | b3 << 8 | b4 << 0; - } - - static Int16 SwapEndianness(Int16 i) - { - return (Int16) ((i << 8) + (i >> 8)); - } - } -} \ No newline at end of file diff --git a/test/Producer-Avro/Producer-Avro.csproj b/test/Producer-Avro/Producer-Avro.csproj deleted file mode 100644 index a00bd721..00000000 --- a/test/Producer-Avro/Producer-Avro.csproj +++ /dev/null @@ -1,15 +0,0 @@ - - - - Exe - net6.0 - Producer_Avro - - - - - - - - - diff --git a/test/Producer-Avro/Program.cs b/test/Producer-Avro/Program.cs deleted file mode 100644 index bad3b283..00000000 --- a/test/Producer-Avro/Program.cs +++ /dev/null @@ -1,106 +0,0 @@ -using com.avro.bean; -using Confluent.Kafka; -using Confluent.Kafka.SyncOverAsync; -using Confluent.SchemaRegistry; -using Confluent.SchemaRegistry.Serdes; -using System; - -namespace Producer -{ - internal class Program - { - private static ProducerConfig producerConfig = new ProducerConfig - { - Acks = Acks.All, - BootstrapServers = "localhost:9092" - }; - - private static CachedSchemaRegistryClient schemaRegistryClient = new CachedSchemaRegistryClient - (new SchemaRegistryConfig - { - Url = "http://localhost:8081" - }); - - private static void Main(string[] args) - { - var topic = args.Length > 0 ? args[0] : "orders"; - - if (topic.Equals("orders")) - { - ProduceOrder(); - } - else if (topic.Equals("product")) - { - ProduceProduct(); - } - } - - static void ProduceOrder() - { - var builder = new ProducerBuilder(producerConfig) - .SetValueSerializer(new AvroSerializer(schemaRegistryClient, new AvroSerializerConfig { AutoRegisterSchemas = true }).AsSyncOverAsync()); - Console.WriteLine($"Writting in orders topic"); - Console.WriteLine("Enter exit for stopping producer, or enter PRODUCT_ID:ORDER_ID|PRICE"); - using (var producer = builder.Build()) - { - string s = Console.ReadLine(); - while (!s.Contains("exit", StringComparison.InvariantCultureIgnoreCase)) - { - string[] r = s.Split(":"); - string[] p = r[1].Split("|"); - producer.Produce("orders", new Message - { - Key = Int32.Parse(r[0]), - Value = new Order - { - order_id = Int32.Parse(p[0]), - price = float.Parse(p[1]), - product_id = Int32.Parse(r[0]) - } - }, (d) => - { - if (d.Status == PersistenceStatus.Persisted) - { - Console.WriteLine("Message sent !"); - } - }); - s = Console.ReadLine(); - } - } - } - - static void ProduceProduct() - { - var builder = new ProducerBuilder(producerConfig) - .SetValueSerializer(new AvroSerializer(schemaRegistryClient, new AvroSerializerConfig { AutoRegisterSchemas = true }).AsSyncOverAsync()); - Console.WriteLine($"Writting in product topic"); - Console.WriteLine("Enter exit for stopping producer, or enter PRODUCT_ID:PRODUCT_NAME|PRICE"); - using (var producer = builder.Build()) - { - string s = Console.ReadLine(); - while (!s.Contains("exit", StringComparison.InvariantCultureIgnoreCase)) - { - string[] r = s.Split(":"); - string[] p = r[1].Split("|"); - producer.Produce("product", new Message - { - Key = Int32.Parse(r[0]), - Value = new Product - { - name = p[0], - price = float.Parse(p[1]), - product_id = Int32.Parse(r[0]) - } - }, (d) => - { - if (d.Status == PersistenceStatus.Persisted) - { - Console.WriteLine("Message sent !"); - } - }); - s = Console.ReadLine(); - } - } - } - } -} diff --git a/test/Producer-Avro/com/avro/bean/Order.cs b/test/Producer-Avro/com/avro/bean/Order.cs deleted file mode 100644 index 65b33703..00000000 --- a/test/Producer-Avro/com/avro/bean/Order.cs +++ /dev/null @@ -1,85 +0,0 @@ -// ------------------------------------------------------------------------------ -// -// Generated by avrogen, version 1.7.7.5 -// Changes to this file may cause incorrect behavior and will be lost if code -// is regenerated -// -// ------------------------------------------------------------------------------ -namespace com.avro.bean -{ - using System; - using System.Collections.Generic; - using System.Text; - using global::Avro; - using global::Avro.Specific; - - public partial class Order : ISpecificRecord - { - public static Schema _SCHEMA = Schema.Parse("{\"type\":\"record\",\"name\":\"Order\",\"namespace\":\"com.avro.bean\",\"fields\":[{\"name\":\"or" + - "der_id\",\"type\":\"int\"},{\"name\":\"price\",\"type\":\"float\"},{\"name\":\"product_id\",\"type" + - "\":\"int\"}]}"); - private int _order_id; - private float _price; - private int _product_id; - public virtual Schema Schema - { - get - { - return Order._SCHEMA; - } - } - public int order_id - { - get - { - return this._order_id; - } - set - { - this._order_id = value; - } - } - public float price - { - get - { - return this._price; - } - set - { - this._price = value; - } - } - public int product_id - { - get - { - return this._product_id; - } - set - { - this._product_id = value; - } - } - public virtual object Get(int fieldPos) - { - switch (fieldPos) - { - case 0: return this.order_id; - case 1: return this.price; - case 2: return this.product_id; - default: throw new AvroRuntimeException("Bad index " + fieldPos + " in Get()"); - }; - } - public virtual void Put(int fieldPos, object fieldValue) - { - switch (fieldPos) - { - case 0: this.order_id = (System.Int32)fieldValue; break; - case 1: this.price = (System.Single)fieldValue; break; - case 2: this.product_id = (System.Int32)fieldValue; break; - default: throw new AvroRuntimeException("Bad index " + fieldPos + " in Put()"); - }; - } - } -} diff --git a/test/Producer-Avro/com/avro/bean/Product.cs b/test/Producer-Avro/com/avro/bean/Product.cs deleted file mode 100644 index 5fca1877..00000000 --- a/test/Producer-Avro/com/avro/bean/Product.cs +++ /dev/null @@ -1,85 +0,0 @@ -// ------------------------------------------------------------------------------ -// -// Generated by avrogen, version 1.7.7.5 -// Changes to this file may cause incorrect behavior and will be lost if code -// is regenerated -// -// ------------------------------------------------------------------------------ -namespace com.avro.bean -{ - using System; - using System.Collections.Generic; - using System.Text; - using global::Avro; - using global::Avro.Specific; - - public partial class Product : ISpecificRecord - { - public static Schema _SCHEMA = Schema.Parse("{\"type\":\"record\",\"name\":\"Product\",\"namespace\":\"com.avro.bean\",\"fields\":[{\"name\":\"" + - "product_id\",\"type\":\"int\"},{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"price\",\"type\"" + - ":\"float\"}]}"); - private int _product_id; - private string _name; - private float _price; - public virtual Schema Schema - { - get - { - return Product._SCHEMA; - } - } - public int product_id - { - get - { - return this._product_id; - } - set - { - this._product_id = value; - } - } - public string name - { - get - { - return this._name; - } - set - { - this._name = value; - } - } - public float price - { - get - { - return this._price; - } - set - { - this._price = value; - } - } - public virtual object Get(int fieldPos) - { - switch (fieldPos) - { - case 0: return this.product_id; - case 1: return this.name; - case 2: return this.price; - default: throw new AvroRuntimeException("Bad index " + fieldPos + " in Get()"); - }; - } - public virtual void Put(int fieldPos, object fieldValue) - { - switch (fieldPos) - { - case 0: this.product_id = (System.Int32)fieldValue; break; - case 1: this.name = (System.String)fieldValue; break; - case 2: this.price = (System.Single)fieldValue; break; - default: throw new AvroRuntimeException("Bad index " + fieldPos + " in Put()"); - }; - } - } -} diff --git a/test/Producer-Avro/order.avsc b/test/Producer-Avro/order.avsc deleted file mode 100644 index 3b14f9ed..00000000 --- a/test/Producer-Avro/order.avsc +++ /dev/null @@ -1,19 +0,0 @@ -{ - "type": "record", - "name": "Order", - "namespace": "com.avro.bean", - "fields": [ - { - "name": "order_id", - "type": "int" - }, - { - "name": "price", - "type": "float" - }, - { - "name": "product_id", - "type": "int" - } - ] -} \ No newline at end of file diff --git a/test/Producer-Avro/product.avsc b/test/Producer-Avro/product.avsc deleted file mode 100644 index ddf461e0..00000000 --- a/test/Producer-Avro/product.avsc +++ /dev/null @@ -1,19 +0,0 @@ -{ - "type": "record", - "name": "Product", - "namespace": "com.avro.bean", - "fields": [ - { - "name": "product_id", - "type": "int" - }, - { - "name": "name", - "type": "string" - }, - { - "name": "price", - "type": "float" - } - ] -} \ No newline at end of file diff --git a/test/Producer/Producer.csproj b/test/Producer/Producer.csproj deleted file mode 100644 index 5653d571..00000000 --- a/test/Producer/Producer.csproj +++ /dev/null @@ -1,12 +0,0 @@ - - - - Exe - net6.0 - - - - - - - diff --git a/test/Producer/Program.cs b/test/Producer/Program.cs deleted file mode 100644 index 1c8e37f0..00000000 --- a/test/Producer/Program.cs +++ /dev/null @@ -1,41 +0,0 @@ -using Confluent.Kafka; -using System; -using System.Security.Cryptography; - -namespace Producer -{ - internal class Program - { - private static void Main(string[] args) - { - var producerConfig = new ProducerConfig - { - Acks = Acks.All, - BootstrapServers = "localhost:9092" - }; - var topic = args.Length > 0 ? args[0] : "test"; - var builder = new ProducerBuilder(producerConfig); - Console.WriteLine($"Writting in {topic} topic"); - Console.WriteLine("Enter exit for stopping producer, or enter KEY:VALUE"); - using (var producer = builder.Build()) - { - string s = Console.ReadLine(); - while (!s.Contains("exit", StringComparison.InvariantCultureIgnoreCase)) - { - string[] r = s.Split(":"); - producer.Produce(topic, new Message { - Key = r[0], - Value = r[1] - }, (d) => - { - if (d.Status == PersistenceStatus.Persisted) - { - Console.WriteLine("Message sent !"); - } - }); - s = Console.ReadLine(); - } - } - } - } -} diff --git a/test/Streamiz.Kafka.Net.IntegrationTests/IntegrationTests.cs b/test/Streamiz.Kafka.Net.IntegrationTests/IntegrationTests.cs index 28f9f419..93934627 100644 --- a/test/Streamiz.Kafka.Net.IntegrationTests/IntegrationTests.cs +++ b/test/Streamiz.Kafka.Net.IntegrationTests/IntegrationTests.cs @@ -31,6 +31,7 @@ public void TearDown() } [Test] + [NonParallelizable] public async Task TestSimpleTopology() { var config = new StreamConfig @@ -64,6 +65,7 @@ await kafkaFixture.Produce( } [Test] + [NonParallelizable] public async Task TestFilteredTopology() { var config = new StreamConfig diff --git a/test/Streamiz.Kafka.Net.IntegrationTests/PerformanceTests.cs b/test/Streamiz.Kafka.Net.IntegrationTests/PerformanceTests.cs index decd64e5..4dfad729 100644 --- a/test/Streamiz.Kafka.Net.IntegrationTests/PerformanceTests.cs +++ b/test/Streamiz.Kafka.Net.IntegrationTests/PerformanceTests.cs @@ -27,18 +27,17 @@ public void TearDown() { kafkaFixture.DisposeAsync().Wait(TimeSpan.FromMinutes(5)); } - - - [Test] - public async Task PerformanceTestStateful() + + private async Task PerformanceRestore(ProcessingGuarantee guarantee, int numberOfRecords) { - int numberResult = 100000; + int numberResult = numberOfRecords; var config = new StreamConfig { - ApplicationId = "performance-test", + ApplicationId = "performance-test-restore", BootstrapServers = kafkaFixture.BootstrapServers, AutoOffsetReset = AutoOffsetReset.Earliest, - LingerMs = 25 + Guarantee = guarantee, + LingerMs = 100 }; string inputTopic = "input-topic", outputTopic = "output-topic"; @@ -51,9 +50,9 @@ public async Task PerformanceTestStateful() .Stream(inputTopic) .GroupByKey() .Count(RocksDb - .As("count-store") - .WithKeySerdes(new StringSerDes()) - .WithValueSerdes(new Int64SerDes())) + .As("count-store") + .WithKeySerdes(new StringSerDes()) + .WithValueSerdes(new Int64SerDes())) .ToStream() .To(outputTopic); @@ -67,19 +66,40 @@ public async Task PerformanceTestStateful() var result = kafkaFixture.ConsumeUntil(outputTopic, numberResult, 60000); stream.Dispose(); - + Assert.IsTrue(result); - Directory.Delete(Path.Combine(config.StateDir, "performance-test"), true); + Directory.Delete(Path.Combine(config.StateDir, "performance-test-restore"), true); + + //restart for restoration + stream = new KafkaStream(t, config); + await kafkaFixture.Produce(inputTopic, "new-zealand", Encoding.UTF8.GetBytes("coucou")); + long startRestoration = DateTime.Now.GetMilliseconds(); + + await stream.StartAsync(); + + var resultNZ = kafkaFixture.ConsumeUntil(outputTopic, 1, 60000); + + long endRestoration = DateTime.Now.GetMilliseconds(); + + stream.Dispose(); + + Assert.IsTrue(resultNZ); + + Directory.Delete(Path.Combine(config.StateDir, "performance-test-restore"), true); + + Console.WriteLine($"Restoration took approximately {endRestoration - startRestoration} ms"); } + [Test] - public async Task PerformanceRestoreTestStateful() + [NonParallelizable] + public async Task PerformanceTestStateful([Values(200000)] int x) { - int numberResult = 200000; + int numberResult = 100000; var config = new StreamConfig { - ApplicationId = "performance-test-restore", + ApplicationId = "performance-test", BootstrapServers = kafkaFixture.BootstrapServers, AutoOffsetReset = AutoOffsetReset.Earliest, LingerMs = 25 @@ -111,29 +131,24 @@ public async Task PerformanceRestoreTestStateful() var result = kafkaFixture.ConsumeUntil(outputTopic, numberResult, 60000); stream.Dispose(); - - Assert.IsTrue(result); - - Directory.Delete(Path.Combine(config.StateDir, "performance-test-restore"), true); - - //restart for restoration - stream = new KafkaStream(t, config); - await kafkaFixture.Produce(inputTopic, "new-zealand", Encoding.UTF8.GetBytes("coucou")); - long startRestoration = DateTime.Now.GetMilliseconds(); - - await stream.StartAsync(); - - var resultNZ = kafkaFixture.ConsumeUntil(outputTopic, 1, 60000); - - long endRestoration = DateTime.Now.GetMilliseconds(); - - stream.Dispose(); - - Assert.IsTrue(resultNZ); - Directory.Delete(Path.Combine(config.StateDir, "performance-test-restore"), true); + Assert.IsTrue(result); - Console.WriteLine($"Restoration took approximately {endRestoration - startRestoration} ms"); + Directory.Delete(Path.Combine(config.StateDir, "performance-test"), true); + } + + [Test] + [NonParallelizable] + public async Task PerformanceRestoreTestStateful([Values(200000)] int x) + { + await PerformanceRestore(ProcessingGuarantee.AT_LEAST_ONCE, x); + } + + [Test] + [NonParallelizable] + public async Task PerformanceRestoreEOSTestStateful([Values(200000)] int x) + { + await PerformanceRestore(ProcessingGuarantee.EXACTLY_ONCE, x); } } } \ No newline at end of file diff --git a/test/Streamiz.Kafka.Net.Tests/Helpers/KafkaProducerException.cs b/test/Streamiz.Kafka.Net.Tests/Helpers/KafkaProducerException.cs index 52466f79..61e265ed 100644 --- a/test/Streamiz.Kafka.Net.Tests/Helpers/KafkaProducerException.cs +++ b/test/Streamiz.Kafka.Net.Tests/Helpers/KafkaProducerException.cs @@ -2,6 +2,7 @@ using Streamiz.Kafka.Net.Mock.Sync; using System; using System.Collections.Generic; +using System.Reflection; using System.Threading; using System.Threading.Tasks; @@ -9,6 +10,7 @@ namespace Streamiz.Kafka.Net.Tests.Helpers { internal class ProducerSyncExceptionOptions { + public IDictionary> ExceptionsToThrow; public bool IsFatal { get; set; } = false; public bool IsRecoverable { get; set; } = false; public bool IsProductionException { get; set; } = false; @@ -50,6 +52,15 @@ private KafkaProducerException(SyncProducer syncProducer) { innerProducer = syncProducer; } + + private void CheckThrowException(string currentMethod) + { + if (options.ExceptionsToThrow != null && + options.ExceptionsToThrow.ContainsKey(currentMethod)) + { + throw options.ExceptionsToThrow[currentMethod].Invoke(); + } + } public KafkaProducerException(SyncProducer syncProducer, ProducerSyncExceptionOptions options) : this(syncProducer) @@ -63,50 +74,61 @@ public KafkaProducerException(SyncProducer syncProducer, ProducerSyncExceptionOp public void AbortTransaction(TimeSpan timeout) { + CheckThrowException(MethodBase.GetCurrentMethod().Name); } public void SetSaslCredentials(string username, string password) { + CheckThrowException(MethodBase.GetCurrentMethod().Name); } public int AddBrokers(string brokers) { + CheckThrowException(MethodBase.GetCurrentMethod().Name); return 0; } public void BeginTransaction() { + CheckThrowException(MethodBase.GetCurrentMethod().Name); } public void CommitTransaction(TimeSpan timeout) { + CheckThrowException(MethodBase.GetCurrentMethod().Name); } public void Dispose() { + CheckThrowException(MethodBase.GetCurrentMethod().Name); } public int Flush(TimeSpan timeout) { + CheckThrowException(MethodBase.GetCurrentMethod().Name); return 0; } public void Flush(CancellationToken cancellationToken = default) { + CheckThrowException(MethodBase.GetCurrentMethod().Name); } public void InitTransactions(TimeSpan timeout) { + CheckThrowException(MethodBase.GetCurrentMethod().Name); } public int Poll(TimeSpan timeout) { + CheckThrowException(MethodBase.GetCurrentMethod().Name); return 0; } private void HandleError(DeliveryReport initReport, Action> deliveryHandler) { + CheckThrowException(MethodBase.GetCurrentMethod().Name); --options.NumberOfError; handleError = options.NumberOfError > 0; @@ -121,7 +143,7 @@ private void HandleError(DeliveryReport initReport, if (options.IsRecoverable) { - throw new ProduceException(new Error(ErrorCode.TransactionCoordinatorFenced, + throw new ProduceException(new Error(ErrorCode.ProducerFenced, "TransactionCoordinatorFenced", false), result); } else @@ -141,7 +163,7 @@ private void HandleError(DeliveryReport initReport, } else if (options.IsRecoverable) { - initReport.Error = new Error(ErrorCode.TransactionCoordinatorFenced, + initReport.Error = new Error(ErrorCode.ProducerFenced, "TransactionCoordinatorFenced", false); deliveryHandler(initReport); @@ -157,6 +179,7 @@ private void HandleError(DeliveryReport initReport, public void Produce(string topic, Message message, Action> deliveryHandler = null) { + CheckThrowException(MethodBase.GetCurrentMethod().Name); if (options.WhiteTopics.Contains(topic) || !handleError) innerProducer.Produce(topic, message, deliveryHandler); else @@ -173,6 +196,7 @@ public void Produce(string topic, Message message, public void Produce(TopicPartition topicPartition, Message message, Action> deliveryHandler = null) { + CheckThrowException(MethodBase.GetCurrentMethod().Name); if (options.WhiteTopics.Contains(topicPartition.Topic) || !handleError) innerProducer.Produce(topicPartition, message, deliveryHandler); else @@ -190,6 +214,7 @@ public void Produce(TopicPartition topicPartition, Message messa public async Task> ProduceAsync(string topic, Message message, CancellationToken cancellationToken = default) { + CheckThrowException(MethodBase.GetCurrentMethod().Name); if (options.WhiteTopics.Contains(topic) || !handleError) return await innerProducer.ProduceAsync(topic, message, cancellationToken); else @@ -199,6 +224,7 @@ public async Task> ProduceAsync(string topic, public async Task> ProduceAsync(TopicPartition topicPartition, Message message, CancellationToken cancellationToken = default) { + CheckThrowException(MethodBase.GetCurrentMethod().Name); if (options.WhiteTopics.Contains(topicPartition.Topic) || !handleError) return await innerProducer.ProduceAsync(topicPartition, message, cancellationToken); else @@ -208,17 +234,17 @@ public async Task> ProduceAsync(TopicPartition to public void SendOffsetsToTransaction(IEnumerable offsets, IConsumerGroupMetadata groupMetadata, TimeSpan timeout) { - throw new NotImplementedException(); + CheckThrowException(MethodBase.GetCurrentMethod().Name); } public void CommitTransaction() { - throw new NotImplementedException(); + CheckThrowException(MethodBase.GetCurrentMethod().Name); } public void AbortTransaction() { - throw new NotImplementedException(); + CheckThrowException(MethodBase.GetCurrentMethod().Name); } } } \ No newline at end of file diff --git a/test/Streamiz.Kafka.Net.Tests/Metrics/TaskMetricsTests.cs b/test/Streamiz.Kafka.Net.Tests/Metrics/TaskMetricsTests.cs index 75136493..19be1b05 100644 --- a/test/Streamiz.Kafka.Net.Tests/Metrics/TaskMetricsTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Metrics/TaskMetricsTests.cs @@ -4,6 +4,7 @@ using Confluent.Kafka; using NUnit.Framework; using Streamiz.Kafka.Net.Crosscutting; +using Streamiz.Kafka.Net.Kafka.Internal; using Streamiz.Kafka.Net.Metrics; using Streamiz.Kafka.Net.Metrics.Internal; using Streamiz.Kafka.Net.Mock; @@ -52,7 +53,9 @@ public void Initialize() var processorTopology = builder.Build().Builder.BuildTopology(id); syncKafkaSupplier = new SyncKafkaSupplier(); - var producer = syncKafkaSupplier.GetProducer(config.ToProducerConfig()); + var streamsProducer = new StreamsProducer( + config, threadId, Guid.NewGuid(), syncKafkaSupplier, ""); + var consumer = syncKafkaSupplier.GetConsumer(config.ToConsumerConfig(), null); topicPartition = new TopicPartition("topic", 0); @@ -64,7 +67,7 @@ public void Initialize() consumer, config, syncKafkaSupplier, - null, + streamsProducer, new MockChangelogRegister(), streamMetricsRegistry); @@ -88,7 +91,7 @@ public void Initialize() public void Dispose() { task.Suspend(); - task.Close(); + task.Close(false); task = null; } @@ -98,7 +101,6 @@ public void TaskMetricsTest() var serdes = new StringSerDes(); var cloneConfig = config.Clone(); cloneConfig.ApplicationId = "consume-test"; - var producer = syncKafkaSupplier.GetProducer(cloneConfig.ToProducerConfig()); var consumer = syncKafkaSupplier.GetConsumer(cloneConfig.ToConsumerConfig("test-consum"), null); consumer.Subscribe("topic2"); @@ -124,7 +126,8 @@ public void TaskMetricsTest() { Assert.IsTrue(task.Process()); Assert.IsTrue(task.CommitNeeded); - task.Commit(); + task.PrepareCommit(); + task.PostCommit(false); } var messagesSink = new List>(); diff --git a/test/Streamiz.Kafka.Net.Tests/Metrics/ThreadMetricsTests.cs b/test/Streamiz.Kafka.Net.Tests/Metrics/ThreadMetricsTests.cs index 30f813ad..bdb3de7e 100644 --- a/test/Streamiz.Kafka.Net.Tests/Metrics/ThreadMetricsTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Metrics/ThreadMetricsTests.cs @@ -81,9 +81,12 @@ public void Initialize() var topo = builder.Build(); thread = StreamThread.Create( - threadId, "c1", + threadId, + Guid.NewGuid(), + "c1", topo.Builder, streamMetricsRegistry, config, - mockKafkaSupplier, mockKafkaSupplier.GetAdmin(config.ToAdminConfig("admin")), + mockKafkaSupplier, + mockKafkaSupplier.GetAdmin(config.ToAdminConfig("admin")), 1) as StreamThread; } diff --git a/test/Streamiz.Kafka.Net.Tests/Private/PauseResumeTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/PauseResumeTests.cs index e056aa11..9a4330ee 100644 --- a/test/Streamiz.Kafka.Net.Tests/Private/PauseResumeTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Private/PauseResumeTests.cs @@ -45,93 +45,5 @@ public void WorkflowCompleteMaxTaskIdleTest() Assert.AreEqual(0, items.Count); } } - - //[Test] - // TODO : fix that - public void WorkflowCompleteBufferedRecordsTest() - { - int maxBuffered = 10; - var token = new System.Threading.CancellationTokenSource(); - var serdes = new StringSerDes(); - var config = new StreamConfig(); - config.ApplicationId = "test-group"; - config.MaxTaskIdleMs = (long) TimeSpan.FromSeconds(100).TotalMilliseconds; - config.BufferedRecordsPerPartition = maxBuffered; - config.PollMs = 10; - - var builder = new StreamBuilder(); - - var stream1 = builder.Stream("topic1"); - var stream2 = builder.Stream("topic2"); - - stream1 - .Join(stream2, (v1, v2) => $"{v1}-{v2}", JoinWindowOptions.Of(TimeSpan.FromSeconds(10))) - .To("output"); - - var topo = builder.Build(); - - var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); - var consumer = supplier.GetConsumer(config.ToConsumerConfig("test-consum"), null); - consumer.Subscribe("output"); - var thread = StreamThread.Create( - "thread-0", "c0", - topo.Builder, new StreamMetricsRegistry(), config, - supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), - 0) as StreamThread; - - thread.Start(token.Token); - - for (int i = 0; i < maxBuffered + 1; ++i) - { - producer.Produce("topic1", new Confluent.Kafka.Message - { - Key = serdes.Serialize("key", new SerializationContext()), - Value = serdes.Serialize($"coucou{i}", new SerializationContext()) - }); - } - - // CONSUME PAUSE AFTER maxBuffered + 1 messages - System.Threading.Thread.Sleep(50); - - // Add one message more with consumer in stream thread in pause - producer.Produce("topic1", new Confluent.Kafka.Message - { - Key = serdes.Serialize("key", new SerializationContext()), - Value = serdes.Serialize($"coucou{maxBuffered + 1}", new SerializationContext()) - }); - - Assert.AreEqual(1, thread.ActiveTasks.Count()); - var task = thread.ActiveTasks.ToArray()[0]; - Assert.IsNotNull(task.Grouper); - Assert.IsFalse(task.Grouper.AllPartitionsBuffered); - Assert.AreEqual(maxBuffered + 1, task.Grouper.NumBuffered()); - Assert.AreEqual(maxBuffered + 1, task.Grouper.NumBuffered(new TopicPartition("topic1", 0))); - Assert.AreEqual(0, task.Grouper.NumBuffered(new TopicPartition("topic2", 0))); - - producer.Produce("topic2", new Confluent.Kafka.Message - { - Key = serdes.Serialize("key", new SerializationContext()), - Value = serdes.Serialize($"test", new SerializationContext()) - }); - - List> records = new List>(); - do - { - records.AddRange(consumer.ConsumeRecords(TimeSpan.FromMilliseconds(100)).ToList()); - } while (records.Count() <= 12); - - Assert.AreEqual(maxBuffered + 2, records.Count()); - for (int i = 0; i < maxBuffered + 2; ++i) - { - var message = records.ToArray()[i]; - Assert.AreEqual("key", serdes.Deserialize(message.Message.Key, new SerializationContext())); - Assert.IsTrue(serdes.Deserialize(message.Message.Value, new SerializationContext()) - .Contains($"coucou{i}-")); - } - - token.Cancel(); - thread.Dispose(); - } } } \ 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 827c8209..12a37145 100644 --- a/test/Streamiz.Kafka.Net.Tests/Private/RecordCollectorTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Private/RecordCollectorTests.cs @@ -13,7 +13,6 @@ namespace Streamiz.Kafka.Net.Tests.Private { public class RecordCollectorTests { - private RecordCollector collector; private StreamConfig config; [SetUp] @@ -22,19 +21,12 @@ 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, - client.Object); } [TearDown] public void Dispose() { - collector.Close(); + // nothing } [Test] @@ -45,9 +37,20 @@ public void MultipleRetry() NumberOfError = 5, }; var supplier = new ProducerSyncExceptionSupplier(options); - var producer = supplier.GetProducer(config.ToProducerConfig("producer-1")); - collector.Init(ref producer); + var producer = new StreamsProducer( + config, + "thread", + Guid.NewGuid(), + supplier, + ""); + + var collector = new RecordCollector( + "test-collector", + config, + new TaskId {Id = 0, Partition = 0}, + producer, + NoRunnableSensor.Empty); // first send => retry queue try @@ -98,9 +101,19 @@ public void TestWithFatalError() IsFatal = true }; var supplier = new ProducerSyncExceptionSupplier(options); - var producer = supplier.GetProducer(config.ToProducerConfig("producer-1")); + var producer = new StreamsProducer( + config, + "thread", + Guid.NewGuid(), + supplier, + ""); - collector.Init(ref producer); + var collector = new RecordCollector( + "test-collector", + config, + new TaskId {Id = 0, Partition = 0}, + producer, + NoRunnableSensor.Empty); // first send => retry queue try @@ -158,9 +171,19 @@ public void TestWithRecoverableError() IsRecoverable = true }; var supplier = new ProducerSyncExceptionSupplier(options); - var producer = supplier.GetProducer(config.ToProducerConfig("producer-1")); + var producer = new StreamsProducer( + config, + "thread", + Guid.NewGuid(), + supplier, + ""); - collector.Init(ref producer); + var collector = new RecordCollector( + "test-collector", + config, + new TaskId {Id = 0, Partition = 0}, + producer, + NoRunnableSensor.Empty); // first send => retry queue try diff --git a/test/Streamiz.Kafka.Net.Tests/Private/StateDirectoryTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/StateDirectoryTests.cs new file mode 100644 index 00000000..ba00a1f5 --- /dev/null +++ b/test/Streamiz.Kafka.Net.Tests/Private/StateDirectoryTests.cs @@ -0,0 +1,66 @@ +using System; +using System.IO; +using NUnit.Framework; +using RocksDbSharp; +using Streamiz.Kafka.Net.Processors.Internal; + +namespace Streamiz.Kafka.Net.Tests.Private; + +public class StateDirectoryTests +{ + [Test] + [NonParallelizable] + public void GettingAnExistingProcessorId() + { + if(File.Exists("./unit-test-1/streamiz-process-metadata")) + File.Delete("./unit-test-1/streamiz-process-metadata"); + + var config = new StreamConfig(); + config.ApplicationId = "test-state-dir"; + config.StateDir = "./unit-test-1"; + + var instance = new StateDirectory(config, true); + var newGuid = Guid.NewGuid(); + Directory.CreateDirectory("./unit-test-1"); + File.WriteAllText("./unit-test-1/streamiz-process-metadata", "{" + @"""processId"": """ + $"{newGuid}"+ @"""" + "}"); + var guid = instance.InitializeProcessId(); + + Directory.Delete("./unit-test-1", true); + Assert.AreEqual(newGuid, guid); + } + + [Test] + [NonParallelizable] + public void FailedDeseriliazeProcessorId() + { + if(File.Exists("./unit-test-2/streamiz-process-metadata")) + File.Delete("./unit-test-2/streamiz-process-metadata"); + + var config = new StreamConfig(); + config.ApplicationId = "test-state-dir"; + config.StateDir = "./unit-test-2"; + + var instance = new StateDirectory(config, true); + var newGuid = Guid.NewGuid(); + Directory.CreateDirectory("./unit-test-2"); + File.WriteAllText("./unit-test-2/streamiz-process-metadata", $"{newGuid}"); + var guid = instance.InitializeProcessId(); + + Directory.Delete("./unit-test-2", true); + Assert.AreNotEqual(newGuid, guid); + } + + [Test] + public void NoPersistenStoreProcessorId() + { + var config = new StreamConfig(); + config.ApplicationId = "test-state-dir"; + config.StateDir = "."; + + var instance = new StateDirectory(config, false); + var guid1 = instance.InitializeProcessId(); + var guid2 = instance.InitializeProcessId(); + + Assert.AreNotEqual(guid1, guid2); + } +} \ No newline at end of file diff --git a/test/Streamiz.Kafka.Net.Tests/Private/StreamTaskTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/StreamTaskTests.cs index b386bcd5..5f127c41 100644 --- a/test/Streamiz.Kafka.Net.Tests/Private/StreamTaskTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Private/StreamTaskTests.cs @@ -7,18 +7,22 @@ using Streamiz.Kafka.Net.SerDes; using System; using System.Collections.Generic; +using System.Globalization; using Streamiz.Kafka.Net.Mock; using Streamiz.Kafka.Net.Table; using Streamiz.Kafka.Net.State.Internal; using System.IO; +using System.Threading.Tasks.Sources; +using Streamiz.Kafka.Net.Errors; +using Streamiz.Kafka.Net.Kafka.Internal; using Streamiz.Kafka.Net.Metrics; namespace Streamiz.Kafka.Net.Tests.Private { public class StreamTaskTests { - [Test] - public void StreamTaskWithEXACTLY_ONCE() + [Test] + public void StreamCloseRunning() { var config = new StreamConfig(); config.ApplicationId = "test-app"; @@ -36,7 +40,12 @@ public void StreamTaskWithEXACTLY_ONCE() var processorTopology = topology.Builder.BuildTopology(id); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); + var streamsProducer = new StreamsProducer( + config, + "thread-0", + Guid.NewGuid(), + supplier, + "log-prefix"); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); @@ -49,7 +58,7 @@ public void StreamTaskWithEXACTLY_ONCE() consumer, config, supplier, - null, + streamsProducer, new MockChangelogRegister() , new StreamMetricsRegistry()); task.InitializeStateStores(); @@ -57,59 +66,62 @@ public void StreamTaskWithEXACTLY_ONCE() task.RestorationIfNeeded(); task.CompleteRestoration(); - List> messages = new List>(); - int offset = 0; - for (int i = 0; i < 5; ++i) - messages.Add( - new ConsumeResult - { - Message = new Message - { - Key = serdes.Serialize($"key{i + 1}", new SerializationContext()), - Value = serdes.Serialize($"value{i + 1}", new SerializationContext()) - }, - TopicPartitionOffset = new TopicPartitionOffset(part, offset++) - }); + Assert.Throws(() => task.Close(false)); + } + + [Test] + public void StreamCloseAlreadyClosed() + { + var config = new StreamConfig(); + config.ApplicationId = "test-app"; + config.Guarantee = ProcessingGuarantee.EXACTLY_ONCE; - task.AddRecords(messages); + var serdes = new StringSerDes(); + var builder = new StreamBuilder(); - Assert.IsTrue(task.CanProcess(DateTime.Now.GetMilliseconds())); + builder.Stream("topic") + .Map((k, v, _) => KeyValuePair.Create(k.ToUpper(), v.ToUpper())) + .To("topic2"); - while (task.CanProcess(DateTime.Now.GetMilliseconds())) - { - Assert.IsTrue(task.Process()); - Assert.IsTrue(task.CommitNeeded); - task.Commit(); - } + var topology = builder.Build(); + TaskId id = new TaskId {Id = 0, Partition = 0}; + var processorTopology = topology.Builder.BuildTopology(id); - // CHECK IN TOPIC topic2 - consumer.Subscribe("topic2"); - List> results = new List>(); - ConsumeResult result = null; - do - { - result = consumer.Consume(100); + var supplier = new SyncKafkaSupplier(); + var streamsProducer = new StreamsProducer( + config, + "thread-0", + Guid.NewGuid(), + supplier, + "log-prefix"); + var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); - if (result != null) - { - results.Add(result); - consumer.Commit(result); - } - } while (result != null); - Assert.AreEqual(5, results.Count); - for (int i = 0; i < 5; ++i) - { - Assert.AreEqual($"KEY{i + 1}", serdes.Deserialize(results[i].Message.Key, new SerializationContext())); - Assert.AreEqual($"VALUE{i + 1}", - serdes.Deserialize(results[i].Message.Value, new SerializationContext())); - } + var part = new TopicPartition("topic", 0); + StreamTask task = new StreamTask( + "thread-0", + id, + new List {part}, + processorTopology, + consumer, + config, + supplier, + streamsProducer, + new MockChangelogRegister() + , new StreamMetricsRegistry()); + task.InitializeStateStores(); + task.InitializeTopology(); + task.RestorationIfNeeded(); + task.CompleteRestoration(); - task.Close(); + task.Suspend(); + task.Close(false); + task.Close(false); } + [Test] - public void StreamTaskSuspendResumeWithEXACTLY_ONCE() + public void StreamTaskRunning() { var config = new StreamConfig(); config.ApplicationId = "test-app"; @@ -127,7 +139,12 @@ public void StreamTaskSuspendResumeWithEXACTLY_ONCE() var processorTopology = topology.Builder.BuildTopology(id); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); + var streamsProducer = new StreamsProducer( + config, + "thread-0", + Guid.NewGuid(), + supplier, + "log-prefix"); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); @@ -140,7 +157,7 @@ public void StreamTaskSuspendResumeWithEXACTLY_ONCE() consumer, config, supplier, - null, + streamsProducer, new MockChangelogRegister() , new StreamMetricsRegistry()); task.InitializeStateStores(); @@ -151,7 +168,6 @@ public void StreamTaskSuspendResumeWithEXACTLY_ONCE() List> messages = new List>(); int offset = 0; for (int i = 0; i < 5; ++i) - { messages.Add( new ConsumeResult { @@ -162,7 +178,6 @@ public void StreamTaskSuspendResumeWithEXACTLY_ONCE() }, TopicPartitionOffset = new TopicPartitionOffset(part, offset++) }); - } task.AddRecords(messages); @@ -172,22 +187,8 @@ public void StreamTaskSuspendResumeWithEXACTLY_ONCE() { Assert.IsTrue(task.Process()); Assert.IsTrue(task.CommitNeeded); - task.Commit(); - } - - task.Suspend(); - task.Resume(); - task.RestorationIfNeeded(); - - task.AddRecords(messages); - - Assert.IsTrue(task.CanProcess(DateTime.Now.GetMilliseconds())); - - while (task.CanProcess(DateTime.Now.GetMilliseconds())) - { - Assert.IsTrue(task.Process()); - Assert.IsTrue(task.CommitNeeded); - task.Commit(); + task.PrepareCommit(); + task.PostCommit(false); } // CHECK IN TOPIC topic2 @@ -205,13 +206,20 @@ public void StreamTaskSuspendResumeWithEXACTLY_ONCE() } } while (result != null); - Assert.AreEqual(10, results.Count); + Assert.AreEqual(5, results.Count); + for (int i = 0; i < 5; ++i) + { + Assert.AreEqual($"KEY{i + 1}", serdes.Deserialize(results[i].Message.Key, new SerializationContext())); + Assert.AreEqual($"VALUE{i + 1}", + serdes.Deserialize(results[i].Message.Value, new SerializationContext())); + } - task.Close(); + task.Suspend(); + task.Close(false); } [Test] - public void StreamTaskSuspendResume() + public void StreamTaskSuspendRecreate() { var config = new StreamConfig(); config.ApplicationId = "test-app"; @@ -230,9 +238,15 @@ public void StreamTaskSuspendResume() var processorTopology = topology.Builder.BuildTopology(id); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); + var streamsProducer = new StreamsProducer( + config, + "thread-0", + Guid.NewGuid(), + supplier, + "log-prefix"); + var part = new TopicPartition("topic", 0); StreamTask task = new StreamTask( "thread-0", @@ -242,7 +256,7 @@ public void StreamTaskSuspendResume() consumer, config, supplier, - producer, + streamsProducer, new MockChangelogRegister() , new StreamMetricsRegistry()); task.InitializeStateStores(); @@ -274,14 +288,31 @@ public void StreamTaskSuspendResume() { Assert.IsTrue(task.Process()); Assert.IsTrue(task.CommitNeeded); - task.Commit(); + task.PrepareCommit(); + task.PostCommit(false); } Assert.IsNotNull(task.GetStore("store")); task.Suspend(); + task.Close(false); Assert.IsNull(task.GetStore("store")); - task.Resume(); + + //recreate task + task = new StreamTask( + "thread-0", + id, + new List {part}, + processorTopology, + consumer, + config, + supplier, + streamsProducer, + new MockChangelogRegister() + , new StreamMetricsRegistry()); + task.InitializeStateStores(); + task.InitializeTopology(); task.RestorationIfNeeded(); + task.CompleteRestoration(); Assert.IsNotNull(task.GetStore("store")); task.AddRecords(messages); @@ -292,7 +323,8 @@ public void StreamTaskSuspendResume() { Assert.IsTrue(task.Process()); Assert.IsTrue(task.CommitNeeded); - task.Commit(); + task.PrepareCommit(); + task.PostCommit(false); } // CHECK IN TOPIC topic2 @@ -312,7 +344,8 @@ public void StreamTaskSuspendResume() Assert.AreEqual(10, results.Count); - task.Close(); + task.Suspend(); + task.Close(false); } [Test] @@ -334,9 +367,15 @@ public void StreamTaskWrittingCheckpoint() var processorTopology = topology.Builder.BuildTopology(id); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); + var streamsProducer = new StreamsProducer( + config, + "thread-0", + Guid.NewGuid(), + supplier, + "log-prefix"); + var part = new TopicPartition("topic", 0); StreamTask task = new StreamTask( "thread-0", @@ -346,7 +385,7 @@ public void StreamTaskWrittingCheckpoint() consumer, config, supplier, - producer, + streamsProducer, new MockChangelogRegister() , new StreamMetricsRegistry()); task.InitializeStateStores(); @@ -378,11 +417,10 @@ public void StreamTaskWrittingCheckpoint() { Assert.IsTrue(task.Process()); Assert.IsTrue(task.CommitNeeded); - task.Commit(); + task.PrepareCommit(); + task.PostCommit(true); } - task.MayWriteCheckpoint(true); - messages = new List>(); for (int i = 0; i < StateManagerTools.OFFSET_DELTA_THRESHOLD_FOR_CHECKPOINT + 10; ++i) { @@ -405,14 +443,15 @@ public void StreamTaskWrittingCheckpoint() Assert.IsTrue(task.Process()); } - task.MayWriteCheckpoint(false); + task.PrepareCommit(); + task.PostCommit(false); var lines = File.ReadAllLines(Path.Combine(config.StateDir, config.ApplicationId, "0-0", ".checkpoint")); Assert.AreEqual(3, lines.Length); Assert.AreEqual("test-app-store-changelog 0 10014", lines[2]); task.Suspend(); - task.Close(); + task.Close(false); Directory.Delete(config.StateDir, true); } diff --git a/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadAssigmentTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadAssigmentTests.cs index cf51a292..63893f04 100644 --- a/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadAssigmentTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadAssigmentTests.cs @@ -1,11 +1,9 @@ -using Confluent.Kafka; using NUnit.Framework; using Streamiz.Kafka.Net.Mock.Kafka; using Streamiz.Kafka.Net.Processors; using Streamiz.Kafka.Net.SerDes; using System; using System.Collections.Generic; -using System.Linq; using System.Threading; using Streamiz.Kafka.Net.Crosscutting; using Streamiz.Kafka.Net.Metrics; @@ -137,7 +135,7 @@ public void Init() thread1 = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(), "c0", topo.Builder, new StreamMetricsRegistry(), config, mockKafkaSupplier, mockKafkaSupplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; diff --git a/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadRebalanceTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadRebalanceTests.cs index bdc10b58..1f152e64 100644 --- a/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadRebalanceTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadRebalanceTests.cs @@ -38,14 +38,16 @@ public void Init() var topo = builder.Build(); + var processId = Guid.NewGuid(); + thread1 = StreamThread.Create( - "thread-0", "c0", + "thread-0", processId, "c0", topo.Builder, new StreamMetricsRegistry(), config, mockKafkaSupplier, mockKafkaSupplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; thread2 = StreamThread.Create( - "thread-1", "c1", + "thread-1", processId, "c1", topo.Builder, new StreamMetricsRegistry(), config, mockKafkaSupplier, mockKafkaSupplier.GetAdmin(config.ToAdminConfig("admin")), 1) as StreamThread; diff --git a/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadRestoreInMemoryTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadRestoreInMemoryTests.cs index f0b27592..774d654b 100644 --- a/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadRestoreInMemoryTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadRestoreInMemoryTests.cs @@ -51,14 +51,15 @@ public void Init() topo.Builder.RewriteTopology(config); topo.Builder.BuildTopology(); + var processId = Guid.NewGuid(); thread1 = StreamThread.Create( - "thread-0", "c0", + "thread-0", processId, "c0", topo.Builder, new StreamMetricsRegistry(), config, mockKafkaSupplier, mockKafkaSupplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; thread2 = StreamThread.Create( - "thread-1", "c1", + "thread-1", processId, "c1", topo.Builder, new StreamMetricsRegistry(), config, mockKafkaSupplier, mockKafkaSupplier.GetAdmin(config.ToAdminConfig("admin")), 1) as StreamThread; diff --git a/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadRestoreRocksDbTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadRestoreRocksDbTests.cs index 4ea6084f..daa798c8 100644 --- a/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadRestoreRocksDbTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadRestoreRocksDbTests.cs @@ -51,14 +51,15 @@ public void Init() topo.Builder.RewriteTopology(config); topo.Builder.BuildTopology(); + var processId = Guid.NewGuid(); thread1 = StreamThread.Create( - "thread-0", "c0", + "thread-0", processId, "c0", topo.Builder, new StreamMetricsRegistry(), config, mockKafkaSupplier, mockKafkaSupplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; thread2 = StreamThread.Create( - "thread-1", "c1", + "thread-1", processId, "c1", topo.Builder, new StreamMetricsRegistry(), config, mockKafkaSupplier, mockKafkaSupplier.GetAdmin(config.ToAdminConfig("admin")), 1) as StreamThread; diff --git a/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadTests.cs index 7eb487f3..0be55c20 100644 --- a/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Private/StreamThreadTests.cs @@ -132,14 +132,6 @@ public void GetSharedAdminClientIdTest() Assert.AreEqual($"thread-client-streamiz-admin", result); } - [Test] - public void GetTaskProducerClientIdTest() - { - var taskId = new TaskId {Id = 0, Partition = 0}; - var result = StreamThread.GetTaskProducerClientId("thread-client", taskId); - Assert.AreEqual($"thread-client-0-0-streamiz-producer", result); - } - [Test] public void GetThreadProducerClientIdTest() { @@ -164,7 +156,7 @@ public void CreateStreamThread() var supplier = new SyncKafkaSupplier(); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(), "c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; @@ -208,7 +200,7 @@ public void StreamThreadNormalWorkflow() var consumer = supplier.GetConsumer(config.ToConsumerConfig("test-consum"), null); consumer.Subscribe("topic2"); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(), "c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; @@ -259,7 +251,7 @@ public void StreamThreadCommitIntervalWorkflow() var consumer = supplier.GetConsumer(config.ToConsumerConfig("test-consum"), null); consumer.Subscribe("topic2"); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(), "c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; @@ -305,7 +297,7 @@ public void CheckIncorrectStateTransition() var supplier = new SyncKafkaSupplier(); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(), "c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; @@ -328,7 +320,7 @@ public void CheckSetStateWithoutStateChangedHandler() var supplier = new SyncKafkaSupplier(); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(),"c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; @@ -353,7 +345,7 @@ public void CheckSetStateStartingWithDeadThread() var supplier = new SyncKafkaSupplier(); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(),"c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; diff --git a/test/Streamiz.Kafka.Net.Tests/Private/StreamsProducerTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/StreamsProducerTests.cs new file mode 100644 index 00000000..4b91a072 --- /dev/null +++ b/test/Streamiz.Kafka.Net.Tests/Private/StreamsProducerTests.cs @@ -0,0 +1,288 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Reflection; +using Confluent.Kafka; +using NUnit.Framework; +using Streamiz.Kafka.Net.Errors; +using Streamiz.Kafka.Net.Kafka.Internal; +using Streamiz.Kafka.Net.Mock.Sync; +using Streamiz.Kafka.Net.Tests.Helpers; + +namespace Streamiz.Kafka.Net.Tests.Private; + +public class StreamsProducerTests +{ + [Test] + public void ThrowExceptionMethodTransactionAtLeastOnce() + { + var kafkaSupplier = new SyncKafkaSupplier(); + var streamConfig = new StreamConfig(); + streamConfig.Guarantee = ProcessingGuarantee.AT_LEAST_ONCE; + + StreamsProducer streamsProducer = new StreamsProducer( + streamConfig, + "thread-1", + Guid.NewGuid(), + kafkaSupplier, + "log-1"); + + Assert.Throws(() => streamsProducer.InitTransaction()); + Assert.Throws(() => streamsProducer.CommitTransaction(null, null)); + } + + [Test] + public void ThrowKafkaRetriableExceptionWhenInitTransaction() + { + var options = new ProducerSyncExceptionOptions() + { + ExceptionsToThrow = new Dictionary> + { + { + typeof(IProducer).GetMethod("InitTransactions")?.Name, + () => new KafkaRetriableException(new Error(ErrorCode.Local_TimedOut)) + } + } + }; + + var kafkaSupplier = new ProducerSyncExceptionSupplier(options); + + var streamConfig = new StreamConfig(); + streamConfig.Guarantee = ProcessingGuarantee.EXACTLY_ONCE; + + StreamsProducer streamsProducer = new StreamsProducer( + streamConfig, + "thread-1", + Guid.NewGuid(), + kafkaSupplier, + "log-1"); + + Assert.Throws(() => streamsProducer.InitTransaction()); + } + + [Test] + public void TestResetProducer() + { + var kafkaSupplier = new SyncKafkaSupplier(); + + var streamConfig = new StreamConfig(); + streamConfig.Guarantee = ProcessingGuarantee.EXACTLY_ONCE; + + StreamsProducer streamsProducer = new StreamsProducer( + streamConfig, + "thread-1", + Guid.NewGuid(), + kafkaSupplier, + "log-1"); + + streamsProducer.InitTransaction(); + + streamsProducer.Produce("topic", new Message() + { + Key = new byte[]{1}, + Value = new byte[]{1}, + }, (_) => {}); + streamsProducer.Produce("topic", new Message() + { + Key = new byte[]{2}, + Value = new byte[]{2}, + }, (_) => {}); + + streamsProducer.ResetProducer(); + + streamsProducer.InitTransaction(); + streamsProducer.Produce("topic", new Message() + { + Key = new byte[]{3}, + Value = new byte[]{3}, + }, (_) => {}); + + var producer = (SyncProducer)kafkaSupplier.GetProducer(null); + Assert.AreEqual(3, producer.GetHistory("topic").ToList().Count); + } + + [Test] + public void ThrowKafkaRetriableExceptionWhenStartTransactionNoRecoverable() + { + var options = new ProducerSyncExceptionOptions() + { + ExceptionsToThrow = new Dictionary> + { + { + typeof(IProducer).GetMethod("BeginTransaction")?.Name, + () => new KafkaException(new Error(ErrorCode.Local_TimedOut)) + } + } + }; + + var kafkaSupplier = new ProducerSyncExceptionSupplier(options); + + var streamConfig = new StreamConfig(); + streamConfig.Guarantee = ProcessingGuarantee.EXACTLY_ONCE; + + StreamsProducer streamsProducer = new StreamsProducer( + streamConfig, + "thread-1", + Guid.NewGuid(), + kafkaSupplier, + "log-1"); + + streamsProducer.InitTransaction(); + Assert.Throws(() => streamsProducer.Produce("topic", null, (_) => { })); + } + + [Test] + public void ThrowKafkaRetriableExceptionWhenStartTransactionRecoverable() + { + var options = new ProducerSyncExceptionOptions() + { + ExceptionsToThrow = new Dictionary> + { + { + typeof(IProducer).GetMethod("BeginTransaction")?.Name, + () => new KafkaException(new Error(ErrorCode.ProducerFenced)) + } + } + }; + + var kafkaSupplier = new ProducerSyncExceptionSupplier(options); + + var streamConfig = new StreamConfig(); + streamConfig.Guarantee = ProcessingGuarantee.EXACTLY_ONCE; + + StreamsProducer streamsProducer = new StreamsProducer( + streamConfig, + "thread-1", + Guid.NewGuid(), + kafkaSupplier, + "log-1"); + + streamsProducer.InitTransaction(); + Assert.Throws(() => streamsProducer.Produce("topic", null, (_) => { })); + } + + [Test] + public void ThrowKafkaRetriableExceptionWhenCommitTransactionNoRecoverable() + { + var options = new ProducerSyncExceptionOptions() + { + ExceptionsToThrow = new Dictionary> + { + { + typeof(IProducer).GetMethods().First(m => m.Name.Equals("CommitTransaction")).Name, + () => new KafkaException(new Error(ErrorCode.Local_TimedOut)) + } + } + }; + + var kafkaSupplier = new ProducerSyncExceptionSupplier(options); + + var streamConfig = new StreamConfig(); + streamConfig.Guarantee = ProcessingGuarantee.EXACTLY_ONCE; + + StreamsProducer streamsProducer = new StreamsProducer( + streamConfig, + "thread-1", + Guid.NewGuid(), + kafkaSupplier, + "log-1"); + + streamsProducer.InitTransaction(); + streamsProducer.Produce("topic", null, (_) => { }); + Assert.Throws(() => streamsProducer.CommitTransaction(null, null)); + } + + [Test] + public void ThrowKafkaRetriableExceptionWhenCommitTransactionRecoverable() + { + var options = new ProducerSyncExceptionOptions() + { + ExceptionsToThrow = new Dictionary> + { + { + typeof(IProducer).GetMethods().First(m => m.Name.Equals("CommitTransaction")).Name, + () => new KafkaException(new Error(ErrorCode.ProducerFenced)) + } + } + }; + + var kafkaSupplier = new ProducerSyncExceptionSupplier(options); + + var streamConfig = new StreamConfig(); + streamConfig.Guarantee = ProcessingGuarantee.EXACTLY_ONCE; + + StreamsProducer streamsProducer = new StreamsProducer( + streamConfig, + "thread-1", + Guid.NewGuid(), + kafkaSupplier, + "log-1"); + + streamsProducer.InitTransaction(); + streamsProducer.Produce("topic", null, (_) => { }); + Assert.Throws(() => streamsProducer.CommitTransaction(null, null)); + } + + [Test] + public void ThrowKafkaRetriableExceptionWhenAbortTransactionNoRecoverable() + { + var options = new ProducerSyncExceptionOptions() + { + ExceptionsToThrow = new Dictionary> + { + { + typeof(IProducer).GetMethods().First(m => m.Name.Equals("AbortTransaction")).Name, + () => new KafkaException(new Error(ErrorCode.Local_TimedOut)) + } + } + }; + + var kafkaSupplier = new ProducerSyncExceptionSupplier(options); + + var streamConfig = new StreamConfig(); + streamConfig.Guarantee = ProcessingGuarantee.EXACTLY_ONCE; + + StreamsProducer streamsProducer = new StreamsProducer( + streamConfig, + "thread-1", + Guid.NewGuid(), + kafkaSupplier, + "log-1"); + + streamsProducer.InitTransaction(); + streamsProducer.Produce("topic", null, (_) => { }); + Assert.Throws(() => streamsProducer.AbortTransaction()); + } + + [Test] + public void ThrowKafkaRetriableExceptionWhenAbortTransactionRecoverable() + { + var options = new ProducerSyncExceptionOptions() + { + ExceptionsToThrow = new Dictionary> + { + { + typeof(IProducer).GetMethods().First(m => m.Name.Equals("AbortTransaction")).Name, + () => new KafkaException(new Error(ErrorCode.ProducerFenced)) + } + } + }; + + var kafkaSupplier = new ProducerSyncExceptionSupplier(options); + + var streamConfig = new StreamConfig(); + streamConfig.Guarantee = ProcessingGuarantee.EXACTLY_ONCE; + + StreamsProducer streamsProducer = new StreamsProducer( + streamConfig, + "thread-1", + Guid.NewGuid(), + kafkaSupplier, + "log-1"); + + streamsProducer.InitTransaction(); + streamsProducer.Produce("topic", null, (_) => { }); + streamsProducer.AbortTransaction(); + Assert.IsFalse(streamsProducer.TransactionInFlight); + } +} \ No newline at end of file diff --git a/test/Streamiz.Kafka.Net.Tests/Private/TaskManagerTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/TaskManagerTests.cs index b0c90e56..1d8f3f09 100644 --- a/test/Streamiz.Kafka.Net.Tests/Private/TaskManagerTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Private/TaskManagerTests.cs @@ -10,8 +10,8 @@ using Streamiz.Kafka.Net.Table; using Streamiz.Kafka.Net.Processors; using Streamiz.Kafka.Net.State; -using Streamiz.Kafka.Net.State.Internal; using System.IO; +using Streamiz.Kafka.Net.Kafka.Internal; using Streamiz.Kafka.Net.Metrics; namespace Streamiz.Kafka.Net.Tests.Private @@ -30,16 +30,22 @@ public void StandardWorkflowTaskManager() var topology = builder.Build(); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); var restoreConsumer = supplier.GetRestoreConsumer(config.ToConsumerConfig()); var storeChangelogReader = new StoreChangelogReader(config, restoreConsumer, "thread-0", new StreamMetricsRegistry()); - var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, producer, + var streamsProducer = new StreamsProducer( + config, + "thread-0", + Guid.NewGuid(), + supplier, + ""); + + var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, storeChangelogReader, new StreamMetricsRegistry()); var taskManager = new TaskManager(topology.Builder, taskCreator, - supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader); + supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader, streamsProducer); taskManager.CreateTasks( new List @@ -98,17 +104,23 @@ public void TaskManagerCommitWithoutCommitNeeed() var topology = builder.Build(); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); var restoreConsumer = supplier.GetRestoreConsumer(config.ToConsumerConfig()); var storeChangelogReader = new StoreChangelogReader(config, restoreConsumer, "thread-0", new StreamMetricsRegistry()); - var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, producer, + var streamsProducer = new StreamsProducer( + config, + "thread-0", + Guid.NewGuid(), + supplier, + ""); + + var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, storeChangelogReader, new StreamMetricsRegistry()); var taskManager = new TaskManager(topology.Builder, taskCreator, - supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader); + supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader, streamsProducer); taskManager.CreateTasks( new List @@ -140,17 +152,23 @@ public void TaskManagerCommit() var topology = builder.Build(); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); var restoreConsumer = supplier.GetRestoreConsumer(config.ToConsumerConfig()); var storeChangelogReader = new StoreChangelogReader(config, restoreConsumer, "thread-0", new StreamMetricsRegistry()); - var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, producer, + var streamsProducer = new StreamsProducer( + config, + "thread-0", + Guid.NewGuid(), + supplier, + ""); + + var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, storeChangelogReader, new StreamMetricsRegistry()); var taskManager = new TaskManager(topology.Builder, taskCreator, - supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader); + supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader, streamsProducer); taskManager.CreateTasks( new List @@ -237,17 +255,23 @@ public void TaskManagerReassignedRevokedPartitions() var topology = builder.Build(); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); var restoreConsumer = supplier.GetRestoreConsumer(config.ToConsumerConfig()); var storeChangelogReader = new StoreChangelogReader(config, restoreConsumer, "thread-0", new StreamMetricsRegistry()); - var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, producer, + var streamsProducer = new StreamsProducer( + config, + "thread-0", + Guid.NewGuid(), + supplier, + ""); + + var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, storeChangelogReader, new StreamMetricsRegistry()); var taskManager = new TaskManager(topology.Builder, taskCreator, - supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader); + supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader, streamsProducer); taskManager.CreateTasks( new List @@ -289,17 +313,23 @@ public void TaskManagerAssignedUnknownPartitions() var topology = builder.Build(); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); var restoreConsumer = supplier.GetRestoreConsumer(config.ToConsumerConfig()); var storeChangelogReader = new StoreChangelogReader(config, restoreConsumer, "thread-0", new StreamMetricsRegistry()); - var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, producer, + var streamsProducer = new StreamsProducer( + config, + "thread-0", + Guid.NewGuid(), + supplier, + ""); + + var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, storeChangelogReader, new StreamMetricsRegistry()); var taskManager = new TaskManager(topology.Builder, taskCreator, - supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader); + supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader, streamsProducer); taskManager.CreateTasks( new List @@ -360,16 +390,22 @@ private void TaskManagerRestorationChangelog(bool persistenStateStore = false) topology.Builder.RewriteTopology(config); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); var restoreConsumer = supplier.GetRestoreConsumer(config.ToConsumerConfig()); var storeChangelogReader = new StoreChangelogReader(config, restoreConsumer, "thread-0", new StreamMetricsRegistry()); - var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, producer, + var streamsProducer = new StreamsProducer( + config, + "thread-0", + Guid.NewGuid(), + supplier, + ""); + + var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, storeChangelogReader, new StreamMetricsRegistry()); var taskManager = new TaskManager(topology.Builder, taskCreator, - supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader); + supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader, streamsProducer); var part = new TopicPartition("topic", 0); diff --git a/test/Streamiz.Kafka.Net.Tests/Private/TaskMigratedExceptionTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/TaskMigratedExceptionTests.cs index 0aff780f..8d0a4473 100644 --- a/test/Streamiz.Kafka.Net.Tests/Private/TaskMigratedExceptionTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Private/TaskMigratedExceptionTests.cs @@ -4,6 +4,8 @@ using System; using System.Collections.Generic; using System.Threading; +using Microsoft.Extensions.Logging; +using Streamiz.Kafka.Net.Mock.Sync; using Streamiz.Kafka.Net.Tests.Helpers; namespace Streamiz.Kafka.Net.Tests.Private @@ -63,6 +65,7 @@ public void ProductionExceptionFatalHandlerFailTest() Assert.AreEqual(new List>(), _return); } + [NonParallelizable] [Test] public void ProductionExceptionRecoverableHandlerFailTestWithParallel() { @@ -80,7 +83,7 @@ private void ProductionExceptionRecoverableHandlerFailTest(bool parallelProcessi var _return = new List>(); var config = new StreamConfig(); var dt = DateTime.Now; - var timeout = TimeSpan.FromSeconds(10); + var timeout = TimeSpan.FromSeconds(1); config.ApplicationId = "test"; @@ -88,7 +91,12 @@ private void ProductionExceptionRecoverableHandlerFailTest(bool parallelProcessi config.PollMs = 10; config.ProductionExceptionHandler += (r) => ProductionExceptionHandlerResponse.FAIL; config.ParallelProcessing = parallelProcessing; - + config.Logger = LoggerFactory.Create((b) => + { + b.SetMinimumLevel(LogLevel.Debug); + b.AddConsole(); + }); + var options = new ProducerSyncExceptionOptions() { IsRecoverable = true, @@ -96,7 +104,6 @@ private void ProductionExceptionRecoverableHandlerFailTest(bool parallelProcessi WhiteTopics = new List {"test"} }; var supplier = new ProducerSyncExceptionSupplier(options); - var builder = new StreamBuilder(); builder .Stream("test") @@ -110,12 +117,20 @@ private void ProductionExceptionRecoverableHandlerFailTest(bool parallelProcessi using (var driver = new TopologyTestDriver(t, config, supplier)) { var inputtopic = driver.CreateInputTopic("test"); - var outputTopic = driver.CreateOuputTopic("test-output"); + //var outputTopic = driver.CreateOuputTopic("test-output"); inputtopic.PipeInput("coucou"); inputtopic.PipeInput("coucou"); - while (_return.Count == 0) ; + while (_return.Count == 0) + { + Thread.Sleep(100); + if (DateTime.Now > dt + timeout) + { + break; + } + } var expected = new List>(); expected.Add(KeyValuePair.Create(null, "coucou")); + expected.Add(KeyValuePair.Create(null, "coucou")); Assert.AreEqual(expected, _return); } } @@ -126,7 +141,9 @@ public void ProduceExceptionRecoverableHandlerFailTestWithoutParallel() ProduceExceptionRecoverableHandlerFailTest(false); } - [Test] public void ProduceExceptionRecoverableHandlerFailTestWithParallel() + [NonParallelizable] + [Test] + public void ProduceExceptionRecoverableHandlerFailTestWithParallel() { ProduceExceptionRecoverableHandlerFailTest(true); } diff --git a/test/Streamiz.Kafka.Net.Tests/Private/TaskScheduledTests.cs b/test/Streamiz.Kafka.Net.Tests/Private/TaskScheduledTests.cs index b51a557b..8188de2e 100644 --- a/test/Streamiz.Kafka.Net.Tests/Private/TaskScheduledTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Private/TaskScheduledTests.cs @@ -6,6 +6,7 @@ using NUnit.Framework; using Streamiz.Kafka.Net.Crosscutting; using Streamiz.Kafka.Net.Errors; +using Streamiz.Kafka.Net.Kafka.Internal; using Streamiz.Kafka.Net.Metrics; using Streamiz.Kafka.Net.Mock; using Streamiz.Kafka.Net.Mock.Sync; @@ -203,16 +204,22 @@ public void StandardSystemTimePunctuator() var topology = builder.Build(); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); var restoreConsumer = supplier.GetRestoreConsumer(config.ToConsumerConfig()); var storeChangelogReader = new StoreChangelogReader(config, restoreConsumer, "thread-0", new StreamMetricsRegistry()); - var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, producer, + var streamsProducer = new StreamsProducer( + config, + "thread-0", + Guid.NewGuid(), + supplier, + ""); + + var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, storeChangelogReader, new StreamMetricsRegistry()); var taskManager = new TaskManager(topology.Builder, taskCreator, - supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader); + supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader, streamsProducer); taskManager.CreateTasks( new List { @@ -246,16 +253,22 @@ public void StandardStreamTimePunctuator() var topology = builder.Build(); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); var restoreConsumer = supplier.GetRestoreConsumer(config.ToConsumerConfig()); var storeChangelogReader = new StoreChangelogReader(config, restoreConsumer, "thread-0", new StreamMetricsRegistry()); - var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, producer, + var streamsProducer = new StreamsProducer( + config, + "thread-0", + Guid.NewGuid(), + supplier, + ""); + + var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, storeChangelogReader, new StreamMetricsRegistry()); var taskManager = new TaskManager(topology.Builder, taskCreator, - supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader); + supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader, streamsProducer); taskManager.CreateTasks( new List { @@ -293,16 +306,22 @@ public void CloseTaskScheduledTask() var topology = builder.Build(); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); var restoreConsumer = supplier.GetRestoreConsumer(config.ToConsumerConfig()); var storeChangelogReader = new StoreChangelogReader(config, restoreConsumer, "thread-0", new StreamMetricsRegistry()); - var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, producer, + var streamsProducer = new StreamsProducer( + config, + "thread-0", + Guid.NewGuid(), + supplier, + ""); + + var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, storeChangelogReader, new StreamMetricsRegistry()); var taskManager = new TaskManager(topology.Builder, taskCreator, - supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader); + supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader, streamsProducer); taskManager.CreateTasks( new List { @@ -360,16 +379,22 @@ private void CreateScheduledException(bool a, bool b, bool c) var topology = builder.Build(); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); var restoreConsumer = supplier.GetRestoreConsumer(config.ToConsumerConfig()); var storeChangelogReader = new StoreChangelogReader(config, restoreConsumer, "thread-0", new StreamMetricsRegistry()); - var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, producer, + var streamsProducer = new StreamsProducer( + config, + "thread-0", + Guid.NewGuid(), + supplier, + ""); + + var taskCreator = new TaskCreator(topology.Builder, config, "thread-0", supplier, storeChangelogReader, new StreamMetricsRegistry()); var taskManager = new TaskManager(topology.Builder, taskCreator, - supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader); + supplier.GetAdmin(config.ToAdminConfig("admin")), consumer, storeChangelogReader, streamsProducer); taskManager.CreateTasks( new List { diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KGroupedStreamCountTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KGroupedStreamCountTests.cs index c4f13e38..b8304ae8 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/KGroupedStreamCountTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KGroupedStreamCountTests.cs @@ -1,9 +1,11 @@ -using System.Collections.Generic; +using System; +using System.Collections.Generic; using System.Linq; using Confluent.Kafka; using NUnit.Framework; using Streamiz.Kafka.Net.Crosscutting; using Streamiz.Kafka.Net.Errors; +using Streamiz.Kafka.Net.Kafka.Internal; using Streamiz.Kafka.Net.Metrics; using Streamiz.Kafka.Net.Mock; using Streamiz.Kafka.Net.Mock.Sync; @@ -40,7 +42,9 @@ public void WithNullMaterialize() var processorTopology = topology.Builder.BuildTopology(id); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); + var streamsProducer = new StreamsProducer( + config, "thread-0", Guid.NewGuid(), supplier, ""); + var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); @@ -53,7 +57,7 @@ public void WithNullMaterialize() consumer, config, supplier, - null, + streamsProducer, new MockChangelogRegister(), new StreamMetricsRegistry()); task.InitializeStateStores(); diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/KGroupedTableCountTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/KGroupedTableCountTests.cs index 490198fc..50a352ac 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/KGroupedTableCountTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/KGroupedTableCountTests.cs @@ -1,9 +1,11 @@ -using System.Collections.Generic; +using System; +using System.Collections.Generic; using System.Linq; using Confluent.Kafka; using NUnit.Framework; using Streamiz.Kafka.Net.Crosscutting; using Streamiz.Kafka.Net.Errors; +using Streamiz.Kafka.Net.Kafka.Internal; using Streamiz.Kafka.Net.Metrics; using Streamiz.Kafka.Net.Mock; using Streamiz.Kafka.Net.Mock.Sync; @@ -41,7 +43,8 @@ public void WithNullMaterialize() var processorTopology = topology.Builder.BuildTopology(id); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); + var streamsProducer = new StreamsProducer( + config, "thread-0", Guid.NewGuid(), supplier, ""); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); @@ -54,7 +57,7 @@ public void WithNullMaterialize() consumer, config, supplier, - null, + streamsProducer, new MockChangelogRegister(), new StreamMetricsRegistry()); task.InitializeStateStores(); diff --git a/test/Streamiz.Kafka.Net.Tests/Processors/TimeWindowKStreamCountTests.cs b/test/Streamiz.Kafka.Net.Tests/Processors/TimeWindowKStreamCountTests.cs index dfe8882f..9586cf5c 100644 --- a/test/Streamiz.Kafka.Net.Tests/Processors/TimeWindowKStreamCountTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Processors/TimeWindowKStreamCountTests.cs @@ -14,6 +14,7 @@ using System.Collections.Generic; using System.Linq; using Avro.Util; +using Streamiz.Kafka.Net.Kafka.Internal; using Streamiz.Kafka.Net.Metrics; using Streamiz.Kafka.Net.Tests.Helpers; @@ -53,7 +54,8 @@ public void WithNullMaterialize() var processorTopology = topology.Builder.BuildTopology(id); var supplier = new SyncKafkaSupplier(); - var producer = supplier.GetProducer(config.ToProducerConfig()); + var streamsProducer = new StreamsProducer( + config, "thread-0", Guid.NewGuid(), supplier, ""); var consumer = supplier.GetConsumer(config.ToConsumerConfig(), null); var part = new TopicPartition("topic", 0); @@ -65,7 +67,7 @@ public void WithNullMaterialize() consumer, config, supplier, - null, + streamsProducer, new MockChangelogRegister(), new StreamMetricsRegistry()); task.InitializeStateStores(); diff --git a/test/Streamiz.Kafka.Net.Tests/Public/ProcessorContextTests.cs b/test/Streamiz.Kafka.Net.Tests/Public/ProcessorContextTests.cs index 075f1fee..383d9dc8 100644 --- a/test/Streamiz.Kafka.Net.Tests/Public/ProcessorContextTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Public/ProcessorContextTests.cs @@ -7,6 +7,8 @@ using Streamiz.Kafka.Net.Processors.Internal; using System; using System.Collections.Generic; +using System.Threading; +using Streamiz.Kafka.Net.Kafka.Internal; using Streamiz.Kafka.Net.Metrics; namespace Streamiz.Kafka.Net.Tests.Public @@ -80,12 +82,16 @@ private StreamTask GetTask(TaskId taskId) config.ApplicationId = "test-app"; var syncKafkaSupplier = new SyncKafkaSupplier(true); + + var streamsProducer = new StreamsProducer( + config, "thread", Guid.NewGuid(), syncKafkaSupplier, "log"); + var streamTask = new StreamTask( "thread", taskId, new List(), new Stream.Internal.ProcessorTopology(null, null, null, null, null, null, null, null), - null, config , syncKafkaSupplier, new SyncProducer(config.ToProducerConfig()), new MockChangelogRegister(), new StreamMetricsRegistry()); + null, config , syncKafkaSupplier, streamsProducer, new MockChangelogRegister(), new StreamMetricsRegistry()); return streamTask; } diff --git a/test/Streamiz.Kafka.Net.Tests/Public/StreamizMetadataTests.cs b/test/Streamiz.Kafka.Net.Tests/Public/StreamizMetadataTests.cs index 53c3c236..9571a0c3 100644 --- a/test/Streamiz.Kafka.Net.Tests/Public/StreamizMetadataTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Public/StreamizMetadataTests.cs @@ -45,7 +45,7 @@ public void ChangeHeadersMetadataTests() var consumer = supplier.GetConsumer(configConsumer.ToConsumerConfig(), null); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(), "c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; @@ -110,7 +110,7 @@ public void GetCurrentHeadersMetadataTests() var consumer = supplier.GetConsumer(configConsumer.ToConsumerConfig(), null); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(),"c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; @@ -172,7 +172,7 @@ public void GetCurrentHeadersMetadataTestsNotConfigured() var consumer = supplier.GetConsumer(configConsumer.ToConsumerConfig(), null); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(),"c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; @@ -232,7 +232,7 @@ public void GetCurrentTopicMetadataTests() var consumer = supplier.GetConsumer(configConsumer.ToConsumerConfig(), null); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(),"c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; @@ -289,7 +289,7 @@ public void GetCurrentTopicMetadataTestsNotConfigured() var consumer = supplier.GetConsumer(configConsumer.ToConsumerConfig(), null); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(),"c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; @@ -348,7 +348,7 @@ public void GetCurrentOffsetMetadataTests() var consumer = supplier.GetConsumer(configConsumer.ToConsumerConfig(), null); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(),"c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; @@ -405,7 +405,7 @@ public void GetCurrentOffsetMetadataTestsNotConfigured() var consumer = supplier.GetConsumer(configConsumer.ToConsumerConfig(), null); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(),"c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; @@ -464,7 +464,7 @@ public void GetCurrentPartitionMetadataTests() var consumer = supplier.GetConsumer(configConsumer.ToConsumerConfig(), null); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(),"c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; @@ -521,7 +521,7 @@ public void GetCurrentPartitionMetadataTestsNotConfigured() var consumer = supplier.GetConsumer(configConsumer.ToConsumerConfig(), null); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(),"c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; @@ -581,7 +581,7 @@ public void GetCurrentTimestampMetadataTests() var consumer = supplier.GetConsumer(configConsumer.ToConsumerConfig(), null); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(),"c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; @@ -640,7 +640,7 @@ public void GetCurrentTimestampMetadataTestsNotConfigured() var consumer = supplier.GetConsumer(configConsumer.ToConsumerConfig(), null); var thread = StreamThread.Create( - "thread-0", "c0", + "thread-0", Guid.NewGuid(),"c0", topo.Builder, new StreamMetricsRegistry(), config, supplier, supplier.GetAdmin(config.ToAdminConfig("admin")), 0) as StreamThread; diff --git a/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedKeyValueBytesStoreTests.cs b/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedKeyValueBytesStoreTests.cs index a0a14c16..5cccb36f 100644 --- a/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedKeyValueBytesStoreTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedKeyValueBytesStoreTests.cs @@ -48,18 +48,13 @@ public void Begin() partition = new TopicPartition("source", 0); kafkaSupplier = new SyncKafkaSupplier(); - - var producerConfig = new ProducerConfig(); - producerConfig.ClientId = "producer-1"; - - var adminConfig = new AdminClientConfig(); - adminConfig.ClientId = "admin-client"; - var producerClient = kafkaSupplier.GetProducer(producerConfig); - var adminClient = kafkaSupplier.GetAdmin(adminConfig); + var streamsProducer = new StreamsProducer(config, + "thread", + Guid.NewGuid(), + kafkaSupplier, "log"); - recordCollector = new RecordCollector("p-1", config, id, new NoRunnableSensor("s", "s", MetricsRecordingLevel.DEBUG), adminClient); - recordCollector.Init(ref producerClient); + recordCollector = new RecordCollector("p-1", config, id, streamsProducer, new NoRunnableSensor("s", "s", MetricsRecordingLevel.DEBUG)); var changelogsTopics = new Dictionary{ { "test-store", "test-store-changelog"} @@ -103,7 +98,7 @@ public void End() { store.Flush(); stateManager.Close(); - recordCollector?.Close(); + recordCollector?.Close(false); } } diff --git a/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedWindowBytesStoreTests.cs b/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedWindowBytesStoreTests.cs index 5870c20c..caf0da24 100644 --- a/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedWindowBytesStoreTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingTimestampedWindowBytesStoreTests.cs @@ -48,18 +48,13 @@ public void Begin() kafkaSupplier = new SyncKafkaSupplier(); - var producerConfig = new ProducerConfig(); - producerConfig.ClientId = "producer-1"; + var streamsProducer = new StreamsProducer(config, + "thread", + Guid.NewGuid(), + kafkaSupplier, "log"); - var adminConfig = new AdminClientConfig(); - adminConfig.ClientId = "admin-client"; + recordCollector = new RecordCollector("p-1", config, id, streamsProducer, new NoRunnableSensor("s", "s", MetricsRecordingLevel.DEBUG)); - 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{ { "test-store", "test-store-changelog"} }; @@ -102,7 +97,7 @@ public void End() { store.Flush(); stateManager.Close(); - recordCollector?.Close(); + recordCollector?.Close(false); } } diff --git a/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingWindowBytesStoreTests.cs b/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingWindowBytesStoreTests.cs index 1d45821c..4692828c 100644 --- a/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingWindowBytesStoreTests.cs +++ b/test/Streamiz.Kafka.Net.Tests/Stores/ChangeLoggingWindowBytesStoreTests.cs @@ -45,18 +45,13 @@ public void Begin() kafkaSupplier = new SyncKafkaSupplier(); - var producerConfig = new ProducerConfig(); - producerConfig.ClientId = "producer-1"; + var streamsProducer = new StreamsProducer(config, + "thread", + Guid.NewGuid(), + kafkaSupplier, "log"); - var adminConfig = new AdminClientConfig(); - adminConfig.ClientId = "admin-client"; + recordCollector = new RecordCollector("p-1", config, id, streamsProducer, new NoRunnableSensor("s", "s", MetricsRecordingLevel.DEBUG)); - 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{ { "test-store", "test-store-changelog"} }; @@ -101,7 +96,7 @@ public void End() { store.Flush(); stateManager.Close(); - recordCollector?.Close(); + recordCollector?.Close(false); } }