├── .vscode ├── settings.json └── launch.json ├── key.snk ├── .DS_Store ├── .github ├── FUNDING.yml ├── ISSUE_TEMPLATE └── workflows │ ├── integration.yml │ └── gh-pages.yml ├── docs ├── favicon.ico └── assets │ ├── state-store.png │ ├── async-topology.png │ ├── request-response.png │ ├── threading-model.png │ └── logo-kafka-stream-net.png ├── resources ├── streamiz_v1.0.ai ├── Untitled.drawio ├── logo-kafka-stream-net.png └── old_logo-kafka-stream-net-1.0.png ├── global.json ├── core ├── Processors │ ├── ICloneableProcessor.cs │ ├── IProcessorSupplier.cs │ ├── Internal │ │ ├── TaskStateTransitionValidator.cs │ │ ├── ThreadStateTransitionValidator.cs │ │ ├── IGlobalStateManager.cs │ │ ├── IChangelogReader.cs │ │ ├── IChangelogRegister.cs │ │ ├── IGlobalStateMaintainer.cs │ │ ├── StaticTopicNameExtractor.cs │ │ ├── WrapperTopicNameExtractor.cs │ │ ├── DefaultRecordTimestampExtractor.cs │ │ ├── StateRestoreCallback.cs │ │ ├── WrapperRecordTimestampExtractor.cs │ │ ├── ObjectDeserialized.cs │ │ ├── GlobalProcessorContext.cs │ │ ├── ExtractRecordMetadataTimestamp.cs │ │ ├── WrapperStreamPartitioner.cs │ │ ├── IStateManager.cs │ │ ├── FailOnInvalidTimestamp.cs │ │ └── TaskId.cs │ ├── PassThroughProcessor.cs │ ├── ThreadStateListener.cs │ ├── IKTableProcessorSupplier.cs │ ├── ITopicManager.cs │ ├── IKStreamAggProcessorSupplier.cs │ ├── IAsyncProcessor.cs │ ├── IThread.cs │ ├── Public │ │ ├── WrappedProcessor.cs │ │ ├── WrappedTransformer.cs │ │ └── PunctuationType.cs │ ├── KStreamMapProcessor.cs │ ├── KStreamPrintProcessor.cs │ ├── KStreamPeekProcessor.cs │ ├── KStreamFilterProcessor.cs │ ├── KStreamTimestampProcessor.cs │ ├── IRecordTimestampExtractor.cs │ ├── KStreamJoinWindowProcessor.cs │ ├── KStreamProcessor.cs │ ├── UnwindowedChangelogTopicConfig.cs │ ├── KStreamBranchProcessor.cs │ ├── KStreamFlatMapProcessor.cs │ ├── KStreamForeachAsyncProcessor.cs │ ├── ITopicNameExtractor.cs │ ├── KStreamFlatMapValuesProcessor.cs │ ├── IStreamPartitioner.cs │ ├── RepartitionTopicConfig.cs │ ├── AbstractKTableKTableJoinProcessor.cs │ ├── KStreamFlatMapAsyncProcessor.cs │ ├── InternalTopicConfig.cs │ ├── KStreamPAPI.cs │ ├── IProcessor.cs │ ├── KTableKTableJoinMergeProcessor.cs │ └── IRecordQueue.cs ├── Metrics │ ├── Stats │ │ ├── IStat.cs │ │ ├── IMeasurable.cs │ │ ├── IMetricValueProvider.cs │ │ ├── IMeasurableStat.cs │ │ ├── CumulativeCount.cs │ │ ├── IGauge.cs │ │ ├── WindowedCount.cs │ │ ├── Value.cs │ │ ├── ImmutableMetricValue.cs │ │ ├── ProviderMetricValue.cs │ │ ├── CumulativeSum.cs │ │ ├── WindowedSum.cs │ │ ├── Avg.cs │ │ ├── Min.cs │ │ └── Max.cs │ ├── Librdkafka │ │ ├── IStatisticsHandler.cs │ │ ├── TopicStatistic.cs │ │ ├── ConsumerGroupStatistic.cs │ │ └── IdempotentProducerStatistic.cs │ ├── MetricUtils.cs │ ├── MetricsRecordingLevel.cs │ └── Internal │ │ ├── NoRunnableSensor.cs │ │ └── RocksDBMetricsRecordingTrigger.cs ├── Streamiz.Kafka.Net.csproj.user ├── State │ ├── Internal │ │ ├── ISegment.cs │ │ ├── ITimestampedStore.cs │ │ ├── ISegments.cs │ │ ├── GlobalStateStoreProviderFacade.cs │ │ ├── IKeySchema.cs │ │ ├── TimestampedWindowStoreMaterializer.cs │ │ └── ISegmentedBytesStore.cs │ ├── IReadOnlySessionStore.cs │ ├── Supplier │ │ ├── ISessionBytesStoreSupplier.cs │ │ ├── IKeyValueBytesStoreSupplier.cs │ │ └── IStoreSupplier.cs │ ├── Cache │ │ ├── ICachedStateStore.cs │ │ ├── ICacheFunction.cs │ │ ├── Internal │ │ │ ├── IClockTime.cs │ │ │ ├── PostEvictionDelegate.cs │ │ │ └── PostEvictionCallbackRegistration.cs │ │ └── CacheEntryValue.cs │ ├── ISessionStore.cs │ ├── ITimestampedKeyValueStore.cs │ ├── RocksDb │ │ ├── Internal │ │ │ ├── RocksDbSegmentedBytesStore.cs │ │ │ └── RocksDbKeyValueSegment.cs │ │ └── IRocksDbAdapter.cs │ ├── Enumerator │ │ ├── EnumeratorExtensions.cs │ │ ├── IWindowStoreEnumerator.cs │ │ ├── IKeyValueEnumerator.cs │ │ └── EmptyEnumerator.cs │ ├── Suppress │ │ ├── ITimeOrderedKeyValueBuffer.cs │ │ ├── Internal │ │ │ ├── BufferKey.cs │ │ │ └── Maybe.cs │ │ └── InMemoryTimeOrderedKeyValueChangeBufferBuilder.cs │ ├── ITimestampedWindowStore.cs │ ├── Logging │ │ ├── ChangeLoggingTimestampedWindowBytesStore.cs │ │ └── ChangeLoggingTimestampedKeyValueBytesStore.cs │ ├── IWindowStore.cs │ ├── InMemory │ │ └── Internal │ │ │ └── InMemoryKeyValueEnumerator.cs │ └── Metered │ │ └── MeteredTimestampedKeyValueStore.cs ├── Stream │ ├── Internal │ │ ├── INameProvider.cs │ │ ├── Graph │ │ │ ├── PassThrough.cs │ │ │ ├── Nodes │ │ │ │ ├── RootNode.cs │ │ │ │ ├── StreamSourceNode.cs │ │ │ │ ├── ProcessorParameters.cs │ │ │ │ ├── StateStoreNode.cs │ │ │ │ ├── ProcessorGraphNode.cs │ │ │ │ └── AbstractRepartitionNode.cs │ │ │ ├── KStreamJoinWindow.cs │ │ │ ├── KStreamMap.cs │ │ │ ├── KStreamFlatMapValues.cs │ │ │ ├── KStreamFlatMap.cs │ │ │ ├── KStreamFilter.cs │ │ │ ├── KStreamTimestampExtractor.cs │ │ │ ├── KStreamProcessorSupplier.cs │ │ │ ├── KStreamPeek.cs │ │ │ ├── KStreamBranch.cs │ │ │ ├── KStreamMapValues.cs │ │ │ ├── KStreamTransformerSupplier.cs │ │ │ ├── KStreamForeachAsync.cs │ │ │ ├── KStreamMapAsync.cs │ │ │ ├── KStreamFlatMapAsync.cs │ │ │ ├── KStreamDropDuplicate.cs │ │ │ ├── KStreamKStreamJoin.cs │ │ │ ├── KStreamReduce.cs │ │ │ ├── KStreamKTableJoin.cs │ │ │ ├── KStreamGlobalKTableJoin.cs │ │ │ └── KStreamMapValuesAsync.cs │ │ ├── Joined.cs │ │ └── ConsumedInternal.cs │ ├── Initializer.cs │ └── ExternalContext.cs ├── Crosscutting │ ├── LongComparer.cs │ ├── WatermarkOffsetsByTopicPartition.cs │ ├── ICloneable.cs │ └── ActionHelper.cs ├── Table │ └── Internal │ │ ├── IKTableValueGetterSupplier.cs │ │ ├── IKTableValueGetter.cs │ │ ├── Change.cs │ │ ├── Graph │ │ ├── Nodes │ │ │ ├── TableProcessorParameters.cs │ │ │ └── GroupedTableRepartitionNode.cs │ │ ├── KTableKTableInnerJoin.cs │ │ ├── KTableKTableLeftJoin.cs │ │ ├── KTableKTableOuterJoin.cs │ │ ├── KTableKTableRightJoin.cs │ │ ├── KTableSource.cs │ │ ├── KTableReduce.cs │ │ └── AbstractKTableKTableJoin.cs │ │ ├── GlobalKTable.cs │ │ ├── GenericKTableValueGetterSupplier.cs │ │ ├── TimestampedKeyValueStoreGetter.cs │ │ ├── WindowKeyValueStoreGetter.cs │ │ └── AbstractKTableKTableJoinValueGetterSupplier.cs ├── Errors │ ├── TaskMigratedException.cs │ ├── NotEnoughtTimeException.cs │ ├── StreamConfigException.cs │ ├── NoneRetryableException.cs │ ├── NotMoreValueException.cs │ ├── StreamProducerException.cs │ ├── TopologyException.cs │ ├── ProductionException.cs │ ├── ProcessorStateException.cs │ ├── IllegalStateException.cs │ └── DeserializationException.cs ├── Mock │ ├── Sync │ │ └── ISyncPublisher.cs │ ├── Pipes │ │ ├── IPipeBuilder.cs │ │ ├── IPipeInput.cs │ │ ├── IPipeOutput.cs │ │ └── SyncPipeInput.cs │ ├── ConcurrentTestInputTopic.cs │ ├── MockChangelogRegister.cs │ ├── MockOffsetCheckpointManager.cs │ ├── IBehaviorTopologyTestDriver.cs │ └── Kafka │ │ └── KafkaPipeBuilder.cs ├── SerDes │ ├── Internal │ │ └── BytesSerDes.cs │ ├── SerDesContext.cs │ ├── CharSerDes.cs │ ├── Int32SerDes.cs │ ├── ByteArraySerDes.cs │ ├── Int64SerDes.cs │ ├── FloatSerDes.cs │ └── DoubleSerDes.cs ├── Kafka │ └── IRecordCollector.cs ├── EnumerableExtensions.cs ├── ExceptionHandlerResponse.cs ├── StreamOptions.cs └── RandomGenerator.cs ├── update-client-kafka.sh ├── environment ├── confs │ ├── prometheus.yaml │ └── otel-collector-config.yaml └── start.sh ├── .gitignore ├── test └── Streamiz.Kafka.Net.Tests │ ├── Helpers │ ├── Proto │ │ ├── order.proto │ │ └── person.proto │ ├── SerdesThrowException.cs │ ├── MockProcessorContext.cs │ ├── DictionaryExtensionsTests.cs │ └── StreamConfigExtension.cs │ ├── Private │ ├── RocksDbWindowKeySchemaTests.cs │ ├── ConcurrentSortedDictionaryTests.cs │ ├── TestLogger.cs │ ├── InternalTopologyBuilderTests.cs │ ├── SerDes │ │ ├── CharSerDesTests.cs │ │ └── Int32SerDesTests.cs │ └── OrderedBytesTests.cs │ ├── Processors │ └── KStreamKStreamWindowed │ │ ├── Base │ │ └── JoinSlidingWindowOptions.cs │ │ └── KStreamKStreamWindowedInnerJoinTests.cs │ ├── Public │ ├── TopologyBuilderTests.cs │ └── StreamTableJoinPropsTests.cs │ ├── log4net.config │ ├── Stores │ ├── CachingInMemoryWindowStoreTests.cs │ └── CachingPersistentWindowStoreTests.cs │ └── Metrics │ └── Stats │ ├── ProviderMetricValueTests.cs │ └── ImmutableMetricValueTests.cs ├── .gitpod.Dockerfile ├── launcher ├── sample-stream │ ├── .dockerignore │ ├── Dockerfile │ └── log4net.config └── sample-stream-demo │ ├── .dockerignore │ ├── sample-stream-demo.csproj │ └── Dockerfile ├── remote └── Streamiz.Kafka.Net.Azure.RemoteStorage │ ├── Internal │ └── AzureTableEntity.cs │ ├── AzureRemoteStorageOptions.cs │ └── AzureRemoteStorageSupplier.cs ├── metrics └── Streamiz.Kafka.Net.Metrics.Prometheus │ └── Internal │ ├── Gauge.cs │ └── PrometheusMetricsExporter.cs ├── Makefile ├── roadmap.md └── LICENSE /.vscode/settings.json: -------------------------------------------------------------------------------- 1 | { 2 | } -------------------------------------------------------------------------------- /key.snk: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/LGouellec/streamiz/HEAD/key.snk -------------------------------------------------------------------------------- /.DS_Store: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/LGouellec/streamiz/HEAD/.DS_Store -------------------------------------------------------------------------------- /.github/FUNDING.yml: -------------------------------------------------------------------------------- 1 | github: LGouellec 2 | custom: https://www.paypal.me/slegouellec 3 | -------------------------------------------------------------------------------- /docs/favicon.ico: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/LGouellec/streamiz/HEAD/docs/favicon.ico -------------------------------------------------------------------------------- /docs/assets/state-store.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/LGouellec/streamiz/HEAD/docs/assets/state-store.png -------------------------------------------------------------------------------- /resources/streamiz_v1.0.ai: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/LGouellec/streamiz/HEAD/resources/streamiz_v1.0.ai -------------------------------------------------------------------------------- /resources/Untitled.drawio: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /docs/assets/async-topology.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/LGouellec/streamiz/HEAD/docs/assets/async-topology.png -------------------------------------------------------------------------------- /docs/assets/request-response.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/LGouellec/streamiz/HEAD/docs/assets/request-response.png -------------------------------------------------------------------------------- /docs/assets/threading-model.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/LGouellec/streamiz/HEAD/docs/assets/threading-model.png -------------------------------------------------------------------------------- /docs/assets/logo-kafka-stream-net.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/LGouellec/streamiz/HEAD/docs/assets/logo-kafka-stream-net.png -------------------------------------------------------------------------------- /resources/logo-kafka-stream-net.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/LGouellec/streamiz/HEAD/resources/logo-kafka-stream-net.png -------------------------------------------------------------------------------- /global.json: -------------------------------------------------------------------------------- 1 | { 2 | "sdk": { 3 | "version": "6.0", 4 | "rollForward": "latestMajor", 5 | "allowPrerelease": true 6 | } 7 | } -------------------------------------------------------------------------------- /resources/old_logo-kafka-stream-net-1.0.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/LGouellec/streamiz/HEAD/resources/old_logo-kafka-stream-net-1.0.png -------------------------------------------------------------------------------- /core/Processors/ICloneableProcessor.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Processors 2 | { 3 | internal interface ICloneableProcessor 4 | { 5 | object Clone(); 6 | } 7 | } -------------------------------------------------------------------------------- /core/Processors/IProcessorSupplier.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Processors 2 | { 3 | internal interface IProcessorSupplier 4 | { 5 | IProcessor Get(); 6 | } 7 | } 8 | -------------------------------------------------------------------------------- /core/Metrics/Stats/IStat.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Metrics.Stats 2 | { 3 | internal interface IStat 4 | { 5 | void Record(MetricConfig config, double value, long timeMs); 6 | } 7 | } -------------------------------------------------------------------------------- /core/Streamiz.Kafka.Net.csproj.user: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | -------------------------------------------------------------------------------- /update-client-kafka.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | for file in `find . -type f -name "*.csproj"`; do 4 | sed -i '' -e "s/Include=\"Confluent\(.*\)\" Version=\"\(.*\)\"/Include=\"Confluent\1\" Version=\"$1\"/g" $file 5 | done -------------------------------------------------------------------------------- /core/Metrics/Stats/IMeasurable.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Metrics.Stats 2 | { 3 | internal interface IMeasurable : IMetricValueProvider 4 | { 5 | double Measure(MetricConfig config, long now); 6 | } 7 | } -------------------------------------------------------------------------------- /environment/confs/prometheus.yaml: -------------------------------------------------------------------------------- 1 | scrape_configs: 2 | - job_name: 'otel-collector' 3 | scrape_interval: 10s 4 | static_configs: 5 | - targets: ['otel-collector:8889'] 6 | - targets: ['otel-collector:8888'] -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | bin 2 | obj 3 | .vs 4 | coverage.*opencover.xml 5 | coverage.json 6 | kafka-stream-net.sln.DotSettings.user 7 | _build 8 | build 9 | TestResults 10 | 11 | .idea/ 12 | .vscode/ 13 | 14 | confidential 15 | roadmap.md -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Helpers/Proto/order.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | package Streamiz.Kafka.Net.Tests.Helpers.Proto; 4 | 5 | message Order { 6 | int32 order_id = 1; 7 | float price = 2; 8 | int32 product_id = 3; 9 | } -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Helpers/Proto/person.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | package Streamiz.Kafka.Net.Tests.Helpers.Proto; 4 | 5 | message Person { 6 | string firstName = 1; 7 | string lastName= 2; 8 | int32 age = 3; 9 | } -------------------------------------------------------------------------------- /core/State/Internal/ISegment.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Crosscutting; 2 | 3 | namespace Streamiz.Kafka.Net.State.Internal 4 | { 5 | internal interface ISegment : IKeyValueStore 6 | { 7 | void Destroy(); 8 | } 9 | } 10 | -------------------------------------------------------------------------------- /core/Processors/Internal/TaskStateTransitionValidator.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Processors.Internal 2 | { 3 | internal interface TaskStateTransitionValidator 4 | { 5 | bool IsValidTransition(TaskStateTransitionValidator newState); 6 | } 7 | } 8 | -------------------------------------------------------------------------------- /core/Processors/Internal/ThreadStateTransitionValidator.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Processors.Internal 2 | { 3 | internal interface ThreadStateTransitionValidator 4 | { 5 | bool IsValidTransition(ThreadStateTransitionValidator newState); 6 | } 7 | } 8 | -------------------------------------------------------------------------------- /core/Stream/Internal/INameProvider.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Stream.Internal 2 | { 3 | internal interface INameProvider 4 | { 5 | string NewProcessorName(string prefix); 6 | 7 | string NewStoreName(string prefix); 8 | } 9 | } 10 | -------------------------------------------------------------------------------- /core/Metrics/Librdkafka/IStatisticsHandler.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Metrics.Librdkafka 2 | { 3 | internal interface IStatisticsHandler 4 | { 5 | void Register(StreamMetricsRegistry metricsRegistry); 6 | void Publish(Statistics statistics); 7 | } 8 | } -------------------------------------------------------------------------------- /core/State/Internal/ITimestampedStore.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | namespace Streamiz.Kafka.Net.State.Internal 3 | { 4 | /// 5 | /// Empty interface used by timestamped state store. 6 | /// 7 | public interface ITimestampedStore 8 | { 9 | } 10 | } -------------------------------------------------------------------------------- /core/Metrics/Stats/IMetricValueProvider.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Metrics.Stats 2 | { 3 | /// 4 | /// Super interface for and 5 | /// 6 | internal interface IMetricValueProvider 7 | { 8 | } 9 | } -------------------------------------------------------------------------------- /core/Crosscutting/LongComparer.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | 3 | namespace Streamiz.Kafka.Net.Crosscutting 4 | { 5 | internal class LongComparer : IComparer 6 | { 7 | public int Compare(long x, long y) 8 | => x.CompareTo(y); 9 | } 10 | } 11 | -------------------------------------------------------------------------------- /.gitpod.Dockerfile: -------------------------------------------------------------------------------- 1 | FROM gitpod/workspace-full:latest 2 | 3 | USER gitpod 4 | #.NET installed via .gitpod.yml task until the following issue is fixed: https://github.com/gitpod-io/gitpod/issues/5090 5 | ENV DOTNET_ROOT=/tmp/dotnet 6 | ENV PATH=$PATH:/tmp/dotnet 7 | 8 | RUN sudo apt-get -y install librocksdb-dev -------------------------------------------------------------------------------- /core/Table/Internal/IKTableValueGetterSupplier.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Table.Internal 4 | { 5 | internal interface IKTableValueGetterSupplier 6 | { 7 | IKTableValueGetter Get(); 8 | 9 | String[] StoreNames { get; } 10 | } 11 | } 12 | -------------------------------------------------------------------------------- /core/Errors/TaskMigratedException.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Errors 4 | { 5 | internal class TaskMigratedException : Exception 6 | { 7 | public TaskMigratedException(string message) 8 | : base(message) 9 | { 10 | } 11 | } 12 | } 13 | -------------------------------------------------------------------------------- /core/Metrics/Stats/IMeasurableStat.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Metrics.Stats 2 | { 3 | /// 4 | /// Used for , , ... 5 | /// 6 | internal interface IMeasurableStat : IStat, IMeasurable 7 | { 8 | 9 | } 10 | } -------------------------------------------------------------------------------- /core/Processors/PassThroughProcessor.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Processors 2 | { 3 | internal class PassThroughProcessor : AbstractProcessor 4 | { 5 | public override void Process(K key, V value) 6 | { 7 | this.Forward(key, value); 8 | } 9 | } 10 | } 11 | -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/PassThrough.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | 3 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 4 | { 5 | internal class PassThrough : IProcessorSupplier 6 | { 7 | public IProcessor Get() => new PassThroughProcessor(); 8 | } 9 | } 10 | -------------------------------------------------------------------------------- /core/Metrics/Stats/CumulativeCount.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Metrics.Stats 2 | { 3 | internal class CumulativeCount : CumulativeSum 4 | { 5 | public override void Record(MetricConfig config, double value, long timeMs) 6 | { 7 | base.Record(config, 1, timeMs); 8 | } 9 | } 10 | } -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Private/RocksDbWindowKeySchemaTests.cs: -------------------------------------------------------------------------------- 1 | using NUnit.Framework; 2 | 3 | namespace Streamiz.Kafka.Net.Tests.Private 4 | { 5 | public class RocksDbWindowKeySchemaTests 6 | { 7 | [Test] 8 | public void test() 9 | { 10 | // TODOTEST 11 | } 12 | } 13 | } -------------------------------------------------------------------------------- /core/State/IReadOnlySessionStore.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.State 2 | { 3 | /// 4 | /// NOT IMPLEMENTED FOR MOMENT 5 | /// 6 | /// 7 | /// 8 | public interface IReadOnlySessionStore 9 | { 10 | } 11 | } 12 | -------------------------------------------------------------------------------- /core/Table/Internal/IKTableValueGetter.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.State; 2 | 3 | namespace Streamiz.Kafka.Net.Table.Internal 4 | { 5 | internal interface IKTableValueGetter 6 | { 7 | void Init(ProcessorContext context); 8 | 9 | ValueAndTimestamp Get(K key); 10 | 11 | void Close(); 12 | } 13 | } 14 | -------------------------------------------------------------------------------- /core/Metrics/Stats/IGauge.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Metrics.Stats 2 | { 3 | internal interface IGauge 4 | { 5 | object Value(MetricConfig config, long now); 6 | } 7 | 8 | internal interface IGauge : IMetricValueProvider, IGauge 9 | { 10 | new T Value(MetricConfig config, long now); 11 | } 12 | } -------------------------------------------------------------------------------- /core/Metrics/Stats/WindowedCount.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Metrics.Stats 2 | { 3 | internal class WindowedCount : WindowedSum 4 | { 5 | protected override void Update(Sample sample, MetricConfig config, double value, long timeMs) 6 | { 7 | base.Update(sample, config, 1.0, timeMs); 8 | } 9 | } 10 | } -------------------------------------------------------------------------------- /core/Processors/Internal/IGlobalStateManager.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | 3 | namespace Streamiz.Kafka.Net.Processors.Internal 4 | { 5 | internal interface IGlobalStateManager : IStateManager 6 | { 7 | ISet Initialize(); 8 | 9 | void SetGlobalProcessorContext(ProcessorContext processorContext); 10 | } 11 | } 12 | -------------------------------------------------------------------------------- /core/State/Supplier/ISessionBytesStoreSupplier.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Crosscutting; 2 | 3 | namespace Streamiz.Kafka.Net.State.Supplier 4 | { 5 | /// 6 | /// NOT IMPLEMENTED FOR MOMENT 7 | /// 8 | public interface ISessionBytesStoreSupplier : IStoreSupplier> 9 | { 10 | } 11 | } 12 | -------------------------------------------------------------------------------- /core/State/Cache/ICachedStateStore.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.Generic; 3 | using Streamiz.Kafka.Net.Table.Internal; 4 | 5 | namespace Streamiz.Kafka.Net.State.Cache 6 | { 7 | internal interface ICachedStateStore 8 | { 9 | bool SetFlushListener(Action>> listener, bool sendOldChanges); 10 | } 11 | } -------------------------------------------------------------------------------- /core/Mock/Sync/ISyncPublisher.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using System; 3 | 4 | namespace Streamiz.Kafka.Net.Mock.Sync 5 | { 6 | internal interface ISyncPublisher 7 | { 8 | public void PublishRecord(string topic, byte[] key, byte[] value, DateTime timestamp, Headers headers); 9 | public void Flush(); 10 | public void Close(); 11 | } 12 | } -------------------------------------------------------------------------------- /core/Mock/Pipes/IPipeBuilder.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Threading; 3 | 4 | namespace Streamiz.Kafka.Net.Mock.Pipes 5 | { 6 | internal interface IPipeBuilder 7 | { 8 | IPipeInput Input(string topic, IStreamConfig configuration); 9 | 10 | IPipeOutput Output(string topic, TimeSpan consumeTimeout, IStreamConfig configuration, CancellationToken token = default); 11 | } 12 | } 13 | -------------------------------------------------------------------------------- /core/State/ISessionStore.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | 3 | namespace Streamiz.Kafka.Net.State 4 | { 5 | /// 6 | /// NOT IMPLEMENTED FOR MOMENT 7 | /// 8 | /// 9 | /// 10 | public interface ISessionStore : IStateStore, IReadOnlySessionStore 11 | { 12 | } 13 | } 14 | -------------------------------------------------------------------------------- /core/Errors/NotEnoughtTimeException.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Errors 4 | { 5 | internal class NotEnoughtTimeException : Exception 6 | { 7 | private long ElapsedTime { get; } 8 | 9 | public NotEnoughtTimeException(string message, long elapsedTime) 10 | : base(message) 11 | { 12 | ElapsedTime = elapsedTime; 13 | } 14 | } 15 | } -------------------------------------------------------------------------------- /core/Processors/Internal/IChangelogReader.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | using Confluent.Kafka; 3 | 4 | namespace Streamiz.Kafka.Net.Processors.Internal 5 | { 6 | internal interface IChangelogReader : IChangelogRegister 7 | { 8 | void Restore(); 9 | void Clear(); 10 | bool IsEmpty { get; } 11 | IEnumerable CompletedChangelogs { get; } 12 | } 13 | } 14 | -------------------------------------------------------------------------------- /core/Processors/Internal/IChangelogRegister.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | using Confluent.Kafka; 3 | 4 | namespace Streamiz.Kafka.Net.Processors.Internal 5 | { 6 | internal interface IChangelogRegister 7 | { 8 | void Register(TopicPartition topicPartition, ProcessorStateManager processorStateManager); 9 | void Unregister(IEnumerable topicPartitions); 10 | } 11 | } 12 | -------------------------------------------------------------------------------- /core/Metrics/Stats/Value.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Metrics.Stats 2 | { 3 | internal class Value : IMeasurableStat 4 | { 5 | private double value = 0.0; 6 | 7 | public void Record(MetricConfig config, double value, long timeMs) 8 | { 9 | this.value = value; 10 | } 11 | 12 | public double Measure(MetricConfig config, long now) 13 | => value; 14 | } 15 | } -------------------------------------------------------------------------------- /core/Processors/ThreadStateListener.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors.Internal; 2 | 3 | namespace Streamiz.Kafka.Net.Processors 4 | { 5 | internal delegate void ThreadStateListener(IThread thread, ThreadStateTransitionValidator old, ThreadStateTransitionValidator @new); 6 | 7 | internal delegate void GlobalThreadStateListener(GlobalStreamThread thread, ThreadStateTransitionValidator old, ThreadStateTransitionValidator @new); 8 | 9 | } 10 | -------------------------------------------------------------------------------- /core/Table/Internal/Change.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Table.Internal 2 | { 3 | internal class Change 4 | { 5 | public T OldValue { get; } 6 | public T NewValue { get; } 7 | 8 | public Change(T old, T @new) 9 | { 10 | OldValue = old; 11 | NewValue = @new; 12 | } 13 | 14 | public override string ToString() => $"OldValue:{OldValue}|NewValue:{NewValue}"; 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /core/Mock/Pipes/IPipeInput.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using System; 3 | 4 | namespace Streamiz.Kafka.Net.Mock.Pipes 5 | { 6 | internal delegate void PipeFlushed(); 7 | 8 | internal interface IPipeInput : IDisposable 9 | { 10 | string TopicName { get; } 11 | event PipeFlushed Flushed; 12 | void Pipe(byte[] key, byte[] value, DateTime timestamp, Headers headers); 13 | void Flush(); 14 | 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /core/Processors/IKTableProcessorSupplier.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Table.Internal; 2 | 3 | namespace Streamiz.Kafka.Net.Processors 4 | { 5 | internal interface IKTableProcessorSupplier 6 | { 7 | void EnableSendingOldValues(); 8 | } 9 | 10 | internal interface IKTableProcessorSupplier : IKTableProcessorSupplier, IProcessorSupplier> 11 | { 12 | IKTableValueGetterSupplier View { get; } 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /launcher/sample-stream/.dockerignore: -------------------------------------------------------------------------------- 1 | **/.dockerignore 2 | **/.env 3 | **/.git 4 | **/.gitignore 5 | **/.project 6 | **/.settings 7 | **/.toolstarget 8 | **/.vs 9 | **/.vscode 10 | **/.idea 11 | **/*.*proj.user 12 | **/*.dbmdl 13 | **/*.jfm 14 | **/azds.yaml 15 | **/bin 16 | **/charts 17 | **/docker-compose* 18 | **/Dockerfile* 19 | **/node_modules 20 | **/npm-debug.log 21 | **/obj 22 | **/secrets.dev.yaml 23 | **/values.dev.yaml 24 | LICENSE 25 | README.md -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/Nodes/RootNode.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors.Internal; 2 | 3 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph.Nodes 4 | { 5 | internal class RootNode : StreamGraphNode 6 | { 7 | public RootNode() : base("ROOT-NODE") 8 | { 9 | HasWrittenToTopology = true; 10 | } 11 | 12 | public override void WriteToTopology(InternalTopologyBuilder builder) 13 | { 14 | } 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /launcher/sample-stream-demo/.dockerignore: -------------------------------------------------------------------------------- 1 | **/.dockerignore 2 | **/.env 3 | **/.git 4 | **/.gitignore 5 | **/.project 6 | **/.settings 7 | **/.toolstarget 8 | **/.vs 9 | **/.vscode 10 | **/.idea 11 | **/*.*proj.user 12 | **/*.dbmdl 13 | **/*.jfm 14 | **/azds.yaml 15 | **/bin 16 | **/charts 17 | **/docker-compose* 18 | **/Dockerfile* 19 | **/node_modules 20 | **/npm-debug.log 21 | **/obj 22 | **/secrets.dev.yaml 23 | **/values.dev.yaml 24 | LICENSE 25 | README.md -------------------------------------------------------------------------------- /core/Processors/ITopicManager.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using System; 3 | using System.Collections.Generic; 4 | using System.Threading.Tasks; 5 | 6 | namespace Streamiz.Kafka.Net.Processors 7 | { 8 | internal interface ITopicManager : IDisposable 9 | { 10 | IAdminClient AdminClient { get; } 11 | 12 | Task> ApplyAsync(int topologyId, IDictionary topics, bool allowAutoCreateTopicsIsEnabled); 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /core/Processors/Internal/IGlobalStateMaintainer.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | using Confluent.Kafka; 3 | 4 | namespace Streamiz.Kafka.Net.Processors.Internal 5 | { 6 | internal interface IGlobalStateMaintainer 7 | { 8 | public void Update(ConsumeResult record); 9 | 10 | public void FlushState(bool force = false); 11 | 12 | public void Close(); 13 | 14 | public IDictionary Initialize(); 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /core/Processors/Internal/StaticTopicNameExtractor.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Processors.Internal 2 | { 3 | internal class StaticTopicNameExtractor : ITopicNameExtractor 4 | { 5 | public string TopicName { get; } 6 | 7 | public StaticTopicNameExtractor(string topicName) 8 | { 9 | TopicName = topicName; 10 | } 11 | 12 | public string Extract(K key, V value, IRecordContext recordContext) => TopicName; 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /core/SerDes/Internal/BytesSerDes.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using Streamiz.Kafka.Net.Crosscutting; 3 | 4 | namespace Streamiz.Kafka.Net.SerDes.Internal 5 | { 6 | internal class BytesSerDes : AbstractSerDes 7 | { 8 | public override Bytes Deserialize(byte[] data, SerializationContext context) 9 | => Bytes.Wrap(data); 10 | 11 | public override byte[] Serialize(Bytes data, SerializationContext context) 12 | => data?.Get; 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/Base/JoinSlidingWindowOptions.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Stream; 2 | 3 | namespace Streamiz.Kafka.Net.Tests.Processors 4 | { 5 | internal class JoinSlidingWindowOptions : JoinWindowOptions 6 | { 7 | public JoinSlidingWindowOptions(long beforeMs, long afterMs, long graceMs, long maintainDurationMs) 8 | : base(beforeMs, afterMs, graceMs, maintainDurationMs) 9 | { 10 | } 11 | } 12 | } -------------------------------------------------------------------------------- /core/Metrics/Stats/ImmutableMetricValue.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Metrics.Stats 2 | { 3 | internal class ImmutableMetricValue : IGauge 4 | { 5 | private T value; 6 | 7 | public ImmutableMetricValue(T value) 8 | { 9 | this.value = value; 10 | } 11 | 12 | public T Value(MetricConfig config, long now) 13 | => value; 14 | 15 | object IGauge.Value(MetricConfig config, long now) 16 | => Value(config, now); 17 | } 18 | } -------------------------------------------------------------------------------- /core/State/ITimestampedKeyValueStore.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.State.Internal; 2 | 3 | namespace Streamiz.Kafka.Net.State 4 | { 5 | /// 6 | /// A key-(value/timestamp) store that supports put/get/delete and range queries. 7 | /// 8 | /// key type 9 | /// value type 10 | public interface ITimestampedKeyValueStore : IKeyValueStore>, ITimestampedStore 11 | { 12 | } 13 | } 14 | -------------------------------------------------------------------------------- /core/Metrics/MetricUtils.cs: -------------------------------------------------------------------------------- 1 | using System.Threading.Tasks; 2 | 3 | namespace Streamiz.Kafka.Net.Metrics 4 | { 5 | internal class MetricUtils 6 | { 7 | public static void ExportMetrics( 8 | StreamMetricsRegistry registry, 9 | IStreamConfig config, 10 | string threadName) 11 | { 12 | var sensors = registry.GetThreadScopeSensor(threadName); 13 | Task.Factory.StartNew(() => config.MetricsReporter?.Invoke(sensors)); 14 | } 15 | } 16 | } -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamJoinWindow.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | 3 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 4 | { 5 | internal class KStreamJoinWindow : IProcessorSupplier 6 | { 7 | private readonly string storeName; 8 | 9 | public KStreamJoinWindow(string storeName) 10 | { 11 | this.storeName = storeName; 12 | } 13 | 14 | public IProcessor Get() => new KStreamJoinWindowProcessor(storeName); 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /core/Table/Internal/Graph/Nodes/TableProcessorParameters.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using Streamiz.Kafka.Net.Stream.Internal.Graph.Nodes; 3 | 4 | namespace Streamiz.Kafka.Net.Table.Internal.Graph.Nodes 5 | { 6 | internal class TableProcessorParameters : ProcessorParameters> 7 | { 8 | public TableProcessorParameters(IProcessorSupplier> processorSupplier, string processorName) 9 | : base(processorSupplier, processorName) 10 | { 11 | } 12 | } 13 | } 14 | -------------------------------------------------------------------------------- /core/Processors/IKStreamAggProcessorSupplier.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Table.Internal; 2 | 3 | namespace Streamiz.Kafka.Net.Processors 4 | { 5 | internal interface IKStreamAggProcessorSupplier 6 | { 7 | IKTableValueGetterSupplier View(); 8 | void EnableSendingOldValues(); 9 | } 10 | 11 | internal interface IKStreamAggProcessorSupplier : 12 | IProcessorSupplier, IKStreamAggProcessorSupplier 13 | { 14 | new void EnableSendingOldValues(); 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /core/Crosscutting/WatermarkOffsetsByTopicPartition.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | 3 | namespace Streamiz.Kafka.Net.Crosscutting 4 | { 5 | internal class WatermarkOffsetsByTopicPartition : WatermarkOffsets 6 | { 7 | public string Topic { get; } 8 | public int Partition { get; } 9 | 10 | public WatermarkOffsetsByTopicPartition(string topic, int partition, Offset low, Offset high) 11 | : base(low, high) 12 | { 13 | Topic = topic; 14 | Partition = partition; 15 | } 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /core/Metrics/Stats/ProviderMetricValue.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Metrics.Stats 4 | { 5 | internal class ProviderMetricValue : IGauge 6 | { 7 | private readonly Func provider; 8 | 9 | public ProviderMetricValue(Func provider) 10 | { 11 | this.provider = provider; 12 | } 13 | 14 | public T Value(MetricConfig config, long now) 15 | => provider(); 16 | 17 | object IGauge.Value(MetricConfig config, long now) 18 | => Value(config, now); 19 | } 20 | } -------------------------------------------------------------------------------- /core/Table/Internal/GlobalKTable.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Table.Internal 2 | { 3 | internal class GlobalKTable : IGlobalKTable 4 | { 5 | public GlobalKTable(IKTableValueGetterSupplier valueGetterSupplier, string queryableStoreName) 6 | { 7 | ValueGetterSupplier = valueGetterSupplier; 8 | QueryableStoreName = queryableStoreName; 9 | } 10 | 11 | public IKTableValueGetterSupplier ValueGetterSupplier { get; } 12 | 13 | public string QueryableStoreName { get; } 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /remote/Streamiz.Kafka.Net.Azure.RemoteStorage/Internal/AzureTableEntity.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using Azure; 3 | using Azure.Data.Tables; 4 | 5 | namespace Streamiz.Kafka.Net.Azure.RemoteStorage.Internal 6 | { 7 | internal class AzureTableEntity : ITableEntity 8 | { 9 | public string PartitionKey { get; set; } 10 | public string RowKey { get; set; } 11 | public DateTimeOffset? Timestamp { get; set; } 12 | public ETag ETag { get; set; } 13 | public byte[] Value { get; set; } 14 | public byte[] Key { get; set; } 15 | } 16 | } -------------------------------------------------------------------------------- /core/State/Cache/ICacheFunction.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Crosscutting; 2 | 3 | namespace Streamiz.Kafka.Net.State.Cache 4 | { 5 | internal interface ICacheFunction 6 | { 7 | byte[] BytesFromCacheKey(Bytes cacheKey); 8 | Bytes Key(Bytes cacheKey); 9 | Bytes CacheKey(Bytes cacheKey); 10 | Bytes CacheKey(Bytes key, long segmentId); 11 | long SegmentId(Bytes key); 12 | long SegmentId(long timestamp); 13 | long SegmentInterval { get; } 14 | int CompareSegmentedKeys(Bytes cacheKey, Bytes storeKey); 15 | } 16 | } -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Public/TopologyBuilderTests.cs: -------------------------------------------------------------------------------- 1 | using NUnit.Framework; 2 | using Streamiz.Kafka.Net.Errors; 3 | 4 | namespace Streamiz.Kafka.Net.Tests.Public 5 | { 6 | public class TopologyBuilderTests 7 | { 8 | [Test] 9 | public void SourceTopicAlreadyAdded() 10 | { 11 | var builder = new StreamBuilder(); 12 | builder.Stream("table"); 13 | builder.Stream("table"); 14 | Assert.Throws(() => builder.Build()); 15 | } 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /core/Processors/IAsyncProcessor.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | using System.Threading.Tasks; 3 | using Confluent.Kafka; 4 | using Streamiz.Kafka.Net.Stream; 5 | 6 | namespace Streamiz.Kafka.Net.Processors 7 | { 8 | internal interface IAsyncProcessor 9 | { 10 | public RetryPolicy Policy { get; } 11 | } 12 | 13 | internal interface IAsyncProcessor : IAsyncProcessor 14 | { 15 | public Task>> ProcessAsync(K key, V value, Headers headers, long timestamp, ExternalContext context); 16 | } 17 | } -------------------------------------------------------------------------------- /core/Processors/IThread.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.Generic; 3 | using System.Threading; 4 | 5 | namespace Streamiz.Kafka.Net.Processors 6 | { 7 | interface IThread : IDisposable 8 | { 9 | int Id { get; } 10 | ThreadState State { get; } 11 | bool IsDisposable { get; } 12 | string Name { get; } 13 | bool IsRunning { get; } 14 | void Run(); 15 | void Start(CancellationToken token); 16 | IEnumerable ActiveTasks { get; } 17 | event ThreadStateListener StateChanged; 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/log4net.config: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /core/Crosscutting/ICloneable.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Crosscutting 2 | { 3 | /// 4 | /// Supports cloning, which creates a new instance of a class with the same value as an existing instance. 5 | /// 6 | /// Type of clone object 7 | public interface ICloneable 8 | { 9 | /// 10 | /// Clone current object to an another instance of T. 11 | /// 12 | /// Return an instance 13 | T Clone(); 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /core/Errors/StreamConfigException.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Errors 4 | { 5 | /// 6 | /// Signals that the configuration in your stream is incorrect or maybe a property is missing 7 | /// 8 | public class StreamConfigException : Exception 9 | { 10 | /// 11 | /// Constructor with exception message 12 | /// 13 | /// Message 14 | public StreamConfigException(string message) 15 | : base(message) 16 | { 17 | } 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /core/Processors/Internal/WrapperTopicNameExtractor.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Processors.Internal 4 | { 5 | internal class WrapperTopicNameExtractor : ITopicNameExtractor 6 | { 7 | private readonly Func inner; 8 | 9 | public WrapperTopicNameExtractor(Func inner) 10 | { 11 | this.inner = inner; 12 | } 13 | 14 | public string Extract(K key, V value, IRecordContext recordContext) 15 | => inner.Invoke(key, value, recordContext); 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamMap.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using System.Collections.Generic; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 5 | { 6 | internal class KStreamMap : IProcessorSupplier 7 | { 8 | public IKeyValueMapper> Mapper { get; } 9 | 10 | public KStreamMap(IKeyValueMapper> mapper) 11 | { 12 | this.Mapper = mapper; 13 | } 14 | 15 | public IProcessor Get() => new KStreamMapProcessor(this.Mapper); 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Stores/CachingInMemoryWindowStoreTests.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Crosscutting; 2 | using Streamiz.Kafka.Net.State; 3 | using Streamiz.Kafka.Net.State.InMemory; 4 | 5 | namespace Streamiz.Kafka.Net.Tests.Stores; 6 | 7 | public class CachingInMemoryWindowStoreTests 8 | : AbstractPersistentWindowStoreTests 9 | { 10 | protected override IWindowStore GetBackWindowStore() 11 | { 12 | return new InMemoryWindowStore( 13 | "test-w-store", 14 | RETENTION_MS, 15 | WINDOW_SIZE, 16 | false 17 | ); 18 | } 19 | } -------------------------------------------------------------------------------- /environment/confs/otel-collector-config.yaml: -------------------------------------------------------------------------------- 1 | receivers: 2 | otlp: 3 | protocols: 4 | grpc: 5 | endpoint: 0.0.0.0:4317 6 | 7 | exporters: 8 | debug: 9 | prometheus: 10 | endpoint: "0.0.0.0:8889" 11 | const_labels: 12 | label1: value1 13 | 14 | processors: 15 | batch: 16 | 17 | extensions: 18 | health_check: 19 | pprof: 20 | endpoint: :1888 21 | zpages: 22 | endpoint: :55679 23 | 24 | service: 25 | extensions: [pprof, zpages, health_check] 26 | pipelines: 27 | metrics: 28 | receivers: [otlp] 29 | processors: [batch] 30 | exporters: [debug, prometheus] -------------------------------------------------------------------------------- /core/Errors/NoneRetryableException.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Errors 4 | { 5 | internal class NoneRetryableException : Exception 6 | { 7 | private int NumberRetry { get; } 8 | private long ElapsedTime { get; } 9 | 10 | public NoneRetryableException( 11 | string message, 12 | int numberRetry, 13 | long ellapsedTime, 14 | Exception innerException) 15 | : base(message, innerException) 16 | { 17 | NumberRetry = numberRetry; 18 | ElapsedTime = ellapsedTime; 19 | } 20 | } 21 | } -------------------------------------------------------------------------------- /core/Processors/Internal/DefaultRecordTimestampExtractor.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Processors.Internal 4 | { 5 | internal class DefaultRecordTimestampExtractor : IRecordTimestampExtractor 6 | { 7 | private readonly Func timestampExtractor; 8 | 9 | public DefaultRecordTimestampExtractor() 10 | { 11 | this.timestampExtractor = (k, v, ctx) => ctx.Timestamp; 12 | } 13 | 14 | public long Extract(K key, V value, IRecordContext recordContext) => timestampExtractor(key, value, recordContext); 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /core/Table/Internal/GenericKTableValueGetterSupplier.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Table.Internal 2 | { 3 | internal class GenericKTableValueGetterSupplier : IKTableValueGetterSupplier 4 | { 5 | private readonly IKTableValueGetter getter; 6 | 7 | public GenericKTableValueGetterSupplier(string[] storeName, IKTableValueGetter getter) 8 | { 9 | this.StoreNames = storeName; 10 | this.getter = getter; 11 | } 12 | 13 | public string[] StoreNames { get; } 14 | 15 | public IKTableValueGetter Get() => getter; 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /core/Metrics/MetricsRecordingLevel.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Metrics 2 | { 3 | /// 4 | /// MetricsRecordingLevel enum 5 | /// 6 | /// - INFO = list high level metrics (client, thread, task) 7 | /// - DEBUG = list all metrics (client, thread, task, processor, state-store) 8 | /// 9 | /// 10 | public enum MetricsRecordingLevel 11 | { 12 | /// 13 | /// Info level metrics 14 | /// 15 | INFO, 16 | /// 17 | /// Debug level metrics 18 | /// 19 | DEBUG 20 | } 21 | } -------------------------------------------------------------------------------- /core/Processors/Internal/StateRestoreCallback.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Crosscutting; 2 | 3 | namespace Streamiz.Kafka.Net.Processors.Internal 4 | { 5 | /// 6 | /// Restoration logic for log-backed state stores upon restart, it takes one record at a time from the logs to apply to the restoring state. 7 | /// 8 | /// Record's key 9 | /// Record's value 10 | /// /// Record's timestamp in Unix milliseconds long format 11 | public delegate void StateRestoreCallback(Bytes key, byte[] value, long timestamp); 12 | } 13 | -------------------------------------------------------------------------------- /core/State/RocksDb/Internal/RocksDbSegmentedBytesStore.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.State.Internal 2 | { 3 | internal class RocksDbSegmentedBytesStore : 4 | AbstractRocksDBSegmentedBytesStore 5 | { 6 | public RocksDbSegmentedBytesStore( 7 | string name, 8 | string metricScope, 9 | long retention, 10 | long segmentInterval, IKeySchema keySchema) 11 | : base(name, 12 | keySchema, 13 | new RocksDbKeyValueSegments(name, retention, segmentInterval, metricScope)) 14 | { 15 | } 16 | } 17 | } -------------------------------------------------------------------------------- /launcher/sample-stream-demo/sample-stream-demo.csproj: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | Exe 5 | net8.0 6 | sample_stream_demo 7 | Linux 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | -------------------------------------------------------------------------------- /core/Errors/NotMoreValueException.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections; 3 | 4 | namespace Streamiz.Kafka.Net.Errors 5 | { 6 | /// 7 | /// Exception throws on a empty rocksdb enumerator when is called. 8 | /// 9 | public class NotMoreValueException : Exception 10 | { 11 | /// 12 | /// Constructor with error message 13 | /// 14 | /// Exception message 15 | public NotMoreValueException(string message) 16 | : base(message) 17 | { 18 | } 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /core/Metrics/Stats/CumulativeSum.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Metrics.Stats 2 | { 3 | internal class CumulativeSum : IMeasurableStat 4 | { 5 | private double total; 6 | 7 | public CumulativeSum() : this(0.0) 8 | { } 9 | 10 | public CumulativeSum(double total) 11 | { 12 | this.total = total; 13 | } 14 | 15 | public virtual void Record(MetricConfig config, double value, long timeMs) 16 | { 17 | total += value; 18 | } 19 | 20 | public double Measure(MetricConfig config, long now) 21 | => total; 22 | } 23 | } -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamFlatMapValues.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using System.Collections.Generic; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 5 | { 6 | internal class KStreamFlatMapValues : IProcessorSupplier 7 | { 8 | private IValueMapperWithKey> Mapper { get; } 9 | 10 | public KStreamFlatMapValues(IValueMapperWithKey> mapper) 11 | { 12 | this.Mapper = mapper; 13 | } 14 | 15 | public IProcessor Get() => new KStreamFlatMapValuesProcessor(this.Mapper); 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Private/ConcurrentSortedDictionaryTests.cs: -------------------------------------------------------------------------------- 1 | using NUnit.Framework; 2 | using Streamiz.Kafka.Net.Crosscutting; 3 | using Streamiz.Kafka.Net.State.Cache.Internal; 4 | 5 | namespace Streamiz.Kafka.Net.Tests.Private; 6 | 7 | // TODO : 8 | public class ConcurrentSortedDictionaryTests 9 | { 10 | private ConcurrentSortedDictionary concurrentSet; 11 | 12 | [SetUp] 13 | public void Init() 14 | { 15 | concurrentSet = new ConcurrentSortedDictionary(); 16 | } 17 | 18 | [TearDown] 19 | public void Dispose() 20 | { 21 | concurrentSet.Clear(); 22 | } 23 | } -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamFlatMap.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using System.Collections.Generic; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 5 | { 6 | internal class KStreamFlatMap : IProcessorSupplier 7 | { 8 | public IKeyValueMapper>> Mapper { get; } 9 | 10 | public KStreamFlatMap(IKeyValueMapper>> mapper) 11 | { 12 | this.Mapper = mapper; 13 | } 14 | 15 | public IProcessor Get() => new KStreamFlatMapProcessor(this.Mapper); 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /core/Errors/StreamProducerException.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using Confluent.Kafka; 3 | 4 | namespace Streamiz.Kafka.Net.Errors 5 | { 6 | internal class StreamProducerException : Exception 7 | { 8 | public ProduceException OriginalProduceException { get; set; } 9 | public ProductionExceptionHandlerResponse Response { get; set; } 10 | 11 | public StreamProducerException(ProduceException originalProduceException, ProductionExceptionHandlerResponse response) 12 | { 13 | OriginalProduceException = originalProduceException; 14 | Response = response; 15 | } 16 | } 17 | } -------------------------------------------------------------------------------- /core/Processors/Internal/WrapperRecordTimestampExtractor.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Processors.Internal 4 | { 5 | internal class WrapperRecordTimestampExtractor : IRecordTimestampExtractor 6 | { 7 | private readonly Func timestampExtractor; 8 | 9 | public WrapperRecordTimestampExtractor(Func timestampExtractor) 10 | { 11 | this.timestampExtractor = timestampExtractor; 12 | } 13 | 14 | public long Extract(K key, V value, IRecordContext recordContext) => timestampExtractor(key, value ,recordContext); 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /core/SerDes/SerDesContext.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.SerDes 2 | { 3 | /// 4 | /// is using for configure instance. 5 | /// It is notably used SchemaAvroSerDes to configure ISchemaRegistryClient with url, auto registry schema, etc .. 6 | /// 7 | public class SerDesContext 8 | { 9 | /// 10 | /// Stream application configuration instance 11 | /// 12 | public IStreamConfig Config { get; } 13 | 14 | internal SerDesContext(IStreamConfig config) 15 | { 16 | Config = config; 17 | } 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /core/State/Enumerator/EnumeratorExtensions.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.Generic; 3 | 4 | namespace Streamiz.Kafka.Net.State.Enumerator 5 | { 6 | internal static class EnumeratorExtensions 7 | { 8 | public static IKeyValueEnumerator ToWrap(this IEnumerable> enumerable) 9 | => new WrapEnumerableKeyValueEnumerator(enumerable); 10 | 11 | public static IKeyValueEnumerator Transform(this IKeyValueEnumerator enumerator, Func> function) 12 | => new TransformKeyValueEnumerator(enumerator, function); 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamFilter.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using System; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 5 | { 6 | internal class KStreamFilter : IProcessorSupplier 7 | { 8 | public Func Predicate { get; } 9 | public bool Not { get; } 10 | 11 | 12 | public KStreamFilter(Func predicate, bool not = false) 13 | { 14 | Predicate = predicate; 15 | Not = not; 16 | } 17 | 18 | public IProcessor Get() => new KStreamFilterProcessor(this.Predicate, this.Not); 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamTimestampExtractor.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using System; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 5 | { 6 | internal class KStreamTimestampExtractor : IProcessorSupplier 7 | { 8 | private readonly Func timestampExtractor; 9 | 10 | public KStreamTimestampExtractor(Func timestampExtractor) 11 | { 12 | this.timestampExtractor = timestampExtractor; 13 | } 14 | 15 | public IProcessor Get() => new KStreamTimestampProcessor(timestampExtractor); 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /core/State/Enumerator/IWindowStoreEnumerator.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.Generic; 3 | 4 | namespace Streamiz.Kafka.Net.State.Enumerator 5 | { 6 | /// 7 | /// Iterator interface of with key typed long used for . 8 | /// Users must call its Dispose() method explicitly upon completeness to release resources, 9 | /// or use "using" keyword. 10 | /// 11 | /// Type of values 12 | public interface IWindowStoreEnumerator : IKeyValueEnumerator 13 | { 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamProcessorSupplier.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using Streamiz.Kafka.Net.Processors.Public; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 5 | { 6 | internal class KStreamProcessorSupplier : IProcessorSupplier 7 | { 8 | private readonly ProcessorSupplier processorSupplier; 9 | 10 | public KStreamProcessorSupplier(ProcessorSupplier processorSupplier) 11 | { 12 | this.processorSupplier = processorSupplier; 13 | } 14 | 15 | public Processors.IProcessor Get() 16 | => new KStreamProcessor(processorSupplier); 17 | } 18 | } -------------------------------------------------------------------------------- /core/Metrics/Internal/NoRunnableSensor.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using Streamiz.Kafka.Net.Metrics.Stats; 3 | 4 | namespace Streamiz.Kafka.Net.Metrics.Internal 5 | { 6 | /// 7 | /// No runnable sensor for test 8 | /// 9 | internal class NoRunnableSensor : Sensor 10 | { 11 | internal NoRunnableSensor(string name, string description, MetricsRecordingLevel metricsRecording) 12 | : base(name, description, metricsRecording) 13 | { 14 | NoRunnable = true; 15 | } 16 | 17 | internal static NoRunnableSensor Empty => 18 | new NoRunnableSensor("unknown", "unknown", MetricsRecordingLevel.INFO); 19 | } 20 | } -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Private/TestLogger.cs: -------------------------------------------------------------------------------- 1 | using Microsoft.Extensions.Logging; 2 | using NUnit.Framework; 3 | 4 | namespace Streamiz.Kafka.Net.Tests.Private 5 | { 6 | public class TestLogger 7 | { 8 | [Test] 9 | public void test() 10 | { 11 | var loggerFactory = LoggerFactory.Create(builder => 12 | { 13 | builder.SetMinimumLevel(LogLevel.Debug); 14 | builder.AddConsole(); 15 | }); 16 | 17 | var logger = loggerFactory.CreateLogger(); 18 | logger.LogInformation("Coucou test"); 19 | logger.LogDebug("Debug Coucou test"); 20 | } 21 | } 22 | } -------------------------------------------------------------------------------- /core/Processors/Internal/ObjectDeserialized.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Processors.Internal 2 | { 3 | internal class ObjectDeserialized 4 | { 5 | public object Bean { get; private set; } 6 | public bool MustBeSkipped { get; private set; } 7 | 8 | public static ObjectDeserialized ObjectSkipped => new ObjectDeserialized(true); 9 | 10 | public ObjectDeserialized(object bean, bool mustBeSkipped) 11 | { 12 | Bean = bean; 13 | MustBeSkipped = mustBeSkipped; 14 | } 15 | 16 | private ObjectDeserialized(bool mustBeSkipped) 17 | { 18 | MustBeSkipped = mustBeSkipped; 19 | } 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamPeek.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using System; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 5 | { 6 | internal class KStreamPeek : IProcessorSupplier 7 | { 8 | public bool ForwardDownStream { get; } 9 | public Action Action { get; } 10 | 11 | public KStreamPeek(Action action, bool forwardDownStream) 12 | { 13 | this.Action = action; 14 | this.ForwardDownStream = forwardDownStream; 15 | } 16 | 17 | public IProcessor Get() => new KStreamPeekProcessor(this.Action, this.ForwardDownStream); 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /core/Table/Internal/Graph/Nodes/GroupedTableRepartitionNode.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.SerDes; 2 | using Streamiz.Kafka.Net.SerDes.Internal; 3 | using Streamiz.Kafka.Net.Stream.Internal.Graph.Nodes; 4 | 5 | namespace Streamiz.Kafka.Net.Table.Internal.Graph.Nodes 6 | { 7 | internal class GroupedTableRepartitionNode : RepartitionNode> 8 | { 9 | public GroupedTableRepartitionNode(string streamGraphNode, string sourceName, ISerDes keySerdes, ISerDes valueSerdes, string sinkName, string repartitionTopic) 10 | : base(streamGraphNode, sourceName, null, keySerdes, new ChangeSerDes(valueSerdes), sinkName, repartitionTopic) 11 | { 12 | } 13 | } 14 | } -------------------------------------------------------------------------------- /launcher/sample-stream-demo/Dockerfile: -------------------------------------------------------------------------------- 1 | FROM mcr.microsoft.com/dotnet/runtime:5.0 AS base 2 | WORKDIR /app 3 | 4 | FROM mcr.microsoft.com/dotnet/sdk:5.0 AS build 5 | WORKDIR /src 6 | COPY ["samples/sample-stream-demo/sample-stream-demo.csproj", "sample-stream-demo/"] 7 | RUN dotnet restore "samples/sample-stream-demo/sample-stream-demo.csproj" 8 | COPY . . 9 | WORKDIR "/src/sample-stream-demo" 10 | RUN dotnet build "sample-stream-demo.csproj" -c Release -o /app/build 11 | 12 | FROM build AS publish 13 | RUN dotnet publish "sample-stream-demo.csproj" -c Release -o /app/publish 14 | 15 | FROM base AS final 16 | WORKDIR /app 17 | COPY --from=publish /app/publish . 18 | ENTRYPOINT ["dotnet", "sample-stream-demo.dll"] 19 | -------------------------------------------------------------------------------- /.github/ISSUE_TEMPLATE: -------------------------------------------------------------------------------- 1 | Description 2 | =========== 3 | 4 | 5 | 6 | 7 | How to reproduce 8 | ================ 9 | 10 | 11 | 12 | 13 | Checklist 14 | ========= 15 | 16 | Please provide the following information: 17 | - [ ] A complete (i.e. we can run it), minimal program demonstrating the problem. No need to supply a project file. 18 | - [ ] A code snippet with your topology builder (ex: builder.Stream("topic").to("an-another-topic");) 19 | - [ ] Streamiz.Kafka.Net nuget version. 20 | - [ ] Apache Kafka version. 21 | - [ ] Client configuration. 22 | - [ ] Operating system. 23 | - [ ] Provide logs (with in debug mode (log4net and StreamConfig.Debug) as necessary in configuration). 24 | - [ ] Critical issue. -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamBranch.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using System; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 5 | { 6 | internal class KStreamBranch : IProcessorSupplier 7 | { 8 | public Func[] Predicates { get; } 9 | public String[] ChildNodes { get; } 10 | 11 | public KStreamBranch(Func[] predicates, 12 | String[] childNodes) 13 | { 14 | this.Predicates = predicates; 15 | this.ChildNodes = childNodes; 16 | } 17 | 18 | public IProcessor Get() => new KStreamBranchProcessor(this.Predicates, this.ChildNodes); 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /core/Metrics/Stats/WindowedSum.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | 3 | namespace Streamiz.Kafka.Net.Metrics.Stats 4 | { 5 | internal class WindowedSum : SampledStat 6 | { 7 | public WindowedSum() 8 | : base(0.0) 9 | { 10 | } 11 | 12 | protected override void Update(Sample sample, MetricConfig config, double value, long timeMs) 13 | { 14 | sample.Value += value; 15 | } 16 | 17 | protected override double Combine(List samples, MetricConfig config, long now) 18 | { 19 | double total = 0.0; 20 | foreach (var sample in samples) 21 | total += sample.Value; 22 | return total; 23 | } 24 | } 25 | } -------------------------------------------------------------------------------- /core/Kafka/IRecordCollector.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | using Confluent.Kafka; 3 | using Streamiz.Kafka.Net.SerDes; 4 | 5 | namespace Streamiz.Kafka.Net.Kafka 6 | { 7 | internal interface IRecordCollector 8 | { 9 | IDictionary CollectorOffsets { get; } 10 | void Initialize(); 11 | void Flush(); 12 | void Close(bool dirty); 13 | void Send(string topic, K key, V value, Headers headers, long timestamp, ISerDes keySerializer, ISerDes valueSerializer); 14 | void Send(string topic, K key, V value, Headers headers, int partition, long timestamp, ISerDes keySerializer, ISerDes valueSerializer); 15 | int PartitionsFor(string topic); 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamMapValues.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using System.Collections.Generic; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 5 | { 6 | internal class KStreamMapValues : IProcessorSupplier 7 | { 8 | private readonly IValueMapperWithKey mapper; 9 | 10 | public KStreamMapValues(IValueMapperWithKey mapper) 11 | { 12 | this.mapper = mapper; 13 | } 14 | 15 | public IProcessor Get() => new KStreamMapProcessor( 16 | new WrappedKeyValueMapper>( 17 | (key, value, c) => new KeyValuePair(key, mapper.Apply(key, value, c)))); 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /launcher/sample-stream/Dockerfile: -------------------------------------------------------------------------------- 1 | FROM mcr.microsoft.com/dotnet/runtime:6.0-alpine AS base 2 | WORKDIR /app 3 | 4 | FROM mcr.microsoft.com/dotnet/sdk:6.0-alpine AS build 5 | WORKDIR /src 6 | COPY ["sample-stream.csproj", "sample-stream/"] 7 | RUN dotnet restore "sample-stream/sample-stream.csproj" 8 | COPY . . 9 | COPY ["Program.cs", "sample-stream/"] 10 | COPY ["log4net.config", "sample-stream/"] 11 | WORKDIR "/src/sample-stream" 12 | RUN dotnet build "sample-stream.csproj" -c Release -o /app/build 13 | 14 | FROM build AS publish 15 | RUN dotnet publish "sample-stream.csproj" -c Release -o /app/publish 16 | 17 | FROM base AS final 18 | WORKDIR /app 19 | COPY --from=publish /app/publish . 20 | RUN mkdir /app/store 21 | ENTRYPOINT ["dotnet", "sample-stream.dll"] 22 | -------------------------------------------------------------------------------- /core/State/Supplier/IKeyValueBytesStoreSupplier.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Crosscutting; 2 | 3 | namespace Streamiz.Kafka.Net.State.Supplier 4 | { 5 | /// 6 | /// A store supplier that can be used to create one or more instances of type <Bytes, byte[]>. 7 | /// For any stores implementing the IKeyValueStore<Byte, byte[]> interface, null value bytes are considered as "not exist". This means: 8 | /// 1. Null value bytes in put operations should be treated as delete. 9 | /// 2. If the key does not exist, get operations should return null value bytes. 10 | /// 11 | public interface IKeyValueBytesStoreSupplier : IStoreSupplier> 12 | { 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /core/Processors/Internal/GlobalProcessorContext.cs: -------------------------------------------------------------------------------- 1 | using System.IO; 2 | using Streamiz.Kafka.Net.Metrics; 3 | 4 | namespace Streamiz.Kafka.Net.Processors.Internal 5 | { 6 | internal class GlobalProcessorContext : ProcessorContext 7 | { 8 | internal GlobalProcessorContext( 9 | IStreamConfig configuration, 10 | IStateManager stateManager, 11 | StreamMetricsRegistry streamMetricsRegistry) 12 | : base(null, configuration, stateManager, streamMetricsRegistry) 13 | { 14 | } 15 | 16 | public override TaskId Id => new TaskId { Id = -1, Partition = -1 }; 17 | 18 | public override string StateDir => $"{Path.Combine(Configuration.StateDir, Configuration.ApplicationId, "global")}"; 19 | } 20 | } -------------------------------------------------------------------------------- /core/Errors/TopologyException.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Stream; 2 | using Streamiz.Kafka.Net.Stream.Internal; 3 | using System; 4 | 5 | namespace Streamiz.Kafka.Net.Errors 6 | { 7 | /// 8 | /// Indicates a pre run time error occurred while parsing the logical topology 9 | /// to construct the physical processor topology. 10 | /// 11 | public class TopologyException : Exception 12 | { 13 | /// 14 | /// Constructor with exception message 15 | /// 16 | /// Exception message 17 | public TopologyException(string message) 18 | : base(message) 19 | { 20 | } 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /core/Processors/Public/WrappedProcessor.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Processors.Public 4 | { 5 | internal class WrappedProcessor : IProcessor, ICloneableProcessor 6 | { 7 | private readonly Action> intern; 8 | 9 | public WrappedProcessor(Action> intern) 10 | { 11 | this.intern = intern; 12 | } 13 | 14 | public void Init(ProcessorContext context) 15 | { } 16 | 17 | public void Process(Record record) 18 | => intern.Invoke(record); 19 | 20 | public void Close() 21 | { } 22 | 23 | public object Clone() 24 | { 25 | return new WrappedProcessor(intern); 26 | } 27 | } 28 | } -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Helpers/SerdesThrowException.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using Streamiz.Kafka.Net.SerDes; 3 | using System; 4 | using System.Text; 5 | 6 | namespace Streamiz.Kafka.Net.Tests.Helpers 7 | { 8 | internal class SerdesThrowException : AbstractSerDes 9 | { 10 | public override string Deserialize(byte[] data, SerializationContext context) 11 | { 12 | if (data.Length % 2 == 0) 13 | throw new NotImplementedException(); 14 | else 15 | return Encoding.UTF8.GetString(data); 16 | } 17 | 18 | public override byte[] Serialize(string data, SerializationContext context) 19 | { 20 | return Encoding.UTF8.GetBytes(data); 21 | } 22 | } 23 | } 24 | -------------------------------------------------------------------------------- /core/Mock/Pipes/IPipeOutput.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.Generic; 3 | using Confluent.Kafka; 4 | 5 | namespace Streamiz.Kafka.Net.Mock.Pipes 6 | { 7 | internal class PipeOutputInfo 8 | { 9 | public string Topic { get; set; } 10 | public int Partition { get; set; } 11 | public long Offset { get; set; } 12 | public long Low { get; set; } 13 | public long High { get; set; } 14 | } 15 | 16 | internal interface IPipeOutput : IDisposable 17 | { 18 | string TopicName { get; } 19 | ConsumeResult Read(); 20 | IEnumerable> ReadList(); 21 | List GetInfos(); 22 | int Size { get; } 23 | bool IsEmpty { get; } 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /core/Processors/Internal/ExtractRecordMetadataTimestamp.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | 3 | namespace Streamiz.Kafka.Net.Processors.Internal 4 | { 5 | internal abstract class ExtractRecordMetadataTimestamp : ITimestampExtractor 6 | { 7 | public long Extract(ConsumeResult record, long partitionTime) 8 | { 9 | if (record.Message.Timestamp.UnixTimestampMs < 0) 10 | { 11 | return onInvalidTimestamp(record, record.Message.Timestamp.UnixTimestampMs, partitionTime); 12 | } 13 | 14 | return record.Message.Timestamp.UnixTimestampMs; 15 | } 16 | 17 | public abstract long onInvalidTimestamp(ConsumeResult record, long recordTimestamp, long partitionTime); 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /core/Table/Internal/TimestampedKeyValueStoreGetter.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.State; 2 | 3 | namespace Streamiz.Kafka.Net.Table.Internal 4 | { 5 | internal class TimestampedKeyValueStoreGetter : IKTableValueGetter 6 | { 7 | private readonly string storeName; 8 | private ITimestampedKeyValueStore store; 9 | 10 | public TimestampedKeyValueStoreGetter(string storeName) 11 | { 12 | this.storeName = storeName; 13 | } 14 | 15 | public void Close() { } 16 | 17 | public ValueAndTimestamp Get(K key) 18 | => store.Get(key); 19 | 20 | public void Init(ProcessorContext context) => 21 | store = (ITimestampedKeyValueStore)context.GetStateStore(storeName); 22 | } 23 | } 24 | -------------------------------------------------------------------------------- /core/State/Internal/ISegments.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | 3 | namespace Streamiz.Kafka.Net.State.Internal 4 | { 5 | internal interface ISegments 6 | where S : ISegment 7 | { 8 | long SegmentId(long timestamp); 9 | string SegmentName(long segmentId); 10 | S GetSegmentForTimestamp(long timestamp); 11 | S GetOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime); 12 | S GetOrCreateSegment(long segmentId, ProcessorContext context); 13 | void OpenExisting(ProcessorContext context, long streamTime); 14 | IEnumerable Segments(long timeFrom, long timeTo, bool forward); 15 | IEnumerable AllSegments(bool forward); 16 | void Flush(); 17 | void Close(); 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /core/Stream/Internal/Joined.cs: -------------------------------------------------------------------------------- 1 | 2 | using System; 3 | using Streamiz.Kafka.Net.SerDes; 4 | 5 | namespace Streamiz.Kafka.Net.Stream.Internal 6 | { 7 | internal class Joined 8 | { 9 | public ISerDes KeySerdes { get; } 10 | public ISerDes ValueSerdes { get; } 11 | public ISerDes OtherValueSerDes { get; } 12 | public string Name { get; } 13 | 14 | public Joined(ISerDes keySerdes, 15 | ISerDes valueSerdes, 16 | ISerDes otherValueSerDes, 17 | String name) 18 | { 19 | KeySerdes = keySerdes; 20 | ValueSerdes = valueSerdes; 21 | OtherValueSerDes = otherValueSerDes; 22 | Name = name; 23 | } 24 | } 25 | } -------------------------------------------------------------------------------- /remote/Streamiz.Kafka.Net.Azure.RemoteStorage/AzureRemoteStorageOptions.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Azure.RemoteStorage 2 | { 3 | public class AzureRemoteStorageOptions 4 | { 5 | public const string storageUriCst = "azure.remote.storage.uri"; 6 | public const string accountNameCst = "azure.remote.storage.account.name"; 7 | public const string storageAccountKeyCst = "azure.remote.storage.account.key"; 8 | 9 | [StreamConfigProperty(storageUriCst)] 10 | public string StorageUri { get; set; } 11 | 12 | [StreamConfigProperty(accountNameCst)] 13 | public string AccountName { get; set; } 14 | 15 | [StreamConfigProperty(storageAccountKeyCst)] 16 | public string StorageAccountKey { get; set; } 17 | } 18 | } -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Helpers/MockProcessorContext.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | using Confluent.Kafka; 3 | using Streamiz.Kafka.Net.Metrics; 4 | using Streamiz.Kafka.Net.Mock; 5 | using Streamiz.Kafka.Net.Processors; 6 | using Streamiz.Kafka.Net.Processors.Internal; 7 | 8 | namespace Streamiz.Kafka.Net.Tests.Helpers; 9 | 10 | public class MockProcessorContext : ProcessorContext 11 | { 12 | public MockProcessorContext(TaskId id, StreamConfig config) 13 | : base(UnassignedStreamTask.Create(), config, new ProcessorStateManager( 14 | id, 15 | new List(), 16 | new Dictionary(), 17 | new MockChangelogRegister(), 18 | new MockOffsetCheckpointManager()), new StreamMetricsRegistry()) 19 | { } 20 | } -------------------------------------------------------------------------------- /core/State/Suppress/ITimeOrderedKeyValueBuffer.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using Streamiz.Kafka.Net.Processors; 3 | using Streamiz.Kafka.Net.SerDes; 4 | using Streamiz.Kafka.Net.State.Suppress.Internal; 5 | 6 | namespace Streamiz.Kafka.Net.State.Suppress 7 | { 8 | internal interface ITimeOrderedKeyValueBuffer : IStateStore 9 | { 10 | long NumRecords { get; } 11 | long BufferSize { get; } 12 | long MinTimestamp { get; } 13 | 14 | bool Put(long timestamp, K key, T value, IRecordContext recordContext); 15 | Maybe> PriorValueForBuffered(K key); 16 | void EvictWhile(Func predicate, Action evictHandler); 17 | void SetSerdesIfNull(ISerDes contextKeySerdes, ISerDes contextValueSerdes); 18 | } 19 | } -------------------------------------------------------------------------------- /metrics/Streamiz.Kafka.Net.Metrics.Prometheus/Internal/Gauge.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | 3 | namespace Streamiz.Kafka.Net.Metrics.Prometheus 4 | { 5 | internal class Gauge 6 | { 7 | public string Key { get; set; } 8 | public string Description { get; set; } 9 | public IReadOnlyDictionary Labels { get; set; } 10 | public double Value { get; private set; } 11 | 12 | public Gauge(string key, string description, IReadOnlyDictionary labels) 13 | { 14 | Key = key; 15 | Description = description; 16 | Labels = labels; 17 | } 18 | 19 | public Gauge SetValue(double value) 20 | { 21 | Value = value; 22 | return this; 23 | } 24 | } 25 | } -------------------------------------------------------------------------------- /core/State/Internal/GlobalStateStoreProviderFacade.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | 3 | namespace Streamiz.Kafka.Net.State.Internal 4 | { 5 | internal class GlobalStateStoreProviderFacade : IStateStoreProvider where T : class 6 | { 7 | GlobalStateStoreProvider globalStateStoreProvider; 8 | 9 | public GlobalStateStoreProviderFacade(GlobalStateStoreProvider globalStateStoreProvider) 10 | { 11 | this.globalStateStoreProvider = globalStateStoreProvider; 12 | } 13 | 14 | public IEnumerable Stores(string storeName, IQueryableStoreType queryableStoreType) 15 | { 16 | return this.globalStateStoreProvider.Stores(StoreQueryParameters.FromNameAndType(storeName, queryableStoreType)); 17 | } 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /core/Processors/KStreamMapProcessor.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Stream; 2 | using System.Collections.Generic; 3 | 4 | namespace Streamiz.Kafka.Net.Processors 5 | { 6 | internal class KStreamMapProcessor : AbstractProcessor 7 | { 8 | private readonly IKeyValueMapper> mapper; 9 | 10 | public KStreamMapProcessor(IKeyValueMapper> mapper) 11 | { 12 | this.mapper = mapper; 13 | } 14 | 15 | 16 | public override void Process(K key, V value) 17 | { 18 | LogProcessingKeyValue(key, value); 19 | KeyValuePair newPair = this.mapper.Apply(key, value, Context.RecordContext); 20 | this.Forward(newPair.Key, newPair.Value); 21 | } 22 | } 23 | } 24 | -------------------------------------------------------------------------------- /Makefile: -------------------------------------------------------------------------------- 1 | # Minimal makefile for Sphinx documentation 2 | # 3 | 4 | # You can set these variables from the command line. 5 | SPHINXOPTS = 6 | SPHINXBUILD = sphinx-build 7 | SPHINXBUILD2 = /usr/local/bin/sphinx-build 8 | SOURCEDIR = docs 9 | BUILDDIR = build 10 | 11 | # Put it first so that "make" without argument is like "make help". 12 | help: 13 | @$(SPHINXBUILD) -M help "$(SOURCEDIR)" "$(BUILDDIR)" $(SPHINXOPTS) $(O) 14 | 15 | .PHONY: help Makefile 16 | 17 | # Local workaround 18 | html2: 19 | @$(SPHINXBUILD2) -M "html" "$(SOURCEDIR)" "$(BUILDDIR)" $(SPHINXOPTS) $(O) 20 | 21 | # Catch-all target: route all unknown targets to Sphinx using the new 22 | # "make mode" option. $(O) is meant as a shortcut for $(SPHINXOPTS). 23 | %: Makefile 24 | @$(SPHINXBUILD) -M $@ "$(SOURCEDIR)" "$(BUILDDIR)" $(SPHINXOPTS) $(O) -------------------------------------------------------------------------------- /core/Errors/ProductionException.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Errors 4 | { 5 | /// 6 | /// Production exception throw when production message is in error and return . 7 | /// 8 | public class ProductionException : Exception 9 | { 10 | /// 11 | /// Production exception throw when production message is in error and return . 12 | /// 13 | /// Exception message 14 | public ProductionException(string message) 15 | : base(message) 16 | { 17 | } 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /core/Mock/ConcurrentTestInputTopic.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | using Streamiz.Kafka.Net.Mock.Pipes; 3 | using Streamiz.Kafka.Net.SerDes; 4 | 5 | namespace Streamiz.Kafka.Net.Mock 6 | { 7 | public class ConcurrentTestInputTopic : TestInputTopic 8 | { 9 | internal ConcurrentTestInputTopic(IPipeInput pipe, IStreamConfig configuration, ISerDes keySerdes, ISerDes valueSerdes) 10 | : base(pipe, configuration, keySerdes, valueSerdes) 11 | { 12 | } 13 | 14 | public override void PipeInput(TestRecord record) 15 | { 16 | base.PipeInput(record); 17 | } 18 | 19 | public override void PipeInputs(IEnumerable> records) 20 | { 21 | base.PipeInputs(records); 22 | } 23 | } 24 | } -------------------------------------------------------------------------------- /core/Processors/KStreamPrintProcessor.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Stream.Internal.Graph; 2 | 3 | namespace Streamiz.Kafka.Net.Processors 4 | { 5 | internal class KStreamPrintProcessor : AbstractProcessor 6 | { 7 | private readonly PrintForeachAction actionPrint; 8 | 9 | public KStreamPrintProcessor(PrintForeachAction actionPrint) 10 | { 11 | this.actionPrint = actionPrint; 12 | } 13 | 14 | public override void Process(K key, V value) 15 | { 16 | LogProcessingKeyValue(key, value); 17 | actionPrint.Apply(key, value, Context.RecordContext); 18 | } 19 | 20 | public override void Close() 21 | { 22 | base.Close(); 23 | actionPrint.Close(); 24 | } 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/Nodes/StreamSourceNode.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors.Internal; 2 | 3 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph.Nodes 4 | { 5 | internal class StreamSourceNode : StreamGraphNode 6 | { 7 | protected string topicName; 8 | protected ConsumedInternal consumed; 9 | 10 | public StreamSourceNode(string topicName, string streamGraphNode, ConsumedInternal consumed) 11 | : base(streamGraphNode) 12 | { 13 | this.topicName = topicName; 14 | this.consumed = consumed; 15 | } 16 | 17 | public override void WriteToTopology(InternalTopologyBuilder builder) 18 | { 19 | builder.AddSourceOperator(topicName, this.streamGraphNode, consumed); 20 | } 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /core/Processors/Internal/WrapperStreamPartitioner.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using Confluent.Kafka; 3 | 4 | namespace Streamiz.Kafka.Net.Processors.Internal 5 | { 6 | internal class WrapperStreamPartitioner : IStreamPartitioner 7 | { 8 | private readonly Func _partitioner; 9 | 10 | public WrapperStreamPartitioner(Func partitioner) 11 | { 12 | _partitioner = partitioner; 13 | } 14 | 15 | public void Initialize(IStreamConfig config) 16 | { 17 | 18 | } 19 | 20 | public Partition Partition(string topic, K key, V value, Partition sourcePartition, int numPartitions) 21 | => _partitioner(topic, key, value, sourcePartition, numPartitions); 22 | } 23 | } -------------------------------------------------------------------------------- /core/Processors/KStreamPeekProcessor.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Processors 4 | { 5 | internal class KStreamPeekProcessor : AbstractProcessor 6 | { 7 | private readonly Action action; 8 | private readonly bool forwardDownStream; 9 | 10 | public KStreamPeekProcessor(Action action, bool forwardDownStream) 11 | { 12 | this.action = action; 13 | this.forwardDownStream = forwardDownStream; 14 | } 15 | 16 | public override void Process(K key, V value) 17 | { 18 | LogProcessingKeyValue(key, value); 19 | action.Invoke(key, value, Context.RecordContext); 20 | if (forwardDownStream) 21 | Forward(key, value); 22 | } 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamTransformerSupplier.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using Streamiz.Kafka.Net.Processors.Public; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 5 | { 6 | internal class KStreamTransformerSupplier : IProcessorSupplier 7 | { 8 | private readonly TransformerSupplier transformerSupplier; 9 | private readonly bool changeKey; 10 | 11 | public KStreamTransformerSupplier(TransformerSupplier transformerSupplier, bool changeKey) 12 | { 13 | this.transformerSupplier = transformerSupplier; 14 | this.changeKey = changeKey; 15 | } 16 | 17 | public Processors.IProcessor Get() 18 | => new KStreamTransformer(transformerSupplier, changeKey); 19 | } 20 | } -------------------------------------------------------------------------------- /core/Mock/MockChangelogRegister.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | using Confluent.Kafka; 3 | using Streamiz.Kafka.Net.Processors.Internal; 4 | 5 | namespace Streamiz.Kafka.Net.Mock 6 | { 7 | internal class MockChangelogRegister : IChangelogRegister 8 | { 9 | private readonly IList restoringPartitions = new List(); 10 | 11 | public MockChangelogRegister(){ } 12 | 13 | public void Register(TopicPartition topicPartition, ProcessorStateManager processorStateManager) 14 | { 15 | restoringPartitions.Add(topicPartition); 16 | } 17 | 18 | public void Unregister(IEnumerable topicPartitions) 19 | { 20 | foreach (var tp in topicPartitions) 21 | restoringPartitions.Remove(tp); 22 | } 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /core/Processors/Public/WrappedTransformer.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Processors.Public 4 | { 5 | internal class WrappedTransformer : ITransformer, ICloneableProcessor 6 | { 7 | private readonly Func, Record> transformer; 8 | 9 | public WrappedTransformer(Func, Record> transformer) 10 | { 11 | this.transformer = transformer; 12 | } 13 | 14 | public void Init(ProcessorContext context) 15 | { } 16 | 17 | public Record Process(Record record) 18 | => transformer.Invoke(record); 19 | 20 | public void Close() 21 | { } 22 | 23 | public object Clone() 24 | => new WrappedTransformer(transformer); 25 | } 26 | } -------------------------------------------------------------------------------- /core/Mock/MockOffsetCheckpointManager.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | using Confluent.Kafka; 3 | using Streamiz.Kafka.Net.Processors.Internal; 4 | using Streamiz.Kafka.Net.State; 5 | 6 | namespace Streamiz.Kafka.Net.Mock 7 | { 8 | internal class MockOffsetCheckpointManager : IOffsetCheckpointManager 9 | { 10 | public MockOffsetCheckpointManager() 11 | { 12 | } 13 | 14 | public void Configure(IStreamConfig config, TaskId taskId) 15 | { 16 | } 17 | 18 | public void Destroy(TaskId taskId) 19 | { 20 | } 21 | 22 | public IDictionary Read(TaskId taskId) 23 | => new Dictionary(); 24 | 25 | public void Write(TaskId taskId, IDictionary data) 26 | { 27 | } 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /core/State/Internal/IKeySchema.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Crosscutting; 2 | using Streamiz.Kafka.Net.State.Enumerator; 3 | using System; 4 | using System.Collections.Generic; 5 | 6 | namespace Streamiz.Kafka.Net.State.Internal 7 | { 8 | internal interface IKeySchema 9 | { 10 | Bytes UpperRange(Bytes key, long to); 11 | 12 | Bytes LowerRange(Bytes key, long from); 13 | 14 | Bytes UpperRangeFixedSize(Bytes key, long to); 15 | 16 | Bytes LowerRangeFixedSize(Bytes key, long from); 17 | 18 | long SegmentTimestamp(Bytes key); 19 | 20 | Func, bool> HasNextCondition(Bytes binaryKeyFrom, Bytes binaryKeyTo, long from, long to); 21 | 22 | IList SegmentsToSearch(ISegments segments, long from, long to, bool forward) 23 | where S : ISegment; 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /core/Stream/Internal/ConsumedInternal.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using Streamiz.Kafka.Net.SerDes; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal 5 | { 6 | internal class ConsumedInternal 7 | { 8 | internal ISerDes KeySerdes { get; } 9 | internal ISerDes ValueSerdes { get; } 10 | internal ITimestampExtractor TimestampExtractor { get; } 11 | internal string Named { get; } 12 | 13 | public ConsumedInternal( 14 | string named, 15 | ISerDes keySerdes, 16 | ISerDes valueSerdes, 17 | ITimestampExtractor timestampExtractor) 18 | { 19 | Named = named; 20 | KeySerdes = keySerdes; 21 | ValueSerdes = valueSerdes; 22 | TimestampExtractor = timestampExtractor; 23 | } 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /core/Table/Internal/WindowKeyValueStoreGetter.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.State; 2 | 3 | namespace Streamiz.Kafka.Net.Table.Internal 4 | { 5 | internal class WindowKeyValueStoreGetter : IKTableValueGetter, V> 6 | { 7 | private readonly string storeName; 8 | private ITimestampedWindowStore store; 9 | 10 | public WindowKeyValueStoreGetter(string storeName) 11 | { 12 | this.storeName = storeName; 13 | } 14 | 15 | public void Close() { } 16 | 17 | public ValueAndTimestamp Get(Windowed key) 18 | { 19 | return store.Fetch(key.Key, key.Window.StartMs); 20 | } 21 | 22 | public void Init(ProcessorContext context) 23 | { 24 | store = (ITimestampedWindowStore)context.GetStateStore(storeName); 25 | } 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /core/State/Enumerator/IKeyValueEnumerator.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | 3 | namespace Streamiz.Kafka.Net.State.Enumerator 4 | { 5 | /// 6 | /// Iterator interface of . 7 | /// Users must call its Dispose() method explicitly upon completeness to release resources, 8 | /// or use "using" keyword. 9 | /// 10 | /// Type of keys 11 | /// Type of values 12 | public interface IKeyValueEnumerator : IEnumerator?> 13 | { 14 | /// 15 | /// Peek at the next key without advancing the iterator. 16 | /// 17 | /// the key of the next value that would be returned from the next call to next 18 | K PeekNextKey(); 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /core/Metrics/Stats/Avg.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.Generic; 3 | 4 | namespace Streamiz.Kafka.Net.Metrics.Stats 5 | { 6 | internal class Avg : SampledStat 7 | { 8 | public Avg() 9 | : base(0.0) 10 | { 11 | } 12 | 13 | protected override void Update(Sample sample, MetricConfig config, double value, long timeMs) 14 | { 15 | sample.Value += value; 16 | } 17 | 18 | protected override double Combine(List samples, MetricConfig config, long now) 19 | { 20 | double total = 0.0; 21 | long count = 0; 22 | foreach (var s in samples) 23 | { 24 | total += s.Value; 25 | count += s.EventCount; 26 | } 27 | 28 | return count == 0 ? Double.NaN : total / count; 29 | } 30 | } 31 | } -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamForeachAsync.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Threading.Tasks; 3 | using Streamiz.Kafka.Net.Processors; 4 | 5 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 6 | { 7 | internal class KStreamForeachAsync : IProcessorSupplier 8 | { 9 | private readonly Func, ExternalContext, Task> asyncCall; 10 | private readonly RetryPolicy retryPolicy; 11 | 12 | public KStreamForeachAsync( 13 | Func, ExternalContext, Task> asyncCall, 14 | RetryPolicy retryPolicy) 15 | { 16 | this.asyncCall = asyncCall; 17 | this.retryPolicy = retryPolicy ?? RetryPolicy.NewBuilder().Build(); 18 | } 19 | 20 | public IProcessor Get() 21 | => new KStreamForeachAsyncProcessor(asyncCall, retryPolicy); 22 | 23 | } 24 | } -------------------------------------------------------------------------------- /.vscode/launch.json: -------------------------------------------------------------------------------- 1 | { 2 | // Use IntelliSense to learn about possible attributes. 3 | // Hover to view descriptions of existing attributes. 4 | // For more information, visit: https://go.microsoft.com/fwlink/?linkid=830387 5 | "version": "0.2.0", 6 | "configurations": [ 7 | { 8 | "name": ".NET Core Launch (console)", 9 | "type": "coreclr", 10 | "request": "launch", 11 | "preLaunchTask": "build", 12 | "program": "${workspaceFolder}/test/Consumer/bin/Debug/net5.0/Consumer.dll", 13 | "args": [], 14 | "cwd": "${workspaceFolder}/test/Consumer", 15 | "console": "internalConsole", 16 | "stopAtEntry": false 17 | }, 18 | { 19 | "name": ".NET Core Attach", 20 | "type": "coreclr", 21 | "request": "attach" 22 | } 23 | ] 24 | } -------------------------------------------------------------------------------- /core/EnumerableExtensions.cs: -------------------------------------------------------------------------------- 1 | #if NETSTANDARD2_0 2 | using System; 3 | using System.Collections.Generic; 4 | using System.Text; 5 | using Confluent.Kafka; 6 | 7 | namespace Streamiz.Kafka.Net 8 | { 9 | internal static class EnumerableExtensions 10 | { 11 | public static HashSet ToHashSet(this IEnumerable source) => source.ToHashSet(comparer: null); 12 | 13 | public static HashSet ToHashSet(this IEnumerable source, IEqualityComparer? comparer) 14 | { 15 | if (source == null) 16 | { 17 | throw new ArgumentNullException(nameof(source)); 18 | } 19 | 20 | // Don't pre-allocate based on knowledge of size, as potentially many elements will be dropped. 21 | return new HashSet(source, comparer); 22 | } 23 | } 24 | } 25 | #endif -------------------------------------------------------------------------------- /core/Processors/KStreamFilterProcessor.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Processors 4 | { 5 | internal class KStreamFilterProcessor : AbstractProcessor 6 | { 7 | private readonly Func predicate; 8 | private readonly bool not; 9 | 10 | 11 | public KStreamFilterProcessor(Func predicate, bool not) 12 | { 13 | this.predicate = predicate; 14 | this.not = not; 15 | } 16 | 17 | public override void Process(K key, V value) 18 | { 19 | LogProcessingKeyValue(key, value); 20 | if ((!not && predicate.Invoke(key, value, Context.RecordContext)) || (not && !predicate.Invoke(key, value, Context.RecordContext))) 21 | { 22 | this.Forward(key, value); 23 | } 24 | } 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /core/State/Suppress/Internal/BufferKey.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.Generic; 3 | using Streamiz.Kafka.Net.Crosscutting; 4 | 5 | namespace Streamiz.Kafka.Net.State.Suppress.Internal 6 | { 7 | internal class BufferKeyComparer : IComparer 8 | { 9 | public int Compare(BufferKey x, BufferKey y) 10 | => x.CompareTo(y); 11 | } 12 | 13 | internal class BufferKey : IComparable 14 | { 15 | public long Time { get; } 16 | public Bytes Key { get; } 17 | 18 | public BufferKey(long time, Bytes key) 19 | { 20 | Time = time; 21 | Key = key; 22 | } 23 | 24 | public int CompareTo(BufferKey other) 25 | { 26 | var compared = Time.CompareTo(other.Time); 27 | return compared == 0 ? Key.CompareTo(other.Key) : compared; 28 | } 29 | } 30 | } -------------------------------------------------------------------------------- /core/Processors/KStreamTimestampProcessor.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Processors 4 | { 5 | internal class KStreamTimestampProcessor : AbstractProcessor 6 | { 7 | private readonly Func timestampExtractor; 8 | 9 | public KStreamTimestampProcessor(Func timestampExtractor) 10 | { 11 | this.timestampExtractor = timestampExtractor; 12 | } 13 | 14 | public override void Process(K key, V value) 15 | { 16 | LogProcessingKeyValue(key, value); 17 | var timestamp = timestampExtractor(key, value, Context.RecordContext); 18 | if (timestamp >= 0) 19 | { 20 | Context.RecordContext.ChangeTimestamp(timestamp); 21 | } 22 | 23 | this.Forward(key, value); 24 | } 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /metrics/Streamiz.Kafka.Net.Metrics.Prometheus/Internal/PrometheusMetricsExporter.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | 3 | namespace Streamiz.Kafka.Net.Metrics.Prometheus 4 | { 5 | /// 6 | /// Export the metrics through the prometheus exporter endpoint 7 | /// 8 | internal class PrometheusMetricsExporter 9 | { 10 | private readonly PrometheusRunner prometheusRunner; 11 | 12 | public PrometheusMetricsExporter(PrometheusRunner prometheusRunner) 13 | { 14 | this.prometheusRunner = prometheusRunner; 15 | } 16 | 17 | /// 18 | /// Expose the current sensors/metrics 19 | /// 20 | /// Sensors to emit 21 | public void ExposeMetrics(IEnumerable sensors) 22 | { 23 | prometheusRunner.Expose(sensors); 24 | } 25 | } 26 | } -------------------------------------------------------------------------------- /core/Processors/IRecordTimestampExtractor.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Processors 2 | { 3 | /// 4 | /// An interface that allows to dynamically determine the timestamp of the record stored in the Kafka topic. 5 | /// 6 | /// Key type 7 | /// Value type 8 | public interface IRecordTimestampExtractor 9 | { 10 | /// 11 | /// Extracts the timestamp of the record stored in the Kafka topic. 12 | /// 13 | /// the record key 14 | /// the record value 15 | /// current context metadata of the record 16 | /// the timestamp of the record 17 | long Extract(K key, V value, IRecordContext recordContext); 18 | 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /core/Table/Internal/AbstractKTableKTableJoinValueGetterSupplier.cs: -------------------------------------------------------------------------------- 1 | using System.Linq; 2 | 3 | namespace Streamiz.Kafka.Net.Table.Internal 4 | { 5 | internal abstract class AbstractKTableKTableJoinValueGetterSupplier : 6 | IKTableValueGetterSupplier 7 | { 8 | protected readonly IKTableValueGetterSupplier getter1; 9 | protected readonly IKTableValueGetterSupplier getter2; 10 | 11 | public AbstractKTableKTableJoinValueGetterSupplier( 12 | IKTableValueGetterSupplier getter1, 13 | IKTableValueGetterSupplier getter2) 14 | { 15 | this.getter1 = getter1; 16 | this.getter2 = getter2; 17 | } 18 | 19 | public string[] StoreNames => getter1.StoreNames.Concat(getter2.StoreNames).ToArray(); 20 | 21 | public abstract IKTableValueGetter Get(); 22 | } 23 | } 24 | -------------------------------------------------------------------------------- /core/Mock/IBehaviorTopologyTestDriver.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using Streamiz.Kafka.Net.SerDes; 3 | using System; 4 | 5 | namespace Streamiz.Kafka.Net.Mock 6 | { 7 | internal interface IBehaviorTopologyTestDriver : IDisposable 8 | { 9 | bool IsRunning { get; } 10 | bool IsStopped { get; } 11 | bool IsError { get; } 12 | void StartDriver(); 13 | TestInputTopic CreateInputTopic(string topicName, ISerDes keySerdes, ISerDes valueSerdes); 14 | TestOutputTopic CreateOutputTopic(string topicName, TimeSpan consumeTimeout, ISerDes keySerdes = null, ISerDes valueSerdes = null); 15 | TestMultiInputTopic CreateMultiInputTopic(string[] topics, ISerDes keySerdes = null, ISerDes valueSerdes = null); 16 | IStateStore GetStateStore(string name); 17 | void TriggerCommit(); 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /core/Table/Internal/Graph/KTableKTableInnerJoin.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using Streamiz.Kafka.Net.Stream; 3 | 4 | namespace Streamiz.Kafka.Net.Table.Internal.Graph 5 | { 6 | internal class KTableKTableInnerJoin : AbstractKTableKTableJoin 7 | { 8 | public KTableKTableInnerJoin(IKTableGetter table1, IKTableGetter table2, IValueJoiner valueJoiner) 9 | : base(table1, table2, valueJoiner) 10 | { } 11 | 12 | public override IKTableValueGetterSupplier View 13 | => new KTableKTableInnerJoinValueGetterSupplier(table1.ValueGetterSupplier, table2.ValueGetterSupplier, valueJoiner); 14 | 15 | public override IProcessor> Get() 16 | => new KTableKTableJoinProcessor(table2.ValueGetterSupplier.Get(), valueJoiner, sendOldValues); 17 | } 18 | } 19 | -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/Nodes/ProcessorParameters.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using System; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph.Nodes 5 | { 6 | internal class ProcessorParameters 7 | { 8 | public IProcessorSupplier Processor { get; private set; } 9 | public string ProcessorName { get; private set; } 10 | 11 | public ProcessorParameters(IProcessorSupplier processorSupplier, String processorName) 12 | { 13 | this.Processor = processorSupplier; 14 | this.ProcessorName = processorName; 15 | } 16 | 17 | 18 | public override string ToString() 19 | { 20 | return "ProcessorParameters{" + 21 | "processor class=" + Processor.Get().GetType() + 22 | ", processor name='" + ProcessorName + '\'' + 23 | '}'; 24 | } 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /environment/start.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | curl -i -X PUT http://localhost:8083/connectors/datagen_order/config \ 4 | -H "Content-Type: application/json" \ 5 | -d '{ 6 | "connector.class": "io.confluent.kafka.connect.datagen.DatagenConnector", 7 | "kafka.topic": "input", 8 | "key.converter": "org.apache.kafka.connect.storage.StringConverter", 9 | "value.converter": "org.apache.kafka.connect.json.JsonConverter", 10 | "value.converter.schemas.enable": "false", 11 | "max.interval": 100, 12 | "iterations": 10000000, 13 | "tasks.max": "1", 14 | "schema.filename": "/home/appuser/order.avsc", 15 | "schema.keyfield": "name" 16 | }' 17 | 18 | # curl -i -X PUT http://localhost:8083/connectors/datagen_order/pause 19 | # curl -i -X PUT http://localhost:8083/connectors/datagen_order/resume 20 | # curl -X DELETE http://localhost:8083/connectors/datagen_order -------------------------------------------------------------------------------- /core/Metrics/Stats/Min.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.Generic; 3 | 4 | namespace Streamiz.Kafka.Net.Metrics.Stats 5 | { 6 | internal class Min : SampledStat 7 | { 8 | public Min() 9 | : base(Double.MaxValue) 10 | { 11 | } 12 | 13 | protected override void Update(Sample sample, MetricConfig config, double value, long timeMs) 14 | { 15 | sample.Value = Math.Min(sample.Value, value); 16 | } 17 | 18 | protected override double Combine(List samples, MetricConfig config, long now) 19 | { 20 | double min = Double.MaxValue; 21 | long count = 0; 22 | foreach (var s in samples) 23 | { 24 | min = Math.Min(min, s.Value); 25 | count += s.EventCount; 26 | } 27 | 28 | return count == 0 ? Double.NaN : min; 29 | } 30 | } 31 | } -------------------------------------------------------------------------------- /core/State/ITimestampedWindowStore.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.State.Internal; 2 | 3 | namespace Streamiz.Kafka.Net.State 4 | { 5 | /// 6 | /// Interface for storing the aggregated values of fixed-size time windows. 7 | ///

8 | /// In contrast to a that stores plain windowedKeys-value pairs, 9 | /// a stores windowedKeys-(value/timestamp) pairs. 10 | ///

11 | /// While the window start- and end-timestamp are fixed per window, the value-side timestamp is used 12 | /// to store the last update timestamp of the corresponding window. 13 | ///
14 | /// Type of keys 15 | /// Type of values 16 | public interface ITimestampedWindowStore : IWindowStore>, ITimestampedStore 17 | { 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /core/Processors/Internal/IStateManager.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.Generic; 3 | using Confluent.Kafka; 4 | 5 | namespace Streamiz.Kafka.Net.Processors.Internal 6 | { 7 | internal interface IStateManager 8 | { 9 | IEnumerable StateStoreNames { get; } 10 | ICollection ChangelogPartitions { get; } 11 | IDictionary ChangelogOffsets { get; } 12 | void Flush(); 13 | void Register(IStateStore store, Action> callback); 14 | void UpdateChangelogOffsets(IDictionary writtenOffsets); 15 | void Close(); 16 | IStateStore GetStore(string name); 17 | TopicPartition GetRegisteredChangelogPartitionFor(string name); 18 | void InitializeOffsetsFromCheckpoint(); 19 | void Checkpoint(); 20 | string ChangelogFor(string storeName); 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /core/Table/Internal/Graph/KTableKTableLeftJoin.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using Streamiz.Kafka.Net.Stream; 3 | 4 | namespace Streamiz.Kafka.Net.Table.Internal.Graph 5 | { 6 | internal class KTableKTableLeftJoin : AbstractKTableKTableJoin 7 | { 8 | public KTableKTableLeftJoin(IKTableGetter table1, IKTableGetter table2, IValueJoiner valueJoiner) 9 | : base(table1, table2, valueJoiner) 10 | { 11 | } 12 | 13 | public override IKTableValueGetterSupplier View 14 | => new KTableKTableLeftJoinValueGetterSupplier(table1.ValueGetterSupplier, table2.ValueGetterSupplier, valueJoiner); 15 | 16 | public override IProcessor> Get() 17 | => new KTableKTableLeftJoinProcessor(table2.ValueGetterSupplier.Get(), valueJoiner, sendOldValues); 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /roadmap.md: -------------------------------------------------------------------------------- 1 | # Roadmap Feature 2 | 3 | - [ ] Dead letter queue mecanism 4 | - [ ] Reafactor External Call with similar behavior like ParrallelConsumer + a real async approach 5 | - [ ] Auto scaling consumption (inspired KEDA) 6 | - [ ] State store restore handler with a batch approach 7 | - [ ] Evict cache store as a batch of records instead of unitary records 8 | - [ ] KIP-612 : end-to-end latency metrics 9 | - [ ] KIP-450 : Sliding windows 10 | - [ ] Session Windows 11 | - [ ] Rename WithRecordTimestamp to AlterRecordTimestamp 12 | - [ ] Versioned State Store 13 | - [ ] https://cwiki.apache.org/confluence/display/KAFKA/KIP-923%3A+Add+A+Grace+Period+to+Stream+Table+Join 14 | - [ ] KIP-424 : Allow suppression of intermediate events based on wall clock time 15 | - [ ] Extensions Streamiz 16 | - [ ] At-most-once processing guarantee 17 | - [ ] Support “broadcast” pattern 18 | - [ ] Replace repartition topics with network shuffles 19 | - [ ] Cross Cluster -------------------------------------------------------------------------------- /core/Table/Internal/Graph/KTableKTableOuterJoin.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using Streamiz.Kafka.Net.Stream; 3 | 4 | namespace Streamiz.Kafka.Net.Table.Internal.Graph 5 | { 6 | internal class KTableKTableOuterJoin : AbstractKTableKTableJoin 7 | { 8 | public KTableKTableOuterJoin(IKTableGetter table1, IKTableGetter table2, IValueJoiner valueJoiner) 9 | : base(table1, table2, valueJoiner) 10 | { 11 | } 12 | 13 | public override IKTableValueGetterSupplier View 14 | => new KTableKTableOuterJoinValueGetterSupplier(table1.ValueGetterSupplier, table2.ValueGetterSupplier, valueJoiner); 15 | 16 | public override IProcessor> Get() 17 | => new KTableKTableOuterJoinProcessor(table2.ValueGetterSupplier.Get(), valueJoiner, sendOldValues); 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /core/Table/Internal/Graph/KTableKTableRightJoin.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using Streamiz.Kafka.Net.Stream; 3 | 4 | namespace Streamiz.Kafka.Net.Table.Internal.Graph 5 | { 6 | internal class KTableKTableRightJoin : AbstractKTableKTableJoin 7 | { 8 | public KTableKTableRightJoin(IKTableGetter table1, IKTableGetter table2, IValueJoiner valueJoiner) 9 | : base(table1, table2, valueJoiner) 10 | { 11 | } 12 | 13 | public override IKTableValueGetterSupplier View 14 | => new KTableKTableRightJoinValueGetterSupplier(table1.ValueGetterSupplier, table2.ValueGetterSupplier, valueJoiner); 15 | 16 | public override IProcessor> Get() 17 | => new KTableKTableRightJoinProcessor(table2.ValueGetterSupplier.Get(), valueJoiner, sendOldValues); 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /.github/workflows/integration.yml: -------------------------------------------------------------------------------- 1 | name: integration 2 | 3 | on: 4 | push: 5 | branches: [ master ] 6 | 7 | jobs: 8 | build: 9 | 10 | runs-on: ubuntu-latest 11 | 12 | steps: 13 | - uses: actions/checkout@v2 14 | - name: Setup .NET 8.0 15 | uses: actions/setup-dotnet@v1 16 | with: 17 | dotnet-version: 8.0.404 18 | # BEGIN Dependencies for RocksDB 19 | - run: sudo apt install -y libc6-dev libgflags-dev libsnappy-dev zlib1g-dev libbz2-dev liblz4-dev libzstd-dev 20 | - run: sudo apt install -y bzip2 lz4 librocksdb-dev 21 | # END Dependencies for RocksDB 22 | - name: Install dependencies 23 | run: dotnet restore 24 | - name: Build 25 | run: dotnet build --configuration Debug --no-restore 26 | - name: Test 27 | run: dotnet test --no-build --no-restore --verbosity normal --configuration Debug test/Streamiz.Kafka.Net.IntegrationTests/Streamiz.Kafka.Net.IntegrationTests.csproj 28 | -------------------------------------------------------------------------------- /core/Metrics/Stats/Max.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.Generic; 3 | 4 | namespace Streamiz.Kafka.Net.Metrics.Stats 5 | { 6 | internal class Max : SampledStat 7 | { 8 | public Max() 9 | : base(Double.NegativeInfinity) 10 | { 11 | } 12 | 13 | protected override void Update(Sample sample, MetricConfig config, double value, long timeMs) 14 | { 15 | sample.Value = Math.Max(sample.Value, value); 16 | } 17 | 18 | protected override double Combine(List samples, MetricConfig config, long now) 19 | { 20 | double max = Double.NegativeInfinity; 21 | long count = 0; 22 | foreach (var s in samples) 23 | { 24 | max = Math.Max(max, s.Value); 25 | count += s.EventCount; 26 | } 27 | 28 | return count == 0 ? Double.NaN : max; 29 | } 30 | } 31 | } -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/Nodes/StateStoreNode.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors.Internal; 2 | using Streamiz.Kafka.Net.State; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph.Nodes 5 | { 6 | internal class StateStoreNode : StreamGraphNode 7 | { 8 | protected readonly IStoreBuilder storeBuilder; 9 | protected readonly string[] processorNodeNames; 10 | 11 | public StateStoreNode(IStoreBuilder storeBuilder, 12 | string streamGraphNode, 13 | params string[] processorNodeNames) 14 | : base(streamGraphNode) 15 | { 16 | this.storeBuilder = storeBuilder; 17 | this.processorNodeNames = processorNodeNames; 18 | } 19 | 20 | public override void WriteToTopology(InternalTopologyBuilder builder) 21 | { 22 | builder.AddStateStore(storeBuilder, false, processorNodeNames); 23 | } 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Private/InternalTopologyBuilderTests.cs: -------------------------------------------------------------------------------- 1 | using NUnit.Framework; 2 | using Streamiz.Kafka.Net.Table; 3 | 4 | namespace Streamiz.Kafka.Net.Tests.Private 5 | { 6 | public class InternalTopologyBuilderTests 7 | { 8 | [Test] 9 | public void MakeInternalTopicGroupsTest() 10 | { 11 | var config = new StreamConfig(); 12 | config.ApplicationId = "MakeInternalTopicGroupsTest"; 13 | 14 | var builder = new StreamBuilder(); 15 | 16 | var inmemory = InMemory.As("table-source"); 17 | inmemory.WithLoggingEnabled(null); 18 | builder.Table("source", inmemory); 19 | 20 | var topology = builder.Build(); 21 | 22 | topology.Builder.RewriteTopology(config); 23 | topology.Builder.BuildTopology(); 24 | 25 | var topicsGroups = topology.Builder.MakeInternalTopicGroups(); 26 | } 27 | } 28 | } -------------------------------------------------------------------------------- /core/Mock/Kafka/KafkaPipeBuilder.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Kafka; 2 | using Streamiz.Kafka.Net.Mock.Pipes; 3 | using System; 4 | using System.Threading; 5 | 6 | namespace Streamiz.Kafka.Net.Mock.Kafka 7 | { 8 | internal class KafkaPipeBuilder : IPipeBuilder 9 | { 10 | private readonly IKafkaSupplier kafkaSupplier; 11 | 12 | public KafkaPipeBuilder(IKafkaSupplier kafkaSupplier) 13 | { 14 | this.kafkaSupplier = kafkaSupplier; 15 | } 16 | 17 | public IPipeInput Input(string topic, IStreamConfig configuration) 18 | { 19 | return new KafkaPipeInput(topic, configuration, kafkaSupplier); 20 | } 21 | 22 | public IPipeOutput Output(string topic, TimeSpan consumeTimeout, IStreamConfig configuration, CancellationToken token = default) 23 | { 24 | return new KafkaPipeOutput(topic, consumeTimeout, configuration, kafkaSupplier, token); 25 | } 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /core/State/Cache/Internal/IClockTime.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.State.Cache.Internal 4 | { 5 | internal interface IClockTime 6 | { 7 | DateTime GetCurrentTime(); 8 | } 9 | 10 | internal class ClockSystemTime : IClockTime 11 | { 12 | public DateTime GetCurrentTime() => DateTime.Now; 13 | } 14 | 15 | 16 | internal class MockSystemTime : IClockTime 17 | { 18 | private DateTime initialTime; 19 | private DateTime currentTime; 20 | 21 | public MockSystemTime(DateTime initialTime) 22 | { 23 | this.initialTime = initialTime; 24 | currentTime = initialTime; 25 | } 26 | 27 | public void AdvanceTime(TimeSpan timeSpan) => currentTime = currentTime.Add(timeSpan); 28 | public void ReduceTime(TimeSpan timeSpan) => currentTime = currentTime.Subtract(timeSpan); 29 | public DateTime GetCurrentTime() => currentTime; 30 | } 31 | } -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Metrics/Stats/ProviderMetricValueTests.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using NUnit.Framework; 3 | using Streamiz.Kafka.Net.Crosscutting; 4 | using Streamiz.Kafka.Net.Metrics.Stats; 5 | 6 | namespace Streamiz.Kafka.Net.Tests.Metrics.Stats 7 | { 8 | public class ProviderMetricValueTests 9 | { 10 | private ProviderMetricValue providerMetricValue; 11 | private readonly MetricConfig config = new MetricConfig(); 12 | private static readonly string value = "hello"; 13 | 14 | [SetUp] 15 | public void Init() 16 | { 17 | providerMetricValue = new ProviderMetricValue(() => value); 18 | } 19 | 20 | [Test] 21 | public void ProviderMetricValue() 22 | { 23 | long now = DateTime.Now.GetMilliseconds(); 24 | var v = providerMetricValue.Value(config, now); 25 | Assert.AreEqual(value, v); 26 | } 27 | } 28 | } -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Metrics/Stats/ImmutableMetricValueTests.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using NUnit.Framework; 3 | using Streamiz.Kafka.Net.Crosscutting; 4 | using Streamiz.Kafka.Net.Metrics.Stats; 5 | 6 | namespace Streamiz.Kafka.Net.Tests.Metrics.Stats 7 | { 8 | public class ImmutableMetricValueTests 9 | { 10 | private ImmutableMetricValue immutableMetricValue; 11 | private readonly MetricConfig config = new MetricConfig(); 12 | private static readonly string value = "hello"; 13 | 14 | [SetUp] 15 | public void Init() 16 | { 17 | immutableMetricValue = new ImmutableMetricValue(value); 18 | } 19 | 20 | [Test] 21 | public void ImmutableMetricValue() 22 | { 23 | long now = DateTime.Now.GetMilliseconds(); 24 | var v = immutableMetricValue.Value(config, now); 25 | Assert.AreEqual(value, v); 26 | } 27 | } 28 | } -------------------------------------------------------------------------------- /core/Errors/ProcessorStateException.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Errors 4 | { 5 | /// 6 | /// Indicates a processor state operation (e.g. put, get) has failed. 7 | /// 8 | public class ProcessorStateException : Exception 9 | { 10 | /// 11 | /// Constructor with message 12 | /// 13 | /// Exception message 14 | public ProcessorStateException(string message) : base(message) 15 | { 16 | } 17 | 18 | /// 19 | /// Constructor with message and innerexception 20 | /// 21 | /// Exception message 22 | /// Inner exception 23 | public ProcessorStateException(string message, Exception innerException) 24 | : base(message, innerException) 25 | { 26 | } 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /core/Processors/KStreamJoinWindowProcessor.cs: -------------------------------------------------------------------------------- 1 | using System.Net.Mime; 2 | using Streamiz.Kafka.Net.State; 3 | 4 | namespace Streamiz.Kafka.Net.Processors 5 | { 6 | internal class KStreamJoinWindowProcessor : AbstractProcessor 7 | { 8 | private readonly string storeName; 9 | private IWindowStore window; 10 | 11 | public KStreamJoinWindowProcessor(string storeName) 12 | { 13 | this.storeName = storeName; 14 | } 15 | 16 | public override void Init(ProcessorContext context) 17 | { 18 | base.Init(context); 19 | 20 | window = (IWindowStore)context.GetStateStore(storeName); 21 | } 22 | 23 | public override void Process(K key, V value) 24 | { 25 | if(key != null) 26 | { 27 | Forward(key, value); 28 | window.Put(key, value, Context.Timestamp); 29 | } 30 | } 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /core/Processors/KStreamProcessor.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors.Public; 2 | 3 | namespace Streamiz.Kafka.Net.Processors 4 | { 5 | internal class KStreamProcessor : KStreamPAPI 6 | { 7 | private readonly Streamiz.Kafka.Net.Processors.Public.IProcessor processor; 8 | 9 | public KStreamProcessor(ProcessorSupplier processorSupplier) 10 | { 11 | processor = processorSupplier.Processor; 12 | } 13 | 14 | public override void Init(ProcessorContext context) 15 | { 16 | base.Init(context); 17 | var newContext = new ProcessorContext(context); 18 | processor.Init(newContext); 19 | } 20 | 21 | public override void Close() 22 | { 23 | base.Close(); 24 | processor.Close(); 25 | } 26 | 27 | public override void Process(Record record) 28 | => processor.Process(record); 29 | } 30 | } -------------------------------------------------------------------------------- /core/State/Logging/ChangeLoggingTimestampedWindowBytesStore.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Crosscutting; 2 | using Streamiz.Kafka.Net.SerDes; 3 | 4 | namespace Streamiz.Kafka.Net.State.Logging 5 | { 6 | internal class ChangeLoggingTimestampedWindowBytesStore : 7 | ChangeLoggingWindowBytesStore 8 | { 9 | public ChangeLoggingTimestampedWindowBytesStore(IWindowStore wrapped, bool retainDuplicates) 10 | : base(wrapped, retainDuplicates) 11 | { } 12 | 13 | protected override void Publish(Bytes key, byte[] valueAndTs) 14 | { 15 | if (valueAndTs != null) 16 | { 17 | (long ts, byte[] data) = ValueAndTimestampSerDes.Extract(valueAndTs); 18 | context.Log(Name, key, data, ts); 19 | } 20 | else 21 | { 22 | context.Log(Name, key, null, context.RecordContext.Timestamp); 23 | } 24 | } 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamMapAsync.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.Generic; 3 | using System.Threading.Tasks; 4 | using Streamiz.Kafka.Net.Processors; 5 | 6 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 7 | { 8 | internal class KStreamMapAsync : IProcessorSupplier 9 | { 10 | private readonly Func, ExternalContext, Task>> asyncMapper; 11 | private readonly RetryPolicy retryPolicy; 12 | 13 | public KStreamMapAsync( 14 | Func, ExternalContext, Task>> asyncMapper, 15 | RetryPolicy retryPolicy) 16 | { 17 | this.asyncMapper = asyncMapper; 18 | this.retryPolicy = retryPolicy ?? RetryPolicy.NewBuilder().Build(); 19 | } 20 | 21 | public IProcessor Get() 22 | => new KStreamMapAsyncProcessor(asyncMapper, retryPolicy); 23 | } 24 | } -------------------------------------------------------------------------------- /core/Errors/IllegalStateException.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Errors 4 | { 5 | /// 6 | /// Signals that a method has been invoked at an illegal or inappropriate time. 7 | /// 8 | public class IllegalStateException : Exception 9 | { 10 | /// 11 | /// Constructor of IllegalStateException 12 | /// 13 | /// Exception message 14 | public IllegalStateException(string message) : base(message) 15 | { 16 | } 17 | 18 | /// 19 | /// Constructor of IllegalStateException 20 | /// 21 | /// Exception message 22 | /// Inner exception 23 | public IllegalStateException(string message, Exception innerException) 24 | : base(message, innerException) 25 | { 26 | } 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /core/State/Cache/CacheEntryValue.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using Streamiz.Kafka.Net.Crosscutting; 3 | using Streamiz.Kafka.Net.Processors; 4 | using Streamiz.Kafka.Net.Processors.Internal; 5 | 6 | namespace Streamiz.Kafka.Net.State.Cache 7 | { 8 | internal class CacheEntryValue 9 | { 10 | public byte[] Value { get; } 11 | public IRecordContext Context { get; } 12 | 13 | internal CacheEntryValue(byte[] value) 14 | { 15 | Context = new RecordContext(); 16 | Value = value; 17 | } 18 | 19 | internal CacheEntryValue(byte[] value, Headers headers, long offset, long timestamp, int partition, string topic) 20 | { 21 | Context = new RecordContext(headers.Clone(), offset, timestamp, partition, topic); 22 | Value = value; 23 | } 24 | 25 | public long Size => 26 | (Value != null ? Value.LongLength : 0) + 27 | Context.MemorySizeEstimate; 28 | } 29 | } -------------------------------------------------------------------------------- /launcher/sample-stream/log4net.config: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | 18 | 19 | 20 | 21 | 22 | 23 | 24 | -------------------------------------------------------------------------------- /remote/Streamiz.Kafka.Net.Azure.RemoteStorage/AzureRemoteStorageSupplier.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Azure.RemoteStorage.Internal; 2 | using Streamiz.Kafka.Net.Crosscutting; 3 | using Streamiz.Kafka.Net.State; 4 | using Streamiz.Kafka.Net.State.Supplier; 5 | 6 | namespace Streamiz.Kafka.Net.Azure.RemoteStorage 7 | { 8 | public class AzureRemoteStorageSupplier : IKeyValueBytesStoreSupplier 9 | { 10 | private AzureRemoteStorageOptions AzureRemoteStorageOptions { get; set; } 11 | public string Name { get; set; } 12 | 13 | public AzureRemoteStorageSupplier(AzureRemoteStorageOptions azureRemoteStorageOptions, string name) 14 | { 15 | AzureRemoteStorageOptions = azureRemoteStorageOptions; 16 | Name = name; 17 | } 18 | 19 | public IKeyValueStore Get() 20 | => new AzureTableStore(AzureRemoteStorageOptions, Name); 21 | 22 | public string MetricsScope => "azure-remote-storage"; 23 | } 24 | } -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Stores/CachingPersistentWindowStoreTests.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using Moq; 3 | using Streamiz.Kafka.Net.Crosscutting; 4 | using Streamiz.Kafka.Net.Processors; 5 | using Streamiz.Kafka.Net.Processors.Internal; 6 | using Streamiz.Kafka.Net.State; 7 | using Streamiz.Kafka.Net.State.Cache; 8 | using Streamiz.Kafka.Net.State.Internal; 9 | 10 | namespace Streamiz.Kafka.Net.Tests.Stores; 11 | 12 | public class CachingPersistentWindowStoreTests : 13 | AbstractPersistentWindowStoreTests 14 | { 15 | protected override IWindowStore GetBackWindowStore() 16 | { 17 | var bytesStore = new RocksDbSegmentedBytesStore( 18 | "test-w-store", 19 | "rocksdb-w", 20 | (long)RETENTION_MS.TotalMilliseconds, 21 | SEGMENT_INTERVAL, 22 | keySchema); 23 | 24 | return new RocksDbWindowStore( 25 | bytesStore, 26 | WINDOW_SIZE 27 | , false); 28 | } 29 | } -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamFlatMapAsync.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.Generic; 3 | using System.Threading.Tasks; 4 | using Streamiz.Kafka.Net.Processors; 5 | 6 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 7 | { 8 | internal class KStreamFlatMapAsync : IProcessorSupplier 9 | { 10 | private readonly Func, ExternalContext, Task>>> asyncMapper; 11 | private readonly RetryPolicy retryPolicy; 12 | 13 | public KStreamFlatMapAsync(Func, ExternalContext, Task>>> asyncMapper, RetryPolicy retryPolicy) 14 | { 15 | this.asyncMapper = asyncMapper; 16 | this.retryPolicy = retryPolicy ?? RetryPolicy.NewBuilder().Build(); 17 | } 18 | 19 | public IProcessor Get() 20 | => new KStreamFlatMapAsyncProcessor(asyncMapper, retryPolicy); 21 | } 22 | 23 | 24 | } -------------------------------------------------------------------------------- /core/Errors/DeserializationException.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Errors 4 | { 5 | /// 6 | /// Deserialization exception throw when deserialization input message is in error and return . 7 | /// 8 | public class DeserializationException : Exception 9 | { 10 | /// 11 | /// Deserialization exception throw when deserialization input message is in error and return . 12 | /// 13 | /// Exception message 14 | /// Inner deserialization exception 15 | public DeserializationException(string message, Exception innerException) : 16 | base(message, innerException) 17 | { 18 | } 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /core/Processors/UnwindowedChangelogTopicConfig.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Crosscutting; 2 | using System.Collections.Generic; 3 | 4 | namespace Streamiz.Kafka.Net.Processors 5 | { 6 | internal class UnwindowedChangelogTopicConfig : InternalTopicConfig 7 | { 8 | public static IDictionary UNWINDOWED_STORE_CHANGELOG_TOPIC_DEFAULT_OVERRIDES = new Dictionary(INTERNAL_TOPIC_DEFAULT_OVERRIDES) 9 | { 10 | { InternalTopicConfigCst.CLEANUP_POLICY_CONFIG, "compact"} 11 | }; 12 | 13 | public override IDictionary GetProperties(IDictionary defaultConfigs, long additionalRetentionMs) 14 | { 15 | IDictionary topicConfig = new Dictionary(UNWINDOWED_STORE_CHANGELOG_TOPIC_DEFAULT_OVERRIDES); 16 | topicConfig.AddRange(defaultConfigs); 17 | topicConfig.AddRange(Configs); 18 | return topicConfig; 19 | } 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Helpers/DictionaryExtensionsTests.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | using NUnit.Framework; 3 | using Streamiz.Kafka.Net.Crosscutting; 4 | 5 | namespace Streamiz.Kafka.Net.Tests.Helpers; 6 | 7 | public class DictionaryExtensionsTests 8 | { 9 | [Test] 10 | public void ToUpdateDictionaryWithMergeOperatorTest() 11 | { 12 | var dictionary = new List> 13 | { 14 | new("a", 1L), 15 | new("b", 1L), 16 | new("c", 1L), 17 | new("d", 1L), 18 | new("a", 2L), 19 | new("b", 3L), 20 | }; 21 | 22 | var results = dictionary.ToUpdateDictionary( 23 | kv => kv.Key, 24 | kv => kv.Value, 25 | (@old, @new) => @new); 26 | 27 | Assert.AreEqual(2L, results["a"]); 28 | Assert.AreEqual(3L, results["b"]); 29 | Assert.AreEqual(1L, results["c"]); 30 | Assert.AreEqual(1L, results["d"]); 31 | } 32 | } -------------------------------------------------------------------------------- /core/State/Internal/TimestampedWindowStoreMaterializer.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Crosscutting; 2 | using Streamiz.Kafka.Net.State.Supplier; 3 | using Streamiz.Kafka.Net.Stream; 4 | using Streamiz.Kafka.Net.Table; 5 | using System; 6 | 7 | namespace Streamiz.Kafka.Net.State.Internal 8 | { 9 | internal class TimestampedWindowStoreMaterializer : 10 | WindowStoreMaterializer> 11 | where W : Window 12 | { 13 | public TimestampedWindowStoreMaterializer(WindowOptions windowsOptions, Materialized> materializedInternal) 14 | : base(windowsOptions, materializedInternal) 15 | { 16 | 17 | } 18 | 19 | protected override IStoreBuilder> GetWindowStoreBuilder(IWindowBytesStoreSupplier supplier) 20 | => Stores.TimestampedWindowStoreBuilder(supplier, materializedInternal.KeySerdes, materializedInternal.ValueSerdes); 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /core/State/Suppress/Internal/Maybe.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Errors; 2 | 3 | namespace Streamiz.Kafka.Net.State.Suppress.Internal 4 | { 5 | internal sealed class Maybe 6 | where T : class 7 | { 8 | private T nullableValue; 9 | 10 | public bool IsDefined { get; private set; } 11 | 12 | public T Value 13 | { 14 | get 15 | { 16 | if (IsDefined) 17 | return nullableValue; 18 | 19 | throw new IllegalStateException("Value is not defined"); 20 | } 21 | private set => nullableValue = value; 22 | } 23 | 24 | public Maybe(T nullableValue, bool isDefined) 25 | { 26 | Value = nullableValue; 27 | IsDefined = isDefined; 28 | } 29 | 30 | public static Maybe Defined(T value) => new Maybe(value, true); 31 | public static Maybe Undefined() => new Maybe(null, false); 32 | } 33 | } -------------------------------------------------------------------------------- /core/State/Logging/ChangeLoggingTimestampedKeyValueBytesStore.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Crosscutting; 2 | using Streamiz.Kafka.Net.State.Enumerator; 3 | using Streamiz.Kafka.Net.State.Internal; 4 | using System.Collections.Generic; 5 | using Streamiz.Kafka.Net.SerDes; 6 | 7 | namespace Streamiz.Kafka.Net.State.Logging 8 | { 9 | internal class ChangeLoggingTimestampedKeyValueBytesStore : ChangeLoggingKeyValueBytesStore 10 | { 11 | public ChangeLoggingTimestampedKeyValueBytesStore(IKeyValueStore wrapped) 12 | : base(wrapped) 13 | { 14 | } 15 | 16 | protected override void Publish(Bytes key, byte[] value) 17 | { 18 | if(value == null) 19 | context.Log(Name, key, null, context.RecordContext.Timestamp); 20 | else 21 | { 22 | (long ts, byte[] data) = ValueAndTimestampSerDes.Extract(value); 23 | context.Log(Name, key, data, ts); 24 | } 25 | } 26 | } 27 | } -------------------------------------------------------------------------------- /core/Stream/Initializer.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Stream 4 | { 5 | /// 6 | /// The initializer interface for creating an initial value in aggregations. 7 | /// Initializer is used in combination with Aggregator. 8 | /// 9 | /// aggregate value type 10 | public interface Initializer 11 | { 12 | /// 13 | /// Return the initial value for an aggregation. 14 | /// 15 | /// the initial value for an aggregation 16 | VA Apply(); 17 | } 18 | 19 | internal class WrappedInitializer : Initializer 20 | { 21 | private readonly Func function; 22 | 23 | public WrappedInitializer(Func function) 24 | { 25 | this.function = function ?? throw new ArgumentNullException($"Initializer function can't be null"); 26 | } 27 | 28 | public T Apply() => function.Invoke(); 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamDropDuplicate.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using Streamiz.Kafka.Net.Processors; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 5 | { 6 | internal class KStreamDropDuplicate : IProcessorSupplier 7 | { 8 | private readonly string _name; 9 | private readonly string _storeName; 10 | private readonly Func _valueComparer; 11 | private readonly TimeSpan _interval; 12 | 13 | public KStreamDropDuplicate( 14 | string name, 15 | string storeName, 16 | Func valueComparer, 17 | TimeSpan interval) 18 | 19 | { 20 | _name = name; 21 | _storeName = storeName; 22 | _valueComparer = valueComparer; 23 | _interval = interval; 24 | } 25 | 26 | public IProcessor Get() 27 | => new KStreamDropDuplicateProcessor(_name, _storeName, _valueComparer, (long)_interval.TotalMilliseconds); 28 | } 29 | } -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/Nodes/ProcessorGraphNode.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors.Internal; 2 | 3 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph.Nodes 4 | { 5 | internal abstract class ProcessorGraphNode : StreamGraphNode 6 | { 7 | protected ProcessorGraphNode(string streamGraphNode) 8 | : base(streamGraphNode) 9 | { 10 | } 11 | } 12 | 13 | internal class ProcessorGraphNode : ProcessorGraphNode 14 | { 15 | public ProcessorParameters ProcessorParameters { get; } 16 | 17 | public ProcessorGraphNode(string streamGraphNode, ProcessorParameters @parameters) : 18 | base(streamGraphNode) 19 | { 20 | ProcessorParameters = @parameters; 21 | } 22 | 23 | public override void WriteToTopology(InternalTopologyBuilder builder) 24 | { 25 | builder.AddProcessor(ProcessorParameters.ProcessorName, ProcessorParameters.Processor, ParentNodeNames()); 26 | } 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /core/Processors/KStreamBranchProcessor.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Processors 4 | { 5 | internal class KStreamBranchProcessor : AbstractProcessor 6 | { 7 | private readonly Func[] predicates; 8 | private readonly string[] childNodes; 9 | 10 | public KStreamBranchProcessor(Func[] predicates, string[] childNodes) 11 | { 12 | this.predicates = predicates; 13 | this.childNodes = childNodes; 14 | } 15 | 16 | public override void Process(K key, V value) 17 | { 18 | LogProcessingKeyValue(key, value); 19 | for (int i = 0; i < this.predicates.Length; i++) 20 | { 21 | if (predicates[i].Invoke(key, value, Context.RecordContext)) 22 | { 23 | Forward(key, value, this.childNodes[i]); 24 | break; 25 | } 26 | } 27 | } 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Private/SerDes/CharSerDesTests.cs: -------------------------------------------------------------------------------- 1 | using NUnit.Framework; 2 | using Streamiz.Kafka.Net.SerDes; 3 | 4 | namespace Streamiz.Kafka.Net.Tests.Private.SerDes 5 | { 6 | public class CharSerDesTests 7 | { 8 | [Test] 9 | public void SerializeData() 10 | { 11 | char i = 'b'; 12 | byte[] b = new byte[] {98, 0}; 13 | var serdes = new CharSerDes(); 14 | var r = serdes.Serialize(i, new Confluent.Kafka.SerializationContext()); 15 | Assert.IsNotNull(r); 16 | Assert.AreEqual(b, r); 17 | } 18 | 19 | [Test] 20 | public void DeserializeData() 21 | { 22 | char i = 'p'; 23 | var serdes = new CharSerDes(); 24 | var r = serdes.Deserialize(serdes.Serialize(i, new Confluent.Kafka.SerializationContext()), 25 | new Confluent.Kafka.SerializationContext()); 26 | Assert.IsNotNull(r); 27 | Assert.AreEqual(i, r); 28 | } 29 | } 30 | } -------------------------------------------------------------------------------- /core/Stream/ExternalContext.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.ObjectModel; 3 | 4 | namespace Streamiz.Kafka.Net.Stream 5 | { 6 | /// 7 | /// External context during an asynchronous processing operation 8 | /// 9 | public class ExternalContext 10 | { 11 | /// 12 | /// Current retry number for current record 13 | /// 14 | public int RetryNumber { get; internal set; } 15 | /// 16 | /// First time of call in milliseconds (epoch) 17 | /// 18 | public long FirstCallEpoch { get; internal set; } 19 | /// 20 | /// Current time of call in milliseconds (epoch) 21 | /// 22 | public long CurrentCallEpoch { get; internal set; } 23 | /// 24 | /// Last exceptions thrown during the last processing iteration 25 | /// 26 | public ReadOnlyCollection LastExceptions { get; internal set; } 27 | } 28 | } -------------------------------------------------------------------------------- /core/Processors/Internal/FailOnInvalidTimestamp.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using Microsoft.Extensions.Logging; 3 | using Streamiz.Kafka.Net.Crosscutting; 4 | using Streamiz.Kafka.Net.Errors; 5 | 6 | namespace Streamiz.Kafka.Net.Processors.Internal 7 | { 8 | internal class FailOnInvalidTimestamp : ExtractRecordMetadataTimestamp 9 | { 10 | private readonly ILogger log = Logger.GetLogger(typeof(FailOnInvalidTimestamp)); 11 | 12 | public override long onInvalidTimestamp(ConsumeResult record, long recordTimestamp, long partitionTime) 13 | { 14 | var message = $"Input record {record} has invalid (negative) timestamp. Possibly because a pre-0.10 producer client was used to write this record to Kafka without embedding a timestamp, or because the input topic was created before upgrading the Kafka cluster to 0.10+. Use a different TimestampExtractor to process this data."; 15 | 16 | log.LogError(message); 17 | throw new StreamsException(message); 18 | } 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Private/SerDes/Int32SerDesTests.cs: -------------------------------------------------------------------------------- 1 | using NUnit.Framework; 2 | using Streamiz.Kafka.Net.SerDes; 3 | 4 | namespace Streamiz.Kafka.Net.Tests.Private.SerDes 5 | { 6 | public class Int32SerDesTests 7 | { 8 | [Test] 9 | public void SerializeData() 10 | { 11 | int i = 100; 12 | byte[] b = new byte[] {100, 0, 0, 0}; 13 | var serdes = new Int32SerDes(); 14 | var r = serdes.Serialize(i, new Confluent.Kafka.SerializationContext()); 15 | Assert.IsNotNull(r); 16 | Assert.AreEqual(b, r); 17 | } 18 | 19 | [Test] 20 | public void DeserializeData() 21 | { 22 | int i = 300; 23 | var serdes = new Int32SerDes(); 24 | var r = serdes.Deserialize(serdes.Serialize(i, new Confluent.Kafka.SerializationContext()), 25 | new Confluent.Kafka.SerializationContext()); 26 | Assert.IsNotNull(r); 27 | Assert.AreEqual(i, r); 28 | } 29 | } 30 | } -------------------------------------------------------------------------------- /core/Processors/KStreamFlatMapProcessor.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Stream; 2 | using System.Collections.Generic; 3 | using Streamiz.Kafka.Net.Crosscutting; 4 | 5 | namespace Streamiz.Kafka.Net.Processors 6 | { 7 | internal class KStreamFlatMapProcessor : AbstractProcessor 8 | { 9 | private readonly IKeyValueMapper>> mapper; 10 | 11 | public KStreamFlatMapProcessor(IKeyValueMapper>> mapper) 12 | { 13 | this.mapper = mapper; 14 | } 15 | 16 | public override void Process(K key, V value) 17 | { 18 | LogProcessingKeyValue(key, value); 19 | var originalHeader = Context.RecordContext.Headers.Clone(); 20 | 21 | foreach (var newPair in mapper.Apply(key, value, Context.RecordContext)) 22 | { 23 | Forward(newPair.Key, newPair.Value); 24 | Context.SetHeaders(originalHeader); 25 | } 26 | } 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /core/State/Suppress/InMemoryTimeOrderedKeyValueChangeBufferBuilder.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.SerDes; 2 | using Streamiz.Kafka.Net.State.Suppress.Internal; 3 | 4 | namespace Streamiz.Kafka.Net.State.Suppress 5 | { 6 | internal class InMemoryTimeOrderedKeyValueChangeBufferBuilder 7 | : AbstractStoreBuilder> 8 | { 9 | public InMemoryTimeOrderedKeyValueChangeBufferBuilder(string name, ISerDes keySerde, ISerDes valueSerde) 10 | : base(name, keySerde, valueSerde) 11 | { 12 | WithCachingDisabled(); // disable explicitly the cache 13 | } 14 | 15 | // Not used 16 | public override bool IsWindowStore => false; 17 | public override long RetentionMs => -1; 18 | 19 | public override InMemoryTimeOrderedKeyValueChangeBuffer Build() => 20 | new( 21 | Name, 22 | LoggingEnabled, 23 | keySerdes, 24 | valueSerdes); 25 | } 26 | } -------------------------------------------------------------------------------- /core/Metrics/Librdkafka/TopicStatistic.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | using Newtonsoft.Json; 3 | 4 | namespace Streamiz.Kafka.Net.Metrics.Librdkafka 5 | { 6 | /// 7 | /// JSON Model from : https://github.com/edenhill/librdkafka/blob/master/STATISTICS.md 8 | /// 9 | internal class TopicStatistic 10 | { 11 | [JsonProperty(PropertyName = "topic")] 12 | public string TopicName; 13 | 14 | [JsonProperty(PropertyName = "metadata_age")] 15 | public long MetadataAge; // Gauge 16 | 17 | [JsonProperty(PropertyName = "batchsize")] 18 | public WindowStatistic BatchSize; // in bytes 19 | 20 | [JsonProperty(PropertyName = "batchcnt")] 21 | public WindowStatistic BatchMessageCounts; 22 | 23 | [JsonProperty(PropertyName = "partitions")] 24 | public Dictionary Partitions; 25 | 26 | public TopicStatistic() 27 | { 28 | Partitions = new Dictionary(); 29 | } 30 | } 31 | } -------------------------------------------------------------------------------- /core/Processors/KStreamForeachAsyncProcessor.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.Generic; 3 | using System.Threading.Tasks; 4 | using Confluent.Kafka; 5 | using Streamiz.Kafka.Net.Stream; 6 | 7 | namespace Streamiz.Kafka.Net.Processors 8 | { 9 | internal class KStreamForeachAsyncProcessor : 10 | AbstractAsyncProcessor 11 | { 12 | private readonly Func, ExternalContext, Task> asyncCall; 13 | 14 | public KStreamForeachAsyncProcessor(Func, ExternalContext, Task> asyncCall, RetryPolicy policy) 15 | : base(policy) 16 | { 17 | this.asyncCall = asyncCall; 18 | } 19 | 20 | public override async Task>> ProcessAsync(K key, V value, Headers headers, long timestamp, ExternalContext context) 21 | { 22 | var record = new ExternalRecord(key, value, headers, timestamp); 23 | await asyncCall(record, context); 24 | return null; 25 | } 26 | } 27 | } -------------------------------------------------------------------------------- /.github/workflows/gh-pages.yml: -------------------------------------------------------------------------------- 1 | name: "Build and deploy Github pages" 2 | 3 | on: 4 | push: 5 | branches: 6 | - master 7 | 8 | jobs: 9 | build-and-deploy: 10 | runs-on: ubuntu-latest 11 | 12 | steps: 13 | - name: "Checkout" 14 | uses: actions/checkout@v2 15 | with: 16 | persist-credentials: false 17 | 18 | - name: "Set up Python" 19 | uses: actions/setup-python@v1 20 | with: 21 | python-version: 3.12 22 | 23 | - name: "Install Python dependencies" 24 | run: | 25 | pip3 install setuptools 26 | pip3 install recommonmark 27 | pip3 install sphinx sphinx-rtd-theme==2.0.0 28 | pip3 install sphinx_markdown_tables 29 | 30 | - name: "Build Sphinx Doc" 31 | run: | 32 | make html 33 | 34 | - name: "Deploy Github Pages" 35 | uses: JamesIves/github-pages-deploy-action@v4 36 | with: 37 | GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} 38 | BRANCH: gh-pages 39 | FOLDER: build/html/ 40 | -------------------------------------------------------------------------------- /core/Table/Internal/Graph/KTableSource.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | 3 | namespace Streamiz.Kafka.Net.Table.Internal.Graph 4 | { 5 | internal class KTableSource : IProcessorSupplier 6 | { 7 | private bool sendOldValues = false; 8 | 9 | public string StoreName { get; } 10 | public string QueryableName { get; private set; } 11 | 12 | public KTableSource(string storeName, string queryableName) 13 | { 14 | this.StoreName = storeName; 15 | this.QueryableName = queryableName; 16 | this.sendOldValues = false; 17 | } 18 | 19 | public void EnableSendingOldValues() 20 | { 21 | this.sendOldValues = true; 22 | this.QueryableName = StoreName; 23 | } 24 | 25 | public void Materialize() 26 | { 27 | this.QueryableName = StoreName; 28 | } 29 | 30 | public IProcessor Get() => new KTableSourceProcessor(this.StoreName, this.QueryableName, this.sendOldValues); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Helpers/StreamConfigExtension.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.IO; 3 | 4 | namespace Streamiz.Kafka.Net.Tests.Helpers 5 | { 6 | internal static class StreamConfigExtension 7 | { 8 | internal static StreamConfig UseRandomRocksDbConfigForTest(this StreamConfig config) 9 | { 10 | Guid guid = Guid.NewGuid(); 11 | config.ApplicationId = $"{config.ApplicationId}-{guid}"; 12 | config.StateDir = $"."; 13 | return config; 14 | } 15 | 16 | internal static StreamConfig RemoveRocksDbFolderForTest(this StreamConfig config) 17 | { 18 | int i = 0, maxRetry = 10; 19 | try 20 | { 21 | while (i < maxRetry) 22 | { 23 | Directory.Delete(Path.Combine(config.StateDir, config.ApplicationId), true); 24 | } 25 | }catch(System.IO.IOException e) 26 | { 27 | ++i; 28 | } 29 | return config; 30 | } 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /core/State/Internal/ISegmentedBytesStore.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Crosscutting; 2 | using Streamiz.Kafka.Net.Processors; 3 | using Streamiz.Kafka.Net.State.Enumerator; 4 | 5 | namespace Streamiz.Kafka.Net.State.Internal 6 | { 7 | internal interface ISegmentedBytesStore : IStateStore 8 | { 9 | IKeyValueEnumerator Fetch(Bytes key, long from, long to); 10 | IKeyValueEnumerator ReverseFetch(Bytes key, long from, long to); 11 | IKeyValueEnumerator Fetch(Bytes fromKey, Bytes toKey, long from, long to); 12 | IKeyValueEnumerator ReverseFetch(Bytes fromKey, Bytes toKey, long from, long to); 13 | IKeyValueEnumerator All(); 14 | IKeyValueEnumerator ReverseAll(); 15 | IKeyValueEnumerator FetchAll(long from, long to); 16 | IKeyValueEnumerator ReverseFetchAll(long from, long to); 17 | void Remove(Bytes key); 18 | void Put(Bytes key, byte[] value); 19 | byte[] Get(Bytes key); 20 | } 21 | } -------------------------------------------------------------------------------- /core/ExceptionHandlerResponse.cs: -------------------------------------------------------------------------------- 1 | 2 | namespace Streamiz.Kafka.Net 3 | { 4 | /// 5 | /// Enumeration that describes the response from the exception handler. 6 | /// 7 | public enum ExceptionHandlerResponse 8 | { 9 | /// 10 | /// Fail processing and stop it ! 11 | /// 12 | FAIL = 0, 13 | /// 14 | /// Continue processing ! 15 | /// 16 | CONTINUE = 1 17 | } 18 | 19 | /// 20 | /// Enumeration that describes the response from the production exception handler. 21 | /// 22 | public enum ProductionExceptionHandlerResponse 23 | { 24 | /// 25 | /// Fail processing and stop it ! 26 | /// 27 | FAIL = 0, 28 | /// 29 | /// Continue processing, so skipping this message 30 | /// 31 | CONTINUE = 1, 32 | /// 33 | /// Retrying to send 34 | /// 35 | RETRY = 2 36 | } 37 | } -------------------------------------------------------------------------------- /core/Processors/ITopicNameExtractor.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | 3 | namespace Streamiz.Kafka.Net.Processors 4 | { 5 | /// 6 | /// An interface that allows to dynamically determine the name of the Kafka topic to send at the sink node of the topology. 7 | /// 8 | /// Key type 9 | /// Value type 10 | public interface ITopicNameExtractor 11 | { 12 | /// 13 | /// Extracts the topic name to send to. The topic name must already exist, since the Kafka Streams library will not 14 | /// try to automatically create the topic with the extracted name. 15 | /// 16 | /// the record key 17 | /// the record value 18 | /// current context metadata of the record 19 | /// the topic name this record should be sent to 20 | String Extract(K key, V value, IRecordContext recordContext); 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Private/OrderedBytesTests.cs: -------------------------------------------------------------------------------- 1 | using NUnit.Framework; 2 | using Streamiz.Kafka.Net.Crosscutting; 3 | using Streamiz.Kafka.Net.SerDes; 4 | using Streamiz.Kafka.Net.State.Internal; 5 | using System; 6 | 7 | namespace Streamiz.Kafka.Net.Tests.Private 8 | { 9 | public class OrderedBytesTest 10 | { 11 | [Test] 12 | public void test() 13 | { 14 | BytesComparer comparer = new BytesComparer(); 15 | var serdes = new StringSerDes(); 16 | var bytes = serdes.Serialize("test", new Confluent.Kafka.SerializationContext()); 17 | long to = DateTime.Now.GetMilliseconds(); 18 | 19 | byte[] maxSuffix = ByteBuffer.Build(12, true) 20 | .PutLong(to) 21 | .PutInt(int.MaxValue) 22 | .ToArray(); 23 | 24 | var bytes2 = OrderedBytes.UpperRange(Bytes.Wrap(bytes), maxSuffix); 25 | var bytes3 = OrderedBytes.LowerRange(Bytes.Wrap(bytes), new byte[12]); 26 | int r = comparer.Compare(bytes2, bytes3); 27 | } 28 | } 29 | } -------------------------------------------------------------------------------- /core/Processors/KStreamFlatMapValuesProcessor.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Stream; 2 | using System.Collections.Generic; 3 | using System.Linq; 4 | using Confluent.Kafka; 5 | using Streamiz.Kafka.Net.Crosscutting; 6 | 7 | namespace Streamiz.Kafka.Net.Processors 8 | { 9 | internal class KStreamFlatMapValuesProcessor : AbstractProcessor 10 | { 11 | private readonly IValueMapperWithKey> mapper; 12 | 13 | public KStreamFlatMapValuesProcessor(IValueMapperWithKey> mapper) 14 | { 15 | this.mapper = mapper; 16 | } 17 | 18 | 19 | public override void Process(K key, V value) 20 | { 21 | LogProcessingKeyValue(key, value); 22 | 23 | foreach (var newValue in this.mapper.Apply(key, value, Context.RecordContext)) 24 | { 25 | var originalHeader = Context.RecordContext.Headers.Clone(); 26 | Forward(key, newValue); 27 | Context.SetHeaders(originalHeader); 28 | } 29 | } 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /core/Processors/IStreamPartitioner.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | 3 | namespace Streamiz.Kafka.Net.Processors 4 | { 5 | public interface IStreamPartitioner 6 | { 7 | /// 8 | /// Initialize the current partitioner 9 | /// 10 | /// Global stream configuration 11 | void Initialize(IStreamConfig config); 12 | 13 | /// 14 | /// Function used to determine how records are distributed among partitions of the topic 15 | /// 16 | /// Sink topic name 17 | /// record's key 18 | /// record's value 19 | /// record's source partition 20 | /// number partitions of the sink topic 21 | /// Return the destination partition for the current record 22 | Partition Partition(string topic, K key, V value, Partition sourcePartition, int numPartitions); 23 | } 24 | } -------------------------------------------------------------------------------- /core/Processors/RepartitionTopicConfig.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | using Streamiz.Kafka.Net.Crosscutting; 3 | 4 | namespace Streamiz.Kafka.Net.Processors 5 | { 6 | internal class RepartitionTopicConfig : InternalTopicConfig 7 | { 8 | public static IDictionary REPARTITION_TOPIC_DEFAULT_OVERRIDES = new Dictionary(INTERNAL_TOPIC_DEFAULT_OVERRIDES) 9 | { 10 | { InternalTopicConfigCst.CLEANUP_POLICY_CONFIG, "delete"}, 11 | { InternalTopicConfigCst.SEGMENT_BYTES_CONFIG, "52428800"}, // 50MB 12 | { InternalTopicConfigCst.RETENTION_MS_CONFIG, "-1"}, // Infinity 13 | }; 14 | 15 | public override IDictionary GetProperties(IDictionary defaultConfigs, long additionalRetentionMs) 16 | { 17 | IDictionary topicConfig = new Dictionary(REPARTITION_TOPIC_DEFAULT_OVERRIDES); 18 | topicConfig.AddRange(defaultConfigs); 19 | topicConfig.AddRange(Configs); 20 | return topicConfig; 21 | } 22 | } 23 | } -------------------------------------------------------------------------------- /core/State/IWindowStore.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | 3 | namespace Streamiz.Kafka.Net.State 4 | { 5 | /// 6 | /// Interface for storing the aggregated values of fixed-size time windows. 7 | /// 8 | /// Type of keys 9 | /// Type of values 10 | public interface IWindowStore : IStateStore, IReadOnlyWindowStore 11 | { 12 | /// 13 | /// Put a key-value pair into the window with given window start timestamp. 14 | /// If serialized value bytes are null it is interpreted as delete. Note that deletes will be 15 | /// ignored in the case of an underlying store that retains duplicates. 16 | /// 17 | /// The key to associate the value to 18 | /// The value; can be null 19 | /// The timestamp of the beginning of the window to put the key/value into 20 | void Put(K key, V value, long windowStartTimestamp); 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamKStreamJoin.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | 3 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 4 | { 5 | internal class KStreamKStreamJoin : IProcessorSupplier 6 | { 7 | private readonly string name; 8 | private readonly string otherWindowName; 9 | private readonly long beforeMs; 10 | private readonly long afterMs; 11 | private readonly IValueJoiner joiner; 12 | private readonly bool outer; 13 | 14 | public KStreamKStreamJoin(string name, string otherWindowName, long beforeMs, long afterMs, IValueJoiner joiner, bool outer) 15 | { 16 | this.name = name; 17 | this.otherWindowName = otherWindowName; 18 | this.beforeMs = beforeMs; 19 | this.afterMs = afterMs; 20 | this.joiner = joiner; 21 | this.outer = outer; 22 | } 23 | 24 | public IProcessor Get() 25 | => new KStreamKStreamJoinProcessor(name, otherWindowName, beforeMs, afterMs, joiner, outer); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /core/Metrics/Librdkafka/ConsumerGroupStatistic.cs: -------------------------------------------------------------------------------- 1 | using Newtonsoft.Json; 2 | 3 | namespace Streamiz.Kafka.Net.Metrics.Librdkafka 4 | { 5 | /// 6 | /// JSON Model from : https://github.com/edenhill/librdkafka/blob/master/STATISTICS.md 7 | /// 8 | internal class ConsumerGroupStatistic 9 | { 10 | [JsonProperty(PropertyName = "state")] 11 | public string State; 12 | 13 | [JsonProperty(PropertyName = "stateage")] 14 | public long StateAge; // Gauge 15 | 16 | [JsonProperty(PropertyName = "joinstate")] 17 | public string LocalConsumerGroupHandlerJoinState; 18 | 19 | [JsonProperty(PropertyName = "rebalance_age")] 20 | public long RebalanceAge; // Gauge 21 | 22 | [JsonProperty(PropertyName = "rebalance_cnt")] 23 | public long TotalNumberOfRelabalance; // Gauge assign or revoke 24 | 25 | [JsonProperty(PropertyName = "rebalance_reason")] 26 | public string RebalanceReason; 27 | 28 | [JsonProperty(PropertyName = "assignment_size")] 29 | public long CurrentAssignmentPartitionCount; // Gauge 30 | } 31 | } -------------------------------------------------------------------------------- /core/Processors/AbstractKTableKTableJoinProcessor.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Stream; 2 | using Streamiz.Kafka.Net.Table.Internal; 3 | 4 | namespace Streamiz.Kafka.Net.Processors 5 | { 6 | internal abstract class AbstractKTableKTableJoinProcessor : AbstractProcessor> 7 | { 8 | protected readonly IKTableValueGetter valueGetter; 9 | protected readonly IValueJoiner joiner; 10 | protected readonly bool sendOldValues; 11 | 12 | public AbstractKTableKTableJoinProcessor(IKTableValueGetter valueGetter, IValueJoiner joiner, bool sendOldValues) 13 | { 14 | this.valueGetter = valueGetter; 15 | this.joiner = joiner; 16 | this.sendOldValues = sendOldValues; 17 | } 18 | 19 | public override void Close() 20 | { 21 | base.Close(); 22 | valueGetter.Close(); 23 | } 24 | 25 | public override void Init(ProcessorContext context) 26 | { 27 | base.Init(context); 28 | valueGetter.Init(context); 29 | } 30 | } 31 | } -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamReduce.cs: -------------------------------------------------------------------------------- 1 | 2 | using Streamiz.Kafka.Net.Processors; 3 | using Streamiz.Kafka.Net.Table.Internal; 4 | 5 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 6 | { 7 | internal class KStreamReduce : IKStreamAggProcessorSupplier 8 | { 9 | private readonly string storeName; 10 | private readonly Reducer reducer; 11 | private bool sendOldValues = false; 12 | 13 | public KStreamReduce(string storeName, Reducer reducer) 14 | { 15 | this.storeName = storeName; 16 | this.reducer = reducer; 17 | } 18 | 19 | public void EnableSendingOldValues() 20 | { 21 | sendOldValues = true; 22 | } 23 | 24 | public IProcessor Get() 25 | => new KStreamReduceProcessor(reducer, storeName, sendOldValues); 26 | 27 | public IKTableValueGetterSupplier View() 28 | => new GenericKTableValueGetterSupplier( 29 | new string[] { storeName }, 30 | new TimestampedKeyValueStoreGetter(storeName)); 31 | } 32 | } -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2020 LGouellec 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | -------------------------------------------------------------------------------- /core/State/Enumerator/EmptyEnumerator.cs: -------------------------------------------------------------------------------- 1 | using System.Collections; 2 | using System.Collections.Generic; 3 | 4 | namespace Streamiz.Kafka.Net.State.Enumerator 5 | { 6 | internal class EmptyKeyValueEnumerator : IKeyValueEnumerator 7 | { 8 | public static EmptyKeyValueEnumerator Empty => new(); 9 | 10 | public KeyValuePair? Current => null; 11 | 12 | object IEnumerator.Current => null; 13 | 14 | public void Dispose() 15 | { 16 | 17 | } 18 | 19 | public bool MoveNext() => false; 20 | 21 | public K PeekNextKey() => default; 22 | 23 | public void Reset() { } 24 | } 25 | 26 | internal class EmptyWindowStoreEnumerator : IWindowStoreEnumerator 27 | { 28 | public KeyValuePair? Current => null; 29 | 30 | object IEnumerator.Current => null; 31 | 32 | public void Dispose() 33 | { 34 | } 35 | 36 | public bool MoveNext() => false; 37 | 38 | 39 | public long PeekNextKey() => 0L; 40 | 41 | public void Reset() 42 | { 43 | } 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /core/Table/Internal/Graph/KTableReduce.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using Streamiz.Kafka.Net.Stream; 3 | 4 | namespace Streamiz.Kafka.Net.Table.Internal.Graph 5 | { 6 | internal class KTableReduce : IKTableProcessorSupplier 7 | { 8 | private readonly string storeName; 9 | private readonly Reducer adder; 10 | private readonly Reducer substractor; 11 | 12 | private bool sendOldValues = false; 13 | 14 | 15 | public KTableReduce(string storeName, Reducer adder, Reducer substractor) 16 | { 17 | this.storeName = storeName; 18 | this.adder = adder; 19 | this.substractor = substractor; 20 | } 21 | 22 | public IKTableValueGetterSupplier View 23 | => new KTableMaterializedValueGetterSupplier(storeName); 24 | 25 | public void EnableSendingOldValues() 26 | { 27 | sendOldValues = true; 28 | } 29 | 30 | public IProcessor> Get() 31 | => new KTableReduceProcessor(storeName, sendOldValues, adder, substractor); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /core/Crosscutting/ActionHelper.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using Streamiz.Kafka.Net.Metrics; 3 | 4 | namespace Streamiz.Kafka.Net.Crosscutting 5 | { 6 | internal static class ActionHelper 7 | { 8 | internal static long MeasureLatency(Action action) 9 | { 10 | long before = DateTime.Now.GetMilliseconds(); 11 | action(); 12 | return DateTime.Now.GetMilliseconds() - before; 13 | } 14 | 15 | internal static T MeasureLatency(Func actionToMeasure, Sensor sensor) 16 | { 17 | long before = DateTime.Now.GetMilliseconds(); 18 | T @return = actionToMeasure.Invoke(); 19 | long after = DateTime.Now.GetMilliseconds(); 20 | sensor.Record(after - before); 21 | return @return; 22 | } 23 | 24 | internal static void MeasureLatency(Action actionToMeasure, Sensor sensor) 25 | { 26 | long before = DateTime.Now.GetMilliseconds(); 27 | actionToMeasure.Invoke(); 28 | long after = DateTime.Now.GetMilliseconds(); 29 | sensor.Record(after - before); 30 | } 31 | } 32 | } -------------------------------------------------------------------------------- /core/State/RocksDb/IRocksDbAdapter.cs: -------------------------------------------------------------------------------- 1 | using RocksDbSharp; 2 | using Streamiz.Kafka.Net.Crosscutting; 3 | using Streamiz.Kafka.Net.State.Enumerator; 4 | using System.Collections.Generic; 5 | 6 | namespace Streamiz.Kafka.Net.State 7 | { 8 | internal interface IRocksDbAdapter 9 | { 10 | void Put(byte[] key, byte[] value); 11 | 12 | void PrepareBatch(IEnumerable> entries, 13 | WriteBatch batch); 14 | 15 | byte[] Get(byte[] key); 16 | 17 | byte[] GetOnly(byte[] key); 18 | 19 | IKeyValueEnumerator Range( 20 | Bytes from, 21 | Bytes to, 22 | bool forward); 23 | 24 | IKeyValueEnumerator All(bool forward); 25 | 26 | long ApproximateNumEntries(); 27 | 28 | void Flush(); 29 | 30 | void PrepareBatchForRestore( 31 | IEnumerable> records, 32 | WriteBatch batch); 33 | 34 | void AddToBatch( 35 | byte[] key, 36 | byte[] value, 37 | WriteBatch batch); 38 | 39 | void Close(); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /core/State/InMemory/Internal/InMemoryKeyValueEnumerator.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Crosscutting; 2 | using Streamiz.Kafka.Net.State.Enumerator; 3 | using System.Collections; 4 | using System.Collections.Generic; 5 | using System.Linq; 6 | 7 | namespace Streamiz.Kafka.Net.State.InMemory.Internal 8 | { 9 | internal class InMemoryKeyValueEnumerator : IKeyValueEnumerator 10 | { 11 | private readonly IKeyValueEnumerator enumerator; 12 | 13 | public InMemoryKeyValueEnumerator(IEnumerable> values, bool forward) 14 | { 15 | enumerator = forward ? values.ToWrap() : values.Reverse().ToWrap(); 16 | } 17 | 18 | public KeyValuePair? Current => enumerator.Current; 19 | 20 | object IEnumerator.Current => Current; 21 | 22 | public void Dispose() 23 | => enumerator.Dispose(); 24 | 25 | public bool MoveNext() 26 | => enumerator.MoveNext(); 27 | 28 | public Bytes PeekNextKey() 29 | => enumerator.Current.Value.Key; 30 | 31 | public void Reset() 32 | => enumerator.Reset(); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /core/StreamOptions.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | 3 | namespace Streamiz.Kafka.Net 4 | { 5 | /// 6 | /// NOT USED FOR MOMENT 7 | /// 8 | internal class StreamOptions 9 | { 10 | internal string Named { get; private set; } = null; 11 | internal ITimestampExtractor Extractor { get; private set; } = null; 12 | 13 | private static StreamOptions Create() => Create(null, null); 14 | 15 | private static StreamOptions Create(string named) => Create(null, named); 16 | 17 | private static StreamOptions Create(ITimestampExtractor extractor) => Create(extractor, null); 18 | 19 | private static StreamOptions Create(ITimestampExtractor extractor, string named) 20 | => new StreamOptions() { Named = named, Extractor = extractor }; 21 | 22 | private StreamOptions WithTimestrampExtractor(ITimestampExtractor extractor) 23 | { 24 | Extractor = extractor; 25 | return this; 26 | } 27 | 28 | private StreamOptions WithNamed(string named) 29 | { 30 | Named = named; 31 | return this; 32 | } 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /core/SerDes/CharSerDes.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using System; 3 | 4 | namespace Streamiz.Kafka.Net.SerDes 5 | { 6 | /// 7 | /// SerDes for . 8 | /// 9 | public class CharSerDes : AbstractSerDes 10 | { 11 | /// 12 | /// Deserialize a record value from a byte array into char value 13 | /// 14 | /// serialized bytes. 15 | /// serialization context 16 | /// deserialized using data; may be null 17 | public override char Deserialize(byte[] data, SerializationContext context) => BitConverter.ToChar(data, 0); 18 | 19 | /// 20 | /// Convert data into a byte array. 21 | /// 22 | /// data 23 | /// serialization context 24 | /// serialized bytes 25 | public override byte[] Serialize(char data, SerializationContext context) => BitConverter.GetBytes(data); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /core/SerDes/Int32SerDes.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using System; 3 | 4 | namespace Streamiz.Kafka.Net.SerDes 5 | { 6 | /// 7 | /// SerDes for . 8 | /// 9 | public class Int32SerDes : AbstractSerDes 10 | { 11 | /// 12 | /// Deserialize a record value from a byte array into value 13 | /// 14 | /// serialized bytes. 15 | /// serialization context 16 | /// deserialized using data; may be null 17 | public override int Deserialize(byte[] data, SerializationContext context) => BitConverter.ToInt32(data, 0); 18 | 19 | /// 20 | /// Convert int32 data into a byte array. 21 | /// 22 | /// data 23 | /// serialization context 24 | /// serialized bytes 25 | public override byte[] Serialize(int data, SerializationContext context) => BitConverter.GetBytes(data); 26 | } 27 | } -------------------------------------------------------------------------------- /core/State/Supplier/IStoreSupplier.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | 3 | namespace Streamiz.Kafka.Net.State.Supplier 4 | { 5 | /// 6 | /// A state store supplier which can create one or more instances. 7 | /// 8 | /// 9 | public interface IStoreSupplier 10 | where T : IStateStore 11 | { 12 | /// 13 | /// Return the name of this state store supplier. 14 | /// This must be a valid Kafka topic name; valid characters are ASCII alphanumerics, '.', '_' and '-'. 15 | /// 16 | string Name { get; set; } 17 | 18 | /// 19 | /// Return a new instance of type . 20 | /// 21 | /// Return a new instance of type . 22 | T Get(); 23 | 24 | /// 25 | /// Return a String that is used as the scope for metrics recorded by Metered stores. 26 | /// 27 | string MetricsScope { get; } 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /core/Processors/KStreamFlatMapAsyncProcessor.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.Generic; 3 | using System.Threading.Tasks; 4 | using Confluent.Kafka; 5 | using Streamiz.Kafka.Net.Stream; 6 | 7 | namespace Streamiz.Kafka.Net.Processors 8 | { 9 | internal class KStreamFlatMapAsyncProcessor : 10 | AbstractAsyncProcessor 11 | { 12 | private readonly Func, ExternalContext, Task>>> asyncMapper; 13 | 14 | public KStreamFlatMapAsyncProcessor( 15 | Func, ExternalContext, Task>>> asyncMapper, 16 | RetryPolicy retryPolicy) : base(retryPolicy) 17 | { 18 | this.asyncMapper = asyncMapper; 19 | } 20 | 21 | public override async Task>> ProcessAsync(K key, V value, Headers headers, long timestamp, ExternalContext context) 22 | { 23 | var record = new ExternalRecord(key, value, headers, timestamp); 24 | 25 | return await asyncMapper( 26 | record, 27 | context); 28 | } 29 | } 30 | } -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamKTableJoin.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using Streamiz.Kafka.Net.Table.Internal; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 5 | { 6 | internal class KStreamKTableJoin : IProcessorSupplier 7 | { 8 | public class KeyValueMapper : IKeyValueMapper 9 | { 10 | public K Apply(K key, V1 value, IRecordContext context) => key; 11 | } 12 | 13 | private readonly IKTableValueGetterSupplier valueGetter; 14 | private readonly IValueJoiner valueJoiner; 15 | private readonly bool leftJoin; 16 | 17 | public KStreamKTableJoin( 18 | IKTableValueGetterSupplier valueGetter, 19 | IValueJoiner valueJoiner, 20 | bool leftJoin) 21 | { 22 | this.valueGetter = valueGetter; 23 | this.valueJoiner = valueJoiner; 24 | this.leftJoin = leftJoin; 25 | } 26 | 27 | public IProcessor Get() 28 | => new KStreamKTableJoinProcessor(valueGetter.Get(), new KeyValueMapper(), valueJoiner, leftJoin); 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /core/SerDes/ByteArraySerDes.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | 3 | namespace Streamiz.Kafka.Net.SerDes 4 | { 5 | /// 6 | /// Basic ByteArraySerdes. 7 | /// This is the default serdes present in if you doesn't set your default key and value serdes. 8 | /// 9 | public class ByteArraySerDes : AbstractSerDes 10 | { 11 | /// 12 | /// Deserialize just return data. 13 | /// 14 | /// serialized bytes 15 | /// serialization context 16 | /// Return data parameter 17 | public override byte[] Deserialize(byte[] data, SerializationContext context) => data; 18 | 19 | /// 20 | /// Serialize just return data. 21 | /// 22 | /// typed data 23 | /// serialization context 24 | /// Return data parameter 25 | public override byte[] Serialize(byte[] data, SerializationContext context) => data; 26 | } 27 | } -------------------------------------------------------------------------------- /core/Processors/Public/PunctuationType.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Processors.Public 2 | { 3 | /// 4 | /// Controls what notion of time is used for punctuation scheduled via 5 | /// 6 | /// - STREAM_TIME - uses "stream time", which is advanced by the processing of messages 7 | /// in accordance with the timestamp as extracted by the in use. 8 | /// NOTE: Only advanced if messages arrive 9 | /// - PROCESSING_TIME - uses system time (the wall-clock time), 10 | /// which is advanced at the polling interval () 11 | /// independent of whether new messages arrive. 12 | /// NOTE: This is best effort only as its granularity is limited 13 | /// by how long an iteration of the processing loop takes to complete 14 | /// 15 | /// 16 | public enum PunctuationType 17 | { 18 | /// 19 | /// Use system time 20 | /// 21 | PROCESSING_TIME, 22 | /// 23 | /// Use stream time 24 | /// 25 | STREAM_TIME 26 | } 27 | } -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Public/StreamTableJoinPropsTests.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using NUnit.Framework; 3 | using Streamiz.Kafka.Net.SerDes; 4 | using Streamiz.Kafka.Net.Stream; 5 | 6 | namespace Streamiz.Kafka.Net.Tests.Public 7 | { 8 | public class StreamTableJoinPropsTests 9 | { 10 | [Test] 11 | public void TestConstructor() 12 | { 13 | var join = new StreamTableJoinProps( 14 | new StringSerDes(), 15 | new Int32SerDes(), 16 | new FloatSerDes()); 17 | 18 | Assert.IsInstanceOf(join.KeySerdes); 19 | Assert.IsInstanceOf(join.LeftValueSerdes); 20 | Assert.IsInstanceOf(join.RightValueSerdes); 21 | } 22 | 23 | [Test] 24 | public void TestConstructorWithNull() 25 | { 26 | var join = new StreamTableJoinProps( 27 | null, null, null); 28 | 29 | Assert.IsNull(join.KeySerdes); 30 | Assert.IsNull(join.LeftValueSerdes); 31 | Assert.IsNull(join.RightValueSerdes); 32 | } 33 | } 34 | } -------------------------------------------------------------------------------- /core/Processors/InternalTopicConfig.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Generic; 2 | 3 | namespace Streamiz.Kafka.Net.Processors 4 | { 5 | internal static class InternalTopicConfigCst 6 | { 7 | public static readonly string RETENTION_MS_CONFIG = "retention.ms"; 8 | public static readonly string MESSAGE_TIMESTAMP_TYPE_CONFIG = "message.timestamp.type"; 9 | public static readonly string CLEANUP_POLICY_CONFIG = "cleanup.policy"; 10 | public static readonly string SEGMENT_BYTES_CONFIG = "segment.bytes"; 11 | } 12 | 13 | internal abstract class InternalTopicConfig 14 | { 15 | public static IDictionary INTERNAL_TOPIC_DEFAULT_OVERRIDES = new Dictionary() 16 | { 17 | {InternalTopicConfigCst.MESSAGE_TIMESTAMP_TYPE_CONFIG, "CreateTime" } 18 | }; 19 | 20 | public string Name { get; set; } 21 | public IDictionary Configs { get; set; } = new Dictionary(); 22 | internal int NumberPartitions { get; set; } 23 | 24 | public abstract IDictionary GetProperties(IDictionary defaultConfigs, long additionalRetentionMs); 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /core/SerDes/Int64SerDes.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using System; 3 | 4 | namespace Streamiz.Kafka.Net.SerDes 5 | { 6 | /// 7 | /// SerDes for or 8 | /// 9 | public class Int64SerDes : AbstractSerDes 10 | { 11 | /// 12 | /// Deserialize a record value from a byte array into value 13 | /// 14 | /// serialized bytes. 15 | /// serialization context 16 | /// deserialized using data; may be null 17 | public override long Deserialize(byte[] data, SerializationContext context) => BitConverter.ToInt64(data, 0); 18 | 19 | /// 20 | /// Convert long data into a byte array. 21 | /// 22 | /// data 23 | /// serialization context 24 | /// serialized bytes 25 | public override byte[] Serialize(long data, SerializationContext context) => BitConverter.GetBytes(data); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /core/State/Cache/Internal/PostEvictionDelegate.cs: -------------------------------------------------------------------------------- 1 | // Licensed to the .NET Foundation under one or more agreements. 2 | // The .NET Foundation licenses this file to you under the MIT license. 3 | // This is a fork from Microsoft.Extensions.Caching.Memory.MemoryCache https://github.com/dotnet/runtime/blob/main/src/libraries/Microsoft.Extensions.Caching.Memory 4 | // The only difference is the compaction process and eviction callback is synchronous whereas the .NET repo is asyncrhonous 5 | 6 | using System; 7 | 8 | namespace Streamiz.Kafka.Net.State.Cache.Internal 9 | { 10 | /// 11 | /// Signature of the callback which gets called when a cache entry expires. 12 | /// 13 | /// The key of the entry being evicted. 14 | /// The value of the entry being evicted. 15 | /// The . 16 | /// The information that was passed when registering the callback. 17 | internal delegate void PostEvictionDelegate(K key, V? value, EvictionReason reason, MemoryCache state) 18 | where K : class, IComparable 19 | where V : class; 20 | } 21 | -------------------------------------------------------------------------------- /core/SerDes/FloatSerDes.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using System; 3 | 4 | namespace Streamiz.Kafka.Net.SerDes 5 | { 6 | /// 7 | /// SerDes for 8 | /// 9 | public class FloatSerDes : AbstractSerDes 10 | { 11 | /// 12 | /// Deserialize a record value from a byte array into value 13 | /// 14 | /// serialized bytes. 15 | /// serialization context 16 | /// deserialized using data; may be null 17 | public override float Deserialize(byte[] data, SerializationContext context) 18 | => BitConverter.ToSingle(data, 0); 19 | 20 | /// 21 | /// Convert double data into a byte array. 22 | /// 23 | /// data 24 | /// serialization context 25 | /// serialized bytes 26 | public override byte[] Serialize(float data, SerializationContext context) 27 | => BitConverter.GetBytes(data); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamGlobalKTableJoin.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using Streamiz.Kafka.Net.Table.Internal; 3 | 4 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 5 | { 6 | // TODO : keyMapper 7 | internal class KStreamGlobalKTableJoin : IProcessorSupplier 8 | { 9 | private readonly IKTableValueGetterSupplier supplier; 10 | private readonly IValueJoiner valueJoiner; 11 | private readonly IKeyValueMapper keyMapper; 12 | private readonly bool leftJoin; 13 | 14 | public KStreamGlobalKTableJoin(IKTableValueGetterSupplier supplier, IValueJoiner valueJoiner, IKeyValueMapper keyMapper, bool leftJoin) 15 | { 16 | this.supplier = supplier; 17 | this.valueJoiner = valueJoiner; 18 | this.leftJoin = leftJoin; 19 | this.keyMapper = keyMapper; 20 | } 21 | 22 | public IProcessor Get() 23 | => new KStreamKTableJoinProcessor( 24 | supplier.Get(), 25 | keyMapper, 26 | valueJoiner, 27 | leftJoin); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /core/Processors/KStreamPAPI.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using Microsoft.Extensions.Logging; 3 | using Streamiz.Kafka.Net.Crosscutting; 4 | using Streamiz.Kafka.Net.Processors.Public; 5 | 6 | namespace Streamiz.Kafka.Net.Processors 7 | { 8 | internal abstract class KStreamPAPI : AbstractProcessor 9 | { 10 | public abstract void Process(Record record); 11 | 12 | public override void Process(K key, V value) 13 | { 14 | if (key == null && StateStores.Count > 0) 15 | { 16 | log.LogWarning($"Skipping record due to null key because your transformer is stateful. topic=[{Context.Topic}] partition=[{Context.Partition}] offset=[{Context.Offset}]"); 17 | droppedRecordsSensor.Record(); 18 | return; 19 | } 20 | 21 | Record record = new Record( 22 | new TopicPartitionOffset(Context.Topic, Context.Partition, Context.Offset), 23 | Context.RecordContext.Headers, 24 | new Timestamp(Context.RecordContext.Timestamp.FromMilliseconds()), 25 | key, 26 | value); 27 | 28 | Process(record); 29 | } 30 | } 31 | } -------------------------------------------------------------------------------- /core/SerDes/DoubleSerDes.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using System; 3 | 4 | namespace Streamiz.Kafka.Net.SerDes 5 | { 6 | /// 7 | /// SerDes for 8 | /// 9 | public class DoubleSerDes : AbstractSerDes 10 | { 11 | /// 12 | /// Deserialize a record value from a byte array into value 13 | /// 14 | /// serialized bytes. 15 | /// serialization context 16 | /// deserialized using data; may be null 17 | public override double Deserialize(byte[] data, SerializationContext context) 18 | => BitConverter.ToDouble(data, 0); 19 | 20 | /// 21 | /// Convert double data into a byte array. 22 | /// 23 | /// data 24 | /// serialization context 25 | /// serialized bytes 26 | public override byte[] Serialize(double data, SerializationContext context) 27 | => BitConverter.GetBytes(data); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/KStreamMapValuesAsync.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Collections.Generic; 3 | using System.Threading.Tasks; 4 | using Streamiz.Kafka.Net.Processors; 5 | 6 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph 7 | { 8 | internal class KStreamMapValuesAsync : IProcessorSupplier 9 | { 10 | private readonly Func, ExternalContext, Task> asyncMapper; 11 | private readonly RetryPolicy retryPolicy; 12 | 13 | public KStreamMapValuesAsync( 14 | Func, ExternalContext, Task> asyncMapper, 15 | RetryPolicy retryPolicy) 16 | { 17 | this.asyncMapper = asyncMapper; 18 | this.retryPolicy = retryPolicy ?? RetryPolicy.NewBuilder().Build(); 19 | } 20 | 21 | public IProcessor Get() 22 | { 23 | async Task> Wrapper(ExternalRecord e, ExternalContext c) 24 | { 25 | var newValue = await asyncMapper(e, c); 26 | return new KeyValuePair(e.Key, newValue); 27 | } 28 | 29 | return new KStreamMapAsyncProcessor(Wrapper, retryPolicy); 30 | } 31 | } 32 | } -------------------------------------------------------------------------------- /core/Processors/IProcessor.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using Streamiz.Kafka.Net.SerDes; 3 | using System.Collections.Generic; 4 | using Streamiz.Kafka.Net.Processors.Internal; 5 | 6 | namespace Streamiz.Kafka.Net.Processors 7 | { 8 | internal interface IProcessor 9 | { 10 | string Name { get; set; } 11 | IList StateStores {get;} 12 | ISerDes Key { get; set; } 13 | ISerDes Value { get; set;} 14 | void Init(ProcessorContext context); 15 | void SetTaskId(TaskId id); 16 | IList Next { get; } 17 | void AddNextProcessor(IProcessor next); 18 | void Close(); 19 | void Process(ConsumeResult record); 20 | void Process(object key, object value); 21 | void Forward(K1 key, V1 value); 22 | void Forward(K1 key, V1 value, string name); 23 | void Forward(K1 key, V1 value, long ts); 24 | } 25 | 26 | internal interface IProcessor : IProcessor 27 | { 28 | ISerDes KeySerDes { get; } 29 | ISerDes ValueSerDes { get; } 30 | void Process(K key, V value); 31 | void Forward(K key, V value); 32 | void Forward(K key, V value, string name); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /core/Metrics/Internal/RocksDBMetricsRecordingTrigger.cs: -------------------------------------------------------------------------------- 1 | using System.Collections.Concurrent; 2 | using System.Collections.Generic; 3 | using System.Runtime.InteropServices; 4 | using Streamiz.Kafka.Net.Errors; 5 | 6 | namespace Streamiz.Kafka.Net.Metrics.Internal 7 | { 8 | internal class RocksDbMetricsRecordingTrigger 9 | { 10 | private ConcurrentDictionary MetricsRecorders { get; } = new(); 11 | 12 | internal void AddMetricsRecorder(RocksDbMetricsRecorder recorder) 13 | { 14 | if (!MetricsRecorders.ContainsKey(recorder.Name)) 15 | { 16 | MetricsRecorders.TryAdd(recorder.Name, recorder); 17 | return; 18 | } 19 | 20 | throw new IllegalStateException( 21 | $"RocksDB metrics recorder for store {recorder.Name} has already been added."); 22 | } 23 | 24 | internal void RemoveMetricsRecorder(RocksDbMetricsRecorder recorder) 25 | => MetricsRecorders.TryRemove(recorder.Name, out RocksDbMetricsRecorder _); 26 | 27 | internal void Run(long now) 28 | { 29 | foreach(var recorder in MetricsRecorders.Values) 30 | recorder.Record(now); 31 | } 32 | } 33 | } -------------------------------------------------------------------------------- /core/Stream/Internal/Graph/Nodes/AbstractRepartitionNode.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.SerDes; 2 | 3 | namespace Streamiz.Kafka.Net.Stream.Internal.Graph.Nodes 4 | { 5 | internal abstract class AbstractRepartitionNode : StreamGraphNode 6 | { 7 | public string SourceName { get; } 8 | public ProcessorParameters ProcessorParameters { get; } 9 | public ISerDes KeySerdes { get; } 10 | public ISerDes ValueSerdes { get; } 11 | public string SinkName { get; } 12 | public string RepartitionTopic { get; } 13 | 14 | 15 | internal AbstractRepartitionNode( 16 | string streamGraphNode, 17 | string sourceName, 18 | ProcessorParameters processorParameters, 19 | ISerDes keySerdes, 20 | ISerDes valueSerdes, 21 | string sinkName, 22 | string repartitionTopic) 23 | : base(streamGraphNode) 24 | { 25 | SourceName = sourceName; 26 | ProcessorParameters = processorParameters; 27 | KeySerdes = keySerdes; 28 | ValueSerdes = valueSerdes; 29 | SinkName = sinkName; 30 | RepartitionTopic = repartitionTopic; 31 | } 32 | } 33 | } -------------------------------------------------------------------------------- /core/Table/Internal/Graph/AbstractKTableKTableJoin.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Processors; 2 | using Streamiz.Kafka.Net.Stream; 3 | 4 | namespace Streamiz.Kafka.Net.Table.Internal.Graph 5 | { 6 | internal abstract class AbstractKTableKTableJoin : IKTableProcessorSupplier 7 | { 8 | protected bool sendOldValues = false; 9 | protected readonly IKTableGetter table1; 10 | protected readonly IKTableGetter table2; 11 | protected readonly IValueJoiner valueJoiner; 12 | 13 | public abstract IKTableValueGetterSupplier View { get; } 14 | public abstract IProcessor> Get(); 15 | 16 | 17 | public AbstractKTableKTableJoin( 18 | IKTableGetter table1, 19 | IKTableGetter table2, 20 | IValueJoiner valueJoiner) 21 | { 22 | this.table1 = table1; 23 | this.table2 = table2; 24 | this.valueJoiner = valueJoiner; 25 | } 26 | 27 | 28 | public void EnableSendingOldValues() 29 | { 30 | table1.EnableSendingOldValues(); 31 | table2.EnableSendingOldValues(); 32 | sendOldValues = true; 33 | } 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /core/Processors/KTableKTableJoinMergeProcessor.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.State; 2 | using Streamiz.Kafka.Net.Table.Internal; 3 | 4 | namespace Streamiz.Kafka.Net.Processors 5 | { 6 | internal class KTableKTableJoinMergeProcessor : AbstractKTableProcessor 7 | { 8 | public KTableKTableJoinMergeProcessor(string queryableStoreName, bool sendOldValues, bool throwExceptionStateNull = false) 9 | : base(queryableStoreName, sendOldValues, throwExceptionStateNull) 10 | { 11 | } 12 | 13 | public override void Process(K key, Change value) 14 | { 15 | if (!string.IsNullOrEmpty(queryableStoreName)) 16 | { 17 | store.Put(key, ValueAndTimestamp.Make(value.NewValue, Context.Timestamp)); 18 | tupleForwarder.MaybeForward(key, value.NewValue, sendOldValues ? value.OldValue : default); 19 | } 20 | else 21 | { 22 | if (sendOldValues) 23 | { 24 | Forward(key, value); 25 | } 26 | else 27 | { 28 | Forward(key, new Change(default, value.NewValue)); 29 | } 30 | } 31 | } 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /core/Metrics/Librdkafka/IdempotentProducerStatistic.cs: -------------------------------------------------------------------------------- 1 | using Newtonsoft.Json; 2 | 3 | namespace Streamiz.Kafka.Net.Metrics.Librdkafka 4 | { 5 | /// 6 | /// JSON Model from : https://github.com/edenhill/librdkafka/blob/master/STATISTICS.md 7 | /// 8 | internal class IdempotentProducerStatistic 9 | { 10 | [JsonProperty(PropertyName = "idemp_state")] 11 | public string State; 12 | 13 | [JsonProperty(PropertyName = "idemp_age")] 14 | public long IdempotentStateAge; // Gauge 15 | 16 | [JsonProperty(PropertyName = "txn_state")] 17 | public string CurrentTransactionalProducerState; 18 | 19 | [JsonProperty(PropertyName = "txn_stateage")] 20 | public long TimeElapsedSinceLastTransactionalProducerStateChange; // Gauge 21 | 22 | [JsonProperty(PropertyName = "txn_may_enq")] 23 | public bool TransactionalStateAllowEnqueuing; 24 | 25 | [JsonProperty(PropertyName = "producer_id")] 26 | public long ProducerId; // Gauge 27 | 28 | [JsonProperty(PropertyName = "producer_epoch")] 29 | public long ProducerEpoch; // Gauge 30 | 31 | [JsonProperty(PropertyName = "epoch_cnt")] 32 | public long NumberOfProducerIdAssignmentSinceStarts; 33 | } 34 | } -------------------------------------------------------------------------------- /core/Mock/Pipes/SyncPipeInput.cs: -------------------------------------------------------------------------------- 1 | using Confluent.Kafka; 2 | using System; 3 | using Streamiz.Kafka.Net.Mock.Sync; 4 | 5 | namespace Streamiz.Kafka.Net.Mock.Pipes 6 | { 7 | internal class SyncPipeInput : IPipeInput 8 | { 9 | private readonly ISyncPublisher publisher; 10 | private readonly string topic; 11 | private static readonly object _lock = new(); 12 | 13 | public SyncPipeInput(ISyncPublisher publisher, string topic) 14 | { 15 | this.publisher = publisher; 16 | this.topic = topic; 17 | } 18 | 19 | public string TopicName => topic; 20 | public event PipeFlushed Flushed; 21 | 22 | public void Dispose() 23 | { 24 | Flush(); 25 | publisher.Close(); 26 | } 27 | 28 | public void Flush() 29 | { 30 | lock (_lock) 31 | { 32 | publisher.Flush(); 33 | Flushed?.Invoke(); 34 | } 35 | } 36 | 37 | public virtual void Pipe(byte[] key, byte[] value, DateTime timestamp, Headers headers) 38 | { 39 | lock (_lock) 40 | { 41 | publisher.PublishRecord(topic, key, value, timestamp, headers); 42 | } 43 | } 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /core/RandomGenerator.cs: -------------------------------------------------------------------------------- 1 | using System; 2 | using System.Linq; 3 | using System.Security.Cryptography; 4 | 5 | namespace Streamiz.Kafka.Net 6 | { 7 | /// 8 | /// Helper random generator 9 | /// 10 | public static class RandomGenerator 11 | { 12 | /// 13 | /// Generates a random integer between 0 and a specified exclusive upper bound using a cryptographically strong random number generator. 14 | /// 15 | public static int GetInt32(int partitionCount) 16 | { 17 | #if NETSTANDARD2_0 18 | byte[] rngBytes = new byte[4]; 19 | RandomNumberGenerator.Create().GetBytes(rngBytes); 20 | return BitConverter.ToInt32(rngBytes, 0); 21 | #else 22 | // Use this when possible as it is more memory efficient 23 | return RandomNumberGenerator.GetInt32(0, partitionCount); 24 | #endif 25 | } 26 | 27 | public static string GetRandomString(int maxLength = 100) 28 | { 29 | var rdLength = GetInt32(maxLength); 30 | const string chars = "ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789"; 31 | return new string(Enumerable.Repeat(chars, rdLength) 32 | .Select(s => s[GetInt32(s.Length)]).ToArray()); 33 | } 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /core/State/Cache/Internal/PostEvictionCallbackRegistration.cs: -------------------------------------------------------------------------------- 1 | // Licensed to the .NET Foundation under one or more agreements. 2 | // The .NET Foundation licenses this file to you under the MIT license. 3 | // This is a fork from Microsoft.Extensions.Caching.Memory.MemoryCache https://github.com/dotnet/runtime/blob/main/src/libraries/Microsoft.Extensions.Caching.Memory 4 | // The only difference is the compaction process and eviction callback is synchronous whereas the .NET repo is asyncrhonous 5 | 6 | using System; 7 | 8 | namespace Streamiz.Kafka.Net.State.Cache.Internal 9 | { 10 | /// 11 | /// Represents a callback delegate that will be fired after an entry is evicted from the cache. 12 | /// 13 | internal class PostEvictionCallbackRegistration 14 | where K : class, IComparable 15 | where V : class 16 | { 17 | /// 18 | /// Gets or sets the callback delegate that will be fired after an entry is evicted from the cache. 19 | /// 20 | public PostEvictionDelegate EvictionCallback { get; set; } 21 | 22 | /// 23 | /// Gets or sets the state to pass to the callback delegate. 24 | /// 25 | public MemoryCache? State { get; set; } 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /core/State/RocksDb/Internal/RocksDbKeyValueSegment.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.State.Internal; 2 | using System; 3 | using Streamiz.Kafka.Net.Metrics.Internal; 4 | 5 | namespace Streamiz.Kafka.Net.State.Internal 6 | { 7 | internal class RocksDbKeyValueSegment 8 | : RocksDbKeyValueStore, IComparable, ISegment 9 | { 10 | private readonly long id; 11 | 12 | public RocksDbKeyValueSegment(string segmentName, string windowName, long id, RocksDbMetricsRecorder recorder) 13 | : base(segmentName, windowName, recorder) 14 | { 15 | this.id = id; 16 | KeyComparator = CompareSegmentedKey; 17 | } 18 | 19 | internal void OpenDB(ProcessorContext context) 20 | => OpenDatabase(context); 21 | 22 | public int CompareTo(RocksDbKeyValueSegment other) 23 | => id.CompareTo(other.id); 24 | 25 | public void Destroy() 26 | => DbDir.Delete(true); 27 | 28 | protected int CompareSegmentedKey(byte[] key1, byte[] key2) 29 | { 30 | var comparer = new WindowKeyBytesComparer(); 31 | var k1 = WindowKeyBytes.Wrap(key1); 32 | var k2 = WindowKeyBytes.Wrap(key2); 33 | return comparer.Compare(k1, k2); 34 | } 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /test/Streamiz.Kafka.Net.Tests/Processors/KStreamKStreamWindowed/KStreamKStreamWindowedInnerJoinTests.cs: -------------------------------------------------------------------------------- 1 | using NUnit.Framework; 2 | using Streamiz.Kafka.Net.Stream; 3 | using System.Linq; 4 | 5 | namespace Streamiz.Kafka.Net.Tests.Processors 6 | { 7 | public class KStreamKStreamWindowedInnerJoinTests : KStreamKStreamWindowedTestsBase 8 | { 9 | protected override JoinWindowOptions CreateJoinWindow() 10 | => new JoinSlidingWindowOptions(0L, 1000L, -1L, JoinSlidingWindowOptions.DEFAULT_RETENTION_MS); 11 | 12 | protected override JoinDelegate GetJoinDelegate(IKStream stream1) 13 | => stream1.Join; 14 | 15 | [Test] 16 | public void StreamStreamInnerJoin() 17 | { 18 | inputTopic2.PipeInput("test", "right-0", TestTime.AddSeconds(0)); 19 | inputTopic1.PipeInput("test", "left-1", TestTime.AddSeconds(1)); 20 | inputTopic2.PipeInput("test", "right-2", TestTime.AddSeconds(2)); 21 | inputTopic2.PipeInput("test", "right-3", TestTime.AddSeconds(3)); 22 | 23 | var records = outputTopic.ReadValueList().ToArray(); 24 | Assert.AreEqual(1, records.Length); 25 | 26 | Assert.That(records[0], Is.EqualTo("left-1-right-2")); 27 | } 28 | } 29 | } -------------------------------------------------------------------------------- /core/Processors/IRecordQueue.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Processors 2 | { 3 | /// 4 | /// RecordQueue is a FIFO queue of (ConsumerRecord + timestamp). It also keeps track of the 5 | /// partition timestamp defined as the largest timestamp seen on the partition so far; this is passed to the 6 | /// timestamp extractor. 7 | /// 8 | /// The type of elements in the queue. 9 | public interface IRecordQueue 10 | { 11 | /// 12 | /// queue one element 13 | /// 14 | /// item 15 | /// return new size of queue 16 | int Queue(T item); 17 | 18 | /// 19 | /// Get the first element in the queue 20 | /// 21 | /// return the first element in the queue 22 | T Poll(); 23 | 24 | /// 25 | /// Actual size of the queue 26 | /// 27 | int Size { get; } 28 | 29 | /// 30 | /// Get if the queue is empty or not 31 | /// 32 | bool IsEmpty { get; } 33 | 34 | /// 35 | /// Clear the queue 36 | /// 37 | void Clear(); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /core/Processors/Internal/TaskId.cs: -------------------------------------------------------------------------------- 1 | namespace Streamiz.Kafka.Net.Processors.Internal 2 | { 3 | /// 4 | /// The task ID representation composed as topic group ID plus the assigned partition ID. 5 | /// 6 | public class TaskId 7 | { 8 | /// 9 | /// The ID of the topic group. 10 | /// 11 | public int Id { get; set; } 12 | 13 | /// 14 | /// The ID of the partition. 15 | /// 16 | public int Partition { get; set; } 17 | 18 | /// 19 | /// 20 | /// 21 | /// 22 | public override string ToString() => $"{Id}-{Partition}"; 23 | 24 | /// 25 | /// 26 | /// 27 | /// 28 | /// 29 | public override bool Equals(object obj) 30 | { 31 | return obj is TaskId id && id.Id.Equals(Id) && id.Partition.Equals(Partition); 32 | } 33 | 34 | /// 35 | /// 36 | /// 37 | /// 38 | public override int GetHashCode() 39 | { 40 | return Id.GetHashCode() & Partition.GetHashCode(); 41 | } 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /core/State/Metered/MeteredTimestampedKeyValueStore.cs: -------------------------------------------------------------------------------- 1 | using Streamiz.Kafka.Net.Crosscutting; 2 | using Streamiz.Kafka.Net.SerDes; 3 | 4 | namespace Streamiz.Kafka.Net.State.Metered 5 | { 6 | internal class MeteredTimestampedKeyValueStore 7 | : MeteredKeyValueStore>, ITimestampedKeyValueStore 8 | { 9 | public MeteredTimestampedKeyValueStore( 10 | IKeyValueStore wrapped, 11 | ISerDes keySerdes, 12 | ISerDes> valueSerdes, 13 | string metricScope) 14 | : base(wrapped, keySerdes, valueSerdes, metricScope) 15 | { } 16 | 17 | public override void InitStoreSerDes(ProcessorContext context) 18 | { 19 | if (!initStoreSerdes) 20 | { 21 | keySerdes ??= context.Configuration.DefaultKeySerDes as ISerDes; 22 | valueSerdes ??= new ValueAndTimestampSerDes(context.Configuration.DefaultValueSerDes as ISerDes); 23 | 24 | keySerdes?.Initialize(new SerDesContext(context.Configuration)); 25 | valueSerdes?.Initialize(new SerDesContext(context.Configuration)); 26 | 27 | initStoreSerdes = true; 28 | } 29 | } 30 | } 31 | } --------------------------------------------------------------------------------